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