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.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  
56  
57    public OpenRegionHandler(final Server server,
58        final RegionServerServices rsServices, HRegionInfo regionInfo,
59        HTableDescriptor htd) {
60      this(server, rsServices, regionInfo, htd, EventType.M_RS_OPEN_REGION, -1);
61    }
62    public OpenRegionHandler(final Server server,
63        final RegionServerServices rsServices, HRegionInfo regionInfo,
64        HTableDescriptor htd, int version) {
65      this(server, rsServices, regionInfo, htd, EventType.M_RS_OPEN_REGION,
66          version);
67    }
68  
69    protected OpenRegionHandler(final Server server,
70        final RegionServerServices rsServices, final HRegionInfo regionInfo,
71        final HTableDescriptor htd, EventType eventType,
72        final int version) {
73      super(server, eventType);
74      this.rsServices = rsServices;
75      this.regionInfo = regionInfo;
76      this.htd = htd;
77      this.version = version;
78    }
79  
80    public HRegionInfo getRegionInfo() {
81      return regionInfo;
82    }
83  
84    @Override
85    public void process() throws IOException {
86      boolean transitionToFailedOpen = false;
87      boolean openSuccessful = false;
88      try {
89        final String name = regionInfo.getRegionNameAsString();
90        if (this.server.isStopped() || this.rsServices.isStopping()) {
91          return;
92        }
93        final String encodedName = regionInfo.getEncodedName();
94  
95        // Check that this region is not already online
96        HRegion region = this.rsServices.getFromOnlineRegions(encodedName);
97  
98        // Open region.  After a successful open, failures in subsequent
99        // processing needs to do a close as part of cleanup.
100       region = openRegion();
101       if (region == null) {
102         tryTransitionToFailedOpen(regionInfo);
103         transitionToFailedOpen = true;
104         return;
105       }
106       boolean failed = true;
107       if (tickleOpening("post_region_open")) {
108         if (updateMeta(region)) {
109           failed = false;
110         }
111       }
112       if (failed || this.server.isStopped() ||
113           this.rsServices.isStopping()) {
114         cleanupFailedOpen(region);
115         tryTransitionToFailedOpen(regionInfo);
116         transitionToFailedOpen = true;
117         return;
118       }
119 
120       if (!transitionToOpened(region)) {
121         // If we fail to transition to opened, it's because of one of two cases:
122         //    (a) we lost our ZK lease
123         // OR (b) someone else opened the region before us
124         // In either case, we don't need to transition to FAILED_OPEN state.
125         // In case (a), the Master will process us as a dead server. In case
126         // (b) the region is already being handled elsewhere anyway.
127         cleanupFailedOpen(region);
128         transitionToFailedOpen = true;
129         return;
130       }
131       // Successful region open, and add it to OnlineRegions
132       this.rsServices.addToOnlineRegions(region);
133       openSuccessful = true;
134       // Done!  Successful region open
135       LOG.debug("Opened " + name + " on server:" +
136         this.server.getServerName());
137     } finally {
138       this.rsServices.removeFromRegionsInTransition(this.regionInfo);
139       if (!openSuccessful && !transitionToFailedOpen) {
140         tryTransitionToFailedOpen(regionInfo);
141       }
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    */
220   static class PostOpenDeployTasksThread extends Thread {
221     private Exception exception = null;
222     private final Server server;
223     private final RegionServerServices services;
224     private final HRegion region;
225     private final AtomicBoolean signaller;
226 
227     PostOpenDeployTasksThread(final HRegion region, final Server server,
228         final RegionServerServices services, final AtomicBoolean signaller) {
229       super("PostOpenDeployTasks:" + region.getRegionInfo().getEncodedName());
230       this.setDaemon(true);
231       this.server = server;
232       this.services = services;
233       this.region = region;
234       this.signaller = signaller;
235     }
236 
237     public void run() {
238       try {
239         this.services.postOpenDeployTasks(this.region,
240           this.server.getCatalogTracker(), false);
241       } catch (Exception e) {
242         LOG.warn("Exception running postOpenDeployTasks; region=" +
243           this.region.getRegionInfo().getEncodedName(), e);
244         this.exception = e;
245       }
246       // We're done.  Set flag then wake up anyone waiting on thread to complete.
247       this.signaller.set(true);
248       synchronized (this.signaller) {
249         this.signaller.notify();
250       }
251     }
252 
253     /**
254      * @return Null or the run exception; call this method after thread is done.
255      */
256     Exception getException() {
257       return this.exception;
258     }
259   }
260 
261 
262   /**
263    * @param r Region we're working on.
264    * @return whether znode is successfully transitioned to OPENED state.
265    * @throws IOException
266    */
267   private boolean transitionToOpened(final HRegion r) throws IOException {
268     boolean result = false;
269     HRegionInfo hri = r.getRegionInfo();
270     final String name = hri.getRegionNameAsString();
271     // Finally, Transition ZK node to OPENED
272     try {
273       if (ZKAssign.transitionNodeOpened(this.server.getZooKeeper(), hri,
274           this.server.getServerName(), this.version) == -1) {
275         LOG.warn("Completed the OPEN of region " + name +
276           " but when transitioning from " +
277           " OPENING to OPENED got a version mismatch, someone else clashed " +
278           "so now unassigning -- closing region on server: " +
279           this.server.getServerName());
280       } else {
281         LOG.debug("region transitioned to opened in zookeeper: " +
282           r.getRegionInfo() + ", server: " + this.server.getServerName());
283         result = true;
284       }
285     } catch (KeeperException e) {
286       LOG.error("Failed transitioning node " + name +
287         " from OPENING to OPENED -- closing region", e);
288     }
289     return result;
290   }
291 
292   /**
293    * @param  Region we're working on.
294    * This is not guaranteed to succeed, we just do our best.
295    * @return whether znode is successfully transitioned to FAILED_OPEN state.
296    */
297   private boolean tryTransitionToFailedOpen(final HRegionInfo hri) {
298     boolean result = false;
299     final String name = hri.getRegionNameAsString();
300     try {
301       LOG.info("Opening of region " + hri + " failed, marking as FAILED_OPEN in ZK");
302       if (ZKAssign.transitionNode(
303           this.server.getZooKeeper(), hri,
304           this.server.getServerName(),
305           EventType.RS_ZK_REGION_OPENING,
306           EventType.RS_ZK_REGION_FAILED_OPEN,
307           this.version) == -1) {
308         LOG.warn("Unable to mark region " + hri + " as FAILED_OPEN. " +
309             "It's likely that the master already timed out this open " +
310             "attempt, and thus another RS already has the region.");
311       } else {
312         result = true;
313       }
314     } catch (KeeperException e) {
315       LOG.error("Failed transitioning node " + name +
316         " from OPENING to FAILED_OPEN", e);
317     }
318     return result;
319   }
320 
321   /**
322    * @return Instance of HRegion if successful open else null.
323    */
324   HRegion openRegion() {
325     HRegion region = null;
326     try {
327       // Instantiate the region.  This also periodically tickles our zk OPENING
328       // state so master doesn't timeout this region in transition.
329       region = HRegion.openHRegion(this.regionInfo, this.htd,
330           this.rsServices.getWAL(), this.server.getConfiguration(),
331           this.rsServices,
332         new CancelableProgressable() {
333           public boolean progress() {
334             // We may lose the znode ownership during the open.  Currently its
335             // too hard interrupting ongoing region open.  Just let it complete
336             // and check we still have the znode after region open.
337             return tickleOpening("open_region_progress");
338           }
339         });
340     } catch (Throwable t) {
341       // We failed open. Our caller will see the 'null' return value
342       // and transition the node back to FAILED_OPEN. If that fails,
343       // we rely on the Timeout Monitor in the master to reassign.
344       LOG.error(
345           "Failed open of region=" + this.regionInfo.getRegionNameAsString()
346               + ", starting to roll back the global memstore size.", t);
347       // Decrease the global memstore size.
348       if (this.rsServices != null) {
349         RegionServerAccounting rsAccounting =
350           this.rsServices.getRegionServerAccounting();
351         if (rsAccounting != null) {
352           rsAccounting.rollbackRegionReplayEditsSize(this.regionInfo.getRegionName());
353         }
354       }
355     }
356     return region;
357   }
358 
359   void cleanupFailedOpen(final HRegion region) throws IOException {
360     if (region != null) region.close();
361   }
362 
363 
364   /**
365    * Update our OPENING state in zookeeper.
366    * Do this so master doesn't timeout this region-in-transition.
367    * @param context Some context to add to logs if failure
368    * @return True if successful transition.
369    */
370   boolean tickleOpening(final String context) {
371     // If previous checks failed... do not try again.
372     if (!isGoodVersion()) return false;
373     String encodedName = this.regionInfo.getEncodedName();
374     try {
375       this.version =
376         ZKAssign.retransitionNodeOpening(server.getZooKeeper(),
377           this.regionInfo, this.server.getServerName(), this.version);
378     } catch (KeeperException e) {
379       server.abort("Exception refreshing OPENING; region=" + encodedName +
380         ", context=" + context, e);
381       this.version = -1;
382     }
383     boolean b = isGoodVersion();
384     if (!b) {
385       LOG.warn("Failed refreshing OPENING; region=" + encodedName +
386         ", context=" + context);
387     }
388     return b;
389   }
390 
391   private boolean isGoodVersion() {
392     return this.version != -1;
393   }
394 }