You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@accumulo.apache.org by "cshannon (via GitHub)" <gi...@apache.org> on 2023/04/14 10:46:21 UTC

[GitHub] [accumulo] cshannon commented on a diff in pull request #3286: Add support for storing ranges in DataFileValue

cshannon commented on code in PR #3286:
URL: https://github.com/apache/accumulo/pull/3286#discussion_r1166679798


##########
core/src/main/java/org/apache/accumulo/core/metadata/schema/DataFileValue.java:
##########
@@ -20,42 +20,76 @@
 
 import static java.nio.charset.StandardCharsets.UTF_8;
 
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Optional;
+
+import org.apache.accumulo.core.data.Range;
 import org.apache.accumulo.core.data.Value;
 import org.apache.accumulo.core.iteratorsImpl.system.InterruptibleIterator;
 import org.apache.accumulo.core.iteratorsImpl.system.TimeSettingIterator;
+import org.apache.accumulo.core.util.json.RangeAdapter;
+
+import com.google.gson.Gson;
+import com.google.gson.JsonSyntaxException;
 
 public class DataFileValue {
-  private long size;
-  private long numEntries;
+
+  private static final Gson gson = RangeAdapter.createRangeGson();
+
+  private final long size;
+  private final long numEntries;
   private long time = -1;
+  private final List<Range> ranges;
 
-  public DataFileValue(long size, long numEntries, long time) {
-    this.size = size;
-    this.numEntries = numEntries;
-    this.time = time;
+  public DataFileValue(final long size, final long numEntries, final long time) {
+    this(size, numEntries, time, null);
+  }
+
+  public DataFileValue(final long size, final long numEntries) {
+    this(size, numEntries, null);
   }
 
-  public DataFileValue(long size, long numEntries) {
+  public DataFileValue(final long size, final long numEntries, final long time,
+      final Collection<Range> ranges) {
     this.size = size;
     this.numEntries = numEntries;
-    this.time = -1;
+    this.time = time;
+    // If ranges is null then just set to empty list and also merge overlapping to reduce
+    // the data stored if possible.
+    this.ranges = Optional.ofNullable(ranges).map(Range::mergeOverlapping)
+        .map(Collections::unmodifiableList).orElse(List.of());
   }
 
-  public DataFileValue(String encodedDFV) {
-    String[] ba = encodedDFV.split(",");
-
-    size = Long.parseLong(ba[0]);
-    numEntries = Long.parseLong(ba[1]);
+  public DataFileValue(long size, long numEntries, Collection<Range> ranges) {
+    this(size, numEntries, -1, ranges);
+  }
 
-    if (ba.length == 3) {
-      time = Long.parseLong(ba[2]);
-    } else {
-      time = -1;
+  public static DataFileValue decode(String encodedDFV) {
+    try {
+      // Optimistically try and decode from Json and fall back to decoding the legacy format
+      // if json parsing fails. Over time the old format will be replaced with the new format
+      // as new values are written or updated.
+      return gson.fromJson(encodedDFV, DataFileValue.class);
+    } catch (JsonSyntaxException e) {
+      return decodeLegacy(encodedDFV);

Review Comment:
   I think that is a good point about the code sticking around a long time which is never great. I was thinking about the upgrade option a bit after @ctubbsii mentioned it but I decided against doing the automatic upgrade in this PR for now just because I wanted to get the new format finalized first for the initial draft version. I agree that once this is finalized/merged we can do a follow on issue for upgrades when the format has been fully decided on and then take away the legacy fallback code.



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

To unsubscribe, e-mail: notifications-unsubscribe@accumulo.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org