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