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 final 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
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 private void readTableCFsZnode() {
162 String currentTableCFs = Bytes.toString(tableCFsTracker.getData(false));
163 this.tableCFs = parseTableCFsFromConfig(currentTableCFs);
164 }
165
166 @Override
167 public PeerState getPeerState() {
168 return peerState;
169 }
170
171
172
173
174
175 @Override
176 public String getId() {
177 return id;
178 }
179
180
181
182
183
184 @Override
185 public ReplicationPeerConfig getPeerConfig() {
186 return peerConfig;
187 }
188
189
190
191
192
193 @Override
194 public Configuration getConfiguration() {
195 return conf;
196 }
197
198
199
200
201
202 @Override
203 public Map<String, List<String>> getTableCFs() {
204 return this.tableCFs;
205 }
206
207 @Override
208 public void abort(String why, Throwable e) {
209 LOG.fatal("The ReplicationPeer coresponding to peer " + peerConfig
210 + " was aborted for the following reason(s):" + why, e);
211 }
212
213 @Override
214 public boolean isAborted() {
215
216
217 return false;
218 }
219
220 @Override
221 public void close() throws IOException {
222
223 }
224
225
226
227
228
229
230
231 public static boolean isStateEnabled(final byte[] bytes) throws DeserializationException {
232 ZooKeeperProtos.ReplicationState.State state = parseStateFrom(bytes);
233 return ZooKeeperProtos.ReplicationState.State.ENABLED == state;
234 }
235
236
237
238
239
240
241 private static ZooKeeperProtos.ReplicationState.State parseStateFrom(final byte[] bytes)
242 throws DeserializationException {
243 ProtobufUtil.expectPBMagicPrefix(bytes);
244 int pblen = ProtobufUtil.lengthOfPBMagic();
245 ZooKeeperProtos.ReplicationState.Builder builder =
246 ZooKeeperProtos.ReplicationState.newBuilder();
247 ZooKeeperProtos.ReplicationState state;
248 try {
249 ProtobufUtil.mergeFrom(builder, bytes, pblen, bytes.length - pblen);
250 state = builder.build();
251 return state.getState();
252 } catch (IOException e) {
253 throw new DeserializationException(e);
254 }
255 }
256
257
258
259
260
261
262
263
264
265 private static boolean ensurePeerEnabled(final ZooKeeperWatcher zookeeper, final String path)
266 throws NodeExistsException, KeeperException {
267 if (ZKUtil.checkExists(zookeeper, path) == -1) {
268
269
270
271 ZKUtil.createNodeIfNotExistsAndWatch(zookeeper, path,
272 ReplicationStateZKBase.ENABLED_ZNODE_BYTES);
273 return true;
274 }
275 return false;
276 }
277
278
279
280
281 public class PeerStateTracker extends ZooKeeperNodeTracker {
282
283 public PeerStateTracker(String peerStateZNode, ZooKeeperWatcher watcher,
284 Abortable abortable) {
285 super(watcher, peerStateZNode, abortable);
286 }
287
288 @Override
289 public synchronized void nodeDataChanged(String path) {
290 if (path.equals(node)) {
291 super.nodeDataChanged(path);
292 try {
293 readPeerStateZnode();
294 } catch (DeserializationException e) {
295 LOG.warn("Failed deserializing the content of " + path, e);
296 }
297 }
298 }
299 }
300
301
302
303
304 public class TableCFsTracker extends ZooKeeperNodeTracker {
305
306 public TableCFsTracker(String tableCFsZNode, ZooKeeperWatcher watcher,
307 Abortable abortable) {
308 super(watcher, tableCFsZNode, abortable);
309 }
310
311 @Override
312 public synchronized void nodeCreated(String path) {
313 if (path.equals(node)) {
314 super.nodeCreated(path);
315 readTableCFsZnode();
316 }
317 }
318
319 @Override
320 public synchronized void nodeDataChanged(String path) {
321 if (path.equals(node)) {
322 super.nodeDataChanged(path);
323 }
324 }
325 }
326 }