View Javadoc

1   /**
2    * Copyright 2010 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
19   */
20  package org.apache.hadoop.hbase.master.handler;
21  
22  import java.io.IOException;
23  import java.util.ArrayList;
24  import java.util.List;
25  import java.util.Map;
26  import java.util.NavigableMap;
27  import java.util.Set;
28  
29  import org.apache.commons.logging.Log;
30  import org.apache.commons.logging.LogFactory;
31  import org.apache.hadoop.hbase.HConstants;
32  import org.apache.hadoop.hbase.HRegionInfo;
33  import org.apache.hadoop.hbase.Server;
34  import org.apache.hadoop.hbase.ServerName;
35  import org.apache.hadoop.hbase.catalog.CatalogTracker;
36  import org.apache.hadoop.hbase.catalog.MetaEditor;
37  import org.apache.hadoop.hbase.catalog.MetaReader;
38  import org.apache.hadoop.hbase.client.Result;
39  import org.apache.hadoop.hbase.executor.EventHandler;
40  import org.apache.hadoop.hbase.master.AssignmentManager;
41  import org.apache.hadoop.hbase.master.AssignmentManager.RegionState;
42  import org.apache.hadoop.hbase.master.DeadServer;
43  import org.apache.hadoop.hbase.master.MasterServices;
44  import org.apache.hadoop.hbase.master.ServerManager;
45  import org.apache.hadoop.hbase.util.Bytes;
46  import org.apache.hadoop.hbase.util.Pair;
47  import org.apache.hadoop.hbase.zookeeper.ZKAssign;
48  import org.apache.zookeeper.KeeperException;
49  
50  /**
51   * Process server shutdown.
52   * Server-to-handle must be already in the deadservers lists.  See
53   * {@link ServerManager#expireServer(ServerName)}
54   */
55  public class ServerShutdownHandler extends EventHandler {
56    private static final Log LOG = LogFactory.getLog(ServerShutdownHandler.class);
57    private final ServerName serverName;
58    private final MasterServices services;
59    private final DeadServer deadServers;
60    private final boolean shouldSplitHlog; // whether to split HLog or not
61  
62    public ServerShutdownHandler(final Server server, final MasterServices services,
63        final DeadServer deadServers, final ServerName serverName,
64        final boolean shouldSplitHlog) {
65      this(server, services, deadServers, serverName, EventType.M_SERVER_SHUTDOWN,
66          shouldSplitHlog);
67    }
68  
69    ServerShutdownHandler(final Server server, final MasterServices services,
70        final DeadServer deadServers, final ServerName serverName, EventType type,
71        final boolean shouldSplitHlog) {
72      super(server, type);
73      this.serverName = serverName;
74      this.server = server;
75      this.services = services;
76      this.deadServers = deadServers;
77      if (!this.deadServers.contains(this.serverName)) {
78        LOG.warn(this.serverName + " is NOT in deadservers; it should be!");
79      }
80      this.shouldSplitHlog = shouldSplitHlog;
81    }
82  
83    @Override
84    public String getInformativeName() {
85      if (serverName != null) {
86        return this.getClass().getSimpleName() + " for " + serverName;
87      } else {
88        return super.getInformativeName();
89      }
90    }
91  
92    /**
93     * Before assign the ROOT region, ensure it haven't 
94     *  been assigned by other place
95     * <p>
96     * Under some scenarios, the ROOT region can be opened twice, so it seemed online
97     * in two regionserver at the same time.
98     * If the ROOT region has been assigned, so the operation can be canceled. 
99     * @throws InterruptedException
100    * @throws IOException
101    * @throws KeeperException
102    */
103   private void verifyAndAssignRoot()
104   throws InterruptedException, IOException, KeeperException {
105     long timeout = this.server.getConfiguration().
106       getLong("hbase.catalog.verification.timeout", 1000);
107     if (!this.server.getCatalogTracker().verifyRootRegionLocation(timeout)) {
108       this.services.getAssignmentManager().assignRoot();
109     } else if (serverName.equals(server.getCatalogTracker().getRootLocation())) {
110       throw new IOException("-ROOT- is onlined on the dead server "
111           + serverName);
112     } else {
113       LOG.info("Skip assigning -ROOT-, because it is online on the "
114           + server.getCatalogTracker().getRootLocation());
115     }
116   }
117 
118   /**
119    * Failed many times, shutdown processing
120    * @throws IOException
121    */
122   private void verifyAndAssignRootWithRetries() throws IOException {
123     int iTimes = this.server.getConfiguration().getInt(
124         "hbase.catalog.verification.retries", 10);
125 
126     long waitTime = this.server.getConfiguration().getLong(
127         "hbase.catalog.verification.timeout", 1000);
128 
129     int iFlag = 0;
130     while (true) {
131       try {
132         verifyAndAssignRoot();
133         break;
134       } catch (KeeperException e) {
135         this.server.abort("In server shutdown processing, assigning root", e);
136         throw new IOException("Aborting", e);
137       } catch (Exception e) {
138         if (iFlag >= iTimes) {
139           this.server.abort("verifyAndAssignRoot failed after" + iTimes
140               + " times retries, aborting", e);
141           throw new IOException("Aborting", e);
142         }
143         try {
144           Thread.sleep(waitTime);
145         } catch (InterruptedException e1) {
146           LOG.warn("Interrupted when is the thread sleep", e1);
147           Thread.currentThread().interrupt();
148           throw new IOException("Interrupted", e1);
149         }
150         iFlag++;
151       }
152     }
153   }
154   
155   /**
156    * @return True if the server we are processing was carrying <code>-ROOT-</code>
157    */
158   boolean isCarryingRoot() {
159     return false;
160   }
161 
162   /**
163    * @return True if the server we are processing was carrying <code>.META.</code>
164    */
165   boolean isCarryingMeta() {
166     return false;
167   }
168 
169   @Override
170   public String toString() {
171     String name = "UnknownServerName";
172     if(server != null && server.getServerName() != null) {
173       name = server.getServerName().toString();
174     }
175     return getClass().getSimpleName() + "-" + name + "-" + getSeqid();
176   }
177 
178   @Override
179   public void process() throws IOException {
180     final ServerName serverName = this.serverName;
181     try {
182       try {
183         if (this.shouldSplitHlog) {
184           LOG.info("Splitting logs for " + serverName);
185           this.services.getMasterFileSystem().splitLog(serverName);
186         } else {
187           LOG.info("Skipping log splitting for " + serverName);
188         }
189       } catch (IOException ioe) {
190         this.services.getExecutorService().submit(this);
191         this.deadServers.add(serverName);
192         throw new IOException("failed log splitting for " +
193           serverName + ", will retry", ioe);
194       }
195 
196       // Assign root and meta if we were carrying them.
197       if (isCarryingRoot()) { // -ROOT-
198         // Check again: region may be assigned to other where because of RIT
199         // timeout
200         if (this.services.getAssignmentManager().isCarryingRoot(serverName)) {
201           LOG.info("Server " + serverName
202               + " was carrying ROOT. Trying to assign.");
203           this.services.getAssignmentManager().regionOffline(
204               HRegionInfo.ROOT_REGIONINFO);
205           verifyAndAssignRootWithRetries();
206         } else {
207           LOG.info("ROOT has been assigned to otherwhere, skip assigning.");
208         }
209       }
210 
211       // Carrying meta?
212       if (isCarryingMeta()) {
213         // Check again: region may be assigned to other where because of RIT
214         // timeout
215         if (this.services.getAssignmentManager().isCarryingMeta(serverName)) {
216           LOG.info("Server " + serverName
217               + " was carrying META. Trying to assign.");
218           this.services.getAssignmentManager().regionOffline(
219               HRegionInfo.FIRST_META_REGIONINFO);
220           this.services.getAssignmentManager().assignMeta();
221         } else {
222           LOG.info("META has been assigned to otherwhere, skip assigning.");
223         }
224       }
225 
226       // We don't want worker thread in the MetaServerShutdownHandler
227       // executor pool to block by waiting availability of -ROOT-
228       // and .META. server. Otherwise, it could run into the following issue:
229       // 1. The current MetaServerShutdownHandler instance For RS1 waits for the .META.
230       //    to come online.
231       // 2. The newly assigned .META. region server RS2 was shutdown right after
232       //    it opens the .META. region. So the MetaServerShutdownHandler
233       //    instance For RS1 will still be blocked.
234       // 3. The new instance of MetaServerShutdownHandler for RS2 is queued.
235       // 4. The newly assigned .META. region server RS3 was shutdown right after
236       //    it opens the .META. region. So the MetaServerShutdownHandler
237       //    instance For RS1 and RS2 will still be blocked.
238       // 5. The new instance of MetaServerShutdownHandler for RS3 is queued.
239       // 6. Repeat until we run out of MetaServerShutdownHandler worker threads
240       // The solution here is to resubmit a ServerShutdownHandler request to process
241       // user regions on that server so that MetaServerShutdownHandler
242       // executor pool is always available.
243       if (isCarryingRoot() || isCarryingMeta()) { // -ROOT- or .META.
244         this.services.getExecutorService().submit(new ServerShutdownHandler(
245           this.server, this.services, this.deadServers, serverName, false));
246         this.deadServers.add(serverName);
247         return;
248       }
249 
250 
251       // Wait on meta to come online; we need it to progress.
252       // TODO: Best way to hold strictly here?  We should build this retry logic
253       // into the MetaReader operations themselves.
254       // TODO: Is the reading of .META. necessary when the Master has state of
255       // cluster in its head?  It should be possible to do without reading .META.
256       // in all but one case. On split, the RS updates the .META.
257       // table and THEN informs the master of the split via zk nodes in
258       // 'unassigned' dir.  Currently the RS puts ephemeral nodes into zk so if
259       // the regionserver dies, these nodes do not stick around and this server
260       // shutdown processing does fixup (see the fixupDaughters method below).
261       // If we wanted to skip the .META. scan, we'd have to change at least the
262       // final SPLIT message to be permanent in zk so in here we'd know a SPLIT
263       // completed (zk is updated after edits to .META. have gone in).  See
264       // {@link SplitTransaction}.  We'd also have to be figure another way for
265       // doing the below .META. daughters fixup.
266       NavigableMap<HRegionInfo, Result> hris = null;
267       while (!this.server.isStopped()) {
268         try {
269           this.server.getCatalogTracker().waitForMeta();
270           hris = MetaReader.getServerUserRegions(this.server.getCatalogTracker(),
271             this.serverName);
272           break;
273         } catch (InterruptedException e) {
274           Thread.currentThread().interrupt();
275           throw new IOException("Interrupted", e);
276         } catch (IOException ioe) {
277           LOG.info("Received exception accessing META during server shutdown of " +
278               serverName + ", retrying META read", ioe);
279         }
280       }
281 
282       // Returns set of regions that had regionplans against the downed server and a list of
283       // the intersection of regions-in-transition and regions that were on the server that died.
284       Pair<Set<HRegionInfo>, List<RegionState>> p = this.services.getAssignmentManager()
285           .processServerShutdown(this.serverName);
286       Set<HRegionInfo> ritsGoingToServer = p.getFirst();
287       List<RegionState> ritsOnServer = p.getSecond();
288 
289       List<HRegionInfo> regionsToAssign = getRegionsToAssign(hris, ritsOnServer, ritsGoingToServer);
290       for (HRegionInfo hri : ritsGoingToServer) {
291         if (!this.services.getAssignmentManager().isRegionAssigned(hri)) {
292           if (!regionsToAssign.contains(hri)) {
293             regionsToAssign.add(hri);
294           }
295         }
296       }
297       for (HRegionInfo hri : regionsToAssign) {
298         this.services.getAssignmentManager().assign(hri, true);
299       }
300       LOG.info(regionsToAssign.size() + " regions which were planned to open on " + this.serverName
301           + " have been re-assigned.");
302     } finally {
303       this.deadServers.finish(serverName);
304     }
305     LOG.info("Finished processing of shutdown of " + serverName);
306   }
307 
308   /**
309    * Figure what to assign from the dead server considering state of RIT and whats up in .META.
310    * @param metaHRIs Regions that .META. says were assigned to the dead server
311    * @param ritsOnServer Regions that were in transition, and on the dead server.
312    * @param ritsGoingToServer Regions that were in transition to the dead server.
313    * @return List of regions to assign or null if aborting.
314    * @throws IOException
315    */
316   private List<HRegionInfo> getRegionsToAssign(final NavigableMap<HRegionInfo, Result> metaHRIs,
317       final List<RegionState> ritsOnServer, Set<HRegionInfo> ritsGoingToServer) throws IOException {
318     List<HRegionInfo> toAssign = new ArrayList<HRegionInfo>();
319     // If no regions on the server, then nothing to assign (Regions that were currently being
320     // assigned will be retried over in the AM#assign method).
321     if (metaHRIs == null || metaHRIs.isEmpty()) return toAssign;
322     // Remove regions that we do not want to reassign such as regions that are
323     // OFFLINE. If region is OFFLINE against this server, its probably being assigned over
324     // in the single region assign method in AM; do not assign it here too. TODO: VERIFY!!!
325     // TODO: Currently OFFLINE is too messy. Its done on single assign but bulk done when bulk
326     // assigning and then there is special handling when master joins a cluster.
327     //
328     // If split, the zk callback will have offlined. Daughters will be in the
329     // list of hris we got from scanning the .META. These should be reassigned. Not the parent.
330     for (RegionState rs : ritsOnServer) {
331       if (!rs.isClosing() && !rs.isPendingClose() && !rs.isSplitting()) {
332         LOG.debug("Removed " + rs.getRegion().getRegionNameAsString()
333             + " from list of regions to assign because region state: " + rs.getState());
334         metaHRIs.remove(rs.getRegion());
335       }
336     }
337 
338     for (Map.Entry<HRegionInfo, Result> e : metaHRIs.entrySet()) {
339       RegionState rit = services.getAssignmentManager().getRegionsInTransition().get(
340           e.getKey().getEncodedName());
341       AssignmentManager assignmentManager = this.services.getAssignmentManager();
342       if (processDeadRegion(e.getKey(), e.getValue(), assignmentManager,
343           this.server.getCatalogTracker())) {
344         ServerName addressFromAM = assignmentManager.getRegionServerOfRegion(e.getKey());
345         if (rit != null && !rit.isClosing() && !rit.isPendingClose() && !rit.isSplitting()
346             && !ritsGoingToServer.contains(e.getKey())) {
347           // Skip regions that were in transition unless CLOSING or
348           // PENDING_CLOSE
349           LOG.info("Skip assigning region " + rit.toString());
350         } else if (addressFromAM != null && !addressFromAM.equals(this.serverName)) {
351           LOG.debug("Skip assigning region " + e.getKey().getRegionNameAsString()
352               + " because it has been opened in " + addressFromAM.getServerName());
353           ritsGoingToServer.remove(e.getKey());
354         } else {
355           if (rit != null) {
356             // clean zk node
357             try {
358               LOG.info("Reassigning region with rs =" + rit + " and deleting zk node if exists");
359               ZKAssign.deleteNodeFailSilent(services.getZooKeeper(), e.getKey());
360             } catch (KeeperException ke) {
361               this.server.abort("Unexpected ZK exception deleting unassigned node " + e.getKey(),
362                   ke);
363               return null;
364             }
365           }
366           toAssign.add(e.getKey());
367         }
368       } else if (rit != null && (rit.isSplitting() || rit.isSplit())) {
369         // This will happen when the RS went down and the call back for the SPLIITING or SPLIT
370         // has not yet happened for node Deleted event. In that case if the region was actually
371         // split but the RS had gone down before completing the split process then will not try
372         // to assign the parent region again. In that case we should make the region offline
373         // and also delete the region from RIT.
374         HRegionInfo region = rit.getRegion();
375         AssignmentManager am = assignmentManager;
376         am.regionOffline(region);
377         ritsGoingToServer.remove(region);
378       }
379       // If the table was partially disabled and the RS went down, we should clear the RIT
380       // and remove the node for the region. The rit that we use may be stale in case the table
381       // was in DISABLING state but though we did assign we will not be clearing the znode in
382       // CLOSING state. Doing this will have no harm. See HBASE-5927
383       toAssign = checkForDisablingOrDisabledTables(ritsGoingToServer, toAssign, rit, assignmentManager);
384     }
385     return toAssign;
386   }
387 
388   private List<HRegionInfo> checkForDisablingOrDisabledTables(Set<HRegionInfo> regionsFromRIT,
389       List<HRegionInfo> toAssign, RegionState rit, AssignmentManager assignmentManager) {
390     if (rit == null) {
391       return toAssign;
392     }
393     if (!rit.isClosing() && !rit.isPendingClose()) {
394       return toAssign;
395     }
396     if (!assignmentManager.getZKTable().isDisablingOrDisabledTable(
397         rit.getRegion().getTableNameAsString())) {
398       return toAssign;
399     }
400     HRegionInfo hri = rit.getRegion();
401     AssignmentManager am = assignmentManager;
402     am.deleteClosingOrClosedNode(hri);
403     am.regionOffline(hri);
404     // To avoid region assignment if table is in disabling or disabled state.
405     toAssign.remove(hri);
406     regionsFromRIT.remove(hri);
407     return toAssign;
408   }
409 
410   /**
411    * Process a dead region from a dead RS. Checks if the region is disabled or
412    * disabling or if the region has a partially completed split.
413    * @param hri
414    * @param result
415    * @param assignmentManager
416    * @param catalogTracker
417    * @return Returns true if specified region should be assigned, false if not.
418    * @throws IOException
419    */
420   public static boolean processDeadRegion(HRegionInfo hri, Result result,
421       AssignmentManager assignmentManager, CatalogTracker catalogTracker)
422   throws IOException {
423     boolean tablePresent = assignmentManager.getZKTable().isTablePresent(
424         hri.getTableNameAsString());
425     if (!tablePresent) {
426       LOG.info("The table " + hri.getTableNameAsString()
427           + " was deleted.  Hence not proceeding.");
428       return false;
429     }
430     // If table is not disabled but the region is offlined,
431     boolean disabled = assignmentManager.getZKTable().isDisabledTable(
432         hri.getTableNameAsString());
433     if (disabled){
434       LOG.info("The table " + hri.getTableNameAsString()
435           + " was disabled.  Hence not proceeding.");
436       return false;
437     }
438     if (hri.isOffline() && hri.isSplit()) {
439       LOG.debug("Offlined and split region " + hri.getRegionNameAsString() +
440         "; checking daughter presence");
441       if (MetaReader.getRegion(catalogTracker, hri.getRegionName()) == null) {
442         return false;
443       }
444       fixupDaughters(result, assignmentManager, catalogTracker);
445       return false;
446     }
447     boolean disabling = assignmentManager.getZKTable().isDisablingTable(
448         hri.getTableNameAsString());
449     if (disabling) {
450       LOG.info("The table " + hri.getTableNameAsString()
451           + " is disabled.  Hence not assigning region" + hri.getEncodedName());
452       return false;
453     }
454     return true;
455   }
456 
457   /**
458    * Check that daughter regions are up in .META. and if not, add them.
459    * @param hris All regions for this server in meta.
460    * @param result The contents of the parent row in .META.
461    * @return the number of daughters missing and fixed
462    * @throws IOException
463    */
464   public static int fixupDaughters(final Result result,
465       final AssignmentManager assignmentManager,
466       final CatalogTracker catalogTracker)
467   throws IOException {
468     int fixedA = fixupDaughter(result, HConstants.SPLITA_QUALIFIER,
469       assignmentManager, catalogTracker);
470     int fixedB = fixupDaughter(result, HConstants.SPLITB_QUALIFIER,
471       assignmentManager, catalogTracker);
472     return fixedA + fixedB;
473   }
474 
475   /**
476    * Check individual daughter is up in .META.; fixup if its not.
477    * @param result The contents of the parent row in .META.
478    * @param qualifier Which daughter to check for.
479    * @return 1 if the daughter is missing and fixed. Otherwise 0
480    * @throws IOException
481    */
482   static int fixupDaughter(final Result result, final byte [] qualifier,
483       final AssignmentManager assignmentManager,
484       final CatalogTracker catalogTracker)
485   throws IOException {
486     HRegionInfo daughter =
487       MetaReader.parseHRegionInfoFromCatalogResult(result, qualifier);
488     if (daughter == null) return 0;
489     if (isDaughterMissing(catalogTracker, daughter)) {
490       LOG.info("Fixup; missing daughter " + daughter.getRegionNameAsString());
491       MetaEditor.addDaughter(catalogTracker, daughter, null);
492 
493       // TODO: Log WARN if the regiondir does not exist in the fs.  If its not
494       // there then something wonky about the split -- things will keep going
495       // but could be missing references to parent region.
496 
497       // And assign it.
498       assignmentManager.assign(daughter, true);
499       return 1;
500     } else {
501       LOG.debug("Daughter " + daughter.getRegionNameAsString() + " present");
502     }
503     return 0;
504   }
505 
506   /**
507    * Look for presence of the daughter OR of a split of the daughter in .META.
508    * Daughter could have been split over on regionserver before a run of the
509    * catalogJanitor had chance to clear reference from parent.
510    * @param daughter Daughter region to search for.
511    * @throws IOException 
512    */
513   private static boolean isDaughterMissing(final CatalogTracker catalogTracker,
514       final HRegionInfo daughter) throws IOException {
515     FindDaughterVisitor visitor = new FindDaughterVisitor(daughter);
516     // Start the scan at what should be the daughter's row in the .META.
517     // We will either 1., find the daughter or some derivative split of the
518     // daughter (will have same table name and start row at least but will sort
519     // after because has larger regionid -- the regionid is timestamp of region
520     // creation), OR, we will not find anything with same table name and start
521     // row.  If the latter, then assume daughter missing and do fixup.
522     byte [] startrow = daughter.getRegionName();
523     MetaReader.fullScan(catalogTracker, visitor, startrow);
524     return !visitor.foundDaughter();
525   }
526 
527   /**
528    * Looks for daughter.  Sets a flag if daughter or some progeny of daughter
529    * is found up in <code>.META.</code>.
530    */
531   static class FindDaughterVisitor implements MetaReader.Visitor {
532     private final HRegionInfo daughter;
533     private boolean found = false;
534 
535     FindDaughterVisitor(final HRegionInfo daughter) {
536       this.daughter = daughter;
537     }
538 
539     /**
540      * @return True if we found a daughter region during our visiting.
541      */
542     boolean foundDaughter() {
543       return this.found;
544     }
545 
546     @Override
547     public boolean visit(Result r) throws IOException {
548       HRegionInfo hri =
549         MetaReader.parseHRegionInfoFromCatalogResult(r, HConstants.REGIONINFO_QUALIFIER);
550       if (hri == null) {
551         LOG.warn("No serialized HRegionInfo in " + r);
552         return true;
553       }
554       byte [] value = r.getValue(HConstants.CATALOG_FAMILY,
555           HConstants.SERVER_QUALIFIER);
556       // See if daughter is assigned to some server
557       if (value == null) return false;
558 
559       // Now see if we have gone beyond the daughter's startrow.
560       if (!Bytes.equals(daughter.getTableName(),
561           hri.getTableName())) {
562         // We fell into another table.  Stop scanning.
563         return false;
564       }
565       // If our start rows do not compare, move on.
566       if (!Bytes.equals(daughter.getStartKey(), hri.getStartKey())) {
567         return false;
568       }
569       // Else, table name and start rows compare.  It means that the daughter
570       // or some derivative split of the daughter is up in .META.  Daughter
571       // exists.
572       this.found = true;
573       return false;
574     }
575   }
576 }