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.conf.Configuration;
24 import org.apache.hadoop.hbase.Abortable;
25 import org.apache.hadoop.hbase.classification.InterfaceAudience;
26 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
27 import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
28 import org.apache.hadoop.hbase.zookeeper.ZKConfig;
29 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
30 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
31 import org.apache.zookeeper.KeeperException;
32
33
34
35
36 @InterfaceAudience.Private
37 public abstract class ReplicationStateZKBase {
38
39
40
41
42
43 protected final String peerStateNodeName;
44
45 protected final String replicationZNode;
46
47 protected final String peersZNode;
48
49 protected final String queuesZNode;
50
51 protected final String hfileRefsZNode;
52
53 protected final String ourClusterKey;
54
55 protected final String tableCFsNodeName;
56
57 protected final ZooKeeperWatcher zookeeper;
58 protected final Configuration conf;
59 protected final Abortable abortable;
60
61
62 public static final byte[] ENABLED_ZNODE_BYTES =
63 toByteArray(ZooKeeperProtos.ReplicationState.State.ENABLED);
64 public static final byte[] DISABLED_ZNODE_BYTES =
65 toByteArray(ZooKeeperProtos.ReplicationState.State.DISABLED);
66 public static final String ZOOKEEPER_ZNODE_REPLICATION_HFILE_REFS_KEY =
67 "zookeeper.znode.replication.hfile.refs";
68 public static final String ZOOKEEPER_ZNODE_REPLICATION_HFILE_REFS_DEFAULT = "hfile-refs";
69
70 public ReplicationStateZKBase(ZooKeeperWatcher zookeeper, Configuration conf,
71 Abortable abortable) {
72 this.zookeeper = zookeeper;
73 this.conf = conf;
74 this.abortable = abortable;
75
76 String replicationZNodeName = conf.get("zookeeper.znode.replication", "replication");
77 String peersZNodeName = conf.get("zookeeper.znode.replication.peers", "peers");
78 String queuesZNodeName = conf.get("zookeeper.znode.replication.rs", "rs");
79 String hfileRefsZNodeName = conf.get(ZOOKEEPER_ZNODE_REPLICATION_HFILE_REFS_KEY,
80 ZOOKEEPER_ZNODE_REPLICATION_HFILE_REFS_DEFAULT);
81 this.peerStateNodeName = conf.get("zookeeper.znode.replication.peers.state", "peer-state");
82 this.tableCFsNodeName = conf.get("zookeeper.znode.replication.peers.tableCFs", "tableCFs");
83 this.ourClusterKey = ZKConfig.getZooKeeperClusterKey(this.conf);
84 this.replicationZNode = ZKUtil.joinZNode(this.zookeeper.baseZNode, replicationZNodeName);
85 this.peersZNode = ZKUtil.joinZNode(replicationZNode, peersZNodeName);
86 this.queuesZNode = ZKUtil.joinZNode(replicationZNode, queuesZNodeName);
87 this.hfileRefsZNode = ZKUtil.joinZNode(replicationZNode, hfileRefsZNodeName);
88 }
89
90
91
92
93
94
95 protected List<String> getListOfReplicatorsZK() throws KeeperException {
96 List<String> result = null;
97 try {
98 result = ZKUtil.listChildrenNoWatch(this.zookeeper, this.queuesZNode);
99 } catch (KeeperException e) {
100 this.abortable.abort("Failed to get list of replicators", e);
101 throw e;
102 }
103 return result;
104 }
105
106
107
108
109
110
111
112 protected static byte[] toByteArray(final ZooKeeperProtos.ReplicationState.State state) {
113 byte[] bytes =
114 ZooKeeperProtos.ReplicationState.newBuilder().setState(state).build().toByteArray();
115 return ProtobufUtil.prependPBMagic(bytes);
116 }
117
118 protected boolean peerExists(String id) throws KeeperException {
119 return ZKUtil.checkExists(this.zookeeper, ZKUtil.joinZNode(this.peersZNode, id)) >= 0;
120 }
121
122
123
124
125
126
127 protected boolean isPeerPath(String path) {
128 return path.split("/").length == peersZNode.split("/").length + 1;
129 }
130
131 protected String getTableCFsNode(String id) {
132 return ZKUtil.joinZNode(this.peersZNode, ZKUtil.joinZNode(id, this.tableCFsNodeName));
133 }
134
135 protected String getPeerStateNode(String id) {
136 return ZKUtil.joinZNode(this.peersZNode, ZKUtil.joinZNode(id, this.peerStateNodeName));
137 }
138
139 protected String getPeerNode(String id) {
140 return ZKUtil.joinZNode(this.peersZNode, id);
141 }
142 }