You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2022/07/01 19:33:01 UTC

[GitHub] [iceberg] rdblue commented on a diff in pull request #4847: Core: Add reference_snapshot_id column and filter to AllManifest table

rdblue commented on code in PR #4847:
URL: https://github.com/apache/iceberg/pull/4847#discussion_r912210579


##########
core/src/main/java/org/apache/iceberg/AllManifestsTable.java:
##########
@@ -204,4 +229,186 @@ public Iterable<FileScanTask> split(long splitSize) {
       return ImmutableList.of(this); // don't split
     }
   }
+
+  static StaticDataTask.Row manifestFileToRow(PartitionSpec spec, ManifestFile manifest, long referenceSnapshotId) {
+    return StaticDataTask.Row.of(
+        manifest.content().id(),
+        manifest.path(),
+        manifest.length(),
+        manifest.partitionSpecId(),
+        manifest.snapshotId(),
+        manifest.content() == ManifestContent.DATA ? manifest.addedFilesCount() : 0,
+        manifest.content() == ManifestContent.DATA ? manifest.existingFilesCount() : 0,
+        manifest.content() == ManifestContent.DATA ? manifest.deletedFilesCount() : 0,
+        manifest.content() == ManifestContent.DELETES ? manifest.addedFilesCount() : 0,
+        manifest.content() == ManifestContent.DELETES ? manifest.existingFilesCount() : 0,
+        manifest.content() == ManifestContent.DELETES ? manifest.deletedFilesCount() : 0,
+        ManifestsTable.partitionSummariesToRows(spec, manifest.partitions()),
+        referenceSnapshotId
+    );
+  }
+
+  private static class SnapshotEvaluator {
+
+    private final Expression boundExpr;
+
+    private SnapshotEvaluator(Expression expr, Types.StructType structType, boolean caseSensitive) {
+      this.boundExpr = Binder.bind(structType, expr, caseSensitive);
+    }
+
+    private boolean eval(Snapshot snapshot) {
+      return new SnapshotEvalVisitor().eval(snapshot);
+    }
+
+    private class SnapshotEvalVisitor extends BoundExpressionVisitor<Boolean> {
+
+      private long snapshotId;
+      private static final boolean ROWS_MIGHT_MATCH = true;
+      private static final boolean ROWS_CANNOT_MATCH = false;
+
+      private boolean eval(Snapshot snapshot) {
+        this.snapshotId = snapshot.snapshotId();
+        return ExpressionVisitors.visitEvaluator(boundExpr, this);
+      }
+
+      @Override
+      public Boolean alwaysTrue() {
+        return ROWS_MIGHT_MATCH;
+      }
+
+      @Override
+      public Boolean alwaysFalse() {
+        return ROWS_CANNOT_MATCH;
+      }
+
+      @Override
+      public Boolean not(Boolean result) {
+        return !result;
+      }
+
+      @Override
+      public Boolean and(Boolean leftResult, Boolean rightResult) {
+        return leftResult && rightResult;
+      }
+
+      @Override
+      public Boolean or(Boolean leftResult, Boolean rightResult) {
+        return leftResult || rightResult;
+      }
+
+      @Override
+      public <T> Boolean isNull(BoundReference<T> ref) {
+        return ROWS_MIGHT_MATCH;
+      }
+
+      @Override
+      public <T> Boolean notNull(BoundReference<T> ref) {
+        return ROWS_MIGHT_MATCH;
+      }
+
+      @Override
+      public <T> Boolean isNaN(BoundReference<T> ref) {
+        if (isSnapshotRef(ref)) {
+          return ROWS_CANNOT_MATCH; // reference_snapshot_id column is never nan
+        } else {
+          return ROWS_MIGHT_MATCH;
+        }
+      }
+
+      @Override
+      public <T> Boolean notNaN(BoundReference<T> ref) {
+        return ROWS_MIGHT_MATCH;
+      }
+
+      @Override
+      public <T> Boolean lt(BoundReference<T> ref, Literal<T> lit) {
+        return compareSnapshotRef(ref, lit, compareResult -> compareResult < 0);
+      }
+
+      @Override
+      public <T> Boolean ltEq(BoundReference<T> ref, Literal<T> lit) {
+        return compareSnapshotRef(ref, lit, compareResult -> compareResult <= 0);
+      }
+
+      @Override
+      public <T> Boolean gt(BoundReference<T> ref, Literal<T> lit) {
+        return compareSnapshotRef(ref, lit, compareResult -> compareResult > 0);
+      }
+
+      @Override
+      public <T> Boolean gtEq(BoundReference<T> ref, Literal<T> lit) {
+        return compareSnapshotRef(ref, lit, compareResult -> compareResult >= 0);
+      }
+
+      @Override
+      public <T> Boolean eq(BoundReference<T> ref, Literal<T> lit) {
+        return compareSnapshotRef(ref, lit, compareResult -> compareResult == 0);
+      }
+
+      @Override
+      public <T> Boolean notEq(BoundReference<T> ref, Literal<T> lit) {
+        return compareSnapshotRef(ref, lit, compareResult -> compareResult != 0);
+      }
+
+      @Override
+      public <T> Boolean in(BoundReference<T> ref, Set<T> literalSet) {
+        if (isSnapshotRef(ref)) {
+          Comparator<Object> longComparator = Comparators.forType(Types.LongType.get());
+          boolean noneMatch = literalSet.stream().noneMatch(lit -> longComparator.compare(snapshotId, lit) == 0);

Review Comment:
   Sorry I'm late to this review, but the `literalSet` will use the correct comparator for a given type.



-- 
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: issues-unsubscribe@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org