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