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
53 private final ServerName sn;
54 private final Server master;
55
56
57
58
59
60
61 ActiveMasterManager(ZooKeeperWatcher watcher, ServerName sn, Server master) {
62 super(watcher);
63 this.sn = sn;
64 this.master = master;
65 }
66
67 @Override
68 public void nodeCreated(String path) {
69 if(path.equals(watcher.masterAddressZNode) && !master.isStopped()) {
70 handleMasterNodeChange();
71 }
72 }
73
74 @Override
75 public void nodeDeleted(String path) {
76 if(path.equals(watcher.masterAddressZNode) && !master.isStopped()) {
77 handleMasterNodeChange();
78 }
79 }
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94 private void handleMasterNodeChange() {
95
96 try {
97 synchronized(clusterHasActiveMaster) {
98 if(ZKUtil.watchAndCheckExists(watcher, watcher.masterAddressZNode)) {
99
100 LOG.debug("A master is now available");
101 clusterHasActiveMaster.set(true);
102 } else {
103
104 LOG.debug("No master available. Notifying waiting threads");
105 clusterHasActiveMaster.set(false);
106
107 clusterHasActiveMaster.notifyAll();
108 }
109 }
110 } catch (KeeperException ke) {
111 master.abort("Received an unexpected KeeperException, aborting", ke);
112 }
113 }
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128 boolean blockUntilBecomingActiveMaster(MonitoredTask startupStatus,
129 ClusterStatusTracker clusterStatusTracker) {
130 while (true) {
131 startupStatus.setStatus("Trying to register in ZK as active master");
132
133
134 try {
135 String backupZNode = ZKUtil.joinZNode(
136 this.watcher.backupMasterAddressesZNode, this.sn.toString());
137 if (ZKUtil.createEphemeralNodeAndWatch(this.watcher,
138 this.watcher.masterAddressZNode, this.sn.getVersionedBytes())) {
139
140
141 LOG.info("Deleting ZNode for " + backupZNode +
142 " from backup master directory");
143 ZKUtil.deleteNodeFailSilent(this.watcher, backupZNode);
144
145
146 startupStatus.setStatus("Successfully registered as active master.");
147 this.clusterHasActiveMaster.set(true);
148 LOG.info("Master=" + this.sn);
149 return true;
150 }
151
152
153
154 this.clusterHasActiveMaster.set(true);
155
156
157
158
159
160
161
162
163
164 LOG.info("Adding ZNode for " + backupZNode +
165 " in backup master directory");
166 ZKUtil.createEphemeralNodeAndWatch(this.watcher, backupZNode,
167 this.sn.getVersionedBytes());
168
169 String msg;
170 byte [] bytes =
171 ZKUtil.getDataAndWatch(this.watcher, this.watcher.masterAddressZNode);
172 if (bytes == null) {
173 msg = ("A master was detected, but went down before its address " +
174 "could be read. Attempting to become the next active master");
175 } else {
176 ServerName currentMaster = ServerName.parseVersionedServerName(bytes);
177 if (ServerName.isSameHostnameAndPort(currentMaster, this.sn)) {
178 msg = ("Current master has this master's address, " +
179 currentMaster + "; master was restarted? Deleting node.");
180
181 ZKUtil.deleteNode(this.watcher, this.watcher.masterAddressZNode);
182 } else {
183 msg = "Another master is the active master, " + currentMaster +
184 "; waiting to become the next active master";
185 }
186 }
187 LOG.info(msg);
188 startupStatus.setStatus(msg);
189 } catch (KeeperException ke) {
190 master.abort("Received an unexpected KeeperException, aborting", ke);
191 return false;
192 }
193 synchronized (this.clusterHasActiveMaster) {
194 while (this.clusterHasActiveMaster.get() && !this.master.isStopped()) {
195 try {
196 this.clusterHasActiveMaster.wait();
197 } catch (InterruptedException e) {
198
199 LOG.debug("Interrupted waiting for master to die", e);
200 }
201 }
202 if (!clusterStatusTracker.isClusterUp()) {
203 this.master.stop("Cluster went down before this master became active");
204 }
205 if (this.master.isStopped()) {
206 return false;
207 }
208
209 }
210 }
211 }
212
213
214
215
216 public boolean isActiveMaster() {
217 try {
218 if (ZKUtil.checkExists(watcher, watcher.masterAddressZNode) >= 0) {
219 return true;
220 }
221 }
222 catch (KeeperException ke) {
223 LOG.info("Received an unexpected KeeperException when checking " +
224 "isActiveMaster : "+ ke);
225 }
226 return false;
227 }
228
229 public void stop() {
230 try {
231
232 byte [] bytes =
233 ZKUtil.getDataAndWatch(watcher, watcher.masterAddressZNode);
234
235 ServerName master = bytes == null ? null : ServerName.parseVersionedServerName(bytes);
236 if (master != null && master.equals(this.sn)) {
237 ZKUtil.deleteNode(watcher, watcher.masterAddressZNode);
238 }
239 } catch (KeeperException e) {
240 LOG.error(this.watcher.prefix("Error deleting our own master address node"), e);
241 }
242 }
243
244
245
246
247 public ServerName getActiveMaster() {
248 ServerName sn = null;
249 String msg;
250 try {
251 byte[] bytes = ZKUtil.getDataAndWatch(this.watcher, this.watcher.masterAddressZNode);
252 if (bytes == null) {
253 msg = "A master was detected, but went down before its address.";
254 LOG.info(msg);
255 } else {
256 sn = ServerName.parseVersionedServerName(bytes);
257 }
258 } catch (KeeperException e) {
259 msg = "Could not find active master";
260 LOG.info(msg);
261 }
262 return sn;
263 }
264 }