flyrain commented on code in PR #5382:
URL: https://github.com/apache/iceberg/pull/5382#discussion_r938109281
##########
core/src/main/java/org/apache/iceberg/util/SnapshotUtil.java:
##########
@@ -138,6 +138,10 @@ public static Snapshot oldestAncestorOf(long snapshotId,
Function<Long, Snapshot
return lastSnapshot;
}
+ public static Snapshot oldestAncestorOf(Table table, long snapshotId) {
Review Comment:
Nit: shall we move this method to line 121. We usually put caller before
callee. Something like this.
```
A Caller of B
B Caller of C
C
```
##########
core/src/main/java/org/apache/iceberg/BaseIncrementalChangelogScan.java:
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.ArrayDeque;
+import java.util.Collection;
+import java.util.Deque;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.ManifestGroup.CreateTasksFunction;
+import org.apache.iceberg.ManifestGroup.TaskContext;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.FluentIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.util.SnapshotUtil;
+import org.apache.iceberg.util.TableScanUtil;
+
+class BaseIncrementalChangelogScan
+ extends BaseIncrementalScan<
+ IncrementalChangelogScan, ChangelogScanTask,
ScanTaskGroup<ChangelogScanTask>>
+ implements IncrementalChangelogScan {
+
+ BaseIncrementalChangelogScan(TableOperations ops, Table table) {
+ this(ops, table, table.schema(), new TableScanContext());
+ }
+
+ BaseIncrementalChangelogScan(
+ TableOperations ops, Table table, Schema schema, TableScanContext
context) {
+ super(ops, table, schema, context);
+ }
+
+ @Override
+ protected IncrementalChangelogScan newRefinedScan(
+ TableOperations newOps, Table newTable, Schema newSchema,
TableScanContext newContext) {
+ return new BaseIncrementalChangelogScan(newOps, newTable, newSchema,
newContext);
+ }
+
+ @Override
+ protected CloseableIterable<ChangelogScanTask> doPlanFiles(
+ Long fromSnapshotIdExclusive, long toSnapshotIdInclusive) {
+
+ Deque<Snapshot> changelogSnapshots =
+ orderedChangelogSnapshots(fromSnapshotIdExclusive,
toSnapshotIdInclusive);
+
+ if (changelogSnapshots.isEmpty()) {
+ return CloseableIterable.empty();
+ }
+
+ Set<Long> changelogSnapshotIds = toSnapshotIds(changelogSnapshots);
+
+ Set<ManifestFile> newDataManifests =
+ FluentIterable.from(changelogSnapshots)
+ .transformAndConcat(snapshot ->
snapshot.dataManifests(table().io()))
+ .filter(manifest ->
changelogSnapshotIds.contains(manifest.snapshotId()))
+ .toSet();
+
+ ManifestGroup manifestGroup =
+ new ManifestGroup(table().io(), newDataManifests, ImmutableList.of())
+ .specsById(table().specs())
+ .caseSensitive(isCaseSensitive())
+ .select(scanColumns())
+ .filterData(filter())
+ .filterManifestEntries(entry ->
changelogSnapshotIds.contains(entry.snapshotId()))
+ .ignoreExisting();
+
+ if (shouldIgnoreResiduals()) {
+ manifestGroup = manifestGroup.ignoreResiduals();
+ }
+
+ if (newDataManifests.size() > 1 && shouldPlanWithExecutor()) {
+ manifestGroup = manifestGroup.planWith(planExecutor());
+ }
+
+ return manifestGroup.plan(new
CreateDataFileChangeTasks(changelogSnapshots));
+ }
+
+ @Override
+ public CloseableIterable<ScanTaskGroup<ChangelogScanTask>> planTasks() {
+ return TableScanUtil.planTaskGroups(
+ planFiles(), targetSplitSize(), splitLookback(), splitOpenFileCost());
+ }
+
+ // builds a collection of changelog snapshots (oldest to newest)
+ // the order of the snapshots is important as it is used to determine change
ordinals
+ private Deque<Snapshot> orderedChangelogSnapshots(Long fromIdExcl, long
toIdIncl) {
+ Deque<Snapshot> changelogSnapshots = new ArrayDeque<>();
+
+ for (Snapshot snapshot : SnapshotUtil.ancestorsBetween(table(), toIdIncl,
fromIdExcl)) {
+ if (!snapshot.operation().equals(DataOperations.REPLACE)) {
+ if (snapshot.deleteManifests(table().io()).size() > 0) {
+ throw new UnsupportedOperationException(
+ "Delete files are currently not supported in changelog scans");
+ }
+
+ changelogSnapshots.addFirst(snapshot);
+ }
+ }
+
+ return changelogSnapshots;
+ }
+
+ private Set<Long> toSnapshotIds(Collection<Snapshot> snapshots) {
+ return
snapshots.stream().map(Snapshot::snapshotId).collect(Collectors.toSet());
+ }
+
+ private static Map<Long, Integer> computeSnapshotOrdinals(Deque<Snapshot>
snapshots) {
Review Comment:
I guess we don't want to reuse seq# as ordinals as we discussed before. The
zero based ordinals should be more consistent and less confusing.
##########
core/src/main/java/org/apache/iceberg/BaseIncrementalScan.java:
##########
@@ -0,0 +1,142 @@
+/*
+ * 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 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.util.SnapshotUtil;
+
+abstract class BaseIncrementalScan<ThisT, T extends ScanTask, G extends
ScanTaskGroup<T>>
+ extends BaseScan<ThisT, T, G> implements IncrementalScan<ThisT, T, G> {
+
+ protected BaseIncrementalScan(
+ TableOperations ops, Table table, Schema schema, TableScanContext
context) {
+ super(ops, table, schema, context);
+ }
+
+ protected abstract CloseableIterable<T> doPlanFiles(
+ Long fromSnapshotIdExclusive, long toSnapshotIdInclusive);
+
+ @Override
+ public ThisT fromSnapshotInclusive(long fromSnapshotId) {
+ Preconditions.checkArgument(
+ table().snapshot(fromSnapshotId) != null,
+ "Cannot find the starting snapshot: %s",
+ fromSnapshotId);
+ TableScanContext newContext =
context().fromSnapshotIdInclusive(fromSnapshotId);
+ return newRefinedScan(tableOps(), table(), schema(), newContext);
+ }
+
+ @Override
+ public ThisT fromSnapshotExclusive(long fromSnapshotId) {
+ // for exclusive behavior, table().snapshot(fromSnapshotId) check can't be
applied
+ // as fromSnapshotId could be matched to a parent snapshot that is already
expired
+ TableScanContext newContext =
context().fromSnapshotIdExclusive(fromSnapshotId);
+ return newRefinedScan(tableOps(), table(), schema(), newContext);
+ }
+
+ @Override
+ public ThisT toSnapshot(long toSnapshotId) {
+ Preconditions.checkArgument(
+ table().snapshot(toSnapshotId) != null, "Cannot find the end snapshot:
%s", toSnapshotId);
+ TableScanContext newContext = context().toSnapshotId(toSnapshotId);
+ return newRefinedScan(tableOps(), table(), schema(), newContext);
+ }
+
+ @Override
+ public CloseableIterable<T> planFiles() {
+ if (scanCurrentLineage() && table().currentSnapshot() == null) {
+ // If the table is empty (no current snapshot) and both from and to
snapshots aren't set,
+ // simply return an empty iterable. In this case, the listener
notification is also skipped.
+ return CloseableIterable.empty();
+ }
+
+ long toSnapshotIdInclusive = toSnapshotIdInclusive();
+ Long fromSnapshotIdExclusive =
fromSnapshotIdExclusive(toSnapshotIdInclusive);
+
+ if (fromSnapshotIdExclusive != null) {
+ Listeners.notifyAll(
+ new IncrementalScanEvent(
+ table().name(),
+ fromSnapshotIdExclusive,
+ toSnapshotIdInclusive,
+ filter(),
+ schema(),
+ false));
+ } else {
+ Listeners.notifyAll(
+ new IncrementalScanEvent(
+ table().name(),
+ SnapshotUtil.oldestAncestorOf(table(),
toSnapshotIdInclusive).snapshotId(),
+ toSnapshotIdInclusive,
+ filter(),
+ schema(),
+ true));
+ }
Review Comment:
Minor suggestion: How about this?
```
Listeners.notifyAll(
new IncrementalScanEvent(
table().name(),
fromSnapshotIdExclusive != null
? fromSnapshotIdExclusive
: SnapshotUtil.oldestAncestorOf(table(),
toSnapshotIdInclusive).snapshotId(),
toSnapshotIdInclusive,
filter(),
schema(),
fromSnapshotIdExclusive != null));
```
##########
core/src/main/java/org/apache/iceberg/BaseScan.java:
##########
@@ -59,6 +90,22 @@ protected TableScanContext context() {
return context;
}
+ protected List<String> scanColumns() {
+ return context.returnColumnStats() ? SCAN_WITH_STATS_COLUMNS :
SCAN_COLUMNS;
+ }
Review Comment:
+1 for the refactor.
--
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: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]