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.Arrays;
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.Set;
32  import java.util.TreeMap;
33  import java.util.concurrent.ConcurrentHashMap;
34  import java.util.concurrent.ConcurrentSkipListSet;
35  import java.util.concurrent.ThreadFactory;
36  import java.util.concurrent.TimeUnit;
37  import java.util.concurrent.atomic.AtomicBoolean;
38  import java.util.concurrent.atomic.AtomicInteger;
39  import java.util.concurrent.locks.Lock;
40  import java.util.concurrent.locks.ReentrantLock;
41  
42  import org.apache.commons.logging.Log;
43  import org.apache.commons.logging.LogFactory;
44  import org.apache.hadoop.classification.InterfaceAudience;
45  import org.apache.hadoop.conf.Configuration;
46  import org.apache.hadoop.hbase.Chore;
47  import org.apache.hadoop.hbase.HBaseIOException;
48  import org.apache.hadoop.hbase.HConstants;
49  import org.apache.hadoop.hbase.HRegionInfo;
50  import org.apache.hadoop.hbase.NotServingRegionException;
51  import org.apache.hadoop.hbase.RegionTransition;
52  import org.apache.hadoop.hbase.Server;
53  import org.apache.hadoop.hbase.ServerName;
54  import org.apache.hadoop.hbase.Stoppable;
55  import org.apache.hadoop.hbase.TableName;
56  import org.apache.hadoop.hbase.TableNotFoundException;
57  import org.apache.hadoop.hbase.catalog.CatalogTracker;
58  import org.apache.hadoop.hbase.catalog.MetaReader;
59  import org.apache.hadoop.hbase.client.Result;
60  import org.apache.hadoop.hbase.exceptions.DeserializationException;
61  import org.apache.hadoop.hbase.executor.EventHandler;
62  import org.apache.hadoop.hbase.executor.EventType;
63  import org.apache.hadoop.hbase.executor.ExecutorService;
64  import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
65  import org.apache.hadoop.hbase.master.RegionState.State;
66  import org.apache.hadoop.hbase.master.balancer.FavoredNodeAssignmentHelper;
67  import org.apache.hadoop.hbase.master.balancer.FavoredNodeLoadBalancer;
68  import org.apache.hadoop.hbase.master.handler.ClosedRegionHandler;
69  import org.apache.hadoop.hbase.master.handler.DisableTableHandler;
70  import org.apache.hadoop.hbase.master.handler.EnableTableHandler;
71  import org.apache.hadoop.hbase.master.handler.OpenedRegionHandler;
72  import org.apache.hadoop.hbase.regionserver.RegionAlreadyInTransitionException;
73  import org.apache.hadoop.hbase.regionserver.RegionMergeTransaction;
74  import org.apache.hadoop.hbase.regionserver.RegionOpeningState;
75  import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
76  import org.apache.hadoop.hbase.regionserver.SplitTransaction;
77  import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
78  import org.apache.hadoop.hbase.util.KeyLocker;
79  import org.apache.hadoop.hbase.util.Pair;
80  import org.apache.hadoop.hbase.util.PairOfSameType;
81  import org.apache.hadoop.hbase.util.Threads;
82  import org.apache.hadoop.hbase.util.Triple;
83  import org.apache.hadoop.hbase.zookeeper.MetaRegionTracker;
84  import org.apache.hadoop.hbase.zookeeper.ZKAssign;
85  import org.apache.hadoop.hbase.zookeeper.ZKTable;
86  import org.apache.hadoop.hbase.zookeeper.ZKUtil;
87  import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
88  import org.apache.hadoop.ipc.RemoteException;
89  import org.apache.zookeeper.AsyncCallback;
90  import org.apache.zookeeper.KeeperException;
91  import org.apache.zookeeper.KeeperException.NoNodeException;
92  import org.apache.zookeeper.KeeperException.NodeExistsException;
93  import org.apache.zookeeper.data.Stat;
94  
95  import com.google.common.base.Preconditions;
96  import com.google.common.collect.LinkedHashMultimap;
97  
98  /**
99   * Manages and performs region assignment.
100  * <p>
101  * Monitors ZooKeeper for events related to regions in transition.
102  * <p>
103  * Handles existing regions in transition during master failover.
104  */
105 @InterfaceAudience.Private
106 public class AssignmentManager extends ZooKeeperListener {
107   private static final Log LOG = LogFactory.getLog(AssignmentManager.class);
108 
109   public static final ServerName HBCK_CODE_SERVERNAME = ServerName.valueOf(HConstants.HBCK_CODE_NAME,
110       -1, -1L);
111 
112   public static final String ASSIGNMENT_TIMEOUT = "hbase.master.assignment.timeoutmonitor.timeout";
113   public static final int DEFAULT_ASSIGNMENT_TIMEOUT_DEFAULT = 600000;
114   public static final String ASSIGNMENT_TIMEOUT_MANAGEMENT = "hbase.assignment.timeout.management";
115   public static final boolean DEFAULT_ASSIGNMENT_TIMEOUT_MANAGEMENT = false;
116 
117   public static final String ALREADY_IN_TRANSITION_WAITTIME
118     = "hbase.assignment.already.intransition.waittime";
119   public static final int DEFAULT_ALREADY_IN_TRANSITION_WAITTIME = 60000; // 1 minute
120 
121   protected final Server server;
122 
123   private ServerManager serverManager;
124 
125   private boolean shouldAssignRegionsWithFavoredNodes;
126 
127   private CatalogTracker catalogTracker;
128 
129   protected final TimeoutMonitor timeoutMonitor;
130 
131   private final TimerUpdater timerUpdater;
132 
133   private LoadBalancer balancer;
134 
135   private final MetricsAssignmentManager metricsAssignmentManager;
136 
137   private final TableLockManager tableLockManager;
138 
139   private AtomicInteger numRegionsOpened = new AtomicInteger(0);
140 
141   final private KeyLocker<String> locker = new KeyLocker<String>();
142 
143   /**
144    * Map of regions to reopen after the schema of a table is changed. Key -
145    * encoded region name, value - HRegionInfo
146    */
147   private final Map <String, HRegionInfo> regionsToReopen;
148 
149   /*
150    * Maximum times we recurse an assignment/unassignment.
151    * See below in {@link #assign()} and {@link #unassign()}.
152    */
153   private final int maximumAttempts;
154 
155   /**
156    * Map of two merging regions from the region to be created.
157    */
158   private final Map<String, PairOfSameType<HRegionInfo>> mergingRegions
159     = new HashMap<String, PairOfSameType<HRegionInfo>>();
160 
161   /**
162    * The sleep time for which the assignment will wait before retrying in case of hbase:meta assignment
163    * failure due to lack of availability of region plan
164    */
165   private final long sleepTimeBeforeRetryingMetaAssignment;
166 
167   /** Plans for region movement. Key is the encoded version of a region name*/
168   // TODO: When do plans get cleaned out?  Ever? In server open and in server
169   // shutdown processing -- St.Ack
170   // All access to this Map must be synchronized.
171   final NavigableMap<String, RegionPlan> regionPlans =
172     new TreeMap<String, RegionPlan>();
173 
174   private final ZKTable zkTable;
175 
176   /**
177    * Contains the server which need to update timer, these servers will be
178    * handled by {@link TimerUpdater}
179    */
180   private final ConcurrentSkipListSet<ServerName> serversInUpdatingTimer;
181 
182   private final ExecutorService executorService;
183 
184   // For unit tests, keep track of calls to ClosedRegionHandler
185   private Map<HRegionInfo, AtomicBoolean> closedRegionHandlerCalled = null;
186 
187   // For unit tests, keep track of calls to OpenedRegionHandler
188   private Map<HRegionInfo, AtomicBoolean> openedRegionHandlerCalled = null;
189 
190   //Thread pool executor service for timeout monitor
191   private java.util.concurrent.ExecutorService threadPoolExecutorService;
192 
193   // A bunch of ZK events workers. Each is a single thread executor service
194   private final java.util.concurrent.ExecutorService zkEventWorkers;
195 
196   private List<EventType> ignoreStatesRSOffline = Arrays.asList(
197       EventType.RS_ZK_REGION_FAILED_OPEN, EventType.RS_ZK_REGION_CLOSED);
198 
199   private final RegionStates regionStates;
200 
201   // The threshold to use bulk assigning. Using bulk assignment
202   // only if assigning at least this many regions to at least this
203   // many servers. If assigning fewer regions to fewer servers,
204   // bulk assigning may be not as efficient.
205   private final int bulkAssignThresholdRegions;
206   private final int bulkAssignThresholdServers;
207 
208   // Should bulk assignment wait till all regions are assigned,
209   // or it is timed out?  This is useful to measure bulk assignment
210   // performance, but not needed in most use cases.
211   private final boolean bulkAssignWaitTillAllAssigned;
212 
213   /**
214    * Indicator that AssignmentManager has recovered the region states so
215    * that ServerShutdownHandler can be fully enabled and re-assign regions
216    * of dead servers. So that when re-assignment happens, AssignmentManager
217    * has proper region states.
218    *
219    * Protected to ease testing.
220    */
221   protected final AtomicBoolean failoverCleanupDone = new AtomicBoolean(false);
222 
223   /** Is the TimeOutManagement activated **/
224   private final boolean tomActivated;
225 
226   /**
227    * A map to track the count a region fails to open in a row.
228    * So that we don't try to open a region forever if the failure is
229    * unrecoverable.  We don't put this information in region states
230    * because we don't expect this to happen frequently; we don't
231    * want to copy this information over during each state transition either.
232    */
233   private final ConcurrentHashMap<String, AtomicInteger>
234     failedOpenTracker = new ConcurrentHashMap<String, AtomicInteger>();
235 
236   /**
237    * For testing only!  Set to true to skip handling of split.
238    */
239   @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="MS_SHOULD_BE_FINAL")
240   public static boolean TEST_SKIP_SPLIT_HANDLING = false;
241 
242   /**
243    * Constructs a new assignment manager.
244    *
245    * @param server
246    * @param serverManager
247    * @param catalogTracker
248    * @param service
249    * @throws KeeperException
250    * @throws IOException
251    */
252   public AssignmentManager(Server server, ServerManager serverManager,
253       CatalogTracker catalogTracker, final LoadBalancer balancer,
254       final ExecutorService service, MetricsMaster metricsMaster,
255       final TableLockManager tableLockManager) throws KeeperException, IOException {
256     super(server.getZooKeeper());
257     this.server = server;
258     this.serverManager = serverManager;
259     this.catalogTracker = catalogTracker;
260     this.executorService = service;
261     this.regionsToReopen = Collections.synchronizedMap
262                            (new HashMap<String, HRegionInfo> ());
263     Configuration conf = server.getConfiguration();
264     // Only read favored nodes if using the favored nodes load balancer.
265     this.shouldAssignRegionsWithFavoredNodes = conf.getClass(
266            HConstants.HBASE_MASTER_LOADBALANCER_CLASS, Object.class).equals(
267            FavoredNodeLoadBalancer.class);
268     this.tomActivated = conf.getBoolean(
269       ASSIGNMENT_TIMEOUT_MANAGEMENT, DEFAULT_ASSIGNMENT_TIMEOUT_MANAGEMENT);
270     if (tomActivated){
271       this.serversInUpdatingTimer =  new ConcurrentSkipListSet<ServerName>();
272       this.timeoutMonitor = new TimeoutMonitor(
273         conf.getInt("hbase.master.assignment.timeoutmonitor.period", 30000),
274         server, serverManager,
275         conf.getInt(ASSIGNMENT_TIMEOUT, DEFAULT_ASSIGNMENT_TIMEOUT_DEFAULT));
276       this.timerUpdater = new TimerUpdater(conf.getInt(
277         "hbase.master.assignment.timerupdater.period", 10000), server);
278       Threads.setDaemonThreadRunning(timerUpdater.getThread(),
279         server.getServerName() + ".timerUpdater");
280     } else {
281       this.serversInUpdatingTimer =  null;
282       this.timeoutMonitor = null;
283       this.timerUpdater = null;
284     }
285     this.zkTable = new ZKTable(this.watcher);
286     // This is the max attempts, not retries, so it should be at least 1.
287     this.maximumAttempts = Math.max(1,
288       this.server.getConfiguration().getInt("hbase.assignment.maximum.attempts", 10));
289     this.sleepTimeBeforeRetryingMetaAssignment = this.server.getConfiguration().getLong(
290         "hbase.meta.assignment.retry.sleeptime", 1000l);
291     this.balancer = balancer;
292     int maxThreads = conf.getInt("hbase.assignment.threads.max", 30);
293     this.threadPoolExecutorService = Threads.getBoundedCachedThreadPool(
294       maxThreads, 60L, TimeUnit.SECONDS, Threads.newDaemonThreadFactory("AM."));
295     this.regionStates = new RegionStates(server, serverManager);
296 
297     this.bulkAssignWaitTillAllAssigned =
298       conf.getBoolean("hbase.bulk.assignment.waittillallassigned", false);
299     this.bulkAssignThresholdRegions = conf.getInt("hbase.bulk.assignment.threshold.regions", 7);
300     this.bulkAssignThresholdServers = conf.getInt("hbase.bulk.assignment.threshold.servers", 3);
301 
302     int workers = conf.getInt("hbase.assignment.zkevent.workers", 20);
303     ThreadFactory threadFactory = Threads.newDaemonThreadFactory("AM.ZK.Worker");
304     zkEventWorkers = Threads.getBoundedCachedThreadPool(workers, 60L,
305             TimeUnit.SECONDS, threadFactory);
306     this.tableLockManager = tableLockManager;
307 
308     this.metricsAssignmentManager = new MetricsAssignmentManager();
309   }
310 
311   void startTimeOutMonitor() {
312     if (tomActivated) {
313       Threads.setDaemonThreadRunning(timeoutMonitor.getThread(), server.getServerName()
314           + ".timeoutMonitor");
315     }
316   }
317 
318   /**
319    * @return Instance of ZKTable.
320    */
321   public ZKTable getZKTable() {
322     // These are 'expensive' to make involving trip to zk ensemble so allow
323     // sharing.
324     return this.zkTable;
325   }
326 
327   /**
328    * This SHOULD not be public. It is public now
329    * because of some unit tests.
330    *
331    * TODO: make it package private and keep RegionStates in the master package
332    */
333   public RegionStates getRegionStates() {
334     return regionStates;
335   }
336 
337   public RegionPlan getRegionReopenPlan(HRegionInfo hri) {
338     return new RegionPlan(hri, null, regionStates.getRegionServerOfRegion(hri));
339   }
340 
341   /**
342    * Add a regionPlan for the specified region.
343    * @param encodedName
344    * @param plan
345    */
346   public void addPlan(String encodedName, RegionPlan plan) {
347     synchronized (regionPlans) {
348       regionPlans.put(encodedName, plan);
349     }
350   }
351 
352   /**
353    * Add a map of region plans.
354    */
355   public void addPlans(Map<String, RegionPlan> plans) {
356     synchronized (regionPlans) {
357       regionPlans.putAll(plans);
358     }
359   }
360 
361   /**
362    * Set the list of regions that will be reopened
363    * because of an update in table schema
364    *
365    * @param regions
366    *          list of regions that should be tracked for reopen
367    */
368   public void setRegionsToReopen(List <HRegionInfo> regions) {
369     for(HRegionInfo hri : regions) {
370       regionsToReopen.put(hri.getEncodedName(), hri);
371     }
372   }
373 
374   /**
375    * Used by the client to identify if all regions have the schema updates
376    *
377    * @param tableName
378    * @return Pair indicating the status of the alter command
379    * @throws IOException
380    */
381   public Pair<Integer, Integer> getReopenStatus(TableName tableName)
382       throws IOException {
383     List <HRegionInfo> hris =
384       MetaReader.getTableRegions(this.server.getCatalogTracker(), tableName, true);
385     Integer pending = 0;
386     for (HRegionInfo hri : hris) {
387       String name = hri.getEncodedName();
388       // no lock concurrent access ok: sequential consistency respected.
389       if (regionsToReopen.containsKey(name)
390           || regionStates.isRegionInTransition(name)) {
391         pending++;
392       }
393     }
394     return new Pair<Integer, Integer>(pending, hris.size());
395   }
396 
397   /**
398    * Used by ServerShutdownHandler to make sure AssignmentManager has completed
399    * the failover cleanup before re-assigning regions of dead servers. So that
400    * when re-assignment happens, AssignmentManager has proper region states.
401    */
402   public boolean isFailoverCleanupDone() {
403     return failoverCleanupDone.get();
404   }
405 
406   /**
407    * To avoid racing with AM, external entities may need to lock a region,
408    * for example, when SSH checks what regions to skip re-assigning.
409    */
410   public Lock acquireRegionLock(final String encodedName) {
411     return locker.acquireLock(encodedName);
412   }
413 
414   /**
415    * Now, failover cleanup is completed. Notify server manager to
416    * process queued up dead servers processing, if any.
417    */
418   void failoverCleanupDone() {
419     failoverCleanupDone.set(true);
420     serverManager.processQueuedDeadServers();
421   }
422 
423   /**
424    * Called on startup.
425    * Figures whether a fresh cluster start of we are joining extant running cluster.
426    * @throws IOException
427    * @throws KeeperException
428    * @throws InterruptedException
429    */
430   void joinCluster() throws IOException,
431       KeeperException, InterruptedException {
432     // Concurrency note: In the below the accesses on regionsInTransition are
433     // outside of a synchronization block where usually all accesses to RIT are
434     // synchronized.  The presumption is that in this case it is safe since this
435     // method is being played by a single thread on startup.
436 
437     // TODO: Regions that have a null location and are not in regionsInTransitions
438     // need to be handled.
439 
440     // Scan hbase:meta to build list of existing regions, servers, and assignment
441     // Returns servers who have not checked in (assumed dead) and their regions
442     Map<ServerName, List<HRegionInfo>> deadServers = rebuildUserRegions();
443 
444     // This method will assign all user regions if a clean server startup or
445     // it will reconstruct master state and cleanup any leftovers from
446     // previous master process.
447     processDeadServersAndRegionsInTransition(deadServers);
448 
449     recoverTableInDisablingState();
450     recoverTableInEnablingState();
451   }
452 
453   /**
454    * Process all regions that are in transition in zookeeper and also
455    * processes the list of dead servers by scanning the META.
456    * Used by master joining an cluster.  If we figure this is a clean cluster
457    * startup, will assign all user regions.
458    * @param deadServers
459    *          Map of dead servers and their regions. Can be null.
460    * @throws KeeperException
461    * @throws IOException
462    * @throws InterruptedException
463    */
464   void processDeadServersAndRegionsInTransition(
465       final Map<ServerName, List<HRegionInfo>> deadServers)
466           throws KeeperException, IOException, InterruptedException {
467     List<String> nodes = ZKUtil.listChildrenNoWatch(watcher,
468       watcher.assignmentZNode);
469 
470     if (nodes == null) {
471       String errorMessage = "Failed to get the children from ZK";
472       server.abort(errorMessage, new IOException(errorMessage));
473       return;
474     }
475 
476     boolean failover = (!serverManager.getDeadServers().isEmpty() || !serverManager
477         .getRequeuedDeadServers().isEmpty());
478 
479     if (!failover) {
480       // If any one region except meta is assigned, it's a failover.
481       Map<HRegionInfo, ServerName> regions = regionStates.getRegionAssignments();
482       for (HRegionInfo hri: regions.keySet()) {
483         if (!hri.isMetaTable()) {
484           LOG.debug("Found " + hri + " out on cluster");
485           failover = true;
486           break;
487         }
488       }
489       if (!failover) {
490         // If any one region except meta is in transition, it's a failover.
491         for (String encodedName: nodes) {
492           RegionState state = regionStates.getRegionState(encodedName);
493           if (state != null && !state.getRegion().isMetaRegion()) {
494             LOG.debug("Found " + state.getRegion().getRegionNameAsString() + " in RITs");
495             failover = true;
496             break;
497           }
498         }
499       }
500     }
501 
502     // If we found user regions out on cluster, its a failover.
503     if (failover) {
504       LOG.info("Found regions out on cluster or in RIT; presuming failover");
505       // Process list of dead servers and regions in RIT.
506       // See HBASE-4580 for more information.
507       processDeadServersAndRecoverLostRegions(deadServers);
508     } else {
509       // Fresh cluster startup.
510       LOG.info("Clean cluster startup. Assigning userregions");
511       assignAllUserRegions();
512     }
513   }
514 
515   /**
516    * If region is up in zk in transition, then do fixup and block and wait until
517    * the region is assigned and out of transition.  Used on startup for
518    * catalog regions.
519    * @param hri Region to look for.
520    * @return True if we processed a region in transition else false if region
521    * was not up in zk in transition.
522    * @throws InterruptedException
523    * @throws KeeperException
524    * @throws IOException
525    */
526   boolean processRegionInTransitionAndBlockUntilAssigned(final HRegionInfo hri)
527       throws InterruptedException, KeeperException, IOException {
528     String encodedRegionName = hri.getEncodedName();
529     if (!processRegionInTransition(encodedRegionName, hri)) {
530       return false; // The region is not in transition
531     }
532     LOG.debug("Waiting on " + HRegionInfo.prettyPrint(encodedRegionName));
533     while (!this.server.isStopped() &&
534         this.regionStates.isRegionInTransition(encodedRegionName)) {
535       RegionState state = this.regionStates.getRegionTransitionState(encodedRegionName);
536       if (state == null || !serverManager.isServerOnline(state.getServerName())) {
537         // The region is not in transition, or not in transition on an online
538         // server. Doesn't help to block here any more. Caller need to
539         // verify the region is actually assigned.
540         break;
541       }
542       this.regionStates.waitForUpdate(100);
543     }
544     return true;
545   }
546 
547   /**
548    * Process failover of new master for region <code>encodedRegionName</code>
549    * up in zookeeper.
550    * @param encodedRegionName Region to process failover for.
551    * @param regionInfo If null we'll go get it from meta table.
552    * @return True if we processed <code>regionInfo</code> as a RIT.
553    * @throws KeeperException
554    * @throws IOException
555    */
556   boolean processRegionInTransition(final String encodedRegionName,
557       final HRegionInfo regionInfo) throws KeeperException, IOException {
558     // We need a lock here to ensure that we will not put the same region twice
559     // It has no reason to be a lock shared with the other operations.
560     // We can do the lock on the region only, instead of a global lock: what we want to ensure
561     // is that we don't have two threads working on the same region.
562     Lock lock = locker.acquireLock(encodedRegionName);
563     try {
564       Stat stat = new Stat();
565       byte [] data = ZKAssign.getDataAndWatch(watcher, encodedRegionName, stat);
566       if (data == null) return false;
567       RegionTransition rt;
568       try {
569         rt = RegionTransition.parseFrom(data);
570       } catch (DeserializationException e) {
571         LOG.warn("Failed parse znode data", e);
572         return false;
573       }
574       HRegionInfo hri = regionInfo;
575       if (hri == null) {
576         // The region info is not passed in. We will try to find the region
577         // from region states map/meta based on the encoded region name. But we
578         // may not be able to find it. This is valid for online merge that
579         // the region may have not been created if the merge is not completed.
580         // Therefore, it is not in meta at master recovery time.
581         hri = regionStates.getRegionInfo(rt.getRegionName());
582         EventType et = rt.getEventType();
583         if (hri == null && et != EventType.RS_ZK_REGION_MERGING
584             && et != EventType.RS_ZK_REQUEST_REGION_MERGE) {
585           LOG.warn("Couldn't find the region in recovering " + rt);
586           return false;
587         }
588       }
589       return processRegionsInTransition(
590         rt, hri, stat.getVersion());
591     } finally {
592       lock.unlock();
593     }
594   }
595 
596   /**
597    * This call is invoked only (1) master assign meta;
598    * (2) during failover mode startup, zk assignment node processing.
599    * The locker is set in the caller. It returns true if the region
600    * is in transition for sure, false otherwise.
601    *
602    * It should be private but it is used by some test too.
603    */
604   boolean processRegionsInTransition(
605       final RegionTransition rt, final HRegionInfo regionInfo,
606       final int expectedVersion) throws KeeperException {
607     EventType et = rt.getEventType();
608     // Get ServerName.  Could not be null.
609     final ServerName sn = rt.getServerName();
610     final byte[] regionName = rt.getRegionName();
611     final String encodedName = HRegionInfo.encodeRegionName(regionName);
612     final String prettyPrintedRegionName = HRegionInfo.prettyPrint(encodedName);
613     LOG.info("Processing " + prettyPrintedRegionName + " in state: " + et);
614 
615     if (regionStates.isRegionInTransition(encodedName)) {
616       LOG.info("Processed region " + prettyPrintedRegionName + " in state: "
617         + et + ", does nothing since the region is already in transition "
618         + regionStates.getRegionTransitionState(encodedName));
619       // Just return
620       return true;
621     }
622     if (!serverManager.isServerOnline(sn)) {
623       // It was transitioning on a dead server, so it's closed now.
624       // Force to OFFLINE and put it in transition, but not assign it
625       // since log splitting for the dead server is not done yet.
626       LOG.debug("RIT " + encodedName + " in state=" + rt.getEventType() +
627         " was on deadserver; forcing offline");
628       if (regionStates.isRegionOnline(regionInfo)) {
629         // Meta could still show the region is assigned to the previous
630         // server. If that server is online, when we reload the meta, the
631         // region is put back to online, we need to offline it.
632         regionStates.regionOffline(regionInfo);
633       }
634       // Put it back in transition so that SSH can re-assign it
635       regionStates.updateRegionState(regionInfo, State.OFFLINE, sn);
636       // No mater the previous server is online or offline,
637       // we need to reset the last region server of the region.
638       regionStates.setLastRegionServerOfRegion(sn, encodedName);
639       if (regionInfo.isMetaRegion()) {
640         // If it's meta region, reset the meta location.
641         // So that master knows the right meta region server.
642         MetaRegionTracker.setMetaLocation(watcher, sn);
643       }
644       // Make sure we know the server is dead.
645       if (!serverManager.isServerDead(sn)) {
646         serverManager.expireServer(sn);
647       }
648       return false;
649     }
650     switch (et) {
651       case M_ZK_REGION_CLOSING:
652         // Insert into RIT & resend the query to the region server: may be the previous master
653         // died before sending the query the first time.
654         final RegionState rsClosing = regionStates.updateRegionState(rt, State.CLOSING);
655         this.executorService.submit(
656           new EventHandler(server, EventType.M_MASTER_RECOVERY) {
657             @Override
658             public void process() throws IOException {
659               ReentrantLock lock = locker.acquireLock(regionInfo.getEncodedName());
660               try {
661                 unassign(regionInfo, rsClosing, expectedVersion, null, true, null);
662                 if (regionStates.isRegionOffline(regionInfo)) {
663                   assign(regionInfo, true);
664                 }
665               } finally {
666                 lock.unlock();
667               }
668             }
669           });
670         break;
671 
672       case RS_ZK_REGION_CLOSED:
673       case RS_ZK_REGION_FAILED_OPEN:
674         // Region is closed, insert into RIT and handle it
675         regionStates.updateRegionState(regionInfo, State.CLOSED, sn);
676         invokeAssign(regionInfo);
677         break;
678 
679       case M_ZK_REGION_OFFLINE:
680         // Insert in RIT and resend to the regionserver
681         regionStates.updateRegionState(rt, State.PENDING_OPEN);
682         final RegionState rsOffline = regionStates.getRegionState(regionInfo);
683         this.executorService.submit(
684           new EventHandler(server, EventType.M_MASTER_RECOVERY) {
685             @Override
686             public void process() throws IOException {
687               ReentrantLock lock = locker.acquireLock(regionInfo.getEncodedName());
688               try {
689                 RegionPlan plan = new RegionPlan(regionInfo, null, sn);
690                 addPlan(encodedName, plan);
691                 assign(rsOffline, false, false);
692               } finally {
693                 lock.unlock();
694               }
695             }
696           });
697         break;
698 
699       case RS_ZK_REGION_OPENING:
700         regionStates.updateRegionState(rt, State.OPENING);
701         break;
702 
703       case RS_ZK_REGION_OPENED:
704         // Region is opened, insert into RIT and handle it
705         // This could be done asynchronously, we would need then to acquire the lock in the
706         //  handler.
707         regionStates.updateRegionState(rt, State.OPEN);
708         new OpenedRegionHandler(server, this, regionInfo, sn, expectedVersion).process();
709         break;
710       case RS_ZK_REQUEST_REGION_SPLIT:
711       case RS_ZK_REGION_SPLITTING:
712       case RS_ZK_REGION_SPLIT:
713         // Splitting region should be online. We could have skipped it during
714         // user region rebuilding since we may consider the split is completed.
715         // Put it in SPLITTING state to avoid complications.
716         regionStates.regionOnline(regionInfo, sn);
717         regionStates.updateRegionState(rt, State.SPLITTING);
718         if (!handleRegionSplitting(
719             rt, encodedName, prettyPrintedRegionName, sn)) {
720           deleteSplittingNode(encodedName, sn);
721         }
722         break;
723       case RS_ZK_REQUEST_REGION_MERGE:
724       case RS_ZK_REGION_MERGING:
725       case RS_ZK_REGION_MERGED:
726         if (!handleRegionMerging(
727             rt, encodedName, prettyPrintedRegionName, sn)) {
728           deleteMergingNode(encodedName, sn);
729         }
730         break;
731       default:
732         throw new IllegalStateException("Received region in state:" + et + " is not valid.");
733     }
734     LOG.info("Processed region " + prettyPrintedRegionName + " in state "
735       + et + ", on " + (serverManager.isServerOnline(sn) ? "" : "dead ")
736       + "server: " + sn);
737     return true;
738   }
739 
740   /**
741    * When a region is closed, it should be removed from the regionsToReopen
742    * @param hri HRegionInfo of the region which was closed
743    */
744   public void removeClosedRegion(HRegionInfo hri) {
745     if (regionsToReopen.remove(hri.getEncodedName()) != null) {
746       LOG.debug("Removed region from reopening regions because it was closed");
747     }
748   }
749 
750   /**
751    * Handles various states an unassigned node can be in.
752    * <p>
753    * Method is called when a state change is suspected for an unassigned node.
754    * <p>
755    * This deals with skipped transitions (we got a CLOSED but didn't see CLOSING
756    * yet).
757    * @param rt
758    * @param expectedVersion
759    */
760   void handleRegion(final RegionTransition rt, int expectedVersion) {
761     if (rt == null) {
762       LOG.warn("Unexpected NULL input for RegionTransition rt");
763       return;
764     }
765     final ServerName sn = rt.getServerName();
766     // Check if this is a special HBCK transition
767     if (sn.equals(HBCK_CODE_SERVERNAME)) {
768       handleHBCK(rt);
769       return;
770     }
771     final long createTime = rt.getCreateTime();
772     final byte[] regionName = rt.getRegionName();
773     String encodedName = HRegionInfo.encodeRegionName(regionName);
774     String prettyPrintedRegionName = HRegionInfo.prettyPrint(encodedName);
775     // Verify this is a known server
776     if (!serverManager.isServerOnline(sn)
777       && !ignoreStatesRSOffline.contains(rt.getEventType())) {
778       LOG.warn("Attempted to handle region transition for server but " +
779         "it is not online: " + prettyPrintedRegionName + ", " + rt);
780       return;
781     }
782 
783     RegionState regionState =
784       regionStates.getRegionState(encodedName);
785     long startTime = System.currentTimeMillis();
786     if (LOG.isDebugEnabled()) {
787       boolean lateEvent = createTime < (startTime - 15000);
788       LOG.debug("Handling " + rt.getEventType() +
789         ", server=" + sn + ", region=" +
790         (prettyPrintedRegionName == null ? "null" : prettyPrintedRegionName) +
791         (lateEvent ? ", which is more than 15 seconds late" : "") +
792         ", current_state=" + regionState);
793     }
794     // We don't do anything for this event,
795     // so separate it out, no need to lock/unlock anything
796     if (rt.getEventType() == EventType.M_ZK_REGION_OFFLINE) {
797       return;
798     }
799 
800     // We need a lock on the region as we could update it
801     Lock lock = locker.acquireLock(encodedName);
802     try {
803       RegionState latestState =
804         regionStates.getRegionState(encodedName);
805       if ((regionState == null && latestState != null)
806           || (regionState != null && latestState == null)
807           || (regionState != null && latestState != null
808             && latestState.getState() != regionState.getState())) {
809         LOG.warn("Region state changed from " + regionState + " to "
810           + latestState + ", while acquiring lock");
811       }
812       long waitedTime = System.currentTimeMillis() - startTime;
813       if (waitedTime > 5000) {
814         LOG.warn("Took " + waitedTime + "ms to acquire the lock");
815       }
816       regionState = latestState;
817       switch (rt.getEventType()) {
818       case RS_ZK_REQUEST_REGION_SPLIT:
819       case RS_ZK_REGION_SPLITTING:
820       case RS_ZK_REGION_SPLIT:
821         if (!handleRegionSplitting(
822             rt, encodedName, prettyPrintedRegionName, sn)) {
823           deleteSplittingNode(encodedName, sn);
824         }
825         break;
826 
827       case RS_ZK_REQUEST_REGION_MERGE:
828       case RS_ZK_REGION_MERGING:
829       case RS_ZK_REGION_MERGED:
830         // Merged region is a new region, we can't find it in the region states now.
831         // However, the two merging regions are not new. They should be in state for merging.
832         if (!handleRegionMerging(
833             rt, encodedName, prettyPrintedRegionName, sn)) {
834           deleteMergingNode(encodedName, sn);
835         }
836         break;
837 
838       case M_ZK_REGION_CLOSING:
839         // Should see CLOSING after we have asked it to CLOSE or additional
840         // times after already being in state of CLOSING
841         if (regionState == null
842             || !regionState.isPendingCloseOrClosingOnServer(sn)) {
843           LOG.warn("Received CLOSING for " + prettyPrintedRegionName
844             + " from " + sn + " but the region isn't PENDING_CLOSE/CLOSING here: "
845             + regionStates.getRegionState(encodedName));
846           return;
847         }
848         // Transition to CLOSING (or update stamp if already CLOSING)
849         regionStates.updateRegionState(rt, State.CLOSING);
850         break;
851 
852       case RS_ZK_REGION_CLOSED:
853         // Should see CLOSED after CLOSING but possible after PENDING_CLOSE
854         if (regionState == null
855             || !regionState.isPendingCloseOrClosingOnServer(sn)) {
856           LOG.warn("Received CLOSED for " + prettyPrintedRegionName
857             + " from " + sn + " but the region isn't PENDING_CLOSE/CLOSING here: "
858             + regionStates.getRegionState(encodedName));
859           return;
860         }
861         // Handle CLOSED by assigning elsewhere or stopping if a disable
862         // If we got here all is good.  Need to update RegionState -- else
863         // what follows will fail because not in expected state.
864         new ClosedRegionHandler(server, this, regionState.getRegion()).process();
865         updateClosedRegionHandlerTracker(regionState.getRegion());
866         break;
867 
868         case RS_ZK_REGION_FAILED_OPEN:
869           if (regionState == null
870               || !regionState.isPendingOpenOrOpeningOnServer(sn)) {
871             LOG.warn("Received FAILED_OPEN for " + prettyPrintedRegionName
872               + " from " + sn + " but the region isn't PENDING_OPEN/OPENING here: "
873               + regionStates.getRegionState(encodedName));
874             return;
875           }
876           AtomicInteger failedOpenCount = failedOpenTracker.get(encodedName);
877           if (failedOpenCount == null) {
878             failedOpenCount = new AtomicInteger();
879             // No need to use putIfAbsent, or extra synchronization since
880             // this whole handleRegion block is locked on the encoded region
881             // name, and failedOpenTracker is updated only in this block
882             failedOpenTracker.put(encodedName, failedOpenCount);
883           }
884           if (failedOpenCount.incrementAndGet() >= maximumAttempts) {
885             regionStates.updateRegionState(rt, State.FAILED_OPEN);
886             // remove the tracking info to save memory, also reset
887             // the count for next open initiative
888             failedOpenTracker.remove(encodedName);
889           } else {
890             // Handle this the same as if it were opened and then closed.
891             regionState = regionStates.updateRegionState(rt, State.CLOSED);
892             if (regionState != null) {
893               // When there are more than one region server a new RS is selected as the
894               // destination and the same is updated in the regionplan. (HBASE-5546)
895               try {
896                 getRegionPlan(regionState.getRegion(), sn, true);
897                 new ClosedRegionHandler(server, this, regionState.getRegion()).process();
898               } catch (HBaseIOException e) {
899                 LOG.warn("Failed to get region plan", e);
900               }
901             }
902           }
903           break;
904 
905         case RS_ZK_REGION_OPENING:
906           // Should see OPENING after we have asked it to OPEN or additional
907           // times after already being in state of OPENING
908           if (regionState == null
909               || !regionState.isPendingOpenOrOpeningOnServer(sn)) {
910             LOG.warn("Received OPENING for " + prettyPrintedRegionName
911               + " from " + sn + " but the region isn't PENDING_OPEN/OPENING here: "
912               + regionStates.getRegionState(encodedName));
913             return;
914           }
915           // Transition to OPENING (or update stamp if already OPENING)
916           regionStates.updateRegionState(rt, State.OPENING);
917           break;
918 
919         case RS_ZK_REGION_OPENED:
920           // Should see OPENED after OPENING but possible after PENDING_OPEN.
921           if (regionState == null
922               || !regionState.isPendingOpenOrOpeningOnServer(sn)) {
923             LOG.warn("Received OPENED for " + prettyPrintedRegionName
924               + " from " + sn + " but the region isn't PENDING_OPEN/OPENING here: "
925               + regionStates.getRegionState(encodedName));
926 
927             // Close it without updating the internal region states,
928             // so as not to create double assignments in unlucky scenarios
929             // mentioned in OpenRegionHandler#process
930             unassign(regionState.getRegion(), null, -1, null, false, sn);
931             return;
932           }
933           // Handle OPENED by removing from transition and deleted zk node
934           regionState = regionStates.updateRegionState(rt, State.OPEN);
935           if (regionState != null) {
936             failedOpenTracker.remove(encodedName); // reset the count, if any
937             new OpenedRegionHandler(
938               server, this, regionState.getRegion(), sn, expectedVersion).process();
939             updateOpenedRegionHandlerTracker(regionState.getRegion());
940           }
941           break;
942 
943         default:
944           throw new IllegalStateException("Received event is not valid.");
945       }
946     } finally {
947       lock.unlock();
948     }
949   }
950 
951   //For unit tests only
952   boolean wasClosedHandlerCalled(HRegionInfo hri) {
953     AtomicBoolean b = closedRegionHandlerCalled.get(hri);
954     //compareAndSet to be sure that unit tests don't see stale values. Means,
955     //we will return true exactly once unless the handler code resets to true
956     //this value.
957     return b == null ? false : b.compareAndSet(true, false);
958   }
959 
960   //For unit tests only
961   boolean wasOpenedHandlerCalled(HRegionInfo hri) {
962     AtomicBoolean b = openedRegionHandlerCalled.get(hri);
963     //compareAndSet to be sure that unit tests don't see stale values. Means,
964     //we will return true exactly once unless the handler code resets to true
965     //this value.
966     return b == null ? false : b.compareAndSet(true, false);
967   }
968 
969   //For unit tests only
970   void initializeHandlerTrackers() {
971     closedRegionHandlerCalled = new HashMap<HRegionInfo, AtomicBoolean>();
972     openedRegionHandlerCalled = new HashMap<HRegionInfo, AtomicBoolean>();
973   }
974 
975   void updateClosedRegionHandlerTracker(HRegionInfo hri) {
976     if (closedRegionHandlerCalled != null) { //only for unit tests this is true
977       closedRegionHandlerCalled.put(hri, new AtomicBoolean(true));
978     }
979   }
980 
981   void updateOpenedRegionHandlerTracker(HRegionInfo hri) {
982     if (openedRegionHandlerCalled != null) { //only for unit tests this is true
983       openedRegionHandlerCalled.put(hri, new AtomicBoolean(true));
984     }
985   }
986 
987   // TODO: processFavoredNodes might throw an exception, for e.g., if the
988   // meta could not be contacted/updated. We need to see how seriously to treat
989   // this problem as. Should we fail the current assignment. We should be able
990   // to recover from this problem eventually (if the meta couldn't be updated
991   // things should work normally and eventually get fixed up).
992   void processFavoredNodes(List<HRegionInfo> regions) throws IOException {
993     if (!shouldAssignRegionsWithFavoredNodes) return;
994     // The AM gets the favored nodes info for each region and updates the meta
995     // table with that info
996     Map<HRegionInfo, List<ServerName>> regionToFavoredNodes =
997         new HashMap<HRegionInfo, List<ServerName>>();
998     for (HRegionInfo region : regions) {
999       regionToFavoredNodes.put(region,
1000           ((FavoredNodeLoadBalancer)this.balancer).getFavoredNodes(region));
1001     }
1002     FavoredNodeAssignmentHelper.updateMetaWithFavoredNodesInfo(regionToFavoredNodes, catalogTracker);
1003   }
1004 
1005   /**
1006    * Handle a ZK unassigned node transition triggered by HBCK repair tool.
1007    * <p>
1008    * This is handled in a separate code path because it breaks the normal rules.
1009    * @param rt
1010    */
1011   private void handleHBCK(RegionTransition rt) {
1012     String encodedName = HRegionInfo.encodeRegionName(rt.getRegionName());
1013     LOG.info("Handling HBCK triggered transition=" + rt.getEventType() +
1014       ", server=" + rt.getServerName() + ", region=" +
1015       HRegionInfo.prettyPrint(encodedName));
1016     RegionState regionState = regionStates.getRegionTransitionState(encodedName);
1017     switch (rt.getEventType()) {
1018       case M_ZK_REGION_OFFLINE:
1019         HRegionInfo regionInfo;
1020         if (regionState != null) {
1021           regionInfo = regionState.getRegion();
1022         } else {
1023           try {
1024             byte [] name = rt.getRegionName();
1025             Pair<HRegionInfo, ServerName> p = MetaReader.getRegion(catalogTracker, name);
1026             regionInfo = p.getFirst();
1027           } catch (IOException e) {
1028             LOG.info("Exception reading hbase:meta doing HBCK repair operation", e);
1029             return;
1030           }
1031         }
1032         LOG.info("HBCK repair is triggering assignment of region=" +
1033             regionInfo.getRegionNameAsString());
1034         // trigger assign, node is already in OFFLINE so don't need to update ZK
1035         assign(regionInfo, false);
1036         break;
1037 
1038       default:
1039         LOG.warn("Received unexpected region state from HBCK: " + rt.toString());
1040         break;
1041     }
1042 
1043   }
1044 
1045   // ZooKeeper events
1046 
1047   /**
1048    * New unassigned node has been created.
1049    *
1050    * <p>This happens when an RS begins the OPENING or CLOSING of a region by
1051    * creating an unassigned node.
1052    *
1053    * <p>When this happens we must:
1054    * <ol>
1055    *   <li>Watch the node for further events</li>
1056    *   <li>Read and handle the state in the node</li>
1057    * </ol>
1058    */
1059   @Override
1060   public void nodeCreated(String path) {
1061     handleAssignmentEvent(path);
1062   }
1063 
1064   /**
1065    * Existing unassigned node has had data changed.
1066    *
1067    * <p>This happens when an RS transitions from OFFLINE to OPENING, or between
1068    * OPENING/OPENED and CLOSING/CLOSED.
1069    *
1070    * <p>When this happens we must:
1071    * <ol>
1072    *   <li>Watch the node for further events</li>
1073    *   <li>Read and handle the state in the node</li>
1074    * </ol>
1075    */
1076   @Override
1077   public void nodeDataChanged(String path) {
1078     handleAssignmentEvent(path);
1079   }
1080 
1081 
1082   // We  don't want to have two events on the same region managed simultaneously.
1083   // For this reason, we need to wait if an event on the same region is currently in progress.
1084   // So we track the region names of the events in progress, and we keep a waiting list.
1085   private final Set<String> regionsInProgress = new HashSet<String>();
1086   // In a LinkedHashMultimap, the put order is kept when we retrieve the collection back. We need
1087   //  this as we want the events to be managed in the same order as we received them.
1088   private final LinkedHashMultimap <String, RegionRunnable>
1089       zkEventWorkerWaitingList = LinkedHashMultimap.create();
1090 
1091   /**
1092    * A specific runnable that works only on a region.
1093    */
1094   private interface RegionRunnable extends Runnable{
1095     /**
1096      * @return - the name of the region it works on.
1097      */
1098     String getRegionName();
1099   }
1100 
1101   /**
1102    * Submit a task, ensuring that there is only one task at a time that working on a given region.
1103    * Order is respected.
1104    */
1105   protected void zkEventWorkersSubmit(final RegionRunnable regRunnable) {
1106 
1107     synchronized (regionsInProgress) {
1108       // If we're there is already a task with this region, we add it to the
1109       //  waiting list and return.
1110       if (regionsInProgress.contains(regRunnable.getRegionName())) {
1111         synchronized (zkEventWorkerWaitingList){
1112           zkEventWorkerWaitingList.put(regRunnable.getRegionName(), regRunnable);
1113         }
1114         return;
1115       }
1116 
1117       // No event in progress on this region => we can submit a new task immediately.
1118       regionsInProgress.add(regRunnable.getRegionName());
1119       zkEventWorkers.submit(new Runnable() {
1120         @Override
1121         public void run() {
1122           try {
1123             regRunnable.run();
1124           } finally {
1125             // now that we have finished, let's see if there is an event for the same region in the
1126             //  waiting list. If it's the case, we can now submit it to the pool.
1127             synchronized (regionsInProgress) {
1128               regionsInProgress.remove(regRunnable.getRegionName());
1129               synchronized (zkEventWorkerWaitingList) {
1130                 java.util.Set<RegionRunnable> waiting = zkEventWorkerWaitingList.get(
1131                     regRunnable.getRegionName());
1132                 if (!waiting.isEmpty()) {
1133                   // We want the first object only. The only way to get it is through an iterator.
1134                   RegionRunnable toSubmit = waiting.iterator().next();
1135                   zkEventWorkerWaitingList.remove(toSubmit.getRegionName(), toSubmit);
1136                   zkEventWorkersSubmit(toSubmit);
1137                 }
1138               }
1139             }
1140           }
1141         }
1142       });
1143     }
1144   }
1145 
1146   @Override
1147   public void nodeDeleted(final String path) {
1148     if (path.startsWith(watcher.assignmentZNode)) {
1149       final String regionName = ZKAssign.getRegionName(watcher, path);
1150       zkEventWorkersSubmit(new RegionRunnable() {
1151         @Override
1152         public String getRegionName() {
1153           return regionName;
1154         }
1155 
1156         @Override
1157         public void run() {
1158           Lock lock = locker.acquireLock(regionName);
1159           try {
1160             RegionState rs = regionStates.getRegionTransitionState(regionName);
1161             if (rs == null) {
1162               rs = regionStates.getRegionState(regionName);
1163               if (rs == null || !rs.isMergingNew()) {
1164                 // MergingNew is an offline state
1165                 return;
1166               }
1167             }
1168 
1169             HRegionInfo regionInfo = rs.getRegion();
1170             String regionNameStr = regionInfo.getRegionNameAsString();
1171             LOG.debug("Znode " + regionNameStr + " deleted, state: " + rs);
1172             boolean disabled = getZKTable().isDisablingOrDisabledTable(regionInfo.getTable());
1173             ServerName serverName = rs.getServerName();
1174             if (serverManager.isServerOnline(serverName)) {
1175               if (rs.isOnServer(serverName)
1176                   && (rs.isOpened() || rs.isSplitting())) {
1177                 regionOnline(regionInfo, serverName);
1178                 if (disabled) {
1179                   // if server is offline, no hurt to unassign again
1180                   LOG.info("Opened " + regionNameStr
1181                     + "but this table is disabled, triggering close of region");
1182                   unassign(regionInfo);
1183                 }
1184               } else if (rs.isMergingNew()) {
1185                 synchronized (regionStates) {
1186                   String p = regionInfo.getEncodedName();
1187                   PairOfSameType<HRegionInfo> regions = mergingRegions.get(p);
1188                   if (regions != null) {
1189                     onlineMergingRegion(disabled, regions.getFirst(), serverName);
1190                     onlineMergingRegion(disabled, regions.getSecond(), serverName);
1191                   }
1192                 }
1193               }
1194             }
1195           } finally {
1196             lock.unlock();
1197           }
1198         }
1199 
1200         private void onlineMergingRegion(boolean disabled,
1201             final HRegionInfo hri, final ServerName serverName) {
1202           RegionState regionState = regionStates.getRegionState(hri);
1203           if (regionState != null && regionState.isMerging()
1204               && regionState.isOnServer(serverName)) {
1205             regionOnline(regionState.getRegion(), serverName);
1206             if (disabled) {
1207               unassign(hri);
1208             }
1209           }
1210         }
1211       });
1212     }
1213   }
1214 
1215   /**
1216    * New unassigned node has been created.
1217    *
1218    * <p>This happens when an RS begins the OPENING, SPLITTING or CLOSING of a
1219    * region by creating a znode.
1220    *
1221    * <p>When this happens we must:
1222    * <ol>
1223    *   <li>Watch the node for further children changed events</li>
1224    *   <li>Watch all new children for changed events</li>
1225    * </ol>
1226    */
1227   @Override
1228   public void nodeChildrenChanged(String path) {
1229     if (path.equals(watcher.assignmentZNode)) {
1230       zkEventWorkers.submit(new Runnable() {
1231         @Override
1232         public void run() {
1233           try {
1234             // Just make sure we see the changes for the new znodes
1235             List<String> children =
1236               ZKUtil.listChildrenAndWatchForNewChildren(
1237                 watcher, watcher.assignmentZNode);
1238             if (children != null) {
1239               Stat stat = new Stat();
1240               for (String child : children) {
1241                 // if region is in transition, we already have a watch
1242                 // on it, so no need to watch it again. So, as I know for now,
1243                 // this is needed to watch splitting nodes only.
1244                 if (!regionStates.isRegionInTransition(child)) {
1245                   ZKAssign.getDataAndWatch(watcher, child, stat);
1246                 }
1247               }
1248             }
1249           } catch (KeeperException e) {
1250             server.abort("Unexpected ZK exception reading unassigned children", e);
1251           }
1252         }
1253       });
1254     }
1255   }
1256 
1257   /**
1258    * Marks the region as online.  Removes it from regions in transition and
1259    * updates the in-memory assignment information.
1260    * <p>
1261    * Used when a region has been successfully opened on a region server.
1262    * @param regionInfo
1263    * @param sn
1264    */
1265   void regionOnline(HRegionInfo regionInfo, ServerName sn) {
1266     numRegionsOpened.incrementAndGet();
1267     regionStates.regionOnline(regionInfo, sn);
1268 
1269     // Remove plan if one.
1270     clearRegionPlan(regionInfo);
1271     // Add the server to serversInUpdatingTimer
1272     addToServersInUpdatingTimer(sn);
1273   }
1274 
1275   /**
1276    * Pass the assignment event to a worker for processing.
1277    * Each worker is a single thread executor service.  The reason
1278    * for just one thread is to make sure all events for a given
1279    * region are processed in order.
1280    *
1281    * @param path
1282    */
1283   private void handleAssignmentEvent(final String path) {
1284     if (path.startsWith(watcher.assignmentZNode)) {
1285       final String regionName = ZKAssign.getRegionName(watcher, path);
1286 
1287       zkEventWorkersSubmit(new RegionRunnable() {
1288         @Override
1289         public String getRegionName() {
1290           return regionName;
1291         }
1292 
1293         @Override
1294         public void run() {
1295           try {
1296             Stat stat = new Stat();
1297             byte [] data = ZKAssign.getDataAndWatch(watcher, path, stat);
1298             if (data == null) return;
1299 
1300             RegionTransition rt = RegionTransition.parseFrom(data);
1301             handleRegion(rt, stat.getVersion());
1302           } catch (KeeperException e) {
1303             server.abort("Unexpected ZK exception reading unassigned node data", e);
1304           } catch (DeserializationException e) {
1305             server.abort("Unexpected exception deserializing node data", e);
1306           }
1307         }
1308       });
1309     }
1310   }
1311 
1312   /**
1313    * Add the server to the set serversInUpdatingTimer, then {@link TimerUpdater}
1314    * will update timers for this server in background
1315    * @param sn
1316    */
1317   private void addToServersInUpdatingTimer(final ServerName sn) {
1318     if (tomActivated){
1319       this.serversInUpdatingTimer.add(sn);
1320     }
1321   }
1322 
1323   /**
1324    * Touch timers for all regions in transition that have the passed
1325    * <code>sn</code> in common.
1326    * Call this method whenever a server checks in.  Doing so helps the case where
1327    * a new regionserver has joined the cluster and its been given 1k regions to
1328    * open.  If this method is tickled every time the region reports in a
1329    * successful open then the 1k-th region won't be timed out just because its
1330    * sitting behind the open of 999 other regions.  This method is NOT used
1331    * as part of bulk assign -- there we have a different mechanism for extending
1332    * the regions in transition timer (we turn it off temporarily -- because
1333    * there is no regionplan involved when bulk assigning.
1334    * @param sn
1335    */
1336   private void updateTimers(final ServerName sn) {
1337     Preconditions.checkState(tomActivated);
1338     if (sn == null) return;
1339 
1340     // This loop could be expensive.
1341     // First make a copy of current regionPlan rather than hold sync while
1342     // looping because holding sync can cause deadlock.  Its ok in this loop
1343     // if the Map we're going against is a little stale
1344     List<Map.Entry<String, RegionPlan>> rps;
1345     synchronized(this.regionPlans) {
1346       rps = new ArrayList<Map.Entry<String, RegionPlan>>(regionPlans.entrySet());
1347     }
1348 
1349     for (Map.Entry<String, RegionPlan> e : rps) {
1350       if (e.getValue() != null && e.getKey() != null && sn.equals(e.getValue().getDestination())) {
1351         RegionState regionState = regionStates.getRegionTransitionState(e.getKey());
1352         if (regionState != null) {
1353           regionState.updateTimestampToNow();
1354         }
1355       }
1356     }
1357   }
1358 
1359   /**
1360    * Marks the region as offline.  Removes it from regions in transition and
1361    * removes in-memory assignment information.
1362    * <p>
1363    * Used when a region has been closed and should remain closed.
1364    * @param regionInfo
1365    */
1366   public void regionOffline(final HRegionInfo regionInfo) {
1367     regionOffline(regionInfo, null);
1368   }
1369 
1370   public void offlineDisabledRegion(HRegionInfo regionInfo) {
1371     // Disabling so should not be reassigned, just delete the CLOSED node
1372     LOG.debug("Table being disabled so deleting ZK node and removing from " +
1373       "regions in transition, skipping assignment of region " +
1374         regionInfo.getRegionNameAsString());
1375     String encodedName = regionInfo.getEncodedName();
1376     deleteNodeInStates(encodedName, "closed", null,
1377       EventType.RS_ZK_REGION_CLOSED, EventType.M_ZK_REGION_OFFLINE);
1378     regionOffline(regionInfo);
1379   }
1380 
1381   // Assignment methods
1382 
1383   /**
1384    * Assigns the specified region.
1385    * <p>
1386    * If a RegionPlan is available with a valid destination then it will be used
1387    * to determine what server region is assigned to.  If no RegionPlan is
1388    * available, region will be assigned to a random available server.
1389    * <p>
1390    * Updates the RegionState and sends the OPEN RPC.
1391    * <p>
1392    * This will only succeed if the region is in transition and in a CLOSED or
1393    * OFFLINE state or not in transition (in-memory not zk), and of course, the
1394    * chosen server is up and running (It may have just crashed!).  If the
1395    * in-memory checks pass, the zk node is forced to OFFLINE before assigning.
1396    *
1397    * @param region server to be assigned
1398    * @param setOfflineInZK whether ZK node should be created/transitioned to an
1399    *                       OFFLINE state before assigning the region
1400    */
1401   public void assign(HRegionInfo region, boolean setOfflineInZK) {
1402     assign(region, setOfflineInZK, false);
1403   }
1404 
1405   /**
1406    * Use care with forceNewPlan. It could cause double assignment.
1407    */
1408   public void assign(HRegionInfo region,
1409       boolean setOfflineInZK, boolean forceNewPlan) {
1410     if (isDisabledorDisablingRegionInRIT(region)) {
1411       return;
1412     }
1413     if (this.serverManager.isClusterShutdown()) {
1414       LOG.info("Cluster shutdown is set; skipping assign of " +
1415         region.getRegionNameAsString());
1416       return;
1417     }
1418     String encodedName = region.getEncodedName();
1419     Lock lock = locker.acquireLock(encodedName);
1420     try {
1421       RegionState state = forceRegionStateToOffline(region, forceNewPlan);
1422       if (state != null) {
1423         if (regionStates.wasRegionOnDeadServer(encodedName)) {
1424           LOG.info("Skip assigning " + region.getRegionNameAsString()
1425             + ", it's host " + regionStates.getLastRegionServerOfRegion(encodedName)
1426             + " is dead but not processed yet");
1427           return;
1428         }
1429         assign(state, setOfflineInZK, forceNewPlan);
1430       }
1431     } finally {
1432       lock.unlock();
1433     }
1434   }
1435 
1436   /**
1437    * Bulk assign regions to <code>destination</code>.
1438    * @param destination
1439    * @param regions Regions to assign.
1440    * @return true if successful
1441    */
1442   boolean assign(final ServerName destination, final List<HRegionInfo> regions) {
1443     long startTime = EnvironmentEdgeManager.currentTimeMillis();
1444     try {
1445       int regionCount = regions.size();
1446       if (regionCount == 0) {
1447         return true;
1448       }
1449       LOG.debug("Assigning " + regionCount + " region(s) to " + destination.toString());
1450       Set<String> encodedNames = new HashSet<String>(regionCount);
1451       for (HRegionInfo region : regions) {
1452         encodedNames.add(region.getEncodedName());
1453       }
1454 
1455       List<HRegionInfo> failedToOpenRegions = new ArrayList<HRegionInfo>();
1456       Map<String, Lock> locks = locker.acquireLocks(encodedNames);
1457       try {
1458         AtomicInteger counter = new AtomicInteger(0);
1459         Map<String, Integer> offlineNodesVersions = new ConcurrentHashMap<String, Integer>();
1460         OfflineCallback cb = new OfflineCallback(
1461           watcher, destination, counter, offlineNodesVersions);
1462         Map<String, RegionPlan> plans = new HashMap<String, RegionPlan>(regions.size());
1463         List<RegionState> states = new ArrayList<RegionState>(regions.size());
1464         for (HRegionInfo region : regions) {
1465           String encodedName = region.getEncodedName();
1466           if (!isDisabledorDisablingRegionInRIT(region)) {
1467             RegionState state = forceRegionStateToOffline(region, false);
1468             boolean onDeadServer = false;
1469             if (state != null) {
1470               if (regionStates.wasRegionOnDeadServer(encodedName)) {
1471                 LOG.info("Skip assigning " + region.getRegionNameAsString()
1472                   + ", it's host " + regionStates.getLastRegionServerOfRegion(encodedName)
1473                   + " is dead but not processed yet");
1474                 onDeadServer = true;
1475               } else if (asyncSetOfflineInZooKeeper(state, cb, destination)) {
1476                 RegionPlan plan = new RegionPlan(region, state.getServerName(), destination);
1477                 plans.put(encodedName, plan);
1478                 states.add(state);
1479                 continue;
1480               }
1481             }
1482             // Reassign if the region wasn't on a dead server
1483             if (!onDeadServer) {
1484               LOG.info("failed to force region state to offline or "
1485                 + "failed to set it offline in ZK, will reassign later: " + region);
1486               failedToOpenRegions.add(region); // assign individually later
1487             }
1488           }
1489           // Release the lock, this region is excluded from bulk assign because
1490           // we can't update its state, or set its znode to offline.
1491           Lock lock = locks.remove(encodedName);
1492           lock.unlock();
1493         }
1494 
1495         // Wait until all unassigned nodes have been put up and watchers set.
1496         int total = states.size();
1497         for (int oldCounter = 0; !server.isStopped();) {
1498           int count = counter.get();
1499           if (oldCounter != count) {
1500             LOG.info(destination.toString() + " unassigned znodes=" + count +
1501               " of total=" + total);
1502             oldCounter = count;
1503           }
1504           if (count >= total) break;
1505           Threads.sleep(5);
1506         }
1507 
1508         if (server.isStopped()) {
1509           return false;
1510         }
1511 
1512         // Add region plans, so we can updateTimers when one region is opened so
1513         // that unnecessary timeout on RIT is reduced.
1514         this.addPlans(plans);
1515 
1516         List<Triple<HRegionInfo, Integer, List<ServerName>>> regionOpenInfos =
1517           new ArrayList<Triple<HRegionInfo, Integer, List<ServerName>>>(states.size());
1518         for (RegionState state: states) {
1519           HRegionInfo region = state.getRegion();
1520           String encodedRegionName = region.getEncodedName();
1521           Integer nodeVersion = offlineNodesVersions.get(encodedRegionName);
1522           if (nodeVersion == null || nodeVersion == -1) {
1523             LOG.warn("failed to offline in zookeeper: " + region);
1524             failedToOpenRegions.add(region); // assign individually later
1525             Lock lock = locks.remove(encodedRegionName);
1526             lock.unlock();
1527           } else {
1528             regionStates.updateRegionState(
1529               region, State.PENDING_OPEN, destination);
1530             List<ServerName> favoredNodes = ServerName.EMPTY_SERVER_LIST;
1531             if (this.shouldAssignRegionsWithFavoredNodes) {
1532               favoredNodes = ((FavoredNodeLoadBalancer)this.balancer).getFavoredNodes(region);
1533             }
1534             regionOpenInfos.add(new Triple<HRegionInfo, Integer,  List<ServerName>>(
1535               region, nodeVersion, favoredNodes));
1536           }
1537         }
1538 
1539         // Move on to open regions.
1540         try {
1541           // Send OPEN RPC. If it fails on a IOE or RemoteException,
1542           // regions will be assigned individually.
1543           long maxWaitTime = System.currentTimeMillis() +
1544             this.server.getConfiguration().
1545               getLong("hbase.regionserver.rpc.startup.waittime", 60000);
1546           for (int i = 1; i <= maximumAttempts && !server.isStopped(); i++) {
1547             try {
1548               List<RegionOpeningState> regionOpeningStateList = serverManager
1549                 .sendRegionOpen(destination, regionOpenInfos);
1550               if (regionOpeningStateList == null) {
1551                 // Failed getting RPC connection to this server
1552                 return false;
1553               }
1554               for (int k = 0, n = regionOpeningStateList.size(); k < n; k++) {
1555                 RegionOpeningState openingState = regionOpeningStateList.get(k);
1556                 if (openingState != RegionOpeningState.OPENED) {
1557                   HRegionInfo region = regionOpenInfos.get(k).getFirst();
1558                   if (openingState == RegionOpeningState.ALREADY_OPENED) {
1559                     processAlreadyOpenedRegion(region, destination);
1560                   } else if (openingState == RegionOpeningState.FAILED_OPENING) {
1561                     // Failed opening this region, reassign it later
1562                     failedToOpenRegions.add(region);
1563                   } else {
1564                     LOG.warn("THIS SHOULD NOT HAPPEN: unknown opening state "
1565                       + openingState + " in assigning region " + region);
1566                   }
1567                 }
1568               }
1569               break;
1570             } catch (IOException e) {
1571               if (e instanceof RemoteException) {
1572                 e = ((RemoteException)e).unwrapRemoteException();
1573               }
1574               if (e instanceof RegionServerStoppedException) {
1575                 LOG.warn("The region server was shut down, ", e);
1576                 // No need to retry, the region server is a goner.
1577                 return false;
1578               } else if (e instanceof ServerNotRunningYetException) {
1579                 long now = System.currentTimeMillis();
1580                 if (now < maxWaitTime) {
1581                   LOG.debug("Server is not yet up; waiting up to " +
1582                     (maxWaitTime - now) + "ms", e);
1583                   Thread.sleep(100);
1584                   i--; // reset the try count
1585                   continue;
1586                 }
1587               } else if (e instanceof java.net.SocketTimeoutException
1588                   && this.serverManager.isServerOnline(destination)) {
1589                 // In case socket is timed out and the region server is still online,
1590                 // the openRegion RPC could have been accepted by the server and
1591                 // just the response didn't go through.  So we will retry to
1592                 // open the region on the same server.
1593                 if (LOG.isDebugEnabled()) {
1594                   LOG.debug("Bulk assigner openRegion() to " + destination
1595                     + " has timed out, but the regions might"
1596                     + " already be opened on it.", e);
1597                 }
1598                 continue;
1599               }
1600               throw e;
1601             }
1602           }
1603         } catch (IOException e) {
1604           // Can be a socket timeout, EOF, NoRouteToHost, etc
1605           LOG.info("Unable to communicate with " + destination
1606             + " in order to assign regions, ", e);
1607           return false;
1608         } catch (InterruptedException e) {
1609           throw new RuntimeException(e);
1610         }
1611       } finally {
1612         for (Lock lock : locks.values()) {
1613           lock.unlock();
1614         }
1615       }
1616 
1617       if (!failedToOpenRegions.isEmpty()) {
1618         for (HRegionInfo region : failedToOpenRegions) {
1619           if (!regionStates.isRegionOnline(region)) {
1620             invokeAssign(region);
1621           }
1622         }
1623       }
1624       LOG.debug("Bulk assigning done for " + destination);
1625       return true;
1626     } finally {
1627       metricsAssignmentManager.updateBulkAssignTime(EnvironmentEdgeManager.currentTimeMillis() - startTime);
1628     }
1629   }
1630 
1631   /**
1632    * Send CLOSE RPC if the server is online, otherwise, offline the region.
1633    *
1634    * The RPC will be sent only to the region sever found in the region state
1635    * if it is passed in, otherwise, to the src server specified. If region
1636    * state is not specified, we don't update region state at all, instead
1637    * we just send the RPC call. This is useful for some cleanup without
1638    * messing around the region states (see handleRegion, on region opened
1639    * on an unexpected server scenario, for an example)
1640    */
1641   private void unassign(final HRegionInfo region,
1642       final RegionState state, final int versionOfClosingNode,
1643       final ServerName dest, final boolean transitionInZK,
1644       final ServerName src) {
1645     ServerName server = src;
1646     if (state != null) {
1647       server = state.getServerName();
1648     }
1649     long maxWaitTime = -1;
1650     for (int i = 1; i <= this.maximumAttempts; i++) {
1651       if (this.server.isStopped() || this.server.isAborted()) {
1652         LOG.debug("Server stopped/aborted; skipping unassign of " + region);
1653         return;
1654       }
1655       // ClosedRegionhandler can remove the server from this.regions
1656       if (!serverManager.isServerOnline(server)) {
1657         LOG.debug("Offline " + region.getRegionNameAsString()
1658           + ", no need to unassign since it's on a dead server: " + server);
1659         if (transitionInZK) {
1660           // delete the node. if no node exists need not bother.
1661           deleteClosingOrClosedNode(region, server);
1662         }
1663         if (state != null) {
1664           regionOffline(region);
1665         }
1666         return;
1667       }
1668       try {
1669         // Send CLOSE RPC
1670         if (serverManager.sendRegionClose(server, region,
1671           versionOfClosingNode, dest, transitionInZK)) {
1672           LOG.debug("Sent CLOSE to " + server + " for region " +
1673             region.getRegionNameAsString());
1674           if (!transitionInZK && state != null) {
1675             // Retry to make sure the region is
1676             // closed so as to avoid double assignment.
1677             unassign(region, state, versionOfClosingNode,
1678               dest, transitionInZK,src);
1679           }
1680           return;
1681         }
1682         // This never happens. Currently regionserver close always return true.
1683         // Todo; this can now happen (0.96) if there is an exception in a coprocessor
1684         LOG.warn("Server " + server + " region CLOSE RPC returned false for " +
1685           region.getRegionNameAsString());
1686       } catch (Throwable t) {
1687         if (t instanceof RemoteException) {
1688           t = ((RemoteException)t).unwrapRemoteException();
1689         }
1690         if (t instanceof NotServingRegionException
1691             || t instanceof RegionServerStoppedException) {
1692           LOG.debug("Offline " + region.getRegionNameAsString()
1693             + ", it's not any more on " + server, t);
1694           if (transitionInZK) {
1695             deleteClosingOrClosedNode(region, server);
1696           }
1697           if (state != null) {
1698             regionOffline(region);
1699           }
1700           return;
1701         } else if (state != null
1702             && t instanceof RegionAlreadyInTransitionException) {
1703           // RS is already processing this region, only need to update the timestamp
1704           LOG.debug("update " + state + " the timestamp.");
1705           state.updateTimestampToNow();
1706           if (maxWaitTime < 0) {
1707             maxWaitTime = EnvironmentEdgeManager.currentTimeMillis()
1708               + this.server.getConfiguration().getLong(ALREADY_IN_TRANSITION_WAITTIME,
1709                 DEFAULT_ALREADY_IN_TRANSITION_WAITTIME);
1710           }
1711           try {
1712             long now = EnvironmentEdgeManager.currentTimeMillis();
1713             if (now < maxWaitTime) {
1714               LOG.debug("Region is already in transition; "
1715                 + "waiting up to " + (maxWaitTime - now) + "ms", t);
1716               Thread.sleep(100);
1717               i--; // reset the try count
1718             }
1719           } catch (InterruptedException ie) {
1720             LOG.warn("Failed to unassign "
1721               + region.getRegionNameAsString() + " since interrupted", ie);
1722             Thread.currentThread().interrupt();
1723             if (!tomActivated) {
1724               regionStates.updateRegionState(region, State.FAILED_CLOSE);
1725             }
1726             return;
1727           }
1728         } else {
1729           LOG.info("Server " + server + " returned " + t + " for "
1730             + region.getRegionNameAsString() + ", try=" + i
1731             + " of " + this.maximumAttempts, t);
1732           // Presume retry or server will expire.
1733         }
1734       }
1735     }
1736     // Run out of attempts
1737     if (!tomActivated && state != null) {
1738       regionStates.updateRegionState(region, State.FAILED_CLOSE);
1739     }
1740   }
1741 
1742   /**
1743    * Set region to OFFLINE unless it is opening and forceNewPlan is false.
1744    */
1745   private RegionState forceRegionStateToOffline(
1746       final HRegionInfo region, final boolean forceNewPlan) {
1747     RegionState state = regionStates.getRegionState(region);
1748     if (state == null) {
1749       LOG.warn("Assigning a region not in region states: " + region);
1750       state = regionStates.createRegionState(region);
1751     }
1752 
1753     ServerName sn = state.getServerName();
1754     if (forceNewPlan && LOG.isDebugEnabled()) {
1755       LOG.debug("Force region state offline " + state);
1756     }
1757 
1758     switch (state.getState()) {
1759     case OPEN:
1760     case OPENING:
1761     case PENDING_OPEN:
1762     case CLOSING:
1763     case PENDING_CLOSE:
1764       if (!forceNewPlan) {
1765         LOG.debug("Skip assigning " +
1766           region + ", it is already " + state);
1767         return null;
1768       }
1769     case FAILED_CLOSE:
1770     case FAILED_OPEN:
1771       unassign(region, state, -1, null, false, null);
1772       state = regionStates.getRegionState(region);
1773       if (state.isFailedClose()) {
1774         // If we can't close the region, we can't re-assign
1775         // it so as to avoid possible double assignment/data loss.
1776         LOG.info("Skip assigning " +
1777           region + ", we couldn't close it: " + state);
1778         return null;
1779       }
1780     case OFFLINE:
1781       // This region could have been open on this server
1782       // for a while. If the server is dead and not processed
1783       // yet, we can move on only if the meta shows the
1784       // region is not on this server actually, or on a server
1785       // not dead, or dead and processed already.
1786       if (regionStates.isServerDeadAndNotProcessed(sn)
1787           && wasRegionOnDeadServerByMeta(region, sn)) {
1788         LOG.info("Skip assigning " + region.getRegionNameAsString()
1789           + ", it is on a dead but not processed yet server");
1790         return null;
1791       }
1792     case CLOSED:
1793       break;
1794     default:
1795       LOG.error("Trying to assign region " + region
1796         + ", which is " + state);
1797       return null;
1798     }
1799     return state;
1800   }
1801 
1802   private boolean wasRegionOnDeadServerByMeta(
1803       final HRegionInfo region, final ServerName sn) {
1804     try {
1805       if (region.isMetaRegion()) {
1806         ServerName server = catalogTracker.getMetaLocation();
1807         return regionStates.isServerDeadAndNotProcessed(server);
1808       }
1809       while (!server.isStopped()) {
1810         try {
1811           catalogTracker.waitForMeta();
1812           Pair<HRegionInfo, ServerName> r =
1813             MetaReader.getRegion(catalogTracker, region.getRegionName());
1814           ServerName server = r == null ? null : r.getSecond();
1815           return regionStates.isServerDeadAndNotProcessed(server);
1816         } catch (IOException ioe) {
1817           LOG.info("Received exception accessing hbase:meta during force assign "
1818             + region.getRegionNameAsString() + ", retrying", ioe);
1819         }
1820       }
1821     } catch (InterruptedException e) {
1822       Thread.currentThread().interrupt();
1823       LOG.info("Interrupted accessing hbase:meta", e);
1824     }
1825     // Call is interrupted or server is stopped.
1826     return regionStates.isServerDeadAndNotProcessed(sn);
1827   }
1828 
1829   /**
1830    * Caller must hold lock on the passed <code>state</code> object.
1831    * @param state
1832    * @param setOfflineInZK
1833    * @param forceNewPlan
1834    */
1835   private void assign(RegionState state,
1836       final boolean setOfflineInZK, final boolean forceNewPlan) {
1837     long startTime = EnvironmentEdgeManager.currentTimeMillis();
1838     try {
1839       RegionState currentState = state;
1840       int versionOfOfflineNode = -1;
1841       RegionPlan plan = null;
1842       long maxWaitTime = -1;
1843       HRegionInfo region = state.getRegion();
1844       RegionOpeningState regionOpenState;
1845       for (int i = 1; i <= maximumAttempts; i++) {
1846         if (server.isStopped() || server.isAborted()) {
1847           LOG.info("Skip assigning " + region.getRegionNameAsString()
1848             + ", the server is stopped/aborted");
1849           return;
1850         }
1851         if (plan == null) { // Get a server for the region at first
1852           try {
1853             plan = getRegionPlan(region, forceNewPlan);
1854           } catch (HBaseIOException e) {
1855             LOG.warn("Failed to get region plan", e);
1856           }
1857         }
1858         if (plan == null) {
1859           LOG.warn("Unable to determine a plan to assign " + region);
1860           if (tomActivated){
1861             this.timeoutMonitor.setAllRegionServersOffline(true);
1862           } else {
1863             if (region.isMetaRegion()) {
1864               try {
1865                 if (i != maximumAttempts) {
1866                   Thread.sleep(this.sleepTimeBeforeRetryingMetaAssignment);
1867                   continue;
1868                 }
1869                 // TODO : Ensure HBCK fixes this
1870                 LOG.error("Unable to determine a plan to assign hbase:meta even after repeated attempts. Run HBCK to fix this");
1871               } catch (InterruptedException e) {
1872                 LOG.error("Got exception while waiting for hbase:meta assignment");
1873                 Thread.currentThread().interrupt();
1874               }
1875             }
1876             regionStates.updateRegionState(region, State.FAILED_OPEN);
1877           }
1878           return;
1879         }
1880         if (setOfflineInZK && versionOfOfflineNode == -1) {
1881           // get the version of the znode after setting it to OFFLINE.
1882           // versionOfOfflineNode will be -1 if the znode was not set to OFFLINE
1883           versionOfOfflineNode = setOfflineInZooKeeper(currentState, plan.getDestination());
1884           if (versionOfOfflineNode != -1) {
1885             if (isDisabledorDisablingRegionInRIT(region)) {
1886               return;
1887             }
1888             // In case of assignment from EnableTableHandler table state is ENABLING. Any how
1889             // EnableTableHandler will set ENABLED after assigning all the table regions. If we
1890             // try to set to ENABLED directly then client API may think table is enabled.
1891             // When we have a case such as all the regions are added directly into hbase:meta and we call
1892             // assignRegion then we need to make the table ENABLED. Hence in such case the table
1893             // will not be in ENABLING or ENABLED state.
1894             TableName tableName = region.getTable();
1895             if (!zkTable.isEnablingTable(tableName) && !zkTable.isEnabledTable(tableName)) {
1896               LOG.debug("Setting table " + tableName + " to ENABLED state.");
1897               setEnabledTable(tableName);
1898             }
1899           }
1900         }
1901         if (setOfflineInZK && versionOfOfflineNode == -1) {
1902           LOG.info("Unable to set offline in ZooKeeper to assign " + region);
1903           // Setting offline in ZK must have been failed due to ZK racing or some
1904           // exception which may make the server to abort. If it is ZK racing,
1905           // we should retry since we already reset the region state,
1906           // existing (re)assignment will fail anyway.
1907           if (!server.isAborted()) {
1908             continue;
1909           }
1910         }
1911         LOG.info("Assigning " + region.getRegionNameAsString() +
1912             " to " + plan.getDestination().toString());
1913         // Transition RegionState to PENDING_OPEN
1914         currentState = regionStates.updateRegionState(region,
1915           State.PENDING_OPEN, plan.getDestination());
1916 
1917         boolean needNewPlan;
1918         final String assignMsg = "Failed assignment of " + region.getRegionNameAsString() +
1919             " to " + plan.getDestination();
1920         try {
1921           List<ServerName> favoredNodes = ServerName.EMPTY_SERVER_LIST;
1922           if (this.shouldAssignRegionsWithFavoredNodes) {
1923             favoredNodes = ((FavoredNodeLoadBalancer)this.balancer).getFavoredNodes(region);
1924           }
1925           regionOpenState = serverManager.sendRegionOpen(
1926               plan.getDestination(), region, versionOfOfflineNode, favoredNodes);
1927 
1928           if (regionOpenState == RegionOpeningState.FAILED_OPENING) {
1929             // Failed opening this region, looping again on a new server.
1930             needNewPlan = true;
1931             LOG.warn(assignMsg + ", regionserver says 'FAILED_OPENING', " +
1932                 " trying to assign elsewhere instead; " +
1933                 "try=" + i + " of " + this.maximumAttempts);
1934           } else {
1935             // we're done
1936             if (regionOpenState == RegionOpeningState.ALREADY_OPENED) {
1937               processAlreadyOpenedRegion(region, plan.getDestination());
1938             }
1939             return;
1940           }
1941 
1942         } catch (Throwable t) {
1943           if (t instanceof RemoteException) {
1944             t = ((RemoteException) t).unwrapRemoteException();
1945           }
1946 
1947           // Should we wait a little before retrying? If the server is starting it's yes.
1948           // If the region is already in transition, it's yes as well: we want to be sure that
1949           //  the region will get opened but we don't want a double assignment.
1950           boolean hold = (t instanceof RegionAlreadyInTransitionException ||
1951               t instanceof ServerNotRunningYetException);
1952 
1953           // In case socket is timed out and the region server is still online,
1954           // the openRegion RPC could have been accepted by the server and
1955           // just the response didn't go through.  So we will retry to
1956           // open the region on the same server to avoid possible
1957           // double assignment.
1958           boolean retry = !hold && (t instanceof java.net.SocketTimeoutException
1959               && this.serverManager.isServerOnline(plan.getDestination()));
1960 
1961 
1962           if (hold) {
1963             LOG.warn(assignMsg + ", waiting a little before trying on the same region server " +
1964               "try=" + i + " of " + this.maximumAttempts, t);
1965 
1966             if (maxWaitTime < 0) {
1967               if (t instanceof RegionAlreadyInTransitionException) {
1968                 maxWaitTime = EnvironmentEdgeManager.currentTimeMillis()
1969                   + this.server.getConfiguration().getLong(ALREADY_IN_TRANSITION_WAITTIME,
1970                     DEFAULT_ALREADY_IN_TRANSITION_WAITTIME);
1971               } else {
1972                 maxWaitTime = this.server.getConfiguration().
1973                   getLong("hbase.regionserver.rpc.startup.waittime", 60000);
1974               }
1975             }
1976             try {
1977               needNewPlan = false;
1978               long now = EnvironmentEdgeManager.currentTimeMillis();
1979               if (now < maxWaitTime) {
1980                 LOG.debug("Server is not yet up or region is already in transition; "
1981                   + "waiting up to " + (maxWaitTime - now) + "ms", t);
1982                 Thread.sleep(100);
1983                 i--; // reset the try count
1984               } else if (!(t instanceof RegionAlreadyInTransitionException)) {
1985                 LOG.debug("Server is not up for a while; try a new one", t);
1986                 needNewPlan = true;
1987               }
1988             } catch (InterruptedException ie) {
1989               LOG.warn("Failed to assign "
1990                   + region.getRegionNameAsString() + " since interrupted", ie);
1991               Thread.currentThread().interrupt();
1992               if (!tomActivated) {
1993                 regionStates.updateRegionState(region, State.FAILED_OPEN);
1994               }
1995               return;
1996             }
1997           } else if (retry) {
1998             needNewPlan = false;
1999             LOG.warn(assignMsg + ", trying to assign to the same region server " +
2000                 "try=" + i + " of " + this.maximumAttempts, t);
2001           } else {
2002             needNewPlan = true;
2003             LOG.warn(assignMsg + ", trying to assign elsewhere instead;" +
2004                 " try=" + i + " of " + this.maximumAttempts, t);
2005           }
2006         }
2007 
2008         if (i == this.maximumAttempts) {
2009           // Don't reset the region state or get a new plan any more.
2010           // This is the last try.
2011           continue;
2012         }
2013 
2014         // If region opened on destination of present plan, reassigning to new
2015         // RS may cause double assignments. In case of RegionAlreadyInTransitionException
2016         // reassigning to same RS.
2017         if (needNewPlan) {
2018           // Force a new plan and reassign. Will return null if no servers.
2019           // The new plan could be the same as the existing plan since we don't
2020           // exclude the server of the original plan, which should not be
2021           // excluded since it could be the only server up now.
2022           RegionPlan newPlan = null;
2023           try {
2024             newPlan = getRegionPlan(region, true);
2025           } catch (HBaseIOException e) {
2026             LOG.warn("Failed to get region plan", e);
2027           }
2028           if (newPlan == null) {
2029             if (tomActivated) {
2030               this.timeoutMonitor.setAllRegionServersOffline(true);
2031             } else {
2032               regionStates.updateRegionState(region, State.FAILED_OPEN);
2033             }
2034             LOG.warn("Unable to find a viable location to assign region " +
2035                 region.getRegionNameAsString());
2036             return;
2037           }
2038 
2039           if (plan != newPlan && !plan.getDestination().equals(newPlan.getDestination())) {
2040             // Clean out plan we failed execute and one that doesn't look like it'll
2041             // succeed anyways; we need a new plan!
2042             // Transition back to OFFLINE
2043             currentState = regionStates.updateRegionState(region, State.OFFLINE);
2044             versionOfOfflineNode = -1;
2045             plan = newPlan;
2046           }
2047         }
2048       }
2049       // Run out of attempts
2050       if (!tomActivated) {
2051         regionStates.updateRegionState(region, State.FAILED_OPEN);
2052       }
2053     } finally {
2054       metricsAssignmentManager.updateAssignmentTime(EnvironmentEdgeManager.currentTimeMillis() - startTime);
2055     }
2056   }
2057 
2058   private void processAlreadyOpenedRegion(HRegionInfo region, ServerName sn) {
2059     // Remove region from in-memory transition and unassigned node from ZK
2060     // While trying to enable the table the regions of the table were
2061     // already enabled.
2062     LOG.debug("ALREADY_OPENED " + region.getRegionNameAsString()
2063       + " to " + sn);
2064     String encodedName = region.getEncodedName();
2065     deleteNodeInStates(encodedName, "offline", sn, EventType.M_ZK_REGION_OFFLINE);
2066     regionStates.regionOnline(region, sn);
2067   }
2068 
2069   private boolean isDisabledorDisablingRegionInRIT(final HRegionInfo region) {
2070     TableName tableName = region.getTable();
2071     boolean disabled = this.zkTable.isDisabledTable(tableName);
2072     if (disabled || this.zkTable.isDisablingTable(tableName)) {
2073       LOG.info("Table " + tableName + (disabled ? " disabled;" : " disabling;") +
2074         " skipping assign of " + region.getRegionNameAsString());
2075       offlineDisabledRegion(region);
2076       return true;
2077     }
2078     return false;
2079   }
2080 
2081   /**
2082    * Set region as OFFLINED up in zookeeper
2083    *
2084    * @param state
2085    * @return the version of the offline node if setting of the OFFLINE node was
2086    *         successful, -1 otherwise.
2087    */
2088   private int setOfflineInZooKeeper(final RegionState state, final ServerName destination) {
2089     if (!state.isClosed() && !state.isOffline()) {
2090       String msg = "Unexpected state : " + state + " .. Cannot transit it to OFFLINE.";
2091       this.server.abort(msg, new IllegalStateException(msg));
2092       return -1;
2093     }
2094     regionStates.updateRegionState(state.getRegion(), State.OFFLINE);
2095     int versionOfOfflineNode;
2096     try {
2097       // get the version after setting the znode to OFFLINE
2098       versionOfOfflineNode = ZKAssign.createOrForceNodeOffline(watcher,
2099         state.getRegion(), destination);
2100       if (versionOfOfflineNode == -1) {
2101         LOG.warn("Attempted to create/force node into OFFLINE state before "
2102             + "completing assignment but failed to do so for " + state);
2103         return -1;
2104       }
2105     } catch (KeeperException e) {
2106       server.abort("Unexpected ZK exception creating/setting node OFFLINE", e);
2107       return -1;
2108     }
2109     return versionOfOfflineNode;
2110   }
2111 
2112   /**
2113    * @param region the region to assign
2114    * @return Plan for passed <code>region</code> (If none currently, it creates one or
2115    * if no servers to assign, it returns null).
2116    */
2117   private RegionPlan getRegionPlan(final HRegionInfo region,
2118       final boolean forceNewPlan)  throws HBaseIOException  {
2119     return getRegionPlan(region, null, forceNewPlan);
2120   }
2121 
2122   /**
2123    * @param region the region to assign
2124    * @param serverToExclude Server to exclude (we know its bad). Pass null if
2125    * all servers are thought to be assignable.
2126    * @param forceNewPlan If true, then if an existing plan exists, a new plan
2127    * will be generated.
2128    * @return Plan for passed <code>region</code> (If none currently, it creates one or
2129    * if no servers to assign, it returns null).
2130    */
2131   private RegionPlan getRegionPlan(final HRegionInfo region,
2132       final ServerName serverToExclude, final boolean forceNewPlan) throws HBaseIOException {
2133     // Pickup existing plan or make a new one
2134     final String encodedName = region.getEncodedName();
2135     final List<ServerName> destServers =
2136       serverManager.createDestinationServersList(serverToExclude);
2137 
2138     if (destServers.isEmpty()){
2139       LOG.warn("Can't move " + encodedName +
2140         ", there is no destination server available.");
2141       return null;
2142     }
2143 
2144     RegionPlan randomPlan = null;
2145     boolean newPlan = false;
2146     RegionPlan existingPlan;
2147 
2148     synchronized (this.regionPlans) {
2149       existingPlan = this.regionPlans.get(encodedName);
2150 
2151       if (existingPlan != null && existingPlan.getDestination() != null) {
2152         LOG.debug("Found an existing plan for " + region.getRegionNameAsString()
2153           + " destination server is " + existingPlan.getDestination() +
2154             " accepted as a dest server = " + destServers.contains(existingPlan.getDestination()));
2155       }
2156 
2157       if (forceNewPlan
2158           || existingPlan == null
2159           || existingPlan.getDestination() == null
2160           || !destServers.contains(existingPlan.getDestination())) {
2161         newPlan = true;
2162         randomPlan = new RegionPlan(region, null,
2163             balancer.randomAssignment(region, destServers));
2164         if (!region.isMetaTable() && shouldAssignRegionsWithFavoredNodes) {
2165           List<HRegionInfo> regions = new ArrayList<HRegionInfo>(1);
2166           regions.add(region);
2167           try {
2168             processFavoredNodes(regions);
2169           } catch (IOException ie) {
2170             LOG.warn("Ignoring exception in processFavoredNodes " + ie);
2171           }
2172         }
2173         this.regionPlans.put(encodedName, randomPlan);
2174       }
2175     }
2176 
2177     if (newPlan) {
2178       if (randomPlan.getDestination() == null) {
2179         LOG.warn("Can't find a destination for " + encodedName);
2180         return null;
2181       }
2182       LOG.debug("No previous transition plan found (or ignoring " +
2183         "an existing plan) for " + region.getRegionNameAsString() +
2184         "; generated random plan=" + randomPlan + "; " +
2185         serverManager.countOfRegionServers() +
2186                " (online=" + serverManager.getOnlineServers().size() +
2187                ", available=" + destServers.size() + ") available servers" +
2188                ", forceNewPlan=" + forceNewPlan);
2189         return randomPlan;
2190       }
2191     LOG.debug("Using pre-existing plan for " +
2192       region.getRegionNameAsString() + "; plan=" + existingPlan);
2193     return existingPlan;
2194   }
2195 
2196   /**
2197    * Unassigns the specified region.
2198    * <p>
2199    * Updates the RegionState and sends the CLOSE RPC unless region is being
2200    * split by regionserver; then the unassign fails (silently) because we
2201    * presume the region being unassigned no longer exists (its been split out
2202    * of existence). TODO: What to do if split fails and is rolled back and
2203    * parent is revivified?
2204    * <p>
2205    * If a RegionPlan is already set, it will remain.
2206    *
2207    * @param region server to be unassigned
2208    */
2209   public void unassign(HRegionInfo region) {
2210     unassign(region, false);
2211   }
2212 
2213 
2214   /**
2215    * Unassigns the specified region.
2216    * <p>
2217    * Updates the RegionState and sends the CLOSE RPC unless region is being
2218    * split by regionserver; then the unassign fails (silently) because we
2219    * presume the region being unassigned no longer exists (its been split out
2220    * of existence). TODO: What to do if split fails and is rolled back and
2221    * parent is revivified?
2222    * <p>
2223    * If a RegionPlan is already set, it will remain.
2224    *
2225    * @param region server to be unassigned
2226    * @param force if region should be closed even if already closing
2227    */
2228   public void unassign(HRegionInfo region, boolean force, ServerName dest) {
2229     // TODO: Method needs refactoring.  Ugly buried returns throughout.  Beware!
2230     LOG.debug("Starting unassign of " + region.getRegionNameAsString()
2231       + " (offlining), current state: " + regionStates.getRegionState(region));
2232 
2233     String encodedName = region.getEncodedName();
2234     // Grab the state of this region and synchronize on it
2235     int versionOfClosingNode = -1;
2236     // We need a lock here as we're going to do a put later and we don't want multiple states
2237     //  creation
2238     ReentrantLock lock = locker.acquireLock(encodedName);
2239     RegionState state = regionStates.getRegionTransitionState(encodedName);
2240     boolean reassign = true;
2241     try {
2242       if (state == null) {
2243         // Region is not in transition.
2244         // We can unassign it only if it's not SPLIT/MERGED.
2245         state = regionStates.getRegionState(encodedName);
2246         if (state != null && state.isUnassignable()) {
2247           LOG.info("Attempting to unassign " + state + ", ignored");
2248           // Offline region will be reassigned below
2249           return;
2250         }
2251         // Create the znode in CLOSING state
2252         try {
2253           if (state == null || state.getServerName() == null) {
2254             // We don't know where the region is, offline it.
2255             // No need to send CLOSE RPC
2256             LOG.warn("Attempting to unassign a region not in RegionStates"
2257               + region.getRegionNameAsString() + ", offlined");
2258             regionOffline(region);
2259             return;
2260           }
2261           versionOfClosingNode = ZKAssign.createNodeClosing(
2262             watcher, region, state.getServerName());
2263           if (versionOfClosingNode == -1) {
2264             LOG.info("Attempting to unassign " +
2265               region.getRegionNameAsString() + " but ZK closing node "
2266               + "can't be created.");
2267             reassign = false; // not unassigned at all
2268             return;
2269           }
2270         } catch (KeeperException e) {
2271           if (e instanceof NodeExistsException) {
2272             // Handle race between master initiated close and regionserver
2273             // orchestrated splitting. See if existing node is in a
2274             // SPLITTING or SPLIT state.  If so, the regionserver started
2275             // an op on node before we could get our CLOSING in.  Deal.
2276             NodeExistsException nee = (NodeExistsException)e;
2277             String path = nee.getPath();
2278             try {
2279               if (isSplitOrSplittingOrMergedOrMerging(path)) {
2280                 LOG.debug(path + " is SPLIT or SPLITTING or MERGED or MERGING; " +
2281                   "skipping unassign because region no longer exists -- its split or merge");
2282                 reassign = false; // no need to reassign for split/merged region
2283                 return;
2284               }
2285             } catch (KeeperException.NoNodeException ke) {
2286               LOG.warn("Failed getData on SPLITTING/SPLIT at " + path +
2287                 "; presuming split and that the region to unassign, " +
2288                 encodedName + ", no longer exists -- confirm", ke);
2289               return;
2290             } catch (KeeperException ke) {
2291               LOG.error("Unexpected zk state", ke);
2292             } catch (DeserializationException de) {
2293               LOG.error("Failed parse", de);
2294             }
2295           }
2296           // If we get here, don't understand whats going on -- abort.
2297           server.abort("Unexpected ZK exception creating node CLOSING", e);
2298           reassign = false; // heading out already
2299           return;
2300         }
2301         state = regionStates.updateRegionState(region, State.PENDING_CLOSE);
2302       } else if (state.isFailedOpen()) {
2303         // The region is not open yet
2304         regionOffline(region);
2305         return;
2306       } else if (force && state.isPendingCloseOrClosing()) {
2307         LOG.debug("Attempting to unassign " + region.getRegionNameAsString() +
2308           " which is already " + state.getState()  +
2309           " but forcing to send a CLOSE RPC again ");
2310         if (state.isFailedClose()) {
2311           state = regionStates.updateRegionState(region, State.PENDING_CLOSE);
2312         }
2313         state.updateTimestampToNow();
2314       } else {
2315         LOG.debug("Attempting to unassign " +
2316           region.getRegionNameAsString() + " but it is " +
2317           "already in transition (" + state.getState() + ", force=" + force + ")");
2318         return;
2319       }
2320 
2321       unassign(region, state, versionOfClosingNode, dest, true, null);
2322     } finally {
2323       lock.unlock();
2324 
2325       // Region is expected to be reassigned afterwards
2326       if (reassign && regionStates.isRegionOffline(region)) {
2327         assign(region, true);
2328       }
2329     }
2330   }
2331 
2332   public void unassign(HRegionInfo region, boolean force){
2333      unassign(region, force, null);
2334   }
2335 
2336   /**
2337    * @param region regioninfo of znode to be deleted.
2338    */
2339   public void deleteClosingOrClosedNode(HRegionInfo region, ServerName sn) {
2340     String encodedName = region.getEncodedName();
2341     deleteNodeInStates(encodedName, "closing", sn, EventType.M_ZK_REGION_CLOSING,
2342       EventType.RS_ZK_REGION_CLOSED);
2343   }
2344 
2345   /**
2346    * @param path
2347    * @return True if znode is in SPLIT or SPLITTING or MERGED or MERGING state.
2348    * @throws KeeperException Can happen if the znode went away in meantime.
2349    * @throws DeserializationException
2350    */
2351   private boolean isSplitOrSplittingOrMergedOrMerging(final String path)
2352       throws KeeperException, DeserializationException {
2353     boolean result = false;
2354     // This may fail if the SPLIT or SPLITTING or MERGED or MERGING znode gets
2355     // cleaned up before we can get data from it.
2356     byte [] data = ZKAssign.getData(watcher, path);
2357     if (data == null) {
2358       LOG.info("Node " + path + " is gone");
2359       return false;
2360     }
2361     RegionTransition rt = RegionTransition.parseFrom(data);
2362     switch (rt.getEventType()) {
2363     case RS_ZK_REQUEST_REGION_SPLIT:
2364     case RS_ZK_REGION_SPLIT:
2365     case RS_ZK_REGION_SPLITTING:
2366     case RS_ZK_REQUEST_REGION_MERGE:
2367     case RS_ZK_REGION_MERGED:
2368     case RS_ZK_REGION_MERGING:
2369       result = true;
2370       break;
2371     default:
2372       LOG.info("Node " + path + " is in " + rt.getEventType());
2373       break;
2374     }
2375     return result;
2376   }
2377 
2378   /**
2379    * Used by unit tests. Return the number of regions opened so far in the life
2380    * of the master. Increases by one every time the master opens a region
2381    * @return the counter value of the number of regions opened so far
2382    */
2383   public int getNumRegionsOpened() {
2384     return numRegionsOpened.get();
2385   }
2386 
2387   /**
2388    * Waits until the specified region has completed assignment.
2389    * <p>
2390    * If the region is already assigned, returns immediately.  Otherwise, method
2391    * blocks until the region is assigned.
2392    * @param regionInfo region to wait on assignment for
2393    * @throws InterruptedException
2394    */
2395   public boolean waitForAssignment(HRegionInfo regionInfo)
2396       throws InterruptedException {
2397     while (!regionStates.isRegionOnline(regionInfo)) {
2398       if (regionStates.isRegionInState(regionInfo, State.FAILED_OPEN)
2399           || this.server.isStopped()) {
2400         return false;
2401       }
2402 
2403       // We should receive a notification, but it's
2404       //  better to have a timeout to recheck the condition here:
2405       //  it lowers the impact of a race condition if any
2406       regionStates.waitForUpdate(100);
2407     }
2408     return true;
2409   }
2410 
2411   /**
2412    * Assigns the hbase:meta region.
2413    * <p>
2414    * Assumes that hbase:meta is currently closed and is not being actively served by
2415    * any RegionServer.
2416    * <p>
2417    * Forcibly unsets the current meta region location in ZooKeeper and assigns
2418    * hbase:meta to a random RegionServer.
2419    * @throws KeeperException
2420    */
2421   public void assignMeta() throws KeeperException {
2422     MetaRegionTracker.deleteMetaLocation(this.watcher);
2423     assign(HRegionInfo.FIRST_META_REGIONINFO, true);
2424   }
2425 
2426   /**
2427    * Assigns specified regions retaining assignments, if any.
2428    * <p>
2429    * This is a synchronous call and will return once every region has been
2430    * assigned.  If anything fails, an exception is thrown
2431    * @throws InterruptedException
2432    * @throws IOException
2433    */
2434   public void assign(Map<HRegionInfo, ServerName> regions)
2435         throws IOException, InterruptedException {
2436     if (regions == null || regions.isEmpty()) {
2437       return;
2438     }
2439     List<ServerName> servers = serverManager.createDestinationServersList();
2440     if (servers == null || servers.isEmpty()) {
2441       throw new IOException("Found no destination server to assign region(s)");
2442     }
2443 
2444     // Reuse existing assignment info
2445     Map<ServerName, List<HRegionInfo>> bulkPlan =
2446       balancer.retainAssignment(regions, servers);
2447 
2448     assign(regions.size(), servers.size(),
2449       "retainAssignment=true", bulkPlan);
2450   }
2451 
2452   /**
2453    * Assigns specified regions round robin, if any.
2454    * <p>
2455    * This is a synchronous call and will return once every region has been
2456    * assigned.  If anything fails, an exception is thrown
2457    * @throws InterruptedException
2458    * @throws IOException
2459    */
2460   public void assign(List<HRegionInfo> regions)
2461         throws IOException, InterruptedException {
2462     if (regions == null || regions.isEmpty()) {
2463       return;
2464     }
2465 
2466     List<ServerName> servers = serverManager.createDestinationServersList();
2467     if (servers == null || servers.isEmpty()) {
2468       throw new IOException("Found no destination server to assign region(s)");
2469     }
2470 
2471     // Generate a round-robin bulk assignment plan
2472     Map<ServerName, List<HRegionInfo>> bulkPlan
2473       = balancer.roundRobinAssignment(regions, servers);
2474     processFavoredNodes(regions);
2475 
2476     assign(regions.size(), servers.size(),
2477       "round-robin=true", bulkPlan);
2478   }
2479 
2480   private void assign(int regions, int totalServers,
2481       String message, Map<ServerName, List<HRegionInfo>> bulkPlan)
2482           throws InterruptedException, IOException {
2483 
2484     int servers = bulkPlan.size();
2485     if (servers == 1 || (regions < bulkAssignThresholdRegions
2486         && servers < bulkAssignThresholdServers)) {
2487 
2488       // Not use bulk assignment.  This could be more efficient in small
2489       // cluster, especially mini cluster for testing, so that tests won't time out
2490       if (LOG.isTraceEnabled()) {
2491         LOG.trace("Not using bulk assignment since we are assigning only " + regions +
2492           " region(s) to " + servers + " server(s)");
2493       }
2494       for (Map.Entry<ServerName, List<HRegionInfo>> plan: bulkPlan.entrySet()) {
2495         if (!assign(plan.getKey(), plan.getValue())) {
2496           for (HRegionInfo region: plan.getValue()) {
2497             if (!regionStates.isRegionOnline(region)) {
2498               invokeAssign(region);
2499             }
2500           }
2501         }
2502       }
2503     } else {
2504       LOG.info("Bulk assigning " + regions + " region(s) across "
2505         + totalServers + " server(s), " + message);
2506 
2507       // Use fixed count thread pool assigning.
2508       BulkAssigner ba = new GeneralBulkAssigner(
2509         this.server, bulkPlan, this, bulkAssignWaitTillAllAssigned);
2510       ba.bulkAssign();
2511       LOG.info("Bulk assigning done");
2512     }
2513   }
2514 
2515   /**
2516    * Assigns all user regions, if any exist.  Used during cluster startup.
2517    * <p>
2518    * This is a synchronous call and will return once every region has been
2519    * assigned.  If anything fails, an exception is thrown and the cluster
2520    * should be shutdown.
2521    * @throws InterruptedException
2522    * @throws IOException
2523    * @throws KeeperException
2524    */
2525   private void assignAllUserRegions()
2526       throws IOException, InterruptedException, KeeperException {
2527     // Cleanup any existing ZK nodes and start watching
2528     ZKAssign.deleteAllNodes(watcher);
2529     ZKUtil.listChildrenAndWatchForNewChildren(this.watcher,
2530       this.watcher.assignmentZNode);
2531     failoverCleanupDone();
2532 
2533     // Skip assignment for regions of tables in DISABLING state because during clean cluster startup
2534     // no RS is alive and regions map also doesn't have any information about the regions.
2535     // See HBASE-6281.
2536     Set<TableName> disabledOrDisablingOrEnabling = ZKTable.getDisabledOrDisablingTables(watcher);
2537     disabledOrDisablingOrEnabling.addAll(ZKTable.getEnablingTables(watcher));
2538     // Scan hbase:meta for all user regions, skipping any disabled tables
2539     Map<HRegionInfo, ServerName> allRegions;
2540     SnapshotOfRegionAssignmentFromMeta snapshotOfRegionAssignment =
2541        new SnapshotOfRegionAssignmentFromMeta(catalogTracker, disabledOrDisablingOrEnabling, true);
2542     snapshotOfRegionAssignment.initialize();
2543     allRegions = snapshotOfRegionAssignment.getRegionToRegionServerMap();
2544     if (allRegions == null || allRegions.isEmpty()) return;
2545 
2546     // Determine what type of assignment to do on startup
2547     boolean retainAssignment = server.getConfiguration().
2548       getBoolean("hbase.master.startup.retainassign", true);
2549 
2550     if (retainAssignment) {
2551       assign(allRegions);
2552     } else {
2553       List<HRegionInfo> regions = new ArrayList<HRegionInfo>(allRegions.keySet());
2554       assign(regions);
2555     }
2556 
2557     for (HRegionInfo hri : allRegions.keySet()) {
2558       TableName tableName = hri.getTable();
2559       if (!zkTable.isEnabledTable(tableName)) {
2560         setEnabledTable(tableName);
2561       }
2562     }
2563   }
2564 
2565   /**
2566    * Wait until no regions in transition.
2567    * @param timeout How long to wait.
2568    * @return True if nothing in regions in transition.
2569    * @throws InterruptedException
2570    */
2571   boolean waitUntilNoRegionsInTransition(final long timeout)
2572       throws InterruptedException {
2573     // Blocks until there are no regions in transition. It is possible that
2574     // there
2575     // are regions in transition immediately after this returns but guarantees
2576     // that if it returns without an exception that there was a period of time
2577     // with no regions in transition from the point-of-view of the in-memory
2578     // state of the Master.
2579     final long endTime = System.currentTimeMillis() + timeout;
2580 
2581     while (!this.server.isStopped() && regionStates.isRegionsInTransition()
2582         && endTime > System.currentTimeMillis()) {
2583       regionStates.waitForUpdate(100);
2584     }
2585 
2586     return !regionStates.isRegionsInTransition();
2587   }
2588 
2589   /**
2590    * Rebuild the list of user regions and assignment information.
2591    * <p>
2592    * Returns a map of servers that are not found to be online and the regions
2593    * they were hosting.
2594    * @return map of servers not online to their assigned regions, as stored
2595    *         in META
2596    * @throws IOException
2597    */
2598   Map<ServerName, List<HRegionInfo>> rebuildUserRegions() throws IOException, KeeperException {
2599     Set<TableName> enablingTables = ZKTable.getEnablingTables(watcher);
2600     Set<TableName> disabledOrEnablingTables = ZKTable.getDisabledTables(watcher);
2601     disabledOrEnablingTables.addAll(enablingTables);
2602     Set<TableName> disabledOrDisablingOrEnabling = ZKTable.getDisablingTables(watcher);
2603     disabledOrDisablingOrEnabling.addAll(disabledOrEnablingTables);
2604 
2605     // Region assignment from META
2606     List<Result> results = MetaReader.fullScan(this.catalogTracker);
2607     // Get any new but slow to checkin region server that joined the cluster
2608     Set<ServerName> onlineServers = serverManager.getOnlineServers().keySet();
2609     // Map of offline servers and their regions to be returned
2610     Map<ServerName, List<HRegionInfo>> offlineServers =
2611       new TreeMap<ServerName, List<HRegionInfo>>();
2612     // Iterate regions in META
2613     for (Result result : results) {
2614       Pair<HRegionInfo, ServerName> region = HRegionInfo.getHRegionInfoAndServerName(result);
2615       if (region == null) continue;
2616       HRegionInfo regionInfo = region.getFirst();
2617       ServerName regionLocation = region.getSecond();
2618       if (regionInfo == null) continue;
2619       regionStates.createRegionState(regionInfo);
2620       if (regionStates.isRegionInState(regionInfo, State.SPLIT)) {
2621         // Split is considered to be completed. If the split znode still
2622         // exists, the region will be put back to SPLITTING state later
2623         LOG.debug("Region " + regionInfo.getRegionNameAsString()
2624            + " split is completed. Hence need not add to regions list");
2625         continue;
2626       }
2627       TableName tableName = regionInfo.getTable();
2628       if (regionLocation == null) {
2629         // regionLocation could be null if createTable didn't finish properly.
2630         // When createTable is in progress, HMaster restarts.
2631         // Some regions have been added to hbase:meta, but have not been assigned.
2632         // When this happens, the region's table must be in ENABLING state.
2633         // It can't be in ENABLED state as that is set when all regions are
2634         // assigned.
2635         // It can't be in DISABLING state, because DISABLING state transitions
2636         // from ENABLED state when application calls disableTable.
2637         // It can't be in DISABLED state, because DISABLED states transitions
2638         // from DISABLING state.
2639         if (!enablingTables.contains(tableName)) {
2640           LOG.warn("Region " + regionInfo.getEncodedName() +
2641             " has null regionLocation." + " But its table " + tableName +
2642             " isn't in ENABLING state.");
2643         }
2644       } else if (!onlineServers.contains(regionLocation)) {
2645         // Region is located on a server that isn't online
2646         List<HRegionInfo> offlineRegions = offlineServers.get(regionLocation);
2647         if (offlineRegions == null) {
2648           offlineRegions = new ArrayList<HRegionInfo>(1);
2649           offlineServers.put(regionLocation, offlineRegions);
2650         }
2651         offlineRegions.add(regionInfo);
2652         // need to enable the table if not disabled or disabling or enabling
2653         // this will be used in rolling restarts
2654         if (!disabledOrDisablingOrEnabling.contains(tableName)
2655             && !getZKTable().isEnabledTable(tableName)) {
2656           setEnabledTable(tableName);
2657         }
2658       } else {
2659         // Region is being served and on an active server
2660         // add only if region not in disabled or enabling table
2661         if (!disabledOrEnablingTables.contains(tableName)) {
2662           regionStates.updateRegionState(regionInfo, State.OPEN, regionLocation);
2663           regionStates.regionOnline(regionInfo, regionLocation);
2664         }
2665         // need to enable the table if not disabled or disabling or enabling
2666         // this will be used in rolling restarts
2667         if (!disabledOrDisablingOrEnabling.contains(tableName)
2668             && !getZKTable().isEnabledTable(tableName)) {
2669           setEnabledTable(tableName);
2670         }
2671       }
2672     }
2673     return offlineServers;
2674   }
2675 
2676   /**
2677    * Recover the tables that were not fully moved to DISABLED state. These
2678    * tables are in DISABLING state when the master restarted/switched.
2679    *
2680    * @throws KeeperException
2681    * @throws TableNotFoundException
2682    * @throws IOException
2683    */
2684   private void recoverTableInDisablingState()
2685       throws KeeperException, TableNotFoundException, IOException {
2686     Set<TableName> disablingTables = ZKTable.getDisablingTables(watcher);
2687     if (disablingTables.size() != 0) {
2688       for (TableName tableName : disablingTables) {
2689         // Recover by calling DisableTableHandler
2690         LOG.info("The table " + tableName
2691             + " is in DISABLING state.  Hence recovering by moving the table"
2692             + " to DISABLED state.");
2693         new DisableTableHandler(this.server, tableName, catalogTracker,
2694             this, tableLockManager, true).prepare().process();
2695       }
2696     }
2697   }
2698 
2699   /**
2700    * Recover the tables that are not fully moved to ENABLED state. These tables
2701    * are in ENABLING state when the master restarted/switched
2702    *
2703    * @throws KeeperException
2704    * @throws org.apache.hadoop.hbase.TableNotFoundException
2705    * @throws IOException
2706    */
2707   private void recoverTableInEnablingState()
2708       throws KeeperException, TableNotFoundException, IOException {
2709     Set<TableName> enablingTables = ZKTable.getEnablingTables(watcher);
2710     if (enablingTables.size() != 0) {
2711       for (TableName tableName : enablingTables) {
2712         // Recover by calling EnableTableHandler
2713         LOG.info("The table " + tableName
2714             + " is in ENABLING state.  Hence recovering by moving the table"
2715             + " to ENABLED state.");
2716         // enableTable in sync way during master startup,
2717         // no need to invoke coprocessor
2718         new EnableTableHandler(this.server, tableName,
2719             catalogTracker, this, tableLockManager, true).prepare().process();
2720       }
2721     }
2722   }
2723 
2724   /**
2725    * Processes list of dead servers from result of hbase:meta scan and regions in RIT
2726    * <p>
2727    * This is used for failover to recover the lost regions that belonged to
2728    * RegionServers which failed while there was no active master or regions
2729    * that were in RIT.
2730    * <p>
2731    *
2732    *
2733    * @param deadServers
2734    *          The list of dead servers which failed while there was no active
2735    *          master. Can be null.
2736    * @throws IOException
2737    * @throws KeeperException
2738    */
2739   private void processDeadServersAndRecoverLostRegions(
2740       Map<ServerName, List<HRegionInfo>> deadServers)
2741           throws IOException, KeeperException {
2742     if (deadServers != null) {
2743       for (Map.Entry<ServerName, List<HRegionInfo>> server: deadServers.entrySet()) {
2744         ServerName serverName = server.getKey();
2745         // We need to keep such info even if the server is known dead
2746         regionStates.setLastRegionServerOfRegions(serverName, server.getValue());
2747         if (!serverManager.isServerDead(serverName)) {
2748           serverManager.expireServer(serverName); // Let SSH do region re-assign
2749         }
2750       }
2751     }
2752     List<String> nodes = ZKUtil.listChildrenAndWatchForNewChildren(
2753       this.watcher, this.watcher.assignmentZNode);
2754     if (!nodes.isEmpty()) {
2755       for (String encodedRegionName : nodes) {
2756         processRegionInTransition(encodedRegionName, null);
2757       }
2758     }
2759 
2760     // Now we can safely claim failover cleanup completed and enable
2761     // ServerShutdownHandler for further processing. The nodes (below)
2762     // in transition, if any, are for regions not related to those
2763     // dead servers at all, and can be done in parallel to SSH.
2764     failoverCleanupDone();
2765   }
2766 
2767   /**
2768    * Set Regions in transitions metrics.
2769    * This takes an iterator on the RegionInTransition map (CLSM), and is not synchronized.
2770    * This iterator is not fail fast, which may lead to stale read; but that's better than
2771    * creating a copy of the map for metrics computation, as this method will be invoked
2772    * on a frequent interval.
2773    */
2774   public void updateRegionsInTransitionMetrics() {
2775     long currentTime = System.currentTimeMillis();
2776     int totalRITs = 0;
2777     int totalRITsOverThreshold = 0;
2778     long oldestRITTime = 0;
2779     int ritThreshold = this.server.getConfiguration().
2780       getInt(HConstants.METRICS_RIT_STUCK_WARNING_THRESHOLD, 60000);
2781     for (RegionState state: regionStates.getRegionsInTransition().values()) {
2782       totalRITs++;
2783       long ritTime = currentTime - state.getStamp();
2784       if (ritTime > ritThreshold) { // more than the threshold
2785         totalRITsOverThreshold++;
2786       }
2787       if (oldestRITTime < ritTime) {
2788         oldestRITTime = ritTime;
2789       }
2790     }
2791     if (this.metricsAssignmentManager != null) {
2792       this.metricsAssignmentManager.updateRITOldestAge(oldestRITTime);
2793       this.metricsAssignmentManager.updateRITCount(totalRITs);
2794       this.metricsAssignmentManager.updateRITCountOverThreshold(totalRITsOverThreshold);
2795     }
2796   }
2797 
2798   /**
2799    * @param region Region whose plan we are to clear.
2800    */
2801   void clearRegionPlan(final HRegionInfo region) {
2802     synchronized (this.regionPlans) {
2803       this.regionPlans.remove(region.getEncodedName());
2804     }
2805   }
2806 
2807   /**
2808    * Wait on region to clear regions-in-transition.
2809    * @param hri Region to wait on.
2810    * @throws IOException
2811    */
2812   public void waitOnRegionToClearRegionsInTransition(final HRegionInfo hri)
2813       throws IOException, InterruptedException {
2814     waitOnRegionToClearRegionsInTransition(hri, -1L);
2815   }
2816 
2817   /**
2818    * Wait on region to clear regions-in-transition or time out
2819    * @param hri
2820    * @param timeOut Milliseconds to wait for current region to be out of transition state.
2821    * @return True when a region clears regions-in-transition before timeout otherwise false
2822    * @throws InterruptedException
2823    */
2824   public boolean waitOnRegionToClearRegionsInTransition(final HRegionInfo hri, long timeOut)
2825       throws InterruptedException {
2826     if (!regionStates.isRegionInTransition(hri)) return true;
2827     long end = (timeOut <= 0) ? Long.MAX_VALUE : EnvironmentEdgeManager.currentTimeMillis()
2828         + timeOut;
2829     // There is already a timeout monitor on regions in transition so I
2830     // should not have to have one here too?
2831     LOG.info("Waiting for " + hri.getEncodedName() +
2832         " to leave regions-in-transition, timeOut=" + timeOut + " ms.");
2833     while (!this.server.isStopped() && regionStates.isRegionInTransition(hri)) {
2834       regionStates.waitForUpdate(100);
2835       if (EnvironmentEdgeManager.currentTimeMillis() > end) {
2836         LOG.info("Timed out on waiting for " + hri.getEncodedName() + " to be assigned.");
2837         return false;
2838       }
2839     }
2840     if (this.server.isStopped()) {
2841       LOG.info("Giving up wait on regions in transition because stoppable.isStopped is set");
2842       return false;
2843     }
2844     return true;
2845   }
2846 
2847   /**
2848    * Update timers for all regions in transition going against the server in the
2849    * serversInUpdatingTimer.
2850    */
2851   public class TimerUpdater extends Chore {
2852 
2853     public TimerUpdater(final int period, final Stoppable stopper) {
2854       super("AssignmentTimerUpdater", period, stopper);
2855     }
2856 
2857     @Override
2858     protected void chore() {
2859       Preconditions.checkState(tomActivated);
2860       ServerName serverToUpdateTimer = null;
2861       while (!serversInUpdatingTimer.isEmpty() && !stopper.isStopped()) {
2862         if (serverToUpdateTimer == null) {
2863           serverToUpdateTimer = serversInUpdatingTimer.first();
2864         } else {
2865           serverToUpdateTimer = serversInUpdatingTimer
2866               .higher(serverToUpdateTimer);
2867         }
2868         if (serverToUpdateTimer == null) {
2869           break;
2870         }
2871         updateTimers(serverToUpdateTimer);
2872         serversInUpdatingTimer.remove(serverToUpdateTimer);
2873       }
2874     }
2875   }
2876 
2877   /**
2878    * Monitor to check for time outs on region transition operations
2879    */
2880   public class TimeoutMonitor extends Chore {
2881     private boolean allRegionServersOffline = false;
2882     private ServerManager serverManager;
2883     private final int timeout;
2884 
2885     /**
2886      * Creates a periodic monitor to check for time outs on region transition
2887      * operations.  This will deal with retries if for some reason something
2888      * doesn't happen within the specified timeout.
2889      * @param period
2890    * @param stopper When {@link Stoppable#isStopped()} is true, this thread will
2891    * cleanup and exit cleanly.
2892      * @param timeout
2893      */
2894     public TimeoutMonitor(final int period, final Stoppable stopper,
2895         ServerManager serverManager,
2896         final int timeout) {
2897       super("AssignmentTimeoutMonitor", period, stopper);
2898       this.timeout = timeout;
2899       this.serverManager = serverManager;
2900     }
2901 
2902     private synchronized void setAllRegionServersOffline(
2903       boolean allRegionServersOffline) {
2904       this.allRegionServersOffline = allRegionServersOffline;
2905     }
2906 
2907     @Override
2908     protected void chore() {
2909       Preconditions.checkState(tomActivated);
2910       boolean noRSAvailable = this.serverManager.createDestinationServersList().isEmpty();
2911 
2912       // Iterate all regions in transition checking for time outs
2913       long now = System.currentTimeMillis();
2914       // no lock concurrent access ok: we will be working on a copy, and it's java-valid to do
2915       //  a copy while another thread is adding/removing items
2916       for (String regionName : regionStates.getRegionsInTransition().keySet()) {
2917         RegionState regionState = regionStates.getRegionTransitionState(regionName);
2918         if (regionState == null) continue;
2919 
2920         if (regionState.getStamp() + timeout <= now) {
2921           // decide on action upon timeout
2922           actOnTimeOut(regionState);
2923         } else if (this.allRegionServersOffline && !noRSAvailable) {
2924           RegionPlan existingPlan = regionPlans.get(regionName);
2925           if (existingPlan == null
2926               || !this.serverManager.isServerOnline(existingPlan
2927                   .getDestination())) {
2928             // if some RSs just came back online, we can start the assignment
2929             // right away
2930             actOnTimeOut(regionState);
2931           }
2932         }
2933       }
2934       setAllRegionServersOffline(noRSAvailable);
2935     }
2936 
2937     private void actOnTimeOut(RegionState regionState) {
2938       HRegionInfo regionInfo = regionState.getRegion();
2939       LOG.info("Regions in transition timed out:  " + regionState);
2940       // Expired! Do a retry.
2941       switch (regionState.getState()) {
2942       case CLOSED:
2943         LOG.info("Region " + regionInfo.getEncodedName()
2944             + " has been CLOSED for too long, waiting on queued "
2945             + "ClosedRegionHandler to run or server shutdown");
2946         // Update our timestamp.
2947         regionState.updateTimestampToNow();
2948         break;
2949       case OFFLINE:
2950         LOG.info("Region has been OFFLINE for too long, " + "reassigning "
2951             + regionInfo.getRegionNameAsString() + " to a random server");
2952         invokeAssign(regionInfo);
2953         break;
2954       case PENDING_OPEN:
2955         LOG.info("Region has been PENDING_OPEN for too "
2956             + "long, reassigning region=" + regionInfo.getRegionNameAsString());
2957         invokeAssign(regionInfo);
2958         break;
2959       case OPENING:
2960         processOpeningState(regionInfo);
2961         break;
2962       case OPEN:
2963         LOG.error("Region has been OPEN for too long, " +
2964             "we don't know where region was opened so can't do anything");
2965         regionState.updateTimestampToNow();
2966         break;
2967 
2968       case PENDING_CLOSE:
2969         LOG.info("Region has been PENDING_CLOSE for too "
2970             + "long, running forced unassign again on region="
2971             + regionInfo.getRegionNameAsString());
2972         invokeUnassign(regionInfo);
2973         break;
2974       case CLOSING:
2975         LOG.info("Region has been CLOSING for too " +
2976           "long, this should eventually complete or the server will " +
2977           "expire, send RPC again");
2978         invokeUnassign(regionInfo);
2979         break;
2980 
2981       case SPLIT:
2982       case SPLITTING:
2983       case FAILED_OPEN:
2984       case FAILED_CLOSE:
2985       case MERGING:
2986         break;
2987 
2988       default:
2989         throw new IllegalStateException("Received event is not valid.");
2990       }
2991     }
2992   }
2993 
2994   private void processOpeningState(HRegionInfo regionInfo) {
2995     LOG.info("Region has been OPENING for too long, reassigning region="
2996         + regionInfo.getRegionNameAsString());
2997     // Should have a ZK node in OPENING state
2998     try {
2999       String node = ZKAssign.getNodeName(watcher, regionInfo.getEncodedName());
3000       Stat stat = new Stat();
3001       byte [] data = ZKAssign.getDataNoWatch(watcher, node, stat);
3002       if (data == null) {
3003         LOG.warn("Data is null, node " + node + " no longer exists");
3004         return;
3005       }
3006       RegionTransition rt = RegionTransition.parseFrom(data);
3007       EventType et = rt.getEventType();
3008       if (et == EventType.RS_ZK_REGION_OPENED) {
3009         LOG.debug("Region has transitioned to OPENED, allowing "
3010             + "watched event handlers to process");
3011         return;
3012       } else if (et != EventType.RS_ZK_REGION_OPENING && et != EventType.RS_ZK_REGION_FAILED_OPEN ) {
3013         LOG.warn("While timing out a region, found ZK node in unexpected state: " + et);
3014         return;
3015       }
3016       invokeAssign(regionInfo);
3017     } catch (KeeperException ke) {
3018       LOG.error("Unexpected ZK exception timing out CLOSING region", ke);
3019     } catch (DeserializationException e) {
3020       LOG.error("Unexpected exception parsing CLOSING region", e);
3021     }
3022   }
3023 
3024   void invokeAssign(HRegionInfo regionInfo) {
3025     threadPoolExecutorService.submit(new AssignCallable(this, regionInfo));
3026   }
3027 
3028   private void invokeUnassign(HRegionInfo regionInfo) {
3029     threadPoolExecutorService.submit(new UnAssignCallable(this, regionInfo));
3030   }
3031 
3032   public boolean isCarryingMeta(ServerName serverName) {
3033     return isCarryingRegion(serverName, HRegionInfo.FIRST_META_REGIONINFO);
3034   }
3035 
3036   /**
3037    * Check if the shutdown server carries the specific region.
3038    * We have a bunch of places that store region location
3039    * Those values aren't consistent. There is a delay of notification.
3040    * The location from zookeeper unassigned node has the most recent data;
3041    * but the node could be deleted after the region is opened by AM.
3042    * The AM's info could be old when OpenedRegionHandler
3043    * processing hasn't finished yet when server shutdown occurs.
3044    * @return whether the serverName currently hosts the region
3045    */
3046   private boolean isCarryingRegion(ServerName serverName, HRegionInfo hri) {
3047     RegionTransition rt = null;
3048     try {
3049       byte [] data = ZKAssign.getData(watcher, hri.getEncodedName());
3050       // This call can legitimately come by null
3051       rt = data == null? null: RegionTransition.parseFrom(data);
3052     } catch (KeeperException e) {
3053       server.abort("Exception reading unassigned node for region=" + hri.getEncodedName(), e);
3054     } catch (DeserializationException e) {
3055       server.abort("Exception parsing unassigned node for region=" + hri.getEncodedName(), e);
3056     }
3057 
3058     ServerName addressFromZK = rt != null? rt.getServerName():  null;
3059     if (addressFromZK != null) {
3060       // if we get something from ZK, we will use the data
3061       boolean matchZK = addressFromZK.equals(serverName);
3062       LOG.debug("Checking region=" + hri.getRegionNameAsString() + ", zk server=" + addressFromZK +
3063         " current=" + serverName + ", matches=" + matchZK);
3064       return matchZK;
3065     }
3066 
3067     ServerName addressFromAM = regionStates.getRegionServerOfRegion(hri);
3068     boolean matchAM = (addressFromAM != null &&
3069       addressFromAM.equals(serverName));
3070     LOG.debug("based on AM, current region=" + hri.getRegionNameAsString() +
3071       " is on server=" + (addressFromAM != null ? addressFromAM : "null") +
3072       " server being checked: " + serverName);
3073 
3074     return matchAM;
3075   }
3076 
3077   /**
3078    * Process shutdown server removing any assignments.
3079    * @param sn Server that went down.
3080    * @return list of regions in transition on this server
3081    */
3082   public List<HRegionInfo> processServerShutdown(final ServerName sn) {
3083     // Clean out any existing assignment plans for this server
3084     synchronized (this.regionPlans) {
3085       for (Iterator <Map.Entry<String, RegionPlan>> i =
3086           this.regionPlans.entrySet().iterator(); i.hasNext();) {
3087         Map.Entry<String, RegionPlan> e = i.next();
3088         ServerName otherSn = e.getValue().getDestination();
3089         // The name will be null if the region is planned for a random assign.
3090         if (otherSn != null && otherSn.equals(sn)) {
3091           // Use iterator's remove else we'll get CME
3092           i.remove();
3093         }
3094       }
3095     }
3096     List<HRegionInfo> regions = regionStates.serverOffline(watcher, sn);
3097     for (Iterator<HRegionInfo> it = regions.iterator(); it.hasNext(); ) {
3098       HRegionInfo hri = it.next();
3099       String encodedName = hri.getEncodedName();
3100 
3101       // We need a lock on the region as we could update it
3102       Lock lock = locker.acquireLock(encodedName);
3103       try {
3104         RegionState regionState =
3105           regionStates.getRegionTransitionState(encodedName);
3106         if (regionState == null
3107             || (regionState.getServerName() != null && !regionState.isOnServer(sn))
3108             || !(regionState.isFailedClose() || regionState.isOffline()
3109               || regionState.isPendingOpenOrOpening())) {
3110           LOG.info("Skip " + regionState + " since it is not opening/failed_close"
3111             + " on the dead server any more: " + sn);
3112           it.remove();
3113         } else {
3114           try {
3115             // Delete the ZNode if exists
3116             ZKAssign.deleteNodeFailSilent(watcher, hri);
3117           } catch (KeeperException ke) {
3118             server.abort("Unexpected ZK exception deleting node " + hri, ke);
3119           }
3120           if (zkTable.isDisablingOrDisabledTable(hri.getTable())) {
3121             regionStates.regionOffline(hri);
3122             it.remove();
3123             continue;
3124           }
3125           // Mark the region offline and assign it again by SSH
3126           regionStates.updateRegionState(hri, State.OFFLINE);
3127         }
3128       } finally {
3129         lock.unlock();
3130       }
3131     }
3132     return regions;
3133   }
3134 
3135   /**
3136    * @param plan Plan to execute.
3137    */
3138   public void balance(final RegionPlan plan) {
3139     synchronized (this.regionPlans) {
3140       this.regionPlans.put(plan.getRegionName(), plan);
3141     }
3142     HRegionInfo hri = plan.getRegionInfo();
3143     TableName tableName = hri.getTable();
3144     if (zkTable.isDisablingOrDisabledTable(tableName)) {
3145       LOG.info("Ignored moving region of disabling/disabled table "
3146         + tableName);
3147       return;
3148     }
3149     unassign(hri, false, plan.getDestination());
3150   }
3151 
3152   public void stop() {
3153     if (tomActivated){
3154       this.timeoutMonitor.interrupt();
3155       this.timerUpdater.interrupt();
3156     }
3157   }
3158 
3159   /**
3160    * Shutdown the threadpool executor service
3161    */
3162   public void shutdown() {
3163     // It's an immediate shutdown, so we're clearing the remaining tasks.
3164     synchronized (zkEventWorkerWaitingList){
3165       zkEventWorkerWaitingList.clear();
3166     }
3167     threadPoolExecutorService.shutdownNow();
3168     zkEventWorkers.shutdownNow();
3169   }
3170 
3171   protected void setEnabledTable(TableName tableName) {
3172     try {
3173       this.zkTable.setEnabledTable(tableName);
3174     } catch (KeeperException e) {
3175       // here we can abort as it is the start up flow
3176       String errorMsg = "Unable to ensure that the table " + tableName
3177           + " will be" + " enabled because of a ZooKeeper issue";
3178       LOG.error(errorMsg);
3179       this.server.abort(errorMsg, e);
3180     }
3181   }
3182 
3183   /**
3184    * Set region as OFFLINED up in zookeeper asynchronously.
3185    * @param state
3186    * @return True if we succeeded, false otherwise (State was incorrect or failed
3187    * updating zk).
3188    */
3189   private boolean asyncSetOfflineInZooKeeper(final RegionState state,
3190       final AsyncCallback.StringCallback cb, final ServerName destination) {
3191     if (!state.isClosed() && !state.isOffline()) {
3192       this.server.abort("Unexpected state trying to OFFLINE; " + state,
3193         new IllegalStateException());
3194       return false;
3195     }
3196     regionStates.updateRegionState(state.getRegion(), State.OFFLINE);
3197     try {
3198       ZKAssign.asyncCreateNodeOffline(watcher, state.getRegion(),
3199         destination, cb, state);
3200     } catch (KeeperException e) {
3201       if (e instanceof NodeExistsException) {
3202         LOG.warn("Node for " + state.getRegion() + " already exists");
3203       } else {
3204         server.abort("Unexpected ZK exception creating/setting node OFFLINE", e);
3205       }
3206       return false;
3207     }
3208     return true;
3209   }
3210 
3211   private boolean deleteNodeInStates(String encodedName,
3212       String desc, ServerName sn, EventType... types) {
3213     try {
3214       for (EventType et: types) {
3215         if (ZKAssign.deleteNode(watcher, encodedName, et, sn)) {
3216           return true;
3217         }
3218       }
3219       LOG.info("Failed to delete the " + desc + " node for "
3220         + encodedName + ". The node type may not match");
3221     } catch (NoNodeException e) {
3222       if (LOG.isDebugEnabled()) {
3223         LOG.debug("The " + desc + " node for " + encodedName + " already deleted");
3224       }
3225     } catch (KeeperException ke) {
3226       server.abort("Unexpected ZK exception deleting " + desc
3227         + " node for the region " + encodedName, ke);
3228     }
3229     return false;
3230   }
3231 
3232   private void deleteMergingNode(String encodedName, ServerName sn) {
3233     deleteNodeInStates(encodedName, "merging", sn, EventType.RS_ZK_REGION_MERGING,
3234       EventType.RS_ZK_REQUEST_REGION_MERGE, EventType.RS_ZK_REGION_MERGED);
3235   }
3236 
3237   private void deleteSplittingNode(String encodedName, ServerName sn) {
3238     deleteNodeInStates(encodedName, "splitting", sn, EventType.RS_ZK_REGION_SPLITTING,
3239       EventType.RS_ZK_REQUEST_REGION_SPLIT, EventType.RS_ZK_REGION_SPLIT);
3240   }
3241 
3242   /**
3243    * A helper to handle region merging transition event.
3244    * It transitions merging regions to MERGING state.
3245    */
3246   private boolean handleRegionMerging(final RegionTransition rt, final String encodedName,
3247       final String prettyPrintedRegionName, final ServerName sn) {
3248     if (!serverManager.isServerOnline(sn)) {
3249       LOG.warn("Dropped merging! ServerName=" + sn + " unknown.");
3250       return false;
3251     }
3252     byte [] payloadOfMerging = rt.getPayload();
3253     List<HRegionInfo> mergingRegions;
3254     try {
3255       mergingRegions = HRegionInfo.parseDelimitedFrom(
3256         payloadOfMerging, 0, payloadOfMerging.length);
3257     } catch (IOException e) {
3258       LOG.error("Dropped merging! Failed reading "  + rt.getEventType()
3259         + " payload for " + prettyPrintedRegionName);
3260       return false;
3261     }
3262     assert mergingRegions.size() == 3;
3263     HRegionInfo p = mergingRegions.get(0);
3264     HRegionInfo hri_a = mergingRegions.get(1);
3265     HRegionInfo hri_b = mergingRegions.get(2);
3266 
3267     RegionState rs_p = regionStates.getRegionState(p);
3268     RegionState rs_a = regionStates.getRegionState(hri_a);
3269     RegionState rs_b = regionStates.getRegionState(hri_b);
3270 
3271     if (!((rs_a == null || rs_a.isOpenOrMergingOnServer(sn))
3272         && (rs_b == null || rs_b.isOpenOrMergingOnServer(sn))
3273         && (rs_p == null || rs_p.isOpenOrMergingNewOnServer(sn)))) {
3274       LOG.warn("Dropped merging! Not in state good for MERGING; rs_p="
3275         + rs_p + ", rs_a=" + rs_a + ", rs_b=" + rs_b);
3276       return false;
3277     }
3278 
3279     EventType et = rt.getEventType();
3280     if (et == EventType.RS_ZK_REQUEST_REGION_MERGE) {
3281       try {
3282         if (RegionMergeTransaction.transitionMergingNode(watcher, p,
3283             hri_a, hri_b, sn, -1, EventType.RS_ZK_REQUEST_REGION_MERGE,
3284             EventType.RS_ZK_REGION_MERGING) == -1) {
3285           byte[] data = ZKAssign.getData(watcher, encodedName);
3286           EventType currentType = null;
3287           if (data != null) {
3288             RegionTransition newRt = RegionTransition.parseFrom(data);
3289             currentType = newRt.getEventType();
3290           }
3291           if (currentType == null || (currentType != EventType.RS_ZK_REGION_MERGED
3292               && currentType != EventType.RS_ZK_REGION_MERGING)) {
3293             LOG.warn("Failed to transition pending_merge node "
3294               + encodedName + " to merging, it's now " + currentType);
3295             return false;
3296           }
3297         }
3298       } catch (Exception e) {
3299         LOG.warn("Failed to transition pending_merge node "
3300           + encodedName + " to merging", e);
3301         return false;
3302       }
3303     }
3304 
3305     synchronized (regionStates) {
3306       regionStates.updateRegionState(hri_a, State.MERGING);
3307       regionStates.updateRegionState(hri_b, State.MERGING);
3308       regionStates.updateRegionState(p, State.MERGING_NEW, sn);
3309 
3310       if (et != EventType.RS_ZK_REGION_MERGED) {
3311         regionStates.regionOffline(p, State.MERGING_NEW);
3312         this.mergingRegions.put(encodedName,
3313           new PairOfSameType<HRegionInfo>(hri_a, hri_b));
3314       } else {
3315         this.mergingRegions.remove(encodedName);
3316         regionOffline(hri_a, State.MERGED);
3317         regionOffline(hri_b, State.MERGED);
3318         regionOnline(p, sn);
3319       }
3320     }
3321 
3322     if (et == EventType.RS_ZK_REGION_MERGED) {
3323       LOG.debug("Handling MERGED event for " + encodedName + "; deleting node");
3324       // Remove region from ZK
3325       try {
3326         boolean successful = false;
3327         while (!successful) {
3328           // It's possible that the RS tickles in between the reading of the
3329           // znode and the deleting, so it's safe to retry.
3330           successful = ZKAssign.deleteNode(watcher, encodedName,
3331             EventType.RS_ZK_REGION_MERGED, sn);
3332         }
3333       } catch (KeeperException e) {
3334         if (e instanceof NoNodeException) {
3335           String znodePath = ZKUtil.joinZNode(watcher.splitLogZNode, encodedName);
3336           LOG.debug("The znode " + znodePath + " does not exist.  May be deleted already.");
3337         } else {
3338           server.abort("Error deleting MERGED node " + encodedName, e);
3339         }
3340       }
3341       LOG.info("Handled MERGED event; merged=" + p.getRegionNameAsString()
3342         + ", region_a=" + hri_a.getRegionNameAsString() + ", region_b="
3343         + hri_b.getRegionNameAsString() + ", on " + sn);
3344 
3345       // User could disable the table before master knows the new region.
3346       if (zkTable.isDisablingOrDisabledTable(p.getTable())) {
3347         unassign(p);
3348       }
3349     }
3350     return true;
3351   }
3352 
3353   /**
3354    * A helper to handle region splitting transition event.
3355    */
3356   private boolean handleRegionSplitting(final RegionTransition rt, final String encodedName,
3357       final String prettyPrintedRegionName, final ServerName sn) {
3358     if (!serverManager.isServerOnline(sn)) {
3359       LOG.warn("Dropped splitting! ServerName=" + sn + " unknown.");
3360       return false;
3361     }
3362     byte [] payloadOfSplitting = rt.getPayload();
3363     List<HRegionInfo> splittingRegions;
3364     try {
3365       splittingRegions = HRegionInfo.parseDelimitedFrom(
3366         payloadOfSplitting, 0, payloadOfSplitting.length);
3367     } catch (IOException e) {
3368       LOG.error("Dropped splitting! Failed reading " + rt.getEventType()
3369         + " payload for " + prettyPrintedRegionName);
3370       return false;
3371     }
3372     assert splittingRegions.size() == 2;
3373     HRegionInfo hri_a = splittingRegions.get(0);
3374     HRegionInfo hri_b = splittingRegions.get(1);
3375 
3376     RegionState rs_p = regionStates.getRegionState(encodedName);
3377     RegionState rs_a = regionStates.getRegionState(hri_a);
3378     RegionState rs_b = regionStates.getRegionState(hri_b);
3379 
3380     if (!((rs_p == null || rs_p.isOpenOrSplittingOnServer(sn))
3381         && (rs_a == null || rs_a.isOpenOrSplittingNewOnServer(sn))
3382         && (rs_b == null || rs_b.isOpenOrSplittingNewOnServer(sn)))) {
3383       LOG.warn("Dropped splitting! Not in state good for SPLITTING; rs_p="
3384         + rs_p + ", rs_a=" + rs_a + ", rs_b=" + rs_b);
3385       return false;
3386     }
3387 
3388     if (rs_p == null) {
3389       // Splitting region should be online
3390       rs_p = regionStates.updateRegionState(rt, State.OPEN);
3391       if (rs_p == null) {
3392         LOG.warn("Received splitting for region " + prettyPrintedRegionName
3393           + " from server " + sn + " but it doesn't exist anymore,"
3394           + " probably already processed its split");
3395         return false;
3396       }
3397       regionStates.regionOnline(rs_p.getRegion(), sn);
3398     }
3399 
3400     HRegionInfo p = rs_p.getRegion();
3401     EventType et = rt.getEventType();
3402     if (et == EventType.RS_ZK_REQUEST_REGION_SPLIT) {
3403       try {
3404         if (SplitTransaction.transitionSplittingNode(watcher, p,
3405             hri_a, hri_b, sn, -1, EventType.RS_ZK_REQUEST_REGION_SPLIT,
3406             EventType.RS_ZK_REGION_SPLITTING) == -1) {
3407           byte[] data = ZKAssign.getData(watcher, encodedName);
3408           EventType currentType = null;
3409           if (data != null) {
3410             RegionTransition newRt = RegionTransition.parseFrom(data);
3411             currentType = newRt.getEventType();
3412           }
3413           if (currentType == null || (currentType != EventType.RS_ZK_REGION_SPLIT
3414               && currentType != EventType.RS_ZK_REGION_SPLITTING)) {
3415             LOG.warn("Failed to transition pending_split node "
3416               + encodedName + " to splitting, it's now " + currentType);
3417             return false;
3418           }
3419         }
3420       } catch (Exception e) {
3421         LOG.warn("Failed to transition pending_split node "
3422           + encodedName + " to splitting", e);
3423         return false;
3424       }
3425     }
3426 
3427     synchronized (regionStates) {
3428       regionStates.updateRegionState(hri_a, State.SPLITTING_NEW, sn);
3429       regionStates.updateRegionState(hri_b, State.SPLITTING_NEW, sn);
3430       regionStates.regionOffline(hri_a, State.SPLITTING_NEW);
3431       regionStates.regionOffline(hri_b, State.SPLITTING_NEW);
3432       regionStates.updateRegionState(rt, State.SPLITTING);
3433 
3434       // The below is for testing ONLY!  We can't do fault injection easily, so
3435       // resort to this kinda uglyness -- St.Ack 02/25/2011.
3436       if (TEST_SKIP_SPLIT_HANDLING) {
3437         LOG.warn("Skipping split message, TEST_SKIP_SPLIT_HANDLING is set");
3438         return true; // return true so that the splitting node stays
3439       }
3440 
3441       if (et == EventType.RS_ZK_REGION_SPLIT) {
3442         regionOffline(p, State.SPLIT);
3443         regionOnline(hri_a, sn);
3444         regionOnline(hri_b, sn);
3445       }
3446     }
3447 
3448     if (et == EventType.RS_ZK_REGION_SPLIT) {
3449       LOG.debug("Handling SPLIT event for " + encodedName + "; deleting node");
3450       // Remove region from ZK
3451       try {
3452         boolean successful = false;
3453         while (!successful) {
3454           // It's possible that the RS tickles in between the reading of the
3455           // znode and the deleting, so it's safe to retry.
3456           successful = ZKAssign.deleteNode(watcher, encodedName,
3457             EventType.RS_ZK_REGION_SPLIT, sn);
3458         }
3459       } catch (KeeperException e) {
3460         if (e instanceof NoNodeException) {
3461           String znodePath = ZKUtil.joinZNode(watcher.splitLogZNode, encodedName);
3462           LOG.debug("The znode " + znodePath + " does not exist.  May be deleted already.");
3463         } else {
3464           server.abort("Error deleting SPLIT node " + encodedName, e);
3465         }
3466       }
3467       LOG.info("Handled SPLIT event; parent=" + p.getRegionNameAsString()
3468         + ", daughter a=" + hri_a.getRegionNameAsString() + ", daughter b="
3469         + hri_b.getRegionNameAsString() + ", on " + sn);
3470 
3471       // User could disable the table before master knows the new region.
3472       if (zkTable.isDisablingOrDisabledTable(p.getTable())) {
3473         unassign(hri_a);
3474         unassign(hri_b);
3475       }
3476     }
3477     return true;
3478   }
3479 
3480   /**
3481    * A region is offline.  The new state should be the specified one,
3482    * if not null.  If the specified state is null, the new state is Offline.
3483    * The specified state can be Split/Merged/Offline/null only.
3484    */
3485   private void regionOffline(final HRegionInfo regionInfo, final State state) {
3486     regionStates.regionOffline(regionInfo, state);
3487     removeClosedRegion(regionInfo);
3488     // remove the region plan as well just in case.
3489     clearRegionPlan(regionInfo);
3490   }
3491 }