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.List;
24  import java.util.Map;
25  import java.util.NavigableMap;
26  
27  import org.apache.commons.logging.Log;
28  import org.apache.commons.logging.LogFactory;
29  import org.apache.hadoop.hbase.HConstants;
30  import org.apache.hadoop.hbase.HRegionInfo;
31  import org.apache.hadoop.hbase.Server;
32  import org.apache.hadoop.hbase.ServerName;
33  import org.apache.hadoop.hbase.catalog.CatalogTracker;
34  import org.apache.hadoop.hbase.catalog.MetaEditor;
35  import org.apache.hadoop.hbase.catalog.MetaReader;
36  import org.apache.hadoop.hbase.client.Result;
37  import org.apache.hadoop.hbase.executor.EventHandler;
38  import org.apache.hadoop.hbase.master.AssignmentManager;
39  import org.apache.hadoop.hbase.master.AssignmentManager.RegionState;
40  import org.apache.hadoop.hbase.master.DeadServer;
41  import org.apache.hadoop.hbase.master.MasterServices;
42  import org.apache.hadoop.hbase.master.ServerManager;
43  import org.apache.hadoop.hbase.util.Bytes;
44  import org.apache.zookeeper.KeeperException;
45  
46  /**
47   * Process server shutdown.
48   * Server-to-handle must be already in the deadservers lists.  See
49   * {@link ServerManager#expireServer(ServerName)}
50   */
51  public class ServerShutdownHandler extends EventHandler {
52    private static final Log LOG = LogFactory.getLog(ServerShutdownHandler.class);
53    private final ServerName serverName;
54    private final MasterServices services;
55    private final DeadServer deadServers;
56    private final boolean shouldSplitHlog; // whether to split HLog or not
57  
58    public ServerShutdownHandler(final Server server, final MasterServices services,
59        final DeadServer deadServers, final ServerName serverName,
60        final boolean shouldSplitHlog) {
61      this(server, services, deadServers, serverName, EventType.M_SERVER_SHUTDOWN,
62          shouldSplitHlog);
63    }
64  
65    ServerShutdownHandler(final Server server, final MasterServices services,
66        final DeadServer deadServers, final ServerName serverName, EventType type,
67        final boolean shouldSplitHlog) {
68      super(server, type);
69      this.serverName = serverName;
70      this.server = server;
71      this.services = services;
72      this.deadServers = deadServers;
73      if (!this.deadServers.contains(this.serverName)) {
74        LOG.warn(this.serverName + " is NOT in deadservers; it should be!");
75      }
76      this.shouldSplitHlog = shouldSplitHlog;
77    }
78  
79    @Override
80    public String getInformativeName() {
81      if (serverName != null) {
82        return this.getClass().getSimpleName() + " for " + serverName;
83      } else {
84        return super.getInformativeName();
85      }
86    }
87  
88    /**
89     * Before assign the ROOT region, ensure it haven't 
90     *  been assigned by other place
91     * <p>
92     * Under some scenarios, the ROOT region can be opened twice, so it seemed online
93     * in two regionserver at the same time.
94     * If the ROOT region has been assigned, so the operation can be canceled. 
95     * @throws InterruptedException
96     * @throws IOException
97     * @throws KeeperException
98     */
99    private void verifyAndAssignRoot()
100   throws InterruptedException, IOException, KeeperException {
101     long timeout = this.server.getConfiguration().
102       getLong("hbase.catalog.verification.timeout", 1000);
103     if (!this.server.getCatalogTracker().verifyRootRegionLocation(timeout)) {
104       this.services.getAssignmentManager().assignRoot();
105     }
106   }
107 
108   /**
109    * Failed many times, shutdown processing
110    * @throws IOException
111    */
112   private void verifyAndAssignRootWithRetries() throws IOException {
113     int iTimes = this.server.getConfiguration().getInt(
114         "hbase.catalog.verification.retries", 10);
115 
116     long waitTime = this.server.getConfiguration().getLong(
117         "hbase.catalog.verification.timeout", 1000);
118 
119     int iFlag = 0;
120     while (true) {
121       try {
122         verifyAndAssignRoot();
123         break;
124       } catch (KeeperException e) {
125         this.server.abort("In server shutdown processing, assigning root", e);
126         throw new IOException("Aborting", e);
127       } catch (Exception e) {
128         if (iFlag >= iTimes) {
129           this.server.abort("verifyAndAssignRoot failed after" + iTimes
130               + " times retries, aborting", e);
131           throw new IOException("Aborting", e);
132         }
133         try {
134           Thread.sleep(waitTime);
135         } catch (InterruptedException e1) {
136           LOG.warn("Interrupted when is the thread sleep", e1);
137           Thread.currentThread().interrupt();
138           throw new IOException("Interrupted", e1);
139         }
140         iFlag++;
141       }
142     }
143   }
144   
145   /**
146    * @return True if the server we are processing was carrying <code>-ROOT-</code>
147    */
148   boolean isCarryingRoot() {
149     return false;
150   }
151 
152   /**
153    * @return True if the server we are processing was carrying <code>.META.</code>
154    */
155   boolean isCarryingMeta() {
156     return false;
157   }
158 
159   @Override
160   public String toString() {
161     String name = "UnknownServerName";
162     if(server != null && server.getServerName() != null) {
163       name = server.getServerName().toString();
164     }
165     return getClass().getSimpleName() + "-" + name + "-" + getSeqid();
166   }
167 
168   @Override
169   public void process() throws IOException {
170     final ServerName serverName = this.serverName;
171 
172     try {
173 
174       try {
175         LOG.info("Splitting logs for " + serverName);
176         this.services.getMasterFileSystem().splitLog(serverName);
177       } catch (IOException ioe) {
178         this.services.getExecutorService().submit(this);
179         this.deadServers.add(serverName);
180         throw new IOException("failed log splitting for " +
181           serverName + ", will retry", ioe);
182       }
183 
184       // Assign root and meta if we were carrying them.
185       if (isCarryingRoot()) { // -ROOT-
186         LOG.info("Server " + serverName +
187             " was carrying ROOT. Trying to assign.");
188         this.services.getAssignmentManager().
189         regionOffline(HRegionInfo.ROOT_REGIONINFO);
190         verifyAndAssignRootWithRetries();
191       }
192 
193       // Carrying meta?
194       if (isCarryingMeta()) {
195         LOG.info("Server " + serverName +
196           " was carrying META. Trying to assign.");
197         this.services.getAssignmentManager().
198         regionOffline(HRegionInfo.FIRST_META_REGIONINFO);
199         this.services.getAssignmentManager().assignMeta();
200       }
201 
202       // We don't want worker thread in the MetaServerShutdownHandler
203       // executor pool to block by waiting availability of -ROOT-
204       // and .META. server. Otherwise, it could run into the following issue:
205       // 1. The current MetaServerShutdownHandler instance For RS1 waits for the .META.
206       //    to come online.
207       // 2. The newly assigned .META. region server RS2 was shutdown right after
208       //    it opens the .META. region. So the MetaServerShutdownHandler
209       //    instance For RS1 will still be blocked.
210       // 3. The new instance of MetaServerShutdownHandler for RS2 is queued.
211       // 4. The newly assigned .META. region server RS3 was shutdown right after
212       //    it opens the .META. region. So the MetaServerShutdownHandler
213       //    instance For RS1 and RS2 will still be blocked.
214       // 5. The new instance of MetaServerShutdownHandler for RS3 is queued.
215       // 6. Repeat until we run out of MetaServerShutdownHandler worker threads
216       // The solution here is to resubmit a ServerShutdownHandler request to process
217       // user regions on that server so that MetaServerShutdownHandler
218       // executor pool is always available.
219       if (isCarryingRoot() || isCarryingMeta()) { // -ROOT- or .META.
220         this.services.getExecutorService().submit(new ServerShutdownHandler(
221           this.server, this.services, this.deadServers, serverName, false));
222         this.deadServers.add(serverName);
223         return;
224       }
225 
226       // Clean out anything in regions in transition.  Being conservative and
227       // doing after log splitting.  Could do some states before -- OPENING?
228       // OFFLINE? -- and then others after like CLOSING that depend on log
229       // splitting.
230       List<RegionState> regionsInTransition =
231         this.services.getAssignmentManager()
232         .processServerShutdown(this.serverName);
233 
234 
235       // Wait on meta to come online; we need it to progress.
236       // TODO: Best way to hold strictly here?  We should build this retry logic
237       // into the MetaReader operations themselves.
238       // TODO: Is the reading of .META. necessary when the Master has state of
239       // cluster in its head?  It should be possible to do without reading .META.
240       // in all but one case. On split, the RS updates the .META.
241       // table and THEN informs the master of the split via zk nodes in
242       // 'unassigned' dir.  Currently the RS puts ephemeral nodes into zk so if
243       // the regionserver dies, these nodes do not stick around and this server
244       // shutdown processing does fixup (see the fixupDaughters method below).
245       // If we wanted to skip the .META. scan, we'd have to change at least the
246       // final SPLIT message to be permanent in zk so in here we'd know a SPLIT
247       // completed (zk is updated after edits to .META. have gone in).  See
248       // {@link SplitTransaction}.  We'd also have to be figure another way for
249       // doing the below .META. daughters fixup.
250       NavigableMap<HRegionInfo, Result> hris = null;
251       while (!this.server.isStopped()) {
252         try {
253           this.server.getCatalogTracker().waitForMeta();
254           hris = MetaReader.getServerUserRegions(this.server.getCatalogTracker(),
255               this.serverName);
256           break;
257         } catch (InterruptedException e) {
258           Thread.currentThread().interrupt();
259           throw new IOException("Interrupted", e);
260         } catch (IOException ioe) {
261           LOG.info("Received exception accessing META during server shutdown of " +
262               serverName + ", retrying META read", ioe);
263         }
264       }
265 
266       // Skip regions that were in transition unless CLOSING or PENDING_CLOSE
267       for (RegionState rit : regionsInTransition) {
268         if (!rit.isClosing() && !rit.isPendingClose()) {
269           LOG.debug("Removed " + rit.getRegion().getRegionNameAsString() +
270           " from list of regions to assign because in RIT" + " region state: "
271           + rit.getState());
272           if (hris != null) hris.remove(rit.getRegion());
273         }
274       }
275 
276       assert regionsInTransition != null;
277       LOG.info("Reassigning " + ((hris == null)? 0: hris.size()) +
278         " region(s) that " + (serverName == null? "null": serverName)  +
279         " was carrying (skipping " +
280         regionsInTransition.size() +
281         " regions(s) that are already in transition)");
282 
283       // Iterate regions that were on this server and assign them
284       if (hris != null) {
285         for (Map.Entry<HRegionInfo, Result> e: hris.entrySet()) {
286           if (processDeadRegion(e.getKey(), e.getValue(),
287               this.services.getAssignmentManager(),
288               this.server.getCatalogTracker())) {
289             this.services.getAssignmentManager().assign(e.getKey(), true);
290           }
291         }
292       }
293     } finally {
294       this.deadServers.finish(serverName);
295     }
296     LOG.info("Finished processing of shutdown of " + serverName);
297   }
298 
299   /**
300    * Process a dead region from a dead RS.  Checks if the region is disabled
301    * or if the region has a partially completed split.
302    * @param hri
303    * @param result
304    * @param assignmentManager
305    * @param catalogTracker
306    * @return Returns true if specified region should be assigned, false if not.
307    * @throws IOException
308    */
309   public static boolean processDeadRegion(HRegionInfo hri, Result result,
310       AssignmentManager assignmentManager, CatalogTracker catalogTracker)
311   throws IOException {
312     // If table is not disabled but the region is offlined,
313     boolean disabled = assignmentManager.getZKTable().isDisabledTable(
314         hri.getTableNameAsString());
315     if (disabled) return false;
316     if (hri.isOffline() && hri.isSplit()) {
317       LOG.debug("Offlined and split region " + hri.getRegionNameAsString() +
318         "; checking daughter presence");
319       fixupDaughters(result, assignmentManager, catalogTracker);
320       return false;
321     }
322     return true;
323   }
324 
325   /**
326    * Check that daughter regions are up in .META. and if not, add them.
327    * @param hris All regions for this server in meta.
328    * @param result The contents of the parent row in .META.
329    * @throws IOException
330    */
331   static void fixupDaughters(final Result result,
332       final AssignmentManager assignmentManager,
333       final CatalogTracker catalogTracker)
334   throws IOException {
335     fixupDaughter(result, HConstants.SPLITA_QUALIFIER, assignmentManager,
336       catalogTracker);
337     fixupDaughter(result, HConstants.SPLITB_QUALIFIER, assignmentManager,
338       catalogTracker);
339   }
340 
341   /**
342    * Check individual daughter is up in .META.; fixup if its not.
343    * @param result The contents of the parent row in .META.
344    * @param qualifier Which daughter to check for.
345    * @throws IOException
346    */
347   static void fixupDaughter(final Result result, final byte [] qualifier,
348       final AssignmentManager assignmentManager,
349       final CatalogTracker catalogTracker)
350   throws IOException {
351     HRegionInfo daughter =
352       MetaReader.parseHRegionInfoFromCatalogResult(result, qualifier);
353     if (daughter == null) return;
354     if (isDaughterMissing(catalogTracker, daughter)) {
355       LOG.info("Fixup; missing daughter " + daughter.getRegionNameAsString());
356       MetaEditor.addDaughter(catalogTracker, daughter, null);
357 
358       // TODO: Log WARN if the regiondir does not exist in the fs.  If its not
359       // there then something wonky about the split -- things will keep going
360       // but could be missing references to parent region.
361 
362       // And assign it.
363       assignmentManager.assign(daughter, true);
364     } else {
365       LOG.debug("Daughter " + daughter.getRegionNameAsString() + " present");
366     }
367   }
368 
369   /**
370    * Look for presence of the daughter OR of a split of the daughter in .META.
371    * Daughter could have been split over on regionserver before a run of the
372    * catalogJanitor had chance to clear reference from parent.
373    * @param daughter Daughter region to search for.
374    * @throws IOException 
375    */
376   private static boolean isDaughterMissing(final CatalogTracker catalogTracker,
377       final HRegionInfo daughter) throws IOException {
378     FindDaughterVisitor visitor = new FindDaughterVisitor(daughter);
379     // Start the scan at what should be the daughter's row in the .META.
380     // We will either 1., find the daughter or some derivative split of the
381     // daughter (will have same table name and start row at least but will sort
382     // after because has larger regionid -- the regionid is timestamp of region
383     // creation), OR, we will not find anything with same table name and start
384     // row.  If the latter, then assume daughter missing and do fixup.
385     byte [] startrow = daughter.getRegionName();
386     MetaReader.fullScan(catalogTracker, visitor, startrow);
387     return !visitor.foundDaughter();
388   }
389 
390   /**
391    * Looks for daughter.  Sets a flag if daughter or some progeny of daughter
392    * is found up in <code>.META.</code>.
393    */
394   static class FindDaughterVisitor implements MetaReader.Visitor {
395     private final HRegionInfo daughter;
396     private boolean found = false;
397 
398     FindDaughterVisitor(final HRegionInfo daughter) {
399       this.daughter = daughter;
400     }
401 
402     /**
403      * @return True if we found a daughter region during our visiting.
404      */
405     boolean foundDaughter() {
406       return this.found;
407     }
408 
409     @Override
410     public boolean visit(Result r) throws IOException {
411       HRegionInfo hri =
412         MetaReader.parseHRegionInfoFromCatalogResult(r, HConstants.REGIONINFO_QUALIFIER);
413       if (hri == null) {
414         LOG.warn("No serialized HRegionInfo in " + r);
415         return true;
416       }
417       byte [] value = r.getValue(HConstants.CATALOG_FAMILY,
418           HConstants.SERVER_QUALIFIER);
419       // See if daughter is assigned to some server
420       if (value == null) return false;
421 
422       // Now see if we have gone beyond the daughter's startrow.
423       if (!Bytes.equals(daughter.getTableName(),
424           hri.getTableName())) {
425         // We fell into another table.  Stop scanning.
426         return false;
427       }
428       // If our start rows do not compare, move on.
429       if (!Bytes.equals(daughter.getStartKey(), hri.getStartKey())) {
430         return false;
431       }
432       // Else, table name and start rows compare.  It means that the daughter
433       // or some derivative split of the daughter is up in .META.  Daughter
434       // exists.
435       this.found = true;
436       return false;
437     }
438   }
439 }