1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21 package org.apache.hadoop.hbase.replication.regionserver;
22
23 import org.apache.commons.logging.Log;
24 import org.apache.commons.logging.LogFactory;
25 import org.apache.hadoop.conf.Configuration;
26 import org.apache.hadoop.fs.FileSystem;
27 import org.apache.hadoop.fs.Path;
28 import org.apache.hadoop.hbase.regionserver.wal.LogActionsListener;
29 import org.apache.hadoop.hbase.replication.ReplicationZookeeperWrapper;
30 import org.apache.zookeeper.WatchedEvent;
31 import org.apache.zookeeper.Watcher;
32
33 import java.io.IOException;
34 import java.util.ArrayList;
35 import java.util.List;
36 import java.util.Map;
37 import java.util.SortedMap;
38 import java.util.SortedSet;
39 import java.util.TreeSet;
40 import java.util.concurrent.atomic.AtomicBoolean;
41
42
43
44
45
46
47
48
49
50
51
52
53 public class ReplicationSourceManager implements LogActionsListener {
54
55 private static final Log LOG =
56 LogFactory.getLog(ReplicationSourceManager.class);
57
58 private final List<ReplicationSourceInterface> sources;
59
60 private final List<ReplicationSourceInterface> oldsources;
61
62 private final AtomicBoolean replicating;
63
64 private final ReplicationZookeeperWrapper zkHelper;
65
66 private final AtomicBoolean stopper;
67
68 private final SortedSet<String> hlogs;
69 private final Configuration conf;
70 private final FileSystem fs;
71
72 private Path latestPath;
73
74 private final List<String> otherRegionServers;
75
76 private final Path logDir;
77
78 private final Path oldLogDir;
79
80
81
82
83
84
85
86
87
88
89
90
91 public ReplicationSourceManager(final ReplicationZookeeperWrapper zkHelper,
92 final Configuration conf,
93 final AtomicBoolean stopper,
94 final FileSystem fs,
95 final AtomicBoolean replicating,
96 final Path logDir,
97 final Path oldLogDir) {
98 this.sources = new ArrayList<ReplicationSourceInterface>();
99 this.replicating = replicating;
100 this.zkHelper = zkHelper;
101 this.stopper = stopper;
102 this.hlogs = new TreeSet<String>();
103 this.oldsources = new ArrayList<ReplicationSourceInterface>();
104 this.conf = conf;
105 this.fs = fs;
106 this.logDir = logDir;
107 this.oldLogDir = oldLogDir;
108 List<String> otherRSs =
109 this.zkHelper.getRegisteredRegionServers(new OtherRegionServerWatcher());
110 this.otherRegionServers = otherRSs == null ? new ArrayList<String>() : otherRSs;
111 }
112
113
114
115
116
117
118
119
120
121
122
123 public void logPositionAndCleanOldLogs(Path log, String id, long position, boolean queueRecovered) {
124 String key = log.getName();
125 LOG.info("Going to report log #" + key + " for position " + position + " in " + log);
126 this.zkHelper.writeReplicationStatus(key.toString(), id, position);
127 synchronized (this.hlogs) {
128 if (!queueRecovered && this.hlogs.first() != key) {
129 SortedSet<String> hlogSet = this.hlogs.headSet(key);
130 LOG.info("Removing " + hlogSet.size() +
131 " logs in the list: " + hlogSet);
132 for (String hlog : hlogSet) {
133 this.zkHelper.removeLogFromList(hlog.toString(), id);
134 }
135 hlogSet.clear();
136 }
137 }
138 }
139
140
141
142
143
144 public void init() throws IOException {
145 for (String id : this.zkHelper.getPeerClusters().keySet()) {
146 ReplicationSourceInterface src = addSource(id);
147 src.startup();
148 }
149 List<String> currentReplicators = this.zkHelper.getListOfReplicators(null);
150 synchronized (otherRegionServers) {
151 LOG.info("Current list of replicators: " + currentReplicators
152 + " other RSs: " + otherRegionServers);
153 }
154
155 for (String rs : currentReplicators) {
156 synchronized (otherRegionServers) {
157 if (!this.otherRegionServers.contains(rs)) {
158 transferQueues(rs);
159 }
160 }
161 }
162 }
163
164
165
166
167
168
169
170 public ReplicationSourceInterface addSource(String id) throws IOException {
171 ReplicationSourceInterface src =
172 getReplicationSource(this.conf, this.fs, this, stopper, replicating, id);
173 this.sources.add(src);
174 synchronized (this.hlogs) {
175 if (this.hlogs.size() > 0) {
176 this.zkHelper.addLogToList(this.hlogs.first(),
177 this.sources.get(0).getPeerClusterZnode());
178 src.enqueueLog(this.latestPath);
179 }
180 }
181 return src;
182 }
183
184
185
186
187 public void join() {
188 if (this.sources.size() == 0) {
189 this.zkHelper.deleteOwnRSZNode();
190 }
191 for (ReplicationSourceInterface source : this.sources) {
192 source.terminate();
193 }
194 }
195
196
197
198
199
200 protected SortedSet<String> getHLogs() {
201 return new TreeSet(this.hlogs);
202 }
203
204
205
206
207
208 public List<ReplicationSourceInterface> getSources() {
209 return this.sources;
210 }
211
212 @Override
213 public void logRolled(Path newLog) {
214 if (this.sources.size() > 0) {
215 this.zkHelper.addLogToList(newLog.getName(),
216 this.sources.get(0).getPeerClusterZnode());
217 }
218 synchronized (this.hlogs) {
219 this.hlogs.add(newLog.getName());
220 }
221 this.latestPath = newLog;
222
223 for (ReplicationSourceInterface source : this.sources) {
224 source.enqueueLog(newLog);
225 }
226 }
227
228
229
230
231
232 public ReplicationZookeeperWrapper getRepZkWrapper() {
233 return zkHelper;
234 }
235
236
237
238
239
240
241
242
243
244
245
246
247 public ReplicationSourceInterface getReplicationSource(
248 final Configuration conf,
249 final FileSystem fs,
250 final ReplicationSourceManager manager,
251 final AtomicBoolean stopper,
252 final AtomicBoolean replicating,
253 final String peerClusterId) throws IOException {
254 ReplicationSourceInterface src;
255 try {
256 Class c = Class.forName(conf.get("replication.replicationsource.implementation",
257 ReplicationSource.class.getCanonicalName()));
258 src = (ReplicationSourceInterface) c.newInstance();
259 } catch (Exception e) {
260 LOG.warn("Passed replication source implemention throws errors, " +
261 "defaulting to ReplicationSource", e);
262 src = new ReplicationSource();
263
264 }
265 src.init(conf, fs, manager, stopper, replicating, peerClusterId);
266 return src;
267 }
268
269
270
271
272
273
274
275
276
277 public void transferQueues(String rsZnode) {
278
279 if (this.stopper.get()) {
280 LOG.info("Not transferring queue since we are shutting down");
281 return;
282 }
283 if (!this.zkHelper.lockOtherRS(rsZnode)) {
284 return;
285 }
286 LOG.info("Moving " + rsZnode + "'s hlogs to my queue");
287 SortedMap<String, SortedSet<String>> newQueues =
288 this.zkHelper.copyQueuesFromRS(rsZnode);
289 if (newQueues == null || newQueues.size() == 0) {
290 return;
291 }
292 this.zkHelper.deleteRsQueues(rsZnode);
293
294 for (Map.Entry<String, SortedSet<String>> entry : newQueues.entrySet()) {
295 String peerId = entry.getKey();
296 try {
297 ReplicationSourceInterface src = getReplicationSource(this.conf,
298 this.fs, this, this.stopper, this.replicating, peerId);
299 this.oldsources.add(src);
300 for (String hlog : entry.getValue()) {
301 src.enqueueLog(new Path(this.oldLogDir, hlog));
302 }
303 src.startup();
304 } catch (IOException e) {
305
306 LOG.error("Failed creating a source", e);
307 }
308 }
309 }
310
311
312
313
314
315 public void closeRecoveredQueue(ReplicationSourceInterface src) {
316 LOG.info("Done with the recovered queue " + src.getPeerClusterZnode());
317 this.oldsources.remove(src);
318 this.zkHelper.deleteSource(src.getPeerClusterZnode());
319 }
320
321
322
323
324
325
326 public class OtherRegionServerWatcher implements Watcher {
327 @Override
328 public void process(WatchedEvent watchedEvent) {
329 LOG.info(" event " + watchedEvent);
330 if (watchedEvent.getType().equals(Event.KeeperState.Expired) ||
331 watchedEvent.getType().equals(Event.KeeperState.Disconnected)) {
332 return;
333 }
334
335 List<String> newRsList = (zkHelper.getRegisteredRegionServers(this));
336 if (newRsList == null) {
337 return;
338 } else {
339 synchronized (otherRegionServers) {
340 otherRegionServers.clear();
341 otherRegionServers.addAll(newRsList);
342 }
343 }
344 if (watchedEvent.getType().equals(Event.EventType.NodeDeleted)) {
345 LOG.info(watchedEvent.getPath() + " znode expired, trying to lock it");
346 String[] rsZnodeParts = watchedEvent.getPath().split("/");
347 transferQueues(rsZnodeParts[rsZnodeParts.length-1]);
348 }
349 }
350 }
351
352
353
354
355
356 public Path getOldLogDir() {
357 return this.oldLogDir;
358 }
359
360
361
362
363
364 public Path getLogDir() {
365 return this.logDir;
366 }
367
368
369
370
371
372 public FileSystem getFs() {
373 return this.fs;
374 }
375
376 }