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 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   * This class is responsible to manage all the replication
66   * sources. There are two classes of sources:
67   * <li> Normal sources are persistent and one per peer cluster</li>
68   * <li> Old sources are recovered from a failed region server and our
69   * only goal is to finish replicating the WAL queue it had up in ZK</li>
70   *
71   * When a region server dies, this class uses a watcher to get notified and it
72   * tries to grab a lock in order to transfer all the queues in a local
73   * old source.
74   *
75   * This class implements the ReplicationListener interface so that it can track changes in
76   * replication state.
77   */
78  @InterfaceAudience.Private
79  public class ReplicationSourceManager implements ReplicationListener {
80    private static final Log LOG =
81        LogFactory.getLog(ReplicationSourceManager.class);
82    // List of all the sources that read this RS's logs
83    private final List<ReplicationSourceInterface> sources;
84    // List of all the sources we got from died RSs
85    private final List<ReplicationSourceInterface> oldsources;
86    private final ReplicationQueues replicationQueues;
87    private final ReplicationTracker replicationTracker;
88    private final ReplicationPeers replicationPeers;
89    // UUID for this cluster
90    private final UUID clusterId;
91    // All about stopping
92    private final Server server;
93    // All logs we are currently tracking
94    private final Map<String, SortedSet<String>> walsById;
95    // Logs for recovered sources we are currently tracking
96    private final Map<String, SortedSet<String>> walsByIdRecoveredQueues;
97    private final Configuration conf;
98    private final FileSystem fs;
99    // The path to the latest log we saw, for new coming sources
100   private Path latestPath;
101   // Path to the wals directories
102   private final Path logDir;
103   // Path to the wal archive
104   private final Path oldLogDir;
105   // The number of ms that we wait before moving znodes, HBASE-3596
106   private final long sleepBeforeFailover;
107   // Homemade executer service for replication
108   private final ThreadPoolExecutor executor;
109 
110   private final Random rand;
111 
112 
113   /**
114    * Creates a replication manager and sets the watch on all the other registered region servers
115    * @param replicationQueues the interface for manipulating replication queues
116    * @param replicationPeers
117    * @param replicationTracker
118    * @param conf the configuration to use
119    * @param server the server for this region server
120    * @param fs the file system to use
121    * @param logDir the directory that contains all wal directories of live RSs
122    * @param oldLogDir the directory where old logs are archived
123    * @param clusterId
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     //CopyOnWriteArrayList is thread-safe.
130     //Generally, reading is more than modifying.
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); // 30 seconds
145     this.clusterId = clusterId;
146     this.replicationTracker.registerListener(this);
147     this.replicationPeers.getAllPeerIds();
148     // It's preferable to failover 1 RS at a time, but with good zk servers
149     // more could be processed at the same time.
150     int nbWorkers = conf.getInt("replication.executor.workers", 1);
151     // use a short 100ms sleep since this could be done inline with a RS startup
152     // even if we fail, other region servers can take care of it
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    * Provide the id of the peer and a log key and this method will figure which
165    * wal it belongs to and will log, for this region server, the current
166    * position. It will also clean old logs from the queue.
167    * @param log Path to the log currently being replicated from
168    * replication status in zookeeper. It will also delete older entries.
169    * @param id id of the peer cluster
170    * @param position current location in the log
171    * @param queueRecovered indicates if this queue comes from another region server
172    * @param holdLogInZK if true then the log is retained in ZK
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    * Cleans a log file and all older files from ZK. Called when we are sure that a
186    * log file is closed and has no more entries.
187    * @param key Path to the log
188    * @param id id of the peer cluster
189    * @param queueRecovered Whether this is a recovered queue
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    * Adds a normal source per registered peer cluster and tries to process all
218    * old region server wal queues
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         // Check if peer exists in hfile-refs queue, if not add it. This can happen in the case
228         // when a peer was added before replication for bulk loaded data was enabled.
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     // Look if there's anything to process after a restart
241     for (String rs : currentReplicators) {
242       if (!otherRegionServers.contains(rs)) {
243         transferQueues(rs);
244       }
245     }
246   }
247 
248   /**
249    * Add a new normal source to this region server
250    * @param id the id of the peer cluster
251    * @return the source that was created
252    * @throws IOException
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       // Add the latest wal to that source's queue
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    * Delete a complete queue of wals associated with a peer cluster
287    * @param peerId Id of the peer cluster queue of wals to delete
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    * Terminate the replication on this region server
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    * Get a copy of the wals of the first source on this rs
311    * @return a sorted set of wal names
312    */
313   protected Map<String, SortedSet<String>> getWALs() {
314     return Collections.unmodifiableMap(walsById);
315   }
316 
317   /**
318    * Get a copy of the wals of the recovered sources on this rs
319    * @return a sorted set of wal names
320    */
321   protected Map<String, SortedSet<String>> getWalsByIdRecoveredQueues() {
322     return Collections.unmodifiableMap(walsByIdRecoveredQueues);
323   }
324 
325   /**
326    * Get a list of all the normal sources of this rs
327    * @return lis of all sources
328    */
329   public List<ReplicationSourceInterface> getSources() {
330     return this.sources;
331   }
332 
333   /**
334    * Get a list of all the old sources of this rs
335    * @return list of all old sources
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           // If there's no slaves, don't need to keep the old wals since
355           // we only consider the last one when a new slave comes in
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     // This only updates the sources we own, not the recovered ones
367     for (ReplicationSourceInterface source : this.sources) {
368       source.enqueueLog(newLog);
369     }
370   }
371 
372   /**
373    * Factory method to create a replication source
374    * @param conf the configuration to use
375    * @param fs the file system to use
376    * @param manager the manager to use
377    * @param server the server object for this region server
378    * @param peerId the id of the peer cluster
379    * @return the created source
380    * @throws IOException
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         // Default to HBase inter-cluster replication endpoint
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           // Override the newly created endpoint from the hook with configured end point
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     // init replication source
431     src.init(conf, fs, manager, replicationQueues, replicationPeers, server, peerId,
432       clusterId, replicationEndpoint, metrics);
433 
434     // init replication endpoint
435     replicationEndpoint.init(new ReplicationEndpoint.Context(replicationPeer.getConfiguration(),
436       fs, peerConfig, peerId, clusterId, replicationPeer, metrics, tableDescriptors));
437 
438     return src;
439   }
440 
441   /**
442    * Transfer all the queues of the specified to this region server.
443    * First it tries to grab a lock and if it works it will move the
444    * znodes and finally will delete the old znodes.
445    *
446    * It creates one old source for any type of source of the old rs.
447    * @param rsZnode
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    * Clear the references to the specified old source
462    * @param src source to clear
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    * Thie method first deletes all the recovered sources for the specified
473    * id, then deletes the normal source (deleting all related data in ZK).
474    * @param id The id of the peer cluster
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     // First close all the recovered sources for this peer
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     // Now look for the one on this cluster
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    * Class responsible to setup new ReplicationSources to take care of the
538    * queues from dead region servers.
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      * @param rsZnode
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       // Wait a bit before transferring the queues, we may be shutting down.
566       // This sleep may not be enough in some cases.
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       // We try to lock that rs' queue directory
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       // Copying over the failed queue is completed.
583       if (newQueues.isEmpty()) {
584         // We either didn't get the lock or the failed region server didn't have any outstanding
585         // WALs to replicate, so we are done.
586         return;
587       }
588 
589       for (Map.Entry<String, SortedSet<String>> entry : newQueues.entrySet()) {
590         String peerId = entry.getKey();
591         try {
592           // there is not an actual peer defined corresponding to peerId for the failover.
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           // TODO manage it
624           LOG.error("Failed creating a source", e);
625         }
626       }
627     }
628   }
629 
630   /**
631    * Get the directory where wals are archived
632    * @return the directory where wals are archived
633    */
634   public Path getOldLogDir() {
635     return this.oldLogDir;
636   }
637 
638   /**
639    * Get the directory where wals are stored by their RSs
640    * @return the directory where wals are stored by their RSs
641    */
642   public Path getLogDir() {
643     return this.logDir;
644   }
645 
646   /**
647    * Get the handle on the local file system
648    * @return Handle on the local file system
649    */
650   public FileSystem getFs() {
651     return this.fs;
652   }
653 
654   /**
655    * Get a string representation of all the sources' metrics
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 }