View Javadoc

1   /**
2    * Copyright 2010 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
19   */
20  package org.apache.hadoop.hbase.master;
21  
22  import java.io.IOException;
23  import java.util.ArrayList;
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.SortedMap;
33  import java.util.TreeMap;
34  import java.util.concurrent.ConcurrentSkipListMap;
35  import java.util.concurrent.atomic.AtomicInteger;
36  import java.util.concurrent.atomic.AtomicReference;
37  
38  import org.apache.commons.logging.Log;
39  import org.apache.commons.logging.LogFactory;
40  import org.apache.hadoop.conf.Configuration;
41  import org.apache.hadoop.fs.FileStatus;
42  import org.apache.hadoop.fs.Path;
43  import org.apache.hadoop.fs.PathFilter;
44  import org.apache.hadoop.hbase.HConstants;
45  import org.apache.hadoop.hbase.HMsg;
46  import org.apache.hadoop.hbase.HRegionInfo;
47  import org.apache.hadoop.hbase.HServerAddress;
48  import org.apache.hadoop.hbase.HServerInfo;
49  import org.apache.hadoop.hbase.HServerLoad;
50  import org.apache.hadoop.hbase.client.Put;
51  import org.apache.hadoop.hbase.ipc.HRegionInterface;
52  import org.apache.hadoop.hbase.regionserver.HRegion;
53  import org.apache.hadoop.hbase.util.Bytes;
54  import org.apache.hadoop.hbase.util.Pair;
55  import org.apache.hadoop.hbase.util.Threads;
56  import org.apache.hadoop.hbase.util.Writables;
57  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWrapper;
58  
59  /**
60   * Class to manage assigning regions to servers, state of root and meta, etc.
61   */
62  public class RegionManager {
63    protected static final Log LOG = LogFactory.getLog(RegionManager.class);
64  
65    private AtomicReference<HServerAddress> rootRegionLocation =
66      new AtomicReference<HServerAddress>(null);
67  
68    private final RootScanner rootScannerThread;
69    final MetaScanner metaScannerThread;
70  
71    /** Set by root scanner to indicate the number of meta regions */
72    private final AtomicInteger numberOfMetaRegions = new AtomicInteger();
73  
74    /** These are the online meta regions */
75    private final NavigableMap<byte [], MetaRegion> onlineMetaRegions =
76      new ConcurrentSkipListMap<byte [], MetaRegion>(Bytes.BYTES_COMPARATOR);
77  
78    private static final byte[] OVERLOADED = Bytes.toBytes("Overloaded");
79  
80    private static final byte [] META_REGION_PREFIX = Bytes.toBytes(".META.,");
81  
82    /**
83     * Map of region name to RegionState for regions that are in transition such as
84     *
85     * unassigned -> pendingOpen -> open
86     * closing -> pendingClose -> closed; if (closed && !offline) -> unassigned
87     *
88     * At the end of a transition, removeRegion is used to remove the region from
89     * the map (since it is no longer in transition)
90     *
91     * Note: Needs to be SortedMap so we can specify a comparator
92     *
93     * @see RegionState inner-class below
94     */
95     final SortedMap<String, RegionState> regionsInTransition =
96      Collections.synchronizedSortedMap(new TreeMap<String, RegionState>());
97  
98    // How many regions to assign a server at a time.
99    private final int maxAssignInOneGo;
100 
101   final HMaster master;
102   private final LoadBalancer loadBalancer;
103 
104   /** Set of regions to split. */
105   private final SortedMap<byte[], Pair<HRegionInfo,HServerAddress>>
106     regionsToSplit = Collections.synchronizedSortedMap(
107         new TreeMap<byte[],Pair<HRegionInfo,HServerAddress>>
108         (Bytes.BYTES_COMPARATOR));
109   /** Set of regions to compact. */
110   private final SortedMap<byte[], Pair<HRegionInfo,HServerAddress>>
111     regionsToCompact = Collections.synchronizedSortedMap(
112         new TreeMap<byte[],Pair<HRegionInfo,HServerAddress>>
113         (Bytes.BYTES_COMPARATOR));
114   /** Set of regions to major compact. */
115   private final SortedMap<byte[], Pair<HRegionInfo,HServerAddress>>
116     regionsToMajorCompact = Collections.synchronizedSortedMap(
117         new TreeMap<byte[],Pair<HRegionInfo,HServerAddress>>
118         (Bytes.BYTES_COMPARATOR));
119   /** Set of regions to flush. */
120   private final SortedMap<byte[], Pair<HRegionInfo,HServerAddress>>
121     regionsToFlush = Collections.synchronizedSortedMap(
122         new TreeMap<byte[],Pair<HRegionInfo,HServerAddress>>
123         (Bytes.BYTES_COMPARATOR));
124   private final int zooKeeperNumRetries;
125   private final int zooKeeperPause;
126 
127   RegionManager(HMaster master) {
128     Configuration conf = master.getConfiguration();
129 
130     this.master = master;
131     this.maxAssignInOneGo = conf.getInt("hbase.regions.percheckin", 10);
132     this.loadBalancer = new LoadBalancer(conf);
133 
134     // The root region
135     rootScannerThread = new RootScanner(master);
136 
137     // Scans the meta table
138     metaScannerThread = new MetaScanner(master);
139 
140     zooKeeperNumRetries = conf.getInt(HConstants.ZOOKEEPER_RETRIES,
141         HConstants.DEFAULT_ZOOKEEPER_RETRIES);
142     zooKeeperPause = conf.getInt(HConstants.ZOOKEEPER_PAUSE,
143         HConstants.DEFAULT_ZOOKEEPER_PAUSE);
144 
145     reassignRootRegion();
146   }
147 
148   void start() {
149     Threads.setDaemonThreadRunning(rootScannerThread,
150       "RegionManager.rootScanner");
151     Threads.setDaemonThreadRunning(metaScannerThread,
152       "RegionManager.metaScanner");
153   }
154 
155   void unsetRootRegion() {
156     synchronized (regionsInTransition) {
157       rootRegionLocation.set(null);
158       regionsInTransition.remove(
159           HRegionInfo.ROOT_REGIONINFO.getRegionNameAsString());
160       LOG.info("-ROOT- region unset (but not set to be reassigned)");
161     }
162   }
163 
164   void reassignRootRegion() {
165     unsetRootRegion();
166     if (!master.getShutdownRequested().get()) {
167       synchronized (regionsInTransition) {
168         RegionState s = new RegionState(HRegionInfo.ROOT_REGIONINFO,
169             RegionState.State.UNASSIGNED);
170         regionsInTransition.put(
171             HRegionInfo.ROOT_REGIONINFO.getRegionNameAsString(), s);
172         LOG.info("ROOT inserted into regionsInTransition");
173       }
174     }
175   }
176 
177   /*
178    * Assigns regions to region servers attempting to balance the load across
179    * all region servers. Note that no synchronization is necessary as the caller
180    * (ServerManager.processMsgs) already owns the monitor for the RegionManager.
181    *
182    * @param info
183    * @param mostLoadedRegions
184    * @param returnMsgs
185    */
186   void assignRegions(HServerInfo info, HRegionInfo[] mostLoadedRegions,
187       ArrayList<HMsg> returnMsgs) {
188     HServerLoad thisServersLoad = info.getLoad();
189     boolean isSingleServer = this.master.numServers() == 1;
190 
191     // figure out what regions need to be assigned and aren't currently being
192     // worked on elsewhere.
193     Set<RegionState> regionsToAssign =
194       regionsAwaitingAssignment(info.getServerAddress(), isSingleServer);
195     if (regionsToAssign.size() == 0) {
196       // There are no regions waiting to be assigned.
197       this.loadBalancer.loadBalancing(info, mostLoadedRegions, returnMsgs);
198     } else {
199       // if there's only one server, just give it all the regions
200       if (isSingleServer) {
201         assignRegionsToOneServer(regionsToAssign, info, returnMsgs);
202       } else {
203         // otherwise, give this server a few regions taking into account the
204         // load of all the other servers.
205         assignRegionsToMultipleServers(thisServersLoad, regionsToAssign,
206             info, returnMsgs);
207       }
208     }
209   }
210 
211   /*
212    * Make region assignments taking into account multiple servers' loads.
213    *
214    * Note that no synchronization is needed while we iterate over
215    * regionsInTransition because this method is only called by assignRegions
216    * whose caller owns the monitor for RegionManager
217    *
218    * TODO: This code is unintelligible.  REWRITE. Add TESTS! St.Ack 09/30/2009
219    * @param thisServersLoad
220    * @param regionsToAssign
221    * @param info
222    * @param returnMsgs
223    */
224   private void assignRegionsToMultipleServers(final HServerLoad thisServersLoad,
225     final Set<RegionState> regionsToAssign, final HServerInfo info,
226     final ArrayList<HMsg> returnMsgs) {
227     boolean isMetaAssign = false;
228     for (RegionState s : regionsToAssign) {
229       if (s.getRegionInfo().isMetaRegion())
230         isMetaAssign = true;
231     }
232     int nRegionsToAssign = regionsToAssign.size();
233     int otherServersRegionsCount =
234       regionsToGiveOtherServers(nRegionsToAssign, thisServersLoad);
235     nRegionsToAssign -= otherServersRegionsCount;
236     if (nRegionsToAssign > 0 || isMetaAssign) {
237       LOG.debug("Assigning for " + info + ": total nregions to assign=" +
238         nRegionsToAssign + ", regions to give other servers than this=" +
239         otherServersRegionsCount + ", isMetaAssign=" + isMetaAssign);
240 
241       // See how many we can assign before this server becomes more heavily
242       // loaded than the next most heavily loaded server.
243       HServerLoad heavierLoad = new HServerLoad();
244       int nservers = computeNextHeaviestLoad(thisServersLoad, heavierLoad);
245       int nregions = 0;
246       // Advance past any less-loaded servers
247       for (HServerLoad load = new HServerLoad(thisServersLoad);
248       load.compareTo(heavierLoad) <= 0 && nregions < nRegionsToAssign;
249       load.setNumberOfRegions(load.getNumberOfRegions() + 1), nregions++) {
250         // continue;
251       }
252       if (nregions < nRegionsToAssign) {
253         // There are some more heavily loaded servers
254         // but we can't assign all the regions to this server.
255         if (nservers > 0) {
256           // There are other servers that can share the load.
257           // Split regions that need assignment across the servers.
258           nregions = (int) Math.ceil((1.0 * nRegionsToAssign)/(1.0 * nservers));
259         } else {
260           // No other servers with same load.
261           // Split regions over all available servers
262           nregions = (int) Math.ceil((1.0 * nRegionsToAssign)/
263               (1.0 * master.getServerManager().numServers()));
264         }
265       } else {
266         // Assign all regions to this server
267         nregions = nRegionsToAssign;
268       }
269       LOG.debug("Assigning " + info + " " + nregions + " regions");
270       assignRegions(regionsToAssign, nregions, info, returnMsgs);
271     }
272   }
273 
274   /*
275    * Assign <code>nregions</code> regions.
276    * @param regionsToAssign
277    * @param nregions
278    * @param info
279    * @param returnMsgs
280    */
281   private void assignRegions(final Set<RegionState> regionsToAssign,
282       final int nregions, final HServerInfo info,
283       final ArrayList<HMsg> returnMsgs) {
284     int count = nregions;
285     if (count > this.maxAssignInOneGo) {
286       count = this.maxAssignInOneGo;
287     }
288     for (RegionState s: regionsToAssign) {
289       doRegionAssignment(s, info, returnMsgs);
290       if (--count <= 0) {
291         break;
292       }
293     }
294   }
295 
296   /*
297    * Assign all to the only server. An unlikely case but still possible.
298    *
299    * Note that no synchronization is needed on regionsInTransition while
300    * iterating on it because the only caller is assignRegions whose caller owns
301    * the monitor for RegionManager
302    *
303    * @param regionsToAssign
304    * @param serverName
305    * @param returnMsgs
306    */
307   private void assignRegionsToOneServer(final Set<RegionState> regionsToAssign,
308       final HServerInfo info, final ArrayList<HMsg> returnMsgs) {
309     for (RegionState s: regionsToAssign) {
310       doRegionAssignment(s, info, returnMsgs);
311     }
312   }
313 
314   /*
315    * Do single region assignment.
316    * @param rs
317    * @param sinfo
318    * @param returnMsgs
319    */
320   private void doRegionAssignment(final RegionState rs,
321       final HServerInfo sinfo, final ArrayList<HMsg> returnMsgs) {
322     String regionName = rs.getRegionInfo().getRegionNameAsString();
323     LOG.info("Assigning region " + regionName + " to " + sinfo.getServerName());
324     rs.setPendingOpen(sinfo.getServerName());
325     synchronized (this.regionsInTransition) {
326       this.regionsInTransition.put(regionName, rs);
327     }
328 
329     returnMsgs.add(new HMsg(HMsg.Type.MSG_REGION_OPEN, rs.getRegionInfo()));
330   }
331 
332   /*
333    * @param nRegionsToAssign
334    * @param thisServersLoad
335    * @return How many regions should go to servers other than this one; i.e.
336    * more lightly loaded servers
337    */
338   private int regionsToGiveOtherServers(final int numUnassignedRegions,
339     final HServerLoad thisServersLoad) {
340     SortedMap<HServerLoad, Set<String>> lightServers =
341       new TreeMap<HServerLoad, Set<String>>();
342     this.master.getLightServers(thisServersLoad, lightServers);
343     // Examine the list of servers that are more lightly loaded than this one.
344     // Pretend that we will assign regions to these more lightly loaded servers
345     // until they reach load equal with ours. Then, see how many regions are left
346     // unassigned. That is how many regions we should assign to this server.
347     int nRegions = 0;
348     for (Map.Entry<HServerLoad, Set<String>> e: lightServers.entrySet()) {
349       HServerLoad lightLoad = new HServerLoad(e.getKey());
350       do {
351         lightLoad.setNumberOfRegions(lightLoad.getNumberOfRegions() + 1);
352         nRegions += 1;
353       } while (lightLoad.compareTo(thisServersLoad) <= 0
354           && nRegions < numUnassignedRegions);
355       nRegions *= e.getValue().size();
356       if (nRegions >= numUnassignedRegions) {
357         break;
358       }
359     }
360     return nRegions;
361   }
362 
363   /*
364    * Get the set of regions that should be assignable in this pass.
365    *
366    * Note that no synchronization on regionsInTransition is needed because the
367    * only caller (assignRegions, whose caller is ServerManager.processMsgs) owns
368    * the monitor for RegionManager
369    */
370   private Set<RegionState> regionsAwaitingAssignment(HServerAddress addr,
371                                                      boolean isSingleServer) {
372     // set of regions we want to assign to this server
373     Set<RegionState> regionsToAssign = new HashSet<RegionState>();
374 
375     boolean isMetaServer = isMetaServer(addr);
376     RegionState rootState = null;
377     // Handle if root is unassigned... only assign root if root is offline.
378     synchronized (this.regionsInTransition) {
379       rootState = regionsInTransition.get(HRegionInfo.ROOT_REGIONINFO.getRegionNameAsString());
380     }
381     if (rootState != null && rootState.isUnassigned()) {
382       // make sure root isnt assigned here first.
383       // if so return 'empty list'
384       // by definition there is no way this could be a ROOT region (since it's
385       // unassigned) so just make sure it isn't hosting META regions (unless
386       // it's the only server left).
387       if (!isMetaServer || isSingleServer) {
388         regionsToAssign.add(rootState);
389       }
390       return regionsToAssign;
391     }
392 
393     // Look over the set of regions that aren't currently assigned to
394     // determine which we should assign to this server.
395     boolean reassigningMetas = numberOfMetaRegions.get() != onlineMetaRegions.size();
396     boolean isMetaOrRoot = isMetaServer || isRootServer(addr);
397     if (reassigningMetas && isMetaOrRoot && !isSingleServer) {
398       return regionsToAssign; // dont assign anything to this server.
399     }
400     synchronized (this.regionsInTransition) {
401       for (RegionState s: regionsInTransition.values()) {
402         HRegionInfo i = s.getRegionInfo();
403         if (i == null) {
404           continue;
405         }
406         if (reassigningMetas &&
407             !i.isMetaRegion()) {
408           // Can't assign user regions until all meta regions have been assigned
409           // and are on-line
410           continue;
411         }
412         if (!i.isMetaRegion() &&
413             !master.getServerManager().canAssignUserRegions()) {
414           LOG.debug("user region " + i.getRegionNameAsString() +
415             " is in transition but not enough servers yet");
416           continue;
417         }
418         if (s.isUnassigned()) {
419           regionsToAssign.add(s);
420         }
421       }
422     }
423     return regionsToAssign;
424   }
425 
426   /*
427    * Figure out the load that is next highest amongst all regionservers. Also,
428    * return how many servers exist at that load.
429    */
430   private int computeNextHeaviestLoad(HServerLoad referenceLoad,
431     HServerLoad heavierLoad) {
432 
433     SortedMap<HServerLoad, Set<String>> heavyServers =
434       new TreeMap<HServerLoad, Set<String>>();
435     synchronized (master.getLoadToServers()) {
436       heavyServers.putAll(
437         master.getLoadToServers().tailMap(referenceLoad));
438     }
439     int nservers = 0;
440     for (Map.Entry<HServerLoad, Set<String>> e : heavyServers.entrySet()) {
441       Set<String> servers = e.getValue();
442       nservers += servers.size();
443       if (e.getKey().compareTo(referenceLoad) == 0) {
444         // This is the load factor of the server we are considering
445         nservers -= 1;
446         continue;
447       }
448 
449       // If we get here, we are at the first load entry that is a
450       // heavier load than the server we are considering
451       heavierLoad.setNumberOfRequests(e.getKey().getNumberOfRequests());
452       heavierLoad.setNumberOfRegions(e.getKey().getNumberOfRegions());
453       break;
454     }
455     return nservers;
456   }
457 
458   /*
459    * The server checking in right now is overloaded. We will tell it to close
460    * some or all of its most loaded regions, allowing it to reduce its load.
461    * The closed regions will then get picked up by other underloaded machines.
462    *
463    * Note that no synchronization is needed because the only caller
464    * (assignRegions) whose caller owns the monitor for RegionManager
465    */
466   void unassignSomeRegions(final HServerInfo info,
467       int numRegionsToClose, final HRegionInfo[] mostLoadedRegions,
468       ArrayList<HMsg> returnMsgs) {
469     LOG.debug("Unassigning " + numRegionsToClose + " regions from " +
470       info.getServerName());
471     int regionIdx = 0;
472     int regionsClosed = 0;
473     int skipped = 0;
474     while (regionsClosed < numRegionsToClose &&
475         regionIdx < mostLoadedRegions.length) {
476       HRegionInfo currentRegion = mostLoadedRegions[regionIdx];
477       regionIdx++;
478       // skip the region if it's meta or root
479       if (currentRegion.isRootRegion() || currentRegion.isMetaTable()) {
480         continue;
481       }
482       final String regionName = currentRegion.getRegionNameAsString();
483       if (regionIsInTransition(regionName)) {
484         skipped++;
485         continue;
486       }
487       if (LOG.isDebugEnabled()) {
488         LOG.debug("Going to close region " + regionName);
489       }
490       // make a message to close the region
491       returnMsgs.add(new HMsg(HMsg.Type.MSG_REGION_CLOSE, currentRegion,
492         OVERLOADED));
493       // mark the region as closing
494       setClosing(info.getServerName(), currentRegion, false);
495       setPendingClose(regionName);
496       // increment the count of regions we've marked
497       regionsClosed++;
498     }
499     LOG.info("Skipped assigning " + skipped + " region(s) to " +
500       info.getServerName() + "because already in transition");
501   }
502 
503   /*
504    * PathFilter that accepts hbase tables only.
505    */
506   static class TableDirFilter implements PathFilter {
507     public boolean accept(final Path path) {
508       // skip the region servers' log dirs && version file
509       // HBASE-1112 want to separate the log dirs from table's data dirs by a
510       // special character.
511       final String pathname = path.getName();
512       return (!pathname.equals(HConstants.HREGION_LOGDIR_NAME)
513               && !pathname.equals(HConstants.VERSION_FILE_NAME));
514     }
515 
516   }
517 
518   /*
519    * PathFilter that accepts all but compaction.dir names.
520    */
521   static class RegionDirFilter implements PathFilter {
522     public boolean accept(Path path) {
523       return !path.getName().equals(HConstants.HREGION_COMPACTIONDIR_NAME);
524     }
525   }
526 
527   /**
528    * @return the rough number of the regions on fs
529    * Note: this method simply counts the regions on fs by accumulating all the dirs
530    * in each table dir (${HBASE_ROOT}/$TABLE) and skipping logfiles, compaction dirs.
531    * @throws IOException
532    */
533   public int countRegionsOnFS() throws IOException {
534     int regions = 0;
535     FileStatus [] tableDirs =
536       this.master.getFileSystem().listStatus(this.master.getRootDir(), new TableDirFilter());
537     FileStatus[] regionDirs;
538     RegionDirFilter rdf = new RegionDirFilter();
539     for(FileStatus tabledir : tableDirs) {
540       if(tabledir.isDir()) {
541         regionDirs = this.master.getFileSystem().listStatus(tabledir.getPath(), rdf);
542         regions += regionDirs.length;
543       }
544     }
545     return regions;
546   }
547 
548   /**
549    * @return Read-only map of online regions.
550    */
551   public Map<byte [], MetaRegion> getOnlineMetaRegions() {
552     synchronized (onlineMetaRegions) {
553       return Collections.unmodifiableMap(onlineMetaRegions);
554     }
555   }
556 
557   public boolean metaRegionsInTransition() {
558     synchronized (onlineMetaRegions) {
559       for (MetaRegion metaRegion : onlineMetaRegions.values()) {
560         String regionName = Bytes.toString(metaRegion.getRegionName());
561         if (regionIsInTransition(regionName)) {
562           return true;
563         }
564       }
565     }
566     return false;
567   }
568 
569   /**
570    * Return a map of the regions in transition on a server.
571    * Returned map entries are region name -> RegionState
572    */
573   Map<String, RegionState> getRegionsInTransitionOnServer(String serverName) {
574     Map<String, RegionState> ret = new HashMap<String, RegionState>();
575     synchronized (regionsInTransition) {
576       for (Map.Entry<String, RegionState> entry : regionsInTransition.entrySet()) {
577         RegionState rs = entry.getValue();
578         if (serverName.equals(rs.getServerName())) {
579           ret.put(entry.getKey(), rs);
580         }
581       }
582     }
583     return ret;
584   }
585 
586   /**
587    * Stop the root and meta scanners so that the region servers serving meta
588    * regions can shut down.
589    */
590   public void stopScanners() {
591     this.rootScannerThread.interruptAndStop();
592     this.metaScannerThread.interruptAndStop();
593   }
594 
595   /** Stop the region assigner */
596   public void stop() {
597     try {
598       if (rootScannerThread.isAlive()) {
599         rootScannerThread.join();       // Wait for the root scanner to finish.
600       }
601     } catch (Exception iex) {
602       LOG.warn("root scanner", iex);
603     }
604     try {
605       if (metaScannerThread.isAlive()) {
606         metaScannerThread.join();       // Wait for meta scanner to finish.
607       }
608     } catch(Exception iex) {
609       LOG.warn("meta scanner", iex);
610     }
611     master.getZooKeeperWrapper().clearRSDirectory();
612     master.getZooKeeperWrapper().close();
613   }
614 
615   /**
616    * Block until meta regions are online or we're shutting down.
617    * @return true if we found meta regions, false if we're closing.
618    */
619   public boolean areAllMetaRegionsOnline() {
620     synchronized (onlineMetaRegions) {
621       return (rootRegionLocation.get() != null &&
622           numberOfMetaRegions.get() == onlineMetaRegions.size());
623     }
624   }
625 
626   /**
627    * Search our map of online meta regions to find the first meta region that
628    * should contain a pointer to <i>newRegion</i>.
629    * @param newRegion
630    * @return MetaRegion where the newRegion should live
631    */
632   public MetaRegion getFirstMetaRegionForRegion(HRegionInfo newRegion) {
633     synchronized (onlineMetaRegions) {
634       if (onlineMetaRegions.size() == 0) {
635         return null;
636       } else if (onlineMetaRegions.size() == 1) {
637         return onlineMetaRegions.get(onlineMetaRegions.firstKey());
638       } else {
639         if (onlineMetaRegions.containsKey(newRegion.getRegionName())) {
640           return onlineMetaRegions.get(newRegion.getRegionName());
641         }
642         return onlineMetaRegions.get(onlineMetaRegions.headMap(
643             newRegion.getRegionName()).lastKey());
644       }
645     }
646   }
647 
648   /**
649    * Get a set of all the meta regions that contain info about a given table.
650    * @param tableName Table you need to know all the meta regions for
651    * @return set of MetaRegion objects that contain the table
652    * @throws NotAllMetaRegionsOnlineException
653    */
654   public Set<MetaRegion> getMetaRegionsForTable(byte [] tableName)
655   throws NotAllMetaRegionsOnlineException {
656     byte [] firstMetaRegion = null;
657     Set<MetaRegion> metaRegions = new HashSet<MetaRegion>();
658     if (Bytes.equals(tableName, HConstants.META_TABLE_NAME)) {
659       if (rootRegionLocation.get() == null) {
660         throw new NotAllMetaRegionsOnlineException(
661             Bytes.toString(HConstants.ROOT_TABLE_NAME));
662       }
663       metaRegions.add(new MetaRegion(rootRegionLocation.get(),
664           HRegionInfo.ROOT_REGIONINFO));
665     } else {
666       if (!areAllMetaRegionsOnline()) {
667         throw new NotAllMetaRegionsOnlineException();
668       }
669       synchronized (onlineMetaRegions) {
670         if (onlineMetaRegions.size() == 1) {
671           firstMetaRegion = onlineMetaRegions.firstKey();
672         } else if (onlineMetaRegions.containsKey(tableName)) {
673           firstMetaRegion = tableName;
674         } else {
675           firstMetaRegion = onlineMetaRegions.headMap(tableName).lastKey();
676         }
677         metaRegions.addAll(onlineMetaRegions.tailMap(firstMetaRegion).values());
678       }
679     }
680     return metaRegions;
681   }
682 
683   /**
684    * Get metaregion that would host passed in row.
685    * @param row Row need to know all the meta regions for
686    * @return MetaRegion for passed row.
687    * @throws NotAllMetaRegionsOnlineException
688    */
689   public MetaRegion getMetaRegionForRow(final byte [] row)
690   throws NotAllMetaRegionsOnlineException {
691     if (!areAllMetaRegionsOnline()) {
692       throw new NotAllMetaRegionsOnlineException();
693     }
694     // Row might be in -ROOT- table.  If so, return -ROOT- region.
695     int prefixlen = META_REGION_PREFIX.length;
696     if (row.length > prefixlen &&
697      Bytes.compareTo(META_REGION_PREFIX, 0, prefixlen, row, 0, prefixlen) == 0) {
698     	return new MetaRegion(this.master.getRegionManager().getRootRegionLocation(),
699     	  HRegionInfo.ROOT_REGIONINFO);
700     }
701     return this.onlineMetaRegions.floorEntry(row).getValue();
702   }
703 
704   /**
705    * Create a new HRegion, put a row for it into META (or ROOT), and mark the
706    * new region unassigned so that it will get assigned to a region server.
707    * @param newRegion HRegionInfo for the region to create
708    * @param server server hosting the META (or ROOT) region where the new
709    * region needs to be noted
710    * @param metaRegionName name of the meta region where new region is to be
711    * written
712    * @throws IOException
713    */
714   public void createRegion(HRegionInfo newRegion, HRegionInterface server,
715       byte [] metaRegionName)
716   throws IOException {
717     // 2. Create the HRegion
718     HRegion region = HRegion.createHRegion(newRegion, this.master.getRootDir(),
719       master.getConfiguration());
720 
721     // 3. Insert into meta
722     HRegionInfo info = region.getRegionInfo();
723     byte [] regionName = region.getRegionName();
724 
725     Put put = new Put(regionName);
726     put.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
727         Writables.getBytes(info));
728     server.put(metaRegionName, put);
729 
730     // 4. Close the new region to flush it to disk.  Close its log file too.
731     region.close();
732     region.getLog().closeAndDelete();
733 
734     // 5. Get it assigned to a server
735     setUnassigned(info, true);
736   }
737 
738   /**
739    * Set a MetaRegion as online.
740    * @param metaRegion
741    */
742   public void putMetaRegionOnline(MetaRegion metaRegion) {
743     onlineMetaRegions.put(metaRegion.getStartKey(), metaRegion);
744   }
745 
746   /**
747    * Get a list of online MetaRegions
748    * @return list of MetaRegion objects
749    */
750   public List<MetaRegion> getListOfOnlineMetaRegions() {
751     List<MetaRegion> regions;
752     synchronized(onlineMetaRegions) {
753       regions = new ArrayList<MetaRegion>(onlineMetaRegions.values());
754     }
755     return regions;
756   }
757 
758   /**
759    * Count of online meta regions
760    * @return count of online meta regions
761    */
762   public int numOnlineMetaRegions() {
763     return onlineMetaRegions.size();
764   }
765 
766   /**
767    * Check if a meta region is online by its name
768    * @param startKey name of the meta region to check
769    * @return true if the region is online, false otherwise
770    */
771   public boolean isMetaRegionOnline(byte [] startKey) {
772     return onlineMetaRegions.containsKey(startKey);
773   }
774 
775   /**
776    * Set an online MetaRegion offline - remove it from the map.
777    * @param startKey Startkey to use finding region to remove.
778    * @return the MetaRegion that was taken offline.
779    */
780   public MetaRegion offlineMetaRegionWithStartKey(byte [] startKey) {
781     LOG.info("META region whose startkey is " + Bytes.toString(startKey) +
782       " removed from onlineMetaRegions");
783     return onlineMetaRegions.remove(startKey);
784   }
785 
786   public boolean isRootServer(HServerAddress server) {
787     return this.master.getRegionManager().getRootRegionLocation() != null &&
788       server.equals(master.getRegionManager().getRootRegionLocation());
789   }
790 
791   /**
792    * Returns the list of byte[] start-keys for any .META. regions hosted
793    * on the indicated server.
794    *
795    * @param server server address
796    * @return list of meta region start-keys.
797    */
798   public List<byte[]> listMetaRegionsForServer(HServerAddress server) {
799     List<byte[]> metas = new ArrayList<byte[]>();
800     for ( MetaRegion region : onlineMetaRegions.values() ) {
801       if (server.equals(region.getServer())) {
802         metas.add(region.getStartKey());
803       }
804     }
805     return metas;
806   }
807 
808   /**
809    * Does this server have any META regions open on it, or any meta
810    * regions being assigned to it?
811    *
812    * @param server Server IP:port
813    * @return true if server has meta region assigned
814    */
815   public boolean isMetaServer(HServerAddress server) {
816     for ( MetaRegion region : onlineMetaRegions.values() ) {
817       if (server.equals(region.getServer())) {
818         return true;
819       }
820     }
821 
822     // This might be expensive, but we need to make sure we dont
823     // get double assignment to the same regionserver.
824     synchronized(regionsInTransition) {
825       for (RegionState s : regionsInTransition.values()) {
826         if (s.getRegionInfo().isMetaRegion()
827             && !s.isUnassigned()
828             && s.getServerName() != null
829             && s.getServerName().equals(server.toString())) {
830           // TODO this code appears to be entirely broken, since
831           // server.toString() has no start code, but s.getServerName()
832           // does!
833           LOG.fatal("I DONT BELIEVE YOU WILL EVER SEE THIS!");
834           // Has an outstanding meta region to be assigned.
835           return true;
836         }
837       }
838     }
839     return false;
840   }
841 
842   /**
843    * Is this server assigned to transition the ROOT table. HBASE-1928
844    *
845    * @param server Server
846    * @return true if server is transitioning the ROOT table
847    */
848   public boolean isRootInTransitionOnThisServer(final String server) {
849     synchronized (this.regionsInTransition) {
850       for (RegionState s : regionsInTransition.values()) {
851         if (s.getRegionInfo().isRootRegion()
852             && !s.isUnassigned()
853             && s.getServerName() != null
854             && s.getServerName().equals(server)) {
855           // Has an outstanding root region to be assigned.
856           return true;
857         }
858       }
859     }
860     return false;
861   }
862 
863   /**
864    * Is this server assigned to transition a META table. HBASE-1928
865    *
866    * @param server Server
867    * @return if this server was transitioning a META table then a not null HRegionInfo pointing to it
868    */
869   public HRegionInfo getMetaServerRegionInfo(final String server) {
870     synchronized (this.regionsInTransition) {
871       for (RegionState s : regionsInTransition.values()) {
872         if (s.getRegionInfo().isMetaRegion()
873             && !s.isUnassigned()
874             && s.getServerName() != null
875             && s.getServerName().equals(server)) {
876           // Has an outstanding meta region to be assigned.
877           return s.getRegionInfo();
878         }
879       }
880     }
881     return null;
882   }
883 
884   /**
885    * Call to take this metaserver offline for immediate reassignment.  Used only
886    * when we know a region has shut down cleanly.
887    *
888    * A meta server is a server that hosts either -ROOT- or any .META. regions.
889    *
890    * If you are considering a unclean shutdown potentially, use ProcessServerShutdown which
891    * calls other methods to immediately unassign root/meta but delay the reassign until the
892    * log has been split.
893    *
894    * @param server the server that went down
895    * @return true if this was in fact a meta server, false if it did not carry meta regions.
896    */
897   public synchronized boolean offlineMetaServer(HServerAddress server) {
898     boolean hasMeta = false;
899 
900     // check to see if ROOT and/or .META. are on this server, reassign them.
901     // use master.getRootRegionLocation.
902     if (master.getRegionManager().getRootRegionLocation() != null &&
903         server.equals(master.getRegionManager().getRootRegionLocation())) {
904       LOG.info("Offlined ROOT server: " + server);
905       reassignRootRegion();
906       hasMeta = true;
907     }
908     // AND
909     for ( MetaRegion region : onlineMetaRegions.values() ) {
910       if (server.equals(region.getServer())) {
911         LOG.info("Offlining META region: " + region);
912         offlineMetaRegionWithStartKey(region.getStartKey());
913         // Set for reassignment.
914         setUnassigned(region.getRegionInfo(), true);
915         hasMeta = true;
916       }
917     }
918     return hasMeta;
919   }
920 
921   /**
922    * Remove a region from the region state map.
923    *
924    * @param info
925    */
926   public void removeRegion(HRegionInfo info) {
927     synchronized (this.regionsInTransition) {
928       this.regionsInTransition.remove(info.getRegionNameAsString());
929     }
930   }
931 
932   /**
933    * @param regionName
934    * @return true if the named region is in a transition state
935    */
936   public boolean regionIsInTransition(String regionName) {
937     synchronized (this.regionsInTransition) {
938       return regionsInTransition.containsKey(regionName);
939     }
940   }
941 
942   /**
943    * @param regionName
944    * @return true if the region is unassigned, pendingOpen or open
945    */
946   public boolean regionIsOpening(String regionName) {
947     synchronized (this.regionsInTransition) {
948       RegionState state = regionsInTransition.get(regionName);
949       if (state != null) {
950         return state.isOpening();
951       }
952     }
953     return false;
954   }
955 
956   /**
957    * Set a region to unassigned
958    * @param info Region to set unassigned
959    * @param force if true mark region unassigned whatever its current state
960    */
961   public void setUnassigned(HRegionInfo info, boolean force) {
962     RegionState s = null;
963     synchronized(this.regionsInTransition) {
964       s = regionsInTransition.get(info.getRegionNameAsString());
965       if (s == null) {
966         s = new RegionState(info, RegionState.State.UNASSIGNED);
967         regionsInTransition.put(info.getRegionNameAsString(), s);
968       }
969     }
970     if (force || (!s.isPendingOpen() && !s.isOpen())) {
971       s.setUnassigned();
972     }
973   }
974 
975   /**
976    * Check if a region is on the unassigned list
977    * @param info HRegionInfo to check for
978    * @return true if on the unassigned list, false if it isn't. Note that this
979    * means a region could not be on the unassigned list AND not be assigned, if
980    * it happens to be between states.
981    */
982   public boolean isUnassigned(HRegionInfo info) {
983     synchronized (regionsInTransition) {
984       RegionState s = regionsInTransition.get(info.getRegionNameAsString());
985       if (s != null) {
986         return s.isUnassigned();
987       }
988     }
989     return false;
990   }
991 
992   /**
993    * Check if a region has been assigned and we're waiting for a response from
994    * the region server.
995    *
996    * @param regionName name of the region
997    * @return true if open, false otherwise
998    */
999   public boolean isPendingOpen(String regionName) {
1000     synchronized (regionsInTransition) {
1001       RegionState s = regionsInTransition.get(regionName);
1002       if (s != null) {
1003         return s.isPendingOpen();
1004       }
1005     }
1006     return false;
1007   }
1008 
1009   /**
1010    * Region has been assigned to a server and the server has told us it is open
1011    * @param regionName
1012    */
1013   public void setOpen(String regionName) {
1014     synchronized (regionsInTransition) {
1015       RegionState s = regionsInTransition.get(regionName);
1016       if (s != null) {
1017         s.setOpen();
1018       }
1019     }
1020   }
1021 
1022   /**
1023    * @param regionName
1024    * @return true if region is marked to be offlined.
1025    */
1026   public boolean isOfflined(String regionName) {
1027     synchronized (regionsInTransition) {
1028       RegionState s = regionsInTransition.get(regionName);
1029       if (s != null) {
1030         return s.isOfflined();
1031       }
1032     }
1033     return false;
1034   }
1035 
1036   /**
1037    * Mark a region as closing
1038    * @param serverName
1039    * @param regionInfo
1040    * @param setOffline
1041    */
1042   public void setClosing(String serverName, final HRegionInfo regionInfo,
1043       final boolean setOffline) {
1044     synchronized (this.regionsInTransition) {
1045       RegionState s =
1046         this.regionsInTransition.get(regionInfo.getRegionNameAsString());
1047       if (s == null) {
1048         s = new RegionState(regionInfo, RegionState.State.CLOSING);
1049       }
1050       // If region was asked to open before getting here, we could be taking
1051       // the wrong server name
1052       if(s.isPendingOpen()) {
1053         serverName = s.getServerName();
1054       }
1055       s.setClosing(serverName, setOffline);
1056       this.regionsInTransition.put(regionInfo.getRegionNameAsString(), s);
1057     }
1058   }
1059 
1060   /**
1061    * Remove the map of region names to region infos waiting to be offlined for a
1062    * given server
1063    *
1064    * @param serverName
1065    * @return set of infos to close
1066    */
1067   public Set<HRegionInfo> getMarkedToClose(String serverName) {
1068     Set<HRegionInfo> result = new HashSet<HRegionInfo>();
1069     synchronized (regionsInTransition) {
1070       for (RegionState s: regionsInTransition.values()) {
1071         if (s.isClosing() && !s.isPendingClose() && !s.isClosed() &&
1072             s.getServerName().compareTo(serverName) == 0) {
1073           result.add(s.getRegionInfo());
1074         }
1075       }
1076     }
1077     return result;
1078   }
1079 
1080   /**
1081    * Called when we have told a region server to close the region
1082    *
1083    * @param regionName
1084    */
1085   public void setPendingClose(String regionName) {
1086     synchronized (regionsInTransition) {
1087       RegionState s = regionsInTransition.get(regionName);
1088       if (s != null) {
1089         s.setPendingClose();
1090       }
1091     }
1092   }
1093 
1094   /**
1095    * @param regionName
1096    */
1097   public void setClosed(String regionName) {
1098     synchronized (regionsInTransition) {
1099       RegionState s = regionsInTransition.get(regionName);
1100       if (s != null) {
1101         s.setClosed();
1102       }
1103     }
1104   }
1105   /**
1106    * Add a meta region to the scan queue
1107    * @param m MetaRegion that needs to get scanned
1108    */
1109   public void addMetaRegionToScan(MetaRegion m) {
1110     metaScannerThread.addMetaRegionToScan(m);
1111   }
1112 
1113   /**
1114    * Check if the initial root scan has been completed.
1115    * @return true if scan completed, false otherwise
1116    */
1117   public boolean isInitialRootScanComplete() {
1118     return rootScannerThread.isInitialScanComplete();
1119   }
1120 
1121   /**
1122    * Check if the initial meta scan has been completed.
1123    * @return true if meta completed, false otherwise
1124    */
1125   public boolean isInitialMetaScanComplete() {
1126     return metaScannerThread.isInitialScanComplete();
1127   }
1128 
1129   /**
1130    * Get the root region location.
1131    * @return HServerAddress describing root region server.
1132    */
1133   public HServerAddress getRootRegionLocation() {
1134     return rootRegionLocation.get();
1135   }
1136 
1137   /**
1138    * Block until either the root region location is available or we're shutting
1139    * down.
1140    */
1141   public void waitForRootRegionLocation() {
1142     synchronized (rootRegionLocation) {
1143       while (!master.getShutdownRequested().get() &&
1144           !master.isClosed() && rootRegionLocation.get() == null) {
1145         // rootRegionLocation will be filled in when we get an 'open region'
1146         // regionServerReport message from the HRegionServer that has been
1147         // allocated the ROOT region below.
1148         try {
1149           // Cycle rather than hold here in case master is closed meantime.
1150           rootRegionLocation.wait(this.master.getThreadWakeFrequency());
1151         } catch (InterruptedException e) {
1152           // continue
1153         }
1154       }
1155     }
1156   }
1157 
1158   /**
1159    * Return the number of meta regions.
1160    * @return number of meta regions
1161    */
1162   public int numMetaRegions() {
1163     return numberOfMetaRegions.get();
1164   }
1165 
1166   /**
1167    * Bump the count of meta regions up one
1168    */
1169   public void incrementNumMetaRegions() {
1170     numberOfMetaRegions.incrementAndGet();
1171   }
1172 
1173   private long getPauseTime(int tries) {
1174     int attempt = tries;
1175     if (attempt >= HConstants.RETRY_BACKOFF.length) {
1176       attempt = HConstants.RETRY_BACKOFF.length - 1;
1177     }
1178     return this.zooKeeperPause * HConstants.RETRY_BACKOFF[attempt];
1179   }
1180 
1181   private void sleep(int attempt) {
1182     try {
1183       Thread.sleep(getPauseTime(attempt));
1184     } catch (InterruptedException e) {
1185       // continue
1186     }
1187   }
1188 
1189   private void writeRootRegionLocationToZooKeeper(HServerAddress address) {
1190     for (int attempt = 0; attempt < zooKeeperNumRetries; ++attempt) {
1191       if (master.getZooKeeperWrapper().writeRootRegionLocation(address)) {
1192         return;
1193       }
1194 
1195       sleep(attempt);
1196     }
1197 
1198     LOG.error("Failed to write root region location to ZooKeeper after " +
1199               zooKeeperNumRetries + " retries, shutting down");
1200 
1201     this.master.shutdown();
1202   }
1203 
1204   /**
1205    * Set the root region location.
1206    * @param address Address of the region server where the root lives
1207    */
1208   public void setRootRegionLocation(HServerAddress address) {
1209     writeRootRegionLocationToZooKeeper(address);
1210     synchronized (rootRegionLocation) {
1211       rootRegionLocation.set(new HServerAddress(address));
1212       rootRegionLocation.notifyAll();
1213     }
1214   }
1215 
1216   /**
1217    * Set the number of meta regions.
1218    * @param num Number of meta regions
1219    */
1220   public void setNumMetaRegions(int num) {
1221     numberOfMetaRegions.set(num);
1222   }
1223 
1224   /**
1225    * @param regionName
1226    * @param info
1227    * @param server
1228    * @param op
1229    */
1230   public void startAction(byte[] regionName, HRegionInfo info,
1231       HServerAddress server, HConstants.Modify op) {
1232     if (LOG.isDebugEnabled()) {
1233       LOG.debug("Adding operation " + op + " from tasklist");
1234     }
1235     switch (op) {
1236       case TABLE_SPLIT:
1237         startAction(regionName, info, server, this.regionsToSplit);
1238         break;
1239       case TABLE_COMPACT:
1240         startAction(regionName, info, server, this.regionsToCompact);
1241         break;
1242       case TABLE_MAJOR_COMPACT:
1243         startAction(regionName, info, server, this.regionsToMajorCompact);
1244         break;
1245       case TABLE_FLUSH:
1246         startAction(regionName, info, server, this.regionsToFlush);
1247         break;
1248       default:
1249         throw new IllegalArgumentException("illegal table action " + op);
1250     }
1251   }
1252 
1253   private void startAction(final byte[] regionName, final HRegionInfo info,
1254       final HServerAddress server,
1255       final SortedMap<byte[], Pair<HRegionInfo,HServerAddress>> map) {
1256     map.put(regionName, new Pair<HRegionInfo,HServerAddress>(info, server));
1257   }
1258 
1259   /**
1260    * @param regionName
1261    * @param op
1262    */
1263   public void endAction(byte[] regionName, HConstants.Modify op) {
1264     if (LOG.isDebugEnabled()) {
1265       LOG.debug("Removing operation " + op + " from tasklist");
1266     }
1267     switch (op) {
1268     case TABLE_SPLIT:
1269       this.regionsToSplit.remove(regionName);
1270       break;
1271     case TABLE_COMPACT:
1272       this.regionsToCompact.remove(regionName);
1273       break;
1274     case TABLE_MAJOR_COMPACT:
1275       this.regionsToMajorCompact.remove(regionName);
1276       break;
1277     case TABLE_FLUSH:
1278       this.regionsToFlush.remove(regionName);
1279       break;
1280     default:
1281       throw new IllegalArgumentException("illegal table action " + op);
1282     }
1283   }
1284 
1285   /**
1286    * @param regionName
1287    */
1288   public void endActions(byte[] regionName) {
1289     regionsToSplit.remove(regionName);
1290     regionsToCompact.remove(regionName);
1291   }
1292 
1293   /**
1294    * Send messages to the given region server asking it to split any
1295    * regions in 'regionsToSplit', etc.
1296    * @param serverInfo
1297    * @param returnMsgs
1298    */
1299   public void applyActions(HServerInfo serverInfo, ArrayList<HMsg> returnMsgs) {
1300     applyActions(serverInfo, returnMsgs, this.regionsToCompact,
1301         HMsg.Type.MSG_REGION_COMPACT);
1302     applyActions(serverInfo, returnMsgs, this.regionsToSplit,
1303       HMsg.Type.MSG_REGION_SPLIT);
1304     applyActions(serverInfo, returnMsgs, this.regionsToFlush,
1305         HMsg.Type.MSG_REGION_FLUSH);
1306     applyActions(serverInfo, returnMsgs, this.regionsToMajorCompact,
1307         HMsg.Type.MSG_REGION_MAJOR_COMPACT);
1308   }
1309 
1310   private void applyActions(final HServerInfo serverInfo,
1311       final ArrayList<HMsg> returnMsgs,
1312       final SortedMap<byte[], Pair<HRegionInfo,HServerAddress>> map,
1313       final HMsg.Type msg) {
1314     HServerAddress addr = serverInfo.getServerAddress();
1315     synchronized (map) {
1316       Iterator<Pair<HRegionInfo, HServerAddress>> i = map.values().iterator();
1317       while (i.hasNext()) {
1318         Pair<HRegionInfo,HServerAddress> pair = i.next();
1319         if (addr.equals(pair.getSecond())) {
1320           if (LOG.isDebugEnabled()) {
1321             LOG.debug("Sending " + msg + " " + pair.getFirst() + " to " + addr);
1322           }
1323           returnMsgs.add(new HMsg(msg, pair.getFirst()));
1324           i.remove();
1325         }
1326       }
1327     }
1328   }
1329 
1330   /**
1331    * Class to balance region servers load.
1332    * It keeps Region Servers load in slop range by unassigning Regions
1333    * from most loaded servers.
1334    *
1335    * Equilibrium is reached when load of all serves are in slop range
1336    * [avgLoadMinusSlop, avgLoadPlusSlop], where
1337    *  avgLoadPlusSlop = Math.ceil(avgLoad * (1 + this.slop)), and
1338    *  avgLoadMinusSlop = Math.floor(avgLoad * (1 - this.slop)) - 1.
1339    */
1340   private class LoadBalancer {
1341     private float slop;                 // hbase.regions.slop
1342     private final int maxRegToClose;    // hbase.regions.close.max
1343 
1344     LoadBalancer(Configuration conf) {
1345       this.slop = conf.getFloat("hbase.regions.slop", (float)0.3);
1346       if (this.slop <= 0) this.slop = 1;
1347       //maxRegToClose to constrain balance closing per one iteration
1348       // -1 to turn off
1349       // TODO: change default in HBASE-862, need a suggestion
1350       this.maxRegToClose = conf.getInt("hbase.regions.close.max", -1);
1351     }
1352 
1353     /**
1354      * Balance server load by unassigning some regions.
1355      *
1356      * @param info - server info
1357      * @param mostLoadedRegions - array of most loaded regions
1358      * @param returnMsgs - array of return massages
1359      */
1360     void loadBalancing(HServerInfo info, HRegionInfo[] mostLoadedRegions,
1361         ArrayList<HMsg> returnMsgs) {
1362       HServerLoad servLoad = info.getLoad();
1363       double avg = master.getAverageLoad();
1364 
1365       // nothing to balance if server load not more then average load
1366       if(servLoad.getLoad() <= Math.ceil(avg) || avg <= 2.0) {
1367         return;
1368       }
1369 
1370       // check if current server is overloaded
1371       int numRegionsToClose = balanceFromOverloaded(info.getServerName(),
1372         servLoad, avg);
1373 
1374       // check if we can unload server by low loaded servers
1375       if(numRegionsToClose <= 0) {
1376         numRegionsToClose = balanceToLowloaded(info.getServerName(), servLoad,
1377             avg);
1378       }
1379 
1380       if(maxRegToClose > 0) {
1381         numRegionsToClose = Math.min(numRegionsToClose, maxRegToClose);
1382       }
1383 
1384       if(numRegionsToClose > 0) {
1385         unassignSomeRegions(info, numRegionsToClose, mostLoadedRegions,
1386             returnMsgs);
1387       }
1388     }
1389 
1390     /*
1391      * Check if server load is not overloaded (with load > avgLoadPlusSlop).
1392      * @return number of regions to unassign.
1393      */
1394     private int balanceFromOverloaded(final String serverName,
1395         HServerLoad srvLoad, double avgLoad) {
1396       int avgLoadPlusSlop = (int)Math.ceil(avgLoad * (1 + this.slop));
1397       int numSrvRegs = srvLoad.getNumberOfRegions();
1398       if (numSrvRegs > avgLoadPlusSlop) {
1399         if (LOG.isDebugEnabled()) {
1400           LOG.debug("Server " + serverName + " is carrying more than its fair " +
1401             "share of regions: " +
1402             "load=" + numSrvRegs + ", avg=" + avgLoad + ", slop=" + this.slop);
1403         }
1404         return numSrvRegs - (int)Math.ceil(avgLoad);
1405       }
1406       return 0;
1407     }
1408 
1409     /*
1410      * Check if server is most loaded and can be unloaded to
1411      * low loaded servers (with load < avgLoadMinusSlop).
1412      * @return number of regions to unassign.
1413      */
1414     private int balanceToLowloaded(String srvName, HServerLoad srvLoad,
1415         double avgLoad) {
1416 
1417       SortedMap<HServerLoad, Set<String>> loadToServers =
1418         master.getLoadToServers();
1419       // check if server most loaded
1420       if (!loadToServers.get(loadToServers.lastKey()).contains(srvName))
1421         return 0;
1422 
1423       // this server is most loaded, we will try to unload it by lowest
1424       // loaded servers
1425       int avgLoadMinusSlop = (int)Math.floor(avgLoad * (1 - this.slop)) - 1;
1426       int lowestLoad = loadToServers.firstKey().getNumberOfRegions();
1427 
1428       if(lowestLoad >= avgLoadMinusSlop)
1429         return 0; // there is no low loaded servers
1430 
1431       int lowSrvCount = loadToServers.get(loadToServers.firstKey()).size();
1432       int numRegionsToClose = 0;
1433 
1434       int numSrvRegs = srvLoad.getNumberOfRegions();
1435       int numMoveToLowLoaded = (avgLoadMinusSlop - lowestLoad) * lowSrvCount;
1436       numRegionsToClose = numSrvRegs - (int)Math.ceil(avgLoad);
1437       numRegionsToClose = Math.min(numRegionsToClose, numMoveToLowLoaded);
1438       if (LOG.isDebugEnabled()) {
1439         LOG.debug("Server(s) are carrying only " + lowestLoad + " regions. " +
1440           "Server " + srvName + " is most loaded (" + numSrvRegs +
1441           "). Shedding " + numRegionsToClose + " regions to pass to " +
1442           " least loaded (numMoveToLowLoaded=" + numMoveToLowLoaded +")");
1443       }
1444       return numRegionsToClose;
1445     }
1446   }
1447 
1448   /**
1449    * @return Snapshot of regionsintransition as a sorted Map.
1450    */
1451   NavigableMap<String, String> getRegionsInTransition() {
1452     NavigableMap<String, String> result = new TreeMap<String, String>();
1453     synchronized (this.regionsInTransition) {
1454       if (this.regionsInTransition.isEmpty()) return result;
1455       for (Map.Entry<String, RegionState> e: this.regionsInTransition.entrySet()) {
1456         result.put(e.getKey(), e.getValue().toString());
1457       }
1458     }
1459     return result;
1460   }
1461 
1462   /**
1463    * @param regionname Name to clear from regions in transistion.
1464    * @return True if we removed an element for the passed regionname.
1465    */
1466   boolean clearFromInTransition(final byte [] regionname) {
1467     boolean result = false;
1468     synchronized (this.regionsInTransition) {
1469       if (this.regionsInTransition.isEmpty()) return result;
1470       for (Map.Entry<String, RegionState> e: this.regionsInTransition.entrySet()) {
1471         if (Bytes.equals(regionname, e.getValue().getRegionName())) {
1472           this.regionsInTransition.remove(e.getKey());
1473           LOG.debug("Removed " + e.getKey() + ", " + e.getValue());
1474           result = true;
1475           break;
1476         }
1477       }
1478     }
1479     return result;
1480   }
1481 
1482   /*
1483    * State of a Region as it transitions from closed to open, etc.  See
1484    * note on regionsInTransition data member above for listing of state
1485    * transitions.
1486    */
1487   static class RegionState implements Comparable<RegionState> {
1488     private final HRegionInfo regionInfo;
1489 
1490     enum State {
1491       UNASSIGNED, // awaiting a server to be assigned
1492       PENDING_OPEN, // told a server to open, hasn't opened yet
1493       OPEN, // has been opened on RS, but not yet marked in META/ROOT
1494       CLOSING, // a msg has been enqueued to close ths region, but not delivered to RS yet
1495       PENDING_CLOSE, // msg has been delivered to RS to close this region
1496       CLOSED // region has been closed but not yet marked in meta
1497 
1498     }
1499 
1500     private State state;
1501 
1502     private boolean isOfflined;
1503 
1504     /* Set when region is assigned or closing */
1505     private String serverName = null;
1506 
1507     /* Constructor */
1508     RegionState(HRegionInfo info, State state) {
1509       this.regionInfo = info;
1510       this.state = state;
1511     }
1512 
1513     synchronized HRegionInfo getRegionInfo() {
1514       return this.regionInfo;
1515     }
1516 
1517     synchronized byte [] getRegionName() {
1518       return this.regionInfo.getRegionName();
1519     }
1520 
1521     /*
1522      * @return Server this region was assigned to
1523      */
1524     synchronized String getServerName() {
1525       return this.serverName;
1526     }
1527 
1528     /*
1529      * @return true if the region is being opened
1530      */
1531     synchronized boolean isOpening() {
1532       return state == State.UNASSIGNED ||
1533         state == State.PENDING_OPEN ||
1534         state == State.OPEN;
1535     }
1536 
1537     /*
1538      * @return true if region is unassigned
1539      */
1540     synchronized boolean isUnassigned() {
1541       return state == State.UNASSIGNED;
1542     }
1543 
1544     /*
1545      * Note: callers of this method (reassignRootRegion,
1546      * regionsAwaitingAssignment, setUnassigned) ensure that this method is not
1547      * called unless it is safe to do so.
1548      */
1549     synchronized void setUnassigned() {
1550       state = State.UNASSIGNED;
1551       this.serverName = null;
1552     }
1553 
1554     synchronized boolean isPendingOpen() {
1555       return state == State.PENDING_OPEN;
1556     }
1557 
1558     /*
1559      * @param serverName Server region was assigned to.
1560      */
1561     synchronized void setPendingOpen(final String serverName) {
1562       if (state != State.UNASSIGNED) {
1563         LOG.warn("Cannot assign a region that is not currently unassigned. " +
1564           "FIX!! State: " + toString());
1565       }
1566       state = State.PENDING_OPEN;
1567       this.serverName = serverName;
1568     }
1569 
1570     synchronized boolean isOpen() {
1571       return state == State.OPEN;
1572     }
1573 
1574     synchronized void setOpen() {
1575       if (state != State.PENDING_OPEN) {
1576         LOG.warn("Cannot set a region as open if it has not been pending. " +
1577           "FIX!! State: " + toString());
1578       }
1579       state = State.OPEN;
1580     }
1581 
1582     synchronized boolean isClosing() {
1583       return state == State.CLOSING;
1584     }
1585 
1586     synchronized void setClosing(String serverName, boolean setOffline) {
1587       state = State.CLOSING;
1588       this.serverName = serverName;
1589       this.isOfflined = setOffline;
1590     }
1591 
1592     synchronized boolean isPendingClose() {
1593       return state == State.PENDING_CLOSE;
1594     }
1595 
1596     synchronized void setPendingClose() {
1597       if (state != State.CLOSING) {
1598         LOG.warn("Cannot set a region as pending close if it has not been " +
1599           "closing.  FIX!! State: " + toString());
1600       }
1601       state = State.PENDING_CLOSE;
1602     }
1603 
1604     synchronized boolean isClosed() {
1605       return state == State.CLOSED;
1606     }
1607 
1608     synchronized void setClosed() {
1609       if (state != State.PENDING_CLOSE &&
1610           state != State.PENDING_OPEN &&
1611           state != State.CLOSING) {
1612         throw new IllegalStateException(
1613             "Cannot set a region to be closed if it was not already marked as" +
1614             " pending close, pending open or closing. State: " + this);
1615       }
1616       state = State.CLOSED;
1617     }
1618 
1619     synchronized boolean isOfflined() {
1620       return (state == State.CLOSING ||
1621         state == State.PENDING_CLOSE) && isOfflined;
1622     }
1623 
1624     @Override
1625     public synchronized String toString() {
1626       return ("name=" + Bytes.toString(getRegionName()) +
1627           ", state=" + this.state);
1628     }
1629 
1630     @Override
1631     public boolean equals(Object o) {
1632       if (this == o) {
1633         return true;
1634       }
1635       if (o == null || getClass() != o.getClass()) {
1636         return false;
1637       }
1638       return this.compareTo((RegionState) o) == 0;
1639     }
1640 
1641     @Override
1642     public int hashCode() {
1643       return Bytes.toString(getRegionName()).hashCode();
1644     }
1645 
1646     public int compareTo(RegionState o) {
1647       if (o == null) {
1648         return 1;
1649       }
1650       return Bytes.compareTo(getRegionName(), o.getRegionName());
1651     }
1652   }
1653 }