View Javadoc

1   /**
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  package org.apache.hadoop.hbase.master;
20  
21  import java.io.IOException;
22  import java.io.InterruptedIOException;
23  import java.util.ArrayList;
24  import java.util.Arrays;
25  import java.util.Collection;
26  import java.util.Collections;
27  import java.util.HashMap;
28  import java.util.HashSet;
29  import java.util.Iterator;
30  import java.util.List;
31  import java.util.Map;
32  import java.util.NavigableMap;
33  import java.util.Random;
34  import java.util.Set;
35  import java.util.TreeMap;
36  import java.util.concurrent.Callable;
37  import java.util.concurrent.ConcurrentHashMap;
38  import java.util.concurrent.CopyOnWriteArrayList;
39  import java.util.concurrent.ThreadFactory;
40  import java.util.concurrent.TimeUnit;
41  import java.util.concurrent.atomic.AtomicBoolean;
42  import java.util.concurrent.atomic.AtomicInteger;
43  import java.util.concurrent.locks.Lock;
44  import java.util.concurrent.locks.ReentrantLock;
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.fs.FileSystem;
50  import org.apache.hadoop.fs.Path;
51  import org.apache.hadoop.hbase.CoordinatedStateException;
52  import org.apache.hadoop.hbase.HBaseIOException;
53  import org.apache.hadoop.hbase.HConstants;
54  import org.apache.hadoop.hbase.HRegionInfo;
55  import org.apache.hadoop.hbase.HRegionLocation;
56  import org.apache.hadoop.hbase.HTableDescriptor;
57  import org.apache.hadoop.hbase.MetaTableAccessor;
58  import org.apache.hadoop.hbase.NotServingRegionException;
59  import org.apache.hadoop.hbase.RegionLocations;
60  import org.apache.hadoop.hbase.RegionStateListener;
61  import org.apache.hadoop.hbase.RegionTransition;
62  import org.apache.hadoop.hbase.ServerName;
63  import org.apache.hadoop.hbase.TableName;
64  import org.apache.hadoop.hbase.TableNotFoundException;
65  import org.apache.hadoop.hbase.TableStateManager;
66  import org.apache.hadoop.hbase.classification.InterfaceAudience;
67  import org.apache.hadoop.hbase.client.RegionReplicaUtil;
68  import org.apache.hadoop.hbase.client.Result;
69  import org.apache.hadoop.hbase.coordination.BaseCoordinatedStateManager;
70  import org.apache.hadoop.hbase.coordination.OpenRegionCoordination;
71  import org.apache.hadoop.hbase.coordination.RegionMergeCoordination;
72  import org.apache.hadoop.hbase.coordination.SplitTransactionCoordination.SplitTransactionDetails;
73  import org.apache.hadoop.hbase.coordination.ZkOpenRegionCoordination;
74  import org.apache.hadoop.hbase.coordination.ZkRegionMergeCoordination;
75  import org.apache.hadoop.hbase.exceptions.DeserializationException;
76  import org.apache.hadoop.hbase.executor.EventHandler;
77  import org.apache.hadoop.hbase.executor.EventType;
78  import org.apache.hadoop.hbase.executor.ExecutorService;
79  import org.apache.hadoop.hbase.ipc.FailedServerException;
80  import org.apache.hadoop.hbase.ipc.RpcClient;
81  import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
82  import org.apache.hadoop.hbase.master.RegionState.State;
83  import org.apache.hadoop.hbase.master.balancer.FavoredNodeAssignmentHelper;
84  import org.apache.hadoop.hbase.master.balancer.FavoredNodeLoadBalancer;
85  import org.apache.hadoop.hbase.master.handler.ClosedRegionHandler;
86  import org.apache.hadoop.hbase.master.handler.DisableTableHandler;
87  import org.apache.hadoop.hbase.master.handler.EnableTableHandler;
88  import org.apache.hadoop.hbase.master.handler.OpenedRegionHandler;
89  import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionStateTransition;
90  import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
91  import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
92  import org.apache.hadoop.hbase.regionserver.RegionAlreadyInTransitionException;
93  import org.apache.hadoop.hbase.regionserver.RegionOpeningState;
94  import org.apache.hadoop.hbase.regionserver.RegionServerAbortedException;
95  import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
96  import org.apache.hadoop.hbase.util.ConfigUtil;
97  import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
98  import org.apache.hadoop.hbase.util.FSUtils;
99  import org.apache.hadoop.hbase.util.KeyLocker;
100 import org.apache.hadoop.hbase.util.Pair;
101 import org.apache.hadoop.hbase.util.PairOfSameType;
102 import org.apache.hadoop.hbase.util.Threads;
103 import org.apache.hadoop.hbase.util.Triple;
104 import org.apache.hadoop.hbase.wal.DefaultWALProvider;
105 import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
106 import org.apache.hadoop.hbase.zookeeper.ZKAssign;
107 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
108 import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
109 import org.apache.hadoop.ipc.RemoteException;
110 import org.apache.hadoop.util.StringUtils;
111 import org.apache.zookeeper.AsyncCallback;
112 import org.apache.zookeeper.KeeperException;
113 import org.apache.zookeeper.KeeperException.NoNodeException;
114 import org.apache.zookeeper.KeeperException.NodeExistsException;
115 import org.apache.zookeeper.data.Stat;
116 
117 import com.google.common.annotations.VisibleForTesting;
118 import com.google.common.collect.LinkedHashMultimap;
119 
120 /**
121  * Manages and performs region assignment.
122  * <p>
123  * Monitors ZooKeeper for events related to regions in transition.
124  * <p>
125  * Handles existing regions in transition during master failover.
126  */
127 @InterfaceAudience.Private
128 public class AssignmentManager extends ZooKeeperListener {
129   private static final Log LOG = LogFactory.getLog(AssignmentManager.class);
130 
131   public static final ServerName HBCK_CODE_SERVERNAME = ServerName.valueOf(HConstants.HBCK_CODE_NAME,
132       -1, -1L);
133 
134   static final String ALREADY_IN_TRANSITION_WAITTIME
135     = "hbase.assignment.already.intransition.waittime";
136   static final int DEFAULT_ALREADY_IN_TRANSITION_WAITTIME = 60000; // 1 minute
137 
138   protected final MasterServices server;
139 
140   private ServerManager serverManager;
141 
142   private boolean shouldAssignRegionsWithFavoredNodes;
143 
144   private LoadBalancer balancer;
145 
146   private final MetricsAssignmentManager metricsAssignmentManager;
147 
148   private final TableLockManager tableLockManager;
149 
150   private AtomicInteger numRegionsOpened = new AtomicInteger(0);
151 
152   final private KeyLocker<String> locker = new KeyLocker<String>();
153 
154   Set<HRegionInfo> replicasToClose = Collections.synchronizedSet(new HashSet<HRegionInfo>());
155 
156   /**
157    * Map of regions to reopen after the schema of a table is changed. Key -
158    * encoded region name, value - HRegionInfo
159    */
160   private final Map <String, HRegionInfo> regionsToReopen;
161 
162   /*
163    * Maximum times we recurse an assignment/unassignment.
164    * See below in {@link #assign()} and {@link #unassign()}.
165    */
166   private final int maximumAttempts;
167 
168   /**
169    * Map of two merging regions from the region to be created.
170    */
171   private final Map<String, PairOfSameType<HRegionInfo>> mergingRegions
172     = new HashMap<String, PairOfSameType<HRegionInfo>>();
173 
174   private final Map<HRegionInfo, PairOfSameType<HRegionInfo>> splitRegions
175   = new HashMap<HRegionInfo, PairOfSameType<HRegionInfo>>();
176 
177   /**
178    * The sleep time for which the assignment will wait before retrying in case of hbase:meta assignment
179    * failure due to lack of availability of region plan or bad region plan
180    */
181   private final long sleepTimeBeforeRetryingMetaAssignment;
182 
183   /** Plans for region movement. Key is the encoded version of a region name*/
184   // TODO: When do plans get cleaned out?  Ever? In server open and in server
185   // shutdown processing -- St.Ack
186   // All access to this Map must be synchronized.
187   final NavigableMap<String, RegionPlan> regionPlans =
188     new TreeMap<String, RegionPlan>();
189 
190   private final TableStateManager tableStateManager;
191 
192   private final ExecutorService executorService;
193 
194   // For unit tests, keep track of calls to ClosedRegionHandler
195   private Map<HRegionInfo, AtomicBoolean> closedRegionHandlerCalled = null;
196 
197   // For unit tests, keep track of calls to OpenedRegionHandler
198   private Map<HRegionInfo, AtomicBoolean> openedRegionHandlerCalled = null;
199 
200   //Thread pool executor service for timeout monitor
201   private java.util.concurrent.ExecutorService threadPoolExecutorService;
202 
203   // A bunch of ZK events workers. Each is a single thread executor service
204   private final java.util.concurrent.ExecutorService zkEventWorkers;
205 
206   private List<EventType> ignoreStatesRSOffline = Arrays.asList(
207       EventType.RS_ZK_REGION_FAILED_OPEN, EventType.RS_ZK_REGION_CLOSED);
208 
209   private final RegionStates regionStates;
210 
211   // The threshold to use bulk assigning. Using bulk assignment
212   // only if assigning at least this many regions to at least this
213   // many servers. If assigning fewer regions to fewer servers,
214   // bulk assigning may be not as efficient.
215   private final int bulkAssignThresholdRegions;
216   private final int bulkAssignThresholdServers;
217   private final int bulkPerRegionOpenTimeGuesstimate;
218 
219   // Should bulk assignment wait till all regions are assigned,
220   // or it is timed out?  This is useful to measure bulk assignment
221   // performance, but not needed in most use cases.
222   private final boolean bulkAssignWaitTillAllAssigned;
223 
224   /**
225    * Indicator that AssignmentManager has recovered the region states so
226    * that ServerShutdownHandler can be fully enabled and re-assign regions
227    * of dead servers. So that when re-assignment happens, AssignmentManager
228    * has proper region states.
229    *
230    * Protected to ease testing.
231    */
232   protected final AtomicBoolean failoverCleanupDone = new AtomicBoolean(false);
233 
234   /**
235    * A map to track the count a region fails to open in a row.
236    * So that we don't try to open a region forever if the failure is
237    * unrecoverable.  We don't put this information in region states
238    * because we don't expect this to happen frequently; we don't
239    * want to copy this information over during each state transition either.
240    */
241   private final ConcurrentHashMap<String, AtomicInteger>
242     failedOpenTracker = new ConcurrentHashMap<String, AtomicInteger>();
243 
244   // A flag to indicate if we are using ZK for region assignment
245   private final boolean useZKForAssignment;
246 
247   // In case not using ZK for region assignment, region states
248   // are persisted in meta with a state store
249   private final RegionStateStore regionStateStore;
250 
251   /**
252    * For testing only!  Set to true to skip handling of split.
253    */
254   @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="MS_SHOULD_BE_FINAL")
255   public static boolean TEST_SKIP_SPLIT_HANDLING = false;
256 
257   /** Listeners that are called on assignment events. */
258   private List<AssignmentListener> listeners = new CopyOnWriteArrayList<AssignmentListener>();
259 
260   private RegionStateListener regionStateListener;
261 
262   public enum ServerHostRegion {
263     NOT_HOSTING_REGION, HOSTING_REGION, UNKNOWN,
264   }
265 
266   /**
267    * Constructs a new assignment manager.
268    *
269    * @param server instance of HMaster this AM running inside
270    * @param serverManager serverManager for associated HMaster
271    * @param balancer implementation of {@link LoadBalancer}
272    * @param service Executor service
273    * @param metricsMaster metrics manager
274    * @param tableLockManager TableLock manager
275    * @throws KeeperException
276    * @throws IOException
277    */
278   public AssignmentManager(MasterServices server, ServerManager serverManager,
279       final LoadBalancer balancer,
280       final ExecutorService service, MetricsMaster metricsMaster,
281       final TableLockManager tableLockManager) throws KeeperException,
282         IOException, CoordinatedStateException {
283     super(server.getZooKeeper());
284     this.server = server;
285     this.serverManager = serverManager;
286     this.executorService = service;
287     this.regionStateStore = new RegionStateStore(server);
288     this.regionsToReopen = Collections.synchronizedMap
289                            (new HashMap<String, HRegionInfo> ());
290     Configuration conf = server.getConfiguration();
291     // Only read favored nodes if using the favored nodes load balancer.
292     this.shouldAssignRegionsWithFavoredNodes = conf.getClass(
293            HConstants.HBASE_MASTER_LOADBALANCER_CLASS, Object.class).equals(
294            FavoredNodeLoadBalancer.class);
295     try {
296       if (server.getCoordinatedStateManager() != null) {
297         this.tableStateManager = server.getCoordinatedStateManager().getTableStateManager();
298       } else {
299         this.tableStateManager = null;
300       }
301     } catch (InterruptedException e) {
302       throw new InterruptedIOException();
303     }
304     // This is the max attempts, not retries, so it should be at least 1.
305     this.maximumAttempts = Math.max(1,
306       this.server.getConfiguration().getInt("hbase.assignment.maximum.attempts", 10));
307     this.sleepTimeBeforeRetryingMetaAssignment = this.server.getConfiguration().getLong(
308         "hbase.meta.assignment.retry.sleeptime", 1000l);
309     this.balancer = balancer;
310     int maxThreads = conf.getInt("hbase.assignment.threads.max", 30);
311     this.threadPoolExecutorService = Threads.getBoundedCachedThreadPool(
312       maxThreads, 60L, TimeUnit.SECONDS, Threads.newDaemonThreadFactory("AM."));
313     this.regionStates = new RegionStates(
314       server, tableStateManager, serverManager, regionStateStore);
315 
316     this.bulkAssignWaitTillAllAssigned =
317       conf.getBoolean("hbase.bulk.assignment.waittillallassigned", false);
318     this.bulkAssignThresholdRegions = conf.getInt("hbase.bulk.assignment.threshold.regions", 7);
319     this.bulkAssignThresholdServers = conf.getInt("hbase.bulk.assignment.threshold.servers", 3);
320     this.bulkPerRegionOpenTimeGuesstimate =
321       conf.getInt("hbase.bulk.assignment.perregion.open.time", 10000);
322 
323     int workers = conf.getInt("hbase.assignment.zkevent.workers", 20);
324     ThreadFactory threadFactory = Threads.newDaemonThreadFactory("AM.ZK.Worker");
325     zkEventWorkers = Threads.getBoundedCachedThreadPool(workers, 60L,
326             TimeUnit.SECONDS, threadFactory);
327     this.tableLockManager = tableLockManager;
328 
329     this.metricsAssignmentManager = new MetricsAssignmentManager();
330     useZKForAssignment = ConfigUtil.useZKForAssignment(conf);
331   }
332 
333   MetricsAssignmentManager getAssignmentManagerMetrics() {
334     return this.metricsAssignmentManager;
335   }
336 
337   /**
338    * Add the listener to the notification list.
339    * @param listener The AssignmentListener to register
340    */
341   public void registerListener(final AssignmentListener listener) {
342     this.listeners.add(listener);
343   }
344 
345   /**
346    * Remove the listener from the notification list.
347    * @param listener The AssignmentListener to unregister
348    */
349   public boolean unregisterListener(final AssignmentListener listener) {
350     return this.listeners.remove(listener);
351   }
352 
353   /**
354    * @return Instance of ZKTableStateManager.
355    */
356   public TableStateManager getTableStateManager() {
357     // These are 'expensive' to make involving trip to zk ensemble so allow
358     // sharing.
359     return this.tableStateManager;
360   }
361 
362   /**
363    * This SHOULD not be public. It is public now
364    * because of some unit tests.
365    *
366    * TODO: make it package private and keep RegionStates in the master package
367    */
368   public RegionStates getRegionStates() {
369     return regionStates;
370   }
371 
372   /**
373    * Used in some tests to mock up region state in meta
374    */
375   @VisibleForTesting
376   RegionStateStore getRegionStateStore() {
377     return regionStateStore;
378   }
379 
380   public RegionPlan getRegionReopenPlan(HRegionInfo hri) {
381     return new RegionPlan(hri, null, regionStates.getRegionServerOfRegion(hri));
382   }
383 
384   /**
385    * Add a regionPlan for the specified region.
386    * @param encodedName
387    * @param plan
388    */
389   public void addPlan(String encodedName, RegionPlan plan) {
390     synchronized (regionPlans) {
391       regionPlans.put(encodedName, plan);
392     }
393   }
394 
395   /**
396    * Add a map of region plans.
397    */
398   public void addPlans(Map<String, RegionPlan> plans) {
399     synchronized (regionPlans) {
400       regionPlans.putAll(plans);
401     }
402   }
403 
404   /**
405    * Set the list of regions that will be reopened
406    * because of an update in table schema
407    *
408    * @param regions
409    *          list of regions that should be tracked for reopen
410    */
411   public void setRegionsToReopen(List <HRegionInfo> regions) {
412     for(HRegionInfo hri : regions) {
413       regionsToReopen.put(hri.getEncodedName(), hri);
414     }
415   }
416 
417   /**
418    * Used by the client to identify if all regions have the schema updates
419    *
420    * @param tableName
421    * @return Pair indicating the status of the alter command
422    * @throws IOException
423    */
424   public Pair<Integer, Integer> getReopenStatus(TableName tableName)
425       throws IOException {
426     List<HRegionInfo> hris;
427     if (TableName.META_TABLE_NAME.equals(tableName)) {
428       hris = new MetaTableLocator().getMetaRegions(server.getZooKeeper());
429     } else {
430       hris = MetaTableAccessor.getTableRegions(server.getZooKeeper(),
431         server.getConnection(), tableName, true);
432     }
433 
434     Integer pending = 0;
435     for (HRegionInfo hri : hris) {
436       String name = hri.getEncodedName();
437       // no lock concurrent access ok: sequential consistency respected.
438       if (regionsToReopen.containsKey(name)
439           || regionStates.isRegionInTransition(name)) {
440         pending++;
441       }
442     }
443     return new Pair<Integer, Integer>(pending, hris.size());
444   }
445 
446   /**
447    * Used by ServerShutdownHandler to make sure AssignmentManager has completed
448    * the failover cleanup before re-assigning regions of dead servers. So that
449    * when re-assignment happens, AssignmentManager has proper region states.
450    */
451   public boolean isFailoverCleanupDone() {
452     return failoverCleanupDone.get();
453   }
454 
455   /**
456    * To avoid racing with AM, external entities may need to lock a region,
457    * for example, when SSH checks what regions to skip re-assigning.
458    */
459   public Lock acquireRegionLock(final String encodedName) {
460     return locker.acquireLock(encodedName);
461   }
462 
463   /**
464    * Now, failover cleanup is completed. Notify server manager to
465    * process queued up dead servers processing, if any.
466    */
467   void failoverCleanupDone() {
468     failoverCleanupDone.set(true);
469     serverManager.processQueuedDeadServers();
470   }
471 
472   /**
473    * Called on startup.
474    * Figures whether a fresh cluster start of we are joining extant running cluster.
475    * @throws IOException
476    * @throws KeeperException
477    * @throws InterruptedException
478    * @throws CoordinatedStateException
479    */
480   void joinCluster() throws IOException,
481       KeeperException, InterruptedException, CoordinatedStateException {
482     long startTime = System.currentTimeMillis();
483     // Concurrency note: In the below the accesses on regionsInTransition are
484     // outside of a synchronization block where usually all accesses to RIT are
485     // synchronized.  The presumption is that in this case it is safe since this
486     // method is being played by a single thread on startup.
487 
488     // TODO: Regions that have a null location and are not in regionsInTransitions
489     // need to be handled.
490 
491     // Scan hbase:meta to build list of existing regions, servers, and assignment
492     // Returns servers who have not checked in (assumed dead) that some regions
493     // were assigned to (according to the meta)
494     Set<ServerName> deadServers = rebuildUserRegions();
495 
496     // This method will assign all user regions if a clean server startup or
497     // it will reconstruct master state and cleanup any leftovers from previous master process.
498     boolean failover = processDeadServersAndRegionsInTransition(deadServers);
499 
500     if (!useZKForAssignment) {
501       // Not use ZK for assignment any more, remove the ZNode
502       ZKUtil.deleteNodeRecursively(watcher, watcher.assignmentZNode);
503     }
504     recoverTableInDisablingState();
505     recoverTableInEnablingState();
506     LOG.info("Joined the cluster in " + (System.currentTimeMillis()
507       - startTime) + "ms, failover=" + failover);
508   }
509 
510   /**
511    * Process all regions that are in transition in zookeeper and also
512    * processes the list of dead servers.
513    * Used by master joining an cluster.  If we figure this is a clean cluster
514    * startup, will assign all user regions.
515    * @param deadServers Set of servers that are offline probably legitimately that were carrying
516    * regions according to a scan of hbase:meta. Can be null.
517    * @throws KeeperException
518    * @throws IOException
519    * @throws InterruptedException
520    */
521   boolean processDeadServersAndRegionsInTransition(final Set<ServerName> deadServers)
522   throws KeeperException, IOException, InterruptedException, CoordinatedStateException {
523     List<String> nodes = ZKUtil.listChildrenNoWatch(watcher, watcher.assignmentZNode);
524 
525     if (useZKForAssignment && nodes == null) {
526       String errorMessage = "Failed to get the children from ZK";
527       server.abort(errorMessage, new IOException(errorMessage));
528       return true; // Doesn't matter in this case
529     }
530 
531     boolean failover = !serverManager.getDeadServers().isEmpty();
532     if (failover) {
533       // This may not be a failover actually, especially if meta is on this master.
534       if (LOG.isDebugEnabled()) {
535         LOG.debug("Found dead servers out on cluster " + serverManager.getDeadServers());
536       }
537     } else {
538       // If any one region except meta is assigned, it's a failover.
539       Set<ServerName> onlineServers = serverManager.getOnlineServers().keySet();
540       for (Map.Entry<HRegionInfo, ServerName> en:
541           regionStates.getRegionAssignments().entrySet()) {
542         HRegionInfo hri = en.getKey();
543         if (!hri.isMetaTable()
544             && onlineServers.contains(en.getValue())) {
545           LOG.debug("Found " + hri + " out on cluster");
546           failover = true;
547           break;
548         }
549       }
550       if (!failover && nodes != null) {
551         // If any one region except meta is in transition, it's a failover.
552         for (String encodedName: nodes) {
553           RegionState regionState = regionStates.getRegionState(encodedName);
554           if (regionState != null && !regionState.getRegion().isMetaRegion()) {
555             LOG.debug("Found " + regionState + " in RITs");
556             failover = true;
557             break;
558           }
559         }
560       }
561     }
562     if (!failover && !useZKForAssignment) {
563       // If any region except meta is in transition on a live server, it's a failover.
564       Map<String, RegionState> regionsInTransition = regionStates.getRegionsInTransition();
565       if (!regionsInTransition.isEmpty()) {
566         Set<ServerName> onlineServers = serverManager.getOnlineServers().keySet();
567         for (RegionState regionState: regionsInTransition.values()) {
568           ServerName serverName = regionState.getServerName();
569           if (!regionState.getRegion().isMetaRegion()
570               && serverName != null && onlineServers.contains(serverName)) {
571             LOG.debug("Found " + regionState + " in RITs");
572             failover = true;
573             break;
574           }
575         }
576       }
577     }
578     if (!failover) {
579       // If we get here, we have a full cluster restart. It is a failover only
580       // if there are some WALs are not split yet. For meta WALs, they should have
581       // been split already, if any. We can walk through those queued dead servers,
582       // if they don't have any WALs, this restart should be considered as a clean one
583       Set<ServerName> queuedDeadServers = serverManager.getRequeuedDeadServers().keySet();
584       if (!queuedDeadServers.isEmpty()) {
585         Configuration conf = server.getConfiguration();
586         Path rootdir = FSUtils.getRootDir(conf);
587         FileSystem fs = rootdir.getFileSystem(conf);
588         for (ServerName serverName: queuedDeadServers) {
589           // In the case of a clean exit, the shutdown handler would have presplit any WALs and
590           // removed empty directories.
591           Path logDir = new Path(rootdir,
592               DefaultWALProvider.getWALDirectoryName(serverName.toString()));
593           Path splitDir = logDir.suffix(DefaultWALProvider.SPLITTING_EXT);
594           if (fs.exists(logDir) || fs.exists(splitDir)) {
595             LOG.debug("Found queued dead server " + serverName);
596             failover = true;
597             break;
598           }
599         }
600         if (!failover) {
601           // We figured that it's not a failover, so no need to
602           // work on these re-queued dead servers any more.
603           LOG.info("AM figured that it's not a failover and cleaned up "
604             + queuedDeadServers.size() + " queued dead servers");
605           serverManager.removeRequeuedDeadServers();
606         }
607       }
608     }
609 
610     Set<TableName> disabledOrDisablingOrEnabling = null;
611     Map<HRegionInfo, ServerName> allRegions = null;
612 
613     if (!failover) {
614       disabledOrDisablingOrEnabling = tableStateManager.getTablesInStates(
615         ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING,
616         ZooKeeperProtos.Table.State.ENABLING);
617 
618       // Clean re/start, mark all user regions closed before reassignment
619       allRegions = regionStates.closeAllUserRegions(
620         disabledOrDisablingOrEnabling);
621     }
622 
623     // Now region states are restored
624     regionStateStore.start();
625 
626     // If we found user regions out on cluster, its a failover.
627     if (failover) {
628       LOG.info("Found regions out on cluster or in RIT; presuming failover");
629       // Process list of dead servers and regions in RIT.
630       // See HBASE-4580 for more information.
631       processDeadServersAndRecoverLostRegions(deadServers);
632     }
633 
634     if (!failover && useZKForAssignment) {
635       // Cleanup any existing ZK nodes and start watching
636       ZKAssign.deleteAllNodes(watcher);
637       ZKUtil.listChildrenAndWatchForNewChildren(this.watcher,
638         this.watcher.assignmentZNode);
639     }
640 
641     // Now we can safely claim failover cleanup completed and enable
642     // ServerShutdownHandler for further processing. The nodes (below)
643     // in transition, if any, are for regions not related to those
644     // dead servers at all, and can be done in parallel to SSH.
645     failoverCleanupDone();
646     if (!failover) {
647       // Fresh cluster startup.
648       LOG.info("Clean cluster startup. Assigning user regions");
649       assignAllUserRegions(allRegions);
650     }
651     // unassign replicas of the split parents and the merged regions
652     // the daughter replicas are opened in assignAllUserRegions if it was
653     // not already opened.
654     for (HRegionInfo h : replicasToClose) {
655       unassign(h);
656     }
657     replicasToClose.clear();
658     return failover;
659   }
660 
661   /**
662    * If region is up in zk in transition, then do fixup and block and wait until
663    * the region is assigned and out of transition.  Used on startup for
664    * catalog regions.
665    * @param hri Region to look for.
666    * @return True if we processed a region in transition else false if region
667    * was not up in zk in transition.
668    * @throws InterruptedException
669    * @throws KeeperException
670    * @throws IOException
671    */
672   boolean processRegionInTransitionAndBlockUntilAssigned(final HRegionInfo hri)
673       throws InterruptedException, KeeperException, IOException {
674     String encodedRegionName = hri.getEncodedName();
675     if (!processRegionInTransition(encodedRegionName, hri)) {
676       return false; // The region is not in transition
677     }
678     LOG.debug("Waiting on " + HRegionInfo.prettyPrint(encodedRegionName));
679     while (!this.server.isStopped() &&
680         this.regionStates.isRegionInTransition(encodedRegionName)) {
681       RegionState state = this.regionStates.getRegionTransitionState(encodedRegionName);
682       if (state == null || !serverManager.isServerOnline(state.getServerName())) {
683         // The region is not in transition, or not in transition on an online
684         // server. Doesn't help to block here any more. Caller need to
685         // verify the region is actually assigned.
686         break;
687       }
688       this.regionStates.waitForUpdate(100);
689     }
690     return true;
691   }
692 
693   /**
694    * Process failover of new master for region <code>encodedRegionName</code>
695    * up in zookeeper.
696    * @param encodedRegionName Region to process failover for.
697    * @param regionInfo If null we'll go get it from meta table.
698    * @return True if we processed <code>regionInfo</code> as a RIT.
699    * @throws KeeperException
700    * @throws IOException
701    */
702   boolean processRegionInTransition(final String encodedRegionName,
703       final HRegionInfo regionInfo) throws KeeperException, IOException {
704     // We need a lock here to ensure that we will not put the same region twice
705     // It has no reason to be a lock shared with the other operations.
706     // We can do the lock on the region only, instead of a global lock: what we want to ensure
707     // is that we don't have two threads working on the same region.
708     Lock lock = locker.acquireLock(encodedRegionName);
709     try {
710       Stat stat = new Stat();
711       byte [] data = ZKAssign.getDataAndWatch(watcher, encodedRegionName, stat);
712       if (data == null) return false;
713       RegionTransition rt;
714       try {
715         rt = RegionTransition.parseFrom(data);
716       } catch (DeserializationException e) {
717         LOG.warn("Failed parse znode data", e);
718         return false;
719       }
720       HRegionInfo hri = regionInfo;
721       if (hri == null) {
722         // The region info is not passed in. We will try to find the region
723         // from region states map/meta based on the encoded region name. But we
724         // may not be able to find it. This is valid for online merge that
725         // the region may have not been created if the merge is not completed.
726         // Therefore, it is not in meta at master recovery time.
727         hri = regionStates.getRegionInfo(rt.getRegionName());
728         EventType et = rt.getEventType();
729         if (hri == null && et != EventType.RS_ZK_REGION_MERGING
730             && et != EventType.RS_ZK_REQUEST_REGION_MERGE) {
731           LOG.warn("Couldn't find the region in recovering " + rt);
732           return false;
733         }
734       }
735 
736       // TODO: This code is tied to ZK anyway, so for now leaving it as is,
737       // will refactor when whole region assignment will be abstracted from ZK
738       BaseCoordinatedStateManager cp =
739         (BaseCoordinatedStateManager) this.server.getCoordinatedStateManager();
740       OpenRegionCoordination openRegionCoordination = cp.getOpenRegionCoordination();
741 
742       ZkOpenRegionCoordination.ZkOpenRegionDetails zkOrd =
743         new ZkOpenRegionCoordination.ZkOpenRegionDetails();
744       zkOrd.setVersion(stat.getVersion());
745       zkOrd.setServerName(cp.getServer().getServerName());
746 
747       return processRegionsInTransition(
748         rt, hri, openRegionCoordination, zkOrd);
749     } finally {
750       lock.unlock();
751     }
752   }
753 
754   /**
755    * This call is invoked only (1) master assign meta;
756    * (2) during failover mode startup, zk assignment node processing.
757    * The locker is set in the caller. It returns true if the region
758    * is in transition for sure, false otherwise.
759    *
760    * It should be private but it is used by some test too.
761    */
762   boolean processRegionsInTransition(
763       final RegionTransition rt, final HRegionInfo regionInfo,
764       OpenRegionCoordination coordination,
765       final OpenRegionCoordination.OpenRegionDetails ord) throws KeeperException {
766     EventType et = rt.getEventType();
767     // Get ServerName.  Could not be null.
768     final ServerName sn = rt.getServerName();
769     final byte[] regionName = rt.getRegionName();
770     final String encodedName = HRegionInfo.encodeRegionName(regionName);
771     final String prettyPrintedRegionName = HRegionInfo.prettyPrint(encodedName);
772     LOG.info("Processing " + prettyPrintedRegionName + " in state: " + et);
773 
774     if (regionStates.isRegionInTransition(encodedName)
775         && (regionInfo.isMetaRegion() || !useZKForAssignment)) {
776       LOG.info("Processed region " + prettyPrintedRegionName + " in state: "
777         + et + ", does nothing since the region is already in transition "
778         + regionStates.getRegionTransitionState(encodedName));
779       // Just return
780       return true;
781     }
782     if (!serverManager.isServerOnline(sn)) {
783       // It was transitioning on a dead server, so it's closed now.
784       // Force to OFFLINE and put it in transition, but not assign it
785       // since log splitting for the dead server is not done yet.
786       LOG.debug("RIT " + encodedName + " in state=" + rt.getEventType() +
787         " was on deadserver; forcing offline");
788       if (regionStates.isRegionOnline(regionInfo)) {
789         // Meta could still show the region is assigned to the previous
790         // server. If that server is online, when we reload the meta, the
791         // region is put back to online, we need to offline it.
792         regionStates.regionOffline(regionInfo);
793         sendRegionClosedNotification(regionInfo);
794       }
795       // Put it back in transition so that SSH can re-assign it
796       regionStates.updateRegionState(regionInfo, State.OFFLINE, sn);
797 
798       if (regionInfo.isMetaRegion()) {
799         // If it's meta region, reset the meta location.
800         // So that master knows the right meta region server.
801         MetaTableLocator.setMetaLocation(watcher, sn, State.OPEN);
802       } else {
803         // No matter the previous server is online or offline,
804         // we need to reset the last region server of the region.
805         regionStates.setLastRegionServerOfRegion(sn, encodedName);
806         // Make sure we know the server is dead.
807         if (!serverManager.isServerDead(sn)) {
808           serverManager.expireServer(sn);
809         }
810       }
811       return false;
812     }
813     switch (et) {
814       case M_ZK_REGION_CLOSING:
815         // Insert into RIT & resend the query to the region server: may be the previous master
816         // died before sending the query the first time.
817         final RegionState rsClosing = regionStates.updateRegionState(rt, State.CLOSING);
818         this.executorService.submit(
819           new EventHandler(server, EventType.M_MASTER_RECOVERY) {
820             @Override
821             public void process() throws IOException {
822               ReentrantLock lock = locker.acquireLock(regionInfo.getEncodedName());
823               try {
824                 final int expectedVersion = ((ZkOpenRegionCoordination.ZkOpenRegionDetails) ord)
825                   .getVersion();
826                 unassign(regionInfo, rsClosing, expectedVersion, null, useZKForAssignment, null);
827                 if (regionStates.isRegionOffline(regionInfo)) {
828                   assign(regionInfo, true);
829                 }
830               } finally {
831                 lock.unlock();
832               }
833             }
834           });
835         break;
836 
837       case RS_ZK_REGION_CLOSED:
838       case RS_ZK_REGION_FAILED_OPEN:
839         // Region is closed, insert into RIT and handle it
840         regionStates.setRegionStateTOCLOSED(regionInfo, sn);
841         if (!replicasToClose.contains(regionInfo)) {
842           invokeAssign(regionInfo);
843         } else {
844           offlineDisabledRegion(regionInfo);
845         }
846         break;
847 
848       case M_ZK_REGION_OFFLINE:
849         // Insert in RIT and resend to the regionserver
850         regionStates.updateRegionState(rt, State.OFFLINE);
851         final RegionState rsOffline = regionStates.getRegionState(regionInfo);
852         this.executorService.submit(
853           new EventHandler(server, EventType.M_MASTER_RECOVERY) {
854             @Override
855             public void process() throws IOException {
856               ReentrantLock lock = locker.acquireLock(regionInfo.getEncodedName());
857               try {
858                 RegionPlan plan = new RegionPlan(regionInfo, null, sn);
859                 addPlan(encodedName, plan);
860                 assign(rsOffline, true, false);
861               } finally {
862                 lock.unlock();
863               }
864             }
865           });
866         break;
867 
868       case RS_ZK_REGION_OPENING:
869         regionStates.updateRegionState(rt, State.OPENING);
870         break;
871 
872       case RS_ZK_REGION_OPENED:
873         // Region is opened, insert into RIT and handle it
874         // This could be done asynchronously, we would need then to acquire the lock in the
875         //  handler.
876         regionStates.updateRegionState(rt, State.OPEN);
877         new OpenedRegionHandler(server, this, regionInfo, coordination, ord).process();
878         break;
879       case RS_ZK_REQUEST_REGION_SPLIT:
880       case RS_ZK_REGION_SPLITTING:
881       case RS_ZK_REGION_SPLIT:
882         // Splitting region should be online. We could have skipped it during
883         // user region rebuilding since we may consider the split is completed.
884         // Put it in SPLITTING state to avoid complications.
885         regionStates.regionOnline(regionInfo, sn);
886         regionStates.updateRegionState(rt, State.SPLITTING);
887         if (!handleRegionSplitting(
888             rt, encodedName, prettyPrintedRegionName, sn)) {
889           deleteSplittingNode(encodedName, sn);
890         }
891         break;
892       case RS_ZK_REQUEST_REGION_MERGE:
893       case RS_ZK_REGION_MERGING:
894       case RS_ZK_REGION_MERGED:
895         if (!handleRegionMerging(
896             rt, encodedName, prettyPrintedRegionName, sn)) {
897           deleteMergingNode(encodedName, sn);
898         }
899         break;
900       default:
901         throw new IllegalStateException("Received region in state:" + et + " is not valid.");
902     }
903     LOG.info("Processed region " + prettyPrintedRegionName + " in state "
904       + et + ", on " + (serverManager.isServerOnline(sn) ? "" : "dead ")
905       + "server: " + sn);
906     return true;
907   }
908 
909   /**
910    * When a region is closed, it should be removed from the regionsToReopen
911    * @param hri HRegionInfo of the region which was closed
912    */
913   public void removeClosedRegion(HRegionInfo hri) {
914     if (regionsToReopen.remove(hri.getEncodedName()) != null) {
915       LOG.debug("Removed region from reopening regions because it was closed");
916     }
917   }
918 
919   /**
920    * Handles various states an unassigned node can be in.
921    * <p>
922    * Method is called when a state change is suspected for an unassigned node.
923    * <p>
924    * This deals with skipped transitions (we got a CLOSED but didn't see CLOSING
925    * yet).
926    * @param rt region transition
927    * @param coordination coordination for opening region
928    * @param ord details about opening region
929    */
930   @edu.umd.cs.findbugs.annotations.SuppressWarnings(
931       value="AT_OPERATION_SEQUENCE_ON_CONCURRENT_ABSTRACTION",
932       justification="Needs work; says access to ConcurrentHashMaps not ATOMIC!!!")
933   void handleRegion(final RegionTransition rt, OpenRegionCoordination coordination,
934                     OpenRegionCoordination.OpenRegionDetails ord) {
935     if (rt == null) {
936       LOG.warn("Unexpected NULL input for RegionTransition rt");
937       return;
938     }
939     final ServerName sn = rt.getServerName();
940     // Check if this is a special HBCK transition
941     if (sn.equals(HBCK_CODE_SERVERNAME)) {
942       handleHBCK(rt);
943       return;
944     }
945     final long createTime = rt.getCreateTime();
946     final byte[] regionName = rt.getRegionName();
947     String encodedName = HRegionInfo.encodeRegionName(regionName);
948     String prettyPrintedRegionName = HRegionInfo.prettyPrint(encodedName);
949     // Verify this is a known server
950     if (!serverManager.isServerOnline(sn)
951       && !ignoreStatesRSOffline.contains(rt.getEventType())) {
952       LOG.warn("Attempted to handle region transition for server but " +
953         "it is not online: " + prettyPrintedRegionName + ", " + rt);
954       return;
955     }
956 
957     RegionState regionState =
958       regionStates.getRegionState(encodedName);
959     long startTime = System.currentTimeMillis();
960     if (LOG.isDebugEnabled()) {
961       boolean lateEvent = createTime < (startTime - 15000);
962       LOG.debug("Handling " + rt.getEventType() +
963         ", server=" + sn + ", region=" +
964         (prettyPrintedRegionName == null ? "null" : prettyPrintedRegionName) +
965         (lateEvent ? ", which is more than 15 seconds late" : "") +
966         ", current_state=" + regionState);
967     }
968     // We don't do anything for this event,
969     // so separate it out, no need to lock/unlock anything
970     if (rt.getEventType() == EventType.M_ZK_REGION_OFFLINE) {
971       return;
972     }
973 
974     // We need a lock on the region as we could update it
975     Lock lock = locker.acquireLock(encodedName);
976     try {
977       RegionState latestState =
978         regionStates.getRegionState(encodedName);
979       if ((regionState == null && latestState != null)
980           || (regionState != null && latestState == null)
981           || (regionState != null && latestState != null
982             && latestState.getState() != regionState.getState())) {
983         LOG.warn("Region state changed from " + regionState + " to "
984           + latestState + ", while acquiring lock");
985       }
986       long waitedTime = System.currentTimeMillis() - startTime;
987       if (waitedTime > 5000) {
988         LOG.warn("Took " + waitedTime + "ms to acquire the lock");
989       }
990       regionState = latestState;
991       switch (rt.getEventType()) {
992       case RS_ZK_REQUEST_REGION_SPLIT:
993       case RS_ZK_REGION_SPLITTING:
994       case RS_ZK_REGION_SPLIT:
995         if (!handleRegionSplitting(
996             rt, encodedName, prettyPrintedRegionName, sn)) {
997           deleteSplittingNode(encodedName, sn);
998         }
999         break;
1000 
1001       case RS_ZK_REQUEST_REGION_MERGE:
1002       case RS_ZK_REGION_MERGING:
1003       case RS_ZK_REGION_MERGED:
1004         // Merged region is a new region, we can't find it in the region states now.
1005         // However, the two merging regions are not new. They should be in state for merging.
1006         if (!handleRegionMerging(
1007             rt, encodedName, prettyPrintedRegionName, sn)) {
1008           deleteMergingNode(encodedName, sn);
1009         }
1010         break;
1011 
1012       case M_ZK_REGION_CLOSING:
1013         // Should see CLOSING after we have asked it to CLOSE or additional
1014         // times after already being in state of CLOSING
1015         if (regionState == null
1016             || !regionState.isPendingCloseOrClosingOnServer(sn)) {
1017           LOG.warn("Received CLOSING for " + prettyPrintedRegionName
1018             + " from " + sn + " but the region isn't PENDING_CLOSE/CLOSING here: "
1019             + regionStates.getRegionState(encodedName));
1020           return;
1021         }
1022         // Transition to CLOSING (or update stamp if already CLOSING)
1023         regionStates.updateRegionState(rt, State.CLOSING);
1024         break;
1025 
1026       case RS_ZK_REGION_CLOSED:
1027         // Should see CLOSED after CLOSING but possible after PENDING_CLOSE
1028         if (regionState == null
1029             || !regionState.isPendingCloseOrClosingOnServer(sn)) {
1030           LOG.warn("Received CLOSED for " + prettyPrintedRegionName
1031             + " from " + sn + " but the region isn't PENDING_CLOSE/CLOSING here: "
1032             + regionStates.getRegionState(encodedName));
1033           return;
1034         }
1035         // Handle CLOSED by assigning elsewhere or stopping if a disable
1036         // If we got here all is good.  Need to update RegionState -- else
1037         // what follows will fail because not in expected state.
1038         new ClosedRegionHandler(server, this, regionState.getRegion()).process();
1039         updateClosedRegionHandlerTracker(regionState.getRegion());
1040         break;
1041 
1042         case RS_ZK_REGION_FAILED_OPEN:
1043           if (regionState == null
1044               || !regionState.isPendingOpenOrOpeningOnServer(sn)) {
1045             LOG.warn("Received FAILED_OPEN for " + prettyPrintedRegionName
1046               + " from " + sn + " but the region isn't PENDING_OPEN/OPENING here: "
1047               + regionStates.getRegionState(encodedName));
1048             return;
1049           }
1050           AtomicInteger failedOpenCount = failedOpenTracker.get(encodedName);
1051           if (failedOpenCount == null) {
1052             failedOpenCount = new AtomicInteger();
1053             // No need to use putIfAbsent, or extra synchronization since
1054             // this whole handleRegion block is locked on the encoded region
1055             // name, and failedOpenTracker is updated only in this block
1056             // FindBugs: AT_OPERATION_SEQUENCE_ON_CONCURRENT_ABSTRACTION
1057             failedOpenTracker.put(encodedName, failedOpenCount);
1058           }
1059           if (failedOpenCount.incrementAndGet() >= maximumAttempts) {
1060             // FindBugs: AT_OPERATION_SEQUENCE_ON_CONCURRENT_ABSTRACTION
1061             regionStates.updateRegionState(rt, State.FAILED_OPEN);
1062             // remove the tracking info to save memory, also reset
1063             // the count for next open initiative
1064             failedOpenTracker.remove(encodedName);
1065           } else {
1066             // Handle this the same as if it were opened and then closed.
1067             regionState = regionStates.setRegionStateTOCLOSED(rt.getRegionName(), sn);
1068             if (regionState != null) {
1069               // When there are more than one region server a new RS is selected as the
1070               // destination and the same is updated in the regionplan. (HBASE-5546)
1071               try {
1072                 getRegionPlan(regionState.getRegion(), sn, true);
1073                 new ClosedRegionHandler(server, this, regionState.getRegion()).process();
1074               } catch (HBaseIOException e) {
1075                 LOG.warn("Failed to get region plan", e);
1076               }
1077             }
1078           }
1079           break;
1080 
1081         case RS_ZK_REGION_OPENING:
1082           // Should see OPENING after we have asked it to OPEN or additional
1083           // times after already being in state of OPENING
1084           if (regionState == null
1085               || !regionState.isPendingOpenOrOpeningOnServer(sn)) {
1086             LOG.warn("Received OPENING for " + prettyPrintedRegionName
1087               + " from " + sn + " but the region isn't PENDING_OPEN/OPENING here: "
1088               + regionStates.getRegionState(encodedName));
1089             return;
1090           }
1091           // Transition to OPENING (or update stamp if already OPENING)
1092           regionStates.updateRegionState(rt, State.OPENING);
1093           break;
1094 
1095         case RS_ZK_REGION_OPENED:
1096           // Should see OPENED after OPENING but possible after PENDING_OPEN.
1097           if (regionState == null
1098               || !regionState.isPendingOpenOrOpeningOnServer(sn)) {
1099             LOG.warn("Received OPENED for " + prettyPrintedRegionName
1100               + " from " + sn + " but the region isn't PENDING_OPEN/OPENING here: "
1101               + regionStates.getRegionState(encodedName));
1102 
1103             if (regionState != null) {
1104               if(regionState.isOpened() && regionState.getServerName().equals(sn)) {
1105                 //if this region was opened before on this rs, we don't have to unassign it. It won't cause
1106                 //double assign. One possible scenario of what happened is HBASE-17275
1107                 failedOpenTracker.remove(encodedName); // reset the count, if any
1108                 new OpenedRegionHandler(
1109                     server, this, regionState.getRegion(), coordination, ord).process();
1110                 updateOpenedRegionHandlerTracker(regionState.getRegion());
1111               } else {
1112                 // Close it without updating the internal region states,
1113                 // so as not to create double assignments in unlucky scenarios
1114                 // mentioned in OpenRegionHandler#process
1115                 unassign(regionState.getRegion(), null, -1, null, false, sn);
1116               }
1117             }
1118             return;
1119           }
1120           // Handle OPENED by removing from transition and deleted zk node
1121           regionState =
1122               regionStates.transitionOpenFromPendingOpenOrOpeningOnServer(rt,regionState, sn);
1123           if (regionState != null) {
1124             failedOpenTracker.remove(encodedName); // reset the count, if any
1125             new OpenedRegionHandler(
1126               server, this, regionState.getRegion(), coordination, ord).process();
1127             updateOpenedRegionHandlerTracker(regionState.getRegion());
1128           }
1129           break;
1130 
1131         default:
1132           throw new IllegalStateException("Received event is not valid.");
1133       }
1134     } finally {
1135       lock.unlock();
1136     }
1137   }
1138 
1139   //For unit tests only
1140   boolean wasClosedHandlerCalled(HRegionInfo hri) {
1141     AtomicBoolean b = closedRegionHandlerCalled.get(hri);
1142     //compareAndSet to be sure that unit tests don't see stale values. Means,
1143     //we will return true exactly once unless the handler code resets to true
1144     //this value.
1145     return b == null ? false : b.compareAndSet(true, false);
1146   }
1147 
1148   //For unit tests only
1149   boolean wasOpenedHandlerCalled(HRegionInfo hri) {
1150     AtomicBoolean b = openedRegionHandlerCalled.get(hri);
1151     //compareAndSet to be sure that unit tests don't see stale values. Means,
1152     //we will return true exactly once unless the handler code resets to true
1153     //this value.
1154     return b == null ? false : b.compareAndSet(true, false);
1155   }
1156 
1157   //For unit tests only
1158   void initializeHandlerTrackers() {
1159     closedRegionHandlerCalled = new HashMap<HRegionInfo, AtomicBoolean>();
1160     openedRegionHandlerCalled = new HashMap<HRegionInfo, AtomicBoolean>();
1161   }
1162 
1163   void updateClosedRegionHandlerTracker(HRegionInfo hri) {
1164     if (closedRegionHandlerCalled != null) { //only for unit tests this is true
1165       closedRegionHandlerCalled.put(hri, new AtomicBoolean(true));
1166     }
1167   }
1168 
1169   void updateOpenedRegionHandlerTracker(HRegionInfo hri) {
1170     if (openedRegionHandlerCalled != null) { //only for unit tests this is true
1171       openedRegionHandlerCalled.put(hri, new AtomicBoolean(true));
1172     }
1173   }
1174 
1175   // TODO: processFavoredNodes might throw an exception, for e.g., if the
1176   // meta could not be contacted/updated. We need to see how seriously to treat
1177   // this problem as. Should we fail the current assignment. We should be able
1178   // to recover from this problem eventually (if the meta couldn't be updated
1179   // things should work normally and eventually get fixed up).
1180   void processFavoredNodes(List<HRegionInfo> regions) throws IOException {
1181     if (!shouldAssignRegionsWithFavoredNodes) return;
1182     // The AM gets the favored nodes info for each region and updates the meta
1183     // table with that info
1184     Map<HRegionInfo, List<ServerName>> regionToFavoredNodes =
1185         new HashMap<HRegionInfo, List<ServerName>>();
1186     for (HRegionInfo region : regions) {
1187       regionToFavoredNodes.put(region,
1188           ((FavoredNodeLoadBalancer)this.balancer).getFavoredNodes(region));
1189     }
1190     FavoredNodeAssignmentHelper.updateMetaWithFavoredNodesInfo(regionToFavoredNodes,
1191       this.server.getConnection());
1192   }
1193 
1194   /**
1195    * Handle a ZK unassigned node transition triggered by HBCK repair tool.
1196    * <p>
1197    * This is handled in a separate code path because it breaks the normal rules.
1198    * @param rt
1199    */
1200   @SuppressWarnings("deprecation")
1201   private void handleHBCK(RegionTransition rt) {
1202     String encodedName = HRegionInfo.encodeRegionName(rt.getRegionName());
1203     LOG.info("Handling HBCK triggered transition=" + rt.getEventType() +
1204       ", server=" + rt.getServerName() + ", region=" +
1205       HRegionInfo.prettyPrint(encodedName));
1206     RegionState regionState = regionStates.getRegionTransitionState(encodedName);
1207     switch (rt.getEventType()) {
1208       case M_ZK_REGION_OFFLINE:
1209         HRegionInfo regionInfo;
1210         if (regionState != null) {
1211           regionInfo = regionState.getRegion();
1212         } else {
1213           try {
1214             byte [] name = rt.getRegionName();
1215             Pair<HRegionInfo, ServerName> p = MetaTableAccessor.getRegion(
1216               this.server.getConnection(), name);
1217             regionInfo = p.getFirst();
1218           } catch (IOException e) {
1219             LOG.info("Exception reading hbase:meta doing HBCK repair operation", e);
1220             return;
1221           }
1222         }
1223         LOG.info("HBCK repair is triggering assignment of region=" +
1224             regionInfo.getRegionNameAsString());
1225         // trigger assign, node is already in OFFLINE so don't need to update ZK
1226         assign(regionInfo, false);
1227         break;
1228 
1229       default:
1230         LOG.warn("Received unexpected region state from HBCK: " + rt.toString());
1231         break;
1232     }
1233 
1234   }
1235 
1236   // ZooKeeper events
1237 
1238   /**
1239    * New unassigned node has been created.
1240    *
1241    * <p>This happens when an RS begins the OPENING or CLOSING of a region by
1242    * creating an unassigned node.
1243    *
1244    * <p>When this happens we must:
1245    * <ol>
1246    *   <li>Watch the node for further events</li>
1247    *   <li>Read and handle the state in the node</li>
1248    * </ol>
1249    */
1250   @Override
1251   public void nodeCreated(String path) {
1252     handleAssignmentEvent(path);
1253   }
1254 
1255   /**
1256    * Existing unassigned node has had data changed.
1257    *
1258    * <p>This happens when an RS transitions from OFFLINE to OPENING, or between
1259    * OPENING/OPENED and CLOSING/CLOSED.
1260    *
1261    * <p>When this happens we must:
1262    * <ol>
1263    *   <li>Watch the node for further events</li>
1264    *   <li>Read and handle the state in the node</li>
1265    * </ol>
1266    */
1267   @Override
1268   public void nodeDataChanged(String path) {
1269     handleAssignmentEvent(path);
1270   }
1271 
1272 
1273   // We  don't want to have two events on the same region managed simultaneously.
1274   // For this reason, we need to wait if an event on the same region is currently in progress.
1275   // So we track the region names of the events in progress, and we keep a waiting list.
1276   private final Set<String> regionsInProgress = new HashSet<String>();
1277   // In a LinkedHashMultimap, the put order is kept when we retrieve the collection back. We need
1278   //  this as we want the events to be managed in the same order as we received them.
1279   private final LinkedHashMultimap <String, RegionRunnable>
1280       zkEventWorkerWaitingList = LinkedHashMultimap.create();
1281 
1282   /**
1283    * A specific runnable that works only on a region.
1284    */
1285   private interface RegionRunnable extends Runnable{
1286     /**
1287      * @return - the name of the region it works on.
1288      */
1289     String getRegionName();
1290   }
1291 
1292   /**
1293    * Submit a task, ensuring that there is only one task at a time that working on a given region.
1294    * Order is respected.
1295    */
1296   protected void zkEventWorkersSubmit(final RegionRunnable regRunnable) {
1297 
1298     synchronized (regionsInProgress) {
1299       // If we're there is already a task with this region, we add it to the
1300       //  waiting list and return.
1301       if (regionsInProgress.contains(regRunnable.getRegionName())) {
1302         synchronized (zkEventWorkerWaitingList){
1303           zkEventWorkerWaitingList.put(regRunnable.getRegionName(), regRunnable);
1304         }
1305         return;
1306       }
1307 
1308       // No event in progress on this region => we can submit a new task immediately.
1309       regionsInProgress.add(regRunnable.getRegionName());
1310       zkEventWorkers.submit(new Runnable() {
1311         @Override
1312         public void run() {
1313           try {
1314             regRunnable.run();
1315           } finally {
1316             // now that we have finished, let's see if there is an event for the same region in the
1317             //  waiting list. If it's the case, we can now submit it to the pool.
1318             synchronized (regionsInProgress) {
1319               regionsInProgress.remove(regRunnable.getRegionName());
1320               synchronized (zkEventWorkerWaitingList) {
1321                 java.util.Set<RegionRunnable> waiting = zkEventWorkerWaitingList.get(
1322                     regRunnable.getRegionName());
1323                 if (!waiting.isEmpty()) {
1324                   // We want the first object only. The only way to get it is through an iterator.
1325                   RegionRunnable toSubmit = waiting.iterator().next();
1326                   zkEventWorkerWaitingList.remove(toSubmit.getRegionName(), toSubmit);
1327                   zkEventWorkersSubmit(toSubmit);
1328                 }
1329               }
1330             }
1331           }
1332         }
1333       });
1334     }
1335   }
1336 
1337   @Override
1338   public void nodeDeleted(final String path) {
1339     if (path.startsWith(watcher.assignmentZNode)) {
1340       final String regionName = ZKAssign.getRegionName(watcher, path);
1341       zkEventWorkersSubmit(new RegionRunnable() {
1342         @Override
1343         public String getRegionName() {
1344           return regionName;
1345         }
1346 
1347         @Override
1348         public void run() {
1349           Lock lock = locker.acquireLock(regionName);
1350           try {
1351             RegionState rs = regionStates.getRegionTransitionState(regionName);
1352             if (rs == null) {
1353               rs = regionStates.getRegionState(regionName);
1354               if (rs == null || !rs.isMergingNew()) {
1355                 // MergingNew is an offline state
1356                 return;
1357               }
1358             }
1359 
1360             HRegionInfo regionInfo = rs.getRegion();
1361             String regionNameStr = regionInfo.getRegionNameAsString();
1362             LOG.debug("Znode " + regionNameStr + " deleted, state: " + rs);
1363 
1364             boolean disabled = getTableStateManager().isTableState(regionInfo.getTable(),
1365                 ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING);
1366 
1367             ServerName serverName = rs.getServerName();
1368             if (serverManager.isServerOnline(serverName)) {
1369               if (rs.isOnServer(serverName) && (rs.isOpened() || rs.isSplitting())) {
1370                 synchronized (regionStates) {
1371                   regionOnline(regionInfo, serverName);
1372                   if (rs.isSplitting() && splitRegions.containsKey(regionInfo)) {
1373                     // Check if the daugter regions are still there, if they are present, offline
1374                     // as its the case of a rollback.
1375                     HRegionInfo hri_a = splitRegions.get(regionInfo).getFirst();
1376                     HRegionInfo hri_b = splitRegions.get(regionInfo).getSecond();
1377                     if (!regionStates.isRegionInTransition(hri_a.getEncodedName())) {
1378                       LOG.warn("Split daughter region not in transition " + hri_a);
1379                     }
1380                     if (!regionStates.isRegionInTransition(hri_b.getEncodedName())) {
1381                       LOG.warn("Split daughter region not in transition" + hri_b);
1382                     }
1383                     regionOffline(hri_a);
1384                     regionOffline(hri_b);
1385                     splitRegions.remove(regionInfo);
1386                   }
1387                   if (disabled) {
1388                     // if server is offline, no hurt to unassign again
1389                     LOG.info("Opened " + regionNameStr
1390                         + "but this table is disabled, triggering close of region");
1391                     unassign(regionInfo);
1392                   }
1393                 }
1394               } else if (rs.isMergingNew()) {
1395                 synchronized (regionStates) {
1396                   String p = regionInfo.getEncodedName();
1397                   PairOfSameType<HRegionInfo> regions = mergingRegions.get(p);
1398                   if (regions != null) {
1399                     onlineMergingRegion(disabled, regions.getFirst(), serverName);
1400                     onlineMergingRegion(disabled, regions.getSecond(), serverName);
1401                   }
1402                 }
1403               }
1404             }
1405           } finally {
1406             lock.unlock();
1407           }
1408         }
1409 
1410         private void onlineMergingRegion(boolean disabled,
1411             final HRegionInfo hri, final ServerName serverName) {
1412           RegionState regionState = regionStates.getRegionState(hri);
1413           if (regionState != null && regionState.isMerging()
1414               && regionState.isOnServer(serverName)) {
1415             regionOnline(regionState.getRegion(), serverName);
1416             if (disabled) {
1417               unassign(hri);
1418             }
1419           }
1420         }
1421       });
1422     }
1423   }
1424 
1425   /**
1426    * New unassigned node has been created.
1427    *
1428    * <p>This happens when an RS begins the OPENING, SPLITTING or CLOSING of a
1429    * region by creating a znode.
1430    *
1431    * <p>When this happens we must:
1432    * <ol>
1433    *   <li>Watch the node for further children changed events</li>
1434    *   <li>Watch all new children for changed events</li>
1435    * </ol>
1436    */
1437   @Override
1438   public void nodeChildrenChanged(String path) {
1439     if (path.equals(watcher.assignmentZNode)) {
1440       zkEventWorkers.submit(new Runnable() {
1441         @Override
1442         public void run() {
1443           try {
1444             // Just make sure we see the changes for the new znodes
1445             List<String> children =
1446               ZKUtil.listChildrenAndWatchForNewChildren(
1447                 watcher, watcher.assignmentZNode);
1448             if (children != null) {
1449               Stat stat = new Stat();
1450               for (String child : children) {
1451                 // if region is in transition, we already have a watch
1452                 // on it, so no need to watch it again. So, as I know for now,
1453                 // this is needed to watch splitting nodes only.
1454                 if (!regionStates.isRegionInTransition(child)) {
1455                   ZKAssign.getDataAndWatch(watcher, child, stat);
1456                 }
1457               }
1458             }
1459           } catch (KeeperException e) {
1460             server.abort("Unexpected ZK exception reading unassigned children", e);
1461           }
1462         }
1463       });
1464     }
1465   }
1466 
1467 
1468   /**
1469    * Marks the region as online.  Removes it from regions in transition and
1470    * updates the in-memory assignment information.
1471    * <p>
1472    * Used when a region has been successfully opened on a region server.
1473    * @param regionInfo
1474    * @param sn
1475    */
1476   void regionOnline(HRegionInfo regionInfo, ServerName sn) {
1477     regionOnline(regionInfo, sn, HConstants.NO_SEQNUM);
1478   }
1479 
1480   void regionOnline(HRegionInfo regionInfo, ServerName sn, long openSeqNum) {
1481     numRegionsOpened.incrementAndGet();
1482     regionStates.regionOnline(regionInfo, sn, openSeqNum);
1483 
1484     // Remove plan if one.
1485     clearRegionPlan(regionInfo);
1486     balancer.regionOnline(regionInfo, sn);
1487 
1488     // Tell our listeners that a region was opened
1489     sendRegionOpenedNotification(regionInfo, sn);
1490   }
1491 
1492   /**
1493    * Pass the assignment event to a worker for processing.
1494    * Each worker is a single thread executor service.  The reason
1495    * for just one thread is to make sure all events for a given
1496    * region are processed in order.
1497    *
1498    * @param path
1499    */
1500   private void handleAssignmentEvent(final String path) {
1501     if (path.startsWith(watcher.assignmentZNode)) {
1502       final String regionName = ZKAssign.getRegionName(watcher, path);
1503 
1504       zkEventWorkersSubmit(new RegionRunnable() {
1505         @Override
1506         public String getRegionName() {
1507           return regionName;
1508         }
1509 
1510         @Override
1511         public void run() {
1512           try {
1513             Stat stat = new Stat();
1514             byte [] data = ZKAssign.getDataAndWatch(watcher, path, stat);
1515             if (data == null) return;
1516 
1517             RegionTransition rt = RegionTransition.parseFrom(data);
1518 
1519             // TODO: This code is tied to ZK anyway, so for now leaving it as is,
1520             // will refactor when whole region assignment will be abstracted from ZK
1521             BaseCoordinatedStateManager csm =
1522               (BaseCoordinatedStateManager) server.getCoordinatedStateManager();
1523             OpenRegionCoordination openRegionCoordination = csm.getOpenRegionCoordination();
1524 
1525             ZkOpenRegionCoordination.ZkOpenRegionDetails zkOrd =
1526               new ZkOpenRegionCoordination.ZkOpenRegionDetails();
1527             zkOrd.setVersion(stat.getVersion());
1528             zkOrd.setServerName(csm.getServer().getServerName());
1529 
1530             handleRegion(rt, openRegionCoordination, zkOrd);
1531           } catch (KeeperException e) {
1532             server.abort("Unexpected ZK exception reading unassigned node data", e);
1533           } catch (DeserializationException e) {
1534             server.abort("Unexpected exception deserializing node data", e);
1535           }
1536         }
1537       });
1538     }
1539   }
1540 
1541   /**
1542    * Marks the region as offline.  Removes it from regions in transition and
1543    * removes in-memory assignment information.
1544    * <p>
1545    * Used when a region has been closed and should remain closed.
1546    * @param regionInfo
1547    */
1548   public void regionOffline(final HRegionInfo regionInfo) {
1549     regionOffline(regionInfo, null);
1550   }
1551 
1552   public void offlineDisabledRegion(HRegionInfo regionInfo) {
1553     if (useZKForAssignment) {
1554       // Disabling so should not be reassigned, just delete the CLOSED node
1555       LOG.debug("Table being disabled so deleting ZK node and removing from " +
1556         "regions in transition, skipping assignment of region " +
1557           regionInfo.getRegionNameAsString());
1558       String encodedName = regionInfo.getEncodedName();
1559       deleteNodeInStates(encodedName, "closed", null,
1560         EventType.RS_ZK_REGION_CLOSED, EventType.M_ZK_REGION_OFFLINE);
1561     }
1562     replicasToClose.remove(regionInfo);
1563     regionOffline(regionInfo);
1564   }
1565 
1566   // Assignment methods
1567 
1568   /**
1569    * Assigns the specified region.
1570    * <p>
1571    * If a RegionPlan is available with a valid destination then it will be used
1572    * to determine what server region is assigned to.  If no RegionPlan is
1573    * available, region will be assigned to a random available server.
1574    * <p>
1575    * Updates the RegionState and sends the OPEN RPC.
1576    * <p>
1577    * This will only succeed if the region is in transition and in a CLOSED or
1578    * OFFLINE state or not in transition (in-memory not zk), and of course, the
1579    * chosen server is up and running (It may have just crashed!).  If the
1580    * in-memory checks pass, the zk node is forced to OFFLINE before assigning.
1581    *
1582    * @param region server to be assigned
1583    * @param setOfflineInZK whether ZK node should be created/transitioned to an
1584    *                       OFFLINE state before assigning the region
1585    */
1586   public void assign(HRegionInfo region, boolean setOfflineInZK) {
1587     assign(region, setOfflineInZK, false);
1588   }
1589 
1590   /**
1591    * Use care with forceNewPlan. It could cause double assignment.
1592    */
1593   @VisibleForTesting
1594   public void assign(HRegionInfo region,
1595       boolean setOfflineInZK, boolean forceNewPlan) {
1596     if (isDisabledorDisablingRegionInRIT(region)) {
1597       return;
1598     }
1599     String encodedName = region.getEncodedName();
1600     Lock lock = locker.acquireLock(encodedName);
1601     try {
1602       RegionState state = forceRegionStateToOffline(region, forceNewPlan);
1603       if (state != null) {
1604         if (regionStates.wasRegionOnDeadServer(encodedName)) {
1605           LOG.info("Skip assigning " + region.getRegionNameAsString()
1606             + ", it's host " + regionStates.getLastRegionServerOfRegion(encodedName)
1607             + " is dead but not processed yet");
1608           return;
1609         }
1610         assign(state, setOfflineInZK && useZKForAssignment, forceNewPlan);
1611       }
1612     } finally {
1613       lock.unlock();
1614     }
1615   }
1616 
1617   /**
1618    * Bulk assign regions to <code>destination</code>.
1619    * @param destination
1620    * @param regions Regions to assign.
1621    * @return true if successful
1622    */
1623   boolean assign(final ServerName destination, final List<HRegionInfo> regions)
1624     throws InterruptedException {
1625     long startTime = EnvironmentEdgeManager.currentTime();
1626     try {
1627       int regionCount = regions.size();
1628       if (regionCount == 0) {
1629         return true;
1630       }
1631       LOG.info("Assigning " + regionCount + " region(s) to " + destination.toString());
1632       Set<String> encodedNames = new HashSet<String>(regionCount);
1633       for (HRegionInfo region : regions) {
1634         encodedNames.add(region.getEncodedName());
1635       }
1636 
1637       List<HRegionInfo> failedToOpenRegions = new ArrayList<HRegionInfo>();
1638       Map<String, Lock> locks = locker.acquireLocks(encodedNames);
1639       try {
1640         AtomicInteger counter = new AtomicInteger(0);
1641         Map<String, Integer> offlineNodesVersions = new ConcurrentHashMap<String, Integer>();
1642         OfflineCallback cb = new OfflineCallback(
1643           watcher, destination, counter, offlineNodesVersions);
1644         Map<String, RegionPlan> plans = new HashMap<String, RegionPlan>(regions.size());
1645         List<RegionState> states = new ArrayList<RegionState>(regions.size());
1646         for (HRegionInfo region : regions) {
1647           String encodedName = region.getEncodedName();
1648           if (!isDisabledorDisablingRegionInRIT(region)) {
1649             RegionState state = forceRegionStateToOffline(region, false);
1650             boolean onDeadServer = false;
1651             if (state != null) {
1652               if (regionStates.wasRegionOnDeadServer(encodedName)) {
1653                 LOG.info("Skip assigning " + region.getRegionNameAsString()
1654                   + ", it's host " + regionStates.getLastRegionServerOfRegion(encodedName)
1655                   + " is dead but not processed yet");
1656                 onDeadServer = true;
1657               } else if (!useZKForAssignment
1658                   || asyncSetOfflineInZooKeeper(state, cb, destination)) {
1659                 RegionPlan plan = new RegionPlan(region, state.getServerName(), destination);
1660                 plans.put(encodedName, plan);
1661                 states.add(state);
1662                 continue;
1663               }
1664             }
1665             // Reassign if the region wasn't on a dead server
1666             if (!onDeadServer) {
1667               LOG.info("failed to force region state to offline or "
1668                 + "failed to set it offline in ZK, will reassign later: " + region);
1669               failedToOpenRegions.add(region); // assign individually later
1670             }
1671           }
1672           // Release the lock, this region is excluded from bulk assign because
1673           // we can't update its state, or set its znode to offline.
1674           Lock lock = locks.remove(encodedName);
1675           lock.unlock();
1676         }
1677 
1678         if (useZKForAssignment) {
1679           // Wait until all unassigned nodes have been put up and watchers set.
1680           int total = states.size();
1681           for (int oldCounter = 0; !server.isStopped();) {
1682             int count = counter.get();
1683             if (oldCounter != count) {
1684               LOG.debug(destination.toString() + " unassigned znodes=" + count +
1685                 " of total=" + total + "; oldCounter=" + oldCounter);
1686               oldCounter = count;
1687             }
1688             if (count >= total) break;
1689             Thread.sleep(5);
1690           }
1691         }
1692 
1693         if (server.isStopped()) {
1694           return false;
1695         }
1696 
1697         // Add region plans, so we can updateTimers when one region is opened so
1698         // that unnecessary timeout on RIT is reduced.
1699         this.addPlans(plans);
1700 
1701         List<Triple<HRegionInfo, Integer, List<ServerName>>> regionOpenInfos =
1702           new ArrayList<Triple<HRegionInfo, Integer, List<ServerName>>>(states.size());
1703         for (RegionState state: states) {
1704           HRegionInfo region = state.getRegion();
1705           String encodedRegionName = region.getEncodedName();
1706           Integer nodeVersion = offlineNodesVersions.get(encodedRegionName);
1707           if (useZKForAssignment && (nodeVersion == null || nodeVersion == -1)) {
1708             LOG.warn("failed to offline in zookeeper: " + region);
1709             failedToOpenRegions.add(region); // assign individually later
1710             Lock lock = locks.remove(encodedRegionName);
1711             lock.unlock();
1712           } else {
1713             regionStates.updateRegionState(
1714               region, State.PENDING_OPEN, destination);
1715             List<ServerName> favoredNodes = ServerName.EMPTY_SERVER_LIST;
1716             if (this.shouldAssignRegionsWithFavoredNodes) {
1717               favoredNodes = ((FavoredNodeLoadBalancer)this.balancer).getFavoredNodes(region);
1718             }
1719             regionOpenInfos.add(new Triple<HRegionInfo, Integer,  List<ServerName>>(
1720               region, nodeVersion, favoredNodes));
1721           }
1722         }
1723 
1724         // Move on to open regions.
1725         try {
1726           // Send OPEN RPC. If it fails on a IOE or RemoteException,
1727           // regions will be assigned individually.
1728           long maxWaitTime = System.currentTimeMillis() +
1729             this.server.getConfiguration().
1730               getLong("hbase.regionserver.rpc.startup.waittime", 60000);
1731           for (int i = 1; i <= maximumAttempts && !server.isStopped(); i++) {
1732             try {
1733               // regionOpenInfos is empty if all regions are in failedToOpenRegions list
1734               if (regionOpenInfos.isEmpty()) {
1735                 break;
1736               }
1737               List<RegionOpeningState> regionOpeningStateList = serverManager
1738                 .sendRegionOpen(destination, regionOpenInfos);
1739               if (regionOpeningStateList == null) {
1740                 // Failed getting RPC connection to this server
1741                 return false;
1742               }
1743               for (int k = 0, n = regionOpeningStateList.size(); k < n; k++) {
1744                 RegionOpeningState openingState = regionOpeningStateList.get(k);
1745                 if (openingState != RegionOpeningState.OPENED) {
1746                   HRegionInfo region = regionOpenInfos.get(k).getFirst();
1747                   if (openingState == RegionOpeningState.ALREADY_OPENED) {
1748                     processAlreadyOpenedRegion(region, destination);
1749                   } else if (openingState == RegionOpeningState.FAILED_OPENING) {
1750                     // Failed opening this region, reassign it later
1751                     failedToOpenRegions.add(region);
1752                   } else {
1753                     LOG.warn("THIS SHOULD NOT HAPPEN: unknown opening state "
1754                       + openingState + " in assigning region " + region);
1755                   }
1756                 }
1757               }
1758               break;
1759             } catch (IOException e) {
1760               if (e instanceof RemoteException) {
1761                 e = ((RemoteException)e).unwrapRemoteException();
1762               }
1763               if (e instanceof RegionServerStoppedException) {
1764                 LOG.warn("The region server was shut down, ", e);
1765                 // No need to retry, the region server is a goner.
1766                 return false;
1767               } else if (e instanceof ServerNotRunningYetException) {
1768                 long now = System.currentTimeMillis();
1769                 if (now < maxWaitTime) {
1770                   LOG.debug("Server is not yet up; waiting up to " +
1771                     (maxWaitTime - now) + "ms", e);
1772                   Thread.sleep(100);
1773                   i--; // reset the try count
1774                   continue;
1775                 }
1776               } else if (e instanceof java.net.SocketTimeoutException
1777                   && this.serverManager.isServerOnline(destination)) {
1778                 // In case socket is timed out and the region server is still online,
1779                 // the openRegion RPC could have been accepted by the server and
1780                 // just the response didn't go through.  So we will retry to
1781                 // open the region on the same server.
1782                 if (LOG.isDebugEnabled()) {
1783                   LOG.debug("Bulk assigner openRegion() to " + destination
1784                     + " has timed out, but the regions might"
1785                     + " already be opened on it.", e);
1786                 }
1787                 // wait and reset the re-try count, server might be just busy.
1788                 Thread.sleep(100);
1789                 i--;
1790                 continue;
1791               }
1792               throw e;
1793             }
1794           }
1795         } catch (IOException e) {
1796           // Can be a socket timeout, EOF, NoRouteToHost, etc
1797           LOG.info("Unable to communicate with " + destination
1798             + " in order to assign regions, ", e);
1799           return false;
1800         }
1801       } finally {
1802         for (Lock lock : locks.values()) {
1803           lock.unlock();
1804         }
1805       }
1806 
1807       if (!failedToOpenRegions.isEmpty()) {
1808         for (HRegionInfo region : failedToOpenRegions) {
1809           if (!regionStates.isRegionOnline(region)) {
1810             invokeAssign(region);
1811           }
1812         }
1813       }
1814 
1815       // wait for assignment completion
1816       ArrayList<HRegionInfo> userRegionSet = new ArrayList<HRegionInfo>(regions.size());
1817       for (HRegionInfo region: regions) {
1818         if (!region.getTable().isSystemTable()) {
1819           userRegionSet.add(region);
1820         }
1821       }
1822       if (!waitForAssignment(userRegionSet, true, userRegionSet.size(),
1823             System.currentTimeMillis())) {
1824         LOG.debug("some user regions are still in transition: " + userRegionSet);
1825       }
1826       LOG.debug("Bulk assigning done for " + destination);
1827       return true;
1828     } finally {
1829       metricsAssignmentManager.updateBulkAssignTime(EnvironmentEdgeManager.currentTime() - startTime);
1830     }
1831   }
1832 
1833   /**
1834    * Send CLOSE RPC if the server is online, otherwise, offline the region.
1835    *
1836    * The RPC will be sent only to the region sever found in the region state
1837    * if it is passed in, otherwise, to the src server specified. If region
1838    * state is not specified, we don't update region state at all, instead
1839    * we just send the RPC call. This is useful for some cleanup without
1840    * messing around the region states (see handleRegion, on region opened
1841    * on an unexpected server scenario, for an example)
1842    */
1843   private void unassign(final HRegionInfo region,
1844       final RegionState state, final int versionOfClosingNode,
1845       final ServerName dest, final boolean transitionInZK,
1846       final ServerName src) {
1847     ServerName server = src;
1848     if (state != null) {
1849       server = state.getServerName();
1850     }
1851     long maxWaitTime = -1;
1852     for (int i = 1; i <= this.maximumAttempts; i++) {
1853       if (this.server.isStopped() || this.server.isAborted()) {
1854         LOG.debug("Server stopped/aborted; skipping unassign of " + region);
1855         return;
1856       }
1857       // ClosedRegionhandler can remove the server from this.regions
1858       if (!serverManager.isServerOnline(server)) {
1859         LOG.debug("Offline " + region.getRegionNameAsString()
1860           + ", no need to unassign since it's on a dead server: " + server);
1861         if (transitionInZK) {
1862           // delete the node. if no node exists need not bother.
1863           deleteClosingOrClosedNode(region, server);
1864         }
1865         if (state != null) {
1866           regionOffline(region);
1867         }
1868         return;
1869       }
1870       try {
1871         // Send CLOSE RPC
1872         if (serverManager.sendRegionClose(server, region,
1873           versionOfClosingNode, dest, transitionInZK)) {
1874           LOG.debug("Sent CLOSE to " + server + " for region " +
1875             region.getRegionNameAsString());
1876           if (useZKForAssignment && !transitionInZK && state != null) {
1877             // Retry to make sure the region is
1878             // closed so as to avoid double assignment.
1879             unassign(region, state, versionOfClosingNode,
1880               dest, transitionInZK, src);
1881           }
1882           return;
1883         }
1884         // This never happens. Currently regionserver close always return true.
1885         // Todo; this can now happen (0.96) if there is an exception in a coprocessor
1886         LOG.warn("Server " + server + " region CLOSE RPC returned false for " +
1887           region.getRegionNameAsString());
1888       } catch (Throwable t) {
1889         long sleepTime = 0;
1890         Configuration conf = this.server.getConfiguration();
1891         if (t instanceof RemoteException) {
1892           t = ((RemoteException)t).unwrapRemoteException();
1893         }
1894         boolean logRetries = true;
1895         if (t instanceof RegionServerAbortedException
1896             || t instanceof RegionServerStoppedException
1897             || t instanceof ServerNotRunningYetException) {
1898           // RS is aborting or stopping, we cannot offline the region since the region may need
1899           // to do WAL recovery. Until we see  the RS expiration, we should retry.
1900           sleepTime = 1 + conf.getInt(RpcClient.FAILED_SERVER_EXPIRY_KEY,
1901             RpcClient.FAILED_SERVER_EXPIRY_DEFAULT);
1902 
1903         } else if (t instanceof NotServingRegionException) {
1904           LOG.debug("Offline " + region.getRegionNameAsString()
1905             + ", it's not any more on " + server, t);
1906           if (transitionInZK) {
1907             deleteClosingOrClosedNode(region, server);
1908           }
1909           if (state != null) {
1910             regionOffline(region);
1911           }
1912           return;
1913         } else if ((t instanceof FailedServerException) || (state != null &&
1914             t instanceof RegionAlreadyInTransitionException)) {
1915           if(t instanceof FailedServerException) {
1916             sleepTime = 1 + conf.getInt(RpcClient.FAILED_SERVER_EXPIRY_KEY,
1917                   RpcClient.FAILED_SERVER_EXPIRY_DEFAULT);
1918           } else {
1919             // RS is already processing this region, only need to update the timestamp
1920             LOG.debug("update " + state + " the timestamp.");
1921             state.updateTimestampToNow();
1922             if (maxWaitTime < 0) {
1923               maxWaitTime =
1924                   EnvironmentEdgeManager.currentTime()
1925                       + conf.getLong(ALREADY_IN_TRANSITION_WAITTIME,
1926                         DEFAULT_ALREADY_IN_TRANSITION_WAITTIME);
1927             }
1928             long now = EnvironmentEdgeManager.currentTime();
1929             if (now < maxWaitTime) {
1930               LOG.debug("Region is already in transition; "
1931                 + "waiting up to " + (maxWaitTime - now) + "ms", t);
1932               sleepTime = 100;
1933               i--; // reset the try count
1934               logRetries = false;
1935             }
1936           }
1937         }
1938 
1939         try {
1940           if (sleepTime > 0) {
1941             Thread.sleep(sleepTime);
1942           }
1943         } catch (InterruptedException ie) {
1944           LOG.warn("Failed to unassign "
1945             + region.getRegionNameAsString() + " since interrupted", ie);
1946           Thread.currentThread().interrupt();
1947           if (state != null) {
1948             regionStates.updateRegionState(region, State.FAILED_CLOSE);
1949           }
1950           return;
1951         }
1952 
1953         if (logRetries) {
1954           LOG.info("Server " + server + " returned " + t + " for "
1955             + region.getRegionNameAsString() + ", try=" + i
1956             + " of " + this.maximumAttempts, t);
1957           // Presume retry or server will expire.
1958         }
1959       }
1960     }
1961     // Run out of attempts
1962     if (state != null) {
1963       regionStates.updateRegionState(region, State.FAILED_CLOSE);
1964     }
1965   }
1966 
1967   /**
1968    * Set region to OFFLINE unless it is opening and forceNewPlan is false.
1969    */
1970   private RegionState forceRegionStateToOffline(
1971       final HRegionInfo region, final boolean forceNewPlan) {
1972     RegionState state = regionStates.getRegionState(region);
1973     if (state == null) {
1974       LOG.warn("Assigning but not in region states: " + region);
1975       state = regionStates.createRegionState(region);
1976     }
1977 
1978     ServerName sn = state.getServerName();
1979     if (forceNewPlan && LOG.isDebugEnabled()) {
1980       LOG.debug("Force region state offline " + state);
1981     }
1982 
1983     switch (state.getState()) {
1984     case OPEN:
1985     case OPENING:
1986     case PENDING_OPEN:
1987     case CLOSING:
1988     case PENDING_CLOSE:
1989       if (!forceNewPlan) {
1990         LOG.debug("Skip assigning " +
1991           region + ", it is already " + state);
1992         return null;
1993       }
1994     case FAILED_CLOSE:
1995     case FAILED_OPEN:
1996       unassign(region, state, -1, null, false, null);
1997       state = regionStates.getRegionState(region);
1998       if (state.isFailedClose()) {
1999         // If we can't close the region, we can't re-assign
2000         // it so as to avoid possible double assignment/data loss.
2001         LOG.info("Skip assigning " +
2002           region + ", we couldn't close it: " + state);
2003         return null;
2004       }
2005     case OFFLINE:
2006       // This region could have been open on this server
2007       // for a while. If the server is dead and not processed
2008       // yet, we can move on only if the meta shows the
2009       // region is not on this server actually, or on a server
2010       // not dead, or dead and processed already.
2011       // In case not using ZK, we don't need this check because
2012       // we have the latest info in memory, and the caller
2013       // will do another round checking any way.
2014       if (useZKForAssignment
2015           && regionStates.isServerDeadAndNotProcessed(sn)
2016           && wasRegionOnDeadServerByMeta(region, sn)) {
2017         if (!regionStates.isRegionInTransition(region)) {
2018           LOG.info("Updating the state to " + State.OFFLINE + " to allow to be reassigned by SSH");
2019           regionStates.updateRegionState(region, State.OFFLINE);
2020         }
2021         LOG.info("Skip assigning " + region.getRegionNameAsString()
2022             + ", it is on a dead but not processed yet server: " + sn);
2023         return null;
2024       }
2025     case CLOSED:
2026       break;
2027     default:
2028       LOG.error("Trying to assign region " + region
2029         + ", which is " + state);
2030       return null;
2031     }
2032     return state;
2033   }
2034 
2035   @SuppressWarnings("deprecation")
2036   protected boolean wasRegionOnDeadServerByMeta(
2037       final HRegionInfo region, final ServerName sn) {
2038     try {
2039       if (region.isMetaRegion()) {
2040         ServerName server = this.server.getMetaTableLocator().
2041           getMetaRegionLocation(this.server.getZooKeeper());
2042         return regionStates.isServerDeadAndNotProcessed(server);
2043       }
2044       while (!server.isStopped()) {
2045         try {
2046           this.server.getMetaTableLocator().waitMetaRegionLocation(server.getZooKeeper());
2047           Result r = MetaTableAccessor.getRegionResult(server.getConnection(),
2048             region.getRegionName());
2049           if (r == null || r.isEmpty()) return false;
2050           ServerName server = HRegionInfo.getServerName(r);
2051           return regionStates.isServerDeadAndNotProcessed(server);
2052         } catch (IOException ioe) {
2053           LOG.info("Received exception accessing hbase:meta during force assign "
2054             + region.getRegionNameAsString() + ", retrying", ioe);
2055         }
2056       }
2057     } catch (InterruptedException e) {
2058       Thread.currentThread().interrupt();
2059       LOG.info("Interrupted accessing hbase:meta", e);
2060     }
2061     // Call is interrupted or server is stopped.
2062     return regionStates.isServerDeadAndNotProcessed(sn);
2063   }
2064 
2065   /**
2066    * Caller must hold lock on the passed <code>state</code> object.
2067    * @param state
2068    * @param setOfflineInZK
2069    * @param forceNewPlan
2070    */
2071   public void assign(RegionState state,
2072       boolean setOfflineInZK, final boolean forceNewPlan) {
2073     long startTime = EnvironmentEdgeManager.currentTime();
2074     try {
2075       Configuration conf = server.getConfiguration();
2076       RegionState currentState = state;
2077       int versionOfOfflineNode = -1;
2078       RegionPlan plan = null;
2079       long maxWaitTime = -1;
2080       HRegionInfo region = state.getRegion();
2081       RegionOpeningState regionOpenState;
2082       Throwable previousException = null;
2083       for (int i = 1; i <= maximumAttempts; i++) {
2084         if (server.isStopped() || server.isAborted()) {
2085           LOG.info("Skip assigning " + region.getRegionNameAsString()
2086             + ", the server is stopped/aborted");
2087           return;
2088         }
2089 
2090         if (plan == null) { // Get a server for the region at first
2091           try {
2092             plan = getRegionPlan(region, forceNewPlan);
2093           } catch (HBaseIOException e) {
2094             LOG.warn("Failed to get region plan", e);
2095           }
2096         }
2097 
2098         if (plan == null) {
2099           LOG.warn("Unable to determine a plan to assign " + region);
2100 
2101           // For meta region, we have to keep retrying until succeeding
2102           if (region.isMetaRegion()) {
2103             if (i == maximumAttempts) {
2104               i = 0; // re-set attempt count to 0 for at least 1 retry
2105 
2106               LOG.warn("Unable to determine a plan to assign a hbase:meta region " + region +
2107                 " after maximumAttempts (" + this.maximumAttempts +
2108                 "). Reset attempts count and continue retrying.");
2109             }
2110             waitForRetryingMetaAssignment();
2111             continue;
2112           }
2113 
2114           regionStates.updateRegionState(region, State.FAILED_OPEN);
2115           return;
2116         }
2117         if (setOfflineInZK && versionOfOfflineNode == -1) {
2118           LOG.info("Setting node as OFFLINED in ZooKeeper for region " + region);
2119           // get the version of the znode after setting it to OFFLINE.
2120           // versionOfOfflineNode will be -1 if the znode was not set to OFFLINE
2121           versionOfOfflineNode = setOfflineInZooKeeper(currentState, plan.getDestination());
2122           if (versionOfOfflineNode != -1) {
2123             if (isDisabledorDisablingRegionInRIT(region)) {
2124               return;
2125             }
2126             // In case of assignment from EnableTableHandler table state is ENABLING. Any how
2127             // EnableTableHandler will set ENABLED after assigning all the table regions. If we
2128             // try to set to ENABLED directly then client API may think table is enabled.
2129             // When we have a case such as all the regions are added directly into hbase:meta and we call
2130             // assignRegion then we need to make the table ENABLED. Hence in such case the table
2131             // will not be in ENABLING or ENABLED state.
2132             TableName tableName = region.getTable();
2133             if (!tableStateManager.isTableState(tableName,
2134               ZooKeeperProtos.Table.State.ENABLED, ZooKeeperProtos.Table.State.ENABLING)) {
2135               LOG.debug("Setting table " + tableName + " to ENABLED state.");
2136               setEnabledTable(tableName);
2137             }
2138           }
2139         }
2140         if (setOfflineInZK && versionOfOfflineNode == -1) {
2141           LOG.info("Unable to set offline in ZooKeeper to assign " + region);
2142           // Setting offline in ZK must have been failed due to ZK racing or some
2143           // exception which may make the server to abort. If it is ZK racing,
2144           // we should retry since we already reset the region state,
2145           // existing (re)assignment will fail anyway.
2146           if (!server.isAborted()) {
2147             continue;
2148           }
2149         }
2150         LOG.info("Assigning " + region.getRegionNameAsString() +
2151             " to " + plan.getDestination().toString());
2152         // Transition RegionState to PENDING_OPEN
2153         currentState = regionStates.updateRegionState(region,
2154           State.PENDING_OPEN, plan.getDestination());
2155 
2156         boolean needNewPlan;
2157         final String assignMsg = "Failed assignment of " + region.getRegionNameAsString() +
2158             " to " + plan.getDestination();
2159         try {
2160           List<ServerName> favoredNodes = ServerName.EMPTY_SERVER_LIST;
2161           if (this.shouldAssignRegionsWithFavoredNodes) {
2162             favoredNodes = ((FavoredNodeLoadBalancer)this.balancer).getFavoredNodes(region);
2163           }
2164           regionOpenState = serverManager.sendRegionOpen(
2165               plan.getDestination(), region, versionOfOfflineNode, favoredNodes);
2166 
2167           if (regionOpenState == RegionOpeningState.FAILED_OPENING) {
2168             // Failed opening this region, looping again on a new server.
2169             needNewPlan = true;
2170             LOG.warn(assignMsg + ", regionserver says 'FAILED_OPENING', " +
2171                 " trying to assign elsewhere instead; " +
2172                 "try=" + i + " of " + this.maximumAttempts);
2173           } else {
2174             // we're done
2175             if (regionOpenState == RegionOpeningState.ALREADY_OPENED) {
2176               processAlreadyOpenedRegion(region, plan.getDestination());
2177             }
2178             return;
2179           }
2180 
2181         } catch (Throwable t) {
2182           if (t instanceof RemoteException) {
2183             t = ((RemoteException) t).unwrapRemoteException();
2184           }
2185           previousException = t;
2186 
2187           // Should we wait a little before retrying? If the server is starting it's yes.
2188           // If the region is already in transition, it's yes as well: we want to be sure that
2189           //  the region will get opened but we don't want a double assignment.
2190           boolean hold = (t instanceof RegionAlreadyInTransitionException ||
2191               t instanceof ServerNotRunningYetException);
2192 
2193           // In case socket is timed out and the region server is still online,
2194           // the openRegion RPC could have been accepted by the server and
2195           // just the response didn't go through.  So we will retry to
2196           // open the region on the same server to avoid possible
2197           // double assignment.
2198           boolean retry = !hold && (t instanceof java.net.SocketTimeoutException
2199               && this.serverManager.isServerOnline(plan.getDestination()));
2200 
2201 
2202           if (hold) {
2203             LOG.warn(assignMsg + ", waiting a little before trying on the same region server " +
2204               "try=" + i + " of " + this.maximumAttempts, t);
2205 
2206             if (maxWaitTime < 0) {
2207               if (t instanceof RegionAlreadyInTransitionException) {
2208                 maxWaitTime = EnvironmentEdgeManager.currentTime()
2209                   + this.server.getConfiguration().getLong(ALREADY_IN_TRANSITION_WAITTIME,
2210                     DEFAULT_ALREADY_IN_TRANSITION_WAITTIME);
2211               } else {
2212                 maxWaitTime = EnvironmentEdgeManager.currentTime()
2213                   + this.server.getConfiguration().getLong(
2214                     "hbase.regionserver.rpc.startup.waittime", 60000);
2215               }
2216             }
2217             try {
2218               needNewPlan = false;
2219               long now = EnvironmentEdgeManager.currentTime();
2220               if (now < maxWaitTime) {
2221                 LOG.debug("Server is not yet up or region is already in transition; "
2222                   + "waiting up to " + (maxWaitTime - now) + "ms", t);
2223                 Thread.sleep(100);
2224                 i--; // reset the try count
2225               } else if (!(t instanceof RegionAlreadyInTransitionException)) {
2226                 LOG.debug("Server is not up for a while; try a new one", t);
2227                 needNewPlan = true;
2228               }
2229             } catch (InterruptedException ie) {
2230               LOG.warn("Failed to assign "
2231                   + region.getRegionNameAsString() + " since interrupted", ie);
2232               regionStates.updateRegionState(region, State.FAILED_OPEN);
2233               Thread.currentThread().interrupt();
2234               return;
2235             }
2236           } else if (retry) {
2237             needNewPlan = false;
2238             i--; // we want to retry as many times as needed as long as the RS is not dead.
2239             LOG.warn(assignMsg + ", trying to assign to the same region server due ", t);
2240           } else {
2241             needNewPlan = true;
2242             LOG.warn(assignMsg + ", trying to assign elsewhere instead;" +
2243                 " try=" + i + " of " + this.maximumAttempts, t);
2244           }
2245         }
2246 
2247         if (i == this.maximumAttempts) {
2248           // For meta region, we have to keep retrying until succeeding
2249           if (region.isMetaRegion()) {
2250             i = 0; // re-set attempt count to 0 for at least 1 retry
2251             LOG.warn(assignMsg +
2252                 ", trying to assign a hbase:meta region reached to maximumAttempts (" +
2253                 this.maximumAttempts + ").  Reset attempt counts and continue retrying.");
2254             waitForRetryingMetaAssignment();
2255           }
2256           else {
2257             // Don't reset the region state or get a new plan any more.
2258             // This is the last try.
2259             continue;
2260           }
2261         }
2262 
2263         // If region opened on destination of present plan, reassigning to new
2264         // RS may cause double assignments. In case of RegionAlreadyInTransitionException
2265         // reassigning to same RS.
2266         if (needNewPlan) {
2267           // Force a new plan and reassign. Will return null if no servers.
2268           // The new plan could be the same as the existing plan since we don't
2269           // exclude the server of the original plan, which should not be
2270           // excluded since it could be the only server up now.
2271           RegionPlan newPlan = null;
2272           try {
2273             newPlan = getRegionPlan(region, true);
2274           } catch (HBaseIOException e) {
2275             LOG.warn("Failed to get region plan", e);
2276           }
2277           if (newPlan == null) {
2278             regionStates.updateRegionState(region, State.FAILED_OPEN);
2279             LOG.warn("Unable to find a viable location to assign region " +
2280                 region.getRegionNameAsString());
2281             return;
2282           }
2283 
2284           if (plan != newPlan && !plan.getDestination().equals(newPlan.getDestination())) {
2285             // Clean out plan we failed execute and one that doesn't look like it'll
2286             // succeed anyways; we need a new plan!
2287             // Transition back to OFFLINE
2288             LOG.info("Region assignment plan changed from " + plan.getDestination() + " to "
2289                 + newPlan.getDestination() + " server.");
2290             currentState = regionStates.updateRegionState(region, State.OFFLINE);
2291             versionOfOfflineNode = -1;
2292             if (useZKForAssignment) {
2293               setOfflineInZK = true;
2294             }
2295             plan = newPlan;
2296           } else if(plan.getDestination().equals(newPlan.getDestination()) &&
2297               previousException instanceof FailedServerException) {
2298             try {
2299               LOG.info("Trying to re-assign " + region.getRegionNameAsString() +
2300                 " to the same failed server.");
2301               Thread.sleep(1 + conf.getInt(RpcClient.FAILED_SERVER_EXPIRY_KEY,
2302                 RpcClient.FAILED_SERVER_EXPIRY_DEFAULT));
2303             } catch (InterruptedException ie) {
2304               LOG.warn("Failed to assign "
2305                   + region.getRegionNameAsString() + " since interrupted", ie);
2306               regionStates.updateRegionState(region, State.FAILED_OPEN);
2307               Thread.currentThread().interrupt();
2308               return;
2309             }
2310           }
2311         }
2312       }
2313       // Run out of attempts
2314       regionStates.updateRegionState(region, State.FAILED_OPEN);
2315     } finally {
2316       metricsAssignmentManager.updateAssignmentTime(EnvironmentEdgeManager.currentTime() - startTime);
2317     }
2318   }
2319 
2320   private void processAlreadyOpenedRegion(HRegionInfo region, ServerName sn) {
2321     // Remove region from in-memory transition and unassigned node from ZK
2322     // While trying to enable the table the regions of the table were
2323     // already enabled.
2324     LOG.debug("ALREADY_OPENED " + region.getRegionNameAsString()
2325       + " to " + sn);
2326     String encodedName = region.getEncodedName();
2327     
2328     //If use ZkForAssignment, region already Opened event should not be handled, 
2329     //leave it to zk event. See HBase-14407.
2330     if(useZKForAssignment){
2331       String node = ZKAssign.getNodeName(watcher, encodedName);
2332       Stat stat = new Stat();
2333       try {
2334         byte[] existingBytes = ZKUtil.getDataNoWatch(watcher, node, stat);
2335         if(existingBytes!=null){
2336           RegionTransition rt= RegionTransition.parseFrom(existingBytes);
2337           EventType et = rt.getEventType();
2338           if (et.equals(EventType.RS_ZK_REGION_OPENED)) {
2339             LOG.debug("ALREADY_OPENED " + region.getRegionNameAsString()
2340               + " and node in "+et+" state");
2341             return;
2342           }
2343         }
2344       } catch (KeeperException ke) {
2345         LOG.warn("Unexpected ZK exception getData " + node
2346           + " node for the region " + encodedName, ke);
2347       } catch (DeserializationException e) {
2348         LOG.warn("Get RegionTransition from zk deserialization failed! ", e);
2349       }
2350       
2351       deleteNodeInStates(encodedName, "offline", sn, EventType.M_ZK_REGION_OFFLINE);
2352     }
2353     
2354     regionStates.regionOnline(region, sn);
2355   }
2356 
2357   private boolean isDisabledorDisablingRegionInRIT(final HRegionInfo region) {
2358     if (this.tableStateManager.isTableState(region.getTable(),
2359         ZooKeeperProtos.Table.State.DISABLED,
2360         ZooKeeperProtos.Table.State.DISABLING) || replicasToClose.contains(region)) {
2361       LOG.info("Table " + region.getTable() + " is disabled or disabling;"
2362         + " skipping assign of " + region.getRegionNameAsString());
2363       offlineDisabledRegion(region);
2364       return true;
2365     }
2366     return false;
2367   }
2368 
2369   /**
2370    * Set region as OFFLINED up in zookeeper
2371    *
2372    * @param state
2373    * @return the version of the offline node if setting of the OFFLINE node was
2374    *         successful, -1 otherwise.
2375    */
2376   private int setOfflineInZooKeeper(final RegionState state, final ServerName destination) {
2377     if (!state.isClosed() && !state.isOffline()) {
2378       String msg = "Unexpected state : " + state + " .. Cannot transit it to OFFLINE.";
2379       this.server.abort(msg, new IllegalStateException(msg));
2380       return -1;
2381     }
2382     regionStates.updateRegionState(state.getRegion(), State.OFFLINE);
2383     int versionOfOfflineNode;
2384     try {
2385       // get the version after setting the znode to OFFLINE
2386       versionOfOfflineNode = ZKAssign.createOrForceNodeOffline(watcher,
2387         state.getRegion(), destination);
2388       if (versionOfOfflineNode == -1) {
2389         LOG.warn("Attempted to create/force node into OFFLINE state before "
2390             + "completing assignment but failed to do so for " + state);
2391         return -1;
2392       }
2393     } catch (KeeperException e) {
2394       server.abort("Unexpected ZK exception creating/setting node OFFLINE", e);
2395       return -1;
2396     }
2397     return versionOfOfflineNode;
2398   }
2399 
2400   /**
2401    * @param region the region to assign
2402    * @return Plan for passed <code>region</code> (If none currently, it creates one or
2403    * if no servers to assign, it returns null).
2404    */
2405   private RegionPlan getRegionPlan(final HRegionInfo region,
2406       final boolean forceNewPlan)  throws HBaseIOException {
2407     return getRegionPlan(region, null, forceNewPlan);
2408   }
2409 
2410   /**
2411    * @param region the region to assign
2412    * @param serverToExclude Server to exclude (we know its bad). Pass null if
2413    * all servers are thought to be assignable.
2414    * @param forceNewPlan If true, then if an existing plan exists, a new plan
2415    * will be generated.
2416    * @return Plan for passed <code>region</code> (If none currently, it creates one or
2417    * if no servers to assign, it returns null).
2418    */
2419   private RegionPlan getRegionPlan(final HRegionInfo region,
2420       final ServerName serverToExclude, final boolean forceNewPlan) throws HBaseIOException {
2421     // Pickup existing plan or make a new one
2422     final String encodedName = region.getEncodedName();
2423     final List<ServerName> destServers =
2424       serverManager.createDestinationServersList(serverToExclude);
2425 
2426     if (destServers.isEmpty()){
2427       LOG.warn("Can't move " + encodedName +
2428         ", there is no destination server available.");
2429       return null;
2430     }
2431 
2432     RegionPlan randomPlan = null;
2433     boolean newPlan = false;
2434     RegionPlan existingPlan;
2435 
2436     synchronized (this.regionPlans) {
2437       existingPlan = this.regionPlans.get(encodedName);
2438 
2439       if (existingPlan != null && existingPlan.getDestination() != null) {
2440         LOG.debug("Found an existing plan for " + region.getRegionNameAsString()
2441           + " destination server is " + existingPlan.getDestination() +
2442             " accepted as a dest server = " + destServers.contains(existingPlan.getDestination()));
2443       }
2444 
2445       if (forceNewPlan
2446           || existingPlan == null
2447           || existingPlan.getDestination() == null
2448           || !destServers.contains(existingPlan.getDestination())) {
2449         newPlan = true;
2450       }
2451     }
2452 
2453     if (newPlan) {
2454       ServerName destination = balancer.randomAssignment(region, destServers);
2455       if (destination == null) {
2456         LOG.warn("Can't find a destination for " + encodedName);
2457         return null;
2458       }
2459       synchronized (this.regionPlans) {
2460         randomPlan = new RegionPlan(region, null, destination);
2461         if (!region.isMetaTable() && shouldAssignRegionsWithFavoredNodes) {
2462           List<HRegionInfo> regions = new ArrayList<HRegionInfo>(1);
2463           regions.add(region);
2464           try {
2465             processFavoredNodes(regions);
2466           } catch (IOException ie) {
2467             LOG.warn("Ignoring exception in processFavoredNodes " + ie);
2468           }
2469         }
2470         this.regionPlans.put(encodedName, randomPlan);
2471       }
2472       LOG.debug("No previous transition plan found (or ignoring " + "an existing plan) for "
2473           + region.getRegionNameAsString() + "; generated random plan=" + randomPlan + "; "
2474           + destServers.size() + " (online=" + serverManager.getOnlineServers().size()
2475           + ") available servers, forceNewPlan=" + forceNewPlan);
2476       return randomPlan;
2477     }
2478     LOG.debug("Using pre-existing plan for " +
2479       region.getRegionNameAsString() + "; plan=" + existingPlan);
2480     return existingPlan;
2481   }
2482 
2483   /**
2484    * Wait for some time before retrying meta table region assignment
2485    */
2486   private void waitForRetryingMetaAssignment() {
2487     try {
2488       Thread.sleep(this.sleepTimeBeforeRetryingMetaAssignment);
2489     } catch (InterruptedException e) {
2490       LOG.error("Got exception while waiting for hbase:meta assignment");
2491       Thread.currentThread().interrupt();
2492     }
2493   }
2494 
2495   /**
2496    * Unassigns the specified region.
2497    * <p>
2498    * Updates the RegionState and sends the CLOSE RPC unless region is being
2499    * split by regionserver; then the unassign fails (silently) because we
2500    * presume the region being unassigned no longer exists (its been split out
2501    * of existence). TODO: What to do if split fails and is rolled back and
2502    * parent is revivified?
2503    * <p>
2504    * If a RegionPlan is already set, it will remain.
2505    *
2506    * @param region server to be unassigned
2507    */
2508   public void unassign(HRegionInfo region) {
2509     unassign(region, false);
2510   }
2511 
2512 
2513   /**
2514    * Unassigns the specified region.
2515    * <p>
2516    * Updates the RegionState and sends the CLOSE RPC unless region is being
2517    * split by regionserver; then the unassign fails (silently) because we
2518    * presume the region being unassigned no longer exists (its been split out
2519    * of existence). TODO: What to do if split fails and is rolled back and
2520    * parent is revivified?
2521    * <p>
2522    * If a RegionPlan is already set, it will remain.
2523    *
2524    * @param region server to be unassigned
2525    * @param force if region should be closed even if already closing
2526    */
2527   public void unassign(HRegionInfo region, boolean force, ServerName dest) {
2528     // TODO: Method needs refactoring.  Ugly buried returns throughout.  Beware!
2529     LOG.debug("Starting unassign of " + region.getRegionNameAsString()
2530       + " (offlining), current state: " + regionStates.getRegionState(region));
2531 
2532     String encodedName = region.getEncodedName();
2533     // Grab the state of this region and synchronize on it
2534     int versionOfClosingNode = -1;
2535     // We need a lock here as we're going to do a put later and we don't want multiple states
2536     //  creation
2537     ReentrantLock lock = locker.acquireLock(encodedName);
2538     RegionState state = regionStates.getRegionTransitionState(encodedName);
2539     boolean reassign = true;
2540     try {
2541       if (state == null) {
2542         // Region is not in transition.
2543         // We can unassign it only if it's not SPLIT/MERGED.
2544         state = regionStates.getRegionState(encodedName);
2545         if (state != null && state.isUnassignable()) {
2546           LOG.info("Attempting to unassign " + state + ", ignored");
2547           // Offline region will be reassigned below
2548           return;
2549         }
2550         // Create the znode in CLOSING state
2551         try {
2552           if (state == null || state.getServerName() == null) {
2553             // We don't know where the region is, offline it.
2554             // No need to send CLOSE RPC
2555             LOG.warn("Attempting to unassign a region not in RegionStates "
2556               + region.getRegionNameAsString() + ", offlined");
2557             regionOffline(region);
2558             return;
2559           }
2560           if (useZKForAssignment) {
2561             versionOfClosingNode = ZKAssign.createNodeClosing(
2562               watcher, region, state.getServerName());
2563             if (versionOfClosingNode == -1) {
2564               LOG.info("Attempting to unassign " +
2565                 region.getRegionNameAsString() + " but ZK closing node "
2566                 + "can't be created.");
2567               reassign = false; // not unassigned at all
2568               return;
2569             }
2570           }
2571         } catch (KeeperException e) {
2572           if (e instanceof NodeExistsException) {
2573             // Handle race between master initiated close and regionserver
2574             // orchestrated splitting. See if existing node is in a
2575             // SPLITTING or SPLIT state.  If so, the regionserver started
2576             // an op on node before we could get our CLOSING in.  Deal.
2577             NodeExistsException nee = (NodeExistsException)e;
2578             String path = nee.getPath();
2579             try {
2580               if (isSplitOrSplittingOrMergedOrMerging(path)) {
2581                 LOG.debug(path + " is SPLIT or SPLITTING or MERGED or MERGING; " +
2582                   "skipping unassign because region no longer exists -- its split or merge");
2583                 reassign = false; // no need to reassign for split/merged region
2584                 return;
2585               }
2586             } catch (KeeperException.NoNodeException ke) {
2587               LOG.warn("Failed getData on SPLITTING/SPLIT at " + path +
2588                 "; presuming split and that the region to unassign, " +
2589                 encodedName + ", no longer exists -- confirm", ke);
2590               return;
2591             } catch (KeeperException ke) {
2592               LOG.error("Unexpected zk state", ke);
2593             } catch (DeserializationException de) {
2594               LOG.error("Failed parse", de);
2595             }
2596           }
2597           // If we get here, don't understand whats going on -- abort.
2598           server.abort("Unexpected ZK exception creating node CLOSING", e);
2599           reassign = false; // heading out already
2600           return;
2601         }
2602         state = regionStates.updateRegionState(region, State.PENDING_CLOSE);
2603       } else if (state.isFailedOpen()) {
2604         // The region is not open yet
2605         regionOffline(region);
2606         return;
2607       } else if (force && state.isPendingCloseOrClosing()) {
2608         LOG.debug("Attempting to unassign " + region.getRegionNameAsString() +
2609           " which is already " + state.getState()  +
2610           " but forcing to send a CLOSE RPC again ");
2611         if (state.isFailedClose()) {
2612           state = regionStates.updateRegionState(region, State.PENDING_CLOSE);
2613         }
2614         state.updateTimestampToNow();
2615       } else {
2616         LOG.debug("Attempting to unassign " +
2617           region.getRegionNameAsString() + " but it is " +
2618           "already in transition (" + state.getState() + ", force=" + force + ")");
2619         return;
2620       }
2621 
2622       unassign(region, state, versionOfClosingNode, dest, useZKForAssignment, null);
2623     } finally {
2624       lock.unlock();
2625 
2626       // Region is expected to be reassigned afterwards
2627       if (!replicasToClose.contains(region) && reassign && regionStates.isRegionOffline(region)) {
2628         assign(region, true);
2629       }
2630     }
2631   }
2632 
2633   public void unassign(HRegionInfo region, boolean force){
2634      unassign(region, force, null);
2635   }
2636 
2637   /**
2638    * @param region regioninfo of znode to be deleted.
2639    */
2640   public void deleteClosingOrClosedNode(HRegionInfo region, ServerName sn) {
2641     String encodedName = region.getEncodedName();
2642     deleteNodeInStates(encodedName, "closing", sn, EventType.M_ZK_REGION_CLOSING,
2643       EventType.RS_ZK_REGION_CLOSED);
2644   }
2645 
2646   /**
2647    * @param path
2648    * @return True if znode is in SPLIT or SPLITTING or MERGED or MERGING state.
2649    * @throws KeeperException Can happen if the znode went away in meantime.
2650    * @throws DeserializationException
2651    */
2652   private boolean isSplitOrSplittingOrMergedOrMerging(final String path)
2653       throws KeeperException, DeserializationException {
2654     boolean result = false;
2655     // This may fail if the SPLIT or SPLITTING or MERGED or MERGING znode gets
2656     // cleaned up before we can get data from it.
2657     byte [] data = ZKAssign.getData(watcher, path);
2658     if (data == null) {
2659       LOG.info("Node " + path + " is gone");
2660       return false;
2661     }
2662     RegionTransition rt = RegionTransition.parseFrom(data);
2663     switch (rt.getEventType()) {
2664     case RS_ZK_REQUEST_REGION_SPLIT:
2665     case RS_ZK_REGION_SPLIT:
2666     case RS_ZK_REGION_SPLITTING:
2667     case RS_ZK_REQUEST_REGION_MERGE:
2668     case RS_ZK_REGION_MERGED:
2669     case RS_ZK_REGION_MERGING:
2670       result = true;
2671       break;
2672     default:
2673       LOG.info("Node " + path + " is in " + rt.getEventType());
2674       break;
2675     }
2676     return result;
2677   }
2678 
2679   /**
2680    * Used by unit tests. Return the number of regions opened so far in the life
2681    * of the master. Increases by one every time the master opens a region
2682    * @return the counter value of the number of regions opened so far
2683    */
2684   public int getNumRegionsOpened() {
2685     return numRegionsOpened.get();
2686   }
2687 
2688   /**
2689    * Waits until the specified region has completed assignment.
2690    * <p>
2691    * If the region is already assigned, returns immediately.  Otherwise, method
2692    * blocks until the region is assigned.
2693    * @param regionInfo region to wait on assignment for
2694    * @return true if the region is assigned false otherwise.
2695    * @throws InterruptedException
2696    */
2697   public boolean waitForAssignment(HRegionInfo regionInfo)
2698       throws InterruptedException {
2699     ArrayList<HRegionInfo> regionSet = new ArrayList<HRegionInfo>(1);
2700     regionSet.add(regionInfo);
2701     return waitForAssignment(regionSet, true, Long.MAX_VALUE);
2702   }
2703 
2704   /**
2705    * Waits until the specified region has completed assignment, or the deadline is reached.
2706    */
2707   protected boolean waitForAssignment(final Collection<HRegionInfo> regionSet,
2708       final boolean waitTillAllAssigned, final int reassigningRegions,
2709       final long minEndTime) throws InterruptedException {
2710     long deadline = minEndTime + bulkPerRegionOpenTimeGuesstimate * (reassigningRegions + 1);
2711     if (deadline < 0) { // Overflow
2712       deadline = Long.MAX_VALUE; // wait forever
2713     }
2714     return waitForAssignment(regionSet, waitTillAllAssigned, deadline);
2715   }
2716 
2717   /**
2718    * Waits until the specified region has completed assignment, or the deadline is reached.
2719    * @param regionSet set of region to wait on. the set is modified and the assigned regions removed
2720    * @param waitTillAllAssigned true if we should wait all the regions to be assigned
2721    * @param deadline the timestamp after which the wait is aborted
2722    * @return true if all the regions are assigned false otherwise.
2723    * @throws InterruptedException
2724    */
2725   protected boolean waitForAssignment(final Collection<HRegionInfo> regionSet,
2726       final boolean waitTillAllAssigned, final long deadline) throws InterruptedException {
2727     // We're not synchronizing on regionsInTransition now because we don't use any iterator.
2728     while (!regionSet.isEmpty() && !server.isStopped() && deadline > System.currentTimeMillis()) {
2729       int failedOpenCount = 0;
2730       Iterator<HRegionInfo> regionInfoIterator = regionSet.iterator();
2731       while (regionInfoIterator.hasNext()) {
2732         HRegionInfo hri = regionInfoIterator.next();
2733         if (regionStates.isRegionOnline(hri) || regionStates.isRegionInState(hri,
2734             State.SPLITTING, State.SPLIT, State.MERGING, State.MERGED)) {
2735           regionInfoIterator.remove();
2736         } else if (regionStates.isRegionInState(hri, State.FAILED_OPEN)) {
2737           failedOpenCount++;
2738         }
2739       }
2740       if (!waitTillAllAssigned) {
2741         // No need to wait, let assignment going on asynchronously
2742         break;
2743       }
2744       if (!regionSet.isEmpty()) {
2745         if (failedOpenCount == regionSet.size()) {
2746           // all the regions we are waiting had an error on open.
2747           break;
2748         }
2749         regionStates.waitForUpdate(100);
2750       }
2751     }
2752     return regionSet.isEmpty();
2753   }
2754 
2755   /**
2756    * Assigns the hbase:meta region or a replica.
2757    * <p>
2758    * Assumes that hbase:meta is currently closed and is not being actively served by
2759    * any RegionServer.
2760    * <p>
2761    * Forcibly unsets the current meta region location in ZooKeeper and assigns
2762    * hbase:meta to a random RegionServer.
2763    * @param hri TODO
2764    * @throws KeeperException
2765    */
2766   public void assignMeta(HRegionInfo hri) throws KeeperException {
2767     this.server.getMetaTableLocator().deleteMetaLocation(this.watcher, hri.getReplicaId());
2768     assign(hri, true);
2769   }
2770 
2771   /**
2772    * Assigns specified regions retaining assignments, if any.
2773    * <p>
2774    * This is a synchronous call and will return once every region has been
2775    * assigned.  If anything fails, an exception is thrown
2776    * @throws InterruptedException
2777    * @throws IOException
2778    */
2779   public void assign(Map<HRegionInfo, ServerName> regions)
2780         throws IOException, InterruptedException {
2781     if (regions == null || regions.isEmpty()) {
2782       return;
2783     }
2784     List<ServerName> servers = serverManager.createDestinationServersList();
2785     if (servers == null || servers.isEmpty()) {
2786       throw new IOException("Found no destination server to assign region(s)");
2787     }
2788 
2789     // Reuse existing assignment info
2790     Map<ServerName, List<HRegionInfo>> bulkPlan =
2791       balancer.retainAssignment(regions, servers);
2792     if (bulkPlan == null) {
2793       throw new IOException("Unable to determine a plan to assign region(s)");
2794     }
2795 
2796     assign(regions.size(), servers.size(),
2797       "retainAssignment=true", bulkPlan);
2798   }
2799 
2800   /**
2801    * Assigns specified regions round robin, if any.
2802    * <p>
2803    * This is a synchronous call and will return once every region has been
2804    * assigned.  If anything fails, an exception is thrown
2805    * @throws InterruptedException
2806    * @throws IOException
2807    */
2808   public void assign(List<HRegionInfo> regions)
2809         throws IOException, InterruptedException {
2810     if (regions == null || regions.isEmpty()) {
2811       return;
2812     }
2813 
2814     List<ServerName> servers = serverManager.createDestinationServersList();
2815     if (servers == null || servers.isEmpty()) {
2816       throw new IOException("Found no destination server to assign region(s)");
2817     }
2818 
2819     // Generate a round-robin bulk assignment plan
2820     Map<ServerName, List<HRegionInfo>> bulkPlan = balancer.roundRobinAssignment(regions, servers);
2821     if (bulkPlan == null) {
2822       throw new IOException("Unable to determine a plan to assign region(s)");
2823     }
2824 
2825     processFavoredNodes(regions);
2826     assign(regions.size(), servers.size(), "round-robin=true", bulkPlan);
2827   }
2828 
2829   private void assign(int regions, int totalServers,
2830       String message, Map<ServerName, List<HRegionInfo>> bulkPlan)
2831           throws InterruptedException, IOException {
2832 
2833     int servers = bulkPlan.size();
2834     if (servers == 1 || (regions < bulkAssignThresholdRegions
2835         && servers < bulkAssignThresholdServers)) {
2836 
2837       // Not use bulk assignment.  This could be more efficient in small
2838       // cluster, especially mini cluster for testing, so that tests won't time out
2839       if (LOG.isTraceEnabled()) {
2840         LOG.trace("Not using bulk assignment since we are assigning only " + regions +
2841           " region(s) to " + servers + " server(s)");
2842       }
2843 
2844       // invoke assignment (async)
2845       ArrayList<HRegionInfo> userRegionSet = new ArrayList<HRegionInfo>(regions);
2846       for (Map.Entry<ServerName, List<HRegionInfo>> plan: bulkPlan.entrySet()) {
2847         if (!assign(plan.getKey(), plan.getValue())) {
2848           for (HRegionInfo region: plan.getValue()) {
2849             if (!regionStates.isRegionOnline(region)) {
2850               invokeAssign(region);
2851               if (!region.getTable().isSystemTable()) {
2852                 userRegionSet.add(region);
2853               }
2854             }
2855           }
2856         }
2857       }
2858 
2859       // wait for assignment completion
2860       if (!waitForAssignment(userRegionSet, true, userRegionSet.size(),
2861             System.currentTimeMillis())) {
2862         LOG.debug("some user regions are still in transition: " + userRegionSet);
2863       }
2864     } else {
2865       LOG.info("Bulk assigning " + regions + " region(s) across "
2866         + totalServers + " server(s), " + message);
2867 
2868       // Use fixed count thread pool assigning.
2869       BulkAssigner ba = new GeneralBulkAssigner(
2870         this.server, bulkPlan, this, bulkAssignWaitTillAllAssigned);
2871       ba.bulkAssign();
2872       LOG.info("Bulk assigning done");
2873     }
2874   }
2875 
2876   /**
2877    * Assigns all user regions, if any exist.  Used during cluster startup.
2878    * <p>
2879    * This is a synchronous call and will return once every region has been
2880    * assigned.  If anything fails, an exception is thrown and the cluster
2881    * should be shutdown.
2882    * @throws InterruptedException
2883    * @throws IOException
2884    */
2885   private void assignAllUserRegions(Map<HRegionInfo, ServerName> allRegions)
2886       throws IOException, InterruptedException {
2887     if (allRegions == null || allRegions.isEmpty()) return;
2888 
2889     // Determine what type of assignment to do on startup
2890     boolean retainAssignment = server.getConfiguration().
2891       getBoolean("hbase.master.startup.retainassign", true);
2892 
2893     Set<HRegionInfo> regionsFromMetaScan = allRegions.keySet();
2894     if (retainAssignment) {
2895       assign(allRegions);
2896     } else {
2897       List<HRegionInfo> regions = new ArrayList<HRegionInfo>(regionsFromMetaScan);
2898       assign(regions);
2899     }
2900 
2901     for (HRegionInfo hri : regionsFromMetaScan) {
2902       TableName tableName = hri.getTable();
2903       if (!tableStateManager.isTableState(tableName,
2904           ZooKeeperProtos.Table.State.ENABLED)) {
2905         setEnabledTable(tableName);
2906       }
2907     }
2908     // assign all the replicas that were not recorded in the meta
2909     assign(replicaRegionsNotRecordedInMeta(regionsFromMetaScan, server));
2910   }
2911 
2912   /**
2913    * Get a list of replica regions that are:
2914    * not recorded in meta yet. We might not have recorded the locations
2915    * for the replicas since the replicas may not have been online yet, master restarted
2916    * in the middle of assigning, ZK erased, etc.
2917    * @param regionsRecordedInMeta the list of regions we know are recorded in meta
2918    * either as a default, or, as the location of a replica
2919    * @param master
2920    * @return list of replica regions
2921    * @throws IOException
2922    */
2923   public static List<HRegionInfo> replicaRegionsNotRecordedInMeta(
2924       Set<HRegionInfo> regionsRecordedInMeta, MasterServices master)throws IOException {
2925     List<HRegionInfo> regionsNotRecordedInMeta = new ArrayList<HRegionInfo>();
2926     for (HRegionInfo hri : regionsRecordedInMeta) {
2927       TableName table = hri.getTable();
2928       HTableDescriptor htd = master.getTableDescriptors().get(table);
2929       // look at the HTD for the replica count. That's the source of truth
2930       int desiredRegionReplication = htd.getRegionReplication();
2931       for (int i = 0; i < desiredRegionReplication; i++) {
2932         HRegionInfo replica = RegionReplicaUtil.getRegionInfoForReplica(hri, i);
2933         if (regionsRecordedInMeta.contains(replica)) continue;
2934         regionsNotRecordedInMeta.add(replica);
2935       }
2936     }
2937     return regionsNotRecordedInMeta;
2938   }
2939 
2940   /**
2941    * Wait until no regions in transition.
2942    * @param timeout How long to wait.
2943    * @return True if nothing in regions in transition.
2944    * @throws InterruptedException
2945    */
2946   boolean waitUntilNoRegionsInTransition(final long timeout)
2947       throws InterruptedException {
2948     // Blocks until there are no regions in transition. It is possible that
2949     // there
2950     // are regions in transition immediately after this returns but guarantees
2951     // that if it returns without an exception that there was a period of time
2952     // with no regions in transition from the point-of-view of the in-memory
2953     // state of the Master.
2954     final long endTime = System.currentTimeMillis() + timeout;
2955 
2956     while (!this.server.isStopped() && regionStates.isRegionsInTransition()
2957         && endTime > System.currentTimeMillis()) {
2958       regionStates.waitForUpdate(100);
2959     }
2960 
2961     return !regionStates.isRegionsInTransition();
2962   }
2963 
2964   /**
2965    * Rebuild the list of user regions and assignment information.
2966    * Updates regionstates with findings as we go through list of regions.
2967    * @return set of servers not online that hosted some regions according to a scan of hbase:meta
2968    * @throws IOException
2969    */
2970   Set<ServerName> rebuildUserRegions() throws
2971       IOException, KeeperException, CoordinatedStateException {
2972     Set<TableName> disabledOrEnablingTables = tableStateManager.getTablesInStates(
2973       ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.ENABLING);
2974 
2975     Set<TableName> disabledOrDisablingOrEnabling = tableStateManager.getTablesInStates(
2976       ZooKeeperProtos.Table.State.DISABLED,
2977       ZooKeeperProtos.Table.State.DISABLING,
2978       ZooKeeperProtos.Table.State.ENABLING);
2979 
2980     // Region assignment from META
2981     List<Result> results = MetaTableAccessor.fullScanOfMeta(server.getConnection());
2982     // Get any new but slow to checkin region server that joined the cluster
2983     Set<ServerName> onlineServers = serverManager.getOnlineServers().keySet();
2984     // Set of offline servers to be returned
2985     Set<ServerName> offlineServers = new HashSet<ServerName>();
2986     // Iterate regions in META
2987     for (Result result : results) {
2988       if (result == null && LOG.isDebugEnabled()){
2989         LOG.debug("null result from meta - ignoring but this is strange.");
2990         continue;
2991       }
2992       // keep a track of replicas to close. These were the replicas of the originally
2993       // unmerged regions. The master might have closed them before but it mightn't
2994       // maybe because it crashed.
2995       PairOfSameType<HRegionInfo> p = MetaTableAccessor.getMergeRegions(result);
2996       if (p.getFirst() != null && p.getSecond() != null) {
2997         int numReplicas = server.getTableDescriptors().get(p.getFirst().
2998             getTable()).getRegionReplication();
2999         for (HRegionInfo merge : p) {
3000           for (int i = 1; i < numReplicas; i++) {
3001             replicasToClose.add(RegionReplicaUtil.getRegionInfoForReplica(merge, i));
3002           }
3003         }
3004       }
3005       RegionLocations rl =  MetaTableAccessor.getRegionLocations(result);
3006       if (rl == null) continue;
3007       HRegionLocation[] locations = rl.getRegionLocations();
3008       if (locations == null) continue;
3009       for (HRegionLocation hrl : locations) {
3010         if (hrl == null) continue;
3011         HRegionInfo regionInfo = hrl.getRegionInfo();
3012         if (regionInfo == null) continue;
3013         int replicaId = regionInfo.getReplicaId();
3014         State state = RegionStateStore.getRegionState(result, replicaId);
3015         // keep a track of replicas to close. These were the replicas of the split parents
3016         // from the previous life of the master. The master should have closed them before
3017         // but it couldn't maybe because it crashed
3018         if (replicaId == 0 && state.equals(State.SPLIT)) {
3019           for (HRegionLocation h : locations) {
3020             replicasToClose.add(h.getRegionInfo());
3021           }
3022         }
3023         ServerName lastHost = hrl.getServerName();
3024         ServerName regionLocation = RegionStateStore.getRegionServer(result, replicaId);
3025         if (tableStateManager.isTableState(regionInfo.getTable(),
3026              ZooKeeperProtos.Table.State.DISABLED)) {
3027           // force region to forget it hosts for disabled/disabling tables.
3028           // see HBASE-13326
3029           lastHost = null;
3030           regionLocation = null;
3031         }
3032         regionStates.createRegionState(regionInfo, state, regionLocation, lastHost);
3033         if (!regionStates.isRegionInState(regionInfo, State.OPEN)) {
3034           // Region is not open (either offline or in transition), skip
3035           continue;
3036         }
3037         TableName tableName = regionInfo.getTable();
3038         if (!onlineServers.contains(regionLocation)) {
3039           // Region is located on a server that isn't online
3040           offlineServers.add(regionLocation);
3041           if (useZKForAssignment) {
3042             regionStates.regionOffline(regionInfo);
3043           }
3044         } else if (!disabledOrEnablingTables.contains(tableName)) {
3045           // Region is being served and on an active server
3046           // add only if region not in disabled or enabling table
3047           regionStates.regionOnline(regionInfo, regionLocation);
3048           balancer.regionOnline(regionInfo, regionLocation);
3049         } else if (useZKForAssignment) {
3050           regionStates.regionOffline(regionInfo);
3051         }
3052         // need to enable the table if not disabled or disabling or enabling
3053         // this will be used in rolling restarts
3054         if (!disabledOrDisablingOrEnabling.contains(tableName)
3055           && !getTableStateManager().isTableState(tableName,
3056             ZooKeeperProtos.Table.State.ENABLED)) {
3057           setEnabledTable(tableName);
3058         }
3059       }
3060     }
3061     return offlineServers;
3062   }
3063 
3064   /**
3065    * Recover the tables that were not fully moved to DISABLED state. These
3066    * tables are in DISABLING state when the master restarted/switched.
3067    *
3068    * @throws KeeperException
3069    * @throws TableNotFoundException
3070    * @throws IOException
3071    */
3072   private void recoverTableInDisablingState()
3073       throws KeeperException, IOException, CoordinatedStateException {
3074     Set<TableName> disablingTables =
3075       tableStateManager.getTablesInStates(ZooKeeperProtos.Table.State.DISABLING);
3076     if (disablingTables.size() != 0) {
3077       for (TableName tableName : disablingTables) {
3078         // Recover by calling DisableTableHandler
3079         LOG.info("The table " + tableName
3080             + " is in DISABLING state.  Hence recovering by moving the table"
3081             + " to DISABLED state.");
3082         new DisableTableHandler(this.server, tableName,
3083             this, tableLockManager, true).prepare().process();
3084       }
3085     }
3086   }
3087 
3088   /**
3089    * Recover the tables that are not fully moved to ENABLED state. These tables
3090    * are in ENABLING state when the master restarted/switched
3091    *
3092    * @throws KeeperException
3093    * @throws org.apache.hadoop.hbase.TableNotFoundException
3094    * @throws IOException
3095    */
3096   private void recoverTableInEnablingState()
3097       throws KeeperException, IOException, CoordinatedStateException {
3098     Set<TableName> enablingTables = tableStateManager.
3099       getTablesInStates(ZooKeeperProtos.Table.State.ENABLING);
3100     if (enablingTables.size() != 0) {
3101       for (TableName tableName : enablingTables) {
3102         // Recover by calling EnableTableHandler
3103         LOG.info("The table " + tableName
3104             + " is in ENABLING state.  Hence recovering by moving the table"
3105             + " to ENABLED state.");
3106         // enableTable in sync way during master startup,
3107         // no need to invoke coprocessor
3108         EnableTableHandler eth = new EnableTableHandler(this.server, tableName,
3109           this, tableLockManager, true);
3110         try {
3111           eth.prepare();
3112         } catch (TableNotFoundException e) {
3113           LOG.warn("Table " + tableName + " not found in hbase:meta to recover.");
3114           continue;
3115         }
3116         eth.process();
3117       }
3118     }
3119   }
3120 
3121   /**
3122    * Processes list of dead servers from result of hbase:meta scan and regions in RIT.
3123    * This is used for failover to recover the lost regions that belonged to
3124    * RegionServers which failed while there was no active master or are offline for whatever
3125    * reason and for regions that were in RIT.
3126    *
3127    * @param deadServers
3128    *          The list of dead servers which failed while there was no active master. Can be null.
3129    * @throws IOException
3130    * @throws KeeperException
3131    */
3132   private void processDeadServersAndRecoverLostRegions(Set<ServerName> deadServers)
3133   throws IOException, KeeperException {
3134     if (deadServers != null && !deadServers.isEmpty()) {
3135       for (ServerName serverName: deadServers) {
3136         if (!serverManager.isServerDead(serverName)) {
3137           serverManager.expireServer(serverName); // Let SSH do region re-assign
3138         }
3139       }
3140     }
3141 
3142     List<String> nodes = useZKForAssignment ?
3143       ZKUtil.listChildrenAndWatchForNewChildren(watcher, watcher.assignmentZNode)
3144       : ZKUtil.listChildrenNoWatch(watcher, watcher.assignmentZNode);
3145     if (nodes != null && !nodes.isEmpty()) {
3146       for (String encodedRegionName : nodes) {
3147         processRegionInTransition(encodedRegionName, null);
3148       }
3149     } else if (!useZKForAssignment) {
3150       processRegionInTransitionZkLess();
3151     }
3152   }
3153 
3154   void processRegionInTransitionZkLess() {
3155     // We need to send RPC call again for PENDING_OPEN/PENDING_CLOSE regions
3156     // in case the RPC call is not sent out yet before the master was shut down
3157     // since we update the state before we send the RPC call. We can't update
3158     // the state after the RPC call. Otherwise, we don't know what's happened
3159     // to the region if the master dies right after the RPC call is out.
3160     Map<String, RegionState> rits = regionStates.getRegionsInTransition();
3161     for (RegionState regionState : rits.values()) {
3162       LOG.info("Processing " + regionState);
3163       ServerName serverName = regionState.getServerName();
3164       // Server could be null in case of FAILED_OPEN when master cannot find a region plan. In that
3165       // case, try assigning it here.
3166       if (serverName != null
3167           && !serverManager.getOnlineServers().containsKey(serverName)) {
3168         LOG.info("Server " + serverName + " isn't online. SSH will handle this");
3169         continue;
3170       }
3171       HRegionInfo regionInfo = regionState.getRegion();
3172       State state = regionState.getState();
3173 
3174       switch (state) {
3175       case CLOSED:
3176         invokeAssign(regionInfo);
3177         break;
3178       case PENDING_OPEN:
3179         retrySendRegionOpen(regionState);
3180         break;
3181       case PENDING_CLOSE:
3182         retrySendRegionClose(regionState);
3183         break;
3184       case FAILED_CLOSE:
3185       case FAILED_OPEN:
3186         invokeUnAssign(regionInfo);
3187         break;
3188       default:
3189         // No process for other states
3190       }
3191     }
3192   }
3193 
3194   /**
3195    * At master failover, for pending_open region, make sure
3196    * sendRegionOpen RPC call is sent to the target regionserver
3197    */
3198   private void retrySendRegionOpen(final RegionState regionState) {
3199     this.executorService.submit(
3200       new EventHandler(server, EventType.M_MASTER_RECOVERY) {
3201         @Override
3202         public void process() throws IOException {
3203           HRegionInfo hri = regionState.getRegion();
3204           ServerName serverName = regionState.getServerName();
3205           ReentrantLock lock = locker.acquireLock(hri.getEncodedName());
3206           try {
3207             for (int i = 1; i <= maximumAttempts; i++) {
3208               if (!serverManager.isServerOnline(serverName)
3209                   || server.isStopped() || server.isAborted()) {
3210                 return; // No need any more
3211               }
3212               try {
3213                 if (!regionState.equals(regionStates.getRegionState(hri))) {
3214                   return; // Region is not in the expected state any more
3215                 }
3216                 List<ServerName> favoredNodes = ServerName.EMPTY_SERVER_LIST;
3217                 if (shouldAssignRegionsWithFavoredNodes) {
3218                   favoredNodes = ((FavoredNodeLoadBalancer)balancer).getFavoredNodes(hri);
3219                 }
3220                 RegionOpeningState regionOpenState = serverManager.sendRegionOpen(
3221                   serverName, hri, -1, favoredNodes);
3222 
3223                 if (regionOpenState == RegionOpeningState.FAILED_OPENING) {
3224                   // Failed opening this region, this means the target server didn't get
3225                   // the original region open RPC, so re-assign it with a new plan
3226                   LOG.debug("Got failed_opening in retry sendRegionOpen for "
3227                     + regionState + ", re-assign it");
3228                   invokeAssign(hri, true);
3229                 }
3230                 return; // Done.
3231               } catch (Throwable t) {
3232                 if (t instanceof RemoteException) {
3233                   t = ((RemoteException) t).unwrapRemoteException();
3234                 }
3235                 // In case SocketTimeoutException/FailedServerException, retry
3236                 if (t instanceof java.net.SocketTimeoutException
3237                     || t instanceof FailedServerException) {
3238                   Threads.sleep(100);
3239                   continue;
3240                 }
3241                 // For other exceptions, re-assign it
3242                 LOG.debug("Got exception in retry sendRegionOpen for "
3243                   + regionState + ", re-assign it", t);
3244                 invokeAssign(hri);
3245                 return; // Done.
3246               }
3247             }
3248           } finally {
3249             lock.unlock();
3250           }
3251         }
3252       });
3253   }
3254 
3255   /**
3256    * At master failover, for pending_close region, make sure
3257    * sendRegionClose RPC call is sent to the target regionserver
3258    */
3259   private void retrySendRegionClose(final RegionState regionState) {
3260     this.executorService.submit(
3261       new EventHandler(server, EventType.M_MASTER_RECOVERY) {
3262         @Override
3263         public void process() throws IOException {
3264           HRegionInfo hri = regionState.getRegion();
3265           ServerName serverName = regionState.getServerName();
3266           ReentrantLock lock = locker.acquireLock(hri.getEncodedName());
3267           try {
3268             for (int i = 1; i <= maximumAttempts; i++) {
3269               if (!serverManager.isServerOnline(serverName)
3270                   || server.isStopped() || server.isAborted()) {
3271                 return; // No need any more
3272               }
3273               try {
3274                 if (!regionState.equals(regionStates.getRegionState(hri))) {
3275                   return; // Region is not in the expected state any more
3276                 }
3277                 if (!serverManager.sendRegionClose(serverName, hri, -1, null, false)) {
3278                   // This means the region is still on the target server
3279                   LOG.debug("Got false in retry sendRegionClose for "
3280                     + regionState + ", re-close it");
3281                   invokeUnAssign(hri);
3282                 }
3283                 return; // Done.
3284               } catch (Throwable t) {
3285                 if (t instanceof RemoteException) {
3286                   t = ((RemoteException) t).unwrapRemoteException();
3287                 }
3288                 // In case SocketTimeoutException/FailedServerException, retry
3289                 if (t instanceof java.net.SocketTimeoutException
3290                     || t instanceof FailedServerException) {
3291                   Threads.sleep(100);
3292                   continue;
3293                 }
3294                 if (!(t instanceof NotServingRegionException
3295                     || t instanceof RegionAlreadyInTransitionException)) {
3296                   // NotServingRegionException/RegionAlreadyInTransitionException
3297                   // means the target server got the original region close request.
3298                   // For other exceptions, re-close it
3299                   LOG.debug("Got exception in retry sendRegionClose for "
3300                     + regionState + ", re-close it", t);
3301                   invokeUnAssign(hri);
3302                 }
3303                 return; // Done.
3304               }
3305             }
3306           } finally {
3307             lock.unlock();
3308           }
3309         }
3310       });
3311   }
3312 
3313   /**
3314    * Set Regions in transitions metrics.
3315    * This takes an iterator on the RegionInTransition map (CLSM), and is not synchronized.
3316    * This iterator is not fail fast, which may lead to stale read; but that's better than
3317    * creating a copy of the map for metrics computation, as this method will be invoked
3318    * on a frequent interval.
3319    */
3320   public void updateRegionsInTransitionMetrics() {
3321     long currentTime = System.currentTimeMillis();
3322     int totalRITs = 0;
3323     int totalRITsOverThreshold = 0;
3324     long oldestRITTime = 0;
3325     int ritThreshold = this.server.getConfiguration().
3326       getInt(HConstants.METRICS_RIT_STUCK_WARNING_THRESHOLD, 60000);
3327     for (RegionState state: regionStates.getRegionsInTransition().values()) {
3328       totalRITs++;
3329       long ritTime = currentTime - state.getStamp();
3330       if (ritTime > ritThreshold) { // more than the threshold
3331         totalRITsOverThreshold++;
3332       }
3333       if (oldestRITTime < ritTime) {
3334         oldestRITTime = ritTime;
3335       }
3336     }
3337     if (this.metricsAssignmentManager != null) {
3338       this.metricsAssignmentManager.updateRITOldestAge(oldestRITTime);
3339       this.metricsAssignmentManager.updateRITCount(totalRITs);
3340       this.metricsAssignmentManager.updateRITCountOverThreshold(totalRITsOverThreshold);
3341     }
3342   }
3343 
3344   /**
3345    * @param region Region whose plan we are to clear.
3346    */
3347   void clearRegionPlan(final HRegionInfo region) {
3348     synchronized (this.regionPlans) {
3349       this.regionPlans.remove(region.getEncodedName());
3350     }
3351   }
3352 
3353   /**
3354    * Wait on region to clear regions-in-transition.
3355    * @param hri Region to wait on.
3356    * @throws IOException
3357    */
3358   public void waitOnRegionToClearRegionsInTransition(final HRegionInfo hri)
3359       throws IOException, InterruptedException {
3360     waitOnRegionToClearRegionsInTransition(hri, -1L);
3361   }
3362 
3363   /**
3364    * Wait on region to clear regions-in-transition or time out
3365    * @param hri
3366    * @param timeOut Milliseconds to wait for current region to be out of transition state.
3367    * @return True when a region clears regions-in-transition before timeout otherwise false
3368    * @throws InterruptedException
3369    */
3370   public boolean waitOnRegionToClearRegionsInTransition(final HRegionInfo hri, long timeOut)
3371       throws InterruptedException {
3372     if (!regionStates.isRegionInTransition(hri)) return true;
3373     long end = (timeOut <= 0) ? Long.MAX_VALUE : EnvironmentEdgeManager.currentTime()
3374         + timeOut;
3375     // There is already a timeout monitor on regions in transition so I
3376     // should not have to have one here too?
3377     LOG.info("Waiting for " + hri.getEncodedName() +
3378         " to leave regions-in-transition, timeOut=" + timeOut + " ms.");
3379     while (!this.server.isStopped() && regionStates.isRegionInTransition(hri)) {
3380       regionStates.waitForUpdate(100);
3381       if (EnvironmentEdgeManager.currentTime() > end) {
3382         LOG.info("Timed out on waiting for " + hri.getEncodedName() + " to be assigned.");
3383         return false;
3384       }
3385     }
3386     if (this.server.isStopped()) {
3387       LOG.info("Giving up wait on regions in transition because stoppable.isStopped is set");
3388       return false;
3389     }
3390     return true;
3391   }
3392 
3393   void invokeAssign(HRegionInfo regionInfo) {
3394     invokeAssign(regionInfo, true);
3395   }
3396 
3397   void invokeAssign(HRegionInfo regionInfo, boolean newPlan) {
3398     threadPoolExecutorService.submit(new AssignCallable(this, regionInfo, newPlan));
3399   }
3400 
3401   void invokeUnAssign(HRegionInfo regionInfo) {
3402     threadPoolExecutorService.submit(new UnAssignCallable(this, regionInfo));
3403   }
3404 
3405   public ServerHostRegion isCarryingMeta(ServerName serverName) {
3406     return isCarryingRegion(serverName, HRegionInfo.FIRST_META_REGIONINFO);
3407   }
3408 
3409   public ServerHostRegion isCarryingMetaReplica(ServerName serverName, int replicaId) {
3410     return isCarryingRegion(serverName,
3411         RegionReplicaUtil.getRegionInfoForReplica(HRegionInfo.FIRST_META_REGIONINFO, replicaId));
3412   }
3413 
3414   public ServerHostRegion isCarryingMetaReplica(ServerName serverName, HRegionInfo metaHri) {
3415     return isCarryingRegion(serverName, metaHri);
3416   }
3417 
3418   /**
3419    * Check if the shutdown server carries the specific region.
3420    * We have a bunch of places that store region location
3421    * Those values aren't consistent. There is a delay of notification.
3422    * The location from zookeeper unassigned node has the most recent data;
3423    * but the node could be deleted after the region is opened by AM.
3424    * The AM's info could be old when OpenedRegionHandler
3425    * processing hasn't finished yet when server shutdown occurs.
3426    * @return whether the serverName currently hosts the region
3427    */
3428   private ServerHostRegion isCarryingRegion(ServerName serverName, HRegionInfo hri) {
3429     RegionTransition rt = null;
3430     try {
3431       byte [] data = ZKAssign.getData(watcher, hri.getEncodedName());
3432       // This call can legitimately come by null
3433       rt = data == null? null: RegionTransition.parseFrom(data);
3434     } catch (KeeperException e) {
3435       server.abort("Exception reading unassigned node for region=" + hri.getEncodedName(), e);
3436     } catch (DeserializationException e) {
3437       server.abort("Exception parsing unassigned node for region=" + hri.getEncodedName(), e);
3438     }
3439 
3440     ServerName addressFromZK = rt != null? rt.getServerName():  null;
3441     if (addressFromZK != null) {
3442       // if we get something from ZK, we will use the data
3443       boolean matchZK = addressFromZK.equals(serverName);
3444       LOG.debug("Checking region=" + hri.getRegionNameAsString() + ", zk server=" + addressFromZK +
3445         " current=" + serverName + ", matches=" + matchZK);
3446       return matchZK ? ServerHostRegion.HOSTING_REGION : ServerHostRegion.NOT_HOSTING_REGION;
3447     }
3448 
3449     ServerName addressFromAM = regionStates.getRegionServerOfRegion(hri);
3450     if (LOG.isDebugEnabled()) {
3451       LOG.debug("based on AM, current region=" + hri.getRegionNameAsString() +
3452         " is on server=" + (addressFromAM != null ? addressFromAM : "null") +
3453         " server being checked: " + serverName);
3454     }
3455     if (addressFromAM != null) {
3456       return addressFromAM.equals(serverName) ?
3457           ServerHostRegion.HOSTING_REGION : ServerHostRegion.NOT_HOSTING_REGION;
3458     }
3459 
3460     if (hri.isMetaRegion() && RegionReplicaUtil.isDefaultReplica(hri)) {
3461       // For the Meta region (default replica), we can do one more check on MetaTableLocator
3462       final ServerName serverNameInZK =
3463           server.getMetaTableLocator().getMetaRegionLocation(this.server.getZooKeeper());
3464       if (LOG.isDebugEnabled()) {
3465         LOG.debug("Based on MetaTableLocator, the META region is on server=" +
3466           (serverNameInZK == null ? "null" : serverNameInZK) +
3467           " server being checked: " + serverName);
3468       }
3469       if (serverNameInZK != null) {
3470         return serverNameInZK.equals(serverName) ?
3471             ServerHostRegion.HOSTING_REGION : ServerHostRegion.NOT_HOSTING_REGION;
3472       }
3473     }
3474 
3475     // Checked everywhere, if reaching here, we are unsure whether the server is carrying region.
3476     return ServerHostRegion.UNKNOWN;
3477   }
3478 
3479   /**
3480    * Clean out crashed server removing any assignments.
3481    * @param sn Server that went down.
3482    * @return list of regions in transition on this server
3483    */
3484   public List<HRegionInfo> cleanOutCrashedServerReferences(final ServerName sn) {
3485     // Clean out any existing assignment plans for this server
3486     synchronized (this.regionPlans) {
3487       for (Iterator <Map.Entry<String, RegionPlan>> i = this.regionPlans.entrySet().iterator();
3488           i.hasNext();) {
3489         Map.Entry<String, RegionPlan> e = i.next();
3490         ServerName otherSn = e.getValue().getDestination();
3491         // The name will be null if the region is planned for a random assign.
3492         if (otherSn != null && otherSn.equals(sn)) {
3493           // Use iterator's remove else we'll get CME
3494           i.remove();
3495         }
3496       }
3497     }
3498     List<HRegionInfo> regions = regionStates.serverOffline(watcher, sn);
3499     for (Iterator<HRegionInfo> it = regions.iterator(); it.hasNext(); ) {
3500       HRegionInfo hri = it.next();
3501       String encodedName = hri.getEncodedName();
3502 
3503       // We need a lock on the region as we could update it
3504       Lock lock = locker.acquireLock(encodedName);
3505       try {
3506         RegionState regionState = regionStates.getRegionTransitionState(encodedName);
3507         if (regionState == null
3508             || (regionState.getServerName() != null && !regionState.isOnServer(sn))
3509             || !(regionState.isFailedClose() || regionState.isOffline()
3510               || regionState.isPendingOpenOrOpening())) {
3511           LOG.info("Skip " + regionState + " since it is not opening/failed_close"
3512             + " on the dead server any more: " + sn);
3513           it.remove();
3514         } else {
3515           try {
3516             // Delete the ZNode if exists
3517             ZKAssign.deleteNodeFailSilent(watcher, hri);
3518           } catch (KeeperException ke) {
3519             server.abort("Unexpected ZK exception deleting node " + hri, ke);
3520           }
3521           if (tableStateManager.isTableState(hri.getTable(),
3522               ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING)) {
3523             regionStates.regionOffline(hri);
3524             it.remove();
3525             continue;
3526           }
3527           // Mark the region offline and assign it again by SSH
3528           regionStates.updateRegionState(hri, State.OFFLINE);
3529         }
3530       } finally {
3531         lock.unlock();
3532       }
3533     }
3534     return regions;
3535   }
3536 
3537   /**
3538    * @param plan Plan to execute.
3539    */
3540   public void balance(final RegionPlan plan) {
3541 
3542     HRegionInfo hri = plan.getRegionInfo();
3543     TableName tableName = hri.getTable();
3544     if (tableStateManager.isTableState(tableName,
3545       ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING)) {
3546       LOG.info("Ignored moving region of disabling/disabled table "
3547         + tableName);
3548       return;
3549     }
3550 
3551     // Move the region only if it's assigned
3552     String encodedName = hri.getEncodedName();
3553     ReentrantLock lock = locker.acquireLock(encodedName);
3554     try {
3555       if (!regionStates.isRegionOnline(hri)) {
3556         RegionState state = regionStates.getRegionState(encodedName);
3557         LOG.info("Ignored moving region not assigned: " + hri + ", "
3558           + (state == null ? "not in region states" : state));
3559         return;
3560       }
3561       synchronized (this.regionPlans) {
3562         this.regionPlans.put(plan.getRegionName(), plan);
3563       }
3564       unassign(hri, false, plan.getDestination());
3565     } finally {
3566       lock.unlock();
3567     }
3568   }
3569 
3570   public void stop() {
3571     shutdown(); // Stop executor service, etc
3572   }
3573 
3574   /**
3575    * Shutdown the threadpool executor service
3576    */
3577   public void shutdown() {
3578     // It's an immediate shutdown, so we're clearing the remaining tasks.
3579     synchronized (zkEventWorkerWaitingList){
3580       zkEventWorkerWaitingList.clear();
3581     }
3582 
3583     // Shutdown the threadpool executor service
3584     threadPoolExecutorService.shutdownNow();
3585     zkEventWorkers.shutdownNow();
3586     regionStateStore.stop();
3587   }
3588 
3589   protected void setEnabledTable(TableName tableName) {
3590     try {
3591       this.tableStateManager.setTableState(tableName,
3592         ZooKeeperProtos.Table.State.ENABLED);
3593     } catch (CoordinatedStateException e) {
3594       // here we can abort as it is the start up flow
3595       String errorMsg = "Unable to ensure that the table " + tableName
3596           + " will be" + " enabled because of a ZooKeeper issue";
3597       LOG.error(errorMsg);
3598       this.server.abort(errorMsg, e);
3599     }
3600   }
3601 
3602   /**
3603    * Set region as OFFLINED up in zookeeper asynchronously.
3604    * @param state
3605    * @return True if we succeeded, false otherwise (State was incorrect or failed
3606    * updating zk).
3607    */
3608   private boolean asyncSetOfflineInZooKeeper(final RegionState state,
3609       final AsyncCallback.StringCallback cb, final ServerName destination) {
3610     if (!state.isClosed() && !state.isOffline()) {
3611       this.server.abort("Unexpected state trying to OFFLINE; " + state,
3612         new IllegalStateException());
3613       return false;
3614     }
3615     regionStates.updateRegionState(state.getRegion(), State.OFFLINE);
3616     try {
3617       ZKAssign.asyncCreateNodeOffline(watcher, state.getRegion(),
3618         destination, cb, state);
3619     } catch (KeeperException e) {
3620       if (e instanceof NodeExistsException) {
3621         LOG.warn("Node for " + state.getRegion() + " already exists");
3622       } else {
3623         server.abort("Unexpected ZK exception creating/setting node OFFLINE", e);
3624       }
3625       return false;
3626     }
3627     return true;
3628   }
3629 
3630   private boolean deleteNodeInStates(String encodedName,
3631       String desc, ServerName sn, EventType... types) {
3632     try {
3633       for (EventType et: types) {
3634         if (ZKAssign.deleteNode(watcher, encodedName, et, sn)) {
3635           return true;
3636         }
3637       }
3638       LOG.info("Failed to delete the " + desc + " node for "
3639         + encodedName + ". The node type may not match");
3640     } catch (NoNodeException e) {
3641       if (LOG.isDebugEnabled()) {
3642         LOG.debug("The " + desc + " node for " + encodedName + " already deleted");
3643       }
3644     } catch (KeeperException ke) {
3645       server.abort("Unexpected ZK exception deleting " + desc
3646         + " node for the region " + encodedName, ke);
3647     }
3648     return false;
3649   }
3650 
3651   private void deleteMergingNode(String encodedName, ServerName sn) {
3652     deleteNodeInStates(encodedName, "merging", sn, EventType.RS_ZK_REGION_MERGING,
3653       EventType.RS_ZK_REQUEST_REGION_MERGE, EventType.RS_ZK_REGION_MERGED);
3654   }
3655 
3656   private void deleteSplittingNode(String encodedName, ServerName sn) {
3657     deleteNodeInStates(encodedName, "splitting", sn, EventType.RS_ZK_REGION_SPLITTING,
3658       EventType.RS_ZK_REQUEST_REGION_SPLIT, EventType.RS_ZK_REGION_SPLIT);
3659   }
3660 
3661   @edu.umd.cs.findbugs.annotations.SuppressWarnings(
3662       value="AT_OPERATION_SEQUENCE_ON_CONCURRENT_ABSTRACTION",
3663       justification="Modification of Maps not ATOMIC!!!! FIX!!!")
3664   private void onRegionFailedOpen(
3665       final HRegionInfo hri, final ServerName sn) {
3666     String encodedName = hri.getEncodedName();
3667     // FindBugs: AT_OPERATION_SEQUENCE_ON_CONCURRENT_ABSTRACTION Worth fixing!!!
3668     AtomicInteger failedOpenCount = failedOpenTracker.get(encodedName);
3669     if (failedOpenCount == null) {
3670       failedOpenCount = new AtomicInteger();
3671       // No need to use putIfAbsent, or extra synchronization since
3672       // this whole handleRegion block is locked on the encoded region
3673       // name, and failedOpenTracker is updated only in this block
3674       // FindBugs: AT_OPERATION_SEQUENCE_ON_CONCURRENT_ABSTRACTION
3675       failedOpenTracker.put(encodedName, failedOpenCount);
3676     }
3677     if (failedOpenCount.incrementAndGet() >= maximumAttempts && !hri.isMetaRegion()) {
3678       // FindBugs: AT_OPERATION_SEQUENCE_ON_CONCURRENT_ABSTRACTION
3679       regionStates.updateRegionState(hri, State.FAILED_OPEN);
3680       // remove the tracking info to save memory, also reset
3681       // the count for next open initiative
3682       failedOpenTracker.remove(encodedName);
3683     } else {
3684       if (hri.isMetaRegion() && failedOpenCount.get() >= maximumAttempts) {
3685         // Log a warning message if a meta region failedOpenCount exceeds maximumAttempts
3686         // so that we are aware of potential problem if it persists for a long time.
3687         LOG.warn("Failed to open the hbase:meta region " +
3688             hri.getRegionNameAsString() + " after" +
3689             failedOpenCount.get() + " retries. Continue retrying.");
3690       }
3691 
3692       // Handle this the same as if it were opened and then closed.
3693       RegionState regionState = regionStates.updateRegionState(hri, State.CLOSED);
3694       if (regionState != null) {
3695         // When there are more than one region server a new RS is selected as the
3696         // destination and the same is updated in the region plan. (HBASE-5546)
3697         if (getTableStateManager().isTableState(hri.getTable(),
3698             ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING) ||
3699             replicasToClose.contains(hri)) {
3700           offlineDisabledRegion(hri);
3701           return;
3702         }
3703         // ZK Node is in CLOSED state, assign it.
3704          regionStates.updateRegionState(hri, RegionState.State.CLOSED);
3705         // This below has to do w/ online enable/disable of a table
3706         removeClosedRegion(hri);
3707         try {
3708           getRegionPlan(hri, sn, true);
3709         } catch (HBaseIOException e) {
3710           LOG.warn("Failed to get region plan", e);
3711         }
3712         invokeAssign(hri, false);
3713       }
3714     }
3715   }
3716 
3717   private void onRegionOpen(final HRegionInfo hri, final ServerName sn, long openSeqNum) {
3718     regionOnline(hri, sn, openSeqNum);
3719     if (useZKForAssignment) {
3720       try {
3721         // Delete the ZNode if exists
3722         ZKAssign.deleteNodeFailSilent(watcher, hri);
3723       } catch (KeeperException ke) {
3724         server.abort("Unexpected ZK exception deleting node " + hri, ke);
3725       }
3726     }
3727 
3728     // reset the count, if any
3729     failedOpenTracker.remove(hri.getEncodedName());
3730     if (getTableStateManager().isTableState(hri.getTable(),
3731         ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING)) {
3732       invokeUnAssign(hri);
3733     }
3734   }
3735 
3736   private void onRegionClosed(final HRegionInfo hri) {
3737     if (getTableStateManager().isTableState(hri.getTable(),
3738         ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING) ||
3739         replicasToClose.contains(hri)) {
3740       offlineDisabledRegion(hri);
3741       return;
3742     }
3743     regionStates.updateRegionState(hri, RegionState.State.CLOSED);
3744     sendRegionClosedNotification(hri);
3745     // This below has to do w/ online enable/disable of a table
3746     removeClosedRegion(hri);
3747     invokeAssign(hri, false);
3748   }
3749 
3750   private String checkInStateForSplit(ServerName sn,
3751       final HRegionInfo p, final HRegionInfo a, final HRegionInfo b) {
3752     final RegionState rs_p = regionStates.getRegionState(p);
3753     RegionState rs_a = regionStates.getRegionState(a);
3754     RegionState rs_b = regionStates.getRegionState(b);
3755     if (!(rs_p.isOpenOrSplittingOnServer(sn)
3756         && (rs_a == null || rs_a.isOpenOrSplittingNewOnServer(sn))
3757         && (rs_b == null || rs_b.isOpenOrSplittingNewOnServer(sn)))) {
3758       return "Not in state good for split";
3759     }
3760     return "";
3761   }
3762 
3763   private String onRegionSplitReverted(ServerName sn,
3764       final HRegionInfo p, final HRegionInfo a, final HRegionInfo b) {
3765     String s = checkInStateForSplit(sn, p, a, b);
3766     if (!org.apache.commons.lang.StringUtils.isEmpty(s)) {
3767       return s;
3768     }
3769 
3770     // Always bring the parent back online. Even if it's not offline
3771     // There's no harm in making it online again.
3772     regionOnline(p, sn);
3773 
3774     // Only offline the region if they are known to exist.
3775     RegionState regionStateA = regionStates.getRegionState(a);
3776     RegionState regionStateB = regionStates.getRegionState(b);
3777     if (regionStateA != null) {
3778       regionOffline(a);
3779     }
3780     if (regionStateB != null) {
3781       regionOffline(b);
3782     }
3783 
3784     if (getTableStateManager().isTableState(p.getTable(),
3785         ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING)) {
3786       invokeUnAssign(p);
3787     }
3788     return null;
3789   }
3790 
3791   private String onRegionSplit(ServerName sn, TransitionCode code,
3792       final HRegionInfo p, final HRegionInfo a, final HRegionInfo b) {
3793     String s = checkInStateForSplit(sn, p, a, b);
3794     if (!org.apache.commons.lang.StringUtils.isEmpty(s)) {
3795       return s;
3796     }
3797 
3798     regionStates.updateRegionState(a, State.SPLITTING_NEW, sn);
3799     regionStates.updateRegionState(b, State.SPLITTING_NEW, sn);
3800     regionStates.updateRegionState(p, State.SPLITTING);
3801 
3802     if (code == TransitionCode.SPLIT) {
3803       if (TEST_SKIP_SPLIT_HANDLING) {
3804         return "Skipping split message, TEST_SKIP_SPLIT_HANDLING is set";
3805       }
3806       regionOffline(p, State.SPLIT);
3807       regionOnline(a, sn, 1);
3808       regionOnline(b, sn, 1);
3809 
3810       // User could disable the table before master knows the new region.
3811       if (getTableStateManager().isTableState(p.getTable(),
3812           ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING)) {
3813         invokeUnAssign(a);
3814         invokeUnAssign(b);
3815       } else {
3816         Callable<Object> splitReplicasCallable = new Callable<Object>() {
3817           @Override
3818           public Object call() {
3819             doSplittingOfReplicas(p, a, b);
3820             return null;
3821           }
3822         };
3823         threadPoolExecutorService.submit(splitReplicasCallable);
3824       }
3825     } else if (code == TransitionCode.SPLIT_PONR) {
3826       try {
3827         regionStates.splitRegion(p, a, b, sn);
3828       } catch (IOException ioe) {
3829         LOG.info("Failed to record split region " + p.getShortNameToLog());
3830         return "Failed to record the splitting in meta";
3831       }
3832     }
3833     return null;
3834   }
3835 
3836   private String onRegionMerge(ServerName sn, TransitionCode code,
3837       final HRegionInfo p, final HRegionInfo a, final HRegionInfo b) {
3838     RegionState rs_p = regionStates.getRegionState(p);
3839     RegionState rs_a = regionStates.getRegionState(a);
3840     RegionState rs_b = regionStates.getRegionState(b);
3841     if (!(rs_a.isOpenOrMergingOnServer(sn) && rs_b.isOpenOrMergingOnServer(sn)
3842         && (rs_p == null || rs_p.isOpenOrMergingNewOnServer(sn)))) {
3843       return "Not in state good for merge";
3844     }
3845 
3846     regionStates.updateRegionState(a, State.MERGING);
3847     regionStates.updateRegionState(b, State.MERGING);
3848     regionStates.updateRegionState(p, State.MERGING_NEW, sn);
3849 
3850     String encodedName = p.getEncodedName();
3851     if (code == TransitionCode.READY_TO_MERGE) {
3852       mergingRegions.put(encodedName,
3853         new PairOfSameType<HRegionInfo>(a, b));
3854     } else if (code == TransitionCode.MERGED) {
3855       mergingRegions.remove(encodedName);
3856       regionOffline(a, State.MERGED);
3857       regionOffline(b, State.MERGED);
3858       regionOnline(p, sn, 1);
3859 
3860       // User could disable the table before master knows the new region.
3861       if (getTableStateManager().isTableState(p.getTable(),
3862           ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING)) {
3863         invokeUnAssign(p);
3864       } else {
3865         Callable<Object> mergeReplicasCallable = new Callable<Object>() {
3866           @Override
3867           public Object call() {
3868             doMergingOfReplicas(p, a, b);
3869             return null;
3870           }
3871         };
3872         threadPoolExecutorService.submit(mergeReplicasCallable);
3873       }
3874     } else if (code == TransitionCode.MERGE_PONR) {
3875       try {
3876         regionStates.mergeRegions(p, a, b, sn);
3877       } catch (IOException ioe) {
3878         LOG.info("Failed to record merged region " + p.getShortNameToLog());
3879         return "Failed to record the merging in meta";
3880       }
3881     }
3882     return null;
3883   }
3884 
3885   private String onRegionMergeReverted(ServerName sn, TransitionCode code,
3886 	      final HRegionInfo p, final HRegionInfo a, final HRegionInfo b) {
3887     RegionState rs_p = regionStates.getRegionState(p);
3888     String encodedName = p.getEncodedName();
3889     mergingRegions.remove(encodedName);
3890 
3891     // Always bring the children back online. Even if they are not offline
3892     // there's no harm in making them online again.
3893     regionOnline(a, sn);
3894     regionOnline(b, sn);
3895 
3896     // Only offline the merging region if it is known to exist.
3897     if (rs_p != null) {
3898       regionOffline(p);
3899     }
3900 
3901     if (getTableStateManager().isTableState(p.getTable(),
3902         ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING)) {
3903       invokeUnAssign(a);
3904       invokeUnAssign(b);
3905     }
3906 
3907     return null;
3908   }
3909 
3910   /**
3911    * A helper to handle region merging transition event.
3912    * It transitions merging regions to MERGING state.
3913    */
3914   private boolean handleRegionMerging(final RegionTransition rt, final String encodedName,
3915       final String prettyPrintedRegionName, final ServerName sn) {
3916     if (!serverManager.isServerOnline(sn)) {
3917       LOG.warn("Dropped merging! ServerName=" + sn + " unknown.");
3918       return false;
3919     }
3920     byte [] payloadOfMerging = rt.getPayload();
3921     List<HRegionInfo> mergingRegions;
3922     try {
3923       mergingRegions = HRegionInfo.parseDelimitedFrom(
3924         payloadOfMerging, 0, payloadOfMerging.length);
3925     } catch (IOException e) {
3926       LOG.error("Dropped merging! Failed reading "  + rt.getEventType()
3927         + " payload for " + prettyPrintedRegionName);
3928       return false;
3929     }
3930     assert mergingRegions.size() == 3;
3931     HRegionInfo p = mergingRegions.get(0);
3932     HRegionInfo hri_a = mergingRegions.get(1);
3933     HRegionInfo hri_b = mergingRegions.get(2);
3934 
3935     RegionState rs_p = regionStates.getRegionState(p);
3936     RegionState rs_a = regionStates.getRegionState(hri_a);
3937     RegionState rs_b = regionStates.getRegionState(hri_b);
3938 
3939     if (!((rs_a == null || rs_a.isOpenOrMergingOnServer(sn))
3940         && (rs_b == null || rs_b.isOpenOrMergingOnServer(sn))
3941         && (rs_p == null || rs_p.isOpenOrMergingNewOnServer(sn)))) {
3942       LOG.warn("Dropped merging! Not in state good for MERGING; rs_p="
3943         + rs_p + ", rs_a=" + rs_a + ", rs_b=" + rs_b);
3944       return false;
3945     }
3946 
3947     EventType et = rt.getEventType();
3948     if (et == EventType.RS_ZK_REQUEST_REGION_MERGE) {
3949       try {
3950         RegionMergeCoordination.RegionMergeDetails std =
3951             ((BaseCoordinatedStateManager) server.getCoordinatedStateManager())
3952                 .getRegionMergeCoordination().getDefaultDetails();
3953         ((BaseCoordinatedStateManager) server.getCoordinatedStateManager())
3954             .getRegionMergeCoordination().processRegionMergeRequest(p, hri_a, hri_b, sn, std);
3955         if (((ZkRegionMergeCoordination.ZkRegionMergeDetails) std).getZnodeVersion() == -1) {
3956           byte[] data = ZKAssign.getData(watcher, encodedName);
3957          EventType currentType = null;
3958           if (data != null) {
3959             RegionTransition newRt = RegionTransition.parseFrom(data);
3960             currentType = newRt.getEventType();
3961           }
3962           if (currentType == null || (currentType != EventType.RS_ZK_REGION_MERGED
3963               && currentType != EventType.RS_ZK_REGION_MERGING)) {
3964             LOG.warn("Failed to transition pending_merge node "
3965               + encodedName + " to merging, it's now " + currentType);
3966             return false;
3967           }
3968         }
3969       } catch (Exception e) {
3970         LOG.warn("Failed to transition pending_merge node "
3971           + encodedName + " to merging", e);
3972         return false;
3973       }
3974     }
3975 
3976     synchronized (regionStates) {
3977       regionStates.updateRegionState(hri_a, State.MERGING);
3978       regionStates.updateRegionState(hri_b, State.MERGING);
3979       regionStates.updateRegionState(p, State.MERGING_NEW, sn);
3980 
3981       if (et != EventType.RS_ZK_REGION_MERGED) {
3982         this.mergingRegions.put(encodedName,
3983           new PairOfSameType<HRegionInfo>(hri_a, hri_b));
3984       } else {
3985         this.mergingRegions.remove(encodedName);
3986         regionOffline(hri_a, State.MERGED);
3987         regionOffline(hri_b, State.MERGED);
3988         regionOnline(p, sn);
3989       }
3990     }
3991 
3992     if (et == EventType.RS_ZK_REGION_MERGED) {
3993       doMergingOfReplicas(p, hri_a, hri_b);
3994       LOG.debug("Handling MERGED event for " + encodedName + "; deleting node");
3995       // Remove region from ZK
3996       try {
3997         boolean successful = false;
3998         while (!successful) {
3999           // It's possible that the RS tickles in between the reading of the
4000           // znode and the deleting, so it's safe to retry.
4001           successful = ZKAssign.deleteNode(watcher, encodedName,
4002             EventType.RS_ZK_REGION_MERGED, sn);
4003         }
4004       } catch (KeeperException e) {
4005         if (e instanceof NoNodeException) {
4006           String znodePath = ZKUtil.joinZNode(watcher.splitLogZNode, encodedName);
4007           LOG.debug("The znode " + znodePath + " does not exist.  May be deleted already.");
4008         } else {
4009           server.abort("Error deleting MERGED node " + encodedName, e);
4010         }
4011       }
4012       LOG.info("Handled MERGED event; merged=" + p.getRegionNameAsString()
4013         + ", region_a=" + hri_a.getRegionNameAsString() + ", region_b="
4014         + hri_b.getRegionNameAsString() + ", on " + sn);
4015 
4016       // User could disable the table before master knows the new region.
4017       if (tableStateManager.isTableState(p.getTable(),
4018           ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING)) {
4019         unassign(p);
4020       }
4021     }
4022     return true;
4023   }
4024 
4025   /**
4026    * A helper to handle region splitting transition event.
4027    */
4028   private boolean handleRegionSplitting(final RegionTransition rt, final String encodedName,
4029       final String prettyPrintedRegionName, final ServerName sn) {
4030     if (!serverManager.isServerOnline(sn)) {
4031       LOG.warn("Dropped splitting! ServerName=" + sn + " unknown.");
4032       return false;
4033     }
4034     byte [] payloadOfSplitting = rt.getPayload();
4035     List<HRegionInfo> splittingRegions;
4036     try {
4037       splittingRegions = HRegionInfo.parseDelimitedFrom(
4038         payloadOfSplitting, 0, payloadOfSplitting.length);
4039     } catch (IOException e) {
4040       LOG.error("Dropped splitting! Failed reading " + rt.getEventType()
4041         + " payload for " + prettyPrintedRegionName);
4042       return false;
4043     }
4044     assert splittingRegions.size() == 2;
4045     HRegionInfo hri_a = splittingRegions.get(0);
4046     HRegionInfo hri_b = splittingRegions.get(1);
4047 
4048     RegionState rs_p = regionStates.getRegionState(encodedName);
4049     RegionState rs_a = regionStates.getRegionState(hri_a);
4050     RegionState rs_b = regionStates.getRegionState(hri_b);
4051 
4052     if (!((rs_p == null || rs_p.isOpenOrSplittingOnServer(sn))
4053         && (rs_a == null || rs_a.isOpenOrSplittingNewOnServer(sn))
4054         && (rs_b == null || rs_b.isOpenOrSplittingNewOnServer(sn)))) {
4055       LOG.warn("Dropped splitting! Not in state good for SPLITTING; rs_p="
4056         + rs_p + ", rs_a=" + rs_a + ", rs_b=" + rs_b);
4057       return false;
4058     }
4059 
4060     if (rs_p == null) {
4061       // Splitting region should be online
4062       rs_p = regionStates.updateRegionState(rt, State.OPEN);
4063       if (rs_p == null) {
4064         LOG.warn("Received splitting for region " + prettyPrintedRegionName
4065           + " from server " + sn + " but it doesn't exist anymore,"
4066           + " probably already processed its split");
4067         return false;
4068       }
4069       regionStates.regionOnline(rs_p.getRegion(), sn);
4070     }
4071 
4072     HRegionInfo p = rs_p.getRegion();
4073     EventType et = rt.getEventType();
4074     if (et == EventType.RS_ZK_REQUEST_REGION_SPLIT) {
4075       try {
4076         SplitTransactionDetails std =
4077             ((BaseCoordinatedStateManager) server.getCoordinatedStateManager())
4078                 .getSplitTransactionCoordination().getDefaultDetails();
4079         if (((BaseCoordinatedStateManager) server.getCoordinatedStateManager())
4080             .getSplitTransactionCoordination().processTransition(p, hri_a, hri_b, sn, std) == -1) {
4081           byte[] data = ZKAssign.getData(watcher, encodedName);
4082           EventType currentType = null;
4083           if (data != null) {
4084             RegionTransition newRt = RegionTransition.parseFrom(data);
4085             currentType = newRt.getEventType();
4086           }
4087           if (currentType == null
4088               || (currentType != EventType.RS_ZK_REGION_SPLIT && currentType != EventType.RS_ZK_REGION_SPLITTING)) {
4089             LOG.warn("Failed to transition pending_split node " + encodedName
4090                 + " to splitting, it's now " + currentType);
4091             return false;
4092           }
4093         }
4094       } catch (Exception e) {
4095         LOG.warn("Failed to transition pending_split node " + encodedName + " to splitting", e);
4096         return false;
4097       }
4098     }
4099 
4100     synchronized (regionStates) {
4101       splitRegions.put(p, new PairOfSameType<HRegionInfo>(hri_a, hri_b));
4102       regionStates.updateRegionState(hri_a, State.SPLITTING_NEW, sn);
4103       regionStates.updateRegionState(hri_b, State.SPLITTING_NEW, sn);
4104       regionStates.updateRegionState(rt, State.SPLITTING);
4105 
4106       // The below is for testing ONLY!  We can't do fault injection easily, so
4107       // resort to this kinda uglyness -- St.Ack 02/25/2011.
4108       if (TEST_SKIP_SPLIT_HANDLING) {
4109         LOG.warn("Skipping split message, TEST_SKIP_SPLIT_HANDLING is set");
4110         return true; // return true so that the splitting node stays
4111       }
4112 
4113       if (et == EventType.RS_ZK_REGION_SPLIT) {
4114         regionOffline(p, State.SPLIT);
4115         regionOnline(hri_a, sn);
4116         regionOnline(hri_b, sn);
4117         splitRegions.remove(p);
4118       }
4119     }
4120 
4121     if (et == EventType.RS_ZK_REGION_SPLIT) {
4122       // split replicas
4123       doSplittingOfReplicas(rs_p.getRegion(), hri_a, hri_b);
4124       LOG.debug("Handling SPLIT event for " + encodedName + "; deleting node");
4125       // Remove region from ZK
4126       try {
4127         boolean successful = false;
4128         while (!successful) {
4129           // It's possible that the RS tickles in between the reading of the
4130           // znode and the deleting, so it's safe to retry.
4131           successful = ZKAssign.deleteNode(watcher, encodedName,
4132             EventType.RS_ZK_REGION_SPLIT, sn);
4133         }
4134       } catch (KeeperException e) {
4135         if (e instanceof NoNodeException) {
4136           String znodePath = ZKUtil.joinZNode(watcher.splitLogZNode, encodedName);
4137           LOG.debug("The znode " + znodePath + " does not exist.  May be deleted already.");
4138         } else {
4139           server.abort("Error deleting SPLIT node " + encodedName, e);
4140         }
4141       }
4142       LOG.info("Handled SPLIT event; parent=" + p.getRegionNameAsString()
4143         + ", daughter a=" + hri_a.getRegionNameAsString() + ", daughter b="
4144         + hri_b.getRegionNameAsString() + ", on " + sn);
4145 
4146       // User could disable the table before master knows the new region.
4147       if (tableStateManager.isTableState(p.getTable(),
4148           ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING)) {
4149         unassign(hri_a);
4150         unassign(hri_b);
4151       }
4152     }
4153     return true;
4154   }
4155 
4156   private void doMergingOfReplicas(HRegionInfo mergedHri, final HRegionInfo hri_a,
4157       final HRegionInfo hri_b) {
4158     // Close replicas for the original unmerged regions. create/assign new replicas
4159     // for the merged parent.
4160     List<HRegionInfo> unmergedRegions = new ArrayList<HRegionInfo>();
4161     unmergedRegions.add(hri_a);
4162     unmergedRegions.add(hri_b);
4163     Map<ServerName, List<HRegionInfo>> map = regionStates.getRegionAssignments(unmergedRegions);
4164     Collection<List<HRegionInfo>> c = map.values();
4165     for (List<HRegionInfo> l : c) {
4166       for (HRegionInfo h : l) {
4167         if (!RegionReplicaUtil.isDefaultReplica(h)) {
4168           LOG.debug("Unassigning un-merged replica " + h);
4169           unassign(h);
4170         }
4171       }
4172     }
4173     int numReplicas = 1;
4174     try {
4175       numReplicas = server.getTableDescriptors().get(mergedHri.getTable()).
4176           getRegionReplication();
4177     } catch (IOException e) {
4178       LOG.warn("Couldn't get the replication attribute of the table " + mergedHri.getTable() +
4179           " due to " + e.getMessage() + ". The assignment of replicas for the merged region " +
4180           "will not be done");
4181     }
4182     List<HRegionInfo> regions = new ArrayList<HRegionInfo>();
4183     for (int i = 1; i < numReplicas; i++) {
4184       regions.add(RegionReplicaUtil.getRegionInfoForReplica(mergedHri, i));
4185     }
4186     try {
4187       assign(regions);
4188     } catch (IOException ioe) {
4189       LOG.warn("Couldn't assign all replica(s) of region " + mergedHri + " because of " +
4190                 ioe.getMessage());
4191     } catch (InterruptedException ie) {
4192       LOG.warn("Couldn't assign all replica(s) of region " + mergedHri+ " because of " +
4193                 ie.getMessage());
4194     }
4195   }
4196 
4197   private void doSplittingOfReplicas(final HRegionInfo parentHri, final HRegionInfo hri_a,
4198       final HRegionInfo hri_b) {
4199     // create new regions for the replica, and assign them to match with the
4200     // current replica assignments. If replica1 of parent is assigned to RS1,
4201     // the replica1s of daughters will be on the same machine
4202     int numReplicas = 1;
4203     try {
4204       numReplicas = server.getTableDescriptors().get(parentHri.getTable()).
4205           getRegionReplication();
4206     } catch (IOException e) {
4207       LOG.warn("Couldn't get the replication attribute of the table " + parentHri.getTable() +
4208           " due to " + e.getMessage() + ". The assignment of daughter replicas " +
4209           "replicas will not be done");
4210     }
4211     // unassign the old replicas
4212     List<HRegionInfo> parentRegion = new ArrayList<HRegionInfo>();
4213     parentRegion.add(parentHri);
4214     Map<ServerName, List<HRegionInfo>> currentAssign =
4215         regionStates.getRegionAssignments(parentRegion);
4216     Collection<List<HRegionInfo>> c = currentAssign.values();
4217     for (List<HRegionInfo> l : c) {
4218       for (HRegionInfo h : l) {
4219         if (!RegionReplicaUtil.isDefaultReplica(h)) {
4220           LOG.debug("Unassigning parent's replica " + h);
4221           unassign(h);
4222         }
4223       }
4224     }
4225     // assign daughter replicas
4226     Map<HRegionInfo, ServerName> map = new HashMap<HRegionInfo, ServerName>();
4227     for (int i = 1; i < numReplicas; i++) {
4228       prepareDaughterReplicaForAssignment(hri_a, parentHri, i, map);
4229       prepareDaughterReplicaForAssignment(hri_b, parentHri, i, map);
4230     }
4231     try {
4232       assign(map);
4233     } catch (IOException e) {
4234       LOG.warn("Caught exception " + e + " while trying to assign replica(s) of daughter(s)");
4235     } catch (InterruptedException e) {
4236       LOG.warn("Caught exception " + e + " while trying to assign replica(s) of daughter(s)");
4237     }
4238   }
4239 
4240   private void prepareDaughterReplicaForAssignment(HRegionInfo daughterHri, HRegionInfo parentHri,
4241       int replicaId, Map<HRegionInfo, ServerName> map) {
4242     HRegionInfo parentReplica = RegionReplicaUtil.getRegionInfoForReplica(parentHri, replicaId);
4243     HRegionInfo daughterReplica = RegionReplicaUtil.getRegionInfoForReplica(daughterHri,
4244         replicaId);
4245     LOG.debug("Created replica region for daughter " + daughterReplica);
4246     ServerName sn;
4247     if ((sn = regionStates.getRegionServerOfRegion(parentReplica)) != null) {
4248       map.put(daughterReplica, sn);
4249     } else {
4250       List<ServerName> servers = serverManager.getOnlineServersList();
4251       sn = servers.get((new Random(System.currentTimeMillis())).nextInt(servers.size()));
4252       map.put(daughterReplica, sn);
4253     }
4254   }
4255 
4256   public Set<HRegionInfo> getReplicasToClose() {
4257     return replicasToClose;
4258   }
4259 
4260   /**
4261    * A region is offline.  The new state should be the specified one,
4262    * if not null.  If the specified state is null, the new state is Offline.
4263    * The specified state can be Split/Merged/Offline/null only.
4264    */
4265   private void regionOffline(final HRegionInfo regionInfo, final State state) {
4266     regionStates.regionOffline(regionInfo, state);
4267     removeClosedRegion(regionInfo);
4268     // remove the region plan as well just in case.
4269     clearRegionPlan(regionInfo);
4270     balancer.regionOffline(regionInfo);
4271 
4272     // Tell our listeners that a region was closed
4273     sendRegionClosedNotification(regionInfo);
4274     // also note that all the replicas of the primary should be closed
4275     if (state != null && state.equals(State.SPLIT)) {
4276       Collection<HRegionInfo> c = new ArrayList<HRegionInfo>(1);
4277       c.add(regionInfo);
4278       Map<ServerName, List<HRegionInfo>> map = regionStates.getRegionAssignments(c);
4279       Collection<List<HRegionInfo>> allReplicas = map.values();
4280       for (List<HRegionInfo> list : allReplicas) {
4281         replicasToClose.addAll(list);
4282       }
4283     }
4284     else if (state != null && state.equals(State.MERGED)) {
4285       Collection<HRegionInfo> c = new ArrayList<HRegionInfo>(1);
4286       c.add(regionInfo);
4287       Map<ServerName, List<HRegionInfo>> map = regionStates.getRegionAssignments(c);
4288       Collection<List<HRegionInfo>> allReplicas = map.values();
4289       for (List<HRegionInfo> list : allReplicas) {
4290         replicasToClose.addAll(list);
4291       }
4292     }
4293   }
4294 
4295   private void sendRegionOpenedNotification(final HRegionInfo regionInfo,
4296       final ServerName serverName) {
4297     if (!this.listeners.isEmpty()) {
4298       for (AssignmentListener listener : this.listeners) {
4299         listener.regionOpened(regionInfo, serverName);
4300       }
4301     }
4302   }
4303 
4304   private void sendRegionClosedNotification(final HRegionInfo regionInfo) {
4305     if (!this.listeners.isEmpty()) {
4306       for (AssignmentListener listener : this.listeners) {
4307         listener.regionClosed(regionInfo);
4308       }
4309     }
4310   }
4311 
4312   /**
4313    * Try to update some region states. If the state machine prevents
4314    * such update, an error message is returned to explain the reason.
4315    *
4316    * It's expected that in each transition there should have just one
4317    * region for opening/closing, 3 regions for splitting/merging.
4318    * These regions should be on the server that requested the change.
4319    *
4320    * Region state machine. Only these transitions
4321    * are expected to be triggered by a region server.
4322    *
4323    * On the state transition:
4324    *  (1) Open/Close should be initiated by master
4325    *      (a) Master sets the region to pending_open/pending_close
4326    *        in memory and hbase:meta after sending the request
4327    *        to the region server
4328    *      (b) Region server reports back to the master
4329    *        after open/close is done (either success/failure)
4330    *      (c) If region server has problem to report the status
4331    *        to master, it must be because the master is down or some
4332    *        temporary network issue. Otherwise, the region server should
4333    *        abort since it must be a bug. If the master is not accessible,
4334    *        the region server should keep trying until the server is
4335    *        stopped or till the status is reported to the (new) master
4336    *      (d) If region server dies in the middle of opening/closing
4337    *        a region, SSH picks it up and finishes it
4338    *      (e) If master dies in the middle, the new master recovers
4339    *        the state during initialization from hbase:meta. Region server
4340    *        can report any transition that has not been reported to
4341    *        the previous active master yet
4342    *  (2) Split/merge is initiated by region servers
4343    *      (a) To split a region, a region server sends a request
4344    *        to master to try to set a region to splitting, together with
4345    *        two daughters (to be created) to splitting new. If approved
4346    *        by the master, the splitting can then move ahead
4347    *      (b) To merge two regions, a region server sends a request to
4348    *        master to try to set the new merged region (to be created) to
4349    *        merging_new, together with two regions (to be merged) to merging.
4350    *        If it is ok with the master, the merge can then move ahead
4351    *      (c) Once the splitting/merging is done, the region server
4352    *        reports the status back to the master either success/failure.
4353    *      (d) Other scenarios should be handled similarly as for
4354    *        region open/close
4355    */
4356   protected String onRegionTransition(final ServerName serverName,
4357       final RegionStateTransition transition) {
4358     TransitionCode code = transition.getTransitionCode();
4359     HRegionInfo hri = HRegionInfo.convert(transition.getRegionInfo(0));
4360     RegionState current = regionStates.getRegionState(hri);
4361     if (LOG.isDebugEnabled()) {
4362       LOG.debug("Got transition " + code + " for "
4363         + (current != null ? current.toString() : hri.getShortNameToLog())
4364         + " from " + serverName);
4365     }
4366     String errorMsg = null;
4367     switch (code) {
4368     case OPENED:
4369       if (current != null && current.isOpened() && current.isOnServer(serverName)) {
4370         LOG.info("Region " + hri.getShortNameToLog() + " is already " + current.getState() + " on "
4371             + serverName);
4372         break;
4373       }
4374     case FAILED_OPEN:
4375       if (current == null
4376           || !current.isPendingOpenOrOpeningOnServer(serverName)) {
4377         errorMsg = hri.getShortNameToLog()
4378           + " is not pending open on " + serverName;
4379       } else if (code == TransitionCode.FAILED_OPEN) {
4380         onRegionFailedOpen(hri, serverName);
4381       } else {
4382         long openSeqNum = HConstants.NO_SEQNUM;
4383         if (transition.hasOpenSeqNum()) {
4384           openSeqNum = transition.getOpenSeqNum();
4385         }
4386         if (openSeqNum < 0) {
4387           errorMsg = "Newly opened region has invalid open seq num " + openSeqNum;
4388         } else {
4389           onRegionOpen(hri, serverName, openSeqNum);
4390         }
4391       }
4392       break;
4393 
4394     case CLOSED:
4395       if (current == null
4396           || !current.isPendingCloseOrClosingOnServer(serverName)) {
4397         errorMsg = hri.getShortNameToLog()
4398           + " is not pending close on " + serverName;
4399       } else {
4400         onRegionClosed(hri);
4401       }
4402       break;
4403 
4404     case READY_TO_SPLIT:
4405       try {
4406         regionStateListener.onRegionSplit(hri);
4407       } catch (IOException exp) {
4408         errorMsg = StringUtils.stringifyException(exp);
4409       }
4410       break;
4411     case SPLIT_PONR:
4412     case SPLIT:
4413       errorMsg =
4414       onRegionSplit(serverName, code, hri, HRegionInfo.convert(transition.getRegionInfo(1)),
4415         HRegionInfo.convert(transition.getRegionInfo(2)));
4416       break;
4417 
4418     case SPLIT_REVERTED:
4419       errorMsg =
4420           onRegionSplitReverted(serverName, hri,
4421             HRegionInfo.convert(transition.getRegionInfo(1)),
4422             HRegionInfo.convert(transition.getRegionInfo(2)));
4423       if (org.apache.commons.lang.StringUtils.isEmpty(errorMsg)) {
4424         try {
4425           regionStateListener.onRegionSplitReverted(hri);
4426         } catch (IOException exp) {
4427           LOG.warn(StringUtils.stringifyException(exp));
4428         }
4429       }
4430       break;
4431     case READY_TO_MERGE:
4432     case MERGE_PONR:
4433     case MERGED:
4434       errorMsg = onRegionMerge(serverName, code, hri,
4435         HRegionInfo.convert(transition.getRegionInfo(1)),
4436         HRegionInfo.convert(transition.getRegionInfo(2)));
4437       if (code == TransitionCode.MERGED && org.apache.commons.lang.StringUtils.isEmpty(errorMsg)) {
4438         try {
4439           regionStateListener.onRegionMerged(hri);
4440         } catch (IOException exp) {
4441           errorMsg = StringUtils.stringifyException(exp);
4442         }
4443       }
4444       break;
4445     case MERGE_REVERTED:
4446         errorMsg = onRegionMergeReverted(serverName, code, hri,
4447                 HRegionInfo.convert(transition.getRegionInfo(1)),
4448                 HRegionInfo.convert(transition.getRegionInfo(2)));
4449       break;
4450 
4451     default:
4452       errorMsg = "Unexpected transition code " + code;
4453     }
4454     if (errorMsg != null) {
4455       LOG.error("Failed to transtion region from " + current + " to "
4456         + code + " by " + serverName + ": " + errorMsg);
4457     }
4458     return errorMsg;
4459   }
4460 
4461   /**
4462    * @return Instance of load balancer
4463    */
4464   public LoadBalancer getBalancer() {
4465     return this.balancer;
4466   }
4467 
4468   public Map<ServerName, List<HRegionInfo>>
4469     getSnapShotOfAssignment(Collection<HRegionInfo> infos) {
4470     return getRegionStates().getRegionAssignments(infos);
4471   }
4472 
4473   void setRegionStateListener(RegionStateListener listener) {
4474     this.regionStateListener = listener;
4475   }
4476 }