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 if (ZKUtil.checkExists(this.zookeeper, this.queuesZNode) < 0) {
44 ZKUtil.createWithParents(this.zookeeper, this.queuesZNode);
45 }
46 } catch (KeeperException e) {
47 throw new ReplicationException("Internal error while initializing a queues client", e);
48 }
49 }
50
51 @Override
52 public List<String> getListOfReplicators() throws KeeperException {
53 return super.getListOfReplicatorsZK();
54 }
55
56 @Override
57 public List<String> getLogsInQueue(String serverName, String queueId) throws KeeperException {
58 String znode = ZKUtil.joinZNode(this.queuesZNode, serverName);
59 znode = ZKUtil.joinZNode(znode, queueId);
60 List<String> result = null;
61 try {
62 result = ZKUtil.listChildrenNoWatch(this.zookeeper, znode);
63 } catch (KeeperException e) {
64 this.abortable.abort("Failed to get list of wals for queueId=" + queueId
65 + " and serverName=" + serverName, e);
66 throw e;
67 }
68 return result;
69 }
70
71 @Override
72 public List<String> getAllQueues(String serverName) throws KeeperException {
73 String znode = ZKUtil.joinZNode(this.queuesZNode, serverName);
74 List<String> result = null;
75 try {
76 result = ZKUtil.listChildrenNoWatch(this.zookeeper, znode);
77 } catch (KeeperException e) {
78 this.abortable.abort("Failed to get list of queues for serverName=" + serverName, e);
79 throw e;
80 }
81 return result;
82 }
83
84 @Override public int getQueuesZNodeCversion() throws KeeperException {
85 try {
86 Stat stat = new Stat();
87 ZKUtil.getDataNoWatch(this.zookeeper, this.queuesZNode, stat);
88 return stat.getCversion();
89 } catch (KeeperException e) {
90 this.abortable.abort("Failed to get stat of replication rs node", e);
91 throw e;
92 }
93 }
94
95 @Override
96 public int getHFileRefsNodeChangeVersion() throws KeeperException {
97 Stat stat = new Stat();
98 try {
99 ZKUtil.getDataNoWatch(this.zookeeper, this.hfileRefsZNode, stat);
100 } catch (KeeperException e) {
101 this.abortable.abort("Failed to get stat of replication hfile references node.", e);
102 throw e;
103 }
104 return stat.getCversion();
105 }
106
107 @Override
108 public List<String> getAllPeersFromHFileRefsQueue() throws KeeperException {
109 List<String> result = null;
110 try {
111 result = ZKUtil.listChildrenNoWatch(this.zookeeper, this.hfileRefsZNode);
112 } catch (KeeperException e) {
113 this.abortable.abort("Failed to get list of all peers in hfile references node.", e);
114 throw e;
115 }
116 return result;
117 }
118
119 @Override
120 public List<String> getReplicableHFiles(String peerId) throws KeeperException {
121 String znode = ZKUtil.joinZNode(this.hfileRefsZNode, peerId);
122 List<String> result = null;
123 try {
124 result = ZKUtil.listChildrenNoWatch(this.zookeeper, znode);
125 } catch (KeeperException e) {
126 this.abortable.abort("Failed to get list of hfile references for peerId=" + peerId, e);
127 throw e;
128 }
129 return result;
130 }
131 }