1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19 package org.apache.hadoop.hbase.replication;
20
21 import java.util.ArrayList;
22 import java.util.List;
23 import java.util.SortedMap;
24 import java.util.SortedSet;
25 import java.util.TreeMap;
26 import java.util.TreeSet;
27
28 import org.apache.commons.logging.Log;
29 import org.apache.commons.logging.LogFactory;
30 import org.apache.hadoop.conf.Configuration;
31 import org.apache.hadoop.hbase.Abortable;
32 import org.apache.hadoop.hbase.HConstants;
33 import org.apache.hadoop.hbase.exceptions.DeserializationException;
34 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
35 import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
36 import org.apache.hadoop.hbase.util.Bytes;
37 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
38 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
39 import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp;
40 import org.apache.zookeeper.KeeperException;
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63 public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements ReplicationQueues {
64
65
66 private String myQueuesZnode;
67
68 private final static String RS_LOCK_ZNODE = "lock";
69
70 private static final Log LOG = LogFactory.getLog(ReplicationQueuesZKImpl.class);
71
72 public ReplicationQueuesZKImpl(final ZooKeeperWatcher zk, Configuration conf,
73 Abortable abortable) {
74 super(zk, conf, abortable);
75 }
76
77 @Override
78 public void init(String serverName) throws ReplicationException {
79 this.myQueuesZnode = ZKUtil.joinZNode(this.queuesZNode, serverName);
80 try {
81 ZKUtil.createWithParents(this.zookeeper, this.myQueuesZnode);
82 } catch (KeeperException e) {
83 throw new ReplicationException("Could not initialize replication queues.", e);
84 }
85 }
86
87 @Override
88 public void removeQueue(String queueId) {
89 try {
90 ZKUtil.deleteNodeRecursively(this.zookeeper, ZKUtil.joinZNode(this.myQueuesZnode, queueId));
91 } catch (KeeperException e) {
92 this.abortable.abort("Failed to delete queue (queueId=" + queueId + ")", e);
93 }
94 }
95
96 @Override
97 public void addLog(String queueId, String filename) throws ReplicationException {
98 String znode = ZKUtil.joinZNode(this.myQueuesZnode, queueId);
99 znode = ZKUtil.joinZNode(znode, filename);
100 try {
101 ZKUtil.createWithParents(this.zookeeper, znode);
102 } catch (KeeperException e) {
103 throw new ReplicationException(
104 "Could not add log because znode could not be created. queueId=" + queueId
105 + ", filename=" + filename);
106 }
107 }
108
109 @Override
110 public void removeLog(String queueId, String filename) {
111 try {
112 String znode = ZKUtil.joinZNode(this.myQueuesZnode, queueId);
113 znode = ZKUtil.joinZNode(znode, filename);
114 ZKUtil.deleteNode(this.zookeeper, znode);
115 } catch (KeeperException e) {
116 this.abortable.abort("Failed to remove hlog from queue (queueId=" + queueId + ", filename="
117 + filename + ")", e);
118 }
119 }
120
121 @Override
122 public void setLogPosition(String queueId, String filename, long position) {
123 try {
124 String znode = ZKUtil.joinZNode(this.myQueuesZnode, queueId);
125 znode = ZKUtil.joinZNode(znode, filename);
126
127 ZKUtil.setData(this.zookeeper, znode, ZKUtil.positionToByteArray(position));
128 } catch (KeeperException e) {
129 this.abortable.abort("Failed to write replication hlog position (filename=" + filename
130 + ", position=" + position + ")", e);
131 }
132 }
133
134 @Override
135 public long getLogPosition(String queueId, String filename) throws ReplicationException {
136 String clusterZnode = ZKUtil.joinZNode(this.myQueuesZnode, queueId);
137 String znode = ZKUtil.joinZNode(clusterZnode, filename);
138 byte[] bytes = null;
139 try {
140 bytes = ZKUtil.getData(this.zookeeper, znode);
141 } catch (KeeperException e) {
142 throw new ReplicationException("Internal Error: could not get position in log for queueId="
143 + queueId + ", filename=" + filename, e);
144 }
145 try {
146 return ZKUtil.parseHLogPositionFrom(bytes);
147 } catch (DeserializationException de) {
148 LOG.warn("Failed to parse HLogPosition for queueId=" + queueId + " and hlog=" + filename
149 + "znode content, continuing.");
150 }
151
152
153 return 0;
154 }
155
156 @Override
157 public SortedMap<String, SortedSet<String>> claimQueues(String regionserverZnode) {
158 SortedMap<String, SortedSet<String>> newQueues = new TreeMap<String, SortedSet<String>>();
159 if (ZKUtil.joinZNode(this.queuesZNode, regionserverZnode).equals(this.myQueuesZnode)) {
160 LOG.warn("An attempt was made to claim our own queues on region server " + regionserverZnode);
161 return newQueues;
162 }
163
164 if (conf.getBoolean(HConstants.ZOOKEEPER_USEMULTI, true)) {
165 LOG.info("Atomically moving " + regionserverZnode + "'s hlogs to my queue");
166 newQueues = copyQueuesFromRSUsingMulti(regionserverZnode);
167 } else {
168 LOG.info("Moving " + regionserverZnode + "'s hlogs to my queue");
169 if (!lockOtherRS(regionserverZnode)) {
170 return newQueues;
171 }
172 newQueues = copyQueuesFromRS(regionserverZnode);
173 deleteAnotherRSQueues(regionserverZnode);
174 }
175 return newQueues;
176 }
177
178 @Override
179 public void removeAllQueues() {
180 try {
181 ZKUtil.deleteNodeRecursively(this.zookeeper, this.myQueuesZnode);
182 } catch (KeeperException e) {
183
184 if (e instanceof KeeperException.SessionExpiredException) {
185 return;
186 }
187 this.abortable.abort("Failed to delete replication queues for region server: "
188 + this.myQueuesZnode, e);
189 }
190 }
191
192 @Override
193 public List<String> getLogsInQueue(String queueId) {
194 String znode = ZKUtil.joinZNode(this.myQueuesZnode, queueId);
195 List<String> result = null;
196 try {
197 result = ZKUtil.listChildrenNoWatch(this.zookeeper, znode);
198 } catch (KeeperException e) {
199 this.abortable.abort("Failed to get list of hlogs for queueId=" + queueId, e);
200 }
201 return result;
202 }
203
204 @Override
205 public List<String> getAllQueues() {
206 List<String> listOfQueues = null;
207 try {
208 listOfQueues = ZKUtil.listChildrenNoWatch(this.zookeeper, this.myQueuesZnode);
209 } catch (KeeperException e) {
210 this.abortable.abort("Failed to get a list of queues for region server: "
211 + this.myQueuesZnode, e);
212 }
213 return listOfQueues;
214 }
215
216
217
218
219
220
221 private boolean lockOtherRS(String znode) {
222 try {
223 String parent = ZKUtil.joinZNode(this.queuesZNode, znode);
224 if (parent.equals(this.myQueuesZnode)) {
225 LOG.warn("Won't lock because this is us, we're dead!");
226 return false;
227 }
228 String p = ZKUtil.joinZNode(parent, RS_LOCK_ZNODE);
229 ZKUtil.createAndWatch(this.zookeeper, p, lockToByteArray(this.myQueuesZnode));
230 } catch (KeeperException e) {
231
232
233
234
235
236 if (e instanceof KeeperException.NoNodeException
237 || e instanceof KeeperException.NodeExistsException) {
238 LOG.info("Won't transfer the queue," + " another RS took care of it because of: "
239 + e.getMessage());
240 } else {
241 LOG.info("Failed lock other rs", e);
242 }
243 return false;
244 }
245 return true;
246 }
247
248
249
250
251
252 private void deleteAnotherRSQueues(String regionserverZnode) {
253 String fullpath = ZKUtil.joinZNode(this.queuesZNode, regionserverZnode);
254 try {
255 List<String> clusters = ZKUtil.listChildrenNoWatch(this.zookeeper, fullpath);
256 for (String cluster : clusters) {
257
258 if (cluster.equals(RS_LOCK_ZNODE)) {
259 continue;
260 }
261 String fullClusterPath = ZKUtil.joinZNode(fullpath, cluster);
262 ZKUtil.deleteNodeRecursively(this.zookeeper, fullClusterPath);
263 }
264
265 ZKUtil.deleteNodeRecursively(this.zookeeper, fullpath);
266 } catch (KeeperException e) {
267 if (e instanceof KeeperException.NoNodeException
268 || e instanceof KeeperException.NotEmptyException) {
269
270
271
272 if (e.getPath().equals(fullpath)) {
273 return;
274 }
275 }
276 this.abortable.abort("Failed to delete replication queues for region server: "
277 + regionserverZnode, e);
278 }
279 }
280
281
282
283
284
285
286
287 private SortedMap<String, SortedSet<String>> copyQueuesFromRSUsingMulti(String znode) {
288 SortedMap<String, SortedSet<String>> queues = new TreeMap<String, SortedSet<String>>();
289
290 String deadRSZnodePath = ZKUtil.joinZNode(this.queuesZNode, znode);
291 List<String> peerIdsToProcess = null;
292 List<ZKUtilOp> listOfOps = new ArrayList<ZKUtil.ZKUtilOp>();
293 try {
294 peerIdsToProcess = ZKUtil.listChildrenNoWatch(this.zookeeper, deadRSZnodePath);
295 if (peerIdsToProcess == null) return queues;
296 for (String peerId : peerIdsToProcess) {
297 ReplicationQueueInfo replicationQueueInfo = new ReplicationQueueInfo(peerId);
298 if (!peerExists(replicationQueueInfo.getPeerId())) {
299 LOG.warn("Peer " + peerId + " didn't exist, skipping the replay");
300
301 continue;
302 }
303 String newPeerId = peerId + "-" + znode;
304 String newPeerZnode = ZKUtil.joinZNode(this.myQueuesZnode, newPeerId);
305
306 String oldClusterZnode = ZKUtil.joinZNode(deadRSZnodePath, peerId);
307 List<String> hlogs = ZKUtil.listChildrenNoWatch(this.zookeeper, oldClusterZnode);
308 if (hlogs == null || hlogs.size() == 0) {
309 listOfOps.add(ZKUtilOp.deleteNodeFailSilent(oldClusterZnode));
310 continue;
311 }
312
313 SortedSet<String> logQueue = new TreeSet<String>();
314 queues.put(newPeerId, logQueue);
315 ZKUtilOp op = ZKUtilOp.createAndFailSilent(newPeerZnode, HConstants.EMPTY_BYTE_ARRAY);
316 listOfOps.add(op);
317
318 for (String hlog : hlogs) {
319 String oldHlogZnode = ZKUtil.joinZNode(oldClusterZnode, hlog);
320 byte[] logOffset = ZKUtil.getData(this.zookeeper, oldHlogZnode);
321 LOG.debug("Creating " + hlog + " with data " + Bytes.toString(logOffset));
322 String newLogZnode = ZKUtil.joinZNode(newPeerZnode, hlog);
323 listOfOps.add(ZKUtilOp.createAndFailSilent(newLogZnode, logOffset));
324
325 listOfOps.add(ZKUtilOp.deleteNodeFailSilent(oldHlogZnode));
326 logQueue.add(hlog);
327 }
328
329 listOfOps.add(ZKUtilOp.deleteNodeFailSilent(oldClusterZnode));
330 }
331
332 listOfOps.add(ZKUtilOp.deleteNodeFailSilent(deadRSZnodePath));
333 LOG.debug(" The multi list size is: " + listOfOps.size());
334 ZKUtil.multiOrSequential(this.zookeeper, listOfOps, false);
335 LOG.info("Atomically moved the dead regionserver logs. ");
336 } catch (KeeperException e) {
337
338 LOG.warn("Got exception in copyQueuesFromRSUsingMulti: ", e);
339 queues.clear();
340 }
341 return queues;
342 }
343
344
345
346
347
348
349
350 private SortedMap<String, SortedSet<String>> copyQueuesFromRS(String znode) {
351
352
353 SortedMap<String, SortedSet<String>> queues = new TreeMap<String, SortedSet<String>>();
354 try {
355 String nodePath = ZKUtil.joinZNode(this.queuesZNode, znode);
356 List<String> clusters = ZKUtil.listChildrenNoWatch(this.zookeeper, nodePath);
357
358 if (clusters == null || clusters.size() <= 1) {
359 return queues;
360 }
361
362 clusters.remove(RS_LOCK_ZNODE);
363 for (String cluster : clusters) {
364 ReplicationQueueInfo replicationQueueInfo = new ReplicationQueueInfo(cluster);
365 if (!peerExists(replicationQueueInfo.getPeerId())) {
366 LOG.warn("Peer " + cluster + " didn't exist, skipping the replay");
367
368 continue;
369 }
370
371
372
373 String newCluster = cluster + "-" + znode;
374 String newClusterZnode = ZKUtil.joinZNode(this.myQueuesZnode, newCluster);
375 String clusterPath = ZKUtil.joinZNode(nodePath, cluster);
376 List<String> hlogs = ZKUtil.listChildrenNoWatch(this.zookeeper, clusterPath);
377
378 if (hlogs == null || hlogs.size() == 0) {
379 continue;
380 }
381 ZKUtil.createNodeIfNotExistsAndWatch(this.zookeeper, newClusterZnode,
382 HConstants.EMPTY_BYTE_ARRAY);
383 SortedSet<String> logQueue = new TreeSet<String>();
384 queues.put(newCluster, logQueue);
385 for (String hlog : hlogs) {
386 String z = ZKUtil.joinZNode(clusterPath, hlog);
387 byte[] positionBytes = ZKUtil.getData(this.zookeeper, z);
388 long position = 0;
389 try {
390 position = ZKUtil.parseHLogPositionFrom(positionBytes);
391 } catch (DeserializationException e) {
392 LOG.warn("Failed parse of hlog position from the following znode: " + z
393 + ", Exception: " + e);
394 }
395 LOG.debug("Creating " + hlog + " with data " + position);
396 String child = ZKUtil.joinZNode(newClusterZnode, hlog);
397
398
399 ZKUtil.createAndWatch(this.zookeeper, child, positionBytes);
400 logQueue.add(hlog);
401 }
402 }
403 } catch (KeeperException e) {
404 this.abortable.abort("Copy queues from rs", e);
405 }
406 return queues;
407 }
408
409
410
411
412
413
414 static byte[] lockToByteArray(final String lockOwner) {
415 byte[] bytes =
416 ZooKeeperProtos.ReplicationLock.newBuilder().setLockOwner(lockOwner).build().toByteArray();
417 return ProtobufUtil.prependPBMagic(bytes);
418 }
419 }