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.IOException;
25  import java.lang.Thread.UncaughtExceptionHandler;
26  import java.util.ArrayList;
27  import java.util.Arrays;
28  import java.util.Date;
29  import java.util.Collections;
30  import java.util.HashMap;
31  import java.util.HashSet;
32  import java.util.Iterator;
33  import java.util.List;
34  import java.util.Map;
35  import java.util.NavigableMap;
36  import java.util.Set;
37  import java.util.SortedMap;
38  import java.util.TreeMap;
39  import java.util.TreeSet;
40  import java.util.concurrent.ConcurrentSkipListMap;
41  import java.util.concurrent.ConcurrentSkipListSet;
42  import java.util.concurrent.Executors;
43  import java.util.concurrent.atomic.AtomicInteger;
44  import java.util.concurrent.atomic.AtomicLong;
45  
46  import org.apache.commons.logging.Log;
47  import org.apache.commons.logging.LogFactory;
48  import org.apache.hadoop.conf.Configuration;
49  import org.apache.hadoop.hbase.Chore;
50  import org.apache.hadoop.hbase.HConstants;
51  import org.apache.hadoop.hbase.HRegionInfo;
52  import org.apache.hadoop.hbase.NotServingRegionException;
53  import org.apache.hadoop.hbase.Server;
54  import org.apache.hadoop.hbase.ServerName;
55  import org.apache.hadoop.hbase.Stoppable;
56  import org.apache.hadoop.hbase.TableNotFoundException;
57  import org.apache.hadoop.hbase.catalog.CatalogTracker;
58  import org.apache.hadoop.hbase.catalog.MetaReader;
59  import org.apache.hadoop.hbase.catalog.RootLocationEditor;
60  import org.apache.hadoop.hbase.client.Result;
61  import org.apache.hadoop.hbase.executor.ExecutorService;
62  import org.apache.hadoop.hbase.executor.RegionTransitionData;
63  import org.apache.hadoop.hbase.executor.EventHandler.EventType;
64  import org.apache.hadoop.hbase.regionserver.RegionAlreadyInTransitionException;
65  import org.apache.hadoop.hbase.regionserver.RegionOpeningState;
66  import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
67  import org.apache.hadoop.hbase.master.RegionPlan;
68  import org.apache.hadoop.hbase.master.handler.ClosedRegionHandler;
69  import org.apache.hadoop.hbase.master.handler.DisableTableHandler;
70  import org.apache.hadoop.hbase.master.handler.EnableTableHandler;
71  import org.apache.hadoop.hbase.master.handler.OpenedRegionHandler;
72  import org.apache.hadoop.hbase.master.handler.ServerShutdownHandler;
73  import org.apache.hadoop.hbase.master.handler.SplitRegionHandler;
74  import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
75  import org.apache.hadoop.hbase.util.Bytes;
76  import org.apache.hadoop.hbase.util.Pair;
77  import org.apache.hadoop.hbase.util.Threads;
78  import org.apache.hadoop.hbase.util.Writables;
79  import org.apache.hadoop.hbase.zookeeper.ZKAssign;
80  import org.apache.hadoop.hbase.zookeeper.ZKTable;
81  import org.apache.hadoop.hbase.zookeeper.ZKUtil;
82  import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
83  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
84  import org.apache.hadoop.ipc.RemoteException;
85  import org.apache.zookeeper.AsyncCallback;
86  import org.apache.zookeeper.KeeperException;
87  import org.apache.zookeeper.KeeperException.NoNodeException;
88  import org.apache.zookeeper.data.Stat;
89  
90  /**
91   * Manages and performs region assignment.
92   * <p>
93   * Monitors ZooKeeper for events related to regions in transition.
94   * <p>
95   * Handles existing regions in transition during master failover.
96   */
97  public class AssignmentManager extends ZooKeeperListener {
98  
99    private static final Log LOG = LogFactory.getLog(AssignmentManager.class);
100 
101   protected Server master;
102 
103   private ServerManager serverManager;
104 
105   private CatalogTracker catalogTracker;
106 
107   private TimeoutMonitor timeoutMonitor;
108 
109   private LoadBalancer balancer;
110 
111   /**
112    * Map of regions to reopen after the schema of a table is changed. Key -
113    * encoded region name, value - HRegionInfo
114    */
115   private final Map <String, HRegionInfo> regionsToReopen;
116 
117   /*
118    * Maximum times we recurse an assignment.  See below in {@link #assign()}.
119    */
120   private final int maximumAssignmentAttempts;
121 
122   /**
123    * Regions currently in transition.  Map of encoded region names to the master
124    * in-memory state for that region.
125    */
126   final ConcurrentSkipListMap<String, RegionState> regionsInTransition =
127     new ConcurrentSkipListMap<String, RegionState>();
128 
129   /** Plans for region movement. Key is the encoded version of a region name*/
130   // TODO: When do plans get cleaned out?  Ever? In server open and in server
131   // shutdown processing -- St.Ack
132   // All access to this Map must be synchronized.
133   final NavigableMap<String, RegionPlan> regionPlans =
134     new TreeMap<String, RegionPlan>();
135 
136   private final ZKTable zkTable;
137 
138   // store all the table names in disabling state
139   Set<String> disablingTables = new HashSet<String>(1);
140   // store all the enabling state tablenames.
141   Set<String> enablingTables = new HashSet<String>(1);
142 
143   /**
144    * Server to regions assignment map.
145    * Contains the set of regions currently assigned to a given server.
146    * This Map and {@link #regions} are tied.  Always update this in tandem
147    * with the other under a lock on {@link #regions}
148    * @see #regions
149    */
150   private final NavigableMap<ServerName, Set<HRegionInfo>> servers =
151     new TreeMap<ServerName, Set<HRegionInfo>>();
152 
153   /**
154    * Region to server assignment map.
155    * Contains the server a given region is currently assigned to.
156    * This Map and {@link #servers} are tied.  Always update this in tandem
157    * with the other under a lock on {@link #regions}
158    * @see #servers
159    */
160   private final SortedMap<HRegionInfo, ServerName> regions =
161     new TreeMap<HRegionInfo, ServerName>();
162 
163   private final ExecutorService executorService;
164 
165   //Thread pool executor service for timeout monitor
166   private java.util.concurrent.ExecutorService threadPoolExecutorService;
167   
168   private List<EventType> ignoreStatesRSOffline = Arrays.asList(new EventType[]{
169       EventType.RS_ZK_REGION_FAILED_OPEN, EventType.RS_ZK_REGION_CLOSED });
170   
171 
172   /**
173    * Constructs a new assignment manager.
174    *
175    * @param master
176    * @param serverManager
177    * @param catalogTracker
178    * @param service
179    * @throws KeeperException
180    * @throws IOException 
181    */
182   public AssignmentManager(Server master, ServerManager serverManager,
183       CatalogTracker catalogTracker, final ExecutorService service)
184   throws KeeperException, IOException {
185     super(master.getZooKeeper());
186     this.master = master;
187     this.serverManager = serverManager;
188     this.catalogTracker = catalogTracker;
189     this.executorService = service;
190     this.regionsToReopen = Collections.synchronizedMap
191                            (new HashMap<String, HRegionInfo> ());
192     Configuration conf = master.getConfiguration();
193     this.timeoutMonitor = new TimeoutMonitor(
194       conf.getInt("hbase.master.assignment.timeoutmonitor.period", 10000),
195       master,
196       conf.getInt("hbase.master.assignment.timeoutmonitor.timeout", 1800000));
197     Threads.setDaemonThreadRunning(timeoutMonitor.getThread(),
198       master.getServerName() + ".timeoutMonitor");
199     this.zkTable = new ZKTable(this.master.getZooKeeper());
200     this.maximumAssignmentAttempts =
201       this.master.getConfiguration().getInt("hbase.assignment.maximum.attempts", 10);
202     this.balancer = LoadBalancerFactory.getLoadBalancer(conf);
203     this.threadPoolExecutorService = Executors.newCachedThreadPool();
204   }
205 
206   /**
207    * Compute the average load across all region servers.
208    * Currently, this uses a very naive computation - just uses the number of
209    * regions being served, ignoring stats about number of requests.
210    * @return the average load
211    */
212   double getAverageLoad() {
213     int totalLoad = 0;
214     int numServers = 0;
215     // Sync on this.regions because access to this.servers always synchronizes
216     // in this order.
217     synchronized (this.regions) {
218       for (Map.Entry<ServerName, Set<HRegionInfo>> e: servers.entrySet()) {
219         numServers++;
220         totalLoad += e.getValue().size();
221       }
222     }
223     return (double)totalLoad / (double)numServers;
224   }
225 
226   /**
227    * @return Instance of ZKTable.
228    */
229   public ZKTable getZKTable() {
230     // These are 'expensive' to make involving trip to zk ensemble so allow
231     // sharing.
232     return this.zkTable;
233   }
234   /**
235    * Returns the RegionServer to which hri is assigned.
236    *
237    * @param hri
238    *          HRegion for which this function returns the region server
239    * @return HServerInfo The region server to which hri belongs
240    */
241   public ServerName getRegionServerOfRegion(HRegionInfo hri) {
242     synchronized (this.regions ) {
243       return regions.get(hri);
244     }
245   }
246 
247   /**
248    * Add a regionPlan for the specified region.
249    */
250   public void addPlan(String encodedName, RegionPlan plan) {
251     synchronized (regionPlans) {
252       regionPlans.put(encodedName, plan);
253     }
254   }
255 
256   /**
257    * Set the list of regions that will be reopened
258    * because of an update in table schema
259    *
260    * @param regions
261    *          list of regions that should be tracked for reopen
262    */
263   public void setRegionsToReopen(List <HRegionInfo> regions) {
264     for(HRegionInfo hri : regions) {
265       regionsToReopen.put(hri.getEncodedName(), hri);
266     }
267   }
268 
269   /**
270    * Used by the client to identify if all regions have the schema updates
271    *
272    * @param tableName
273    * @return Pair indicating the status of the alter command
274    * @throws IOException
275    * @throws InterruptedException 
276    */
277   public Pair<Integer, Integer> getReopenStatus(byte[] tableName)
278   throws IOException, InterruptedException {
279     List <HRegionInfo> hris =
280       MetaReader.getTableRegions(this.master.getCatalogTracker(), tableName);
281     Integer pending = 0;
282     for(HRegionInfo hri : hris) {
283       if(regionsToReopen.get(hri.getEncodedName()) != null) {
284         pending++;
285       }
286     }
287     return new Pair<Integer, Integer>(pending, hris.size());
288   }
289   /**
290    * Reset all unassigned znodes.  Called on startup of master.
291    * Call {@link #assignAllUserRegions()} after root and meta have been assigned.
292    * @throws IOException
293    * @throws KeeperException
294    */
295   void cleanoutUnassigned() throws IOException, KeeperException {
296     // Cleanup any existing ZK nodes and start watching
297     ZKAssign.deleteAllNodes(watcher);
298     ZKUtil.listChildrenAndWatchForNewChildren(this.watcher,
299       this.watcher.assignmentZNode);
300   }
301 
302   /**
303    * Called on startup.
304    * Figures whether a fresh cluster start of we are joining extant running cluster.
305    * @throws IOException
306    * @throws KeeperException
307    * @throws InterruptedException
308    */
309   void joinCluster() throws IOException, KeeperException, InterruptedException {
310     // Concurrency note: In the below the accesses on regionsInTransition are
311     // outside of a synchronization block where usually all accesses to RIT are
312     // synchronized.  The presumption is that in this case it is safe since this
313     // method is being played by a single thread on startup.
314 
315     // TODO: Regions that have a null location and are not in regionsInTransitions
316     // need to be handled.
317 
318     // Scan META to build list of existing regions, servers, and assignment
319     // Returns servers who have not checked in (assumed dead) and their regions
320     Map<ServerName,List<Pair<HRegionInfo,Result>>> deadServers =
321       rebuildUserRegions();
322 
323     processDeadServersAndRegionsInTransition(deadServers);
324 
325     // Recover the tables that were not fully moved to DISABLED state.
326     // These tables are in DISABLING state when the master restarted/switched.
327     boolean isWatcherCreated = recoverTableInDisablingState(this.disablingTables);
328     recoverTableInEnablingState(this.enablingTables, isWatcherCreated);
329   }
330 
331   /**
332    * Process all regions that are in transition up in zookeeper.  Used by
333    * master joining an already running cluster.
334    * @throws KeeperException
335    * @throws IOException
336    * @throws InterruptedException
337    */
338   void processDeadServersAndRegionsInTransition()
339   throws KeeperException, IOException, InterruptedException {
340     // Pass null to signify no dead servers in this context.
341     processDeadServersAndRegionsInTransition(null);
342   }
343 
344   /**
345    * Process all regions that are in transition in zookeeper and also
346    * processes the list of dead servers by scanning the META. 
347    * Used by master joining an cluster.
348    * @param deadServers
349    *          Map of dead servers and their regions. Can be null.
350    * @throws KeeperException
351    * @throws IOException
352    * @throws InterruptedException
353    */
354   void processDeadServersAndRegionsInTransition(
355       final Map<ServerName, List<Pair<HRegionInfo, Result>>> deadServers)
356   throws KeeperException, IOException, InterruptedException {
357     List<String> nodes = ZKUtil.listChildrenAndWatchForNewChildren(watcher,
358       watcher.assignmentZNode);
359     // Run through all regions.  If they are not assigned and not in RIT, then
360     // its a clean cluster startup, else its a failover.
361     boolean regionsToProcess = false;
362     for (Map.Entry<HRegionInfo, ServerName> e: this.regions.entrySet()) {
363       if (!e.getKey().isMetaTable()
364           && e.getValue() != null) {
365         LOG.debug("Found " + e + " out on cluster");
366         regionsToProcess = true;
367         break;
368       }
369       if (nodes.contains(e.getKey().getEncodedName())) {
370         LOG.debug("Found " + e.getKey().getRegionNameAsString() + " in RITs");
371         // Could be a meta region.
372         regionsToProcess = true;
373         break;
374       }
375     }
376 
377     // If we found user regions out on cluster, its a failover.
378     if (regionsToProcess) {
379       LOG.info("Found regions out on cluster or in RIT; failover");
380       // Process list of dead servers and regions in RIT.
381       // See HBASE-4580 for more information.
382       processDeadServersAndRecoverLostRegions(deadServers, nodes);
383 
384     } else {
385       // Fresh cluster startup.
386       LOG.info("Clean cluster startup. Assigning userregions");
387       cleanoutUnassigned();
388       assignAllUserRegions();
389     }
390   }
391 
392   /**
393    * If region is up in zk in transition, then do fixup and block and wait until
394    * the region is assigned and out of transition.  Used on startup for
395    * catalog regions.
396    * @param hri Region to look for.
397    * @return True if we processed a region in transition else false if region
398    * was not up in zk in transition.
399    * @throws InterruptedException
400    * @throws KeeperException
401    * @throws IOException
402    */
403   boolean processRegionInTransitionAndBlockUntilAssigned(final HRegionInfo hri)
404   throws InterruptedException, KeeperException, IOException {
405     boolean intransistion =
406       processRegionInTransition(hri.getEncodedName(), hri, null);
407     if (!intransistion) return intransistion;
408     debugLog(hri, "Waiting on " + HRegionInfo.prettyPrint(hri.getEncodedName()));
409     synchronized(this.regionsInTransition) {
410       while (!this.master.isStopped() &&
411           this.regionsInTransition.containsKey(hri.getEncodedName())) {
412         this.regionsInTransition.wait();
413       }
414     }
415     return intransistion;
416   }
417 
418   /**
419    * Process failover of new master for region <code>encodedRegionName</code>
420    * up in zookeeper.
421    * @param encodedRegionName Region to process failover for.
422    * @param regionInfo If null we'll go get it from meta table.
423    * @param deadServers Can be null 
424    * @return True if we processed <code>regionInfo</code> as a RIT.
425    * @throws KeeperException
426    * @throws IOException
427    */
428   boolean processRegionInTransition(final String encodedRegionName,
429       final HRegionInfo regionInfo,
430       final Map<ServerName,List<Pair<HRegionInfo,Result>>> deadServers)
431   throws KeeperException, IOException {
432     Stat stat = new Stat();
433     RegionTransitionData data = ZKAssign.getDataAndWatch(watcher,
434         encodedRegionName, stat);
435     if (data == null) return false;
436     HRegionInfo hri = regionInfo;
437     if (hri == null) {
438       Pair<HRegionInfo, ServerName> p =
439         MetaReader.getRegion(catalogTracker, data.getRegionName());
440       if (p == null) return false;
441       hri = p.getFirst();
442     }
443     processRegionsInTransition(data, hri, deadServers, stat.getVersion());
444     return true;
445   }
446 
447   void processRegionsInTransition(final RegionTransitionData data,
448       final HRegionInfo regionInfo,
449       final Map<ServerName, List<Pair<HRegionInfo, Result>>> deadServers,
450       int expectedVersion)
451   throws KeeperException {
452     String encodedRegionName = regionInfo.getEncodedName();
453     LOG.info("Processing region " + regionInfo.getRegionNameAsString() +
454       " in state " + data.getEventType());
455     synchronized (regionsInTransition) {
456       switch (data.getEventType()) {
457       case M_ZK_REGION_CLOSING:
458         // If zk node of the region was updated by a live server skip this
459         // region and just add it into RIT.
460         if (isOnDeadServer(regionInfo, deadServers) &&
461             (data.getOrigin() == null || !serverManager.isServerOnline(data.getOrigin()))) {
462           // If was on dead server, its closed now. Force to OFFLINE and this
463           // will get it reassigned if appropriate
464           forceOffline(regionInfo, data);
465         } else {
466           // Just insert region into RIT.
467           // If this never updates the timeout will trigger new assignment
468           regionsInTransition.put(encodedRegionName, new RegionState(
469             regionInfo, RegionState.State.CLOSING,
470             data.getStamp(), data.getOrigin()));
471         }
472         break;
473 
474       case RS_ZK_REGION_CLOSED:
475       case RS_ZK_REGION_FAILED_OPEN:
476         // Region is closed, insert into RIT and handle it
477         addToRITandCallClose(regionInfo, RegionState.State.CLOSED, data);
478         break;
479 
480       case M_ZK_REGION_OFFLINE:
481         // Region is offline, insert into RIT and handle it like a closed
482         addToRITandCallClose(regionInfo, RegionState.State.OFFLINE, data);
483         break;
484 
485       case RS_ZK_REGION_OPENING:
486         // TODO: Could check if it was on deadServers.  If it was, then we could
487         // do what happens in TimeoutMonitor when it sees this condition.
488 
489         // Just insert region into RIT
490         // If this never updates the timeout will trigger new assignment
491         if (regionInfo.isMetaTable()) {
492           regionsInTransition.put(encodedRegionName, new RegionState(
493               regionInfo, RegionState.State.OPENING, data.getStamp(), data
494                   .getOrigin()));
495           // If ROOT or .META. table is waiting for timeout monitor to assign
496           // it may take lot of time when the assignment.timeout.period is
497           // the default value which may be very long.  We will not be able
498           // to serve any request during this time.
499           // So we will assign the ROOT and .META. region immediately.
500           processOpeningState(regionInfo);
501           break;
502         }
503         regionsInTransition.put(encodedRegionName, new RegionState(regionInfo,
504             RegionState.State.OPENING, data.getStamp(), data.getOrigin()));
505         break;
506 
507       case RS_ZK_REGION_OPENED:
508         // Region is opened, insert into RIT and handle it
509         regionsInTransition.put(encodedRegionName, new RegionState(
510             regionInfo, RegionState.State.OPEN,
511             data.getStamp(), data.getOrigin()));
512         ServerName sn = data.getOrigin() == null? null: data.getOrigin();
513         // sn could be null if this server is no longer online.  If
514         // that is the case, just let this RIT timeout; it'll be assigned
515         // to new server then.
516         if (sn == null) {
517           LOG.warn("Region in transition " + regionInfo.getEncodedName() +
518             " references a null server; letting RIT timeout so will be " +
519             "assigned elsewhere");
520         } else if (!serverManager.isServerOnline(sn)
521             && (isOnDeadServer(regionInfo, deadServers)
522                 || regionInfo.isMetaRegion() || regionInfo.isRootRegion())) {
523           forceOffline(regionInfo, data);
524         } else {
525           new OpenedRegionHandler(master, this, regionInfo, sn, expectedVersion)
526               .process();
527         }
528         break;
529       }
530     }
531   }
532 
533   /**
534    * Put the region <code>hri</code> into an offline state up in zk.
535    * @param hri
536    * @param oldData
537    * @throws KeeperException
538    */
539   private void forceOffline(final HRegionInfo hri,
540       final RegionTransitionData oldData)
541   throws KeeperException {
542     // If was on dead server, its closed now.  Force to OFFLINE and then
543     // handle it like a close; this will get it reassigned if appropriate
544     debugLog(hri, "RIT " + hri.getEncodedName() + " in state=" +
545       oldData.getEventType() + " was on deadserver; forcing offline");
546     ZKAssign.createOrForceNodeOffline(this.watcher, hri,
547       this.master.getServerName());
548     addToRITandCallClose(hri, RegionState.State.OFFLINE, oldData);
549   }
550 
551   /**
552    * Add to the in-memory copy of regions in transition and then call close
553    * handler on passed region <code>hri</code>
554    * @param hri
555    * @param state
556    * @param oldData
557    */
558   private void addToRITandCallClose(final HRegionInfo hri,
559       final RegionState.State state, final RegionTransitionData oldData) {
560     this.regionsInTransition.put(hri.getEncodedName(),
561       new RegionState(hri, state, oldData.getStamp(), oldData.getOrigin()));
562     new ClosedRegionHandler(this.master, this, hri).process();
563   }
564 
565   /**
566    * When a region is closed, it should be removed from the regionsToReopen
567    * @param hri HRegionInfo of the region which was closed
568    */
569   public void removeClosedRegion(HRegionInfo hri) {
570     if (!regionsToReopen.isEmpty()) {
571       if (regionsToReopen.remove(hri.getEncodedName()) != null) {
572           LOG.debug("Removed region from reopening regions because it was closed");
573       }
574     }
575   }
576 
577   /**
578    * @param regionInfo
579    * @param deadServers Map of deadServers and the regions they were carrying;
580    * can be null.
581    * @return True if the passed regionInfo in the passed map of deadServers?
582    */
583   private boolean isOnDeadServer(final HRegionInfo regionInfo,
584       final Map<ServerName, List<Pair<HRegionInfo, Result>>> deadServers) {
585     if (deadServers == null) return false;
586     for (Map.Entry<ServerName, List<Pair<HRegionInfo, Result>>> deadServer:
587         deadServers.entrySet()) {
588       for (Pair<HRegionInfo, Result> e: deadServer.getValue()) {
589         if (e.getFirst().equals(regionInfo)) return true;
590       }
591     }
592     return false;
593   }
594 
595   /**
596    * Handles various states an unassigned node can be in.
597    * <p>
598    * Method is called when a state change is suspected for an unassigned node.
599    * <p>
600    * This deals with skipped transitions (we got a CLOSED but didn't see CLOSING
601    * yet).
602    * @param data
603    * @param expectedVersion
604    */
605   private void handleRegion(final RegionTransitionData data, int expectedVersion) {
606     synchronized(regionsInTransition) {
607       if (data == null || data.getOrigin() == null) {
608         LOG.warn("Unexpected NULL input " + data);
609         return;
610       }
611       ServerName sn = data.getOrigin();
612       // Check if this is a special HBCK transition
613       if (sn.equals(HConstants.HBCK_CODE_SERVERNAME)) {
614         handleHBCK(data);
615         return;
616       }
617       String encodedName = HRegionInfo.encodeRegionName(data.getRegionName());
618       String prettyPrintedRegionName = HRegionInfo.prettyPrint(encodedName);
619       // Verify this is a known server
620       if (!serverManager.isServerOnline(sn) &&
621           !this.master.getServerName().equals(sn)
622           && !ignoreStatesRSOffline.contains(data.getEventType())) {
623         LOG.warn("Attempted to handle region transition for server but " +
624           "server is not online: " + prettyPrintedRegionName);
625         return;
626       }
627       // Printing if the event was created a long time ago helps debugging
628       boolean lateEvent = data.getStamp() <
629           (System.currentTimeMillis() - 15000);
630       LOG.debug("Handling transition=" + data.getEventType() +
631         ", server=" + data.getOrigin() + ", region=" +
632           (prettyPrintedRegionName == null? "null": prettyPrintedRegionName)  +
633           (lateEvent? ", which is more than 15 seconds late" : ""));
634       RegionState regionState = regionsInTransition.get(encodedName);
635       switch (data.getEventType()) {
636         case M_ZK_REGION_OFFLINE:
637           // Nothing to do.
638           break;
639 
640         case RS_ZK_REGION_SPLITTING:
641           if (!isInStateForSplitting(regionState)) break;
642           addSplittingToRIT(sn, encodedName);
643           break;
644 
645         case RS_ZK_REGION_SPLIT:
646           // RegionState must be null, or SPLITTING or PENDING_CLOSE.
647           if (!isInStateForSplitting(regionState)) break;
648           // If null, add SPLITTING state before going to SPLIT
649           if (regionState == null) {
650             regionState = addSplittingToRIT(sn, encodedName);
651             String message = "Received SPLIT for region " + prettyPrintedRegionName +
652               " from server " + sn;
653             // If still null, it means we cannot find it and it was already processed
654             if (regionState == null) {
655               LOG.warn(message + " but it doesn't exist anymore," +
656                   " probably already processed its split");
657               break;
658             }
659             LOG.info(message +
660                 " but region was not first in SPLITTING state; continuing");
661           }
662           // Check it has daughters.
663           byte [] payload = data.getPayload();
664           List<HRegionInfo> daughters = null;
665           try {
666             daughters = Writables.getHRegionInfos(payload, 0, payload.length);
667           } catch (IOException e) {
668             LOG.error("Dropped split! Failed reading split payload for " +
669               prettyPrintedRegionName);
670             break;
671           }
672           assert daughters.size() == 2;
673           // Assert that we can get a serverinfo for this server.
674           if (!this.serverManager.isServerOnline(sn)) {
675             LOG.error("Dropped split! ServerName=" + sn + " unknown.");
676             break;
677           }
678           // Run handler to do the rest of the SPLIT handling.
679           this.executorService.submit(new SplitRegionHandler(master, this,
680             regionState.getRegion(), sn, daughters));
681           break;
682 
683         case M_ZK_REGION_CLOSING:
684           // Should see CLOSING after we have asked it to CLOSE or additional
685           // times after already being in state of CLOSING
686           if (regionState == null ||
687               (!regionState.isPendingClose() && !regionState.isClosing())) {
688             LOG.warn("Received CLOSING for region " + prettyPrintedRegionName +
689               " from server " + data.getOrigin() + " but region was in " +
690               " the state " + regionState + " and not " +
691               "in expected PENDING_CLOSE or CLOSING states");
692             return;
693           }
694           // Transition to CLOSING (or update stamp if already CLOSING)
695           regionState.update(RegionState.State.CLOSING,
696               data.getStamp(), data.getOrigin());
697           break;
698 
699         case RS_ZK_REGION_CLOSED:
700           // Should see CLOSED after CLOSING but possible after PENDING_CLOSE
701           if (regionState == null ||
702               (!regionState.isPendingClose() && !regionState.isClosing())) {
703             LOG.warn("Received CLOSED for region " + prettyPrintedRegionName +
704                 " from server " + data.getOrigin() + " but region was in " +
705                 " the state " + regionState + " and not " +
706                 "in expected PENDING_CLOSE or CLOSING states");
707             return;
708           }
709           // Handle CLOSED by assigning elsewhere or stopping if a disable
710           // If we got here all is good.  Need to update RegionState -- else
711           // what follows will fail because not in expected state.
712           regionState.update(RegionState.State.CLOSED,
713               data.getStamp(), data.getOrigin());
714 	  removeClosedRegion(regionState.getRegion());
715           this.executorService.submit(new ClosedRegionHandler(master,
716             this, regionState.getRegion()));
717           break;
718           
719         case RS_ZK_REGION_FAILED_OPEN:
720           if (regionState == null ||
721               (!regionState.isPendingOpen() && !regionState.isOpening())) {
722             LOG.warn("Received FAILED_OPEN for region " + prettyPrintedRegionName +
723                 " from server " + data.getOrigin() + " but region was in " +
724                 " the state " + regionState + " and not in PENDING_OPEN or OPENING");
725             return;
726           }
727           // Handle this the same as if it were opened and then closed.
728           regionState.update(RegionState.State.CLOSED,
729               data.getStamp(), data.getOrigin());
730           this.executorService.submit(new ClosedRegionHandler(master,
731             this, regionState.getRegion()));
732           break;
733 
734         case RS_ZK_REGION_OPENING:
735           // Should see OPENING after we have asked it to OPEN or additional
736           // times after already being in state of OPENING
737           if (regionState == null ||
738               (!regionState.isPendingOpen() && !regionState.isOpening())) {
739             LOG.warn("Received OPENING for region " +
740                 prettyPrintedRegionName +
741                 " from server " + data.getOrigin() + " but region was in " +
742                 " the state " + regionState + " and not " +
743                 "in expected PENDING_OPEN or OPENING states");
744             return;
745           }
746           // Transition to OPENING (or update stamp if already OPENING)
747           regionState.update(RegionState.State.OPENING,
748               data.getStamp(), data.getOrigin());
749           break;
750 
751         case RS_ZK_REGION_OPENED:
752           // Should see OPENED after OPENING but possible after PENDING_OPEN
753           if (regionState == null ||
754               (!regionState.isPendingOpen() && !regionState.isOpening())) {
755             LOG.warn("Received OPENED for region " +
756                 prettyPrintedRegionName +
757                 " from server " + data.getOrigin() + " but region was in " +
758                 " the state " + regionState + " and not " +
759                 "in expected PENDING_OPEN or OPENING states");
760             return;
761           }
762           // Handle OPENED by removing from transition and deleted zk node
763           regionState.update(RegionState.State.OPEN,
764               data.getStamp(), data.getOrigin());
765           this.executorService.submit(
766             new OpenedRegionHandler(master, this, regionState.getRegion(),
767               data.getOrigin(), expectedVersion));
768           break;
769       }
770     }
771   }
772 
773   /**
774    * @return Returns true if this RegionState is splittable; i.e. the
775    * RegionState is currently in splitting state or pending_close or
776    * null (Anything else will return false). (Anything else will return false).
777    */
778   private boolean isInStateForSplitting(final RegionState rs) {
779     if (rs == null) return true;
780     if (rs.isSplitting()) return true;
781     if (convertPendingCloseToSplitting(rs)) return true;
782     LOG.warn("Dropped region split! Not in state good for SPLITTING; rs=" + rs);
783     return false;
784   }
785 
786   /**
787    * If the passed regionState is in PENDING_CLOSE, clean up PENDING_CLOSE
788    * state and convert it to SPLITTING instead.
789    * This can happen in case where master wants to close a region at same time
790    * a regionserver starts a split.  The split won.  Clean out old PENDING_CLOSE
791    * state.
792    * @param rs
793    * @return True if we converted from PENDING_CLOSE to SPLITTING
794    */
795   private boolean convertPendingCloseToSplitting(final RegionState rs) {
796     if (!rs.isPendingClose()) return false;
797     LOG.debug("Converting PENDING_CLOSE to SPLITING; rs=" + rs);
798     rs.update(RegionState.State.SPLITTING);
799     // Clean up existing state.  Clear from region plans seems all we
800     // have to do here by way of clean up of PENDING_CLOSE.
801     clearRegionPlan(rs.getRegion());
802     return true;
803   }
804 
805   /**
806    * @param serverName
807    * @param encodedName
808    * @return The SPLITTING RegionState we added to RIT for the passed region
809    * <code>encodedName</code>
810    */
811   private RegionState addSplittingToRIT(final ServerName serverName,
812       final String encodedName) {
813     RegionState regionState = null;
814     synchronized (this.regions) {
815       regionState = findHRegionInfoThenAddToRIT(serverName, encodedName);
816       if (regionState != null) {
817         regionState.update(RegionState.State.SPLITTING,
818           System.currentTimeMillis(), serverName);
819       }
820     }
821     return regionState;
822   }
823 
824   /**
825    * Caller must hold lock on <code>this.regions</code>.
826    * @param serverName
827    * @param encodedName
828    * @return The instance of RegionState that was added to RIT or null if error.
829    */
830   private RegionState findHRegionInfoThenAddToRIT(final ServerName serverName,
831       final String encodedName) {
832     HRegionInfo hri = findHRegionInfo(serverName, encodedName);
833     if (hri == null) {
834       LOG.warn("Region " + encodedName + " not found on server " + serverName +
835         "; failed processing");
836       return null;
837     }
838     // Add to regions in transition, then update state to SPLITTING.
839     return addToRegionsInTransition(hri);
840   }
841 
842   /**
843    * Caller must hold lock on <code>this.regions</code>.
844    * @param serverName
845    * @param encodedName
846    * @return Found HRegionInfo or null.
847    */
848   private HRegionInfo findHRegionInfo(final ServerName sn,
849       final String encodedName) {
850     if (!this.serverManager.isServerOnline(sn)) return null;
851     Set<HRegionInfo> hris = this.servers.get(sn);
852     HRegionInfo foundHri = null;
853     for (HRegionInfo hri: hris) {
854       if (hri.getEncodedName().equals(encodedName)) {
855         foundHri = hri;
856         break;
857       }
858     }
859     return foundHri;
860   }
861 
862   /**
863    * Handle a ZK unassigned node transition triggered by HBCK repair tool.
864    * <p>
865    * This is handled in a separate code path because it breaks the normal rules.
866    * @param data
867    */
868   private void handleHBCK(RegionTransitionData data) {
869     String encodedName = HRegionInfo.encodeRegionName(data.getRegionName());
870     LOG.info("Handling HBCK triggered transition=" + data.getEventType() +
871       ", server=" + data.getOrigin() + ", region=" +
872       HRegionInfo.prettyPrint(encodedName));
873     RegionState regionState = regionsInTransition.get(encodedName);
874     switch (data.getEventType()) {
875       case M_ZK_REGION_OFFLINE:
876         HRegionInfo regionInfo = null;
877         if (regionState != null) {
878           regionInfo = regionState.getRegion();
879         } else {
880           try {
881             regionInfo = MetaReader.getRegion(catalogTracker,
882                 data.getRegionName()).getFirst();
883           } catch (IOException e) {
884             LOG.info("Exception reading META doing HBCK repair operation", e);
885             return;
886           }
887         }
888         LOG.info("HBCK repair is triggering assignment of region=" +
889             regionInfo.getRegionNameAsString());
890         // trigger assign, node is already in OFFLINE so don't need to update ZK
891         assign(regionInfo, false);
892         break;
893 
894       default:
895         LOG.warn("Received unexpected region state from HBCK (" +
896             data.getEventType() + ")");
897         break;
898     }
899   }
900 
901   // ZooKeeper events
902 
903   /**
904    * New unassigned node has been created.
905    *
906    * <p>This happens when an RS begins the OPENING or CLOSING of a region by
907    * creating an unassigned node.
908    *
909    * <p>When this happens we must:
910    * <ol>
911    *   <li>Watch the node for further events</li>
912    *   <li>Read and handle the state in the node</li>
913    * </ol>
914    */
915   @Override
916   public void nodeCreated(String path) {
917     if(path.startsWith(watcher.assignmentZNode)) {
918       try {
919         Stat stat = new Stat();
920         RegionTransitionData data = ZKAssign.getDataAndWatch(watcher, path, stat);
921         if (data == null) {
922           return;
923         }
924         handleRegion(data, stat.getVersion());
925       } catch (KeeperException e) {
926         master.abort("Unexpected ZK exception reading unassigned node data", e);
927       }
928     }
929   }
930 
931   /**
932    * Existing unassigned node has had data changed.
933    *
934    * <p>This happens when an RS transitions from OFFLINE to OPENING, or between
935    * OPENING/OPENED and CLOSING/CLOSED.
936    *
937    * <p>When this happens we must:
938    * <ol>
939    *   <li>Watch the node for further events</li>
940    *   <li>Read and handle the state in the node</li>
941    * </ol>
942    */
943   @Override
944   public void nodeDataChanged(String path) {
945     if(path.startsWith(watcher.assignmentZNode)) {
946       try {
947         Stat stat = new Stat();
948         RegionTransitionData data = ZKAssign.getDataAndWatch(watcher, path, stat);
949         if (data == null) {
950           return;
951         }
952         handleRegion(data, stat.getVersion());
953       } catch (KeeperException e) {
954         master.abort("Unexpected ZK exception reading unassigned node data", e);
955       }
956     }
957   }
958 
959   @Override
960   public void nodeDeleted(String path) {
961     // Added so we notice when ephemeral nodes go away; in particular,
962     // SPLITTING or SPLIT nodes added by a regionserver splitting.
963     if (path.startsWith(this.watcher.assignmentZNode)) {
964       String regionName =
965         ZKAssign.getRegionName(this.master.getZooKeeper(), path);
966       RegionState rs = this.regionsInTransition.get(regionName);
967       if (rs != null) {
968         HRegionInfo regionInfo = rs.getRegion();
969         if (rs.isSplitting() || rs.isSplit()) {
970           LOG.debug("Ephemeral node deleted, regionserver crashed?, " +
971             "clearing from RIT; rs=" + rs);
972           clearRegionFromTransition(rs.getRegion());
973         } else {
974           LOG.debug("The znode of region " + regionInfo.getRegionNameAsString()
975               + " has been deleted.");
976           if (rs.isOpened()) {
977             makeRegionOnline(rs, regionInfo);
978           }
979         }
980       }
981     }
982   }
983   
984   private void makeRegionOnline(RegionState rs, HRegionInfo regionInfo) {
985     regionOnline(regionInfo, rs.serverName);
986     LOG.info("The master has opened the region "
987         + regionInfo.getRegionNameAsString() + " that was online on "
988         + rs.serverName);
989     if (this.getZKTable().isDisablingOrDisabledTable(
990         regionInfo.getTableNameAsString())) {
991       debugLog(regionInfo, "Opened region "
992           + regionInfo.getRegionNameAsString() + " but "
993           + "this table is disabled, triggering close of region");
994       unassign(regionInfo);
995     }
996   }
997 
998 
999   /**
1000    * New unassigned node has been created.
1001    *
1002    * <p>This happens when an RS begins the OPENING, SPLITTING or CLOSING of a
1003    * region by creating a znode.
1004    *
1005    * <p>When this happens we must:
1006    * <ol>
1007    *   <li>Watch the node for further children changed events</li>
1008    *   <li>Watch all new children for changed events</li>
1009    * </ol>
1010    */
1011   @Override
1012   public void nodeChildrenChanged(String path) {
1013     if(path.equals(watcher.assignmentZNode)) {
1014       try {
1015         // Just make sure we see the changes for the new znodes
1016         ZKUtil.listChildrenAndWatchThem(watcher,
1017             watcher.assignmentZNode);
1018       } catch(KeeperException e) {
1019         master.abort("Unexpected ZK exception reading unassigned children", e);
1020       }
1021     }
1022   }
1023 
1024   /**
1025    * Marks the region as online.  Removes it from regions in transition and
1026    * updates the in-memory assignment information.
1027    * <p>
1028    * Used when a region has been successfully opened on a region server.
1029    * @param regionInfo
1030    * @param sn
1031    */
1032   public void regionOnline(HRegionInfo regionInfo, ServerName sn) {
1033     synchronized (this.regionsInTransition) {
1034       RegionState rs =
1035         this.regionsInTransition.remove(regionInfo.getEncodedName());
1036       if (rs != null) {
1037         this.regionsInTransition.notifyAll();
1038       }
1039     }
1040     synchronized (this.regions) {
1041       // Add check
1042       ServerName oldSn = this.regions.get(regionInfo);
1043       if (oldSn != null) LOG.warn("Overwriting " + regionInfo.getEncodedName() +
1044         " on " + oldSn + " with " + sn);
1045       
1046       if (isServerOnline(sn)) {
1047         this.regions.put(regionInfo, sn);
1048         addToServers(sn, regionInfo);
1049         this.regions.notifyAll();
1050       } else {
1051         LOG.info("The server is not in online servers, ServerName=" + 
1052           sn.getServerName() + ", region=" + regionInfo.getEncodedName());
1053       }
1054     }
1055     // Remove plan if one.
1056     clearRegionPlan(regionInfo);
1057     // Update timers for all regions in transition going against this server.
1058     updateTimers(sn);
1059   }
1060 
1061   /**
1062    * Touch timers for all regions in transition that have the passed
1063    * <code>sn</code> in common.
1064    * Call this method whenever a server checks in.  Doing so helps the case where
1065    * a new regionserver has joined the cluster and its been given 1k regions to
1066    * open.  If this method is tickled every time the region reports in a
1067    * successful open then the 1k-th region won't be timed out just because its
1068    * sitting behind the open of 999 other regions.  This method is NOT used
1069    * as part of bulk assign -- there we have a different mechanism for extending
1070    * the regions in transition timer (we turn it off temporarily -- because
1071    * there is no regionplan involved when bulk assigning.
1072    * @param sn
1073    */
1074   private void updateTimers(final ServerName sn) {
1075     // This loop could be expensive.
1076     // First make a copy of current regionPlan rather than hold sync while
1077     // looping because holding sync can cause deadlock.  Its ok in this loop
1078     // if the Map we're going against is a little stale
1079     Map<String, RegionPlan> copy = new HashMap<String, RegionPlan>();
1080     synchronized(this.regionPlans) {
1081       copy.putAll(this.regionPlans);
1082     }
1083     for (Map.Entry<String, RegionPlan> e: copy.entrySet()) {
1084       if (e.getValue() == null || e.getValue().getDestination() == null) continue;
1085       if (!e.getValue().getDestination().equals(sn)) continue;
1086       RegionState rs = null;
1087       synchronized (this.regionsInTransition) {
1088         rs = this.regionsInTransition.get(e.getKey());
1089       }
1090       if (rs == null) continue;
1091       rs.updateTimestampToNow();
1092     }
1093   }
1094 
1095   /**
1096    * Marks the region as offline.  Removes it from regions in transition and
1097    * removes in-memory assignment information.
1098    * <p>
1099    * Used when a region has been closed and should remain closed.
1100    * @param regionInfo
1101    */
1102   public void regionOffline(final HRegionInfo regionInfo) {
1103     synchronized(this.regionsInTransition) {
1104       if (this.regionsInTransition.remove(regionInfo.getEncodedName()) != null) {
1105         this.regionsInTransition.notifyAll();
1106       }
1107     }
1108     // remove the region plan as well just in case.
1109     clearRegionPlan(regionInfo);
1110     setOffline(regionInfo);
1111   }
1112 
1113   /**
1114    * Sets the region as offline by removing in-memory assignment information but
1115    * retaining transition information.
1116    * <p>
1117    * Used when a region has been closed but should be reassigned.
1118    * @param regionInfo
1119    */
1120   public void setOffline(HRegionInfo regionInfo) {
1121     synchronized (this.regions) {
1122       ServerName sn = this.regions.remove(regionInfo);
1123       if (sn == null) return;
1124       Set<HRegionInfo> serverRegions = this.servers.get(sn);
1125       if (!serverRegions.remove(regionInfo)) {
1126         LOG.warn("No " + regionInfo + " on " + sn);
1127       }
1128     }
1129   }
1130 
1131   public void offlineDisabledRegion(HRegionInfo regionInfo) {
1132     // Disabling so should not be reassigned, just delete the CLOSED node
1133     LOG.debug("Table being disabled so deleting ZK node and removing from " +
1134         "regions in transition, skipping assignment of region " +
1135           regionInfo.getRegionNameAsString());
1136     try {
1137       if (!ZKAssign.deleteClosedNode(watcher, regionInfo.getEncodedName())) {
1138         // Could also be in OFFLINE mode
1139         ZKAssign.deleteOfflineNode(watcher, regionInfo.getEncodedName());
1140       }
1141     } catch (KeeperException.NoNodeException nne) {
1142       LOG.debug("Tried to delete closed node for " + regionInfo + " but it " +
1143           "does not exist so just offlining");
1144     } catch (KeeperException e) {
1145       this.master.abort("Error deleting CLOSED node in ZK", e);
1146     }
1147     regionOffline(regionInfo);
1148   }
1149 
1150   // Assignment methods
1151 
1152   /**
1153    * Assigns the specified region.
1154    * <p>
1155    * If a RegionPlan is available with a valid destination then it will be used
1156    * to determine what server region is assigned to.  If no RegionPlan is
1157    * available, region will be assigned to a random available server.
1158    * <p>
1159    * Updates the RegionState and sends the OPEN RPC.
1160    * <p>
1161    * This will only succeed if the region is in transition and in a CLOSED or
1162    * OFFLINE state or not in transition (in-memory not zk), and of course, the
1163    * chosen server is up and running (It may have just crashed!).  If the
1164    * in-memory checks pass, the zk node is forced to OFFLINE before assigning.
1165    *
1166    * @param region server to be assigned
1167    * @param setOfflineInZK whether ZK node should be created/transitioned to an
1168    *                       OFFLINE state before assigning the region
1169    */
1170   public void assign(HRegionInfo region, boolean setOfflineInZK) {
1171     assign(region, setOfflineInZK, false);
1172   }
1173 
1174   public void assign(HRegionInfo region, boolean setOfflineInZK,
1175       boolean forceNewPlan) {
1176     assign(region, setOfflineInZK, forceNewPlan, false);
1177   }
1178 
1179   /**
1180    * @param region
1181    * @param setOfflineInZK
1182    * @param forceNewPlan
1183    * @param hijack
1184    *          - true new assignment is needed, false otherwise
1185    */
1186   public void assign(HRegionInfo region, boolean setOfflineInZK,
1187       boolean forceNewPlan, boolean hijack) {
1188     //If hijack is true do not call disableRegionIfInRIT as 
1189     // we have not yet moved the znode to OFFLINE state.
1190     if (!hijack && isDisabledorDisablingRegionInRIT(region)) {
1191       return;
1192     }
1193     if (this.serverManager.isClusterShutdown()) {
1194       LOG.info("Cluster shutdown is set; skipping assign of " +
1195         region.getRegionNameAsString());
1196       return;
1197     }
1198     RegionState state = addToRegionsInTransition(region,
1199         hijack);
1200     synchronized (state) {
1201       assign(region, state, setOfflineInZK, forceNewPlan, hijack);
1202     }
1203   }
1204 
1205   /**
1206    * Bulk assign regions to <code>destination</code>.
1207    * @param destination
1208    * @param regions Regions to assign.
1209    */
1210   void assign(final ServerName destination,
1211       final List<HRegionInfo> regions) {
1212     if (regions.size() == 0) {
1213       return;
1214     }
1215     LOG.debug("Bulk assigning " + regions.size() + " region(s) to " +
1216       destination.toString());
1217 
1218     List<RegionState> states = new ArrayList<RegionState>(regions.size());
1219     synchronized (this.regionsInTransition) {
1220       for (HRegionInfo region: regions) {
1221         states.add(forceRegionStateToOffline(region));
1222       }
1223     }
1224     // Presumption is that only this thread will be updating the state at this
1225     // time; i.e. handlers on backend won't be trying to set it to OPEN, etc.
1226     AtomicInteger counter = new AtomicInteger(0);
1227     CreateUnassignedAsyncCallback cb =
1228       new CreateUnassignedAsyncCallback(this.watcher, destination, counter);
1229     for (RegionState state: states) {
1230       if (!asyncSetOfflineInZooKeeper(state, cb, state)) {
1231         return;
1232       }
1233     }
1234     // Wait until all unassigned nodes have been put up and watchers set.
1235     int total = regions.size();
1236     for (int oldCounter = 0; true;) {
1237       int count = counter.get();
1238       if (oldCounter != count) {
1239         LOG.info(destination.toString() + " unassigned znodes=" + count +
1240           " of total=" + total);
1241         oldCounter = count;
1242       }
1243       if (count == total) break;
1244       Threads.sleep(1);
1245     }
1246     // Move on to open regions.
1247     try {
1248       // Send OPEN RPC. If it fails on a IOE or RemoteException, the
1249       // TimeoutMonitor will pick up the pieces.
1250       long maxWaitTime = System.currentTimeMillis() +
1251         this.master.getConfiguration().
1252           getLong("hbase.regionserver.rpc.startup.waittime", 60000);
1253       while (!this.master.isStopped()) {
1254         try {
1255           this.serverManager.sendRegionOpen(destination, regions);
1256           break;
1257         } catch (RemoteException e) {
1258           IOException decodedException = e.unwrapRemoteException();
1259           if (decodedException instanceof RegionServerStoppedException) {
1260             LOG.warn("The region server was shut down, ", decodedException);
1261             // No need to retry, the region server is a goner.
1262             return;
1263           } else if (decodedException instanceof ServerNotRunningYetException) {
1264             // This is the one exception to retry.  For all else we should just fail
1265             // the startup.
1266             long now = System.currentTimeMillis();
1267             if (now > maxWaitTime) throw e;
1268             LOG.debug("Server is not yet up; waiting up to " +
1269                 (maxWaitTime - now) + "ms", e);
1270             Thread.sleep(1000);
1271           }
1272 
1273           throw decodedException;
1274         }
1275       }
1276     } catch (IOException e) {
1277       // Can be a socket timeout, EOF, NoRouteToHost, etc
1278       LOG.info("Unable to communicate with the region server in order" +
1279           " to assign regions", e);
1280     } catch (InterruptedException e) {
1281       throw new RuntimeException(e);
1282     }
1283     LOG.debug("Bulk assigning done for " + destination.toString());
1284   }
1285 
1286   /**
1287    * Callback handler for create unassigned znodes used during bulk assign.
1288    */
1289   static class CreateUnassignedAsyncCallback implements AsyncCallback.StringCallback {
1290     private final Log LOG = LogFactory.getLog(CreateUnassignedAsyncCallback.class);
1291     private final ZooKeeperWatcher zkw;
1292     private final ServerName destination;
1293     private final AtomicInteger counter;
1294 
1295     CreateUnassignedAsyncCallback(final ZooKeeperWatcher zkw,
1296         final ServerName destination, final AtomicInteger counter) {
1297       this.zkw = zkw;
1298       this.destination = destination;
1299       this.counter = counter;
1300     }
1301 
1302     @Override
1303     public void processResult(int rc, String path, Object ctx, String name) {
1304       if (rc != 0) {
1305         // Thisis resultcode.  If non-zero, need to resubmit.
1306         LOG.warn("rc != 0 for " + path + " -- retryable connectionloss -- " +
1307           "FIX see http://wiki.apache.org/hadoop/ZooKeeper/FAQ#A2");
1308         this.zkw.abort("Connectionloss writing unassigned at " + path +
1309           ", rc=" + rc, null);
1310         return;
1311       }
1312       LOG.debug("rs=" + (RegionState)ctx + ", server=" + this.destination.toString());
1313       // Async exists to set a watcher so we'll get triggered when
1314       // unassigned node changes.
1315       this.zkw.getRecoverableZooKeeper().getZooKeeper().exists(path, this.zkw,
1316         new ExistsUnassignedAsyncCallback(this.counter, destination), ctx);
1317     }
1318   }
1319 
1320   /**
1321    * Callback handler for the exists call that sets watcher on unassigned znodes.
1322    * Used during bulk assign on startup.
1323    */
1324   static class ExistsUnassignedAsyncCallback implements AsyncCallback.StatCallback {
1325     private final Log LOG = LogFactory.getLog(ExistsUnassignedAsyncCallback.class);
1326     private final AtomicInteger counter;
1327     private ServerName destination;
1328 
1329     ExistsUnassignedAsyncCallback(final AtomicInteger counter, ServerName destination) {
1330       this.counter = counter;
1331       this.destination = destination;
1332     }
1333 
1334     @Override
1335     public void processResult(int rc, String path, Object ctx, Stat stat) {
1336       if (rc != 0) {
1337         // Thisis resultcode.  If non-zero, need to resubmit.
1338         LOG.warn("rc != 0 for " + path + " -- retryable connectionloss -- " +
1339           "FIX see http://wiki.apache.org/hadoop/ZooKeeper/FAQ#A2");
1340         return;
1341       }
1342       RegionState state = (RegionState)ctx;
1343       LOG.debug("rs=" + state);
1344       // Transition RegionState to PENDING_OPEN here in master; means we've
1345       // sent the open.  We're a little ahead of ourselves here since we've not
1346       // yet sent out the actual open but putting this state change after the
1347       // call to open risks our writing PENDING_OPEN after state has been moved
1348       // to OPENING by the regionserver.
1349       state.update(RegionState.State.PENDING_OPEN, System.currentTimeMillis(), destination);
1350       this.counter.addAndGet(1);
1351     }
1352   }
1353 
1354   /**
1355    * @param region
1356    * @return The current RegionState
1357    */
1358   private RegionState addToRegionsInTransition(final HRegionInfo region) {
1359     return addToRegionsInTransition(region, false);
1360   }
1361   /**
1362    * @param region
1363    * @param hijack
1364    * @return The current RegionState
1365    */
1366   private RegionState addToRegionsInTransition(final HRegionInfo region,
1367       boolean hijack) {
1368     synchronized (regionsInTransition) {
1369       return forceRegionStateToOffline(region, hijack);
1370     }
1371   }
1372   /**
1373    * Sets regions {@link RegionState} to {@link RegionState.State#OFFLINE}.
1374    * Caller must hold lock on this.regionsInTransition.
1375    * @param region
1376    * @return Amended RegionState.
1377    */
1378   private RegionState forceRegionStateToOffline(final HRegionInfo region) {
1379     return forceRegionStateToOffline(region, false);
1380   }
1381 
1382   /**
1383    * Sets regions {@link RegionState} to {@link RegionState.State#OFFLINE}.
1384    * Caller must hold lock on this.regionsInTransition.
1385    * @param region
1386    * @param hijack
1387    * @return Amended RegionState.
1388    */
1389   private RegionState forceRegionStateToOffline(final HRegionInfo region,
1390       boolean hijack) {
1391     String encodedName = region.getEncodedName();
1392     RegionState state = this.regionsInTransition.get(encodedName);
1393     if (state == null) {
1394       state = new RegionState(region, RegionState.State.OFFLINE);
1395       this.regionsInTransition.put(encodedName, state);
1396     } else {
1397       // If we are reassigning the node do not force in-memory state to OFFLINE.
1398       // Based on the znode state we will decide if to change
1399       // in-memory state to OFFLINE or not. It will
1400       // be done before setting the znode to OFFLINE state.
1401       if (!hijack) {
1402         LOG.debug("Forcing OFFLINE; was=" + state);
1403         state.update(RegionState.State.OFFLINE);
1404       }
1405     }
1406     return state;
1407   }
1408 
1409   /**
1410    * Caller must hold lock on the passed <code>state</code> object.
1411    * @param state
1412    * @param setOfflineInZK
1413    * @param forceNewPlan
1414    * @param hijack
1415    */
1416   private void assign(final HRegionInfo region, final RegionState state,
1417       final boolean setOfflineInZK, final boolean forceNewPlan,
1418       boolean hijack) {
1419     for (int i = 0; i < this.maximumAssignmentAttempts; i++) {
1420       int versionOfOfflineNode = -1;
1421       if (setOfflineInZK) {
1422         // get the version of the znode after setting it to OFFLINE.
1423         // versionOfOfflineNode will be -1 if the znode was not set to OFFLINE
1424         versionOfOfflineNode = setOfflineInZooKeeper(state,
1425             hijack);
1426         if(versionOfOfflineNode != -1){
1427           if (isDisabledorDisablingRegionInRIT(region)) {
1428             return;
1429           }
1430         }
1431       }
1432       
1433       if (setOfflineInZK && versionOfOfflineNode == -1) {
1434         return;
1435       }
1436       
1437       if (this.master.isStopped()) {
1438         LOG.debug("Server stopped; skipping assign of " + state);
1439         return;
1440       }
1441       RegionPlan plan = getRegionPlan(state, forceNewPlan);
1442       if (plan == null) {
1443         debugLog(state.getRegion(),
1444             "Unable to determine a plan to assign " + state);
1445         return; // Should get reassigned later when RIT times out.
1446       }
1447       try {
1448         debugLog(state.getRegion(),
1449           "Assigning region " + state.getRegion().getRegionNameAsString() +
1450           " to " + plan.getDestination().toString());
1451         // Transition RegionState to PENDING_OPEN
1452         state.update(RegionState.State.PENDING_OPEN, System.currentTimeMillis(),
1453             plan.getDestination());
1454         // Send OPEN RPC. This can fail if the server on other end is is not up.
1455         // Pass the version that was obtained while setting the node to OFFLINE.
1456         RegionOpeningState regionOpenState = serverManager.sendRegionOpen(plan
1457             .getDestination(), state.getRegion(), versionOfOfflineNode);
1458         if (regionOpenState == RegionOpeningState.ALREADY_OPENED) {
1459           // Remove region from in-memory transition and unassigned node from ZK
1460           // While trying to enable the table the regions of the table were
1461           // already enabled.
1462           debugLog(state.getRegion(),
1463               "ALREADY_OPENED region " + state.getRegion().getRegionNameAsString() +
1464               " to " + plan.getDestination().toString());
1465           String encodedRegionName = state.getRegion()
1466               .getEncodedName();
1467           try {
1468             ZKAssign.deleteOfflineNode(master.getZooKeeper(), encodedRegionName);
1469           } catch (KeeperException.NoNodeException e) {
1470             if(LOG.isDebugEnabled()){
1471               LOG.debug("The unassigned node "+encodedRegionName+" doesnot exist.");
1472             }
1473           } catch (KeeperException e) {
1474             master.abort(
1475                 "Error deleting OFFLINED node in ZK for transition ZK node ("
1476                     + encodedRegionName + ")", e);
1477           }
1478           synchronized (this.regionsInTransition) {
1479             this.regionsInTransition.remove(plan.getRegionInfo()
1480                 .getEncodedName());
1481           }
1482           synchronized (this.regions) {
1483             this.regions.put(plan.getRegionInfo(), plan.getDestination());
1484           }
1485         }
1486         break;
1487       } catch (Throwable t) {
1488         if (t instanceof RemoteException) {
1489           t = ((RemoteException) t).unwrapRemoteException();
1490           if (t instanceof RegionAlreadyInTransitionException) {
1491             String errorMsg = "Failed assignment in: " + plan.getDestination()
1492                 + " due to " + t.getMessage();
1493             LOG.error(errorMsg, t);
1494             return;
1495           }
1496         }
1497         LOG.warn("Failed assignment of " +
1498           state.getRegion().getRegionNameAsString() + " to " +
1499           plan.getDestination() + ", trying to assign elsewhere instead; " +
1500           "retry=" + i, t);
1501         // Clean out plan we failed execute and one that doesn't look like it'll
1502         // succeed anyways; we need a new plan!
1503         // Transition back to OFFLINE
1504         state.update(RegionState.State.OFFLINE);
1505         // Force a new plan and reassign.  Will return null if no servers.
1506         if (getRegionPlan(state, plan.getDestination(), true) == null) {
1507           LOG.warn("Unable to find a viable location to assign region " +
1508             state.getRegion().getRegionNameAsString());
1509           return;
1510         }
1511       }
1512     }
1513   }
1514 
1515   private void debugLog(HRegionInfo region, String string) {
1516     if (region.isMetaTable()) {
1517       LOG.info(string);
1518     } else {
1519       LOG.debug(string);
1520     }
1521   }
1522 
1523   private boolean isDisabledorDisablingRegionInRIT(final HRegionInfo region) {
1524     String tableName = region.getTableNameAsString();
1525     boolean disabled = this.zkTable.isDisabledTable(tableName);
1526     if (disabled || this.zkTable.isDisablingTable(tableName)) {
1527       LOG.info("Table " + tableName + (disabled ? " disabled;" : " disabling;") +
1528         " skipping assign of " + region.getRegionNameAsString());
1529       offlineDisabledRegion(region);
1530       return true;
1531     }
1532     return false;
1533   }
1534 
1535   /**
1536    * Set region as OFFLINED up in zookeeper
1537    * 
1538    * @param state
1539    * @param hijack
1540    *          - true if needs to be hijacked and reassigned, false otherwise.
1541    * @return the version of the offline node if setting of the OFFLINE node was
1542    *         successful, -1 otherwise.
1543    */
1544   int setOfflineInZooKeeper(final RegionState state,
1545       boolean hijack) {
1546     // In case of reassignment the current state in memory need not be
1547     // OFFLINE. 
1548     if (!hijack && !state.isClosed() && !state.isOffline()) {
1549       this.master.abort("Unexpected state trying to OFFLINE; " + state,
1550           new IllegalStateException());
1551       return -1;
1552     }
1553     boolean allowZNodeCreation = false;
1554     // Under reassignment if the current state is PENDING_OPEN
1555     // or OPENING then refresh the in-memory state to PENDING_OPEN. This is
1556     // important because if the region was in 
1557     // RS_OPENING state for a long time the master will try to force the znode
1558     // to OFFLINE state meanwhile the RS could have opened the corresponding
1559     // region and the state in znode will be RS_ZK_REGION_OPENED.
1560     // For all other cases we can change the in-memory state to OFFLINE.
1561     if (hijack &&
1562         (state.getState().equals(RegionState.State.PENDING_OPEN) || 
1563             state.getState().equals(RegionState.State.OPENING))) {
1564       state.update(RegionState.State.PENDING_OPEN);
1565       allowZNodeCreation = false;
1566     } else {
1567       state.update(RegionState.State.OFFLINE);
1568       allowZNodeCreation = true;
1569     }
1570     int versionOfOfflineNode = -1;
1571     try {
1572       // get the version after setting the znode to OFFLINE
1573       versionOfOfflineNode = ZKAssign.createOrForceNodeOffline(master.getZooKeeper(), 
1574           state.getRegion(), this.master.getServerName(),
1575           hijack, allowZNodeCreation);
1576       if (versionOfOfflineNode == -1) {
1577         LOG.warn("Attempted to create/force node into OFFLINE state before "
1578             + "completing assignment but failed to do so for " + state);
1579         return -1;
1580       }
1581     } catch (KeeperException e) {
1582       master.abort("Unexpected ZK exception creating/setting node OFFLINE", e);
1583       return -1;
1584     }
1585     return versionOfOfflineNode;
1586   }
1587 
1588   /**
1589    * Set region as OFFLINED up in zookeeper asynchronously.
1590    * @param state
1591    * @return True if we succeeded, false otherwise (State was incorrect or failed
1592    * updating zk).
1593    */
1594   boolean asyncSetOfflineInZooKeeper(final RegionState state,
1595       final AsyncCallback.StringCallback cb, final Object ctx) {
1596     if (!state.isClosed() && !state.isOffline()) {
1597         new RuntimeException("Unexpected state trying to OFFLINE; " + state);
1598       this.master.abort("Unexpected state trying to OFFLINE; " + state,
1599         new IllegalStateException());
1600       return false;
1601     }
1602     state.update(RegionState.State.OFFLINE);
1603     try {
1604       ZKAssign.asyncCreateNodeOffline(master.getZooKeeper(), state.getRegion(),
1605         this.master.getServerName(), cb, ctx);
1606     } catch (KeeperException e) {
1607       master.abort("Unexpected ZK exception creating/setting node OFFLINE", e);
1608       return false;
1609     }
1610     return true;
1611   }
1612 
1613   /**
1614    * @param state
1615    * @return Plan for passed <code>state</code> (If none currently, it creates one or
1616    * if no servers to assign, it returns null).
1617    */
1618   RegionPlan getRegionPlan(final RegionState state,
1619       final boolean forceNewPlan) {
1620     return getRegionPlan(state, null, forceNewPlan);
1621   }
1622 
1623   /**
1624    * @param state
1625    * @param serverToExclude Server to exclude (we know its bad). Pass null if
1626    * all servers are thought to be assignable.
1627    * @param forceNewPlan If true, then if an existing plan exists, a new plan
1628    * will be generated.
1629    * @return Plan for passed <code>state</code> (If none currently, it creates one or
1630    * if no servers to assign, it returns null).
1631    */
1632   RegionPlan getRegionPlan(final RegionState state,
1633       final ServerName serverToExclude, final boolean forceNewPlan) {
1634     // Pickup existing plan or make a new one
1635     final String encodedName = state.getRegion().getEncodedName();
1636     final List<ServerName> servers = this.serverManager.getOnlineServersList();
1637     final List<ServerName> drainingServers = this.serverManager.getDrainingServersList();
1638 
1639     if (serverToExclude != null) servers.remove(serverToExclude);
1640 
1641     // Loop through the draining server list and remove them from the server
1642     // list.
1643     if (!drainingServers.isEmpty()) {
1644       for (final ServerName server: drainingServers) {
1645         LOG.debug("Removing draining server: " + server +
1646             " from eligible server pool.");
1647         servers.remove(server);
1648       }
1649     }
1650 
1651     if (servers.isEmpty()) return null;
1652 
1653     RegionPlan randomPlan = new RegionPlan(state.getRegion(), null,
1654       balancer.randomAssignment(servers));
1655     boolean newPlan = false;
1656     RegionPlan existingPlan = null;
1657 
1658     synchronized (this.regionPlans) {
1659       existingPlan = this.regionPlans.get(encodedName);
1660 
1661       if (existingPlan != null && existingPlan.getDestination() != null) {
1662         LOG.debug("Found an existing plan for " +
1663             state.getRegion().getRegionNameAsString() +
1664        " destination server is + " + existingPlan.getDestination().toString());
1665       }
1666 
1667       if (forceNewPlan
1668           || existingPlan == null
1669           || existingPlan.getDestination() == null
1670           || drainingServers.contains(existingPlan.getDestination())) {
1671         newPlan = true;
1672         this.regionPlans.put(encodedName, randomPlan);
1673       }
1674     }
1675 
1676     if (newPlan) {
1677       debugLog(state.getRegion(), "No previous transition plan was found (or we are ignoring " +
1678         "an existing plan) for " + state.getRegion().getRegionNameAsString() +
1679         " so generated a random one; " + randomPlan + "; " +
1680         serverManager.countOfRegionServers() +
1681                " (online=" + serverManager.getOnlineServers().size() +
1682                ", exclude=" + drainingServers.size() + ") available servers");
1683         return randomPlan;
1684       }
1685       debugLog(state.getRegion(), "Using pre-existing plan for region " +
1686                state.getRegion().getRegionNameAsString() + "; plan=" + existingPlan);
1687       return existingPlan;
1688   }
1689 
1690   /**
1691    * Unassign the list of regions. Configuration knobs:
1692    * hbase.bulk.waitbetween.reopen indicates the number of milliseconds to
1693    * wait before unassigning another region from this region server
1694    *
1695    * @param regions
1696    * @throws InterruptedException
1697    */
1698   public void unassign(List<HRegionInfo> regions) {
1699     int waitTime = this.master.getConfiguration().getInt(
1700         "hbase.bulk.waitbetween.reopen", 0);
1701     for (HRegionInfo region : regions) {
1702       if (isRegionInTransition(region) != null)
1703         continue;
1704       unassign(region, false);
1705       while (isRegionInTransition(region) != null) {
1706         try {
1707           Thread.sleep(10);
1708         } catch (InterruptedException e) {
1709           // Do nothing, continue
1710         }
1711       }
1712       if (waitTime > 0)
1713         try {
1714           Thread.sleep(waitTime);
1715         } catch (InterruptedException e) {
1716           // Do nothing, continue
1717         }
1718     }
1719   }
1720 
1721   /**
1722    * Unassigns the specified region.
1723    * <p>
1724    * Updates the RegionState and sends the CLOSE RPC.
1725    * <p>
1726    * If a RegionPlan is already set, it will remain.
1727    *
1728    * @param region server to be unassigned
1729    */
1730   public void unassign(HRegionInfo region) {
1731     unassign(region, false);
1732   }
1733 
1734   /**
1735    * Unassigns the specified region.
1736    * <p>
1737    * Updates the RegionState and sends the CLOSE RPC.
1738    * <p>
1739    * If a RegionPlan is already set, it will remain.
1740    *
1741    * @param region server to be unassigned
1742    * @param force if region should be closed even if already closing
1743    */
1744   public void unassign(HRegionInfo region, boolean force) {
1745     debugLog(region, "Starting unassignment of region " +
1746       region.getRegionNameAsString() + " (offlining)");
1747     synchronized (this.regions) {
1748       // Check if this region is currently assigned
1749       if (!regions.containsKey(region)) {
1750         debugLog(region, "Attempted to unassign region " +
1751           region.getRegionNameAsString() + " but it is not " +
1752           "currently assigned anywhere");
1753         return;
1754       }
1755     }
1756     String encodedName = region.getEncodedName();
1757     // Grab the state of this region and synchronize on it
1758     RegionState state;
1759     synchronized (regionsInTransition) {
1760       state = regionsInTransition.get(encodedName);
1761       if (state == null) {
1762 
1763          // Create the znode in CLOSING state
1764         try {
1765           ZKAssign.createNodeClosing(
1766             master.getZooKeeper(), region, master.getServerName());
1767         } catch (KeeperException e) {
1768           master.abort("Unexpected ZK exception creating node CLOSING", e);
1769           return;
1770         }
1771         state = new RegionState(region, RegionState.State.PENDING_CLOSE);
1772         regionsInTransition.put(encodedName, state);
1773       } else if (force && (state.isPendingClose() || state.isClosing())) {
1774         debugLog(region,
1775             "Attempting to unassign region " + region.getRegionNameAsString() + 
1776                 " which is already " + state.getState()  + 
1777                 " but forcing to send a CLOSE RPC again ");
1778         state.update(state.getState());
1779       } else {
1780         debugLog(region, "Attempting to unassign region " +
1781           region.getRegionNameAsString() + " but it is " +
1782           "already in transition (" + state.getState() + ")");
1783         return;
1784       }
1785     }
1786     // Send CLOSE RPC
1787     ServerName server = null;
1788     synchronized (this.regions) {
1789       server = regions.get(region);
1790     }
1791     try {
1792       // TODO: We should consider making this look more like it does for the
1793       // region open where we catch all throwables and never abort
1794       if (serverManager.sendRegionClose(server, state.getRegion())) {
1795         debugLog(region, "Sent CLOSE to " + server + " for region " +
1796           region.getRegionNameAsString());
1797         return;
1798       }
1799       // This never happens. Currently regionserver close always return true.
1800       LOG.warn("Server " + server + " region CLOSE RPC returned false for " +
1801         region.getEncodedName());
1802     } catch (NotServingRegionException nsre) {
1803       LOG.info("Server " + server + " returned " + nsre + " for " +
1804         region.getEncodedName());
1805       // Presume that master has stale data.  Presume remote side just split.
1806       // Presume that the split message when it comes in will fix up the master's
1807       // in memory cluster state.
1808     } catch (Throwable t) {
1809       if (t instanceof RemoteException) {
1810         t = ((RemoteException)t).unwrapRemoteException();
1811         if (t instanceof NotServingRegionException) {
1812           if (checkIfRegionBelongsToDisabling(region)) {
1813             // Remove from the regionsinTransition map
1814             LOG.info("While trying to recover the table "
1815                 + region.getTableNameAsString()
1816                 + " to DISABLED state the region " + region
1817                 + " was offlined but the table was in DISABLING state");
1818             synchronized (this.regionsInTransition) {
1819               this.regionsInTransition.remove(region.getEncodedName());
1820             }
1821             // Remove from the regionsMap
1822             synchronized (this.regions) {
1823               this.regions.remove(region);
1824             }
1825           }
1826         }
1827         // RS is already processing this region, only need to update the timestamp
1828         if (t instanceof RegionAlreadyInTransitionException) {
1829           debugLog(region, "update " + state + " the timestamp.");
1830           state.update(state.getState());
1831         }
1832       }
1833       LOG.info("Server " + server + " returned " + t + " for " +
1834         region.getEncodedName());
1835       // Presume retry or server will expire.
1836     }
1837   }
1838 
1839   /**
1840    * Waits until the specified region has completed assignment.
1841    * <p>
1842    * If the region is already assigned, returns immediately.  Otherwise, method
1843    * blocks until the region is assigned.
1844    * @param regionInfo region to wait on assignment for
1845    * @throws InterruptedException
1846    */
1847   public void waitForAssignment(HRegionInfo regionInfo)
1848   throws InterruptedException {
1849     synchronized(regions) {
1850       while(!regions.containsKey(regionInfo)) {
1851         regions.wait();
1852       }
1853     }
1854   }
1855 
1856   /**
1857    * Assigns the ROOT region.
1858    * <p>
1859    * Assumes that ROOT is currently closed and is not being actively served by
1860    * any RegionServer.
1861    * <p>
1862    * Forcibly unsets the current root region location in ZooKeeper and assigns
1863    * ROOT to a random RegionServer.
1864    * @throws KeeperException
1865    */
1866   public void assignRoot() throws KeeperException {
1867     RootLocationEditor.deleteRootLocation(this.master.getZooKeeper());
1868     assign(HRegionInfo.ROOT_REGIONINFO, true);
1869   }
1870 
1871   /**
1872    * Assigns the META region.
1873    * <p>
1874    * Assumes that META is currently closed and is not being actively served by
1875    * any RegionServer.
1876    * <p>
1877    * Forcibly assigns META to a random RegionServer.
1878    */
1879   public void assignMeta() {
1880     // Force assignment to a random server
1881     assign(HRegionInfo.FIRST_META_REGIONINFO, true);
1882   }
1883 
1884   /**
1885    * Assigns all user regions, if any.  Used during cluster startup.
1886    * <p>
1887    * This is a synchronous call and will return once every region has been
1888    * assigned.  If anything fails, an exception is thrown
1889    * @throws InterruptedException
1890    * @throws IOException
1891    */
1892   public void assignUserRegions(List<HRegionInfo> regions, List<ServerName> servers)
1893   throws IOException, InterruptedException {
1894     if (regions == null)
1895       return;
1896     Map<ServerName, List<HRegionInfo>> bulkPlan = null;
1897     // Generate a round-robin bulk assignment plan
1898     bulkPlan = balancer.roundRobinAssignment(regions, servers);
1899     LOG.info("Bulk assigning " + regions.size() + " region(s) round-robin across " +
1900                servers.size() + " server(s)");
1901     // Use fixed count thread pool assigning.
1902     BulkAssigner ba = new StartupBulkAssigner(this.master, bulkPlan, this);
1903     ba.bulkAssign();
1904     LOG.info("Bulk assigning done");
1905   }
1906 
1907   /**
1908    * Assigns all user regions, if any exist.  Used during cluster startup.
1909    * <p>
1910    * This is a synchronous call and will return once every region has been
1911    * assigned.  If anything fails, an exception is thrown and the cluster
1912    * should be shutdown.
1913    * @throws InterruptedException
1914    * @throws IOException
1915    */
1916   public void assignAllUserRegions() throws IOException, InterruptedException {
1917     // Get all available servers
1918     List<ServerName> servers = serverManager.getOnlineServersList();
1919 
1920     // If there are no servers we need not proceed with region assignment.
1921     if(servers.isEmpty()) return;
1922 
1923     // Scan META for all user regions, skipping any disabled tables
1924     Map<HRegionInfo, ServerName> allRegions =
1925       MetaReader.fullScan(catalogTracker, this.zkTable.getDisabledTables(), true);
1926     if (allRegions == null || allRegions.isEmpty()) return;
1927 
1928     // Determine what type of assignment to do on startup
1929     boolean retainAssignment = master.getConfiguration().
1930       getBoolean("hbase.master.startup.retainassign", true);
1931 
1932     Map<ServerName, List<HRegionInfo>> bulkPlan = null;
1933     if (retainAssignment) {
1934       // Reuse existing assignment info
1935       bulkPlan = balancer.retainAssignment(allRegions, servers);
1936     } else {
1937       // assign regions in round-robin fashion
1938       assignUserRegions(new ArrayList<HRegionInfo>(allRegions.keySet()), servers);
1939       return;
1940     }
1941     LOG.info("Bulk assigning " + allRegions.size() + " region(s) across " +
1942       servers.size() + " server(s), retainAssignment=" + retainAssignment);
1943 
1944     // Use fixed count thread pool assigning.
1945     BulkAssigner ba = new StartupBulkAssigner(this.master, bulkPlan, this);
1946     ba.bulkAssign();
1947     LOG.info("Bulk assigning done");
1948   }
1949 
1950   /**
1951    * Run bulk assign on startup.  Does one RCP per regionserver passing a
1952    * batch of reginons using {@link SingleServerBulkAssigner}.
1953    * Uses default {@link #getUncaughtExceptionHandler()}
1954    * which will abort the Server if exception.
1955    */
1956   static class StartupBulkAssigner extends BulkAssigner {
1957     final Map<ServerName, List<HRegionInfo>> bulkPlan;
1958     final AssignmentManager assignmentManager;
1959 
1960     StartupBulkAssigner(final Server server,
1961         final Map<ServerName, List<HRegionInfo>> bulkPlan,
1962         final AssignmentManager am) {
1963       super(server);
1964       this.bulkPlan = bulkPlan;
1965       this.assignmentManager = am;
1966     }
1967 
1968     @Override
1969     public boolean bulkAssign(boolean sync) throws InterruptedException {
1970       // Disable timing out regions in transition up in zk while bulk assigning.
1971       this.assignmentManager.timeoutMonitor.bulkAssign(true);
1972       try {
1973         return super.bulkAssign(sync);
1974       } finally {
1975         // Reenable timing out regions in transition up in zi.
1976         this.assignmentManager.timeoutMonitor.bulkAssign(false);
1977       }
1978     }
1979 
1980     @Override
1981     protected String getThreadNamePrefix() {
1982       return this.server.getServerName() + "-StartupBulkAssigner";
1983     }
1984 
1985     @Override
1986     protected void populatePool(java.util.concurrent.ExecutorService pool) {
1987       for (Map.Entry<ServerName, List<HRegionInfo>> e: this.bulkPlan.entrySet()) {
1988         pool.execute(new SingleServerBulkAssigner(e.getKey(), e.getValue(),
1989           this.assignmentManager));
1990       }
1991     }
1992 
1993     protected boolean waitUntilDone(final long timeout)
1994     throws InterruptedException {
1995       Set<HRegionInfo> regionSet = new HashSet<HRegionInfo>();
1996       for (List<HRegionInfo> regionList : bulkPlan.values()) {
1997         regionSet.addAll(regionList);
1998       }
1999       return this.assignmentManager.waitUntilNoRegionsInTransition(timeout, regionSet);
2000     }
2001 
2002     @Override
2003     protected long getTimeoutOnRIT() {
2004       // Guess timeout.  Multiply the number of regions on a random server
2005       // by how long we thing one region takes opening.
2006       long perRegionOpenTimeGuesstimate =
2007         this.server.getConfiguration().getLong("hbase.bulk.assignment.perregion.open.time", 1000);
2008       int regionsPerServer =
2009         this.bulkPlan.entrySet().iterator().next().getValue().size();
2010       long timeout = perRegionOpenTimeGuesstimate * regionsPerServer;
2011       LOG.debug("Timeout-on-RIT=" + timeout);
2012       return timeout;
2013     }
2014   }
2015 
2016   /**
2017    * Bulk user region assigner.
2018    * If failed assign, lets timeout in RIT do cleanup.
2019    */
2020   static class GeneralBulkAssigner extends StartupBulkAssigner {
2021     GeneralBulkAssigner(final Server server,
2022         final Map<ServerName, List<HRegionInfo>> bulkPlan,
2023         final AssignmentManager am) {
2024       super(server, bulkPlan, am);
2025     }
2026 
2027     @Override
2028     protected UncaughtExceptionHandler getUncaughtExceptionHandler() {
2029       return new UncaughtExceptionHandler() {
2030         @Override
2031         public void uncaughtException(Thread t, Throwable e) {
2032           LOG.warn("Assigning regions in " + t.getName(), e);
2033         }
2034       };
2035     }
2036   }
2037 
2038   /**
2039    * Manage bulk assigning to a server.
2040    */
2041   static class SingleServerBulkAssigner implements Runnable {
2042     private final ServerName regionserver;
2043     private final List<HRegionInfo> regions;
2044     private final AssignmentManager assignmentManager;
2045 
2046     SingleServerBulkAssigner(final ServerName regionserver,
2047         final List<HRegionInfo> regions, final AssignmentManager am) {
2048       this.regionserver = regionserver;
2049       this.regions = regions;
2050       this.assignmentManager = am;
2051     }
2052     @Override
2053     public void run() {
2054       this.assignmentManager.assign(this.regionserver, this.regions);
2055     }
2056   }
2057 
2058   /**
2059    * Wait until no regions in transition.
2060    * @param timeout How long to wait.
2061    * @return True if nothing in regions in transition.
2062    * @throws InterruptedException
2063    */
2064   boolean waitUntilNoRegionsInTransition(final long timeout)
2065   throws InterruptedException {
2066     // Blocks until there are no regions in transition. It is possible that
2067     // there
2068     // are regions in transition immediately after this returns but guarantees
2069     // that if it returns without an exception that there was a period of time
2070     // with no regions in transition from the point-of-view of the in-memory
2071     // state of the Master.
2072     long startTime = System.currentTimeMillis();
2073     long remaining = timeout;
2074     synchronized (regionsInTransition) {
2075       while (regionsInTransition.size() > 0 && !this.master.isStopped()
2076           && remaining > 0) {
2077         regionsInTransition.wait(remaining);
2078         remaining = timeout - (System.currentTimeMillis() - startTime);
2079       }
2080     }
2081     return regionsInTransition.isEmpty();
2082   }
2083 
2084   /**
2085    * Wait until no regions from set regions are in transition.
2086    * @param timeout How long to wait.
2087    * @param regions set of regions to wait for
2088    * @return True if nothing in regions in transition.
2089    * @throws InterruptedException
2090    */
2091   boolean waitUntilNoRegionsInTransition(final long timeout, Set<HRegionInfo> regions)
2092   throws InterruptedException {
2093     // Blocks until there are no regions in transition.
2094     long startTime = System.currentTimeMillis();
2095     long remaining = timeout;
2096     boolean stillInTransition = true;
2097     synchronized (regionsInTransition) {
2098       while (regionsInTransition.size() > 0 && !this.master.isStopped() &&
2099           remaining > 0 && stillInTransition) {
2100         int count = 0;
2101         for (RegionState rs : regionsInTransition.values()) {
2102           if (regions.contains(rs.getRegion())) {
2103             count++;
2104             break;
2105           }
2106         }
2107         if (count == 0) {
2108           stillInTransition = false;
2109           break;
2110         }
2111         regionsInTransition.wait(remaining);
2112         remaining = timeout - (System.currentTimeMillis() - startTime);
2113       }
2114     }
2115     return stillInTransition;
2116   }
2117 
2118   /**
2119    * Rebuild the list of user regions and assignment information.
2120    * <p>
2121    * Returns a map of servers that are not found to be online and the regions
2122    * they were hosting.
2123    * @return map of servers not online to their assigned regions, as stored
2124    *         in META
2125    * @throws IOException
2126    */
2127   Map<ServerName, List<Pair<HRegionInfo, Result>>> rebuildUserRegions()
2128   throws IOException, KeeperException {
2129     // Region assignment from META
2130     List<Result> results = MetaReader.fullScan(this.catalogTracker);
2131     // Map of offline servers and their regions to be returned
2132     Map<ServerName, List<Pair<HRegionInfo,Result>>> offlineServers =
2133       new TreeMap<ServerName, List<Pair<HRegionInfo, Result>>>();
2134     // Iterate regions in META
2135     for (Result result : results) {
2136       Pair<HRegionInfo, ServerName> region = MetaReader.parseCatalogResult(result);
2137       if (region == null) continue;
2138       HRegionInfo regionInfo = region.getFirst();
2139       ServerName regionLocation = region.getSecond();
2140       String tableName = regionInfo.getTableNameAsString();
2141       if (regionLocation == null) {
2142         // regionLocation could be null if createTable didn't finish properly.
2143         // When createTable is in progress, HMaster restarts.
2144         // Some regions have been added to .META., but have not been assigned.
2145         // When this happens, the region's table must be in ENABLING state.
2146         // It can't be in ENABLED state as that is set when all regions are
2147         // assigned.
2148         // It can't be in DISABLING state, because DISABLING state transitions
2149         // from ENABLED state when application calls disableTable.
2150         // It can't be in DISABLED state, because DISABLED states transitions
2151         // from DISABLING state.
2152         if (false == checkIfRegionsBelongsToEnabling(regionInfo)) {
2153           LOG.warn("Region " + regionInfo.getEncodedName() +
2154             " has null regionLocation." + " But its table " + tableName +
2155             " isn't in ENABLING state.");
2156         }
2157         addTheTablesInPartialState(this.disablingTables, this.enablingTables, regionInfo,
2158             tableName);
2159       } else if (!this.serverManager.isServerOnline(regionLocation)) {
2160         // Region is located on a server that isn't online
2161         List<Pair<HRegionInfo, Result>> offlineRegions =
2162           offlineServers.get(regionLocation);
2163         if (offlineRegions == null) {
2164           offlineRegions = new ArrayList<Pair<HRegionInfo,Result>>(1);
2165           offlineServers.put(regionLocation, offlineRegions);
2166         }
2167         offlineRegions.add(new Pair<HRegionInfo,Result>(regionInfo, result));
2168       } else {
2169         // Region is being served and on an active server
2170         // add only if region not in disabled and enabling table
2171         if (false == checkIfRegionBelongsToDisabled(regionInfo)
2172             && false == checkIfRegionsBelongsToEnabling(regionInfo)) {
2173           regions.put(regionInfo, regionLocation);
2174           addToServers(regionLocation, regionInfo);
2175         }
2176         addTheTablesInPartialState(this.disablingTables, this.enablingTables, regionInfo,
2177             tableName);
2178       }
2179     }
2180     return offlineServers;
2181   }
2182 
2183   private void addTheTablesInPartialState(Set<String> disablingTables,
2184       Set<String> enablingTables, HRegionInfo regionInfo,
2185       String disablingTableName) {
2186     if (checkIfRegionBelongsToDisabling(regionInfo)) {
2187       disablingTables.add(disablingTableName);
2188     } else if (checkIfRegionsBelongsToEnabling(regionInfo)) {
2189       enablingTables.add(disablingTableName);
2190     }
2191   }
2192 
2193   /**
2194    * Recover the tables that were not fully moved to DISABLED state. These
2195    * tables are in DISABLING state when the master restarted/switched.
2196    * 
2197    * @param disablingTables
2198    * @return
2199    * @throws KeeperException
2200    * @throws TableNotFoundException
2201    * @throws IOException
2202    */
2203   private boolean recoverTableInDisablingState(Set<String> disablingTables)
2204       throws KeeperException, TableNotFoundException, IOException {
2205     boolean isWatcherCreated = false;
2206     if (disablingTables.size() != 0) {
2207       // Create a watcher on the zookeeper node
2208       ZKUtil.listChildrenAndWatchForNewChildren(watcher,
2209           watcher.assignmentZNode);
2210       isWatcherCreated = true;
2211       for (String tableName : disablingTables) {
2212         // Recover by calling DisableTableHandler
2213         LOG.info("The table " + tableName
2214             + " is in DISABLING state.  Hence recovering by moving the table"
2215             + " to DISABLED state.");
2216         new DisableTableHandler(this.master, tableName.getBytes(),
2217             catalogTracker, this, true).process();
2218       }
2219     }
2220     return isWatcherCreated;
2221   }
2222 
2223   /**
2224    * Recover the tables that are not fully moved to ENABLED state. These tables
2225    * are in ENABLING state when the master restarted/switched
2226    * 
2227    * @param enablingTables
2228    * @param isWatcherCreated
2229    * @throws KeeperException
2230    * @throws TableNotFoundException
2231    * @throws IOException
2232    */
2233   private void recoverTableInEnablingState(Set<String> enablingTables,
2234       boolean isWatcherCreated) throws KeeperException, TableNotFoundException,
2235       IOException {
2236     if (enablingTables.size() != 0) {
2237       if (false == isWatcherCreated) {
2238         ZKUtil.listChildrenAndWatchForNewChildren(watcher,
2239             watcher.assignmentZNode);
2240       }
2241       for (String tableName : enablingTables) {
2242         // Recover by calling EnableTableHandler
2243         LOG.info("The table " + tableName
2244             + " is in ENABLING state.  Hence recovering by moving the table"
2245             + " to ENABLED state.");
2246         // enableTable in sync way during master startup,
2247         // no need to invoke coprocessor
2248         new EnableTableHandler(this.master, tableName.getBytes(),
2249             catalogTracker, this, true).process();
2250       }
2251     }
2252   }
2253 
2254   private boolean checkIfRegionsBelongsToEnabling(HRegionInfo regionInfo) {
2255     String tableName = regionInfo.getTableNameAsString();
2256     return getZKTable().isEnablingTable(tableName);
2257   }
2258 
2259   private boolean checkIfRegionBelongsToDisabled(HRegionInfo regionInfo) {
2260     String tableName = regionInfo.getTableNameAsString();
2261     return getZKTable().isDisabledTable(tableName);
2262   }
2263 
2264   private boolean checkIfRegionBelongsToDisabling(HRegionInfo regionInfo) {
2265     String tableName = regionInfo.getTableNameAsString();
2266     return getZKTable().isDisablingTable(tableName);
2267   }
2268 
2269   /**
2270    * Processes list of dead servers from result of META scan and regions in RIT
2271    * <p>
2272    * This is used for failover to recover the lost regions that belonged to
2273    * RegionServers which failed while there was no active master or regions 
2274    * that were in RIT.
2275    * <p>
2276    * 
2277    * @param deadServers
2278    *          The list of dead servers which failed while there was no active
2279    *          master. Can be null.
2280    * @param nodes
2281    *          The regions in RIT
2282    * @throws IOException
2283    * @throws KeeperException
2284    */
2285   private void processDeadServersAndRecoverLostRegions(
2286       Map<ServerName, List<Pair<HRegionInfo, Result>>> deadServers,
2287       List<String> nodes) throws IOException, KeeperException {
2288     if (null != deadServers) {
2289       for (Map.Entry<ServerName, List<Pair<HRegionInfo, Result>>> deadServer : 
2290         deadServers.entrySet()) {
2291         List<Pair<HRegionInfo, Result>> regions = deadServer.getValue();
2292         for (Pair<HRegionInfo, Result> region : regions) {
2293           HRegionInfo regionInfo = region.getFirst();
2294           Result result = region.getSecond();
2295           // If region was in transition (was in zk) force it offline for
2296           // reassign
2297           try {
2298             RegionTransitionData data = ZKAssign.getData(watcher,
2299                 regionInfo.getEncodedName());
2300 
2301             // If zk node of this region has been updated by a live server,
2302             // we consider that this region is being handled.
2303             // So we should skip it and process it in
2304             // processRegionsInTransition.
2305             if (data != null && data.getOrigin() != null && 
2306                 serverManager.isServerOnline(data.getOrigin())) {
2307               LOG.info("The region " + regionInfo.getEncodedName()
2308                   + "is being handled on " + data.getOrigin());
2309               continue;
2310             }
2311             // Process with existing RS shutdown code
2312             boolean assign = ServerShutdownHandler.processDeadRegion(
2313                 regionInfo, result, this, this.catalogTracker);
2314             if (assign) {
2315               ZKAssign.createOrForceNodeOffline(watcher, regionInfo,
2316                   master.getServerName());
2317               if (!nodes.contains(regionInfo.getEncodedName())) {
2318                 nodes.add(regionInfo.getEncodedName());
2319               }
2320             }
2321           } catch (KeeperException.NoNodeException nne) {
2322             // This is fine
2323           }
2324         }
2325       }
2326     }
2327 
2328     if (!nodes.isEmpty()) {
2329       for (String encodedRegionName : nodes) {
2330         processRegionInTransition(encodedRegionName, null, deadServers);
2331       }
2332     }
2333   }
2334 
2335   /*
2336    * Presumes caller has taken care of necessary locking modifying servers Map.
2337    * @param hsi
2338    * @param hri
2339    */
2340   private void addToServers(final ServerName sn, final HRegionInfo hri) {
2341     Set<HRegionInfo> hris = servers.get(sn);
2342     if (hris == null) {
2343       hris = new ConcurrentSkipListSet<HRegionInfo>();
2344       servers.put(sn, hris);
2345     }
2346     if (!hris.contains(hri)) hris.add(hri);
2347   }
2348 
2349   /**
2350    * @return A copy of the Map of regions currently in transition.
2351    */
2352   public NavigableMap<String, RegionState> getRegionsInTransition() {
2353     synchronized (this.regionsInTransition) {
2354       return new TreeMap<String, RegionState>(this.regionsInTransition);
2355     }
2356   }
2357 
2358   /**
2359    * @return True if regions in transition.
2360    */
2361   public boolean isRegionsInTransition() {
2362     synchronized (this.regionsInTransition) {
2363       return !this.regionsInTransition.isEmpty();
2364     }
2365   }
2366 
2367   /**
2368    * @param hri Region to check.
2369    * @return Returns null if passed region is not in transition else the current
2370    * RegionState
2371    */
2372   public RegionState isRegionInTransition(final HRegionInfo hri) {
2373     synchronized (this.regionsInTransition) {
2374       return this.regionsInTransition.get(hri.getEncodedName());
2375     }
2376   }
2377 
2378   /**
2379    * Clears the specified region from being in transition.
2380    * @param hri Region to remove.
2381    */
2382   public void clearRegionFromTransition(HRegionInfo hri) {
2383     synchronized (this.regionsInTransition) {
2384       this.regionsInTransition.remove(hri.getEncodedName());
2385     }
2386     synchronized (this.regions) {
2387       this.regions.remove(hri);
2388       for (Set<HRegionInfo> regions : this.servers.values()) {
2389         regions.remove(hri);
2390       }
2391     }
2392     clearRegionPlan(hri);
2393   }
2394 
2395   /**
2396    * @param region Region whose plan we are to clear.
2397    */
2398   void clearRegionPlan(final HRegionInfo region) {
2399     synchronized (this.regionPlans) {
2400       this.regionPlans.remove(region.getEncodedName());
2401     }
2402   }
2403 
2404   /**
2405    * Wait on region to clear regions-in-transition.
2406    * @param hri Region to wait on.
2407    * @throws IOException
2408    */
2409   public void waitOnRegionToClearRegionsInTransition(final HRegionInfo hri)
2410   throws IOException {
2411     if (isRegionInTransition(hri) == null) return;
2412     RegionState rs = null;
2413     // There is already a timeout monitor on regions in transition so I
2414     // should not have to have one here too?
2415     while(!this.master.isStopped() && (rs = isRegionInTransition(hri)) != null) {
2416       Threads.sleep(1000);
2417       LOG.info("Waiting on " + rs + " to clear regions-in-transition");
2418     }
2419     if (this.master.isStopped()) {
2420       LOG.info("Giving up wait on regions in " +
2421         "transition because stoppable.isStopped is set");
2422     }
2423   }
2424 
2425 
2426   /**
2427    * Gets the online regions of the specified table.
2428    * This method looks at the in-memory state.  It does not go to <code>.META.</code>.
2429    * Only returns <em>online</em> regions.  If a region on this table has been
2430    * closed during a disable, etc., it will be included in the returned list.
2431    * So, the returned list may not necessarily be ALL regions in this table, its
2432    * all the ONLINE regions in the table.
2433    * @param tableName
2434    * @return Online regions from <code>tableName</code>
2435    */
2436   public List<HRegionInfo> getRegionsOfTable(byte[] tableName) {
2437     List<HRegionInfo> tableRegions = new ArrayList<HRegionInfo>();
2438     HRegionInfo boundary =
2439       new HRegionInfo(tableName, null, null);
2440     synchronized (this.regions) {
2441       for (HRegionInfo regionInfo: this.regions.tailMap(boundary).keySet()) {
2442         if(Bytes.equals(regionInfo.getTableName(), tableName)) {
2443           tableRegions.add(regionInfo);
2444         } else {
2445           break;
2446         }
2447       }
2448     }
2449     return tableRegions;
2450   }
2451 
2452   /**
2453    * Monitor to check for time outs on region transition operations
2454    */
2455   public class TimeoutMonitor extends Chore {
2456     private final int timeout;
2457     private boolean bulkAssign = false;
2458 
2459     /**
2460      * Creates a periodic monitor to check for time outs on region transition
2461      * operations.  This will deal with retries if for some reason something
2462      * doesn't happen within the specified timeout.
2463      * @param period
2464    * @param stopper When {@link Stoppable#isStopped()} is true, this thread will
2465    * cleanup and exit cleanly.
2466      * @param timeout
2467      */
2468     public TimeoutMonitor(final int period, final Stoppable stopper,
2469         final int timeout) {
2470       super("AssignmentTimeoutMonitor", period, stopper);
2471       this.timeout = timeout;
2472     }
2473 
2474     /**
2475      * @param bulkAssign If true, we'll suspend checking regions in transition
2476      * up in zookeeper.  If false, will reenable check.
2477      * @return Old setting for bulkAssign.
2478      */
2479     public boolean bulkAssign(final boolean bulkAssign) {
2480       boolean result = this.bulkAssign;
2481       this.bulkAssign = bulkAssign;
2482       return result;
2483     }
2484 
2485     @Override
2486     protected void chore() {
2487       // If bulkAssign in progress, suspend checks
2488       if (this.bulkAssign) return;
2489       synchronized (regionsInTransition) {
2490         // Iterate all regions in transition checking for time outs
2491         long now = System.currentTimeMillis();
2492         for (RegionState regionState : regionsInTransition.values()) {
2493           if (regionState.getStamp() + timeout <= now) {
2494            //decide on action upon timeout
2495             actOnTimeOut(regionState);
2496           }
2497         }
2498       }
2499     }
2500 
2501     private void actOnTimeOut(RegionState regionState) {
2502       HRegionInfo regionInfo = regionState.getRegion();
2503       LOG.info("Regions in transition timed out:  " + regionState);
2504       // Expired! Do a retry.
2505       switch (regionState.getState()) {
2506       case CLOSED:
2507         LOG.info("Region " + regionInfo.getEncodedName()
2508             + " has been CLOSED for too long, waiting on queued "
2509             + "ClosedRegionHandler to run or server shutdown");
2510         // Update our timestamp.
2511         regionState.updateTimestampToNow();
2512         break;
2513       case OFFLINE:
2514         LOG.info("Region has been OFFLINE for too long, " + "reassigning "
2515             + regionInfo.getRegionNameAsString() + " to a random server");
2516         invokeAssign(regionInfo);
2517         break;
2518       case PENDING_OPEN:
2519         LOG.info("Region has been PENDING_OPEN for too "
2520             + "long, reassigning region=" + regionInfo.getRegionNameAsString());
2521         invokeAssign(regionInfo);
2522         break;
2523       case OPENING:
2524         processOpeningState(regionInfo);
2525         break;
2526       case OPEN:
2527         LOG.error("Region has been OPEN for too long, " +
2528             "we don't know where region was opened so can't do anything");
2529         synchronized (regionState) {
2530           regionState.updateTimestampToNow();
2531         }
2532         break;
2533 
2534       case PENDING_CLOSE:
2535         LOG.info("Region has been PENDING_CLOSE for too "
2536             + "long, running forced unassign again on region="
2537             + regionInfo.getRegionNameAsString());
2538         invokeUnassign(regionInfo);
2539         break;
2540       case CLOSING:
2541         LOG.info("Region has been CLOSING for too " +
2542           "long, this should eventually complete or the server will " +
2543           "expire, send RPC again");
2544         invokeUnassign(regionInfo);
2545         break;
2546       }
2547     }
2548   }
2549   
2550   private void processOpeningState(HRegionInfo regionInfo) {
2551     LOG.info("Region has been OPENING for too " + "long, reassigning region="
2552         + regionInfo.getRegionNameAsString());
2553     // Should have a ZK node in OPENING state
2554     try {
2555       String node = ZKAssign.getNodeName(watcher, regionInfo.getEncodedName());
2556       Stat stat = new Stat();
2557       RegionTransitionData dataInZNode = ZKAssign.getDataNoWatch(watcher, node,
2558           stat);
2559       if (dataInZNode == null) {
2560         LOG.warn("Data is null, node " + node + " no longer exists");
2561         return;
2562       }
2563       if (dataInZNode.getEventType() == EventType.RS_ZK_REGION_OPENED) {
2564         LOG.debug("Region has transitioned to OPENED, allowing "
2565             + "watched event handlers to process");
2566         return;
2567       } else if (dataInZNode.getEventType() != EventType.RS_ZK_REGION_OPENING &&
2568           dataInZNode.getEventType() != EventType.RS_ZK_REGION_FAILED_OPEN ) {
2569         LOG.warn("While timing out a region in state OPENING, "
2570             + "found ZK node in unexpected state: "
2571             + dataInZNode.getEventType());
2572         return;
2573       }
2574       invokeAssign(regionInfo);
2575     } catch (KeeperException ke) {
2576       LOG.error("Unexpected ZK exception timing out CLOSING region", ke);
2577       return;
2578     }
2579     return;
2580   }
2581 
2582   private void invokeAssign(HRegionInfo regionInfo) {
2583     threadPoolExecutorService.submit(new AssignCallable(this, regionInfo));
2584   }
2585 
2586   private void invokeUnassign(HRegionInfo regionInfo) {
2587     threadPoolExecutorService.submit(new UnAssignCallable(this, regionInfo));
2588   }
2589 
2590   public boolean isCarryingRoot(ServerName serverName) {
2591     return isCarryingRegion(serverName, HRegionInfo.ROOT_REGIONINFO);
2592   }
2593 
2594   public boolean isCarryingMeta(ServerName serverName) {
2595     return isCarryingRegion(serverName, HRegionInfo.FIRST_META_REGIONINFO);
2596   }
2597   /**
2598    * Check if the shutdown server carries the specific region.
2599    * We have a bunch of places that store region location
2600    * Those values aren't consistent. There is a delay of notification.
2601    * The location from zookeeper unassigned node has the most recent data;
2602    * but the node could be deleted after the region is opened by AM.
2603    * The AM's info could be old when OpenedRegionHandler
2604    * processing hasn't finished yet when server shutdown occurs.
2605    * @return whether the serverName currently hosts the region
2606    */
2607   public boolean isCarryingRegion(ServerName serverName, HRegionInfo hri) {
2608     RegionTransitionData data = null;
2609     try {
2610       data = ZKAssign.getData(master.getZooKeeper(), hri.getEncodedName());
2611     } catch (KeeperException e) {
2612       master.abort("Unexpected ZK exception reading unassigned node for region="
2613         + hri.getEncodedName(), e);
2614     }
2615 
2616     ServerName addressFromZK = (data != null && data.getOrigin() != null) ?
2617       data.getOrigin() : null;
2618     if (addressFromZK != null) {
2619       // if we get something from ZK, we will use the data
2620       boolean matchZK = (addressFromZK != null &&
2621         addressFromZK.equals(serverName));
2622       LOG.debug("based on ZK, current region=" + hri.getRegionNameAsString() +
2623           " is on server=" + addressFromZK +
2624           " server being checked=: " + serverName);
2625       return matchZK;
2626     }
2627 
2628     ServerName addressFromAM = getRegionServerOfRegion(hri);
2629     boolean matchAM = (addressFromAM != null &&
2630       addressFromAM.equals(serverName));
2631     LOG.debug("based on AM, current region=" + hri.getRegionNameAsString() +
2632       " is on server=" + (addressFromAM != null ? addressFromAM : "null") +
2633       " server being checked: " + serverName);
2634 
2635     return matchAM;
2636   }
2637   /**
2638    * Process shutdown server removing any assignments.
2639    * @param sn Server that went down.
2640    * @return list of regions in transition on this server
2641    */
2642   public List<RegionState> processServerShutdown(final ServerName sn) {
2643     // Clean out any existing assignment plans for this server
2644     synchronized (this.regionPlans) {
2645       for (Iterator <Map.Entry<String, RegionPlan>> i =
2646           this.regionPlans.entrySet().iterator(); i.hasNext();) {
2647         Map.Entry<String, RegionPlan> e = i.next();
2648         ServerName otherSn = e.getValue().getDestination();
2649         // The name will be null if the region is planned for a random assign.
2650         if (otherSn != null && otherSn.equals(sn)) {
2651           // Use iterator's remove else we'll get CME
2652           i.remove();
2653         }
2654       }
2655     }
2656     // TODO: Do we want to sync on RIT here?
2657     // Remove this server from map of servers to regions, and remove all regions
2658     // of this server from online map of regions.
2659     Set<HRegionInfo> deadRegions = null;
2660     List<RegionState> rits = new ArrayList<RegionState>();
2661     synchronized (this.regions) {
2662       Set<HRegionInfo> assignedRegions = this.servers.remove(sn);
2663       if (assignedRegions == null || assignedRegions.isEmpty()) {
2664         // No regions on this server, we are done, return empty list of RITs
2665         return rits;
2666       }
2667       deadRegions = new TreeSet<HRegionInfo>(assignedRegions);
2668       for (HRegionInfo region : deadRegions) {
2669         this.regions.remove(region);
2670       }
2671     }
2672     // See if any of the regions that were online on this server were in RIT
2673     // If they are, normal timeouts will deal with them appropriately so
2674     // let's skip a manual re-assignment.
2675     synchronized (regionsInTransition) {
2676       for (RegionState region : this.regionsInTransition.values()) {
2677         if (deadRegions.remove(region.getRegion())) {
2678           rits.add(region);
2679         }
2680       }
2681     }
2682     return rits;
2683   }
2684 
2685   /**
2686    * Update inmemory structures.
2687    * @param sn Server that reported the split
2688    * @param parent Parent region that was split
2689    * @param a Daughter region A
2690    * @param b Daughter region B
2691    */
2692   public void handleSplitReport(final ServerName sn, final HRegionInfo parent,
2693       final HRegionInfo a, final HRegionInfo b) {
2694     regionOffline(parent);
2695     regionOnline(a, sn);
2696     regionOnline(b, sn);
2697 
2698     // There's a possibility that the region was splitting while a user asked
2699     // the master to disable, we need to make sure we close those regions in
2700     // that case. This is not racing with the region server itself since RS
2701     // report is done after the split transaction completed.
2702     if (this.zkTable.isDisablingOrDisabledTable(
2703         parent.getTableNameAsString())) {
2704       unassign(a);
2705       unassign(b);
2706     }
2707   }
2708 
2709   /**
2710    * @return A clone of current assignments. Note, this is assignments only.
2711    * If a new server has come in and it has no regions, it will not be included
2712    * in the returned Map.
2713    */
2714   Map<ServerName, List<HRegionInfo>> getAssignments() {
2715     // This is an EXPENSIVE clone.  Cloning though is the safest thing to do.
2716     // Can't let out original since it can change and at least the loadbalancer
2717     // wants to iterate this exported list.  We need to synchronize on regions
2718     // since all access to this.servers is under a lock on this.regions.
2719     Map<ServerName, List<HRegionInfo>> result = null;
2720     synchronized (this.regions) {
2721       result = new HashMap<ServerName, List<HRegionInfo>>(this.servers.size());
2722       for (Map.Entry<ServerName, Set<HRegionInfo>> e: this.servers.entrySet()) {
2723         result.put(e.getKey(), new ArrayList<HRegionInfo>(e.getValue()));
2724       }
2725     }
2726     return result;
2727   }
2728 
2729   /**
2730    * @param encodedRegionName Region encoded name.
2731    * @return Null or a {@link Pair} instance that holds the full {@link HRegionInfo}
2732    * and the hosting servers {@link ServerName}.
2733    */
2734   Pair<HRegionInfo, ServerName> getAssignment(final byte [] encodedRegionName) {
2735     String name = Bytes.toString(encodedRegionName);
2736     synchronized(this.regions) {
2737       for (Map.Entry<HRegionInfo, ServerName> e: this.regions.entrySet()) {
2738         if (e.getKey().getEncodedName().equals(name)) {
2739           return new Pair<HRegionInfo, ServerName>(e.getKey(), e.getValue());
2740         }
2741       }
2742     }
2743     return null;
2744   }
2745 
2746   /**
2747    * @param plan Plan to execute.
2748    */
2749   void balance(final RegionPlan plan) {
2750     synchronized (this.regionPlans) {
2751       this.regionPlans.put(plan.getRegionName(), plan);
2752     }
2753     unassign(plan.getRegionInfo());
2754   }
2755 
2756   /**
2757    * Run through remaining regionservers and unassign all catalog regions.
2758    */
2759   void unassignCatalogRegions() {
2760     this.servers.entrySet();
2761     synchronized (this.regions) {
2762       for (Map.Entry<ServerName, Set<HRegionInfo>> e: this.servers.entrySet()) {
2763         Set<HRegionInfo> regions = e.getValue();
2764         if (regions == null || regions.isEmpty()) continue;
2765         for (HRegionInfo hri: regions) {
2766           if (hri.isMetaRegion()) {
2767             unassign(hri);
2768           }
2769         }
2770       }
2771     }
2772   }
2773 
2774   /**
2775    * State of a Region while undergoing transitions.
2776    */
2777   public static class RegionState implements org.apache.hadoop.io.Writable {
2778     private HRegionInfo region;
2779 
2780     public enum State {
2781       OFFLINE,        // region is in an offline state
2782       PENDING_OPEN,   // sent rpc to server to open but has not begun
2783       OPENING,        // server has begun to open but not yet done
2784       OPEN,           // server opened region and updated meta
2785       PENDING_CLOSE,  // sent rpc to server to close but has not begun
2786       CLOSING,        // server has begun to close but not yet done
2787       CLOSED,         // server closed region and updated meta
2788       SPLITTING,      // server started split of a region
2789       SPLIT           // server completed split of a region
2790     }
2791 
2792     private State state;
2793     // Many threads can update the state at the stamp at the same time
2794     private final AtomicLong stamp;
2795     private ServerName serverName;
2796 
2797     public RegionState() {
2798       this.stamp = new AtomicLong(System.currentTimeMillis());
2799     }
2800 
2801     RegionState(HRegionInfo region, State state) {
2802       this(region, state, System.currentTimeMillis(), null);
2803     }
2804 
2805     RegionState(HRegionInfo region, State state, long stamp, ServerName serverName) {
2806       this.region = region;
2807       this.state = state;
2808       this.stamp = new AtomicLong(stamp);
2809       this.serverName = serverName;
2810     }
2811 
2812     public void update(State state, long stamp, ServerName serverName) {
2813       this.state = state;
2814       updateTimestamp(stamp);
2815       this.serverName = serverName;
2816     }
2817 
2818     public void update(State state) {
2819       this.state = state;
2820       updateTimestampToNow();
2821       this.serverName = null;
2822     }
2823 
2824     public void updateTimestamp(long stamp) {
2825       this.stamp.set(stamp);
2826     }
2827 
2828     public void updateTimestampToNow() {
2829       this.stamp.set(System.currentTimeMillis());
2830     }
2831 
2832     public State getState() {
2833       return state;
2834     }
2835 
2836     public long getStamp() {
2837       return stamp.get();
2838     }
2839 
2840     public HRegionInfo getRegion() {
2841       return region;
2842     }
2843 
2844     public boolean isClosing() {
2845       return state == State.CLOSING;
2846     }
2847 
2848     public boolean isClosed() {
2849       return state == State.CLOSED;
2850     }
2851 
2852     public boolean isPendingClose() {
2853       return state == State.PENDING_CLOSE;
2854     }
2855 
2856     public boolean isOpening() {
2857       return state == State.OPENING;
2858     }
2859 
2860     public boolean isOpened() {
2861       return state == State.OPEN;
2862     }
2863 
2864     public boolean isPendingOpen() {
2865       return state == State.PENDING_OPEN;
2866     }
2867 
2868     public boolean isOffline() {
2869       return state == State.OFFLINE;
2870     }
2871 
2872     public boolean isSplitting() {
2873       return state == State.SPLITTING;
2874     }
2875  
2876     public boolean isSplit() {
2877       return state == State.SPLIT;
2878     }
2879 
2880     @Override
2881     public String toString() {
2882       return region.getRegionNameAsString()
2883         + " state=" + state
2884         + ", ts=" + stamp
2885         + ", server=" + serverName;
2886     }
2887 
2888     /**
2889      * A slower (but more easy-to-read) stringification 
2890      */
2891     public String toDescriptiveString() {
2892       long lstamp = stamp.get();
2893       long relTime = System.currentTimeMillis() - lstamp;
2894       
2895       return region.getRegionNameAsString()
2896         + " state=" + state
2897         + ", ts=" + new Date(lstamp) + " (" + (relTime/1000) + "s ago)"
2898         + ", server=" + serverName;
2899     }
2900 
2901     @Override
2902     public void readFields(DataInput in) throws IOException {
2903       region = new HRegionInfo();
2904       region.readFields(in);
2905       state = State.valueOf(in.readUTF());
2906       stamp.set(in.readLong());
2907     }
2908 
2909     @Override
2910     public void write(DataOutput out) throws IOException {
2911       region.write(out);
2912       out.writeUTF(state.name());
2913       out.writeLong(stamp.get());
2914     }
2915   }
2916 
2917   public void stop() {
2918     this.timeoutMonitor.interrupt();
2919   }
2920   
2921   /**
2922    * Check whether the RegionServer is online.
2923    */
2924   public boolean isServerOnline(ServerName serverName) {
2925     return this.serverManager.isServerOnline(serverName);
2926   }
2927   /**
2928    * Shutdown the threadpool executor service
2929    */
2930   public void shutdown() {
2931     if (null != threadPoolExecutorService) {
2932       this.threadPoolExecutorService.shutdown();
2933     }
2934   }
2935 }