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.hbase.classification.InterfaceAudience;
31 import org.apache.hadoop.conf.Configuration;
32 import org.apache.hadoop.hbase.Abortable;
33 import org.apache.hadoop.hbase.HConstants;
34 import org.apache.hadoop.hbase.exceptions.DeserializationException;
35 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
36 import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
37 import org.apache.hadoop.hbase.util.Bytes;
38 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
39 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
40 import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp;
41 import org.apache.zookeeper.KeeperException;
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64 @InterfaceAudience.Private
65 public class ReplicationQueuesZKImpl extends ReplicationStateZKBase implements ReplicationQueues {
66
67
68 private String myQueuesZnode;
69
70 private final static String RS_LOCK_ZNODE = "lock";
71
72 private static final Log LOG = LogFactory.getLog(ReplicationQueuesZKImpl.class);
73
74 public ReplicationQueuesZKImpl(final ZooKeeperWatcher zk, Configuration conf,
75 Abortable abortable) {
76 super(zk, conf, abortable);
77 }
78
79 @Override
80 public void init(String serverName) throws ReplicationException {
81 this.myQueuesZnode = ZKUtil.joinZNode(this.queuesZNode, serverName);
82 try {
83 ZKUtil.createWithParents(this.zookeeper, this.myQueuesZnode);
84 } catch (KeeperException e) {
85 throw new ReplicationException("Could not initialize replication queues.", e);
86 }
87
88 if (replicationForBulkLoadEnabled) {
89 try {
90 ZKUtil.createWithParents(this.zookeeper, this.hfileRefsZNode);
91 } catch (KeeperException e) {
92 throw new ReplicationException("Could not initialize hfile references replication queue.",
93 e);
94 }
95 }
96 }
97
98 @Override
99 public void removeQueue(String queueId) {
100 try {
101 ZKUtil.deleteNodeRecursively(this.zookeeper, ZKUtil.joinZNode(this.myQueuesZnode, queueId));
102 } catch (KeeperException e) {
103 this.abortable.abort("Failed to delete queue (queueId=" + queueId + ")", e);
104 }
105 }
106
107 @Override
108 public void addLog(String queueId, String filename) throws ReplicationException {
109 String znode = ZKUtil.joinZNode(this.myQueuesZnode, queueId);
110 znode = ZKUtil.joinZNode(znode, filename);
111 try {
112 ZKUtil.createWithParents(this.zookeeper, znode);
113 } catch (KeeperException e) {
114 throw new ReplicationException(
115 "Could not add log because znode could not be created. queueId=" + queueId
116 + ", filename=" + filename);
117 }
118 }
119
120 @Override
121 public void removeLog(String queueId, String filename) {
122 try {
123 String znode = ZKUtil.joinZNode(this.myQueuesZnode, queueId);
124 znode = ZKUtil.joinZNode(znode, filename);
125 ZKUtil.deleteNode(this.zookeeper, znode);
126 } catch (KeeperException e) {
127 this.abortable.abort("Failed to remove wal from queue (queueId=" + queueId + ", filename="
128 + filename + ")", e);
129 }
130 }
131
132 @Override
133 public void setLogPosition(String queueId, String filename, long position) {
134 try {
135 String znode = ZKUtil.joinZNode(this.myQueuesZnode, queueId);
136 znode = ZKUtil.joinZNode(znode, filename);
137
138 ZKUtil.setData(this.zookeeper, znode, ZKUtil.positionToByteArray(position));
139 } catch (KeeperException e) {
140 this.abortable.abort("Failed to write replication wal position (filename=" + filename
141 + ", position=" + position + ")", e);
142 }
143 }
144
145 @Override
146 public long getLogPosition(String queueId, String filename) throws ReplicationException {
147 String clusterZnode = ZKUtil.joinZNode(this.myQueuesZnode, queueId);
148 String znode = ZKUtil.joinZNode(clusterZnode, filename);
149 byte[] bytes = null;
150 try {
151 bytes = ZKUtil.getData(this.zookeeper, znode);
152 } catch (KeeperException e) {
153 throw new ReplicationException("Internal Error: could not get position in log for queueId="
154 + queueId + ", filename=" + filename, e);
155 } catch (InterruptedException e) {
156 Thread.currentThread().interrupt();
157 return 0;
158 }
159 try {
160 return ZKUtil.parseWALPositionFrom(bytes);
161 } catch (DeserializationException de) {
162 LOG.warn("Failed to parse WALPosition for queueId=" + queueId + " and wal=" + filename
163 + "znode content, continuing.");
164 }
165
166
167 return 0;
168 }
169
170 @Override
171 public boolean isThisOurZnode(String znode) {
172 return ZKUtil.joinZNode(this.queuesZNode, znode).equals(this.myQueuesZnode);
173 }
174
175 @Override
176 public SortedMap<String, SortedSet<String>> claimQueues(String regionserverZnode) {
177 SortedMap<String, SortedSet<String>> newQueues = new TreeMap<String, SortedSet<String>>();
178
179 if (conf.getBoolean(HConstants.ZOOKEEPER_USEMULTI, true)) {
180 LOG.info("Atomically moving " + regionserverZnode + "'s wals to my queue");
181 newQueues = copyQueuesFromRSUsingMulti(regionserverZnode);
182 } else {
183 LOG.info("Moving " + regionserverZnode + "'s wals to my queue");
184 if (!lockOtherRS(regionserverZnode)) {
185 return newQueues;
186 }
187 newQueues = copyQueuesFromRS(regionserverZnode);
188 deleteAnotherRSQueues(regionserverZnode);
189 }
190 return newQueues;
191 }
192
193 @Override
194 public void removeAllQueues() {
195 try {
196 ZKUtil.deleteNodeRecursively(this.zookeeper, this.myQueuesZnode);
197 } catch (KeeperException e) {
198
199 if (e instanceof KeeperException.SessionExpiredException) {
200 return;
201 }
202 this.abortable.abort("Failed to delete replication queues for region server: "
203 + this.myQueuesZnode, e);
204 }
205 }
206
207 @Override
208 public List<String> getLogsInQueue(String queueId) {
209 String znode = ZKUtil.joinZNode(this.myQueuesZnode, queueId);
210 List<String> result = null;
211 try {
212 result = ZKUtil.listChildrenNoWatch(this.zookeeper, znode);
213 } catch (KeeperException e) {
214 this.abortable.abort("Failed to get list of wals for queueId=" + queueId, e);
215 }
216 return result;
217 }
218
219 @Override
220 public List<String> getAllQueues() {
221 List<String> listOfQueues = null;
222 try {
223 listOfQueues = ZKUtil.listChildrenNoWatch(this.zookeeper, this.myQueuesZnode);
224 } catch (KeeperException e) {
225 this.abortable.abort("Failed to get a list of queues for region server: "
226 + this.myQueuesZnode, e);
227 }
228 return listOfQueues;
229 }
230
231
232
233
234
235
236 private boolean lockOtherRS(String znode) {
237 try {
238 String parent = ZKUtil.joinZNode(this.queuesZNode, znode);
239 if (parent.equals(this.myQueuesZnode)) {
240 LOG.warn("Won't lock because this is us, we're dead!");
241 return false;
242 }
243 String p = ZKUtil.joinZNode(parent, RS_LOCK_ZNODE);
244 ZKUtil.createAndWatch(this.zookeeper, p, lockToByteArray(this.myQueuesZnode));
245 } catch (KeeperException e) {
246
247
248
249
250
251 if (e instanceof KeeperException.NoNodeException
252 || e instanceof KeeperException.NodeExistsException) {
253 LOG.info("Won't transfer the queue," + " another RS took care of it because of: "
254 + e.getMessage());
255 } else {
256 LOG.info("Failed lock other rs", e);
257 }
258 return false;
259 }
260 return true;
261 }
262
263
264
265
266
267 private void deleteAnotherRSQueues(String regionserverZnode) {
268 String fullpath = ZKUtil.joinZNode(this.queuesZNode, regionserverZnode);
269 try {
270 List<String> clusters = ZKUtil.listChildrenNoWatch(this.zookeeper, fullpath);
271 for (String cluster : clusters) {
272
273 if (cluster.equals(RS_LOCK_ZNODE)) {
274 continue;
275 }
276 String fullClusterPath = ZKUtil.joinZNode(fullpath, cluster);
277 ZKUtil.deleteNodeRecursively(this.zookeeper, fullClusterPath);
278 }
279
280 ZKUtil.deleteNodeRecursively(this.zookeeper, fullpath);
281 } catch (KeeperException e) {
282 if (e instanceof KeeperException.NoNodeException
283 || e instanceof KeeperException.NotEmptyException) {
284
285
286
287 if (e.getPath().equals(fullpath)) {
288 return;
289 }
290 }
291 this.abortable.abort("Failed to delete replication queues for region server: "
292 + regionserverZnode, e);
293 }
294 }
295
296
297
298
299
300
301
302 private SortedMap<String, SortedSet<String>> copyQueuesFromRSUsingMulti(String znode) {
303 SortedMap<String, SortedSet<String>> queues = new TreeMap<String, SortedSet<String>>();
304
305 String deadRSZnodePath = ZKUtil.joinZNode(this.queuesZNode, znode);
306 List<String> peerIdsToProcess = null;
307 List<ZKUtilOp> listOfOps = new ArrayList<ZKUtil.ZKUtilOp>();
308 try {
309 peerIdsToProcess = ZKUtil.listChildrenNoWatch(this.zookeeper, deadRSZnodePath);
310 if (peerIdsToProcess == null) return queues;
311 for (String peerId : peerIdsToProcess) {
312 ReplicationQueueInfo replicationQueueInfo = new ReplicationQueueInfo(peerId);
313 if (!peerExists(replicationQueueInfo.getPeerId())) {
314 LOG.warn("Peer " + peerId + " didn't exist, skipping the replay");
315
316 continue;
317 }
318 String newPeerId = peerId + "-" + znode;
319 String newPeerZnode = ZKUtil.joinZNode(this.myQueuesZnode, newPeerId);
320
321 String oldClusterZnode = ZKUtil.joinZNode(deadRSZnodePath, peerId);
322 List<String> wals = ZKUtil.listChildrenNoWatch(this.zookeeper, oldClusterZnode);
323 if (wals == null || wals.size() == 0) {
324 listOfOps.add(ZKUtilOp.deleteNodeFailSilent(oldClusterZnode));
325 continue;
326 }
327
328 SortedSet<String> logQueue = new TreeSet<String>();
329 queues.put(newPeerId, logQueue);
330 ZKUtilOp op = ZKUtilOp.createAndFailSilent(newPeerZnode, HConstants.EMPTY_BYTE_ARRAY);
331 listOfOps.add(op);
332
333 for (String wal : wals) {
334 String oldWalZnode = ZKUtil.joinZNode(oldClusterZnode, wal);
335 byte[] logOffset = ZKUtil.getData(this.zookeeper, oldWalZnode);
336 LOG.debug("Creating " + wal + " with data " + Bytes.toString(logOffset));
337 String newLogZnode = ZKUtil.joinZNode(newPeerZnode, wal);
338 listOfOps.add(ZKUtilOp.createAndFailSilent(newLogZnode, logOffset));
339
340 listOfOps.add(ZKUtilOp.deleteNodeFailSilent(oldWalZnode));
341 logQueue.add(wal);
342 }
343
344 listOfOps.add(ZKUtilOp.deleteNodeFailSilent(oldClusterZnode));
345 }
346
347
348
349 listOfOps.add(ZKUtilOp.deleteNodeFailSilent(deadRSZnodePath));
350 LOG.debug(" The multi list size is: " + listOfOps.size());
351 ZKUtil.multiOrSequential(this.zookeeper, listOfOps, false);
352 LOG.info("Atomically moved the dead regionserver logs. ");
353 } catch (KeeperException e) {
354
355 LOG.warn("Got exception in copyQueuesFromRSUsingMulti: ", e);
356 queues.clear();
357 } catch (InterruptedException e) {
358 LOG.warn("Got exception in copyQueuesFromRSUsingMulti: ", e);
359 queues.clear();
360 Thread.currentThread().interrupt();
361 }
362 return queues;
363 }
364
365
366
367
368
369
370
371 private SortedMap<String, SortedSet<String>> copyQueuesFromRS(String znode) {
372
373
374 SortedMap<String, SortedSet<String>> queues = new TreeMap<String, SortedSet<String>>();
375 try {
376 String nodePath = ZKUtil.joinZNode(this.queuesZNode, znode);
377 List<String> clusters = ZKUtil.listChildrenNoWatch(this.zookeeper, nodePath);
378
379 if (clusters == null || clusters.size() <= 1) {
380 return queues;
381 }
382
383 clusters.remove(RS_LOCK_ZNODE);
384 for (String cluster : clusters) {
385 ReplicationQueueInfo replicationQueueInfo = new ReplicationQueueInfo(cluster);
386 if (!peerExists(replicationQueueInfo.getPeerId())) {
387 LOG.warn("Peer " + cluster + " didn't exist, skipping the replay");
388
389 continue;
390 }
391
392
393
394 String newCluster = cluster + "-" + znode;
395 String newClusterZnode = ZKUtil.joinZNode(this.myQueuesZnode, newCluster);
396 String clusterPath = ZKUtil.joinZNode(nodePath, cluster);
397 List<String> wals = ZKUtil.listChildrenNoWatch(this.zookeeper, clusterPath);
398
399 if (wals == null || wals.size() == 0) {
400 continue;
401 }
402 ZKUtil.createNodeIfNotExistsAndWatch(this.zookeeper, newClusterZnode,
403 HConstants.EMPTY_BYTE_ARRAY);
404 SortedSet<String> logQueue = new TreeSet<String>();
405 queues.put(newCluster, logQueue);
406 for (String wal : wals) {
407 String z = ZKUtil.joinZNode(clusterPath, wal);
408 byte[] positionBytes = ZKUtil.getData(this.zookeeper, z);
409 long position = 0;
410 try {
411 position = ZKUtil.parseWALPositionFrom(positionBytes);
412 } catch (DeserializationException e) {
413 LOG.warn("Failed parse of wal position from the following znode: " + z
414 + ", Exception: " + e);
415 }
416 LOG.debug("Creating " + wal + " with data " + position);
417 String child = ZKUtil.joinZNode(newClusterZnode, wal);
418
419
420 ZKUtil.createAndWatch(this.zookeeper, child, positionBytes);
421 logQueue.add(wal);
422 }
423 }
424 } catch (KeeperException e) {
425 this.abortable.abort("Copy queues from rs", e);
426 } catch (InterruptedException e) {
427 LOG.warn(e);
428 Thread.currentThread().interrupt();
429 }
430 return queues;
431 }
432
433
434
435
436
437
438 static byte[] lockToByteArray(final String lockOwner) {
439 byte[] bytes =
440 ZooKeeperProtos.ReplicationLock.newBuilder().setLockOwner(lockOwner).build().toByteArray();
441 return ProtobufUtil.prependPBMagic(bytes);
442 }
443
444 @Override
445 public void addHFileRefs(String peerId, List<String> files) throws ReplicationException {
446 String peerZnode = ZKUtil.joinZNode(this.hfileRefsZNode, peerId);
447 boolean debugEnabled = LOG.isDebugEnabled();
448 if (debugEnabled) {
449 LOG.debug("Adding hfile references " + files + " in queue " + peerZnode);
450 }
451 List<ZKUtilOp> listOfOps = new ArrayList<ZKUtil.ZKUtilOp>();
452 int size = files.size();
453 for (int i = 0; i < size; i++) {
454 listOfOps.add(ZKUtilOp.createAndFailSilent(ZKUtil.joinZNode(peerZnode, files.get(i)),
455 HConstants.EMPTY_BYTE_ARRAY));
456 }
457 if (debugEnabled) {
458 LOG.debug(" The multi list size for adding hfile references in zk for node " + peerZnode
459 + " is " + listOfOps.size());
460 }
461 try {
462 ZKUtil.multiOrSequential(this.zookeeper, listOfOps, true);
463 } catch (KeeperException e) {
464 throw new ReplicationException("Failed to create hfile reference znode=" + e.getPath(), e);
465 }
466 }
467
468 @Override
469 public void removeHFileRefs(String peerId, List<String> files) {
470 String peerZnode = ZKUtil.joinZNode(this.hfileRefsZNode, peerId);
471 boolean debugEnabled = LOG.isDebugEnabled();
472 if (debugEnabled) {
473 LOG.debug("Removing hfile references " + files + " from queue " + peerZnode);
474 }
475 List<ZKUtilOp> listOfOps = new ArrayList<ZKUtil.ZKUtilOp>();
476 int size = files.size();
477 for (int i = 0; i < size; i++) {
478 listOfOps.add(ZKUtilOp.deleteNodeFailSilent(ZKUtil.joinZNode(peerZnode, files.get(i))));
479 }
480 if (debugEnabled) {
481 LOG.debug(" The multi list size for removing hfile references in zk for node " + peerZnode
482 + " is " + listOfOps.size());
483 }
484 try {
485 ZKUtil.multiOrSequential(this.zookeeper, listOfOps, true);
486 } catch (KeeperException e) {
487 LOG.error("Failed to remove hfile reference znode=" + e.getPath(), e);
488 }
489 }
490
491 @Override
492 public void addPeerToHFileRefs(String peerId) throws ReplicationException {
493 String peerZnode = ZKUtil.joinZNode(this.hfileRefsZNode, peerId);
494 try {
495 if (ZKUtil.checkExists(this.zookeeper, peerZnode) == -1) {
496 LOG.info("Adding peer " + peerId + " to hfile reference queue.");
497 ZKUtil.createWithParents(this.zookeeper, peerZnode);
498 }
499 } catch (KeeperException e) {
500 throw new ReplicationException("Failed to add peer " + peerId + " to hfile reference queue.",
501 e);
502 }
503 }
504 }