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.annotations.VisibleForTesting;
22  
23  import java.io.IOException;
24  import java.util.ArrayList;
25  import java.util.Collection;
26  import java.util.Collections;
27  import java.util.HashMap;
28  import java.util.HashSet;
29  import java.util.Iterator;
30  import java.util.List;
31  import java.util.Map;
32  import java.util.NavigableMap;
33  import java.util.Random;
34  import java.util.Set;
35  import java.util.TreeMap;
36  import java.util.concurrent.Callable;
37  import java.util.concurrent.ConcurrentHashMap;
38  import java.util.concurrent.CopyOnWriteArrayList;
39  import java.util.concurrent.TimeUnit;
40  import java.util.concurrent.atomic.AtomicBoolean;
41  import java.util.concurrent.atomic.AtomicInteger;
42  import java.util.concurrent.locks.Lock;
43  import java.util.concurrent.locks.ReentrantLock;
44  
45  import org.apache.commons.logging.Log;
46  import org.apache.commons.logging.LogFactory;
47  import org.apache.hadoop.conf.Configuration;
48  import org.apache.hadoop.fs.FileStatus;
49  import org.apache.hadoop.fs.FileSystem;
50  import org.apache.hadoop.fs.Path;
51  import org.apache.hadoop.hbase.CoordinatedStateException;
52  import org.apache.hadoop.hbase.HBaseIOException;
53  import org.apache.hadoop.hbase.HConstants;
54  import org.apache.hadoop.hbase.HRegionInfo;
55  import org.apache.hadoop.hbase.HRegionLocation;
56  import org.apache.hadoop.hbase.HTableDescriptor;
57  import org.apache.hadoop.hbase.MetaTableAccessor;
58  import org.apache.hadoop.hbase.NotServingRegionException;
59  import org.apache.hadoop.hbase.RegionLocations;
60  import org.apache.hadoop.hbase.RegionStateListener;
61  import org.apache.hadoop.hbase.ServerName;
62  import org.apache.hadoop.hbase.TableName;
63  import org.apache.hadoop.hbase.TableNotFoundException;
64  import org.apache.hadoop.hbase.classification.InterfaceAudience;
65  import org.apache.hadoop.hbase.client.MasterSwitchType;
66  import org.apache.hadoop.hbase.client.RegionReplicaUtil;
67  import org.apache.hadoop.hbase.client.Result;
68  import org.apache.hadoop.hbase.client.TableState;
69  import org.apache.hadoop.hbase.executor.EventHandler;
70  import org.apache.hadoop.hbase.executor.EventType;
71  import org.apache.hadoop.hbase.executor.ExecutorService;
72  import org.apache.hadoop.hbase.ipc.FailedServerException;
73  import org.apache.hadoop.hbase.ipc.RpcClient;
74  import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
75  import org.apache.hadoop.hbase.master.RegionState.State;
76  import org.apache.hadoop.hbase.master.balancer.FavoredNodeAssignmentHelper;
77  import org.apache.hadoop.hbase.master.balancer.FavoredNodeLoadBalancer;
78  import org.apache.hadoop.hbase.master.handler.DisableTableHandler;
79  import org.apache.hadoop.hbase.master.handler.EnableTableHandler;
80  import org.apache.hadoop.hbase.master.normalizer.NormalizationPlan.PlanType;
81  import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionStateTransition;
82  import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionStateTransition.TransitionCode;
83  import org.apache.hadoop.hbase.quotas.QuotaExceededException;
84  import org.apache.hadoop.hbase.regionserver.RegionOpeningState;
85  import org.apache.hadoop.hbase.regionserver.RegionServerAbortedException;
86  import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
87  import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
88  import org.apache.hadoop.hbase.util.FSUtils;
89  import org.apache.hadoop.hbase.util.KeyLocker;
90  import org.apache.hadoop.hbase.util.Pair;
91  import org.apache.hadoop.hbase.util.PairOfSameType;
92  import org.apache.hadoop.hbase.util.Threads;
93  import org.apache.hadoop.hbase.wal.AbstractFSWALProvider;
94  import org.apache.hadoop.hbase.zookeeper.MetaTableLocator;
95  import org.apache.hadoop.ipc.RemoteException;
96  import org.apache.hadoop.util.StringUtils;
97  import org.apache.zookeeper.KeeperException;
98  
99  /**
100  * Manages and performs region assignment.
101  * Related communications with regionserver are all done over RPC.
102  */
103 @InterfaceAudience.Private
104 public class AssignmentManager {
105   private static final Log LOG = LogFactory.getLog(AssignmentManager.class);
106 
107   protected final MasterServices server;
108 
109   private ServerManager serverManager;
110 
111   private boolean shouldAssignRegionsWithFavoredNodes;
112 
113   private LoadBalancer balancer;
114 
115   private final MetricsAssignmentManager metricsAssignmentManager;
116 
117   private final TableLockManager tableLockManager;
118 
119   private AtomicInteger numRegionsOpened = new AtomicInteger(0);
120 
121   final private KeyLocker<String> locker = new KeyLocker<String>();
122 
123   Set<HRegionInfo> replicasToClose = Collections.synchronizedSet(new HashSet<HRegionInfo>());
124 
125   /**
126    * Map of regions to reopen after the schema of a table is changed. Key -
127    * encoded region name, value - HRegionInfo
128    */
129   private final Map <String, HRegionInfo> regionsToReopen;
130 
131   /*
132    * Maximum times we recurse an assignment/unassignment.
133    * See below in {@link #assign()} and {@link #unassign()}.
134    */
135   private final int maximumAttempts;
136 
137   /**
138    * The sleep time for which the assignment will wait before retrying in case of
139    * hbase:meta assignment failure due to lack of availability of region plan or bad region plan
140    */
141   private final long sleepTimeBeforeRetryingMetaAssignment;
142 
143   /** Plans for region movement. Key is the encoded version of a region name*/
144   // TODO: When do plans get cleaned out?  Ever? In server open and in server
145   // shutdown processing -- St.Ack
146   // All access to this Map must be synchronized.
147   final NavigableMap<String, RegionPlan> regionPlans =
148     new TreeMap<String, RegionPlan>();
149 
150   private final TableStateManager tableStateManager;
151 
152   private final ExecutorService executorService;
153 
154   // Thread pool executor service. TODO, consolidate with executorService?
155   private java.util.concurrent.ExecutorService threadPoolExecutorService;
156 
157   private final RegionStates regionStates;
158 
159   // The threshold to use bulk assigning. Using bulk assignment
160   // only if assigning at least this many regions to at least this
161   // many servers. If assigning fewer regions to fewer servers,
162   // bulk assigning may be not as efficient.
163   private final int bulkAssignThresholdRegions;
164   private final int bulkAssignThresholdServers;
165   private final int bulkPerRegionOpenTimeGuesstimate;
166 
167   // Should bulk assignment wait till all regions are assigned,
168   // or it is timed out?  This is useful to measure bulk assignment
169   // performance, but not needed in most use cases.
170   private final boolean bulkAssignWaitTillAllAssigned;
171 
172   /**
173    * Indicator that AssignmentManager has recovered the region states so
174    * that ServerShutdownHandler can be fully enabled and re-assign regions
175    * of dead servers. So that when re-assignment happens, AssignmentManager
176    * has proper region states.
177    *
178    * Protected to ease testing.
179    */
180   protected final AtomicBoolean failoverCleanupDone = new AtomicBoolean(false);
181 
182   /**
183    * A map to track the count a region fails to open in a row.
184    * So that we don't try to open a region forever if the failure is
185    * unrecoverable.  We don't put this information in region states
186    * because we don't expect this to happen frequently; we don't
187    * want to copy this information over during each state transition either.
188    */
189   private final ConcurrentHashMap<String, AtomicInteger>
190     failedOpenTracker = new ConcurrentHashMap<String, AtomicInteger>();
191 
192   // In case not using ZK for region assignment, region states
193   // are persisted in meta with a state store
194   private final RegionStateStore regionStateStore;
195 
196   /**
197    * For testing only!  Set to true to skip handling of split.
198    */
199   @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="MS_SHOULD_BE_FINAL")
200   public static boolean TEST_SKIP_SPLIT_HANDLING = false;
201 
202   /** Listeners that are called on assignment events. */
203   private List<AssignmentListener> listeners = new CopyOnWriteArrayList<AssignmentListener>();
204 
205   private RegionStateListener regionStateListener;
206 
207   /**
208    * Constructs a new assignment manager.
209    *
210    * @param server instance of HMaster this AM running inside
211    * @param serverManager serverManager for associated HMaster
212    * @param balancer implementation of {@link LoadBalancer}
213    * @param service Executor service
214    * @param metricsMaster metrics manager
215    * @param tableLockManager TableLock manager
216    * @throws IOException
217    */
218   public AssignmentManager(MasterServices server, ServerManager serverManager,
219       final LoadBalancer balancer,
220       final ExecutorService service, MetricsMaster metricsMaster,
221       final TableLockManager tableLockManager,
222       final TableStateManager tableStateManager)
223           throws IOException {
224     this.server = server;
225     this.serverManager = serverManager;
226     this.executorService = service;
227     this.regionStateStore = new RegionStateStore(server);
228     this.regionsToReopen = Collections.synchronizedMap
229                            (new HashMap<String, HRegionInfo> ());
230     Configuration conf = server.getConfiguration();
231     // Only read favored nodes if using the favored nodes load balancer.
232     this.shouldAssignRegionsWithFavoredNodes = conf.getClass(
233            HConstants.HBASE_MASTER_LOADBALANCER_CLASS, Object.class).equals(
234            FavoredNodeLoadBalancer.class);
235 
236     this.tableStateManager = tableStateManager;
237 
238     // This is the max attempts, not retries, so it should be at least 1.
239     this.maximumAttempts = Math.max(1,
240       this.server.getConfiguration().getInt("hbase.assignment.maximum.attempts", 10));
241     this.sleepTimeBeforeRetryingMetaAssignment = this.server.getConfiguration().getLong(
242         "hbase.meta.assignment.retry.sleeptime", 1000l);
243     this.balancer = balancer;
244     int maxThreads = conf.getInt("hbase.assignment.threads.max", 30);
245     this.threadPoolExecutorService = Threads.getBoundedCachedThreadPool(
246       maxThreads, 60L, TimeUnit.SECONDS, Threads.newDaemonThreadFactory("AM."));
247     this.regionStates = new RegionStates(
248       server, tableStateManager, serverManager, regionStateStore);
249 
250     this.bulkAssignWaitTillAllAssigned =
251       conf.getBoolean("hbase.bulk.assignment.waittillallassigned", false);
252     this.bulkAssignThresholdRegions = conf.getInt("hbase.bulk.assignment.threshold.regions", 7);
253     this.bulkAssignThresholdServers = conf.getInt("hbase.bulk.assignment.threshold.servers", 3);
254     this.bulkPerRegionOpenTimeGuesstimate =
255       conf.getInt("hbase.bulk.assignment.perregion.open.time", 10000);
256 
257     this.metricsAssignmentManager = new MetricsAssignmentManager();
258     this.tableLockManager = tableLockManager;
259   }
260 
261   /**
262    * Add the listener to the notification list.
263    * @param listener The AssignmentListener to register
264    */
265   public void registerListener(final AssignmentListener listener) {
266     this.listeners.add(listener);
267   }
268 
269   /**
270    * Remove the listener from the notification list.
271    * @param listener The AssignmentListener to unregister
272    */
273   public boolean unregisterListener(final AssignmentListener listener) {
274     return this.listeners.remove(listener);
275   }
276 
277   /**
278    * @return Instance of ZKTableStateManager.
279    */
280   public TableStateManager getTableStateManager() {
281     // These are 'expensive' to make involving trip to zk ensemble so allow
282     // sharing.
283     return this.tableStateManager;
284   }
285 
286   /**
287    * This SHOULD not be public. It is public now
288    * because of some unit tests.
289    *
290    * TODO: make it package private and keep RegionStates in the master package
291    */
292   public RegionStates getRegionStates() {
293     return regionStates;
294   }
295 
296   /**
297    * Used in some tests to mock up region state in meta
298    */
299   @VisibleForTesting
300   RegionStateStore getRegionStateStore() {
301     return regionStateStore;
302   }
303 
304   public RegionPlan getRegionReopenPlan(HRegionInfo hri) {
305     return new RegionPlan(hri, null, regionStates.getRegionServerOfRegion(hri));
306   }
307 
308   /**
309    * Add a regionPlan for the specified region.
310    * @param encodedName
311    * @param plan
312    */
313   public void addPlan(String encodedName, RegionPlan plan) {
314     synchronized (regionPlans) {
315       regionPlans.put(encodedName, plan);
316     }
317   }
318 
319   /**
320    * Add a map of region plans.
321    */
322   public void addPlans(Map<String, RegionPlan> plans) {
323     synchronized (regionPlans) {
324       regionPlans.putAll(plans);
325     }
326   }
327 
328   /**
329    * Set the list of regions that will be reopened
330    * because of an update in table schema
331    *
332    * @param regions
333    *          list of regions that should be tracked for reopen
334    */
335   public void setRegionsToReopen(List <HRegionInfo> regions) {
336     for(HRegionInfo hri : regions) {
337       regionsToReopen.put(hri.getEncodedName(), hri);
338     }
339   }
340 
341   /**
342    * Used by the client to identify if all regions have the schema updates
343    *
344    * @param tableName
345    * @return Pair indicating the status of the alter command
346    * @throws IOException
347    */
348   public Pair<Integer, Integer> getReopenStatus(TableName tableName)
349       throws IOException {
350     List<HRegionInfo> hris;
351     if (TableName.META_TABLE_NAME.equals(tableName)) {
352       hris = new MetaTableLocator().getMetaRegions(server.getZooKeeper());
353     } else {
354       hris = MetaTableAccessor.getTableRegions(server.getConnection(), tableName, true);
355     }
356 
357     Integer pending = 0;
358     for (HRegionInfo hri : hris) {
359       String name = hri.getEncodedName();
360       // no lock concurrent access ok: sequential consistency respected.
361       if (regionsToReopen.containsKey(name)
362           || regionStates.isRegionInTransition(name)) {
363         pending++;
364       }
365     }
366     return new Pair<Integer, Integer>(pending, hris.size());
367   }
368 
369   /**
370    * Used by ServerShutdownHandler to make sure AssignmentManager has completed
371    * the failover cleanup before re-assigning regions of dead servers. So that
372    * when re-assignment happens, AssignmentManager has proper region states.
373    */
374   public boolean isFailoverCleanupDone() {
375     return failoverCleanupDone.get();
376   }
377 
378   /**
379    * To avoid racing with AM, external entities may need to lock a region,
380    * for example, when SSH checks what regions to skip re-assigning.
381    */
382   public Lock acquireRegionLock(final String encodedName) {
383     return locker.acquireLock(encodedName);
384   }
385 
386   /**
387    * Now, failover cleanup is completed. Notify server manager to
388    * process queued up dead servers processing, if any.
389    */
390   void failoverCleanupDone() {
391     failoverCleanupDone.set(true);
392     serverManager.processQueuedDeadServers();
393   }
394 
395   /**
396    * Called on startup.
397    * Figures whether a fresh cluster start of we are joining extant running cluster.
398    * @throws IOException
399    * @throws KeeperException
400    * @throws InterruptedException
401    * @throws CoordinatedStateException
402    */
403   void joinCluster()
404   throws IOException, KeeperException, InterruptedException, CoordinatedStateException {
405     long startTime = System.currentTimeMillis();
406     // Concurrency note: In the below the accesses on regionsInTransition are
407     // outside of a synchronization block where usually all accesses to RIT are
408     // synchronized.  The presumption is that in this case it is safe since this
409     // method is being played by a single thread on startup.
410 
411     // TODO: Regions that have a null location and are not in regionsInTransitions
412     // need to be handled.
413 
414     // Scan hbase:meta to build list of existing regions, servers, and assignment
415     // Returns servers who have not checked in (assumed dead) that some regions
416     // were assigned to (according to the meta)
417     Set<ServerName> deadServers = rebuildUserRegions();
418 
419     // This method will assign all user regions if a clean server startup or
420     // it will reconstruct master state and cleanup any leftovers from previous master process.
421     boolean failover = processDeadServersAndRegionsInTransition(deadServers);
422 
423     recoverTableInDisablingState();
424     recoverTableInEnablingState();
425     LOG.info("Joined the cluster in " + (System.currentTimeMillis()
426       - startTime) + "ms, failover=" + failover);
427   }
428 
429   /**
430    * Process all regions that are in transition in zookeeper and also
431    * processes the list of dead servers.
432    * Used by master joining an cluster.  If we figure this is a clean cluster
433    * startup, will assign all user regions.
434    * @param deadServers Set of servers that are offline probably legitimately that were carrying
435    * regions according to a scan of hbase:meta. Can be null.
436    * @throws IOException
437    * @throws InterruptedException
438    */
439   boolean processDeadServersAndRegionsInTransition(final Set<ServerName> deadServers)
440   throws KeeperException, IOException, InterruptedException, CoordinatedStateException {
441     // TODO Needed? List<String> nodes = ZKUtil.listChildrenNoWatch(watcher, watcher.assignmentZNode);
442     boolean failover = !serverManager.getDeadServers().isEmpty();
443     if (failover) {
444       // This may not be a failover actually, especially if meta is on this master.
445       if (LOG.isDebugEnabled()) {
446         LOG.debug("Found dead servers out on cluster " + serverManager.getDeadServers());
447       }
448       // Check if there are any regions on these servers
449       failover = false;
450       for (ServerName serverName : serverManager.getDeadServers().copyServerNames()) {
451         if (regionStates.getRegionAssignments().values().contains(serverName)) {
452           LOG.debug("Found regions on dead server: " + serverName);
453           failover = true;
454           break;
455         }
456       }
457     }
458     Set<ServerName> onlineServers = serverManager.getOnlineServers().keySet();
459     if (!failover) {
460       // If any one region except meta is assigned, it's a failover.
461       for (Map.Entry<HRegionInfo, ServerName> en:
462           regionStates.getRegionAssignments().entrySet()) {
463         HRegionInfo hri = en.getKey();
464         if (!hri.isMetaTable()
465             && onlineServers.contains(en.getValue())) {
466           LOG.debug("Found region " + hri + " out on cluster");
467           failover = true;
468           break;
469         }
470       }
471     }
472     if (!failover) {
473       // If any region except meta is in transition on a live server, it's a failover.
474       Set<RegionState> regionsInTransition = regionStates.getRegionsInTransition();
475       if (!regionsInTransition.isEmpty()) {
476         for (RegionState regionState: regionsInTransition) {
477           ServerName serverName = regionState.getServerName();
478           if (!regionState.getRegion().isMetaRegion()
479               && serverName != null && onlineServers.contains(serverName)) {
480             LOG.debug("Found " + regionState + " for region " +
481               regionState.getRegion().getRegionNameAsString() + " for server " +
482                 serverName + "in RITs");
483             failover = true;
484             break;
485           }
486         }
487       }
488     }
489     if (!failover) {
490       // If we get here, we have a full cluster restart. It is a failover only
491       // if there are some WALs are not split yet. For meta WALs, they should have
492       // been split already, if any. We can walk through those queued dead servers,
493       // if they don't have any WALs, this restart should be considered as a clean one
494       Set<ServerName> queuedDeadServers = serverManager.getRequeuedDeadServers().keySet();
495       if (!queuedDeadServers.isEmpty()) {
496         Configuration conf = server.getConfiguration();
497         Path rootdir = FSUtils.getRootDir(conf);
498         FileSystem fs = rootdir.getFileSystem(conf);
499         for (ServerName serverName: queuedDeadServers) {
500           // In the case of a clean exit, the shutdown handler would have presplit any WALs and
501           // removed empty directories.
502           Path logDir = new Path(rootdir,
503             AbstractFSWALProvider.getWALDirectoryName(serverName.toString()));
504           Path splitDir = logDir.suffix(AbstractFSWALProvider.SPLITTING_EXT);
505           if (checkWals(fs, logDir) || checkWals(fs, splitDir)) {
506             LOG.debug("Found queued dead server " + serverName);
507             failover = true;
508             break;
509           }
510         }
511         if (!failover) {
512           // We figured that it's not a failover, so no need to
513           // work on these re-queued dead servers any more.
514           LOG.info("AM figured that it's not a failover and cleaned up "
515             + queuedDeadServers.size() + " queued dead servers");
516           serverManager.removeRequeuedDeadServers();
517         }
518       }
519     }
520 
521     Set<TableName> disabledOrDisablingOrEnabling = null;
522     Map<HRegionInfo, ServerName> allRegions = null;
523 
524     if (!failover) {
525       disabledOrDisablingOrEnabling = tableStateManager.getTablesInStates(
526         TableState.State.DISABLED, TableState.State.DISABLING,
527         TableState.State.ENABLING);
528 
529       // Clean re/start, mark all user regions closed before reassignment
530       allRegions = regionStates.closeAllUserRegions(
531         disabledOrDisablingOrEnabling);
532     }
533 
534     // Now region states are restored
535     regionStateStore.start();
536 
537     if (failover) {
538       if (deadServers != null && !deadServers.isEmpty()) {
539         for (ServerName serverName: deadServers) {
540           if (!serverManager.isServerDead(serverName)) {
541             serverManager.expireServer(serverName); // Let SSH do region re-assign
542           }
543         }
544       }
545       processRegionsInTransition(regionStates.getRegionsInTransition());
546     }
547 
548     // Now we can safely claim failover cleanup completed and enable
549     // ServerShutdownHandler for further processing. The nodes (below)
550     // in transition, if any, are for regions not related to those
551     // dead servers at all, and can be done in parallel to SSH.
552     failoverCleanupDone();
553     if (!failover) {
554       // Fresh cluster startup.
555       LOG.info("Clean cluster startup. Don't reassign user regions");
556       assignAllUserRegions(allRegions);
557     } else {
558       LOG.info("Failover! Reassign user regions");
559     }
560     // unassign replicas of the split parents and the merged regions
561     // the daughter replicas are opened in assignAllUserRegions if it was
562     // not already opened.
563     for (HRegionInfo h : replicasToClose) {
564       unassign(h);
565     }
566     replicasToClose.clear();
567     return failover;
568   }
569 
570   private boolean checkWals(FileSystem fs, Path dir) throws IOException {
571     if (!fs.exists(dir)) {
572       LOG.debug(dir + " doesn't exist");
573       return false;
574     }
575     if (!fs.getFileStatus(dir).isDirectory()) {
576       LOG.warn(dir + " is not a directory");
577       return false;
578     }
579     FileStatus[] files = FSUtils.listStatus(fs, dir);
580     if (files == null || files.length == 0) {
581       LOG.debug(dir + " has no files");
582       return false;
583     }
584     for (int i = 0; i < files.length; i++) {
585       if (files[i].isFile() && files[i].getLen() > 0) {
586         LOG.debug(dir + " has a non-empty file: " + files[i].getPath());
587         return true;
588       } else if (files[i].isDirectory() && checkWals(fs, dir)) {
589         LOG.debug(dir + " is a directory and has a non-empty file: " + files[i].getPath());
590         return true;
591       }
592     }
593     LOG.debug("Found 0 non-empty wal files for :" + dir);
594     return false;
595   }
596 
597   /**
598    * When a region is closed, it should be removed from the regionsToReopen
599    * @param hri HRegionInfo of the region which was closed
600    */
601   public void removeClosedRegion(HRegionInfo hri) {
602     if (regionsToReopen.remove(hri.getEncodedName()) != null) {
603       LOG.debug("Removed region from reopening regions because it was closed");
604     }
605   }
606 
607   // TODO: processFavoredNodes might throw an exception, for e.g., if the
608   // meta could not be contacted/updated. We need to see how seriously to treat
609   // this problem as. Should we fail the current assignment. We should be able
610   // to recover from this problem eventually (if the meta couldn't be updated
611   // things should work normally and eventually get fixed up).
612   void processFavoredNodes(List<HRegionInfo> regions) throws IOException {
613     if (!shouldAssignRegionsWithFavoredNodes) return;
614     // The AM gets the favored nodes info for each region and updates the meta
615     // table with that info
616     Map<HRegionInfo, List<ServerName>> regionToFavoredNodes =
617         new HashMap<HRegionInfo, List<ServerName>>();
618     for (HRegionInfo region : regions) {
619       regionToFavoredNodes.put(region,
620           ((FavoredNodeLoadBalancer)this.balancer).getFavoredNodes(region));
621     }
622     FavoredNodeAssignmentHelper.updateMetaWithFavoredNodesInfo(regionToFavoredNodes,
623       this.server.getConnection());
624   }
625 
626   /**
627    * Marks the region as online.  Removes it from regions in transition and
628    * updates the in-memory assignment information.
629    * <p>
630    * Used when a region has been successfully opened on a region server.
631    * @param regionInfo
632    * @param sn
633    */
634   void regionOnline(HRegionInfo regionInfo, ServerName sn) {
635     regionOnline(regionInfo, sn, HConstants.NO_SEQNUM);
636   }
637 
638   void regionOnline(HRegionInfo regionInfo, ServerName sn, long openSeqNum) {
639     numRegionsOpened.incrementAndGet();
640     regionStates.regionOnline(regionInfo, sn, openSeqNum);
641 
642     // Remove plan if one.
643     clearRegionPlan(regionInfo);
644     balancer.regionOnline(regionInfo, sn);
645 
646     // Tell our listeners that a region was opened
647     sendRegionOpenedNotification(regionInfo, sn);
648   }
649 
650   /**
651    * Marks the region as offline.  Removes it from regions in transition and
652    * removes in-memory assignment information.
653    * <p>
654    * Used when a region has been closed and should remain closed.
655    * @param regionInfo
656    */
657   public void regionOffline(final HRegionInfo regionInfo) {
658     regionOffline(regionInfo, null);
659   }
660 
661   public void offlineDisabledRegion(HRegionInfo regionInfo) {
662     replicasToClose.remove(regionInfo);
663     regionOffline(regionInfo);
664   }
665 
666   // Assignment methods
667 
668   /**
669    * Assigns the specified region.
670    * <p>
671    * If a RegionPlan is available with a valid destination then it will be used
672    * to determine what server region is assigned to.  If no RegionPlan is
673    * available, region will be assigned to a random available server.
674    * <p>
675    * Updates the RegionState and sends the OPEN RPC.
676    * <p>
677    * This will only succeed if the region is in transition and in a CLOSED or
678    * OFFLINE state or not in transition, and of course, the
679    * chosen server is up and running (It may have just crashed!).
680    *
681    * @param region server to be assigned
682    */
683   public void assign(HRegionInfo region) {
684     assign(region, false);
685   }
686 
687   /**
688    * Use care with forceNewPlan. It could cause double assignment.
689    */
690   public void assign(HRegionInfo region, boolean forceNewPlan) {
691     if (isDisabledorDisablingRegionInRIT(region)) {
692       return;
693     }
694     String encodedName = region.getEncodedName();
695     Lock lock = locker.acquireLock(encodedName);
696     try {
697       RegionState state = forceRegionStateToOffline(region, forceNewPlan);
698       if (state != null) {
699         if (regionStates.wasRegionOnDeadServer(encodedName)) {
700           LOG.info("Skip assigning " + region.getRegionNameAsString()
701             + ", it's host " + regionStates.getLastRegionServerOfRegion(encodedName)
702             + " is dead but not processed yet");
703           return;
704         }
705         assign(state, forceNewPlan);
706       }
707     } finally {
708       lock.unlock();
709     }
710   }
711 
712   /**
713    * Bulk assign regions to <code>destination</code>.
714    * @param destination
715    * @param regions Regions to assign.
716    * @return true if successful
717    */
718   boolean assign(final ServerName destination, final List<HRegionInfo> regions)
719     throws InterruptedException {
720     long startTime = EnvironmentEdgeManager.currentTime();
721     try {
722       int regionCount = regions.size();
723       if (regionCount == 0) {
724         return true;
725       }
726       LOG.info("Assigning " + regionCount + " region(s) to " + destination.toString());
727       Set<String> encodedNames = new HashSet<String>(regionCount);
728       for (HRegionInfo region : regions) {
729         encodedNames.add(region.getEncodedName());
730       }
731 
732       List<HRegionInfo> failedToOpenRegions = new ArrayList<HRegionInfo>();
733       Map<String, Lock> locks = locker.acquireLocks(encodedNames);
734       try {
735         Map<String, RegionPlan> plans = new HashMap<String, RegionPlan>(regionCount);
736         List<RegionState> states = new ArrayList<RegionState>(regionCount);
737         for (HRegionInfo region : regions) {
738           String encodedName = region.getEncodedName();
739           if (!isDisabledorDisablingRegionInRIT(region)) {
740             RegionState state = forceRegionStateToOffline(region, false);
741             boolean onDeadServer = false;
742             if (state != null) {
743               if (regionStates.wasRegionOnDeadServer(encodedName)) {
744                 LOG.info("Skip assigning " + region.getRegionNameAsString()
745                   + ", it's host " + regionStates.getLastRegionServerOfRegion(encodedName)
746                   + " is dead but not processed yet");
747                 onDeadServer = true;
748               } else {
749                 RegionPlan plan = new RegionPlan(region, state.getServerName(), destination);
750                 plans.put(encodedName, plan);
751                 states.add(state);
752                 continue;
753               }
754             }
755             // Reassign if the region wasn't on a dead server
756             if (!onDeadServer) {
757               LOG.info("failed to force region state to offline, "
758                 + "will reassign later: " + region);
759               failedToOpenRegions.add(region); // assign individually later
760             }
761           }
762           // Release the lock, this region is excluded from bulk assign because
763           // we can't update its state, or set its znode to offline.
764           Lock lock = locks.remove(encodedName);
765           lock.unlock();
766         }
767 
768         if (server.isStopped()) {
769           return false;
770         }
771 
772         // Add region plans, so we can updateTimers when one region is opened so
773         // that unnecessary timeout on RIT is reduced.
774         this.addPlans(plans);
775 
776         List<Pair<HRegionInfo, List<ServerName>>> regionOpenInfos =
777           new ArrayList<Pair<HRegionInfo, List<ServerName>>>(states.size());
778         for (RegionState state: states) {
779           HRegionInfo region = state.getRegion();
780           regionStates.updateRegionState(
781             region, State.PENDING_OPEN, destination);
782           List<ServerName> favoredNodes = ServerName.EMPTY_SERVER_LIST;
783           if (this.shouldAssignRegionsWithFavoredNodes) {
784             favoredNodes = ((FavoredNodeLoadBalancer)this.balancer).getFavoredNodes(region);
785           }
786           regionOpenInfos.add(new Pair<HRegionInfo, List<ServerName>>(
787             region, favoredNodes));
788         }
789 
790         // Move on to open regions.
791         try {
792           // Send OPEN RPC. If it fails on a IOE or RemoteException,
793           // regions will be assigned individually.
794           Configuration conf = server.getConfiguration();
795           long maxWaitTime = System.currentTimeMillis() +
796             conf.getLong("hbase.regionserver.rpc.startup.waittime", 60000);
797           for (int i = 1; i <= maximumAttempts && !server.isStopped(); i++) {
798             try {
799               List<RegionOpeningState> regionOpeningStateList = serverManager
800                 .sendRegionOpen(destination, regionOpenInfos);
801               for (int k = 0, n = regionOpeningStateList.size(); k < n; k++) {
802                 RegionOpeningState openingState = regionOpeningStateList.get(k);
803                 if (openingState != RegionOpeningState.OPENED) {
804                   HRegionInfo region = regionOpenInfos.get(k).getFirst();
805                   LOG.info("Got opening state " + openingState
806                     + ", will reassign later: " + region);
807                   // Failed opening this region, reassign it later
808                   forceRegionStateToOffline(region, true);
809                   failedToOpenRegions.add(region);
810                 }
811               }
812               break;
813             } catch (IOException e) {
814               if (e instanceof RemoteException) {
815                 e = ((RemoteException)e).unwrapRemoteException();
816               }
817               if (e instanceof RegionServerStoppedException) {
818                 LOG.warn("The region server was shut down, ", e);
819                 // No need to retry, the region server is a goner.
820                 return false;
821               } else if (e instanceof ServerNotRunningYetException) {
822                 long now = System.currentTimeMillis();
823                 if (now < maxWaitTime) {
824                   if (LOG.isDebugEnabled()) {
825                     LOG.debug("Server is not yet up; waiting up to " +
826                       (maxWaitTime - now) + "ms", e);
827                   }
828                   Thread.sleep(100);
829                   i--; // reset the try count
830                   continue;
831                 }
832               } else if (e instanceof java.net.SocketTimeoutException
833                   && this.serverManager.isServerOnline(destination)) {
834                 // In case socket is timed out and the region server is still online,
835                 // the openRegion RPC could have been accepted by the server and
836                 // just the response didn't go through.  So we will retry to
837                 // open the region on the same server.
838                 if (LOG.isDebugEnabled()) {
839                   LOG.debug("Bulk assigner openRegion() to " + destination
840                     + " has timed out, but the regions might"
841                     + " already be opened on it.", e);
842                 }
843                 // wait and reset the re-try count, server might be just busy.
844                 Thread.sleep(100);
845                 i--;
846                 continue;
847               } else if (e instanceof FailedServerException && i < maximumAttempts) {
848                 // In case the server is in the failed server list, no point to
849                 // retry too soon. Retry after the failed_server_expiry time
850                 long sleepTime = 1 + conf.getInt(RpcClient.FAILED_SERVER_EXPIRY_KEY,
851                   RpcClient.FAILED_SERVER_EXPIRY_DEFAULT);
852                 if (LOG.isDebugEnabled()) {
853                   LOG.debug(destination + " is on failed server list; waiting "
854                     + sleepTime + "ms", e);
855                 }
856                 Thread.sleep(sleepTime);
857                 continue;
858               }
859               throw e;
860             }
861           }
862         } catch (IOException e) {
863           // Can be a socket timeout, EOF, NoRouteToHost, etc
864           LOG.info("Unable to communicate with " + destination
865             + " in order to assign regions, ", e);
866           for (RegionState state: states) {
867             HRegionInfo region = state.getRegion();
868             forceRegionStateToOffline(region, true);
869           }
870           return false;
871         }
872       } finally {
873         for (Lock lock : locks.values()) {
874           lock.unlock();
875         }
876       }
877 
878       if (!failedToOpenRegions.isEmpty()) {
879         for (HRegionInfo region : failedToOpenRegions) {
880           if (!regionStates.isRegionOnline(region)) {
881             invokeAssign(region);
882           }
883         }
884       }
885 
886       // wait for assignment completion
887       ArrayList<HRegionInfo> userRegionSet = new ArrayList<HRegionInfo>(regions.size());
888       for (HRegionInfo region: regions) {
889         if (!region.getTable().isSystemTable()) {
890           userRegionSet.add(region);
891         }
892       }
893       if (!waitForAssignment(userRegionSet, true, userRegionSet.size(),
894             System.currentTimeMillis())) {
895         LOG.debug("some user regions are still in transition: " + userRegionSet);
896       }
897       LOG.debug("Bulk assigning done for " + destination);
898       return true;
899     } finally {
900       metricsAssignmentManager.updateBulkAssignTime(EnvironmentEdgeManager.currentTime() - startTime);
901     }
902   }
903 
904   /**
905    * Send CLOSE RPC if the server is online, otherwise, offline the region.
906    *
907    * The RPC will be sent only to the region sever found in the region state
908    * if it is passed in, otherwise, to the src server specified. If region
909    * state is not specified, we don't update region state at all, instead
910    * we just send the RPC call. This is useful for some cleanup without
911    * messing around the region states (see handleRegion, on region opened
912    * on an unexpected server scenario, for an example)
913    */
914   private void unassign(final HRegionInfo region,
915       final ServerName server, final ServerName dest) {
916     for (int i = 1; i <= this.maximumAttempts; i++) {
917       if (this.server.isStopped() || this.server.isAborted()) {
918         LOG.debug("Server stopped/aborted; skipping unassign of " + region);
919         return;
920       }
921       if (!serverManager.isServerOnline(server)) {
922         LOG.debug("Offline " + region.getRegionNameAsString()
923           + ", no need to unassign since it's on a dead server: " + server);
924         regionStates.updateRegionState(region, State.OFFLINE);
925         return;
926       }
927       try {
928         // Send CLOSE RPC
929         if (serverManager.sendRegionClose(server, region, dest)) {
930           LOG.debug("Sent CLOSE to " + server + " for region " +
931             region.getRegionNameAsString());
932           return;
933         }
934         // This never happens. Currently regionserver close always return true.
935         // Todo; this can now happen (0.96) if there is an exception in a coprocessor
936         LOG.warn("Server " + server + " region CLOSE RPC returned false for " +
937           region.getRegionNameAsString());
938       } catch (Throwable t) {
939         long sleepTime = 0;
940         Configuration conf = this.server.getConfiguration();
941         if (t instanceof RemoteException) {
942           t = ((RemoteException)t).unwrapRemoteException();
943         }
944         if (t instanceof RegionServerAbortedException
945             || t instanceof RegionServerStoppedException
946             || t instanceof ServerNotRunningYetException) {
947           // RS is aborting, we cannot offline the region since the region may need to do WAL
948           // recovery. Until we see  the RS expiration, we should retry.
949           sleepTime = 1 + conf.getInt(RpcClient.FAILED_SERVER_EXPIRY_KEY,
950             RpcClient.FAILED_SERVER_EXPIRY_DEFAULT);
951 
952         } else if (t instanceof NotServingRegionException) {
953           LOG.debug("Offline " + region.getRegionNameAsString()
954             + ", it's not any more on " + server, t);
955           regionStates.updateRegionState(region, State.OFFLINE);
956           return;
957         } else if (t instanceof FailedServerException && i < maximumAttempts) {
958           // In case the server is in the failed server list, no point to
959           // retry too soon. Retry after the failed_server_expiry time
960           sleepTime = 1 + conf.getInt(RpcClient.FAILED_SERVER_EXPIRY_KEY,
961           RpcClient.FAILED_SERVER_EXPIRY_DEFAULT);
962           if (LOG.isDebugEnabled()) {
963             LOG.debug(server + " is on failed server list; waiting " + sleepTime + "ms", t);
964           }
965        }
966        try {
967          if (sleepTime > 0) {
968            Thread.sleep(sleepTime);
969          }
970        } catch (InterruptedException ie) {
971          LOG.warn("Interrupted unassign " + region.getRegionNameAsString(), ie);
972          Thread.currentThread().interrupt();
973          regionStates.updateRegionState(region, State.FAILED_CLOSE);
974          return;
975        }
976        LOG.info("Server " + server + " returned " + t + " for "
977          + region.getRegionNameAsString() + ", try=" + i
978          + " of " + this.maximumAttempts, t);
979       }
980     }
981     // Run out of attempts
982     regionStates.updateRegionState(region, State.FAILED_CLOSE);
983   }
984 
985   /**
986    * Set region to OFFLINE unless it is opening and forceNewPlan is false.
987    */
988   private RegionState forceRegionStateToOffline(
989       final HRegionInfo region, final boolean forceNewPlan) {
990     RegionState state = regionStates.getRegionState(region);
991     if (state == null) {
992       LOG.warn("Assigning but not in region states: " + region);
993       state = regionStates.createRegionState(region);
994     }
995 
996     if (forceNewPlan && LOG.isDebugEnabled()) {
997       LOG.debug("Force region state offline " + state);
998     }
999 
1000     switch (state.getState()) {
1001     case OPEN:
1002     case OPENING:
1003     case PENDING_OPEN:
1004     case CLOSING:
1005     case PENDING_CLOSE:
1006       if (!forceNewPlan) {
1007         LOG.debug("Skip assigning " +
1008           region + ", it is already " + state);
1009         return null;
1010       }
1011     case FAILED_CLOSE:
1012     case FAILED_OPEN:
1013       regionStates.updateRegionState(region, State.PENDING_CLOSE);
1014       unassign(region, state.getServerName(), null);
1015       state = regionStates.getRegionState(region);
1016       if (!state.isOffline() && !state.isClosed()) {
1017         // If the region isn't offline, we can't re-assign
1018         // it now. It will be assigned automatically after
1019         // the regionserver reports it's closed.
1020         return null;
1021       }
1022     case OFFLINE:
1023     case CLOSED:
1024       break;
1025     default:
1026       LOG.error("Trying to assign region " + region
1027         + ", which is " + state);
1028       return null;
1029     }
1030     return state;
1031   }
1032 
1033   /**
1034    * Caller must hold lock on the passed <code>state</code> object.
1035    * @param state
1036    * @param forceNewPlan
1037    */
1038   private void assign(RegionState state, boolean forceNewPlan) {
1039     long startTime = EnvironmentEdgeManager.currentTime();
1040     try {
1041       Configuration conf = server.getConfiguration();
1042       RegionPlan plan = null;
1043       long maxWaitTime = -1;
1044       HRegionInfo region = state.getRegion();
1045       Throwable previousException = null;
1046       for (int i = 1; i <= maximumAttempts; i++) {
1047         if (server.isStopped() || server.isAborted()) {
1048           LOG.info("Skip assigning " + region.getRegionNameAsString()
1049             + ", the server is stopped/aborted");
1050           return;
1051         }
1052 
1053         if (plan == null) { // Get a server for the region at first
1054           try {
1055             plan = getRegionPlan(region, forceNewPlan);
1056           } catch (HBaseIOException e) {
1057             LOG.warn("Failed to get region plan", e);
1058           }
1059         }
1060 
1061         if (plan == null) {
1062           LOG.warn("Unable to determine a plan to assign " + region);
1063 
1064           // For meta region, we have to keep retrying until succeeding
1065           if (region.isMetaRegion()) {
1066             if (i == maximumAttempts) {
1067               i = 0; // re-set attempt count to 0 for at least 1 retry
1068 
1069               LOG.warn("Unable to determine a plan to assign a hbase:meta region " + region +
1070                 " after maximumAttempts (" + this.maximumAttempts +
1071                 "). Reset attempts count and continue retrying.");
1072             }
1073             waitForRetryingMetaAssignment();
1074             continue;
1075           }
1076 
1077           regionStates.updateRegionState(region, State.FAILED_OPEN);
1078           return;
1079         }
1080         LOG.info("Assigning " + region.getRegionNameAsString() +
1081             " to " + plan.getDestination());
1082         // Transition RegionState to PENDING_OPEN
1083        regionStates.updateRegionState(region,
1084           State.PENDING_OPEN, plan.getDestination());
1085 
1086         boolean needNewPlan = false;
1087         final String assignMsg = "Failed assignment of " + region.getRegionNameAsString() +
1088             " to " + plan.getDestination();
1089         try {
1090           List<ServerName> favoredNodes = ServerName.EMPTY_SERVER_LIST;
1091           if (this.shouldAssignRegionsWithFavoredNodes) {
1092             favoredNodes = ((FavoredNodeLoadBalancer)this.balancer).getFavoredNodes(region);
1093           }
1094           serverManager.sendRegionOpen(plan.getDestination(), region, favoredNodes);
1095           return; // we're done
1096         } catch (Throwable t) {
1097           if (t instanceof RemoteException) {
1098             t = ((RemoteException) t).unwrapRemoteException();
1099           }
1100           previousException = t;
1101 
1102           // Should we wait a little before retrying? If the server is starting it's yes.
1103           boolean hold = (t instanceof ServerNotRunningYetException);
1104 
1105           // In case socket is timed out and the region server is still online,
1106           // the openRegion RPC could have been accepted by the server and
1107           // just the response didn't go through.  So we will retry to
1108           // open the region on the same server.
1109           boolean retry = !hold && (t instanceof java.net.SocketTimeoutException
1110               && this.serverManager.isServerOnline(plan.getDestination()));
1111 
1112           if (hold) {
1113             LOG.warn(assignMsg + ", waiting a little before trying on the same region server " +
1114               "try=" + i + " of " + this.maximumAttempts, t);
1115 
1116             if (maxWaitTime < 0) {
1117               maxWaitTime = EnvironmentEdgeManager.currentTime()
1118                 + this.server.getConfiguration().getLong(
1119                   "hbase.regionserver.rpc.startup.waittime", 60000);
1120             }
1121             try {
1122               long now = EnvironmentEdgeManager.currentTime();
1123               if (now < maxWaitTime) {
1124                 if (LOG.isDebugEnabled()) {
1125                   LOG.debug("Server is not yet up; waiting up to "
1126                     + (maxWaitTime - now) + "ms", t);
1127                 }
1128                 Thread.sleep(100);
1129                 i--; // reset the try count
1130               } else {
1131                 LOG.debug("Server is not up for a while; try a new one", t);
1132                 needNewPlan = true;
1133               }
1134             } catch (InterruptedException ie) {
1135               LOG.warn("Failed to assign "
1136                   + region.getRegionNameAsString() + " since interrupted", ie);
1137               regionStates.updateRegionState(region, State.FAILED_OPEN);
1138               Thread.currentThread().interrupt();
1139               return;
1140             }
1141           } else if (retry) {
1142             i--; // we want to retry as many times as needed as long as the RS is not dead.
1143             if (LOG.isDebugEnabled()) {
1144               LOG.debug(assignMsg + ", trying to assign to the same region server due ", t);
1145             }
1146           } else {
1147             needNewPlan = true;
1148             LOG.warn(assignMsg + ", trying to assign elsewhere instead;" +
1149                 " try=" + i + " of " + this.maximumAttempts, t);
1150           }
1151         }
1152 
1153         if (i == this.maximumAttempts) {
1154           // For meta region, we have to keep retrying until succeeding
1155           if (region.isMetaRegion()) {
1156             i = 0; // re-set attempt count to 0 for at least 1 retry
1157             LOG.warn(assignMsg +
1158                 ", trying to assign a hbase:meta region reached to maximumAttempts (" +
1159                 this.maximumAttempts + ").  Reset attempt counts and continue retrying.");
1160             waitForRetryingMetaAssignment();
1161           }
1162           else {
1163             // Don't reset the region state or get a new plan any more.
1164             // This is the last try.
1165             continue;
1166           }
1167         }
1168 
1169         // If region opened on destination of present plan, reassigning to new
1170         // RS may cause double assignments. In case of RegionAlreadyInTransitionException
1171         // reassigning to same RS.
1172         if (needNewPlan) {
1173           // Force a new plan and reassign. Will return null if no servers.
1174           // The new plan could be the same as the existing plan since we don't
1175           // exclude the server of the original plan, which should not be
1176           // excluded since it could be the only server up now.
1177           RegionPlan newPlan = null;
1178           try {
1179             newPlan = getRegionPlan(region, true);
1180           } catch (HBaseIOException e) {
1181             LOG.warn("Failed to get region plan", e);
1182           }
1183           if (newPlan == null) {
1184             regionStates.updateRegionState(region, State.FAILED_OPEN);
1185             LOG.warn("Unable to find a viable location to assign region " +
1186                 region.getRegionNameAsString());
1187             return;
1188           }
1189 
1190           if (plan != newPlan && !plan.getDestination().equals(newPlan.getDestination())) {
1191             // Clean out plan we failed execute and one that doesn't look like it'll
1192             // succeed anyways; we need a new plan!
1193             // Transition back to OFFLINE
1194             regionStates.updateRegionState(region, State.OFFLINE);
1195             plan = newPlan;
1196           } else if(plan.getDestination().equals(newPlan.getDestination()) &&
1197               previousException instanceof FailedServerException) {
1198             try {
1199               LOG.info("Trying to re-assign " + region.getRegionNameAsString() +
1200                 " to the same failed server.");
1201               Thread.sleep(1 + conf.getInt(RpcClient.FAILED_SERVER_EXPIRY_KEY,
1202                 RpcClient.FAILED_SERVER_EXPIRY_DEFAULT));
1203             } catch (InterruptedException ie) {
1204               LOG.warn("Failed to assign "
1205                   + region.getRegionNameAsString() + " since interrupted", ie);
1206               regionStates.updateRegionState(region, State.FAILED_OPEN);
1207               Thread.currentThread().interrupt();
1208               return;
1209             }
1210           }
1211         }
1212       }
1213       // Run out of attempts
1214       regionStates.updateRegionState(region, State.FAILED_OPEN);
1215     } finally {
1216       metricsAssignmentManager.updateAssignmentTime(EnvironmentEdgeManager.currentTime() - startTime);
1217     }
1218   }
1219 
1220   private boolean isDisabledorDisablingRegionInRIT(final HRegionInfo region) {
1221     if (this.tableStateManager.isTableState(region.getTable(),
1222             TableState.State.DISABLED,
1223             TableState.State.DISABLING) || replicasToClose.contains(region)) {
1224       LOG.info("Table " + region.getTable() + " is disabled or disabling;"
1225         + " skipping assign of " + region.getRegionNameAsString());
1226       offlineDisabledRegion(region);
1227       return true;
1228     }
1229     return false;
1230   }
1231 
1232   /**
1233    * @param region the region to assign
1234    * @param forceNewPlan If true, then if an existing plan exists, a new plan
1235    * will be generated.
1236    * @return Plan for passed <code>region</code> (If none currently, it creates one or
1237    * if no servers to assign, it returns null).
1238    */
1239   private RegionPlan getRegionPlan(final HRegionInfo region,
1240       final boolean forceNewPlan) throws HBaseIOException {
1241     // Pickup existing plan or make a new one
1242     final String encodedName = region.getEncodedName();
1243     final List<ServerName> destServers =
1244       serverManager.createDestinationServersList();
1245 
1246     if (destServers.isEmpty()){
1247       LOG.warn("Can't move " + encodedName +
1248         ", there is no destination server available.");
1249       return null;
1250     }
1251 
1252     RegionPlan randomPlan = null;
1253     boolean newPlan = false;
1254     RegionPlan existingPlan;
1255 
1256     synchronized (this.regionPlans) {
1257       existingPlan = this.regionPlans.get(encodedName);
1258 
1259       if (existingPlan != null && existingPlan.getDestination() != null) {
1260         LOG.debug("Found an existing plan for " + region.getRegionNameAsString()
1261           + " destination server is " + existingPlan.getDestination() +
1262             " accepted as a dest server = " + destServers.contains(existingPlan.getDestination()));
1263       }
1264 
1265       if (forceNewPlan
1266           || existingPlan == null
1267           || existingPlan.getDestination() == null
1268           || !destServers.contains(existingPlan.getDestination())) {
1269         newPlan = true;
1270         try {
1271           randomPlan = new RegionPlan(region, null,
1272               balancer.randomAssignment(region, destServers));
1273         } catch (IOException ex) {
1274           LOG.warn("Failed to create new plan.",ex);
1275           return null;
1276         }
1277         if (!region.isMetaTable() && shouldAssignRegionsWithFavoredNodes) {
1278           List<HRegionInfo> regions = new ArrayList<HRegionInfo>(1);
1279           regions.add(region);
1280           try {
1281             processFavoredNodes(regions);
1282           } catch (IOException ie) {
1283             LOG.warn("Ignoring exception in processFavoredNodes " + ie);
1284           }
1285         }
1286         this.regionPlans.put(encodedName, randomPlan);
1287       }
1288     }
1289 
1290     if (newPlan) {
1291       if (randomPlan.getDestination() == null) {
1292         LOG.warn("Can't find a destination for " + encodedName);
1293         return null;
1294       }
1295       if (LOG.isDebugEnabled()) {
1296         LOG.debug("No previous transition plan found (or ignoring " +
1297           "an existing plan) for " + region.getRegionNameAsString() +
1298           "; generated random plan=" + randomPlan + "; " + destServers.size() +
1299           " (online=" + serverManager.getOnlineServers().size() +
1300           ") available servers, forceNewPlan=" + forceNewPlan);
1301       }
1302       return randomPlan;
1303     }
1304     if (LOG.isDebugEnabled()) {
1305       LOG.debug("Using pre-existing plan for " +
1306         region.getRegionNameAsString() + "; plan=" + existingPlan);
1307     }
1308     return existingPlan;
1309   }
1310 
1311   /**
1312    * Wait for some time before retrying meta table region assignment
1313    */
1314   private void waitForRetryingMetaAssignment() {
1315     try {
1316       Thread.sleep(this.sleepTimeBeforeRetryingMetaAssignment);
1317     } catch (InterruptedException e) {
1318       LOG.error("Got exception while waiting for hbase:meta assignment");
1319       Thread.currentThread().interrupt();
1320     }
1321   }
1322 
1323   /**
1324    * Unassigns the specified region.
1325    * <p>
1326    * Updates the RegionState and sends the CLOSE RPC unless region is being
1327    * split by regionserver; then the unassign fails (silently) because we
1328    * presume the region being unassigned no longer exists (its been split out
1329    * of existence). TODO: What to do if split fails and is rolled back and
1330    * parent is revivified?
1331    * <p>
1332    * If a RegionPlan is already set, it will remain.
1333    *
1334    * @param region server to be unassigned
1335    */
1336   public void unassign(HRegionInfo region) {
1337     unassign(region, null);
1338   }
1339 
1340 
1341   /**
1342    * Unassigns the specified region.
1343    * <p>
1344    * Updates the RegionState and sends the CLOSE RPC unless region is being
1345    * split by regionserver; then the unassign fails (silently) because we
1346    * presume the region being unassigned no longer exists (its been split out
1347    * of existence). TODO: What to do if split fails and is rolled back and
1348    * parent is revivified?
1349    * <p>
1350    * If a RegionPlan is already set, it will remain.
1351    *
1352    * @param region server to be unassigned
1353    * @param dest the destination server of the region
1354    */
1355   public void unassign(HRegionInfo region, ServerName dest) {
1356     // TODO: Method needs refactoring.  Ugly buried returns throughout.  Beware!
1357     LOG.debug("Starting unassign of " + region.getRegionNameAsString()
1358       + " (offlining), current state: " + regionStates.getRegionState(region));
1359 
1360     String encodedName = region.getEncodedName();
1361     // Grab the state of this region and synchronize on it
1362     // We need a lock here as we're going to do a put later and we don't want multiple states
1363     //  creation
1364     ReentrantLock lock = locker.acquireLock(encodedName);
1365     RegionState state = regionStates.getRegionTransitionState(encodedName);
1366     try {
1367       if (state == null || state.isFailedClose()) {
1368         if (state == null) {
1369           // Region is not in transition.
1370           // We can unassign it only if it's not SPLIT/MERGED.
1371           state = regionStates.getRegionState(encodedName);
1372           if (state != null && state.isUnassignable()) {
1373             LOG.info("Attempting to unassign " + state + ", ignored");
1374             // Offline region will be reassigned below
1375             return;
1376           }
1377           if (state == null || state.getServerName() == null) {
1378             // We don't know where the region is, offline it.
1379             // No need to send CLOSE RPC
1380             LOG.warn("Attempting to unassign a region not in RegionStates "
1381               + region.getRegionNameAsString() + ", offlined");
1382             regionOffline(region);
1383             return;
1384           }
1385         }
1386         state = regionStates.updateRegionState(
1387           region, State.PENDING_CLOSE);
1388       } else if (state.isFailedOpen()) {
1389         // The region is not open yet
1390         regionOffline(region);
1391         return;
1392       } else {
1393         LOG.debug("Attempting to unassign " +
1394           region.getRegionNameAsString() + " but it is " +
1395           "already in transition (" + state.getState());
1396         return;
1397       }
1398 
1399       unassign(region, state.getServerName(), dest);
1400     } finally {
1401       lock.unlock();
1402 
1403       // Region is expected to be reassigned afterwards
1404       if (!replicasToClose.contains(region)
1405           && regionStates.isRegionInState(region, State.OFFLINE)) {
1406         assign(region);
1407       }
1408     }
1409   }
1410 
1411   /**
1412    * Used by unit tests. Return the number of regions opened so far in the life
1413    * of the master. Increases by one every time the master opens a region
1414    * @return the counter value of the number of regions opened so far
1415    */
1416   public int getNumRegionsOpened() {
1417     return numRegionsOpened.get();
1418   }
1419 
1420   /**
1421    * Waits until the specified region has completed assignment.
1422    * <p>
1423    * If the region is already assigned, returns immediately.  Otherwise, method
1424    * blocks until the region is assigned.
1425    * @param regionInfo region to wait on assignment for
1426    * @return true if the region is assigned false otherwise.
1427    * @throws InterruptedException
1428    */
1429   public boolean waitForAssignment(HRegionInfo regionInfo)
1430       throws InterruptedException {
1431     ArrayList<HRegionInfo> regionSet = new ArrayList<HRegionInfo>(1);
1432     regionSet.add(regionInfo);
1433     return waitForAssignment(regionSet, true, Long.MAX_VALUE);
1434   }
1435 
1436   /**
1437    * Waits until the specified region has completed assignment, or the deadline is reached.
1438    */
1439   protected boolean waitForAssignment(final Collection<HRegionInfo> regionSet,
1440       final boolean waitTillAllAssigned, final int reassigningRegions,
1441       final long minEndTime) throws InterruptedException {
1442     long deadline = minEndTime + bulkPerRegionOpenTimeGuesstimate * (reassigningRegions + 1);
1443     return waitForAssignment(regionSet, waitTillAllAssigned, deadline);
1444   }
1445 
1446   /**
1447    * Waits until the specified region has completed assignment, or the deadline is reached.
1448    * @param regionSet set of region to wait on. the set is modified and the assigned regions removed
1449    * @param waitTillAllAssigned true if we should wait all the regions to be assigned
1450    * @param deadline the timestamp after which the wait is aborted
1451    * @return true if all the regions are assigned false otherwise.
1452    * @throws InterruptedException
1453    */
1454   protected boolean waitForAssignment(final Collection<HRegionInfo> regionSet,
1455       final boolean waitTillAllAssigned, final long deadline) throws InterruptedException {
1456     // We're not synchronizing on regionsInTransition now because we don't use any iterator.
1457     while (!regionSet.isEmpty() && !server.isStopped() && deadline > System.currentTimeMillis()) {
1458       int failedOpenCount = 0;
1459       Iterator<HRegionInfo> regionInfoIterator = regionSet.iterator();
1460       while (regionInfoIterator.hasNext()) {
1461         HRegionInfo hri = regionInfoIterator.next();
1462         if (regionStates.isRegionOnline(hri) || regionStates.isRegionInState(hri,
1463             State.SPLITTING, State.SPLIT, State.MERGING, State.MERGED)) {
1464           regionInfoIterator.remove();
1465         } else if (regionStates.isRegionInState(hri, State.FAILED_OPEN)) {
1466           failedOpenCount++;
1467         }
1468       }
1469       if (!waitTillAllAssigned) {
1470         // No need to wait, let assignment going on asynchronously
1471         break;
1472       }
1473       if (!regionSet.isEmpty()) {
1474         if (failedOpenCount == regionSet.size()) {
1475           // all the regions we are waiting had an error on open.
1476           break;
1477         }
1478         regionStates.waitForUpdate(100);
1479       }
1480     }
1481     return regionSet.isEmpty();
1482   }
1483 
1484   /**
1485    * Assigns the hbase:meta region or a replica.
1486    * <p>
1487    * Assumes that hbase:meta is currently closed and is not being actively served by
1488    * any RegionServer.
1489    * @param hri TODO
1490    */
1491   public void assignMeta(HRegionInfo hri) throws KeeperException {
1492     regionStates.updateRegionState(hri, State.OFFLINE);
1493     assign(hri);
1494   }
1495 
1496   /**
1497    * Assigns specified regions retaining assignments, if any.
1498    * <p>
1499    * This is a synchronous call and will return once every region has been
1500    * assigned.  If anything fails, an exception is thrown
1501    * @throws InterruptedException
1502    * @throws IOException
1503    */
1504   public void assign(Map<HRegionInfo, ServerName> regions)
1505         throws IOException, InterruptedException {
1506     if (regions == null || regions.isEmpty()) {
1507       return;
1508     }
1509     List<ServerName> servers = serverManager.createDestinationServersList();
1510     if (servers == null || servers.isEmpty()) {
1511       throw new IOException("Found no destination server to assign region(s)");
1512     }
1513 
1514     // Reuse existing assignment info
1515     Map<ServerName, List<HRegionInfo>> bulkPlan =
1516       balancer.retainAssignment(regions, servers);
1517     if (bulkPlan == null) {
1518       throw new IOException("Unable to determine a plan to assign region(s)");
1519     }
1520 
1521     processBogusAssignments(bulkPlan);
1522 
1523     assign(regions.size(), servers.size(),
1524       "retainAssignment=true", bulkPlan);
1525   }
1526 
1527   /**
1528    * Assigns specified regions round robin, if any.
1529    * <p>
1530    * This is a synchronous call and will return once every region has been
1531    * assigned.  If anything fails, an exception is thrown
1532    * @throws InterruptedException
1533    * @throws IOException
1534    */
1535   public void assign(List<HRegionInfo> regions)
1536         throws IOException, InterruptedException {
1537     if (regions == null || regions.isEmpty()) {
1538       return;
1539     }
1540 
1541     List<ServerName> servers = serverManager.createDestinationServersList();
1542     if (servers == null || servers.isEmpty()) {
1543       throw new IOException("Found no destination server to assign region(s)");
1544     }
1545 
1546     // Generate a round-robin bulk assignment plan
1547     Map<ServerName, List<HRegionInfo>> bulkPlan = balancer.roundRobinAssignment(regions, servers);
1548     if (bulkPlan == null) {
1549       throw new IOException("Unable to determine a plan to assign region(s)");
1550     }
1551 
1552     processBogusAssignments(bulkPlan);
1553 
1554     processFavoredNodes(regions);
1555     assign(regions.size(), servers.size(), "round-robin=true", bulkPlan);
1556   }
1557 
1558   private void assign(int regions, int totalServers,
1559       String message, Map<ServerName, List<HRegionInfo>> bulkPlan)
1560           throws InterruptedException, IOException {
1561 
1562     int servers = bulkPlan.size();
1563     if (servers == 1 || (regions < bulkAssignThresholdRegions
1564         && servers < bulkAssignThresholdServers)) {
1565 
1566       // Not use bulk assignment.  This could be more efficient in small
1567       // cluster, especially mini cluster for testing, so that tests won't time out
1568       if (LOG.isTraceEnabled()) {
1569         LOG.trace("Not using bulk assignment since we are assigning only " + regions +
1570           " region(s) to " + servers + " server(s)");
1571       }
1572 
1573       // invoke assignment (async)
1574       ArrayList<HRegionInfo> userRegionSet = new ArrayList<HRegionInfo>(regions);
1575       for (Map.Entry<ServerName, List<HRegionInfo>> plan: bulkPlan.entrySet()) {
1576         if (!assign(plan.getKey(), plan.getValue()) && !server.isStopped()) {
1577           for (HRegionInfo region: plan.getValue()) {
1578             if (!regionStates.isRegionOnline(region)) {
1579               invokeAssign(region);
1580               if (!region.getTable().isSystemTable()) {
1581                 userRegionSet.add(region);
1582               }
1583             }
1584           }
1585         }
1586       }
1587 
1588       // wait for assignment completion
1589       if (!waitForAssignment(userRegionSet, true, userRegionSet.size(),
1590             System.currentTimeMillis())) {
1591         LOG.debug("some user regions are still in transition: " + userRegionSet);
1592       }
1593     } else {
1594       LOG.info("Bulk assigning " + regions + " region(s) across "
1595         + totalServers + " server(s), " + message);
1596 
1597       // Use fixed count thread pool assigning.
1598       BulkAssigner ba = new GeneralBulkAssigner(
1599         this.server, bulkPlan, this, bulkAssignWaitTillAllAssigned);
1600       ba.bulkAssign();
1601       LOG.info("Bulk assigning done");
1602     }
1603   }
1604 
1605   /**
1606    * Assigns all user regions, if any exist.  Used during cluster startup.
1607    * <p>
1608    * This is a synchronous call and will return once every region has been
1609    * assigned.  If anything fails, an exception is thrown and the cluster
1610    * should be shutdown.
1611    * @throws InterruptedException
1612    * @throws IOException
1613    */
1614   private void assignAllUserRegions(Map<HRegionInfo, ServerName> allRegions)
1615       throws IOException, InterruptedException {
1616     if (allRegions == null || allRegions.isEmpty()) return;
1617 
1618     // Determine what type of assignment to do on startup
1619     boolean retainAssignment = server.getConfiguration().
1620       getBoolean("hbase.master.startup.retainassign", true);
1621 
1622     Set<HRegionInfo> regionsFromMetaScan = allRegions.keySet();
1623     if (retainAssignment) {
1624       assign(allRegions);
1625     } else {
1626       List<HRegionInfo> regions = new ArrayList<HRegionInfo>(regionsFromMetaScan);
1627       assign(regions);
1628     }
1629 
1630     for (HRegionInfo hri : regionsFromMetaScan) {
1631       TableName tableName = hri.getTable();
1632       if (!tableStateManager.isTableState(tableName,
1633               TableState.State.ENABLED)) {
1634         setEnabledTable(tableName);
1635       }
1636     }
1637     // assign all the replicas that were not recorded in the meta
1638     assign(replicaRegionsNotRecordedInMeta(regionsFromMetaScan, (MasterServices)server));
1639   }
1640 
1641   /**
1642    * Get a list of replica regions that are:
1643    * not recorded in meta yet. We might not have recorded the locations
1644    * for the replicas since the replicas may not have been online yet, master restarted
1645    * in the middle of assigning, ZK erased, etc.
1646    * @param regionsRecordedInMeta the list of regions we know are recorded in meta
1647    * either as a default, or, as the location of a replica
1648    * @param master
1649    * @return list of replica regions
1650    * @throws IOException
1651    */
1652   public static List<HRegionInfo> replicaRegionsNotRecordedInMeta(
1653       Set<HRegionInfo> regionsRecordedInMeta, MasterServices master)throws IOException {
1654     List<HRegionInfo> regionsNotRecordedInMeta = new ArrayList<HRegionInfo>();
1655     for (HRegionInfo hri : regionsRecordedInMeta) {
1656       TableName table = hri.getTable();
1657       HTableDescriptor htd = master.getTableDescriptors().get(table);
1658       // look at the HTD for the replica count. That's the source of truth
1659       int desiredRegionReplication = htd.getRegionReplication();
1660       for (int i = 0; i < desiredRegionReplication; i++) {
1661         HRegionInfo replica = RegionReplicaUtil.getRegionInfoForReplica(hri, i);
1662         if (regionsRecordedInMeta.contains(replica)) continue;
1663         regionsNotRecordedInMeta.add(replica);
1664       }
1665     }
1666     return regionsNotRecordedInMeta;
1667   }
1668 
1669   /**
1670    * Rebuild the list of user regions and assignment information.
1671    * Updates regionstates with findings as we go through list of regions.
1672    * @return set of servers not online that hosted some regions according to a scan of hbase:meta
1673    * @throws IOException
1674    */
1675   Set<ServerName> rebuildUserRegions() throws
1676           IOException, KeeperException {
1677     Set<TableName> disabledOrEnablingTables = tableStateManager.getTablesInStates(
1678             TableState.State.DISABLED, TableState.State.ENABLING);
1679 
1680     Set<TableName> disabledOrDisablingOrEnabling = tableStateManager.getTablesInStates(
1681             TableState.State.DISABLED,
1682             TableState.State.DISABLING,
1683             TableState.State.ENABLING);
1684 
1685     // Region assignment from META
1686     List<Result> results = MetaTableAccessor.fullScanRegions(server.getConnection());
1687     // Get any new but slow to checkin region server that joined the cluster
1688     Set<ServerName> onlineServers = serverManager.getOnlineServers().keySet();
1689     // Set of offline servers to be returned
1690     Set<ServerName> offlineServers = new HashSet<ServerName>();
1691     // Iterate regions in META
1692     for (Result result : results) {
1693       if (result == null && LOG.isDebugEnabled()){
1694         LOG.debug("null result from meta - ignoring but this is strange.");
1695         continue;
1696       }
1697       // keep a track of replicas to close. These were the replicas of the originally
1698       // unmerged regions. The master might have closed them before but it mightn't
1699       // maybe because it crashed.
1700       PairOfSameType<HRegionInfo> p = MetaTableAccessor.getMergeRegions(result);
1701       if (p.getFirst() != null && p.getSecond() != null) {
1702         int numReplicas = ((MasterServices)server).getTableDescriptors().get(p.getFirst().
1703             getTable()).getRegionReplication();
1704         for (HRegionInfo merge : p) {
1705           for (int i = 1; i < numReplicas; i++) {
1706             replicasToClose.add(RegionReplicaUtil.getRegionInfoForReplica(merge, i));
1707           }
1708         }
1709       }
1710       RegionLocations rl =  MetaTableAccessor.getRegionLocations(result);
1711       if (rl == null) {
1712         continue;
1713       }
1714       HRegionLocation[] locations = rl.getRegionLocations();
1715       if (locations == null) {
1716         continue;
1717       }
1718       for (HRegionLocation hrl : locations) {
1719         if (hrl == null) continue;
1720         HRegionInfo regionInfo = hrl.getRegionInfo();
1721         if (regionInfo == null) continue;
1722         int replicaId = regionInfo.getReplicaId();
1723         State state = RegionStateStore.getRegionState(result, replicaId);
1724         // keep a track of replicas to close. These were the replicas of the split parents
1725         // from the previous life of the master. The master should have closed them before
1726         // but it couldn't maybe because it crashed
1727         if (replicaId == 0 && state.equals(State.SPLIT)) {
1728           for (HRegionLocation h : locations) {
1729             replicasToClose.add(h.getRegionInfo());
1730           }
1731         }
1732         ServerName lastHost = hrl.getServerName();
1733         ServerName regionLocation = RegionStateStore.getRegionServer(result, replicaId);
1734         regionStates.createRegionState(regionInfo, state, regionLocation, lastHost);
1735         if (!regionStates.isRegionInState(regionInfo, State.OPEN)) {
1736           // Region is not open (either offline or in transition), skip
1737           continue;
1738         }
1739         TableName tableName = regionInfo.getTable();
1740         if (!onlineServers.contains(regionLocation)) {
1741           // Region is located on a server that isn't online
1742           offlineServers.add(regionLocation);
1743         } else if (!disabledOrEnablingTables.contains(tableName)) {
1744           // Region is being served and on an active server
1745           // add only if region not in disabled or enabling table
1746           regionStates.regionOnline(regionInfo, regionLocation);
1747           balancer.regionOnline(regionInfo, regionLocation);
1748         }
1749         // need to enable the table if not disabled or disabling or enabling
1750         // this will be used in rolling restarts
1751         if (!disabledOrDisablingOrEnabling.contains(tableName)
1752           && !getTableStateManager().isTableState(tableName,
1753                 TableState.State.ENABLED)) {
1754           setEnabledTable(tableName);
1755         }
1756       }
1757     }
1758     return offlineServers;
1759   }
1760 
1761   /**
1762    * Recover the tables that were not fully moved to DISABLED state. These
1763    * tables are in DISABLING state when the master restarted/switched.
1764    *
1765    * @throws KeeperException
1766    * @throws TableNotFoundException
1767    * @throws IOException
1768    */
1769   private void recoverTableInDisablingState()
1770           throws KeeperException, IOException {
1771     Set<TableName> disablingTables =
1772             tableStateManager.getTablesInStates(TableState.State.DISABLING);
1773     if (disablingTables.size() != 0) {
1774       for (TableName tableName : disablingTables) {
1775         // Recover by calling DisableTableHandler
1776         LOG.info("The table " + tableName
1777             + " is in DISABLING state.  Hence recovering by moving the table"
1778             + " to DISABLED state.");
1779         new DisableTableHandler(this.server, tableName,
1780             this, tableLockManager, true).prepare().process();
1781       }
1782     }
1783   }
1784 
1785   /**
1786    * Recover the tables that are not fully moved to ENABLED state. These tables
1787    * are in ENABLING state when the master restarted/switched
1788    *
1789    * @throws KeeperException
1790    * @throws org.apache.hadoop.hbase.TableNotFoundException
1791    * @throws IOException
1792    */
1793   private void recoverTableInEnablingState()
1794           throws KeeperException, IOException {
1795     Set<TableName> enablingTables = tableStateManager.
1796             getTablesInStates(TableState.State.ENABLING);
1797     if (enablingTables.size() != 0) {
1798       for (TableName tableName : enablingTables) {
1799         // Recover by calling EnableTableHandler
1800         LOG.info("The table " + tableName
1801             + " is in ENABLING state.  Hence recovering by moving the table"
1802             + " to ENABLED state.");
1803         // enableTable in sync way during master startup,
1804         // no need to invoke coprocessor
1805         EnableTableHandler eth = new EnableTableHandler(this.server, tableName,
1806           this, tableLockManager, true);
1807         try {
1808           eth.prepare();
1809         } catch (TableNotFoundException e) {
1810           LOG.warn("Table " + tableName + " not found in hbase:meta to recover.");
1811           continue;
1812         }
1813         eth.process();
1814       }
1815     }
1816   }
1817 
1818   /**
1819    * Processes list of regions in transition at startup
1820    */
1821   void processRegionsInTransition(Collection<RegionState> regionsInTransition) {
1822     // We need to send RPC call again for PENDING_OPEN/PENDING_CLOSE regions
1823     // in case the RPC call is not sent out yet before the master was shut down
1824     // since we update the state before we send the RPC call. We can't update
1825     // the state after the RPC call. Otherwise, we don't know what's happened
1826     // to the region if the master dies right after the RPC call is out.
1827     for (RegionState regionState: regionsInTransition) {
1828       LOG.info("Processing " + regionState);
1829       ServerName serverName = regionState.getServerName();
1830       // Server could be null in case of FAILED_OPEN when master cannot find a region plan. In that
1831       // case, try assigning it here.
1832       if (serverName != null && !serverManager.getOnlineServers().containsKey(serverName)) {
1833         LOG.info("Server " + serverName + " isn't online. SSH will handle this");
1834         continue; // SSH will handle it
1835       }
1836       HRegionInfo regionInfo = regionState.getRegion();
1837       RegionState.State state = regionState.getState();
1838       switch (state) {
1839       case CLOSED:
1840         invokeAssign(regionState.getRegion());
1841         break;
1842       case PENDING_OPEN:
1843         retrySendRegionOpen(regionState);
1844         break;
1845       case PENDING_CLOSE:
1846         retrySendRegionClose(regionState);
1847         break;
1848       case FAILED_CLOSE:
1849       case FAILED_OPEN:
1850         invokeUnAssign(regionInfo);
1851         break;
1852       default:
1853           // No process for other states
1854           break;
1855       }
1856     }
1857   }
1858 
1859   /**
1860    * At master failover, for pending_open region, make sure
1861    * sendRegionOpen RPC call is sent to the target regionserver
1862    */
1863   private void retrySendRegionOpen(final RegionState regionState) {
1864     this.executorService.submit(
1865       new EventHandler(server, EventType.M_MASTER_RECOVERY) {
1866         @Override
1867         public void process() throws IOException {
1868           HRegionInfo hri = regionState.getRegion();
1869           ServerName serverName = regionState.getServerName();
1870           ReentrantLock lock = locker.acquireLock(hri.getEncodedName());
1871           try {
1872             for (int i = 1; i <= maximumAttempts; i++) {
1873               if (!serverManager.isServerOnline(serverName)
1874                   || server.isStopped() || server.isAborted()) {
1875                 return; // No need any more
1876               }
1877               try {
1878                 if (!regionState.equals(regionStates.getRegionState(hri))) {
1879                   return; // Region is not in the expected state any more
1880                 }
1881                 List<ServerName> favoredNodes = ServerName.EMPTY_SERVER_LIST;
1882                 if (shouldAssignRegionsWithFavoredNodes) {
1883                   favoredNodes = ((FavoredNodeLoadBalancer)balancer).getFavoredNodes(hri);
1884                 }
1885                 serverManager.sendRegionOpen(serverName, hri, favoredNodes);
1886                 return; // we're done
1887               } catch (Throwable t) {
1888                 if (t instanceof RemoteException) {
1889                   t = ((RemoteException) t).unwrapRemoteException();
1890                 }
1891                 if (t instanceof FailedServerException && i < maximumAttempts) {
1892                   // In case the server is in the failed server list, no point to
1893                   // retry too soon. Retry after the failed_server_expiry time
1894                   try {
1895                     Configuration conf = this.server.getConfiguration();
1896                     long sleepTime = 1 + conf.getInt(RpcClient.FAILED_SERVER_EXPIRY_KEY,
1897                       RpcClient.FAILED_SERVER_EXPIRY_DEFAULT);
1898                     if (LOG.isDebugEnabled()) {
1899                       LOG.debug(serverName + " is on failed server list; waiting "
1900                         + sleepTime + "ms", t);
1901                     }
1902                     Thread.sleep(sleepTime);
1903                     continue;
1904                   } catch (InterruptedException ie) {
1905                     LOG.warn("Failed to assign "
1906                       + hri.getRegionNameAsString() + " since interrupted", ie);
1907                     regionStates.updateRegionState(hri, State.FAILED_OPEN);
1908                     Thread.currentThread().interrupt();
1909                     return;
1910                   }
1911                 }
1912                 if (serverManager.isServerOnline(serverName)
1913                     && t instanceof java.net.SocketTimeoutException) {
1914                   i--; // reset the try count
1915                 } else {
1916                   LOG.info("Got exception in retrying sendRegionOpen for "
1917                     + regionState + "; try=" + i + " of " + maximumAttempts, t);
1918                 }
1919                 Threads.sleep(100);
1920               }
1921             }
1922             // Run out of attempts
1923             regionStates.updateRegionState(hri, State.FAILED_OPEN);
1924           } finally {
1925             lock.unlock();
1926           }
1927         }
1928       });
1929   }
1930 
1931   /**
1932    * At master failover, for pending_close region, make sure
1933    * sendRegionClose RPC call is sent to the target regionserver
1934    */
1935   private void retrySendRegionClose(final RegionState regionState) {
1936     this.executorService.submit(
1937       new EventHandler(server, EventType.M_MASTER_RECOVERY) {
1938         @Override
1939         public void process() throws IOException {
1940           HRegionInfo hri = regionState.getRegion();
1941           ServerName serverName = regionState.getServerName();
1942           ReentrantLock lock = locker.acquireLock(hri.getEncodedName());
1943           try {
1944             for (int i = 1; i <= maximumAttempts; i++) {
1945               if (!serverManager.isServerOnline(serverName)
1946                   || server.isStopped() || server.isAborted()) {
1947                 return; // No need any more
1948               }
1949               try {
1950                 if (!regionState.equals(regionStates.getRegionState(hri))) {
1951                   return; // Region is not in the expected state any more
1952                 }
1953                 serverManager.sendRegionClose(serverName, hri, null);
1954                 return; // Done.
1955               } catch (Throwable t) {
1956                 if (t instanceof RemoteException) {
1957                   t = ((RemoteException) t).unwrapRemoteException();
1958                 }
1959                 if (t instanceof FailedServerException && i < maximumAttempts) {
1960                   // In case the server is in the failed server list, no point to
1961                   // retry too soon. Retry after the failed_server_expiry time
1962                   try {
1963                     Configuration conf = this.server.getConfiguration();
1964                     long sleepTime = 1 + conf.getInt(RpcClient.FAILED_SERVER_EXPIRY_KEY,
1965                       RpcClient.FAILED_SERVER_EXPIRY_DEFAULT);
1966                     if (LOG.isDebugEnabled()) {
1967                       LOG.debug(serverName + " is on failed server list; waiting "
1968                         + sleepTime + "ms", t);
1969                     }
1970                     Thread.sleep(sleepTime);
1971                     continue;
1972                   } catch (InterruptedException ie) {
1973                     LOG.warn("Failed to unassign "
1974                       + hri.getRegionNameAsString() + " since interrupted", ie);
1975                     regionStates.updateRegionState(hri, RegionState.State.FAILED_CLOSE);
1976                     Thread.currentThread().interrupt();
1977                     return;
1978                   }
1979                 }
1980                 if (serverManager.isServerOnline(serverName)
1981                     && t instanceof java.net.SocketTimeoutException) {
1982                   i--; // reset the try count
1983                 } else {
1984                   LOG.info("Got exception in retrying sendRegionClose for "
1985                     + regionState + "; try=" + i + " of " + maximumAttempts, t);
1986                 }
1987                 Threads.sleep(100);
1988               }
1989             }
1990             // Run out of attempts
1991             regionStates.updateRegionState(hri, State.FAILED_CLOSE);
1992           } finally {
1993             lock.unlock();
1994           }
1995         }
1996       });
1997   }
1998 
1999   /**
2000    * Set Regions in transitions metrics.
2001    * This takes an iterator on the RegionInTransition map (CLSM), and is not synchronized.
2002    * This iterator is not fail fast, which may lead to stale read; but that's better than
2003    * creating a copy of the map for metrics computation, as this method will be invoked
2004    * on a frequent interval.
2005    */
2006   public void updateRegionsInTransitionMetrics() {
2007     long currentTime = System.currentTimeMillis();
2008     int totalRITs = 0;
2009     int totalRITsOverThreshold = 0;
2010     long oldestRITTime = 0;
2011     int ritThreshold = this.server.getConfiguration().
2012       getInt(HConstants.METRICS_RIT_STUCK_WARNING_THRESHOLD, 60000);
2013     for (RegionState state: regionStates.getRegionsInTransition()) {
2014       totalRITs++;
2015       long ritTime = currentTime - state.getStamp();
2016       if (ritTime > ritThreshold) { // more than the threshold
2017         totalRITsOverThreshold++;
2018       }
2019       if (oldestRITTime < ritTime) {
2020         oldestRITTime = ritTime;
2021       }
2022     }
2023     if (this.metricsAssignmentManager != null) {
2024       this.metricsAssignmentManager.updateRITOldestAge(oldestRITTime);
2025       this.metricsAssignmentManager.updateRITCount(totalRITs);
2026       this.metricsAssignmentManager.updateRITCountOverThreshold(totalRITsOverThreshold);
2027     }
2028   }
2029 
2030   /**
2031    * @param region Region whose plan we are to clear.
2032    */
2033   private void clearRegionPlan(final HRegionInfo region) {
2034     synchronized (this.regionPlans) {
2035       this.regionPlans.remove(region.getEncodedName());
2036     }
2037   }
2038 
2039   /**
2040    * Wait on region to clear regions-in-transition.
2041    * @param hri Region to wait on.
2042    * @throws IOException
2043    */
2044   public void waitOnRegionToClearRegionsInTransition(final HRegionInfo hri)
2045       throws IOException, InterruptedException {
2046     waitOnRegionToClearRegionsInTransition(hri, -1L);
2047   }
2048 
2049   /**
2050    * Wait on region to clear regions-in-transition or time out
2051    * @param hri
2052    * @param timeOut Milliseconds to wait for current region to be out of transition state.
2053    * @return True when a region clears regions-in-transition before timeout otherwise false
2054    * @throws InterruptedException
2055    */
2056   public boolean waitOnRegionToClearRegionsInTransition(final HRegionInfo hri, long timeOut)
2057       throws InterruptedException {
2058     if (!regionStates.isRegionInTransition(hri)) {
2059       return true;
2060     }
2061     long end = (timeOut <= 0) ? Long.MAX_VALUE : EnvironmentEdgeManager.currentTime()
2062         + timeOut;
2063     // There is already a timeout monitor on regions in transition so I
2064     // should not have to have one here too?
2065     LOG.info("Waiting for " + hri.getEncodedName() +
2066         " to leave regions-in-transition, timeOut=" + timeOut + " ms.");
2067     while (!this.server.isStopped() && regionStates.isRegionInTransition(hri)) {
2068       regionStates.waitForUpdate(100);
2069       if (EnvironmentEdgeManager.currentTime() > end) {
2070         LOG.info("Timed out on waiting for " + hri.getEncodedName() + " to be assigned.");
2071         return false;
2072       }
2073     }
2074     if (this.server.isStopped()) {
2075       LOG.info("Giving up wait on regions in transition because stoppable.isStopped is set");
2076       return false;
2077     }
2078     return true;
2079   }
2080 
2081   void invokeAssign(HRegionInfo regionInfo) {
2082     threadPoolExecutorService.submit(new AssignCallable(this, regionInfo));
2083   }
2084 
2085   void invokeUnAssign(HRegionInfo regionInfo) {
2086     threadPoolExecutorService.submit(new UnAssignCallable(this, regionInfo));
2087   }
2088 
2089   public boolean isCarryingMeta(ServerName serverName) {
2090     return isCarryingRegion(serverName, HRegionInfo.FIRST_META_REGIONINFO);
2091   }
2092 
2093   public boolean isCarryingMetaReplica(ServerName serverName, int replicaId) {
2094     return isCarryingRegion(serverName,
2095         RegionReplicaUtil.getRegionInfoForReplica(HRegionInfo.FIRST_META_REGIONINFO, replicaId));
2096   }
2097 
2098   public boolean isCarryingMetaReplica(ServerName serverName, HRegionInfo metaHri) {
2099     return isCarryingRegion(serverName, metaHri);
2100   }
2101 
2102   /**
2103    * Check if the shutdown server carries the specific region.
2104    * @return whether the serverName currently hosts the region
2105    */
2106   private boolean isCarryingRegion(ServerName serverName, HRegionInfo hri) {
2107     RegionState regionState = regionStates.getRegionTransitionState(hri);
2108     ServerName transitionAddr = regionState != null? regionState.getServerName(): null;
2109     if (transitionAddr != null) {
2110       boolean matchTransitionAddr = transitionAddr.equals(serverName);
2111       LOG.debug("Checking region=" + hri.getRegionNameAsString()
2112         + ", transitioning on server=" + matchTransitionAddr
2113         + " server being checked: " + serverName
2114         + ", matches=" + matchTransitionAddr);
2115       return matchTransitionAddr;
2116     }
2117 
2118     ServerName assignedAddr = regionStates.getRegionServerOfRegion(hri);
2119     boolean matchAssignedAddr = serverName.equals(assignedAddr);
2120     LOG.debug("based on AM, current region=" + hri.getRegionNameAsString()
2121       + " is on server=" + assignedAddr + ", server being checked: "
2122       + serverName);
2123     return matchAssignedAddr;
2124   }
2125 
2126   /**
2127    * Clean out crashed server removing any assignments.
2128    * @param sn Server that went down.
2129    * @return list of regions in transition on this server
2130    */
2131   public List<HRegionInfo> cleanOutCrashedServerReferences(final ServerName sn) {
2132     // Clean out any existing assignment plans for this server
2133     synchronized (this.regionPlans) {
2134       for (Iterator <Map.Entry<String, RegionPlan>> i = this.regionPlans.entrySet().iterator();
2135           i.hasNext();) {
2136         Map.Entry<String, RegionPlan> e = i.next();
2137         ServerName otherSn = e.getValue().getDestination();
2138         // The name will be null if the region is planned for a random assign.
2139         if (otherSn != null && otherSn.equals(sn)) {
2140           // Use iterator's remove else we'll get CME
2141           i.remove();
2142         }
2143       }
2144     }
2145     List<HRegionInfo> rits = regionStates.serverOffline(sn);
2146     for (Iterator<HRegionInfo> it = rits.iterator(); it.hasNext(); ) {
2147       HRegionInfo hri = it.next();
2148       String encodedName = hri.getEncodedName();
2149 
2150       // We need a lock on the region as we could update it
2151       Lock lock = locker.acquireLock(encodedName);
2152       try {
2153         RegionState regionState = regionStates.getRegionTransitionState(encodedName);
2154         if (regionState == null
2155             || (regionState.getServerName() != null && !regionState.isOnServer(sn))
2156             || !RegionStates.isOneOfStates(regionState, State.PENDING_OPEN,
2157                 State.OPENING, State.FAILED_OPEN, State.FAILED_CLOSE, State.OFFLINE)) {
2158           LOG.info("Skip " + regionState + " since it is not opening/failed_close"
2159             + " on the dead server any more: " + sn);
2160           it.remove();
2161         } else {
2162           if (tableStateManager.isTableState(hri.getTable(),
2163                   TableState.State.DISABLED, TableState.State.DISABLING)) {
2164             regionStates.regionOffline(hri);
2165             it.remove();
2166             continue;
2167           }
2168           // Mark the region offline and assign it again by SSH
2169           regionStates.updateRegionState(hri, State.OFFLINE);
2170         }
2171       } finally {
2172         lock.unlock();
2173       }
2174     }
2175     return rits;
2176   }
2177 
2178   /**
2179    * @param plan Plan to execute.
2180    */
2181   public void balance(final RegionPlan plan) {
2182 
2183     HRegionInfo hri = plan.getRegionInfo();
2184     TableName tableName = hri.getTable();
2185     if (tableStateManager.isTableState(tableName,
2186             TableState.State.DISABLED, TableState.State.DISABLING)) {
2187       LOG.info("Ignored moving region of disabling/disabled table "
2188         + tableName);
2189       return;
2190     }
2191 
2192     // Move the region only if it's assigned
2193     String encodedName = hri.getEncodedName();
2194     ReentrantLock lock = locker.acquireLock(encodedName);
2195     try {
2196       if (!regionStates.isRegionOnline(hri)) {
2197         RegionState state = regionStates.getRegionState(encodedName);
2198         LOG.info("Ignored moving region not assigned: " + hri + ", "
2199           + (state == null ? "not in region states" : state));
2200         return;
2201       }
2202       synchronized (this.regionPlans) {
2203         this.regionPlans.put(plan.getRegionName(), plan);
2204       }
2205       unassign(hri, plan.getDestination());
2206     } finally {
2207       lock.unlock();
2208     }
2209   }
2210 
2211   public void stop() {
2212     // Shutdown the threadpool executor service
2213     threadPoolExecutorService.shutdownNow();
2214     regionStateStore.stop();
2215   }
2216 
2217   protected void setEnabledTable(TableName tableName) {
2218     try {
2219       this.tableStateManager.setTableState(tableName,
2220               TableState.State.ENABLED);
2221     } catch (IOException e) {
2222       // here we can abort as it is the start up flow
2223       String errorMsg = "Unable to ensure that the table " + tableName
2224           + " will be" + " enabled because of a ZooKeeper issue";
2225       LOG.error(errorMsg);
2226       this.server.abort(errorMsg, e);
2227     }
2228   }
2229 
2230   @edu.umd.cs.findbugs.annotations.SuppressWarnings(
2231       value="AT_OPERATION_SEQUENCE_ON_CONCURRENT_ABSTRACTION",
2232       justification="Worth fixing but not the end of the world.")
2233   private String onRegionFailedOpen(final RegionState current,
2234       final HRegionInfo hri, final ServerName serverName) {
2235     // The region must be opening on this server.
2236     // If current state is failed_open on the same server,
2237     // it could be a reportRegionTransition RPC retry.
2238     if (current == null || !current.isOpeningOrFailedOpenOnServer(serverName)) {
2239       return hri.getShortNameToLog() + " is not opening on " + serverName;
2240     }
2241 
2242     // Just return in case of retrying
2243     if (current.isFailedOpen()) {
2244       return null;
2245     }
2246 
2247     String encodedName = hri.getEncodedName();
2248     // FindBugs: AT_OPERATION_SEQUENCE_ON_CONCURRENT_ABSTRACTION Worth fixing!!!
2249     AtomicInteger failedOpenCount = failedOpenTracker.get(encodedName);
2250     if (failedOpenCount == null) {
2251       failedOpenCount = new AtomicInteger();
2252       // No need to use putIfAbsent, or extra synchronization since
2253       // this whole handleRegion block is locked on the encoded region
2254       // name, and failedOpenTracker is updated only in this block
2255       failedOpenTracker.put(encodedName, failedOpenCount);
2256     }
2257     if (failedOpenCount.incrementAndGet() >= maximumAttempts && !hri.isMetaRegion()) {
2258       regionStates.updateRegionState(hri, State.FAILED_OPEN);
2259       // remove the tracking info to save memory, also reset
2260       // the count for next open initiative
2261       failedOpenTracker.remove(encodedName);
2262     } else {
2263       if (hri.isMetaRegion() && failedOpenCount.get() >= maximumAttempts) {
2264         // Log a warning message if a meta region failedOpenCount exceeds maximumAttempts
2265         // so that we are aware of potential problem if it persists for a long time.
2266         LOG.warn("Failed to open the hbase:meta region " +
2267             hri.getRegionNameAsString() + " after" +
2268             failedOpenCount.get() + " retries. Continue retrying.");
2269       }
2270 
2271       // Handle this the same as if it were opened and then closed.
2272       RegionState regionState = regionStates.updateRegionState(hri, State.CLOSED);
2273       if (regionState != null) {
2274         // When there are more than one region server a new RS is selected as the
2275         // destination and the same is updated in the region plan. (HBASE-5546)
2276         if (getTableStateManager().isTableState(hri.getTable(),
2277                 TableState.State.DISABLED, TableState.State.DISABLING) ||
2278                 replicasToClose.contains(hri)) {
2279           offlineDisabledRegion(hri);
2280           return null;
2281         }
2282         regionStates.updateRegionState(hri, RegionState.State.CLOSED);
2283         // This below has to do w/ online enable/disable of a table
2284         removeClosedRegion(hri);
2285         try {
2286           getRegionPlan(hri, true);
2287         } catch (HBaseIOException e) {
2288           LOG.warn("Failed to get region plan", e);
2289         }
2290         invokeAssign(hri);
2291       }
2292     }
2293     // Null means no error
2294     return null;
2295   }
2296 
2297   private String onRegionOpen(final RegionState current, final HRegionInfo hri,
2298       final ServerName serverName, final RegionStateTransition transition) {
2299     // The region must be opening on this server.
2300     // If current state is already opened on the same server,
2301     // it could be a reportRegionTransition RPC retry.
2302     if (current == null || !current.isOpeningOrOpenedOnServer(serverName)) {
2303       return hri.getShortNameToLog() + " is not opening on " + serverName;
2304     }
2305 
2306     // Just return in case of retrying
2307     if (current.isOpened()) {
2308       return null;
2309     }
2310 
2311     long openSeqNum = transition.hasOpenSeqNum()
2312       ? transition.getOpenSeqNum() : HConstants.NO_SEQNUM;
2313     if (openSeqNum < 0) {
2314       return "Newly opened region has invalid open seq num " + openSeqNum;
2315     }
2316     regionOnline(hri, serverName, openSeqNum);
2317 
2318     // reset the count, if any
2319     failedOpenTracker.remove(hri.getEncodedName());
2320     if (getTableStateManager().isTableState(hri.getTable(),
2321             TableState.State.DISABLED, TableState.State.DISABLING)) {
2322       invokeUnAssign(hri);
2323     }
2324     return null;
2325   }
2326 
2327   private String onRegionClosed(final RegionState current,
2328       final HRegionInfo hri, final ServerName serverName) {
2329     // Region will be usually assigned right after closed. When a RPC retry comes
2330     // in, the region may already have moved away from closed state. However, on the
2331     // region server side, we don't care much about the response for this transition.
2332     // We only make sure master has got and processed this report, either
2333     // successfully or not. So this is fine, not a problem at all.
2334     if (current == null || !current.isClosingOrClosedOnServer(serverName)) {
2335       return hri.getShortNameToLog() + " is not closing on " + serverName;
2336     }
2337 
2338     // Just return in case of retrying
2339     if (current.isClosed()) {
2340       return null;
2341     }
2342 
2343     if (getTableStateManager().isTableState(hri.getTable(), TableState.State.DISABLED,
2344         TableState.State.DISABLING) || replicasToClose.contains(hri)) {
2345       offlineDisabledRegion(hri);
2346       return null;
2347     }
2348 
2349     regionStates.updateRegionState(hri, RegionState.State.CLOSED);
2350     sendRegionClosedNotification(hri);
2351     // This below has to do w/ online enable/disable of a table
2352     removeClosedRegion(hri);
2353     invokeAssign(hri);
2354     return null;
2355   }
2356 
2357   private String onRegionReadyToSplit(final RegionState current, final HRegionInfo hri,
2358       final ServerName serverName, final RegionStateTransition transition) {
2359     // The region must be opened on this server.
2360     // If current state is already splitting on the same server,
2361     // it could be a reportRegionTransition RPC retry.
2362     if (current == null || !current.isSplittingOrOpenedOnServer(serverName)) {
2363       return hri.getShortNameToLog() + " is not opening on " + serverName;
2364     }
2365 
2366     if (!((HMaster)server).getSplitOrMergeTracker().isSplitOrMergeEnabled(
2367             MasterSwitchType.SPLIT)) {
2368       return "split switch is off!";
2369     }
2370 
2371     // Just return in case of retrying
2372     if (current.isSplitting()) {
2373       return null;
2374     }
2375 
2376     final HRegionInfo a = HRegionInfo.convert(transition.getRegionInfo(1));
2377     final HRegionInfo b = HRegionInfo.convert(transition.getRegionInfo(2));
2378     RegionState rs_a = regionStates.getRegionState(a);
2379     RegionState rs_b = regionStates.getRegionState(b);
2380     if (rs_a != null || rs_b != null) {
2381       return "Some daughter is already existing. "
2382         + "a=" + rs_a + ", b=" + rs_b;
2383     }
2384 
2385     // Server holding is not updated at this stage.
2386     // It is done after PONR.
2387     regionStates.updateRegionState(hri, State.SPLITTING);
2388     regionStates.createRegionState(
2389       a, State.SPLITTING_NEW, serverName, null);
2390     regionStates.createRegionState(
2391       b, State.SPLITTING_NEW, serverName, null);
2392     return null;
2393   }
2394 
2395   private String onRegionSplitPONR(final RegionState current, final HRegionInfo hri,
2396       final ServerName serverName, final RegionStateTransition transition) {
2397     // The region must be splitting on this server, and the daughters must be in
2398     // splitting_new state. To check RPC retry, we use server holding info.
2399     if (current == null || !current.isSplittingOnServer(serverName)) {
2400       return hri.getShortNameToLog() + " is not splitting on " + serverName;
2401     }
2402 
2403     final HRegionInfo a = HRegionInfo.convert(transition.getRegionInfo(1));
2404     final HRegionInfo b = HRegionInfo.convert(transition.getRegionInfo(2));
2405     RegionState rs_a = regionStates.getRegionState(a);
2406     RegionState rs_b = regionStates.getRegionState(b);
2407 
2408     // Master could have restarted and lost the new region
2409     // states, if so, they must be lost together
2410     if (rs_a == null && rs_b == null) {
2411       rs_a = regionStates.createRegionState(
2412         a, State.SPLITTING_NEW, serverName, null);
2413       rs_b = regionStates.createRegionState(
2414         b, State.SPLITTING_NEW, serverName, null);
2415     }
2416 
2417     if (rs_a == null || !rs_a.isSplittingNewOnServer(serverName)
2418         || rs_b == null || !rs_b.isSplittingNewOnServer(serverName)) {
2419       return "Some daughter is not known to be splitting on " + serverName
2420         + ", a=" + rs_a + ", b=" + rs_b;
2421     }
2422 
2423     // Just return in case of retrying
2424     if (!regionStates.isRegionOnServer(hri, serverName)) {
2425       return null;
2426     }
2427 
2428     try {
2429       regionStates.splitRegion(hri, a, b, serverName);
2430     } catch (IOException ioe) {
2431       LOG.info("Failed to record split region " + hri.getShortNameToLog());
2432       return "Failed to record the splitting in meta";
2433     }
2434     return null;
2435   }
2436 
2437   private String onRegionSplit(final RegionState current, final HRegionInfo hri,
2438       final ServerName serverName, final RegionStateTransition transition) {
2439     // The region must be splitting on this server, and the daughters must be in
2440     // splitting_new state.
2441     // If current state is already split on the same server,
2442     // it could be a reportRegionTransition RPC retry.
2443     if (current == null || !current.isSplittingOrSplitOnServer(serverName)) {
2444       return hri.getShortNameToLog() + " is not splitting on " + serverName;
2445     }
2446 
2447     // Just return in case of retrying
2448     if (current.isSplit()) {
2449       return null;
2450     }
2451 
2452     final HRegionInfo a = HRegionInfo.convert(transition.getRegionInfo(1));
2453     final HRegionInfo b = HRegionInfo.convert(transition.getRegionInfo(2));
2454     RegionState rs_a = regionStates.getRegionState(a);
2455     RegionState rs_b = regionStates.getRegionState(b);
2456     if (rs_a == null || !rs_a.isSplittingNewOnServer(serverName)
2457         || rs_b == null || !rs_b.isSplittingNewOnServer(serverName)) {
2458       return "Some daughter is not known to be splitting on " + serverName
2459         + ", a=" + rs_a + ", b=" + rs_b;
2460     }
2461 
2462     if (TEST_SKIP_SPLIT_HANDLING) {
2463       return "Skipping split message, TEST_SKIP_SPLIT_HANDLING is set";
2464     }
2465     regionOffline(hri, State.SPLIT);
2466     regionOnline(a, serverName, 1);
2467     regionOnline(b, serverName, 1);
2468 
2469     // User could disable the table before master knows the new region.
2470     if (getTableStateManager().isTableState(hri.getTable(),
2471         TableState.State.DISABLED, TableState.State.DISABLING)) {
2472       invokeUnAssign(a);
2473       invokeUnAssign(b);
2474     } else {
2475       Callable<Object> splitReplicasCallable = new Callable<Object>() {
2476         @Override
2477         public Object call() {
2478           doSplittingOfReplicas(hri, a, b);
2479           return null;
2480         }
2481       };
2482       threadPoolExecutorService.submit(splitReplicasCallable);
2483     }
2484     return null;
2485   }
2486 
2487   private String onRegionSplitReverted(final RegionState current, final HRegionInfo hri,
2488       final ServerName serverName, final RegionStateTransition transition) {
2489     // The region must be splitting on this server, and the daughters must be in
2490     // splitting_new state.
2491     // If the region is in open state, it could be an RPC retry.
2492     if (current == null || !current.isSplittingOrOpenedOnServer(serverName)) {
2493       return hri.getShortNameToLog() + " is not splitting on " + serverName;
2494     }
2495 
2496     // Just return in case of retrying
2497     if (current.isOpened()) {
2498       return null;
2499     }
2500 
2501     final HRegionInfo a = HRegionInfo.convert(transition.getRegionInfo(1));
2502     final HRegionInfo b = HRegionInfo.convert(transition.getRegionInfo(2));
2503     RegionState rs_a = regionStates.getRegionState(a);
2504     RegionState rs_b = regionStates.getRegionState(b);
2505     if (rs_a == null || !rs_a.isSplittingNewOnServer(serverName)
2506         || rs_b == null || !rs_b.isSplittingNewOnServer(serverName)) {
2507       return "Some daughter is not known to be splitting on " + serverName
2508         + ", a=" + rs_a + ", b=" + rs_b;
2509     }
2510 
2511     regionOnline(hri, serverName);
2512     regionOffline(a);
2513     regionOffline(b);
2514     if (getTableStateManager().isTableState(hri.getTable(),
2515         TableState.State.DISABLED, TableState.State.DISABLING)) {
2516       invokeUnAssign(hri);
2517     }
2518     return null;
2519   }
2520 
2521   private String onRegionReadyToMerge(final RegionState current, final HRegionInfo hri,
2522       final ServerName serverName, final RegionStateTransition transition) {
2523     // The region must be new, and the daughters must be open on this server.
2524     // If the region is in merge_new state, it could be an RPC retry.
2525     if (current != null && !current.isMergingNewOnServer(serverName)) {
2526       return "Merging daughter region already exists, p=" + current;
2527     }
2528 
2529     if (!((HMaster)server).getSplitOrMergeTracker().isSplitOrMergeEnabled(
2530             MasterSwitchType.MERGE)) {
2531       return "merge switch is off!";
2532     }
2533     // Just return in case of retrying
2534     if (current != null) {
2535       return null;
2536     }
2537 
2538     final HRegionInfo a = HRegionInfo.convert(transition.getRegionInfo(1));
2539     final HRegionInfo b = HRegionInfo.convert(transition.getRegionInfo(2));
2540     Set<String> encodedNames = new HashSet<String>(2);
2541     encodedNames.add(a.getEncodedName());
2542     encodedNames.add(b.getEncodedName());
2543     Map<String, Lock> locks = locker.acquireLocks(encodedNames);
2544     try {
2545       RegionState rs_a = regionStates.getRegionState(a);
2546       RegionState rs_b = regionStates.getRegionState(b);
2547       if (rs_a == null || !rs_a.isOpenedOnServer(serverName)
2548           || rs_b == null || !rs_b.isOpenedOnServer(serverName)) {
2549         return "Some daughter is not in a state to merge on " + serverName
2550           + ", a=" + rs_a + ", b=" + rs_b;
2551       }
2552 
2553       regionStates.updateRegionState(a, State.MERGING);
2554       regionStates.updateRegionState(b, State.MERGING);
2555       regionStates.createRegionState(
2556         hri, State.MERGING_NEW, serverName, null);
2557       return null;
2558     } finally {
2559       for (Lock lock: locks.values()) {
2560         lock.unlock();
2561       }
2562     }
2563   }
2564 
2565   private String onRegionMergePONR(final RegionState current, final HRegionInfo hri,
2566       final ServerName serverName, final RegionStateTransition transition) {
2567     // The region must be in merging_new state, and the daughters must be
2568     // merging. To check RPC retry, we use server holding info.
2569     if (current != null && !current.isMergingNewOnServer(serverName)) {
2570       return hri.getShortNameToLog() + " is not merging on " + serverName;
2571     }
2572 
2573     final HRegionInfo a = HRegionInfo.convert(transition.getRegionInfo(1));
2574     final HRegionInfo b = HRegionInfo.convert(transition.getRegionInfo(2));
2575     RegionState rs_a = regionStates.getRegionState(a);
2576     RegionState rs_b = regionStates.getRegionState(b);
2577     if (rs_a == null || !rs_a.isMergingOnServer(serverName)
2578         || rs_b == null || !rs_b.isMergingOnServer(serverName)) {
2579       return "Some daughter is not known to be merging on " + serverName
2580         + ", a=" + rs_a + ", b=" + rs_b;
2581     }
2582 
2583     // Master could have restarted and lost the new region state
2584     if (current == null) {
2585       regionStates.createRegionState(
2586         hri, State.MERGING_NEW, serverName, null);
2587     }
2588 
2589     // Just return in case of retrying
2590     if (regionStates.isRegionOnServer(hri, serverName)) {
2591       return null;
2592     }
2593 
2594     try {
2595       regionStates.mergeRegions(hri, a, b, serverName);
2596     } catch (IOException ioe) {
2597       LOG.info("Failed to record merged region " + hri.getShortNameToLog());
2598       return "Failed to record the merging in meta";
2599     }
2600     return null;
2601   }
2602 
2603   private String onRegionMerged(final RegionState current, final HRegionInfo hri,
2604       final ServerName serverName, final RegionStateTransition transition) {
2605     // The region must be in merging_new state, and the daughters must be
2606     // merging on this server.
2607     // If current state is already opened on the same server,
2608     // it could be a reportRegionTransition RPC retry.
2609     if (current == null || !current.isMergingNewOrOpenedOnServer(serverName)) {
2610       return hri.getShortNameToLog() + " is not merging on " + serverName;
2611     }
2612 
2613     // Just return in case of retrying
2614     if (current.isOpened()) {
2615       return null;
2616     }
2617 
2618     final HRegionInfo a = HRegionInfo.convert(transition.getRegionInfo(1));
2619     final HRegionInfo b = HRegionInfo.convert(transition.getRegionInfo(2));
2620     RegionState rs_a = regionStates.getRegionState(a);
2621     RegionState rs_b = regionStates.getRegionState(b);
2622     if (rs_a == null || !rs_a.isMergingOnServer(serverName)
2623         || rs_b == null || !rs_b.isMergingOnServer(serverName)) {
2624       return "Some daughter is not known to be merging on " + serverName
2625         + ", a=" + rs_a + ", b=" + rs_b;
2626     }
2627 
2628     regionOffline(a, State.MERGED);
2629     regionOffline(b, State.MERGED);
2630     regionOnline(hri, serverName, 1);
2631 
2632     // User could disable the table before master knows the new region.
2633     if (getTableStateManager().isTableState(hri.getTable(),
2634         TableState.State.DISABLED, TableState.State.DISABLING)) {
2635       invokeUnAssign(hri);
2636     } else {
2637       Callable<Object> mergeReplicasCallable = new Callable<Object>() {
2638         @Override
2639         public Object call() {
2640           doMergingOfReplicas(hri, a, b);
2641           return null;
2642         }
2643       };
2644       threadPoolExecutorService.submit(mergeReplicasCallable);
2645     }
2646     return null;
2647   }
2648 
2649   private String onRegionMergeReverted(final RegionState current, final HRegionInfo hri,
2650       final ServerName serverName, final RegionStateTransition transition) {
2651     // The region must be in merging_new state, and the daughters must be
2652     // merging on this server.
2653     // If the region is in offline state, it could be an RPC retry.
2654     if (current == null || !current.isMergingNewOrOfflineOnServer(serverName)) {
2655       return hri.getShortNameToLog() + " is not merging on " + serverName;
2656     }
2657 
2658     // Just return in case of retrying
2659     if (current.isOffline()) {
2660       return null;
2661     }
2662 
2663     final HRegionInfo a = HRegionInfo.convert(transition.getRegionInfo(1));
2664     final HRegionInfo b = HRegionInfo.convert(transition.getRegionInfo(2));
2665     RegionState rs_a = regionStates.getRegionState(a);
2666     RegionState rs_b = regionStates.getRegionState(b);
2667     if (rs_a == null || !rs_a.isMergingOnServer(serverName)
2668         || rs_b == null || !rs_b.isMergingOnServer(serverName)) {
2669       return "Some daughter is not known to be merging on " + serverName
2670         + ", a=" + rs_a + ", b=" + rs_b;
2671     }
2672 
2673     regionOnline(a, serverName);
2674     regionOnline(b, serverName);
2675     regionOffline(hri);
2676 
2677     if (getTableStateManager().isTableState(hri.getTable(),
2678         TableState.State.DISABLED, TableState.State.DISABLING)) {
2679       invokeUnAssign(a);
2680       invokeUnAssign(b);
2681     }
2682     return null;
2683   }
2684 
2685   private void doMergingOfReplicas(HRegionInfo mergedHri, final HRegionInfo hri_a,
2686       final HRegionInfo hri_b) {
2687     // Close replicas for the original unmerged regions. create/assign new replicas
2688     // for the merged parent.
2689     List<HRegionInfo> unmergedRegions = new ArrayList<HRegionInfo>();
2690     unmergedRegions.add(hri_a);
2691     unmergedRegions.add(hri_b);
2692     Map<ServerName, List<HRegionInfo>> map = regionStates.getRegionAssignments(unmergedRegions);
2693     Collection<List<HRegionInfo>> c = map.values();
2694     for (List<HRegionInfo> l : c) {
2695       for (HRegionInfo h : l) {
2696         if (!RegionReplicaUtil.isDefaultReplica(h)) {
2697           LOG.debug("Unassigning un-merged replica " + h);
2698           unassign(h);
2699         }
2700       }
2701     }
2702     int numReplicas = 1;
2703     try {
2704       numReplicas = ((MasterServices)server).getTableDescriptors().get(mergedHri.getTable()).
2705           getRegionReplication();
2706     } catch (IOException e) {
2707       LOG.warn("Couldn't get the replication attribute of the table " + mergedHri.getTable() +
2708           " due to " + e.getMessage() + ". The assignment of replicas for the merged region " +
2709           "will not be done");
2710     }
2711     List<HRegionInfo> regions = new ArrayList<HRegionInfo>();
2712     for (int i = 1; i < numReplicas; i++) {
2713       regions.add(RegionReplicaUtil.getRegionInfoForReplica(mergedHri, i));
2714     }
2715     try {
2716       assign(regions);
2717     } catch (IOException ioe) {
2718       LOG.warn("Couldn't assign all replica(s) of region " + mergedHri + " because of " +
2719                 ioe.getMessage());
2720     } catch (InterruptedException ie) {
2721       LOG.warn("Couldn't assign all replica(s) of region " + mergedHri+ " because of " +
2722                 ie.getMessage());
2723     }
2724   }
2725 
2726   private void doSplittingOfReplicas(final HRegionInfo parentHri, final HRegionInfo hri_a,
2727       final HRegionInfo hri_b) {
2728     // create new regions for the replica, and assign them to match with the
2729     // current replica assignments. If replica1 of parent is assigned to RS1,
2730     // the replica1s of daughters will be on the same machine
2731     int numReplicas = 1;
2732     try {
2733       numReplicas = ((MasterServices)server).getTableDescriptors().get(parentHri.getTable()).
2734           getRegionReplication();
2735     } catch (IOException e) {
2736       LOG.warn("Couldn't get the replication attribute of the table " + parentHri.getTable() +
2737           " due to " + e.getMessage() + ". The assignment of daughter replicas " +
2738           "replicas will not be done");
2739     }
2740     // unassign the old replicas
2741     List<HRegionInfo> parentRegion = new ArrayList<HRegionInfo>();
2742     parentRegion.add(parentHri);
2743     Map<ServerName, List<HRegionInfo>> currentAssign =
2744         regionStates.getRegionAssignments(parentRegion);
2745     Collection<List<HRegionInfo>> c = currentAssign.values();
2746     for (List<HRegionInfo> l : c) {
2747       for (HRegionInfo h : l) {
2748         if (!RegionReplicaUtil.isDefaultReplica(h)) {
2749           LOG.debug("Unassigning parent's replica " + h);
2750           unassign(h);
2751         }
2752       }
2753     }
2754     // assign daughter replicas
2755     Map<HRegionInfo, ServerName> map = new HashMap<HRegionInfo, ServerName>();
2756     for (int i = 1; i < numReplicas; i++) {
2757       prepareDaughterReplicaForAssignment(hri_a, parentHri, i, map);
2758       prepareDaughterReplicaForAssignment(hri_b, parentHri, i, map);
2759     }
2760     try {
2761       assign(map);
2762     } catch (IOException e) {
2763       LOG.warn("Caught exception " + e + " while trying to assign replica(s) of daughter(s)");
2764     } catch (InterruptedException e) {
2765       LOG.warn("Caught exception " + e + " while trying to assign replica(s) of daughter(s)");
2766     }
2767   }
2768 
2769   private void prepareDaughterReplicaForAssignment(HRegionInfo daughterHri, HRegionInfo parentHri,
2770       int replicaId, Map<HRegionInfo, ServerName> map) {
2771     HRegionInfo parentReplica = RegionReplicaUtil.getRegionInfoForReplica(parentHri, replicaId);
2772     HRegionInfo daughterReplica = RegionReplicaUtil.getRegionInfoForReplica(daughterHri,
2773         replicaId);
2774     LOG.debug("Created replica region for daughter " + daughterReplica);
2775     ServerName sn;
2776     if ((sn = regionStates.getRegionServerOfRegion(parentReplica)) != null) {
2777       map.put(daughterReplica, sn);
2778     } else {
2779       List<ServerName> servers = serverManager.getOnlineServersList();
2780       sn = servers.get((new Random(System.currentTimeMillis())).nextInt(servers.size()));
2781       map.put(daughterReplica, sn);
2782     }
2783   }
2784 
2785   public Set<HRegionInfo> getReplicasToClose() {
2786     return replicasToClose;
2787   }
2788 
2789   /**
2790    * A region is offline.  The new state should be the specified one,
2791    * if not null.  If the specified state is null, the new state is Offline.
2792    * The specified state can be Split/Merged/Offline/null only.
2793    */
2794   private void regionOffline(final HRegionInfo regionInfo, final State state) {
2795     regionStates.regionOffline(regionInfo, state);
2796     removeClosedRegion(regionInfo);
2797     // remove the region plan as well just in case.
2798     clearRegionPlan(regionInfo);
2799     balancer.regionOffline(regionInfo);
2800 
2801     // Tell our listeners that a region was closed
2802     sendRegionClosedNotification(regionInfo);
2803     // also note that all the replicas of the primary should be closed
2804     if (state != null && state.equals(State.SPLIT)) {
2805       Collection<HRegionInfo> c = new ArrayList<HRegionInfo>(1);
2806       c.add(regionInfo);
2807       Map<ServerName, List<HRegionInfo>> map = regionStates.getRegionAssignments(c);
2808       Collection<List<HRegionInfo>> allReplicas = map.values();
2809       for (List<HRegionInfo> list : allReplicas) {
2810         replicasToClose.addAll(list);
2811       }
2812     }
2813     else if (state != null && state.equals(State.MERGED)) {
2814       Collection<HRegionInfo> c = new ArrayList<HRegionInfo>(1);
2815       c.add(regionInfo);
2816       Map<ServerName, List<HRegionInfo>> map = regionStates.getRegionAssignments(c);
2817       Collection<List<HRegionInfo>> allReplicas = map.values();
2818       for (List<HRegionInfo> list : allReplicas) {
2819         replicasToClose.addAll(list);
2820       }
2821     }
2822   }
2823 
2824   private void sendRegionOpenedNotification(final HRegionInfo regionInfo,
2825       final ServerName serverName) {
2826     if (!this.listeners.isEmpty()) {
2827       for (AssignmentListener listener : this.listeners) {
2828         listener.regionOpened(regionInfo, serverName);
2829       }
2830     }
2831   }
2832 
2833   private void sendRegionClosedNotification(final HRegionInfo regionInfo) {
2834     if (!this.listeners.isEmpty()) {
2835       for (AssignmentListener listener : this.listeners) {
2836         listener.regionClosed(regionInfo);
2837       }
2838     }
2839   }
2840 
2841   /**
2842    * Try to update some region states. If the state machine prevents
2843    * such update, an error message is returned to explain the reason.
2844    *
2845    * It's expected that in each transition there should have just one
2846    * region for opening/closing, 3 regions for splitting/merging.
2847    * These regions should be on the server that requested the change.
2848    *
2849    * Region state machine. Only these transitions
2850    * are expected to be triggered by a region server.
2851    *
2852    * On the state transition:
2853    *  (1) Open/Close should be initiated by master
2854    *      (a) Master sets the region to pending_open/pending_close
2855    *        in memory and hbase:meta after sending the request
2856    *        to the region server
2857    *      (b) Region server reports back to the master
2858    *        after open/close is done (either success/failure)
2859    *      (c) If region server has problem to report the status
2860    *        to master, it must be because the master is down or some
2861    *        temporary network issue. Otherwise, the region server should
2862    *        abort since it must be a bug. If the master is not accessible,
2863    *        the region server should keep trying until the server is
2864    *        stopped or till the status is reported to the (new) master
2865    *      (d) If region server dies in the middle of opening/closing
2866    *        a region, SSH picks it up and finishes it
2867    *      (e) If master dies in the middle, the new master recovers
2868    *        the state during initialization from hbase:meta. Region server
2869    *        can report any transition that has not been reported to
2870    *        the previous active master yet
2871    *  (2) Split/merge is initiated by region servers
2872    *      (a) To split a region, a region server sends a request
2873    *        to master to try to set a region to splitting, together with
2874    *        two daughters (to be created) to splitting new. If approved
2875    *        by the master, the splitting can then move ahead
2876    *      (b) To merge two regions, a region server sends a request to
2877    *        master to try to set the new merged region (to be created) to
2878    *        merging_new, together with two regions (to be merged) to merging.
2879    *        If it is ok with the master, the merge can then move ahead
2880    *      (c) Once the splitting/merging is done, the region server
2881    *        reports the status back to the master either success/failure.
2882    *      (d) Other scenarios should be handled similarly as for
2883    *        region open/close
2884    */
2885   protected String onRegionTransition(final ServerName serverName,
2886       final RegionStateTransition transition) {
2887     TransitionCode code = transition.getTransitionCode();
2888     HRegionInfo hri = HRegionInfo.convert(transition.getRegionInfo(0));
2889     Lock lock = locker.acquireLock(hri.getEncodedName());
2890     try {
2891       RegionState current = regionStates.getRegionState(hri);
2892       if (LOG.isDebugEnabled()) {
2893         LOG.debug("Got transition " + code + " for "
2894           + (current != null ? current.toString() : hri.getShortNameToLog())
2895           + " from " + serverName);
2896       }
2897       String errorMsg = null;
2898       switch (code) {
2899       case OPENED:
2900         errorMsg = onRegionOpen(current, hri, serverName, transition);
2901         break;
2902       case FAILED_OPEN:
2903         errorMsg = onRegionFailedOpen(current, hri, serverName);
2904         break;
2905       case CLOSED:
2906         errorMsg = onRegionClosed(current, hri, serverName);
2907         break;
2908       case READY_TO_SPLIT:
2909         try {
2910           regionStateListener.onRegionSplit(hri);
2911           errorMsg = onRegionReadyToSplit(current, hri, serverName, transition);
2912         } catch (IOException exp) {
2913             if (exp instanceof QuotaExceededException) {
2914               server.getRegionNormalizer().planSkipped(hri, PlanType.SPLIT);
2915             }
2916             errorMsg = StringUtils.stringifyException(exp);
2917         }
2918         break;
2919       case SPLIT_PONR:
2920         errorMsg = onRegionSplitPONR(current, hri, serverName, transition);
2921         break;
2922       case SPLIT:
2923         errorMsg = onRegionSplit(current, hri, serverName, transition);
2924         break;
2925       case SPLIT_REVERTED:
2926         errorMsg = onRegionSplitReverted(current, hri, serverName, transition);
2927         if (org.apache.commons.lang.StringUtils.isEmpty(errorMsg)) {
2928           try {
2929             regionStateListener.onRegionSplitReverted(hri);
2930           } catch (IOException exp) {
2931             LOG.warn(StringUtils.stringifyException(exp));
2932           }
2933         }
2934         break;
2935       case READY_TO_MERGE:
2936         errorMsg = onRegionReadyToMerge(current, hri, serverName, transition);
2937         break;
2938       case MERGE_PONR:
2939         errorMsg = onRegionMergePONR(current, hri, serverName, transition);
2940         break;
2941       case MERGED:
2942         try {
2943           errorMsg = onRegionMerged(current, hri, serverName, transition);
2944           regionStateListener.onRegionMerged(hri);
2945         } catch (IOException exp) {
2946           errorMsg = StringUtils.stringifyException(exp);
2947         }
2948         break;
2949       case MERGE_REVERTED:
2950         errorMsg = onRegionMergeReverted(current, hri, serverName, transition);
2951         break;
2952 
2953       default:
2954         errorMsg = "Unexpected transition code " + code;
2955       }
2956       if (errorMsg != null) {
2957         LOG.info("Could not transition region from " + current + " on "
2958           + code + " by " + serverName + ": " + errorMsg);
2959       }
2960       return errorMsg;
2961     } finally {
2962       lock.unlock();
2963     }
2964   }
2965 
2966   private void processBogusAssignments(Map<ServerName, List<HRegionInfo>> bulkPlan) {
2967     if (bulkPlan.containsKey(LoadBalancer.BOGUS_SERVER_NAME)) {
2968       // Found no plan for some regions, put those regions in RIT
2969       for (HRegionInfo hri : bulkPlan.get(LoadBalancer.BOGUS_SERVER_NAME)) {
2970         regionStates.updateRegionState(hri, State.FAILED_OPEN);
2971       }
2972       bulkPlan.remove(LoadBalancer.BOGUS_SERVER_NAME);
2973     }
2974   }
2975 
2976   /**
2977    * @return Instance of load balancer
2978    */
2979   public LoadBalancer getBalancer() {
2980     return this.balancer;
2981   }
2982 
2983   public Map<ServerName, List<HRegionInfo>>
2984     getSnapShotOfAssignment(Collection<HRegionInfo> infos) {
2985     return getRegionStates().getRegionAssignments(infos);
2986   }
2987 
2988   void setRegionStateListener(RegionStateListener listener) {
2989     this.regionStateListener = listener;
2990   }
2991 }