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.IOException;
22 import java.util.ArrayList;
23 import java.util.Collections;
24 import java.util.List;
25 import java.util.UUID;
26
27 import org.apache.commons.logging.Log;
28 import org.apache.commons.logging.LogFactory;
29 import org.apache.hadoop.hbase.classification.InterfaceAudience;
30 import org.apache.hadoop.hbase.Abortable;
31 import org.apache.hadoop.hbase.ServerName;
32 import org.apache.hadoop.hbase.zookeeper.ZKClusterId;
33 import org.apache.hadoop.hbase.zookeeper.ZKUtil;
34 import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
35 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
36 import org.apache.zookeeper.KeeperException;
37 import org.apache.zookeeper.KeeperException.AuthFailedException;
38 import org.apache.zookeeper.KeeperException.ConnectionLossException;
39 import org.apache.zookeeper.KeeperException.SessionExpiredException;
40
41
42
43
44
45 @InterfaceAudience.Private
46 public abstract class HBaseReplicationEndpoint extends BaseReplicationEndpoint
47 implements Abortable {
48
49 private static final Log LOG = LogFactory.getLog(HBaseReplicationEndpoint.class);
50
51 private ZooKeeperWatcher zkw = null;
52
53 private List<ServerName> regionServers = new ArrayList<ServerName>(0);
54 private long lastRegionServerUpdate;
55
56 protected synchronized void disconnect() {
57 if (zkw != null) {
58 zkw.close();
59 }
60 }
61
62
63
64
65
66 protected void reconnect(KeeperException ke) {
67 if (ke instanceof ConnectionLossException || ke instanceof SessionExpiredException
68 || ke instanceof AuthFailedException) {
69 String clusterKey = ctx.getPeerConfig().getClusterKey();
70 LOG.warn("Lost the ZooKeeper connection for peer " + clusterKey, ke);
71 try {
72 reloadZkWatcher();
73 } catch (IOException io) {
74 LOG.warn("Creation of ZookeeperWatcher failed for peer " + clusterKey, io);
75 }
76 }
77 }
78
79 @Override
80 protected void doStart() {
81 try {
82 reloadZkWatcher();
83 notifyStarted();
84 } catch (IOException e) {
85 notifyFailed(e);
86 }
87 }
88
89 @Override
90 protected void doStop() {
91 disconnect();
92 close();
93 notifyStopped();
94 }
95
96 @Override
97
98
99
100
101 public synchronized UUID getPeerUUID() {
102 UUID peerUUID = null;
103 try {
104 peerUUID = ZKClusterId.getUUIDForCluster(zkw);
105 } catch (KeeperException ke) {
106 reconnect(ke);
107 }
108 return peerUUID;
109 }
110
111
112
113
114
115 protected synchronized ZooKeeperWatcher getZkw() {
116 return zkw;
117 }
118
119
120
121
122
123 synchronized void reloadZkWatcher() throws IOException {
124 if (zkw != null) zkw.close();
125 zkw = new ZooKeeperWatcher(ctx.getConfiguration(),
126 "connection to cluster: " + ctx.getPeerId(), this);
127 getZkw().registerListener(new PeerRegionServerListener(this));
128 }
129
130 @Override
131 public void abort(String why, Throwable e) {
132 LOG.error("The HBaseReplicationEndpoint corresponding to peer " + ctx.getPeerId()
133 + " was aborted for the following reason(s):" + why, e);
134 }
135
136 @Override
137 public boolean isAborted() {
138
139 return false;
140 }
141
142
143
144
145
146
147 protected static List<ServerName> fetchSlavesAddresses(ZooKeeperWatcher zkw)
148 throws KeeperException {
149 List<String> children = ZKUtil.listChildrenAndWatchForNewChildren(zkw, zkw.rsZNode);
150 if (children == null) {
151 return Collections.emptyList();
152 }
153 List<ServerName> addresses = new ArrayList<ServerName>(children.size());
154 for (String child : children) {
155 addresses.add(ServerName.parseServerName(child));
156 }
157 return addresses;
158 }
159
160
161
162
163
164
165
166
167
168
169 public synchronized List<ServerName> getRegionServers() {
170 try {
171 setRegionServers(fetchSlavesAddresses(this.getZkw()));
172 } catch (KeeperException ke) {
173 if (LOG.isDebugEnabled()) {
174 LOG.debug("Fetch slaves addresses failed", ke);
175 }
176 reconnect(ke);
177 }
178 return regionServers;
179 }
180
181
182
183
184
185 public synchronized void setRegionServers(List<ServerName> regionServers) {
186 this.regionServers = regionServers;
187 lastRegionServerUpdate = System.currentTimeMillis();
188 }
189
190
191
192
193
194
195 public long getLastRegionServerUpdate() {
196 return lastRegionServerUpdate;
197 }
198
199
200
201
202 public static class PeerRegionServerListener extends ZooKeeperListener {
203
204 private final HBaseReplicationEndpoint replicationEndpoint;
205 private final String regionServerListNode;
206
207 public PeerRegionServerListener(HBaseReplicationEndpoint replicationPeer) {
208 super(replicationPeer.getZkw());
209 this.replicationEndpoint = replicationPeer;
210 this.regionServerListNode = replicationEndpoint.getZkw().rsZNode;
211 }
212
213 @Override
214 public synchronized void nodeChildrenChanged(String path) {
215 if (path.equals(regionServerListNode)) {
216 try {
217 LOG.info("Detected change to peer region servers, fetching updated list");
218 replicationEndpoint.setRegionServers(fetchSlavesAddresses(replicationEndpoint.getZkw()));
219 } catch (KeeperException e) {
220 LOG.error("Error reading slave addresses", e);
221 }
222 }
223 }
224 }
225 }