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