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



##########
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:
       Not sure if this scanner is closed.  
   
   This would be a follow on issue, thinking now that we are using RFile we 
could possibly have a dedicated little block cache for RFile recovery.  This 
could make opening the same rfiles multiple times for recovery purposes much 
faster.  These rfiles are opened here and later.  However they may also be 
opened between tablets. 

##########
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());

Review comment:
       Could make formatRow private and add a static method `Range 
LogFileKey.toRange(LogFileKey start, LogFileKey end)`.  My thinking is that 
this brings all the code for translating into one place in the LogFileKey class.

##########
File path: 
server/tserver/src/main/java/org/apache/accumulo/tserver/logger/LogFileKey.java
##########
@@ -189,4 +196,158 @@ public String toString() {
     }
     throw new RuntimeException("Unknown type of entry: " + event);
   }
+
+  /**
+   * Converts LogFileKey to Key. Creates a Key containing all of the 
LogFileKey fields. The fields
+   * are stored so the Key sorts maintaining the legacy sort order. The row of 
the Key is composed
+   * of 3 fields: EventNum + tabletID + seq. The EventNum is the byte returned 
by eventType(). The
+   * column family is always the event. The column qualifier is dependent of 
the type of event and
+   * could be empty.
+   *
+   * <pre>
+   *     Key Schema:
+   *     Row = EventNum + tabletID + seq
+   *     Family = event
+   *     Qualifier = tserverSession OR filename OR KeyExtent
+   * </pre>
+   */
+  public Key toKey() throws IOException {
+    byte[] formattedRow;
+    Text family = new Text(event.name());
+    var kb = Key.builder();
+    switch (event) {
+      case OPEN:
+        formattedRow = formatRow(0, 0);
+        return kb.row(formattedRow).family(family).qualifier(new 
Text(tserverSession)).build();
+      case COMPACTION_START:
+        formattedRow = formatRow(tabletId, seq);
+        return kb.row(formattedRow).family(family).qualifier(new 
Text(filename)).build();

Review comment:
       ```suggestion
           return 
kb.row(formattedRow).family(family).qualifier(filename).build();
   ```

##########
File path: 
server/tserver/src/main/java/org/apache/accumulo/tserver/logger/LogFileKey.java
##########
@@ -189,4 +196,158 @@ public String toString() {
     }
     throw new RuntimeException("Unknown type of entry: " + event);
   }
+
+  /**
+   * Converts LogFileKey to Key. Creates a Key containing all of the 
LogFileKey fields. The fields
+   * are stored so the Key sorts maintaining the legacy sort order. The row of 
the Key is composed
+   * of 3 fields: EventNum + tabletID + seq. The EventNum is the byte returned 
by eventType(). The
+   * column family is always the event. The column qualifier is dependent of 
the type of event and
+   * could be empty.
+   *
+   * <pre>
+   *     Key Schema:
+   *     Row = EventNum + tabletID + seq
+   *     Family = event
+   *     Qualifier = tserverSession OR filename OR KeyExtent
+   * </pre>
+   */
+  public Key toKey() throws IOException {
+    byte[] formattedRow;
+    Text family = new Text(event.name());
+    var kb = Key.builder();
+    switch (event) {
+      case OPEN:
+        formattedRow = formatRow(0, 0);
+        return kb.row(formattedRow).family(family).qualifier(new 
Text(tserverSession)).build();
+      case COMPACTION_START:
+        formattedRow = formatRow(tabletId, seq);
+        return kb.row(formattedRow).family(family).qualifier(new 
Text(filename)).build();
+      case MUTATION:
+      case MANY_MUTATIONS:
+      case COMPACTION_FINISH:
+        return kb.row(formatRow(tabletId, seq)).family(family).build();
+      case DEFINE_TABLET:
+        formattedRow = formatRow(tabletId, seq);
+        DataOutputBuffer buffer = new DataOutputBuffer();
+        tablet.writeTo(buffer);
+        var q = copyOf(buffer.getData(), buffer.getLength());
+        buffer.close();
+        return kb.row(formattedRow).family(family).qualifier(q).build();
+      default:
+        throw new AssertionError("Invalid event type in LogFileKey: " + event);
+    }
+  }
+
+  /**
+   * Get the first byte for the event. The only possible values are 0-4. This 
is used as the highest
+   * byte in the row.
+   */
+  private byte getEventByte() {
+    int evenTypeInteger = eventType(event);
+    return (byte) (evenTypeInteger & 0xff);
+  }
+
+  /**
+   * Get the byte encoded row for this LogFileKey as a Text object.
+   */
+  public Text formatRow() {
+    return new Text(formatRow(tabletId, seq));
+  }
+
+  /**
+   * Format the row using 13 bytes encoded to allow proper sorting of the 
RFile Key. The highest
+   * byte is for the event number, 4 bytes for the tabletId and 8 bytes for 
the sequence long.
+   */
+  private byte[] formatRow(int tabletId, long seq) {
+    byte eventNum = getEventByte();
+    // These will not sort properly when encoded if negative. Negative is not 
expected currently,
+    // defending against future changes and/or bugs.
+    Preconditions.checkArgument(eventNum >= 0 && seq >= 0);
+    byte[] row = new byte[13];
+    // encode the signed integer so negatives will sort properly for tabletId
+    int encodedTabletId = tabletId ^ 0x80000000;
+
+    row[0] = eventNum;
+    row[1] = (byte) ((encodedTabletId >>> 24) & 0xff);
+    row[2] = (byte) ((encodedTabletId >>> 16) & 0xff);
+    row[3] = (byte) ((encodedTabletId >>> 8) & 0xff);
+    row[4] = (byte) (encodedTabletId & 0xff);
+    row[5] = (byte) (seq >>> 56);
+    row[6] = (byte) (seq >>> 48);
+    row[7] = (byte) (seq >>> 40);
+    row[8] = (byte) (seq >>> 32);
+    row[9] = (byte) (seq >>> 24);
+    row[10] = (byte) (seq >>> 16);
+    row[11] = (byte) (seq >>> 8);
+    row[12] = (byte) (seq); // >>> 0
+    return row;
+  }
+
+  /**
+   * Extract the tabletId integer from the byte encoded Row.
+   */
+  private static int getTabletId(byte[] row) {
+    int encoded = ((row[1] << 24) + (row[2] << 16) + (row[3] << 8) + row[4]);
+    return encoded ^ 0x80000000;
+  }
+
+  /**
+   * Extract the sequence long from the byte encoded Row.
+   */
+  private static long getSequence(byte[] row) {
+    // @formatter:off
+    return (((long) row[5] << 56) +
+            ((long) (row[6] & 255) << 48) +
+            ((long) (row[7] & 255) << 40) +
+            ((long) (row[8] & 255) << 32) +
+            ((long) (row[9] & 255) << 24) +
+            ((row[10] & 255) << 16) +
+            ((row[11] & 255) << 8) +
+            ((row[12] & 255)));
+    // @formatter:on
+  }
+
+  /**
+   * Create LogFileKey from row. Follows schema defined by {@link #toKey()}
+   */
+  public static LogFileKey fromKey(Key key) {
+    var logFileKey = new LogFileKey();
+    byte[] rowParts = key.getRow().getBytes();
+
+    logFileKey.tabletId = getTabletId(rowParts);
+    logFileKey.seq = getSequence(rowParts);
+    logFileKey.event = LogEvents.valueOf(key.getColumnFamilyData().toString());
+    // verify event number in row matches column family
+    if (eventType(logFileKey.event) != rowParts[0]) {
+      throw new AssertionError("Event in row differs from column family. Key: 
" + key);
+    }
+
+    // handle special cases of what is stored in the qualifier
+    switch (logFileKey.event) {
+      case OPEN:
+        logFileKey.tserverSession = key.getColumnQualifier().toString();
+        break;
+      case COMPACTION_START:
+        logFileKey.filename = key.getColumnQualifier().toString();

Review comment:
       ```suggestion
           logFileKey.filename = key.getColumnQualifierData().toString();
   ```

##########
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 {

Review comment:
       Could colFamToFetch be removed?  The first byte in the row is derived 
from the event type.  Not sure, but it seems like the columns fetched always 
correspond to the ones that map to this first byte of the row.  If so then the 
request to fetch columns is never filtering anything.  If its not needed, 
removing it would simplify the code.

##########
File path: 
server/tserver/src/main/java/org/apache/accumulo/tserver/logger/LogFileKey.java
##########
@@ -189,4 +196,158 @@ public String toString() {
     }
     throw new RuntimeException("Unknown type of entry: " + event);
   }
+
+  /**
+   * Converts LogFileKey to Key. Creates a Key containing all of the 
LogFileKey fields. The fields
+   * are stored so the Key sorts maintaining the legacy sort order. The row of 
the Key is composed
+   * of 3 fields: EventNum + tabletID + seq. The EventNum is the byte returned 
by eventType(). The
+   * column family is always the event. The column qualifier is dependent of 
the type of event and
+   * could be empty.
+   *
+   * <pre>
+   *     Key Schema:
+   *     Row = EventNum + tabletID + seq
+   *     Family = event
+   *     Qualifier = tserverSession OR filename OR KeyExtent
+   * </pre>
+   */
+  public Key toKey() throws IOException {
+    byte[] formattedRow;
+    Text family = new Text(event.name());

Review comment:
       Thinking the key builder can handle string
   
   ```suggestion
       String family = event.name();
   ```

##########
File path: 
server/tserver/src/main/java/org/apache/accumulo/tserver/logger/LogFileKey.java
##########
@@ -189,4 +196,158 @@ public String toString() {
     }
     throw new RuntimeException("Unknown type of entry: " + event);
   }
+
+  /**
+   * Converts LogFileKey to Key. Creates a Key containing all of the 
LogFileKey fields. The fields
+   * are stored so the Key sorts maintaining the legacy sort order. The row of 
the Key is composed
+   * of 3 fields: EventNum + tabletID + seq. The EventNum is the byte returned 
by eventType(). The
+   * column family is always the event. The column qualifier is dependent of 
the type of event and
+   * could be empty.
+   *
+   * <pre>
+   *     Key Schema:
+   *     Row = EventNum + tabletID + seq
+   *     Family = event
+   *     Qualifier = tserverSession OR filename OR KeyExtent
+   * </pre>
+   */
+  public Key toKey() throws IOException {
+    byte[] formattedRow;
+    Text family = new Text(event.name());
+    var kb = Key.builder();
+    switch (event) {
+      case OPEN:
+        formattedRow = formatRow(0, 0);
+        return kb.row(formattedRow).family(family).qualifier(new 
Text(tserverSession)).build();

Review comment:
       ```suggestion
           return 
kb.row(formattedRow).family(family).qualifier(tserverSession).build();
   ```

##########
File path: 
server/tserver/src/main/java/org/apache/accumulo/tserver/log/RecoveryLogsIterator.java
##########
@@ -93,11 +121,45 @@ public void remove() {
 
   @Override
   public void close() {
-    for (CloseableIterator<?> reader : iterators) {
-      try {
-        reader.close();
-      } catch (IOException e) {
-        LOG.debug("Failed to close reader", e);
+    scanners.forEach(ScannerBase::close);
+  }
+
+  /**
+   * Check for sorting signal files (finished/failed) and get the logs in the 
provided directory.
+   */
+  private List<Path> getFiles(VolumeManager fs, Path directory) throws 
IOException {
+    boolean foundFinish = false;
+    List<Path> logFiles = new ArrayList<>();
+    for (FileStatus child : fs.listStatus(directory)) {
+      if (child.getPath().getName().startsWith("_"))
+        continue;
+      if (SortedLogState.isFinished(child.getPath().getName())) {
+        foundFinish = true;
+        continue;
+      }
+      if (SortedLogState.FAILED.getMarker().equals(child.getPath().getName())) 
{

Review comment:
       This seems to just ignore failed markers.  Is this what it did before?  
I have a vague memory of an issue around this and we switched to ignoring it, 
but not sure.

##########
File path: 
server/tserver/src/main/java/org/apache/accumulo/tserver/logger/LogFileKey.java
##########
@@ -189,4 +196,158 @@ public String toString() {
     }
     throw new RuntimeException("Unknown type of entry: " + event);
   }
+
+  /**
+   * Converts LogFileKey to Key. Creates a Key containing all of the 
LogFileKey fields. The fields
+   * are stored so the Key sorts maintaining the legacy sort order. The row of 
the Key is composed
+   * of 3 fields: EventNum + tabletID + seq. The EventNum is the byte returned 
by eventType(). The
+   * column family is always the event. The column qualifier is dependent of 
the type of event and
+   * could be empty.
+   *
+   * <pre>
+   *     Key Schema:
+   *     Row = EventNum + tabletID + seq
+   *     Family = event
+   *     Qualifier = tserverSession OR filename OR KeyExtent
+   * </pre>
+   */
+  public Key toKey() throws IOException {
+    byte[] formattedRow;
+    Text family = new Text(event.name());
+    var kb = Key.builder();
+    switch (event) {
+      case OPEN:
+        formattedRow = formatRow(0, 0);
+        return kb.row(formattedRow).family(family).qualifier(new 
Text(tserverSession)).build();
+      case COMPACTION_START:
+        formattedRow = formatRow(tabletId, seq);
+        return kb.row(formattedRow).family(family).qualifier(new 
Text(filename)).build();
+      case MUTATION:
+      case MANY_MUTATIONS:
+      case COMPACTION_FINISH:
+        return kb.row(formatRow(tabletId, seq)).family(family).build();
+      case DEFINE_TABLET:
+        formattedRow = formatRow(tabletId, seq);
+        DataOutputBuffer buffer = new DataOutputBuffer();
+        tablet.writeTo(buffer);
+        var q = copyOf(buffer.getData(), buffer.getLength());
+        buffer.close();
+        return kb.row(formattedRow).family(family).qualifier(q).build();
+      default:
+        throw new AssertionError("Invalid event type in LogFileKey: " + event);
+    }
+  }
+
+  /**
+   * Get the first byte for the event. The only possible values are 0-4. This 
is used as the highest
+   * byte in the row.
+   */
+  private byte getEventByte() {
+    int evenTypeInteger = eventType(event);
+    return (byte) (evenTypeInteger & 0xff);
+  }
+
+  /**
+   * Get the byte encoded row for this LogFileKey as a Text object.
+   */
+  public Text formatRow() {
+    return new Text(formatRow(tabletId, seq));
+  }
+
+  /**
+   * Format the row using 13 bytes encoded to allow proper sorting of the 
RFile Key. The highest
+   * byte is for the event number, 4 bytes for the tabletId and 8 bytes for 
the sequence long.
+   */
+  private byte[] formatRow(int tabletId, long seq) {
+    byte eventNum = getEventByte();
+    // These will not sort properly when encoded if negative. Negative is not 
expected currently,
+    // defending against future changes and/or bugs.
+    Preconditions.checkArgument(eventNum >= 0 && seq >= 0);
+    byte[] row = new byte[13];
+    // encode the signed integer so negatives will sort properly for tabletId
+    int encodedTabletId = tabletId ^ 0x80000000;
+
+    row[0] = eventNum;
+    row[1] = (byte) ((encodedTabletId >>> 24) & 0xff);
+    row[2] = (byte) ((encodedTabletId >>> 16) & 0xff);
+    row[3] = (byte) ((encodedTabletId >>> 8) & 0xff);
+    row[4] = (byte) (encodedTabletId & 0xff);
+    row[5] = (byte) (seq >>> 56);
+    row[6] = (byte) (seq >>> 48);
+    row[7] = (byte) (seq >>> 40);
+    row[8] = (byte) (seq >>> 32);
+    row[9] = (byte) (seq >>> 24);
+    row[10] = (byte) (seq >>> 16);
+    row[11] = (byte) (seq >>> 8);
+    row[12] = (byte) (seq); // >>> 0
+    return row;
+  }
+
+  /**
+   * Extract the tabletId integer from the byte encoded Row.
+   */
+  private static int getTabletId(byte[] row) {
+    int encoded = ((row[1] << 24) + (row[2] << 16) + (row[3] << 8) + row[4]);
+    return encoded ^ 0x80000000;
+  }
+
+  /**
+   * Extract the sequence long from the byte encoded Row.
+   */
+  private static long getSequence(byte[] row) {
+    // @formatter:off
+    return (((long) row[5] << 56) +
+            ((long) (row[6] & 255) << 48) +
+            ((long) (row[7] & 255) << 40) +
+            ((long) (row[8] & 255) << 32) +
+            ((long) (row[9] & 255) << 24) +
+            ((row[10] & 255) << 16) +
+            ((row[11] & 255) << 8) +
+            ((row[12] & 255)));
+    // @formatter:on
+  }
+
+  /**
+   * Create LogFileKey from row. Follows schema defined by {@link #toKey()}
+   */
+  public static LogFileKey fromKey(Key key) {
+    var logFileKey = new LogFileKey();
+    byte[] rowParts = key.getRow().getBytes();
+
+    logFileKey.tabletId = getTabletId(rowParts);
+    logFileKey.seq = getSequence(rowParts);
+    logFileKey.event = LogEvents.valueOf(key.getColumnFamilyData().toString());
+    // verify event number in row matches column family
+    if (eventType(logFileKey.event) != rowParts[0]) {
+      throw new AssertionError("Event in row differs from column family. Key: 
" + key);
+    }
+
+    // handle special cases of what is stored in the qualifier
+    switch (logFileKey.event) {
+      case OPEN:
+        logFileKey.tserverSession = key.getColumnQualifier().toString();

Review comment:
       ```suggestion
           logFileKey.tserverSession = key.getColumnQualifierData().toString();
   ```




-- 
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