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