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