accumulo-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From ktur...@apache.org
Subject [accumulo] 02/04: Fixes #449 suggested updates to PR apache/accumulo#458
Date Tue, 08 May 2018 15:36:27 GMT
This is an automated email from the ASF dual-hosted git repository.

kturner pushed a commit to branch 1.9
in repository https://gitbox.apache.org/repos/asf/accumulo.git

commit e56340c3f873a7beb15e9ff3740639ec80fe0eff
Author: Christopher Tubbs <ctubbsii@apache.org>
AuthorDate: Wed May 2 17:53:25 2018 -0400

    Fixes #449 suggested updates to PR apache/accumulo#458
---
 .../{MultiReader.java => RecoveryLogReader.java}   |  93 ++++++++++++--
 .../accumulo/tserver/log/RecoveryLogsIterator.java | 143 +++++----------------
 .../accumulo/tserver/log/SortedLogRecovery.java    |  82 ++++++++----
 .../apache/accumulo/tserver/logger/LogEvents.java  |   3 +
 .../apache/accumulo/tserver/logger/LogFileKey.java |   2 +-
 .../apache/accumulo/tserver/logger/LogReader.java  |   4 +-
 ...ReaderTest.java => RecoveryLogsReaderTest.java} |  10 +-
 7 files changed, 179 insertions(+), 158 deletions(-)

diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/MultiReader.java
b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/RecoveryLogReader.java
similarity index 64%
rename from server/tserver/src/main/java/org/apache/accumulo/tserver/log/MultiReader.java
rename to server/tserver/src/main/java/org/apache/accumulo/tserver/log/RecoveryLogReader.java
index e354797..ab8b631 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/MultiReader.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/RecoveryLogReader.java
@@ -18,10 +18,15 @@ package org.apache.accumulo.tserver.log;
 
 import java.io.EOFException;
 import java.io.IOException;
+import java.util.AbstractMap;
+import java.util.Iterator;
+import java.util.Map.Entry;
 import java.util.Objects;
 
 import org.apache.accumulo.server.fs.VolumeManager;
 import org.apache.accumulo.server.log.SortedLogState;
+import org.apache.accumulo.tserver.logger.LogFileKey;
+import org.apache.accumulo.tserver.logger.LogFileValue;
 import org.apache.commons.collections.buffer.PriorityBuffer;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
@@ -32,23 +37,23 @@ import org.apache.hadoop.io.MapFile.Reader;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
 
+import com.google.common.base.Preconditions;
+
 /**
- * Provide simple Map.Reader methods over multiple Maps.
+ * A class which reads sorted recovery logs produced from a single WAL.
  *
  * Presently only supports next() and seek() and works on all the Map directories within
a
  * directory. The primary purpose of this class is to merge the results of multiple Reduce
jobs that
  * result in Map output files.
  */
