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;
21  
22  import java.io.DataInput;
23  import java.io.DataOutput;
24  import java.io.EOFException;
25  import java.io.IOException;
26  import java.net.ConnectException;
27  import java.util.ArrayList;
28  import java.util.HashMap;
29  import java.util.Iterator;
30  import java.util.List;
31  import java.util.Map;
32  import java.util.NavigableMap;
33  import java.util.Set;
34  import java.util.SortedMap;
35  import java.util.TreeMap;
36  import java.util.TreeSet;
37  import java.util.concurrent.ConcurrentSkipListMap;
38  import java.util.concurrent.atomic.AtomicInteger;
39  
40  import org.apache.commons.logging.Log;
41  import org.apache.commons.logging.LogFactory;
42  import org.apache.hadoop.conf.Configuration;
43  import org.apache.hadoop.hbase.Chore;
44  import org.apache.hadoop.hbase.HConstants;
45  import org.apache.hadoop.hbase.HRegionInfo;
46  import org.apache.hadoop.hbase.HServerAddress;
47  import org.apache.hadoop.hbase.HServerInfo;
48  import org.apache.hadoop.hbase.HTableDescriptor;
49  import org.apache.hadoop.hbase.NotServingRegionException;
50  import org.apache.hadoop.hbase.Server;
51  import org.apache.hadoop.hbase.Stoppable;
52  import org.apache.hadoop.hbase.catalog.CatalogTracker;
53  import org.apache.hadoop.hbase.catalog.MetaReader;
54  import org.apache.hadoop.hbase.catalog.RootLocationEditor;
55  import org.apache.hadoop.hbase.client.Result;
56  import org.apache.hadoop.hbase.executor.ExecutorService;
57  import org.apache.hadoop.hbase.executor.RegionTransitionData;
58  import org.apache.hadoop.hbase.executor.EventHandler.EventType;
59  import org.apache.hadoop.hbase.master.LoadBalancer.RegionPlan;
60  import org.apache.hadoop.hbase.master.handler.ClosedRegionHandler;
61  import org.apache.hadoop.hbase.master.handler.OpenedRegionHandler;
62  import org.apache.hadoop.hbase.master.handler.ServerShutdownHandler;
63  import org.apache.hadoop.hbase.util.Bytes;
64  import org.apache.hadoop.hbase.util.Pair;
65  import org.apache.hadoop.hbase.util.Threads;
66  import org.apache.hadoop.hbase.zookeeper.ZKAssign;
67  import org.apache.hadoop.hbase.zookeeper.ZKTable;
68  import org.apache.hadoop.hbase.zookeeper.ZKUtil;
69  import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
70  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
71  import org.apache.hadoop.hbase.zookeeper.ZKUtil.NodeAndData;
72  import org.apache.hadoop.io.Writable;
73  import org.apache.hadoop.ipc.RemoteException;
74  import org.apache.zookeeper.AsyncCallback;
75  import org.apache.zookeeper.KeeperException;
76  import org.apache.zookeeper.KeeperException.NoNodeException;
77  import org.apache.zookeeper.data.Stat;
78  
79  /**
80   * Manages and performs region assignment.
81   * <p>
82   * Monitors ZooKeeper for events related to regions in transition.
83   * <p>
84   * Handles existing regions in transition during master failover.
85   */
86  public class AssignmentManager extends ZooKeeperListener {
87    private static final Log LOG = LogFactory.getLog(AssignmentManager.class);
88  
89    protected Server master;
90  
91    private ServerManager serverManager;
92  
93    private CatalogTracker catalogTracker;
94  
95    private TimeoutMonitor timeoutMonitor;
96  
97    /*
98     * Maximum times we recurse an assignment.  See below in {@link #assign()}.
99     */
100   private final int maximumAssignmentAttempts;
101 
102   /**
103    * Regions currently in transition.  Map of encoded region names to the master
104    * in-memory state for that region.
105    */
106   final ConcurrentSkipListMap<String, RegionState> regionsInTransition =
107     new ConcurrentSkipListMap<String, RegionState>();
108 
109   /** Plans for region movement. Key is the encoded version of a region name*/
110   // TODO: When do plans get cleaned out?  Ever? In server open and in server
111   // shutdown processing -- St.Ack
112   // All access to this Map must be synchronized.
113   final NavigableMap<String, RegionPlan> regionPlans =
114     new TreeMap<String, RegionPlan>();
115 
116   private final ZKTable zkTable;
117 
118   /**
119    * Server to regions assignment map.
120    * Contains the set of regions currently assigned to a given server.
121    * This Map and {@link #regions} are tied.  Always update this in tandem
122    * with the other under a lock on {@link #regions}
123    * @see #regions
124    */
125   private final NavigableMap<HServerInfo, List<HRegionInfo>> servers =
126     new TreeMap<HServerInfo, List<HRegionInfo>>();
127 
128   /**
129    * Region to server assignment map.
130    * Contains the server a given region is currently assigned to.
131    * This Map and {@link #servers} are tied.  Always update this in tandem
132    * with the other under a lock on {@link #regions}
133    * @see #servers
134    */
135   private final SortedMap<HRegionInfo,HServerInfo> regions =
136     new TreeMap<HRegionInfo,HServerInfo>();
137 
138   private final ExecutorService executorService;
139 
140   /**
141    * Constructs a new assignment manager.
142    *
143    * @param master
144    * @param serverManager
145    * @param catalogTracker
146    * @param service
147    * @throws KeeperException
148    */
149   public AssignmentManager(Server master, ServerManager serverManager,
150       CatalogTracker catalogTracker, final ExecutorService service)
151   throws KeeperException {
152     super(master.getZooKeeper());
153     this.master = master;
154     this.serverManager = serverManager;
155     this.catalogTracker = catalogTracker;
156     this.executorService = service;
157     Configuration conf = master.getConfiguration();
158     this.timeoutMonitor = new TimeoutMonitor(
159       conf.getInt("hbase.master.assignment.timeoutmonitor.period", 10000),
160       master,
161       conf.getInt("hbase.master.assignment.timeoutmonitor.timeout", 30000));
162     Threads.setDaemonThreadRunning(timeoutMonitor,
163       master.getServerName() + ".timeoutMonitor");
164     this.zkTable = new ZKTable(this.master.getZooKeeper());
165     this.maximumAssignmentAttempts =
166       this.master.getConfiguration().getInt("hbase.assignment.maximum.attempts", 10);
167   }
168 
169   /**
170    * @return Instance of ZKTable.
171    */
172   public ZKTable getZKTable() {
173     // These are 'expensive' to make involving trip to zk ensemble so allow
174     // sharing.
175     return this.zkTable;
176   }
177 
178   /**
179    * Reset all unassigned znodes.  Called on startup of master.
180    * Call {@link #assignAllUserRegions()} after root and meta have been assigned.
181    * @throws IOException
182    * @throws KeeperException
183    */
184   void cleanoutUnassigned() throws IOException, KeeperException {
185     // Cleanup any existing ZK nodes and start watching
186     ZKAssign.deleteAllNodes(watcher);
187     ZKUtil.listChildrenAndWatchForNewChildren(this.watcher,
188       this.watcher.assignmentZNode);
189   }
190 
191   /**
192    * Handle failover.  Restore state from META and ZK.  Handle any regions in
193    * transition.  Presumes <code>.META.</code> and <code>-ROOT-</code> deployed.
194    * @throws KeeperException
195    * @throws IOException
196    */
197   void processFailover() throws KeeperException, IOException {
198     // Concurrency note: In the below the accesses on regionsInTransition are
199     // outside of a synchronization block where usually all accesses to RIT are
200     // synchronized.  The presumption is that in this case it is safe since this
201     // method is being played by a single thread on startup.
202 
203     // TODO: Check list of user regions and their assignments against regionservers.
204     // TODO: Regions that have a null location and are not in regionsInTransitions
205     // need to be handled.
206 
207     // Scan META to build list of existing regions, servers, and assignment
208     // Returns servers who have not checked in (assumed dead) and their regions
209     Map<HServerInfo,List<Pair<HRegionInfo,Result>>> deadServers =
210       rebuildUserRegions();
211     // Process list of dead servers
212     processDeadServers(deadServers);
213     // Check existing regions in transition
214     List<String> nodes = ZKUtil.listChildrenAndWatchForNewChildren(watcher,
215         watcher.assignmentZNode);
216     if (nodes.isEmpty()) {
217       LOG.info("No regions in transition in ZK to process on failover");
218       return;
219     }
220     LOG.info("Failed-over master needs to process " + nodes.size() +
221         " regions in transition");
222     for (String encodedRegionName: nodes) {
223       processRegionInTransition(encodedRegionName, null);
224     }
225   }
226 
227   /**
228    * If region is up in zk in transition, then do fixup and block and wait until
229    * the region is assigned and out of transition.  Used on startup for
230    * catalog regions.
231    * @param hri Region to look for.
232    * @return True if we processed a region in transition else false if region
233    * was not up in zk in transition.
234    * @throws InterruptedException
235    * @throws KeeperException
236    * @throws IOException
237    */
238   boolean processRegionInTransitionAndBlockUntilAssigned(final HRegionInfo hri)
239   throws InterruptedException, KeeperException, IOException {
240     boolean intransistion = processRegionInTransition(hri.getEncodedName(), hri);
241     if (!intransistion) return intransistion;
242     synchronized(this.regionsInTransition) {
243       while (!this.master.isStopped() &&
244           this.regionsInTransition.containsKey(hri.getEncodedName())) {
245         this.regionsInTransition.wait();
246       }
247     }
248     return intransistion;
249   }
250 
251   /**
252    * Process failover of <code>encodedName</code>.  Look in
253    * @param encodedRegionName Region to process failover for.
254    * @param encodedRegionName RegionInfo.  If null we'll go get it from meta table.
255    * @return
256    * @throws KeeperException
257    * @throws IOException
258    */
259   boolean processRegionInTransition(final String encodedRegionName,
260       final HRegionInfo regionInfo)
261   throws KeeperException, IOException {
262     RegionTransitionData data = ZKAssign.getData(watcher, encodedRegionName);
263     if (data == null) return false;
264     HRegionInfo hri = regionInfo;
265     if (hri == null) {
266       Pair<HRegionInfo, HServerAddress> p =
267         MetaReader.getRegion(catalogTracker, data.getRegionName());
268       if (p == null) return false;
269       hri = p.getFirst();
270     }
271     processRegionsInTransition(data, hri);
272     return true;
273   }
274 
275   void processRegionsInTransition(final RegionTransitionData data,
276       final HRegionInfo regionInfo)
277   throws KeeperException {
278     String encodedRegionName = regionInfo.getEncodedName();
279     LOG.info("Processing region " + regionInfo.getRegionNameAsString() +
280       " in state " + data.getEventType());
281     synchronized (regionsInTransition) {
282       switch (data.getEventType()) {
283       case RS_ZK_REGION_CLOSING:
284         // Just insert region into RIT.
285         // If this never updates the timeout will trigger new assignment
286         regionsInTransition.put(encodedRegionName, new RegionState(
287             regionInfo, RegionState.State.CLOSING, data.getStamp()));
288         break;
289 
290       case RS_ZK_REGION_CLOSED:
291         // Region is closed, insert into RIT and handle it
292         regionsInTransition.put(encodedRegionName, new RegionState(
293             regionInfo, RegionState.State.CLOSED, data.getStamp()));
294         new ClosedRegionHandler(master, this, regionInfo).process();
295         break;
296 
297       case M_ZK_REGION_OFFLINE:
298         // Region is offline, insert into RIT and handle it like a closed
299         regionsInTransition.put(encodedRegionName, new RegionState(
300             regionInfo, RegionState.State.OFFLINE, data.getStamp()));
301         new ClosedRegionHandler(master, this, regionInfo).process();
302         break;
303 
304       case RS_ZK_REGION_OPENING:
305         // Just insert region into RIT
306         // If this never updates the timeout will trigger new assignment
307         regionsInTransition.put(encodedRegionName, new RegionState(
308             regionInfo, RegionState.State.OPENING, data.getStamp()));
309         break;
310 
311       case RS_ZK_REGION_OPENED:
312         // Region is opened, insert into RIT and handle it
313         regionsInTransition.put(encodedRegionName, new RegionState(
314             regionInfo, RegionState.State.OPENING, data.getStamp()));
315         HServerInfo hsi = serverManager.getServerInfo(data.getServerName());
316         // hsi could be null if this server is no longer online.  If
317         // that the case, just let this RIT timeout; it'll be assigned
318         // to new server then.
319         if (hsi == null) {
320           LOG.warn("Region in transition " + regionInfo.getEncodedName() +
321             " references a server no longer up " + data.getServerName() +
322             "; letting RIT timeout so will be assigned elsewhere");
323           break;
324         }
325         new OpenedRegionHandler(master, this, regionInfo, hsi).process();
326         break;
327       }
328     }
329   }
330 
331   /**
332    * Handles various states an unassigned node can be in.
333    * <p>
334    * Method is called when a state change is suspected for an unassigned node.
335    * <p>
336    * This deals with skipped transitions (we got a CLOSED but didn't see CLOSING
337    * yet).
338    * @param data
339    */
340   private void handleRegion(final RegionTransitionData data) {
341     synchronized(regionsInTransition) {
342       if (data == null || data.getServerName() == null) {
343         LOG.warn("Unexpected NULL input " + data);
344         return;
345       }
346       // Check if this is a special HBCK transition
347       if (data.getServerName().equals(HConstants.HBCK_CODE_NAME)) {
348         handleHBCK(data);
349         return;
350       }
351       // Verify this is a known server
352       if (!serverManager.isServerOnline(data.getServerName()) &&
353           !this.master.getServerName().equals(data.getServerName())) {
354         LOG.warn("Attempted to handle region transition for server but " +
355           "server is not online: " + data.getRegionName());
356         return;
357       }
358       String encodedName = HRegionInfo.encodeRegionName(data.getRegionName());
359       String prettyPrintedRegionName = HRegionInfo.prettyPrint(encodedName);
360       LOG.debug("Handling transition=" + data.getEventType() +
361         ", server=" + data.getServerName() + ", region=" + prettyPrintedRegionName);
362       RegionState regionState = regionsInTransition.get(encodedName);
363       switch (data.getEventType()) {
364         case M_ZK_REGION_OFFLINE:
365           // Nothing to do.
366           break;
367 
368         case RS_ZK_REGION_CLOSING:
369           // Should see CLOSING after we have asked it to CLOSE or additional
370           // times after already being in state of CLOSING
371           if (regionState == null ||
372               (!regionState.isPendingClose() && !regionState.isClosing())) {
373             LOG.warn("Received CLOSING for region " + prettyPrintedRegionName +
374               " from server " + data.getServerName() + " but region was in " +
375               " the state " + regionState + " and not " +
376               "in expected PENDING_CLOSE or CLOSING states");
377             return;
378           }
379           // Transition to CLOSING (or update stamp if already CLOSING)
380           regionState.update(RegionState.State.CLOSING, data.getStamp());
381           break;
382 
383         case RS_ZK_REGION_CLOSED:
384           // Should see CLOSED after CLOSING but possible after PENDING_CLOSE
385           if (regionState == null ||
386               (!regionState.isPendingClose() && !regionState.isClosing())) {
387             LOG.warn("Received CLOSED for region " + prettyPrintedRegionName +
388                 " from server " + data.getServerName() + " but region was in " +
389                 " the state " + regionState + " and not " +
390                 "in expected PENDING_CLOSE or CLOSING states");
391             return;
392           }
393           // Handle CLOSED by assigning elsewhere or stopping if a disable
394           // If we got here all is good.  Need to update RegionState -- else
395           // what follows will fail because not in expected state.
396           regionState.update(RegionState.State.CLOSED, data.getStamp());
397           this.executorService.submit(new ClosedRegionHandler(master,
398             this, regionState.getRegion()));
399           break;
400 
401         case RS_ZK_REGION_OPENING:
402           // Should see OPENING after we have asked it to OPEN or additional
403           // times after already being in state of OPENING
404           if(regionState == null ||
405               (!regionState.isPendingOpen() && !regionState.isOpening())) {
406             LOG.warn("Received OPENING for region " +
407                 prettyPrintedRegionName +
408                 " from server " + data.getServerName() + " but region was in " +
409                 " the state " + regionState + " and not " +
410                 "in expected PENDING_OPEN or OPENING states");
411             return;
412           }
413           // Transition to OPENING (or update stamp if already OPENING)
414           regionState.update(RegionState.State.OPENING, data.getStamp());
415           break;
416 
417         case RS_ZK_REGION_OPENED:
418           // Should see OPENED after OPENING but possible after PENDING_OPEN
419           if(regionState == null ||
420               (!regionState.isPendingOpen() && !regionState.isOpening())) {
421             LOG.warn("Received OPENED for region " +
422                 prettyPrintedRegionName +
423                 " from server " + data.getServerName() + " but region was in " +
424                 " the state " + regionState + " and not " +
425                 "in expected PENDING_OPEN or OPENING states");
426             return;
427           }
428           // Handle OPENED by removing from transition and deleted zk node
429           regionState.update(RegionState.State.OPEN, data.getStamp());
430           this.executorService.submit(
431             new OpenedRegionHandler(master, this, regionState.getRegion(),
432               this.serverManager.getServerInfo(data.getServerName())));
433           break;
434       }
435     }
436   }
437 
438   /**
439    * Handle a ZK unassigned node transition triggered by HBCK repair tool.
440    * <p>
441    * This is handled in a separate code path because it breaks the normal rules.
442    * @param data
443    */
444   private void handleHBCK(RegionTransitionData data) {
445     String encodedName = HRegionInfo.encodeRegionName(data.getRegionName());
446     LOG.info("Handling HBCK triggered transition=" + data.getEventType() +
447       ", server=" + data.getServerName() + ", region=" +
448       HRegionInfo.prettyPrint(encodedName));
449     RegionState regionState = regionsInTransition.get(encodedName);
450     switch (data.getEventType()) {
451       case M_ZK_REGION_OFFLINE:
452         HRegionInfo regionInfo = null;
453         if (regionState != null) {
454           regionInfo = regionState.getRegion();
455         } else {
456           try {
457             regionInfo = MetaReader.getRegion(catalogTracker,
458                 data.getRegionName()).getFirst();
459           } catch (IOException e) {
460             LOG.info("Exception reading META doing HBCK repair operation", e);
461             return;
462           }
463         }
464         LOG.info("HBCK repair is triggering assignment of region=" +
465             regionInfo.getRegionNameAsString());
466         // trigger assign, node is already in OFFLINE so don't need to update ZK
467         assign(regionInfo, false);
468         break;
469 
470       default:
471         LOG.warn("Received unexpected region state from HBCK (" +
472             data.getEventType() + ")");
473         break;
474     }
475   }
476 
477   // ZooKeeper events
478 
479   /**
480    * New unassigned node has been created.
481    *
482    * <p>This happens when an RS begins the OPENING or CLOSING of a region by
483    * creating an unassigned node.
484    *
485    * <p>When this happens we must:
486    * <ol>
487    *   <li>Watch the node for further events</li>
488    *   <li>Read and handle the state in the node</li>
489    * </ol>
490    */
491   @Override
492   public void nodeCreated(String path) {
493     if(path.startsWith(watcher.assignmentZNode)) {
494       synchronized(regionsInTransition) {
495         try {
496           RegionTransitionData data = ZKAssign.getData(watcher, path);
497           if(data == null) {
498             return;
499           }
500           handleRegion(data);
501         } catch (KeeperException e) {
502           master.abort("Unexpected ZK exception reading unassigned node data", e);
503         }
504       }
505     }
506   }
507 
508   /**
509    * Existing unassigned node has had data changed.
510    *
511    * <p>This happens when an RS transitions from OFFLINE to OPENING, or between
512    * OPENING/OPENED and CLOSING/CLOSED.
513    *
514    * <p>When this happens we must:
515    * <ol>
516    *   <li>Watch the node for further events</li>
517    *   <li>Read and handle the state in the node</li>
518    * </ol>
519    */
520   @Override
521   public void nodeDataChanged(String path) {
522     if(path.startsWith(watcher.assignmentZNode)) {
523       synchronized(regionsInTransition) {
524         try {
525           RegionTransitionData data = ZKAssign.getData(watcher, path);
526           if(data == null) {
527             return;
528           }
529           handleRegion(data);
530         } catch (KeeperException e) {
531           master.abort("Unexpected ZK exception reading unassigned node data", e);
532         }
533       }
534     }
535   }
536 
537   /**
538    * New unassigned node has been created.
539    *
540    * <p>This happens when an RS begins the OPENING or CLOSING of a region by
541    * creating an unassigned node.
542    *
543    * <p>When this happens we must:
544    * <ol>
545    *   <li>Watch the node for further children changed events</li>
546    *   <li>Watch all new children for changed events</li>
547    *   <li>Read all children and handle them</li>
548    * </ol>
549    */
550   @Override
551   public void nodeChildrenChanged(String path) {
552     if(path.equals(watcher.assignmentZNode)) {
553       synchronized(regionsInTransition) {
554         try {
555           List<NodeAndData> newNodes = ZKUtil.watchAndGetNewChildren(watcher,
556               watcher.assignmentZNode);
557           for(NodeAndData newNode : newNodes) {
558             LOG.debug("Handling new unassigned node: " + newNode);
559             handleRegion(RegionTransitionData.fromBytes(newNode.getData()));
560           }
561         } catch(KeeperException e) {
562           master.abort("Unexpected ZK exception reading unassigned children", e);
563         }
564       }
565     }
566   }
567 
568   /**
569    * Marks the region as online.  Removes it from regions in transition and
570    * updates the in-memory assignment information.
571    * <p>
572    * Used when a region has been successfully opened on a region server.
573    * @param regionInfo
574    * @param serverInfo
575    */
576   public void regionOnline(HRegionInfo regionInfo, HServerInfo serverInfo) {
577     synchronized (this.regionsInTransition) {
578       RegionState rs =
579         this.regionsInTransition.remove(regionInfo.getEncodedName());
580       if (rs != null) {
581         this.regionsInTransition.notifyAll();
582       }
583     }
584     synchronized (this.regions) {
585       // Add check
586       HServerInfo hsi = this.regions.get(regionInfo);
587       if (hsi != null) LOG.warn("Overwriting " + regionInfo.getEncodedName() +
588         " on " + hsi);
589       this.regions.put(regionInfo, serverInfo);
590       addToServers(serverInfo, regionInfo);
591       this.regions.notifyAll();
592     }
593     // Remove plan if one.
594     clearRegionPlan(regionInfo);
595     // Update timers for all regions in transition going against this server.
596     updateTimers(serverInfo);
597   }
598 
599   /**
600    * Touch timers for all regions in transition that have the passed
601    * <code>hsi</code> in common.
602    * Call this method whenever a server checks in.  Doing so helps the case where
603    * a new regionserver has joined the cluster and its been given 1k regions to
604    * open.  If this method is tickled every time the region reports in a
605    * successful open then the 1k-th region won't be timed out just because its
606    * sitting behind the open of 999 other regions.  This method is NOT used
607    * as part of bulk assign -- there we have a different mechanism for extending
608    * the regions in transition timer (we turn it off temporarily -- because
609    * there is no regionplan involved when bulk assigning.
610    * @param hsi
611    */
612   private void updateTimers(final HServerInfo hsi) {
613     // This loop could be expensive.
614     // First make a copy of current regionPlan rather than hold sync while
615     // looping because holding sync can cause deadlock.  Its ok in this loop
616     // if the Map we're going against is a little stale
617     Map<String, RegionPlan> copy = new HashMap<String, RegionPlan>();
618     synchronized(this.regionPlans) {
619       copy.putAll(this.regionPlans);
620     }
621     for (Map.Entry<String, RegionPlan> e: copy.entrySet()) {
622       if (!e.getValue().getDestination().equals(hsi)) continue;
623       RegionState rs = null;
624       synchronized (this.regionsInTransition) {
625         rs = this.regionsInTransition.get(e.getKey());
626       }
627       if (rs == null) continue;
628       synchronized (rs) {
629         rs.update(rs.getState());
630       }
631     }
632   }
633 
634   /**
635    * Marks the region as offline.  Removes it from regions in transition and
636    * removes in-memory assignment information.
637    * <p>
638    * Used when a region has been closed and should remain closed.
639    * @param regionInfo
640    */
641   public void regionOffline(final HRegionInfo regionInfo) {
642     synchronized(this.regionsInTransition) {
643       if (this.regionsInTransition.remove(regionInfo.getEncodedName()) != null) {
644         this.regionsInTransition.notifyAll();
645       }
646     }
647     // remove the region plan as well just in case.
648     clearRegionPlan(regionInfo);
649     setOffline(regionInfo);
650   }
651 
652   /**
653    * Sets the region as offline by removing in-memory assignment information but
654    * retaining transition information.
655    * <p>
656    * Used when a region has been closed but should be reassigned.
657    * @param regionInfo
658    */
659   public void setOffline(HRegionInfo regionInfo) {
660     synchronized (this.regions) {
661       HServerInfo serverInfo = this.regions.remove(regionInfo);
662       if (serverInfo == null) return;
663       List<HRegionInfo> serverRegions = this.servers.get(serverInfo);
664       if (!serverRegions.remove(regionInfo)) {
665         LOG.warn("No " + regionInfo + " on " + serverInfo);
666       }
667     }
668   }
669 
670   public void offlineDisabledRegion(HRegionInfo regionInfo) {
671     // Disabling so should not be reassigned, just delete the CLOSED node
672     LOG.debug("Table being disabled so deleting ZK node and removing from " +
673         "regions in transition, skipping assignment of region " +
674           regionInfo.getRegionNameAsString());
675     try {
676       if (!ZKAssign.deleteClosedNode(watcher, regionInfo.getEncodedName())) {
677         // Could also be in OFFLINE mode
678         ZKAssign.deleteOfflineNode(watcher, regionInfo.getEncodedName());
679       }
680     } catch (KeeperException.NoNodeException nne) {
681       LOG.debug("Tried to delete closed node for " + regionInfo + " but it " +
682           "does not exist so just offlining");
683     } catch (KeeperException e) {
684       this.master.abort("Error deleting CLOSED node in ZK", e);
685     }
686     regionOffline(regionInfo);
687   }
688 
689   // Assignment methods
690 
691   /**
692    * Assigns the specified region.
693    * <p>
694    * If a RegionPlan is available with a valid destination then it will be used
695    * to determine what server region is assigned to.  If no RegionPlan is
696    * available, region will be assigned to a random available server.
697    * <p>
698    * Updates the RegionState and sends the OPEN RPC.
699    * <p>
700    * This will only succeed if the region is in transition and in a CLOSED or
701    * OFFLINE state or not in transition (in-memory not zk), and of course, the
702    * chosen server is up and running (It may have just crashed!).  If the
703    * in-memory checks pass, the zk node is forced to OFFLINE before assigning.
704    *
705    * @param region server to be assigned
706    * @param setOfflineInZK whether ZK node should be created/transitioned to an
707    *                       OFFLINE state before assigning the region
708    */
709   public void assign(HRegionInfo region, boolean setOfflineInZK) {
710     assign(region, setOfflineInZK, false);
711   }
712 
713   public void assign(HRegionInfo region, boolean setOfflineInZK,
714       boolean forceNewPlan) {
715     String tableName = region.getTableDesc().getNameAsString();
716     boolean disabled = this.zkTable.isDisabledTable(tableName);
717     if (disabled || this.zkTable.isDisablingTable(tableName)) {
718       LOG.info("Table " + tableName + (disabled? " disabled;": " disabling;") +
719         " skipping assign of " + region.getRegionNameAsString());
720       offlineDisabledRegion(region);
721       return;
722     }
723     if (this.serverManager.isClusterShutdown()) {
724       LOG.info("Cluster shutdown is set; skipping assign of " +
725         region.getRegionNameAsString());
726       return;
727     }
728     RegionState state = addToRegionsInTransition(region);
729     synchronized (state) {
730       assign(state, setOfflineInZK, forceNewPlan);
731     }
732   }
733 
734   /**
735    * Bulk assign regions to <code>destination</code>.  If we fail in any way,
736    * we'll abort the server.
737    * @param destination
738    * @param regions Regions to assign.
739    */
740   void assign(final HServerInfo destination,
741       final List<HRegionInfo> regions) {
742     LOG.debug("Bulk assigning " + regions.size() + " region(s) to " +
743       destination.getServerName());
744 
745     List<RegionState> states = new ArrayList<RegionState>(regions.size());
746     synchronized (this.regionsInTransition) {
747       for (HRegionInfo region: regions) {
748         states.add(forceRegionStateToOffline(region));
749       }
750     }
751     // Presumption is that only this thread will be updating the state at this
752     // time; i.e. handlers on backend won't be trying to set it to OPEN, etc.
753     AtomicInteger counter = new AtomicInteger(0);
754     CreateUnassignedAsyncCallback cb =
755       new CreateUnassignedAsyncCallback(this.watcher, destination, counter);
756     for (RegionState state: states) {
757       if (!asyncSetOfflineInZooKeeper(state, cb, state)) {
758         return;
759       }
760     }
761     // Wait until all unassigned nodes have been put up and watchers set.
762     int total = regions.size();
763     for (int oldCounter = 0; true;) {
764       int count = counter.get();
765       if (oldCounter != count) {
766         LOG.info(destination.getServerName() + " unassigned znodes=" + count +
767           " of total=" + total);
768         oldCounter = count;
769       }
770       if (count == total) break;
771       Threads.sleep(1);
772     }
773     // Move on to open regions.
774     try {
775       // Send OPEN RPC. This can fail if the server on other end is is not up.
776       this.serverManager.sendRegionOpen(destination, regions);
777     } catch (Throwable t) {
778       this.master.abort("Failed assignment of regions to " + destination, t);
779       return;
780     }
781     LOG.debug("Bulk assigning done for " + destination.getServerName());
782   }
783 
784   /**
785    * Callback handler for create unassigned znodes used during bulk assign.
786    */
787   static class CreateUnassignedAsyncCallback implements AsyncCallback.StringCallback {
788     private final Log LOG = LogFactory.getLog(CreateUnassignedAsyncCallback.class);
789     private final ZooKeeperWatcher zkw;
790     private final HServerInfo destination;
791     private final AtomicInteger counter;
792 
793     CreateUnassignedAsyncCallback(final ZooKeeperWatcher zkw,
794         final HServerInfo destination, final AtomicInteger counter) {
795       this.zkw = zkw;
796       this.destination = destination;
797       this.counter = counter;
798     }
799 
800     @Override
801     public void processResult(int rc, String path, Object ctx, String name) {
802       if (rc != 0) {
803         // Thisis resultcode.  If non-zero, need to resubmit.
804         LOG.warn("rc != 0 for " + path + " -- retryable connectionloss -- " +
805           "FIX see http://wiki.apache.org/hadoop/ZooKeeper/FAQ#A2");
806         this.zkw.abort("Connectionloss writing unassigned at " + path +
807           ", rc=" + rc, null);
808         return;
809       }
810       LOG.debug("rs=" + (RegionState)ctx + ", server=" + this.destination.getServerName());
811       // Async exists to set a watcher so we'll get triggered when
812       // unassigned node changes.
813       this.zkw.getZooKeeper().exists(path, this.zkw,
814         new ExistsUnassignedAsyncCallback(this.counter), ctx);
815     }
816   }
817 
818   /**
819    * Callback handler for the exists call that sets watcher on unassigned znodes.
820    * Used during bulk assign on startup.
821    */
822   static class ExistsUnassignedAsyncCallback implements AsyncCallback.StatCallback {
823     private final Log LOG = LogFactory.getLog(ExistsUnassignedAsyncCallback.class);
824     private final AtomicInteger counter;
825 
826     ExistsUnassignedAsyncCallback(final AtomicInteger counter) {
827       this.counter = counter;
828     }
829 
830     @Override
831     public void processResult(int rc, String path, Object ctx, Stat stat) {
832       if (rc != 0) {
833         // Thisis resultcode.  If non-zero, need to resubmit.
834         LOG.warn("rc != 0 for " + path + " -- retryable connectionloss -- " +
835           "FIX see http://wiki.apache.org/hadoop/ZooKeeper/FAQ#A2");
836         return;
837       }
838       RegionState state = (RegionState)ctx;
839       LOG.debug("rs=" + state);
840       // Transition RegionState to PENDING_OPEN here in master; means we've
841       // sent the open.  We're a little ahead of ourselves here since we've not
842       // yet sent out the actual open but putting this state change after the
843       // call to open risks our writing PENDING_OPEN after state has been moved
844       // to OPENING by the regionserver.
845       state.update(RegionState.State.PENDING_OPEN);
846       this.counter.addAndGet(1);
847     }
848   }
849 
850   /**
851    * @param region
852    * @return
853    */
854   private RegionState addToRegionsInTransition(final HRegionInfo region) {
855     synchronized (regionsInTransition) {
856       return forceRegionStateToOffline(region);
857     }
858   }
859 
860   /**
861    * Sets regions {@link RegionState} to {@link RegionState.State#OFFLINE}.
862    * Caller must hold lock on this.regionsInTransition.
863    * @param region
864    * @return Amended RegionState.
865    */
866   private RegionState forceRegionStateToOffline(final HRegionInfo region) {
867     String encodedName = region.getEncodedName();
868     RegionState state = this.regionsInTransition.get(encodedName);
869     if (state == null) {
870       state = new RegionState(region, RegionState.State.OFFLINE);
871       this.regionsInTransition.put(encodedName, state);
872     } else {
873       LOG.debug("Forcing OFFLINE; was=" + state);
874       state.update(RegionState.State.OFFLINE);
875     }
876     return state;
877   }
878 
879   /**
880    * Caller must hold lock on the passed <code>state</code> object.
881    * @param state
882    * @param setOfflineInZK
883    * @param forceNewPlan
884    */
885   private void assign(final RegionState state, final boolean setOfflineInZK,
886       final boolean forceNewPlan) {
887     for (int i = 0; i < this.maximumAssignmentAttempts; i++) {
888       if (setOfflineInZK && !setOfflineInZooKeeper(state)) return;
889       if (this.master.isStopped()) {
890         LOG.debug("Server stopped; skipping assign of " + state);
891         return;
892       }
893       RegionPlan plan = getRegionPlan(state, forceNewPlan);
894       if (plan == null) return; // Should get reassigned later when RIT times out.
895       try {
896         LOG.debug("Assigning region " + state.getRegion().getRegionNameAsString() +
897           " to " + plan.getDestination().getServerName());
898         // Transition RegionState to PENDING_OPEN
899         state.update(RegionState.State.PENDING_OPEN);
900         // Send OPEN RPC. This can fail if the server on other end is is not up.
901         serverManager.sendRegionOpen(plan.getDestination(), state.getRegion());
902         break;
903       } catch (Throwable t) {
904         LOG.warn("Failed assignment of " +
905           state.getRegion().getRegionNameAsString() + " to " +
906           plan.getDestination() + ", trying to assign elsewhere instead; " +
907           "retry=" + i, t);
908         // Clean out plan we failed execute and one that doesn't look like it'll
909         // succeed anyways; we need a new plan!
910         // Transition back to OFFLINE
911         state.update(RegionState.State.OFFLINE);
912         // Force a new plan and reassign.  Will return null if no servers.
913         if (getRegionPlan(state, plan.getDestination(), true) == null) {
914           LOG.warn("Unable to find a viable location to assign region " +
915             state.getRegion().getRegionNameAsString());
916           return;
917         }
918       }
919     }
920   }
921 
922   /**
923    * Set region as OFFLINED up in zookeeper
924    * @param state
925    * @return True if we succeeded, false otherwise (State was incorrect or failed
926    * updating zk).
927    */
928   boolean setOfflineInZooKeeper(final RegionState state) {
929     if (!state.isClosed() && !state.isOffline()) {
930         new RuntimeException("Unexpected state trying to OFFLINE; " + state);
931       this.master.abort("Unexpected state trying to OFFLINE; " + state,
932         new IllegalStateException());
933       return false;
934     }
935     state.update(RegionState.State.OFFLINE);
936     try {
937       if(!ZKAssign.createOrForceNodeOffline(master.getZooKeeper(),
938           state.getRegion(), master.getServerName())) {
939         LOG.warn("Attempted to create/force node into OFFLINE state before " +
940           "completing assignment but failed to do so for " + state);
941         return false;
942       }
943     } catch (KeeperException e) {
944       master.abort("Unexpected ZK exception creating/setting node OFFLINE", e);
945       return false;
946     }
947     return true;
948   }
949 
950   /**
951    * Set region as OFFLINED up in zookeeper asynchronously.
952    * @param state
953    * @return True if we succeeded, false otherwise (State was incorrect or failed
954    * updating zk).
955    */
956   boolean asyncSetOfflineInZooKeeper(final RegionState state,
957       final AsyncCallback.StringCallback cb, final Object ctx) {
958     if (!state.isClosed() && !state.isOffline()) {
959         new RuntimeException("Unexpected state trying to OFFLINE; " + state);
960       this.master.abort("Unexpected state trying to OFFLINE; " + state,
961         new IllegalStateException());
962       return false;
963     }
964     state.update(RegionState.State.OFFLINE);
965     try {
966       ZKAssign.asyncCreateNodeOffline(master.getZooKeeper(), state.getRegion(),
967         master.getServerName(), cb, ctx);
968     } catch (KeeperException e) {
969       master.abort("Unexpected ZK exception creating/setting node OFFLINE", e);
970       return false;
971     }
972     return true;
973   }
974 
975   /**
976    * @param state
977    * @return Plan for passed <code>state</code> (If none currently, it creates one or
978    * if no servers to assign, it returns null).
979    */
980   RegionPlan getRegionPlan(final RegionState state,
981       final boolean forceNewPlan) {
982     return getRegionPlan(state, null, forceNewPlan);
983   }
984 
985   /**
986    * @param state
987    * @param serverToExclude Server to exclude (we know its bad). Pass null if
988    * all servers are thought to be assignable.
989    * @param forceNewPlan If true, then if an existing plan exists, a new plan
990    * will be generated.
991    * @return Plan for passed <code>state</code> (If none currently, it creates one or
992    * if no servers to assign, it returns null).
993    */
994   RegionPlan getRegionPlan(final RegionState state,
995       final HServerInfo serverToExclude, final boolean forceNewPlan) {
996     // Pickup existing plan or make a new one
997     String encodedName = state.getRegion().getEncodedName();
998     List<HServerInfo> servers = this.serverManager.getOnlineServersList();
999     // The remove below hinges on the fact that the call to
1000     // serverManager.getOnlineServersList() returns a copy
1001     if (serverToExclude != null) servers.remove(serverToExclude);
1002     if (servers.isEmpty()) return null;
1003     RegionPlan randomPlan = new RegionPlan(state.getRegion(), null,
1004       LoadBalancer.randomAssignment(servers));
1005     boolean newPlan = false;
1006     RegionPlan existingPlan = null;
1007     synchronized (this.regionPlans) {
1008       existingPlan = this.regionPlans.get(encodedName);
1009       if (forceNewPlan || existingPlan == null 
1010               || existingPlan.getDestination() == null 
1011               || existingPlan.getDestination().equals(serverToExclude)) {
1012         newPlan = true;
1013         this.regionPlans.put(encodedName, randomPlan);
1014       }
1015     }
1016     if (newPlan) {
1017       LOG.debug("No previous transition plan was found (or we are ignoring " +
1018         "an existing plan) for " + state.getRegion().getRegionNameAsString() +
1019         " so generated a random one; " + randomPlan + "; " +
1020         serverManager.countOfRegionServers() +
1021         " (online=" + serverManager.getOnlineServers().size() +
1022         ", exclude=" + serverToExclude + ") available servers");
1023         return randomPlan;
1024       }
1025       LOG.debug("Using pre-existing plan for region " +
1026         state.getRegion().getRegionNameAsString() + "; plan=" + existingPlan);
1027       return existingPlan;
1028   }
1029 
1030   /**
1031    * Unassigns the specified region.
1032    * <p>
1033    * Updates the RegionState and sends the CLOSE RPC.
1034    * <p>
1035    * If a RegionPlan is already set, it will remain.
1036    *
1037    * @param region server to be unassigned
1038    */
1039   public void unassign(HRegionInfo region) {
1040     unassign(region, false);
1041   }
1042 
1043   /**
1044    * Unassigns the specified region.
1045    * <p>
1046    * Updates the RegionState and sends the CLOSE RPC.
1047    * <p>
1048    * If a RegionPlan is already set, it will remain.
1049    *
1050    * @param region server to be unassigned
1051    * @param force if region should be closed even if already closing
1052    */
1053   public void unassign(HRegionInfo region, boolean force) {
1054     LOG.debug("Starting unassignment of region " +
1055       region.getRegionNameAsString() + " (offlining)");
1056     synchronized (this.regions) {
1057       // Check if this region is currently assigned
1058       if (!regions.containsKey(region)) {
1059         LOG.debug("Attempted to unassign region " +
1060           region.getRegionNameAsString() + " but it is not " +
1061           "currently assigned anywhere");
1062         return;
1063       }
1064     }
1065     String encodedName = region.getEncodedName();
1066     // Grab the state of this region and synchronize on it
1067     RegionState state;
1068     synchronized (regionsInTransition) {
1069       state = regionsInTransition.get(encodedName);
1070       if (state == null) {
1071         state = new RegionState(region, RegionState.State.PENDING_CLOSE);
1072         regionsInTransition.put(encodedName, state);
1073       } else if (force && state.isPendingClose()) {
1074         LOG.debug("Attempting to unassign region " +
1075             region.getRegionNameAsString() + " which is already pending close "
1076             + "but forcing an additional close");
1077         state.update(RegionState.State.PENDING_CLOSE);
1078       } else {
1079         LOG.debug("Attempting to unassign region " +
1080           region.getRegionNameAsString() + " but it is " +
1081           "already in transition (" + state.getState() + ")");
1082         return;
1083       }
1084     }
1085     // Send CLOSE RPC
1086     HServerInfo server = null;
1087     synchronized (this.regions) {
1088       server = regions.get(region);
1089     }
1090     try {
1091       // TODO: We should consider making this look more like it does for the
1092       // region open where we catch all throwables and never abort
1093       if (serverManager.sendRegionClose(server, state.getRegion())) {
1094         LOG.debug("Sent CLOSE to " + server + " for region " +
1095           region.getRegionNameAsString());
1096         return;
1097       }
1098       // This never happens. Currently regionserver close always return true.
1099       LOG.debug("Server " + server + " region CLOSE RPC returned false for " +
1100         region.getEncodedName());
1101     } catch (NotServingRegionException nsre) {
1102       LOG.info("Server " + server + " returned " + nsre + " for " +
1103         region.getEncodedName());
1104       // Presume that master has stale data.  Presume remote side just split.
1105       // Presume that the split message when it comes in will fix up the master's
1106       // in memory cluster state.
1107       return;
1108     } catch (ConnectException e) {
1109       LOG.info("Failed connect to " + server + ", message=" + e.getMessage() +
1110         ", region=" + region.getEncodedName());
1111       // Presume that regionserver just failed and we haven't got expired
1112       // server from zk yet.  Let expired server deal with clean up.
1113     } catch (java.net.SocketTimeoutException e) {
1114       LOG.info("Server " + server + " returned " + e.getMessage() + " for " +
1115         region.getEncodedName());
1116       // Presume retry or server will expire.
1117     } catch (EOFException e) {
1118       LOG.info("Server " + server + " returned " + e.getMessage() + " for " +
1119         region.getEncodedName());
1120       // Presume retry or server will expire.
1121     } catch (RemoteException re) {
1122       IOException ioe = re.unwrapRemoteException();
1123       if (ioe instanceof NotServingRegionException) {
1124         // Failed to close, so pass through and reassign
1125         LOG.debug("Server " + server + " returned " + ioe + " for " +
1126           region.getEncodedName());
1127       } else if (ioe instanceof EOFException) {
1128         // Failed to close, so pass through and reassign
1129         LOG.debug("Server " + server + " returned " + ioe + " for " +
1130           region.getEncodedName());
1131       } else {
1132         this.master.abort("Remote unexpected exception", ioe);
1133       }
1134     } catch (Throwable t) {
1135       // For now call abort if unexpected exception -- radical, but will get
1136       // fellas attention. St.Ack 20101012
1137       this.master.abort("Remote unexpected exception", t);
1138     }
1139   }
1140 
1141   /**
1142    * Waits until the specified region has completed assignment.
1143    * <p>
1144    * If the region is already assigned, returns immediately.  Otherwise, method
1145    * blocks until the region is assigned.
1146    * @param regionInfo region to wait on assignment for
1147    * @throws InterruptedException
1148    */
1149   public void waitForAssignment(HRegionInfo regionInfo)
1150   throws InterruptedException {
1151     synchronized(regions) {
1152       while(!regions.containsKey(regionInfo)) {
1153         regions.wait();
1154       }
1155     }
1156   }
1157 
1158   /**
1159    * Assigns the ROOT region.
1160    * <p>
1161    * Assumes that ROOT is currently closed and is not being actively served by
1162    * any RegionServer.
1163    * <p>
1164    * Forcibly unsets the current root region location in ZooKeeper and assigns
1165    * ROOT to a random RegionServer.
1166    * @throws KeeperException
1167    */
1168   public void assignRoot() throws KeeperException {
1169     RootLocationEditor.deleteRootLocation(this.master.getZooKeeper());
1170     assign(HRegionInfo.ROOT_REGIONINFO, true);
1171   }
1172 
1173   /**
1174    * Assigns the META region.
1175    * <p>
1176    * Assumes that META is currently closed and is not being actively served by
1177    * any RegionServer.
1178    * <p>
1179    * Forcibly assigns META to a random RegionServer.
1180    */
1181   public void assignMeta() {
1182     // Force assignment to a random server
1183     assign(HRegionInfo.FIRST_META_REGIONINFO, true);
1184   }
1185 
1186   /**
1187    * Assigns list of user regions in round-robin fashion, if any exist.
1188    * <p>
1189    * This is a synchronous call and will return once every region has been
1190    * assigned.  If anything fails, an exception is thrown
1191    * @throws InterruptedException
1192    * @throws IOException
1193    */
1194   public void assignUserRegions(List<HRegionInfo> regions, List<HServerInfo> servers) throws IOException, InterruptedException {
1195     if (regions == null)
1196       return;
1197     Map<HServerInfo, List<HRegionInfo>> bulkPlan = null;
1198     // Generate a round-robin bulk assignment plan
1199     bulkPlan = LoadBalancer.roundRobinAssignment(regions, servers);
1200     LOG.info("Bulk assigning " + regions.size() + " region(s) round-robin across " +
1201                servers.size() + " server(s)");
1202     // Use fixed count thread pool assigning.
1203     BulkAssigner ba = new BulkStartupAssigner(this.master, bulkPlan, this);
1204     ba.bulkAssign();
1205     LOG.info("Bulk assigning done");
1206   }
1207 
1208   /**
1209    * Assigns all user regions, if any exist.  Used during cluster startup.
1210    * <p>
1211    * This is a synchronous call and will return once every region has been
1212    * assigned.  If anything fails, an exception is thrown and the cluster
1213    * should be shutdown.
1214    * @throws InterruptedException
1215    * @throws IOException
1216    */
1217   public void assignAllUserRegions() throws IOException, InterruptedException {
1218     // Get all available servers
1219     List<HServerInfo> servers = serverManager.getOnlineServersList();
1220 
1221     // Scan META for all user regions, skipping any disabled tables
1222     Map<HRegionInfo,HServerAddress> allRegions =
1223       MetaReader.fullScan(catalogTracker, this.zkTable.getDisabledTables(), true);
1224     if (allRegions == null || allRegions.isEmpty()) return;
1225 
1226     // Determine what type of assignment to do on startup
1227     boolean retainAssignment = master.getConfiguration().
1228       getBoolean("hbase.master.startup.retainassign", true);
1229 
1230     Map<HServerInfo, List<HRegionInfo>> bulkPlan = null;
1231     if (retainAssignment) {
1232       // Reuse existing assignment info
1233       bulkPlan = LoadBalancer.retainAssignment(allRegions, servers);
1234     } else {
1235       // assign regions in round-robin fashion
1236       assignUserRegions(new ArrayList<HRegionInfo>(allRegions.keySet()), servers);
1237       return;
1238     }
1239     LOG.info("Bulk assigning " + allRegions.size() + " region(s) across " +
1240       servers.size() + " server(s), retainAssignment=" + retainAssignment);
1241 
1242     // Use fixed count thread pool assigning.
1243     BulkAssigner ba = new BulkStartupAssigner(this.master, bulkPlan, this);
1244     ba.bulkAssign();
1245     LOG.info("Bulk assigning done");
1246   }
1247 
1248   /**
1249    * Run bulk assign on startup.
1250    */
1251   static class BulkStartupAssigner extends BulkAssigner {
1252     private final Map<HServerInfo, List<HRegionInfo>> bulkPlan;
1253     private final AssignmentManager assignmentManager;
1254 
1255     BulkStartupAssigner(final Server server,
1256         final Map<HServerInfo, List<HRegionInfo>> bulkPlan,
1257         final AssignmentManager am) {
1258       super(server);
1259       this.bulkPlan = bulkPlan;
1260       this.assignmentManager = am;
1261     }
1262 
1263     @Override
1264     public boolean bulkAssign() throws InterruptedException {
1265       // Disable timing out regions in transition up in zk while bulk assigning.
1266       this.assignmentManager.timeoutMonitor.bulkAssign(true);
1267       try {
1268         return super.bulkAssign();
1269       } finally {
1270         // Reenable timing out regions in transition up in zi.
1271         this.assignmentManager.timeoutMonitor.bulkAssign(false);
1272       }
1273     }
1274 
1275     @Override
1276    protected String getThreadNamePrefix() {
1277     return super.getThreadNamePrefix() + "-startup";
1278    }
1279 
1280     @Override
1281     protected void populatePool(java.util.concurrent.ExecutorService pool) {
1282       for (Map.Entry<HServerInfo, List<HRegionInfo>> e: this.bulkPlan.entrySet()) {
1283         pool.execute(new SingleServerBulkAssigner(e.getKey(), e.getValue(),
1284           this.assignmentManager));
1285       }
1286     }
1287 
1288     protected boolean waitUntilDone(final long timeout)
1289     throws InterruptedException {
1290       return this.assignmentManager.waitUntilNoRegionsInTransition(timeout);
1291     }
1292   }
1293 
1294   /**
1295    * Manage bulk assigning to a server.
1296    */
1297   static class SingleServerBulkAssigner implements Runnable {
1298     private final HServerInfo regionserver;
1299     private final List<HRegionInfo> regions;
1300     private final AssignmentManager assignmentManager;
1301 
1302     SingleServerBulkAssigner(final HServerInfo regionserver,
1303         final List<HRegionInfo> regions, final AssignmentManager am) {
1304       this.regionserver = regionserver;
1305       this.regions = regions;
1306       this.assignmentManager = am;
1307     }
1308     @Override
1309     public void run() {
1310       this.assignmentManager.assign(this.regionserver, this.regions);
1311     }
1312   }
1313 
1314   /**
1315    * Wait until no regions in transition.
1316    * @param timeout How long to wait.
1317    * @return True if nothing in regions in transition.
1318    * @throws InterruptedException
1319    */
1320   boolean waitUntilNoRegionsInTransition(final long timeout)
1321   throws InterruptedException {
1322     // Blocks until there are no regions in transition. It is possible that
1323     // there
1324     // are regions in transition immediately after this returns but guarantees
1325     // that if it returns without an exception that there was a period of time
1326     // with no regions in transition from the point-of-view of the in-memory
1327     // state of the Master.
1328     long startTime = System.currentTimeMillis();
1329     long remaining = timeout;
1330     synchronized (regionsInTransition) {
1331       while (regionsInTransition.size() > 0 && !this.master.isStopped()
1332           && remaining > 0) {
1333         regionsInTransition.wait(remaining);
1334         remaining = timeout - (System.currentTimeMillis() - startTime);
1335       }
1336     }
1337     return regionsInTransition.isEmpty();
1338   }
1339 
1340   /**
1341    * Rebuild the list of user regions and assignment information.
1342    * <p>
1343    * Returns a map of servers that are not found to be online and the regions
1344    * they were hosting.
1345    * @return map of servers not online to their assigned regions, as stored
1346    *         in META
1347    * @throws IOException
1348    */
1349   private Map<HServerInfo,List<Pair<HRegionInfo,Result>>> rebuildUserRegions()
1350   throws IOException {
1351     // Region assignment from META
1352     List<Result> results = MetaReader.fullScanOfResults(catalogTracker);
1353     // Map of offline servers and their regions to be returned
1354     Map<HServerInfo,List<Pair<HRegionInfo,Result>>> offlineServers =
1355       new TreeMap<HServerInfo,List<Pair<HRegionInfo,Result>>>();
1356     // Iterate regions in META
1357     for (Result result : results) {
1358       Pair<HRegionInfo,HServerInfo> region =
1359         MetaReader.metaRowToRegionPairWithInfo(result);
1360       if (region == null) continue;
1361       HServerInfo regionLocation = region.getSecond();
1362       HRegionInfo regionInfo = region.getFirst();
1363       if (regionLocation == null) {
1364         // Region not being served, add to region map with no assignment
1365         // If this needs to be assigned out, it will also be in ZK as RIT
1366         this.regions.put(regionInfo, null);
1367       } else if (!serverManager.isServerOnline(
1368           regionLocation.getServerName())) {
1369         // Region is located on a server that isn't online
1370         List<Pair<HRegionInfo,Result>> offlineRegions =
1371           offlineServers.get(regionLocation);
1372         if (offlineRegions == null) {
1373           offlineRegions = new ArrayList<Pair<HRegionInfo,Result>>(1);
1374           offlineServers.put(regionLocation, offlineRegions);
1375         }
1376         offlineRegions.add(new Pair<HRegionInfo,Result>(regionInfo, result));
1377       } else {
1378         // Region is being served and on an active server
1379         regions.put(regionInfo, regionLocation);
1380         addToServers(regionLocation, regionInfo);
1381       }
1382     }
1383     return offlineServers;
1384   }
1385 
1386   /**
1387    * Processes list of dead servers from result of META scan.
1388    * <p>
1389    * This is used as part of failover to handle RegionServers which failed
1390    * while there was no active master.
1391    * <p>
1392    * Method stubs in-memory data to be as expected by the normal server shutdown
1393    * handler.
1394    *
1395    * @param deadServers
1396    * @throws IOException
1397    * @throws KeeperException
1398    */
1399   private void processDeadServers(
1400       Map<HServerInfo, List<Pair<HRegionInfo, Result>>> deadServers)
1401   throws IOException, KeeperException {
1402     for (Map.Entry<HServerInfo, List<Pair<HRegionInfo,Result>>> deadServer :
1403       deadServers.entrySet()) {
1404       List<Pair<HRegionInfo,Result>> regions = deadServer.getValue();
1405       for (Pair<HRegionInfo,Result> region : regions) {
1406         HRegionInfo regionInfo = region.getFirst();
1407         Result result = region.getSecond();
1408         // If region was in transition (was in zk) force it offline for reassign
1409         try {
1410           ZKAssign.createOrForceNodeOffline(watcher, regionInfo,
1411               master.getServerName());
1412         } catch (KeeperException.NoNodeException nne) {
1413           // This is fine
1414         }
1415         // Process with existing RS shutdown code
1416         ServerShutdownHandler.processDeadRegion(regionInfo, result, this,
1417             this.catalogTracker);
1418       }
1419     }
1420   }
1421 
1422   /*
1423    * Presumes caller has taken care of necessary locking modifying servers Map.
1424    * @param hsi
1425    * @param hri
1426    */
1427   private void addToServers(final HServerInfo hsi, final HRegionInfo hri) {
1428     List<HRegionInfo> hris = servers.get(hsi);
1429     if (hris == null) {
1430       hris = new ArrayList<HRegionInfo>();
1431       servers.put(hsi, hris);
1432     }
1433     hris.add(hri);
1434   }
1435 
1436   /**
1437    * @return A copy of the Map of regions currently in transition.
1438    */
1439   public NavigableMap<String, RegionState> getRegionsInTransition() {
1440     synchronized (this.regionsInTransition) {
1441       return new TreeMap<String, RegionState>(this.regionsInTransition);
1442     }
1443   }
1444 
1445   /**
1446    * @return True if regions in transition.
1447    */
1448   public boolean isRegionsInTransition() {
1449     synchronized (this.regionsInTransition) {
1450       return !this.regionsInTransition.isEmpty();
1451     }
1452   }
1453 
1454   /**
1455    * @param hri Region to check.
1456    * @return Returns null if passed region is not in transition else the current
1457    * RegionState
1458    */
1459   public RegionState isRegionInTransition(final HRegionInfo hri) {
1460     synchronized (this.regionsInTransition) {
1461       return this.regionsInTransition.get(hri.getEncodedName());
1462     }
1463   }
1464 
1465   /**
1466    * Clears the specified region from being in transition.
1467    * <p>
1468    * Used only by HBCK tool.
1469    * @param hri
1470    */
1471   public void clearRegionFromTransition(HRegionInfo hri) {
1472     synchronized (this.regionsInTransition) {
1473       this.regionsInTransition.remove(hri.getEncodedName());
1474     }
1475     synchronized (this.regions) {
1476       this.regions.remove(hri);
1477       for (List<HRegionInfo> regions : this.servers.values()) {
1478         for (int i=0;i<regions.size();i++) {
1479           if (regions.get(i).equals(hri)) {
1480             regions.remove(i);
1481             break;
1482           }
1483         }
1484       }
1485     }
1486     clearRegionPlan(hri);
1487   }
1488 
1489   /**
1490    * @param region Region whose plan we are to clear.
1491    */
1492   void clearRegionPlan(final HRegionInfo region) {
1493     synchronized (this.regionPlans) {
1494       this.regionPlans.remove(region.getEncodedName());
1495     }
1496   }
1497 
1498   /**
1499    * Wait on region to clear regions-in-transition.
1500    * @param hri Region to wait on.
1501    * @throws IOException
1502    */
1503   public void waitOnRegionToClearRegionsInTransition(final HRegionInfo hri)
1504   throws IOException {
1505     if (isRegionInTransition(hri) == null) return;
1506     RegionState rs = null;
1507     // There is already a timeout monitor on regions in transition so I
1508     // should not have to have one here too?
1509     while(!this.master.isStopped() && (rs = isRegionInTransition(hri)) != null) {
1510       Threads.sleep(1000);
1511       LOG.info("Waiting on " + rs + " to clear regions-in-transition");
1512     }
1513     if (this.master.isStopped()) {
1514       LOG.info("Giving up wait on regions in " +
1515         "transition because stoppable.isStopped is set");
1516     }
1517   }
1518 
1519 
1520   /**
1521    * Gets the online regions of the specified table.
1522    * This method looks at the in-memory state.  It does not go to <code>.META.</code>.
1523    * Only returns <em>online</em> regions.  If a region on this table has been
1524    * closed during a disable, etc., it will be included in the returned list.
1525    * So, the returned list may not necessarily be ALL regions in this table, its
1526    * all the ONLINE regions in the table.
1527    * @param tableName
1528    * @return Online regions from <code>tableName</code>
1529    */
1530   public List<HRegionInfo> getRegionsOfTable(byte[] tableName) {
1531     List<HRegionInfo> tableRegions = new ArrayList<HRegionInfo>();
1532     HRegionInfo boundary =
1533       new HRegionInfo(new HTableDescriptor(tableName), null, null);
1534     synchronized (this.regions) {
1535       for (HRegionInfo regionInfo: this.regions.tailMap(boundary).keySet()) {
1536         if(Bytes.equals(regionInfo.getTableDesc().getName(), tableName)) {
1537           tableRegions.add(regionInfo);
1538         } else {
1539           break;
1540         }
1541       }
1542     }
1543     return tableRegions;
1544   }
1545 
1546   /**
1547    * Monitor to check for time outs on region transition operations
1548    */
1549   public class TimeoutMonitor extends Chore {
1550     private final int timeout;
1551     private boolean bulkAssign = false;
1552 
1553     /**
1554      * Creates a periodic monitor to check for time outs on region transition
1555      * operations.  This will deal with retries if for some reason something
1556      * doesn't happen within the specified timeout.
1557      * @param period
1558    * @param stopper When {@link Stoppable#isStopped()} is true, this thread will
1559    * cleanup and exit cleanly.
1560      * @param timeout
1561      */
1562     public TimeoutMonitor(final int period, final Stoppable stopper,
1563         final int timeout) {
1564       super("AssignmentTimeoutMonitor", period, stopper);
1565       this.timeout = timeout;
1566     }
1567 
1568     /**
1569      * @param bulkAssign If true, we'll suspend checking regions in transition
1570      * up in zookeeper.  If false, will reenable check.
1571      * @return Old setting for bulkAssign.
1572      */
1573     public boolean bulkAssign(final boolean bulkAssign) {
1574       boolean result = this.bulkAssign;
1575       this.bulkAssign = bulkAssign;
1576       return result;
1577     }
1578 
1579     @Override
1580     protected void chore() {
1581       // If bulkAssign in progress, suspend checks
1582       if (this.bulkAssign) return;
1583       synchronized (regionsInTransition) {
1584         // Iterate all regions in transition checking for time outs
1585         long now = System.currentTimeMillis();
1586         for (RegionState regionState : regionsInTransition.values()) {
1587           if(regionState.getStamp() + timeout <= now) {
1588             HRegionInfo regionInfo = regionState.getRegion();
1589             LOG.info("Regions in transition timed out:  " + regionState);
1590             // Expired!  Do a retry.
1591             switch (regionState.getState()) {
1592               case CLOSED:
1593                 LOG.info("Region " + regionInfo.getEncodedName() +
1594                   " has been CLOSED for too long, waiting on queued " +
1595                   "ClosedRegionHandler to run or server shutdown");
1596                 // Update our timestamp.
1597                 synchronized(regionState) {
1598                   regionState.update(regionState.getState());
1599                 }
1600                 break;
1601               case OFFLINE:
1602                 LOG.info("Region has been OFFLINE for too long, " +
1603                   "reassigning " + regionInfo.getRegionNameAsString() +
1604                   " to a random server");
1605                 assign(regionState.getRegion(), false);
1606                 break;
1607               case PENDING_OPEN:
1608                 LOG.info("Region has been PENDING_OPEN for too " +
1609                     "long, reassigning region=" +
1610                     regionInfo.getRegionNameAsString());
1611                 assign(regionState.getRegion(), false, true);
1612                 break;
1613               case OPENING:
1614                 LOG.info("Region has been OPENING for too " +
1615                   "long, reassigning region=" +
1616                   regionInfo.getRegionNameAsString());
1617                 // Should have a ZK node in OPENING state
1618                 try {
1619                   String node = ZKAssign.getNodeName(watcher,
1620                       regionInfo.getEncodedName());
1621                   Stat stat = new Stat();
1622                   RegionTransitionData data = ZKAssign.getDataNoWatch(watcher,
1623                       node, stat);
1624                   if (data.getEventType() == EventType.RS_ZK_REGION_OPENED) {
1625                     LOG.debug("Region has transitioned to OPENED, allowing " +
1626                         "watched event handlers to process");
1627                     break;
1628                   } else if (data.getEventType() !=
1629                       EventType.RS_ZK_REGION_OPENING) {
1630                     LOG.warn("While timing out a region in state OPENING, " +
1631                         "found ZK node in unexpected state: " +
1632                         data.getEventType());
1633                     break;
1634                   }
1635                   // Attempt to transition node into OFFLINE
1636                   try {
1637                     data = new RegionTransitionData(
1638                       EventType.M_ZK_REGION_OFFLINE, regionInfo.getRegionName(),
1639                       master.getServerName());
1640                     if (ZKUtil.setData(watcher, node, data.getBytes(),
1641                         stat.getVersion())) {
1642                       // Node is now OFFLINE, let's trigger another assignment
1643                       ZKUtil.getDataAndWatch(watcher, node); // re-set the watch
1644                       LOG.info("Successfully transitioned region=" +
1645                           regionInfo.getRegionNameAsString() + " into OFFLINE" +
1646                           " and forcing a new assignment");
1647                       assign(regionState, false, true);
1648                     }
1649                   } catch (KeeperException.NoNodeException nne) {
1650                     // Node did not exist, can't time this out
1651                   }
1652                 } catch (KeeperException ke) {
1653                   LOG.error("Unexpected ZK exception timing out CLOSING region",
1654                       ke);
1655                   break;
1656                 }
1657                 break;
1658               case OPEN:
1659                 LOG.error("Region has been OPEN for too long, " +
1660                 "we don't know where region was opened so can't do anything");
1661                 break;
1662               case PENDING_CLOSE:
1663                 LOG.info("Region has been PENDING_CLOSE for too " +
1664                     "long, running forced unassign again on region=" +
1665                     regionInfo.getRegionNameAsString());
1666                   try {
1667                     // If the server got the RPC, it will transition the node
1668                     // to CLOSING, so only do something here if no node exists
1669                     if (!ZKUtil.watchAndCheckExists(watcher,
1670                         ZKAssign.getNodeName(watcher,
1671                             regionInfo.getEncodedName()))) {
1672                       unassign(regionInfo, true);
1673                     }
1674                   } catch (NoNodeException e) {
1675                     LOG.debug("Node no longer existed so not forcing another " +
1676                         "unassignment");
1677                   } catch (KeeperException e) {
1678                     LOG.warn("Unexpected ZK exception timing out a region " +
1679                         "close", e);
1680                   }
1681                   break;
1682               case CLOSING:
1683                 LOG.info("Region has been CLOSING for too " +
1684                   "long, this should eventually complete or the server will " +
1685                   "expire, doing nothing");
1686                 break;
1687             }
1688           }
1689         }
1690       }
1691     }
1692   }
1693 
1694   /**
1695    * Process shutdown server removing any assignments.
1696    * @param hsi Server that went down.
1697    * @return list of regions in transition on this server
1698    */
1699   public List<RegionState> processServerShutdown(final HServerInfo hsi) {
1700     // Clean out any existing assignment plans for this server
1701     synchronized (this.regionPlans) {
1702       for (Iterator <Map.Entry<String, RegionPlan>> i =
1703           this.regionPlans.entrySet().iterator(); i.hasNext();) {
1704         Map.Entry<String, RegionPlan> e = i.next();
1705         if (e.getValue().getDestination().equals(hsi)) {
1706           // Use iterator's remove else we'll get CME
1707           i.remove();
1708         }
1709       }
1710     }
1711     // TODO: Do we want to sync on RIT here?
1712     // Remove this server from map of servers to regions, and remove all regions
1713     // of this server from online map of regions.
1714     Set<HRegionInfo> deadRegions = null;
1715     List<RegionState> rits = new ArrayList<RegionState>();
1716     synchronized (this.regions) {
1717       List<HRegionInfo> assignedRegions = this.servers.remove(hsi);
1718       if (assignedRegions == null || assignedRegions.isEmpty()) {
1719         // No regions on this server, we are done, return empty list of RITs
1720         return rits;
1721       }
1722       deadRegions = new TreeSet<HRegionInfo>(assignedRegions);
1723       for (HRegionInfo region : deadRegions) {
1724         this.regions.remove(region);
1725       }
1726     }
1727     // See if any of the regions that were online on this server were in RIT
1728     // If they are, normal timeouts will deal with them appropriately so
1729     // let's skip a manual re-assignment.
1730     synchronized (regionsInTransition) {
1731       for (RegionState region : this.regionsInTransition.values()) {
1732         if (deadRegions.remove(region.getRegion())) {
1733           rits.add(region);
1734         }
1735       }
1736     }
1737     return rits;
1738   }
1739 
1740   /**
1741    * Update inmemory structures.
1742    * @param hsi Server that reported the split
1743    * @param parent Parent region that was split
1744    * @param a Daughter region A
1745    * @param b Daughter region B
1746    */
1747   public void handleSplitReport(final HServerInfo hsi, final HRegionInfo parent,
1748       final HRegionInfo a, final HRegionInfo b) {
1749     regionOffline(parent);
1750     // Remove any CLOSING node, if exists, due to race between master & rs
1751     // for close & split.  Not putting into regionOffline method because it is
1752     // called from various locations.
1753     try {
1754       RegionTransitionData node = ZKAssign.getDataNoWatch(this.watcher,
1755         parent.getEncodedName(), null);
1756       if (node != null) {
1757         if (node.getEventType().equals(EventType.RS_ZK_REGION_CLOSING)) {
1758           ZKAssign.deleteClosingNode(this.watcher, parent);
1759         } else {
1760           LOG.warn("Split report has RIT node (shouldnt have one): " +
1761             parent + " node: " + node);
1762         }
1763       }
1764     } catch (KeeperException e) {
1765       LOG.warn("Exception while validating RIT during split report", e);
1766     }
1767 
1768     regionOnline(a, hsi);
1769     regionOnline(b, hsi);
1770 
1771     // There's a possibility that the region was splitting while a user asked
1772     // the master to disable, we need to make sure we close those regions in
1773     // that case. This is not racing with the region server itself since RS
1774     // report is done after the split transaction completed.
1775     if (this.zkTable.isDisablingOrDisabledTable(
1776         parent.getTableDesc().getNameAsString())) {
1777       unassign(a);
1778       unassign(b);
1779     }
1780   }
1781 
1782   /**
1783    * @return A clone of current assignments. Note, this is assignments only.
1784    * If a new server has come in and it has no regions, it will not be included
1785    * in the returned Map.
1786    */
1787   Map<HServerInfo, List<HRegionInfo>> getAssignments() {
1788     // This is an EXPENSIVE clone.  Cloning though is the safest thing to do.
1789     // Can't let out original since it can change and at least the loadbalancer
1790     // wants to iterate this exported list.  We need to synchronize on regions
1791     // since all access to this.servers is under a lock on this.regions.
1792     Map<HServerInfo, List<HRegionInfo>> result = null;
1793     synchronized (this.regions) {
1794       result = new HashMap<HServerInfo, List<HRegionInfo>>(this.servers.size());
1795       for (Map.Entry<HServerInfo, List<HRegionInfo>> e: this.servers.entrySet()) {
1796         List<HRegionInfo> shallowCopy = new ArrayList<HRegionInfo>(e.getValue());
1797         HServerInfo clone = new HServerInfo(e.getKey());
1798         // Set into server load the number of regions this server is carrying
1799         // The load balancer calculation needs it at least and its handy.
1800         clone.getLoad().setNumberOfRegions(e.getValue().size());
1801         result.put(clone, shallowCopy);
1802       }
1803     }
1804     return result;
1805   }
1806 
1807   /**
1808    * @param encodedRegionName Region encoded name.
1809    * @return Null or a {@link Pair} instance that holds the full {@link HRegionInfo}
1810    * and the hosting servers {@link HServerInfo}.
1811    */
1812   Pair<HRegionInfo, HServerInfo> getAssignment(final byte [] encodedRegionName) {
1813     String name = Bytes.toString(encodedRegionName);
1814     synchronized(this.regions) {
1815       for (Map.Entry<HRegionInfo, HServerInfo> e: this.regions.entrySet()) {
1816         if (e.getKey().getEncodedName().equals(name)) {
1817           return new Pair<HRegionInfo, HServerInfo>(e.getKey(), e.getValue());
1818         }
1819       }
1820     }
1821     return null;
1822   }
1823 
1824   /**
1825    * @param plan Plan to execute.
1826    */
1827   void balance(final RegionPlan plan) {
1828     synchronized (this.regionPlans) {
1829       this.regionPlans.put(plan.getRegionName(), plan);
1830     }
1831     unassign(plan.getRegionInfo());
1832   }
1833 
1834   /**
1835    * State of a Region while undergoing transitions.
1836    */
1837   public static class RegionState implements Writable {
1838     private HRegionInfo region;
1839 
1840     public enum State {
1841       OFFLINE,        // region is in an offline state
1842       PENDING_OPEN,   // sent rpc to server to open but has not begun
1843       OPENING,        // server has begun to open but not yet done
1844       OPEN,           // server opened region and updated meta
1845       PENDING_CLOSE,  // sent rpc to server to close but has not begun
1846       CLOSING,        // server has begun to close but not yet done
1847       CLOSED          // server closed region and updated meta
1848     }
1849 
1850     private State state;
1851     private long stamp;
1852 
1853     public RegionState() {}
1854 
1855     RegionState(HRegionInfo region, State state) {
1856       this(region, state, System.currentTimeMillis());
1857     }
1858 
1859     RegionState(HRegionInfo region, State state, long stamp) {
1860       this.region = region;
1861       this.state = state;
1862       this.stamp = stamp;
1863     }
1864 
1865     public void update(State state, long stamp) {
1866       this.state = state;
1867       this.stamp = stamp;
1868     }
1869 
1870     public void update(State state) {
1871       this.state = state;
1872       this.stamp = System.currentTimeMillis();
1873     }
1874 
1875     public State getState() {
1876       return state;
1877     }
1878 
1879     public long getStamp() {
1880       return stamp;
1881     }
1882 
1883     public HRegionInfo getRegion() {
1884       return region;
1885     }
1886 
1887     public boolean isClosing() {
1888       return state == State.CLOSING;
1889     }
1890 
1891     public boolean isClosed() {
1892       return state == State.CLOSED;
1893     }
1894 
1895     public boolean isPendingClose() {
1896       return state == State.PENDING_CLOSE;
1897     }
1898 
1899     public boolean isOpening() {
1900       return state == State.OPENING;
1901     }
1902 
1903     public boolean isOpened() {
1904       return state == State.OPEN;
1905     }
1906 
1907     public boolean isPendingOpen() {
1908       return state == State.PENDING_OPEN;
1909     }
1910 
1911     public boolean isOffline() {
1912       return state == State.OFFLINE;
1913     }
1914 
1915     @Override
1916     public String toString() {
1917       return region.getRegionNameAsString() + " state=" + state +
1918         ", ts=" + stamp;
1919     }
1920 
1921     @Override
1922     public void readFields(DataInput in) throws IOException {
1923       region = new HRegionInfo();
1924       region.readFields(in);
1925       state = State.valueOf(in.readUTF());
1926       stamp = in.readLong();
1927     }
1928 
1929     @Override
1930     public void write(DataOutput out) throws IOException {
1931       region.write(out);
1932       out.writeUTF(state.name());
1933       out.writeLong(stamp);
1934     }
1935   }
1936 
1937   public void stop() {
1938     this.timeoutMonitor.interrupt();
1939   }
1940 }