View Javadoc

1   /*
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  
20  package org.apache.hadoop.hbase.replication.regionserver;
21  
22  import com.google.common.collect.Sets;
23  import com.google.common.util.concurrent.ThreadFactoryBuilder;
24  
25  import java.io.IOException;
26  import java.util.ArrayList;
27  import java.util.Collections;
28  import java.util.HashMap;
29  import java.util.List;
30  import java.util.Map;
31  import java.util.Random;
32  import java.util.Set;
33  import java.util.SortedSet;
34  import java.util.TreeSet;
35  import java.util.UUID;
36  import java.util.concurrent.ConcurrentHashMap;
37  import java.util.concurrent.CopyOnWriteArrayList;
38  import java.util.concurrent.LinkedBlockingQueue;
39  import java.util.concurrent.RejectedExecutionException;
40  import java.util.concurrent.ThreadPoolExecutor;
41  import java.util.concurrent.TimeUnit;
42  import java.util.concurrent.atomic.AtomicLong;
43  
44  import org.apache.commons.logging.Log;
45  import org.apache.commons.logging.LogFactory;
46  import org.apache.hadoop.conf.Configuration;
47  import org.apache.hadoop.fs.FileSystem;
48  import org.apache.hadoop.fs.Path;
49  import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
50  import org.apache.hadoop.hbase.HConstants;
51  import org.apache.hadoop.hbase.Server;
52  import org.apache.hadoop.hbase.TableDescriptors;
53  import org.apache.hadoop.hbase.TableName;
54  import org.apache.hadoop.hbase.classification.InterfaceAudience;
55  import org.apache.hadoop.hbase.regionserver.HRegionServer;
56  import org.apache.hadoop.hbase.regionserver.RegionServerCoprocessorHost;
57  import org.apache.hadoop.hbase.replication.ReplicationEndpoint;
58  import org.apache.hadoop.hbase.replication.ReplicationException;
59  import org.apache.hadoop.hbase.replication.ReplicationListener;
60  import org.apache.hadoop.hbase.replication.ReplicationPeer;
61  import org.apache.hadoop.hbase.replication.ReplicationPeer.PeerState;
62  import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
63  import org.apache.hadoop.hbase.replication.ReplicationPeers;
64  import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
65  import org.apache.hadoop.hbase.replication.ReplicationQueues;
66  import org.apache.hadoop.hbase.replication.ReplicationSourceWithoutPeerException;
67  import org.apache.hadoop.hbase.replication.ReplicationTracker;
68  import org.apache.hadoop.hbase.util.Pair;
69  import org.apache.hadoop.hbase.wal.DefaultWALProvider;
70  
71  /**
72   * This class is responsible to manage all the replication
73   * sources. There are two classes of sources:
74   * <ul>
75   * <li> Normal sources are persistent and one per peer cluster</li>
76   * <li> Old sources are recovered from a failed region server and our
77   * only goal is to finish replicating the WAL queue it had up in ZK</li>
78   * </ul>
79   *
80   * When a region server dies, this class uses a watcher to get notified and it
81   * tries to grab a lock in order to transfer all the queues in a local
82   * old source.
83   *
84   * This class implements the ReplicationListener interface so that it can track changes in
85   * replication state.
86   */
87  @InterfaceAudience.Private
88  public class ReplicationSourceManager implements ReplicationListener {
89    private static final Log LOG =
90        LogFactory.getLog(ReplicationSourceManager.class);
91    // List of all the sources that read this RS's logs
92    private final List<ReplicationSourceInterface> sources;
93    // List of all the sources we got from died RSs
94    private final List<ReplicationSourceInterface> oldsources;
95    private final ReplicationQueues replicationQueues;
96    private final ReplicationTracker replicationTracker;
97    private final ReplicationPeers replicationPeers;
98    // UUID for this cluster
99    private final UUID clusterId;
100   // All about stopping
101   private final Server server;
102   // All logs we are currently tracking
103   // Index structure of the map is: peer_id->logPrefix/logGroup->logs
104   private final Map<String, Map<String, SortedSet<String>>> walsById;
105   // Logs for recovered sources we are currently tracking
106   private final Map<String, Map<String, SortedSet<String>>> walsByIdRecoveredQueues;
107   private final Configuration conf;
108   private final FileSystem fs;
109   // The paths to the latest log of each wal group, for new coming peers
110   private final Map<String, Path> latestPaths;
111   // Path to the wals directories
112   private final Path logDir;
113   // Path to the wal archive
114   private final Path oldLogDir;
115   // The number of ms that we wait before moving znodes, HBASE-3596
116   private final long sleepBeforeFailover;
117   // Homemade executer service for replication
118   private final ThreadPoolExecutor executor;
119 
120   private final Random rand;
121   private final boolean replicationForBulkLoadDataEnabled;
122 
123   private AtomicLong totalBufferUsed = new AtomicLong();
124 
125   /**
126    * Creates a replication manager and sets the watch on all the other registered region servers
127    * @param replicationQueues the interface for manipulating replication queues
128    * @param replicationPeers the replication peers maintenance class
129    * @param replicationTracker the replication tracker to track the states
130    * @param conf the configuration to use
131    * @param server the server for this region server
132    * @param fs the file system to use
133    * @param logDir the directory that contains all wal directories of live RSs
134    * @param oldLogDir the directory where old logs are archived
135    * @param clusterId the cluster id of the source cluster
136    */
137   public ReplicationSourceManager(final ReplicationQueues replicationQueues,
138       final ReplicationPeers replicationPeers, final ReplicationTracker replicationTracker,
139       final Configuration conf, final Server server, final FileSystem fs, final Path logDir,
140       final Path oldLogDir, final UUID clusterId) {
141     //CopyOnWriteArrayList is thread-safe.
142     //Generally, reading is more than modifying.
143     this.sources = new CopyOnWriteArrayList<ReplicationSourceInterface>();
144     this.replicationQueues = replicationQueues;
145     this.replicationPeers = replicationPeers;
146     this.replicationTracker = replicationTracker;
147     this.server = server;
148     this.walsById = new HashMap<String, Map<String, SortedSet<String>>>();
149     this.walsByIdRecoveredQueues = new ConcurrentHashMap<String, Map<String, SortedSet<String>>>();
150     this.oldsources = new CopyOnWriteArrayList<ReplicationSourceInterface>();
151     this.conf = conf;
152     this.fs = fs;
153     this.logDir = logDir;
154     this.oldLogDir = oldLogDir;
155     this.sleepBeforeFailover =
156         conf.getLong("replication.sleep.before.failover", 30000); // 30 seconds
157     this.clusterId = clusterId;
158     this.replicationTracker.registerListener(this);
159     this.replicationPeers.getAllPeerIds();
160     // It's preferable to failover 1 RS at a time, but with good zk servers
161     // more could be processed at the same time.
162     int nbWorkers = conf.getInt("replication.executor.workers", 1);
163     // use a short 100ms sleep since this could be done inline with a RS startup
164     // even if we fail, other region servers can take care of it
165     this.executor = new ThreadPoolExecutor(nbWorkers, nbWorkers,
166         100, TimeUnit.MILLISECONDS,
167         new LinkedBlockingQueue<Runnable>());
168     ThreadFactoryBuilder tfb = new ThreadFactoryBuilder();
169     tfb.setNameFormat("ReplicationExecutor-%d");
170     tfb.setDaemon(true);
171     this.executor.setThreadFactory(tfb.build());
172     this.rand = new Random();
173     this.latestPaths = new HashMap<>();
174     replicationForBulkLoadDataEnabled =
175         conf.getBoolean(HConstants.REPLICATION_BULKLOAD_ENABLE_KEY,
176           HConstants.REPLICATION_BULKLOAD_ENABLE_DEFAULT);
177   }
178 
179   /**
180    * Provide the id of the peer and a log key and this method will figure which
181    * wal it belongs to and will log, for this region server, the current
182    * position. It will also clean old logs from the queue.
183    * @param log Path to the log currently being replicated from
184    *            replication status in zookeeper. It will also delete older entries.
185    * @param id id of the replication queue
186    * @param position current location in the log
187    * @param queueRecovered indicates if this queue comes from another region server
188    * @param holdLogInZK if true then the log is retained in ZK
189    */
190   public synchronized void logPositionAndCleanOldLogs(Path log, String id, long position,
191       boolean queueRecovered, boolean holdLogInZK) {
192     String fileName = log.getName();
193     this.replicationQueues.setLogPosition(id, fileName, position);
194     if (holdLogInZK) {
195       return;
196     }
197 
198     cleanOldLogs(fileName, id, queueRecovered);
199   }
200 
201   /**
202    * Cleans a log file and all older files from ZK. Called when we are sure that a
203    * log file is closed and has no more entries.
204    * @param key Path to the log
205    * @param id id of the peer cluster
206    * @param queueRecovered Whether this is a recovered queue
207    */
208   public void cleanOldLogs(String key, String id, boolean queueRecovered) {
209     String logPrefix = DefaultWALProvider.getWALPrefixFromWALName(key);
210     if (queueRecovered) {
211       Map<String, SortedSet<String>> walsForPeer = walsByIdRecoveredQueues.get(id);
212       if(walsForPeer != null) {
213         SortedSet<String> wals = walsForPeer.get(logPrefix);
214         if (wals != null && !wals.first().equals(key)) {
215           cleanOldLogs(wals, key, id);
216         }
217       }
218     } else {
219       synchronized (this.walsById) {
220         SortedSet<String> wals = getLogsWithPrefix(id, logPrefix);
221         if (wals != null && !wals.first().equals(key)) {
222           cleanOldLogs(wals, key, id);
223         }
224       }
225     }
226   }
227 
228   private void cleanOldLogs(SortedSet<String> wals, String key, String id) {
229     SortedSet<String> walSet = wals.headSet(key);
230     LOG.debug("Removing " + walSet.size() + " logs in the list: " + walSet);
231     try {
232       for (String wal : walSet) {
233         this.replicationQueues.removeLog(id, wal);
234       }
235     } catch (ReplicationSourceWithoutPeerException rspe) {
236       // This means the source is running and replication peer have been removed
237       // We should call the removePeer workflow to terminate the source gracefully
238       LOG.warn("Replication peer " + id + " has been removed and source is still running", rspe);
239       String peerId = id;
240       if (peerId.contains("-")) {
241         peerId = peerId.split("-")[0];
242       }
243       schedulePeerRemoval(peerId);
244     }
245     walSet.clear();
246   }
247 
248   /**
249    * Get logs with log prefix for the given wal group
250    * @param walGroupId wal group ID
251    * @param logPrefix log prefix
252    * @return logs with the given prefix
253    */
254   public SortedSet<String> getLogsWithPrefix(String walGroupId, String logPrefix) {
255     return walsById.get(walGroupId).get(logPrefix);
256   }
257 
258   /**
259    * Adds a normal source per registered peer cluster and tries to process all
260    * old region server wal queues
261    */
262   protected void init() throws IOException, ReplicationException {
263     for (String id : this.replicationPeers.getPeerIds()) {
264       addSource(id);
265       if (replicationForBulkLoadDataEnabled) {
266         // Check if peer exists in hfile-refs queue, if not add it. This can happen in the case
267         // when a peer was added before replication for bulk loaded data was enabled.
268         this.replicationQueues.addPeerToHFileRefs(id);
269       }
270     }
271     List<String> currentReplicators = this.replicationQueues.getListOfReplicators();
272     if (currentReplicators == null || currentReplicators.isEmpty()) {
273       return;
274     }
275     List<String> otherRegionServers = replicationTracker.getListOfRegionServers();
276     LOG.info("Current list of replicators: " + currentReplicators + " other RSs: "
277         + otherRegionServers);
278 
279     // Look if there's anything to process after a restart
280     for (String rs : currentReplicators) {
281       if (!otherRegionServers.contains(rs)) {
282         transferQueues(rs);
283       }
284     }
285   }
286 
287   /**
288    * Add sources for the given peer cluster on this region server. For the newly added peer, we only
289    * need to enqueue the latest log of each wal group and do replication
290    * @param id the id of the peer cluster
291    * @return the source that was created
292    * @throws IOException IO Exception
293    */
294   protected ReplicationSourceInterface addSource(String id) throws IOException,
295       ReplicationException {
296     ReplicationPeerConfig peerConfig = replicationPeers.getReplicationPeerConfig(id);
297     ReplicationPeer peer = replicationPeers.getPeer(id);
298     ReplicationSourceInterface src =
299         getReplicationSource(this.conf, this.fs, this, this.replicationQueues,
300           this.replicationPeers, server, id, this.clusterId, peerConfig, peer);
301     synchronized (this.walsById) {
302       this.sources.add(src);
303       Map<String, SortedSet<String>> walsByGroup = new HashMap<String, SortedSet<String>>();
304       this.walsById.put(id, walsByGroup);
305       // Add the latest wal to that source's queue
306       synchronized (latestPaths) {
307         if (!latestPaths.isEmpty()) {
308           for (Map.Entry<String, Path> walPrefixAndPath : latestPaths.entrySet()) {
309             Path walPath = walPrefixAndPath.getValue();
310             SortedSet<String> wals = new TreeSet<>();
311             wals.add(walPath.getName());
312             walsByGroup.put(walPrefixAndPath.getKey(), wals);
313             try {
314               this.replicationQueues.addLog(id, walPath.getName());
315             } catch (ReplicationException e) {
316               String message =
317                   "Cannot add log to queue when creating a new source, queueId=" + id
318                       + ", filename=" + walPath.getName();
319               server.stop(message);
320               throw e;
321             }
322             src.enqueueLog(walPath);
323           }
324         }
325       }
326     }
327     src.startup();
328     return src;
329   }
330 
331   /**
332    * Delete a complete queue of wals associated with a peer cluster
333    * @param peerId Id of the peer cluster queue of wals to delete
334    */
335   public void deleteSource(String peerId, boolean closeConnection) {
336     this.replicationQueues.removeQueue(peerId);
337     if (closeConnection) {
338       this.replicationPeers.peerRemoved(peerId);
339     }
340   }
341 
342   /**
343    * Terminate the replication on this region server
344    */
345   public void join() {
346     this.executor.shutdown();
347     if (this.sources.size() == 0) {
348       this.replicationQueues.removeAllQueues();
349     }
350     for (ReplicationSourceInterface source : this.sources) {
351       source.terminate("Region server is closing");
352     }
353   }
354 
355   /**
356    * Get a copy of the wals of the first source on this rs
357    * @return a sorted set of wal names
358    */
359   protected Map<String, Map<String, SortedSet<String>>> getWALs() {
360     return Collections.unmodifiableMap(walsById);
361   }
362 
363   /**
364    * Get a copy of the wals of the recovered sources on this rs
365    * @return a sorted set of wal names
366    */
367   protected Map<String, Map<String, SortedSet<String>>> getWalsByIdRecoveredQueues() {
368     return Collections.unmodifiableMap(walsByIdRecoveredQueues);
369   }
370 
371   /**
372    * Get a list of all the normal sources of this rs
373    * @return lis of all sources
374    */
375   public List<ReplicationSourceInterface> getSources() {
376     return this.sources;
377   }
378 
379   /**
380    * Get a list of all the old sources of this rs
381    * @return list of all old sources
382    */
383   public List<ReplicationSourceInterface> getOldSources() {
384     return this.oldsources;
385   }
386 
387   /**
388    * Get the normal source for a given peer
389    * @param peerId the replication peer Id
390    * @return the normal source for the give peer if it exists, otherwise null.
391    */
392   public ReplicationSourceInterface getSource(String peerId) {
393     for (ReplicationSourceInterface source: getSources()) {
394       if (source.getPeerClusterId().equals(peerId)) {
395         return source;
396       }
397     }
398     return null;
399   }
400 
401   List<String> getAllQueues() {
402     return replicationQueues.getAllQueues();
403   }
404 
405   void preLogRoll(Path newLog) throws IOException {
406     recordLog(newLog);
407     String logName = newLog.getName();
408     String logPrefix = DefaultWALProvider.getWALPrefixFromWALName(logName);
409     synchronized (latestPaths) {
410       latestPaths.put(logPrefix, newLog);
411     }
412   }
413 
414   /**
415    * Check and enqueue the given log to the correct source. If there's still no source for the
416    * group to which the given log belongs, create one
417    * @param logPath the log path to check and enqueue
418    * @throws IOException IO Exception
419    */
420   private void recordLog(Path logPath) throws IOException {
421     String logName = logPath.getName();
422     String logPrefix = DefaultWALProvider.getWALPrefixFromWALName(logName);
423     // update replication queues on ZK
424     // synchronize on replicationPeers to avoid adding source for the to-be-removed peer
425     synchronized (replicationPeers) {
426       for (String id : replicationPeers.getPeerIds()) {
427         try {
428           this.replicationQueues.addLog(id, logName);
429         } catch (ReplicationException e) {
430           throw new IOException("Cannot add log to replication queue"
431               + " when creating a new source, queueId=" + id + ", filename=" + logName, e);
432         }
433       }
434     }
435     // update walsById map
436     synchronized (walsById) {
437       for (Map.Entry<String, Map<String, SortedSet<String>>> entry : this.walsById.entrySet()) {
438         String peerId = entry.getKey();
439         Map<String, SortedSet<String>> walsByPrefix = entry.getValue();
440         boolean existingPrefix = false;
441         for (Map.Entry<String, SortedSet<String>> walsEntry : walsByPrefix.entrySet()) {
442           SortedSet<String> wals = walsEntry.getValue();
443           if (this.sources.isEmpty()) {
444             // If there's no slaves, don't need to keep the old wals since
445             // we only consider the last one when a new slave comes in
446             wals.clear();
447           }
448           if (logPrefix.equals(walsEntry.getKey())) {
449             wals.add(logName);
450             existingPrefix = true;
451           }
452         }
453         if (!existingPrefix) {
454           // The new log belongs to a new group, add it into this peer
455           LOG.debug("Start tracking logs for wal group " + logPrefix + " for peer " + peerId);
456           SortedSet<String> wals = new TreeSet<String>();
457           wals.add(logName);
458           walsByPrefix.put(logPrefix, wals);
459         }
460       }
461     }
462   }
463 
464   void postLogRoll(Path newLog) throws IOException {
465     // This only updates the sources we own, not the recovered ones
466     for (ReplicationSourceInterface source : this.sources) {
467       source.enqueueLog(newLog);
468     }
469   }
470 
471   public AtomicLong getTotalBufferUsed() {
472     return totalBufferUsed;
473   }
474 
475   /**
476    * Factory method to create a replication source
477    * @param conf the configuration to use
478    * @param fs the file system to use
479    * @param manager the manager to use
480    * @param server the server object for this region server
481    * @param peerId the id of the peer cluster
482    * @return the created source
483    * @throws IOException IO Exception
484    */
485   protected ReplicationSourceInterface getReplicationSource(final Configuration conf,
486       final FileSystem fs, final ReplicationSourceManager manager,
487       final ReplicationQueues replicationQueues, final ReplicationPeers replicationPeers,
488       final Server server, final String peerId, final UUID clusterId,
489       final ReplicationPeerConfig peerConfig, final ReplicationPeer replicationPeer)
490       throws IOException {
491     RegionServerCoprocessorHost rsServerHost = null;
492     TableDescriptors tableDescriptors = null;
493     if (server instanceof HRegionServer) {
494       rsServerHost = ((HRegionServer) server).getRegionServerCoprocessorHost();
495       tableDescriptors = ((HRegionServer) server).getTableDescriptors();
496     }
497     ReplicationSourceInterface src;
498     try {
499       @SuppressWarnings("rawtypes")
500       Class c = Class.forName(conf.get("replication.replicationsource.implementation",
501           ReplicationSource.class.getCanonicalName()));
502       src = (ReplicationSourceInterface) c.newInstance();
503     } catch (Exception e) {
504       LOG.warn("Passed replication source implementation throws errors, " +
505           "defaulting to ReplicationSource", e);
506       src = new ReplicationSource();
507     }
508 
509     ReplicationEndpoint replicationEndpoint = null;
510     try {
511       String replicationEndpointImpl = peerConfig.getReplicationEndpointImpl();
512       if (replicationEndpointImpl == null) {
513         // Default to HBase inter-cluster replication endpoint
514         replicationEndpointImpl = HBaseInterClusterReplicationEndpoint.class.getName();
515       }
516       @SuppressWarnings("rawtypes")
517       Class c = Class.forName(replicationEndpointImpl);
518       replicationEndpoint = (ReplicationEndpoint) c.newInstance();
519       if(rsServerHost != null) {
520         ReplicationEndpoint newReplicationEndPoint = rsServerHost
521             .postCreateReplicationEndPoint(replicationEndpoint);
522         if(newReplicationEndPoint != null) {
523           // Override the newly created endpoint from the hook with configured end point
524           replicationEndpoint = newReplicationEndPoint;
525         }
526       }
527     } catch (Exception e) {
528       LOG.warn("Passed replication endpoint implementation throws errors"
529           + " while initializing ReplicationSource for peer: " + peerId, e);
530       throw new IOException(e);
531     }
532 
533     MetricsSource metrics = new MetricsSource(peerId);
534     // init replication source
535     src.init(conf, fs, manager, replicationQueues, replicationPeers, server, peerId,
536       clusterId, replicationEndpoint, metrics);
537 
538     // init replication endpoint
539     replicationEndpoint.init(new ReplicationEndpoint.Context(
540       conf, replicationPeer.getConfiguration(),
541       fs, peerId, clusterId, replicationPeer, metrics, tableDescriptors, server));
542 
543     return src;
544   }
545 
546   /**
547    * Transfer all the queues of the specified to this region server.
548    * First it tries to grab a lock and if it works it will move the
549    * znodes and finally will delete the old znodes.
550    *
551    * It creates one old source for any type of source of the old rs.
552    * @param rsZnode znode for region server from where to transfer the queues
553    */
554   private void transferQueues(String rsZnode) {
555     NodeFailoverWorker transfer =
556         new NodeFailoverWorker(rsZnode, this.replicationQueues, this.replicationPeers,
557             this.clusterId);
558     try {
559       this.executor.execute(transfer);
560     } catch (RejectedExecutionException ex) {
561       CompatibilitySingletonFactory.getInstance(MetricsReplicationSourceFactory.class)
562         .getGlobalSource().incrFailedRecoveryQueue();
563       LOG.info("Cancelling the transfer of " + rsZnode + " because of " + ex.getMessage());
564     }
565   }
566 
567   /**
568    * Clear the references to the specified old source
569    * @param src source to clear
570    */
571   public void closeRecoveredQueue(ReplicationSourceInterface src) {
572     LOG.info("Done with the recovered queue " + src.getPeerClusterZnode());
573     src.getSourceMetrics().clear();
574     this.oldsources.remove(src);
575     deleteSource(src.getPeerClusterZnode(), false);
576     this.walsByIdRecoveredQueues.remove(src.getPeerClusterZnode());
577   }
578 
579   /**
580    * Clear the references to the specified old source
581    * @param src source to clear
582    */
583   public void closeQueue(ReplicationSourceInterface src) {
584     LOG.info("Done with the queue " + src.getPeerClusterZnode());
585     src.getSourceMetrics().clear();
586     this.sources.remove(src);
587     deleteSource(src.getPeerClusterZnode(), true);
588     this.walsById.remove(src.getPeerClusterZnode());
589   }
590 
591   /**
592    * This method first deletes all the recovered sources for the specified
593    * id, then deletes the normal source (deleting all related data in ZK).
594    * @param id The id of the peer cluster
595    */
596   public void removePeer(String id) {
597     LOG.info("Closing the following queue " + id + ", currently have "
598         + sources.size() + " and another "
599         + oldsources.size() + " that were recovered");
600     String terminateMessage = "Replication stream was removed by a user";
601     List<ReplicationSourceInterface> oldSourcesToDelete =
602         new ArrayList<ReplicationSourceInterface>();
603     // synchronized on oldsources to avoid adding recovered source for the to-be-removed peer
604     // see NodeFailoverWorker.run
605     synchronized (oldsources) {
606       // First close all the recovered sources for this peer
607       for (ReplicationSourceInterface src : oldsources) {
608         if (id.equals(src.getPeerClusterId())) {
609           oldSourcesToDelete.add(src);
610         }
611       }
612       for (ReplicationSourceInterface src : oldSourcesToDelete) {
613         src.terminate(terminateMessage);
614         closeRecoveredQueue(src);
615       }
616     }
617     LOG.info("Number of deleted recovered sources for " + id + ": "
618         + oldSourcesToDelete.size());
619     // Now look for the one on this cluster
620     List<ReplicationSourceInterface> srcToRemove = new ArrayList<ReplicationSourceInterface>();
621     // synchronize on replicationPeers to avoid adding source for the to-be-removed peer
622     synchronized (this.replicationPeers) {
623       for (ReplicationSourceInterface src : this.sources) {
624         if (id.equals(src.getPeerClusterId())) {
625           srcToRemove.add(src);
626         }
627       }
628       if (srcToRemove.isEmpty()) {
629         LOG.error("The peer we wanted to remove is missing a ReplicationSourceInterface. " +
630             "This could mean that ReplicationSourceInterface initialization failed for this peer " +
631             "and that replication on this peer may not be caught up. peerId=" + id);
632       }
633       for (ReplicationSourceInterface toRemove : srcToRemove) {
634         toRemove.terminate(terminateMessage);
635         closeQueue(toRemove);
636       }
637       deleteSource(id, true);
638     }
639   }
640 
641   @Override
642   public void regionServerRemoved(String regionserver) {
643     transferQueues(regionserver);
644   }
645 
646   /**
647    * We want to run the peer removal in a separate thread when the peer removal
648    * is called from ReplicationSource shipper thread on encountering NoNodeException
649    * because peerRemoved terminate the source which might leave replication source
650    * in orphaned state.
651    * See HBASE-25741.
652    * @param peerId peer ID to be removed.
653    */
654   private void schedulePeerRemoval(final String peerId) {
655     LOG.info(String.format("Scheduling an async peer removal for peer %s", peerId));
656     this.executor.submit(new Runnable() {
657       @Override public void run() {
658         peerRemoved(peerId);
659       }
660     });
661   }
662 
663   @Override
664   public void peerRemoved(String peerId) {
665     removePeer(peerId);
666     this.replicationQueues.removePeerFromHFileRefs(peerId);
667   }
668 
669   @Override
670   public void peerListChanged(List<String> peerIds) {
671     for (String id : peerIds) {
672       try {
673         boolean added = this.replicationPeers.peerAdded(id);
674         if (added) {
675           addSource(id);
676           if (replicationForBulkLoadDataEnabled) {
677             this.replicationQueues.addPeerToHFileRefs(id);
678           }
679         }
680       } catch (Exception e) {
681         LOG.error("Error while adding a new peer", e);
682       }
683     }
684   }
685 
686   Set<Path> getLastestPath() {
687     synchronized (latestPaths) {
688       return Sets.newHashSet(latestPaths.values());
689     }
690   }
691 
692   /**
693    * Class responsible to setup new ReplicationSources to take care of the
694    * queues from dead region servers.
695    */
696   class NodeFailoverWorker extends Thread {
697 
698     private String rsZnode;
699     private final ReplicationQueues rq;
700     private final ReplicationPeers rp;
701     private final UUID clusterId;
702 
703     /**
704      * @param rsZnode znode for dead region server
705      */
706     public NodeFailoverWorker(String rsZnode) {
707       this(rsZnode, replicationQueues, replicationPeers, ReplicationSourceManager.this.clusterId);
708     }
709 
710     public NodeFailoverWorker(String rsZnode, final ReplicationQueues replicationQueues,
711         final ReplicationPeers replicationPeers, final UUID clusterId) {
712       super("Failover-for-"+rsZnode);
713       this.rsZnode = rsZnode;
714       this.rq = replicationQueues;
715       this.rp = replicationPeers;
716       this.clusterId = clusterId;
717     }
718 
719     @Override
720     public void run() {
721       if (this.rq.isThisOurZnode(rsZnode)) {
722         return;
723       }
724       // Wait a bit before transferring the queues, we may be shutting down.
725       // This sleep may not be enough in some cases.
726       try {
727         Thread.sleep(sleepBeforeFailover + (long) (rand.nextFloat() * sleepBeforeFailover));
728       } catch (InterruptedException e) {
729         LOG.warn("Interrupted while waiting before transferring a queue.");
730         Thread.currentThread().interrupt();
731       }
732       // We try to lock that rs' queue directory
733       if (server.isStopped()) {
734         LOG.info("Not transferring queue since we are shutting down");
735         return;
736       }
737 
738       Map<String, SortedSet<String>> newQueues = new HashMap<>();
739       List<String> peers = rq.getUnClaimedQueueIds(rsZnode);
740       while (peers != null && !peers.isEmpty()) {
741         Pair<String, SortedSet<String>> peer = this.rq.claimQueue(rsZnode,
742             peers.get(rand.nextInt(peers.size())));
743         long sleep = sleepBeforeFailover/2;
744         if (peer != null) {
745           newQueues.put(peer.getFirst(), peer.getSecond());
746           sleep = sleepBeforeFailover;
747         }
748         try {
749           Thread.sleep(sleep);
750         } catch (InterruptedException e) {
751           LOG.warn("Interrupted while waiting before transferring a queue.");
752           Thread.currentThread().interrupt();
753         }
754         peers = rq.getUnClaimedQueueIds(rsZnode);
755       }
756       if (peers != null) {
757         rq.removeReplicatorIfQueueIsEmpty(rsZnode);
758       }
759 
760       // Copying over the failed queue is completed.
761       if (newQueues.isEmpty()) {
762         // We either didn't get the lock or the failed region server didn't have any outstanding
763         // WALs to replicate, so we are done.
764         return;
765       }
766 
767       for (Map.Entry<String, SortedSet<String>> entry : newQueues.entrySet()) {
768         String peerId = entry.getKey();
769         SortedSet<String> walsSet = entry.getValue();
770         try {
771           // there is not an actual peer defined corresponding to peerId for the failover.
772           ReplicationQueueInfo replicationQueueInfo = new ReplicationQueueInfo(peerId);
773           String actualPeerId = replicationQueueInfo.getPeerId();
774           ReplicationPeer peer = replicationPeers.getPeer(actualPeerId);
775           ReplicationPeerConfig peerConfig = null;
776           try {
777             peerConfig = replicationPeers.getReplicationPeerConfig(actualPeerId);
778           } catch (ReplicationException ex) {
779             LOG.warn("Received exception while getting replication peer config, skipping replay"
780                 + ex);
781           }
782           if (peer == null || peerConfig == null) {
783             LOG.warn("Skipping failover for peer:" + actualPeerId + " of node" + rsZnode);
784             replicationQueues.removeQueue(peerId);
785             continue;
786           }
787           if (server instanceof ReplicationSyncUp.DummyServer
788               && peer.getPeerState().equals(PeerState.DISABLED)) {
789             LOG.warn("Peer " + actualPeerId + " is disbaled. ReplicationSyncUp tool will skip "
790                 + "replicating data to this peer.");
791             continue;
792           }
793           // track sources in walsByIdRecoveredQueues
794           Map<String, SortedSet<String>> walsByGroup = new HashMap<String, SortedSet<String>>();
795           walsByIdRecoveredQueues.put(peerId, walsByGroup);
796           for (String wal : walsSet) {
797             String walPrefix = DefaultWALProvider.getWALPrefixFromWALName(wal);
798             SortedSet<String> wals = walsByGroup.get(walPrefix);
799             if (wals == null) {
800               wals = new TreeSet<String>();
801               walsByGroup.put(walPrefix, wals);
802             }
803             wals.add(wal);
804           }
805 
806           // enqueue sources
807           ReplicationSourceInterface src =
808               getReplicationSource(conf, fs, ReplicationSourceManager.this, this.rq, this.rp,
809                 server, peerId, this.clusterId, peerConfig, peer);
810           // synchronized on oldsources to avoid adding recovered source for the to-be-removed peer
811           // see removePeer
812           synchronized (oldsources) {
813             if (!this.rp.getPeerIds().contains(src.getPeerClusterId())) {
814               src.terminate("Recovered queue doesn't belong to any current peer");
815               closeRecoveredQueue(src);
816               continue;
817             }
818             oldsources.add(src);
819             for (String wal : walsSet) {
820               src.enqueueLog(new Path(oldLogDir, wal));
821             }
822             src.startup();
823           }
824         } catch (IOException e) {
825           // TODO manage it
826           LOG.error("Failed creating a source", e);
827         }
828       }
829     }
830   }
831 
832   /**
833    * Get the directory where wals are archived
834    * @return the directory where wals are archived
835    */
836   public Path getOldLogDir() {
837     return this.oldLogDir;
838   }
839 
840   /**
841    * Get the directory where wals are stored by their RSs
842    * @return the directory where wals are stored by their RSs
843    */
844   public Path getLogDir() {
845     return this.logDir;
846   }
847 
848   /**
849    * Get the handle on the local file system
850    * @return Handle on the local file system
851    */
852   public FileSystem getFs() {
853     return this.fs;
854   }
855 
856   /**
857    * Get the ReplicationPeers used by this ReplicationSourceManager
858    * @return the ReplicationPeers used by this ReplicationSourceManager
859    */
860   public ReplicationPeers getReplicationPeers() {
861     return this.replicationPeers;
862   }
863 
864   /**
865    * Get a string representation of all the sources' metrics
866    */
867   public String getStats() {
868     StringBuffer stats = new StringBuffer();
869     for (ReplicationSourceInterface source : sources) {
870       stats.append("Normal source for cluster " + source.getPeerClusterId() + ": ");
871       stats.append(source.getStats() + "\n");
872     }
873     for (ReplicationSourceInterface oldSource : oldsources) {
874       stats.append("Recovered source for cluster/machine(s) " + oldSource.getPeerClusterId()+": ");
875       stats.append(oldSource.getStats()+ "\n");
876     }
877     return stats.toString();
878   }
879 
880   public void addHFileRefs(TableName tableName, byte[] family, List<Pair<Path, Path>> pairs)
881       throws ReplicationException {
882     for (ReplicationSourceInterface source : this.sources) {
883       source.addHFileRefs(tableName, family, pairs);
884     }
885   }
886 
887   public void cleanUpHFileRefs(String peerId, List<String> files) {
888     this.replicationQueues.removeHFileRefs(peerId, files);
889   }
890 }