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