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