View Javadoc

1   /*
2    * Copyright 2010 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
19   */
20  package org.apache.hadoop.hbase.replication.regionserver;
21  
22  import java.io.EOFException;
23  import java.io.FileNotFoundException;
24  import java.io.IOException;
25  import java.util.ArrayList;
26  import java.util.Arrays;
27  import java.util.Comparator;
28  import java.util.HashSet;
29  import java.util.List;
30  import java.util.NavigableMap;
31  import java.util.Random;
32  import java.util.Set;
33  import java.util.concurrent.CountDownLatch;
34  import java.util.concurrent.PriorityBlockingQueue;
35  import java.util.concurrent.TimeUnit;
36  import java.util.concurrent.atomic.AtomicBoolean;
37  
38  import org.apache.commons.logging.Log;
39  import org.apache.commons.logging.LogFactory;
40  import org.apache.hadoop.conf.Configuration;
41  import org.apache.hadoop.fs.FileStatus;
42  import org.apache.hadoop.fs.FileSystem;
43  import org.apache.hadoop.fs.Path;
44  import org.apache.hadoop.hbase.HConstants;
45  import org.apache.hadoop.hbase.HServerAddress;
46  import org.apache.hadoop.hbase.KeyValue;
47  import org.apache.hadoop.hbase.Stoppable;
48  import org.apache.hadoop.hbase.client.HConnection;
49  import org.apache.hadoop.hbase.client.HConnectionManager;
50  import org.apache.hadoop.hbase.ipc.HRegionInterface;
51  import org.apache.hadoop.hbase.regionserver.wal.HLog;
52  import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
53  import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
54  import org.apache.hadoop.hbase.replication.ReplicationZookeeper;
55  import org.apache.hadoop.hbase.util.Bytes;
56  import org.apache.hadoop.hbase.util.Threads;
57  import org.apache.zookeeper.KeeperException;
58  
59  /**
60   * Class that handles the source of a replication stream.
61   * Currently does not handle more than 1 slave
62   * For each slave cluster it selects a random number of peers
63   * using a replication ratio. For example, if replication ration = 0.1
64   * and slave cluster has 100 region servers, 10 will be selected.
65   * <p/>
66   * A stream is considered down when we cannot contact a region server on the
67   * peer cluster for more than 55 seconds by default.
68   * <p/>
69   *
70   */
71  public class ReplicationSource extends Thread
72      implements ReplicationSourceInterface {
73  
74    private static final Log LOG = LogFactory.getLog(ReplicationSource.class);
75    // Queue of logs to process
76    private PriorityBlockingQueue<Path> queue;
77    // container of entries to replicate
78    private HLog.Entry[] entriesArray;
79    private HConnection conn;
80    // Helper class for zookeeper
81    private ReplicationZookeeper zkHelper;
82    private Configuration conf;
83    // ratio of region servers to chose from a slave cluster
84    private float ratio;
85    private Random random;
86    // should we replicate or not?
87    private AtomicBoolean replicating;
88    // id of the peer cluster this source replicates to
89    private String peerClusterId;
90    // The manager of all sources to which we ping back our progress
91    private ReplicationSourceManager manager;
92    // Should we stop everything?
93    private Stoppable stopper;
94    // List of chosen sinks (region servers)
95    private List<HServerAddress> currentPeers;
96    // How long should we sleep for each retry
97    private long sleepForRetries;
98    // Max size in bytes of entriesArray
99    private long replicationQueueSizeCapacity;
100   // Max number of entries in entriesArray
101   private int replicationQueueNbCapacity;
102   // Our reader for the current log
103   private HLog.Reader reader;
104   // Current position in the log
105   private long position = 0;
106   // Path of the current log
107   private volatile Path currentPath;
108   private FileSystem fs;
109   // id of this cluster
110   private byte clusterId;
111   // total number of edits we replicated
112   private long totalReplicatedEdits = 0;
113   // The znode we currently play with
114   private String peerClusterZnode;
115   // Indicates if this queue is recovered (and will be deleted when depleted)
116   private boolean queueRecovered;
117   // List of all the dead region servers that had this queue (if recovered)
118   private String[] deadRegionServers;
119   // Maximum number of retries before taking bold actions
120   private long maxRetriesMultiplier;
121   // Current number of entries that we need to replicate
122   private int currentNbEntries = 0;
123   // Current number of operations (Put/Delete) that we need to replicate
124   private int currentNbOperations = 0;
125   // Indicates if this particular source is running
126   private volatile boolean running = true;
127   // Metrics for this source
128   private ReplicationSourceMetrics metrics;
129   // If source is enabled, replication happens. If disabled, nothing will be
130   // replicated but HLogs will still be queued
131   private AtomicBoolean sourceEnabled = new AtomicBoolean();
132 
133   /**
134    * Instantiation method used by region servers
135    *
136    * @param conf configuration to use
137    * @param fs file system to use
138    * @param manager replication manager to ping to
139    * @param stopper     the atomic boolean to use to stop the regionserver
140    * @param replicating the atomic boolean that starts/stops replication
141    * @param peerClusterZnode the name of our znode
142    * @throws IOException
143    */
144   public void init(final Configuration conf,
145                    final FileSystem fs,
146                    final ReplicationSourceManager manager,
147                    final Stoppable stopper,
148                    final AtomicBoolean replicating,
149                    final String peerClusterZnode)
150       throws IOException {
151     this.stopper = stopper;
152     this.conf = conf;
153     this.replicationQueueSizeCapacity =
154         this.conf.getLong("replication.source.size.capacity", 1024*1024*64);
155     this.replicationQueueNbCapacity =
156         this.conf.getInt("replication.source.nb.capacity", 25000);
157     this.entriesArray = new HLog.Entry[this.replicationQueueNbCapacity];
158     for (int i = 0; i < this.replicationQueueNbCapacity; i++) {
159       this.entriesArray[i] = new HLog.Entry();
160     }
161     this.maxRetriesMultiplier =
162         this.conf.getLong("replication.source.maxretriesmultiplier", 10);
163     this.queue =
164         new PriorityBlockingQueue<Path>(
165             conf.getInt("hbase.regionserver.maxlogs", 32),
166             new LogsComparator());
167     this.conn = HConnectionManager.getConnection(conf);
168     this.zkHelper = manager.getRepZkWrapper();
169     this.ratio = this.conf.getFloat("replication.source.ratio", 0.1f);
170     this.currentPeers = new ArrayList<HServerAddress>();
171     this.random = new Random();
172     this.replicating = replicating;
173     this.manager = manager;
174     this.sleepForRetries =
175         this.conf.getLong("replication.source.sleepforretries", 1000);
176     this.fs = fs;
177     this.clusterId = Byte.valueOf(zkHelper.getClusterId());
178     this.metrics = new ReplicationSourceMetrics(peerClusterZnode);
179 
180     // Finally look if this is a recovered queue
181     this.checkIfQueueRecovered(peerClusterZnode);
182   }
183 
184   // The passed znode will be either the id of the peer cluster or
185   // the handling story of that queue in the form of id-servername-*
186   private void checkIfQueueRecovered(String peerClusterZnode) {
187     String[] parts = peerClusterZnode.split("-");
188     this.queueRecovered = parts.length != 1;
189     this.peerClusterId = this.queueRecovered ?
190         parts[0] : peerClusterZnode;
191     this.peerClusterZnode = peerClusterZnode;
192     this.deadRegionServers = new String[parts.length-1];
193     // Extract all the places where we could find the hlogs
194     for (int i = 1; i < parts.length; i++) {
195       this.deadRegionServers[i-1] = parts[i];
196     }
197   }
198 
199   /**
200    * Select a number of peers at random using the ratio. Mininum 1.
201    */
202   private void chooseSinks() throws KeeperException {
203     this.currentPeers.clear();
204     List<HServerAddress> addresses =
205         this.zkHelper.getSlavesAddresses(peerClusterId);
206     Set<HServerAddress> setOfAddr = new HashSet<HServerAddress>();
207     int nbPeers = (int) (Math.ceil(addresses.size() * ratio));
208     LOG.info("Getting " + nbPeers +
209         " rs from peer cluster # " + peerClusterId);
210     for (int i = 0; i < nbPeers; i++) {
211       HServerAddress address;
212       // Make sure we get one address that we don't already have
213       do {
214         address = addresses.get(this.random.nextInt(addresses.size()));
215       } while (setOfAddr.contains(address));
216       LOG.info("Choosing peer " + address);
217       setOfAddr.add(address);
218     }
219     this.currentPeers.addAll(setOfAddr);
220   }
221 
222   @Override
223   public void enqueueLog(Path log) {
224     this.queue.put(log);
225     this.metrics.sizeOfLogQueue.set(queue.size());
226   }
227 
228   @Override
229   public void run() {
230     connectToPeers();
231     // We were stopped while looping to connect to sinks, just abort
232     if (this.stopper.isStopped()) {
233       return;
234     }
235     // If this is recovered, the queue is already full and the first log
236     // normally has a position (unless the RS failed between 2 logs)
237     if (this.queueRecovered) {
238       try {
239         this.position = this.zkHelper.getHLogRepPosition(
240             this.peerClusterZnode, this.queue.peek().getName());
241       } catch (KeeperException e) {
242         this.terminate("Couldn't get the position of this recovered queue " +
243             peerClusterZnode, e);
244       }
245     }
246     int sleepMultiplier = 1;
247     // Loop until we close down
248     while (!stopper.isStopped() && this.running) {
249       // Sleep until replication is enabled again
250       if (!this.replicating.get() || !this.sourceEnabled.get()) {
251         if (sleepForRetries("Replication is disabled", sleepMultiplier)) {
252           sleepMultiplier++;
253         }
254         continue;
255       }
256       // Get a new path
257       if (!getNextPath()) {
258         if (sleepForRetries("No log to process", sleepMultiplier)) {
259           sleepMultiplier++;
260         }
261         continue;
262       }
263       // Open a reader on it
264       if (!openReader(sleepMultiplier)) {
265         // Reset the sleep multiplier, else it'd be reused for the next file
266         sleepMultiplier = 1;
267         continue;
268       }
269 
270       // If we got a null reader but didn't continue, then sleep and continue
271       if (this.reader == null) {
272         if (sleepForRetries("Unable to open a reader", sleepMultiplier)) {
273           sleepMultiplier++;
274         }
275         continue;
276       }
277 
278       boolean gotIOE = false;
279       currentNbEntries = 0;
280       try {
281         if(readAllEntriesToReplicateOrNextFile()) {
282           continue;
283         }
284       } catch (IOException ioe) {
285         LOG.warn(peerClusterZnode + " Got: ", ioe);
286         gotIOE = true;
287         if (ioe.getCause() instanceof EOFException) {
288 
289           boolean considerDumping = false;
290           if (this.queueRecovered) {
291             try {
292               FileStatus stat = this.fs.getFileStatus(this.currentPath);
293               if (stat.getLen() == 0) {
294                 LOG.warn(peerClusterZnode + " Got EOF and the file was empty");
295               }
296               considerDumping = true;
297             } catch (IOException e) {
298               LOG.warn(peerClusterZnode + " Got while getting file size: ", e);
299             }
300           } else if (currentNbEntries != 0) {
301             LOG.warn(peerClusterZnode + " Got EOF while reading, " +
302                 "looks like this file is broken? " + currentPath);
303             considerDumping = true;
304             currentNbEntries = 0;
305           }
306 
307           if (considerDumping &&
308               sleepMultiplier == this.maxRetriesMultiplier &&
309               processEndOfFile()) {
310             continue;
311           }
312         }
313       } finally {
314         try {
315           // if current path is null, it means we processEndOfFile hence
316           if (this.currentPath != null && !gotIOE) {
317             this.position = this.reader.getPosition();
318           }
319           if (this.reader != null) {
320             this.reader.close();
321           }
322         } catch (IOException e) {
323           gotIOE = true;
324           LOG.warn("Unable to finalize the tailing of a file", e);
325         }
326       }
327 
328       // If we didn't get anything to replicate, or if we hit a IOE,
329       // wait a bit and retry.
330       // But if we need to stop, don't bother sleeping
331       if (!stopper.isStopped() && (gotIOE || currentNbEntries == 0)) {
332         this.manager.logPositionAndCleanOldLogs(this.currentPath,
333             this.peerClusterZnode, this.position, queueRecovered);
334         if (sleepForRetries("Nothing to replicate", sleepMultiplier)) {
335           sleepMultiplier++;
336         }
337         continue;
338       }
339       sleepMultiplier = 1;
340       shipEdits();
341 
342     }
343     LOG.debug("Source exiting " + peerClusterId);
344   }
345 
346   /**
347    * Read all the entries from the current log files and retain those
348    * that need to be replicated. Else, process the end of the current file.
349    * @return true if we got nothing and went to the next file, false if we got
350    * entries
351    * @throws IOException
352    */
353   protected boolean readAllEntriesToReplicateOrNextFile() throws IOException{
354     long seenEntries = 0;
355     if (this.position != 0) {
356       this.reader.seek(this.position);
357     }
358     HLog.Entry entry = this.reader.next(this.entriesArray[currentNbEntries]);
359     while (entry != null) {
360       WALEdit edit = entry.getEdit();
361       this.metrics.logEditsReadRate.inc(1);
362       seenEntries++;
363       // Remove all KVs that should not be replicated
364       removeNonReplicableEdits(edit);
365       HLogKey logKey = entry.getKey();
366       // Don't replicate catalog entries, if the WALEdit wasn't
367       // containing anything to replicate and if we're currently not set to replicate
368       if (!(Bytes.equals(logKey.getTablename(), HConstants.ROOT_TABLE_NAME) ||
369           Bytes.equals(logKey.getTablename(), HConstants.META_TABLE_NAME)) &&
370           edit.size() != 0 && replicating.get()) {
371         logKey.setClusterId(this.clusterId);
372         currentNbOperations += countDistinctRowKeys(edit);
373         currentNbEntries++;
374       } else {
375         this.metrics.logEditsFilteredRate.inc(1);
376       }
377       // Stop if too many entries or too big
378       if ((this.reader.getPosition() - this.position)
379           >= this.replicationQueueSizeCapacity ||
380           currentNbEntries >= this.replicationQueueNbCapacity) {
381         break;
382       }
383       entry = this.reader.next(entriesArray[currentNbEntries]);
384     }
385     LOG.debug("currentNbOperations:" + currentNbOperations +
386         " and seenEntries:" + seenEntries +
387         " and size: " + (this.reader.getPosition() - this.position));
388     // If we didn't get anything and the queue has an object, it means we
389     // hit the end of the file for sure
390     return seenEntries == 0 && processEndOfFile();
391   }
392 
393   private void connectToPeers() {
394     // Connect to peer cluster first, unless we have to stop
395     while (!this.stopper.isStopped() && this.currentPeers.size() == 0) {
396       try {
397         chooseSinks();
398         Thread.sleep(this.sleepForRetries);
399       } catch (InterruptedException e) {
400         LOG.error("Interrupted while trying to connect to sinks", e);
401       } catch (KeeperException e) {
402         LOG.error("Error talking to zookeeper, retrying", e);
403       }
404     }
405   }
406 
407   /**
408    * Poll for the next path
409    * @return true if a path was obtained, false if not
410    */
411   protected boolean getNextPath() {
412     try {
413       if (this.currentPath == null) {
414         this.currentPath = queue.poll(this.sleepForRetries, TimeUnit.MILLISECONDS);
415         this.metrics.sizeOfLogQueue.set(queue.size());
416       }
417     } catch (InterruptedException e) {
418       LOG.warn("Interrupted while reading edits", e);
419     }
420     return this.currentPath != null;
421   }
422 
423   /**
424    * Open a reader on the current path
425    *
426    * @param sleepMultiplier by how many times the default sleeping time is augmented
427    * @return true if we should continue with that file, false if we are over with it
428    */
429   protected boolean openReader(int sleepMultiplier) {
430     try {
431       LOG.debug("Opening log for replication " + this.currentPath.getName() +
432           " at " + this.position);
433       try {
434        this.reader = null;
435        this.reader = HLog.getReader(this.fs, this.currentPath, this.conf);
436       } catch (FileNotFoundException fnfe) {
437         if (this.queueRecovered) {
438           // We didn't find the log in the archive directory, look if it still
439           // exists in the dead RS folder (there could be a chain of failures
440           // to look at)
441           LOG.info("NB dead servers : " + deadRegionServers.length);
442           for (int i = this.deadRegionServers.length - 1; i >= 0; i--) {
443 
444             Path deadRsDirectory =
445                 new Path(manager.getLogDir().getParent(), this.deadRegionServers[i]);
446             Path possibleLogLocation =
447                 new Path(deadRsDirectory, currentPath.getName());
448             LOG.info("Possible location " + possibleLogLocation.toUri().toString());
449             if (this.manager.getFs().exists(possibleLogLocation)) {
450               // We found the right new location
451               LOG.info("Log " + this.currentPath + " still exists at " +
452                   possibleLogLocation);
453               // Breaking here will make us sleep since reader is null
454               return true;
455             }
456           }
457           // TODO What happens if the log was missing from every single location?
458           // Although we need to check a couple of times as the log could have
459           // been moved by the master between the checks
460           // It can also happen if a recovered queue wasn't properly cleaned,
461           // such that the znode pointing to a log exists but the log was
462           // deleted a long time ago.
463           // For the moment, we'll throw the IO and processEndOfFile
464           throw new IOException("File from recovered queue is " +
465               "nowhere to be found", fnfe);
466         } else {
467           // If the log was archived, continue reading from there
468           Path archivedLogLocation =
469               new Path(manager.getOldLogDir(), currentPath.getName());
470           if (this.manager.getFs().exists(archivedLogLocation)) {
471             currentPath = archivedLogLocation;
472             LOG.info("Log " + this.currentPath + " was moved to " +
473                 archivedLogLocation);
474             // Open the log at the new location
475             this.openReader(sleepMultiplier);
476 
477           }
478           // TODO What happens the log is missing in both places?
479         }
480       }
481     } catch (IOException ioe) {
482       LOG.warn(peerClusterZnode + " Got: ", ioe);
483       // TODO Need a better way to determinate if a file is really gone but
484       // TODO without scanning all logs dir
485       if (sleepMultiplier == this.maxRetriesMultiplier) {
486         LOG.warn("Waited too long for this file, considering dumping");
487         return !processEndOfFile();
488       }
489     }
490     return true;
491   }
492 
493   /**
494    * Do the sleeping logic
495    * @param msg Why we sleep
496    * @param sleepMultiplier by how many times the default sleeping time is augmented
497    * @return True if <code>sleepMultiplier</code> is &lt; <code>maxRetriesMultiplier</code>
498    */
499   protected boolean sleepForRetries(String msg, int sleepMultiplier) {
500     try {
501       LOG.debug(msg + ", sleeping " + sleepForRetries + " times " + sleepMultiplier);
502       Thread.sleep(this.sleepForRetries * sleepMultiplier);
503     } catch (InterruptedException e) {
504       LOG.debug("Interrupted while sleeping between retries");
505     }
506     return sleepMultiplier < maxRetriesMultiplier;
507   }
508 
509   /**
510    * We only want KVs that are scoped other than local
511    * @param edit The KV to check for replication
512    */
513   protected void removeNonReplicableEdits(WALEdit edit) {
514     NavigableMap<byte[], Integer> scopes = edit.getScopes();
515     List<KeyValue> kvs = edit.getKeyValues();
516     for (int i = 0; i < edit.size(); i++) {
517       KeyValue kv = kvs.get(i);
518       // The scope will be null or empty if
519       // there's nothing to replicate in that WALEdit
520       if (scopes == null || !scopes.containsKey(kv.getFamily())) {
521         kvs.remove(i);
522         i--;
523       }
524     }
525   }
526 
527   /**
528    * Count the number of different row keys in the given edit because of
529    * mini-batching. We assume that there's at least one KV in the WALEdit.
530    * @param edit edit to count row keys from
531    * @return number of different row keys
532    */
533   private int countDistinctRowKeys(WALEdit edit) {
534     List<KeyValue> kvs = edit.getKeyValues();
535     int distinctRowKeys = 1;
536     KeyValue lastKV = kvs.get(0);
537     for (int i = 0; i < edit.size(); i++) {
538       if (!kvs.get(i).matchingRow(lastKV)) {
539         distinctRowKeys++;
540       }
541     }
542     return distinctRowKeys;
543   }
544 
545   /**
546    * Do the shipping logic
547    */
548   protected void shipEdits() {
549     int sleepMultiplier = 1;
550     if (this.currentNbEntries == 0) {
551       LOG.warn("Was given 0 edits to ship");
552       return;
553     }
554     while (!this.stopper.isStopped()) {
555       try {
556         HRegionInterface rrs = getRS();
557         LOG.debug("Replicating " + currentNbEntries);
558         rrs.replicateLogEntries(Arrays.copyOf(this.entriesArray, currentNbEntries));
559         this.manager.logPositionAndCleanOldLogs(this.currentPath,
560             this.peerClusterZnode, this.position, queueRecovered);
561         this.totalReplicatedEdits += currentNbEntries;
562         this.metrics.shippedBatchesRate.inc(1);
563         this.metrics.shippedOpsRate.inc(
564             this.currentNbOperations);
565         this.metrics.setAgeOfLastShippedOp(
566             this.entriesArray[this.entriesArray.length-1].getKey().getWriteTime());
567         LOG.debug("Replicated in total: " + this.totalReplicatedEdits);
568         break;
569 
570       } catch (IOException ioe) {
571         LOG.warn("Unable to replicate because ", ioe);
572         try {
573           boolean down;
574           do {
575             down = isSlaveDown();
576             if (down) {
577               LOG.debug("The region server we tried to ping didn't answer, " +
578                   "sleeping " + sleepForRetries + " times " + sleepMultiplier);
579               Thread.sleep(this.sleepForRetries * sleepMultiplier);
580               if (sleepMultiplier < maxRetriesMultiplier) {
581                 sleepMultiplier++;
582               } else {
583                 chooseSinks();
584               }
585             }
586           } while (!this.stopper.isStopped() && down);
587         } catch (InterruptedException e) {
588           LOG.debug("Interrupted while trying to contact the peer cluster");
589         } catch (KeeperException e) {
590           LOG.error("Error talking to zookeeper, retrying", e);
591         }
592 
593       }
594     }
595   }
596 
597   /**
598    * If the queue isn't empty, switch to the next one
599    * Else if this is a recovered queue, it means we're done!
600    * Else we'll just continue to try reading the log file
601    * @return true if we're done with the current file, false if we should
602    * continue trying to read from it
603    */
604   protected boolean processEndOfFile() {
605     if (this.queue.size() != 0) {
606       this.currentPath = null;
607       this.position = 0;
608       return true;
609     } else if (this.queueRecovered) {
610       this.manager.closeRecoveredQueue(this);
611       LOG.info("Finished recovering the queue");
612       this.running = false;
613       return true;
614     }
615     return false;
616   }
617 
618   public void startup() {
619     String n = Thread.currentThread().getName();
620     Thread.UncaughtExceptionHandler handler =
621         new Thread.UncaughtExceptionHandler() {
622           public void uncaughtException(final Thread t, final Throwable e) {
623             terminate("Uncaught exception during runtime", new Exception(e));
624           }
625         };
626     Threads.setDaemonThreadRunning(
627         this, n + ".replicationSource," + peerClusterZnode, handler);
628   }
629 
630   public void terminate(String reason) {
631     terminate(reason, null);
632   }
633 
634   public void terminate(String reason, Exception cause) {
635     if (cause == null) {
636       LOG.info("Closing source "
637           + this.peerClusterZnode + " because: " + reason);
638 
639     } else {
640       LOG.error("Closing source " + this.peerClusterZnode
641           + " because an error occurred: " + reason, cause);
642     }
643     this.running = false;
644     Threads.shutdown(this, this.sleepForRetries);
645   }
646 
647   /**
648    * Get a new region server at random from this peer
649    * @return
650    * @throws IOException
651    */
652   private HRegionInterface getRS() throws IOException {
653     if (this.currentPeers.size() == 0) {
654       throw new IOException(this.peerClusterZnode + " has 0 region servers");
655     }
656     HServerAddress address =
657         currentPeers.get(random.nextInt(this.currentPeers.size()));
658     return this.conn.getHRegionConnection(address);
659   }
660 
661   /**
662    * Check if the slave is down by trying to establish a connection
663    * @return true if down, false if up
664    * @throws InterruptedException
665    */
666   public boolean isSlaveDown() throws InterruptedException {
667     final CountDownLatch latch = new CountDownLatch(1);
668     Thread pingThread = new Thread() {
669       public void run() {
670         try {
671           HRegionInterface rrs = getRS();
672           // Dummy call which should fail
673           rrs.getHServerInfo();
674           latch.countDown();
675         } catch (IOException ex) {
676           LOG.info("Slave cluster looks down: " + ex.getMessage());
677         }
678       }
679     };
680     pingThread.start();
681     // awaits returns true if countDown happened
682     boolean down = ! latch.await(this.sleepForRetries, TimeUnit.MILLISECONDS);
683     pingThread.interrupt();
684     return down;
685   }
686 
687   public String getPeerClusterZnode() {
688     return this.peerClusterZnode;
689   }
690 
691   public String getPeerClusterId() {
692     return this.peerClusterId;
693   }
694 
695   public Path getCurrentPath() {
696     return this.currentPath;
697   }
698 
699   public void setSourceEnabled(boolean status) {
700     this.sourceEnabled.set(status);
701   }
702 
703   /**
704    * Comparator used to compare logs together based on their start time
705    */
706   public static class LogsComparator implements Comparator<Path> {
707 
708     @Override
709     public int compare(Path o1, Path o2) {
710       return Long.valueOf(getTS(o1)).compareTo(getTS(o2));
711     }
712 
713     @Override
714     public boolean equals(Object o) {
715       return true;
716     }
717 
718     /**
719      * Split a path to get the start time
720      * For example: 10.20.20.171%3A60020.1277499063250
721      * @param p path to split
722      * @return start time
723      */
724     private long getTS(Path p) {
725       String[] parts = p.getName().split("\\.");
726       return Long.parseLong(parts[parts.length-1]);
727     }
728   }
729 }