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