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 boolean isThisOurZnode(String znode) {
158 return ZKUtil.joinZNode(this.queuesZNode, znode).equals(this.myQueuesZnode);
159 }
160
161 @Override
162 public SortedMap<String, SortedSet<String>> claimQueues(String regionserverZnode) {
163 SortedMap<String, SortedSet<String>> newQueues = new TreeMap<String, SortedSet<String>>();
164
165 if (conf.getBoolean(HConstants.ZOOKEEPER_USEMULTI, true)) {
166 LOG.info("Atomically moving " + regionserverZnode + "'s hlogs to my queue");
167 newQueues = copyQueuesFromRSUsingMulti(regionserverZnode);
168 } else {
169 LOG.info("Moving " + regionserverZnode + "'s hlogs to my queue");
170 if (!lockOtherRS(regionserverZnode)) {
171 return newQueues;
172 }
173 newQueues = copyQueuesFromRS(regionserverZnode);
174 deleteAnotherRSQueues(regionserverZnode);
175 }
176 return newQueues;
177 }
178
179 @Override
180 public void removeAllQueues() {
181 try {
182 ZKUtil.deleteNodeRecursively(this.zookeeper, this.myQueuesZnode);
183 } catch (KeeperException e) {
184
185 if (e instanceof KeeperException.SessionExpiredException) {
186 return;
187 }
188 this.abortable.abort("Failed to delete replication queues for region server: "
189 + this.myQueuesZnode, e);
190 }
191 }
192
193 @Override
194 public List<String> getLogsInQueue(String queueId) {
195 String znode = ZKUtil.joinZNode(this.myQueuesZnode, queueId);
196 List<String> result = null;
197 try {
198 result = ZKUtil.listChildrenNoWatch(this.zookeeper, znode);
199 } catch (KeeperException e) {
200 this.abortable.abort("Failed to get list of hlogs for queueId=" + queueId, e);
201 }
202 return result;
203 }
204
205 @Override
206 public List<String> getAllQueues() {
207 List<String> listOfQueues = null;
208 try {
209 listOfQueues = ZKUtil.listChildrenNoWatch(this.zookeeper, this.myQueuesZnode);
210 } catch (KeeperException e) {
211 this.abortable.abort("Failed to get a list of queues for region server: "
212 + this.myQueuesZnode, e);
213 }
214 return listOfQueues;
215 }
216
217
218
219
220
221
222 private boolean lockOtherRS(String znode) {
223 try {
224 String parent = ZKUtil.joinZNode(this.queuesZNode, znode);
225 if (parent.equals(this.myQueuesZnode)) {
226 LOG.warn("Won't lock because this is us, we're dead!");
227 return false;
228 }
229 String p = ZKUtil.joinZNode(parent, RS_LOCK_ZNODE);
230 ZKUtil.createAndWatch(this.zookeeper, p, lockToByteArray(this.myQueuesZnode));
231 } catch (KeeperException e) {
232
233
234
235
236
237 if (e instanceof KeeperException.NoNodeException
238 || e instanceof KeeperException.NodeExistsException) {
239 LOG.info("Won't transfer the queue," + " another RS took care of it because of: "
240 + e.getMessage());
241 } else {
242 LOG.info("Failed lock other rs", e);
243 }
244 return false;
245 }
246 return true;
247 }
248
249
250
251
252
253 private void deleteAnotherRSQueues(String regionserverZnode) {
254 String fullpath = ZKUtil.joinZNode(this.queuesZNode, regionserverZnode);
255 try {
256 List<String> clusters = ZKUtil.listChildrenNoWatch(this.zookeeper, fullpath);
257 for (String cluster : clusters) {
258
259 if (cluster.equals(RS_LOCK_ZNODE)) {
260 continue;
261 }
262 String fullClusterPath = ZKUtil.joinZNode(fullpath, cluster);
263 ZKUtil.deleteNodeRecursively(this.zookeeper, fullClusterPath);
264 }
265
266 ZKUtil.deleteNodeRecursively(this.zookeeper, fullpath);
267 } catch (KeeperException e) {
268 if (e instanceof KeeperException.NoNodeException
269 || e instanceof KeeperException.NotEmptyException) {
270
271
272
273 if (e.getPath().equals(fullpath)) {
274 return;
275 }
276 }
277 this.abortable.abort("Failed to delete replication queues for region server: "
278 + regionserverZnode, e);
279 }
280 }
281
282
283
284
285
286
287
288 private SortedMap<String, SortedSet<String>> copyQueuesFromRSUsingMulti(String znode) {
289 SortedMap<String, SortedSet<String>> queues = new TreeMap<String, SortedSet<String>>();
290
291 String deadRSZnodePath = ZKUtil.joinZNode(this.queuesZNode, znode);
292 List<String> peerIdsToProcess = null;
293 List<ZKUtilOp> listOfOps = new ArrayList<ZKUtil.ZKUtilOp>();
294 try {
295 peerIdsToProcess = ZKUtil.listChildrenNoWatch(this.zookeeper, deadRSZnodePath);
296 if (peerIdsToProcess == null) return queues;
297 for (String peerId : peerIdsToProcess) {
298 ReplicationQueueInfo replicationQueueInfo = new ReplicationQueueInfo(peerId);
299 if (!peerExists(replicationQueueInfo.getPeerId())) {
300 LOG.warn("Peer " + peerId + " didn't exist, skipping the replay");
301
302 continue;
303 }
304 String newPeerId = peerId + "-" + znode;
305 String newPeerZnode = ZKUtil.joinZNode(this.myQueuesZnode, newPeerId);
306
307 String oldClusterZnode = ZKUtil.joinZNode(deadRSZnodePath, peerId);
308 List<String> hlogs = ZKUtil.listChildrenNoWatch(this.zookeeper, oldClusterZnode);
309 if (hlogs == null || hlogs.size() == 0) {
310 listOfOps.add(ZKUtilOp.deleteNodeFailSilent(oldClusterZnode));
311 continue;
312 }
313
314 SortedSet<String> logQueue = new TreeSet<String>();
315 queues.put(newPeerId, logQueue);
316 ZKUtilOp op = ZKUtilOp.createAndFailSilent(newPeerZnode, HConstants.EMPTY_BYTE_ARRAY);
317 listOfOps.add(op);
318
319 for (String hlog : hlogs) {
320 String oldHlogZnode = ZKUtil.joinZNode(oldClusterZnode, hlog);
321 byte[] logOffset = ZKUtil.getData(this.zookeeper, oldHlogZnode);
322 LOG.debug("Creating " + hlog + " with data " + Bytes.toString(logOffset));
323 String newLogZnode = ZKUtil.joinZNode(newPeerZnode, hlog);
324 listOfOps.add(ZKUtilOp.createAndFailSilent(newLogZnode, logOffset));
325
326 listOfOps.add(ZKUtilOp.deleteNodeFailSilent(oldHlogZnode));
327 logQueue.add(hlog);
328 }
329
330 listOfOps.add(ZKUtilOp.deleteNodeFailSilent(oldClusterZnode));
331 }
332
333 listOfOps.add(ZKUtilOp.deleteNodeFailSilent(deadRSZnodePath));
334 LOG.debug(" The multi list size is: " + listOfOps.size());
335 ZKUtil.multiOrSequential(this.zookeeper, listOfOps, false);
336 LOG.info("Atomically moved the dead regionserver logs. ");
337 } catch (KeeperException e) {
338
339 LOG.warn("Got exception in copyQueuesFromRSUsingMulti: ", e);
340 queues.clear();
341 }
342 return queues;
343 }
344
345
346
347
348
349
350
351 private SortedMap<String, SortedSet<String>> copyQueuesFromRS(String znode) {
352
353
354 SortedMap<String, SortedSet<String>> queues = new TreeMap<String, SortedSet<String>>();
355 try {
356 String nodePath = ZKUtil.joinZNode(this.queuesZNode, znode);
357 List<String> clusters = ZKUtil.listChildrenNoWatch(this.zookeeper, nodePath);
358
359 if (clusters == null || clusters.size() <= 1) {
360 return queues;
361 }
362
363 clusters.remove(RS_LOCK_ZNODE);
364 for (String cluster : clusters) {
365 ReplicationQueueInfo replicationQueueInfo = new ReplicationQueueInfo(cluster);
366 if (!peerExists(replicationQueueInfo.getPeerId())) {
367 LOG.warn("Peer " + cluster + " didn't exist, skipping the replay");
368
369 continue;
370 }
371
372
373
374 String newCluster = cluster + "-" + znode;
375 String newClusterZnode = ZKUtil.joinZNode(this.myQueuesZnode, newCluster);
376 String clusterPath = ZKUtil.joinZNode(nodePath, cluster);
377 List<String> hlogs = ZKUtil.listChildrenNoWatch(this.zookeeper, clusterPath);
378
379 if (hlogs == null || hlogs.size() == 0) {
380 continue;
381 }
382 ZKUtil.createNodeIfNotExistsAndWatch(this.zookeeper, newClusterZnode,
383 HConstants.EMPTY_BYTE_ARRAY);
384 SortedSet<String> logQueue = new TreeSet<String>();
385 queues.put(newCluster, logQueue);
386 for (String hlog : hlogs) {
387 String z = ZKUtil.joinZNode(clusterPath, hlog);
388 byte[] positionBytes = ZKUtil.getData(this.zookeeper, z);
389 long position = 0;
390 try {
391 position = ZKUtil.parseHLogPositionFrom(positionBytes);
392 } catch (DeserializationException e) {
393 LOG.warn("Failed parse of hlog position from the following znode: " + z
394 + ", Exception: " + e);
395 }
396 LOG.debug("Creating " + hlog + " with data " + position);
397 String child = ZKUtil.joinZNode(newClusterZnode, hlog);
398
399
400 ZKUtil.createAndWatch(this.zookeeper, child, positionBytes);
401 logQueue.add(hlog);
402 }
403 }
404 } catch (KeeperException e) {
405 this.abortable.abort("Copy queues from rs", e);
406 }
407 return queues;
408 }
409
410
411
412
413
414
415 static byte[] lockToByteArray(final String lockOwner) {
416 byte[] bytes =
417 ZooKeeperProtos.ReplicationLock.newBuilder().setLockOwner(lockOwner).build().toByteArray();
418 return ProtobufUtil.prependPBMagic(bytes);
419 }
420 }