1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20 package org.apache.hadoop.hbase.replication;
21
22 import java.io.IOException;
23 import java.util.ArrayList;
24 import java.util.List;
25 import java.util.concurrent.atomic.AtomicBoolean;
26
27 import org.apache.commons.logging.Log;
28 import org.apache.commons.logging.LogFactory;
29 import org.apache.hadoop.conf.Configuration;
30 import org.apache.hadoop.hbase.Abortable;
31 import org.apache.hadoop.hbase.ServerName;
32 import org.apache.hadoop.hbase.replication.ReplicationZookeeper.PeerState;
33 import org.apache.hadoop.hbase.util.Bytes;
34 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
35 import org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker;
36 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
37 import org.apache.zookeeper.KeeperException;
38
39
40
41
42
43
44 public class ReplicationPeer implements Abortable {
45 private static final Log LOG = LogFactory.getLog(ReplicationPeer.class);
46
47 private final String clusterKey;
48 private final String id;
49 private List<ServerName> regionServers = new ArrayList<ServerName>(0);
50 private final AtomicBoolean peerEnabled = new AtomicBoolean();
51
52 private ZooKeeperWatcher zkw;
53 private final Configuration conf;
54
55 private PeerStateTracker peerStateTracker;
56
57
58
59
60
61
62
63
64 public ReplicationPeer(Configuration conf, String key,
65 String id) throws IOException {
66 this.conf = conf;
67 this.clusterKey = key;
68 this.id = id;
69 this.reloadZkWatcher();
70 }
71
72
73
74
75
76
77
78
79 public void startStateTracker(ZooKeeperWatcher zookeeper, String peerStateNode)
80 throws KeeperException {
81 if (ZKUtil.checkExists(zookeeper, peerStateNode) == -1) {
82
83
84
85 ZKUtil.createNodeIfNotExistsAndWatch(zookeeper, peerStateNode,
86 Bytes.toBytes(PeerState.ENABLED.name()));
87 }
88 this.peerStateTracker = new PeerStateTracker(peerStateNode, zookeeper,
89 this);
90 this.peerStateTracker.start();
91 this.readPeerStateZnode();
92 }
93
94 private void readPeerStateZnode() {
95 String currentState = Bytes.toString(peerStateTracker.getData(false));
96 this.peerEnabled.set(PeerState.ENABLED.equals(PeerState
97 .valueOf(currentState)));
98 }
99
100
101
102
103
104
105 public String getClusterKey() {
106 return clusterKey;
107 }
108
109
110
111
112
113 public AtomicBoolean getPeerEnabled() {
114 return peerEnabled;
115 }
116
117
118
119
120
121
122 public List<ServerName> getRegionServers() {
123 return regionServers;
124 }
125
126
127
128
129
130 public void setRegionServers(List<ServerName> regionServers) {
131 this.regionServers = regionServers;
132 }
133
134
135
136
137
138 public ZooKeeperWatcher getZkw() {
139 return zkw;
140 }
141
142
143
144
145
146 public String getId() {
147 return id;
148 }
149
150
151
152
153
154 public Configuration getConfiguration() {
155 return conf;
156 }
157
158 @Override
159 public void abort(String why, Throwable e) {
160 LOG.fatal("The ReplicationPeer coresponding to peer " + clusterKey
161 + " was aborted for the following reason(s):" + why, e);
162 }
163
164
165
166
167
168 public void reloadZkWatcher() throws IOException {
169 if (zkw != null) zkw.close();
170 zkw = new ZooKeeperWatcher(conf,
171 "connection to cluster: " + id, this);
172 }
173
174 @Override
175 public boolean isAborted() {
176
177
178 return false;
179 }
180
181
182
183
184 public class PeerStateTracker extends ZooKeeperNodeTracker {
185
186 public PeerStateTracker(String peerStateZNode, ZooKeeperWatcher watcher,
187 Abortable abortable) {
188 super(watcher, peerStateZNode, abortable);
189 }
190
191 @Override
192 public synchronized void nodeDataChanged(String path) {
193 if (path.equals(node)) {
194 super.nodeDataChanged(path);
195 readPeerStateZnode();
196 }
197 }
198 }
199 }