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