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