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