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.io.Closeable;
22 import java.io.IOException;
23 import java.util.HashMap;
24 import java.util.List;
25 import java.util.Map;
26
27 import org.apache.commons.logging.Log;
28 import org.apache.commons.logging.LogFactory;
29 import org.apache.hadoop.hbase.classification.InterfaceAudience;
30 import org.apache.hadoop.conf.Configuration;
31 import org.apache.hadoop.hbase.Abortable;
32 import org.apache.hadoop.hbase.TableName;
33 import org.apache.hadoop.hbase.client.replication.ReplicationAdmin;
34 import org.apache.hadoop.hbase.exceptions.DeserializationException;
35 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
36 import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
37 import org.apache.hadoop.hbase.util.Bytes;
38 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
39 import org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker;
40 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
41 import org.apache.zookeeper.KeeperException;
42 import org.apache.zookeeper.KeeperException.NodeExistsException;
43
44 import com.google.protobuf.InvalidProtocolBufferException;
45
46 @InterfaceAudience.Private
47 public class ReplicationPeerZKImpl implements ReplicationPeer, Abortable, Closeable {
48 private static final Log LOG = LogFactory.getLog(ReplicationPeerZKImpl.class);
49
50 private final ReplicationPeerConfig peerConfig;
51 private final String id;
52 private volatile PeerState peerState;
53 private volatile Map<TableName, List<String>> tableCFs = new HashMap<TableName, List<String>>();
54 private final Configuration conf;
55
56 private PeerStateTracker peerStateTracker;
57 private TableCFsTracker tableCFsTracker;
58
59
60
61
62
63
64
65
66 public ReplicationPeerZKImpl(Configuration conf, String id, ReplicationPeerConfig peerConfig)
67 throws ReplicationException {
68 this.conf = conf;
69 this.peerConfig = peerConfig;
70 this.id = id;
71 }
72
73
74
75
76
77
78
79
80 public void startStateTracker(ZooKeeperWatcher zookeeper, String peerStateNode)
81 throws KeeperException {
82 ensurePeerEnabled(zookeeper, peerStateNode);
83 this.peerStateTracker = new PeerStateTracker(peerStateNode, zookeeper, this);
84 this.peerStateTracker.start();
85 try {
86 this.readPeerStateZnode();
87 } catch (DeserializationException e) {
88 throw ZKUtil.convert(e);
89 }
90 }
91
92 private void readPeerStateZnode() throws DeserializationException {
93 this.peerState =
94 isStateEnabled(this.peerStateTracker.getData(false))
95 ? PeerState.ENABLED
96 : PeerState.DISABLED;
97 }
98
99
100
101
102
103
104
105
106 public void startTableCFsTracker(ZooKeeperWatcher zookeeper, String tableCFsNode)
107 throws KeeperException {
108 this.tableCFsTracker = new TableCFsTracker(tableCFsNode, zookeeper,
109 this);
110 this.tableCFsTracker.start();
111 this.readTableCFsZnode();
112 }
113
114 private void readTableCFsZnode() {
115 String currentTableCFs = Bytes.toString(tableCFsTracker.getData(false));
116 this.tableCFs = ReplicationAdmin.parseTableCFsFromConfig(currentTableCFs);
117 }
118
119 @Override
120 public PeerState getPeerState() {
121 return peerState;
122 }
123
124
125
126
127
128 @Override
129 public String getId() {
130 return id;
131 }
132
133
134
135
136
137 @Override
138 public ReplicationPeerConfig getPeerConfig() {
139 return peerConfig;
140 }
141
142
143
144
145
146 @Override
147 public Configuration getConfiguration() {
148 return conf;
149 }
150
151
152
153
154
155 @Override
156 public Map<TableName, List<String>> getTableCFs() {
157 return this.tableCFs;
158 }
159
160 @Override
161 public void abort(String why, Throwable e) {
162 LOG.fatal("The ReplicationPeer coresponding to peer " + peerConfig
163 + " was aborted for the following reason(s):" + why, e);
164 }
165
166 @Override
167 public boolean isAborted() {
168
169
170 return false;
171 }
172
173 @Override
174 public void close() throws IOException {
175
176 }
177
178
179
180
181
182
183
184 public static boolean isStateEnabled(final byte[] bytes) throws DeserializationException {
185 ZooKeeperProtos.ReplicationState.State state = parseStateFrom(bytes);
186 return ZooKeeperProtos.ReplicationState.State.ENABLED == state;
187 }
188
189
190
191
192
193
194 private static ZooKeeperProtos.ReplicationState.State parseStateFrom(final byte[] bytes)
195 throws DeserializationException {
196 ProtobufUtil.expectPBMagicPrefix(bytes);
197 int pblen = ProtobufUtil.lengthOfPBMagic();
198 ZooKeeperProtos.ReplicationState.Builder builder =
199 ZooKeeperProtos.ReplicationState.newBuilder();
200 ZooKeeperProtos.ReplicationState state;
201 try {
202 state = builder.mergeFrom(bytes, pblen, bytes.length - pblen).build();
203 return state.getState();
204 } catch (InvalidProtocolBufferException e) {
205 throw new DeserializationException(e);
206 }
207 }
208
209
210
211
212
213
214
215
216
217 private static boolean ensurePeerEnabled(final ZooKeeperWatcher zookeeper, final String path)
218 throws NodeExistsException, KeeperException {
219 if (ZKUtil.checkExists(zookeeper, path) == -1) {
220
221
222
223 ZKUtil.createNodeIfNotExistsAndWatch(zookeeper, path,
224 ReplicationStateZKBase.ENABLED_ZNODE_BYTES);
225 return true;
226 }
227 return false;
228 }
229
230
231
232
233 public class PeerStateTracker extends ZooKeeperNodeTracker {
234
235 public PeerStateTracker(String peerStateZNode, ZooKeeperWatcher watcher,
236 Abortable abortable) {
237 super(watcher, peerStateZNode, abortable);
238 }
239
240 @Override
241 public synchronized void nodeDataChanged(String path) {
242 if (path.equals(node)) {
243 super.nodeDataChanged(path);
244 try {
245 readPeerStateZnode();
246 } catch (DeserializationException e) {
247 LOG.warn("Failed deserializing the content of " + path, e);
248 }
249 }
250 }
251 }
252
253
254
255
256 public class TableCFsTracker extends ZooKeeperNodeTracker {
257
258 public TableCFsTracker(String tableCFsZNode, ZooKeeperWatcher watcher,
259 Abortable abortable) {
260 super(watcher, tableCFsZNode, abortable);
261 }
262
263 @Override
264 public synchronized void nodeCreated(String path) {
265 if (path.equals(node)) {
266 super.nodeCreated(path);
267 readTableCFsZnode();
268 }
269 }
270
271 @Override
272 public synchronized void nodeDataChanged(String path) {
273 if (path.equals(node)) {
274 super.nodeDataChanged(path);
275 }
276 }
277 }
278 }