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