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