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.ArrayList;
24 import java.util.HashMap;
25 import java.util.List;
26 import java.util.Map;
27
28 import org.apache.commons.logging.Log;
29 import org.apache.commons.logging.LogFactory;
30 import org.apache.hadoop.hbase.classification.InterfaceAudience;
31 import org.apache.hadoop.conf.Configuration;
32 import org.apache.hadoop.hbase.Abortable;
33 import org.apache.hadoop.hbase.exceptions.DeserializationException;
34 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
35 import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos;
36 import org.apache.hadoop.hbase.util.Bytes;
37 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
38 import org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker;
39 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
40 import org.apache.zookeeper.KeeperException;
41 import org.apache.zookeeper.KeeperException.NodeExistsException;
42
43 @InterfaceAudience.Private
44 public class ReplicationPeerZKImpl implements ReplicationPeer, Abortable, Closeable {
45 private static final Log LOG = LogFactory.getLog(ReplicationPeerZKImpl.class);
46
47 private ReplicationPeerConfig peerConfig;
48 private final String id;
49 private volatile PeerState peerState;
50 private volatile Map<String, List<String>> tableCFs = new HashMap<String, List<String>>();
51 private final Configuration conf;
52
53 private PeerStateTracker peerStateTracker;
54 private TableCFsTracker tableCFsTracker;
55 private PeerConfigTracker peerConfigTracker;
56
57
58
59
60
61
62
63 public ReplicationPeerZKImpl(Configuration conf, String id, ReplicationPeerConfig peerConfig)
64 throws ReplicationException {
65 this.conf = conf;
66 this.peerConfig = peerConfig;
67 this.id = id;
68 }
69
70
71
72
73
74
75
76
77 public void startStateTracker(ZooKeeperWatcher zookeeper, String peerStateNode)
78 throws KeeperException {
79 ensurePeerEnabled(zookeeper, peerStateNode);
80 this.peerStateTracker = new PeerStateTracker(peerStateNode, zookeeper, this);
81 this.peerStateTracker.start();
82 try {
83 this.readPeerStateZnode();
84 } catch (DeserializationException e) {
85 throw ZKUtil.convert(e);
86 }
87 }
88
89 private void readPeerStateZnode() throws DeserializationException {
90 this.peerState =
91 isStateEnabled(this.peerStateTracker.getData(false))
92 ? PeerState.ENABLED
93 : PeerState.DISABLED;
94 }
95
96
97
98
99
100
101
102
103 public void startTableCFsTracker(ZooKeeperWatcher zookeeper, String tableCFsNode)
104 throws KeeperException {
105 this.tableCFsTracker = new TableCFsTracker(tableCFsNode, zookeeper,
106 this);
107 this.tableCFsTracker.start();
108 this.readTableCFsZnode();
109 }
110
111 static Map<String, List<String>> parseTableCFsFromConfig(String tableCFsConfig) {
112 if (tableCFsConfig == null || tableCFsConfig.trim().length() == 0) {
113 return null;
114 }
115
116 Map<String, List<String>> tableCFsMap = null;
117
118
119
120 String[] tables = tableCFsConfig.split(";");
121 for (String tab : tables) {
122
123 tab = tab.trim();
124 if (tab.length() == 0) {
125 continue;
126 }
127
128
129 String[] pair = tab.split(":");
130 String tabName = pair[0].trim();
131 if (pair.length > 2 || tabName.length() == 0) {
132 LOG.error("ignore invalid tableCFs setting: " + tab);
133 continue;
134 }
135
136
137 List<String> cfs = null;
138 if (pair.length == 2) {
139 String[] cfsList = pair[1].split(",");
140 for (String cf : cfsList) {
141 String cfName = cf.trim();
142 if (cfName.length() > 0) {
143 if (cfs == null) {
144 cfs = new ArrayList<String>();
145 }
146 cfs.add(cfName);
147 }
148 }
149 }
150
151
152 if (tableCFsMap == null) {
153 tableCFsMap = new HashMap<String, List<String>>();
154 }
155 tableCFsMap.put(tabName, cfs);
156 }
157
158 return tableCFsMap;
159 }
160
161
162
163
164
165
166 public void startPeerConfigTracker(ZooKeeperWatcher zookeeper, String peerConfigNode)
167 throws KeeperException {
168 this.peerConfigTracker = new PeerConfigTracker(peerConfigNode, zookeeper,
169 this);
170 this.peerConfigTracker.start();
171 this.readPeerConfig();
172 }
173
174 private void readTableCFsZnode() {
175 String currentTableCFs = Bytes.toString(tableCFsTracker.getData(false));
176 this.tableCFs = parseTableCFsFromConfig(currentTableCFs);
177 }
178
179 private ReplicationPeerConfig readPeerConfig() {
180 try {
181 byte[] data = peerConfigTracker.getData(false);
182 if (data != null) {
183 this.peerConfig = ReplicationSerDeHelper.parsePeerFrom(data);
184 }
185 } catch (DeserializationException e) {
186 LOG.error("", e);
187 }
188 return this.peerConfig;
189 }
190 @Override
191 public PeerState getPeerState() {
192 return peerState;
193 }
194
195
196
197
198
199 @Override
200 public String getId() {
201 return id;
202 }
203
204
205
206
207
208 @Override
209 public ReplicationPeerConfig getPeerConfig() {
210 return peerConfig;
211 }
212
213
214
215
216
217 @Override
218 public Configuration getConfiguration() {
219 return conf;
220 }
221
222
223
224
225
226 @Override
227 public Map<String, List<String>> getTableCFs() {
228 return this.tableCFs;
229 }
230
231 @Override
232 public void trackPeerConfigChanges(ReplicationPeerConfigListener listener) {
233 if (this.peerConfigTracker != null){
234 this.peerConfigTracker.setListener(listener);
235 }
236 }
237
238 @Override
239 public void abort(String why, Throwable e) {
240 LOG.fatal("The ReplicationPeer coresponding to peer " + peerConfig
241 + " was aborted for the following reason(s):" + why, e);
242 }
243
244 @Override
245 public boolean isAborted() {
246
247
248 return false;
249 }
250
251 @Override
252 public void close() throws IOException {
253
254 }
255
256
257
258
259
260
261
262 public static boolean isStateEnabled(final byte[] bytes) throws DeserializationException {
263 ZooKeeperProtos.ReplicationState.State state = parseStateFrom(bytes);
264 return ZooKeeperProtos.ReplicationState.State.ENABLED == state;
265 }
266
267
268
269
270
271
272 private static ZooKeeperProtos.ReplicationState.State parseStateFrom(final byte[] bytes)
273 throws DeserializationException {
274 ProtobufUtil.expectPBMagicPrefix(bytes);
275 int pblen = ProtobufUtil.lengthOfPBMagic();
276 ZooKeeperProtos.ReplicationState.Builder builder =
277 ZooKeeperProtos.ReplicationState.newBuilder();
278 ZooKeeperProtos.ReplicationState state;
279 try {
280 ProtobufUtil.mergeFrom(builder, bytes, pblen, bytes.length - pblen);
281 state = builder.build();
282 return state.getState();
283 } catch (IOException e) {
284 throw new DeserializationException(e);
285 }
286 }
287
288
289
290
291
292
293
294
295
296 private static boolean ensurePeerEnabled(final ZooKeeperWatcher zookeeper, final String path)
297 throws NodeExistsException, KeeperException {
298 if (ZKUtil.checkExists(zookeeper, path) == -1) {
299
300
301
302 ZKUtil.createNodeIfNotExistsAndWatch(zookeeper, path,
303 ReplicationStateZKBase.ENABLED_ZNODE_BYTES);
304 return true;
305 }
306 return false;
307 }
308
309
310
311
312 public class PeerStateTracker extends ZooKeeperNodeTracker {
313
314 public PeerStateTracker(String peerStateZNode, ZooKeeperWatcher watcher,
315 Abortable abortable) {
316 super(watcher, peerStateZNode, abortable);
317 }
318
319 @Override
320 public synchronized void nodeDataChanged(String path) {
321 if (path.equals(node)) {
322 super.nodeDataChanged(path);
323 try {
324 readPeerStateZnode();
325 } catch (DeserializationException e) {
326 LOG.warn("Failed deserializing the content of " + path, e);
327 }
328 }
329 }
330 }
331
332
333
334
335 public class TableCFsTracker extends ZooKeeperNodeTracker {
336
337 public TableCFsTracker(String tableCFsZNode, ZooKeeperWatcher watcher,
338 Abortable abortable) {
339 super(watcher, tableCFsZNode, abortable);
340 }
341
342 @Override
343 public synchronized void nodeCreated(String path) {
344 if (path.equals(node)) {
345 super.nodeCreated(path);
346 readTableCFsZnode();
347 }
348 }
349
350 @Override
351 public synchronized void nodeDataChanged(String path) {
352 if (path.equals(node)) {
353 super.nodeDataChanged(path);
354 }
355 }
356 }
357
358
359
360
361 public class PeerConfigTracker extends ZooKeeperNodeTracker {
362
363 private ReplicationPeerConfigListener listener;
364
365 public PeerConfigTracker(String peerConfigNode, ZooKeeperWatcher watcher,
366 Abortable abortable) {
367 super(watcher, peerConfigNode, abortable);
368 }
369
370 public synchronized void setListener(ReplicationPeerConfigListener listener){
371 this.listener = listener;
372 }
373
374 @Override
375 public synchronized void nodeCreated(String path) {
376 if (path.equals(node)) {
377 super.nodeCreated(path);
378 ReplicationPeerConfig config = readPeerConfig();
379 if (listener != null){
380 listener.peerConfigUpdated(config);
381 }
382 }
383 }
384
385 @Override
386 public synchronized void nodeDataChanged(String path) {
387
388 if (path.equals(node)) {
389 super.nodeDataChanged(path);
390 }
391 }
392 }
393 }