Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
77 changes: 65 additions & 12 deletions core/src/main/java/org/apache/iceberg/PositionDeletesTable.java
Original file line number Diff line number Diff line change
Expand Up @@ -24,8 +24,10 @@
import java.util.List;
import java.util.Map;
import java.util.function.Function;
import org.apache.iceberg.expressions.Expression;
import org.apache.iceberg.expressions.Expressions;
import org.apache.iceberg.expressions.ManifestEvaluator;
import org.apache.iceberg.expressions.Projections;
import org.apache.iceberg.expressions.ResidualEvaluator;
import org.apache.iceberg.io.CloseableIterable;
import org.apache.iceberg.io.CloseableIterator;
Expand Down Expand Up @@ -130,18 +132,28 @@ private Schema calculateSchema() {
public static class PositionDeletesBatchScan
extends SnapshotScan<BatchScan, ScanTask, ScanTaskGroup<ScanTask>> implements BatchScan {

private Expression baseTableFilter = Expressions.alwaysTrue();

protected PositionDeletesBatchScan(Table table, Schema schema) {
super(table, schema, TableScanContext.empty());
}

/** @deprecated the API will be removed in v1.5.0 */
@Deprecated
protected PositionDeletesBatchScan(Table table, Schema schema, TableScanContext context) {
super(table, schema, context);
}

protected PositionDeletesBatchScan(
Table table, Schema schema, TableScanContext context, Expression baseTableFilter) {
super(table, schema, context);
this.baseTableFilter = baseTableFilter;
}

@Override
protected PositionDeletesBatchScan newRefinedScan(
Table newTable, Schema newSchema, TableScanContext newContext) {
return new PositionDeletesBatchScan(newTable, newSchema, newContext);
return new PositionDeletesBatchScan(newTable, newSchema, newContext, baseTableFilter);
}

@Override
Expand All @@ -155,39 +167,70 @@ protected List<String> scanColumns() {
return context().returnColumnStats() ? DELETE_SCAN_WITH_STATS_COLUMNS : DELETE_SCAN_COLUMNS;
}

/**
* Sets a filter that applies on base table of this position deletes table, to use for this
* scan.
*
* <p>Only the partition expressions part of the filter will be applied to the position deletes
* table, as the schema of the base table does not otherwise match the schema of position
* deletes table.
*
* <ul>
* <li>Only the partition expressions of the filter that can be projected on the base table
* partition specs, via {@link
* org.apache.iceberg.expressions.Projections.ProjectionEvaluator#project(Expression)}
* will be evaluated. Note, not all partition expressions can be projected.
* <li>Because it cannot apply beyond the partition expression, this filter will not
* contribute to the residuals of tasks returned by this scan. (See {@link
* PositionDeletesScanTask#residual()})
* </ul>
*
* @param expr expression filter that applies on the base table of this posiiton deletes table
* @return this for method chaining
*/
public BatchScan baseTableFilter(Expression expr) {
return new PositionDeletesBatchScan(
table(), schema(), context(), Expressions.and(baseTableFilter, expr));
}

@Override
protected CloseableIterable<ScanTask> doPlanFiles() {
String schemaString = SchemaParser.toJson(tableSchema());

// prepare transformed partition specs and caches
Map<Integer, PartitionSpec> transformedSpecs = transformSpecs(tableSchema(), table().specs());

LoadingCache<Integer, String> specStringCache =
partitionCacheOf(transformedSpecs, PartitionSpecParser::toJson);
LoadingCache<Integer, ManifestEvaluator> deletesTableEvalCache =
partitionCacheOf(
transformedSpecs,
spec -> ManifestEvaluator.forRowFilter(filter(), spec, isCaseSensitive()));
LoadingCache<Integer, ManifestEvaluator> baseTableEvalCache =
partitionCacheOf(
table().specs(), // evaluate base table filters on base table specs
spec -> ManifestEvaluator.forRowFilter(baseTableFilter, spec, isCaseSensitive()));
LoadingCache<Integer, ResidualEvaluator> residualCache =
partitionCacheOf(
transformedSpecs,
spec ->
ResidualEvaluator.of(
spec,
// there are no applicable filters in the base table's filter
// that we can use to evaluate on the position deletes table
shouldIgnoreResiduals() ? Expressions.alwaysTrue() : filter(),
isCaseSensitive()));

LoadingCache<Integer, String> specStringCache =
partitionCacheOf(transformedSpecs, PartitionSpecParser::toJson);

LoadingCache<Integer, ManifestEvaluator> evalCache =
partitionCacheOf(
transformedSpecs,
spec -> ManifestEvaluator.forRowFilter(filter(), spec, isCaseSensitive()));

// iterate through delete manifests
List<ManifestFile> manifests = snapshot().deleteManifests(table().io());

CloseableIterable<ManifestFile> matchingManifests =
Copy link
Contributor

@aokolnychyi aokolnychyi Jul 27, 2023

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Shall we do this filter only if either of the filter expression is non-trivial? Otherwise, what's the point of doing this work?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Well, this is for manifests, so shouldn't matter that much. Never mind.

CloseableIterable.filter(
scanMetrics().skippedDeleteManifests(),
CloseableIterable.withNoopClose(manifests),
manifest -> evalCache.get(manifest.partitionSpecId()).eval(manifest));

manifest ->
baseTableEvalCache.get(manifest.partitionSpecId()).eval(manifest)
&& deletesTableEvalCache.get(manifest.partitionSpecId()).eval(manifest));
matchingManifests =
CloseableIterable.count(scanMetrics().scannedDeleteManifests(), matchingManifests);

Expand All @@ -196,7 +239,12 @@ protected CloseableIterable<ScanTask> doPlanFiles() {
matchingManifests,
manifest ->
posDeletesScanTasks(
manifest, schemaString, transformedSpecs, residualCache, specStringCache));
manifest,
table().specs().get(manifest.partitionSpecId()),
schemaString,
transformedSpecs,
residualCache,
specStringCache));

