View Javadoc

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