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