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.net.InetAddress;
24  import java.util.ArrayList;
25  import java.util.Collections;
26  import java.util.HashMap;
27  import java.util.HashSet;
28  import java.util.Iterator;
29  import java.util.List;
30  import java.util.Map;
31  import java.util.Set;
32  import java.util.concurrent.ConcurrentHashMap;
33  
34  import org.apache.commons.logging.Log;
35  import org.apache.commons.logging.LogFactory;
36  import org.apache.hadoop.conf.Configuration;
37  import org.apache.hadoop.hbase.ClockOutOfSyncException;
38  import org.apache.hadoop.hbase.HRegionInfo;
39  import org.apache.hadoop.hbase.HServerAddress;
40  import org.apache.hadoop.hbase.HServerLoad;
41  import org.apache.hadoop.hbase.PleaseHoldException;
42  import org.apache.hadoop.hbase.Server;
43  import org.apache.hadoop.hbase.ServerName;
44  import org.apache.hadoop.hbase.YouAreDeadException;
45  import org.apache.hadoop.hbase.ZooKeeperConnectionException;
46  import org.apache.hadoop.hbase.client.HConnection;
47  import org.apache.hadoop.hbase.client.HConnectionManager;
48  import org.apache.hadoop.hbase.client.RetriesExhaustedException;
49  import org.apache.hadoop.hbase.ipc.HRegionInterface;
50  import org.apache.hadoop.hbase.master.handler.MetaServerShutdownHandler;
51  import org.apache.hadoop.hbase.master.handler.ServerShutdownHandler;
52  import org.apache.hadoop.hbase.monitoring.MonitoredTask;
53  import org.apache.hadoop.hbase.regionserver.RegionOpeningState;
54  
55  /**
56   * The ServerManager class manages info about region servers.
57   * <p>
58   * Maintains lists of online and dead servers.  Processes the startups,
59   * shutdowns, and deaths of region servers.
60   * <p>
61   * Servers are distinguished in two different ways.  A given server has a
62   * location, specified by hostname and port, and of which there can only be one
63   * online at any given time.  A server instance is specified by the location
64   * (hostname and port) as well as the startcode (timestamp from when the server
65   * was started).  This is used to differentiate a restarted instance of a given
66   * server from the original instance.
67   */
68  public class ServerManager {
69    public static final String WAIT_ON_REGIONSERVERS_MAXTOSTART =
70        "hbase.master.wait.on.regionservers.maxtostart";
71  
72    public static final String WAIT_ON_REGIONSERVERS_MINTOSTART =
73        "hbase.master.wait.on.regionservers.mintostart";
74  
75    public static final String WAIT_ON_REGIONSERVERS_TIMEOUT =
76        "hbase.master.wait.on.regionservers.timeout";
77  
78    public static final String WAIT_ON_REGIONSERVERS_INTERVAL =
79        "hbase.master.wait.on.regionservers.interval";
80  
81    private static final Log LOG = LogFactory.getLog(ServerManager.class);
82  
83    // Set if we are to shutdown the cluster.
84    private volatile boolean clusterShutdown = false;
85  
86    /** Map of registered servers to their current load */
87    private final Map<ServerName, HServerLoad> onlineServers =
88      new ConcurrentHashMap<ServerName, HServerLoad>();
89  
90    // TODO: This is strange to have two maps but HSI above is used on both sides
91    /**
92     * Map from full server-instance name to the RPC connection for this server.
93     */
94    private final Map<ServerName, HRegionInterface> serverConnections =
95      new HashMap<ServerName, HRegionInterface>();
96  
97    /**
98     * List of region servers <ServerName> that should not get any more new
99     * regions.
100    */
101   private final ArrayList<ServerName> drainingServers =
102     new ArrayList<ServerName>();
103 
104   private final Server master;
105   private final MasterServices services;
106   private final HConnection connection;
107 
108   private final DeadServer deadservers;
109 
110   private final long maxSkew;
111   private final long warningSkew;
112 
113   /**
114    * Set of region servers which are dead but not expired immediately. If one
115    * server died before master enables ServerShutdownHandler, the server will be
116    * added to set and will be expired through calling
117    * {@link ServerManager#expireDeadNotExpiredServers()} by master.
118    */
119   private Set<ServerName> deadNotExpiredServers = new HashSet<ServerName>();
120 
121   /**
122    * Constructor.
123    * @param master
124    * @param services
125    * @throws ZooKeeperConnectionException
126    */
127   public ServerManager(final Server master, final MasterServices services)
128       throws ZooKeeperConnectionException {
129     this(master, services, true);
130   }
131 
132   ServerManager(final Server master, final MasterServices services,
133       final boolean connect) throws ZooKeeperConnectionException {
134     this.master = master;
135     this.services = services;
136     Configuration c = master.getConfiguration();
137     maxSkew = c.getLong("hbase.master.maxclockskew", 30000);
138     warningSkew = c.getLong("hbase.master.warningclockskew", 10000);
139     this.deadservers = new DeadServer();
140     this.connection = connect ? HConnectionManager.getConnection(c) : null;
141   }
142 
143   /**
144    * Let the server manager know a new regionserver has come online
145    * @param ia The remote address
146    * @param port The remote port
147    * @param serverStartcode
148    * @param serverCurrentTime The current time of the region server in ms
149    * @return The ServerName we know this server as.
150    * @throws IOException
151    */
152   ServerName regionServerStartup(final InetAddress ia, final int port,
153     final long serverStartcode, long serverCurrentTime)
154   throws IOException {
155     // Test for case where we get a region startup message from a regionserver
156     // that has been quickly restarted but whose znode expiration handler has
157     // not yet run, or from a server whose fail we are currently processing.
158     // Test its host+port combo is present in serverAddresstoServerInfo.  If it
159     // is, reject the server and trigger its expiration. The next time it comes
160     // in, it should have been removed from serverAddressToServerInfo and queued
161     // for processing by ProcessServerShutdown.
162     ServerName sn = new ServerName(ia.getHostName(), port, serverStartcode);
163     checkClockSkew(sn, serverCurrentTime);
164     checkIsDead(sn, "STARTUP");
165     checkAlreadySameHostPort(sn);
166     recordNewServer(sn, HServerLoad.EMPTY_HSERVERLOAD);
167     return sn;
168   }
169 
170   void regionServerReport(ServerName sn, HServerLoad hsl)
171   throws YouAreDeadException, PleaseHoldException {
172     checkIsDead(sn, "REPORT");
173     if (!this.onlineServers.containsKey(sn)) {
174       // Already have this host+port combo and its just different start code?
175       checkAlreadySameHostPort(sn);
176       // Just let the server in. Presume master joining a running cluster.
177       // recordNewServer is what happens at the end of reportServerStartup.
178       // The only thing we are skipping is passing back to the regionserver
179       // the ServerName to use. Here we presume a master has already done
180       // that so we'll press on with whatever it gave us for ServerName.
181       recordNewServer(sn, hsl);
182     } else {
183       this.onlineServers.put(sn, hsl);
184     }
185   }
186 
187   /**
188    * Test to see if we have a server of same host and port already.
189    * @param serverName
190    * @throws PleaseHoldException
191    */
192   void checkAlreadySameHostPort(final ServerName serverName)
193   throws PleaseHoldException {
194     ServerName existingServer =
195       ServerName.findServerWithSameHostnamePort(getOnlineServersList(), serverName);
196     if (existingServer != null) {
197       String message = "Server serverName=" + serverName +
198         " rejected; we already have " + existingServer.toString() +
199         " registered with same hostname and port";
200       LOG.info(message);
201       if (existingServer.getStartcode() < serverName.getStartcode()) {
202         LOG.info("Triggering server recovery; existingServer " +
203           existingServer + " looks stale, new server:" + serverName);
204         expireServer(existingServer);
205       }
206       if (services.isServerShutdownHandlerEnabled()) {
207         // master has completed the initialization
208         throw new PleaseHoldException(message);
209       }
210     }
211   }
212 
213   /**
214    * Checks if the clock skew between the server and the master. If the clock skew exceeds the 
215    * configured max, it will throw an exception; if it exceeds the configured warning threshold, 
216    * it will log a warning but start normally.
217    * @param serverName Incoming servers's name
218    * @param serverCurrentTime
219    * @throws ClockOutOfSyncException if the skew exceeds the configured max value
220    */
221   private void checkClockSkew(final ServerName serverName, final long serverCurrentTime)
222   throws ClockOutOfSyncException {
223     long skew = System.currentTimeMillis() - serverCurrentTime;
224     if (skew > maxSkew) {
225       String message = "Server " + serverName + " has been " +
226         "rejected; Reported time is too far out of sync with master.  " +
227         "Time difference of " + skew + "ms > max allowed of " + maxSkew + "ms";
228       LOG.warn(message);
229       throw new ClockOutOfSyncException(message);
230     } else if (skew > warningSkew){
231       String message = "Reported time for server " + serverName + " is out of sync with master " +
232         "by " + skew + "ms. (Warning threshold is " + warningSkew + "ms; " + 
233         "error threshold is " + maxSkew + "ms)";
234       LOG.warn(message);
235     }
236   }
237 
238   /**
239    * If this server is on the dead list, reject it with a YouAreDeadException.
240    * If it was dead but came back with a new start code, remove the old entry
241    * from the dead list.
242    * @param serverName
243    * @param what START or REPORT
244    * @throws YouAreDeadException
245    */
246   private void checkIsDead(final ServerName serverName, final String what)
247       throws YouAreDeadException {
248     if (this.deadservers.isDeadServer(serverName)) {
249       // host name, port and start code all match with existing one of the
250       // dead servers. So, this server must be dead.
251       String message = "Server " + what + " rejected; currently processing " +
252           serverName + " as dead server";
253       LOG.debug(message);
254       throw new YouAreDeadException(message);
255     }
256 
257     // remove dead server with same hostname and port of newly checking in rs after master
258     // initialization.See HBASE-5916 for more information.
259     if ((this.services == null || ((HMaster) this.services).isInitialized())
260         && this.deadservers.cleanPreviousInstance(serverName)) {
261       // This server has now become alive after we marked it as dead.
262       // We removed it's previous entry from the dead list to reflect it.
263       LOG.debug(what + ":" + " Server " + serverName + " came back up," +
264           " removed it from the dead servers list");
265     }
266   }
267 
268   /**
269    * Adds the onlineServers list.
270    * @param hsl
271    * @param serverName The remote servers name.
272    */
273   void recordNewServer(final ServerName serverName, final  HServerLoad hsl) {
274     LOG.info("Registering server=" + serverName);
275     this.onlineServers.put(serverName, hsl);
276     this.serverConnections.remove(serverName);
277   }
278 
279   /**
280    * @param serverName
281    * @return HServerLoad if serverName is known else null
282    */
283   public HServerLoad getLoad(final ServerName serverName) {
284     return this.onlineServers.get(serverName);
285   }
286 
287   /**
288    * @param address
289    * @return HServerLoad if serverName is known else null
290    * @deprecated Use {@link #getLoad(HServerAddress)}
291    */
292   public HServerLoad getLoad(final HServerAddress address) {
293     ServerName sn = new ServerName(address.toString(), ServerName.NON_STARTCODE);
294     ServerName actual =
295       ServerName.findServerWithSameHostnamePort(this.getOnlineServersList(), sn);
296     return actual == null? null: getLoad(actual);
297   }
298 
299   /**
300    * Compute the average load across all region servers.
301    * Currently, this uses a very naive computation - just uses the number of
302    * regions being served, ignoring stats about number of requests.
303    * @return the average load
304    */
305   public double getAverageLoad() {
306     int totalLoad = 0;
307     int numServers = 0;
308     double averageLoad = 0.0;
309     for (HServerLoad hsl: this.onlineServers.values()) {
310         numServers++;
311         totalLoad += hsl.getNumberOfRegions();
312     }
313     averageLoad = (double)totalLoad / (double)numServers;
314     return averageLoad;
315   }
316 
317   /** @return the count of active regionservers */
318   int countOfRegionServers() {
319     // Presumes onlineServers is a concurrent map
320     return this.onlineServers.size();
321   }
322 
323   /**
324    * @return Read-only map of servers to serverinfo
325    */
326   public Map<ServerName, HServerLoad> getOnlineServers() {
327     // Presumption is that iterating the returned Map is OK.
328     synchronized (this.onlineServers) {
329       return Collections.unmodifiableMap(this.onlineServers);
330     }
331   }
332 
333   public Set<ServerName> getDeadServers() {
334     return this.deadservers.clone();
335   }
336 
337   /**
338    * Checks if any dead servers are currently in progress.
339    * @return true if any RS are being processed as dead, false if not
340    */
341   public boolean areDeadServersInProgress() {
342     return this.deadservers.areDeadServersInProgress();
343   }
344 
345   void letRegionServersShutdown() {
346     long previousLogTime = 0;
347     while (!onlineServers.isEmpty()) {
348 
349       if (System.currentTimeMillis() > (previousLogTime + 1000)) {
350         StringBuilder sb = new StringBuilder();
351         for (ServerName key : this.onlineServers.keySet()) {
352           if (sb.length() > 0) {
353             sb.append(", ");
354           }
355           sb.append(key);
356         }
357         LOG.info("Waiting on regionserver(s) to go down " + sb.toString());
358         previousLogTime = System.currentTimeMillis();
359       }
360 
361       synchronized (onlineServers) {
362         try {
363           onlineServers.wait(100);
364         } catch (InterruptedException ignored) {
365           // continue
366         }
367       }
368     }
369   }
370 
371   /*
372    * Expire the passed server.  Add it to list of deadservers and queue a
373    * shutdown processing.
374    */
375   public synchronized void expireServer(final ServerName serverName) {
376     if (!services.isServerShutdownHandlerEnabled()) {
377       LOG.info("Master doesn't enable ServerShutdownHandler during initialization, "
378           + "delay expiring server " + serverName);
379       this.deadNotExpiredServers.add(serverName);
380       return;
381     }
382     if (!this.onlineServers.containsKey(serverName)) {
383       LOG.warn("Received expiration of " + serverName +
384         " but server is not currently online");
385       return;
386     }
387     if (this.deadservers.contains(serverName)) {
388       // TODO: Can this happen?  It shouldn't be online in this case?
389       LOG.warn("Received expiration of " + serverName +
390           " but server shutdown is already in progress");
391       return;
392     }
393     // Remove the server from the known servers lists and update load info BUT
394     // add to deadservers first; do this so it'll show in dead servers list if
395     // not in online servers list.
396     this.deadservers.add(serverName);
397     this.onlineServers.remove(serverName);
398     synchronized (onlineServers) {
399       onlineServers.notifyAll();
400     }
401     this.serverConnections.remove(serverName);
402     // If cluster is going down, yes, servers are going to be expiring; don't
403     // process as a dead server
404     if (this.clusterShutdown) {
405       LOG.info("Cluster shutdown set; " + serverName +
406         " expired; onlineServers=" + this.onlineServers.size());
407       if (this.onlineServers.isEmpty()) {
408         master.stop("Cluster shutdown set; onlineServer=0");
409       }
410       return;
411     }
412 
413     boolean carryingRoot = services.getAssignmentManager().isCarryingRoot(serverName);
414     boolean carryingMeta = services.getAssignmentManager().isCarryingMeta(serverName);
415     if (carryingRoot || carryingMeta) {
416       this.services.getExecutorService().submit(new MetaServerShutdownHandler(this.master,
417         this.services, this.deadservers, serverName, carryingRoot, carryingMeta));
418     } else {
419       this.services.getExecutorService().submit(new ServerShutdownHandler(this.master,
420         this.services, this.deadservers, serverName, true));
421     }
422     LOG.debug("Added=" + serverName +
423       " to dead servers, submitted shutdown handler to be executed, root=" +
424         carryingRoot + ", meta=" + carryingMeta);
425   }
426 
427   /**
428    * Expire the servers which died during master's initialization. It will be
429    * called after HMaster#assignRootAndMeta.
430    * @throws IOException
431    * */
432   synchronized void expireDeadNotExpiredServers() throws IOException {
433     if (!services.isServerShutdownHandlerEnabled()) {
434       throw new IOException("Master hasn't enabled ServerShutdownHandler ");
435     }
436     Iterator<ServerName> serverIterator = deadNotExpiredServers.iterator();
437     while (serverIterator.hasNext()) {
438       expireServer(serverIterator.next());
439       serverIterator.remove();
440     }
441   }
442 
443   /*
444    * Remove the server from the drain list.
445    */
446   public boolean removeServerFromDrainList(final ServerName sn) {
447     // Warn if the server (sn) is not online.  ServerName is of the form:
448     // <hostname> , <port> , <startcode>
449 
450     if (!this.isServerOnline(sn)) {
451       LOG.warn("Server " + sn + " is not currently online. " +
452                "Removing from draining list anyway, as requested.");
453     }
454     // Remove the server from the draining servers lists.
455     return this.drainingServers.remove(sn);
456   }
457 
458   /*
459    * Add the server to the drain list.
460    */
461   public boolean addServerToDrainList(final ServerName sn) {
462     // Warn if the server (sn) is not online.  ServerName is of the form:
463     // <hostname> , <port> , <startcode>
464 
465     if (!this.isServerOnline(sn)) {
466       LOG.warn("Server " + sn + " is not currently online. " +
467                "Ignoring request to add it to draining list.");
468       return false;
469     }
470     // Add the server to the draining servers lists, if it's not already in
471     // it.
472     if (this.drainingServers.contains(sn)) {
473       LOG.warn("Server " + sn + " is already in the draining server list." +
474                "Ignoring request to add it again.");
475       return false;
476     }
477     return this.drainingServers.add(sn);
478   }
479 
480   // RPC methods to region servers
481 
482   /**
483    * Sends an OPEN RPC to the specified server to open the specified region.
484    * <p>
485    * Open should not fail but can if server just crashed.
486    * <p>
487    * @param server server to open a region
488    * @param region region to open
489    * @param versionOfOfflineNode that needs to be present in the offline node
490    * when RS tries to change the state from OFFLINE to other states.
491    */
492   public RegionOpeningState sendRegionOpen(final ServerName server,
493       HRegionInfo region, int versionOfOfflineNode)
494   throws IOException {
495     HRegionInterface hri = getServerConnection(server);
496     if (hri == null) {
497       LOG.warn("Attempting to send OPEN RPC to server " + server.toString() +
498         " failed because no RPC connection found to this server");
499       return RegionOpeningState.FAILED_OPENING;
500     }
501     return (versionOfOfflineNode == -1) ? hri.openRegion(region) : hri
502         .openRegion(region, versionOfOfflineNode);
503   }
504 
505   /**
506    * Sends an OPEN RPC to the specified server to open the specified region.
507    * <p>
508    * Open should not fail but can if server just crashed.
509    * <p>
510    * @param server server to open a region
511    * @param regions regions to open
512    */
513   public void sendRegionOpen(ServerName server, List<HRegionInfo> regions)
514   throws IOException {
515     HRegionInterface hri = getServerConnection(server);
516     if (hri == null) {
517       LOG.warn("Attempting to send OPEN RPC to server " + server.toString() +
518         " failed because no RPC connection found to this server");
519       return;
520     }
521     hri.openRegions(regions);
522   }
523 
524   /**
525    * Sends an CLOSE RPC to the specified server to close the specified region.
526    * <p>
527    * A region server could reject the close request because it either does not
528    * have the specified region or the region is being split.
529    * @param server server to open a region
530    * @param region region to open
531    * @param versionOfClosingNode
532    *   the version of znode to compare when RS transitions the znode from
533    *   CLOSING state.
534    * @return true if server acknowledged close, false if not
535    * @throws IOException
536    */
537   public boolean sendRegionClose(ServerName server, HRegionInfo region,
538     int versionOfClosingNode) throws IOException {
539     if (server == null) throw new NullPointerException("Passed server is null");
540     HRegionInterface hri = getServerConnection(server);
541     if (hri == null) {
542       throw new IOException("Attempting to send CLOSE RPC to server " +
543         server.toString() + " for region " +
544         region.getRegionNameAsString() +
545         " failed because no RPC connection found to this server");
546     }
547     return hri.closeRegion(region, versionOfClosingNode);
548   }
549 
550   /**
551    * @param sn
552    * @return
553    * @throws IOException
554    * @throws RetriesExhaustedException wrapping a ConnectException if failed
555    * putting up proxy.
556    */
557   private HRegionInterface getServerConnection(final ServerName sn)
558   throws IOException {
559     HRegionInterface hri = this.serverConnections.get(sn);
560     if (hri == null) {
561       LOG.debug("New connection to " + sn.toString());
562       hri = this.connection.getHRegionConnection(sn.getHostname(), sn.getPort());
563       this.serverConnections.put(sn, hri);
564     }
565     return hri;
566   }
567 
568   /**
569    * Wait for the region servers to report in.
570    * We will wait until one of this condition is met:
571    *  - the master is stopped
572    *  - the 'hbase.master.wait.on.regionservers.maxtostart' number of
573    *    region servers is reached
574    *  - the 'hbase.master.wait.on.regionservers.mintostart' is reached AND
575    *   there have been no new region server in for
576    *      'hbase.master.wait.on.regionservers.interval' time AND
577    *   the 'hbase.master.wait.on.regionservers.timeout' is reached
578    *
579    * @throws InterruptedException
580    */
581   public void waitForRegionServers(MonitoredTask status)
582   throws InterruptedException {
583     final long interval = this.master.getConfiguration().
584       getLong(WAIT_ON_REGIONSERVERS_INTERVAL, 1500);
585     final long timeout = this.master.getConfiguration().
586       getLong(WAIT_ON_REGIONSERVERS_TIMEOUT, 4500);
587     int minToStart = this.master.getConfiguration().
588       getInt(WAIT_ON_REGIONSERVERS_MINTOSTART, 1);
589     if (minToStart < 1) {
590       LOG.warn(String.format(
591         "The value of '%s' (%d) can not be less than 1, ignoring.",
592         WAIT_ON_REGIONSERVERS_MINTOSTART, minToStart));
593       minToStart = 1;
594     }
595     int maxToStart = this.master.getConfiguration().
596       getInt(WAIT_ON_REGIONSERVERS_MAXTOSTART, Integer.MAX_VALUE);
597     if (maxToStart < minToStart) {
598         LOG.warn(String.format(
599             "The value of '%s' (%d) is set less than '%s' (%d), ignoring.",
600             WAIT_ON_REGIONSERVERS_MAXTOSTART, maxToStart,
601             WAIT_ON_REGIONSERVERS_MINTOSTART, minToStart));
602         maxToStart = Integer.MAX_VALUE;
603     }
604 
605     long now =  System.currentTimeMillis();
606     final long startTime = now;
607     long slept = 0;
608     long lastLogTime = 0;
609     long lastCountChange = startTime;
610     int count = countOfRegionServers();
611     int oldCount = 0;
612     while (
613       !this.master.isStopped() &&
614         count < maxToStart &&
615         (lastCountChange+interval > now || timeout > slept || count < minToStart)
616       ){
617 
618       // Log some info at every interval time or if there is a change
619       if (oldCount != count || lastLogTime+interval < now){
620         lastLogTime = now;
621         String msg =
622           "Waiting for region servers count to settle; currently"+
623             " checked in " + count + ", slept for " + slept + " ms," +
624             " expecting minimum of " + minToStart + ", maximum of "+ maxToStart+
625             ", timeout of "+timeout+" ms, interval of "+interval+" ms.";
626         LOG.info(msg);
627         status.setStatus(msg);
628       }
629 
630       // We sleep for some time
631       final long sleepTime = 50;
632       Thread.sleep(sleepTime);
633       now =  System.currentTimeMillis();
634       slept = now - startTime;
635 
636       oldCount = count;
637       count = countOfRegionServers();
638       if (count != oldCount) {
639         lastCountChange = now;
640       }
641     }
642 
643     LOG.info("Finished waiting for region servers count to settle;" +
644       " checked in " + count + ", slept for " + slept + " ms," +
645       " expecting minimum of " + minToStart + ", maximum of "+ maxToStart+","+
646       " master is "+ (this.master.isStopped() ? "stopped.": "running.")
647     );
648   }
649 
650   /**
651    * @return A copy of the internal list of online servers.
652    */
653   public List<ServerName> getOnlineServersList() {
654     // TODO: optimize the load balancer call so we don't need to make a new list
655     // TODO: FIX. THIS IS POPULAR CALL.
656     return new ArrayList<ServerName>(this.onlineServers.keySet());
657   }
658 
659   /**
660    * @return A copy of the internal list of draining servers.
661    */
662   public List<ServerName> getDrainingServersList() {
663     return new ArrayList<ServerName>(this.drainingServers);
664   }
665 
666   /**
667    * @return A copy of the internal set of deadNotExpired servers.
668    */
669   Set<ServerName> getDeadNotExpiredServers() {
670     return new HashSet<ServerName>(this.deadNotExpiredServers);
671   }
672 
673   public boolean isServerOnline(ServerName serverName) {
674     return onlineServers.containsKey(serverName);
675   }
676 
677   public void shutdownCluster() {
678     this.clusterShutdown = true;
679     this.master.stop("Cluster shutdown requested");
680   }
681 
682   public boolean isClusterShutdown() {
683     return this.clusterShutdown;
684   }
685 
686   /**
687    * Stop the ServerManager.  Currently closes the connection to the master.
688    */
689   public void stop() {
690     if (connection != null) {
691       try {
692         connection.close();
693       } catch (IOException e) {
694         LOG.error("Attempt to close connection to master failed", e);
695       }
696     }
697   }
698     
699   /**
700    * To clear any dead server with same host name and port of any online server
701    */
702   void clearDeadServersWithSameHostNameAndPortOfOnlineServer() {
703     ServerName sn = null;
704     for (ServerName serverName : getOnlineServersList()) {
705       while ((sn = ServerName.
706           findServerWithSameHostnamePort(this.deadservers, serverName)) != null) {
707         this.deadservers.remove(sn);
708       }
709     }
710   }
711 
712 }