View Javadoc

1   /*
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  package org.apache.hadoop.hbase.replication.regionserver;
20  
21  import java.io.EOFException;
22  import java.io.FileNotFoundException;
23  import java.io.IOException;
24  import java.util.ArrayList;
25  import java.util.Comparator;
26  import java.util.List;
27  import java.util.UUID;
28  import java.util.concurrent.PriorityBlockingQueue;
29  import java.util.concurrent.TimeUnit;
30  
31  import org.apache.commons.lang.StringUtils;
32  import org.apache.commons.logging.Log;
33  import org.apache.commons.logging.LogFactory;
34  import org.apache.hadoop.classification.InterfaceAudience;
35  import org.apache.hadoop.conf.Configuration;
36  import org.apache.hadoop.fs.FileStatus;
37  import org.apache.hadoop.fs.FileSystem;
38  import org.apache.hadoop.fs.Path;
39  import org.apache.hadoop.hbase.CellUtil;
40  import org.apache.hadoop.hbase.HConstants;
41  import org.apache.hadoop.hbase.KeyValue;
42  import org.apache.hadoop.hbase.Stoppable;
43  import org.apache.hadoop.hbase.regionserver.wal.HLog;
44  import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
45  import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
46  import org.apache.hadoop.hbase.replication.ChainWALEntryFilter;
47  import org.apache.hadoop.hbase.replication.ReplicationEndpoint;
48  import org.apache.hadoop.hbase.replication.ReplicationException;
49  import org.apache.hadoop.hbase.replication.ReplicationPeers;
50  import org.apache.hadoop.hbase.replication.ReplicationQueueInfo;
51  import org.apache.hadoop.hbase.replication.ReplicationQueues;
52  import org.apache.hadoop.hbase.replication.SystemTableWALEntryFilter;
53  import org.apache.hadoop.hbase.replication.WALEntryFilter;
54  import org.apache.hadoop.hbase.util.Threads;
55  
56  import com.google.common.collect.Lists;
57  import com.google.common.util.concurrent.ListenableFuture;
58  import com.google.common.util.concurrent.Service;
59  
60  /**
61   * Class that handles the source of a replication stream.
62   * Currently does not handle more than 1 slave
63   * For each slave cluster it selects a random number of peers
64   * using a replication ratio. For example, if replication ration = 0.1
65   * and slave cluster has 100 region servers, 10 will be selected.
66   * <p/>
67   * A stream is considered down when we cannot contact a region server on the
68   * peer cluster for more than 55 seconds by default.
69   * <p/>
70   *
71   */
72  @InterfaceAudience.Private
73  public class ReplicationSource extends Thread
74      implements ReplicationSourceInterface {
75  
76    public static final Log LOG = LogFactory.getLog(ReplicationSource.class);
77    // Queue of logs to process
78    private PriorityBlockingQueue<Path> queue;
79    private ReplicationQueues replicationQueues;
80    private ReplicationPeers replicationPeers;
81  
82    private Configuration conf;
83    private ReplicationQueueInfo replicationQueueInfo;
84    // id of the peer cluster this source replicates to
85    private String peerId;
86    // The manager of all sources to which we ping back our progress
87    private ReplicationSourceManager manager;
88    // Should we stop everything?
89    private Stoppable stopper;
90    // How long should we sleep for each retry
91    private long sleepForRetries;
92    // Max size in bytes of entriesArray
93    private long replicationQueueSizeCapacity;
94    // Max number of entries in entriesArray
95    private int replicationQueueNbCapacity;
96    // Our reader for the current log
97    private HLog.Reader reader;
98    // Last position in the log that we sent to ZooKeeper
99    private long lastLoggedPosition = -1;
100   // Path of the current log
101   private volatile Path currentPath;
102   private FileSystem fs;
103   // id of this cluster
104   private UUID clusterId;
105   // id of the other cluster
106   private UUID peerClusterId;
107   // total number of edits we replicated
108   private long totalReplicatedEdits = 0;
109   // total number of edits we replicated
110   private long totalReplicatedOperations = 0;
111   // The znode we currently play with
112   private String peerClusterZnode;
113   // Maximum number of retries before taking bold actions
114   private int maxRetriesMultiplier;
115   // Current number of operations (Put/Delete) that we need to replicate
116   private int currentNbOperations = 0;
117   // Current size of data we need to replicate
118   private int currentSize = 0;
119   // Indicates if this particular source is running
120   private volatile boolean running = true;
121   // Metrics for this source
122   private MetricsSource metrics;
123   // Handle on the log reader helper
124   private ReplicationHLogReaderManager repLogReader;
125   //WARN threshold for the number of queued logs, defaults to 2
126   private int logQueueWarnThreshold;
127   // ReplicationEndpoint which will handle the actual replication
128   private ReplicationEndpoint replicationEndpoint;
129   // A filter (or a chain of filters) for the WAL entries.
130   private WALEntryFilter walEntryFilter;
131   // Context for ReplicationEndpoint#replicate()
132   private ReplicationEndpoint.ReplicateContext replicateContext;
133   // throttler
134   private ReplicationThrottler throttler;
135 
136   /**
137    * Instantiation method used by region servers
138    *
139    * @param conf configuration to use
140    * @param fs file system to use
141    * @param manager replication manager to ping to
142    * @param stopper     the atomic boolean to use to stop the regionserver
143    * @param peerClusterZnode the name of our znode
144    * @param clusterId unique UUID for the cluster
145    * @param replicationEndpoint the replication endpoint implementation
146    * @param metrics metrics for replication source
147    * @throws IOException
148    */
149   @Override
150   public void init(final Configuration conf, final FileSystem fs,
151       final ReplicationSourceManager manager, final ReplicationQueues replicationQueues,
152       final ReplicationPeers replicationPeers, final Stoppable stopper,
153       final String peerClusterZnode, final UUID clusterId, ReplicationEndpoint replicationEndpoint,
154       final MetricsSource metrics)
155           throws IOException {
156     this.stopper = stopper;
157     this.conf = conf;
158     decorateConf();
159     this.replicationQueueSizeCapacity =
160         this.conf.getLong("replication.source.size.capacity", 1024*1024*64);
161     this.replicationQueueNbCapacity =
162         this.conf.getInt("replication.source.nb.capacity", 25000);
163     this.maxRetriesMultiplier = this.conf.getInt("replication.source.maxretriesmultiplier", 10);
164     this.queue =
165         new PriorityBlockingQueue<Path>(
166             this.conf.getInt("hbase.regionserver.maxlogs", 32),
167             new LogsComparator());
168     long bandwidth = this.conf.getLong("replication.source.per.peer.node.bandwidth", 0);
169     this.throttler = new ReplicationThrottler((double)bandwidth/10.0);
170     this.replicationQueues = replicationQueues;
171     this.replicationPeers = replicationPeers;
172     this.manager = manager;
173     this.sleepForRetries =
174         this.conf.getLong("replication.source.sleepforretries", 1000);
175     this.fs = fs;
176     this.metrics = metrics;
177     this.repLogReader = new ReplicationHLogReaderManager(this.fs, this.conf);
178     this.clusterId = clusterId;
179 
180     this.peerClusterZnode = peerClusterZnode;
181     this.replicationQueueInfo = new ReplicationQueueInfo(peerClusterZnode);
182     // ReplicationQueueInfo parses the peerId out of the znode for us
183     this.peerId = this.replicationQueueInfo.getPeerId();
184     this.logQueueWarnThreshold = this.conf.getInt("replication.source.log.queue.warn", 2);
185     this.replicationEndpoint = replicationEndpoint;
186 
187     this.replicateContext = new ReplicationEndpoint.ReplicateContext();
188   }
189 
190   private void decorateConf() {
191     String replicationCodec = this.conf.get(HConstants.REPLICATION_CODEC_CONF_KEY);
192     if (StringUtils.isNotEmpty(replicationCodec)) {
193       this.conf.set(HConstants.RPC_CODEC_CONF_KEY, replicationCodec);
194     }
195   }
196 
197   @Override
198   public void enqueueLog(Path log) {
199     this.queue.put(log);
200     int queueSize = queue.size();
201     this.metrics.setSizeOfLogQueue(queueSize);
202     // This will log a warning for each new log that gets created above the warn threshold
203     if (queueSize > this.logQueueWarnThreshold) {
204       LOG.warn("Queue size: " + queueSize +
205         " exceeds value of replication.source.log.queue.warn: " + logQueueWarnThreshold);
206     }
207   }
208 
209   private void uninitialize() {
210     LOG.debug("Source exiting " + this.peerId);
211     metrics.clear();
212     if (replicationEndpoint.state() == Service.State.STARTING
213         || replicationEndpoint.state() == Service.State.RUNNING) {
214       replicationEndpoint.stopAndWait();
215     }
216   }
217 
218   @Override
219   public void run() {
220     // We were stopped while looping to connect to sinks, just abort
221     if (!this.isActive()) {
222       uninitialize();
223       return;
224     }
225 
226     try {
227       // start the endpoint, connect to the cluster
228       Service.State state = replicationEndpoint.start().get();
229       if (state != Service.State.RUNNING) {
230         LOG.warn("ReplicationEndpoint was not started. Exiting");
231         uninitialize();
232         return;
233       }
234     } catch (Exception ex) {
235       LOG.warn("Error starting ReplicationEndpoint, exiting", ex);
236       throw new RuntimeException(ex);
237     }
238 
239     // get the WALEntryFilter from ReplicationEndpoint and add it to default filters
240     ArrayList<WALEntryFilter> filters = Lists.newArrayList(
241       (WALEntryFilter)new SystemTableWALEntryFilter());
242     WALEntryFilter filterFromEndpoint = this.replicationEndpoint.getWALEntryfilter();
243     if (filterFromEndpoint != null) {
244       filters.add(filterFromEndpoint);
245     }
246     this.walEntryFilter = new ChainWALEntryFilter(filters);
247 
248     int sleepMultiplier = 1;
249     // delay this until we are in an asynchronous thread
250     while (this.isActive() && this.peerClusterId == null) {
251       this.peerClusterId = replicationEndpoint.getPeerUUID();
252       if (this.isActive() && this.peerClusterId == null) {
253         if (sleepForRetries("Cannot contact the peer's zk ensemble", sleepMultiplier)) {
254           sleepMultiplier++;
255         }
256       }
257     }
258     // We were stopped while looping to contact peer's zk ensemble, just abort
259     if (!this.isActive()) {
260       uninitialize();
261       return;
262     }
263 
264     // resetting to 1 to reuse later
265     sleepMultiplier = 1;
266 
267     // In rare case, zookeeper setting may be messed up. That leads to the incorrect
268     // peerClusterId value, which is the same as the source clusterId
269     if (clusterId.equals(peerClusterId) && !replicationEndpoint.canReplicateToSameCluster()) {
270       this.terminate("ClusterId " + clusterId + " is replicating to itself: peerClusterId "
271           + peerClusterId + " which is not allowed by ReplicationEndpoint:"
272           + replicationEndpoint.getClass().getName(), null, false);
273     }
274     LOG.info("Replicating "+clusterId + " -> " + peerClusterId);
275 
276     // If this is recovered, the queue is already full and the first log
277     // normally has a position (unless the RS failed between 2 logs)
278     if (this.replicationQueueInfo.isQueueRecovered()) {
279       try {
280         this.repLogReader.setPosition(this.replicationQueues.getLogPosition(this.peerClusterZnode,
281           this.queue.peek().getName()));
282         if (LOG.isTraceEnabled()) {
283           LOG.trace("Recovered queue started with log " + this.queue.peek() +
284               " at position " + this.repLogReader.getPosition());
285         }
286       } catch (ReplicationException e) {
287         this.terminate("Couldn't get the position of this recovered queue " +
288             this.peerClusterZnode, e);
289       }
290     }
291     // Loop until we close down
292     while (isActive()) {
293       // Sleep until replication is enabled again
294       if (!isPeerEnabled()) {
295         if (sleepForRetries("Replication is disabled", sleepMultiplier)) {
296           sleepMultiplier++;
297         }
298         continue;
299       }
300       Path oldPath = getCurrentPath(); //note that in the current scenario,
301                                        //oldPath will be null when a log roll
302                                        //happens.
303       // Get a new path
304       boolean hasCurrentPath = getNextPath();
305       if (getCurrentPath() != null && oldPath == null) {
306         sleepMultiplier = 1; //reset the sleepMultiplier on a path change
307       }
308       if (!hasCurrentPath) {
309         if (sleepForRetries("No log to process", sleepMultiplier)) {
310           sleepMultiplier++;
311         }
312         continue;
313       }
314       boolean currentWALisBeingWrittenTo = false;
315       //For WAL files we own (rather than recovered), take a snapshot of whether the
316       //current WAL file (this.currentPath) is in use (for writing) NOW!
317       //Since the new WAL paths are enqueued only after the prev WAL file
318       //is 'closed', presence of an element in the queue means that
319       //the previous WAL file was closed, else the file is in use (currentPath)
320       //We take the snapshot now so that we are protected against races
321       //where a new file gets enqueued while the current file is being processed
322       //(and where we just finished reading the current file).
323       if (!this.replicationQueueInfo.isQueueRecovered() && queue.size() == 0) {
324         currentWALisBeingWrittenTo = true;
325       }
326       // Open a reader on it
327       if (!openReader(sleepMultiplier)) {
328         // Reset the sleep multiplier, else it'd be reused for the next file
329         sleepMultiplier = 1;
330         continue;
331       }
332 
333       // If we got a null reader but didn't continue, then sleep and continue
334       if (this.reader == null) {
335         if (sleepForRetries("Unable to open a reader", sleepMultiplier)) {
336           sleepMultiplier++;
337         }
338         continue;
339       }
340 
341       boolean gotIOE = false;
342       currentNbOperations = 0;
343       List<HLog.Entry> entries = new ArrayList<HLog.Entry>(1);
344       currentSize = 0;
345       try {
346         if (readAllEntriesToReplicateOrNextFile(currentWALisBeingWrittenTo, entries)) {
347           continue;
348         }
349       } catch (IOException ioe) {
350         LOG.warn(this.peerClusterZnode + " Got: ", ioe);
351         gotIOE = true;
352         if (ioe.getCause() instanceof EOFException) {
353 
354           boolean considerDumping = false;
355           if (this.replicationQueueInfo.isQueueRecovered()) {
356             try {
357               FileStatus stat = this.fs.getFileStatus(this.currentPath);
358               if (stat.getLen() == 0) {
359                 LOG.warn(this.peerClusterZnode + " Got EOF and the file was empty");
360               }
361               considerDumping = true;
362             } catch (IOException e) {
363               LOG.warn(this.peerClusterZnode + " Got while getting file size: ", e);
364             }
365           }
366 
367           if (considerDumping &&
368               sleepMultiplier == this.maxRetriesMultiplier &&
369               processEndOfFile()) {
370             continue;
371           }
372         }
373       } finally {
374         try {
375           this.reader = null;
376           this.repLogReader.closeReader();
377         } catch (IOException e) {
378           gotIOE = true;
379           LOG.warn("Unable to finalize the tailing of a file", e);
380         }
381       }
382 
383       // If we didn't get anything to replicate, or if we hit a IOE,
384       // wait a bit and retry.
385       // But if we need to stop, don't bother sleeping
386       if (this.isActive() && (gotIOE || entries.isEmpty())) {
387         if (this.lastLoggedPosition != this.repLogReader.getPosition()) {
388           this.manager.logPositionAndCleanOldLogs(this.currentPath,
389               this.peerClusterZnode, this.repLogReader.getPosition(),
390               this.replicationQueueInfo.isQueueRecovered(), currentWALisBeingWrittenTo);
391           this.lastLoggedPosition = this.repLogReader.getPosition();
392         }
393         // Reset the sleep multiplier if nothing has actually gone wrong
394         if (!gotIOE) {
395           sleepMultiplier = 1;
396           // if there was nothing to ship and it's not an error
397           // set "ageOfLastShippedOp" to <now> to indicate that we're current
398           this.metrics.setAgeOfLastShippedOp(System.currentTimeMillis());
399         }
400         if (sleepForRetries("Nothing to replicate", sleepMultiplier)) {
401           sleepMultiplier++;
402         }
403         continue;
404       }
405       sleepMultiplier = 1;
406       shipEdits(currentWALisBeingWrittenTo, entries);
407     }
408     uninitialize();
409   }
410 
411   /**
412    * Read all the entries from the current log files and retain those
413    * that need to be replicated. Else, process the end of the current file.
414    * @param currentWALisBeingWrittenTo is the current WAL being written to
415    * @param entries resulting entries to be replicated
416    * @return true if we got nothing and went to the next file, false if we got
417    * entries
418    * @throws IOException
419    */
420   protected boolean readAllEntriesToReplicateOrNextFile(boolean currentWALisBeingWrittenTo,
421       List<HLog.Entry> entries) throws IOException{
422     long seenEntries = 0;
423     if (LOG.isTraceEnabled()) {
424       LOG.trace("Seeking in " + this.currentPath + " at position "
425           + this.repLogReader.getPosition());
426     }
427     this.repLogReader.seek();
428     long positionBeforeRead = this.repLogReader.getPosition();
429     HLog.Entry entry =
430         this.repLogReader.readNextAndSetPosition();
431     while (entry != null) {
432       this.metrics.incrLogEditsRead();
433       seenEntries++;
434 
435       // don't replicate if the log entries have already been consumed by the cluster
436       if (replicationEndpoint.canReplicateToSameCluster()
437           || !entry.getKey().getClusterIds().contains(peerClusterId)) {
438         // Remove all KVs that should not be replicated
439         entry = walEntryFilter.filter(entry);
440         WALEdit edit = null;
441         HLogKey logKey = null;
442         if (entry != null) {
443           edit = entry.getEdit();
444           logKey = entry.getKey();
445         }
446 
447         if (edit != null && edit.size() != 0) {
448           //Mark that the current cluster has the change
449           logKey.addClusterId(clusterId);
450           currentNbOperations += countDistinctRowKeys(edit);
451           entries.add(entry);
452           currentSize += entry.getEdit().heapSize();
453         } else {
454           this.metrics.incrLogEditsFiltered();
455         }
456       }
457       // Stop if too many entries or too big
458       if (currentSize >= this.replicationQueueSizeCapacity ||
459           entries.size() >= this.replicationQueueNbCapacity) {
460         break;
461       }
462       try {
463         entry = this.repLogReader.readNextAndSetPosition();
464       } catch (IOException ie) {
465         LOG.debug("Break on IOE: " + ie.getMessage());
466         break;
467       }
468     }
469     metrics.incrLogReadInBytes(this.repLogReader.getPosition() - positionBeforeRead);
470     if (currentWALisBeingWrittenTo) {
471       return false;
472     }
473     // If we didn't get anything and the queue has an object, it means we
474     // hit the end of the file for sure
475     return seenEntries == 0 && processEndOfFile();
476   }
477 
478   /**
479    * Poll for the next path
480    * @return true if a path was obtained, false if not
481    */
482   protected boolean getNextPath() {
483     try {
484       if (this.currentPath == null) {
485         this.currentPath = queue.poll(this.sleepForRetries, TimeUnit.MILLISECONDS);
486         this.metrics.setSizeOfLogQueue(queue.size());
487         if (this.currentPath != null) {
488           this.manager.cleanOldLogs(this.currentPath.getName(),
489               this.peerId,
490               this.replicationQueueInfo.isQueueRecovered());
491           if (LOG.isTraceEnabled()) {
492             LOG.trace("New log: " + this.currentPath);
493           }
494         }
495       }
496     } catch (InterruptedException e) {
497       LOG.warn("Interrupted while reading edits", e);
498     }
499     return this.currentPath != null;
500   }
501 
502   /**
503    * Open a reader on the current path
504    *
505    * @param sleepMultiplier by how many times the default sleeping time is augmented
506    * @return true if we should continue with that file, false if we are over with it
507    */
508   protected boolean openReader(int sleepMultiplier) {
509     try {
510       try {
511         if (LOG.isTraceEnabled()) {
512           LOG.trace("Opening log " + this.currentPath);
513         }
514         this.reader = repLogReader.openReader(this.currentPath);
515       } catch (FileNotFoundException fnfe) {
516         if (this.replicationQueueInfo.isQueueRecovered()) {
517           // We didn't find the log in the archive directory, look if it still
518           // exists in the dead RS folder (there could be a chain of failures
519           // to look at)
520           List<String> deadRegionServers = this.replicationQueueInfo.getDeadRegionServers();
521           LOG.info("NB dead servers : " + deadRegionServers.size());
522           for (String curDeadServerName : deadRegionServers) {
523             Path deadRsDirectory =
524                 new Path(manager.getLogDir().getParent(), curDeadServerName);
525             Path[] locs = new Path[] {
526                 new Path(deadRsDirectory, currentPath.getName()),
527                 new Path(deadRsDirectory.suffix(HLog.SPLITTING_EXT),
528                                           currentPath.getName()),
529             };
530             for (Path possibleLogLocation : locs) {
531               LOG.info("Possible location " + possibleLogLocation.toUri().toString());
532               if (this.manager.getFs().exists(possibleLogLocation)) {
533                 // We found the right new location
534                 LOG.info("Log " + this.currentPath + " still exists at " +
535                     possibleLogLocation);
536                 // Breaking here will make us sleep since reader is null
537                 return true;
538               }
539             }
540           }
541           // In the case of disaster/recovery, HMaster may be shutdown/crashed before flush data
542           // from .logs to .oldlogs. Loop into .logs folders and check whether a match exists
543           if (stopper instanceof ReplicationSyncUp.DummyServer) {
544             FileStatus[] rss = fs.listStatus(manager.getLogDir());
545             for (FileStatus rs : rss) {
546               Path p = rs.getPath();
547               FileStatus[] logs = fs.listStatus(p);
548               for (FileStatus log : logs) {
549                 p = new Path(p, log.getPath().getName());
550                 if (p.getName().equals(currentPath.getName())) {
551                   currentPath = p;
552                   LOG.info("Log " + this.currentPath + " exists under " + manager.getLogDir());
553                   // Open the log at the new location
554                   this.openReader(sleepMultiplier);
555                   return true;
556                 }
557               }
558             }
559           }
560 
561           // TODO What happens if the log was missing from every single location?
562           // Although we need to check a couple of times as the log could have
563           // been moved by the master between the checks
564           // It can also happen if a recovered queue wasn't properly cleaned,
565           // such that the znode pointing to a log exists but the log was
566           // deleted a long time ago.
567           // For the moment, we'll throw the IO and processEndOfFile
568           throw new IOException("File from recovered queue is " +
569               "nowhere to be found", fnfe);
570         } else {
571           // If the log was archived, continue reading from there
572           Path archivedLogLocation =
573               new Path(manager.getOldLogDir(), currentPath.getName());
574           if (this.manager.getFs().exists(archivedLogLocation)) {
575             currentPath = archivedLogLocation;
576             LOG.info("Log " + this.currentPath + " was moved to " +
577                 archivedLogLocation);
578             // Open the log at the new location
579             this.openReader(sleepMultiplier);
580 
581           }
582           // TODO What happens the log is missing in both places?
583         }
584       }
585     } catch (IOException ioe) {
586       if (ioe instanceof EOFException && isCurrentLogEmpty()) return true;
587       LOG.warn(this.peerClusterZnode + " Got: ", ioe);
588       this.reader = null;
589       if (ioe.getCause() instanceof NullPointerException) {
590         // Workaround for race condition in HDFS-4380
591         // which throws a NPE if we open a file before any data node has the most recent block
592         // Just sleep and retry. Will require re-reading compressed HLogs for compressionContext.
593         LOG.warn("Got NPE opening reader, will retry.");
594       } else if (sleepMultiplier == this.maxRetriesMultiplier) {
595         // TODO Need a better way to determine if a file is really gone but
596         // TODO without scanning all logs dir
597         LOG.warn("Waited too long for this file, considering dumping");
598         return !processEndOfFile();
599       }
600     }
601     return true;
602   }
603 
604   /*
605    * Checks whether the current log file is empty, and it is not a recovered queue. This is to
606    * handle scenario when in an idle cluster, there is no entry in the current log and we keep on
607    * trying to read the log file and get EOFException. In case of a recovered queue the last log
608    * file may be empty, and we don't want to retry that.
609    */
610   private boolean isCurrentLogEmpty() {
611     return (this.repLogReader.getPosition() == 0 &&
612         !this.replicationQueueInfo.isQueueRecovered() && queue.size() == 0);
613   }
614 
615   /**
616    * Do the sleeping logic
617    * @param msg Why we sleep
618    * @param sleepMultiplier by how many times the default sleeping time is augmented
619    * @return True if <code>sleepMultiplier</code> is &lt; <code>maxRetriesMultiplier</code>
620    */
621   protected boolean sleepForRetries(String msg, int sleepMultiplier) {
622     try {
623       if (LOG.isTraceEnabled()) {
624         LOG.trace(msg + ", sleeping " + sleepForRetries + " times " + sleepMultiplier);
625       }
626       Thread.sleep(this.sleepForRetries * sleepMultiplier);
627     } catch (InterruptedException e) {
628       LOG.debug("Interrupted while sleeping between retries");
629       Thread.currentThread().interrupt();
630     }
631     return sleepMultiplier < maxRetriesMultiplier;
632   }
633 
634   /**
635    * Count the number of different row keys in the given edit because of
636    * mini-batching. We assume that there's at least one KV in the WALEdit.
637    * @param edit edit to count row keys from
638    * @return number of different row keys
639    */
640   private int countDistinctRowKeys(WALEdit edit) {
641     List<KeyValue> kvs = edit.getKeyValues();
642     int distinctRowKeys = 1;
643     KeyValue lastKV = kvs.get(0);
644     for (int i = 0; i < edit.size(); i++) {
645       if (!CellUtil.matchingRow(kvs.get(i), lastKV)) {
646         distinctRowKeys++;
647       }
648     }
649     return distinctRowKeys;
650   }
651 
652   /**
653    * Do the shipping logic
654    * @param currentWALisBeingWrittenTo was the current WAL being (seemingly)
655    * written to when this method was called
656    */
657   protected void shipEdits(boolean currentWALisBeingWrittenTo, List<HLog.Entry> entries) {
658     int sleepMultiplier = 1;
659     if (entries.isEmpty()) {
660       LOG.warn("Was given 0 edits to ship");
661       return;
662     }
663     while (this.isActive()) {
664       try {
665         if (this.throttler.isEnabled()) {
666           long sleepTicks = this.throttler.getNextSleepInterval(currentSize);
667           if (sleepTicks > 0) {
668             try {
669               if (LOG.isTraceEnabled()) {
670                 LOG.trace("To sleep " + sleepTicks + "ms for throttling control");
671               }
672               Thread.sleep(sleepTicks);
673             } catch (InterruptedException e) {
674               LOG.debug("Interrupted while sleeping for throttling control");
675               Thread.currentThread().interrupt();
676               // current thread might be interrupted to terminate
677               // directly go back to while() for confirm this
678               continue;
679             }
680             // reset throttler's cycle start tick when sleep for throttling occurs
681             this.throttler.resetStartTick();
682           }
683         }
684         replicateContext.setEntries(entries).setSize(currentSize);
685 
686         // send the edits to the endpoint. Will block until the edits are shipped and acknowledged
687         boolean replicated = replicationEndpoint.replicate(replicateContext);
688 
689         if (!replicated) {
690           continue;
691         }
692 
693         if (this.lastLoggedPosition != this.repLogReader.getPosition()) {
694           this.manager.logPositionAndCleanOldLogs(this.currentPath,
695               this.peerClusterZnode, this.repLogReader.getPosition(),
696               this.replicationQueueInfo.isQueueRecovered(), currentWALisBeingWrittenTo);
697           this.lastLoggedPosition = this.repLogReader.getPosition();
698         }
699         if (this.throttler.isEnabled()) {
700           this.throttler.addPushSize(currentSize);
701         }
702         this.totalReplicatedEdits += entries.size();
703         this.totalReplicatedOperations += currentNbOperations;
704         this.metrics.shipBatch(this.currentNbOperations, this.currentSize/1024);
705         this.metrics.setAgeOfLastShippedOp(entries.get(entries.size()-1).getKey().getWriteTime());
706         if (LOG.isTraceEnabled()) {
707           LOG.trace("Replicated " + this.totalReplicatedEdits + " entries in total, or "
708               + this.totalReplicatedOperations + " operations");
709         }
710         break;
711       } catch (Exception ex) {
712         LOG.warn(replicationEndpoint.getClass().getName() + " threw unknown exception:" +
713             org.apache.hadoop.util.StringUtils.stringifyException(ex));
714         if (sleepForRetries("ReplicationEndpoint threw exception", sleepMultiplier)) {
715           sleepMultiplier++;
716         }
717       }
718     }
719   }
720 
721   /**
722    * check whether the peer is enabled or not
723    *
724    * @return true if the peer is enabled, otherwise false
725    */
726   protected boolean isPeerEnabled() {
727     return this.replicationPeers.getStatusOfPeer(this.peerId);
728   }
729 
730   /**
731    * If the queue isn't empty, switch to the next one
732    * Else if this is a recovered queue, it means we're done!
733    * Else we'll just continue to try reading the log file
734    * @return true if we're done with the current file, false if we should
735    * continue trying to read from it
736    */
737   protected boolean processEndOfFile() {
738     if (this.queue.size() != 0) {
739       if (LOG.isTraceEnabled()) {
740         String filesize = "N/A";
741         try {
742           FileStatus stat = this.fs.getFileStatus(this.currentPath);
743           filesize = stat.getLen()+"";
744         } catch (IOException ex) {}
745         LOG.trace("Reached the end of a log, stats: " + getStats() +
746             ", and the length of the file is " + filesize);
747       }
748       this.currentPath = null;
749       this.repLogReader.finishCurrentFile();
750       this.reader = null;
751       return true;
752     } else if (this.replicationQueueInfo.isQueueRecovered()) {
753       this.manager.closeRecoveredQueue(this);
754       LOG.info("Finished recovering the queue with the following stats " + getStats());
755       this.running = false;
756       return true;
757     }
758     return false;
759   }
760 
761   @Override
762   public void startup() {
763     String n = Thread.currentThread().getName();
764     Thread.UncaughtExceptionHandler handler =
765         new Thread.UncaughtExceptionHandler() {
766           @Override
767           public void uncaughtException(final Thread t, final Throwable e) {
768             LOG.error("Unexpected exception in ReplicationSource," +
769               " currentPath=" + currentPath, e);
770           }
771         };
772     Threads.setDaemonThreadRunning(
773         this, n + ".replicationSource," +
774         this.peerClusterZnode, handler);
775   }
776 
777   @Override
778   public void terminate(String reason) {
779     terminate(reason, null);
780   }
781 
782   @Override
783   public void terminate(String reason, Exception cause) {
784     terminate(reason, cause, true);
785   }
786 
787   public void terminate(String reason, Exception cause, boolean join) {
788     if (cause == null) {
789       LOG.info("Closing source "
790           + this.peerClusterZnode + " because: " + reason);
791 
792     } else {
793       LOG.error("Closing source " + this.peerClusterZnode
794           + " because an error occurred: " + reason, cause);
795     }
796     this.running = false;
797     this.interrupt();
798     ListenableFuture<Service.State> future = null;
799     if (this.replicationEndpoint != null) {
800       future = this.replicationEndpoint.stop();
801     }
802     if (join) {
803       Threads.shutdown(this, this.sleepForRetries);
804       if (future != null) {
805         try {
806           future.get();
807         } catch (Exception e) {
808           LOG.warn("Got exception:" + e);
809         }
810       }
811     }
812   }
813 
814   @Override
815   public String getPeerClusterZnode() {
816     return this.peerClusterZnode;
817   }
818 
819   @Override
820   public String getPeerClusterId() {
821     return this.peerId;
822   }
823 
824   @Override
825   public Path getCurrentPath() {
826     return this.currentPath;
827   }
828 
829   private boolean isActive() {
830     return !this.stopper.isStopped() && this.running && !isInterrupted();
831   }
832 
833   /**
834    * Comparator used to compare logs together based on their start time
835    */
836   public static class LogsComparator implements Comparator<Path> {
837 
838     @Override
839     public int compare(Path o1, Path o2) {
840       return Long.valueOf(getTS(o1)).compareTo(getTS(o2));
841     }
842 
843     /**
844      * Split a path to get the start time
845      * For example: 10.20.20.171%3A60020.1277499063250
846      * @param p path to split
847      * @return start time
848      */
849     private long getTS(Path p) {
850       String[] parts = p.getName().split("\\.");
851       return Long.parseLong(parts[parts.length-1]);
852     }
853   }
854 
855   @Override
856   public String getStats() {
857     long position = this.repLogReader.getPosition();
858     return "Total replicated edits: " + totalReplicatedEdits +
859       ", currently replicating from: " + this.currentPath +
860       " at position: " + position;
861   }
862 }