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/05/17 06:09:24 UTC

[GitHub] [iceberg] hameizi commented on a diff in pull request #4744: Core: Incremental append scan impl

hameizi commented on code in PR #4744:
URL: https://github.com/apache/iceberg/pull/4744#discussion_r874402526


##########
core/src/main/java/org/apache/iceberg/BaseIncrementalAppendScan.java:
##########
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg;
+
+import java.util.List;
+import java.util.Set;
+import org.apache.iceberg.events.IncrementalScanEvent;
+import org.apache.iceberg.events.Listeners;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.FluentIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.util.SnapshotUtil;
+import org.apache.iceberg.util.TableScanUtil;
+
+class BaseIncrementalAppendScan extends BaseScan<IncrementalAppendScan> implements IncrementalAppendScan {
+
+  BaseIncrementalAppendScan(TableOperations ops, Table table) {
+    this(ops, table, table.schema(), new TableScanContext());
+  }
+
+  BaseIncrementalAppendScan(TableOperations ops, Table table, Schema schema, TableScanContext context) {
+    super(ops, table, schema, context);
+  }
+
+  @Override
+  protected IncrementalAppendScan newRefinedScan(
+      TableOperations newOps, Table newTable, Schema newSchema, TableScanContext newContext) {
+    return new BaseIncrementalAppendScan(newOps, newTable, newSchema, newContext);
+  }
+
+  @Override
+  public IncrementalAppendScan fromSnapshotInclusive(long fromSnapshotId) {
+    Snapshot startingSnapshotInclusive = table().snapshot(fromSnapshotId);
+    Preconditions.checkArgument(startingSnapshotInclusive != null,
+        "Cannot find the starting snapshot: %d", fromSnapshotId);
+    // To achieve inclusive behavior, we pass the parent snapshot id to the TableScanContext.
+    // If parent id is null, leave the TableScanContext#fromSnapshotId as null.
+    // This way, TableScanContext#fromSnapshotId always capture the exclusive behavior.
+    if (startingSnapshotInclusive.parentId() != null) {
+      return newRefinedScan(tableOps(), table(), schema(),
+          context().fromSnapshotId(startingSnapshotInclusive.parentId()));
+    }
+    return this;
+  }
+
+  @Override
+  public IncrementalAppendScan fromSnapshotExclusive(long fromSnapshotId) {
+    Preconditions.checkArgument(table().snapshot(fromSnapshotId) != null,
+        "Cannot find starting snapshot: %d", fromSnapshotId);
+    return newRefinedScan(tableOps(), table(), schema(), context().fromSnapshotId(fromSnapshotId));
+  }
+
+  @Override
+  public IncrementalAppendScan toSnapshot(long toSnapshotId) {
+    Preconditions.checkArgument(table().snapshot(toSnapshotId) != null,
+        "Cannot find end snapshot: %d", toSnapshotId);
+    return newRefinedScan(tableOps(), table(), schema(),  context().toSnapshotId(toSnapshotId));
+  }
+
+  @Override
+  public CloseableIterable<FileScanTask> planFiles() {
+    // Return an empty iterable when table is empty
+    if (table().currentSnapshot() == null) {
+      return CloseableIterable.empty();
+    }
+
+    long toSnapshotIdInclusive = context().toSnapshotId() != null ?
+        context().toSnapshotId() : table().currentSnapshot().snapshotId();
+    if (context().fromSnapshotId() != null) {
+      Preconditions.checkArgument(
+          SnapshotUtil.isAncestorOf(table(), toSnapshotIdInclusive, context().fromSnapshotId()),
+          "Starting snapshot (exclusive) %d is not an ancestor of end snapshot %d",
+          context().fromSnapshotId(), toSnapshotIdInclusive);
+    }
+
+    // appendsBetween handles null fromSnapshotId (exclusive) properly
+    List<Snapshot> snapshots = appendsBetween(table(), context().fromSnapshotId(), toSnapshotIdInclusive);
+    if (snapshots.isEmpty()) {
+      return CloseableIterable.empty();
+    }
+
+    Set<Long> snapshotIds = Sets.newHashSet(Iterables.transform(snapshots, Snapshot::snapshotId));
+    Set<ManifestFile> manifests = FluentIterable
+        .from(snapshots)
+        .transformAndConcat(Snapshot::dataManifests)
+        .filter(manifestFile -> snapshotIds.contains(manifestFile.snapshotId()))
+        .toSet();
+
+    ManifestGroup manifestGroup = new ManifestGroup(tableOps().io(), manifests)
+        .caseSensitive(context().caseSensitive())
+        .select(context().returnColumnStats() ? DataTableScan.SCAN_WITH_STATS_COLUMNS : DataTableScan.SCAN_COLUMNS)
+        .filterData(context().rowFilter())
+        .filterManifestEntries(
+            manifestEntry ->
+                snapshotIds.contains(manifestEntry.snapshotId()) &&
+                    manifestEntry.status() == ManifestEntry.Status.ADDED)
+        .specsById(tableOps().current().specsById())
+        .ignoreDeleted();
+
+    if (context().ignoreResiduals()) {
+      manifestGroup = manifestGroup.ignoreResiduals();
+    }
+
+    long fromSnapshotId = resolveFromSnapshotIdForListenerEvent(
+        table(), context().fromSnapshotId(), toSnapshotIdInclusive);
+    Listeners.notifyAll(new IncrementalScanEvent(table().name(), fromSnapshotId,
+        toSnapshotIdInclusive, context().rowFilter(), table().schema()));
+
+    if (manifests.size() > 1 &&
+        (DataTableScan.PLAN_SCANS_WITH_WORKER_POOL || context().planWithCustomizedExecutor())) {
+      manifestGroup = manifestGroup.planWith(context().planExecutor());
+    }
+
+    return manifestGroup.planFiles();

Review Comment:
   `manifestGroup.planFiles()` will apply all delete-file whose seq number greater than datafile even though the delete file is not belong to the snapshots we scan, is it the intended effect? 



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