1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18 package org.apache.hadoop.hbase.zookeeper;
19
20 import org.apache.hadoop.hbase.classification.InterfaceAudience;
21 import org.apache.hadoop.hbase.Abortable;
22 import org.apache.hadoop.hbase.HConstants;
23 import org.apache.hadoop.hbase.ServerName;
24 import org.apache.hadoop.hbase.exceptions.DeserializationException;
25 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
26 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
27 import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
28 import org.apache.zookeeper.KeeperException;
29 import org.apache.zookeeper.data.Stat;
30
31 import java.io.IOException;
32 import java.io.InterruptedIOException;
33 import com.google.protobuf.InvalidProtocolBufferException;
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53 @InterfaceAudience.Private
54 public class MasterAddressTracker extends ZooKeeperNodeTracker {
55
56
57
58
59
60
61
62
63
64
65
66 public MasterAddressTracker(ZooKeeperWatcher watcher, Abortable abortable) {
67 super(watcher, watcher.getMasterAddressZNode(), abortable);
68 }
69
70
71
72
73
74
75 public ServerName getMasterAddress() {
76 return getMasterAddress(false);
77 }
78
79
80
81
82
83
84 public int getMasterInfoPort() {
85 try {
86 final ZooKeeperProtos.Master master = parse(this.getData(false));
87 if (master == null) {
88 return 0;
89 }
90 return master.getInfoPort();
91 } catch (DeserializationException e) {
92 LOG.warn("Failed parse master zk node data", e);
93 return 0;
94 }
95 }
96
97
98
99
100
101
102 public int getBackupMasterInfoPort(final ServerName sn) {
103 String backupZNode = ZKUtil.joinZNode(watcher.backupMasterAddressesZNode, sn.toString());
104 try {
105 byte[] data = ZKUtil.getData(watcher, backupZNode);
106 final ZooKeeperProtos.Master backup = parse(data);
107 if (backup == null) {
108 return 0;
109 }
110 return backup.getInfoPort();
111 } catch (Exception e) {
112 LOG.warn("Failed to get backup master: " + sn + "'s info port.", e);
113 return 0;
114 }
115 }
116
117
118
119
120
121
122
123
124
125 public ServerName getMasterAddress(final boolean refresh) {
126 try {
127 return ServerName.parseFrom(super.getData(refresh));
128 } catch (DeserializationException e) {
129 LOG.warn("Failed parse", e);
130 return null;
131 }
132 }
133
134
135
136
137
138
139
140
141
142
143
144 public static ServerName getMasterAddress(final ZooKeeperWatcher zkw)
145 throws KeeperException, IOException {
146 byte [] data;
147 try {
148 data = ZKUtil.getData(zkw, zkw.getMasterAddressZNode());
149 } catch (InterruptedException e) {
150 throw new InterruptedIOException();
151 }
152
153 if (data == null){
154 throw new IOException("Can't get master address from ZooKeeper; znode data == null");
155 }
156 try {
157 return ServerName.parseFrom(data);
158 } catch (DeserializationException e) {
159 KeeperException ke = new KeeperException.DataInconsistencyException();
160 ke.initCause(e);
161 throw ke;
162 }
163 }
164
165
166
167
168
169
170
171
172
173
174
175
176 public static int getMasterInfoPort(final ZooKeeperWatcher zkw) throws KeeperException,
177 IOException {
178 byte[] data;
179 try {
180 data = ZKUtil.getData(zkw, zkw.getMasterAddressZNode());
181 } catch (InterruptedException e) {
182 throw new InterruptedIOException();
183 }
184
185 if (data == null) {
186 throw new IOException("Can't get master address from ZooKeeper; znode data == null");
187 }
188 try {
189 return parse(data).getInfoPort();
190 } catch (DeserializationException e) {
191 KeeperException ke = new KeeperException.DataInconsistencyException();
192 ke.initCause(e);
193 throw ke;
194 }
195 }
196
197
198
199
200
201
202
203
204
205
206
207
208 public static boolean setMasterAddress(final ZooKeeperWatcher zkw,
209 final String znode, final ServerName master, int infoPort)
210 throws KeeperException {
211 return ZKUtil.createEphemeralNodeAndWatch(zkw, znode, toByteArray(master, infoPort));
212 }
213
214
215
216
217
218 public boolean hasMaster() {
219 return super.getData(false) != null;
220 }
221
222
223
224
225
226
227 static byte[] toByteArray(final ServerName sn, int infoPort) {
228 ZooKeeperProtos.Master.Builder mbuilder = ZooKeeperProtos.Master.newBuilder();
229 HBaseProtos.ServerName.Builder snbuilder = HBaseProtos.ServerName.newBuilder();
230 snbuilder.setHostName(sn.getHostname());
231 snbuilder.setPort(sn.getPort());
232 snbuilder.setStartCode(sn.getStartcode());
233 mbuilder.setMaster(snbuilder.build());
234 mbuilder.setRpcVersion(HConstants.RPC_CURRENT_VERSION);
235 mbuilder.setInfoPort(infoPort);
236 return ProtobufUtil.prependPBMagic(mbuilder.build().toByteArray());
237 }
238
239
240
241
242
243
244 public static ZooKeeperProtos.Master parse(byte[] data) throws DeserializationException {
245 if (data == null) {
246 return null;
247 }
248 int prefixLen = ProtobufUtil.lengthOfPBMagic();
249 try {
250 return ZooKeeperProtos.Master.PARSER.parseFrom(data, prefixLen, data.length - prefixLen);
251 } catch (InvalidProtocolBufferException e) {
252 throw new DeserializationException(e);
253 }
254 }
255
256
257
258
259
260 public static boolean deleteIfEquals(ZooKeeperWatcher zkw, final String content) {
261 if (content == null){
262 throw new IllegalArgumentException("Content must not be null");
263 }
264
265 try {
266 Stat stat = new Stat();
267 byte[] data = ZKUtil.getDataNoWatch(zkw, zkw.getMasterAddressZNode(), stat);
268 ServerName sn = ServerName.parseFrom(data);
269 if (sn != null && content.equals(sn.toString())) {
270 return (ZKUtil.deleteNode(zkw, zkw.getMasterAddressZNode(), stat.getVersion()));
271 }
272 } catch (KeeperException e) {
273 LOG.warn("Can't get or delete the master znode", e);
274 } catch (DeserializationException e) {
275 LOG.warn("Can't get or delete the master znode", e);
276 }
277
278 return false;
279 }
280 }