1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18 package org.apache.hadoop.hbase.zookeeper;
19
20 import java.io.IOException;
21 import java.util.List;
22 import java.util.NavigableSet;
23 import java.util.TreeSet;
24
25 import org.apache.commons.logging.Log;
26 import org.apache.commons.logging.LogFactory;
27 import org.apache.hadoop.hbase.classification.InterfaceAudience;
28 import org.apache.hadoop.hbase.Abortable;
29 import org.apache.hadoop.hbase.ServerName;
30 import org.apache.hadoop.hbase.master.ServerListener;
31 import org.apache.hadoop.hbase.master.ServerManager;
32 import org.apache.zookeeper.KeeperException;
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48 @InterfaceAudience.Private
49 public class DrainingServerTracker extends ZooKeeperListener {
50 private static final Log LOG = LogFactory.getLog(DrainingServerTracker.class);
51
52 private ServerManager serverManager;
53 private final NavigableSet<ServerName> drainingServers = new TreeSet<ServerName>();
54 private Abortable abortable;
55
56 public DrainingServerTracker(ZooKeeperWatcher watcher,
57 Abortable abortable, ServerManager serverManager) {
58 super(watcher);
59 this.abortable = abortable;
60 this.serverManager = serverManager;
61 }
62
63
64
65
66
67
68
69
70 public void start() throws KeeperException, IOException {
71 watcher.registerListener(this);
72
73 serverManager.registerListener(new ServerListener() {
74 @Override
75 public void serverAdded(ServerName sn) {
76 if (drainingServers.contains(sn)){
77 serverManager.addServerToDrainList(sn);
78 }
79 }
80
81 @Override
82 public void waiting() {
83
84 }
85
86 @Override
87 public void serverRemoved(ServerName serverName) {
88
89 }
90 });
91 List<String> servers =
92 ZKUtil.listChildrenAndWatchThem(watcher, watcher.drainingZNode);
93 add(servers);
94 }
95
96 private void add(final List<String> servers) throws IOException {
97 synchronized(this.drainingServers) {
98 this.drainingServers.clear();
99 for (String n: servers) {
100 final ServerName sn = ServerName.valueOf(ZKUtil.getNodeName(n));
101 this.drainingServers.add(sn);
102 this.serverManager.addServerToDrainList(sn);
103 LOG.info("Draining RS node created, adding to list [" +
104 sn + "]");
105
106 }
107 }
108 }
109
110 private void remove(final ServerName sn) {
111 synchronized(this.drainingServers) {
112 this.drainingServers.remove(sn);
113 this.serverManager.removeServerFromDrainList(sn);
114 }
115 }
116
117 @Override
118 public void nodeDeleted(final String path) {
119 if(path.startsWith(watcher.drainingZNode)) {
120 final ServerName sn = ServerName.valueOf(ZKUtil.getNodeName(path));
121 LOG.info("Draining RS node deleted, removing from list [" +
122 sn + "]");
123 remove(sn);
124 }
125 }
126
127 @Override
128 public void nodeChildrenChanged(final String path) {
129 if(path.equals(watcher.drainingZNode)) {
130 try {
131 final List<String> newNodes =
132 ZKUtil.listChildrenAndWatchThem(watcher, watcher.drainingZNode);
133 add(newNodes);
134 } catch (KeeperException e) {
135 abortable.abort("Unexpected zk exception getting RS nodes", e);
136 } catch (IOException e) {
137 abortable.abort("Unexpected zk exception getting RS nodes", e);
138 }
139 }
140 }
141 }