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