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