keith-turner commented on a change in pull request #2117:
URL: https://github.com/apache/accumulo/pull/2117#discussion_r652154940



##########
File path: 
server/tserver/src/main/java/org/apache/accumulo/tserver/log/RecoveryLogsIterator.java
##########
@@ -19,61 +19,87 @@
 package org.apache.accumulo.tserver.log;
 
 import java.io.IOException;
+import java.util.AbstractMap;
 import java.util.ArrayList;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Map.Entry;
 
+import org.apache.accumulo.core.client.Scanner;
+import org.apache.accumulo.core.client.ScannerBase;
+import org.apache.accumulo.core.client.rfile.RFile;
+import org.apache.accumulo.core.data.Key;
+import org.apache.accumulo.core.data.Range;
+import org.apache.accumulo.core.data.Value;
+import org.apache.accumulo.server.ServerContext;
 import org.apache.accumulo.server.fs.VolumeManager;
+import org.apache.accumulo.server.log.SortedLogState;
+import org.apache.accumulo.tserver.logger.LogEvents;
 import org.apache.accumulo.tserver.logger.LogFileKey;
 import org.apache.accumulo.tserver.logger.LogFileValue;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.google.common.collect.Iterators;
-import com.google.common.collect.UnmodifiableIterator;
 
 /**
  * Iterates over multiple sorted recovery logs merging them into a single 
sorted stream.
  */
-public class RecoveryLogsIterator implements 
CloseableIterator<Entry<LogFileKey,LogFileValue>> {
+public class RecoveryLogsIterator
+    implements Iterator<Entry<LogFileKey,LogFileValue>>, AutoCloseable {
 
   private static final Logger LOG = 
LoggerFactory.getLogger(RecoveryLogsIterator.class);
 
-  List<CloseableIterator<Entry<LogFileKey,LogFileValue>>> iterators;
-  private UnmodifiableIterator<Entry<LogFileKey,LogFileValue>> iter;
+  private final List<Scanner> scanners;
+  private final Iterator<Entry<Key,Value>> iter;
 
   /**
-   * Iterates only over keys in the range [start,end].
+   * Scans the files in each recoveryLogDir over the range [start,end].
    */
-  RecoveryLogsIterator(VolumeManager fs, List<Path> recoveryLogPaths, 
LogFileKey start,
-      LogFileKey end) throws IOException {
+  RecoveryLogsIterator(ServerContext context, List<Path> recoveryLogDirs, 
LogFileKey start,
+      LogFileKey end, boolean checkFirstKey, LogEvents... colFamToFetch) 
throws IOException {
 
-    iterators = new ArrayList<>(recoveryLogPaths.size());
+    List<Iterator<Entry<Key,Value>>> iterators = new 
ArrayList<>(recoveryLogDirs.size());
+    scanners = new ArrayList<>();
+    Range range = new Range(start.formatRow(), end.formatRow());
+    var vm = context.getVolumeManager();
 
-    try {
-      for (Path log : recoveryLogPaths) {
-        LOG.debug("Opening recovery log {}", log.getName());
-        RecoveryLogReader rlr = new RecoveryLogReader(fs, log, start, end);
-        if (rlr.hasNext()) {
+    for (Path logDir : recoveryLogDirs) {
+      LOG.debug("Opening recovery log dir {}", logDir.getName());
+      List<Path> logFiles = getFiles(vm, logDir);
+      var fs = vm.getFileSystemByPath(logDir);
+
+      // only check the first key once to prevent extra iterator creation and 
seeking
+      if (checkFirstKey) {
+        validateFirstKey(
+            
RFile.newScanner().from(logFiles.stream().map(Path::toString).toArray(String[]::new))

Review comment:
       using a dedicate recovery Rfile block cache would solve what #882 was 
trying to address.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to