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