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