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.List;
22
23 import org.apache.hadoop.hbase.classification.InterfaceAudience;
24 import org.apache.hadoop.conf.Configuration;
25 import org.apache.hadoop.hbase.Abortable;
26 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
27 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
28 import org.apache.zookeeper.KeeperException;
29 import org.apache.zookeeper.data.Stat;
30
31 @InterfaceAudience.Private
32 public class ReplicationQueuesClientZKImpl extends ReplicationStateZKBase implements
33 ReplicationQueuesClient {
34
35 public ReplicationQueuesClientZKImpl(final ZooKeeperWatcher zk, Configuration conf,
36 Abortable abortable) {
37 super(zk, conf, abortable);
38 }
39
40 @Override
41 public void init() throws ReplicationException {
42 try {
43 ZKUtil.createWithParents(this.zookeeper, this.queuesZNode);
44 } catch (KeeperException e) {
45 throw new ReplicationException("Internal error while initializing a queues client", e);
46 }
47 }
48
49 @Override
50 public List<String> getLogsInQueue(String serverName, String queueId) throws KeeperException {
51 String znode = ZKUtil.joinZNode(this.queuesZNode, serverName);
52 znode = ZKUtil.joinZNode(znode, queueId);
53 List<String> result = null;
54 try {
55 result = ZKUtil.listChildrenNoWatch(this.zookeeper, znode);
56 } catch (KeeperException e) {
57 this.abortable.abort("Failed to get list of wals for queueId=" + queueId
58 + " and serverName=" + serverName, e);
59 throw e;
60 }
61 return result;
62 }
63
64 @Override
65 public List<String> getAllQueues(String serverName) throws KeeperException {
66 String znode = ZKUtil.joinZNode(this.queuesZNode, serverName);
67 List<String> result = null;
68 try {
69 result = ZKUtil.listChildrenNoWatch(this.zookeeper, znode);
70 } catch (KeeperException e) {
71 this.abortable.abort("Failed to get list of queues for serverName=" + serverName, e);
72 throw e;
73 }
74 return result;
75 }
76
77 @Override public int getQueuesZNodeCversion() throws KeeperException {
78 try {
79 Stat stat = new Stat();
80 ZKUtil.getDataNoWatch(this.zookeeper, this.queuesZNode, stat);
81 return stat.getCversion();
82 } catch (KeeperException e) {
83 this.abortable.abort("Failed to get stat of replication rs node", e);
84 throw e;
85 }
86 }
87 }