View Javadoc

1   /**
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
17   */
18  package org.apache.hadoop.hbase.master;
19  
20  import java.io.IOException;
21  import java.util.ArrayList;
22  import java.util.Collection;
23  import java.util.Collections;
24  import java.util.HashMap;
25  import java.util.HashSet;
26  import java.util.Iterator;
27  import java.util.List;
28  import java.util.Map;
29  import java.util.Set;
30  import java.util.TreeMap;
31  
32  import org.apache.commons.logging.Log;
33  import org.apache.commons.logging.LogFactory;
34  import org.apache.hadoop.hbase.classification.InterfaceAudience;
35  import org.apache.hadoop.conf.Configuration;
36  import org.apache.hadoop.hbase.HConstants;
37  import org.apache.hadoop.hbase.HRegionInfo;
38  import org.apache.hadoop.hbase.RegionTransition;
39  import org.apache.hadoop.hbase.HTableDescriptor;
40  import org.apache.hadoop.hbase.MetaTableAccessor;
41  import org.apache.hadoop.hbase.Server;
42  import org.apache.hadoop.hbase.ServerLoad;
43  import org.apache.hadoop.hbase.ServerName;
44  import org.apache.hadoop.hbase.TableName;
45  import org.apache.hadoop.hbase.TableStateManager;
46  import org.apache.hadoop.hbase.client.RegionReplicaUtil;
47  import org.apache.hadoop.hbase.MetaTableAccessor;
48  import org.apache.hadoop.hbase.master.RegionState.State;
49  import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
50  import org.apache.hadoop.hbase.util.Bytes;
51  import org.apache.hadoop.hbase.util.FSUtils;
52  import org.apache.hadoop.hbase.util.Pair;
53  import org.apache.hadoop.hbase.zookeeper.ZKAssign;
54  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
55  import org.apache.zookeeper.KeeperException;
56  
57  import com.google.common.annotations.VisibleForTesting;
58  import com.google.common.base.Preconditions;
59  
60  /**
61   * Region state accountant. It holds the states of all regions in the memory.
62   * In normal scenario, it should match the meta table and the true region states.
63   *
64   * This map is used by AssignmentManager to track region states.
65   */
66  @InterfaceAudience.Private
67  public class RegionStates {
68    private static final Log LOG = LogFactory.getLog(RegionStates.class);
69  
70    /**
71     * Regions currently in transition.
72     */
73    final HashMap<String, RegionState> regionsInTransition =
74      new HashMap<String, RegionState>();
75  
76    /**
77     * Region encoded name to state map.
78     * All the regions should be in this map.
79     */
80    private final Map<String, RegionState> regionStates =
81      new HashMap<String, RegionState>();
82  
83    /**
84     * Holds mapping of table -> region state
85     */
86    private final Map<TableName, Map<String, RegionState>> regionStatesTableIndex =
87        new HashMap<TableName, Map<String, RegionState>>();
88  
89    /**
90     * Server to regions assignment map.
91     * Contains the set of regions currently assigned to a given server.
92     */
93    private final Map<ServerName, Set<HRegionInfo>> serverHoldings =
94      new HashMap<ServerName, Set<HRegionInfo>>();
95  
96    /**
97     * Maintains the mapping from the default region to the replica regions.
98     */
99    private final Map<HRegionInfo, Set<HRegionInfo>> defaultReplicaToOtherReplicas =
100     new HashMap<HRegionInfo, Set<HRegionInfo>>();
101 
102   /**
103    * Region to server assignment map.
104    * Contains the server a given region is currently assigned to.
105    */
106   private final TreeMap<HRegionInfo, ServerName> regionAssignments =
107     new TreeMap<HRegionInfo, ServerName>();
108 
109   /**
110    * Encoded region name to server assignment map for re-assignment
111    * purpose. Contains the server a given region is last known assigned
112    * to, which has not completed log splitting, so not assignable.
113    * If a region is currently assigned, this server info in this
114    * map should be the same as that in regionAssignments.
115    * However the info in regionAssignments is cleared when the region
116    * is offline while the info in lastAssignments is cleared when
117    * the region is closed or the server is dead and processed.
118    */
119   private final HashMap<String, ServerName> lastAssignments =
120     new HashMap<String, ServerName>();
121 
122   /**
123    * Encoded region name to server assignment map for the
124    * purpose to clean up serverHoldings when a region is online
125    * on a new server. When the region is offline from the previous
126    * server, we cleaned up regionAssignments so that it has the
127    * latest assignment map. But we didn't clean up serverHoldings
128    * to match the meta. We need this map to find out the old server
129    * whose serverHoldings needs cleanup, given a moved region.
130    */
131   private final HashMap<String, ServerName> oldAssignments =
132     new HashMap<String, ServerName>();
133 
134   /**
135    * Map a host port pair string to the latest start code
136    * of a region server which is known to be dead. It is dead
137    * to us, but server manager may not know it yet.
138    */
139   private final HashMap<String, Long> deadServers =
140     new HashMap<String, Long>();
141 
142   /**
143    * Map a dead servers to the time when log split is done.
144    * Since log splitting is not ordered, we have to remember
145    * all processed instances. The map is cleaned up based
146    * on a configured time. By default, we assume a dead
147    * server should be done with log splitting in two hours.
148    */
149   private final HashMap<ServerName, Long> processedServers =
150     new HashMap<ServerName, Long>();
151   private long lastProcessedServerCleanTime;
152 
153   private final TableStateManager tableStateManager;
154   private final RegionStateStore regionStateStore;
155   private final ServerManager serverManager;
156   private final Server server;
157 
158   // The maximum time to keep a log split info in region states map
159   static final String LOG_SPLIT_TIME = "hbase.master.maximum.logsplit.keeptime";
160   static final long DEFAULT_LOG_SPLIT_TIME = 7200000L; // 2 hours
161 
162   RegionStates(final Server master, final TableStateManager tableStateManager,
163       final ServerManager serverManager, final RegionStateStore regionStateStore) {
164     this.tableStateManager = tableStateManager;
165     this.regionStateStore = regionStateStore;
166     this.serverManager = serverManager;
167     this.server = master;
168   }
169 
170   /**
171    * @return an unmodifiable the region assignment map
172    */
173   public synchronized Map<HRegionInfo, ServerName> getRegionAssignments() {
174     return Collections.unmodifiableMap(regionAssignments);
175   }
176 
177   /**
178    * Return the replicas (including default) for the regions grouped by ServerName
179    * @param regions
180    * @return a pair containing the groupings as a map
181    */
182   synchronized Map<ServerName, List<HRegionInfo>> getRegionAssignments(
183     Collection<HRegionInfo> regions) {
184     Map<ServerName, List<HRegionInfo>> map = new HashMap<ServerName, List<HRegionInfo>>();
185     for (HRegionInfo region : regions) {
186       HRegionInfo defaultReplica = RegionReplicaUtil.getRegionInfoForDefaultReplica(region);
187       Set<HRegionInfo> allReplicas = defaultReplicaToOtherReplicas.get(defaultReplica);
188       if (allReplicas != null) {
189         for (HRegionInfo hri : allReplicas) {
190           ServerName server = regionAssignments.get(hri);
191           if (server != null) {
192             List<HRegionInfo> regionsOnServer = map.get(server);
193             if (regionsOnServer == null) {
194               regionsOnServer = new ArrayList<HRegionInfo>(1);
195               map.put(server, regionsOnServer);
196             }
197             regionsOnServer.add(hri);
198           }
199         }
200       }
201     }
202     return map;
203   }
204 
205   public synchronized ServerName getRegionServerOfRegion(HRegionInfo hri) {
206     return regionAssignments.get(hri);
207   }
208 
209   /**
210    * Get regions in transition and their states
211    */
212   @SuppressWarnings("unchecked")
213   public synchronized Map<String, RegionState> getRegionsInTransition() {
214     return (Map<String, RegionState>)regionsInTransition.clone();
215   }
216 
217   /**
218    * @return True if specified region in transition.
219    */
220   public synchronized boolean isRegionInTransition(final HRegionInfo hri) {
221     return regionsInTransition.containsKey(hri.getEncodedName());
222   }
223 
224   /**
225    * @return True if specified region in transition.
226    */
227   public synchronized boolean isRegionInTransition(final String encodedName) {
228     return regionsInTransition.containsKey(encodedName);
229   }
230 
231   /**
232    * @return True if any region in transition.
233    */
234   public synchronized boolean isRegionsInTransition() {
235     return !regionsInTransition.isEmpty();
236   }
237 
238   /**
239    * @return True if hbase:meta table region is in transition.
240    */
241   public synchronized boolean isMetaRegionInTransition() {
242     for (RegionState state : regionsInTransition.values()) {
243       if (state.getRegion().isMetaRegion()) return true;
244     }
245     return false;
246   }
247 
248   /**
249    * @return True if specified region assigned, and not in transition.
250    */
251   public synchronized boolean isRegionOnline(final HRegionInfo hri) {
252     return !isRegionInTransition(hri) && regionAssignments.containsKey(hri);
253   }
254 
255   /**
256    * @return True if specified region offline/closed, but not in transition.
257    * If the region is not in the map, it is offline to us too.
258    */
259   public synchronized boolean isRegionOffline(final HRegionInfo hri) {
260     return getRegionState(hri) == null || (!isRegionInTransition(hri)
261       && isRegionInState(hri, State.OFFLINE, State.CLOSED));
262   }
263 
264   /**
265    * @return True if specified region is in one of the specified states.
266    */
267   public boolean isRegionInState(
268       final HRegionInfo hri, final State... states) {
269     return isRegionInState(hri.getEncodedName(), states);
270   }
271 
272   /**
273    * @return True if specified region is in one of the specified states.
274    */
275   public boolean isRegionInState(
276       final String encodedName, final State... states) {
277     RegionState regionState = getRegionState(encodedName);
278     return isOneOfStates(regionState, states);
279   }
280 
281   /**
282    * Wait for the state map to be updated by assignment manager.
283    */
284   public synchronized void waitForUpdate(
285       final long timeout) throws InterruptedException {
286     this.wait(timeout);
287   }
288 
289   /**
290    * Get region transition state
291    */
292   public RegionState getRegionTransitionState(final HRegionInfo hri) {
293     return getRegionTransitionState(hri.getEncodedName());
294   }
295 
296   /**
297    * Get region transition state
298    */
299   public synchronized RegionState
300       getRegionTransitionState(final String encodedName) {
301     return regionsInTransition.get(encodedName);
302   }
303 
304   /**
305    * Add a list of regions to RegionStates. If a region is split
306    * and offline, its state will be SPLIT. Otherwise, its state will
307    * be OFFLINE. Region already in RegionStates will be skipped.
308    */
309   public void createRegionStates(
310       final List<HRegionInfo> hris) {
311     for (HRegionInfo hri: hris) {
312       createRegionState(hri);
313     }
314   }
315 
316   /**
317    * Add a region to RegionStates. If the region is split
318    * and offline, its state will be SPLIT. Otherwise, its state will
319    * be OFFLINE. If it is already in RegionStates, this call has
320    * no effect, and the original state is returned.
321    */
322   public RegionState createRegionState(final HRegionInfo hri) {
323     return createRegionState(hri, null, null, null);
324   }
325 
326   /**
327    * Add a region to RegionStates with the specified state.
328    * If the region is already in RegionStates, this call has
329    * no effect, and the original state is returned.
330    *
331    * @param hri the region info to create a state for
332    * @param newState the state to the region in set to
333    * @param serverName the server the region is transitioning on
334    * @param lastHost the last server that hosts the region
335    * @return the current state
336    */
337   public synchronized RegionState createRegionState(final HRegionInfo hri,
338       State newState, ServerName serverName, ServerName lastHost) {
339     if (newState == null || (newState == State.OPEN && serverName == null)) {
340       newState =  State.OFFLINE;
341     }
342     if (hri.isOffline() && hri.isSplit()) {
343       newState = State.SPLIT;
344       serverName = null;
345     }
346     String encodedName = hri.getEncodedName();
347     RegionState regionState = regionStates.get(encodedName);
348     if (regionState != null) {
349       LOG.warn("Tried to create a state for a region already in RegionStates, "
350         + "used existing: " + regionState + ", ignored new: " + newState);
351     } else {
352       regionState = new RegionState(hri, newState, serverName);
353       putRegionState(regionState);
354       if (newState == State.OPEN) {
355         if (!serverName.equals(lastHost)) {
356           LOG.warn("Open region's last host " + lastHost
357             + " should be the same as the current one " + serverName
358             + ", ignored the last and used the current one");
359           lastHost = serverName;
360         }
361         lastAssignments.put(encodedName, lastHost);
362         regionAssignments.put(hri, lastHost);
363       } else if (!regionState.isUnassignable()) {
364         regionsInTransition.put(encodedName, regionState);
365       }
366       if (lastHost != null && newState != State.SPLIT) {
367         addToServerHoldings(lastHost, hri);
368         if (newState != State.OPEN) {
369           oldAssignments.put(encodedName, lastHost);
370         }
371       }
372     }
373     return regionState;
374   }
375 
376   private RegionState putRegionState(RegionState regionState) {
377     HRegionInfo hri = regionState.getRegion();
378     String encodedName = hri.getEncodedName();
379     TableName table = hri.getTable();
380     RegionState oldState = regionStates.put(encodedName, regionState);
381     Map<String, RegionState> map = regionStatesTableIndex.get(table);
382     if (map == null) {
383       map = new HashMap<String, RegionState>();
384       regionStatesTableIndex.put(table, map);
385     }
386     map.put(encodedName, regionState);
387     return oldState;
388   }
389 
390   /**
391    * Update a region state. It will be put in transition if not already there.
392    */
393   public RegionState updateRegionState(
394       final HRegionInfo hri, final State state) {
395     RegionState regionState = getRegionState(hri.getEncodedName());
396     return updateRegionState(hri, state,
397       regionState == null ? null : regionState.getServerName());
398   }
399 
400   /**
401    * Update a region state. It will be put in transition if not already there.
402    *
403    * If we can't find the region info based on the region name in
404    * the transition, log a warning and return null.
405    */
406   public RegionState updateRegionState(
407       final RegionTransition transition, final State state) {
408     byte [] regionName = transition.getRegionName();
409     HRegionInfo regionInfo = getRegionInfo(regionName);
410     if (regionInfo == null) {
411       String prettyRegionName = HRegionInfo.prettyPrint(
412         HRegionInfo.encodeRegionName(regionName));
413       LOG.warn("Failed to find region " + prettyRegionName
414         + " in updating its state to " + state
415         + " based on region transition " + transition);
416       return null;
417     }
418     return updateRegionState(regionInfo, state,
419       transition.getServerName());
420   }
421 
422   /**
423    * Transition a region state to OPEN from OPENING/PENDING_OPEN
424    */
425   public synchronized RegionState transitionOpenFromPendingOpenOrOpeningOnServer(
426       final RegionTransition transition, final RegionState fromState, final ServerName sn) {
427     if(fromState.isPendingOpenOrOpeningOnServer(sn)){
428       return updateRegionState(transition, State.OPEN);
429     }
430     return null;
431   }
432 
433   /**
434    * Update a region state. It will be put in transition if not already there.
435    */
436   public RegionState updateRegionState(
437       final HRegionInfo hri, final State state, final ServerName serverName) {
438     return updateRegionState(hri, state, serverName, HConstants.NO_SEQNUM);
439   }
440 
441   public void regionOnline(
442       final HRegionInfo hri, final ServerName serverName) {
443     regionOnline(hri, serverName, HConstants.NO_SEQNUM);
444   }
445 
446   /**
447    * A region is online, won't be in transition any more.
448    * We can't confirm it is really online on specified region server
449    * because it hasn't been put in region server's online region list yet.
450    */
451   public void regionOnline(final HRegionInfo hri,
452       final ServerName serverName, long openSeqNum) {
453     String encodedName = hri.getEncodedName();
454     if (!serverManager.isServerOnline(serverName)) {
455       // This is possible if the region server dies before master gets a
456       // chance to handle ZK event in time. At this time, if the dead server
457       // is already processed by SSH, we should ignore this event.
458       // If not processed yet, ignore and let SSH deal with it.
459       LOG.warn("Ignored, " + encodedName
460         + " was opened on a dead server: " + serverName);
461       return;
462     }
463     updateRegionState(hri, State.OPEN, serverName, openSeqNum);
464 
465     synchronized (this) {
466       regionsInTransition.remove(encodedName);
467       ServerName oldServerName = regionAssignments.put(hri, serverName);
468       if (!serverName.equals(oldServerName)) {
469         if (LOG.isDebugEnabled()) {
470           LOG.debug("Onlined " + hri.getShortNameToLog() + " on " + serverName + " " + hri);
471         } else {
472           LOG.debug("Onlined " + hri.getShortNameToLog() + " on " + serverName);
473         }
474         addToServerHoldings(serverName, hri);
475         addToReplicaMapping(hri);
476         if (oldServerName == null) {
477           oldServerName = oldAssignments.remove(encodedName);
478         }
479         if (oldServerName != null
480             && !oldServerName.equals(serverName)
481             && serverHoldings.containsKey(oldServerName)) {
482           LOG.info("Offlined " + hri.getShortNameToLog() + " from " + oldServerName);
483           removeFromServerHoldings(oldServerName, hri);
484         }
485       }
486     }
487   }
488 
489   private void addToServerHoldings(ServerName serverName, HRegionInfo hri) {
490     Set<HRegionInfo> regions = serverHoldings.get(serverName);
491     if (regions == null) {
492       regions = new HashSet<HRegionInfo>();
493       serverHoldings.put(serverName, regions);
494     }
495     regions.add(hri);
496   }
497 
498   private void addToReplicaMapping(HRegionInfo hri) {
499     HRegionInfo defaultReplica = RegionReplicaUtil.getRegionInfoForDefaultReplica(hri);
500     Set<HRegionInfo> replicas =
501         defaultReplicaToOtherReplicas.get(defaultReplica);
502     if (replicas == null) {
503       replicas = new HashSet<HRegionInfo>();
504       defaultReplicaToOtherReplicas.put(defaultReplica, replicas);
505     }
506     replicas.add(hri);
507   }
508 
509   private void removeFromServerHoldings(ServerName serverName, HRegionInfo hri) {
510     Set<HRegionInfo> oldRegions = serverHoldings.get(serverName);
511     oldRegions.remove(hri);
512     if (oldRegions.isEmpty()) {
513       serverHoldings.remove(serverName);
514     }
515   }
516 
517   private void removeFromReplicaMapping(HRegionInfo hri) {
518     HRegionInfo defaultReplica = RegionReplicaUtil.getRegionInfoForDefaultReplica(hri);
519     Set<HRegionInfo> replicas = defaultReplicaToOtherReplicas.get(defaultReplica);
520     if (replicas != null) {
521       replicas.remove(hri);
522       if (replicas.isEmpty()) {
523         defaultReplicaToOtherReplicas.remove(defaultReplica);
524       }
525     }
526   }
527 
528   /**
529    * A dead server's wals have been split so that all the regions
530    * used to be open on it can be safely assigned now. Mark them assignable.
531    */
532   public synchronized void logSplit(final ServerName serverName) {
533     for (Iterator<Map.Entry<String, ServerName>> it
534         = lastAssignments.entrySet().iterator(); it.hasNext();) {
535       Map.Entry<String, ServerName> e = it.next();
536       if (e.getValue().equals(serverName)) {
537         it.remove();
538       }
539     }
540     long now = System.currentTimeMillis();
541     if (LOG.isDebugEnabled()) {
542       LOG.debug("Adding to processed servers " + serverName);
543     }
544     processedServers.put(serverName, Long.valueOf(now));
545     Configuration conf = server.getConfiguration();
546     long obsoleteTime = conf.getLong(LOG_SPLIT_TIME, DEFAULT_LOG_SPLIT_TIME);
547     // Doesn't have to be very accurate about the clean up time
548     if (now > lastProcessedServerCleanTime + obsoleteTime) {
549       lastProcessedServerCleanTime = now;
550       long cutoff = now - obsoleteTime;
551       for (Iterator<Map.Entry<ServerName, Long>> it
552           = processedServers.entrySet().iterator(); it.hasNext();) {
553         Map.Entry<ServerName, Long> e = it.next();
554         if (e.getValue().longValue() < cutoff) {
555           if (LOG.isDebugEnabled()) {
556             LOG.debug("Removed from processed servers " + e.getKey());
557           }
558           it.remove();
559         }
560       }
561     }
562   }
563 
564   /**
565    * Log split is done for a given region, so it is assignable now.
566    */
567   public void logSplit(final HRegionInfo region) {
568     clearLastAssignment(region);
569   }
570 
571   public synchronized void clearLastAssignment(final HRegionInfo region) {
572     lastAssignments.remove(region.getEncodedName());
573   }
574 
575   /**
576    * A region is offline, won't be in transition any more.
577    */
578   public void regionOffline(final HRegionInfo hri) {
579     regionOffline(hri, null);
580   }
581 
582   /**
583    * A region is offline, won't be in transition any more. Its state
584    * should be the specified expected state, which can only be
585    * Split/Merged/Offline/null(=Offline)/SplittingNew/MergingNew.
586    */
587   public void regionOffline(
588       final HRegionInfo hri, final State expectedState) {
589     Preconditions.checkArgument(expectedState == null
590       || RegionState.isUnassignable(expectedState),
591         "Offlined region should not be " + expectedState);
592     if (isRegionInState(hri, State.SPLITTING_NEW, State.MERGING_NEW)) {
593       // Remove it from all region maps
594       deleteRegion(hri);
595       return;
596     }
597     State newState =
598       expectedState == null ? State.OFFLINE : expectedState;
599     updateRegionState(hri, newState);
600     String encodedName = hri.getEncodedName();
601     synchronized (this) {
602       regionsInTransition.remove(encodedName);
603       ServerName oldServerName = regionAssignments.remove(hri);
604       if (oldServerName != null && serverHoldings.containsKey(oldServerName)) {
605         if (newState == State.MERGED || newState == State.SPLIT
606             || hri.isMetaRegion() || tableStateManager.isTableState(hri.getTable(),
607               ZooKeeperProtos.Table.State.DISABLED, ZooKeeperProtos.Table.State.DISABLING)) {
608           // Offline the region only if it's merged/split, or the table is disabled/disabling.
609           // Otherwise, offline it from this server only when it is online on a different server.
610           LOG.info("Offlined " + hri.getShortNameToLog() + " from " + oldServerName);
611           removeFromServerHoldings(oldServerName, hri);
612           removeFromReplicaMapping(hri);
613         } else {
614           // Need to remember it so that we can offline it from this
615           // server when it is online on a different server.
616           oldAssignments.put(encodedName, oldServerName);
617         }
618       }
619     }
620   }
621 
622   /**
623    * A server is offline, all regions on it are dead.
624    */
625   public List<HRegionInfo> serverOffline(final ZooKeeperWatcher watcher, final ServerName sn) {
626     // Offline all regions on this server not already in transition.
627     List<HRegionInfo> rits = new ArrayList<HRegionInfo>();
628     Set<HRegionInfo> regionsToCleanIfNoMetaEntry = new HashSet<HRegionInfo>();
629     // Offline regions outside the loop and synchronized block to avoid
630     // ConcurrentModificationException and deadlock in case of meta anassigned,
631     // but RegionState a blocked.
632     Set<HRegionInfo> regionsToOffline = new HashSet<HRegionInfo>();
633     synchronized (this) {
634       Set<HRegionInfo> assignedRegions = serverHoldings.get(sn);
635       if (assignedRegions == null) {
636         assignedRegions = new HashSet<HRegionInfo>();
637       }
638 
639       for (HRegionInfo region : assignedRegions) {
640         // Offline open regions, no need to offline if SPLIT/MERGED/OFFLINE
641         if (isRegionOnline(region)) {
642           regionsToOffline.add(region);
643         } else if (isRegionInState(region, State.SPLITTING, State.MERGING)) {
644           LOG.debug("Offline splitting/merging region " + getRegionState(region));
645           try {
646             // Delete the ZNode if exists
647             ZKAssign.deleteNodeFailSilent(watcher, region);
648             regionsToOffline.add(region);
649           } catch (KeeperException ke) {
650             server.abort("Unexpected ZK exception deleting node " + region, ke);
651           }
652         }
653       }
654 
655       for (RegionState state : regionsInTransition.values()) {
656         HRegionInfo hri = state.getRegion();
657         if (assignedRegions.contains(hri)) {
658           // Region is open on this region server, but in transition.
659           // This region must be moving away from this server, or splitting/merging.
660           // SSH will handle it, either skip assigning, or re-assign.
661           LOG.info("Transitioning " + state + " will be handled by SSH for " + sn);
662         } else if (sn.equals(state.getServerName())) {
663           // Region is in transition on this region server, and this
664           // region is not open on this server. So the region must be
665           // moving to this server from another one (i.e. opening or
666           // pending open on this server, was open on another one.
667           // Offline state is also kind of pending open if the region is in
668           // transition. The region could be in failed_close state too if we have
669           // tried several times to open it while this region server is not reachable)
670           if (state.isPendingOpenOrOpening() || state.isFailedClose() || state.isOffline()) {
671             LOG.info("Found region in " + state + " to be reassigned by SSH for " + sn);
672             rits.add(hri);
673           } else if(state.isSplittingNew()) {
674             regionsToCleanIfNoMetaEntry.add(state.getRegion());
675           } else {
676             LOG.warn("THIS SHOULD NOT HAPPEN: unexpected " + state);
677           }
678         }
679       }
680       this.notifyAll();
681     }
682 
683     for (HRegionInfo hri : regionsToOffline) {
684       regionOffline(hri);
685     }
686 
687     cleanIfNoMetaEntry(regionsToCleanIfNoMetaEntry);
688     return rits;
689   }
690 
691   /**
692    * This method does an RPC to hbase:meta. Do not call this method with a lock/synchronize held.
693    * @param hris The hris to check if empty in hbase:meta and if so, clean them up.
694    */
695   private void cleanIfNoMetaEntry(Set<HRegionInfo> hris) {
696     if (hris.isEmpty()) return;
697     for (HRegionInfo hri: hris) {
698       try {
699         // This is RPC to meta table. It is done while we have a synchronize on
700         // regionstates. No progress will be made if meta is not available at this time.
701         // This is a cleanup task. Not critical.
702         if (MetaTableAccessor.getRegion(server.getConnection(), hri.getEncodedNameAsBytes()) ==
703             null) {
704           regionOffline(hri);
705           FSUtils.deleteRegionDir(server.getConfiguration(), hri);
706         }
707       } catch (IOException e) {
708         LOG.warn("Got exception while deleting " + hri + " directories from file system.", e);
709       }
710     }
711   }
712 
713   /**
714    * Gets the online regions of the specified table.
715    * This method looks at the in-memory state.  It does not go to <code>hbase:meta</code>.
716    * Only returns <em>online</em> regions.  If a region on this table has been
717    * closed during a disable, etc., it will be included in the returned list.
718    * So, the returned list may not necessarily be ALL regions in this table, its
719    * all the ONLINE regions in the table.
720    * @param tableName
721    * @return Online regions from <code>tableName</code>
722    */
723   public synchronized List<HRegionInfo> getRegionsOfTable(TableName tableName) {
724     List<HRegionInfo> tableRegions = new ArrayList<HRegionInfo>();
725     // boundary needs to have table's name but regionID 0 so that it is sorted
726     // before all table's regions.
727     HRegionInfo boundary = new HRegionInfo(tableName, null, null, false, 0L);
728     for (HRegionInfo hri: regionAssignments.tailMap(boundary).keySet()) {
729       if(!hri.getTable().equals(tableName)) break;
730       tableRegions.add(hri);
731     }
732     return tableRegions;
733   }
734 
735   /**
736    * Gets current state of all regions of the table.
737    * This method looks at the in-memory state.  It does not go to <code>hbase:meta</code>.
738    * Method guaranteed to return keys for all states
739    * in {@link org.apache.hadoop.hbase.master.RegionState.State}
740    *
741    * @param tableName
742    * @return Online regions from <code>tableName</code>
743    */
744   public synchronized Map<RegionState.State, List<HRegionInfo>>
745   getRegionByStateOfTable(TableName tableName) {
746     Map<RegionState.State, List<HRegionInfo>> tableRegions =
747         new HashMap<State, List<HRegionInfo>>();
748     for (State state : State.values()) {
749       tableRegions.put(state, new ArrayList<HRegionInfo>());
750     }
751     Map<String, RegionState> indexMap = regionStatesTableIndex.get(tableName);
752     if (indexMap == null)
753       return tableRegions;
754     for (RegionState regionState : indexMap.values()) {
755       tableRegions.get(regionState.getState()).add(regionState.getRegion());
756     }
757     return tableRegions;
758   }
759 
760   /**
761    * Wait on region to clear regions-in-transition.
762    * <p>
763    * If the region isn't in transition, returns immediately.  Otherwise, method
764    * blocks until the region is out of transition.
765    */
766   public synchronized void waitOnRegionToClearRegionsInTransition(
767       final HRegionInfo hri) throws InterruptedException {
768     if (!isRegionInTransition(hri)) return;
769 
770     while(!server.isStopped() && isRegionInTransition(hri)) {
771       RegionState rs = getRegionState(hri);
772       LOG.info("Waiting on " + rs + " to clear regions-in-transition");
773       waitForUpdate(100);
774     }
775 
776     if (server.isStopped()) {
777       LOG.info("Giving up wait on region in " +
778         "transition because stoppable.isStopped is set");
779     }
780   }
781 
782   /**
783    * A table is deleted. Remove its regions from all internal maps.
784    * We loop through all regions assuming we don't delete tables too much.
785    */
786   public void tableDeleted(final TableName tableName) {
787     Set<HRegionInfo> regionsToDelete = new HashSet<HRegionInfo>();
788     synchronized (this) {
789       for (RegionState state: regionStates.values()) {
790         HRegionInfo region = state.getRegion();
791         if (region.getTable().equals(tableName)) {
792           regionsToDelete.add(region);
793         }
794       }
795     }
796     for (HRegionInfo region: regionsToDelete) {
797       deleteRegion(region);
798     }
799   }
800 
801   /**
802    * Get a copy of all regions assigned to a server
803    */
804   public synchronized Set<HRegionInfo> getServerRegions(ServerName serverName) {
805     Set<HRegionInfo> regions = serverHoldings.get(serverName);
806     if (regions == null) return null;
807     return new HashSet<HRegionInfo>(regions);
808   }
809 
810   /**
811    * Remove a region from all state maps.
812    */
813   @VisibleForTesting
814   public synchronized void deleteRegion(final HRegionInfo hri) {
815     String encodedName = hri.getEncodedName();
816     regionsInTransition.remove(encodedName);
817     regionStates.remove(encodedName);
818     TableName table = hri.getTable();
819     Map<String, RegionState> indexMap = regionStatesTableIndex.get(table);
820     indexMap.remove(encodedName);
821     if (indexMap.size() == 0)
822       regionStatesTableIndex.remove(table);
823     lastAssignments.remove(encodedName);
824     ServerName sn = regionAssignments.remove(hri);
825     if (sn != null) {
826       Set<HRegionInfo> regions = serverHoldings.get(sn);
827       regions.remove(hri);
828     }
829   }
830 
831   /**
832    * Checking if a region was assigned to a server which is not online now.
833    * If so, we should hold re-assign this region till SSH has split its wals.
834    * Once logs are split, the last assignment of this region will be reset,
835    * which means a null last assignment server is ok for re-assigning.
836    *
837    * A region server could be dead but we don't know it yet. We may
838    * think it's online falsely. Therefore if a server is online, we still
839    * need to confirm it reachable and having the expected start code.
840    */
841   synchronized boolean wasRegionOnDeadServer(final String encodedName) {
842     ServerName server = lastAssignments.get(encodedName);
843     return isServerDeadAndNotProcessed(server);
844   }
845 
846   synchronized boolean isServerDeadAndNotProcessed(ServerName server) {
847     if (server == null) return false;
848     if (serverManager.isServerOnline(server)) {
849       String hostAndPort = server.getHostAndPort();
850       long startCode = server.getStartcode();
851       Long deadCode = deadServers.get(hostAndPort);
852       if (deadCode == null || startCode > deadCode.longValue()) {
853         if (serverManager.isServerReachable(server)) {
854           return false;
855         }
856         // The size of deadServers won't grow unbounded.
857         deadServers.put(hostAndPort, Long.valueOf(startCode));
858       }
859       // Watch out! If the server is not dead, the region could
860       // remain unassigned. That's why ServerManager#isServerReachable
861       // should use some retry.
862       //
863       // We cache this info since it is very unlikely for that
864       // instance to come back up later on. We don't want to expire
865       // the server since we prefer to let it die naturally.
866       LOG.warn("Couldn't reach online server " + server);
867     }
868     // Now, we know it's dead. Check if it's processed
869     return !processedServers.containsKey(server);
870   }
871 
872  /**
873    * Get the last region server a region was on for purpose of re-assignment,
874    * i.e. should the re-assignment be held back till log split is done?
875    */
876   synchronized ServerName getLastRegionServerOfRegion(final String encodedName) {
877     return lastAssignments.get(encodedName);
878   }
879 
880   synchronized void setLastRegionServerOfRegions(
881       final ServerName serverName, final List<HRegionInfo> regionInfos) {
882     for (HRegionInfo hri: regionInfos) {
883       setLastRegionServerOfRegion(serverName, hri.getEncodedName());
884     }
885   }
886 
887   synchronized void setLastRegionServerOfRegion(
888       final ServerName serverName, final String encodedName) {
889     lastAssignments.put(encodedName, serverName);
890   }
891 
892   void splitRegion(HRegionInfo p,
893       HRegionInfo a, HRegionInfo b, ServerName sn) throws IOException {
894 
895     regionStateStore.splitRegion(p, a, b, sn, getRegionReplication(p));
896     synchronized (this) {
897       // After PONR, split is considered to be done.
898       // Update server holdings to be aligned with the meta.
899       Set<HRegionInfo> regions = serverHoldings.get(sn);
900       if (regions == null) {
901         throw new IllegalStateException(sn + " should host some regions");
902       }
903       regions.remove(p);
904       regions.add(a);
905       regions.add(b);
906     }
907   }
908 
909   void mergeRegions(HRegionInfo p,
910       HRegionInfo a, HRegionInfo b, ServerName sn) throws IOException {
911     regionStateStore.mergeRegions(p, a, b, sn, getRegionReplication(a));
912     synchronized (this) {
913       // After PONR, merge is considered to be done.
914       // Update server holdings to be aligned with the meta.
915       Set<HRegionInfo> regions = serverHoldings.get(sn);
916       if (regions == null) {
917         throw new IllegalStateException(sn + " should host some regions");
918       }
919       regions.remove(a);
920       regions.remove(b);
921       regions.add(p);
922     }
923   }
924 
925   private int getRegionReplication(HRegionInfo r) throws IOException {
926     if (tableStateManager != null) {
927       HTableDescriptor htd = ((MasterServices)server).getTableDescriptors().get(r.getTable());
928       if (htd != null) {
929         return htd.getRegionReplication();
930       }
931     }
932     return 1;
933   }
934 
935   /**
936    * At cluster clean re/start, mark all user regions closed except those of tables
937    * that are excluded, such as disabled/disabling/enabling tables. All user regions
938    * and their previous locations are returned.
939    */
940   synchronized Map<HRegionInfo, ServerName> closeAllUserRegions(Set<TableName> excludedTables) {
941     boolean noExcludeTables = excludedTables == null || excludedTables.isEmpty();
942     Set<HRegionInfo> toBeClosed = new HashSet<HRegionInfo>(regionStates.size());
943     for(RegionState state: regionStates.values()) {
944       HRegionInfo hri = state.getRegion();
945       if (state.isSplit() || hri.isSplit()) {
946         continue;
947       }
948       TableName tableName = hri.getTable();
949       if (!TableName.META_TABLE_NAME.equals(tableName)
950           && (noExcludeTables || !excludedTables.contains(tableName))) {
951         toBeClosed.add(hri);
952       }
953     }
954     Map<HRegionInfo, ServerName> allUserRegions =
955       new HashMap<HRegionInfo, ServerName>(toBeClosed.size());
956     for (HRegionInfo hri: toBeClosed) {
957       RegionState regionState = updateRegionState(hri, State.CLOSED);
958       allUserRegions.put(hri, regionState.getServerName());
959     }
960     return allUserRegions;
961   }
962 
963   /**
964    * Compute the average load across all region servers.
965    * Currently, this uses a very naive computation - just uses the number of
966    * regions being served, ignoring stats about number of requests.
967    * @return the average load
968    */
969   protected synchronized double getAverageLoad() {
970     int numServers = 0, totalLoad = 0;
971     for (Map.Entry<ServerName, Set<HRegionInfo>> e: serverHoldings.entrySet()) {
972       Set<HRegionInfo> regions = e.getValue();
973       ServerName serverName = e.getKey();
974       int regionCount = regions.size();
975       if (serverManager.isServerOnline(serverName)) {
976         totalLoad += regionCount;
977         numServers++;
978       }
979     }
980     if (numServers > 1) {
981       // The master region server holds only a couple regions.
982       // Don't consider this server in calculating the average load
983       // if there are other region servers to avoid possible confusion.
984       Set<HRegionInfo> hris = serverHoldings.get(server.getServerName());
985       if (hris != null) {
986         totalLoad -= hris.size();
987         numServers--;
988       }
989     }
990     return numServers == 0 ? 0.0 :
991       (double)totalLoad / (double)numServers;
992   }
993 
994   /**
995    * This is an EXPENSIVE clone.  Cloning though is the safest thing to do.
996    * Can't let out original since it can change and at least the load balancer
997    * wants to iterate this exported list.  We need to synchronize on regions
998    * since all access to this.servers is under a lock on this.regions.
999    *
1000    * @return A clone of current assignments by table.
1001    */
1002   protected Map<TableName, Map<ServerName, List<HRegionInfo>>>
1003       getAssignmentsByTable() {
1004     Map<TableName, Map<ServerName, List<HRegionInfo>>> result =
1005       new HashMap<TableName, Map<ServerName,List<HRegionInfo>>>();
1006     synchronized (this) {
1007       if (!server.getConfiguration().getBoolean(
1008             HConstants.HBASE_MASTER_LOADBALANCE_BYTABLE, false)) {
1009         Map<ServerName, List<HRegionInfo>> svrToRegions =
1010           new HashMap<ServerName, List<HRegionInfo>>(serverHoldings.size());
1011         for (Map.Entry<ServerName, Set<HRegionInfo>> e: serverHoldings.entrySet()) {
1012           svrToRegions.put(e.getKey(), new ArrayList<HRegionInfo>(e.getValue()));
1013         }
1014         result.put(TableName.valueOf(HConstants.ENSEMBLE_TABLE_NAME), svrToRegions);
1015       } else {
1016         for (Map.Entry<ServerName, Set<HRegionInfo>> e: serverHoldings.entrySet()) {
1017           for (HRegionInfo hri: e.getValue()) {
1018             if (hri.isMetaRegion()) continue;
1019             TableName tablename = hri.getTable();
1020             Map<ServerName, List<HRegionInfo>> svrToRegions = result.get(tablename);
1021             if (svrToRegions == null) {
1022               svrToRegions = new HashMap<ServerName, List<HRegionInfo>>(serverHoldings.size());
1023               result.put(tablename, svrToRegions);
1024             }
1025             List<HRegionInfo> regions = svrToRegions.get(e.getKey());
1026             if (regions == null) {
1027               regions = new ArrayList<HRegionInfo>();
1028               svrToRegions.put(e.getKey(), regions);
1029             }
1030             regions.add(hri);
1031           }
1032         }
1033       }
1034     }
1035 
1036     Map<ServerName, ServerLoad>
1037       onlineSvrs = serverManager.getOnlineServers();
1038     // Take care of servers w/o assignments, and remove servers in draining mode
1039     List<ServerName> drainingServers = this.serverManager.getDrainingServersList();
1040     for (Map<ServerName, List<HRegionInfo>> map: result.values()) {
1041       for (ServerName svr: onlineSvrs.keySet()) {
1042         if (!map.containsKey(svr)) {
1043           map.put(svr, new ArrayList<HRegionInfo>());
1044         }
1045       }
1046       map.keySet().removeAll(drainingServers);
1047     }
1048     return result;
1049   }
1050 
1051   protected RegionState getRegionState(final HRegionInfo hri) {
1052     return getRegionState(hri.getEncodedName());
1053   }
1054 
1055   /**
1056    * Returns a clone of region assignments per server
1057    * @return a Map of ServerName to a List of HRegionInfo's
1058    */
1059   protected synchronized Map<ServerName, List<HRegionInfo>> getRegionAssignmentsByServer() {
1060     Map<ServerName, List<HRegionInfo>> regionsByServer =
1061         new HashMap<ServerName, List<HRegionInfo>>(serverHoldings.size());
1062     for (Map.Entry<ServerName, Set<HRegionInfo>> e: serverHoldings.entrySet()) {
1063       regionsByServer.put(e.getKey(), new ArrayList<HRegionInfo>(e.getValue()));
1064     }
1065     return regionsByServer;
1066   }
1067 
1068   protected synchronized RegionState getRegionState(final String encodedName) {
1069     return regionStates.get(encodedName);
1070   }
1071 
1072   /**
1073    * Get the HRegionInfo from cache, if not there, from the hbase:meta table
1074    * @param  regionName
1075    * @return HRegionInfo for the region
1076    */
1077   @SuppressWarnings("deprecation")
1078   protected HRegionInfo getRegionInfo(final byte [] regionName) {
1079     String encodedName = HRegionInfo.encodeRegionName(regionName);
1080     RegionState regionState = getRegionState(encodedName);
1081     if (regionState != null) {
1082       return regionState.getRegion();
1083     }
1084 
1085     try {
1086       Pair<HRegionInfo, ServerName> p =
1087         MetaTableAccessor.getRegion(server.getConnection(), regionName);
1088       HRegionInfo hri = p == null ? null : p.getFirst();
1089       if (hri != null) {
1090         createRegionState(hri);
1091       }
1092       return hri;
1093     } catch (IOException e) {
1094       server.abort("Aborting because error occoured while reading "
1095         + Bytes.toStringBinary(regionName) + " from hbase:meta", e);
1096       return null;
1097     }
1098   }
1099 
1100   static boolean isOneOfStates(RegionState regionState, State... states) {
1101     State s = regionState != null ? regionState.getState() : null;
1102     for (State state: states) {
1103       if (s == state) return true;
1104     }
1105     return false;
1106   }
1107 
1108   /**
1109    * Update a region state. It will be put in transition if not already there.
1110    */
1111   private RegionState updateRegionState(final HRegionInfo hri,
1112       final State state, final ServerName serverName, long openSeqNum) {
1113     if (state == State.FAILED_CLOSE || state == State.FAILED_OPEN) {
1114       LOG.warn("Failed to open/close " + hri.getShortNameToLog()
1115         + " on " + serverName + ", set to " + state);
1116     }
1117 
1118     String encodedName = hri.getEncodedName();
1119     RegionState regionState = new RegionState(
1120       hri, state, System.currentTimeMillis(), serverName);
1121     RegionState oldState = getRegionState(encodedName);
1122     if (!regionState.equals(oldState)) {
1123       LOG.info("Transition " + oldState + " to " + regionState);
1124       // Persist region state before updating in-memory info, if needed
1125       regionStateStore.updateRegionState(openSeqNum, regionState, oldState);
1126     }
1127 
1128     synchronized (this) {
1129       regionsInTransition.put(encodedName, regionState);
1130       putRegionState(regionState);
1131 
1132       // For these states, region should be properly closed.
1133       // There should be no log splitting issue.
1134       if ((state == State.CLOSED || state == State.MERGED
1135           || state == State.SPLIT) && lastAssignments.containsKey(encodedName)) {
1136         ServerName last = lastAssignments.get(encodedName);
1137         if (last.equals(serverName)) {
1138           lastAssignments.remove(encodedName);
1139         } else {
1140           LOG.warn(encodedName + " moved to " + state + " on "
1141             + serverName + ", expected " + last);
1142         }
1143       }
1144 
1145       // Once a region is opened, record its last assignment right away.
1146       if (serverName != null && state == State.OPEN) {
1147         ServerName last = lastAssignments.get(encodedName);
1148         if (!serverName.equals(last)) {
1149           lastAssignments.put(encodedName, serverName);
1150           if (last != null && isServerDeadAndNotProcessed(last)) {
1151             LOG.warn(encodedName + " moved to " + serverName
1152               + ", while it's previous host " + last
1153               + " is dead but not processed yet");
1154           }
1155         }
1156       }
1157 
1158       // notify the change
1159       this.notifyAll();
1160     }
1161     return regionState;
1162   }
1163 }