-@SuppressWarnings({"rawtypes", "unchecked"})
-public class MultiReader {
+public class RecoveryLogReader {
 
   /**
    * Group together the next key/value from a Reader with the Reader
-   *
    */
   private static class Index implements Comparable<Index> {
     Reader reader;
-    WritableComparable key;
+    WritableComparable<?> key;
     Writable value;
     boolean cached = false;
 
@@ -62,7 +67,7 @@ public class MultiReader {
 
     public Index(Reader reader) {
       this.reader = reader;
-      key = (WritableComparable) create(reader.getKeyClass());
+      key = (WritableComparable<?>) create(reader.getKeyClass());
       value = (Writable) create(reader.getValueClass());
     }
 
@@ -92,7 +97,9 @@ public class MultiReader {
           return 1;
         if (!o.cached)
           return -1;
-        return key.compareTo(o.key);
+        @SuppressWarnings({"unchecked", "rawtypes"})
+        int result = ((WritableComparable) key).compareTo(o.key);
+        return result;
       } catch (IOException ex) {
         throw new RuntimeException(ex);
       }
@@ -101,7 +108,7 @@ public class MultiReader {
 
   private PriorityBuffer heap = new PriorityBuffer();
 
-  public MultiReader(VolumeManager fs, Path directory) throws IOException {
+  public RecoveryLogReader(VolumeManager fs, Path directory) throws IOException {
     boolean foundFinish = false;
     for (FileStatus child : fs.listStatus(directory)) {
       if (child.getPath().getName().startsWith("_"))
@@ -127,7 +134,7 @@ public class MultiReader {
     dest.readFields(input);
   }
 
-  public synchronized boolean next(WritableComparable key, Writable val) throws IOException
{
+  public synchronized boolean next(WritableComparable<?> key, Writable val) throws
IOException {
     Index elt = (Index) heap.remove();
     try {
       elt.cache();
@@ -144,13 +151,13 @@ public class MultiReader {
     return true;
   }
 
-  public synchronized boolean seek(WritableComparable key) throws IOException {
+  synchronized boolean seek(WritableComparable<?> key) throws IOException {
     PriorityBuffer reheap = new PriorityBuffer(heap.size());
     boolean result = false;
     for (Object obj : heap) {
       Index index = (Index) obj;
       try {
-        WritableComparable found = index.reader.getClosest(key, index.value, true);
+        WritableComparable<?> found = index.reader.getClosest(key, index.value, true);
         if (found != null && found.equals(key)) {
           result = true;
         }
@@ -164,7 +171,7 @@ public class MultiReader {
     return result;
   }
 
-  public void close() throws IOException {
+  void close() throws IOException {
     IOException problem = null;
     for (Object obj : heap) {
       Index index = (Index) obj;
@@ -179,4 +186,66 @@ public class MultiReader {
     heap = null;
   }
 
+  volatile boolean returnedIterator = false;
+
+  // TODO make this primary entry into this class, and remove volatile boolean and make rest
private
+  Iterator<Entry<LogFileKey,LogFileValue>> getIterator(LogFileKey start, LogFileKey
end)
+      throws IOException {
+    Preconditions.checkState(!returnedIterator, "Each reader can have only one iterator");
+    returnedIterator = true;
+    return new RecoveryLogReaderIterator(this, start, end);
+  }
+
+  private static class RecoveryLogReaderIterator
+      implements Iterator<Entry<LogFileKey,LogFileValue>> {
+
+    private RecoveryLogReader reader;
+    private LogFileKey key = new LogFileKey();
+    private LogFileValue value = new LogFileValue();
+    private boolean hasNext;
+    private LogFileKey end;
+
+    RecoveryLogReaderIterator(RecoveryLogReader reader, LogFileKey start, LogFileKey end)
+        throws IOException {
+      this.reader = reader;
+      this.end = end;
+
+      reader.seek(start);
+
+      hasNext = reader.next(key, value);
+
+      if (hasNext && key.compareTo(start) < 0) {
+        throw new IllegalStateException("First key is less than start " + key + " " + start);
+      }
+
+      if (hasNext && key.compareTo(end) > 0) {
+        hasNext = false;
+      }
+    }
+
+    @Override
+    public boolean hasNext() {
+      return hasNext;
+    }
+
+    @Override
+    public Entry<LogFileKey,LogFileValue> next() {
+      Preconditions.checkState(hasNext);
+      Entry<LogFileKey,LogFileValue> entry = new AbstractMap.SimpleImmutableEntry<>(key,
value);
+
+      key = new LogFileKey();
+      value = new LogFileValue();
+      try {
+        hasNext = reader.next(key, value);
+        if (hasNext && key.compareTo(end) > 0) {
+          hasNext = false;
+        }
+      } catch (IOException e) {
+        throw new IllegalStateException(e);
+      }
+
+      return entry;
+    }
+  }
+
 }
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/RecoveryLogsIterator.java
b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/RecoveryLogsIterator.java
index ba25322..95ee0c5 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/RecoveryLogsIterator.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/RecoveryLogsIterator.java
@@ -19,8 +19,6 @@ package org.apache.accumulo.tserver.log;
 import static org.apache.accumulo.tserver.logger.LogEvents.OPEN;
 
 import java.io.IOException;
-import java.io.UncheckedIOException;
-import java.util.AbstractMap;
 import java.util.ArrayList;
 import java.util.Comparator;
 import java.util.Iterator;
@@ -28,76 +26,34 @@ import java.util.List;
 import java.util.Map.Entry;
 
 import org.apache.accumulo.server.fs.VolumeManager;
-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.Path;
-import org.mortbay.log.Log;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Iterators;
 import com.google.common.collect.PeekingIterator;
 import com.google.common.collect.UnmodifiableIterator;
 
 /**
- * Iterates over multiple recovery logs merging them into a single sorted stream.
+ * Iterates over multiple sorted recovery logs merging them into a single sorted stream.
  */
 public class RecoveryLogsIterator
     implements Iterator<Entry<LogFileKey,LogFileValue>>, AutoCloseable {
 
-  private List<MultiReader> readers;
-  private UnmodifiableIterator<Entry<LogFileKey,LogFileValue>> iter;
-
-  private static class MultiReaderIterator implements Iterator<Entry<LogFileKey,LogFileValue>>
{
-
-    private MultiReader reader;
-    private LogFileKey key = new LogFileKey();
-    private LogFileValue value = new LogFileValue();
-    private boolean hasNext;
-    private LogFileKey end;
-
-    MultiReaderIterator(MultiReader reader, LogFileKey start, LogFileKey end) throws IOException
{
-      this.reader = reader;
-      this.end = end;
-
-      reader.seek(start);
-
-      hasNext = reader.next(key, value);
-
-      if (hasNext && key.compareTo(start) < 0) {
-        throw new IllegalStateException("First key is less than start " + key + " " + start);
-      }
-
-      if (hasNext && key.compareTo(end) > 0) {
-        hasNext = false;
-      }
-    }
+  private static final Logger LOG = LoggerFactory.getLogger(RecoveryLogsIterator.class);
 
-    @Override
-    public boolean hasNext() {
-      return hasNext;
-    }
-
-    @Override
-    public Entry<LogFileKey,LogFileValue> next() {
-      Preconditions.checkState(hasNext);
-      Entry<LogFileKey,LogFileValue> entry = new AbstractMap.SimpleImmutableEntry<>(key,
value);
-
-      key = new LogFileKey();
-      value = new LogFileValue();
-      try {
-        hasNext = reader.next(key, value);
-        if (hasNext && key.compareTo(end) > 0) {
-          hasNext = false;
-        }
-      } catch (IOException e) {
-        throw new UncheckedIOException(e);
-      }
-
-      return entry;
-    }
-  }
+  private List<RecoveryLogReader> readers;
+  private UnmodifiableIterator<Entry<LogFileKey,LogFileValue>> iter;
 
+  /**
+   * Ensures source iterator provides data in sorted order
+   */
+  // TODO add unit test and move to RecoveryLogReader
+  @VisibleForTesting
   static class SortCheckIterator implements Iterator<Entry<LogFileKey,LogFileValue>>
{
 
     private PeekingIterator<Entry<LogFileKey,LogFileValue>> source;
@@ -125,8 +81,9 @@ public class RecoveryLogsIterator
     }
   }
 
-  private MultiReader open(VolumeManager fs, Path log) throws IOException {
-    MultiReader reader = new MultiReader(fs, log);
+  // TODO get rid of this (push down into iterator in RecoveryLogReader)
+  private RecoveryLogReader open(VolumeManager fs, Path log) throws IOException {
+    RecoveryLogReader reader = new RecoveryLogReader(fs, log);
     LogFileKey key = new LogFileKey();
     LogFileValue value = new LogFileValue();
     if (!reader.next(key, value)) {
@@ -134,58 +91,19 @@ public class RecoveryLogsIterator
       return null;
     }
     if (key.event != OPEN) {
-      reader.close();
-      throw new RuntimeException("First log entry value is not OPEN");
+      RuntimeException e = new IllegalStateException(
+          "First log entry value is not OPEN (" + log + ")");
+      try {
+        reader.close();
+      } catch (Exception e2) {
+        e.addSuppressed(e2);
+      }
+      throw e;
     }
 
     return reader;
   }
 
-  static LogFileKey maxKey(LogEvents event) {
-    LogFileKey key = new LogFileKey();
-    key.event = event;
-    key.tid = Integer.MAX_VALUE;
-    key.seq = Long.MAX_VALUE;
-    return key;
-  }
-
-  static LogFileKey maxKey(LogEvents event, int tid) {
-    LogFileKey key = maxKey(event);
-    key.tid = tid;
-    return key;
-  }
-
-  static LogFileKey minKey(LogEvents event) {
-    LogFileKey key = new LogFileKey();
-    key.event = event;
-    key.tid = 0;
-    key.seq = 0;
-    return key;
-  }
-
-  static LogFileKey minKey(LogEvents event, int tid) {
-    LogFileKey key = minKey(event);
-    key.tid = tid;
-    return key;
-  }
-
-  /**
-   * Iterates only over keys with the specified event (some events are equivalent for sorting)
and
-   * tid type.
-   */
-  RecoveryLogsIterator(VolumeManager fs, List<Path> recoveryLogPaths, LogEvents event,
int tid)
-      throws IOException {
-    this(fs, recoveryLogPaths, minKey(event, tid), maxKey(event, tid));
-  }
-
-  /**
-   * Iterates only over keys with the specified event (some events are equivalent for sorting).
-   */
-  RecoveryLogsIterator(VolumeManager fs, List<Path> recoveryLogPaths, LogEvents event)
-      throws IOException {
-    this(fs, recoveryLogPaths, minKey(event), maxKey(event));
-  }
-
   /**
    * Iterates only over keys between [start,end].
    */
@@ -197,11 +115,10 @@ public class RecoveryLogsIterator
 
     try {
       for (Path log : recoveryLogPaths) {
-        MultiReader reader = open(fs, log);
+        RecoveryLogReader reader = open(fs, log);
         if (reader != null) {
           readers.add(reader);
-          iterators.add(
-              new SortCheckIterator(log.getName(), new MultiReaderIterator(reader, start,
end)));
+          iterators.add(new SortCheckIterator(log.getName(), reader.getIterator(start, end)));
         }
       }
 
@@ -213,7 +130,11 @@ public class RecoveryLogsIterator
       });
 
     } catch (RuntimeException | IOException e) {
-      close();
+      try {
+        close();
+      } catch (Exception e2) {
+        e.addSuppressed(e2);
+      }
       throw e;
     }
   }
@@ -230,11 +151,11 @@ public class RecoveryLogsIterator
 
   @Override
   public void close() {
-    for (MultiReader reader : readers) {
+    for (RecoveryLogReader reader : readers) {
       try {
         reader.close();
       } catch (IOException e) {
-        Log.debug("Failed to close reader", e);
+        LOG.debug("Failed to close reader", e);
       }
     }
   }
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/SortedLogRecovery.java
b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/SortedLogRecovery.java
index 2c2a463..f84c906 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/log/SortedLogRecovery.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/log/SortedLogRecovery.java
@@ -17,8 +17,6 @@
 package org.apache.accumulo.tserver.log;
 
 import static com.google.common.base.Preconditions.checkState;
-import static org.apache.accumulo.tserver.log.RecoveryLogsIterator.maxKey;
-import static org.apache.accumulo.tserver.log.RecoveryLogsIterator.minKey;
 import static org.apache.accumulo.tserver.logger.LogEvents.COMPACTION_FINISH;
 import static org.apache.accumulo.tserver.logger.LogEvents.COMPACTION_START;
 import static org.apache.accumulo.tserver.logger.LogEvents.DEFINE_TABLET;
@@ -63,10 +61,40 @@ public class SortedLogRecovery {
     this.fs = fs;
   }
 
+  static LogFileKey maxKey(LogEvents event) {
+    LogFileKey key = new LogFileKey();
+    key.event = event;
+    key.tid = Integer.MAX_VALUE;
+    key.seq = Long.MAX_VALUE;
+    return key;
+  }
+
+  static LogFileKey maxKey(LogEvents event, int tabletId) {
+    LogFileKey key = maxKey(event);
+    key.tid = tabletId;
+    return key;
+  }
+
+  static LogFileKey minKey(LogEvents event) {
+    LogFileKey key = new LogFileKey();
+    key.event = event;
+    key.tid = 0;
+    key.seq = 0;
+    return key;
+  }
+
+  static LogFileKey minKey(LogEvents event, int tabletId) {
+    LogFileKey key = minKey(event);
+    key.tid = tabletId;
+    return key;
+  }
+
   private int findMaxTabletId(KeyExtent extent, List<Path> recoveryLogs) throws IOException
{
-    int tid = -1;
+    int tabletId = -1;
+
+    try (RecoveryLogsIterator rli = new RecoveryLogsIterator(fs, recoveryLogs,
+        minKey(DEFINE_TABLET), maxKey(DEFINE_TABLET))) {
 
-    try (RecoveryLogsIterator rli = new RecoveryLogsIterator(fs, recoveryLogs, DEFINE_TABLET))
{
       KeyExtent alternative = extent;
       if (extent.isRootTablet()) {
         alternative = RootTable.OLD_EXTENT;
@@ -79,16 +107,16 @@ public class SortedLogRecovery {
 
         if (key.tablet.equals(extent) || key.tablet.equals(alternative)) {
           checkState(key.tid >= 0, "Tid %s for %s is negative", key.tid, extent);
-          checkState(tid == -1 || key.tid >= tid); // should only fail if bug in
-                                                   // RecoveryLogsIterator
+          checkState(tabletId == -1 || key.tid >= tabletId); // should only fail if bug
in
+          // RecoveryLogsIterator
 
-          if (tid != key.tid) {
-            tid = key.tid;
+          if (tabletId != key.tid) {
+            tabletId = key.tid;
           }
         }
       }
     }
-    return tid;
+    return tabletId;
   }
 
   private String getPathSuffix(String pathString) {
@@ -124,7 +152,7 @@ public class SortedLogRecovery {
 
   }
 
-  private long findLastStartToFinish(List<Path> recoveryLogs, Set<String> tabletFiles,
int tid)
+  private long findLastStartToFinish(List<Path> recoveryLogs, Set<String> tabletFiles,
int tabletId)
       throws IOException {
     HashSet<String> suffixes = new HashSet<>();
     for (String path : tabletFiles)
@@ -133,8 +161,8 @@ public class SortedLogRecovery {
     long lastStart = 0;
     long recoverySeq = 0;
 
-    try (RecoveryLogsIterator rli = new RecoveryLogsIterator(fs, recoveryLogs, COMPACTION_START,
-        tid)) {
+    try (RecoveryLogsIterator rli = new RecoveryLogsIterator(fs, recoveryLogs,
+        minKey(COMPACTION_START, tabletId), maxKey(COMPACTION_START, tabletId))) {
 
       DeduplicatingIterator ddi = new DeduplicatingIterator(rli);
 
@@ -146,8 +174,8 @@ public class SortedLogRecovery {
       while (ddi.hasNext()) {
         LogFileKey key = ddi.next().getKey();
 
-        checkState(key.seq >= 0, "Unexpected negative seq %s for tid %s", key.seq, tid);
-        checkState(key.tid == tid); // should only fail if bug elsewhere
+        checkState(key.seq >= 0, "Unexpected negative seq %s for tabletId %s", key.seq,
tabletId);
+        checkState(key.tid == tabletId); // should only fail if bug elsewhere
 
         if (key.event == COMPACTION_START) {
           checkState(key.seq >= lastStart); // should only fail if bug elsewhere
@@ -176,33 +204,33 @@ public class SortedLogRecovery {
       }
 
       if (firstEventWasFinish && !sawStartFinish) {
-        throw new IllegalStateException(
-            "COMPACTION_FINISH (without preceding COMPACTION_START) is not followed by a
successful minor compaction.");
+        throw new IllegalStateException("COMPACTION_FINISH (without preceding COMPACTION_START)"
+            + " is not followed by a successful minor compaction.");
       }
 
       if (lastStartFile != null && suffixes.contains(getPathSuffix(lastStartFile)))
{
         // There was no compaction finish event, however the last compaction start event
has a file
         // in the metadata table, so the compaction finished.
         log.debug("Considering compaction start {} {} finished because file {} in metadata
table",
-            tid, lastStart, getPathSuffix(lastStartFile));
+            tabletId, lastStart, getPathSuffix(lastStartFile));
         recoverySeq = lastStart;
       }
     }
     return recoverySeq;
   }
 
-  private void playbackMutations(List<Path> recoveryLogs, MutationReceiver mr, int
tid,
+  private void playbackMutations(List<Path> recoveryLogs, MutationReceiver mr, int
tabletId,
       long recoverySeq) throws IOException {
-    LogFileKey start = minKey(MUTATION, tid);
+    LogFileKey start = minKey(MUTATION, tabletId);
     start.seq = recoverySeq;
 
-    LogFileKey end = maxKey(MUTATION, tid);
+    LogFileKey end = maxKey(MUTATION, tabletId);
 
     try (RecoveryLogsIterator rli = new RecoveryLogsIterator(fs, recoveryLogs, start, end))
{
       while (rli.hasNext()) {
         Entry<LogFileKey,LogFileValue> entry = rli.next();
 
-        checkState(entry.getKey().tid == tid); // should only fail if bug elsewhere
+        checkState(entry.getKey().tid == tabletId); // should only fail if bug elsewhere
         checkState(entry.getKey().seq >= recoverySeq); // should only fail if bug elsewhere
 
         if (entry.getKey().event == MUTATION) {
@@ -233,20 +261,20 @@ public class SortedLogRecovery {
     // A tablet may leave a tserver and then come back, in which case it would have a different
and
     // higher tablet id. Only want to consider events in the log related to the last time
the tablet
     // was loaded.
-    int tid = findMaxTabletId(extent, recoveryLogs);
+    int tabletId = findMaxTabletId(extent, recoveryLogs);
 
-    if (tid == -1) {
+    if (tabletId == -1) {
       log.info("Tablet {} is not defined in recovery logs {} ", extent, asNames(recoveryLogs));
       return;
     }
 
     // Find the seq # for the last compaction that started and finished
-    long recoverySeq = findLastStartToFinish(recoveryLogs, tabletFiles, tid);
+    long recoverySeq = findLastStartToFinish(recoveryLogs, tabletFiles, tabletId);
 
-    log.info("Recovering mutations, tablet:{} tid:{} seq:{} logs:{}", extent, tid, recoverySeq,
-        asNames(recoveryLogs));
+    log.info("Recovering mutations, tablet:{} tabletId:{} seq:{} logs:{}", extent, tabletId,
+        recoverySeq, asNames(recoveryLogs));
 
     // Replay all mutations that were written after the last successful compaction started.
-    playbackMutations(recoveryLogs, mr, tid, recoverySeq);
+    playbackMutations(recoveryLogs, mr, tabletId, recoverySeq);
   }
 }
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/logger/LogEvents.java
b/server/tserver/src/main/java/org/apache/accumulo/tserver/logger/LogEvents.java
index 137fe06..044629e 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/logger/LogEvents.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/logger/LogEvents.java
@@ -17,6 +17,9 @@
 package org.apache.accumulo.tserver.logger;
 
 public enum LogEvents {
+  // TODO add unit test to verify ordinals, rather than rely on dubious comments
+  // TODO if possible, rename COMPACTION to "FLUSH" (or at least "MINC") without changing
+  // serialization
   // DO NOT CHANGE ORDER OF ENUMS, ORDER IS USED IN SERIALIZATION
   OPEN, DEFINE_TABLET, MUTATION, MANY_MUTATIONS, COMPACTION_START, COMPACTION_FINISH;
 }
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/logger/LogFileKey.java
b/server/tserver/src/main/java/org/apache/accumulo/tserver/logger/LogFileKey.java
index 6cf9d08..399874a 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/logger/LogFileKey.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/logger/LogFileKey.java
@@ -34,7 +34,7 @@ public class LogFileKey implements WritableComparable<LogFileKey>
{
   public String filename = null;
   public KeyExtent tablet = null;
   public long seq = -1;
-  public int tid = -1;
+  public int tid = -1; // TODO rename to tabletId
   public static final int VERSION = 2;
   public String tserverSession;
 
diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/logger/LogReader.java
b/server/tserver/src/main/java/org/apache/accumulo/tserver/logger/LogReader.java
index 927e345..ef26a36 100644
--- a/server/tserver/src/main/java/org/apache/accumulo/tserver/logger/LogReader.java
+++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/logger/LogReader.java
@@ -37,7 +37,7 @@ import org.apache.accumulo.server.fs.VolumeManagerImpl;
 import org.apache.accumulo.tserver.log.DfsLogger;
 import org.apache.accumulo.tserver.log.DfsLogger.DFSLoggerInputStreams;
 import org.apache.accumulo.tserver.log.DfsLogger.LogHeaderIncompleteException;
-import org.apache.accumulo.tserver.log.MultiReader;
+import org.apache.accumulo.tserver.log.RecoveryLogReader;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
@@ -129,7 +129,7 @@ public class LogReader {
         }
       } else {
         // read the log entries sorted in a map file
-        MultiReader input = new MultiReader(fs, path);
+        RecoveryLogReader input = new RecoveryLogReader(fs, path);
         while (input.next(key, value)) {
           printLogEvent(key, value, row, rowMatcher, ke, tabletIds, opts.maxMutations);
         }
diff --git a/server/tserver/src/test/java/org/apache/accumulo/tserver/log/MultiReaderTest.java
b/server/tserver/src/test/java/org/apache/accumulo/tserver/log/RecoveryLogsReaderTest.java
similarity index 93%
rename from server/tserver/src/test/java/org/apache/accumulo/tserver/log/MultiReaderTest.java
rename to server/tserver/src/test/java/org/apache/accumulo/tserver/log/RecoveryLogsReaderTest.java
index a6df17c..166a381 100644
--- a/server/tserver/src/test/java/org/apache/accumulo/tserver/log/MultiReaderTest.java
+++ b/server/tserver/src/test/java/org/apache/accumulo/tserver/log/RecoveryLogsReaderTest.java
@@ -35,7 +35,7 @@ import org.junit.Before;
 import org.junit.Test;
 import org.junit.rules.TemporaryFolder;
 
-public class MultiReaderTest {
+public class RecoveryLogsReaderTest {
 
   VolumeManager fs;
   TemporaryFolder root = new TemporaryFolder(new File(System.getProperty("user.dir") + "/target"));
@@ -73,7 +73,7 @@ public class MultiReaderTest {
     root.create();
   }
 
-  private void scan(MultiReader reader, int start) throws IOException {
+  private void scan(RecoveryLogReader reader, int start) throws IOException {
     IntWritable key = new IntWritable();
     BytesWritable value = new BytesWritable();
 
@@ -85,7 +85,7 @@ public class MultiReaderTest {
     }
   }
 
-  private void scanOdd(MultiReader reader, int start) throws IOException {
+  private void scanOdd(RecoveryLogReader reader, int start) throws IOException {
     IntWritable key = new IntWritable();
     BytesWritable value = new BytesWritable();
 
@@ -98,7 +98,7 @@ public class MultiReaderTest {
   @Test
   public void testMultiReader() throws IOException {
     Path manyMaps = new Path("file://" + root.getRoot().getAbsolutePath() + "/manyMaps");
-    MultiReader reader = new MultiReader(fs, manyMaps);
+    RecoveryLogReader reader = new RecoveryLogReader(fs, manyMaps);
     IntWritable key = new IntWritable();
     BytesWritable value = new BytesWritable();
 
@@ -128,7 +128,7 @@ public class MultiReaderTest {
     reader.close();
 
     fs.deleteRecursively(new Path(manyMaps, "even"));
-    reader = new MultiReader(fs, manyMaps);
+    reader = new RecoveryLogReader(fs, manyMaps);
     key.set(501);
     assertTrue(reader.seek(key));
     scanOdd(reader, 501);

-- 
To stop receiving notification emails like this one, please contact
kturner@apache.org.

Mime
View raw message