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.regionserver.handler;
21  
22  import java.io.IOException;
23  import java.util.concurrent.atomic.AtomicBoolean;
24  
25  import org.apache.commons.logging.Log;
26  import org.apache.commons.logging.LogFactory;
27  import org.apache.hadoop.hbase.HRegionInfo;
28  import org.apache.hadoop.hbase.HTableDescriptor;
29  import org.apache.hadoop.hbase.Server;
30  import org.apache.hadoop.hbase.executor.EventHandler;
31  import org.apache.hadoop.hbase.regionserver.HRegion;
32  import org.apache.hadoop.hbase.regionserver.RegionServerAccounting;
33  import org.apache.hadoop.hbase.regionserver.RegionServerServices;
34  import org.apache.hadoop.hbase.regionserver.wal.HLog;
35  import org.apache.hadoop.hbase.util.CancelableProgressable;
36  import org.apache.hadoop.hbase.zookeeper.ZKAssign;
37  import org.apache.zookeeper.KeeperException;
38  
39  /**
40   * Handles opening of a region on a region server.
41   * <p>
42   * This is executed after receiving an OPEN RPC from the master or client.
43   */
44  public class OpenRegionHandler extends EventHandler {
45    private static final Log LOG = LogFactory.getLog(OpenRegionHandler.class);
46  
47    protected final RegionServerServices rsServices;
48  
49    private final HRegionInfo regionInfo;
50    private final HTableDescriptor htd;
51  
52    // We get version of our znode at start of open process and monitor it across
53    // the total open. We'll fail the open if someone hijacks our znode; we can
54    // tell this has happened if version is not as expected.
55    private volatile int version = -1;
56  
57  
58    public OpenRegionHandler(final Server server,
59        final RegionServerServices rsServices, HRegionInfo regionInfo,
60        HTableDescriptor htd) {
61      this(server, rsServices, regionInfo, htd, EventType.M_RS_OPEN_REGION, -1);
62    }
63    public OpenRegionHandler(final Server server,
64        final RegionServerServices rsServices, HRegionInfo regionInfo,
65        HTableDescriptor htd, int version) {
66      this(server, rsServices, regionInfo, htd, EventType.M_RS_OPEN_REGION,
67          version);
68    }
69  
70    protected OpenRegionHandler(final Server server,
71        final RegionServerServices rsServices, final HRegionInfo regionInfo,
72        final HTableDescriptor htd, EventType eventType,
73        final int version) {
74      super(server, eventType);
75      this.rsServices = rsServices;
76      this.regionInfo = regionInfo;
77      this.htd = htd;
78      this.version = version;
79    }
80  
81    public HRegionInfo getRegionInfo() {
82      return regionInfo;
83    }
84  
85    @Override
86    public void process() throws IOException {
87      boolean transitionToFailedOpen = false;
88      boolean openSuccessful = false;
89      try {
90        final String name = regionInfo.getRegionNameAsString();
91        if (this.server.isStopped() || this.rsServices.isStopping()) {
92          return;
93        }
94        final String encodedName = regionInfo.getEncodedName();
95  
96        // Check that this region is not already online
97        HRegion region = this.rsServices.getFromOnlineRegions(encodedName);
98  
99        // Open region.  After a successful open, failures in subsequent
100       // processing needs to do a close as part of cleanup.
101       region = openRegion();
102       if (region == null) {
103         tryTransitionToFailedOpen(regionInfo);
104         transitionToFailedOpen = true;
105         return;
106       }
107       boolean failed = true;
108       if (tickleOpening("post_region_open")) {
109         if (updateMeta(region)) {
110           failed = false;
111         }
112       }
113       if (failed || this.server.isStopped() ||
114           this.rsServices.isStopping()) {
115         cleanupFailedOpen(region);
116         tryTransitionToFailedOpen(regionInfo);
117         transitionToFailedOpen = true;
118         return;
119       }
120 
121       if (!transitionToOpened(region)) {
122         // If we fail to transition to opened, it's because of one of two cases:
123         //    (a) we lost our ZK lease
124         // OR (b) someone else opened the region before us
125         // In either case, we don't need to transition to FAILED_OPEN state.
126         // In case (a), the Master will process us as a dead server. In case
127         // (b) the region is already being handled elsewhere anyway.
128         cleanupFailedOpen(region);
129         transitionToFailedOpen = true;
130         return;
131       }
132       // Successful region open, and add it to OnlineRegions
133       this.rsServices.addToOnlineRegions(region);
134       openSuccessful = true;
135       // Done!  Successful region open
136       LOG.debug("Opened " + name + " on server:" +
137         this.server.getServerName());
138     } finally {
139       this.rsServices.removeFromRegionsInTransition(this.regionInfo);
140       if (!openSuccessful && !transitionToFailedOpen) {
141         tryTransitionToFailedOpen(regionInfo);
142       }
143     }
144   }
145 
146   /**
147    * Update ZK, ROOT or META.  This can take a while if for example the
148    * .META. is not available -- if server hosting .META. crashed and we are
149    * waiting on it to come back -- so run in a thread and keep updating znode
150    * state meantime so master doesn't timeout our region-in-transition.
151    * Caller must cleanup region if this fails.
152    */
153   boolean updateMeta(final HRegion r) {
154     if (this.server.isStopped() || this.rsServices.isStopping()) {
155       return false;
156     }
157     // Object we do wait/notify on.  Make it boolean.  If set, we're done.
158     // Else, wait.
159     final AtomicBoolean signaller = new AtomicBoolean(false);
160     PostOpenDeployTasksThread t = new PostOpenDeployTasksThread(r,
161       this.server, this.rsServices, signaller);
162     t.start();
163     int assignmentTimeout = this.server.getConfiguration().
164       getInt("hbase.master.assignment.timeoutmonitor.period", 10000);
165     // Total timeout for meta edit.  If we fail adding the edit then close out
166     // the region and let it be assigned elsewhere.
167     long timeout = assignmentTimeout * 10;
168     long now = System.currentTimeMillis();
169     long endTime = now + timeout;
170     // Let our period at which we update OPENING state to be be 1/3rd of the
171     // regions-in-transition timeout period.
172     long period = Math.max(1, assignmentTimeout/ 3);
173     long lastUpdate = now;
174     boolean tickleOpening = true;
175     while (!signaller.get() && t.isAlive() && !this.server.isStopped() &&
176         !this.rsServices.isStopping() && (endTime > now)) {
177       long elapsed = now - lastUpdate;
178       if (elapsed > period) {
179         // Only tickle OPENING if postOpenDeployTasks is taking some time.
180         lastUpdate = now;
181         tickleOpening = tickleOpening("post_open_deploy");
182       }
183       synchronized (signaller) {
184         try {
185           signaller.wait(period);
186         } catch (InterruptedException e) {
187           // Go to the loop check.
188         }
189       }
190       now = System.currentTimeMillis();
191     }
192     // Is thread still alive?  We may have left above loop because server is
193     // stopping or we timed out the edit.  Is so, interrupt it.
194     if (t.isAlive()) {
195       if (!signaller.get()) {
196         // Thread still running; interrupt
197         LOG.debug("Interrupting thread " + t);
198         t.interrupt();
199       }
200       try {
201         t.join();
202       } catch (InterruptedException ie) {
203         LOG.warn("Interrupted joining " +
204           r.getRegionInfo().getRegionNameAsString(), ie);
205         Thread.currentThread().interrupt();
206       }
207     }
208 
209     // Was there an exception opening the region?  This should trigger on
210     // InterruptedException too.  If so, we failed.  Even if tickle opening fails
211     // then it is a failure.
212     return ((!Thread.interrupted() && t.getException() == null) && tickleOpening);
213   }
214 
215   /**
216    * Thread to run region post open tasks. Call {@link #getException()} after
217    * the thread finishes to check for exceptions running
218    * {@link RegionServerServices#postOpenDeployTasks(HRegion, org.apache.hadoop.hbase.catalog.CatalogTracker, boolean)}
219    * .
220    */
221   static class PostOpenDeployTasksThread extends Thread {
222     private Exception exception = null;
223     private final Server server;
224     private final RegionServerServices services;
225     private final HRegion region;
226     private final AtomicBoolean signaller;
227 
228     PostOpenDeployTasksThread(final HRegion region, final Server server,
229         final RegionServerServices services, final AtomicBoolean signaller) {
230       super("PostOpenDeployTasks:" + region.getRegionInfo().getEncodedName());
231       this.setDaemon(true);
232       this.server = server;
233       this.services = services;
234       this.region = region;
235       this.signaller = signaller;
236     }
237 
238     public void run() {
239       try {
240         this.services.postOpenDeployTasks(this.region,
241           this.server.getCatalogTracker(), false);
242       } catch (KeeperException e) {
243         server.abort("Exception running postOpenDeployTasks; region=" +
244             this.region.getRegionInfo().getEncodedName(), e);
245       } catch (Exception e) {
246         LOG.warn("Exception running postOpenDeployTasks; region=" +
247           this.region.getRegionInfo().getEncodedName(), e);
248         this.exception = e;
249       }
250       // We're done.  Set flag then wake up anyone waiting on thread to complete.
251       this.signaller.set(true);
252       synchronized (this.signaller) {
253         this.signaller.notify();
254       }
255     }
256 
257     /**
258      * @return Null or the run exception; call this method after thread is done.
259      */
260     Exception getException() {
261       return this.exception;
262     }
263   }
264 
265 
266   /**
267    * @param r Region we're working on.
268    * @return whether znode is successfully transitioned to OPENED state.
269    * @throws IOException
270    */
271   private boolean transitionToOpened(final HRegion r) throws IOException {
272     boolean result = false;
273     HRegionInfo hri = r.getRegionInfo();
274     final String name = hri.getRegionNameAsString();
275     // Finally, Transition ZK node to OPENED
276     try {
277       if (ZKAssign.transitionNodeOpened(this.server.getZooKeeper(), hri,
278           this.server.getServerName(), this.version) == -1) {
279         LOG.warn("Completed the OPEN of region " + name +
280           " but when transitioning from " +
281           " OPENING to OPENED got a version mismatch, someone else clashed " +
282           "so now unassigning -- closing region on server: " +
283           this.server.getServerName());
284       } else {
285         LOG.debug("region transitioned to opened in zookeeper: " +
286           r.getRegionInfo() + ", server: " + this.server.getServerName());
287         result = true;
288       }
289     } catch (KeeperException e) {
290       LOG.error("Failed transitioning node " + name +
291         " from OPENING to OPENED -- closing region", e);
292     }
293     return result;
294   }
295 
296   /**
297    * @param  Region we're working on.
298    * This is not guaranteed to succeed, we just do our best.
299    * @return whether znode is successfully transitioned to FAILED_OPEN state.
300    */
301   private boolean tryTransitionToFailedOpen(final HRegionInfo hri) {
302     boolean result = false;
303     final String name = hri.getRegionNameAsString();
304     try {
305       LOG.info("Opening of region " + hri + " failed, marking as FAILED_OPEN in ZK");
306       if (ZKAssign.transitionNode(
307           this.server.getZooKeeper(), hri,
308           this.server.getServerName(),
309           EventType.RS_ZK_REGION_OPENING,
310           EventType.RS_ZK_REGION_FAILED_OPEN,
311           this.version) == -1) {
312         LOG.warn("Unable to mark region " + hri + " as FAILED_OPEN. " +
313             "It's likely that the master already timed out this open " +
314             "attempt, and thus another RS already has the region.");
315       } else {
316         result = true;
317       }
318     } catch (KeeperException e) {
319       LOG.error("Failed transitioning node " + name +
320         " from OPENING to FAILED_OPEN", e);
321     }
322     return result;
323   }
324 
325   /**
326    * @return Instance of HRegion if successful open else null.
327    */
328   HRegion openRegion() {
329     HRegion region = null;
330     try {
331       // Instantiate the region.  This also periodically tickles our zk OPENING
332       // state so master doesn't timeout this region in transition.
333       region = HRegion.openHRegion(this.regionInfo, this.htd,
334           this.rsServices.getWAL(this.regionInfo), 
335           this.server.getConfiguration(),
336           this.rsServices,
337         new CancelableProgressable() {
338           public boolean progress() {
339             // We may lose the znode ownership during the open.  Currently its
340             // too hard interrupting ongoing region open.  Just let it complete
341             // and check we still have the znode after region open.
342             return tickleOpening("open_region_progress");
343           }
344         });
345     } catch (Throwable t) {
346       // We failed open. Our caller will see the 'null' return value
347       // and transition the node back to FAILED_OPEN. If that fails,
348       // we rely on the Timeout Monitor in the master to reassign.
349       LOG.error(
350           "Failed open of region=" + this.regionInfo.getRegionNameAsString()
351               + ", starting to roll back the global memstore size.", t);
352       // Decrease the global memstore size.
353       if (this.rsServices != null) {
354         RegionServerAccounting rsAccounting =
355           this.rsServices.getRegionServerAccounting();
356         if (rsAccounting != null) {
357           rsAccounting.rollbackRegionReplayEditsSize(this.regionInfo.getRegionName());
358         }
359       }
360     }
361     return region;
362   }
363 
364   void cleanupFailedOpen(final HRegion region) throws IOException {
365     if (region != null) region.close();
366   }
367 
368 
369   /**
370    * Update our OPENING state in zookeeper.
371    * Do this so master doesn't timeout this region-in-transition.
372    * @param context Some context to add to logs if failure
373    * @return True if successful transition.
374    */
375   boolean tickleOpening(final String context) {
376     // If previous checks failed... do not try again.
377     if (!isGoodVersion()) return false;
378     String encodedName = this.regionInfo.getEncodedName();
379     try {
380       this.version =
381         ZKAssign.retransitionNodeOpening(server.getZooKeeper(),
382           this.regionInfo, this.server.getServerName(), this.version);
383     } catch (KeeperException e) {
384       server.abort("Exception refreshing OPENING; region=" + encodedName +
385         ", context=" + context, e);
386       this.version = -1;
387     }
388     boolean b = isGoodVersion();
389     if (!b) {
390       LOG.warn("Failed refreshing OPENING; region=" + encodedName +
391         ", context=" + context);
392     }
393     return b;
394   }
395 
396   private boolean isGoodVersion() {
397     return this.version != -1;
398   }
399 }