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