1 /**
2  *
3  * Licensed to the Apache Software Foundation (ASF) under one
4  * or more contributor license agreements.  See the NOTICE file
5  * distributed with this work for additional information
6  * regarding copyright ownership.  The ASF licenses this file
7  * to you under the Apache License, Version 2.0 (the
8  * "License"); you may not use this file except in compliance
9  * with the License.  You may obtain a copy of the License at
10  *
11  *     http://www.apache.org/licenses/LICENSE-2.0
12  *
13  * Unless required by applicable law or agreed to in writing, software
14  * distributed under the License is distributed on an "AS IS" BASIS,
15  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16  * See the License for the specific language governing permissions and
17  * limitations under the License.
18  */
19 package org.apache.hadoop.hbase.master;
20 
21 import java.io.IOException;
22 import java.net.InetAddress;
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.Map.Entry;
31 import java.util.Set;
32 import java.util.concurrent.ConcurrentHashMap;
33 import java.util.concurrent.ConcurrentNavigableMap;
34 import java.util.concurrent.ConcurrentSkipListMap;
35 import java.util.concurrent.CopyOnWriteArrayList;
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.HConstants;
42 import org.apache.hadoop.hbase.HRegionInfo;
43 import org.apache.hadoop.hbase.NotServingRegionException;
44 import org.apache.hadoop.hbase.RegionLoad;
45 import org.apache.hadoop.hbase.Server;
46 import org.apache.hadoop.hbase.ServerLoad;
47 import org.apache.hadoop.hbase.ServerName;
48 import org.apache.hadoop.hbase.YouAreDeadException;
49 import org.apache.hadoop.hbase.ZooKeeperConnectionException;
50 import org.apache.hadoop.hbase.classification.InterfaceAudience;
51 import org.apache.hadoop.hbase.client.ClusterConnection;
52 import org.apache.hadoop.hbase.client.ConnectionFactory;
53 import org.apache.hadoop.hbase.client.RetriesExhaustedException;
54 import org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController;
55 import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
56 import org.apache.hadoop.hbase.master.balancer.BaseLoadBalancer;
57 import org.apache.hadoop.hbase.master.procedure.ServerCrashProcedure;
58 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
59 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
60 import org.apache.hadoop.hbase.protobuf.RequestConverter;
61 import org.apache.hadoop.hbase.protobuf.ResponseConverter;
62 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
63 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionRequest;
64 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionResponse;
65 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ServerInfo;
66 import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest;
67 import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds;
68 import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.StoreSequenceId;
69 import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
70 import org.apache.hadoop.hbase.regionserver.HRegionServer;
71 import org.apache.hadoop.hbase.regionserver.RegionOpeningState;
72 import org.apache.hadoop.hbase.util.Bytes;
73 import org.apache.hadoop.hbase.util.Triple;
74 import org.apache.hadoop.hbase.util.RetryCounter;
75 import org.apache.hadoop.hbase.util.RetryCounterFactory;
76 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
77 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
78 import org.apache.zookeeper.KeeperException;
79 
80 import com.google.common.annotations.VisibleForTesting;
81 import com.google.protobuf.ByteString;
82 import com.google.protobuf.ServiceException;
83 
84 /**
85  * The ServerManager class manages info about region servers.
86  * <p>
87  * Maintains lists of online and dead servers.  Processes the startups,
88  * shutdowns, and deaths of region servers.
89  * <p>
90  * Servers are distinguished in two different ways.  A given server has a
91  * location, specified by hostname and port, and of which there can only be one
92  * online at any given time.  A server instance is specified by the location
93  * (hostname and port) as well as the startcode (timestamp from when the server
94  * was started).  This is used to differentiate a restarted instance of a given
95  * server from the original instance.
96  * <p>
97  * If a sever is known not to be running any more, it is called dead. The dead
98  * server needs to be handled by a ServerShutdownHandler.  If the handler is not
99  * enabled yet, the server can't be handled right away so it is queued up.
100  * After the handler is enabled, the server will be submitted to a handler to handle.
101  * However, the handler may be just partially enabled.  If so,
102  * the server cannot be fully processed, and be queued up for further processing.
103  * A server is fully processed only after the handler is fully enabled
104  * and has completed the handling.
105  */
106 @InterfaceAudience.Private
107 public class ServerManager {
108   public static final String WAIT_ON_REGIONSERVERS_MAXTOSTART =
109       "hbase.master.wait.on.regionservers.maxtostart";
110 
111   public static final String WAIT_ON_REGIONSERVERS_MINTOSTART =
112       "hbase.master.wait.on.regionservers.mintostart";
113 
114   public static final String WAIT_ON_REGIONSERVERS_TIMEOUT =
115       "hbase.master.wait.on.regionservers.timeout";
116 
117   public static final String WAIT_ON_REGIONSERVERS_INTERVAL =
118       "hbase.master.wait.on.regionservers.interval";
119 
120   private static final Log LOG = LogFactory.getLog(ServerManager.class);
121 
122   // Set if we are to shutdown the cluster.
123   private volatile boolean clusterShutdown = false;
124 
125   /**
126    * The last flushed sequence id for a region.
127    */
128   private final ConcurrentNavigableMap<byte[], Long> flushedSequenceIdByRegion =
129     new ConcurrentSkipListMap<byte[], Long>(Bytes.BYTES_COMPARATOR);
130 
131   /**
132    * The last flushed sequence id for a store in a region.
133    */
134   private final ConcurrentNavigableMap<byte[], ConcurrentNavigableMap<byte[], Long>>
135     storeFlushedSequenceIdsByRegion =
136     new ConcurrentSkipListMap<byte[], ConcurrentNavigableMap<byte[], Long>>(Bytes.BYTES_COMPARATOR);
137 
138   /** Map of registered servers to their current load */
139   private final ConcurrentHashMap<ServerName, ServerLoad> onlineServers =
140     new ConcurrentHashMap<ServerName, ServerLoad>();
141 
142   /**
143    * Map of admin interfaces per registered regionserver; these interfaces we use to control
144    * regionservers out on the cluster
145    */
146   private final Map<ServerName, AdminService.BlockingInterface> rsAdmins =
147     new HashMap<ServerName, AdminService.BlockingInterface>();
148 
149   /**
150    * List of region servers <ServerName> that should not get any more new
151    * regions.
152    */
153   private final ArrayList<ServerName> drainingServers =
154     new ArrayList<ServerName>();
155 
156   private final Server master;
157   private final MasterServices services;
158   private final ClusterConnection connection;
159 
160   private final DeadServer deadservers = new DeadServer();
161 
162   private final long maxSkew;
163   private final long warningSkew;
164 
165   private final RetryCounterFactory pingRetryCounterFactory;
166   private final RpcControllerFactory rpcControllerFactory;
167 
168   /**
169    * Set of region servers which are dead but not processed immediately. If one
170    * server died before master enables ServerShutdownHandler, the server will be
171    * added to this set and will be processed through calling
172    * {@link ServerManager#processQueuedDeadServers()} by master.
173    * <p>
174    * A dead server is a server instance known to be dead, not listed in the /hbase/rs
175    * znode any more. It may have not been submitted to ServerShutdownHandler yet
176    * because the handler is not enabled.
177    * <p>
178    * A dead server, which has been submitted to ServerShutdownHandler while the
179    * handler is not enabled, is queued up.
180    * <p>
181    * So this is a set of region servers known to be dead but not submitted to
182    * ServerShutdownHander for processing yet.
183    */
184   private Set<ServerName> queuedDeadServers = new HashSet<ServerName>();
185 
186   /**
187    * Set of region servers which are dead and submitted to ServerShutdownHandler to process but not
188    * fully processed immediately.
189    * <p>
190    * If one server died before assignment manager finished the failover cleanup, the server will be
191    * added to this set and will be processed through calling
192    * {@link ServerManager#processQueuedDeadServers()} by assignment manager.
193    * <p>
194    * The Boolean value indicates whether log split is needed inside ServerShutdownHandler
195    * <p>
196    * ServerShutdownHandler processes a dead server submitted to the handler after the handler is
197    * enabled. It may not be able to complete the processing because meta is not yet online or master
198    * is currently in startup mode. In this case, the dead server will be parked in this set
199    * temporarily.
200    */
201   private Map<ServerName, Boolean> requeuedDeadServers
202     = new ConcurrentHashMap<ServerName, Boolean>();
203 
204   /** Listeners that are called on server events. */
205   private List<ServerListener> listeners = new CopyOnWriteArrayList<ServerListener>();
206 
207   /**
208    * Constructor.
209    * @param master
210    * @param services
211    * @throws ZooKeeperConnectionException
212    */
ServerManager(final Server master, final MasterServices services)213   public ServerManager(final Server master, final MasterServices services)
214       throws IOException {
215     this(master, services, true);
216   }
217 
ServerManager(final Server master, final MasterServices services, final boolean connect)218   ServerManager(final Server master, final MasterServices services,
219       final boolean connect) throws IOException {
220     this.master = master;
221     this.services = services;
222     Configuration c = master.getConfiguration();
223     maxSkew = c.getLong("hbase.master.maxclockskew", 30000);
224     warningSkew = c.getLong("hbase.master.warningclockskew", 10000);
225     this.connection = connect ? (ClusterConnection)ConnectionFactory.createConnection(c) : null;
226     int pingMaxAttempts = Math.max(1, master.getConfiguration().getInt(
227       "hbase.master.maximum.ping.server.attempts", 10));
228     int pingSleepInterval = Math.max(1, master.getConfiguration().getInt(
229       "hbase.master.ping.server.retry.sleep.interval", 100));
230     this.pingRetryCounterFactory = new RetryCounterFactory(pingMaxAttempts, pingSleepInterval);
231     this.rpcControllerFactory = this.connection == null
232         ? null
233         : connection.getRpcControllerFactory();
234   }
235 
236   /**
237    * Add the listener to the notification list.
238    * @param listener The ServerListener to register
239    */
registerListener(final ServerListener listener)240   public void registerListener(final ServerListener listener) {
241     this.listeners.add(listener);
242   }
243 
244   /**
245    * Remove the listener from the notification list.
246    * @param listener The ServerListener to unregister
247    */
unregisterListener(final ServerListener listener)248   public boolean unregisterListener(final ServerListener listener) {
249     return this.listeners.remove(listener);
250   }
251 
252   /**
253    * Let the server manager know a new regionserver has come online
254    * @param request the startup request
255    * @param ia the InetAddress from which request is received
256    * @return The ServerName we know this server as.
257    * @throws IOException
258    */
regionServerStartup(RegionServerStartupRequest request, InetAddress ia)259   ServerName regionServerStartup(RegionServerStartupRequest request, InetAddress ia)
260       throws IOException {
261     // Test for case where we get a region startup message from a regionserver
262     // that has been quickly restarted but whose znode expiration handler has
263     // not yet run, or from a server whose fail we are currently processing.
264     // Test its host+port combo is present in serverAddresstoServerInfo.  If it
265     // is, reject the server and trigger its expiration. The next time it comes
266     // in, it should have been removed from serverAddressToServerInfo and queued
267     // for processing by ProcessServerShutdown.
268 
269     final String hostname = request.hasUseThisHostnameInstead() ?
270         request.getUseThisHostnameInstead() :ia.getHostName();
271     ServerName sn = ServerName.valueOf(hostname, request.getPort(),
272       request.getServerStartCode());
273     checkClockSkew(sn, request.getServerCurrentTime());
274     checkIsDead(sn, "STARTUP");
275     if (!checkAndRecordNewServer(sn, ServerLoad.EMPTY_SERVERLOAD)) {
276       LOG.warn("THIS SHOULD NOT HAPPEN, RegionServerStartup"
277         + " could not record the server: " + sn);
278     }
279     return sn;
280   }
281 
getOrCreateStoreFlushedSequenceId( byte[] regionName)282   private ConcurrentNavigableMap<byte[], Long> getOrCreateStoreFlushedSequenceId(
283     byte[] regionName) {
284     ConcurrentNavigableMap<byte[], Long> storeFlushedSequenceId =
285         storeFlushedSequenceIdsByRegion.get(regionName);
286     if (storeFlushedSequenceId != null) {
287       return storeFlushedSequenceId;
288     }
289     storeFlushedSequenceId = new ConcurrentSkipListMap<byte[], Long>(Bytes.BYTES_COMPARATOR);
290     ConcurrentNavigableMap<byte[], Long> alreadyPut =
291         storeFlushedSequenceIdsByRegion.putIfAbsent(regionName, storeFlushedSequenceId);
292     return alreadyPut == null ? storeFlushedSequenceId : alreadyPut;
293   }
294   /**
295    * Updates last flushed sequence Ids for the regions on server sn
296    * @param sn
297    * @param hsl
298    */
updateLastFlushedSequenceIds(ServerName sn, ServerLoad hsl)299   private void updateLastFlushedSequenceIds(ServerName sn, ServerLoad hsl) {
300     Map<byte[], RegionLoad> regionsLoad = hsl.getRegionsLoad();
301     for (Entry<byte[], RegionLoad> entry : regionsLoad.entrySet()) {
302       byte[] encodedRegionName = Bytes.toBytes(HRegionInfo.encodeRegionName(entry.getKey()));
303       Long existingValue = flushedSequenceIdByRegion.get(encodedRegionName);
304       long l = entry.getValue().getCompleteSequenceId();
305       // Don't let smaller sequence ids override greater sequence ids.
306       if (LOG.isTraceEnabled()) {
307         LOG.trace(Bytes.toString(encodedRegionName) + ", existingValue=" + existingValue +
308           ", completeSequenceId=" + l);
309       }
310       if (existingValue == null || (l != HConstants.NO_SEQNUM && l > existingValue)) {
311         flushedSequenceIdByRegion.put(encodedRegionName, l);
312       } else if (l != HConstants.NO_SEQNUM && l < existingValue) {
313         LOG.warn("RegionServer " + sn + " indicates a last flushed sequence id ("
314             + l + ") that is less than the previous last flushed sequence id ("
315             + existingValue + ") for region " + Bytes.toString(entry.getKey()) + " Ignoring.");
316       }
317       ConcurrentNavigableMap<byte[], Long> storeFlushedSequenceId =
318           getOrCreateStoreFlushedSequenceId(encodedRegionName);
319       for (StoreSequenceId storeSeqId : entry.getValue().getStoreCompleteSequenceId()) {
320         byte[] family = storeSeqId.getFamilyName().toByteArray();
321         existingValue = storeFlushedSequenceId.get(family);
322         l = storeSeqId.getSequenceId();
323         if (LOG.isTraceEnabled()) {
324           LOG.trace(Bytes.toString(encodedRegionName) + ", family=" + Bytes.toString(family) +
325             ", existingValue=" + existingValue + ", completeSequenceId=" + l);
326         }
327         // Don't let smaller sequence ids override greater sequence ids.
328         if (existingValue == null || (l != HConstants.NO_SEQNUM && l > existingValue.longValue())) {
329           storeFlushedSequenceId.put(family, l);
330         }
331       }
332     }
333   }
334 
regionServerReport(ServerName sn, ServerLoad sl)335   void regionServerReport(ServerName sn,
336       ServerLoad sl) throws YouAreDeadException {
337     checkIsDead(sn, "REPORT");
338     if (null == this.onlineServers.replace(sn, sl)) {
339       // Already have this host+port combo and its just different start code?
340       // Just let the server in. Presume master joining a running cluster.
341       // recordNewServer is what happens at the end of reportServerStartup.
342       // The only thing we are skipping is passing back to the regionserver
343       // the ServerName to use. Here we presume a master has already done
344       // that so we'll press on with whatever it gave us for ServerName.
345       if (!checkAndRecordNewServer(sn, sl)) {
346         LOG.info("RegionServerReport ignored, could not record the server: " + sn);
347         return; // Not recorded, so no need to move on
348       }
349     }
350     updateLastFlushedSequenceIds(sn, sl);
351   }
352 
353   /**
354    * Check is a server of same host and port already exists,
355    * if not, or the existed one got a smaller start code, record it.
356    *
357    * @param sn the server to check and record
358    * @param sl the server load on the server
359    * @return true if the server is recorded, otherwise, false
360    */
checkAndRecordNewServer( final ServerName serverName, final ServerLoad sl)361   boolean checkAndRecordNewServer(
362       final ServerName serverName, final ServerLoad sl) {
363     ServerName existingServer = null;
364     synchronized (this.onlineServers) {
365       existingServer = findServerWithSameHostnamePortWithLock(serverName);
366       if (existingServer != null && (existingServer.getStartcode() > serverName.getStartcode())) {
367         LOG.info("Server serverName=" + serverName + " rejected; we already have "
368             + existingServer.toString() + " registered with same hostname and port");
369         return false;
370       }
371       recordNewServerWithLock(serverName, sl);
372     }
373 
374     // Tell our listeners that a server was added
375     if (!this.listeners.isEmpty()) {
376       for (ServerListener listener : this.listeners) {
377         listener.serverAdded(serverName);
378       }
379     }
380 
381     // Note that we assume that same ts means same server, and don't expire in that case.
382     //  TODO: ts can theoretically collide due to clock shifts, so this is a bit hacky.
383     if (existingServer != null && (existingServer.getStartcode() < serverName.getStartcode())) {
384       LOG.info("Triggering server recovery; existingServer " +
385           existingServer + " looks stale, new server:" + serverName);
386       expireServer(existingServer);
387     }
388     return true;
389   }
390 
391   /**
392    * Checks if the clock skew between the server and the master. If the clock skew exceeds the
393    * configured max, it will throw an exception; if it exceeds the configured warning threshold,
394    * it will log a warning but start normally.
395    * @param serverName Incoming servers's name
396    * @param serverCurrentTime
397    * @throws ClockOutOfSyncException if the skew exceeds the configured max value
398    */
checkClockSkew(final ServerName serverName, final long serverCurrentTime)399   private void checkClockSkew(final ServerName serverName, final long serverCurrentTime)
400   throws ClockOutOfSyncException {
401     long skew = Math.abs(System.currentTimeMillis() - serverCurrentTime);
402     if (skew > maxSkew) {
403       String message = "Server " + serverName + " has been " +
404         "rejected; Reported time is too far out of sync with master.  " +
405         "Time difference of " + skew + "ms > max allowed of " + maxSkew + "ms";
406       LOG.warn(message);
407       throw new ClockOutOfSyncException(message);
408     } else if (skew > warningSkew){
409       String message = "Reported time for server " + serverName + " is out of sync with master " +
410         "by " + skew + "ms. (Warning threshold is " + warningSkew + "ms; " +
411         "error threshold is " + maxSkew + "ms)";
412       LOG.warn(message);
413     }
414   }
415 
416   /**
417    * If this server is on the dead list, reject it with a YouAreDeadException.
418    * If it was dead but came back with a new start code, remove the old entry
419    * from the dead list.
420    * @param serverName
421    * @param what START or REPORT
422    * @throws org.apache.hadoop.hbase.YouAreDeadException
423    */
checkIsDead(final ServerName serverName, final String what)424   private void checkIsDead(final ServerName serverName, final String what)
425       throws YouAreDeadException {
426     if (this.deadservers.isDeadServer(serverName)) {
427       // host name, port and start code all match with existing one of the
428       // dead servers. So, this server must be dead.
429       String message = "Server " + what + " rejected; currently processing " +
430           serverName + " as dead server";
431       LOG.debug(message);
432       throw new YouAreDeadException(message);
433     }
434     // remove dead server with same hostname and port of newly checking in rs after master
435     // initialization.See HBASE-5916 for more information.
436     if ((this.services == null || ((HMaster) this.services).isInitialized())
437         && this.deadservers.cleanPreviousInstance(serverName)) {
438       // This server has now become alive after we marked it as dead.
439       // We removed it's previous entry from the dead list to reflect it.
440       LOG.debug(what + ":" + " Server " + serverName + " came back up," +
441           " removed it from the dead servers list");
442     }
443   }
444 
445   /**
446    * Assumes onlineServers is locked.
447    * @return ServerName with matching hostname and port.
448    */
findServerWithSameHostnamePortWithLock( final ServerName serverName)449   private ServerName findServerWithSameHostnamePortWithLock(
450       final ServerName serverName) {
451     for (ServerName sn: this.onlineServers.keySet()) {
452       if (ServerName.isSameHostnameAndPort(serverName, sn)) return sn;
453     }
454     return null;
455   }
456 
457   /**
458    * Adds the onlineServers list. onlineServers should be locked.
459    * @param serverName The remote servers name.
460    * @param sl
461    * @return Server load from the removed server, if any.
462    */
463   @VisibleForTesting
recordNewServerWithLock(final ServerName serverName, final ServerLoad sl)464   void recordNewServerWithLock(final ServerName serverName, final ServerLoad sl) {
465     LOG.info("Registering server=" + serverName);
466     this.onlineServers.put(serverName, sl);
467     this.rsAdmins.remove(serverName);
468   }
469 
getLastFlushedSequenceId(byte[] encodedRegionName)470   public RegionStoreSequenceIds getLastFlushedSequenceId(byte[] encodedRegionName) {
471     RegionStoreSequenceIds.Builder builder = RegionStoreSequenceIds.newBuilder();
472     Long seqId = flushedSequenceIdByRegion.get(encodedRegionName);
473     builder.setLastFlushedSequenceId(seqId != null ? seqId.longValue() : HConstants.NO_SEQNUM);
474     Map<byte[], Long> storeFlushedSequenceId =
475         storeFlushedSequenceIdsByRegion.get(encodedRegionName);
476     if (storeFlushedSequenceId != null) {
477       for (Map.Entry<byte[], Long> entry : storeFlushedSequenceId.entrySet()) {
478         builder.addStoreSequenceId(StoreSequenceId.newBuilder()
479             .setFamilyName(ByteString.copyFrom(entry.getKey()))
480             .setSequenceId(entry.getValue().longValue()).build());
481       }
482     }
483     return builder.build();
484   }
485 
486   /**
487    * @param serverName
488    * @return ServerLoad if serverName is known else null
489    */
getLoad(final ServerName serverName)490   public ServerLoad getLoad(final ServerName serverName) {
491     return this.onlineServers.get(serverName);
492   }
493 
494   /**
495    * Compute the average load across all region servers.
496    * Currently, this uses a very naive computation - just uses the number of
497    * regions being served, ignoring stats about number of requests.
498    * @return the average load
499    */
getAverageLoad()500   public double getAverageLoad() {
501     int totalLoad = 0;
502     int numServers = 0;
503     for (ServerLoad sl: this.onlineServers.values()) {
504         numServers++;
505         totalLoad += sl.getNumberOfRegions();
506     }
507     return numServers == 0 ? 0 :
508       (double)totalLoad / (double)numServers;
509   }
510 
511   /** @return the count of active regionservers */
countOfRegionServers()512   public int countOfRegionServers() {
513     // Presumes onlineServers is a concurrent map
514     return this.onlineServers.size();
515   }
516 
517   /**
518    * @return Read-only map of servers to serverinfo
519    */
getOnlineServers()520   public Map<ServerName, ServerLoad> getOnlineServers() {
521     // Presumption is that iterating the returned Map is OK.
522     synchronized (this.onlineServers) {
523       return Collections.unmodifiableMap(this.onlineServers);
524     }
525   }
526 
527 
getDeadServers()528   public DeadServer getDeadServers() {
529     return this.deadservers;
530   }
531 
532   /**
533    * Checks if any dead servers are currently in progress.
534    * @return true if any RS are being processed as dead, false if not
535    */
areDeadServersInProgress()536   public boolean areDeadServersInProgress() {
537     return this.deadservers.areDeadServersInProgress();
538   }
539 
letRegionServersShutdown()540   void letRegionServersShutdown() {
541     long previousLogTime = 0;
542     ServerName sn = master.getServerName();
543     ZooKeeperWatcher zkw = master.getZooKeeper();
544     int onlineServersCt;
545     while ((onlineServersCt = onlineServers.size()) > 0){
546 
547       if (System.currentTimeMillis() > (previousLogTime + 1000)) {
548         Set<ServerName> remainingServers = onlineServers.keySet();
549         synchronized (onlineServers) {
550           if (remainingServers.size() == 1 && remainingServers.contains(sn)) {
551             // Master will delete itself later.
552             return;
553           }
554         }
555         StringBuilder sb = new StringBuilder();
556         // It's ok here to not sync on onlineServers - merely logging
557         for (ServerName key : remainingServers) {
558           if (sb.length() > 0) {
559             sb.append(", ");
560           }
561           sb.append(key);
562         }
563         LOG.info("Waiting on regionserver(s) to go down " + sb.toString());
564         previousLogTime = System.currentTimeMillis();
565       }
566 
567       try {
568         List<String> servers = ZKUtil.listChildrenNoWatch(zkw, zkw.rsZNode);
569         if (servers == null || servers.size() == 0 || (servers.size() == 1
570             && servers.contains(sn.toString()))) {
571           LOG.info("ZK shows there is only the master self online, exiting now");
572           // Master could have lost some ZK events, no need to wait more.
573           break;
574         }
575       } catch (KeeperException ke) {
576         LOG.warn("Failed to list regionservers", ke);
577         // ZK is malfunctioning, don't hang here
578         break;
579       }
580       synchronized (onlineServers) {
581         try {
582           if (onlineServersCt == onlineServers.size()) onlineServers.wait(100);
583         } catch (InterruptedException ignored) {
584           // continue
585         }
586       }
587     }
588   }
589 
590   /*
591    * Expire the passed server.  Add it to list of dead servers and queue a
592    * shutdown processing.
593    */
expireServer(final ServerName serverName)594   public synchronized void expireServer(final ServerName serverName) {
595     if (serverName.equals(master.getServerName())) {
596       if (!(master.isAborted() || master.isStopped())) {
597         master.stop("We lost our znode?");
598       }
599       return;
600     }
601     if (!services.isServerCrashProcessingEnabled()) {
602       LOG.info("Master doesn't enable ServerShutdownHandler during initialization, "
603           + "delay expiring server " + serverName);
604       this.queuedDeadServers.add(serverName);
605       return;
606     }
607     if (this.deadservers.isDeadServer(serverName)) {
608       // TODO: Can this happen?  It shouldn't be online in this case?
609       LOG.warn("Expiration of " + serverName +
610           " but server shutdown already in progress");
611       return;
612     }
613     moveFromOnelineToDeadServers(serverName);
614 
615     // If cluster is going down, yes, servers are going to be expiring; don't
616     // process as a dead server
617     if (this.clusterShutdown) {
618       LOG.info("Cluster shutdown set; " + serverName +
619         " expired; onlineServers=" + this.onlineServers.size());
620       if (this.onlineServers.isEmpty()) {
621         master.stop("Cluster shutdown set; onlineServer=0");
622       }
623       return;
624     }
625 
626     boolean carryingMeta = services.getAssignmentManager().isCarryingMeta(serverName) ==
627         AssignmentManager.ServerHostRegion.HOSTING_REGION;
628     this.services.getMasterProcedureExecutor().
629       submitProcedure(new ServerCrashProcedure(serverName, true, carryingMeta));
630     LOG.debug("Added=" + serverName +
631       " to dead servers, submitted shutdown handler to be executed meta=" + carryingMeta);
632 
633     // Tell our listeners that a server was removed
634     if (!this.listeners.isEmpty()) {
635       for (ServerListener listener : this.listeners) {
636         listener.serverRemoved(serverName);
637       }
638     }
639   }
640 
641   @VisibleForTesting
moveFromOnelineToDeadServers(final ServerName sn)642   public void moveFromOnelineToDeadServers(final ServerName sn) {
643     synchronized (onlineServers) {
644       if (!this.onlineServers.containsKey(sn)) {
645         LOG.warn("Expiration of " + sn + " but server not online");
646       }
647       // Remove the server from the known servers lists and update load info BUT
648       // add to deadservers first; do this so it'll show in dead servers list if
649       // not in online servers list.
650       this.deadservers.add(sn);
651       this.onlineServers.remove(sn);
652       onlineServers.notifyAll();
653     }
654     this.rsAdmins.remove(sn);
655   }
656 
processDeadServer(final ServerName serverName, boolean shouldSplitWal)657   public synchronized void processDeadServer(final ServerName serverName, boolean shouldSplitWal) {
658     // When assignment manager is cleaning up the zookeeper nodes and rebuilding the
659     // in-memory region states, region servers could be down. Meta table can and
660     // should be re-assigned, log splitting can be done too. However, it is better to
661     // wait till the cleanup is done before re-assigning user regions.
662     //
663     // We should not wait in the server shutdown handler thread since it can clog
664     // the handler threads and meta table could not be re-assigned in case
665     // the corresponding server is down. So we queue them up here instead.
666     if (!services.getAssignmentManager().isFailoverCleanupDone()) {
667       requeuedDeadServers.put(serverName, shouldSplitWal);
668       return;
669     }
670 
671     this.deadservers.add(serverName);
672     this.services.getMasterProcedureExecutor().
673     submitProcedure(new ServerCrashProcedure(serverName, shouldSplitWal, false));
674   }
675 
676   /**
677    * Process the servers which died during master's initialization. It will be
678    * called after HMaster#assignMeta and AssignmentManager#joinCluster.
679    * */
processQueuedDeadServers()680   synchronized void processQueuedDeadServers() {
681     if (!services.isServerCrashProcessingEnabled()) {
682       LOG.info("Master hasn't enabled ServerShutdownHandler");
683     }
684     Iterator<ServerName> serverIterator = queuedDeadServers.iterator();
685     while (serverIterator.hasNext()) {
686       ServerName tmpServerName = serverIterator.next();
687       expireServer(tmpServerName);
688       serverIterator.remove();
689       requeuedDeadServers.remove(tmpServerName);
690     }
691 
692     if (!services.getAssignmentManager().isFailoverCleanupDone()) {
693       LOG.info("AssignmentManager hasn't finished failover cleanup; waiting");
694     }
695 
696     for(ServerName tmpServerName : requeuedDeadServers.keySet()){
697       processDeadServer(tmpServerName, requeuedDeadServers.get(tmpServerName));
698     }
699     requeuedDeadServers.clear();
700   }
701 
702   /*
703    * Remove the server from the drain list.
704    */
removeServerFromDrainList(final ServerName sn)705   public boolean removeServerFromDrainList(final ServerName sn) {
706     // Warn if the server (sn) is not online.  ServerName is of the form:
707     // <hostname> , <port> , <startcode>
708 
709     if (!this.isServerOnline(sn)) {
710       LOG.warn("Server " + sn + " is not currently online. " +
711                "Removing from draining list anyway, as requested.");
712     }
713     // Remove the server from the draining servers lists.
714     return this.drainingServers.remove(sn);
715   }
716 
717   /*
718    * Add the server to the drain list.
719    */
addServerToDrainList(final ServerName sn)720   public boolean addServerToDrainList(final ServerName sn) {
721     // Warn if the server (sn) is not online.  ServerName is of the form:
722     // <hostname> , <port> , <startcode>
723 
724     if (!this.isServerOnline(sn)) {
725       LOG.warn("Server " + sn + " is not currently online. " +
726                "Ignoring request to add it to draining list.");
727       return false;
728     }
729     // Add the server to the draining servers lists, if it's not already in
730     // it.
731     if (this.drainingServers.contains(sn)) {
732       LOG.warn("Server " + sn + " is already in the draining server list." +
733                "Ignoring request to add it again.");
734       return false;
735     }
736     return this.drainingServers.add(sn);
737   }
738 
739   // RPC methods to region servers
740 
741   /**
742    * Sends an OPEN RPC to the specified server to open the specified region.
743    * <p>
744    * Open should not fail but can if server just crashed.
745    * <p>
746    * @param server server to open a region
747    * @param region region to open
748    * @param versionOfOfflineNode that needs to be present in the offline node
749    * when RS tries to change the state from OFFLINE to other states.
750    * @param favoredNodes
751    */
sendRegionOpen(final ServerName server, HRegionInfo region, int versionOfOfflineNode, List<ServerName> favoredNodes)752   public RegionOpeningState sendRegionOpen(final ServerName server,
753       HRegionInfo region, int versionOfOfflineNode, List<ServerName> favoredNodes)
754   throws IOException {
755     AdminService.BlockingInterface admin = getRsAdmin(server);
756     if (admin == null) {
757       LOG.warn("Attempting to send OPEN RPC to server " + server.toString() +
758         " failed because no RPC connection found to this server");
759       return RegionOpeningState.FAILED_OPENING;
760     }
761     OpenRegionRequest request = RequestConverter.buildOpenRegionRequest(server,
762       region, versionOfOfflineNode, favoredNodes,
763       (RecoveryMode.LOG_REPLAY == this.services.getMasterFileSystem().getLogRecoveryMode()));
764     try {
765       OpenRegionResponse response = admin.openRegion(null, request);
766       return ResponseConverter.getRegionOpeningState(response);
767     } catch (ServiceException se) {
768       throw ProtobufUtil.getRemoteException(se);
769     }
770   }
771 
772   /**
773    * Sends an OPEN RPC to the specified server to open the specified region.
774    * <p>
775    * Open should not fail but can if server just crashed.
776    * <p>
777    * @param server server to open a region
778    * @param regionOpenInfos info of a list of regions to open
779    * @return a list of region opening states
780    */
sendRegionOpen(ServerName server, List<Triple<HRegionInfo, Integer, List<ServerName>>> regionOpenInfos)781   public List<RegionOpeningState> sendRegionOpen(ServerName server,
782       List<Triple<HRegionInfo, Integer, List<ServerName>>> regionOpenInfos)
783   throws IOException {
784     AdminService.BlockingInterface admin = getRsAdmin(server);
785     if (admin == null) {
786       LOG.warn("Attempting to send OPEN RPC to server " + server.toString() +
787         " failed because no RPC connection found to this server");
788       return null;
789     }
790 
791     OpenRegionRequest request = RequestConverter.buildOpenRegionRequest(server, regionOpenInfos,
792       (RecoveryMode.LOG_REPLAY == this.services.getMasterFileSystem().getLogRecoveryMode()));
793     try {
794       OpenRegionResponse response = admin.openRegion(null, request);
795       return ResponseConverter.getRegionOpeningStateList(response);
796     } catch (ServiceException se) {
797       throw ProtobufUtil.getRemoteException(se);
798     }
799   }
800 
newRpcController()801   private PayloadCarryingRpcController newRpcController() {
802     return rpcControllerFactory == null ? null : rpcControllerFactory.newController();
803   }
804 
805   /**
806    * Sends an CLOSE RPC to the specified server to close the specified region.
807    * <p>
808    * A region server could reject the close request because it either does not
809    * have the specified region or the region is being split.
810    * @param server server to open a region
811    * @param region region to open
812    * @param versionOfClosingNode
813    *   the version of znode to compare when RS transitions the znode from
814    *   CLOSING state.
815    * @param dest - if the region is moved to another server, the destination server. null otherwise.
816    * @return true if server acknowledged close, false if not
817    * @throws IOException
818    */
sendRegionClose(ServerName server, HRegionInfo region, int versionOfClosingNode, ServerName dest, boolean transitionInZK)819   public boolean sendRegionClose(ServerName server, HRegionInfo region,
820     int versionOfClosingNode, ServerName dest, boolean transitionInZK) throws IOException {
821     if (server == null) throw new NullPointerException("Passed server is null");
822     AdminService.BlockingInterface admin = getRsAdmin(server);
823     if (admin == null) {
824       throw new IOException("Attempting to send CLOSE RPC to server " +
825         server.toString() + " for region " +
826         region.getRegionNameAsString() +
827         " failed because no RPC connection found to this server");
828     }
829     PayloadCarryingRpcController controller = newRpcController();
830     return ProtobufUtil.closeRegion(controller, admin, server, region.getRegionName(),
831       versionOfClosingNode, dest, transitionInZK);
832   }
833 
sendRegionClose(ServerName server, HRegionInfo region, int versionOfClosingNode)834   public boolean sendRegionClose(ServerName server,
835       HRegionInfo region, int versionOfClosingNode) throws IOException {
836     return sendRegionClose(server, region, versionOfClosingNode, null, true);
837   }
838 
839   /**
840    * Sends a WARMUP RPC to the specified server to warmup the specified region.
841    * <p>
842    * A region server could reject the close request because it either does not
843    * have the specified region or the region is being split.
844    * @param server server to warmup a region
845    * @param region region to  warmup
846    */
sendRegionWarmup(ServerName server, HRegionInfo region)847   public void sendRegionWarmup(ServerName server,
848       HRegionInfo region) {
849     if (server == null) return;
850     try {
851       AdminService.BlockingInterface admin = getRsAdmin(server);
852       PayloadCarryingRpcController controller = newRpcController();
853       ProtobufUtil.warmupRegion(controller, admin, region);
854     } catch (IOException e) {
855       LOG.error("Received exception in RPC for warmup server:" +
856         server + "region: " + region +
857         "exception: " + e);
858     }
859   }
860 
861   /**
862    * Contacts a region server and waits up to timeout ms
863    * to close the region.  This bypasses the active hmaster.
864    */
closeRegionSilentlyAndWait(ClusterConnection connection, ServerName server, HRegionInfo region, long timeout)865   public static void closeRegionSilentlyAndWait(ClusterConnection connection,
866     ServerName server, HRegionInfo region, long timeout) throws IOException, InterruptedException {
867     AdminService.BlockingInterface rs = connection.getAdmin(server);
868     PayloadCarryingRpcController controller = connection.getRpcControllerFactory().newController();
869     try {
870       ProtobufUtil.closeRegion(controller, rs, server, region.getRegionName(), false);
871     } catch (IOException e) {
872       LOG.warn("Exception when closing region: " + region.getRegionNameAsString(), e);
873     }
874     long expiration = timeout + System.currentTimeMillis();
875     while (System.currentTimeMillis() < expiration) {
876       try {
877         HRegionInfo rsRegion =
878           ProtobufUtil.getRegionInfo(controller, rs, region.getRegionName());
879         if (rsRegion == null) return;
880       } catch (IOException ioe) {
881         if (ioe instanceof NotServingRegionException) // no need to retry again
882           return;
883         LOG.warn("Exception when retrieving regioninfo from: "
884           + region.getRegionNameAsString(), ioe);
885       }
886       Thread.sleep(1000);
887     }
888     throw new IOException("Region " + region + " failed to close within"
889         + " timeout " + timeout);
890   }
891 
892   /**
893    * Sends an MERGE REGIONS RPC to the specified server to merge the specified
894    * regions.
895    * <p>
896    * A region server could reject the close request because it either does not
897    * have the specified region.
898    * @param server server to merge regions
899    * @param region_a region to merge
900    * @param region_b region to merge
901    * @param forcible true if do a compulsory merge, otherwise we will only merge
902    *          two adjacent regions
903    * @throws IOException
904    */
sendRegionsMerge(ServerName server, HRegionInfo region_a, HRegionInfo region_b, boolean forcible)905   public void sendRegionsMerge(ServerName server, HRegionInfo region_a,
906       HRegionInfo region_b, boolean forcible) throws IOException {
907     if (server == null)
908       throw new NullPointerException("Passed server is null");
909     if (region_a == null || region_b == null)
910       throw new NullPointerException("Passed region is null");
911     AdminService.BlockingInterface admin = getRsAdmin(server);
912     if (admin == null) {
913       throw new IOException("Attempting to send MERGE REGIONS RPC to server "
914           + server.toString() + " for region "
915           + region_a.getRegionNameAsString() + ","
916           + region_b.getRegionNameAsString()
917           + " failed because no RPC connection found to this server");
918     }
919     PayloadCarryingRpcController controller = newRpcController();
920     ProtobufUtil.mergeRegions(controller, admin, region_a, region_b, forcible);
921   }
922 
923   /**
924    * Check if a region server is reachable and has the expected start code
925    */
isServerReachable(ServerName server)926   public boolean isServerReachable(ServerName server) {
927     if (server == null) throw new NullPointerException("Passed server is null");
928 
929 
930     RetryCounter retryCounter = pingRetryCounterFactory.create();
931     while (retryCounter.shouldRetry()) {
932       synchronized (this.onlineServers) {
933         if (this.deadservers.isDeadServer(server)) {
934           return false;
935         }
936       }
937       try {
938         PayloadCarryingRpcController controller = newRpcController();
939         AdminService.BlockingInterface admin = getRsAdmin(server);
940         if (admin != null) {
941           ServerInfo info = ProtobufUtil.getServerInfo(controller, admin);
942           return info != null && info.hasServerName()
943             && server.getStartcode() == info.getServerName().getStartCode();
944         }
945       } catch (IOException ioe) {
946         if (LOG.isDebugEnabled()) {
947           LOG.debug("Couldn't reach " + server + ", try=" + retryCounter.getAttemptTimes() + " of "
948               + retryCounter.getMaxAttempts(), ioe);
949         }
950         try {
951           retryCounter.sleepUntilNextRetry();
952         } catch(InterruptedException ie) {
953           Thread.currentThread().interrupt();
954           break;
955         }
956       }
957     }
958     return false;
959   }
960 
961     /**
962     * @param sn
963     * @return Admin interface for the remote regionserver named <code>sn</code>
964     * @throws IOException
965     * @throws RetriesExhaustedException wrapping a ConnectException if failed
966     */
getRsAdmin(final ServerName sn)967   private AdminService.BlockingInterface getRsAdmin(final ServerName sn)
968   throws IOException {
969     AdminService.BlockingInterface admin = this.rsAdmins.get(sn);
970     if (admin == null) {
971       LOG.debug("New admin connection to " + sn.toString());
972       if (sn.equals(master.getServerName()) && master instanceof HRegionServer) {
973         // A master is also a region server now, see HBASE-10569 for details
974         admin = ((HRegionServer)master).getRSRpcServices();
975       } else {
976         admin = this.connection.getAdmin(sn);
977       }
978       this.rsAdmins.put(sn, admin);
979     }
980     return admin;
981   }
982 
983   /**
984    * Wait for the region servers to report in.
985    * We will wait until one of this condition is met:
986    *  - the master is stopped
987    *  - the 'hbase.master.wait.on.regionservers.maxtostart' number of
988    *    region servers is reached
989    *  - the 'hbase.master.wait.on.regionservers.mintostart' is reached AND
990    *   there have been no new region server in for
991    *      'hbase.master.wait.on.regionservers.interval' time AND
992    *   the 'hbase.master.wait.on.regionservers.timeout' is reached
993    *
994    * @throws InterruptedException
995    */
waitForRegionServers(MonitoredTask status)996   public void waitForRegionServers(MonitoredTask status)
997   throws InterruptedException {
998     final long interval = this.master.getConfiguration().
999       getLong(WAIT_ON_REGIONSERVERS_INTERVAL, 1500);
1000     final long timeout = this.master.getConfiguration().
1001       getLong(WAIT_ON_REGIONSERVERS_TIMEOUT, 4500);
1002     int defaultMinToStart = 1;
1003     if (BaseLoadBalancer.tablesOnMaster(master.getConfiguration())) {
1004       // If we assign regions to master, we'd like to start
1005       // at least another region server so that we don't
1006       // assign all regions to master if other region servers
1007       // don't come up in time.
1008       defaultMinToStart = 2;
1009     }
1010     int minToStart = this.master.getConfiguration().
1011       getInt(WAIT_ON_REGIONSERVERS_MINTOSTART, defaultMinToStart);
1012     if (minToStart < 1) {
1013       LOG.warn(String.format(
1014         "The value of '%s' (%d) can not be less than 1, ignoring.",
1015         WAIT_ON_REGIONSERVERS_MINTOSTART, minToStart));
1016       minToStart = 1;
1017     }
1018     int maxToStart = this.master.getConfiguration().
1019       getInt(WAIT_ON_REGIONSERVERS_MAXTOSTART, Integer.MAX_VALUE);
1020     if (maxToStart < minToStart) {
1021         LOG.warn(String.format(
1022             "The value of '%s' (%d) is set less than '%s' (%d), ignoring.",
1023             WAIT_ON_REGIONSERVERS_MAXTOSTART, maxToStart,
1024             WAIT_ON_REGIONSERVERS_MINTOSTART, minToStart));
1025         maxToStart = Integer.MAX_VALUE;
1026     }
1027 
1028     long now =  System.currentTimeMillis();
1029     final long startTime = now;
1030     long slept = 0;
1031     long lastLogTime = 0;
1032     long lastCountChange = startTime;
1033     int count = countOfRegionServers();
1034     int oldCount = 0;
1035     while (!this.master.isStopped() && count < maxToStart
1036         && (lastCountChange+interval > now || timeout > slept || count < minToStart)) {
1037       // Log some info at every interval time or if there is a change
1038       if (oldCount != count || lastLogTime+interval < now){
1039         lastLogTime = now;
1040         String msg =
1041           "Waiting for region servers count to settle; currently"+
1042             " checked in " + count + ", slept for " + slept + " ms," +
1043             " expecting minimum of " + minToStart + ", maximum of "+ maxToStart+
1044             ", timeout of "+timeout+" ms, interval of "+interval+" ms.";
1045         LOG.info(msg);
1046         status.setStatus(msg);
1047       }
1048 
1049       // We sleep for some time
1050       final long sleepTime = 50;
1051       Thread.sleep(sleepTime);
1052       now =  System.currentTimeMillis();
1053       slept = now - startTime;
1054 
1055       oldCount = count;
1056       count = countOfRegionServers();
1057       if (count != oldCount) {
1058         lastCountChange = now;
1059       }
1060     }
1061 
1062     LOG.info("Finished waiting for region servers count to settle;" +
1063       " checked in " + count + ", slept for " + slept + " ms," +
1064       " expecting minimum of " + minToStart + ", maximum of "+ maxToStart+","+
1065       " master is "+ (this.master.isStopped() ? "stopped.": "running")
1066     );
1067   }
1068 
1069   /**
1070    * @return A copy of the internal list of online servers.
1071    */
getOnlineServersList()1072   public List<ServerName> getOnlineServersList() {
1073     // TODO: optimize the load balancer call so we don't need to make a new list
1074     // TODO: FIX. THIS IS POPULAR CALL.
1075     return new ArrayList<ServerName>(this.onlineServers.keySet());
1076   }
1077 
1078   /**
1079    * @return A copy of the internal list of draining servers.
1080    */
getDrainingServersList()1081   public List<ServerName> getDrainingServersList() {
1082     return new ArrayList<ServerName>(this.drainingServers);
1083   }
1084 
1085   /**
1086    * @return A copy of the internal set of deadNotExpired servers.
1087    */
getDeadNotExpiredServers()1088   Set<ServerName> getDeadNotExpiredServers() {
1089     return new HashSet<ServerName>(this.queuedDeadServers);
1090   }
1091 
1092   /**
1093    * During startup, if we figure it is not a failover, i.e. there is
1094    * no more WAL files to split, we won't try to recover these dead servers.
1095    * So we just remove them from the queue. Use caution in calling this.
1096    */
removeRequeuedDeadServers()1097   void removeRequeuedDeadServers() {
1098     requeuedDeadServers.clear();
1099   }
1100 
1101   /**
1102    * @return A copy of the internal map of requeuedDeadServers servers and their corresponding
1103    *         splitlog need flag.
1104    */
getRequeuedDeadServers()1105   Map<ServerName, Boolean> getRequeuedDeadServers() {
1106     return Collections.unmodifiableMap(this.requeuedDeadServers);
1107   }
1108 
isServerOnline(ServerName serverName)1109   public boolean isServerOnline(ServerName serverName) {
1110     return serverName != null && onlineServers.containsKey(serverName);
1111   }
1112 
1113   /**
1114    * Check if a server is known to be dead.  A server can be online,
1115    * or known to be dead, or unknown to this manager (i.e, not online,
1116    * not known to be dead either. it is simply not tracked by the
1117    * master any more, for example, a very old previous instance).
1118    */
isServerDead(ServerName serverName)1119   public synchronized boolean isServerDead(ServerName serverName) {
1120     return serverName == null || deadservers.isDeadServer(serverName)
1121       || queuedDeadServers.contains(serverName)
1122       || requeuedDeadServers.containsKey(serverName);
1123   }
1124 
shutdownCluster()1125   public void shutdownCluster() {
1126     this.clusterShutdown = true;
1127     this.master.stop("Cluster shutdown requested");
1128   }
1129 
isClusterShutdown()1130   public boolean isClusterShutdown() {
1131     return this.clusterShutdown;
1132   }
1133 
1134   /**
1135    * Stop the ServerManager.  Currently closes the connection to the master.
1136    */
stop()1137   public void stop() {
1138     if (connection != null) {
1139       try {
1140         connection.close();
1141       } catch (IOException e) {
1142         LOG.error("Attempt to close connection to master failed", e);
1143       }
1144     }
1145   }
1146 
1147   /**
1148    * Creates a list of possible destinations for a region. It contains the online servers, but not
1149    *  the draining or dying servers.
1150    *  @param serverToExclude can be null if there is no server to exclude
1151    */
createDestinationServersList(final ServerName serverToExclude)1152   public List<ServerName> createDestinationServersList(final ServerName serverToExclude){
1153     final List<ServerName> destServers = getOnlineServersList();
1154 
1155     if (serverToExclude != null){
1156       destServers.remove(serverToExclude);
1157     }
1158 
1159     // Loop through the draining server list and remove them from the server list
1160     final List<ServerName> drainingServersCopy = getDrainingServersList();
1161     if (!drainingServersCopy.isEmpty()) {
1162       for (final ServerName server: drainingServersCopy) {
1163         destServers.remove(server);
1164       }
1165     }
1166 
1167     // Remove the deadNotExpired servers from the server list.
1168     removeDeadNotExpiredServers(destServers);
1169     return destServers;
1170   }
1171 
1172   /**
1173    * Calls {@link #createDestinationServersList} without server to exclude.
1174    */
createDestinationServersList()1175   public List<ServerName> createDestinationServersList(){
1176     return createDestinationServersList(null);
1177   }
1178 
1179     /**
1180     * Loop through the deadNotExpired server list and remove them from the
1181     * servers.
1182     * This function should be used carefully outside of this class. You should use a high level
1183     *  method such as {@link #createDestinationServersList()} instead of managing you own list.
1184     */
removeDeadNotExpiredServers(List<ServerName> servers)1185   void removeDeadNotExpiredServers(List<ServerName> servers) {
1186     Set<ServerName> deadNotExpiredServersCopy = this.getDeadNotExpiredServers();
1187     if (!deadNotExpiredServersCopy.isEmpty()) {
1188       for (ServerName server : deadNotExpiredServersCopy) {
1189         LOG.debug("Removing dead but not expired server: " + server
1190           + " from eligible server pool.");
1191         servers.remove(server);
1192       }
1193     }
1194   }
1195 
1196   /**
1197    * To clear any dead server with same host name and port of any online server
1198    */
clearDeadServersWithSameHostNameAndPortOfOnlineServer()1199   void clearDeadServersWithSameHostNameAndPortOfOnlineServer() {
1200     for (ServerName serverName : getOnlineServersList()) {
1201       deadservers.cleanAllPreviousInstances(serverName);
1202     }
1203   }
1204 }
1205