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 void preLogRoll(Path newLog) throws IOException {
278 if (!this.replicating.get()) {
279 LOG.warn("Replication stopped, won't add new log");
280 return;
281 }
282
283 synchronized (this.hlogsById) {
284 String name = newLog.getName();
285 for (ReplicationSourceInterface source : this.sources) {
286 try {
287 this.zkHelper.addLogToList(name, source.getPeerClusterZnode());
288 } catch (KeeperException ke) {
289 throw new IOException("Cannot add log to zk for replication", ke);
290 }
291 }
292 for (SortedSet<String> hlogs : this.hlogsById.values()) {
293 if (this.sources.isEmpty()) {
294
295
296 hlogs.clear();
297 }
298 hlogs.add(name);
299 }
300 }
301
302 this.latestPath = newLog;
303 }
304
305 void postLogRoll(Path newLog) throws IOException {
306 if (!this.replicating.get()) {
307 LOG.warn("Replication stopped, won't add new log");
308 return;
309 }
310
311
312 for (ReplicationSourceInterface source : this.sources) {
313 source.enqueueLog(newLog);
314 }
315 }
316
317
318
319
320
321 public ReplicationZookeeper getRepZkWrapper() {
322 return zkHelper;
323 }
324
325
326
327
328
329
330
331
332
333
334
335
336 public ReplicationSourceInterface getReplicationSource(
337 final Configuration conf,
338 final FileSystem fs,
339 final ReplicationSourceManager manager,
340 final Stoppable stopper,
341 final AtomicBoolean replicating,
342 final String peerId) throws IOException {
343 ReplicationSourceInterface src;
344 try {
345 @SuppressWarnings("rawtypes")
346 Class c = Class.forName(conf.get("replication.replicationsource.implementation",
347 ReplicationSource.class.getCanonicalName()));
348 src = (ReplicationSourceInterface) c.newInstance();
349 } catch (Exception e) {
350 LOG.warn("Passed replication source implementation throws errors, " +
351 "defaulting to ReplicationSource", e);
352 src = new ReplicationSource();
353
354 }
355 src.init(conf, fs, manager, stopper, replicating, peerId);
356 return src;
357 }
358
359
360
361
362
363
364
365
366
367 public void transferQueues(String rsZnode) {
368 NodeFailoverWorker transfer = new NodeFailoverWorker(rsZnode);
369 try {
370 this.executor.execute(transfer);
371 } catch (RejectedExecutionException ex) {
372 LOG.info("Cancelling the transfer of " + rsZnode +
373 " because of " + ex.getMessage());
374 }
375 }
376
377
378
379
380
381 public void closeRecoveredQueue(ReplicationSourceInterface src) {
382 LOG.info("Done with the recovered queue " + src.getPeerClusterZnode());
383 this.oldsources.remove(src);
384 this.zkHelper.deleteSource(src.getPeerClusterZnode(), false);
385 }
386
387
388
389
390
391
392 public void removePeer(String id) {
393 LOG.info("Closing the following queue " + id + ", currently have "
394 + sources.size() + " and another "
395 + oldsources.size() + " that were recovered");
396 String terminateMessage = "Replication stream was removed by a user";
397 ReplicationSourceInterface srcToRemove = null;
398 List<ReplicationSourceInterface> oldSourcesToDelete =
399 new ArrayList<ReplicationSourceInterface>();
400
401 for (ReplicationSourceInterface src : oldsources) {
402 if (id.equals(src.getPeerClusterId())) {
403 oldSourcesToDelete.add(src);
404 }
405 }
406 for (ReplicationSourceInterface src : oldSourcesToDelete) {
407 src.terminate(terminateMessage);
408 closeRecoveredQueue((src));
409 }
410 LOG.info("Number of deleted recovered sources for " + id + ": "
411 + oldSourcesToDelete.size());
412
413 for (ReplicationSourceInterface src : this.sources) {
414 if (id.equals(src.getPeerClusterId())) {
415 srcToRemove = src;
416 break;
417 }
418 }
419 if (srcToRemove == null) {
420 LOG.error("The queue we wanted to close is missing " + id);
421 return;
422 }
423 srcToRemove.terminate(terminateMessage);
424 this.sources.remove(srcToRemove);
425 this.zkHelper.deleteSource(id, true);
426 }
427
428
429
430
431
432
433
434
435
436 private boolean refreshOtherRegionServersList() {
437 List<String> newRsList = zkHelper.getRegisteredRegionServers();
438 if (newRsList == null) {
439 return false;
440 } else {
441 synchronized (otherRegionServers) {
442 otherRegionServers.clear();
443 otherRegionServers.addAll(newRsList);
444 }
445 }
446 return true;
447 }
448
449
450
451
452
453
454 public class OtherRegionServerWatcher extends ZooKeeperListener {
455
456
457
458
459 public OtherRegionServerWatcher(ZooKeeperWatcher watcher) {
460 super(watcher);
461 }
462
463
464
465
466
467 public void nodeCreated(String path) {
468 refreshListIfRightPath(path);
469 }
470
471
472
473
474
475 public void nodeDeleted(String path) {
476 if (stopper.isStopped()) {
477 return;
478 }
479 boolean cont = refreshListIfRightPath(path);
480 if (!cont) {
481 return;
482 }
483 LOG.info(path + " znode expired, trying to lock it");
484 transferQueues(ReplicationZookeeper.getZNodeName(path));
485 }
486
487
488
489
490
491 public void nodeChildrenChanged(String path) {
492 if (stopper.isStopped()) {
493 return;
494 }
495 refreshListIfRightPath(path);
496 }
497
498 private boolean refreshListIfRightPath(String path) {
499 if (!path.startsWith(zkHelper.getZookeeperWatcher().rsZNode)) {
500 return false;
501 }
502 return refreshOtherRegionServersList();
503 }
504 }
505
506
507
508
509 public class PeersWatcher extends ZooKeeperListener {
510
511
512
513
514 public PeersWatcher(ZooKeeperWatcher watcher) {
515 super(watcher);
516 }
517
518
519
520
521
522 public void nodeDeleted(String path) {
523 List<String> peers = refreshPeersList(path);
524 if (peers == null) {
525 return;
526 }
527 String id = ReplicationZookeeper.getZNodeName(path);
528 removePeer(id);
529 }
530
531
532
533
534
535 public void nodeChildrenChanged(String path) {
536 List<String> peers = refreshPeersList(path);
537 if (peers == null) {
538 return;
539 }
540 for (String id : peers) {
541 try {
542 boolean added = zkHelper.connectToPeer(id);
543 if (added) {
544 addSource(id);
545 }
546 } catch (IOException e) {
547
548 LOG.error("Error while adding a new peer", e);
549 } catch (KeeperException e) {
550 LOG.error("Error while adding a new peer", e);
551 }
552 }
553 }
554
555
556
557
558
559
560
561
562 private List<String> refreshPeersList(String path) {
563 if (!path.startsWith(zkHelper.getPeersZNode())) {
564 return null;
565 }
566 return zkHelper.listPeersIdsAndWatch();
567 }
568 }
569
570
571
572
573
574 class NodeFailoverWorker extends Thread {
575
576 private String rsZnode;
577
578
579
580
581
582 public NodeFailoverWorker(String rsZnode) {
583 super("Failover-for-"+rsZnode);
584 this.rsZnode = rsZnode;
585 }
586
587 @Override
588 public void run() {
589
590
591 try {
592 Thread.sleep(sleepBeforeFailover + (long) (rand.nextFloat() * sleepBeforeFailover));
593 } catch (InterruptedException e) {
594 LOG.warn("Interrupted while waiting before transferring a queue.");
595 Thread.currentThread().interrupt();
596 }
597
598 if (stopper.isStopped()) {
599 LOG.info("Not transferring queue since we are shutting down");
600 return;
601 }
602 SortedMap<String, SortedSet<String>> newQueues = null;
603
604
605 if (conf.getBoolean(HConstants.ZOOKEEPER_USEMULTI, true)) {
606 LOG.info("Atomically moving " + rsZnode + "'s hlogs to my queue");
607 newQueues = zkHelper.copyQueuesFromRSUsingMulti(rsZnode);
608 } else {
609 LOG.info("Moving " + rsZnode + "'s hlogs to my queue");
610 if (!zkHelper.lockOtherRS(rsZnode)) {
611 return;
612 }
613 newQueues = zkHelper.copyQueuesFromRS(rsZnode);
614 zkHelper.deleteRsQueues(rsZnode);
615 }
616
617 if (newQueues.isEmpty()) {
618 return;
619 }
620
621 for (Map.Entry<String, SortedSet<String>> entry : newQueues.entrySet()) {
622 String peerId = entry.getKey();
623 try {
624 ReplicationSourceInterface src = getReplicationSource(conf,
625 fs, ReplicationSourceManager.this, stopper, replicating, peerId);
626 if (!zkHelper.getPeerClusters().containsKey(src.getPeerClusterId())) {
627 src.terminate("Recovered queue doesn't belong to any current peer");
628 break;
629 }
630 oldsources.add(src);
631 for (String hlog : entry.getValue()) {
632 src.enqueueLog(new Path(oldLogDir, hlog));
633 }
634 src.startup();
635 } catch (IOException e) {
636
637 LOG.error("Failed creating a source", e);
638 }
639 }
640 }
641 }
642
643
644
645
646
647 public Path getOldLogDir() {
648 return this.oldLogDir;
649 }
650
651
652
653
654
655 public Path getLogDir() {
656 return this.logDir;
657 }
658
659
660
661
662
663 public FileSystem getFs() {
664 return this.fs;
665 }
666 }