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