1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
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.SortedMap;
33 import java.util.concurrent.ConcurrentHashMap;
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.hbase.classification.InterfaceAudience;
40 import org.apache.hadoop.conf.Configuration;
41 import org.apache.hadoop.hbase.ClockOutOfSyncException;
42 import org.apache.hadoop.hbase.HRegionInfo;
43 import org.apache.hadoop.hbase.RegionLoad;
44 import org.apache.hadoop.hbase.Server;
45 import org.apache.hadoop.hbase.ServerLoad;
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.master.handler.MetaServerShutdownHandler;
53 import org.apache.hadoop.hbase.master.handler.ServerShutdownHandler;
54 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
55 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
56 import org.apache.hadoop.hbase.protobuf.RequestConverter;
57 import org.apache.hadoop.hbase.protobuf.ResponseConverter;
58 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService;
59 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionRequest;
60 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionResponse;
61 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ServerInfo;
62 import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode;
63 import org.apache.hadoop.hbase.regionserver.RegionOpeningState;
64 import org.apache.hadoop.hbase.util.Bytes;
65 import org.apache.hadoop.hbase.util.Triple;
66
67 import com.google.common.annotations.VisibleForTesting;
68 import com.google.protobuf.ServiceException;
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92 @InterfaceAudience.Private
93 public class ServerManager {
94 public static final String WAIT_ON_REGIONSERVERS_MAXTOSTART =
95 "hbase.master.wait.on.regionservers.maxtostart";
96
97 public static final String WAIT_ON_REGIONSERVERS_MINTOSTART =
98 "hbase.master.wait.on.regionservers.mintostart";
99
100 public static final String WAIT_ON_REGIONSERVERS_TIMEOUT =
101 "hbase.master.wait.on.regionservers.timeout";
102
103 public static final String WAIT_ON_REGIONSERVERS_INTERVAL =
104 "hbase.master.wait.on.regionservers.interval";
105
106 private static final Log LOG = LogFactory.getLog(ServerManager.class);
107
108
109 private volatile boolean clusterShutdown = false;
110
111 private final SortedMap<byte[], Long> flushedSequenceIdByRegion =
112 new ConcurrentSkipListMap<byte[], Long>(Bytes.BYTES_COMPARATOR);
113
114
115 private final ConcurrentHashMap<ServerName, ServerLoad> onlineServers =
116 new ConcurrentHashMap<ServerName, ServerLoad>();
117
118
119
120
121
122 private final Map<ServerName, AdminService.BlockingInterface> rsAdmins =
123 new HashMap<ServerName, AdminService.BlockingInterface>();
124
125
126
127
128
129 private final ArrayList<ServerName> drainingServers =
130 new ArrayList<ServerName>();
131
132 private final Server master;
133 private final MasterServices services;
134 private final HConnection connection;
135
136 private final DeadServer deadservers = new DeadServer();
137
138 private final long maxSkew;
139 private final long warningSkew;
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157 private Set<ServerName> queuedDeadServers = new HashSet<ServerName>();
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174 private Map<ServerName, Boolean> requeuedDeadServers
175 = new ConcurrentHashMap<ServerName, Boolean>();
176
177
178 private List<ServerListener> listeners = new CopyOnWriteArrayList<ServerListener>();
179
180
181
182
183
184
185
186 public ServerManager(final Server master, final MasterServices services)
187 throws IOException {
188 this(master, services, true);
189 }
190
191 @SuppressWarnings("deprecation")
192 ServerManager(final Server master, final MasterServices services,
193 final boolean connect) throws IOException {
194 this.master = master;
195 this.services = services;
196 Configuration c = master.getConfiguration();
197 maxSkew = c.getLong("hbase.master.maxclockskew", 30000);
198 warningSkew = c.getLong("hbase.master.warningclockskew", 10000);
199 this.connection = connect ? HConnectionManager.getConnection(c) : null;
200 }
201
202
203
204
205
206 public void registerListener(final ServerListener listener) {
207 this.listeners.add(listener);
208 }
209
210
211
212
213
214 public boolean unregisterListener(final ServerListener listener) {
215 return this.listeners.remove(listener);
216 }
217
218
219
220
221
222
223
224
225
226
227 ServerName regionServerStartup(final InetAddress ia, final int port,
228 final long serverStartcode, long serverCurrentTime)
229 throws IOException {
230
231
232
233
234
235
236
237 ServerName sn = ServerName.valueOf(ia.getHostName(), port, serverStartcode);
238 checkClockSkew(sn, serverCurrentTime);
239 checkIsDead(sn, "STARTUP");
240 if (!checkAndRecordNewServer(sn, ServerLoad.EMPTY_SERVERLOAD)) {
241 LOG.warn("THIS SHOULD NOT HAPPEN, RegionServerStartup"
242 + " could not record the server: " + sn);
243 }
244 return sn;
245 }
246
247
248
249
250
251
252 private void updateLastFlushedSequenceIds(ServerName sn, ServerLoad hsl) {
253 Map<byte[], RegionLoad> regionsLoad = hsl.getRegionsLoad();
254 for (Entry<byte[], RegionLoad> entry : regionsLoad.entrySet()) {
255 Long existingValue = flushedSequenceIdByRegion.get(entry.getKey());
256 long l = entry.getValue().getCompleteSequenceId();
257 if (existingValue != null) {
258 if (l != -1 && l < existingValue) {
259 LOG.warn("RegionServer " + sn +
260 " indicates a last flushed sequence id (" + entry.getValue() +
261 ") that is less than the previous last flushed sequence id (" +
262 existingValue + ") for region " +
263 Bytes.toString(entry.getKey()) + " Ignoring.");
264
265 continue;
266
267 }
268 }
269 flushedSequenceIdByRegion.put(entry.getKey(), l);
270 }
271 }
272
273 void regionServerReport(ServerName sn,
274 ServerLoad sl) throws YouAreDeadException {
275 checkIsDead(sn, "REPORT");
276 if (null == this.onlineServers.replace(sn, sl)) {
277
278
279
280
281
282
283 if (!checkAndRecordNewServer(sn, sl)) {
284 LOG.info("RegionServerReport ignored, could not record the server: " + sn);
285 return;
286 }
287 }
288 updateLastFlushedSequenceIds(sn, sl);
289 }
290
291
292
293
294
295
296
297
298
299 boolean checkAndRecordNewServer(
300 final ServerName serverName, final ServerLoad sl) {
301 ServerName existingServer = null;
302 synchronized (this.onlineServers) {
303 existingServer = findServerWithSameHostnamePortWithLock(serverName);
304 if (existingServer != null && (existingServer.getStartcode() > serverName.getStartcode())) {
305 LOG.info("Server serverName=" + serverName + " rejected; we already have "
306 + existingServer.toString() + " registered with same hostname and port");
307 return false;
308 }
309 recordNewServerWithLock(serverName, sl);
310 }
311
312
313 if (!this.listeners.isEmpty()) {
314 for (ServerListener listener : this.listeners) {
315 listener.serverAdded(serverName);
316 }
317 }
318
319
320
321 if (existingServer != null && (existingServer.getStartcode() < serverName.getStartcode())) {
322 LOG.info("Triggering server recovery; existingServer " +
323 existingServer + " looks stale, new server:" + serverName);
324 expireServer(existingServer);
325 }
326 return true;
327 }
328
329
330
331
332
333
334
335
336
337 private void checkClockSkew(final ServerName serverName, final long serverCurrentTime)
338 throws ClockOutOfSyncException {
339 long skew = Math.abs(System.currentTimeMillis() - serverCurrentTime);
340 if (skew > maxSkew) {
341 String message = "Server " + serverName + " has been " +
342 "rejected; Reported time is too far out of sync with master. " +
343 "Time difference of " + skew + "ms > max allowed of " + maxSkew + "ms";
344 LOG.warn(message);
345 throw new ClockOutOfSyncException(message);
346 } else if (skew > warningSkew){
347 String message = "Reported time for server " + serverName + " is out of sync with master " +
348 "by " + skew + "ms. (Warning threshold is " + warningSkew + "ms; " +
349 "error threshold is " + maxSkew + "ms)";
350 LOG.warn(message);
351 }
352 }
353
354
355
356
357
358
359
360
361
362 private void checkIsDead(final ServerName serverName, final String what)
363 throws YouAreDeadException {
364 if (this.deadservers.isDeadServer(serverName)) {
365
366
367 String message = "Server " + what + " rejected; currently processing " +
368 serverName + " as dead server";
369 LOG.debug(message);
370 throw new YouAreDeadException(message);
371 }
372
373
374 if ((this.services == null || ((HMaster) this.services).isInitialized())
375 && this.deadservers.cleanPreviousInstance(serverName)) {
376
377
378 LOG.debug(what + ":" + " Server " + serverName + " came back up," +
379 " removed it from the dead servers list");
380 }
381 }
382
383
384
385
386
387 private ServerName findServerWithSameHostnamePortWithLock(
388 final ServerName serverName) {
389 for (ServerName sn: this.onlineServers.keySet()) {
390 if (ServerName.isSameHostnameAndPort(serverName, sn)) return sn;
391 }
392 return null;
393 }
394
395
396
397
398
399
400
401 @VisibleForTesting
402 void recordNewServerWithLock(final ServerName serverName, final ServerLoad sl) {
403 LOG.info("Registering server=" + serverName);
404 this.onlineServers.put(serverName, sl);
405 this.rsAdmins.remove(serverName);
406 }
407
408 public long getLastFlushedSequenceId(byte[] regionName) {
409 long seqId = -1;
410 if (flushedSequenceIdByRegion.containsKey(regionName)) {
411 seqId = flushedSequenceIdByRegion.get(regionName);
412 }
413 return seqId;
414 }
415
416
417
418
419
420 public ServerLoad getLoad(final ServerName serverName) {
421 return this.onlineServers.get(serverName);
422 }
423
424
425
426
427
428
429
430 public double getAverageLoad() {
431 int totalLoad = 0;
432 int numServers = 0;
433 double averageLoad;
434 for (ServerLoad sl: this.onlineServers.values()) {
435 numServers++;
436 totalLoad += sl.getNumberOfRegions();
437 }
438 averageLoad = (double)totalLoad / (double)numServers;
439 return averageLoad;
440 }
441
442
443 int countOfRegionServers() {
444
445 return this.onlineServers.size();
446 }
447
448
449
450
451 public Map<ServerName, ServerLoad> getOnlineServers() {
452
453 synchronized (this.onlineServers) {
454 return Collections.unmodifiableMap(this.onlineServers);
455 }
456 }
457
458
459 public DeadServer getDeadServers() {
460 return this.deadservers;
461 }
462
463
464
465
466
467 public boolean areDeadServersInProgress() {
468 return this.deadservers.areDeadServersInProgress();
469 }
470
471 void letRegionServersShutdown() {
472 long previousLogTime = 0;
473 int onlineServersCt;
474 while ((onlineServersCt = onlineServers.size()) > 0) {
475
476 if (System.currentTimeMillis() > (previousLogTime + 1000)) {
477 StringBuilder sb = new StringBuilder();
478
479 for (ServerName key : this.onlineServers.keySet()) {
480 if (sb.length() > 0) {
481 sb.append(", ");
482 }
483 sb.append(key);
484 }
485 LOG.info("Waiting on regionserver(s) to go down " + sb.toString());
486 previousLogTime = System.currentTimeMillis();
487 }
488
489 synchronized (onlineServers) {
490 try {
491 if (onlineServersCt == onlineServers.size()) onlineServers.wait(100);
492 } catch (InterruptedException ignored) {
493
494 }
495 }
496 }
497 }
498
499
500
501
502
503 public synchronized void expireServer(final ServerName serverName) {
504 if (!services.isServerShutdownHandlerEnabled()) {
505 LOG.info("Master doesn't enable ServerShutdownHandler during initialization, "
506 + "delay expiring server " + serverName);
507 this.queuedDeadServers.add(serverName);
508 return;
509 }
510 if (this.deadservers.isDeadServer(serverName)) {
511
512 LOG.warn("Expiration of " + serverName +
513 " but server shutdown already in progress");
514 return;
515 }
516 synchronized (onlineServers) {
517 if (!this.onlineServers.containsKey(serverName)) {
518 LOG.warn("Expiration of " + serverName + " but server not online");
519 }
520
521
522
523 this.deadservers.add(serverName);
524 this.onlineServers.remove(serverName);
525 onlineServers.notifyAll();
526 }
527 this.rsAdmins.remove(serverName);
528
529
530 if (this.clusterShutdown) {
531 LOG.info("Cluster shutdown set; " + serverName +
532 " expired; onlineServers=" + this.onlineServers.size());
533 if (this.onlineServers.isEmpty()) {
534 master.stop("Cluster shutdown set; onlineServer=0");
535 }
536 return;
537 }
538
539 boolean carryingMeta = services.getAssignmentManager().isCarryingMeta(serverName);
540 if (carryingMeta) {
541 this.services.getExecutorService().submit(new MetaServerShutdownHandler(this.master,
542 this.services, this.deadservers, serverName));
543 } else {
544 this.services.getExecutorService().submit(new ServerShutdownHandler(this.master,
545 this.services, this.deadservers, serverName, true));
546 }
547 LOG.debug("Added=" + serverName +
548 " to dead servers, submitted shutdown handler to be executed meta=" + carryingMeta);
549
550
551 if (!this.listeners.isEmpty()) {
552 for (ServerListener listener : this.listeners) {
553 listener.serverRemoved(serverName);
554 }
555 }
556 }
557
558 public synchronized void processDeadServer(final ServerName serverName) {
559 this.processDeadServer(serverName, false);
560 }
561
562 public synchronized void processDeadServer(final ServerName serverName, boolean shouldSplitHlog) {
563
564
565
566
567
568
569
570
571 if (!services.getAssignmentManager().isFailoverCleanupDone()) {
572 requeuedDeadServers.put(serverName, shouldSplitHlog);
573 return;
574 }
575
576 this.deadservers.add(serverName);
577 this.services.getExecutorService().submit(
578 new ServerShutdownHandler(this.master, this.services, this.deadservers, serverName,
579 shouldSplitHlog));
580 }
581
582
583
584
585
586 synchronized void processQueuedDeadServers() {
587 if (!services.isServerShutdownHandlerEnabled()) {
588 LOG.info("Master hasn't enabled ServerShutdownHandler");
589 }
590 Iterator<ServerName> serverIterator = queuedDeadServers.iterator();
591 while (serverIterator.hasNext()) {
592 ServerName tmpServerName = serverIterator.next();
593 expireServer(tmpServerName);
594 serverIterator.remove();
595 requeuedDeadServers.remove(tmpServerName);
596 }
597
598 if (!services.getAssignmentManager().isFailoverCleanupDone()) {
599 LOG.info("AssignmentManager hasn't finished failover cleanup; waiting");
600 }
601
602 for(ServerName tmpServerName : requeuedDeadServers.keySet()){
603 processDeadServer(tmpServerName, requeuedDeadServers.get(tmpServerName));
604 }
605 requeuedDeadServers.clear();
606 }
607
608
609
610
611 public boolean removeServerFromDrainList(final ServerName sn) {
612
613
614
615 if (!this.isServerOnline(sn)) {
616 LOG.warn("Server " + sn + " is not currently online. " +
617 "Removing from draining list anyway, as requested.");
618 }
619
620 return this.drainingServers.remove(sn);
621 }
622
623
624
625
626 public boolean addServerToDrainList(final ServerName sn) {
627
628
629
630 if (!this.isServerOnline(sn)) {
631 LOG.warn("Server " + sn + " is not currently online. " +
632 "Ignoring request to add it to draining list.");
633 return false;
634 }
635
636
637 if (this.drainingServers.contains(sn)) {
638 LOG.warn("Server " + sn + " is already in the draining server list." +
639 "Ignoring request to add it again.");
640 return false;
641 }
642 return this.drainingServers.add(sn);
643 }
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658 public RegionOpeningState sendRegionOpen(final ServerName server,
659 HRegionInfo region, int versionOfOfflineNode, List<ServerName> favoredNodes)
660 throws IOException {
661 AdminService.BlockingInterface admin = getRsAdmin(server);
662 if (admin == null) {
663 LOG.warn("Attempting to send OPEN RPC to server " + server.toString() +
664 " failed because no RPC connection found to this server");
665 return RegionOpeningState.FAILED_OPENING;
666 }
667 OpenRegionRequest request = RequestConverter.buildOpenRegionRequest(server,
668 region, versionOfOfflineNode, favoredNodes,
669 (RecoveryMode.LOG_REPLAY == this.services.getMasterFileSystem().getLogRecoveryMode()));
670 try {
671 OpenRegionResponse response = admin.openRegion(null, request);
672 return ResponseConverter.getRegionOpeningState(response);
673 } catch (ServiceException se) {
674 throw ProtobufUtil.getRemoteException(se);
675 }
676 }
677
678
679
680
681
682
683
684
685
686
687 public List<RegionOpeningState> sendRegionOpen(ServerName server,
688 List<Triple<HRegionInfo, Integer, List<ServerName>>> regionOpenInfos)
689 throws IOException {
690 AdminService.BlockingInterface admin = getRsAdmin(server);
691 if (admin == null) {
692 LOG.warn("Attempting to send OPEN RPC to server " + server.toString() +
693 " failed because no RPC connection found to this server");
694 return null;
695 }
696
697 OpenRegionRequest request = RequestConverter.buildOpenRegionRequest(server, regionOpenInfos,
698 (RecoveryMode.LOG_REPLAY == this.services.getMasterFileSystem().getLogRecoveryMode()));
699 try {
700 OpenRegionResponse response = admin.openRegion(null, request);
701 return ResponseConverter.getRegionOpeningStateList(response);
702 } catch (ServiceException se) {
703 throw ProtobufUtil.getRemoteException(se);
704 }
705 }
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721 public boolean sendRegionClose(ServerName server, HRegionInfo region,
722 int versionOfClosingNode, ServerName dest, boolean transitionInZK) throws IOException {
723 if (server == null) throw new NullPointerException("Passed server is null");
724 AdminService.BlockingInterface admin = getRsAdmin(server);
725 if (admin == null) {
726 throw new IOException("Attempting to send CLOSE RPC to server " +
727 server.toString() + " for region " +
728 region.getRegionNameAsString() +
729 " failed because no RPC connection found to this server");
730 }
731 return ProtobufUtil.closeRegion(admin, server, region.getRegionName(),
732 versionOfClosingNode, dest, transitionInZK);
733 }
734
735 public boolean sendRegionClose(ServerName server,
736 HRegionInfo region, int versionOfClosingNode) throws IOException {
737 return sendRegionClose(server, region, versionOfClosingNode, null, true);
738 }
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753 public void sendRegionsMerge(ServerName server, HRegionInfo region_a,
754 HRegionInfo region_b, boolean forcible) throws IOException {
755 if (server == null)
756 throw new NullPointerException("Passed server is null");
757 if (region_a == null || region_b == null)
758 throw new NullPointerException("Passed region is null");
759 AdminService.BlockingInterface admin = getRsAdmin(server);
760 if (admin == null) {
761 throw new IOException("Attempting to send MERGE REGIONS RPC to server "
762 + server.toString() + " for region "
763 + region_a.getRegionNameAsString() + ","
764 + region_b.getRegionNameAsString()
765 + " failed because no RPC connection found to this server");
766 }
767 ProtobufUtil.mergeRegions(admin, region_a, region_b, forcible);
768 }
769
770
771
772
773 public boolean isServerReachable(ServerName server) {
774 if (server == null) throw new NullPointerException("Passed server is null");
775 int maximumAttempts = Math.max(1, master.getConfiguration().getInt(
776 "hbase.master.maximum.ping.server.attempts", 10));
777 for (int i = 0; i < maximumAttempts; i++) {
778 try {
779 AdminService.BlockingInterface admin = getRsAdmin(server);
780 if (admin != null) {
781 ServerInfo info = ProtobufUtil.getServerInfo(admin);
782 return info != null && info.hasServerName()
783 && server.getStartcode() == info.getServerName().getStartCode();
784 }
785 } catch (IOException ioe) {
786 LOG.debug("Couldn't reach " + server + ", try=" + i
787 + " of " + maximumAttempts, ioe);
788 }
789 }
790 return false;
791 }
792
793
794
795
796
797
798
799 private AdminService.BlockingInterface getRsAdmin(final ServerName sn)
800 throws IOException {
801 AdminService.BlockingInterface admin = this.rsAdmins.get(sn);
802 if (admin == null) {
803 LOG.debug("New admin connection to " + sn.toString());
804 admin = this.connection.getAdmin(sn);
805 this.rsAdmins.put(sn, admin);
806 }
807 return admin;
808 }
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823 public void waitForRegionServers(MonitoredTask status)
824 throws InterruptedException {
825 final long interval = this.master.getConfiguration().
826 getLong(WAIT_ON_REGIONSERVERS_INTERVAL, 1500);
827 final long timeout = this.master.getConfiguration().
828 getLong(WAIT_ON_REGIONSERVERS_TIMEOUT, 4500);
829 int minToStart = this.master.getConfiguration().
830 getInt(WAIT_ON_REGIONSERVERS_MINTOSTART, 1);
831 if (minToStart < 1) {
832 LOG.warn(String.format(
833 "The value of '%s' (%d) can not be less than 1, ignoring.",
834 WAIT_ON_REGIONSERVERS_MINTOSTART, minToStart));
835 minToStart = 1;
836 }
837 int maxToStart = this.master.getConfiguration().
838 getInt(WAIT_ON_REGIONSERVERS_MAXTOSTART, Integer.MAX_VALUE);
839 if (maxToStart < minToStart) {
840 LOG.warn(String.format(
841 "The value of '%s' (%d) is set less than '%s' (%d), ignoring.",
842 WAIT_ON_REGIONSERVERS_MAXTOSTART, maxToStart,
843 WAIT_ON_REGIONSERVERS_MINTOSTART, minToStart));
844 maxToStart = Integer.MAX_VALUE;
845 }
846
847 long now = System.currentTimeMillis();
848 final long startTime = now;
849 long slept = 0;
850 long lastLogTime = 0;
851 long lastCountChange = startTime;
852 int count = countOfRegionServers();
853 int oldCount = 0;
854 while (
855 !this.master.isStopped() &&
856 count < maxToStart &&
857 (lastCountChange+interval > now || timeout > slept || count < minToStart)
858 ){
859
860
861 if (oldCount != count || lastLogTime+interval < now){
862 lastLogTime = now;
863 String msg =
864 "Waiting for region servers count to settle; currently"+
865 " checked in " + count + ", slept for " + slept + " ms," +
866 " expecting minimum of " + minToStart + ", maximum of "+ maxToStart+
867 ", timeout of "+timeout+" ms, interval of "+interval+" ms.";
868 LOG.info(msg);
869 status.setStatus(msg);
870 }
871
872
873 final long sleepTime = 50;
874 Thread.sleep(sleepTime);
875 now = System.currentTimeMillis();
876 slept = now - startTime;
877
878 oldCount = count;
879 count = countOfRegionServers();
880 if (count != oldCount) {
881 lastCountChange = now;
882 }
883 }
884
885 LOG.info("Finished waiting for region servers count to settle;" +
886 " checked in " + count + ", slept for " + slept + " ms," +
887 " expecting minimum of " + minToStart + ", maximum of "+ maxToStart+","+
888 " master is "+ (this.master.isStopped() ? "stopped.": "running.")
889 );
890 }
891
892
893
894
895 public List<ServerName> getOnlineServersList() {
896
897
898 return new ArrayList<ServerName>(this.onlineServers.keySet());
899 }
900
901
902
903
904 public List<ServerName> getDrainingServersList() {
905 return new ArrayList<ServerName>(this.drainingServers);
906 }
907
908
909
910
911 Set<ServerName> getDeadNotExpiredServers() {
912 return new HashSet<ServerName>(this.queuedDeadServers);
913 }
914
915
916
917
918
919
920 void removeRequeuedDeadServers() {
921 requeuedDeadServers.clear();
922 }
923
924
925
926
927
928 Map<ServerName, Boolean> getRequeuedDeadServers() {
929 return Collections.unmodifiableMap(this.requeuedDeadServers);
930 }
931
932 public boolean isServerOnline(ServerName serverName) {
933 return serverName != null && onlineServers.containsKey(serverName);
934 }
935
936
937
938
939
940
941
942 public synchronized boolean isServerDead(ServerName serverName) {
943 return serverName == null || deadservers.isDeadServer(serverName)
944 || queuedDeadServers.contains(serverName)
945 || requeuedDeadServers.containsKey(serverName);
946 }
947
948 public void shutdownCluster() {
949 this.clusterShutdown = true;
950 this.master.stop("Cluster shutdown requested");
951 }
952
953 public boolean isClusterShutdown() {
954 return this.clusterShutdown;
955 }
956
957
958
959
960 public void stop() {
961 if (connection != null) {
962 try {
963 connection.close();
964 } catch (IOException e) {
965 LOG.error("Attempt to close connection to master failed", e);
966 }
967 }
968 }
969
970
971
972
973
974
975 public List<ServerName> createDestinationServersList(final ServerName serverToExclude){
976 final List<ServerName> destServers = getOnlineServersList();
977
978 if (serverToExclude != null){
979 destServers.remove(serverToExclude);
980 }
981
982
983 final List<ServerName> drainingServersCopy = getDrainingServersList();
984 if (!drainingServersCopy.isEmpty()) {
985 for (final ServerName server: drainingServersCopy) {
986 destServers.remove(server);
987 }
988 }
989
990
991 removeDeadNotExpiredServers(destServers);
992
993 return destServers;
994 }
995
996
997
998
999 public List<ServerName> createDestinationServersList(){
1000 return createDestinationServersList(null);
1001 }
1002
1003
1004
1005
1006
1007
1008
1009 void removeDeadNotExpiredServers(List<ServerName> servers) {
1010 Set<ServerName> deadNotExpiredServersCopy = this.getDeadNotExpiredServers();
1011 if (!deadNotExpiredServersCopy.isEmpty()) {
1012 for (ServerName server : deadNotExpiredServersCopy) {
1013 LOG.debug("Removing dead but not expired server: " + server
1014 + " from eligible server pool.");
1015 servers.remove(server);
1016 }
1017 }
1018 }
1019
1020
1021
1022
1023 void clearDeadServersWithSameHostNameAndPortOfOnlineServer() {
1024 for (ServerName serverName : getOnlineServersList()) {
1025 deadservers.cleanAllPreviousInstances(serverName);
1026 }
1027 }
1028 }