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.util.Bytes;
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 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 startupStatus.setStatus("Trying to register in ZK as active master");
130 boolean cleanSetOfActiveMaster = true;
131
132
133 try {
134 if (ZKUtil.createEphemeralNodeAndWatch(this.watcher,
135 this.watcher.masterAddressZNode, sn.getVersionedBytes())) {
136
137 startupStatus.setStatus("Successfully registered as active master.");
138 this.clusterHasActiveMaster.set(true);
139 LOG.info("Master=" + this.sn);
140 return cleanSetOfActiveMaster;
141 }
142 cleanSetOfActiveMaster = false;
143
144
145
146 this.clusterHasActiveMaster.set(true);
147 byte [] bytes =
148 ZKUtil.getDataAndWatch(this.watcher, this.watcher.masterAddressZNode);
149 ServerName currentMaster = ServerName.parseVersionedServerName(bytes);
150 if (ServerName.isSameHostnameAndPort(currentMaster, this.sn)) {
151 String msg = ("Current master has this master's address, " + currentMaster +
152 "; master was restarted? Waiting on znode to expire...");
153 LOG.info(msg);
154 startupStatus.setStatus(msg);
155
156 ZKUtil.deleteNode(this.watcher, this.watcher.masterAddressZNode);
157 } else {
158 String msg = "Another master is the active master, " + currentMaster +
159 "; waiting to become the next active master";
160 LOG.info(msg);
161 startupStatus.setStatus(msg);
162 }
163 } catch (KeeperException ke) {
164 master.abort("Received an unexpected KeeperException, aborting", ke);
165 return false;
166 }
167 synchronized (this.clusterHasActiveMaster) {
168 while (this.clusterHasActiveMaster.get() && !this.master.isStopped()) {
169 try {
170 this.clusterHasActiveMaster.wait();
171 } catch (InterruptedException e) {
172
173 LOG.debug("Interrupted waiting for master to die", e);
174 }
175 }
176 if (this.master.isStopped()) {
177 return cleanSetOfActiveMaster;
178 }
179
180 blockUntilBecomingActiveMaster(startupStatus);
181 }
182 return cleanSetOfActiveMaster;
183 }
184
185
186
187
188 public boolean isActiveMaster() {
189 try {
190 if (ZKUtil.checkExists(watcher, watcher.masterAddressZNode) >= 0) {
191 return true;
192 }
193 }
194 catch (KeeperException ke) {
195 LOG.info("Received an unexpected KeeperException when checking " +
196 "isActiveMaster : "+ ke);
197 }
198 return false;
199 }
200
201 public void stop() {
202 try {
203
204 byte [] bytes =
205 ZKUtil.getDataAndWatch(watcher, watcher.masterAddressZNode);
206
207 ServerName master = new ServerName(Bytes.toString(bytes));
208 if (master != null && master.equals(this.sn)) {
209 ZKUtil.deleteNode(watcher, watcher.masterAddressZNode);
210 }
211 } catch (KeeperException e) {
212 LOG.error(this.watcher.prefix("Error deleting our own master address node"), e);
213 }
214 }
215 }