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.HashMap;
22 import java.util.List;
23 import java.util.Map;
24
25 import org.apache.hadoop.hbase.classification.InterfaceAudience;
26 import org.apache.hadoop.conf.Configuration;
27 import org.apache.hadoop.hbase.Abortable;
28 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
29 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
30 import org.apache.zookeeper.KeeperException;
31 import org.apache.zookeeper.data.Stat;
32
33 @InterfaceAudience.Private
34 public class ReplicationQueuesClientZKImpl extends ReplicationStateZKBase implements
35 ReplicationQueuesClient {
36
37 public ReplicationQueuesClientZKImpl(final ZooKeeperWatcher zk, Configuration conf,
38 Abortable abortable) {
39 super(zk, conf, abortable);
40 }
41
42 @Override
43 public void init() throws ReplicationException {
44 try {
45 if (ZKUtil.checkExists(this.zookeeper, this.queuesZNode) < 0) {
46 ZKUtil.createWithParents(this.zookeeper, this.queuesZNode);
47 }
48 } catch (KeeperException e) {
49 throw new ReplicationException("Internal error while initializing a queues client", e);
50 }
51 }
52
53 @Override
54 public List<String> getListOfReplicators() throws KeeperException {
55 return super.getListOfReplicatorsZK();
56 }
57
58 @Override
59 public List<String> getLogsInQueue(String serverName, String queueId) throws KeeperException {
60 String znode = ZKUtil.joinZNode(this.queuesZNode, serverName);
61 znode = ZKUtil.joinZNode(znode, queueId);
62 List<String> result = null;
63 try {
64 result = ZKUtil.listChildrenNoWatch(this.zookeeper, znode);
65 } catch (KeeperException e) {
66 this.abortable.abort("Failed to get list of wals for queueId=" + queueId
67 + " and serverName=" + serverName, e);
68 throw e;
69 }
70 return result;
71 }
72
73 @Override
74 public List<String> getAllQueues(String serverName) throws KeeperException {
75 String znode = ZKUtil.joinZNode(this.queuesZNode, serverName);
76 List<String> result = null;
77 try {
78 result = ZKUtil.listChildrenNoWatch(this.zookeeper, znode);
79 } catch (KeeperException e) {
80 this.abortable.abort("Failed to get list of queues for serverName=" + serverName, e);
81 throw e;
82 }
83 return result;
84 }
85
86 @Override public int getQueuesZNodeCversion() throws KeeperException {
87 try {
88 Stat stat = new Stat();
89 ZKUtil.getDataNoWatch(this.zookeeper, this.queuesZNode, stat);
90 return stat.getCversion();
91 } catch (KeeperException e) {
92 this.abortable.abort("Failed to get stat of replication rs node", e);
93 throw e;
94 }
95 }
96
97 @Override public Map<String, Integer> getReplicatorsZNodeCversion()
98 throws KeeperException {
99 List<String> rss = super.getListOfReplicatorsZK();
100 Map<String, Integer> rsToCversion = new HashMap<>();
101 for (String rs : rss) {
102 Stat stat = new Stat();
103 ZKUtil.getDataNoWatch(this.zookeeper, ZKUtil.joinZNode(this.queuesZNode, rs), stat);
104 rsToCversion.put(rs, stat.getCversion());
105 }
106 return rsToCversion;
107 }
108
109 @Override
110 public int getHFileRefsNodeChangeVersion() throws KeeperException {
111 Stat stat = new Stat();
112 try {
113 ZKUtil.getDataNoWatch(this.zookeeper, this.hfileRefsZNode, stat);
114 } catch (KeeperException e) {
115 this.abortable.abort("Failed to get stat of replication hfile references node.", e);
116 throw e;
117 }
118 return stat.getCversion();
119 }
120
121 @Override
122 public List<String> getAllPeersFromHFileRefsQueue() throws KeeperException {
123 List<String> result = null;
124 try {
125 result = ZKUtil.listChildrenNoWatch(this.zookeeper, this.hfileRefsZNode);
126 } catch (KeeperException e) {
127 this.abortable.abort("Failed to get list of all peers in hfile references node.", e);
128 throw e;
129 }
130 return result;
131 }
132
133 @Override
134 public List<String> getReplicableHFiles(String peerId) throws KeeperException {
135 String znode = ZKUtil.joinZNode(this.hfileRefsZNode, peerId);
136 List<String> result = null;
137 try {
138 result = ZKUtil.listChildrenNoWatch(this.zookeeper, znode);
139 } catch (KeeperException e) {
140 this.abortable.abort("Failed to get list of hfile references for peerId=" + peerId, e);
141 throw e;
142 }
143 return result;
144 }
145 }