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