if (planExecutor() != null) {
return new ParallelIterable<>(tasks, planExecutor());
Expand All @@ -207,6 +255,7 @@ protected CloseableIterable<ScanTask> doPlanFiles() {

private CloseableIterable<ScanTask> posDeletesScanTasks(
ManifestFile manifest,
PartitionSpec spec,
String schemaString,
Map<Integer, PartitionSpec> transformedSpecs,
LoadingCache<Integer, ResidualEvaluator> residualCache,
Expand All @@ -223,12 +272,16 @@ public void close() throws IOException {

@Override
public CloseableIterator<ScanTask> iterator() {
Expression partitionFilter =
Projections.inclusive(spec, isCaseSensitive()).project(baseTableFilter);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

You could have cached this and used ManifestEvaluator.forPartitionFilter but it is probably not worth it.


// Filter partitions
CloseableIterable<ManifestEntry<DeleteFile>> deleteFileEntries =
ManifestFiles.readDeleteManifest(manifest, table().io(), transformedSpecs)
.caseSensitive(isCaseSensitive())
.select(scanColumns())
.filterRows(filter())
.filterPartitions(partitionFilter)
.scanMetrics(scanMetrics())
.liveEntries();

Expand Down
205 changes: 195 additions & 10 deletions core/src/test/java/org/apache/iceberg/TestMetadataTableScans.java
Original file line number Diff line number Diff line change
Expand Up @@ -53,22 +53,50 @@ public TestMetadataTableScans(int formatVersion) {
super(formatVersion);
}

private void preparePartitionedTable() {
preparePartitionedTableData();
private void preparePartitionedTable(boolean transactional) {
preparePartitionedTableData(transactional);

if (formatVersion == 2) {
table.newRowDelta().addDeletes(FILE_A_DELETES).commit();
table.newRowDelta().addDeletes(FILE_B_DELETES).commit();
table.newRowDelta().addDeletes(FILE_C2_DELETES).commit();
table.newRowDelta().addDeletes(FILE_D2_DELETES).commit();
if (transactional) {
table
.newRowDelta()
.addDeletes(FILE_A_DELETES)
.addDeletes(FILE_B_DELETES)
.addDeletes(FILE_C2_DELETES)
.addDeletes(FILE_D2_DELETES)
.commit();
} else {
table.newRowDelta().addDeletes(FILE_A_DELETES).commit();
table.newRowDelta().addDeletes(FILE_B_DELETES).commit();
table.newRowDelta().addDeletes(FILE_C2_DELETES).commit();
table.newRowDelta().addDeletes(FILE_D2_DELETES).commit();
}
}
}

private void preparePartitionedTable() {
preparePartitionedTable(false);
}

private void preparePartitionedTableData(boolean transactional) {
if (transactional) {
table
.newFastAppend()
.appendFile(FILE_A)
.appendFile(FILE_B)
.appendFile(FILE_C)
.appendFile(FILE_D)
.commit();
} else {
table.newFastAppend().appendFile(FILE_A).commit();
table.newFastAppend().appendFile(FILE_C).commit();
table.newFastAppend().appendFile(FILE_D).commit();
table.newFastAppend().appendFile(FILE_B).commit();
}
}

private void preparePartitionedTableData() {
table.newFastAppend().appendFile(FILE_A).commit();
table.newFastAppend().appendFile(FILE_C).commit();
table.newFastAppend().appendFile(FILE_D).commit();
table.newFastAppend().appendFile(FILE_B).commit();
preparePartitionedTableData(false);
}

@Test
Expand Down Expand Up @@ -1261,6 +1289,163 @@ public void testPositionDeletesWithFilter() {
constantsMap(posDeleteTask, partitionType).get(MetadataColumns.FILE_PATH.fieldId()));
}

@Test
public void testPositionDeletesBaseTableFilterManifestLevel() {
testPositionDeletesBaseTableFilter(false);
}

@Test
public void testPositionDeletesBaseTableFilterEntriesLevel() {
testPositionDeletesBaseTableFilter(true);
}

private void testPositionDeletesBaseTableFilter(boolean transactional) {
Assume.assumeTrue("Position deletes supported only for v2 tables", formatVersion == 2);
preparePartitionedTable(transactional);

PositionDeletesTable positionDeletesTable = new PositionDeletesTable(table);

Expression expression =
Expressions.and(
Expressions.equal("data", "u"), // hashes to bucket 0
Expressions.greaterThan("id", 0));
BatchScan scan =
((PositionDeletesTable.PositionDeletesBatchScan) positionDeletesTable.newBatchScan())
.baseTableFilter(expression);
assertThat(scan).isExactlyInstanceOf(PositionDeletesTable.PositionDeletesBatchScan.class);
PositionDeletesTable.PositionDeletesBatchScan deleteScan =
(PositionDeletesTable.PositionDeletesBatchScan) scan;

List<ScanTask> tasks = Lists.newArrayList(scan.planFiles());

Assert.assertEquals(
"Expected to scan one delete manifest",
1,
deleteScan.scanMetrics().scannedDeleteManifests().value());
int expectedSkippedManifests = transactional ? 0 : 3;
Assert.assertEquals(
"Wrong number of manifests skipped",
expectedSkippedManifests,
deleteScan.scanMetrics().skippedDeleteManifests().value());

assertThat(tasks).hasSize(1);

ScanTask task = tasks.get(0);
assertThat(task).isInstanceOf(PositionDeletesScanTask.class);

Types.StructType partitionType = Partitioning.partitionType(table);
PositionDeletesScanTask posDeleteTask = (PositionDeletesScanTask) task;

// base table filter should only be used to evaluate partitions
posDeleteTask.residual().isEquivalentTo(Expressions.alwaysTrue());

int filePartition = posDeleteTask.file().partition().get(0, Integer.class);
StructLike taskPartitionStruct =
(StructLike)
constantsMap(posDeleteTask, partitionType).get(MetadataColumns.PARTITION_COLUMN_ID);
int taskPartition = taskPartitionStruct.get(0, Integer.class);
Assert.assertEquals("Expected correct partition on task's file", 0, filePartition);
Assert.assertEquals("Expected correct partition on task's column", 0, taskPartition);

Assert.assertEquals(
"Expected correct partition spec id on task", 0, posDeleteTask.file().specId());
Assert.assertEquals(
"Expected correct partition spec id on constant column",
0,
constantsMap(posDeleteTask, partitionType).get(MetadataColumns.SPEC_ID.fieldId()));

Assert.assertEquals(
"Expected correct delete file on task", FILE_A_DELETES.path(), posDeleteTask.file().path());
Assert.assertEquals(
"Expected correct delete file on constant column",
FILE_A_DELETES.path(),
constantsMap(posDeleteTask, partitionType).get(MetadataColumns.FILE_PATH.fieldId()));
}

@Test
public void testPositionDeletesWithBaseTableFilterNot() {
Assume.assumeTrue("Position deletes supported only for v2 tables", formatVersion == 2);

// use identity rather than bucket partition spec,
// as bucket.project does not support projecting notEq
table.updateSpec().removeField("data_bucket").addField("id").commit();
PartitionSpec spec = table.spec();

String path0 = "/path/to/data-0-deletes.parquet";
PartitionData partitionData0 = new PartitionData(spec.partitionType());
partitionData0.set(0, 0);
DeleteFile deleteFileA =
FileMetadata.deleteFileBuilder(spec)
.ofPositionDeletes()
.withPath(path0)
.withFileSizeInBytes(10)
.withPartition(partitionData0)
.withRecordCount(1)
.build();

String path1 = "/path/to/data-1-deletes.parquet";
PartitionData partitionData1 = new PartitionData(spec.partitionType());
partitionData1.set(0, 1);
DeleteFile deleteFileB =
FileMetadata.deleteFileBuilder(spec)
.ofPositionDeletes()
.withPath(path1)
.withFileSizeInBytes(10)
.withPartition(partitionData1)
.withRecordCount(1)
.build();
table.newRowDelta().addDeletes(deleteFileA).addDeletes(deleteFileB).commit();

PositionDeletesTable positionDeletesTable = new PositionDeletesTable(table);

Expression expression = Expressions.not(Expressions.equal("id", 0));
BatchScan scan =
((PositionDeletesTable.PositionDeletesBatchScan) positionDeletesTable.newBatchScan())
.baseTableFilter(expression);
assertThat(scan).isExactlyInstanceOf(PositionDeletesTable.PositionDeletesBatchScan.class);
PositionDeletesTable.PositionDeletesBatchScan deleteScan =
(PositionDeletesTable.PositionDeletesBatchScan) scan;

List<ScanTask> tasks = Lists.newArrayList(scan.planFiles());

Assert.assertEquals(
"Expected to scan one delete manifest",
1,
deleteScan.scanMetrics().scannedDeleteManifests().value());
assertThat(tasks).hasSize(1);

ScanTask task = tasks.get(0);
assertThat(task).isInstanceOf(PositionDeletesScanTask.class);

Types.StructType partitionType = Partitioning.partitionType(table);
PositionDeletesScanTask posDeleteTask = (PositionDeletesScanTask) task;

// base table filter should only be used to evaluate partitions
posDeleteTask.residual().isEquivalentTo(Expressions.alwaysTrue());

int filePartition = posDeleteTask.file().partition().get(0, Integer.class);
StructLike taskPartitionStruct =
(StructLike)
constantsMap(posDeleteTask, partitionType).get(MetadataColumns.PARTITION_COLUMN_ID);
int taskPartition =
taskPartitionStruct.get(1, Integer.class); // new partition field in position 1
Assert.assertEquals("Expected correct partition on task's file", 1, filePartition);
Assert.assertEquals("Expected correct partition on task's column", 1, taskPartition);

Assert.assertEquals(
"Expected correct partition spec id on task", 1, posDeleteTask.file().specId());
Assert.assertEquals(
"Expected correct partition spec id on constant column",
1,
constantsMap(posDeleteTask, partitionType).get(MetadataColumns.SPEC_ID.fieldId()));

Assert.assertEquals("Expected correct delete file on task", path1, posDeleteTask.file().path());
Assert.assertEquals(
"Expected correct delete file on constant column",
path1,
constantsMap(posDeleteTask, partitionType).get(MetadataColumns.FILE_PATH.fieldId()));
}

@Test
public void testPositionDeletesResiduals() {
Assume.assumeTrue("Position deletes supported only for v2 tables", formatVersion == 2);
Expand Down
Loading