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 import com.google.common.collect.ImmutableList;
21 import java.io.IOException;
22 import java.io.InterruptedIOException;
23 import java.util.List;
24 import java.util.concurrent.atomic.AtomicBoolean;
25
26 import org.apache.commons.logging.Log;
27 import org.apache.commons.logging.LogFactory;
28 import org.apache.hadoop.hbase.classification.InterfaceAudience;
29 import org.apache.hadoop.hbase.Server;
30 import org.apache.hadoop.hbase.ServerName;
31 import org.apache.hadoop.hbase.ZNodeClearer;
32 import org.apache.hadoop.hbase.exceptions.DeserializationException;
33 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
34 import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
35 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
36 import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
37 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
38 import org.apache.zookeeper.KeeperException;
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54 @InterfaceAudience.Private
55 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="JLM_JSR166_UTILCONCURRENT_MONITORENTER",
56 justification="Use of an atomic type both as monitor and condition variable is intended")
57 public class ActiveMasterManager extends ZooKeeperListener {
58 private static final Log LOG = LogFactory.getLog(ActiveMasterManager.class);
59
60 final AtomicBoolean clusterHasActiveMaster = new AtomicBoolean(false);
61 final AtomicBoolean clusterShutDown = new AtomicBoolean(false);
62
63
64 private final ServerName sn;
65 private int infoPort;
66 private final Server master;
67
68
69
70
71 volatile ServerName activeMasterServerName;
72
73
74 private volatile ImmutableList<ServerName> backupMasters;
75
76
77
78
79
80
81 ActiveMasterManager(ZooKeeperWatcher watcher, ServerName sn, Server master)
82 throws InterruptedIOException {
83 super(watcher);
84 watcher.registerListener(this);
85 this.sn = sn;
86 this.master = master;
87 updateBackupMasters();
88 }
89
90
91 public void setInfoPort(int infoPort) {
92 this.infoPort = infoPort;
93 }
94
95 @Override
96 public void nodeCreated(String path) {
97 handle(path);
98 }
99
100 @Override
101 public void nodeChildrenChanged(String path) {
102 if (path.equals(watcher.backupMasterAddressesZNode)) {
103 try {
104 updateBackupMasters();
105 } catch (InterruptedIOException ioe) {
106 LOG.error("Error updating backup masters", ioe);
107 }
108 }
109 }
110
111 @Override
112 public void nodeDeleted(String path) {
113
114
115
116
117
118
119
120 if(path.equals(watcher.clusterStateZNode) && !master.isStopped()) {
121 clusterShutDown.set(true);
122 }
123 handle(path);
124 }
125
126 void handle(final String path) {
127 if (path.equals(watcher.getMasterAddressZNode()) && !master.isStopped()) {
128 handleMasterNodeChange();
129 }
130 }
131
132 private void updateBackupMasters() throws InterruptedIOException {
133 backupMasters =
134 ImmutableList.copyOf(MasterAddressTracker.getBackupMastersAndRenewWatch(watcher));
135 }
136
137
138
139
140 private void fetchAndSetActiveMasterServerName() {
141 LOG.debug("Attempting to fetch active master sn from zk");
142 try {
143 activeMasterServerName = MasterAddressTracker.getMasterAddress(watcher);
144 } catch (IOException | KeeperException e) {
145
146 LOG.error("Error fetching active master information", e);
147 }
148 }
149
150
151
152
153 public ServerName getActiveMasterServerName() {
154 if (!clusterHasActiveMaster.get()) {
155 return null;
156 }
157 if (activeMasterServerName == null) {
158 fetchAndSetActiveMasterServerName();
159 }
160
161 return activeMasterServerName;
162 }
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177 private void handleMasterNodeChange() {
178
179 try {
180 synchronized(clusterHasActiveMaster) {
181 if (ZKUtil.watchAndCheckExists(watcher, watcher.getMasterAddressZNode())) {
182
183 LOG.debug("A master is now available");
184 clusterHasActiveMaster.set(true);
185 } else {
186
187 LOG.debug("No master available. Notifying waiting threads");
188 clusterHasActiveMaster.set(false);
189
190 clusterHasActiveMaster.notifyAll();
191 }
192
193
194 activeMasterServerName = null;
195 }
196 } catch (KeeperException ke) {
197 master.abort("Received an unexpected KeeperException, aborting", ke);
198 }
199 }
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215 boolean blockUntilBecomingActiveMaster(
216 int checkInterval, MonitoredTask startupStatus) {
217 String backupZNode = ZKUtil.joinZNode(
218 this.watcher.backupMasterAddressesZNode, this.sn.toString());
219 while (!(master.isAborted() || master.isStopped())) {
220 startupStatus.setStatus("Trying to register in ZK as active master");
221
222
223 try {
224 if (MasterAddressTracker.setMasterAddress(this.watcher,
225 this.watcher.getMasterAddressZNode(), this.sn, infoPort)) {
226
227
228
229 if (ZKUtil.checkExists(this.watcher, backupZNode) != -1) {
230 LOG.info("Deleting ZNode for " + backupZNode + " from backup master directory");
231 ZKUtil.deleteNodeFailSilent(this.watcher, backupZNode);
232 }
233
234 ZNodeClearer.writeMyEphemeralNodeOnDisk(this.sn.toString());
235
236
237 startupStatus.setStatus("Successfully registered as active master.");
238 this.clusterHasActiveMaster.set(true);
239 LOG.info("Registered Active Master=" + this.sn);
240 activeMasterServerName = sn;
241 return true;
242 }
243
244
245
246 activeMasterServerName = null;
247
248
249 this.clusterHasActiveMaster.set(true);
250
251 String msg;
252 byte[] bytes =
253 ZKUtil.getDataAndWatch(this.watcher, this.watcher.getMasterAddressZNode());
254 if (bytes == null) {
255 msg = ("A master was detected, but went down before its address " +
256 "could be read. Attempting to become the next active master");
257 } else {
258 ServerName currentMaster;
259 try {
260 currentMaster = ServerName.parseFrom(bytes);
261 } catch (DeserializationException e) {
262 LOG.warn("Failed parse", e);
263
264 continue;
265 }
266 if (ServerName.isSameHostnameAndPort(currentMaster, this.sn)) {
267 msg = ("Current master has this master's address, " +
268 currentMaster + "; master was restarted? Deleting node.");
269
270 ZKUtil.deleteNode(this.watcher, this.watcher.getMasterAddressZNode());
271
272
273
274
275 ZNodeClearer.deleteMyEphemeralNodeOnDisk();
276 } else {
277 msg = "Another master is the active master, " + currentMaster +
278 "; waiting to become the next active master";
279 }
280 }
281 LOG.info(msg);
282 startupStatus.setStatus(msg);
283 } catch (KeeperException ke) {
284 master.abort("Received an unexpected KeeperException, aborting", ke);
285 return false;
286 }
287 synchronized (this.clusterHasActiveMaster) {
288 while (clusterHasActiveMaster.get() && !master.isStopped()) {
289 try {
290 clusterHasActiveMaster.wait(checkInterval);
291 } catch (InterruptedException e) {
292
293
294 LOG.debug("Interrupted waiting for master to die", e);
295 }
296 }
297 if (clusterShutDown.get()) {
298 this.master.stop(
299 "Cluster went down before this master became active");
300 }
301 }
302 }
303 return false;
304 }
305
306
307
308
309 boolean hasActiveMaster() {
310 try {
311 if (ZKUtil.checkExists(watcher, watcher.getMasterAddressZNode()) >= 0) {
312 return true;
313 }
314 }
315 catch (KeeperException ke) {
316 LOG.info("Received an unexpected KeeperException when checking " +
317 "isActiveMaster : "+ ke);
318 }
319 return false;
320 }
321
322 public void stop() {
323 try {
324 synchronized (clusterHasActiveMaster) {
325
326
327 clusterHasActiveMaster.notifyAll();
328 }
329
330 ServerName activeMaster = null;
331 try {
332 activeMaster = MasterAddressTracker.getMasterAddress(this.watcher);
333 } catch (IOException e) {
334 LOG.warn("Failed get of master address: " + e.toString());
335 }
336 if (activeMaster != null && activeMaster.equals(this.sn)) {
337 ZKUtil.deleteNode(watcher, watcher.getMasterAddressZNode());
338
339
340 ZNodeClearer.deleteMyEphemeralNodeOnDisk();
341 }
342 } catch (KeeperException e) {
343 LOG.error(this.watcher.prefix("Error deleting our own master address node"), e);
344 }
345 }
346
347
348
349
350 public List<ServerName> getBackupMasters() {
351 return backupMasters;
352 }
353 }