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.HashMap;
23  import java.util.HashSet;
24  import java.util.Iterator;
25  import java.util.List;
26  import java.util.Map;
27  import java.util.Set;
28  import java.util.TreeMap;
29  
30  import org.apache.commons.logging.Log;
31  import org.apache.commons.logging.LogFactory;
32  import org.apache.hadoop.classification.InterfaceAudience;
33  import org.apache.hadoop.conf.Configuration;
34  import org.apache.hadoop.hbase.HRegionInfo;
35  import org.apache.hadoop.hbase.RegionTransition;
36  import org.apache.hadoop.hbase.Server;
37  import org.apache.hadoop.hbase.ServerLoad;
38  import org.apache.hadoop.hbase.ServerName;
39  import org.apache.hadoop.hbase.TableName;
40  import org.apache.hadoop.hbase.catalog.MetaReader;
41  import org.apache.hadoop.hbase.master.RegionState.State;
42  import org.apache.hadoop.hbase.util.Bytes;
43  import org.apache.hadoop.hbase.util.Pair;
44  import org.apache.hadoop.hbase.zookeeper.ZKAssign;
45  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
46  import org.apache.zookeeper.KeeperException;
47  
48  import com.google.common.base.Preconditions;
49  
50  /**
51   * Region state accountant. It holds the states of all regions in the memory.
52   * In normal scenario, it should match the meta table and the true region states.
53   *
54   * This map is used by AssignmentManager to track region states.
55   */
56  @InterfaceAudience.Private
57  public class RegionStates {
58    private static final Log LOG = LogFactory.getLog(RegionStates.class);
59  
60    /**
61     * Regions currently in transition.
62     */
63    final HashMap<String, RegionState> regionsInTransition;
64  
65    /**
66     * Region encoded name to state map.
67     * All the regions should be in this map.
68     */
69    private final Map<String, RegionState> regionStates;
70  
71    /**
72     * Server to regions assignment map.
73     * Contains the set of regions currently assigned to a given server.
74     */
75    private final Map<ServerName, Set<HRegionInfo>> serverHoldings;
76  
77    /**
78     * Region to server assignment map.
79     * Contains the server a given region is currently assigned to.
80     */
81    private final TreeMap<HRegionInfo, ServerName> regionAssignments;
82  
83    /**
84     * Encoded region name to server assignment map for re-assignment
85     * purpose. Contains the server a given region is last known assigned
86     * to, which has not completed log splitting, so not assignable.
87     * If a region is currently assigned, this server info in this
88     * map should be the same as that in regionAssignments.
89     * However the info in regionAssignments is cleared when the region
90     * is offline while the info in lastAssignments is cleared when
91     * the region is closed or the server is dead and processed.
92     */
93    private final HashMap<String, ServerName> lastAssignments;
94  
95    /**
96     * Map a host port pair string to the latest start code
97     * of a region server which is known to be dead. It is dead
98     * to us, but server manager may not know it yet.
99     */
100   private final HashMap<String, Long> deadServers;
101 
102   /**
103    * Map a dead servers to the time when log split is done.
104    * Since log splitting is not ordered, we have to remember
105    * all processed instances. The map is cleaned up based
106    * on a configured time. By default, we assume a dead
107    * server should be done with log splitting in two hours.
108    */
109   private final HashMap<ServerName, Long> processedServers;
110   private long lastProcessedServerCleanTime;
111 
112   private final ServerManager serverManager;
113   private final Server server;
114 
115   // The maximum time to keep a log split info in region states map
116   static final String LOG_SPLIT_TIME = "hbase.master.maximum.logsplit.keeptime";
117   static final long DEFAULT_LOG_SPLIT_TIME = 7200000L; // 2 hours
118 
119   RegionStates(final Server master, final ServerManager serverManager) {
120     regionStates = new HashMap<String, RegionState>();
121     regionsInTransition = new HashMap<String, RegionState>();
122     serverHoldings = new HashMap<ServerName, Set<HRegionInfo>>();
123     regionAssignments = new TreeMap<HRegionInfo, ServerName>();
124     lastAssignments = new HashMap<String, ServerName>();
125     processedServers = new HashMap<ServerName, Long>();
126     deadServers = new HashMap<String, Long>();
127     this.serverManager = serverManager;
128     this.server = master;
129   }
130 
131   /**
132    * @return an unmodifiable the region assignment map
133    */
134   @SuppressWarnings("unchecked")
135   public synchronized Map<HRegionInfo, ServerName> getRegionAssignments() {
136     return (Map<HRegionInfo, ServerName>)regionAssignments.clone();
137   }
138 
139   public synchronized ServerName getRegionServerOfRegion(HRegionInfo hri) {
140     return regionAssignments.get(hri);
141   }
142 
143   /**
144    * Get regions in transition and their states
145    */
146   @SuppressWarnings("unchecked")
147   public synchronized Map<String, RegionState> getRegionsInTransition() {
148     return (Map<String, RegionState>)regionsInTransition.clone();
149   }
150 
151   /**
152    * @return True if specified region in transition.
153    */
154   public synchronized boolean isRegionInTransition(final HRegionInfo hri) {
155     return regionsInTransition.containsKey(hri.getEncodedName());
156   }
157 
158   /**
159    * @return True if specified region in transition.
160    */
161   public synchronized boolean isRegionInTransition(final String encodedName) {
162     return regionsInTransition.containsKey(encodedName);
163   }
164 
165   /**
166    * @return True if any region in transition.
167    */
168   public synchronized boolean isRegionsInTransition() {
169     return !regionsInTransition.isEmpty();
170   }
171 
172   /**
173    * @return True if specified region assigned, and not in transition.
174    */
175   public synchronized boolean isRegionOnline(final HRegionInfo hri) {
176     return !isRegionInTransition(hri) && regionAssignments.containsKey(hri);
177   }
178 
179   /**
180    * @return True if specified region offline/closed, but not in transition.
181    * If the region is not in the map, it is offline to us too.
182    */
183   public synchronized boolean isRegionOffline(final HRegionInfo hri) {
184     return getRegionState(hri) == null || (!isRegionInTransition(hri)
185       && isRegionInState(hri, State.OFFLINE, State.CLOSED));
186   }
187 
188   /**
189    * @return True if specified region is in one of the specified states.
190    */
191   public synchronized boolean isRegionInState(
192       final HRegionInfo hri, final State... states) {
193     return isRegionInState(hri.getEncodedName(), states);
194   }
195 
196   /**
197    * @return True if specified region is in one of the specified states.
198    */
199   public synchronized boolean isRegionInState(
200       final String encodedName, final State... states) {
201     RegionState regionState = getRegionState(encodedName);
202     State s = regionState != null ? regionState.getState() : null;
203     for (State state: states) {
204       if (s == state) return true;
205     }
206     return false;
207   }
208 
209   /**
210    * Wait for the state map to be updated by assignment manager.
211    */
212   public synchronized void waitForUpdate(
213       final long timeout) throws InterruptedException {
214     this.wait(timeout);
215   }
216 
217   /**
218    * Get region transition state
219    */
220   public synchronized RegionState
221       getRegionTransitionState(final HRegionInfo hri) {
222     return regionsInTransition.get(hri.getEncodedName());
223   }
224 
225   /**
226    * Get region transition state
227    */
228   public synchronized RegionState
229       getRegionTransitionState(final String encodedName) {
230     return regionsInTransition.get(encodedName);
231   }
232 
233   /**
234    * Add a list of regions to RegionStates. If a region is split
235    * and offline, its state will be SPLIT. Otherwise, its state will
236    * be OFFLINE. Region already in RegionStates will be skipped.
237    */
238   public synchronized void createRegionStates(
239       final List<HRegionInfo> hris) {
240     for (HRegionInfo hri: hris) {
241       createRegionState(hri);
242     }
243   }
244 
245   /**
246    * Add a region to RegionStates. If the region is split
247    * and offline, its state will be SPLIT. Otherwise, its state will
248    * be OFFLINE. If it is already in RegionStates, this call has
249    * no effect, and the original state is returned.
250    */
251   public synchronized RegionState createRegionState(final HRegionInfo hri) {
252     State newState = (hri.isOffline() && hri.isSplit()) ? State.SPLIT : State.OFFLINE;
253     String encodedName = hri.getEncodedName();
254     RegionState regionState = regionStates.get(encodedName);
255     if (regionState != null) {
256       LOG.warn("Tried to create a state for a region already in RegionStates, "
257         + "used existing: " + regionState + ", ignored new: " + newState);
258     } else {
259       regionState = new RegionState(hri, newState);
260       regionStates.put(encodedName, regionState);
261     }
262     return regionState;
263   }
264 
265   /**
266    * Update a region state. It will be put in transition if not already there.
267    */
268   public synchronized RegionState updateRegionState(
269       final HRegionInfo hri, final State state) {
270     RegionState regionState = regionStates.get(hri.getEncodedName());
271     return updateRegionState(hri, state,
272       regionState == null ? null : regionState.getServerName());
273   }
274 
275   /**
276    * Update a region state. It will be put in transition if not already there.
277    *
278    * If we can't find the region info based on the region name in
279    * the transition, log a warning and return null.
280    */
281   public synchronized RegionState updateRegionState(
282       final RegionTransition transition, final State state) {
283     byte [] regionName = transition.getRegionName();
284     HRegionInfo regionInfo = getRegionInfo(regionName);
285     if (regionInfo == null) {
286       String prettyRegionName = HRegionInfo.prettyPrint(
287         HRegionInfo.encodeRegionName(regionName));
288       LOG.warn("Failed to find region " + prettyRegionName
289         + " in updating its state to " + state
290         + " based on region transition " + transition);
291       return null;
292     }
293     return updateRegionState(regionInfo, state,
294       transition.getServerName());
295   }
296 
297   /**
298    * Update a region state. It will be put in transition if not already there.
299    */
300   public synchronized RegionState updateRegionState(
301       final HRegionInfo hri, final State state, final ServerName serverName) {
302     if (state == State.FAILED_CLOSE || state == State.FAILED_OPEN) {
303       LOG.warn("Failed to open/close " + hri.getShortNameToLog()
304         + " on " + serverName + ", set to " + state);
305     }
306 
307     String encodedName = hri.getEncodedName();
308     RegionState regionState = new RegionState(
309       hri, state, System.currentTimeMillis(), serverName);
310     regionsInTransition.put(encodedName, regionState);
311     RegionState oldState = regionStates.put(encodedName, regionState);
312     ServerName oldServerName = oldState == null ? null : oldState.getServerName();
313     if (oldState == null || oldState.getState() != regionState.getState()
314         || (oldServerName == null && serverName != null)
315         || (oldServerName != null && !oldServerName.equals(serverName))) {
316       LOG.info("Transitioned " + oldState + " to " + regionState);
317     }
318 
319     // For these states, region should be properly closed.
320     // There should be no log splitting issue.
321     if ((state == State.CLOSED || state == State.MERGED
322         || state == State.SPLIT) && lastAssignments.containsKey(encodedName)) {
323       ServerName last = lastAssignments.get(encodedName);
324       if (last.equals(serverName)) {
325         lastAssignments.remove(encodedName);
326       } else {
327         LOG.warn(encodedName + " moved to " + state + " on "
328           + serverName + ", expected " + last);
329       }
330     }
331 
332     // Once a region is opened, record its last assignment right away.
333     if (serverName != null && state == State.OPEN) {
334       ServerName last = lastAssignments.get(encodedName);
335       if (!serverName.equals(last)) {
336         lastAssignments.put(encodedName, serverName);
337         if (last != null && isServerDeadAndNotProcessed(last)) {
338           LOG.warn(encodedName + " moved to " + serverName
339             + ", while it's previous host " + last
340             + " is dead but not processed yet");
341         }
342       }
343     }
344 
345     // notify the change
346     this.notifyAll();
347     return regionState;
348   }
349 
350   /**
351    * A region is online, won't be in transition any more.
352    * We can't confirm it is really online on specified region server
353    * because it hasn't been put in region server's online region list yet.
354    */
355   public synchronized void regionOnline(
356       final HRegionInfo hri, final ServerName serverName) {
357     if (!serverManager.isServerOnline(serverName)) {
358       // This is possible if the region server dies before master gets a
359       // chance to handle ZK event in time. At this time, if the dead server
360       // is already processed by SSH, we should ignore this event.
361       // If not processed yet, ignore and let SSH deal with it.
362       LOG.warn("Ignored, " + hri.getEncodedName()
363         + " was opened on a dead server: " + serverName);
364       return;
365     }
366 
367     String encodedName = hri.getEncodedName();
368     RegionState oldState = regionStates.get(encodedName);
369     if (oldState == null) {
370       LOG.warn("Online region not in RegionStates: " + hri.getShortNameToLog());
371     }
372     updateRegionState(hri, State.OPEN, serverName);
373     regionsInTransition.remove(encodedName);
374 
375     ServerName oldServerName = regionAssignments.put(hri, serverName);
376     if (!serverName.equals(oldServerName)) {
377       LOG.info("Onlined " + hri.getShortNameToLog() + " on " + serverName);
378       Set<HRegionInfo> regions = serverHoldings.get(serverName);
379       if (regions == null) {
380         regions = new HashSet<HRegionInfo>();
381         serverHoldings.put(serverName, regions);
382       }
383       regions.add(hri);
384       if (oldServerName != null) {
385         LOG.info("Offlined " + hri.getShortNameToLog() + " from " + oldServerName);
386         Set<HRegionInfo> oldRegions = serverHoldings.get(oldServerName);
387         oldRegions.remove(hri);
388         if (oldRegions.isEmpty()) {
389           serverHoldings.remove(oldServerName);
390         }
391       }
392     }
393   }
394 
395   /**
396    * A dead server's hlogs have been split so that all the regions
397    * used to be open on it can be safely assigned now. Mark them assignable.
398    */
399   public synchronized void logSplit(final ServerName serverName) {
400     for (Iterator<Map.Entry<String, ServerName>> it
401         = lastAssignments.entrySet().iterator(); it.hasNext();) {
402       Map.Entry<String, ServerName> e = it.next();
403       if (e.getValue().equals(serverName)) {
404         it.remove();
405       }
406     }
407     long now = System.currentTimeMillis();
408     processedServers.put(serverName, Long.valueOf(now));
409     Configuration conf = server.getConfiguration();
410     long obsoleteTime = conf.getLong(LOG_SPLIT_TIME, DEFAULT_LOG_SPLIT_TIME);
411     // Doesn't have to be very accurate about the clean up time
412     if (now > lastProcessedServerCleanTime + obsoleteTime) {
413       lastProcessedServerCleanTime = now;
414       long cutoff = now - obsoleteTime;
415       for (Iterator<Map.Entry<ServerName, Long>> it
416           = processedServers.entrySet().iterator(); it.hasNext();) {
417         Map.Entry<ServerName, Long> e = it.next();
418         if (e.getValue().longValue() < cutoff) {
419           it.remove();
420         }
421       }
422     }
423   }
424 
425   /**
426    * Log split is done for a given region, so it is assignable now.
427    */
428   public synchronized void logSplit(final HRegionInfo region) {
429     lastAssignments.remove(region.getEncodedName());
430   }
431 
432   /**
433    * A region is offline, won't be in transition any more.
434    */
435   public void regionOffline(final HRegionInfo hri) {
436     regionOffline(hri, null);
437   }
438 
439   /**
440    * A region is offline, won't be in transition any more. Its state
441    * should be the specified expected state, which can only be
442    * Split/Merged/Offline/null(=Offline)/SplittingNew/MergingNew.
443    */
444   public synchronized void regionOffline(
445       final HRegionInfo hri, final State expectedState) {
446     Preconditions.checkArgument(expectedState == null
447       || RegionState.isUnassignable(expectedState),
448         "Offlined region should not be " + expectedState);
449     String encodedName = hri.getEncodedName();
450     State newState =
451       expectedState == null ? State.OFFLINE : expectedState;
452     updateRegionState(hri, newState);
453     regionsInTransition.remove(encodedName);
454 
455     ServerName oldServerName = regionAssignments.remove(hri);
456     if (oldServerName != null) {
457       LOG.info("Offlined " + hri.getShortNameToLog() + " from " + oldServerName);
458       Set<HRegionInfo> oldRegions = serverHoldings.get(oldServerName);
459       oldRegions.remove(hri);
460       if (oldRegions.isEmpty()) {
461         serverHoldings.remove(oldServerName);
462       }
463     }
464   }
465 
466   /**
467    * A server is offline, all regions on it are dead.
468    */
469   public synchronized List<HRegionInfo> serverOffline(
470       final ZooKeeperWatcher watcher, final ServerName sn) {
471     // Offline all regions on this server not already in transition.
472     List<HRegionInfo> rits = new ArrayList<HRegionInfo>();
473     Set<HRegionInfo> assignedRegions = serverHoldings.get(sn);
474     if (assignedRegions == null) {
475       assignedRegions = new HashSet<HRegionInfo>();
476     }
477 
478     // Offline regions outside the loop to avoid ConcurrentModificationException
479     Set<HRegionInfo> regionsToOffline = new HashSet<HRegionInfo>();
480     for (HRegionInfo region : assignedRegions) {
481       // Offline open regions, no need to offline if SPLIT/MERGED/OFFLINE
482       if (isRegionOnline(region)) {
483         regionsToOffline.add(region);
484       } else {
485         if (isRegionInState(region, State.SPLITTING, State.MERGING)) {
486           LOG.debug("Offline splitting/merging region " + getRegionState(region));
487           try {
488             // Delete the ZNode if exists
489             ZKAssign.deleteNodeFailSilent(watcher, region);
490             regionsToOffline.add(region);
491           } catch (KeeperException ke) {
492             server.abort("Unexpected ZK exception deleting node " + region, ke);
493           }
494         }
495       }
496     }
497 
498     for (HRegionInfo hri : regionsToOffline) {
499       regionOffline(hri);
500     }
501 
502     for (RegionState state : regionsInTransition.values()) {
503       HRegionInfo hri = state.getRegion();
504       if (assignedRegions.contains(hri)) {
505         // Region is open on this region server, but in transition.
506         // This region must be moving away from this server, or splitting/merging.
507         // SSH will handle it, either skip assigning, or re-assign.
508         LOG.info("Transitioning " + state + " will be handled by SSH for " + sn);
509       } else if (sn.equals(state.getServerName())) {
510         // Region is in transition on this region server, and this
511         // region is not open on this server. So the region must be
512         // moving to this server from another one (i.e. opening or
513         // pending open on this server, was open on another one.
514         // Offline state is also kind of pending open if the region is in
515         // transition. The region could be in failed_close state too if we have
516         // tried several times to open it while this region server is not reachable)
517         if (state.isPendingOpenOrOpening() || state.isFailedClose() || state.isOffline()) {
518           LOG.info("Found region in " + state + " to be reassigned by SSH for " + sn);
519           rits.add(hri);
520         } else {
521           LOG.warn("THIS SHOULD NOT HAPPEN: unexpected " + state);
522         }
523       }
524     }
525 
526     this.notifyAll();
527     return rits;
528   }
529 
530   /**
531    * Gets the online regions of the specified table.
532    * This method looks at the in-memory state.  It does not go to <code>hbase:meta</code>.
533    * Only returns <em>online</em> regions.  If a region on this table has been
534    * closed during a disable, etc., it will be included in the returned list.
535    * So, the returned list may not necessarily be ALL regions in this table, its
536    * all the ONLINE regions in the table.
537    * @param tableName
538    * @return Online regions from <code>tableName</code>
539    */
540   public synchronized List<HRegionInfo> getRegionsOfTable(TableName tableName) {
541     List<HRegionInfo> tableRegions = new ArrayList<HRegionInfo>();
542     // boundary needs to have table's name but regionID 0 so that it is sorted
543     // before all table's regions.
544     HRegionInfo boundary = new HRegionInfo(tableName, null, null, false, 0L);
545     for (HRegionInfo hri: regionAssignments.tailMap(boundary).keySet()) {
546       if(!hri.getTable().equals(tableName)) break;
547       tableRegions.add(hri);
548     }
549     return tableRegions;
550   }
551 
552 
553   /**
554    * Wait on region to clear regions-in-transition.
555    * <p>
556    * If the region isn't in transition, returns immediately.  Otherwise, method
557    * blocks until the region is out of transition.
558    */
559   public synchronized void waitOnRegionToClearRegionsInTransition(
560       final HRegionInfo hri) throws InterruptedException {
561     if (!isRegionInTransition(hri)) return;
562 
563     while(!server.isStopped() && isRegionInTransition(hri)) {
564       RegionState rs = getRegionState(hri);
565       LOG.info("Waiting on " + rs + " to clear regions-in-transition");
566       waitForUpdate(100);
567     }
568 
569     if (server.isStopped()) {
570       LOG.info("Giving up wait on region in " +
571         "transition because stoppable.isStopped is set");
572     }
573   }
574 
575   /**
576    * Checking if a region was assigned to a server which is not online now.
577    * If so, we should hold re-assign this region till SSH has split its hlogs.
578    * Once logs are split, the last assignment of this region will be reset,
579    * which means a null last assignment server is ok for re-assigning.
580    *
581    * A region server could be dead but we don't know it yet. We may
582    * think it's online falsely. Therefore if a server is online, we still
583    * need to confirm it reachable and having the expected start code.
584    */
585   synchronized boolean wasRegionOnDeadServer(final String encodedName) {
586     ServerName server = lastAssignments.get(encodedName);
587     return isServerDeadAndNotProcessed(server);
588   }
589 
590   synchronized boolean isServerDeadAndNotProcessed(ServerName server) {
591     if (server == null) return false;
592     if (serverManager.isServerOnline(server)) {
593       String hostAndPort = server.getHostAndPort();
594       long startCode = server.getStartcode();
595       Long deadCode = deadServers.get(hostAndPort);
596       if (deadCode == null || startCode > deadCode.longValue()) {
597         if (serverManager.isServerReachable(server)) {
598           return false;
599         }
600         // The size of deadServers won't grow unbounded.
601         deadServers.put(hostAndPort, Long.valueOf(startCode));
602       }
603       // Watch out! If the server is not dead, the region could
604       // remain unassigned. That's why ServerManager#isServerReachable
605       // should use some retry.
606       //
607       // We cache this info since it is very unlikely for that
608       // instance to come back up later on. We don't want to expire
609       // the server since we prefer to let it die naturally.
610       LOG.warn("Couldn't reach online server " + server);
611     }
612     // Now, we know it's dead. Check if it's processed
613     return !processedServers.containsKey(server);
614   }
615 
616  /**
617    * Get the last region server a region was on for purpose of re-assignment,
618    * i.e. should the re-assignment be held back till log split is done?
619    */
620   synchronized ServerName getLastRegionServerOfRegion(final String encodedName) {
621     return lastAssignments.get(encodedName);
622   }
623 
624   synchronized void setLastRegionServerOfRegions(
625       final ServerName serverName, final List<HRegionInfo> regionInfos) {
626     for (HRegionInfo hri: regionInfos) {
627       setLastRegionServerOfRegion(serverName, hri.getEncodedName());
628     }
629   }
630 
631   synchronized void setLastRegionServerOfRegion(
632       final ServerName serverName, final String encodedName) {
633     lastAssignments.put(encodedName, serverName);
634   }
635 
636   /**
637    * Compute the average load across all region servers.
638    * Currently, this uses a very naive computation - just uses the number of
639    * regions being served, ignoring stats about number of requests.
640    * @return the average load
641    */
642   protected synchronized double getAverageLoad() {
643     int numServers = 0, totalLoad = 0;
644     for (Map.Entry<ServerName, Set<HRegionInfo>> e: serverHoldings.entrySet()) {
645       Set<HRegionInfo> regions = e.getValue();
646       ServerName serverName = e.getKey();
647       int regionCount = regions.size();
648       if (regionCount > 0 || serverManager.isServerOnline(serverName)) {
649         totalLoad += regionCount;
650         numServers++;
651       }
652     }
653     return numServers == 0 ? 0.0 :
654       (double)totalLoad / (double)numServers;
655   }
656 
657   /**
658    * This is an EXPENSIVE clone.  Cloning though is the safest thing to do.
659    * Can't let out original since it can change and at least the load balancer
660    * wants to iterate this exported list.  We need to synchronize on regions
661    * since all access to this.servers is under a lock on this.regions.
662    *
663    * @return A clone of current assignments by table.
664    */
665   protected Map<TableName, Map<ServerName, List<HRegionInfo>>>
666       getAssignmentsByTable() {
667     Map<TableName, Map<ServerName, List<HRegionInfo>>> result =
668       new HashMap<TableName, Map<ServerName,List<HRegionInfo>>>();
669     synchronized (this) {
670       if (!server.getConfiguration().getBoolean("hbase.master.loadbalance.bytable", false)) {
671         Map<ServerName, List<HRegionInfo>> svrToRegions =
672           new HashMap<ServerName, List<HRegionInfo>>(serverHoldings.size());
673         for (Map.Entry<ServerName, Set<HRegionInfo>> e: serverHoldings.entrySet()) {
674           svrToRegions.put(e.getKey(), new ArrayList<HRegionInfo>(e.getValue()));
675         }
676         result.put(TableName.valueOf("ensemble"), svrToRegions);
677       } else {
678         for (Map.Entry<ServerName, Set<HRegionInfo>> e: serverHoldings.entrySet()) {
679           for (HRegionInfo hri: e.getValue()) {
680             if (hri.isMetaRegion()) continue;
681             TableName tablename = hri.getTable();
682             Map<ServerName, List<HRegionInfo>> svrToRegions = result.get(tablename);
683             if (svrToRegions == null) {
684               svrToRegions = new HashMap<ServerName, List<HRegionInfo>>(serverHoldings.size());
685               result.put(tablename, svrToRegions);
686             }
687             List<HRegionInfo> regions = svrToRegions.get(e.getKey());
688             if (regions == null) {
689               regions = new ArrayList<HRegionInfo>();
690               svrToRegions.put(e.getKey(), regions);
691             }
692             regions.add(hri);
693           }
694         }
695       }
696     }
697 
698     Map<ServerName, ServerLoad>
699       onlineSvrs = serverManager.getOnlineServers();
700     // Take care of servers w/o assignments.
701     for (Map<ServerName, List<HRegionInfo>> map: result.values()) {
702       for (ServerName svr: onlineSvrs.keySet()) {
703         if (!map.containsKey(svr)) {
704           map.put(svr, new ArrayList<HRegionInfo>());
705         }
706       }
707     }
708     return result;
709   }
710 
711   protected synchronized RegionState getRegionState(final HRegionInfo hri) {
712     return regionStates.get(hri.getEncodedName());
713   }
714 
715   protected synchronized RegionState getRegionState(final String encodedName) {
716     return regionStates.get(encodedName);
717   }
718 
719   /**
720    * Get the HRegionInfo from cache, if not there, from the hbase:meta table
721    * @param  regionName
722    * @return HRegionInfo for the region
723    */
724   protected HRegionInfo getRegionInfo(final byte [] regionName) {
725     String encodedName = HRegionInfo.encodeRegionName(regionName);
726     RegionState regionState = regionStates.get(encodedName);
727     if (regionState != null) {
728       return regionState.getRegion();
729     }
730 
731     try {
732       Pair<HRegionInfo, ServerName> p =
733         MetaReader.getRegion(server.getCatalogTracker(), regionName);
734       HRegionInfo hri = p == null ? null : p.getFirst();
735       if (hri != null) {
736         createRegionState(hri);
737       }
738       return hri;
739     } catch (IOException e) {
740       server.abort("Aborting because error occoured while reading "
741         + Bytes.toStringBinary(regionName) + " from hbase:meta", e);
742       return null;
743     }
744   }
745 }