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.List;
27  import java.util.Map;
28  import java.util.Set;
29  import java.util.concurrent.ConcurrentHashMap;
30  
31  import org.apache.commons.logging.Log;
32  import org.apache.commons.logging.LogFactory;
33  import org.apache.hadoop.conf.Configuration;
34  import org.apache.hadoop.hbase.ClockOutOfSyncException;
35  import org.apache.hadoop.hbase.HMsg;
36  import org.apache.hadoop.hbase.HRegionInfo;
37  import org.apache.hadoop.hbase.HServerAddress;
38  import org.apache.hadoop.hbase.HServerInfo;
39  import org.apache.hadoop.hbase.HServerLoad;
40  import org.apache.hadoop.hbase.PleaseHoldException;
41  import org.apache.hadoop.hbase.Server;
42  import org.apache.hadoop.hbase.YouAreDeadException;
43  import org.apache.hadoop.hbase.catalog.CatalogTracker;
44  import org.apache.hadoop.hbase.client.HConnection;
45  import org.apache.hadoop.hbase.client.HConnectionManager;
46  import org.apache.hadoop.hbase.client.RetriesExhaustedException;
47  import org.apache.hadoop.hbase.ipc.HRegionInterface;
48  import org.apache.hadoop.hbase.master.handler.MetaServerShutdownHandler;
49  import org.apache.hadoop.hbase.master.handler.ServerShutdownHandler;
50  import org.apache.hadoop.hbase.master.metrics.MasterMetrics;
51  import org.apache.hadoop.hbase.regionserver.Leases.LeaseStillHeldException;
52  
53  /**
54   * The ServerManager class manages info about region servers - HServerInfo,
55   * load numbers, dying servers, etc.
56   * <p>
57   * Maintains lists of online and dead servers.  Processes the startups,
58   * shutdowns, and deaths of region servers.
59   * <p>
60   * Servers are distinguished in two different ways.  A given server has a
61   * location, specified by hostname and port, and of which there can only be one
62   * online at any given time.  A server instance is specified by the location
63   * (hostname and port) as well as the startcode (timestamp from when the server
64   * was started).  This is used to differentiate a restarted instance of a given
65   * server from the original instance.
66   */
67  public class ServerManager {
68    private static final Log LOG = LogFactory.getLog(ServerManager.class);
69  
70    // Set if we are to shutdown the cluster.
71    private volatile boolean clusterShutdown = false;
72  
73    /** The map of known server names to server info */
74    private final Map<String, HServerInfo> onlineServers =
75      new ConcurrentHashMap<String, HServerInfo>();
76  
77    // TODO: This is strange to have two maps but HSI above is used on both sides
78    /**
79     * Map from full server-instance name to the RPC connection for this server.
80     */
81    private final Map<String, HRegionInterface> serverConnections =
82      new HashMap<String, HRegionInterface>();
83  
84    private final Server master;
85    private final MasterServices services;
86  
87    // Reporting to track master metrics.
88    private final MasterMetrics metrics;
89  
90    private final DeadServer deadservers;
91  
92    private final long maxSkew;
93  
94    /**
95     * Constructor.
96     * @param master
97     * @param services
98     * @param metrics
99     */
100   public ServerManager(final Server master, final MasterServices services,
101       MasterMetrics metrics) {
102     this.master = master;
103     this.services = services;
104     this.metrics = metrics;
105     Configuration c = master.getConfiguration();
106     maxSkew = c.getLong("hbase.master.maxclockskew", 30000);
107     this.deadservers =
108       new DeadServer(c.getInt("hbase.master.maxdeadservers", 100));
109   }
110 
111   /**
112    * Let the server manager know a new regionserver has come online
113    * @param serverInfo
114    * @param serverCurrentTime The current time of the region server in ms
115    * @throws IOException
116    */
117   void regionServerStartup(final HServerInfo serverInfo, long serverCurrentTime)
118   throws IOException {
119     // Test for case where we get a region startup message from a regionserver
120     // that has been quickly restarted but whose znode expiration handler has
121     // not yet run, or from a server whose fail we are currently processing.
122     // Test its host+port combo is present in serverAddresstoServerInfo.  If it
123     // is, reject the server and trigger its expiration. The next time it comes
124     // in, it should have been removed from serverAddressToServerInfo and queued
125     // for processing by ProcessServerShutdown.
126     HServerInfo info = new HServerInfo(serverInfo);
127     checkIsDead(info.getServerName(), "STARTUP");
128     checkAlreadySameHostPort(info);
129     checkClockSkew(info, serverCurrentTime);
130     recordNewServer(info, false, null);
131   }
132 
133   /**
134    * Test to see if we have a server of same host and port already.
135    * @param serverInfo
136    * @throws PleaseHoldException
137    */
138   void checkAlreadySameHostPort(final HServerInfo serverInfo)
139   throws PleaseHoldException {
140     String hostAndPort = serverInfo.getServerAddress().toString();
141     HServerInfo existingServer =
142       haveServerWithSameHostAndPortAlready(serverInfo.getHostnamePort());
143     if (existingServer != null) {
144       String message = "Server start rejected; we already have " + hostAndPort +
145         " registered; existingServer=" + existingServer + ", newServer=" + serverInfo;
146       LOG.info(message);
147       if (existingServer.getStartCode() < serverInfo.getStartCode()) {
148         LOG.info("Triggering server recovery; existingServer " +
149           existingServer.getServerName() + " looks stale");
150         expireServer(existingServer);
151       }
152       throw new PleaseHoldException(message);
153     }
154   }
155 
156   private HServerInfo haveServerWithSameHostAndPortAlready(final String hostnamePort) {
157     synchronized (this.onlineServers) {
158       for (Map.Entry<String, HServerInfo> e: this.onlineServers.entrySet()) {
159         if (e.getValue().getHostnamePort().equals(hostnamePort)) {
160           return e.getValue();
161         }
162       }
163     }
164     return null;
165   }
166 
167   /**
168    * Checks if the clock skew between the server and the master. If the clock
169    * skew is too much it will throw an Exception.
170    * @throws ClockOutOfSyncException
171    */
172   private void checkClockSkew(final HServerInfo serverInfo,
173       final long serverCurrentTime)
174   throws ClockOutOfSyncException {
175     long skew = System.currentTimeMillis() - serverCurrentTime;
176     if (skew > maxSkew) {
177       String message = "Server " + serverInfo.getServerName() + " has been " +
178         "rejected; Reported time is too far out of sync with master.  " +
179         "Time difference of " + skew + "ms > max allowed of " + maxSkew + "ms";
180       LOG.warn(message);
181       throw new ClockOutOfSyncException(message);
182     }
183   }
184 
185   /**
186    * If this server is on the dead list, reject it with a LeaseStillHeldException
187    * @param serverName Server name formatted as host_port_startcode.
188    * @param what START or REPORT
189    * @throws LeaseStillHeldException
190    */
191   private void checkIsDead(final String serverName, final String what)
192   throws YouAreDeadException {
193     if (!this.deadservers.isDeadServer(serverName)) return;
194     String message = "Server " + what + " rejected; currently processing " +
195       serverName + " as dead server";
196     LOG.debug(message);
197     throw new YouAreDeadException(message);
198   }
199 
200   /**
201    * Adds the HSI to the RS list
202    * @param info The region server informations
203    * @param useInfoLoad True if the load from the info should be used; e.g.
204    * under a master failover
205    * @param hri Region interface.  Can be null.
206    */
207   void recordNewServer(HServerInfo info, boolean useInfoLoad,
208       HRegionInterface hri) {
209     HServerLoad load = useInfoLoad? info.getLoad(): new HServerLoad();
210     String serverName = info.getServerName();
211     LOG.info("Registering server=" + serverName + ", regionCount=" +
212       load.getLoad() + ", userLoad=" + useInfoLoad);
213     info.setLoad(load);
214     // TODO: Why did we update the RS location ourself?  Shouldn't RS do this?
215     // masterStatus.getZooKeeper().updateRSLocationGetWatch(info, watcher);
216     // -- If I understand the question, the RS does not update the location
217     // because could be disagreement over locations because of DNS issues; only
218     // master does DNS now -- St.Ack 20100929.
219     this.onlineServers.put(serverName, info);
220     if (hri == null) {
221       serverConnections.remove(serverName);
222     } else {
223       serverConnections.put(serverName, hri);
224     }
225   }
226 
227   /**
228    * Called to process the messages sent from the region server to the master
229    * along with the heart beat.
230    *
231    * @param serverInfo
232    * @param msgs
233    * @param mostLoadedRegions Array of regions the region server is submitting
234    * as candidates to be rebalanced, should it be overloaded
235    * @return messages from master to region server indicating what region
236    * server should do.
237    *
238    * @throws IOException
239    */
240   HMsg [] regionServerReport(final HServerInfo serverInfo,
241     final HMsg [] msgs, final HRegionInfo[] mostLoadedRegions)
242   throws IOException {
243     // Be careful. This method does returns in the middle.
244     HServerInfo info = new HServerInfo(serverInfo);
245 
246     // Check if dead.  If it is, it'll get a 'You Are Dead!' exception.
247     checkIsDead(info.getServerName(), "REPORT");
248 
249     // If we don't know this server, tell it shutdown.
250     HServerInfo storedInfo = this.onlineServers.get(info.getServerName());
251     if (storedInfo == null) {
252       // Maybe we already have this host+port combo and its just different
253       // start code?
254       checkAlreadySameHostPort(info);
255       // Just let the server in. Presume master joining a running cluster.
256       // recordNewServer is what happens at the end of reportServerStartup.
257       // The only thing we are skipping is passing back to the regionserver
258       // the HServerInfo to use. Here we presume a master has already done
259       // that so we'll press on with whatever it gave us for HSI.
260       recordNewServer(info, true, null);
261       // If msgs, put off their processing but this is not enough because
262       // its possible that the next time the server reports in, we'll still
263       // not be up and serving. For example, if a split, we'll need the
264       // regions and servers setup in the master before the below
265       // handleSplitReport will work. TODO: FIx!!
266       if (msgs.length > 0)
267         throw new PleaseHoldException("FIX! Putting off " +
268           "message processing because not yet rwady but possible we won't be " +
269           "ready next on next report");
270     }
271 
272     // Check startcodes
273     if (raceThatShouldNotHappenAnymore(storedInfo, info)) {
274       return HMsg.STOP_REGIONSERVER_ARRAY;
275     }
276 
277     for (HMsg msg: msgs) {
278       LOG.info("Received " + msg + " from " + serverInfo.getServerName());
279       switch (msg.getType()) {
280       case REGION_SPLIT:
281         this.services.getAssignmentManager().handleSplitReport(serverInfo,
282             msg.getRegionInfo(), msg.getDaughterA(), msg.getDaughterB());
283         break;
284 
285         default:
286           LOG.error("Unhandled msg type " + msg);
287       }
288     }
289 
290     HMsg [] reply = null;
291     int numservers = countOfRegionServers();
292     if (this.clusterShutdown) {
293       if (numservers <= 2) {
294         // Shutdown needs to be staggered; the meta regions need to close last
295         // in case they need to be updated during the close melee.  If <= 2
296         // servers left, then these are the two that were carrying root and meta
297         // most likely (TODO: This presumes unsplittable meta -- FIX). Tell
298         // these servers can shutdown now too.
299         reply = HMsg.STOP_REGIONSERVER_ARRAY;
300       }
301     }
302     return processRegionServerAllsWell(info, mostLoadedRegions, reply);
303   }
304 
305   private boolean raceThatShouldNotHappenAnymore(final HServerInfo storedInfo,
306       final HServerInfo reportedInfo) {
307     if (storedInfo.getStartCode() != reportedInfo.getStartCode()) {
308       // TODO: I don't think this possible any more.  We check startcodes when
309       // server comes in on regionServerStartup -- St.Ack
310       // This state is reachable if:
311       // 1) RegionServer A started
312       // 2) RegionServer B started on the same machine, then clobbered A in regionServerStartup.
313       // 3) RegionServer A returns, expecting to work as usual.
314       // The answer is to ask A to shut down for good.
315       LOG.warn("Race condition detected: " + reportedInfo.getServerName());
316       synchronized (this.onlineServers) {
317         removeServerInfo(reportedInfo.getServerName());
318         notifyOnlineServers();
319       }
320       return true;
321     }
322     return false;
323   }
324 
325   /**
326    *  RegionServer is checking in, no exceptional circumstances
327    * @param serverInfo
328    * @param mostLoadedRegions
329    * @param msgs
330    * @return
331    * @throws IOException
332    */
333   private HMsg[] processRegionServerAllsWell(HServerInfo serverInfo,
334       final HRegionInfo[] mostLoadedRegions, HMsg[] msgs)
335   throws IOException {
336     // Refresh the info object and the load information
337     this.onlineServers.put(serverInfo.getServerName(), serverInfo);
338     HServerLoad load = serverInfo.getLoad();
339     if (load != null && this.metrics != null) {
340       this.metrics.incrementRequests(load.getNumberOfRequests());
341     }
342     // No more piggyback messages on heartbeats for other stuff
343     return msgs;
344   }
345 
346   /**
347    * @param serverName
348    * @return True if we removed server from the list.
349    */
350   private boolean removeServerInfo(final String serverName) {
351     HServerInfo info = this.onlineServers.remove(serverName);
352     if (info != null) {
353       return true;
354     }
355     return false;
356   }
357 
358   /**
359    * Compute the average load across all region servers.
360    * Currently, this uses a very naive computation - just uses the number of
361    * regions being served, ignoring stats about number of requests.
362    * @return the average load
363    */
364   public double getAverageLoad() {
365     int totalLoad = 0;
366     int numServers = 0;
367     double averageLoad = 0.0;
368     for (HServerInfo hsi : onlineServers.values()) {
369         numServers++;
370         totalLoad += hsi.getLoad().getNumberOfRegions();
371     }
372     averageLoad = (double)totalLoad / (double)numServers;
373     return averageLoad;
374   }
375 
376   /** @return the count of active regionservers */
377   int countOfRegionServers() {
378     // Presumes onlineServers is a concurrent map
379     return this.onlineServers.size();
380   }
381 
382   /**
383    * @param name server name
384    * @return HServerInfo for the given server address
385    */
386   public HServerInfo getServerInfo(String name) {
387     return this.onlineServers.get(name);
388   }
389 
390   /**
391    * @return Read-only map of servers to serverinfo
392    */
393   public Map<String, HServerInfo> getOnlineServers() {
394     // Presumption is that iterating the returned Map is OK.
395     synchronized (this.onlineServers) {
396       return Collections.unmodifiableMap(this.onlineServers);
397     }
398   }
399 
400   public Set<String> getDeadServers() {
401     return this.deadservers.clone();
402   }
403 
404   /**
405    * Checks if any dead servers are currently in progress.
406    * @return true if any RS are being processed as dead, false if not
407    */
408   public boolean areDeadServersInProgress() {
409     return this.deadservers.areDeadServersInProgress();
410   }
411 
412   /**
413    * @param hsa
414    * @return The HServerInfo whose HServerAddress is <code>hsa</code> or null
415    * if nothing found.
416    */
417   public HServerInfo getHServerInfo(final HServerAddress hsa) {
418     synchronized(this.onlineServers) {
419       // TODO: This is primitive.  Do a better search.
420       for (Map.Entry<String, HServerInfo> e: this.onlineServers.entrySet()) {
421         if (e.getValue().getServerAddress().equals(hsa)) {
422           return e.getValue();
423         }
424       }
425     }
426     return null;
427   }
428 
429   private void notifyOnlineServers() {
430     synchronized (this.onlineServers) {
431       this.onlineServers.notifyAll();
432     }
433   }
434 
435   /*
436    * Wait on regionservers to report in
437    * with {@link #regionServerReport(HServerInfo, HMsg[])} so they get notice
438    * the master is going down.  Waits until all region servers come back with
439    * a MSG_REGIONSERVER_STOP.
440    */
441   void letRegionServersShutdown() {
442     synchronized (onlineServers) {
443       while (onlineServers.size() > 0) {
444         StringBuilder sb = new StringBuilder();
445         for (String key: this.onlineServers.keySet()) {
446           if (sb.length() > 0) {
447             sb.append(", ");
448           }
449           sb.append(key);
450         }
451         LOG.info("Waiting on regionserver(s) to go down " + sb.toString());
452         try {
453           this.onlineServers.wait(1000);
454         } catch (InterruptedException e) {
455           // continue
456         }
457       }
458     }
459   }
460 
461   /*
462    * Expire the passed server.  Add it to list of deadservers and queue a
463    * shutdown processing.
464    */
465   public synchronized void expireServer(final HServerInfo hsi) {
466     // First check a server to expire.  ServerName is of the form:
467     // <hostname> , <port> , <startcode>
468     String serverName = hsi.getServerName();
469     HServerInfo info = this.onlineServers.get(serverName);
470     if (info == null) {
471       LOG.warn("Received expiration of " + hsi.getServerName() +
472         " but server is not currently online");
473       return;
474     }
475     if (this.deadservers.contains(serverName)) {
476       // TODO: Can this happen?  It shouldn't be online in this case?
477       LOG.warn("Received expiration of " + hsi.getServerName() +
478           " but server shutdown is already in progress");
479       return;
480     }
481     // Remove the server from the known servers lists and update load info BUT
482     // add to deadservers first; do this so it'll show in dead servers list if
483     // not in online servers list.
484     this.deadservers.add(serverName);
485     this.onlineServers.remove(serverName);
486     this.serverConnections.remove(serverName);
487     // If cluster is going down, yes, servers are going to be expiring; don't
488     // process as a dead server
489     if (this.clusterShutdown) {
490       LOG.info("Cluster shutdown set; " + hsi.getServerName() +
491         " expired; onlineServers=" + this.onlineServers.size());
492       if (this.onlineServers.isEmpty()) {
493         master.stop("Cluster shutdown set; onlineServer=0");
494       }
495       return;
496     }
497     CatalogTracker ct = this.master.getCatalogTracker();
498     // Was this server carrying root?
499     boolean carryingRoot;
500     try {
501       HServerAddress address = ct.getRootLocation();
502       carryingRoot = address != null &&
503         hsi.getServerAddress().equals(address);
504     } catch (InterruptedException e) {
505       Thread.currentThread().interrupt();
506       LOG.info("Interrupted");
507       return;
508     }
509     // Was this server carrying meta?  Can't ask CatalogTracker because it
510     // may have reset the meta location as null already (it may have already
511     // run into fact that meta is dead).  I can ask assignment manager. It
512     // has an inmemory list of who has what.  This list will be cleared as we
513     // process the dead server but should be  find asking it now.
514     HServerAddress address = ct.getMetaLocation();
515     boolean carryingMeta =
516       address != null && hsi.getServerAddress().equals(address);
517     if (carryingRoot || carryingMeta) {
518       this.services.getExecutorService().submit(new MetaServerShutdownHandler(this.master,
519         this.services, this.deadservers, info, carryingRoot, carryingMeta));
520     } else {
521       this.services.getExecutorService().submit(new ServerShutdownHandler(this.master,
522         this.services, this.deadservers, info));
523     }
524     LOG.debug("Added=" + serverName +
525       " to dead servers, submitted shutdown handler to be executed, root=" +
526         carryingRoot + ", meta=" + carryingMeta);
527   }
528 
529   // RPC methods to region servers
530 
531   /**
532    * Sends an OPEN RPC to the specified server to open the specified region.
533    * <p>
534    * Open should not fail but can if server just crashed.
535    * <p>
536    * @param server server to open a region
537    * @param region region to open
538    */
539   public void sendRegionOpen(HServerInfo server, HRegionInfo region)
540   throws IOException {
541     HRegionInterface hri = getServerConnection(server);
542     if (hri == null) {
543       LOG.warn("Attempting to send OPEN RPC to server " + server.getServerName()
544           + " failed because no RPC connection found to this server");
545       return;
546     }
547     hri.openRegion(region);
548   }
549 
550   /**
551    * Sends an OPEN RPC to the specified server to open the specified region.
552    * <p>
553    * Open should not fail but can if server just crashed.
554    * <p>
555    * @param server server to open a region
556    * @param regions regions to open
557    */
558   public void sendRegionOpen(HServerInfo server, List<HRegionInfo> regions)
559   throws IOException {
560     HRegionInterface hri = getServerConnection(server);
561     if (hri == null) {
562       LOG.warn("Attempting to send OPEN RPC to server " + server.getServerName()
563           + " failed because no RPC connection found to this server");
564       return;
565     }
566     hri.openRegions(regions);
567   }
568 
569   /**
570    * Sends an CLOSE RPC to the specified server to close the specified region.
571    * <p>
572    * A region server could reject the close request because it either does not
573    * have the specified region or the region is being split.
574    * @param server server to open a region
575    * @param region region to open
576    * @return true if server acknowledged close, false if not
577    * @throws IOException
578    */
579   public boolean sendRegionClose(HServerInfo server, HRegionInfo region)
580   throws IOException {
581     if (server == null) throw new NullPointerException("Passed server is null");
582     HRegionInterface hri = getServerConnection(server);
583     if (hri == null) {
584       throw new IOException("Attempting to send CLOSE RPC to server " +
585         server.getServerName() + " for region " +
586         region.getRegionNameAsString() +
587         " failed because no RPC connection found to this server");
588     }
589     return hri.closeRegion(region);
590   }
591 
592   /**
593    * @param info
594    * @return
595    * @throws IOException
596    * @throws RetriesExhaustedException wrapping a ConnectException if failed
597    * putting up proxy.
598    */
599   private HRegionInterface getServerConnection(HServerInfo info)
600   throws IOException {
601     HConnection connection =
602       HConnectionManager.getConnection(this.master.getConfiguration());
603     HRegionInterface hri = serverConnections.get(info.getServerName());
604     if (hri == null) {
605       LOG.debug("New connection to " + info.getServerName());
606       hri = connection.getHRegionConnection(info.getServerAddress(), false);
607       this.serverConnections.put(info.getServerName(), hri);
608     }
609     return hri;
610   }
611 
612   /**
613    * Waits for the regionservers to report in.
614    * @return Count of regions out on cluster
615    * @throws InterruptedException
616    */
617   public int waitForRegionServers()
618   throws InterruptedException {
619     long interval = this.master.getConfiguration().
620       getLong("hbase.master.wait.on.regionservers.interval", 1500);
621     long timeout = this.master.getConfiguration().
622       getLong("hbase.master.wait.on.regionservers.timeout", 4500);
623     int minToStart = this.master.getConfiguration().
624       getInt("hbase.master.wait.on.regionservers.mintostart", 1);
625     int maxToStart = this.master.getConfiguration().
626       getInt("hbase.master.wait.on.regionservers.maxtostart", Integer.MAX_VALUE);
627     // So, number of regionservers > 0 and its been n since last check in, break,
628     // else just stall here
629     int count = 0;
630     long slept = 0;
631     for (int oldcount = countOfRegionServers(); !this.master.isStopped();) {
632       Thread.sleep(interval);
633       slept += interval;
634       count = countOfRegionServers();
635       if (count == oldcount && count >= minToStart && slept >= timeout) {
636         LOG.info("Finished waiting for regionserver count to settle; " +
637             "count=" + count + ", sleptFor=" + slept);
638         break;
639       }
640       if (count >= maxToStart) {
641         LOG.info("At least the max configured number of regionserver(s) have " +
642             "checked in: " + count);
643         break;
644       }
645       if (count == 0) {
646         LOG.info("Waiting on regionserver(s) to checkin");
647       } else {
648         LOG.info("Waiting on regionserver(s) count to settle; currently=" + count);
649       }
650       oldcount = count;
651     }
652     // Count how many regions deployed out on cluster.  If fresh start, it'll
653     // be none but if not a fresh start, we'll have registered servers when
654     // they came in on the {@link #regionServerReport(HServerInfo)} as opposed to
655     // {@link #regionServerStartup(HServerInfo)} and it'll be carrying an
656     // actual server load.
657     int regionCount = 0;
658     for (Map.Entry<String, HServerInfo> e: this.onlineServers.entrySet()) {
659       HServerLoad load = e.getValue().getLoad();
660       if (load != null) regionCount += load.getLoad();
661     }
662     LOG.info("Exiting wait on regionserver(s) to checkin; count=" + count +
663       ", stopped=" + this.master.isStopped() +
664       ", count of regions out on cluster=" + regionCount);
665     return regionCount;
666   }
667 
668   /**
669    * @return A copy of the internal list of online servers.
670    */
671   public List<HServerInfo> getOnlineServersList() {
672     // TODO: optimize the load balancer call so we don't need to make a new list
673     return new ArrayList<HServerInfo>(onlineServers.values());
674   }
675 
676   public boolean isServerOnline(String serverName) {
677     return onlineServers.containsKey(serverName);
678   }
679 
680   public void shutdownCluster() {
681     this.clusterShutdown = true;
682     this.master.stop("Cluster shutdown requested");
683   }
684 
685   public boolean isClusterShutdown() {
686     return this.clusterShutdown;
687   }
688 
689   /**
690    * Stop the ServerManager.  Currently does nothing.
691    */
692   public void stop() {
693 
694   }
695 }