Skip to content

Commit

Permalink
Core: Follow-up to IncrementalAppendScan implementation (apache#4886)
Browse files Browse the repository at this point in the history
  • Loading branch information
stevenzwu authored Jun 1, 2022
1 parent 01e640c commit 9ab94f8
Show file tree
Hide file tree
Showing 8 changed files with 50 additions and 52 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -59,7 +59,7 @@ public IncrementalAppendScan fromSnapshotInclusive(long fromSnapshotId) {
public IncrementalAppendScan 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
return newRefinedScan(tableOps(), table(), schema(), context().fromSnapshotId(fromSnapshotId));
return newRefinedScan(tableOps(), table(), schema(), context().fromSnapshotIdExclusive(fromSnapshotId));
}

@Override
Expand All @@ -71,46 +71,23 @@ public IncrementalAppendScan toSnapshot(long toSnapshotId) {

@Override
public CloseableIterable<FileScanTask> planFiles() {
long toSnapshotIdInclusive;
if (context().toSnapshotId() != null) {
toSnapshotIdInclusive = context().toSnapshotId();
} else if (table().currentSnapshot() != null) {
toSnapshotIdInclusive = table().currentSnapshot().snapshotId();
} else if (context().fromSnapshotId() != null) {
throw new IllegalArgumentException(
"Invalid config: end snapshot is not set, start snapshot is set, and table has no current snapshot");
} else {
// It is an empty table (no current snapshot). Both from and to snapshots aren't set either.
// In this case, listener notification is also skipped
Long fromSnapshotId = context().fromSnapshotId();
Long toSnapshotId = context().toSnapshotId();
if (fromSnapshotId == null && toSnapshotId == null && table().currentSnapshot() == null) {
// If it is an empty table (no current snapshot) and both from and to snapshots aren't set either,
// simply return an empty iterable. In this case, listener notification is also skipped.
return CloseableIterable.empty();
}

long toSnapshotIdInclusive = toSnapshotIdInclusive();
// fromSnapshotIdExclusive can be null. appendsBetween handles null fromSnapshotIdExclusive properly
// by finding the oldest ancestor of end snapshot.
Long fromSnapshotIdExclusive = context().fromSnapshotId();
if (context().fromSnapshotId() != null) {
if (context().fromSnapshotInclusive()) {
// validate the fromSnapshotId is an ancestor of toSnapshotId
Preconditions.checkArgument(
SnapshotUtil.isAncestorOf(table(), toSnapshotIdInclusive, context().fromSnapshotId()),
"Starting snapshot (inclusive) %s is not an ancestor of end snapshot %s",
context().fromSnapshotId(), toSnapshotIdInclusive);
// for inclusive behavior fromSnapshotIdExclusive is set to the parent snapshot id, which can be null.
fromSnapshotIdExclusive = table().snapshot(context().fromSnapshotId()).parentId();
} else {
// validate the parent snapshot id an ancestor of toSnapshotId
Preconditions.checkArgument(
SnapshotUtil.isParentAncestorOf(table(), toSnapshotIdInclusive, context().fromSnapshotId()),
"Starting snapshot (exclusive) %s is not a parent ancestor of end snapshot %s",
context().fromSnapshotId(), toSnapshotIdInclusive);
}
}

Long fromSnapshotIdExclusive = fromSnapshotIdExclusive(fromSnapshotId, toSnapshotIdInclusive);
if (fromSnapshotIdExclusive != null) {
Listeners.notifyAll(new IncrementalScanEvent(table().name(), fromSnapshotIdExclusive,
toSnapshotIdInclusive, context().rowFilter(), table().schema(), false));
} else {
Snapshot oldestAncestorSnapshot = SnapshotUtil.oldestAncestor(toSnapshotIdInclusive, table()::snapshot);
Snapshot oldestAncestorSnapshot = SnapshotUtil.oldestAncestorOf(toSnapshotIdInclusive, table()::snapshot);
Listeners.notifyAll(new IncrementalScanEvent(table().name(), oldestAncestorSnapshot.snapshotId(),
toSnapshotIdInclusive, context().rowFilter(), table().schema(), true));
}
Expand All @@ -131,6 +108,40 @@ public CloseableIterable<CombinedScanTask> planTasks() {
return TableScanUtil.planTasks(splitFiles, targetSplitSize(), splitLookback(), splitOpenFileCost());
}

private Long fromSnapshotIdExclusive(Long fromSnapshotId, long toSnapshotIdInclusive) {
if (fromSnapshotId != null) {
if (context().fromSnapshotInclusive()) {
// validate the fromSnapshotId is an ancestor of toSnapshotId
Preconditions.checkArgument(
SnapshotUtil.isAncestorOf(table(), toSnapshotIdInclusive, fromSnapshotId),
"Starting snapshot (inclusive) %s is not an ancestor of end snapshot %s",
fromSnapshotId, toSnapshotIdInclusive);
// for inclusive behavior fromSnapshotIdExclusive is set to the parent snapshot id, which can be null.
return table().snapshot(fromSnapshotId).parentId();
} else {
// validate the parent snapshot id an ancestor of toSnapshotId
Preconditions.checkArgument(
SnapshotUtil.isParentAncestorOf(table(), toSnapshotIdInclusive, fromSnapshotId),
"Starting snapshot (exclusive) %s is not a parent ancestor of end snapshot %s",
fromSnapshotId, toSnapshotIdInclusive);
return fromSnapshotId;
}
} else {
return null;
}
}

private long toSnapshotIdInclusive() {
if (context().toSnapshotId() != null) {
return context().toSnapshotId();
} else {
Snapshot currentSnapshot = table().currentSnapshot();
Preconditions.checkArgument(currentSnapshot != null,
"Invalid config: end snapshot is not set and table has no current snapshot");
return currentSnapshot.snapshotId();
}
}

private CloseableIterable<FileScanTask> appendFilesFromSnapshots(List<Snapshot> snapshots) {
Set<Long> snapshotIds = Sets.newHashSet(Iterables.transform(snapshots, Snapshot::snapshotId));
Set<ManifestFile> manifests = FluentIterable
Expand Down
2 changes: 1 addition & 1 deletion core/src/main/java/org/apache/iceberg/BaseScan.java
Original file line number Diff line number Diff line change
Expand Up @@ -134,7 +134,7 @@ public long splitOpenFileCost() {
/**
* Resolve the schema to be projected lazily.
*
* if there are selected columns from scan context, selected columns are projected to the table schema.
* If there are selected columns from scan context, selected columns are projected to the table schema.
* Otherwise, projected schema from scan context shall be returned.
*
* @param context scan context
Expand Down
5 changes: 0 additions & 5 deletions core/src/main/java/org/apache/iceberg/BaseTableScan.java
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,6 @@

package org.apache.iceberg;

import java.util.Collection;
import java.util.Map;
import java.util.concurrent.ExecutorService;
import org.apache.iceberg.events.Listeners;
Expand Down Expand Up @@ -61,10 +60,6 @@ protected boolean shouldIgnoreResiduals() {
return context().ignoreResiduals();
}

protected Collection<String> selectedColumns() {
return context().selectedColumns();
}

protected ExecutorService planExecutor() {
return context().planExecutor();
}
Expand Down
11 changes: 1 addition & 10 deletions core/src/main/java/org/apache/iceberg/DataTableScan.java
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,6 @@
import org.apache.iceberg.io.FileIO;
import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
import org.apache.iceberg.util.PropertyUtil;
import org.apache.iceberg.util.SnapshotUtil;

public class DataTableScan extends BaseTableScan {
Expand Down Expand Up @@ -51,7 +50,7 @@ public TableScan appendsBetween(long fromSnapshotId, long toSnapshotId) {
Preconditions.checkState(snapshotId() == null,
"Cannot enable incremental scan, scan-snapshot set to id=%s", snapshotId());
return new IncrementalDataTableScan(tableOps(), table(), schema(),
context().fromSnapshotId(fromSnapshotId).toSnapshotId(toSnapshotId));
context().fromSnapshotIdExclusive(fromSnapshotId).toSnapshotId(toSnapshotId));
}

@Override
Expand Down Expand Up @@ -99,12 +98,4 @@ public CloseableIterable<FileScanTask> doPlanFiles() {

return manifestGroup.planFiles();
}

@Override
public long targetSplitSize() {
long tableValue = tableOps().current().propertyAsLong(
TableProperties.SPLIT_SIZE,
TableProperties.SPLIT_SIZE_DEFAULT);
return PropertyUtil.propertyAsLong(options(), TableProperties.SPLIT_SIZE, tableValue);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -56,7 +56,7 @@ public TableScan useSnapshot(long scanSnapshotId) {
public TableScan appendsBetween(long fromSnapshotId, long toSnapshotId) {
validateSnapshotIdsRefinement(fromSnapshotId, toSnapshotId);
return new IncrementalDataTableScan(tableOps(), table(), schema(),
context().fromSnapshotId(fromSnapshotId).toSnapshotId(toSnapshotId));
context().fromSnapshotIdExclusive(fromSnapshotId).toSnapshotId(toSnapshotId));
}

@Override
Expand Down
4 changes: 2 additions & 2 deletions core/src/main/java/org/apache/iceberg/TableScanContext.java
Original file line number Diff line number Diff line change
Expand Up @@ -169,10 +169,10 @@ Long fromSnapshotId() {
return fromSnapshotId;
}

TableScanContext fromSnapshotId(long id) {
TableScanContext fromSnapshotIdExclusive(long id) {
return new TableScanContext(snapshotId, rowFilter, ignoreResiduals,
caseSensitive, colStats, projectedSchema, selectedColumns, options, id, toSnapshotId,
planExecutor, fromSnapshotInclusive);
planExecutor, false);
}

TableScanContext fromSnapshotIdInclusive(long id) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -132,7 +132,7 @@ public static Snapshot oldestAncestor(Table table) {
* @param lookup lookup function from snapshot ID to snapshot
* @return null if there is no current snapshot in the table, else the oldest Snapshot.
*/
public static Snapshot oldestAncestor(long snapshotId, Function<Long, Snapshot> lookup) {
public static Snapshot oldestAncestorOf(long snapshotId, Function<Long, Snapshot> lookup) {
Snapshot lastSnapshot = null;
for (Snapshot snapshot : ancestorsOf(snapshotId, lookup)) {
lastSnapshot = snapshot;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -109,6 +109,7 @@ public IncrementalScanEvent event() {
Assert.assertEquals(table.schema(), listener1.event().projection());
Assert.assertEquals(Expressions.alwaysTrue(), listener1.event().filter());
Assert.assertEquals("test", listener1.event().tableName());
Assert.assertEquals(false, listener1.event().isFromSnapshotInclusive());
}

@Test
Expand Down

0 comments on commit 9ab94f8

Please sign in to comment.