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 java.io.IOException;
24 import java.util.ArrayList;
25 import java.util.Collections;
26 import java.util.HashMap;
27 import java.util.List;
28 import java.util.Map;
29 import java.util.SortedMap;
30 import java.util.SortedSet;
31 import java.util.TreeSet;
32 import java.util.concurrent.LinkedBlockingQueue;
33 import java.util.concurrent.RejectedExecutionException;
34 import java.util.concurrent.ThreadPoolExecutor;
35 import java.util.concurrent.TimeUnit;
36 import java.util.concurrent.atomic.AtomicBoolean;
37
38 import com.google.common.util.concurrent.ThreadFactoryBuilder;
39 import org.apache.commons.logging.Log;
40 import org.apache.commons.logging.LogFactory;
41 import org.apache.hadoop.conf.Configuration;
42 import org.apache.hadoop.fs.FileSystem;
43 import org.apache.hadoop.fs.Path;
44 import org.apache.hadoop.hbase.Stoppable;
45 import org.apache.hadoop.hbase.replication.ReplicationZookeeper;
46 import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
47 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
48 import org.apache.zookeeper.KeeperException;
49
50
51
52
53
54
55
56
57
58
59
60
61 public class ReplicationSourceManager {
62 private static final Log LOG =
63 LogFactory.getLog(ReplicationSourceManager.class);
64
65 private final List<ReplicationSourceInterface> sources;
66
67 private final List<ReplicationSourceInterface> oldsources;
68
69 private final AtomicBoolean replicating;
70
71 private final ReplicationZookeeper zkHelper;
72
73 private final Stoppable stopper;
74
75 private final Map<String, SortedSet<String>> hlogsById;
76 private final Configuration conf;
77 private final FileSystem fs;
78
79 private Path latestPath;
80
81 private final List<String> otherRegionServers;
82
83 private final Path logDir;
84
85 private final Path oldLogDir;
86
87 private final long sleepBeforeFailover;
88
89 private final ThreadPoolExecutor executor;
90
91
92
93
94
95
96
97
98
99
100
101
102 public ReplicationSourceManager(final ReplicationZookeeper zkHelper,
103 final Configuration conf,
104 final Stoppable stopper,
105 final FileSystem fs,
106 final AtomicBoolean replicating,
107 final Path logDir,
108 final Path oldLogDir) {
109 this.sources = new ArrayList<ReplicationSourceInterface>();
110 this.replicating = replicating;
111 this.zkHelper = zkHelper;
112 this.stopper = stopper;
113 this.hlogsById = new HashMap<String, SortedSet<String>>();
114 this.oldsources = new ArrayList<ReplicationSourceInterface>();
115 this.conf = conf;
116 this.fs = fs;
117 this.logDir = logDir;
118 this.oldLogDir = oldLogDir;
119 this.sleepBeforeFailover = conf.getLong("replication.sleep.before.failover", 2000);
120 this.zkHelper.registerRegionServerListener(
121 new OtherRegionServerWatcher(this.zkHelper.getZookeeperWatcher()));
122 List<String> otherRSs =
123 this.zkHelper.getRegisteredRegionServers();
124 this.zkHelper.registerRegionServerListener(
125 new PeersWatcher(this.zkHelper.getZookeeperWatcher()));
126 this.zkHelper.listPeersIdsAndWatch();
127 this.otherRegionServers = otherRSs == null ? new ArrayList<String>() : otherRSs;
128
129
130 int nbWorkers = conf.getInt("replication.executor.workers", 1);
131
132
133 this.executor = new ThreadPoolExecutor(nbWorkers, nbWorkers,
134 100, TimeUnit.MILLISECONDS,
135 new LinkedBlockingQueue<Runnable>());
136 ThreadFactoryBuilder tfb = new ThreadFactoryBuilder();
137 tfb.setNameFormat("ReplicationExecutor-%d");
138 this.executor.setThreadFactory(tfb.build());
139 }
140
141
142
143
144
145
146
147
148
149
150
151 public void logPositionAndCleanOldLogs(Path log, String id, long position, boolean queueRecovered) {
152 String key = log.getName();
153 LOG.info("Going to report log #" + key + " for position " + position + " in " + log);
154 this.zkHelper.writeReplicationStatus(key, id, position);
155 synchronized (this.hlogsById) {
156 SortedSet<String> hlogs = this.hlogsById.get(id);
157 if (!queueRecovered && hlogs.first() != key) {
158 SortedSet<String> hlogSet = hlogs.headSet(key);
159 LOG.info("Removing " + hlogSet.size() +
160 " logs in the list: " + hlogSet);
161 for (String hlog : hlogSet) {
162 this.zkHelper.removeLogFromList(hlog, id);
163 }
164 hlogSet.clear();
165 }
166 }
167 }
168
169
170
171
172
173 public void init() throws IOException {
174 for (String id : this.zkHelper.getPeerClusters().keySet()) {
175 addSource(id);
176 }
177 List<String> currentReplicators = this.zkHelper.getListOfReplicators();
178 if (currentReplicators == null || currentReplicators.size() == 0) {
179 return;
180 }
181 synchronized (otherRegionServers) {
182 LOG.info("Current list of replicators: " + currentReplicators
183 + " other RSs: " + otherRegionServers);
184 }
185
186 for (String rs : currentReplicators) {
187 synchronized (otherRegionServers) {
188 if (!this.otherRegionServers.contains(rs)) {
189 transferQueues(rs);
190 }
191 }
192 }
193 }
194
195
196
197
198
199
200
201 public ReplicationSourceInterface addSource(String id) throws IOException {
202 ReplicationSourceInterface src =
203 getReplicationSource(this.conf, this.fs, this, stopper, replicating, id);
204
205 src.setSourceEnabled(true);
206 synchronized (this.hlogsById) {
207 this.sources.add(src);
208 this.hlogsById.put(id, new TreeSet<String>());
209
210 if (this.latestPath != null) {
211 String name = this.latestPath.getName();
212 this.hlogsById.get(id).add(name);
213 try {
214 this.zkHelper.addLogToList(name, src.getPeerClusterZnode());
215 } catch (KeeperException ke) {
216 String message = "Cannot add log to zk for" +
217 " replication when creating a new source";
218 stopper.stop(message);
219 throw new IOException(message, ke);
220 }
221 src.enqueueLog(this.latestPath);
222 }
223 }
224 src.startup();
225 return src;
226 }
227
228
229
230
231 public void join() {
232 this.executor.shutdown();
233 if (this.sources.size() == 0) {
234 this.zkHelper.deleteOwnRSZNode();
235 }
236 for (ReplicationSourceInterface source : this.sources) {
237 source.terminate("Region server is closing");
238 }
239 }
240
241
242
243
244
245 protected Map<String, SortedSet<String>> getHLogs() {
246 return Collections.unmodifiableMap(hlogsById);
247 }
248
249
250
251
252
253 public List<ReplicationSourceInterface> getSources() {
254 return this.sources;
255 }
256
257 void logRolled(Path newLog) throws IOException {
258 if (!this.replicating.get()) {
259 LOG.warn("Replication stopped, won't add new log");
260 return;
261 }
262
263 synchronized (this.hlogsById) {
264 String name = newLog.getName();
265 for (ReplicationSourceInterface source : this.sources) {
266 try {
267 this.zkHelper.addLogToList(name, source.getPeerClusterZnode());
268 } catch (KeeperException ke) {
269 throw new IOException("Cannot add log to zk for replication", ke);
270 }
271 }
272 for (SortedSet<String> hlogs : this.hlogsById.values()) {
273 if (this.sources.isEmpty()) {
274
275
276 hlogs.clear();
277 }
278 hlogs.add(name);
279 }
280 }
281
282 this.latestPath = newLog;
283
284 for (ReplicationSourceInterface source : this.sources) {
285 source.enqueueLog(newLog);
286 }
287 }
288
289
290
291
292
293 public ReplicationZookeeper getRepZkWrapper() {
294 return zkHelper;
295 }
296
297
298
299
300
301
302
303
304
305
306
307
308 public ReplicationSourceInterface getReplicationSource(
309 final Configuration conf,
310 final FileSystem fs,
311 final ReplicationSourceManager manager,
312 final Stoppable stopper,
313 final AtomicBoolean replicating,
314 final String peerId) throws IOException {
315 ReplicationSourceInterface src;
316 try {
317 @SuppressWarnings("rawtypes")
318 Class c = Class.forName(conf.get("replication.replicationsource.implementation",
319 ReplicationSource.class.getCanonicalName()));
320 src = (ReplicationSourceInterface) c.newInstance();
321 } catch (Exception e) {
322 LOG.warn("Passed replication source implementation throws errors, " +
323 "defaulting to ReplicationSource", e);
324 src = new ReplicationSource();
325
326 }
327 src.init(conf, fs, manager, stopper, replicating, peerId);
328 return src;
329 }
330
331
332
333
334
335
336
337
338
339 public void transferQueues(String rsZnode) {
340 NodeFailoverWorker transfer = new NodeFailoverWorker(rsZnode);
341 try {
342 this.executor.execute(transfer);
343 } catch (RejectedExecutionException ex) {
344 LOG.info("Cancelling the transfer of " + rsZnode +
345 " because of " + ex.getMessage());
346 }
347 }
348
349
350
351
352
353 public void closeRecoveredQueue(ReplicationSourceInterface src) {
354 LOG.info("Done with the recovered queue " + src.getPeerClusterZnode());
355 this.oldsources.remove(src);
356 this.zkHelper.deleteSource(src.getPeerClusterZnode(), false);
357 }
358
359
360
361
362
363
364 public void removePeer(String id) {
365 LOG.info("Closing the following queue " + id + ", currently have "
366 + sources.size() + " and another "
367 + oldsources.size() + " that were recovered");
368 String terminateMessage = "Replication stream was removed by a user";
369 ReplicationSourceInterface srcToRemove = null;
370 List<ReplicationSourceInterface> oldSourcesToDelete =
371 new ArrayList<ReplicationSourceInterface>();
372
373 for (ReplicationSourceInterface src : oldsources) {
374 if (id.equals(src.getPeerClusterId())) {
375 oldSourcesToDelete.add(src);
376 }
377 }
378 for (ReplicationSourceInterface src : oldSourcesToDelete) {
379 src.terminate(terminateMessage);
380 closeRecoveredQueue((src));
381 }
382 LOG.info("Number of deleted recovered sources for " + id + ": "
383 + oldSourcesToDelete.size());
384
385 for (ReplicationSourceInterface src : this.sources) {
386 if (id.equals(src.getPeerClusterId())) {
387 srcToRemove = src;
388 break;
389 }
390 }
391 if (srcToRemove == null) {
392 LOG.error("The queue we wanted to close is missing " + id);
393 return;
394 }
395 srcToRemove.terminate(terminateMessage);
396 this.sources.remove(srcToRemove);
397 this.zkHelper.deleteSource(id, true);
398 }
399
400
401
402
403
404
405 public class OtherRegionServerWatcher extends ZooKeeperListener {
406
407
408
409
410 public OtherRegionServerWatcher(ZooKeeperWatcher watcher) {
411 super(watcher);
412 }
413
414
415
416
417
418 public void nodeCreated(String path) {
419 refreshRegionServersList(path);
420 }
421
422
423
424
425
426 public void nodeDeleted(String path) {
427 if (stopper.isStopped()) {
428 return;
429 }
430 boolean cont = refreshRegionServersList(path);
431 if (!cont) {
432 return;
433 }
434 LOG.info(path + " znode expired, trying to lock it");
435 transferQueues(ReplicationZookeeper.getZNodeName(path));
436 }
437
438
439
440
441
442 public void nodeChildrenChanged(String path) {
443 if (stopper.isStopped()) {
444 return;
445 }
446 refreshRegionServersList(path);
447 }
448
449 private boolean refreshRegionServersList(String path) {
450 if (!path.startsWith(zkHelper.getZookeeperWatcher().rsZNode)) {
451 return false;
452 }
453 List<String> newRsList = (zkHelper.getRegisteredRegionServers());
454 if (newRsList == null) {
455 return false;
456 } else {
457 synchronized (otherRegionServers) {
458 otherRegionServers.clear();
459 otherRegionServers.addAll(newRsList);
460 }
461 }
462 return true;
463 }
464 }
465
466
467
468
469 public class PeersWatcher extends ZooKeeperListener {
470
471
472
473
474 public PeersWatcher(ZooKeeperWatcher watcher) {
475 super(watcher);
476 }
477
478
479
480
481
482 public void nodeDeleted(String path) {
483 List<String> peers = refreshPeersList(path);
484 if (peers == null) {
485 return;
486 }
487 String id = ReplicationZookeeper.getZNodeName(path);
488 removePeer(id);
489 }
490
491
492
493
494
495 public void nodeChildrenChanged(String path) {
496 List<String> peers = refreshPeersList(path);
497 if (peers == null) {
498 return;
499 }
500 for (String id : peers) {
501 try {
502 boolean added = zkHelper.connectToPeer(id);
503 if (added) {
504 addSource(id);
505 }
506 } catch (IOException e) {
507
508 LOG.error("Error while adding a new peer", e);
509 } catch (KeeperException e) {
510 LOG.error("Error while adding a new peer", e);
511 }
512 }
513 }
514
515
516
517
518
519
520
521
522 private List<String> refreshPeersList(String path) {
523 if (!path.startsWith(zkHelper.getPeersZNode())) {
524 return null;
525 }
526 return zkHelper.listPeersIdsAndWatch();
527 }
528 }
529
530
531
532
533
534 class NodeFailoverWorker extends Thread {
535
536 private String rsZnode;
537
538
539
540
541
542 public NodeFailoverWorker(String rsZnode) {
543 super("Failover-for-"+rsZnode);
544 this.rsZnode = rsZnode;
545 }
546
547 @Override
548 public void run() {
549
550
551 try {
552 Thread.sleep(sleepBeforeFailover);
553 } catch (InterruptedException e) {
554 LOG.warn("Interrupted while waiting before transferring a queue.");
555 Thread.currentThread().interrupt();
556 }
557
558 if (stopper.isStopped()) {
559 LOG.info("Not transferring queue since we are shutting down");
560 return;
561 }
562 if (!zkHelper.lockOtherRS(rsZnode)) {
563 return;
564 }
565 LOG.info("Moving " + rsZnode + "'s hlogs to my queue");
566 SortedMap<String, SortedSet<String>> newQueues =
567 zkHelper.copyQueuesFromRS(rsZnode);
568 zkHelper.deleteRsQueues(rsZnode);
569 if (newQueues == null || newQueues.size() == 0) {
570 return;
571 }
572
573 for (Map.Entry<String, SortedSet<String>> entry : newQueues.entrySet()) {
574 String peerId = entry.getKey();
575 try {
576 ReplicationSourceInterface src = getReplicationSource(conf,
577 fs, ReplicationSourceManager.this, stopper, replicating, peerId);
578 if (!zkHelper.getPeerClusters().containsKey(src.getPeerClusterId())) {
579 src.terminate("Recovered queue doesn't belong to any current peer");
580 break;
581 }
582 oldsources.add(src);
583 for (String hlog : entry.getValue()) {
584 src.enqueueLog(new Path(oldLogDir, hlog));
585 }
586
587 src.setSourceEnabled(true);
588 src.startup();
589 } catch (IOException e) {
590
591 LOG.error("Failed creating a source", e);
592 }
593 }
594 }
595 }
596
597
598
599
600
601 public Path getOldLogDir() {
602 return this.oldLogDir;
603 }
604
605
606
607
608
609 public Path getLogDir() {
610 return this.logDir;
611 }
612
613
614
615
616
617 public FileSystem getFs() {
618 return this.fs;
619 }
620 }