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 = this.conf.getInt("replication.source.maxretriesmultiplier", 10);
175     this.socketTimeoutMultiplier = this.conf.getInt("replication.source.socketTimeoutMultiplier",
176         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       } else if (oldPath != null && !oldPath.getName().equals(getCurrentPath().getName())) {
356         this.manager.cleanOldLogs(getCurrentPath().getName(), this.peerId, this.queueRecovered);
357       }
358       boolean currentWALisBeingWrittenTo = false;
359       //For WAL files we own (rather than recovered), take a snapshot of whether the
360       //current WAL file (this.currentPath) is in use (for writing) NOW!
361       //Since the new WAL paths are enqueued only after the prev WAL file
362       //is 'closed', presence of an element in the queue means that
363       //the previous WAL file was closed, else the file is in use (currentPath)
364       //We take the snapshot now so that we are protected against races
365       //where a new file gets enqueued while the current file is being processed
366       //(and where we just finished reading the current file).
367       if (!this.queueRecovered && queue.size() == 0) {
368         currentWALisBeingWrittenTo = true;
369       }
370       // Open a reader on it
371       if (!openReader(sleepMultiplier)) {
372         // Reset the sleep multiplier, else it'd be reused for the next file
373         sleepMultiplier = 1;
374         continue;
375       }
376 
377       // If we got a null reader but didn't continue, then sleep and continue
378       if (this.reader == null) {
379         if (sleepForRetries("Unable to open a reader", sleepMultiplier)) {
380           sleepMultiplier++;
381         }
382         continue;
383       }
384 
385       boolean gotIOE = false;
386       currentNbOperations = 0;
387       currentNbEntries = 0;
388       currentSize = 0;
389       try {
390         if (readAllEntriesToReplicateOrNextFile(currentWALisBeingWrittenTo)) {
391           continue;
392         }
393       } catch (IOException ioe) {
394         LOG.warn(peerClusterZnode + " Got: ", ioe);
395         gotIOE = true;
396         if (ioe.getCause() instanceof EOFException) {
397 
398           boolean considerDumping = false;
399           if (this.queueRecovered) {
400             try {
401               FileStatus stat = this.fs.getFileStatus(this.currentPath);
402               if (stat.getLen() == 0) {
403                 LOG.warn(peerClusterZnode + " Got EOF and the file was empty");
404               }
405               considerDumping = true;
406             } catch (IOException e) {
407               LOG.warn(peerClusterZnode + " Got while getting file size: ", e);
408             }
409           } else if (currentNbEntries != 0) {
410             LOG.warn(peerClusterZnode + " Got EOF while reading, " +
411                 "looks like this file is broken? " + currentPath);
412             considerDumping = true;
413             currentNbEntries = 0;
414           }
415 
416           if (considerDumping &&
417               sleepMultiplier == this.maxRetriesMultiplier &&
418               processEndOfFile()) {
419             continue;
420           }
421         }
422       } finally {
423         try {
424           this.reader = null;
425           this.repLogReader.closeReader();
426         } catch (IOException e) {
427           gotIOE = true;
428           LOG.warn("Unable to finalize the tailing of a file", e);
429         }
430       }
431 
432       // If we didn't get anything to replicate, or if we hit a IOE,
433       // wait a bit and retry.
434       // But if we need to stop, don't bother sleeping
435       if (this.isActive() && (gotIOE || currentNbEntries == 0)) {
436         if (this.lastLoggedPosition != this.repLogReader.getPosition()) {
437           this.manager.logPositionAndCleanOldLogs(this.currentPath,
438               this.peerClusterZnode, this.repLogReader.getPosition(), queueRecovered, currentWALisBeingWrittenTo);
439           this.lastLoggedPosition = this.repLogReader.getPosition();
440         }
441         if (sleepForRetries("Nothing to replicate", sleepMultiplier)) {
442           sleepMultiplier++;
443         }
444         continue;
445       }
446       sleepMultiplier = 1;
447       shipEdits(currentWALisBeingWrittenTo);
448 
449     }
450     if (this.conn != null) {
451       try {
452         this.conn.close();
453       } catch (IOException e) {
454         LOG.debug("Attempt to close connection failed", e);
455       }
456     }
457     LOG.debug("Source exiting " + peerId);
458   }
459 
460   /**
461    * Read all the entries from the current log files and retain those
462    * that need to be replicated. Else, process the end of the current file.
463    * @param currentWALisBeingWrittenTo is the current WAL being written to
464    * @return true if we got nothing and went to the next file, false if we got
465    * entries
466    * @throws IOException
467    */
468   protected boolean readAllEntriesToReplicateOrNextFile(boolean currentWALisBeingWrittenTo)
469       throws IOException{
470     long seenEntries = 0;
471     this.repLogReader.seek();
472     HLog.Entry entry =
473         this.repLogReader.readNextAndSetPosition(this.entriesArray, this.currentNbEntries);
474     while (entry != null) {
475       WALEdit edit = entry.getEdit();
476       this.metrics.logEditsReadRate.inc(1);
477       seenEntries++;
478       // Remove all KVs that should not be replicated
479       HLogKey logKey = entry.getKey();
480       // don't replicate if the log entries originated in the peer
481       if (!logKey.getClusterId().equals(peerClusterId)) {
482         removeNonReplicableEdits(edit);
483         // Don't replicate catalog entries, if the WALEdit wasn't
484         // containing anything to replicate and if we're currently not set to replicate
485         if (!(Bytes.equals(logKey.getTablename(), HConstants.ROOT_TABLE_NAME) ||
486             Bytes.equals(logKey.getTablename(), HConstants.META_TABLE_NAME)) &&
487             edit.size() != 0 && replicating.get()) {
488           // Only set the clusterId if is a local key.
489           // This ensures that the originator sets the cluster id
490           // and all replicas retain the initial cluster id.
491           // This is *only* place where a cluster id other than the default is set.
492           if (HConstants.DEFAULT_CLUSTER_ID == logKey.getClusterId()) {
493             logKey.setClusterId(this.clusterId);
494           }
495           currentNbOperations += countDistinctRowKeys(edit);
496           currentNbEntries++;
497           currentSize += entry.getEdit().heapSize();
498         } else {
499           this.metrics.logEditsFilteredRate.inc(1);
500         }
501       }
502       // Stop if too many entries or too big
503       if (currentSize >= this.replicationQueueSizeCapacity ||
504           currentNbEntries >= this.replicationQueueNbCapacity) {
505         break;
506       }
507       try {
508         entry = this.repLogReader.readNextAndSetPosition(this.entriesArray, this.currentNbEntries);
509       } catch (IOException ie) {
510         LOG.debug("Break on IOE: " + ie.getMessage());
511         break;
512       }
513     }
514     LOG.debug("currentNbOperations:" + currentNbOperations +
515         " and seenEntries:" + seenEntries +
516         " and size: " + this.currentSize);
517     if (currentWALisBeingWrittenTo) {
518       return false;
519     }
520     // If we didn't get anything and the queue has an object, it means we
521     // hit the end of the file for sure
522     return seenEntries == 0 && processEndOfFile();
523   }
524 
525   private void connectToPeers() {
526     int sleepMultiplier = 1;
527 
528     // Connect to peer cluster first, unless we have to stop
529     while (this.isActive() && this.currentPeers.size() == 0) {
530 
531       chooseSinks();
532       if (this.isActive() && this.currentPeers.size() == 0) {
533         if (sleepForRetries("Waiting for peers", sleepMultiplier)) {
534           sleepMultiplier++;
535         }
536       }
537     }
538   }
539 
540   /**
541    * Poll for the next path
542    * @return true if a path was obtained, false if not
543    */
544   protected boolean getNextPath() {
545     try {
546       if (this.currentPath == null) {
547         this.currentPath = queue.poll(this.sleepForRetries, TimeUnit.MILLISECONDS);
548         this.metrics.sizeOfLogQueue.set(queue.size());
549       }
550     } catch (InterruptedException e) {
551       LOG.warn("Interrupted while reading edits", e);
552     }
553     return this.currentPath != null;
554   }
555 
556   /**
557    * Open a reader on the current path
558    *
559    * @param sleepMultiplier by how many times the default sleeping time is augmented
560    * @return true if we should continue with that file, false if we are over with it
561    */
562   protected boolean openReader(int sleepMultiplier) {
563     try {
564       LOG.debug("Opening log for replication " + this.currentPath.getName() +
565           " at " + this.repLogReader.getPosition());
566       try {
567         this.reader = repLogReader.openReader(this.currentPath);
568       } catch (FileNotFoundException fnfe) {
569         if (this.queueRecovered) {
570           // We didn't find the log in the archive directory, look if it still
571           // exists in the dead RS folder (there could be a chain of failures
572           // to look at)
573           LOG.info("NB dead servers : " + deadRegionServers.size());
574           for (String curDeadServerName : deadRegionServers) {
575             Path deadRsDirectory =
576                 new Path(manager.getLogDir().getParent(), curDeadServerName);
577             Path[] locs = new Path[] {
578                 new Path(deadRsDirectory, currentPath.getName()),
579                 new Path(deadRsDirectory.suffix(HLog.SPLITTING_EXT),
580                                           currentPath.getName()),
581             };
582             for (Path possibleLogLocation : locs) {
583               LOG.info("Possible location " + possibleLogLocation.toUri().toString());
584               if (this.manager.getFs().exists(possibleLogLocation)) {
585                 // We found the right new location
586                 LOG.info("Log " + this.currentPath + " still exists at " +
587                     possibleLogLocation);
588                 // Breaking here will make us sleep since reader is null
589                 return true;
590               }
591             }
592           }
593           // TODO What happens if the log was missing from every single location?
594           // Although we need to check a couple of times as the log could have
595           // been moved by the master between the checks
596           // It can also happen if a recovered queue wasn't properly cleaned,
597           // such that the znode pointing to a log exists but the log was
598           // deleted a long time ago.
599           // For the moment, we'll throw the IO and processEndOfFile
600           throw new IOException("File from recovered queue is " +
601               "nowhere to be found", fnfe);
602         } else {
603           // If the log was archived, continue reading from there
604           Path archivedLogLocation =
605               new Path(manager.getOldLogDir(), currentPath.getName());
606           if (this.manager.getFs().exists(archivedLogLocation)) {
607             currentPath = archivedLogLocation;
608             LOG.info("Log " + this.currentPath + " was moved to " +
609                 archivedLogLocation);
610             // Open the log at the new location
611             this.openReader(sleepMultiplier);
612 
613           }
614           // TODO What happens the log is missing in both places?
615         }
616       }
617     } catch (IOException ioe) {
618       if (ioe instanceof EOFException && isCurrentLogEmpty()) return true;
619       LOG.warn(peerClusterZnode + " Got: ", ioe);
620       this.reader = null;
621       if (ioe.getCause() instanceof NullPointerException) {
622         // Workaround for race condition in HDFS-4380
623         // which throws a NPE if we open a file before any data node has the most recent block
624         // Just sleep and retry.  Will require re-reading compressed HLogs for compressionContext.
625         LOG.warn("Got NPE opening reader, will retry.");
626       } else if (sleepMultiplier == this.maxRetriesMultiplier) {
627         // TODO Need a better way to determine if a file is really gone but
628         // TODO without scanning all logs dir  
629         LOG.warn("Waited too long for this file, considering dumping");
630         return !processEndOfFile();
631       }
632     }
633     return true;
634   }
635 
636   /*
637    * Checks whether the current log file is empty, and it is not a recovered queue. This is to
638    * handle scenario when in an idle cluster, there is no entry in the current log and we keep on
639    * trying to read the log file and get EOFEception. In case of a recovered queue the last log file
640    * may be empty, and we don't want to retry that.
641    */
642   private boolean isCurrentLogEmpty() {
643     return (this.repLogReader.getPosition() == 0 && !queueRecovered && queue.size() == 0);
644   }
645 
646   /**
647    * Do the sleeping logic
648    * @param msg Why we sleep
649    * @param sleepMultiplier by how many times the default sleeping time is augmented
650    * @return True if <code>sleepMultiplier</code> is &lt; <code>maxRetriesMultiplier</code>
651    */
652   protected boolean sleepForRetries(String msg, int sleepMultiplier) {
653     try {
654       LOG.debug(msg + ", sleeping " + sleepForRetries + " times " + sleepMultiplier);
655       Thread.sleep(this.sleepForRetries * sleepMultiplier);
656     } catch (InterruptedException e) {
657       LOG.debug("Interrupted while sleeping between retries");
658     }
659     return sleepMultiplier < maxRetriesMultiplier;
660   }
661 
662   /**
663    * We only want KVs that are scoped other than local
664    * @param edit The KV to check for replication
665    */
666   protected void removeNonReplicableEdits(WALEdit edit) {
667     NavigableMap<byte[], Integer> scopes = edit.getScopes();
668     List<KeyValue> kvs = edit.getKeyValues();
669     for (int i = edit.size()-1; i >= 0; i--) {
670       KeyValue kv = kvs.get(i);
671       // The scope will be null or empty if
672       // there's nothing to replicate in that WALEdit
673       if (scopes == null || !scopes.containsKey(kv.getFamily())) {
674         kvs.remove(i);
675       }
676     }
677   }
678 
679   /**
680    * Count the number of different row keys in the given edit because of
681    * mini-batching. We assume that there's at least one KV in the WALEdit.
682    * @param edit edit to count row keys from
683    * @return number of different row keys
684    */
685   private int countDistinctRowKeys(WALEdit edit) {
686     List<KeyValue> kvs = edit.getKeyValues();
687     int distinctRowKeys = 1;
688     KeyValue lastKV = kvs.get(0);
689     for (int i = 0; i < edit.size(); i++) {
690       if (!kvs.get(i).matchingRow(lastKV)) {
691         distinctRowKeys++;
692       }
693     }
694     return distinctRowKeys;
695   }
696 
697   /**
698    * Do the shipping logic
699    * @param currentWALisBeingWrittenTo was the current WAL being (seemingly) 
700    * written to when this method was called
701    */
702   protected void shipEdits(boolean currentWALisBeingWrittenTo) {
703     int sleepMultiplier = 1;
704     if (this.currentNbEntries == 0) {
705       LOG.warn("Was given 0 edits to ship");
706       return;
707     }
708     while (this.isActive()) {
709       if (!isPeerEnabled()) {
710         if (sleepForRetries("Replication is disabled", sleepMultiplier)) {
711           sleepMultiplier++;
712         }
713         continue;
714       }
715       try {
716         HRegionInterface rrs = getRS();
717         LOG.debug("Replicating " + currentNbEntries);
718         rrs.replicateLogEntries(Arrays.copyOf(this.entriesArray, currentNbEntries));
719         if (this.lastLoggedPosition != this.repLogReader.getPosition()) {
720           this.manager.logPositionAndCleanOldLogs(this.currentPath,
721               this.peerClusterZnode, this.repLogReader.getPosition(), queueRecovered, currentWALisBeingWrittenTo);
722           this.lastLoggedPosition = this.repLogReader.getPosition();
723         }
724         this.totalReplicatedEdits += currentNbEntries;
725         this.metrics.shippedBatchesRate.inc(1);
726         this.metrics.shippedOpsRate.inc(
727             this.currentNbOperations);
728         this.metrics.setAgeOfLastShippedOp(
729             this.entriesArray[currentNbEntries-1].getKey().getWriteTime());
730         LOG.debug("Replicated in total: " + this.totalReplicatedEdits);
731         break;
732 
733       } catch (IOException ioe) {
734         // Didn't ship anything, but must still age the last time we did
735         this.metrics.refreshAgeOfLastShippedOp();
736         if (ioe instanceof RemoteException) {
737           ioe = ((RemoteException) ioe).unwrapRemoteException();
738           LOG.warn("Can't replicate because of an error on the remote cluster: ", ioe);
739           if (ioe instanceof TableNotFoundException) {
740             if (sleepForRetries("A table is missing in the peer cluster. "
741                 + "Replication cannot proceed without losing data.", sleepMultiplier)) {
742               sleepMultiplier++;
743             }
744           }
745         } else {
746           if (ioe instanceof SocketTimeoutException) {
747             // This exception means we waited for more than 60s and nothing
748             // happened, the cluster is alive and calling it right away
749             // even for a test just makes things worse.
750             sleepForRetries("Encountered a SocketTimeoutException. Since the " +
751               "call to the remote cluster timed out, which is usually " +
752               "caused by a machine failure or a massive slowdown",
753               this.socketTimeoutMultiplier);
754           } else if (ioe instanceof ConnectException) {
755             LOG.warn("Peer is unavailable, rechecking all sinks: ", ioe);
756             chooseSinks();
757           } else {
758             LOG.warn("Can't replicate because of a local or network error: ", ioe);
759           }
760         }
761 
762         try {
763           boolean down;
764           // Spin while the slave is down and we're not asked to shutdown/close
765           do {
766             down = isSlaveDown();
767             if (down) {
768               if (sleepForRetries("Since we are unable to replicate", sleepMultiplier)) {
769                 sleepMultiplier++;
770               } else {
771                 chooseSinks();
772               }
773             }
774           } while (this.isActive() && down );
775         } catch (InterruptedException e) {
776           LOG.debug("Interrupted while trying to contact the peer cluster");
777         }
778       }
779     }
780   }
781 
782   /**
783    * check whether the peer is enabled or not
784    *
785    * @return true if the peer is enabled, otherwise false
786    */
787   protected boolean isPeerEnabled() {
788     return this.replicating.get() && this.zkHelper.getPeerEnabled(peerId);
789   }
790 
791   /**
792    * If the queue isn't empty, switch to the next one
793    * Else if this is a recovered queue, it means we're done!
794    * Else we'll just continue to try reading the log file
795    * @return true if we're done with the current file, false if we should
796    * continue trying to read from it
797    */
798   protected boolean processEndOfFile() {
799     if (this.queue.size() != 0) {
800       this.currentPath = null;
801       this.repLogReader.finishCurrentFile();
802       this.reader = null;
803       return true;
804     } else if (this.queueRecovered) {
805       this.manager.closeRecoveredQueue(this);
806       LOG.info("Finished recovering the queue");
807       this.running = false;
808       return true;
809     }
810     return false;
811   }
812 
813   public void startup() {
814     String n = Thread.currentThread().getName();
815     Thread.UncaughtExceptionHandler handler =
816         new Thread.UncaughtExceptionHandler() {
817           public void uncaughtException(final Thread t, final Throwable e) {
818             LOG.error("Unexpected exception in ReplicationSource," +
819               " currentPath=" + currentPath, e);
820           }
821         };
822     Threads.setDaemonThreadRunning(
823         this, n + ".replicationSource," + peerClusterZnode, handler);
824   }
825 
826   public void terminate(String reason) {
827     terminate(reason, null);
828   }
829 
830   public void terminate(String reason, Exception cause) {
831     if (cause == null) {
832       LOG.info("Closing source "
833           + this.peerClusterZnode + " because: " + reason);
834 
835     } else {
836       LOG.error("Closing source " + this.peerClusterZnode
837           + " because an error occurred: " + reason, cause);
838     }
839     this.running = false;
840     // Only wait for the thread to die if it's not us
841     if (!Thread.currentThread().equals(this)) {
842       Threads.shutdown(this, this.sleepForRetries);
843     }
844   }
845 
846   /**
847    * Get a new region server at random from this peer
848    * @return
849    * @throws IOException
850    */
851   private HRegionInterface getRS() throws IOException {
852     if (this.currentPeers.size() == 0) {
853       throw new IOException(this.peerClusterZnode + " has 0 region servers");
854     }
855     ServerName address =
856         currentPeers.get(random.nextInt(this.currentPeers.size()));
857     return this.conn.getHRegionConnection(address.getHostname(), address.getPort());
858   }
859 
860   /**
861    * Check if the slave is down by trying to establish a connection
862    * @return true if down, false if up
863    * @throws InterruptedException
864    */
865   public boolean isSlaveDown() throws InterruptedException {
866     final CountDownLatch latch = new CountDownLatch(1);
867     Thread pingThread = new Thread() {
868       public void run() {
869         try {
870           HRegionInterface rrs = getRS();
871           // Dummy call which should fail
872           rrs.getHServerInfo();
873           latch.countDown();
874         } catch (IOException ex) {
875           if (ex instanceof RemoteException) {
876             ex = ((RemoteException) ex).unwrapRemoteException();
877           }
878           LOG.info("Slave cluster looks down: " + ex.getMessage());
879         }
880       }
881     };
882     pingThread.start();
883     // awaits returns true if countDown happened
884     boolean down = ! latch.await(this.sleepForRetries, TimeUnit.MILLISECONDS);
885     pingThread.interrupt();
886     return down;
887   }
888 
889   public String getPeerClusterZnode() {
890     return this.peerClusterZnode;
891   }
892 
893   public String getPeerClusterId() {
894     return this.peerId;
895   }
896 
897   public Path getCurrentPath() {
898     return this.currentPath;
899   }
900 
901   private boolean isActive() {
902     return !this.stopper.isStopped() && this.running;
903   }
904 
905   /**
906    * Comparator used to compare logs together based on their start time
907    */
908   public static class LogsComparator implements Comparator<Path> {
909 
910     @Override
911     public int compare(Path o1, Path o2) {
912       return Long.valueOf(getTS(o1)).compareTo(getTS(o2));
913     }
914 
915     @Override
916     public boolean equals(Object o) {
917       return true;
918     }
919 
920     /**
921      * Split a path to get the start time
922      * For example: 10.20.20.171%3A60020.1277499063250
923      * @param p path to split
924      * @return start time
925      */
926     private long getTS(Path p) {
927       String[] parts = p.getName().split("\\.");
928       return Long.parseLong(parts[parts.length-1]);
929     }
930   }
931 }