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 synchronized (this.hlogsById) {
165 SortedSet<String> hlogs = this.hlogsById.get(id);
166 if (!queueRecovered && hlogs.first() != key) {
167 SortedSet<String> hlogSet = hlogs.headSet(key);
168 LOG.info("Removing " + hlogSet.size() +
169 " logs in the list: " + hlogSet);
170 for (String hlog : hlogSet) {
171 this.zkHelper.removeLogFromList(hlog, id);
172 }
173 hlogSet.clear();
174 }
175 }
176 }
177
178
179
180
181
182 public void init() throws IOException {
183 for (String id : this.zkHelper.getPeerClusters().keySet()) {
184 addSource(id);
185 }
186 List<String> currentReplicators = this.zkHelper.getListOfReplicators();
187 if (currentReplicators == null || currentReplicators.size() == 0) {
188 return;
189 }
190 synchronized (otherRegionServers) {
191 refreshOtherRegionServersList();
192 LOG.info("Current list of replicators: " + currentReplicators
193 + " other RSs: " + otherRegionServers);
194 }
195
196 for (String rs : currentReplicators) {
197 synchronized (otherRegionServers) {
198 if (!this.otherRegionServers.contains(rs)) {
199 transferQueues(rs);
200 }
201 }
202 }
203 }
204
205
206
207
208
209
210
211 public ReplicationSourceInterface addSource(String id) throws IOException {
212 ReplicationSourceInterface src =
213 getReplicationSource(this.conf, this.fs, this, stopper, replicating, id);
214 synchronized (this.hlogsById) {
215 this.sources.add(src);
216 this.hlogsById.put(id, new TreeSet<String>());
217
218 if (this.latestPath != null) {
219 String name = this.latestPath.getName();
220 this.hlogsById.get(id).add(name);
221 try {
222 this.zkHelper.addLogToList(name, src.getPeerClusterZnode());
223 } catch (KeeperException ke) {
224 String message = "Cannot add log to zk for" +
225 " replication when creating a new source";
226 stopper.stop(message);
227 throw new IOException(message, ke);
228 }
229 src.enqueueLog(this.latestPath);
230 }
231 }
232 src.startup();
233 return src;
234 }
235
236
237
238
239 public void join() {
240 this.executor.shutdown();
241 if (this.sources.size() == 0) {
242 this.zkHelper.deleteOwnRSZNode();
243 }
244 for (ReplicationSourceInterface source : this.sources) {
245 source.terminate("Region server is closing");
246 }
247 }
248
249
250
251
252
253 protected Map<String, SortedSet<String>> getHLogs() {
254 return Collections.unmodifiableMap(hlogsById);
255 }
256
257
258
259
260
261 public List<ReplicationSourceInterface> getSources() {
262 return this.sources;
263 }
264
265 void preLogRoll(Path newLog) throws IOException {
266 if (!this.replicating.get()) {
267 LOG.warn("Replication stopped, won't add new log");
268 return;
269 }
270
271 synchronized (this.hlogsById) {
272 String name = newLog.getName();
273 for (ReplicationSourceInterface source : this.sources) {
274 try {
275 this.zkHelper.addLogToList(name, source.getPeerClusterZnode());
276 } catch (KeeperException ke) {
277 throw new IOException("Cannot add log to zk for replication", ke);
278 }
279 }
280 for (SortedSet<String> hlogs : this.hlogsById.values()) {
281 if (this.sources.isEmpty()) {
282
283
284 hlogs.clear();
285 }
286 hlogs.add(name);
287 }
288 }
289
290 this.latestPath = newLog;
291 }
292
293 void postLogRoll(Path newLog) throws IOException {
294 if (!this.replicating.get()) {
295 LOG.warn("Replication stopped, won't add new log");
296 return;
297 }
298
299
300 for (ReplicationSourceInterface source : this.sources) {
301 source.enqueueLog(newLog);
302 }
303 }
304
305
306
307
308
309 public ReplicationZookeeper getRepZkWrapper() {
310 return zkHelper;
311 }
312
313
314
315
316
317
318
319
320
321
322
323
324 public ReplicationSourceInterface getReplicationSource(
325 final Configuration conf,
326 final FileSystem fs,
327 final ReplicationSourceManager manager,
328 final Stoppable stopper,
329 final AtomicBoolean replicating,
330 final String peerId) throws IOException {
331 ReplicationSourceInterface src;
332 try {
333 @SuppressWarnings("rawtypes")
334 Class c = Class.forName(conf.get("replication.replicationsource.implementation",
335 ReplicationSource.class.getCanonicalName()));
336 src = (ReplicationSourceInterface) c.newInstance();
337 } catch (Exception e) {
338 LOG.warn("Passed replication source implementation throws errors, " +
339 "defaulting to ReplicationSource", e);
340 src = new ReplicationSource();
341
342 }
343 src.init(conf, fs, manager, stopper, replicating, peerId);
344 return src;
345 }
346
347
348
349
350
351
352
353
354
355 public void transferQueues(String rsZnode) {
356 NodeFailoverWorker transfer = new NodeFailoverWorker(rsZnode);
357 try {
358 this.executor.execute(transfer);
359 } catch (RejectedExecutionException ex) {
360 LOG.info("Cancelling the transfer of " + rsZnode +
361 " because of " + ex.getMessage());
362 }
363 }
364
365
366
367
368
369 public void closeRecoveredQueue(ReplicationSourceInterface src) {
370 LOG.info("Done with the recovered queue " + src.getPeerClusterZnode());
371 this.oldsources.remove(src);
372 this.zkHelper.deleteSource(src.getPeerClusterZnode(), false);
373 }
374
375
376
377
378
379
380 public void removePeer(String id) {
381 LOG.info("Closing the following queue " + id + ", currently have "
382 + sources.size() + " and another "
383 + oldsources.size() + " that were recovered");
384 String terminateMessage = "Replication stream was removed by a user";
385 ReplicationSourceInterface srcToRemove = null;
386 List<ReplicationSourceInterface> oldSourcesToDelete =
387 new ArrayList<ReplicationSourceInterface>();
388
389 for (ReplicationSourceInterface src : oldsources) {
390 if (id.equals(src.getPeerClusterId())) {
391 oldSourcesToDelete.add(src);
392 }
393 }
394 for (ReplicationSourceInterface src : oldSourcesToDelete) {
395 src.terminate(terminateMessage);
396 closeRecoveredQueue((src));
397 }
398 LOG.info("Number of deleted recovered sources for " + id + ": "
399 + oldSourcesToDelete.size());
400
401 for (ReplicationSourceInterface src : this.sources) {
402 if (id.equals(src.getPeerClusterId())) {
403 srcToRemove = src;
404 break;
405 }
406 }
407 if (srcToRemove == null) {
408 LOG.error("The queue we wanted to close is missing " + id);
409 return;
410 }
411 srcToRemove.terminate(terminateMessage);
412 this.sources.remove(srcToRemove);
413 this.zkHelper.deleteSource(id, true);
414 }
415
416
417
418
419
420
421
422
423
424 private boolean refreshOtherRegionServersList() {
425 List<String> newRsList = zkHelper.getRegisteredRegionServers();
426 if (newRsList == null) {
427 return false;
428 } else {
429 synchronized (otherRegionServers) {
430 otherRegionServers.clear();
431 otherRegionServers.addAll(newRsList);
432 }
433 }
434 return true;
435 }
436
437
438
439
440
441
442 public class OtherRegionServerWatcher extends ZooKeeperListener {
443
444
445
446
447 public OtherRegionServerWatcher(ZooKeeperWatcher watcher) {
448 super(watcher);
449 }
450
451
452
453
454
455 public void nodeCreated(String path) {
456 refreshListIfRightPath(path);
457 }
458
459
460
461
462
463 public void nodeDeleted(String path) {
464 if (stopper.isStopped()) {
465 return;
466 }
467 boolean cont = refreshListIfRightPath(path);
468 if (!cont) {
469 return;
470 }
471 LOG.info(path + " znode expired, trying to lock it");
472 transferQueues(ReplicationZookeeper.getZNodeName(path));
473 }
474
475
476
477
478
479 public void nodeChildrenChanged(String path) {
480 if (stopper.isStopped()) {
481 return;
482 }
483 refreshListIfRightPath(path);
484 }
485
486 private boolean refreshListIfRightPath(String path) {
487 if (!path.startsWith(zkHelper.getZookeeperWatcher().rsZNode)) {
488 return false;
489 }
490 return refreshOtherRegionServersList();
491 }
492 }
493
494
495
496
497 public class PeersWatcher extends ZooKeeperListener {
498
499
500
501
502 public PeersWatcher(ZooKeeperWatcher watcher) {
503 super(watcher);
504 }
505
506
507
508
509
510 public void nodeDeleted(String path) {
511 List<String> peers = refreshPeersList(path);
512 if (peers == null) {
513 return;
514 }
515 String id = ReplicationZookeeper.getZNodeName(path);
516 removePeer(id);
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 }