1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20 package org.apache.hadoop.hbase.master;
21
22 import java.util.concurrent.atomic.AtomicBoolean;
23
24 import org.apache.commons.logging.Log;
25 import org.apache.commons.logging.LogFactory;
26 import org.apache.hadoop.hbase.Server;
27 import org.apache.hadoop.hbase.ServerName;
28 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
29 import org.apache.hadoop.hbase.zookeeper.ClusterStatusTracker;
30 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
31 import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
32 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
33 import org.apache.zookeeper.KeeperException;
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48 public class ActiveMasterManager extends ZooKeeperListener {
49 private static final Log LOG = LogFactory.getLog(ActiveMasterManager.class);
50
51 final AtomicBoolean clusterHasActiveMaster = new AtomicBoolean(false);
52 final AtomicBoolean clusterShutDown = new AtomicBoolean(false);
53
54 private final ServerName sn;
55 private final Server master;
56
57
58
59
60
61
62 ActiveMasterManager(ZooKeeperWatcher watcher, ServerName sn, Server master) {
63 super(watcher);
64 this.sn = sn;
65 this.master = master;
66 }
67
68 @Override
69 public void nodeCreated(String path) {
70 if(path.equals(watcher.masterAddressZNode) && !master.isStopped()) {
71 handleMasterNodeChange();
72 }
73 }
74
75 @Override
76 public void nodeDeleted(String path) {
77
78
79
80
81
82
83
84 if (path.equals(watcher.clusterStateZNode) && !master.isStopped()) {
85 clusterShutDown.set(true);
86 }
87 if(path.equals(watcher.masterAddressZNode) && !master.isStopped()) {
88 handleMasterNodeChange();
89 }
90 }
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105 private void handleMasterNodeChange() {
106
107 try {
108 synchronized(clusterHasActiveMaster) {
109 if(ZKUtil.watchAndCheckExists(watcher, watcher.masterAddressZNode)) {
110
111 LOG.debug("A master is now available");
112 clusterHasActiveMaster.set(true);
113 } else {
114
115 LOG.debug("No master available. Notifying waiting threads");
116 clusterHasActiveMaster.set(false);
117
118 clusterHasActiveMaster.notifyAll();
119 }
120 }
121 } catch (KeeperException ke) {
122 master.abort("Received an unexpected KeeperException, aborting", ke);
123 }
124 }
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139 boolean blockUntilBecomingActiveMaster(MonitoredTask startupStatus) {
140 while (true) {
141 startupStatus.setStatus("Trying to register in ZK as active master");
142
143
144 try {
145 String backupZNode = ZKUtil.joinZNode(
146 this.watcher.backupMasterAddressesZNode, this.sn.toString());
147 if (ZKUtil.createEphemeralNodeAndWatch(this.watcher,
148 this.watcher.masterAddressZNode, this.sn.getVersionedBytes())) {
149
150
151 LOG.info("Deleting ZNode for " + backupZNode +
152 " from backup master directory");
153 ZKUtil.deleteNodeFailSilent(this.watcher, backupZNode);
154
155
156 startupStatus.setStatus("Successfully registered as active master.");
157 this.clusterHasActiveMaster.set(true);
158 LOG.info("Master=" + this.sn);
159 return true;
160 }
161
162
163
164 this.clusterHasActiveMaster.set(true);
165
166
167
168
169
170
171
172
173
174 LOG.info("Adding ZNode for " + backupZNode +
175 " in backup master directory");
176 ZKUtil.createEphemeralNodeAndWatch(this.watcher, backupZNode,
177 this.sn.getVersionedBytes());
178
179 String msg;
180 byte [] bytes =
181 ZKUtil.getDataAndWatch(this.watcher, this.watcher.masterAddressZNode);
182 if (bytes == null) {
183 msg = ("A master was detected, but went down before its address " +
184 "could be read. Attempting to become the next active master");
185 } else {
186 ServerName currentMaster = ServerName.parseVersionedServerName(bytes);
187 if (ServerName.isSameHostnameAndPort(currentMaster, this.sn)) {
188 msg = ("Current master has this master's address, " +
189 currentMaster + "; master was restarted? Deleting node.");
190
191 ZKUtil.deleteNode(this.watcher, this.watcher.masterAddressZNode);
192 } else {
193 msg = "Another master is the active master, " + currentMaster +
194 "; waiting to become the next active master";
195 }
196 }
197 LOG.info(msg);
198 startupStatus.setStatus(msg);
199 } catch (KeeperException ke) {
200 master.abort("Received an unexpected KeeperException, aborting", ke);
201 return false;
202 }
203 synchronized (this.clusterHasActiveMaster) {
204 while (this.clusterHasActiveMaster.get() && !this.master.isStopped()) {
205 try {
206 this.clusterHasActiveMaster.wait();
207 } catch (InterruptedException e) {
208
209 LOG.debug("Interrupted waiting for master to die", e);
210 }
211 }
212 if (clusterShutDown.get()) {
213 this.master.stop("Cluster went down before this master became active");
214 }
215 if (this.master.isStopped()) {
216 return false;
217 }
218
219 }
220 }
221 }
222
223
224
225
226 public boolean isActiveMaster() {
227 try {
228 if (ZKUtil.checkExists(watcher, watcher.masterAddressZNode) >= 0) {
229 return true;
230 }
231 }
232 catch (KeeperException ke) {
233 LOG.info("Received an unexpected KeeperException when checking " +
234 "isActiveMaster : "+ ke);
235 }
236 return false;
237 }
238
239 public void stop() {
240 try {
241
242 byte [] bytes =
243 ZKUtil.getDataAndWatch(watcher, watcher.masterAddressZNode);
244
245 ServerName master = bytes == null ? null : ServerName.parseVersionedServerName(bytes);
246 if (master != null && master.equals(this.sn)) {
247 ZKUtil.deleteNode(watcher, watcher.masterAddressZNode);
248 }
249 } catch (KeeperException e) {
250 LOG.error(this.watcher.prefix("Error deleting our own master address node"), e);
251 }
252 }
253
254
255
256
257 public ServerName getActiveMaster() {
258 ServerName sn = null;
259 String msg;
260 try {
261 byte[] bytes = ZKUtil.getDataAndWatch(this.watcher, this.watcher.masterAddressZNode);
262 if (bytes == null) {
263 msg = "A master was detected, but went down before its address.";
264 LOG.info(msg);
265 } else {
266 sn = ServerName.parseVersionedServerName(bytes);
267 }
268 } catch (KeeperException e) {
269 msg = "Could not find active master";
270 LOG.info(msg);
271 }
272 return sn;
273 }
274 }