You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@accumulo.apache.org by cs...@apache.org on 2023/07/19 11:26:20 UTC

[accumulo] branch no-chop-merge updated: Move range validation to AbstractTabletFile (#3621)

This is an automated email from the ASF dual-hosted git repository.

cshannon pushed a commit to branch no-chop-merge
in repository https://gitbox.apache.org/repos/asf/accumulo.git


The following commit(s) were added to refs/heads/no-chop-merge by this push:
     new 647710e96c Move range validation to AbstractTabletFile (#3621)
647710e96c is described below

commit 647710e96cbef763b47c1b97e3bd48ee465f1944
Author: Christopher L. Shannon <ch...@gmail.com>
AuthorDate: Wed Jul 19 07:26:14 2023 -0400

    Move range validation to AbstractTabletFile (#3621)
    
    Also add validation to StoredTabletFile validation method and on
    serialization
---
 .../accumulo/core/metadata/AbstractTabletFile.java | 27 +++++++++++++----
 .../core/metadata/ReferencedTabletFile.java        |  3 +-
 .../accumulo/core/metadata/StoredTabletFile.java   | 34 ++++------------------
 3 files changed, 28 insertions(+), 36 deletions(-)

diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/AbstractTabletFile.java b/core/src/main/java/org/apache/accumulo/core/metadata/AbstractTabletFile.java
index 89f4b4bc6b..231e4d97e9 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/AbstractTabletFile.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/AbstractTabletFile.java
@@ -20,6 +20,7 @@ package org.apache.accumulo.core.metadata;
 
 import java.util.Objects;
 
+import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Range;
 import org.apache.hadoop.fs.Path;
 
@@ -39,12 +40,7 @@ public abstract class AbstractTabletFile<T extends AbstractTabletFile<T>>
 
   protected AbstractTabletFile(Path path, Range range) {
     this.path = Objects.requireNonNull(path);
-    this.range = Objects.requireNonNull(range);
-    // Ensure consistency by requiring ranges use true/false for inclusivity
-    // for start and end keys
-    Preconditions.checkArgument(
-        !hasRange() || (range.isStartKeyInclusive() && !range.isEndKeyInclusive()),
-        "The Range for a TabletFile must be startKeyInclusive=true and endKeyInclusive=false");
+    this.range = requireRowRange(range);
   }
 
   @Override
@@ -62,4 +58,23 @@ public abstract class AbstractTabletFile<T extends AbstractTabletFile<T>>
     return !range.isInfiniteStartKey() || !range.isInfiniteStopKey();
   }
 
+  static Range requireRowRange(Range range) {
+    if (!range.isInfiniteStartKey()) {
+      Preconditions.checkArgument(range.isStartKeyInclusive() && isOnlyRowSet(range.getStartKey()),
+          "Range is not a row range %s", range);
+    }
+
+    if (!range.isInfiniteStopKey()) {
+      Preconditions.checkArgument(!range.isEndKeyInclusive() && isOnlyRowSet(range.getEndKey()),
+          "Range is not a row range %s", range);
+    }
+
+    return range;
+  }
+
+  private static boolean isOnlyRowSet(Key key) {
+    return key.getColumnFamilyData().length() == 0 && key.getColumnQualifierData().length() == 0
+        && key.getColumnVisibilityData().length() == 0 && key.getTimestamp() == Long.MAX_VALUE;
+  }
+
 }
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/ReferencedTabletFile.java b/core/src/main/java/org/apache/accumulo/core/metadata/ReferencedTabletFile.java
index bd5af63091..0300f0ee31 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/ReferencedTabletFile.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/ReferencedTabletFile.java
@@ -19,7 +19,6 @@
 package org.apache.accumulo.core.metadata;
 
 import static org.apache.accumulo.core.Constants.HDFS_TABLES_DIR;
-import static org.apache.accumulo.core.metadata.StoredTabletFile.requireRowRange;
 
 import java.net.URI;
 import java.util.Comparator;
@@ -166,7 +165,7 @@ public class ReferencedTabletFile extends AbstractTabletFile<ReferencedTabletFil
    * qualify an absolute path or create a new file.
    */
   public ReferencedTabletFile(Path metaPath, Range range) {
-    super(Objects.requireNonNull(metaPath), requireRowRange(range));
+    super(Objects.requireNonNull(metaPath), range);
     log.trace("Parsing TabletFile from {}", metaPath);
     parts = parsePath(metaPath);
   }
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/StoredTabletFile.java b/core/src/main/java/org/apache/accumulo/core/metadata/StoredTabletFile.java
index 545867bba2..71d3d6b9b4 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/StoredTabletFile.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/StoredTabletFile.java
@@ -33,7 +33,6 @@ import org.apache.hadoop.io.DataInputBuffer;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.io.Text;
 
-import com.google.common.base.Preconditions;
 import com.google.gson.Gson;
 
 /**
@@ -151,7 +150,11 @@ public class StoredTabletFile extends AbstractTabletFile<StoredTabletFile> {
    * Validates that the provided metadata string for the StoredTabletFile is valid.
    */
   public static void validate(String metadataEntry) {
+    final TabletFileCq tabletFileCq = deserialize(metadataEntry);
+    // Validate the path
     ReferencedTabletFile.parsePath(deserialize(metadataEntry).path);
+    // Validate the range
+    requireRowRange(tabletFileCq.range);
   }
 
   public static StoredTabletFile of(final Text metadataEntry) {
@@ -162,32 +165,12 @@ public class StoredTabletFile extends AbstractTabletFile<StoredTabletFile> {
     return new StoredTabletFile(metadataEntry);
   }
 
-  private static boolean isOnlyRowSet(Key key) {
-    return key.getColumnFamilyData().length() == 0 && key.getColumnQualifierData().length() == 0
-        && key.getColumnVisibilityData().length() == 0 && key.getTimestamp() == Long.MAX_VALUE;
-  }
-
-  static Range requireRowRange(Range range) {
-    if (!range.isInfiniteStartKey()) {
-      Preconditions.checkArgument(range.isStartKeyInclusive() && isOnlyRowSet(range.getStartKey()),
-          "Range is not a row range %s", range);
-    }
-
-    if (!range.isInfiniteStopKey()) {
-      Preconditions.checkArgument(!range.isEndKeyInclusive() && isOnlyRowSet(range.getEndKey()),
-          "Range is not a row range %s", range);
-    }
-
-    return range;
-  }
-
   public static StoredTabletFile of(final URI path, Range range) {
     return of(new Path(Objects.requireNonNull(path)), range);
   }
 
   public static StoredTabletFile of(final Path path, Range range) {
-    return new StoredTabletFile(
-        new TabletFileCq(Objects.requireNonNull(path), requireRowRange(range)));
+    return new StoredTabletFile(new TabletFileCq(Objects.requireNonNull(path), range));
   }
 
   private static final Gson gson = ByteArrayToBase64TypeAdapter.createBase64Gson();
@@ -215,14 +198,9 @@ public class StoredTabletFile extends AbstractTabletFile<StoredTabletFile> {
   }
 
   public static String serialize(String path, Range range) {
+    requireRowRange(range);
     final TabletFileCqMetadataGson metadata = new TabletFileCqMetadataGson();
     metadata.path = Objects.requireNonNull(path);
-
-    // TODO - Add validation on start/end rows exclusive/inclusive in a Range if not null?
-    // If we can guarantee start is exlusive and end is inclusive then we don't need to encode
-    // those boolean values or store them.
-    // Should we validate and enforce this when we serialize here or even earlier when we crate the
-    // TabletFile object with a range?
     metadata.startRow = encodeRow(range.getStartKey());
     metadata.endRow = encodeRow(range.getEndKey());