diff --git a/api/src/main/java/org/apache/iceberg/Snapshot.java b/api/src/main/java/org/apache/iceberg/Snapshot.java index d10a6c3a49d1..0d1b81166414 100644 --- a/api/src/main/java/org/apache/iceberg/Snapshot.java +++ b/api/src/main/java/org/apache/iceberg/Snapshot.java @@ -120,6 +120,26 @@ public interface Snapshot extends Serializable { */ Iterable deletedFiles(); + /** + * Return all delete files added to the table in this snapshot. + *

+ * The files returned include the following columns: file_path, file_format, partition, + * record_count, and file_size_in_bytes. Other columns will be null. + * + * @return all delete files added to the table in this snapshot. + */ + Iterable addedDeleteFiles(); + + /** + * Return all delete files deleted from the table in this snapshot. + *

+ * The files returned include the following columns: file_path, file_format, partition, + * record_count, and file_size_in_bytes. Other columns will be null. + * + * @return all delete files deleted from the table in this snapshot. + */ + Iterable deletedDeleteFiles(); + /** * Return the location of this snapshot's manifest list, or null if it is not separate. * diff --git a/api/src/main/java/org/apache/iceberg/TableScan.java b/api/src/main/java/org/apache/iceberg/TableScan.java index fe040c02d229..b384a5d8a7f0 100644 --- a/api/src/main/java/org/apache/iceberg/TableScan.java +++ b/api/src/main/java/org/apache/iceberg/TableScan.java @@ -48,6 +48,17 @@ public interface TableScan { */ TableScan useSnapshot(long snapshotId); + + /** + * Create a new {@link TableScan} from this scan's configuration that will scan the manifests + * specified by the parameters. + * + * @param manifests specified manifest group which will be scanned + * @return a new scan based on this with the given manifest group + * @throws IllegalArgumentException if the snapshot cannot be found + */ + TableScan useManifests(Iterable manifests); + /** * Create a new {@link TableScan} from this scan's configuration that will use the most recent * snapshot as of the given time in milliseconds. diff --git a/core/src/main/java/org/apache/iceberg/BaseSnapshot.java b/core/src/main/java/org/apache/iceberg/BaseSnapshot.java index 7626d4d2b6c7..d2829c0b7f95 100644 --- a/core/src/main/java/org/apache/iceberg/BaseSnapshot.java +++ b/core/src/main/java/org/apache/iceberg/BaseSnapshot.java @@ -23,11 +23,13 @@ import java.util.Arrays; import java.util.List; import java.util.Map; +import java.util.function.Function; import org.apache.iceberg.exceptions.RuntimeIOException; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; import org.apache.iceberg.relocated.com.google.common.base.Objects; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableCollection; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Lists; @@ -49,8 +51,10 @@ class BaseSnapshot implements Snapshot { private transient List allManifests = null; private transient List dataManifests = null; private transient List deleteManifests = null; - private transient List cachedAdds = null; - private transient List cachedDeletes = null; + private transient List cachedAddedDataFiles = null; + private transient List cachedDeletedDataFiles = null; + private transient List cachedAddedDeleteFiles = null; + private transient List cachedDeletedDeleteFiles = null; /** * For testing only. @@ -175,40 +179,73 @@ public List deleteManifests() { @Override public List addedFiles() { - if (cachedAdds == null) { - cacheChanges(); + if (cachedAddedDataFiles == null) { + cacheDataFileChanges(); } - return cachedAdds; + return cachedAddedDataFiles; } @Override public List deletedFiles() { - if (cachedDeletes == null) { - cacheChanges(); + if (cachedDeletedDataFiles == null) { + cacheDataFileChanges(); } - return cachedDeletes; + return cachedDeletedDataFiles; } @Override - public String manifestListLocation() { - return manifestListLocation; + public Iterable addedDeleteFiles() { + if (cachedAddedDeleteFiles == null) { + cacheDeleteFileChanges(); + } + return cachedAddedDeleteFiles; } - private void cacheChanges() { - if (io == null) { - throw new IllegalStateException("Cannot cache changes: FileIO is null"); + @Override + public Iterable deletedDeleteFiles() { + if (cachedDeletedDeleteFiles == null) { + cacheDeleteFileChanges(); } + return cachedDeletedDeleteFiles; + } + @Override + public String manifestListLocation() { + return manifestListLocation; + } + + private void cacheDataFileChanges() { ImmutableList.Builder adds = ImmutableList.builder(); ImmutableList.Builder deletes = ImmutableList.builder(); + collectFileChanges(adds, deletes, dataManifests(), + manifest -> ManifestFiles.read(manifest, io, null).entries() + ); + this.cachedAddedDataFiles = adds.build(); + this.cachedDeletedDataFiles = deletes.build(); + } + private void cacheDeleteFileChanges() { + ImmutableList.Builder adds = ImmutableList.builder(); + ImmutableList.Builder deletes = ImmutableList.builder(); + collectFileChanges(adds, deletes, deleteManifests(), + manifest -> ManifestFiles.readDeleteManifest(manifest, io, null).entries() + ); + this.cachedAddedDeleteFiles = adds.build(); + this.cachedDeletedDeleteFiles = deletes.build(); + } + + private > void collectFileChanges( + ImmutableCollection.Builder adds, ImmutableCollection.Builder deletes, Iterable manifests, + Function>> manifestReader) { // read only manifests that were created by this snapshot - Iterable changedManifests = Iterables.filter(dataManifests(), + Iterable changedManifests = Iterables.filter(manifests, manifest -> Objects.equal(manifest.snapshotId(), snapshotId)); - try (CloseableIterable> entries = new ManifestGroup(io, changedManifests) - .ignoreExisting() - .entries()) { - for (ManifestEntry entry : entries) { + + try (CloseableIterable> entries = CloseableIterable.filter( + CloseableIterable.concat(Iterables.transform(changedManifests, manifestReader::apply)), + entry -> entry.status() != ManifestEntry.Status.EXISTING + )) { + for (ManifestEntry entry : entries) { switch (entry.status()) { case ADDED: adds.add(entry.file().copy()); @@ -224,9 +261,6 @@ private void cacheChanges() { } catch (IOException e) { throw new RuntimeIOException(e, "Failed to close entries while caching changes"); } - - this.cachedAdds = adds.build(); - this.cachedDeletes = deletes.build(); } @Override diff --git a/core/src/main/java/org/apache/iceberg/BaseTableScan.java b/core/src/main/java/org/apache/iceberg/BaseTableScan.java index 651bfa386e71..25458016448b 100644 --- a/core/src/main/java/org/apache/iceberg/BaseTableScan.java +++ b/core/src/main/java/org/apache/iceberg/BaseTableScan.java @@ -114,6 +114,11 @@ public TableScan appendsAfter(long fromSnapshotId) { throw new UnsupportedOperationException("Incremental scan is not supported"); } + @Override + public TableScan useManifests(Iterable manifests) { + throw new UnsupportedOperationException("Specific manifests scan is not supported"); + } + @Override public TableScan useSnapshot(long scanSnapshotId) { Preconditions.checkArgument(context.snapshotId() == null, diff --git a/core/src/main/java/org/apache/iceberg/DataTableScan.java b/core/src/main/java/org/apache/iceberg/DataTableScan.java index f589a1d6a1c3..055d43a195ab 100644 --- a/core/src/main/java/org/apache/iceberg/DataTableScan.java +++ b/core/src/main/java/org/apache/iceberg/DataTableScan.java @@ -64,6 +64,14 @@ public TableScan appendsAfter(long fromSnapshotId) { return appendsBetween(fromSnapshotId, currentSnapshot.snapshotId()); } + @Override + public TableScan useManifests(Iterable manifests) { + Long scanSnapshotId = snapshotId(); + Preconditions.checkState(scanSnapshotId == null, + "Cannot enable specific manifests scan, scan-snapshot set to id=%s", scanSnapshotId); + return new ManifestsDataTableScan(tableOps(), table(), schema(), context().useManifests(manifests)); + } + @Override public TableScan useSnapshot(long scanSnapshotId) { // call method in superclass just for the side effect of argument validation; diff --git a/core/src/main/java/org/apache/iceberg/DeleteFileIndex.java b/core/src/main/java/org/apache/iceberg/DeleteFileIndex.java index 8ebdb366a284..39313b39cdec 100644 --- a/core/src/main/java/org/apache/iceberg/DeleteFileIndex.java +++ b/core/src/main/java/org/apache/iceberg/DeleteFileIndex.java @@ -462,7 +462,7 @@ private Iterable>> deleteManifestRea Iterable matchingManifests = evalCache == null ? deleteManifests : Iterables.filter(deleteManifests, manifest -> manifest.content() == ManifestContent.DELETES && - (manifest.hasAddedFiles() || manifest.hasExistingFiles()) && + (manifest.hasAddedFiles() || manifest.hasExistingFiles() || manifest.hasDeletedFiles()) && evalCache.get(manifest.partitionSpecId()).eval(manifest)); return Iterables.transform( diff --git a/core/src/main/java/org/apache/iceberg/ManifestsDataTableScan.java b/core/src/main/java/org/apache/iceberg/ManifestsDataTableScan.java new file mode 100644 index 000000000000..9f4cf4febc38 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/ManifestsDataTableScan.java @@ -0,0 +1,92 @@ +/* + * 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.io.CloseableIterable; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.apache.iceberg.util.ThreadPools; + +public class ManifestsDataTableScan extends DataTableScan { + + ManifestsDataTableScan(TableOperations ops, Table table, Schema schema, TableScanContext context) { + super(ops, table, schema, context); + Preconditions.checkState(context.manifests() != null && !Iterables.isEmpty(context.manifests()), + "Scanned manifests cannot be null or empty"); + } + + @Override + public TableScan asOfTime(long timestampMillis) { + throw new UnsupportedOperationException(String.format( + "Cannot scan table as of time %s: configured for specific manifests %s", + timestampMillis, context().manifests())); + } + + @Override + public TableScan useSnapshot(long scanSnapshotId) { + throw new UnsupportedOperationException(String.format( + "Cannot scan table using scan snapshot id %s: configured for specific manifests %s", + scanSnapshotId, context().manifests())); + } + + @Override + public TableScan appendsBetween(long fromSnapshotId, long toSnapshotId) { + throw new UnsupportedOperationException(String.format( + "Cannot scan table in snapshots (%s, %s]: configured for specific manifests %s", + fromSnapshotId, toSnapshotId, context().manifests())); + } + + @Override + public TableScan appendsAfter(long fromSnapshotId) { + throw new UnsupportedOperationException(String.format( + "Cannot scan appends after %s: configured for specific manifests %s", + fromSnapshotId, context().manifests())); + } + + @Override + public TableScan useManifests(Iterable manifests) { + return new ManifestsDataTableScan(tableOps(), table(), schema(), context().useManifests(manifests)); + } + + @Override + protected TableScan newRefinedScan(TableOperations ops, Table table, Schema schema, TableScanContext context) { + return new ManifestsDataTableScan(ops, table, schema, context); + } + + @Override + public CloseableIterable planFiles() { + ManifestGroup manifestGroup = new ManifestGroup(tableOps().io(), context().manifests()) + .caseSensitive(isCaseSensitive()) + .select(colStats() ? SCAN_WITH_STATS_COLUMNS : SCAN_COLUMNS) + .filterData(filter()) + .specsById(tableOps().current().specsById()) + .ignoreDeleted(); + + if (shouldIgnoreResiduals()) { + manifestGroup = manifestGroup.ignoreResiduals(); + } + + if (PLAN_SCANS_WITH_WORKER_POOL && Iterables.size(context().manifests()) > 1) { + manifestGroup = manifestGroup.planWith(ThreadPools.getWorkerPool()); + } + + return manifestGroup.planFiles(); + } +} diff --git a/core/src/main/java/org/apache/iceberg/TableScanContext.java b/core/src/main/java/org/apache/iceberg/TableScanContext.java index dcf7d9753530..b8aad04358fd 100644 --- a/core/src/main/java/org/apache/iceberg/TableScanContext.java +++ b/core/src/main/java/org/apache/iceberg/TableScanContext.java @@ -40,6 +40,7 @@ final class TableScanContext { private final ImmutableMap options; private final Long fromSnapshotId; private final Long toSnapshotId; + private final Iterable manifests; TableScanContext() { this.snapshotId = null; @@ -52,12 +53,13 @@ final class TableScanContext { this.options = ImmutableMap.of(); this.fromSnapshotId = null; this.toSnapshotId = null; + this.manifests = null; } private TableScanContext(Long snapshotId, Expression rowFilter, boolean ignoreResiduals, boolean caseSensitive, boolean colStats, Schema projectedSchema, Collection selectedColumns, ImmutableMap options, - Long fromSnapshotId, Long toSnapshotId) { + Long fromSnapshotId, Long toSnapshotId, Iterable manifests) { this.snapshotId = snapshotId; this.rowFilter = rowFilter; this.ignoreResiduals = ignoreResiduals; @@ -68,6 +70,7 @@ private TableScanContext(Long snapshotId, Expression rowFilter, boolean ignoreRe this.options = options; this.fromSnapshotId = fromSnapshotId; this.toSnapshotId = toSnapshotId; + this.manifests = manifests; } Long snapshotId() { @@ -75,8 +78,8 @@ Long snapshotId() { } TableScanContext useSnapshotId(Long scanSnapshotId) { - return new TableScanContext(scanSnapshotId, rowFilter, ignoreResiduals, - caseSensitive, colStats, projectedSchema, selectedColumns, options, fromSnapshotId, toSnapshotId); + return new TableScanContext(scanSnapshotId, rowFilter, ignoreResiduals, caseSensitive, colStats, projectedSchema, + selectedColumns, options, fromSnapshotId, toSnapshotId, manifests); } Expression rowFilter() { @@ -84,8 +87,8 @@ Expression rowFilter() { } TableScanContext filterRows(Expression filter) { - return new TableScanContext(snapshotId, filter, ignoreResiduals, - caseSensitive, colStats, projectedSchema, selectedColumns, options, fromSnapshotId, toSnapshotId); + return new TableScanContext(snapshotId, filter, ignoreResiduals, caseSensitive, colStats, projectedSchema, + selectedColumns, options, fromSnapshotId, toSnapshotId, manifests); } boolean ignoreResiduals() { @@ -93,8 +96,8 @@ boolean ignoreResiduals() { } TableScanContext ignoreResiduals(boolean shouldIgnoreResiduals) { - return new TableScanContext(snapshotId, rowFilter, shouldIgnoreResiduals, - caseSensitive, colStats, projectedSchema, selectedColumns, options, fromSnapshotId, toSnapshotId); + return new TableScanContext(snapshotId, rowFilter, shouldIgnoreResiduals, caseSensitive, colStats, projectedSchema, + selectedColumns, options, fromSnapshotId, toSnapshotId, manifests); } boolean caseSensitive() { @@ -102,8 +105,8 @@ boolean caseSensitive() { } TableScanContext setCaseSensitive(boolean isCaseSensitive) { - return new TableScanContext(snapshotId, rowFilter, ignoreResiduals, - isCaseSensitive, colStats, projectedSchema, selectedColumns, options, fromSnapshotId, toSnapshotId); + return new TableScanContext(snapshotId, rowFilter, ignoreResiduals, isCaseSensitive, colStats, projectedSchema, + selectedColumns, options, fromSnapshotId, toSnapshotId, manifests); } boolean returnColumnStats() { @@ -111,8 +114,8 @@ boolean returnColumnStats() { } TableScanContext shouldReturnColumnStats(boolean returnColumnStats) { - return new TableScanContext(snapshotId, rowFilter, ignoreResiduals, - caseSensitive, returnColumnStats, projectedSchema, selectedColumns, options, fromSnapshotId, toSnapshotId); + return new TableScanContext(snapshotId, rowFilter, ignoreResiduals, caseSensitive, returnColumnStats, + projectedSchema, selectedColumns, options, fromSnapshotId, toSnapshotId, manifests); } Collection selectedColumns() { @@ -122,7 +125,7 @@ Collection selectedColumns() { TableScanContext selectColumns(Collection columns) { Preconditions.checkState(projectedSchema == null, "Cannot select columns when projection schema is set"); return new TableScanContext(snapshotId, rowFilter, ignoreResiduals, - caseSensitive, colStats, null, columns, options, fromSnapshotId, toSnapshotId); + caseSensitive, colStats, null, columns, options, fromSnapshotId, toSnapshotId, manifests); } Schema projectedSchema() { @@ -132,7 +135,7 @@ Schema projectedSchema() { TableScanContext project(Schema schema) { Preconditions.checkState(selectedColumns == null, "Cannot set projection schema when columns are selected"); return new TableScanContext(snapshotId, rowFilter, ignoreResiduals, - caseSensitive, colStats, schema, null, options, fromSnapshotId, toSnapshotId); + caseSensitive, colStats, schema, null, options, fromSnapshotId, toSnapshotId, manifests); } Map options() { @@ -143,8 +146,8 @@ TableScanContext withOption(String property, String value) { ImmutableMap.Builder builder = ImmutableMap.builder(); builder.putAll(options); builder.put(property, value); - return new TableScanContext(snapshotId, rowFilter, ignoreResiduals, - caseSensitive, colStats, projectedSchema, selectedColumns, builder.build(), fromSnapshotId, toSnapshotId); + return new TableScanContext(snapshotId, rowFilter, ignoreResiduals, caseSensitive, colStats, projectedSchema, + selectedColumns, builder.build(), fromSnapshotId, toSnapshotId, manifests); } Long fromSnapshotId() { @@ -152,8 +155,8 @@ Long fromSnapshotId() { } TableScanContext fromSnapshotId(long id) { - return new TableScanContext(snapshotId, rowFilter, ignoreResiduals, - caseSensitive, colStats, projectedSchema, selectedColumns, options, id, toSnapshotId); + return new TableScanContext(snapshotId, rowFilter, ignoreResiduals, caseSensitive, colStats, projectedSchema, + selectedColumns, options, id, toSnapshotId, manifests); } Long toSnapshotId() { @@ -161,7 +164,16 @@ Long toSnapshotId() { } TableScanContext toSnapshotId(long id) { - return new TableScanContext(snapshotId, rowFilter, ignoreResiduals, - caseSensitive, colStats, projectedSchema, selectedColumns, options, fromSnapshotId, id); + return new TableScanContext(snapshotId, rowFilter, ignoreResiduals, caseSensitive, colStats, projectedSchema, + selectedColumns, options, fromSnapshotId, id, manifests); + } + + Iterable manifests() { + return this.manifests; + } + + TableScanContext useManifests(Iterable scanManifests) { + return new TableScanContext(snapshotId, rowFilter, ignoreResiduals, caseSensitive, colStats, projectedSchema, + selectedColumns, options, fromSnapshotId, toSnapshotId, scanManifests); } } diff --git a/core/src/test/java/org/apache/iceberg/MockFileScanTask.java b/core/src/test/java/org/apache/iceberg/MockFileScanTask.java index aef8065b68b7..97030f6c8aeb 100644 --- a/core/src/test/java/org/apache/iceberg/MockFileScanTask.java +++ b/core/src/test/java/org/apache/iceberg/MockFileScanTask.java @@ -19,6 +19,7 @@ package org.apache.iceberg; +import org.apache.iceberg.expressions.ResidualEvaluator; import org.mockito.Mockito; public class MockFileScanTask extends BaseFileScanTask { @@ -40,6 +41,12 @@ public MockFileScanTask(DataFile file, DeleteFile[] deleteFiles) { this.length = file.fileSizeInBytes(); } + public MockFileScanTask(DataFile file, DeleteFile[] deleteFiles, String schemaString, String specString, + ResidualEvaluator residuals) { + super(file, deleteFiles, schemaString, specString, residuals); + this.length = file.fileSizeInBytes(); + } + public static MockFileScanTask mockTask(long length, int sortOrderId) { DataFile mockFile = Mockito.mock(DataFile.class); Mockito.when(mockFile.fileSizeInBytes()).thenReturn(length); diff --git a/core/src/test/java/org/apache/iceberg/TableTestBase.java b/core/src/test/java/org/apache/iceberg/TableTestBase.java index 82e607646102..c7d8b7f2169b 100644 --- a/core/src/test/java/org/apache/iceberg/TableTestBase.java +++ b/core/src/test/java/org/apache/iceberg/TableTestBase.java @@ -33,6 +33,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Iterators; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Sets; @@ -361,6 +362,24 @@ void validateSnapshot(Snapshot old, Snapshot snap, Long sequenceNumber, DataFile Assert.assertEquals("Schema ID should match", table.schema().schemaId(), (int) snap.schemaId()); } + void validateFiles(Iterable dataFiles, DataFile... expectedFiles) { + Set expectedFilePaths = Sets.newHashSet(); + for (DataFile file : expectedFiles) { + expectedFilePaths.add(file.path()); + } + Set actualFilePaths = Sets.newHashSet(Iterables.transform(dataFiles, DataFile::path)); + Assert.assertEquals("Files should match", expectedFilePaths, actualFilePaths); + } + + void validateDeleteFiles(Iterable deleteFiles, DeleteFile... expectedFiles) { + Set expectedFilePaths = Sets.newHashSet(); + for (DeleteFile file : expectedFiles) { + expectedFilePaths.add(file.path()); + } + Set actualFilePaths = Sets.newHashSet(Iterables.transform(deleteFiles, DeleteFile::path)); + Assert.assertEquals("Delete files should match", expectedFilePaths, actualFilePaths); + } + void validateTableFiles(Table tbl, DataFile... expectedFiles) { Set expectedFilePaths = Sets.newHashSet(); for (DataFile file : expectedFiles) { diff --git a/core/src/test/java/org/apache/iceberg/TestSnapshot.java b/core/src/test/java/org/apache/iceberg/TestSnapshot.java index d918accb7355..499396aa5e67 100644 --- a/core/src/test/java/org/apache/iceberg/TestSnapshot.java +++ b/core/src/test/java/org/apache/iceberg/TestSnapshot.java @@ -19,6 +19,8 @@ package org.apache.iceberg; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.junit.Assume; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -82,4 +84,35 @@ public void testAppendFoundFiles() { validateSnapshot(oldSnapshot, newSnapshot, FILE_A, FILE_B); } + @Test + public void testFileChanges() { + Assume.assumeFalse("Only support equality-delete in format v2.", formatVersion < 2); + + table.newRowDelta() + .addRows(FILE_A) + .addRows(FILE_B) + .addDeletes(FILE_A_DELETES) + .addDeletes(FILE_B_DELETES) + .commit(); + Snapshot oldSnapshot = table.currentSnapshot(); + + validateFiles(oldSnapshot.addedFiles(), FILE_A, FILE_B); + validateFiles(oldSnapshot.deletedFiles()); + validateDeleteFiles(oldSnapshot.addedDeleteFiles(), FILE_A_DELETES, FILE_B_DELETES); + validateDeleteFiles(oldSnapshot.deletedDeleteFiles()); + + table.newRewrite().rewriteFiles( + Sets.newHashSet(FILE_B), + Sets.newHashSet(FILE_A_DELETES, FILE_B_DELETES), + Sets.newHashSet(FILE_C), + Sets.newHashSet(FILE_A2_DELETES) + ).validateFromSnapshot(oldSnapshot.snapshotId()).commit(); + Snapshot newSnapshot = table.currentSnapshot(); + + validateFiles(newSnapshot.addedFiles(), FILE_C); + validateFiles(newSnapshot.deletedFiles(), FILE_B); + validateDeleteFiles(newSnapshot.addedDeleteFiles(), FILE_A2_DELETES); + validateDeleteFiles(newSnapshot.deletedDeleteFiles(), FILE_A_DELETES, FILE_B_DELETES); + } + } diff --git a/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/CommitResult.java b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/CommitResult.java new file mode 100644 index 000000000000..fc26909fa19e --- /dev/null +++ b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/CommitResult.java @@ -0,0 +1,81 @@ +/* + * 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.flink.sink; + +import java.io.Serializable; +import org.apache.iceberg.io.WriteResult; + +class CommitResult implements Serializable { + + private final long sequenceNumber; + private final long snapshotId; + private final WriteResult writeResult; + + private CommitResult(long snapshotId, + long sequenceNumber, + WriteResult writeResult) { + this.snapshotId = snapshotId; + this.sequenceNumber = sequenceNumber; + this.writeResult = writeResult; + } + + long sequenceNumber() { + return sequenceNumber; + } + + long snapshotId() { + return snapshotId; + } + + WriteResult writeResult() { + return writeResult; + } + + static Builder builder(long sequenceNumber, long snapshotId) { + return new Builder(sequenceNumber, snapshotId); + } + + static class Builder { + + private final long sequenceNumber; + private final long snapshotId; + private final WriteResult.Builder writeResult; + + private Builder(long sequenceNumber, long snapshotId) { + this.sequenceNumber = sequenceNumber; + this.snapshotId = snapshotId; + this.writeResult = WriteResult.builder(); + } + + Builder add(WriteResult result) { + this.writeResult.add(result); + return this; + } + + Builder addAll(Iterable results) { + this.writeResult.addAll(results); + return this; + } + + CommitResult build() { + return new CommitResult(snapshotId, sequenceNumber, writeResult.build()); + } + } +} diff --git a/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java index d20859377ffc..0afce353e261 100644 --- a/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java +++ b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java @@ -20,6 +20,7 @@ package org.apache.iceberg.flink.sink; import java.io.IOException; +import java.util.Collection; import java.util.List; import java.util.function.Supplier; import org.apache.iceberg.DataFile; @@ -55,12 +56,59 @@ static ManifestFile writeDataFiles(OutputFile outputFile, PartitionSpec spec, Li return writer.toManifestFile(); } + static ManifestFile writeDeleteFiles(OutputFile outputFile, PartitionSpec spec, List deleteFiles) + throws IOException { + ManifestWriter writer = ManifestFiles.writeDeleteManifest(FORMAT_V2, spec, outputFile, + DUMMY_SNAPSHOT_ID); + + try (ManifestWriter closeableWriter = writer) { + closeableWriter.addAll(deleteFiles); + } + return writer.toManifestFile(); + } + + static ManifestFile referenceDataFiles(OutputFile outputFile, + long sequenceNumber, + long snapshotId, + PartitionSpec spec, + List dataFiles) throws IOException { + ManifestWriter writer = ManifestFiles.write(FORMAT_V2, spec, outputFile, DUMMY_SNAPSHOT_ID); + + try (ManifestWriter closeableWriter = writer) { + dataFiles.forEach(datafile -> closeableWriter.existing(datafile, snapshotId, sequenceNumber)); + } + + return writer.toManifestFile(); + } + + static ManifestFile referenceDeleteFiles(OutputFile outputFile, + long sequenceNumber, + long snapshotId, + PartitionSpec spec, + List deleteFiles) throws IOException { + ManifestWriter writer = ManifestFiles.writeDeleteManifest(FORMAT_V2, spec, outputFile, + DUMMY_SNAPSHOT_ID); + + try (ManifestWriter closeableWriter = writer) { + deleteFiles.forEach(deleteFile -> closeableWriter.existing(deleteFile, snapshotId, sequenceNumber)); + } + + return writer.toManifestFile(); + } + static List readDataFiles(ManifestFile manifestFile, FileIO io) throws IOException { try (CloseableIterable dataFiles = ManifestFiles.read(manifestFile, io)) { return Lists.newArrayList(dataFiles); } } + static List readDeleteFiles(ManifestFile manifestFile, FileIO io) throws IOException { + try (CloseableIterable deleteFiles = ManifestFiles.readDeleteManifest(manifestFile, io, null)) { + return Lists.newArrayList(deleteFiles); + } + + } + static ManifestOutputFileFactory createOutputFileFactory(Table table, String flinkJobId, String operatorUniqueId, int subTaskId, long attemptNumber) { TableOperations ops = ((HasTableOperations) table).operations(); @@ -82,20 +130,35 @@ static DeltaManifests writeCompletedFiles(WriteResult result, // Write the completed delete files into a newly created delete manifest file. if (result.deleteFiles() != null && result.deleteFiles().length > 0) { - OutputFile deleteManifestFile = outputFileSupplier.get(); + deleteManifest = writeDeleteFiles(outputFileSupplier.get(), spec, Lists.newArrayList(result.deleteFiles())); + } - ManifestWriter deleteManifestWriter = ManifestFiles.writeDeleteManifest(FORMAT_V2, spec, - deleteManifestFile, DUMMY_SNAPSHOT_ID); - try (ManifestWriter writer = deleteManifestWriter) { - for (DeleteFile deleteFile : result.deleteFiles()) { - writer.add(deleteFile); - } - } + return new DeltaManifests(dataManifest, deleteManifest, result.referencedDataFiles()); + } + + static DeltaManifests writeExistingFiles(long sequenceNumber, + long snapshotId, + Collection dataFiles, + Collection deleteFiles, + Supplier outputFileSupplier, + PartitionSpec spec) throws IOException { + + ManifestFile dataManifest = null; + ManifestFile deleteManifest = null; - deleteManifest = deleteManifestWriter.toManifestFile(); + // Write the exists data files into a newly created data manifest file. + if (dataFiles != null && dataFiles.size() > 0) { + dataManifest = referenceDataFiles(outputFileSupplier.get(), sequenceNumber, snapshotId, spec, + Lists.newArrayList(dataFiles)); } - return new DeltaManifests(dataManifest, deleteManifest, result.referencedDataFiles()); + // Write the exists delete files into a newly created delete manifest file. + if (deleteFiles != null && deleteFiles.size() > 0) { + deleteManifest = referenceDeleteFiles(outputFileSupplier.get(), sequenceNumber, snapshotId, spec, + Lists.newArrayList(deleteFiles)); + } + + return new DeltaManifests(dataManifest, deleteManifest); } static WriteResult readCompletedFiles(DeltaManifests deltaManifests, FileIO io) throws IOException { @@ -108,10 +171,7 @@ static WriteResult readCompletedFiles(DeltaManifests deltaManifests, FileIO io) // Read the completed delete files from persisted delete manifests file. if (deltaManifests.deleteManifest() != null) { - try (CloseableIterable deleteFiles = ManifestFiles - .readDeleteManifest(deltaManifests.deleteManifest(), io, null)) { - builder.addDeleteFiles(deleteFiles); - } + builder.addDeleteFiles(readDeleteFiles(deltaManifests.deleteManifest(), io)); } return builder.addReferencedDataFiles(deltaManifests.referencedDataFiles()) diff --git a/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java index 6714ae8357a1..d623796e8abe 100644 --- a/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java +++ b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java @@ -27,7 +27,6 @@ import java.util.function.Function; import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.common.typeinfo.Types; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSink; import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator; @@ -71,6 +70,9 @@ public class FlinkSink { private static final String ICEBERG_STREAM_WRITER_NAME = IcebergStreamWriter.class.getSimpleName(); private static final String ICEBERG_FILES_COMMITTER_NAME = IcebergFilesCommitter.class.getSimpleName(); + private static final String ICEBERG_REWRITE_TASKS_EMITTER_NAME = IcebergRewriteTaskEmitter.class.getSimpleName(); + private static final String ICEBERG_STREAM_REWRITER_NAME = IcebergStreamRewriter.class.getSimpleName(); + private static final String ICEBERG_REWRITE_FILES_COMMITTER_NAME = IcebergRewriteFilesCommitter.class.getSimpleName(); private FlinkSink() { } @@ -131,6 +133,8 @@ public static class Builder { private boolean upsert = false; private List equalityFieldColumns = null; private String uidPrefix = null; + private boolean rewrite = false; + private Integer rewriteParallelism = null; private Builder() { } @@ -267,6 +271,29 @@ public Builder uidPrefix(String newPrefix) { return this; } + /** + * Configuring whether to enable the rewrite operator. The rewrite operator will rewrite committed files between + * multiple checkpoints. + * + * @param enabled indicate whether it should rewrite committed files. + * @return {@link Builder} to connect the iceberg table. + */ + public Builder rewrite(boolean enabled) { + this.rewrite = enabled; + return this; + } + + /** + * Configuring the rewrite parallel number for iceberg stream rewriter. + * + * @param newRewriteParallelism the number of parallel iceberg stream rewriter. + * @return {@link Builder} to connect the iceberg table. + */ + public Builder rewriteParallelism(int newRewriteParallelism) { + this.rewriteParallelism = newRewriteParallelism; + return this; + } + private DataStreamSink chainIcebergOperators() { Preconditions.checkArgument(inputCreator != null, "Please use forRowData() or forMapperOutputType() to initialize the input DataStream."); @@ -293,12 +320,20 @@ private DataStreamSink chainIcebergOperators() { // Add parallel writers that append rows to files SingleOutputStreamOperator writerStream = appendWriter(distributeStream, flinkRowType); + boolean rewriteEnabled = rewrite || PropertyUtil.propertyAsBoolean(table.properties(), + FlinkSinkOptions.STREAMING_REWRITE_ENABLE, FlinkSinkOptions.STREAMING_REWRITE_ENABLE_DEFAULT); + // Add single-parallelism committer that commits files // after successful checkpoint or end of input - SingleOutputStreamOperator committerStream = appendCommitter(writerStream); + SingleOutputStreamOperator committerStream = appendCommitter(writerStream); + + // Add parallel rewriter and single-parallelism committer to rewrite committed files + // when streaming rewrite is enable. + SingleOutputStreamOperator rewriterStream = rewriteEnabled ? + appendRewriter(committerStream) : committerStream; // Add dummy discard sink - return appendDummySink(committerStream); + return appendDummySink(rewriterStream); } /** @@ -327,7 +362,7 @@ private String operatorName(String suffix) { } @SuppressWarnings("unchecked") - private DataStreamSink appendDummySink(SingleOutputStreamOperator committerStream) { + private DataStreamSink appendDummySink(SingleOutputStreamOperator committerStream) { DataStreamSink resultStream = committerStream .addSink(new DiscardingSink()) .name(operatorName(String.format("IcebergSink %s", this.table.name()))) @@ -338,10 +373,45 @@ private DataStreamSink appendDummySink(SingleOutputStreamOperator c return resultStream; } - private SingleOutputStreamOperator appendCommitter(SingleOutputStreamOperator writerStream) { + private SingleOutputStreamOperator appendRewriter( + SingleOutputStreamOperator committerStream) { + + Integer fileRewriterParallelism = rewriteParallelism; + if (fileRewriterParallelism == null) { + // Fallback to use rewrite parallelism parsed from table properties if don't specify in job level. + String parallelism = table.properties().get(FlinkSinkOptions.STREAMING_REWRITE_PARALLELISM); + fileRewriterParallelism = parallelism != null ? Integer.valueOf(parallelism) : + FlinkSinkOptions.STREAMING_REWRITE_PARALLELISM_DEFAULT; + } + + IcebergRewriteTaskEmitter emitter = new IcebergRewriteTaskEmitter(tableLoader); + SingleOutputStreamOperator emitterStream = committerStream + .transform(operatorName(ICEBERG_REWRITE_TASKS_EMITTER_NAME), TypeInformation.of(RewriteTask.class), emitter) + .setParallelism(1) + .setMaxParallelism(1); + if (uidPrefix != null) { + emitterStream = emitterStream.uid(uidPrefix + "-rewriter"); + } + + IcebergStreamRewriter rewriter = new IcebergStreamRewriter(tableLoader); + SingleOutputStreamOperator rewriterStream = emitterStream + .transform(operatorName(ICEBERG_STREAM_REWRITER_NAME), TypeInformation.of(RewriteResult.class), rewriter); + if (fileRewriterParallelism != null) { + rewriterStream = rewriterStream.setParallelism(fileRewriterParallelism); + } + + IcebergRewriteFilesCommitter committer = new IcebergRewriteFilesCommitter(tableLoader); + return rewriterStream + .transform(operatorName(ICEBERG_REWRITE_FILES_COMMITTER_NAME), TypeInformation.of(Void.class), committer) + .setParallelism(1) + .setMaxParallelism(1); + } + + private SingleOutputStreamOperator appendCommitter( + SingleOutputStreamOperator writerStream) { IcebergFilesCommitter filesCommitter = new IcebergFilesCommitter(tableLoader, overwrite); - SingleOutputStreamOperator committerStream = writerStream - .transform(operatorName(ICEBERG_FILES_COMMITTER_NAME), Types.VOID, filesCommitter) + SingleOutputStreamOperator committerStream = writerStream + .transform(operatorName(ICEBERG_FILES_COMMITTER_NAME), TypeInformation.of(CommitResult.class), filesCommitter) .setParallelism(1) .setMaxParallelism(1); if (uidPrefix != null) { diff --git a/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSinkOptions.java b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSinkOptions.java new file mode 100644 index 000000000000..2a75c282111d --- /dev/null +++ b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSinkOptions.java @@ -0,0 +1,96 @@ +/* + * 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.flink.sink; + +import org.apache.iceberg.TableProperties; + +/** + * Flink iceberg sinker options + */ +public class FlinkSinkOptions { + + private FlinkSinkOptions() { + } + + public static final String STREAMING_REWRITE_ENABLE = "flink.rewrite.enable"; + public static final boolean STREAMING_REWRITE_ENABLE_DEFAULT = false; + + public static final String STREAMING_REWRITE_PARALLELISM = "flink.rewrite.parallelism"; + public static final Integer STREAMING_REWRITE_PARALLELISM_DEFAULT = null; // use flink job default parallelism + + public static final String STREAMING_REWRITE_CASE_SENSITIVE = "flink.rewrite.case-sensitive"; + public static final boolean STREAMING_REWRITE_CASE_SENSITIVE_DEFAULT = false; + + /** + * The output file size attempt to generate when rewriting files. + *

+ * Defaults will use the {@link TableProperties#WRITE_TARGET_FILE_SIZE_BYTES} value. + */ + public static final String STREAMING_REWRITE_TARGET_FILE_SIZE_BYTES = "flink.rewrite.target-file-size-bytes"; + + /** + * Adjusts files which will be considered for rewriting. + * Files smaller than this value will be considered for rewriting. + *

+ * Defaults to 75% of the target file size + */ + public static final String STREAMING_REWRITE_MIN_FILE_SIZE_BYTES = "flink.rewrite.min-file-size-bytes"; + public static final double STREAMING_REWRITE_MIN_FILE_SIZE_DEFAULT_RATIO = 0.75d; + + /** + * Adjusts files which will be considered for rewriting. + * Files larger than this value will be considered for rewriting. + *

+ * Defaults to 180% of the target file size + */ + public static final String STREAMING_REWRITE_MAX_FILE_SIZE_BYTES = "flink.rewrite.max-file-size-bytes"; + public static final double STREAMING_REWRITE_MAX_FILE_SIZE_DEFAULT_RATIO = 1.80d; + + /** + * The minimum number of files that need to be in a file group for it to be considered for rewriting, + * if the total size of that group is reach the {@link #STREAMING_REWRITE_TARGET_FILE_SIZE_BYTES} or + * the total number of files of that group is reach to the {@link #STREAMING_REWRITE_MAX_GROUP_FILES}. + *

+ * Defaults to 2 files, which is mean at less 2 files in a file group will be considered for rewriting. + */ + public static final String STREAMING_REWRITE_MIN_GROUP_FILES = "flink.rewrite.min-group-files"; + public static final int STREAMING_REWRITE_MIN_GROUP_FILES_DEFAULT = 2; + + /** + * The maximum number of files that allow to be in a file group for it to be considered for rewriting. + * Once the total number of files of that group is reach to this value, the file group will be rewritten + * regardless of whether the total size of that group reaches the {@link #STREAMING_REWRITE_TARGET_FILE_SIZE_BYTES}. + *

+ * Defaults to Integer.MAX_VALUE, which means this feature is not enabled by default. + */ + public static final String STREAMING_REWRITE_MAX_GROUP_FILES = "flink.rewrite.max-group-files"; + public static final int STREAMING_REWRITE_MAX_GROUP_FILES_DEFAULT = Integer.MAX_VALUE; + + /** + * The maximum number of commits will be wait for a file group. If no more file append to this file group after + * this number of commits, this file group will be rewritten regardless of whether the total size of that group + * reaches the {@link #STREAMING_REWRITE_TARGET_FILE_SIZE_BYTES}. + *

+ * Defaults to Integer.MAX_VALUE, which means this feature is not enabled by default. + */ + public static final String STREAMING_REWRITE_MAX_WAITING_COMMITS = "flink.rewrite.nums-of-commit-after-append"; + public static final int STREAMING_REWRITE_MAX_WAITING_COMMITS_DEFAULT = Integer.MAX_VALUE; + +} diff --git a/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java index beffff68fde8..6b23b876a1b1 100644 --- a/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java +++ b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.util.Arrays; +import java.util.Collections; import java.util.Comparator; import java.util.List; import java.util.Map; @@ -45,6 +46,7 @@ import org.apache.iceberg.Snapshot; import org.apache.iceberg.SnapshotUpdate; import org.apache.iceberg.Table; +import org.apache.iceberg.events.CreateSnapshotEvent; import org.apache.iceberg.flink.TableLoader; import org.apache.iceberg.io.WriteResult; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; @@ -58,8 +60,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -class IcebergFilesCommitter extends AbstractStreamOperator - implements OneInputStreamOperator, BoundedOneInput { +class IcebergFilesCommitter extends AbstractStreamOperator + implements OneInputStreamOperator, BoundedOneInput { private static final long serialVersionUID = 1L; private static final long INITIAL_CHECKPOINT_ID = -1L; @@ -276,6 +278,8 @@ private void commitDeltaTxn(NavigableMap pendingResults, Stri } commitOperation(appendFiles, numFiles, 0, "append", newFlinkJobId, checkpointId); + + emitCommitResult(appendFiles, pendingResults.values()); } else { // To be compatible with iceberg format V2. for (Map.Entry e : pendingResults.entrySet()) { @@ -299,6 +303,8 @@ private void commitDeltaTxn(NavigableMap pendingResults, Stri Arrays.stream(result.deleteFiles()).forEach(rowDelta::addDeletes); commitOperation(rowDelta, numDataFiles, numDeleteFiles, "rowDelta", newFlinkJobId, e.getKey()); + + emitCommitResult(rowDelta, Collections.singleton(result)); } } } @@ -316,11 +322,26 @@ private void commitOperation(SnapshotUpdate operation, int numDataFiles, int LOG.info("Committed in {} ms", duration); } + private void emitCommitResult(SnapshotUpdate operation, Iterable results) { + Preconditions.checkArgument(operation.updateEvent() instanceof CreateSnapshotEvent, + "Operation update event should be instance of CreateSnapshotEvent, but not %s", operation.updateEvent()); + + CreateSnapshotEvent event = (CreateSnapshotEvent) operation.updateEvent(); + CommitResult commitResult = CommitResult.builder(event.sequenceNumber(), event.snapshotId()) + .addAll(results) + .build(); + emit(commitResult); + } + @Override public void processElement(StreamRecord element) { this.writeResultsOfCurrentCkpt.add(element.getValue()); } + private void emit(CommitResult result) { + output.collect(new StreamRecord<>(result)); + } + @Override public void endInput() throws IOException { // Flush the buffered data files into 'dataFilesPerCheckpoint' firstly. diff --git a/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergRewriteFilesCommitter.java b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergRewriteFilesCommitter.java new file mode 100644 index 000000000000..d65dd241fe94 --- /dev/null +++ b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergRewriteFilesCommitter.java @@ -0,0 +1,124 @@ +/* + * 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.flink.sink; + +import java.util.Map; +import org.apache.flink.runtime.state.StateSnapshotContext; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.iceberg.HasTableOperations; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.RewriteFiles; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableOperations; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.util.Tasks; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +class IcebergRewriteFilesCommitter extends AbstractStreamOperator + implements OneInputStreamOperator { + + private static final long serialVersionUID = 1L; + + private static final Logger LOG = LoggerFactory.getLogger(IcebergRewriteFilesCommitter.class); + + private final TableLoader tableLoader; + + private transient Table table; + private transient TableOperations ops; + private transient PartitionSpec spec; + + private final Map rewriteResults = Maps.newLinkedHashMap(); + + IcebergRewriteFilesCommitter(TableLoader tableLoader) { + this.tableLoader = tableLoader; + } + + @Override + public void open() throws Exception { + this.tableLoader.open(); + this.table = tableLoader.loadTable(); + this.ops = ((HasTableOperations) table).operations(); + this.spec = table.spec(); + } + + @Override + public void snapshotState(StateSnapshotContext context) throws Exception { + super.snapshotState(context); + long checkpointId = context.getCheckpointId(); + LOG.info("Start to commit rewrite results, table: {}, checkpointId: {}", table, checkpointId); + + commitRewriteResults(); + } + + @Override + public void processElement(StreamRecord record) throws Exception { + RewriteResult result = record.getValue(); + + rewriteResults.computeIfAbsent(result.startingSnapshotId(), k -> RewriteResult.builder(k, spec.partitionType())) + .partitions(result.partitions()) + .addAddedDataFiles(result.addedDataFiles()) + .addRewrittenDataFiles(result.rewrittenDataFiles()); + } + + private void commitRewriteResults() { + // Refresh the table to get the committed snapshot of rewrite results. + table.refresh(); + + for (RewriteResult.Builder builder : rewriteResults.values()) { + commitRewriteResult(builder.build()); + } + rewriteResults.clear(); + } + + private void commitRewriteResult(RewriteResult result) { + LOG.info("Committing rewrite file groups of table {}: {}.", table, result); + + long start = System.currentTimeMillis(); + try { + long sequenceNumber = table.snapshot(result.startingSnapshotId()).sequenceNumber(); + RewriteFiles rewriteFiles = table.newRewrite() + .validateFromSnapshot(result.startingSnapshotId()) + .rewriteFiles(result.rewrittenDataFiles(), result.addedDataFiles(), sequenceNumber); + rewriteFiles.commit(); + LOG.info("Committed rewrite file groups in {} ms.", System.currentTimeMillis() - start); + } catch (Exception e) { + LOG.error("Cannot commit rewrite file groups, attempting to clean up written files.", e); + + Tasks.foreach(Iterables.transform(result.addedDataFiles(), f -> f.path().toString())) + .noRetry() + .suppressFailureWhenFinished() + .onFailure((location, exc) -> LOG.warn("Failed to delete: {}", location, exc)) + .run(ops.io()::deleteFile); + } + } + + @Override + public void close() throws Exception { + super.close(); + if (tableLoader != null) { + tableLoader.close(); + } + } +} diff --git a/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergRewriteTaskEmitter.java b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergRewriteTaskEmitter.java new file mode 100644 index 000000000000..830214a9eb34 --- /dev/null +++ b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergRewriteTaskEmitter.java @@ -0,0 +1,536 @@ +/* + * 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.flink.sink; + +import java.io.IOException; +import java.math.RoundingMode; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.Deque; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.NavigableMap; +import java.util.Set; +import org.apache.flink.api.common.state.ListState; +import org.apache.flink.api.common.state.ListStateDescriptor; +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.api.common.typeinfo.TypeHint; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.core.io.SimpleVersionedSerialization; +import org.apache.flink.runtime.state.StateInitializationContext; +import org.apache.flink.runtime.state.StateSnapshotContext; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.ContentFile; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataOperations; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileContent; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.base.Strings; +import org.apache.iceberg.relocated.com.google.common.collect.FluentIterable; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +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.Maps; +import org.apache.iceberg.relocated.com.google.common.collect.Multimap; +import org.apache.iceberg.relocated.com.google.common.collect.Multimaps; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.relocated.com.google.common.math.LongMath; +import org.apache.iceberg.util.PropertyUtil; +import org.apache.iceberg.util.SnapshotUtil; +import org.apache.iceberg.util.StructLikeWrapper; +import org.apache.iceberg.util.TableScanUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class IcebergRewriteTaskEmitter extends AbstractStreamOperator + implements OneInputStreamOperator { + + private static final long serialVersionUID = 1L; + private static final long DUMMY_SNAPSHOT_ID = -1L; + private static final long MAX_CHECKPOINT_ID = Long.MAX_VALUE; + private static final Set VALIDATE_DATA_CHANGE_FILES_OPERATIONS = + ImmutableSet.of(DataOperations.APPEND, DataOperations.OVERWRITE, DataOperations.DELETE); + + private static final Logger LOG = LoggerFactory.getLogger(IcebergRewriteTaskEmitter.class); + private static final String FLINK_JOB_ID = "flink.job-id"; + + private final TableLoader tableLoader; + + private transient Table table; + private transient String flinkJobId; + private transient boolean caseSensitive; + private transient StreamingBinPackStrategy strategy; + private transient ManifestOutputFileFactory manifestOutputFileFactory; + + private transient long lastReceivedSnapshotId; + private transient Map> pendingFileGroupsByPartition; + private transient NavigableMap> rewrittenFileGroupsPerCheckpoint; + + private static final ListStateDescriptor JOB_ID_DESCRIPTOR = new ListStateDescriptor<>( + "iceberg-flink-job-id", BasicTypeInfo.STRING_TYPE_INFO); + private transient ListState jobIdState; + + private static final ListStateDescriptor RECEIVED_SNAPSHOT_ID_DESCRIPTOR = new ListStateDescriptor<>( + "iceberg-streaming-rewrite-received-snapshot-id", BasicTypeInfo.LONG_TYPE_INFO); + private transient ListState receivedSnapshotIdState; + + private static final ListStateDescriptor REWRITE_FILE_GROUPS_DESCRIPTOR = new ListStateDescriptor<>( + "iceberg-streaming-rewrite-pending-file-groups-state", TypeInformation.of(new TypeHint() {})); + private transient ListState pendingRewriteFileGroupsState; + + public IcebergRewriteTaskEmitter(TableLoader tableLoader) { + this.tableLoader = tableLoader; + } + + @Override + public void initializeState(StateInitializationContext context) throws Exception { + super.initializeState(context); + + // Open the table loader and load the table. + this.tableLoader.open(); + this.table = tableLoader.loadTable(); + this.strategy = new StreamingBinPackStrategy(); + + this.caseSensitive = PropertyUtil.propertyAsBoolean(table.properties(), + FlinkSinkOptions.STREAMING_REWRITE_CASE_SENSITIVE, + FlinkSinkOptions.STREAMING_REWRITE_CASE_SENSITIVE_DEFAULT); + + this.lastReceivedSnapshotId = DUMMY_SNAPSHOT_ID; + this.pendingFileGroupsByPartition = Maps.newHashMap(); + this.rewrittenFileGroupsPerCheckpoint = Maps.newTreeMap(); + + this.flinkJobId = getContainingTask().getEnvironment().getJobID().toString(); + int subTaskId = getRuntimeContext().getIndexOfThisSubtask(); + int attemptId = getRuntimeContext().getAttemptNumber(); + this.manifestOutputFileFactory = FlinkManifestUtil.createOutputFileFactory(table, flinkJobId, + getOperatorID().toString(), subTaskId, attemptId); + + this.jobIdState = context.getOperatorStateStore().getListState(JOB_ID_DESCRIPTOR); + this.receivedSnapshotIdState = context.getOperatorStateStore().getListState(RECEIVED_SNAPSHOT_ID_DESCRIPTOR); + this.pendingRewriteFileGroupsState = context.getOperatorStateStore().getListState(REWRITE_FILE_GROUPS_DESCRIPTOR); + if (context.isRestored()) { + String restoredFlinkJobId = jobIdState.get().iterator().next(); + Preconditions.checkState(!Strings.isNullOrEmpty(restoredFlinkJobId), + "Flink job id parsed from checkpoint snapshot shouldn't be null or empty"); + + for (byte[] bytes : pendingRewriteFileGroupsState.get()) { + RewriteFileGroup fileGroup = SimpleVersionedSerialization.readVersionAndDeSerialize( + RewriteFileGroup.Serializer.INSTANCE, bytes); + StructLikeWrapper wrapper = StructLikeWrapper.forType(table.spec().partitionType()).set(fileGroup.partition()); + pendingFileGroupsByPartition.computeIfAbsent(wrapper, p -> Lists.newLinkedList()).add(fileGroup); + } + + this.lastReceivedSnapshotId = receivedSnapshotIdState.get().iterator().next(); + long lastCommittedSnapshotId = getLastCommittedSnapshotId(table, restoredFlinkJobId); + // The last received snapshot id would be negative on first commit. + if (lastCommittedSnapshotId > 0 && (lastReceivedSnapshotId == DUMMY_SNAPSHOT_ID || + SnapshotUtil.isAncestorOf(table, lastCommittedSnapshotId, lastReceivedSnapshotId))) { + // Restore last received snapshot id from state will lose the last committed snapshot of restored flink job, + // because the committer emit committed result after checkpoint so that the last received snapshot id + // has not updated when checkpointing. Therefore, we need to append all data files and delete files + // which are committed by the restored flink job and append all eq-delete files which are committed by other + // writer to catch up to the last committed snapshot of the restored flink job. + appendFilesWithin(lastReceivedSnapshotId, lastCommittedSnapshotId, restoredFlinkJobId); + emitRewriteTask(); + this.lastReceivedSnapshotId = lastCommittedSnapshotId; + } + } + } + + @Override + public void snapshotState(StateSnapshotContext context) throws Exception { + super.snapshotState(context); + long checkpointId = context.getCheckpointId(); + LOG.info("Start to flush rewrite tasks to state backend, table: {}, checkpointId: {}", table, checkpointId); + + jobIdState.clear(); + jobIdState.add(flinkJobId); + + receivedSnapshotIdState.clear(); + receivedSnapshotIdState.add(lastReceivedSnapshotId); + + List pendingRewriteFileGroups = Lists.newArrayListWithCapacity(pendingFileGroupsByPartition.size()); + for (Deque rewriteFileGroups : pendingFileGroupsByPartition.values()) { + for (RewriteFileGroup rewriteFileGroup : rewriteFileGroups) { + pendingRewriteFileGroups.add(SimpleVersionedSerialization.writeVersionAndSerialize( + RewriteFileGroup.Serializer.INSTANCE, rewriteFileGroup)); + } + } + pendingRewriteFileGroupsState.clear(); + pendingRewriteFileGroupsState.addAll(pendingRewriteFileGroups); + + // Setting checkpoint id to rewritten file groups which is rewritten in this checkpoint, + // to prevent delete asynchronously rewritten file groups which is rewritten in next checkpoint. + List rewrittenFileGroups = rewrittenFileGroupsPerCheckpoint.remove(MAX_CHECKPOINT_ID); + if (rewrittenFileGroups != null && !rewrittenFileGroups.isEmpty()) { + rewrittenFileGroupsPerCheckpoint.put(checkpointId, rewrittenFileGroups); + } + } + + @Override + public void notifyCheckpointComplete(long checkpointId) throws Exception { + super.notifyCheckpointComplete(checkpointId); + + // Delete rewritten file groups manifests after checkpoint to prevent lost manifest when restore from checkpoint. + NavigableMap> pendingMap = rewrittenFileGroupsPerCheckpoint.headMap( + checkpointId, true); + pendingMap.values().forEach(fileGroups -> fileGroups.forEach(fileGroup -> { + for (ManifestFile file : fileGroup.manifestFiles()) { + try { + table.io().deleteFile(file.path()); + } catch (Exception e) { + LOG.warn("The file group {} has been rewritten, but we failed to clean the temporary manifests: {}", + fileGroup, file.path(), e); + } + } + })); + } + + @Override + public void processElement(StreamRecord record) throws Exception { + CommitResult committed = record.getValue(); + + // Refresh the table to get the last committed snapshot of the commit result. + table.refresh(); + Snapshot committedSnapshot = table.snapshot(committed.snapshotId()); + if (lastReceivedSnapshotId != DUMMY_SNAPSHOT_ID && lastReceivedSnapshotId != committedSnapshot.parentId()) { + // Concurrent writer will commit other snapshot and make lastReceivedSnapshot and committedSnapshot discontinuous. + // Concurrent writer maybe adding equality delete files which could delete any this job committed records + // and should be applied to the data file we received before. Therefore, we need to emit all delete files + // between (lastReceivedSnapshotId, committedSnapshotId) to avoid missing any equality delete files + // when doing compaction. + // See more details in https://github.com/apache/iceberg/pull/3323#issuecomment-962068331 + appendFilesWithin(lastReceivedSnapshotId, committedSnapshot.parentId(), flinkJobId); + } + + LOG.info("Append files for snapshot id {} with flink job id {}", committed.snapshotId(), flinkJobId); + Iterable dataFiles = Arrays.asList(committed.writeResult().dataFiles()); + Iterable deleteFiles = Arrays.asList(committed.writeResult().deleteFiles()); + appendFilesFor(committed.sequenceNumber(), committed.snapshotId(), dataFiles, deleteFiles); + + emitRewriteTask(); + + lastReceivedSnapshotId = committed.snapshotId(); + } + + private void appendFilesWithin(long fromSnapshotId, long toSnapshotId, String jobId) throws IOException { + LOG.info("Append files from snapshot id {} to snapshot id {} with flink job id {}", + fromSnapshotId, toSnapshotId, jobId); + List snapshotIds = SnapshotUtil.snapshotIdsBetween(table, fromSnapshotId, toSnapshotId); + Collections.reverse(snapshotIds); // emit file groups on sequence number + for (Long snapshotId : snapshotIds) { + Snapshot snapshot = table.snapshot(snapshotId); + if (!VALIDATE_DATA_CHANGE_FILES_OPERATIONS.contains(snapshot.operation())) { + continue; + } + + // Only emit data files which are committed by the flink job of the job id. + Iterable dataFiles = isCommittedByFlinkJob(snapshot, jobId) ? + snapshot.addedFiles() : Collections.emptyList(); + + // Emit all delete files which are committed by the flink job of the job id. + // And only emit eq-delete files which are committed by others. + Iterable deleteFiles = Iterables.filter(snapshot.addedDeleteFiles(), + deleteFile -> isCommittedByFlinkJob(snapshot, jobId) || deleteFile.content() == FileContent.EQUALITY_DELETES); + + appendFilesFor(snapshot.sequenceNumber(), snapshotId, dataFiles, deleteFiles); + } + } + + private void appendFilesFor(long sequenceNumber, long snapshotId, + Iterable dataFiles, Iterable deleteFiles) throws IOException { + Map> dataFileGroup = groupFilesByPartition(dataFiles); + Map> deleteFileGroup = groupFilesByPartition(deleteFiles); + + Set partitions = Sets.union(dataFileGroup.keySet(), deleteFileGroup.keySet()); + for (StructLikeWrapper partition : partitions) { + Collection partitionDataFiles = dataFileGroup.getOrDefault(partition, Collections.emptyList()); + Collection partitionDeleteFiles = deleteFileGroup.getOrDefault(partition, Collections.emptyList()); + appendFiles(partition, sequenceNumber, snapshotId, partitionDataFiles, partitionDeleteFiles); + } + } + + private void appendFiles(StructLikeWrapper partition, long sequenceNumber, long snapshotId, + Collection dataFiles, Collection deleteFiles) throws IOException { + Deque bins = pendingFileGroupsByPartition.computeIfAbsent(partition, p -> Lists.newLinkedList()); + strategy.packTo(bins, partition.get(), sequenceNumber, snapshotId, dataFiles, deleteFiles); + } + + private > Map> groupFilesByPartition(Iterable files) { + Multimap filesByPartition = Multimaps.newListMultimap(Maps.newHashMap(), Lists::newArrayList); + for (F file : files) { + PartitionSpec spec = table.specs().get(file.specId()); + StructLikeWrapper partition = StructLikeWrapper.forType(spec.partitionType()).set(file.partition()); + filesByPartition.put(partition, file); + } + return filesByPartition.asMap(); + } + + private void emitRewriteTask() { + Iterator> iterator = pendingFileGroupsByPartition.values().iterator(); + while (iterator.hasNext()) { + + Deque bins = iterator.next(); + for (RewriteFileGroup fileGroup : strategy.pickFrom(bins)) { + + Iterable tasks = strategy.planTasks(fileGroup); + + tasks.forEach(task -> emit(new RewriteTask(fileGroup.latestSnapshotId(), fileGroup.partition(), task))); + + rewrittenFileGroupsPerCheckpoint.computeIfAbsent(MAX_CHECKPOINT_ID, k -> Lists.newArrayList()).add(fileGroup); + } + + if (bins.isEmpty() || bins.peek().isEmpty()) { + iterator.remove(); + } + } + } + + private void emit(RewriteTask rewriteTask) { + LOG.info("Emit rewrite task: {}.", rewriteTask); + output.collect(new StreamRecord<>(rewriteTask)); + } + + @Override + public void close() throws Exception { + super.close(); + if (tableLoader != null) { + tableLoader.close(); + } + } + + private static long getLastCommittedSnapshotId(Table table, String flinkJobId) { + Snapshot snapshot = table.currentSnapshot(); + while (snapshot != null) { + if (isCommittedByFlinkJob(snapshot, flinkJobId)) { + return snapshot.snapshotId(); + } + + Long parentSnapshotId = snapshot.parentId(); + snapshot = parentSnapshotId != null ? table.snapshot(parentSnapshotId) : null; + } + return DUMMY_SNAPSHOT_ID; + } + + private static int getCommitNumAfter(Table table, long snapshotId, String flinkJobId) { + int counter = 0; + Snapshot snapshot = table.currentSnapshot(); + while (snapshot != null && snapshot.snapshotId() != snapshotId) { + if (isCommittedByFlinkJob(snapshot, flinkJobId)) { + counter++; + } + + Long parentSnapshotId = snapshot.parentId(); + snapshot = parentSnapshotId != null ? table.snapshot(parentSnapshotId) : null; + } + return counter; + } + + private static boolean isCommittedByFlinkJob(Snapshot snapshot, String flinkJobId) { + String snapshotFlinkJobId = snapshot.summary().get(FLINK_JOB_ID); + return snapshotFlinkJobId != null && snapshotFlinkJobId.equals(flinkJobId); + } + + private class StreamingBinPackStrategy { + + private final long minFileSize; + private final long maxFileSize; + private final long targetFileSize; + private final long writeMaxFileSize; + private final int minGroupFiles; + private final int maxGroupFiles; + private final int maxWaitingCommits; + + private StreamingBinPackStrategy() { + this.targetFileSize = PropertyUtil.propertyAsLong(table.properties(), + FlinkSinkOptions.STREAMING_REWRITE_TARGET_FILE_SIZE_BYTES, + PropertyUtil.propertyAsLong(table.properties(), + TableProperties.WRITE_TARGET_FILE_SIZE_BYTES, + TableProperties.WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT)); + + this.minFileSize = PropertyUtil.propertyAsLong(table.properties(), + FlinkSinkOptions.STREAMING_REWRITE_MIN_FILE_SIZE_BYTES, + (long) (targetFileSize * FlinkSinkOptions.STREAMING_REWRITE_MIN_FILE_SIZE_DEFAULT_RATIO)); + + this.maxFileSize = PropertyUtil.propertyAsLong(table.properties(), + FlinkSinkOptions.STREAMING_REWRITE_MAX_FILE_SIZE_BYTES, + (long) (targetFileSize * FlinkSinkOptions.STREAMING_REWRITE_MAX_FILE_SIZE_DEFAULT_RATIO)); + + this.writeMaxFileSize = (long) (targetFileSize + ((maxFileSize - targetFileSize) * 0.5)); + + this.minGroupFiles = PropertyUtil.propertyAsInt(table.properties(), + FlinkSinkOptions.STREAMING_REWRITE_MIN_GROUP_FILES, + FlinkSinkOptions.STREAMING_REWRITE_MIN_GROUP_FILES_DEFAULT); + + this.maxGroupFiles = PropertyUtil.propertyAsInt(table.properties(), + FlinkSinkOptions.STREAMING_REWRITE_MAX_GROUP_FILES, + FlinkSinkOptions.STREAMING_REWRITE_MAX_GROUP_FILES_DEFAULT); + + this.maxWaitingCommits = PropertyUtil.propertyAsInt(table.properties(), + FlinkSinkOptions.STREAMING_REWRITE_MAX_WAITING_COMMITS, + FlinkSinkOptions.STREAMING_REWRITE_MAX_WAITING_COMMITS_DEFAULT); + + validateOptions(); + } + + private boolean shouldBeRewritten(DataFile dataFile) { + return dataFile.fileSizeInBytes() < minFileSize || dataFile.fileSizeInBytes() > maxFileSize; + } + + private void packTo(Deque bins, StructLike partition, long sequenceNumber, long snapshotId, + Collection dataFiles, Collection deleteFiles) throws IOException { + + RewriteFileGroup bin = bins.peekLast(); + if (bin == null) { + bin = new RewriteFileGroup(partition); + bins.addLast(bin); + } + + long rewriteFilesSize = 0; + List pendingFiles = Lists.newArrayList(); + for (DataFile dataFile : dataFiles) { + // Only stat not reach target size data files which should be considered for rewriting + // and keep rewrite file group size as multiple of target file size. + rewriteFilesSize += shouldBeRewritten(dataFile) ? dataFile.fileSizeInBytes() : 0; + pendingFiles.add(dataFile); + + // Rolling to a new rewrite file group to prevent file fragmentation. + if (rewriteFilesSize + bin.rewriteFilesSize() >= targetFileSize || + pendingFiles.size() + bin.totalFilesCount() >= maxGroupFiles) { + // Write all delete files to manifest to ensure delete files can be applied to previous data files. + DeltaManifests deltaManifests = FlinkManifestUtil.writeExistingFiles( + sequenceNumber, snapshotId, pendingFiles, deleteFiles, + () -> manifestOutputFileFactory.createTmp(), table.spec() + ); + bin.add(sequenceNumber, snapshotId, rewriteFilesSize, pendingFiles.size(), deltaManifests); + + bin = new RewriteFileGroup(partition); + bins.add(bin); + rewriteFilesSize = 0; + pendingFiles.clear(); + } + } + + // Append all delete files or remain files to current rewrite file group. + if ((dataFiles.isEmpty() && !deleteFiles.isEmpty()) || !pendingFiles.isEmpty()) { + DeltaManifests deltaManifests = FlinkManifestUtil.writeExistingFiles( + sequenceNumber, snapshotId, pendingFiles, deleteFiles, + () -> manifestOutputFileFactory.createTmp(), table.spec() + ); + bin.add(sequenceNumber, snapshotId, rewriteFilesSize, pendingFiles.size(), deltaManifests); + } + } + + private List pickFrom(Deque bins) { + List picked = Lists.newArrayList(); + while (!bins.isEmpty() && canPick(bins.peek())) { + picked.add(bins.poll()); + } + return picked; + } + + private boolean canPick(RewriteFileGroup bin) { + return !bin.isEmpty() && (bin.rewriteFilesSize() >= targetFileSize || bin.totalFilesCount() >= maxGroupFiles || + getCommitNumAfter(table, bin.latestSnapshotId(), flinkJobId) > maxWaitingCommits); + } + + private Iterable planTasks(RewriteFileGroup fileGroup) { + CloseableIterable scanTasks = table.newScan() + .useManifests(fileGroup.manifestFiles()) + .caseSensitive(caseSensitive) + .ignoreResiduals() + .planFiles(); + + CloseableIterable filtered = CloseableIterable.withNoopClose( + FluentIterable.from(scanTasks).filter(scanTask -> shouldBeRewritten(scanTask.file()))); + + CloseableIterable splitFiles = TableScanUtil.splitFiles(filtered, splitSize(totalSize(filtered))); + return Iterables.filter( + TableScanUtil.planTasks(splitFiles, writeMaxFileSize, 1, 0), + task -> task.files().size() >= minGroupFiles || totalSize(task.files()) >= targetFileSize + ); + } + + private long totalSize(Iterable tasks) { + return FluentIterable.from(tasks).stream().mapToLong(FileScanTask::length).sum(); + } + + private long splitSize(long totalSizeInBytes) { + if (totalSizeInBytes < targetFileSize) { + return targetFileSize; + } + + if (LongMath.mod(totalSizeInBytes, targetFileSize) > minFileSize) { + return LongMath.divide(totalSizeInBytes, targetFileSize, RoundingMode.CEILING); + } + + return totalSizeInBytes / LongMath.divide(totalSizeInBytes, targetFileSize, RoundingMode.FLOOR); + } + + private void validateOptions() { + Preconditions.checkArgument(minFileSize >= 0, + "Cannot set %s to a negative number, %d < 0", + FlinkSinkOptions.STREAMING_REWRITE_MIN_FILE_SIZE_BYTES, minFileSize); + + Preconditions.checkArgument(maxFileSize > minFileSize, + "Cannot set %s greater than or equal to %s, %d >= %d", + FlinkSinkOptions.STREAMING_REWRITE_MIN_FILE_SIZE_BYTES, + FlinkSinkOptions.STREAMING_REWRITE_MAX_FILE_SIZE_BYTES, minFileSize, maxFileSize); + + Preconditions.checkArgument(targetFileSize > minFileSize, + "Cannot set %s greater than or equal to %s, %d >= %d", + FlinkSinkOptions.STREAMING_REWRITE_MIN_FILE_SIZE_BYTES, + FlinkSinkOptions.STREAMING_REWRITE_TARGET_FILE_SIZE_BYTES, minFileSize, targetFileSize); + + Preconditions.checkArgument(targetFileSize < maxFileSize, + "Cannot set %s is greater than or equal to %s, %d >= %d", + FlinkSinkOptions.STREAMING_REWRITE_MAX_FILE_SIZE_BYTES, + FlinkSinkOptions.STREAMING_REWRITE_TARGET_FILE_SIZE_BYTES, maxFileSize, targetFileSize); + + Preconditions.checkArgument(minGroupFiles > 0, + "Cannot set %s to a negative number, %d < 0", + FlinkSinkOptions.STREAMING_REWRITE_MIN_GROUP_FILES, minGroupFiles); + + Preconditions.checkArgument(maxGroupFiles > minGroupFiles, + "Cannot set %s is greater than or equal to %s, %d >= %d", + FlinkSinkOptions.STREAMING_REWRITE_MIN_GROUP_FILES, + FlinkSinkOptions.STREAMING_REWRITE_MAX_GROUP_FILES, minGroupFiles, maxGroupFiles); + + Preconditions.checkArgument(maxWaitingCommits > 0, + "Cannot set %s to a negative number, %d < 0", + FlinkSinkOptions.STREAMING_REWRITE_MAX_WAITING_COMMITS, maxWaitingCommits); + } + } +} diff --git a/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamRewriter.java b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamRewriter.java new file mode 100644 index 000000000000..75d260de8621 --- /dev/null +++ b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamRewriter.java @@ -0,0 +1,149 @@ +/* + * 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.flink.sink; + +import java.io.IOException; +import java.util.List; +import java.util.Locale; +import java.util.stream.Collectors; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.SerializableTable; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.source.DataIterator; +import org.apache.iceberg.flink.source.RowDataFileScanTaskReader; +import org.apache.iceberg.io.TaskWriter; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.util.PropertyUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +class IcebergStreamRewriter extends AbstractStreamOperator + implements OneInputStreamOperator { + + private static final long serialVersionUID = 1L; + + private static final Logger LOG = LoggerFactory.getLogger(IcebergStreamRewriter.class); + + private final TableLoader tableLoader; + + private transient Table table; + private transient RowDataFileScanTaskReader rowDataReader; + private transient TaskWriterFactory taskWriterFactory; + + IcebergStreamRewriter(TableLoader tableLoader) { + this.tableLoader = tableLoader; + } + + @Override + public void open() throws Exception { + super.open(); + + // Open the table loader and load the table. + this.tableLoader.open(); + this.table = tableLoader.loadTable(); + + // init dependence + String nameMapping = PropertyUtil.propertyAsString(table.properties(), + TableProperties.DEFAULT_NAME_MAPPING, null); + boolean caseSensitive = PropertyUtil.propertyAsBoolean(table.properties(), + FlinkSinkOptions.STREAMING_REWRITE_CASE_SENSITIVE, FlinkSinkOptions.STREAMING_REWRITE_CASE_SENSITIVE_DEFAULT); + this.rowDataReader = new RowDataFileScanTaskReader(table.schema(), table.schema(), nameMapping, caseSensitive); + + int subTaskId = getRuntimeContext().getIndexOfThisSubtask(); + int attemptId = getRuntimeContext().getAttemptNumber(); + String formatString = PropertyUtil.propertyAsString(table.properties(), + TableProperties.DEFAULT_FILE_FORMAT, TableProperties.DEFAULT_FILE_FORMAT_DEFAULT); + FileFormat format = FileFormat.valueOf(formatString.toUpperCase(Locale.ENGLISH)); + RowType flinkSchema = FlinkSchemaUtil.convert(table.schema()); + this.taskWriterFactory = new RowDataTaskWriterFactory( + SerializableTable.copyOf(table), flinkSchema, Long.MAX_VALUE, format, null, false); + taskWriterFactory.initialize(subTaskId, attemptId); + } + + @Override + public void processElement(StreamRecord record) throws Exception { + RewriteTask rewriteTask = record.getValue(); + + LOG.info("Rewriting task {}.", rewriteTask); + long start = System.currentTimeMillis(); + RewriteResult rewriteResult = rewrite(rewriteTask.snapshotId(), rewriteTask.partition(), rewriteTask.task()); + LOG.info("Rewritten task {} in {} ms.", rewriteTask, System.currentTimeMillis() - start); + + emit(rewriteResult); + } + + private RewriteResult rewrite(long snapshotId, StructLike partition, CombinedScanTask task) throws IOException { + TaskWriter writer = taskWriterFactory.create(); + try (DataIterator iterator = new DataIterator<>(rowDataReader, task, table.io(), table.encryption())) { + while (iterator.hasNext()) { + RowData rowData = iterator.next(); + writer.write(rowData); + } + } catch (Throwable originalThrowable) { + try { + writer.abort(); + } catch (Throwable inner) { + if (originalThrowable != inner) { + originalThrowable.addSuppressed(inner); + LOG.warn("Suppressing exception in catch: {}", inner.getMessage(), inner); + } + } + + if (originalThrowable instanceof Exception) { + throw originalThrowable; + } else { + throw new RuntimeException(originalThrowable); + } + } + + List addedDataFiles = Lists.newArrayList(writer.dataFiles()); + List currentDataFiles = task.files().stream().map(FileScanTask::file) .collect(Collectors.toList()); + + return RewriteResult.builder(snapshotId, table.spec().partitionType()) + .partition(partition) + .addAddedDataFiles(addedDataFiles) + .addRewrittenDataFiles(currentDataFiles) + .build(); + } + + private void emit(RewriteResult result) { + output.collect(new StreamRecord<>(result)); + } + + @Override + public void close() throws Exception { + super.close(); + if (tableLoader != null) { + tableLoader.close(); + } + } +} diff --git a/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java index b7d575bb446b..9f3f8b2e08ce 100644 --- a/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java +++ b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java @@ -20,6 +20,7 @@ package org.apache.iceberg.flink.sink; import java.util.Map; +import java.util.UUID; import java.util.concurrent.atomic.AtomicInteger; import org.apache.iceberg.FileFormat; import org.apache.iceberg.TableOperations; @@ -51,25 +52,33 @@ class ManifestOutputFileFactory { this.attemptNumber = attemptNumber; } - private String generatePath(long checkpointId) { - return FileFormat.AVRO.addExtension(String.format("%s-%s-%05d-%d-%d-%05d", flinkJobId, operatorUniqueId, - subTaskId, attemptNumber, checkpointId, fileCount.incrementAndGet())); + private String generatePath(String version) { + return FileFormat.AVRO.addExtension(String.format("%s-%s-%05d-%d-%s-%05d", flinkJobId, operatorUniqueId, + subTaskId, attemptNumber, version, fileCount.incrementAndGet())); } - OutputFile create(long checkpointId) { + private OutputFile create(String filePath) { String flinkManifestDir = props.get(FLINK_MANIFEST_LOCATION); String newManifestFullPath; if (Strings.isNullOrEmpty(flinkManifestDir)) { // User don't specify any flink manifest directory, so just use the default metadata path. - newManifestFullPath = ops.metadataFileLocation(generatePath(checkpointId)); + newManifestFullPath = ops.metadataFileLocation(filePath); } else { - newManifestFullPath = String.format("%s/%s", stripTrailingSlash(flinkManifestDir), generatePath(checkpointId)); + newManifestFullPath = String.format("%s/%s", stripTrailingSlash(flinkManifestDir), filePath); } return io.newOutputFile(newManifestFullPath); } + OutputFile create(long checkpointId) { + return create(generatePath(String.valueOf(checkpointId))); + } + + OutputFile createTmp() { + return create(generatePath(UUID.randomUUID().toString())); + } + private static String stripTrailingSlash(String path) { String result = path; while (result.endsWith("/")) { diff --git a/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/RewriteFileGroup.java b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/RewriteFileGroup.java new file mode 100644 index 000000000000..9ee511d6b39e --- /dev/null +++ b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/RewriteFileGroup.java @@ -0,0 +1,198 @@ +/* + * 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.flink.sink; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.util.List; +import org.apache.flink.core.io.SimpleVersionedSerialization; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; + +class RewriteFileGroup { + + private long latestSequenceNumber; + private long latestSnapshotId; + private long rewriteFilesSize; + private int totalFilesCount; + private final StructLike partition; + private final List manifestsList; + + RewriteFileGroup(StructLike partition) { + this.latestSequenceNumber = 0; + this.latestSnapshotId = 0; + this.rewriteFilesSize = 0; + this.totalFilesCount = 0; + this.partition = partition; + this.manifestsList = Lists.newArrayList(); + } + + private RewriteFileGroup(long latestSequenceNumber, long latestSnapshotId, long rewriteFilesSize, int totalFilesCount, + StructLike partition, List manifestsList) { + this.latestSequenceNumber = latestSequenceNumber; + this.latestSnapshotId = latestSnapshotId; + this.rewriteFilesSize = rewriteFilesSize; + this.totalFilesCount = totalFilesCount; + this.partition = partition; + this.manifestsList = manifestsList; + } + + long latestSequenceNumber() { + return latestSequenceNumber; + } + + long latestSnapshotId() { + return latestSnapshotId; + } + + StructLike partition() { + return partition; + } + + long rewriteFilesSize() { + return rewriteFilesSize; + } + + int totalFilesCount() { + return totalFilesCount; + } + + List manifestsList() { + return manifestsList; + } + + Iterable manifestFiles() { + return Iterables.concat(Lists.transform(manifestsList, DeltaManifests::manifests)); + } + + boolean isEmpty() { + return manifestsList.isEmpty(); + } + + void add(long sequenceNumber, long snapshotId, long deltaFilesSize, int deltaFilesCount, DeltaManifests manifests) + throws IOException { + if (manifests == null || manifests.manifests().isEmpty()) { + return; + } + + // v1 table sequence number is always 0. + if (sequenceNumber >= latestSequenceNumber) { + this.latestSequenceNumber = sequenceNumber; + this.latestSnapshotId = snapshotId; + } + + this.rewriteFilesSize += deltaFilesSize; + this.totalFilesCount += deltaFilesCount; + this.manifestsList.add(manifests); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("latestSequenceNumber", latestSequenceNumber) + .add("latestSnapshotId", latestSnapshotId) + .add("partition", partition) + .add("rewriteFilesSize", rewriteFilesSize) + .add("totalFilesCount", totalFilesCount) + .toString(); + } + + static class Serializer implements SimpleVersionedSerializer { + + static final Serializer INSTANCE = new Serializer(); + + @Override + public int getVersion() { + return 1; + } + + @Override + public byte[] serialize(RewriteFileGroup rewriteFileGroup) throws IOException { + Preconditions.checkNotNull(rewriteFileGroup, "RewriteFileGroup to be serialized should not be null"); + + ByteArrayOutputStream binaryOut = new ByteArrayOutputStream(); + ObjectOutputStream out = new ObjectOutputStream(binaryOut); + + out.writeLong(rewriteFileGroup.latestSequenceNumber()); + out.writeLong(rewriteFileGroup.latestSnapshotId()); + out.writeLong(rewriteFileGroup.rewriteFilesSize()); + out.writeInt(rewriteFileGroup.totalFilesCount()); + out.writeObject(rewriteFileGroup.partition()); + + int size = rewriteFileGroup.manifestsList().size(); + out.writeInt(size); + for (DeltaManifests manifests : rewriteFileGroup.manifestsList()) { + byte[] data = SimpleVersionedSerialization.writeVersionAndSerialize( + DeltaManifestsSerializer.INSTANCE, manifests); + out.writeInt(data.length); + out.write(data); + } + out.flush(); + return binaryOut.toByteArray(); + } + + @Override + public RewriteFileGroup deserialize(int version, byte[] serialized) throws IOException { + if (version == 1) { + return deserializeV1(serialized); + } else { + throw new RuntimeException("Unknown serialize version: " + version); + } + } + + private RewriteFileGroup deserializeV1(byte[] serialized) throws IOException { + ByteArrayInputStream binaryIn = new ByteArrayInputStream(serialized); + ObjectInputStream in = new ObjectInputStream(binaryIn); + + long latestSequenceNumber = in.readLong(); + long latestSnapshotId = in.readLong(); + long rewriteFilesSize = in.readLong(); + int totalFilesCount = in.readInt(); + StructLike partition; + try { + partition = (StructLike) in.readObject(); + } catch (ClassNotFoundException e) { + throw new IOException("Fail to read partition bytes", e); + } + + int size = in.readInt(); + List manifestsList = Lists.newArrayListWithCapacity(size); + for (int i = 0; i < size; i++) { + int length = in.readInt(); + byte[] data = new byte[length]; + in.readFully(data); + DeltaManifests deltaManifests = SimpleVersionedSerialization.readVersionAndDeSerialize( + DeltaManifestsSerializer.INSTANCE, data); + manifestsList.add(deltaManifests); + } + + return new RewriteFileGroup(latestSequenceNumber, latestSnapshotId, rewriteFilesSize, totalFilesCount, partition, + manifestsList); + } + } +} diff --git a/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/RewriteResult.java b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/RewriteResult.java new file mode 100644 index 000000000000..60ec1f9b684a --- /dev/null +++ b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/RewriteResult.java @@ -0,0 +1,116 @@ +/* + * 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.flink.sink; + +import java.util.Collection; +import java.util.Set; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.StructLikeSet; + +class RewriteResult { + + private final long startingSnapshotId; + private final Collection partitions; + private final Set addedDataFiles; + private final Set rewrittenDataFiles; + + private RewriteResult(long startingSnapshotId, + Collection partitions, + Set addedDataFiles, + Set rewrittenDataFiles) { + this.startingSnapshotId = startingSnapshotId; + this.partitions = partitions; + this.addedDataFiles = addedDataFiles; + this.rewrittenDataFiles = rewrittenDataFiles; + } + + long startingSnapshotId() { + return startingSnapshotId; + } + + Collection partitions() { + return partitions; + } + + Set addedDataFiles() { + return addedDataFiles; + } + + Set rewrittenDataFiles() { + return rewrittenDataFiles; + } + + static Builder builder(long startingSnapshotId, Types.StructType partitionType) { + return new Builder(startingSnapshotId, partitionType); + } + + static class Builder { + private final long startingSnapshotId; + private final StructLikeSet partitions; + private final Set addedFiles; + private final Set rewrittenDataFiles; + + private Builder(long startingSnapshotId, Types.StructType partitionType) { + this.startingSnapshotId = startingSnapshotId; + this.partitions = StructLikeSet.create(partitionType); + this.addedFiles = Sets.newHashSet(); + this.rewrittenDataFiles = Sets.newHashSet(); + } + + Builder partition(StructLike newPartition) { + this.partitions.add(newPartition); + return this; + } + + Builder partitions(Collection newPartitions) { + this.partitions.addAll(newPartitions); + return this; + } + + Builder addAddedDataFiles(Iterable dataFiles) { + Iterables.addAll(addedFiles, dataFiles); + return this; + } + + Builder addRewrittenDataFiles(Iterable dataFiles) { + Iterables.addAll(rewrittenDataFiles, dataFiles); + return this; + } + + RewriteResult build() { + return new RewriteResult(startingSnapshotId, Sets.newHashSet(partitions), addedFiles, rewrittenDataFiles); + } + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("partitions", partitions) + .add("startingSnapshotId", startingSnapshotId) + .add("numRewrittenFiles", Iterables.size(rewrittenDataFiles)) + .add("numAddedFiles", Iterables.size(addedDataFiles)) + .toString(); + } +} diff --git a/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/RewriteTask.java b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/RewriteTask.java new file mode 100644 index 000000000000..ee625a17aa11 --- /dev/null +++ b/flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/sink/RewriteTask.java @@ -0,0 +1,65 @@ +/* + * 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.flink.sink; + +import java.io.Serializable; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; + +class RewriteTask implements Serializable { + + private final long snapshotId; + private final StructLike partition; + private final CombinedScanTask task; + private final long totalFilesCount; + private final long totalSizeInBytes; + + RewriteTask(long snapshotId, StructLike partition, CombinedScanTask task) { + this.snapshotId = snapshotId; + this.partition = partition; + this.task = task; + this.totalFilesCount = task.files().size(); + this.totalSizeInBytes = task.files().stream().mapToLong(FileScanTask::length).sum(); + } + + long snapshotId() { + return snapshotId; + } + + StructLike partition() { + return partition; + } + + CombinedScanTask task() { + return task; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("snapshotId", snapshotId) + .add("partition", partition) + .add("totalFilesCount", totalFilesCount) + .add("totalSizeInBytes", totalSizeInBytes) + .toString(); + } +} diff --git a/flink/v1.14/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java b/flink/v1.14/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java index 4cfccfbadccb..fb761603f3e7 100644 --- a/flink/v1.14/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java +++ b/flink/v1.14/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java @@ -34,13 +34,14 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.iceberg.DataFile; -import org.apache.iceberg.DataFiles; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.FileFormat; import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.PartitionKey; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; +import org.apache.iceberg.StructLike; import org.apache.iceberg.Table; import org.apache.iceberg.TableScan; import org.apache.iceberg.data.GenericRecord; @@ -49,15 +50,12 @@ import org.apache.iceberg.deletes.EqualityDeleteWriter; import org.apache.iceberg.deletes.PositionDeleteWriter; import org.apache.iceberg.encryption.EncryptedOutputFile; -import org.apache.iceberg.flink.sink.FlinkAppenderFactory; -import org.apache.iceberg.hadoop.HadoopInputFile; import org.apache.iceberg.hadoop.HadoopTables; import org.apache.iceberg.io.CloseableIterable; -import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.io.DataWriter; import org.apache.iceberg.io.FileAppenderFactory; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; 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.Maps; @@ -105,6 +103,13 @@ public static Record createRecord(Integer id, String data) { return record; } + public static StructLike createPartition(String data) { + PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("data").build(); + PartitionKey wrapper = new PartitionKey(spec, SCHEMA); + wrapper.partition(createRecord(null, data)); + return wrapper; + } + public static RowData createRowData(Integer id, String data) { return GenericRowData.of(id, StringData.fromString(data)); } @@ -125,49 +130,46 @@ public static RowData createUpdateAfter(Integer id, String data) { return GenericRowData.ofKind(RowKind.UPDATE_AFTER, id, StringData.fromString(data)); } - public static DataFile writeFile(Schema schema, PartitionSpec spec, Configuration conf, - String location, String filename, List rows) + public static DataFile writeDataFile(Table table, FileFormat format, String filename, + FileAppenderFactory appenderFactory, + StructLike partition, List rows) throws IOException { - Path path = new Path(location, filename); - FileFormat fileFormat = FileFormat.fromFileName(filename); - Preconditions.checkNotNull(fileFormat, "Cannot determine format for file: %s", filename); - - RowType flinkSchema = FlinkSchemaUtil.convert(schema); - FileAppenderFactory appenderFactory = - new FlinkAppenderFactory(schema, flinkSchema, ImmutableMap.of(), spec); + String location = partition == null ? table.locationProvider().newDataLocation(filename) : + table.locationProvider().newDataLocation(table.spec(), partition, filename); + EncryptedOutputFile outputFile = table.encryption().encrypt(fromPath(new Path(location), new Configuration())); - FileAppender appender = appenderFactory.newAppender(fromPath(path, conf), fileFormat); - try (FileAppender closeableAppender = appender) { - closeableAppender.addAll(rows); + DataWriter dataWriter = appenderFactory.newDataWriter(outputFile, format, partition); + try (DataWriter writer = dataWriter) { + writer.write(rows); } - return DataFiles.builder(spec) - .withInputFile(HadoopInputFile.fromPath(path, conf)) - .withMetrics(appender.metrics()) - .build(); + return dataWriter.toDataFile(); } - public static DeleteFile writeEqDeleteFile(Table table, FileFormat format, String tablePath, String filename, + public static DeleteFile writeEqDeleteFile(Table table, FileFormat format, String filename, FileAppenderFactory appenderFactory, - List deletes) throws IOException { - EncryptedOutputFile outputFile = - table.encryption().encrypt(fromPath(new Path(tablePath, filename), new Configuration())); + StructLike partition, List deletes) + throws IOException { + String location = partition == null ? table.locationProvider().newDataLocation(filename) : + table.locationProvider().newDataLocation(table.spec(), partition, filename); + EncryptedOutputFile outputFile = table.encryption().encrypt(fromPath(new Path(location), new Configuration())); - EqualityDeleteWriter eqWriter = appenderFactory.newEqDeleteWriter(outputFile, format, null); + EqualityDeleteWriter eqWriter = appenderFactory.newEqDeleteWriter(outputFile, format, partition); try (EqualityDeleteWriter writer = eqWriter) { writer.deleteAll(deletes); } return eqWriter.toDeleteFile(); } - public static DeleteFile writePosDeleteFile(Table table, FileFormat format, String tablePath, - String filename, + public static DeleteFile writePosDeleteFile(Table table, FileFormat format, String filename, FileAppenderFactory appenderFactory, - List> positions) throws IOException { - EncryptedOutputFile outputFile = - table.encryption().encrypt(fromPath(new Path(tablePath, filename), new Configuration())); + StructLike partition, List> positions) + throws IOException { + String location = partition == null ? table.locationProvider().newDataLocation(filename) : + table.locationProvider().newDataLocation(table.spec(), partition, filename); + EncryptedOutputFile outputFile = table.encryption().encrypt(fromPath(new Path(location), new Configuration())); - PositionDeleteWriter posWriter = appenderFactory.newPosDeleteWriter(outputFile, format, null); + PositionDeleteWriter posWriter = appenderFactory.newPosDeleteWriter(outputFile, format, partition); try (PositionDeleteWriter writer = posWriter) { for (Pair p : positions) { writer.delete(p.first(), p.second()); diff --git a/flink/v1.14/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java b/flink/v1.14/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java index 4a47656e847d..151ccf778233 100644 --- a/flink/v1.14/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java +++ b/flink/v1.14/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java @@ -230,18 +230,18 @@ public ManifestFile deserialize(int version, byte[] serialized) throws IOExcepti } private DataFile writeDataFile(String filename, List rows) throws IOException { - return SimpleDataUtil.writeFile(table.schema(), table.spec(), CONF, - tablePath, FileFormat.PARQUET.addExtension(filename), rows); + return SimpleDataUtil.writeDataFile(table, FileFormat.PARQUET, filename, appenderFactory, null, rows); } private DeleteFile writeEqDeleteFile(String filename, List deletes) throws IOException { - return SimpleDataUtil.writeEqDeleteFile(table, FileFormat.PARQUET, tablePath, filename, appenderFactory, deletes); + return SimpleDataUtil + .writeEqDeleteFile(table, FileFormat.PARQUET, filename, appenderFactory, null, deletes); } private DeleteFile writePosDeleteFile(String filename, List> positions) throws IOException { return SimpleDataUtil - .writePosDeleteFile(table, FileFormat.PARQUET, tablePath, filename, appenderFactory, positions); + .writePosDeleteFile(table, FileFormat.PARQUET, filename, appenderFactory, null, positions); } private List generateDataFiles(int fileNum) throws IOException { diff --git a/flink/v1.14/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java b/flink/v1.14/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java index 9c23d8a0889f..f8a250594eb7 100644 --- a/flink/v1.14/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java +++ b/flink/v1.14/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java @@ -23,6 +23,7 @@ import java.io.IOException; import java.nio.file.Files; import java.nio.file.Path; +import java.util.Arrays; import java.util.List; import java.util.Locale; import java.util.stream.Collectors; @@ -40,6 +41,7 @@ import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; import org.apache.flink.table.data.RowData; import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.ContentFile; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.FileFormat; @@ -47,6 +49,7 @@ import org.apache.iceberg.ManifestContent; import org.apache.iceberg.ManifestFile; import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.StructLike; import org.apache.iceberg.TableTestBase; import org.apache.iceberg.flink.FlinkSchemaUtil; import org.apache.iceberg.flink.SimpleDataUtil; @@ -74,6 +77,7 @@ public class TestIcebergFilesCommitter extends TableTestBase { private String tablePath; private File flinkManifestFolder; + private FileAppenderFactory appenderFactory; private final FileFormat format; @@ -85,7 +89,7 @@ public static Object[][] parameters() { new Object[] {"parquet", 1}, new Object[] {"parquet", 2}, new Object[] {"orc", 1}, - new Object[] {"orc", 2} + new Object[] {"orc", 2}, }; } @@ -113,6 +117,8 @@ public void setupTable() throws IOException { .set(FLINK_MANIFEST_LOCATION, flinkManifestFolder.getAbsolutePath()) .set(MAX_CONTINUOUS_EMPTY_COMMITS, "1") .commit(); + + appenderFactory = createDeletableAppenderFactory(); } @Test @@ -120,7 +126,7 @@ public void testCommitTxnWithoutDataFiles() throws Exception { long checkpointId = 0; long timestamp = 0; JobID jobId = new JobID(); - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { harness.setup(); harness.open(); @@ -152,7 +158,7 @@ public void testMaxContinuousEmptyCommits() throws Exception { JobID jobId = new JobID(); long checkpointId = 0; long timestamp = 0; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { harness.setup(); harness.open(); @@ -183,7 +189,7 @@ public void testCommitTxn() throws Exception { long timestamp = 0; JobID jobID = new JobID(); - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobID)) { + try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobID)) { harness.setup(); harness.open(); assertSnapshotSize(0); @@ -191,7 +197,7 @@ public void testCommitTxn() throws Exception { List rows = Lists.newArrayListWithExpectedSize(3); for (int i = 1; i <= 3; i++) { RowData rowData = SimpleDataUtil.createRowData(i, "hello" + i); - DataFile dataFile = writeDataFile("data-" + i, ImmutableList.of(rowData)); + DataFile dataFile = writeDataFile("data-" + i, null, ImmutableList.of(rowData)); harness.processElement(of(dataFile), ++timestamp); rows.add(rowData); @@ -218,14 +224,14 @@ public void testOrderedEventsBetweenCheckpoints() throws Exception { long timestamp = 0; JobID jobId = new JobID(); - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { harness.setup(); harness.open(); assertMaxCommittedCheckpointId(jobId, -1L); RowData row1 = SimpleDataUtil.createRowData(1, "hello"); - DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row1)); + DataFile dataFile1 = writeDataFile("data-1", null, ImmutableList.of(row1)); harness.processElement(of(dataFile1), ++timestamp); assertMaxCommittedCheckpointId(jobId, -1L); @@ -236,7 +242,7 @@ public void testOrderedEventsBetweenCheckpoints() throws Exception { assertFlinkManifests(1); RowData row2 = SimpleDataUtil.createRowData(2, "world"); - DataFile dataFile2 = writeDataFile("data-2", ImmutableList.of(row2)); + DataFile dataFile2 = writeDataFile("data-2", null, ImmutableList.of(row2)); harness.processElement(of(dataFile2), ++timestamp); assertMaxCommittedCheckpointId(jobId, -1L); @@ -269,14 +275,14 @@ public void testDisorderedEventsBetweenCheckpoints() throws Exception { long timestamp = 0; JobID jobId = new JobID(); - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { harness.setup(); harness.open(); assertMaxCommittedCheckpointId(jobId, -1L); RowData row1 = SimpleDataUtil.createRowData(1, "hello"); - DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row1)); + DataFile dataFile1 = writeDataFile("data-1", null, ImmutableList.of(row1)); harness.processElement(of(dataFile1), ++timestamp); assertMaxCommittedCheckpointId(jobId, -1L); @@ -287,7 +293,7 @@ public void testDisorderedEventsBetweenCheckpoints() throws Exception { assertFlinkManifests(1); RowData row2 = SimpleDataUtil.createRowData(2, "world"); - DataFile dataFile2 = writeDataFile("data-2", ImmutableList.of(row2)); + DataFile dataFile2 = writeDataFile("data-2", null, ImmutableList.of(row2)); harness.processElement(of(dataFile2), ++timestamp); assertMaxCommittedCheckpointId(jobId, -1L); @@ -318,7 +324,7 @@ public void testRecoveryFromValidSnapshot() throws Exception { OperatorSubtaskState snapshot; JobID jobId = new JobID(); - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { harness.setup(); harness.open(); @@ -327,7 +333,7 @@ public void testRecoveryFromValidSnapshot() throws Exception { RowData row = SimpleDataUtil.createRowData(1, "hello"); expectedRows.add(row); - DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row)); + DataFile dataFile1 = writeDataFile("data-1", null, ImmutableList.of(row)); harness.processElement(of(dataFile1), ++timestamp); snapshot = harness.snapshot(++checkpointId, ++timestamp); @@ -342,7 +348,7 @@ public void testRecoveryFromValidSnapshot() throws Exception { } // Restore from the given snapshot - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { harness.setup(); harness.initializeState(snapshot); harness.open(); @@ -353,7 +359,7 @@ public void testRecoveryFromValidSnapshot() throws Exception { RowData row = SimpleDataUtil.createRowData(2, "world"); expectedRows.add(row); - DataFile dataFile = writeDataFile("data-2", ImmutableList.of(row)); + DataFile dataFile = writeDataFile("data-2", null, ImmutableList.of(row)); harness.processElement(of(dataFile), ++timestamp); harness.snapshot(++checkpointId, ++timestamp); @@ -377,7 +383,7 @@ public void testRecoveryFromSnapshotWithoutCompletedNotification() throws Except OperatorSubtaskState snapshot; List expectedRows = Lists.newArrayList(); JobID jobId = new JobID(); - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { harness.setup(); harness.open(); @@ -386,7 +392,7 @@ public void testRecoveryFromSnapshotWithoutCompletedNotification() throws Except RowData row = SimpleDataUtil.createRowData(1, "hello"); expectedRows.add(row); - DataFile dataFile = writeDataFile("data-1", ImmutableList.of(row)); + DataFile dataFile = writeDataFile("data-1", null, ImmutableList.of(row)); harness.processElement(of(dataFile), ++timestamp); snapshot = harness.snapshot(++checkpointId, ++timestamp); @@ -395,7 +401,7 @@ public void testRecoveryFromSnapshotWithoutCompletedNotification() throws Except assertFlinkManifests(1); } - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { harness.setup(); harness.initializeState(snapshot); harness.open(); @@ -419,7 +425,7 @@ public void testRecoveryFromSnapshotWithoutCompletedNotification() throws Except RowData row = SimpleDataUtil.createRowData(2, "world"); expectedRows.add(row); - DataFile dataFile = writeDataFile("data-2", ImmutableList.of(row)); + DataFile dataFile = writeDataFile("data-2", null, ImmutableList.of(row)); harness.processElement(of(dataFile), ++timestamp); snapshot = harness.snapshot(++checkpointId, ++timestamp); @@ -428,7 +434,7 @@ public void testRecoveryFromSnapshotWithoutCompletedNotification() throws Except // Redeploying flink job from external checkpoint. JobID newJobId = new JobID(); - try (OneInputStreamOperatorTestHarness harness = createStreamSink(newJobId)) { + try (OneInputStreamOperatorTestHarness harness = createStreamSink(newJobId)) { harness.setup(); harness.initializeState(snapshot); harness.open(); @@ -443,7 +449,7 @@ public void testRecoveryFromSnapshotWithoutCompletedNotification() throws Except RowData row = SimpleDataUtil.createRowData(3, "foo"); expectedRows.add(row); - DataFile dataFile = writeDataFile("data-3", ImmutableList.of(row)); + DataFile dataFile = writeDataFile("data-3", null, ImmutableList.of(row)); harness.processElement(of(dataFile), ++timestamp); harness.snapshot(++checkpointId, ++timestamp); @@ -466,7 +472,7 @@ public void testStartAnotherJobToWriteSameTable() throws Exception { List tableRows = Lists.newArrayList(); JobID oldJobId = new JobID(); - try (OneInputStreamOperatorTestHarness harness = createStreamSink(oldJobId)) { + try (OneInputStreamOperatorTestHarness harness = createStreamSink(oldJobId)) { harness.setup(); harness.open(); @@ -477,7 +483,7 @@ public void testStartAnotherJobToWriteSameTable() throws Exception { rows.add(SimpleDataUtil.createRowData(i, "hello" + i)); tableRows.addAll(rows); - DataFile dataFile = writeDataFile(String.format("data-%d", i), rows); + DataFile dataFile = writeDataFile(String.format("data-%d", i), null, rows); harness.processElement(of(dataFile), ++timestamp); harness.snapshot(++checkpointId, ++timestamp); assertFlinkManifests(1); @@ -495,7 +501,7 @@ public void testStartAnotherJobToWriteSameTable() throws Exception { checkpointId = 0; timestamp = 0; JobID newJobId = new JobID(); - try (OneInputStreamOperatorTestHarness harness = createStreamSink(newJobId)) { + try (OneInputStreamOperatorTestHarness harness = createStreamSink(newJobId)) { harness.setup(); harness.open(); @@ -506,7 +512,7 @@ public void testStartAnotherJobToWriteSameTable() throws Exception { rows.add(SimpleDataUtil.createRowData(2, "world")); tableRows.addAll(rows); - DataFile dataFile = writeDataFile("data-new-1", rows); + DataFile dataFile = writeDataFile("data-new-1", null, rows); harness.processElement(of(dataFile), ++timestamp); harness.snapshot(++checkpointId, ++timestamp); assertFlinkManifests(1); @@ -529,7 +535,7 @@ public void testMultipleJobsWriteSameTable() throws Exception { int jobIndex = i % 3; int checkpointId = i / 3; JobID jobId = jobs[jobIndex]; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { harness.setup(); harness.open(); @@ -539,7 +545,7 @@ public void testMultipleJobsWriteSameTable() throws Exception { List rows = Lists.newArrayList(SimpleDataUtil.createRowData(i, "word-" + i)); tableRows.addAll(rows); - DataFile dataFile = writeDataFile(String.format("data-%d", i), rows); + DataFile dataFile = writeDataFile(String.format("data-%d", i), null, rows); harness.processElement(of(dataFile), ++timestamp); harness.snapshot(checkpointId + 1, ++timestamp); assertFlinkManifests(1); @@ -556,7 +562,7 @@ public void testMultipleJobsWriteSameTable() throws Exception { @Test public void testBoundedStream() throws Exception { JobID jobId = new JobID(); - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { harness.setup(); harness.open(); @@ -566,7 +572,7 @@ public void testBoundedStream() throws Exception { List tableRows = Lists.newArrayList(SimpleDataUtil.createRowData(1, "word-1")); - DataFile dataFile = writeDataFile("data-1", tableRows); + DataFile dataFile = writeDataFile("data-1", null, tableRows); harness.processElement(of(dataFile), 1); ((BoundedOneInput) harness.getOneInputOperator()).endInput(); @@ -583,14 +589,14 @@ public void testFlinkManifests() throws Exception { final long checkpoint = 10; JobID jobId = new JobID(); - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { harness.setup(); harness.open(); assertMaxCommittedCheckpointId(jobId, -1L); RowData row1 = SimpleDataUtil.createRowData(1, "hello"); - DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row1)); + DataFile dataFile1 = writeDataFile("data-1", null, ImmutableList.of(row1)); harness.processElement(of(dataFile1), ++timestamp); assertMaxCommittedCheckpointId(jobId, -1L); @@ -625,16 +631,15 @@ public void testDeleteFiles() throws Exception { long checkpoint = 10; JobID jobId = new JobID(); - FileAppenderFactory appenderFactory = createDeletableAppenderFactory(); - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { harness.setup(); harness.open(); assertMaxCommittedCheckpointId(jobId, -1L); RowData row1 = SimpleDataUtil.createInsert(1, "aaa"); - DataFile dataFile1 = writeDataFile("data-file-1", ImmutableList.of(row1)); + DataFile dataFile1 = writeDataFile("data-file-1", null, ImmutableList.of(row1)); harness.processElement(of(dataFile1), ++timestamp); assertMaxCommittedCheckpointId(jobId, -1L); @@ -660,10 +665,10 @@ public void testDeleteFiles() throws Exception { // 4. process both data files and delete files. RowData row2 = SimpleDataUtil.createInsert(2, "bbb"); - DataFile dataFile2 = writeDataFile("data-file-2", ImmutableList.of(row2)); + DataFile dataFile2 = writeDataFile("data-file-2", null, ImmutableList.of(row2)); RowData delete1 = SimpleDataUtil.createDelete(1, "aaa"); - DeleteFile deleteFile1 = writeEqDeleteFile(appenderFactory, "delete-file-1", ImmutableList.of(delete1)); + DeleteFile deleteFile1 = writeEqDeleteFile("delete-file-1", null, ImmutableList.of(delete1)); harness.processElement(WriteResult.builder() .addDataFiles(dataFile2) .addDeleteFiles(deleteFile1) @@ -691,9 +696,8 @@ public void testCommitTwoCheckpointsInSingleTxn() throws Exception { long checkpoint = 10; JobID jobId = new JobID(); - FileAppenderFactory appenderFactory = createDeletableAppenderFactory(); - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { harness.setup(); harness.open(); @@ -702,8 +706,8 @@ public void testCommitTwoCheckpointsInSingleTxn() throws Exception { RowData insert1 = SimpleDataUtil.createInsert(1, "aaa"); RowData insert2 = SimpleDataUtil.createInsert(2, "bbb"); RowData delete3 = SimpleDataUtil.createDelete(3, "ccc"); - DataFile dataFile1 = writeDataFile("data-file-1", ImmutableList.of(insert1, insert2)); - DeleteFile deleteFile1 = writeEqDeleteFile(appenderFactory, "delete-file-1", ImmutableList.of(delete3)); + DataFile dataFile1 = writeDataFile("data-file-1", null, ImmutableList.of(insert1, insert2)); + DeleteFile deleteFile1 = writeEqDeleteFile("delete-file-1", null, ImmutableList.of(delete3)); harness.processElement(WriteResult.builder() .addDataFiles(dataFile1) .addDeleteFiles(deleteFile1) @@ -715,8 +719,8 @@ public void testCommitTwoCheckpointsInSingleTxn() throws Exception { RowData insert4 = SimpleDataUtil.createInsert(4, "ddd"); RowData delete2 = SimpleDataUtil.createDelete(2, "bbb"); - DataFile dataFile2 = writeDataFile("data-file-2", ImmutableList.of(insert4)); - DeleteFile deleteFile2 = writeEqDeleteFile(appenderFactory, "delete-file-2", ImmutableList.of(delete2)); + DataFile dataFile2 = writeDataFile("data-file-2", null, ImmutableList.of(insert4)); + DeleteFile deleteFile2 = writeEqDeleteFile("delete-file-2", null, ImmutableList.of(delete2)); harness.processElement(WriteResult.builder() .addDataFiles(dataFile2) .addDeleteFiles(deleteFile2) @@ -735,16 +739,91 @@ public void testCommitTwoCheckpointsInSingleTxn() throws Exception { } } - private DeleteFile writeEqDeleteFile(FileAppenderFactory appenderFactory, - String filename, List deletes) throws IOException { - return SimpleDataUtil.writeEqDeleteFile(table, FileFormat.PARQUET, tablePath, filename, appenderFactory, deletes); + @Test + public void testEmitCommitResult() throws Exception { + long timestamp = 0; + long checkpoint = 10; + + JobID jobId = new JobID(); + try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + harness.setup(); + harness.open(); + + assertMaxCommittedCheckpointId(jobId, -1L); + + // construct write results + DataFile dataFile = writeDataFile("data-file", null, ImmutableList.of( + SimpleDataUtil.createRowData(1, "aaa"), + SimpleDataUtil.createRowData(2, "bbb"), + SimpleDataUtil.createRowData(3, "ccc") + )); + DeleteFile deleteFile = writePosDeleteFile("pos-delete", null, ImmutableList.of( + Pair.of(dataFile.path(), 0L), + Pair.of(dataFile.path(), 2L) + )); + + WriteResult writeResult = formatVersion < 2 ? WriteResult.builder().addDataFiles(dataFile).build() : + WriteResult.builder() + .addDataFiles(dataFile) + .addDeleteFiles(deleteFile) + .addReferencedDataFiles(dataFile.path()) + .build(); + + // construct expect result + List expected = formatVersion < 2 ? ImmutableList.of( + SimpleDataUtil.createRowData(1, "aaa"), + SimpleDataUtil.createRowData(2, "bbb"), + SimpleDataUtil.createRowData(3, "ccc") + ) : ImmutableList.of(SimpleDataUtil.createRowData(2, "bbb")); + + // 0. process all write result + harness.processElement(writeResult, ++timestamp); + assertMaxCommittedCheckpointId(jobId, -1L); + + // 1. snapshotState for checkpoint#1 + harness.snapshot(checkpoint, ++timestamp); + assertFlinkManifests(formatVersion < 2 ? 1 : 2); + + // 2. notifyCheckpointComplete for checkpoint + harness.notifyOfCompletedCheckpoint(checkpoint); + SimpleDataUtil.assertTableRows(table, expected); + assertMaxCommittedCheckpointId(jobId, checkpoint); + assertFlinkManifests(0); + + // 3. emit commit results + List commitResults = harness.extractOutputValues(); + Assert.assertEquals(1, commitResults.size()); + + CommitResult commitResult = commitResults.get(0); + Assert.assertEquals(table.currentSnapshot().sequenceNumber(), commitResult.sequenceNumber()); + Assert.assertEquals(table.currentSnapshot().snapshotId(), commitResult.snapshotId()); + Assert.assertEquals( + Arrays.stream(writeResult.dataFiles()).map(ContentFile::path).collect(Collectors.toSet()), + Arrays.stream(commitResult.writeResult().dataFiles()).map(ContentFile::path).collect(Collectors.toSet()) + ); + Assert.assertEquals( + Arrays.stream(writeResult.deleteFiles()).map(ContentFile::path).collect(Collectors.toSet()), + Arrays.stream(commitResult.writeResult().deleteFiles()).map(ContentFile::path).collect(Collectors.toSet()) + ); + Assert.assertEquals( + Arrays.stream(writeResult.referencedDataFiles()).collect(Collectors.toSet()), + Arrays.stream(commitResult.writeResult().referencedDataFiles()).collect(Collectors.toSet()) + ); + } } - private DeleteFile writePosDeleteFile(FileAppenderFactory appenderFactory, - String filename, - List> positions) throws IOException { - return SimpleDataUtil.writePosDeleteFile(table, FileFormat.PARQUET, tablePath, filename, appenderFactory, - positions); + private DataFile writeDataFile(String filename, StructLike partition, List rows) throws IOException { + return SimpleDataUtil.writeDataFile(table, format, filename, appenderFactory, partition, rows); + } + + private DeleteFile writeEqDeleteFile(String filename, StructLike partition, List deletes) + throws IOException { + return SimpleDataUtil.writeEqDeleteFile(table, format, filename, appenderFactory, partition, deletes); + } + + private DeleteFile writePosDeleteFile(String filename, StructLike partition, List> positions) + throws IOException { + return SimpleDataUtil.writePosDeleteFile(table, format, filename, appenderFactory, partition, positions); } private FileAppenderFactory createDeletableAppenderFactory() { @@ -771,10 +850,6 @@ private List assertFlinkManifests(int expectedCount) throws IOException { return manifests; } - private DataFile writeDataFile(String filename, List rows) throws IOException { - return SimpleDataUtil.writeFile(table.schema(), table.spec(), CONF, tablePath, format.addExtension(filename), rows); - } - private void assertMaxCommittedCheckpointId(JobID jobID, long expectedId) { table.refresh(); long actualId = IcebergFilesCommitter.getMaxCommittedCheckpointId(table, jobID.toString()); @@ -786,7 +861,7 @@ private void assertSnapshotSize(int expectedSnapshotSize) { Assert.assertEquals(expectedSnapshotSize, Lists.newArrayList(table.snapshots()).size()); } - private OneInputStreamOperatorTestHarness createStreamSink(JobID jobID) + private OneInputStreamOperatorTestHarness createStreamSink(JobID jobID) throws Exception { TestOperatorFactory factory = TestOperatorFactory.of(tablePath); return new OneInputStreamOperatorTestHarness<>(factory, createEnvironment(jobID)); @@ -805,8 +880,8 @@ private static MockEnvironment createEnvironment(JobID jobID) { .build(); } - private static class TestOperatorFactory extends AbstractStreamOperatorFactory - implements OneInputStreamOperatorFactory { + private static class TestOperatorFactory extends AbstractStreamOperatorFactory + implements OneInputStreamOperatorFactory { private final String tablePath; private TestOperatorFactory(String tablePath) { @@ -819,7 +894,8 @@ private static TestOperatorFactory of(String tablePath) { @Override @SuppressWarnings("unchecked") - public > T createStreamOperator(StreamOperatorParameters param) { + public > T createStreamOperator( + StreamOperatorParameters param) { IcebergFilesCommitter committer = new IcebergFilesCommitter(new TestTableLoader(tablePath), false); committer.setup(param.getContainingTask(), param.getStreamConfig(), param.getOutput()); return (T) committer; diff --git a/flink/v1.14/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergRewriteFilesCommitter.java b/flink/v1.14/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergRewriteFilesCommitter.java new file mode 100644 index 000000000000..49e3f4738900 --- /dev/null +++ b/flink/v1.14/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergRewriteFilesCommitter.java @@ -0,0 +1,327 @@ +/* + * 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.flink.sink; + +import java.io.File; +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.Set; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.operators.testutils.MockEnvironment; +import org.apache.flink.runtime.operators.testutils.MockEnvironmentBuilder; +import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider; +import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.FileMetadata; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.RowDelta; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.events.CreateSnapshotEvent; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +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.junit.Assert; +import org.junit.Assume; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +@RunWith(Parameterized.class) +public class TestIcebergRewriteFilesCommitter { + + @Rule + public TemporaryFolder temp = new TemporaryFolder(); + + private Table table; + private String tablePath; + + private final FileFormat format; + private final int formatVersion; + + @Parameterized.Parameters(name = "FileFormat = {0}, FormatVersion={1}") + public static Object[][] parameters() { + return new Object[][] { + new Object[] {"avro", 1}, + new Object[] {"avro", 2}, + new Object[] {"parquet", 1}, + new Object[] {"parquet", 2}, + new Object[] {"orc", 1}, + new Object[] {"orc", 2} + }; + } + + public TestIcebergRewriteFilesCommitter(String format, int formatVersion) { + this.format = FileFormat.valueOf(format.toUpperCase(Locale.ENGLISH)); + this.formatVersion = formatVersion; + } + + @Before + public void setupTable() throws IOException { + File tableDir = temp.newFolder(); + Assert.assertTrue(tableDir.delete()); + + tablePath = tableDir.getAbsolutePath(); + File flinkManifestFolder = temp.newFolder(); + + // Construct the iceberg table. + Map props = ImmutableMap.of( + TableProperties.DEFAULT_FILE_FORMAT, format.name(), + TableProperties.FORMAT_VERSION, String.valueOf(formatVersion), + ManifestOutputFileFactory.FLINK_MANIFEST_LOCATION, flinkManifestFolder.getAbsolutePath() + ); + table = SimpleDataUtil.createTable(tablePath, props, true); + } + + @Test + public void testCommitPreCheckpoint() throws Exception { + long timestamp = 0; + long checkpointId = 0; + JobID jobID = new JobID(); + try (OneInputStreamOperatorTestHarness harness = createStreamOpr(jobID)) { + harness.setup(); + harness.open(); + + StructLike partitionA = SimpleDataUtil.createPartition("aaa"); + StructLike partitionB = SimpleDataUtil.createPartition("bbb"); + + DataFile rewrittenFileA1 = createDataFile("data-old-1", partitionA); + DataFile rewrittenFileA2 = createDataFile("data-old-2", partitionA); + DataFile rewrittenFileB1 = createDataFile("data-old-1", partitionB); + DataFile rewrittenFileB2 = createDataFile("data-old-2", partitionB); + long snapshot1 = commit(rewrittenFileA1, rewrittenFileA2, rewrittenFileB1, rewrittenFileB2); + + DataFile addedFileA1 = createDataFile("data-new-1", partitionA); + RewriteResult rewriteResultA1 = RewriteResult.builder(snapshot1, table.spec().partitionType()) + .partition(partitionA) + .addRewrittenDataFiles(ImmutableList.of(rewrittenFileA1, rewrittenFileA2)) + .addAddedDataFiles(ImmutableList.of(addedFileA1)) + .build(); + harness.processElement(rewriteResultA1, ++timestamp); + assertSnapshots(1); + + DataFile addedFileB1 = createDataFile("data-new-1", partitionB); + RewriteResult rewriteResultB1 = RewriteResult.builder(snapshot1, table.spec().partitionType()) + .partition(partitionB) + .addRewrittenDataFiles(ImmutableList.of(rewrittenFileB1, rewrittenFileB2)) + .addAddedDataFiles(ImmutableList.of(addedFileB1)) + .build(); + harness.processElement(rewriteResultB1, ++timestamp); + assertSnapshots(1); + + DataFile rewrittenFileA3 = createDataFile("data-old-3", partitionA); + long snapshot2 = commit(rewrittenFileA3); + + DataFile addedFileA2 = createDataFile("data-new-2", partitionA); + DataFile addedFileA3 = createDataFile("data-new-3", partitionA); + RewriteResult rewriteResultA2 = RewriteResult.builder(snapshot2, table.spec().partitionType()) + .partition(partitionA) + .addRewrittenDataFiles(ImmutableList.of(rewrittenFileA3)) + .addAddedDataFiles(ImmutableList.of(addedFileA2, addedFileA3)) + .build(); + harness.processElement(rewriteResultA2, ++timestamp); + assertSnapshots(2); + + // rewrite results should be committed in same time if they have same snapshot id. + harness.snapshot(++checkpointId, ++timestamp); + assertSnapshots(4); + + List snapshots = Lists.newArrayList(table.snapshots().iterator()); + Assert.assertEquals("Snapshot id should equal", snapshot1, snapshots.get(0).snapshotId()); + Assert.assertEquals("Snapshot id should equal", snapshot2, snapshots.get(1).snapshotId()); + validateSnapshotFiles(snapshots.get(2).snapshotId(), addedFileA1, addedFileB1, rewrittenFileA3); + validateSnapshotFiles(snapshots.get(3).snapshotId(), addedFileA1, addedFileB1, addedFileA2, addedFileA3); + } + } + + @Test + public void testCommitRewriteValidation() throws Exception { + Assume.assumeTrue("Sequence number is only supported in iceberg format v2.", formatVersion > 1); + + long timestamp = 0; + long checkpointId = 0; + JobID jobID = new JobID(); + try (OneInputStreamOperatorTestHarness harness = createStreamOpr(jobID)) { + harness.setup(); + harness.open(); + + StructLike partition = SimpleDataUtil.createPartition(null); + + DataFile rewrittenFile = createDataFile("data-old", partition); + long snapshot1 = commit(rewrittenFile); + + // add deletes files + DeleteFile posDeleteFile = createPosDeleteFile("pos-delete", partition); + DeleteFile eqDeleteFile = createEqDeleteFile("eq-delete", partition); + long snapshot2 = commit(ImmutableList.of(), ImmutableList.of(posDeleteFile, eqDeleteFile)); + + DataFile addedFile = createDataFile("data-new-1", partition); + RewriteResult rewriteResult = RewriteResult.builder(snapshot1, table.spec().partitionType()) + .partition(partition) + .addRewrittenDataFiles(ImmutableList.of(rewrittenFile)) + .addAddedDataFiles(ImmutableList.of(addedFile)) + .build(); + harness.processElement(rewriteResult, ++timestamp); + assertSnapshots(2); + + // should commit rewrite fail and abort for found new position delete for rewritten data file. + harness.snapshot(++checkpointId, ++timestamp); + assertSnapshots(2); + + List snapshots = Lists.newArrayList(table.snapshots().iterator()); + Assert.assertEquals("Snapshot id should equal", snapshot1, snapshots.get(0).snapshotId()); + Assert.assertEquals("Snapshot id should equal", snapshot2, snapshots.get(1).snapshotId()); + } + } + + private long commit(DataFile... dataFiles) { + return commit(Arrays.asList(dataFiles), ImmutableList.of()); + } + + private long commit(List dataFiles, List deleteFiles) { + RowDelta rowDelta = table.newRowDelta(); + dataFiles.forEach(rowDelta::addRows); + deleteFiles.forEach(rowDelta::addDeletes); + rowDelta.commit(); + CreateSnapshotEvent event = (CreateSnapshotEvent) rowDelta.updateEvent(); + return event.snapshotId(); + } + + private String getFilePath(String filename, StructLike partition) { + return partition == null ? table.locationProvider().newDataLocation(filename) : + table.locationProvider().newDataLocation(table.spec(), partition, filename); + } + + private DataFile createDataFile(String filename, StructLike partition) { + return DataFiles.builder(table.spec()) + .withFormat(format) + .withPath(getFilePath(filename, partition)) + .withFileSizeInBytes(10) + .withPartition(partition) + .withRecordCount(1) + .build(); + } + + private DeleteFile createPosDeleteFile(String filename, StructLike partition) { + return FileMetadata.deleteFileBuilder(table.spec()) + .ofPositionDeletes() + .withFormat(format) + .withPath(getFilePath(filename, partition)) + .withFileSizeInBytes(10) + .withPartition(partition) + .withRecordCount(1) + .build(); + } + + private DeleteFile createEqDeleteFile(String filename, StructLike partition) { + return FileMetadata.deleteFileBuilder(table.spec()) + .ofEqualityDeletes() + .withFormat(format) + .withPath(getFilePath(filename, partition)) + .withFileSizeInBytes(10) + .withPartition(partition) + .withRecordCount(1) + .build(); + } + + private void assertSnapshots(int expectedCount) { + table.refresh(); + int actualCount = Iterables.size(table.snapshots()); + Assert.assertEquals(expectedCount, actualCount); + } + + private void validateSnapshotFiles(long snapshotId, DataFile... expectedFiles) { + Set expectedFilePaths = Sets.newHashSet(); + for (DataFile file : expectedFiles) { + expectedFilePaths.add(file.path()); + } + Set actualFilePaths = Sets.newHashSet(); + for (FileScanTask task : table.newScan().useSnapshot(snapshotId).planFiles()) { + actualFilePaths.add(task.file().path()); + } + Assert.assertEquals("Files should match", expectedFilePaths, actualFilePaths); + } + + private OneInputStreamOperatorTestHarness createStreamOpr(JobID jobID) + throws Exception { + TestOperatorFactory factory = TestOperatorFactory.of(tablePath); + return new OneInputStreamOperatorTestHarness<>(factory, createEnvironment(jobID)); + } + + private static MockEnvironment createEnvironment(JobID jobID) { + return new MockEnvironmentBuilder() + .setTaskName("test task") + .setManagedMemorySize(32 * 1024) + .setInputSplitProvider(new MockInputSplitProvider()) + .setBufferSize(256) + .setTaskConfiguration(new org.apache.flink.configuration.Configuration()) + .setExecutionConfig(new ExecutionConfig()) + .setMaxParallelism(16) + .setJobID(jobID) + .build(); + } + + private static class TestOperatorFactory extends AbstractStreamOperatorFactory + implements OneInputStreamOperatorFactory { + private final String tablePath; + + private TestOperatorFactory(String tablePath) { + this.tablePath = tablePath; + } + + private static TestOperatorFactory of(String tablePath) { + return new TestOperatorFactory(tablePath); + } + + @Override + @SuppressWarnings("unchecked") + public > T createStreamOperator( + StreamOperatorParameters param) { + IcebergRewriteFilesCommitter operator = new IcebergRewriteFilesCommitter(TableLoader.fromHadoopTable(tablePath)); + operator.setup(param.getContainingTask(), param.getStreamConfig(), param.getOutput()); + return (T) operator; + } + + @Override + public Class getStreamOperatorClass(ClassLoader classLoader) { + return IcebergRewriteFilesCommitter.class; + } + } +} diff --git a/flink/v1.14/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergRewriteTaskEmitter.java b/flink/v1.14/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergRewriteTaskEmitter.java new file mode 100644 index 000000000000..408994ec6f24 --- /dev/null +++ b/flink/v1.14/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergRewriteTaskEmitter.java @@ -0,0 +1,1158 @@ +/* + * 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.flink.sink; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.Arrays; +import java.util.Comparator; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.stream.Collectors; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; +import org.apache.flink.runtime.operators.testutils.MockEnvironment; +import org.apache.flink.runtime.operators.testutils.MockEnvironmentBuilder; +import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider; +import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; +import org.apache.iceberg.BaseCombinedScanTask; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.ContentFile; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.FileMetadata; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.MockFileScanTask; +import org.apache.iceberg.RowDelta; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.actions.RewriteDataFiles; +import org.apache.iceberg.events.CreateSnapshotEvent; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +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.Pair; +import org.apache.iceberg.util.StructLikeWrapper; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +@RunWith(Parameterized.class) +public class TestIcebergRewriteTaskEmitter { + + private static final String FLINK_JOB_ID = "flink.job-id"; + + @Rule + public TemporaryFolder temp = new TemporaryFolder(); + + private Table table; + private String tablePath; + private File flinkManifestFolder; + + private final FileFormat format; + private final int formatVersion; + + @Parameterized.Parameters(name = "FileFormat = {0}, FormatVersion={1}") + public static Object[][] parameters() { + return new Object[][] { + new Object[] {"avro", 1}, + new Object[] {"avro", 2}, + new Object[] {"parquet", 1}, + new Object[] {"parquet", 2}, + new Object[] {"orc", 1}, + new Object[] {"orc", 2}, + }; + } + + public TestIcebergRewriteTaskEmitter(String format, int formatVersion) { + this.format = FileFormat.valueOf(format.toUpperCase(Locale.ENGLISH)); + this.formatVersion = formatVersion; + } + + @Before + public void setupTable() throws IOException { + File tableDir = temp.newFolder(); + Assert.assertTrue(tableDir.delete()); + + tablePath = tableDir.getAbsolutePath(); + + // Construct the iceberg table. + flinkManifestFolder = temp.newFolder(); + Map props = ImmutableMap.of( + TableProperties.DEFAULT_FILE_FORMAT, format.name(), + TableProperties.FORMAT_VERSION, String.valueOf(formatVersion), + ManifestOutputFileFactory.FLINK_MANIFEST_LOCATION, flinkManifestFolder.getAbsolutePath() + ); + table = SimpleDataUtil.createTable(tablePath, props, false); + } + + @Test + public void testSplitLargeFile() throws Exception { + long targetFileSize = 5000; + + table.updateProperties() + .set(FlinkSinkOptions.STREAMING_REWRITE_TARGET_FILE_SIZE_BYTES, String.valueOf(targetFileSize)) + .set(FlinkSinkOptions.STREAMING_REWRITE_MAX_GROUP_FILES, String.valueOf(Integer.MAX_VALUE)) + .commit(); + + long timestamp = 0; + long checkpoint = 0; + JobID jobID = new JobID(); + try (OneInputStreamOperatorTestHarness harness = createStreamOpr(jobID)) { + harness.setup(); + harness.open(); + + StructLike partition = SimpleDataUtil.createPartition(null); + + // qualified file should not trigger emit. + DataFile qualifiedFile = createDataFile("data-qualified", partition, (long) (targetFileSize * 1.2)); + CommitResult commit1 = commit( + jobID.toString(), + ImmutableList.of(qualifiedFile), + ImmutableList.of(), + ImmutableList.of() + ); + harness.processElement(commit1, ++timestamp); + Assert.assertTrue(harness.extractOutputValues().isEmpty()); + harness.snapshot(++checkpoint, timestamp); + harness.notifyOfCompletedCheckpoint(checkpoint); + assertFlinkManifests(1); + + // large file will trigger emit. + DataFile largeFile = createDataFile("data-large", partition, (targetFileSize * 2)); + DeleteFile posDeleteFile = createPosDeleteFile("pos-delete", partition); + DeleteFile eqDeleteFile = createEqDeleteFile("eq-delete", partition); + CommitResult commit2 = commit( + jobID.toString(), + ImmutableList.of(largeFile), + formatVersion > 1 ? ImmutableList.of(posDeleteFile, eqDeleteFile) : ImmutableList.of(), + formatVersion > 1 ? ImmutableList.of(largeFile.path()) : ImmutableList.of() + ); + + harness.processElement(commit2, ++timestamp); + assertFlinkManifests(formatVersion > 1 ? 3 : 2); // write delta manifests for pre snapshot and pre partition. + Assert.assertEquals(2, harness.extractOutputValues().size()); + + harness.snapshot(++checkpoint, timestamp); + harness.notifyOfCompletedCheckpoint(checkpoint); + assertFlinkManifests(0); // delete rewritten file groups manifests after checkpoint. + + // qualified file no need to rewrite and should be ignored. + List expected = formatVersion > 1 ? Lists.newArrayList( + createRewriteTask(commit2.snapshotId(), partition, + combined(largeFile, posDeleteFile) + ), + createRewriteTask(commit2.snapshotId(), partition, + combined(largeFile, posDeleteFile) + ) + ) : Lists.newArrayList( + createRewriteTask(commit2.snapshotId(), partition, + combined(largeFile) + ), + createRewriteTask(commit2.snapshotId(), partition, + combined(largeFile) + ) + ); + List actual = harness.extractOutputValues(); + assertRewriteTasks(expected, actual); + + actual.forEach(task -> Assert.assertEquals(targetFileSize, task.task().files().iterator().next().length())); + } + } + + @Test + public void testCombineMixedFiles() throws Exception { + long targetFileSize = 5000; + + table.updateProperties() + .set(FlinkSinkOptions.STREAMING_REWRITE_TARGET_FILE_SIZE_BYTES, String.valueOf(targetFileSize)) + .set(FlinkSinkOptions.STREAMING_REWRITE_MAX_GROUP_FILES, String.valueOf(Integer.MAX_VALUE)) + .commit(); + + long timestamp = 0; + long checkpoint = 0; + JobID jobID = new JobID(); + try (OneInputStreamOperatorTestHarness harness = createStreamOpr(jobID)) { + harness.setup(); + harness.open(); + + StructLike partition = SimpleDataUtil.createPartition(null); + + // qualified file should not trigger emit. + DataFile qualifiedFile = createDataFile("data-qualified", partition, (long) (targetFileSize * 1.2)); + CommitResult commit1 = commit( + jobID.toString(), + ImmutableList.of(qualifiedFile), + ImmutableList.of(), + ImmutableList.of() + ); + harness.processElement(commit1, ++timestamp); + Assert.assertTrue(harness.extractOutputValues().isEmpty()); + harness.snapshot(++checkpoint, timestamp); + harness.notifyOfCompletedCheckpoint(checkpoint); + assertFlinkManifests(1); + + // small files will trigger emit + DataFile largeFile = createDataFile("data-large", partition, (long) (targetFileSize * 0.7)); + DataFile mediumFile = createDataFile("data-medium ", partition, (long) (targetFileSize * 0.2)); + DataFile smallFile = createDataFile("data-small", partition, (long) (targetFileSize * 0.1)); + DeleteFile posDeleteFile = createPosDeleteFile("pos-delete", partition); + DeleteFile eqDeleteFile = createEqDeleteFile("eq-delete", partition); + CommitResult commit2 = commit( + jobID.toString(), + ImmutableList.of(largeFile, mediumFile, smallFile), + formatVersion > 1 ? ImmutableList.of(posDeleteFile, eqDeleteFile) : ImmutableList.of(), + formatVersion > 1 ? ImmutableList.of(largeFile.path()) : ImmutableList.of() + ); + + harness.processElement(commit2, ++timestamp); + assertFlinkManifests(formatVersion > 1 ? 3 : 2); // write delta manifests for pre snapshot and pre partition. + Assert.assertEquals(1, harness.extractOutputValues().size()); + + harness.snapshot(++checkpoint, timestamp); + harness.notifyOfCompletedCheckpoint(checkpoint); + assertFlinkManifests(0); // delete rewritten file groups manifests after checkpoint. + + // qualified file no need to rewrite and should be ignored. + List expected = formatVersion > 1 ? Lists.newArrayList( + createRewriteTask(commit2.snapshotId(), partition, + combined(largeFile, posDeleteFile), + combined(mediumFile, posDeleteFile), + combined(smallFile, posDeleteFile) + ) + ) : Lists.newArrayList( + createRewriteTask(commit2.snapshotId(), partition, + combined(largeFile), + combined(mediumFile), + combined(smallFile) + ) + ); + List actual = harness.extractOutputValues(); + assertRewriteTasks(expected, actual); + } + } + + @Test + public void testCombineFilesByFilesCount() throws Exception { + table.updateProperties().set(FlinkSinkOptions.STREAMING_REWRITE_MAX_GROUP_FILES, String.valueOf(3)).commit(); + + long timestamp = 0; + long checkpoint = 0; + JobID jobID = new JobID(); + try (OneInputStreamOperatorTestHarness harness = createStreamOpr(jobID)) { + harness.setup(); + harness.open(); + + StructLike partition = SimpleDataUtil.createPartition(null); + + DataFile dataFile1 = createDataFile("data-1", partition); + DataFile dataFile2 = createDataFile("data-2", partition); + DeleteFile posDeleteFile = createPosDeleteFile("pos-delete", partition); + CommitResult commit1 = commit( + jobID.toString(), + ImmutableList.of(dataFile1, dataFile2), + formatVersion > 1 ? ImmutableList.of(posDeleteFile) : ImmutableList.of(), + formatVersion > 1 ? ImmutableList.of(dataFile1.path()) : ImmutableList.of() + ); + harness.processElement(commit1, ++timestamp); + Assert.assertTrue(harness.extractOutputValues().isEmpty()); + harness.snapshot(++checkpoint, timestamp); + harness.notifyOfCompletedCheckpoint(checkpoint); + assertFlinkManifests(formatVersion > 1 ? 2 : 1); + + // emit rewrite task when reach files count. + DataFile dataFile3 = createDataFile("data-3", partition); + DeleteFile eqDeleteFile = createEqDeleteFile("eq-delete", partition); + CommitResult commit2 = commit( + jobID.toString(), + ImmutableList.of(dataFile3), + formatVersion > 1 ? ImmutableList.of(eqDeleteFile) : ImmutableList.of(), + ImmutableList.of() + ); + harness.processElement(commit2, ++timestamp); + assertFlinkManifests(formatVersion > 1 ? 4 : 2); // write delta manifests for pre snapshot and pre partition. + Assert.assertEquals(1, harness.extractOutputValues().size()); + harness.snapshot(++checkpoint, timestamp); + harness.notifyOfCompletedCheckpoint(checkpoint); + assertFlinkManifests(0); // delete rewritten file groups manifests after checkpoint. + + List expected = formatVersion > 1 ? Lists.newArrayList( + createRewriteTask(commit2.snapshotId(), partition, + combined(dataFile1, posDeleteFile, eqDeleteFile), + combined(dataFile2, posDeleteFile, eqDeleteFile), + combined(dataFile3) + ) + ) : Lists.newArrayList( + createRewriteTask(commit2.snapshotId(), partition, + combined(dataFile1), + combined(dataFile2), + combined(dataFile3) + ) + ); + List actual = harness.extractOutputValues(); + assertRewriteTasks(expected, actual); + } + } + + @Test + public void testFlushPendingRewriteFiles() throws Exception { + table.updateSpec().addField("data").commit(); + + table.updateProperties() + .set(RewriteDataFiles.TARGET_FILE_SIZE_BYTES, String.valueOf(Integer.MAX_VALUE)) + .set(FlinkSinkOptions.STREAMING_REWRITE_MAX_GROUP_FILES, String.valueOf(Integer.MAX_VALUE)) + .set(FlinkSinkOptions.STREAMING_REWRITE_MIN_GROUP_FILES, String.valueOf(2)) + .set(FlinkSinkOptions.STREAMING_REWRITE_MAX_WAITING_COMMITS, String.valueOf(1)) + .commit(); + + long timestamp = 0; + long checkpoint = 10; + JobID jobId = new JobID(); + try (OneInputStreamOperatorTestHarness harness = createStreamOpr(jobId)) { + harness.setup(); + harness.open(); + + StructLike partitionPass = SimpleDataUtil.createPartition("hour=00"); + StructLike partitionCurr = SimpleDataUtil.createPartition("hour=01"); + + // add files to pass partition + DataFile remainDataFile1 = createDataFile("data-remain-1", partitionPass); + DeleteFile remainPosDeleteFile = createPosDeleteFile("pos-delete-remain", partitionPass); + CommitResult commit1 = commit( + jobId.toString(), + ImmutableList.of(remainDataFile1), + formatVersion > 1 ? ImmutableList.of(remainPosDeleteFile) : ImmutableList.of(), + formatVersion > 1 ? ImmutableList.of(remainDataFile1.path()) : ImmutableList.of() + ); + harness.processElement(commit1, ++timestamp); + Assert.assertTrue(harness.extractOutputValues().isEmpty()); + harness.snapshot(++checkpoint, timestamp); + harness.notifyOfCompletedCheckpoint(checkpoint); + assertFlinkManifests(formatVersion > 1 ? 2 : 1); + + // start write files to curr partition and never have new file will add to pass partition. + DataFile remainDataFile2 = createDataFile("data-remain-2", partitionPass); + DeleteFile remainEqDeleteFile = createEqDeleteFile("eq-delete-remain", partitionPass); + DataFile newDataFile1 = createDataFile("data-new-1", partitionCurr); + CommitResult commit2 = commit( + jobId.toString(), + ImmutableList.of(remainDataFile2, newDataFile1), + formatVersion > 1 ? ImmutableList.of(remainEqDeleteFile) : ImmutableList.of(), + ImmutableList.of() + ); + harness.processElement(commit2, ++timestamp); + Assert.assertTrue(harness.extractOutputValues().isEmpty()); + harness.snapshot(++checkpoint, timestamp); + harness.notifyOfCompletedCheckpoint(checkpoint); + assertFlinkManifests(formatVersion > 1 ? 5 : 3); + + // pass partition wait one commit. + DataFile newDataFile2 = createDataFile("data-new-2", partitionCurr); + CommitResult commit3 = commit( + jobId.toString(), + ImmutableList.of(newDataFile2), + ImmutableList.of(), + ImmutableList.of() + ); + harness.processElement(commit3, ++timestamp); + Assert.assertTrue(harness.extractOutputValues().isEmpty()); + harness.snapshot(++checkpoint, timestamp); + harness.notifyOfCompletedCheckpoint(checkpoint); + assertFlinkManifests(formatVersion > 1 ? 6 : 4); + + // flush pass partition files as rewrite task. + DataFile newDataFile3 = createDataFile("data-new-3", partitionCurr); + CommitResult commit4 = commit( + jobId.toString(), + ImmutableList.of(newDataFile3), + ImmutableList.of(), + ImmutableList.of() + ); + harness.processElement(commit4, ++timestamp); + assertFlinkManifests(formatVersion > 1 ? 7 : 5); // write delta manifests for pre snapshot and pre partition. + Assert.assertEquals(1, harness.extractOutputValues().size()); + harness.snapshot(++checkpoint, timestamp); + harness.notifyOfCompletedCheckpoint(checkpoint); + assertFlinkManifests(3); // delete rewritten file groups manifests after checkpoint. + + // rewrite task snapshot id should be the latest commit of the pass partition. + List expected = formatVersion > 1 ? Lists.newArrayList( + createRewriteTask(commit2.snapshotId(), partitionPass, + combined(remainDataFile1, remainPosDeleteFile, remainEqDeleteFile), + combined(remainDataFile2) + ) + ) : Lists.newArrayList( + createRewriteTask(commit2.snapshotId(), partitionPass, + combined(remainDataFile1), + combined(remainDataFile2) + ) + ); + List actual = harness.extractOutputValues(); + assertRewriteTasks(expected, actual); + + } + } + + @Test + public void testRecoveryFromValidSnapshotForUnpartitionedTable() throws Exception { + table.updateSpec().addField("data").commit(); + table.updateProperties().set(FlinkSinkOptions.STREAMING_REWRITE_MAX_GROUP_FILES, String.valueOf(4)).commit(); + + long timestamp = 0; + long checkpoint = 10; + OperatorSubtaskState snapshot; + + StructLike partition = SimpleDataUtil.createPartition(null); + + // appended files in old flink job, should be included in rewrite tasks + JobID oldJobId = new JobID(); + DataFile dataFile = createDataFile("data-txn1-1", partition); + CommitResult commit1 = commit( + oldJobId.toString(), + ImmutableList.of(dataFile), + ImmutableList.of(), + ImmutableList.of() + ); + + try (OneInputStreamOperatorTestHarness harness = createStreamOpr(oldJobId)) { + harness.setup(); + harness.open(); + + harness.processElement(commit1, ++timestamp); + assertFlinkManifests(1); + Assert.assertTrue(harness.extractOutputValues().isEmpty()); + + snapshot = harness.snapshot(++checkpoint, timestamp); + harness.notifyOfCompletedCheckpoint(checkpoint); + assertFlinkManifests(1); // no rewritten file groups manifests should be deleted. + } + + // add not appended files + DataFile missingDataFile11 = createDataFile("data-txn2-1", partition); + DataFile missingDataFile12 = createDataFile("data-txn2-2", partition); + DeleteFile missingDeleteFile11 = createPosDeleteFile("pos-delete-txn2-1", partition); + DeleteFile missingDeleteFile12 = createEqDeleteFile("eq-delete-txn2-1", partition); + CommitResult commit2 = commit( + oldJobId.toString(), + ImmutableList.of(missingDataFile11, missingDataFile12), + formatVersion > 1 ? ImmutableList.of(missingDeleteFile11, missingDeleteFile12) : ImmutableList.of(), + formatVersion > 1 ? ImmutableList.of(missingDataFile11.path()) : ImmutableList.of() + ); + + // add other files to table between two txn, only the added eq-delete files should be appended + DataFile otherDataFile = createDataFile("data-other-1", partition); + DeleteFile otherPosDeleteFile = createPosDeleteFile("pos-delete-other-1", partition); + DeleteFile otherEqDeleteFile = createEqDeleteFile("eq-delete-other-1", partition); + commit( + null, + ImmutableList.of(otherDataFile), + formatVersion > 1 ? ImmutableList.of(otherPosDeleteFile, otherEqDeleteFile) : ImmutableList.of(), + formatVersion > 1 ? ImmutableList.of(otherDataFile.path()) : ImmutableList.of() + ); + + // appended latest committed files + DataFile missingDataFile21 = createDataFile("data-txn3-1", partition); + DeleteFile missingDeleteFile22 = createEqDeleteFile("eq-delete-txn3-2", partition); + CommitResult commit3 = commit( + oldJobId.toString(), + ImmutableList.of(missingDataFile21), + formatVersion > 1 ? ImmutableList.of(missingDeleteFile22) : ImmutableList.of(), + ImmutableList.of() + ); + + JobID newJobId = new JobID(); + try (OneInputStreamOperatorTestHarness harness = createStreamOpr(newJobId)) { + harness.setup(); + harness.initializeState(snapshot); + harness.open(); + + assertFlinkManifests(formatVersion > 1 ? 6 : 3); // write delta manifests for pre snapshot and pre partition. + Assert.assertEquals(1, harness.extractOutputValues().size()); + + harness.snapshot(++checkpoint, timestamp); + harness.notifyOfCompletedCheckpoint(checkpoint); + assertFlinkManifests(0); // delete rewritten file groups manifests after checkpoint. + + List expected = formatVersion > 1 ? Lists.newArrayList( + createRewriteTask(commit3.snapshotId(), partition, + combined(dataFile, missingDeleteFile11, missingDeleteFile12, otherEqDeleteFile, missingDeleteFile22), + combined(missingDataFile11, missingDeleteFile11, otherEqDeleteFile, missingDeleteFile22), + combined(missingDataFile12, missingDeleteFile11, otherEqDeleteFile, missingDeleteFile22), + combined(missingDataFile21) + ) + ) : Lists.newArrayList( + createRewriteTask(commit3.snapshotId(), partition, + combined(dataFile), + combined(missingDataFile11), + combined(missingDataFile12), + combined(missingDataFile21) + ) + ); + List actual = harness.extractOutputValues(); + assertRewriteTasks(expected, actual); + } + } + + @Test + public void testRecoveryFromValidSnapshotForPartitionedTable() throws Exception { + table.updateSpec().addField("data").commit(); + table.updateProperties().set(FlinkSinkOptions.STREAMING_REWRITE_MAX_GROUP_FILES, String.valueOf(4)).commit(); + + long timestamp = 0; + long checkpoint = 10; + OperatorSubtaskState snapshot; + + StructLike partitionA = SimpleDataUtil.createPartition("aaa"); + StructLike partitionB = SimpleDataUtil.createPartition("bbb"); + + // appended files in old flink job, should be included in rewrite tasks + JobID oldJobId = new JobID(); + DataFile dataFileA = createDataFile("data-txn1-1", partitionA); + DataFile dataFileB = createDataFile("data-txn1-2", partitionB); + CommitResult commit1 = commit( + oldJobId.toString(), + ImmutableList.of(dataFileA, dataFileB), + ImmutableList.of(), + ImmutableList.of() + ); + + try (OneInputStreamOperatorTestHarness harness = createStreamOpr(oldJobId)) { + harness.setup(); + harness.open(); + + harness.processElement(commit1, ++timestamp); + assertFlinkManifests(2); + Assert.assertTrue(harness.extractOutputValues().isEmpty()); + + snapshot = harness.snapshot(++checkpoint, timestamp); + harness.notifyOfCompletedCheckpoint(checkpoint); + assertFlinkManifests(2); // no rewritten file groups manifests should be deleted. + } + + // add not appended files + DataFile missingDataFileA11 = createDataFile("data-txn2-1", partitionA); + DataFile missingDataFileA12 = createDataFile("data-txn2-2", partitionA); + DeleteFile missingDeleteFileA11 = createPosDeleteFile("pos-delete-txn2-1", partitionA); + DeleteFile missingDeleteFileA12 = createEqDeleteFile("eq-delete-txn2-1", partitionA); + DataFile missingDataFileB11 = createDataFile("data-txn2-1", partitionB); + DataFile missingDataFileB12 = createDataFile("data-txn2-2", partitionB); + CommitResult commit2 = commit( + oldJobId.toString(), + ImmutableList.of(missingDataFileA11, missingDataFileA12, missingDataFileB11, missingDataFileB12), + formatVersion > 1 ? ImmutableList.of(missingDeleteFileA11, missingDeleteFileA12) : ImmutableList.of(), + formatVersion > 1 ? ImmutableList.of(missingDataFileA11.path()) : ImmutableList.of() + ); + + // add other files to table between two txn, only the added eq-delete files should be appended + DataFile otherDataFileA = createDataFile("data-other-1", partitionA); + DeleteFile otherPosDeleteFileA = createPosDeleteFile("pos-delete-other-1", partitionA); + DeleteFile otherEqDeleteFileA = createEqDeleteFile("eq-delete-other-1", partitionA); + DataFile otherDataFileB = createDataFile("data-other-1", partitionB); + commit( + null, + ImmutableList.of(otherDataFileA, otherDataFileB), + formatVersion > 1 ? ImmutableList.of(otherPosDeleteFileA, otherEqDeleteFileA) : ImmutableList.of(), + formatVersion > 1 ? ImmutableList.of(otherDataFileA.path()) : ImmutableList.of() + ); + + // appended latest committed files + DataFile missingDataFileA21 = createDataFile("data-txn3-1", partitionA); + DeleteFile missingDeleteFileA21 = createEqDeleteFile("eq-delete-txn3-1", partitionA); + DataFile missingDataFileB21 = createDataFile("data-txn3-1", partitionB); + DeleteFile missingDeleteFileB22 = createEqDeleteFile("eq-delete-txn3-2", partitionB); + CommitResult commit3 = commit( + oldJobId.toString(), + ImmutableList.of(missingDataFileA21, missingDataFileB21), + formatVersion > 1 ? ImmutableList.of(missingDeleteFileA21, missingDeleteFileB22) : ImmutableList.of(), + ImmutableList.of() + ); + + JobID newJobId = new JobID(); + try (OneInputStreamOperatorTestHarness harness = createStreamOpr(newJobId)) { + harness.setup(); + harness.initializeState(snapshot); + harness.open(); + + assertFlinkManifests(formatVersion > 1 ? 10 : 6); // write delta manifests for pre snapshot and pre partition. + Assert.assertEquals(2, harness.extractOutputValues().size()); + + harness.snapshot(++checkpoint, timestamp); + harness.notifyOfCompletedCheckpoint(checkpoint); + assertFlinkManifests(0); // delete rewritten file groups manifests after checkpoint. + + List expected = formatVersion > 1 ? Lists.newArrayList( + createRewriteTask(commit3.snapshotId(), partitionA, + combined(dataFileA, missingDeleteFileA11, missingDeleteFileA12, otherEqDeleteFileA, missingDeleteFileA21), + combined(missingDataFileA11, missingDeleteFileA11, otherEqDeleteFileA, missingDeleteFileA21), + combined(missingDataFileA12, missingDeleteFileA11, otherEqDeleteFileA, missingDeleteFileA21), + combined(missingDataFileA21) + ), + createRewriteTask(commit3.snapshotId(), partitionB, + combined(dataFileB, missingDeleteFileB22), + combined(missingDataFileB11, missingDeleteFileB22), + combined(missingDataFileB12, missingDeleteFileB22), + combined(missingDataFileB21) + ) + ) : Lists.newArrayList( + createRewriteTask(commit3.snapshotId(), partitionA, + combined(dataFileA), + combined(missingDataFileA11), + combined(missingDataFileA12), + combined(missingDataFileA21) + ), + createRewriteTask(commit3.snapshotId(), partitionB, + combined(dataFileB), + combined(missingDataFileB11), + combined(missingDataFileB12), + combined(missingDataFileB21) + ) + ); + List actual = harness.extractOutputValues(); + assertRewriteTasks(expected, actual); + } + } + + @Test + public void testContinuousCommitsForUnpartitionedTable() throws Exception { + table.updateProperties().set(FlinkSinkOptions.STREAMING_REWRITE_MAX_GROUP_FILES, String.valueOf(3)).commit(); + + long timestamp = 0; + long checkpoint = 10; + JobID currentJobId = new JobID(); + try (OneInputStreamOperatorTestHarness harness = createStreamOpr(currentJobId)) { + harness.setup(); + harness.open(); + + StructLike partition = SimpleDataUtil.createPartition(null); + + // add exists files to table which should be ignored by emitter + DataFile existDataFile1 = createDataFile("data-exist-1", partition); + DataFile existDataFile2 = createDataFile("data-exist-2", partition); + commit(null, ImmutableList.of(existDataFile1, existDataFile2), ImmutableList.of(), ImmutableList.of()); + + // Txn#1: init last emitted snapshot id and emit last committed files + DataFile newDataFile11 = createDataFile("data-txn1-1", partition); + DataFile newDataFile12 = createDataFile("data-txn1-2", partition); + DeleteFile newDeleteFile11 = createPosDeleteFile("pos-delete-txn1-1", partition); + CommitResult commit1 = commit( + currentJobId.toString(), + ImmutableList.of(newDataFile11, newDataFile12), + formatVersion > 1 ? ImmutableList.of(newDeleteFile11) : ImmutableList.of(), + formatVersion > 1 ? ImmutableList.of(newDataFile11.path()) : ImmutableList.of() + ); + + harness.processElement(commit1, ++timestamp); + assertFlinkManifests(formatVersion > 1 ? 2 : 1); + Assert.assertTrue(harness.extractOutputValues().isEmpty()); + + harness.snapshot(++checkpoint, timestamp); + harness.notifyOfCompletedCheckpoint(checkpoint); + assertFlinkManifests(formatVersion > 1 ? 2 : 1); // no rewritten file groups manifests should be deleted. + + // Txn#2: only emit last committed files + DataFile newDataFile21 = createDataFile("data-txn2-1", partition); + DeleteFile newDeleteFile21 = createEqDeleteFile("eq-delete-txn2-1", partition); + CommitResult commit2 = commit( + currentJobId.toString(), + ImmutableList.of(newDataFile21), + formatVersion > 1 ? ImmutableList.of(newDeleteFile21) : ImmutableList.of(), + ImmutableList.of() + ); + harness.processElement(commit2, ++timestamp); + assertFlinkManifests(formatVersion > 1 ? 4 : 2); // write delta manifests for pre snapshot and pre partition. + Assert.assertEquals(1, harness.extractOutputValues().size()); + + harness.snapshot(++checkpoint, timestamp); + harness.notifyOfCompletedCheckpoint(checkpoint); + assertFlinkManifests(0); // delete rewritten file groups manifests after checkpoint. + + List expected = formatVersion > 1 ? Lists.newArrayList( + createRewriteTask(commit2.snapshotId(), partition, + combined(newDataFile11, newDeleteFile11, newDeleteFile21), + combined(newDataFile12, newDeleteFile11, newDeleteFile21), + combined(newDataFile21) + ) + ) : Lists.newArrayList( + createRewriteTask(commit2.snapshotId(), partition, + combined(newDataFile11), + combined(newDataFile12), + combined(newDataFile21) + ) + ); + List actual = harness.extractOutputValues(); + assertRewriteTasks(expected, actual); + } + } + + @Test + public void testContinuousCommitsForPartitionedTable() throws Exception { + table.updateSpec().addField("data").commit(); + table.updateProperties().set(FlinkSinkOptions.STREAMING_REWRITE_MAX_GROUP_FILES, String.valueOf(3)).commit(); + + long timestamp = 0; + long checkpoint = 10; + JobID currentJobId = new JobID(); + try (OneInputStreamOperatorTestHarness harness = createStreamOpr(currentJobId)) { + harness.setup(); + harness.open(); + + StructLike partitionA = SimpleDataUtil.createPartition("aaa"); + StructLike partitionB = SimpleDataUtil.createPartition("bbb"); + + // add exists files to table which should be ignored by emitter + DataFile existDataFileA = createDataFile("data-exist-1", partitionA); + DataFile existDataFileB = createDataFile("data-exist-2", partitionB); + commit(null, ImmutableList.of(existDataFileA, existDataFileB), ImmutableList.of(), ImmutableList.of()); + + // Txn#1: init last received snapshot id and emit last committed files + DataFile newDataFileA11 = createDataFile("data-txn1-1", partitionA); + DataFile newDataFileA12 = createDataFile("data-txn1-2", partitionA); + DeleteFile newDeleteFileA11 = createPosDeleteFile("pos-delete-txn1-1", partitionA); + DataFile newDataFileB11 = createDataFile("data-txn1-1", partitionB); + DataFile newDataFileB12 = createDataFile("data-txn1-1", partitionB); + CommitResult commit1 = commit( + currentJobId.toString(), + ImmutableList.of(newDataFileA11, newDataFileA12, newDataFileB11, newDataFileB12), + formatVersion > 1 ? ImmutableList.of(newDeleteFileA11) : ImmutableList.of(), + formatVersion > 1 ? ImmutableList.of(newDataFileA11.path()) : ImmutableList.of() + ); + + harness.processElement(commit1, ++timestamp); + assertFlinkManifests(formatVersion > 1 ? 3 : 2); + Assert.assertTrue(harness.extractOutputValues().isEmpty()); + + harness.snapshot(++checkpoint, timestamp); + harness.notifyOfCompletedCheckpoint(checkpoint); + assertFlinkManifests(formatVersion > 1 ? 3 : 2); // no rewritten file groups manifests should be deleted. + + // Txn#2: only append last committed files + DataFile newDataFileA21 = createDataFile("data-txn2-1", partitionA); + DeleteFile newDeleteFileA21 = createEqDeleteFile("eq-delete-txn2-1", partitionA); + DataFile newDataFileB21 = createDataFile("data-txn2-1", partitionB); + DeleteFile newDeleteFileB21 = createEqDeleteFile("eq-delete-txn2-1", partitionB); + CommitResult commit2 = commit( + currentJobId.toString(), + ImmutableList.of(newDataFileA21, newDataFileB21), + formatVersion > 1 ? ImmutableList.of(newDeleteFileA21, newDeleteFileB21) : ImmutableList.of(), + ImmutableList.of() + ); + harness.processElement(commit2, ++timestamp); + assertFlinkManifests(formatVersion > 1 ? 7 : 4); // write delta manifests for pre snapshot and pre partition. + Assert.assertEquals(2, harness.extractOutputValues().size()); + + harness.snapshot(++checkpoint, timestamp); + harness.notifyOfCompletedCheckpoint(checkpoint); + assertFlinkManifests(0); // delete rewritten file groups manifests after checkpoint. + + List expected = formatVersion > 1 ? Lists.newArrayList( + createRewriteTask(commit2.snapshotId(), partitionA, + combined(newDataFileA11, newDeleteFileA11, newDeleteFileA21), + combined(newDataFileA12, newDeleteFileA11, newDeleteFileA21), + combined(newDataFileA21) + ), + createRewriteTask(commit2.snapshotId(), partitionB, + combined(newDataFileB11, newDeleteFileB21), + combined(newDataFileB12, newDeleteFileB21), + combined(newDataFileB21) + ) + ) : Lists.newArrayList( + createRewriteTask(commit2.snapshotId(), partitionA, + combined(newDataFileA11), + combined(newDataFileA12), + combined(newDataFileA21) + ), + createRewriteTask(commit2.snapshotId(), partitionB, + combined(newDataFileB11), + combined(newDataFileB12), + combined(newDataFileB21) + ) + ); + List actual = harness.extractOutputValues(); + assertRewriteTasks(expected, actual); + } + } + + @Test + public void testDiscontinuousCommitsForUnpartitionedTable() throws Exception { + table.updateProperties().set(FlinkSinkOptions.STREAMING_REWRITE_MAX_GROUP_FILES, String.valueOf(3)).commit(); + + long timestamp = 0; + long checkpoint = 10; + JobID currentJobId = new JobID(); + JobID otherJobId = new JobID(); + try (OneInputStreamOperatorTestHarness harness = createStreamOpr(currentJobId)) { + harness.setup(); + harness.open(); + + StructLike partition = SimpleDataUtil.createPartition(null); + + // add exists files to table which should be ignored by emitter + DataFile existDataFile1 = createDataFile("data-exist-1", partition); + DataFile existDataFile2 = createDataFile("data-exist-2", partition); + commit(null, ImmutableList.of(existDataFile1, existDataFile2), ImmutableList.of(), ImmutableList.of()); + + // Txn#1: init last emitted snapshot id and emit last committed files + DataFile newDataFile11 = createDataFile("data-txn1-1", partition); + DataFile newDataFile12 = createDataFile("data-txn1-2", partition); + DeleteFile newDeleteFile11 = createPosDeleteFile("pos-delete-txn1-1", partition); + CommitResult commit1 = commit( + currentJobId.toString(), + ImmutableList.of(newDataFile11, newDataFile12), + formatVersion > 1 ? ImmutableList.of(newDeleteFile11) : ImmutableList.of(), + formatVersion > 1 ? ImmutableList.of(newDataFile11.path()) : ImmutableList.of() + ); + + harness.processElement(commit1, ++timestamp); + assertFlinkManifests(formatVersion > 1 ? 2 : 1); + Assert.assertTrue(harness.extractOutputValues().isEmpty()); + + harness.snapshot(++checkpoint, timestamp); + harness.notifyOfCompletedCheckpoint(checkpoint); + assertFlinkManifests(formatVersion > 1 ? 2 : 1); // no rewritten file groups manifests should be deleted. + + // add other files to table between two txn, the added eq-delete files should be emitted + DataFile otherDataFile = createDataFile("data-other-1", partition); + DeleteFile otherPosDeleteFile = createPosDeleteFile("pos-delete-other-1", partition); + DeleteFile otherEqDeleteFile = createEqDeleteFile("eq-delete-other-1", partition); + commit( + otherJobId.toString(), + ImmutableList.of(otherDataFile), + formatVersion > 1 ? ImmutableList.of(otherPosDeleteFile, otherEqDeleteFile) : ImmutableList.of(), + formatVersion > 1 ? ImmutableList.of(otherDataFile.path()) : ImmutableList.of() + ); + + // Txn#2: only emit last committed files + DataFile newDataFile21 = createDataFile("data-txn2-1", partition); + DeleteFile newDeleteFile21 = createEqDeleteFile("eq-delete-txn2-1", partition); + CommitResult commit2 = commit( + currentJobId.toString(), + ImmutableList.of(newDataFile21), + formatVersion > 1 ? ImmutableList.of(newDeleteFile21) : ImmutableList.of(), + ImmutableList.of() + ); + harness.processElement(commit2, ++timestamp); + assertFlinkManifests(formatVersion > 1 ? 5 : 2); // write delta manifests for pre snapshot and pre partition. + Assert.assertEquals(1, harness.extractOutputValues().size()); + + harness.snapshot(++checkpoint, timestamp); + harness.notifyOfCompletedCheckpoint(checkpoint); + assertFlinkManifests(0); // delete rewritten file groups manifests after checkpoint. + + List expected = formatVersion > 1 ? Lists.newArrayList( + createRewriteTask(commit2.snapshotId(), partition, + combined(newDataFile11, newDeleteFile11, otherEqDeleteFile, newDeleteFile21), + combined(newDataFile12, newDeleteFile11, otherEqDeleteFile, newDeleteFile21), + combined(newDataFile21) + ) + ) : Lists.newArrayList( + createRewriteTask(commit2.snapshotId(), partition, + combined(newDataFile11), + combined(newDataFile12), + combined(newDataFile21) + ) + ); + List actual = harness.extractOutputValues(); + assertRewriteTasks(expected, actual); + } + } + + @Test + public void testDiscontinuousCommitsForPartitionedTable() throws Exception { + table.updateSpec().addField("data").commit(); + table.updateProperties().set(FlinkSinkOptions.STREAMING_REWRITE_MAX_GROUP_FILES, String.valueOf(3)).commit(); + + long timestamp = 0; + long checkpoint = 10; + JobID currentJobId = new JobID(); + JobID otherJobId = new JobID(); + try (OneInputStreamOperatorTestHarness harness = createStreamOpr(currentJobId)) { + harness.setup(); + harness.open(); + + StructLike partitionA = SimpleDataUtil.createPartition("aaa"); + StructLike partitionB = SimpleDataUtil.createPartition("bbb"); + + // add exists files to table which should be ignored by emitter + DataFile existDataFileA = createDataFile("data-exist-1", partitionA); + DataFile existDataFileB = createDataFile("data-exist-2", partitionB); + commit(null, ImmutableList.of(existDataFileA, existDataFileB), ImmutableList.of(), ImmutableList.of()); + + // Txn#1: init last received snapshot id and emit last committed files + DataFile newDataFileA11 = createDataFile("data-txn1-1", partitionA); + DataFile newDataFileA12 = createDataFile("data-txn1-2", partitionA); + DeleteFile newDeleteFileA11 = createPosDeleteFile("pos-delete-txn1-1", partitionA); + DataFile newDataFileB11 = createDataFile("data-txn1-1", partitionB); + DataFile newDataFileB12 = createDataFile("data-txn1-1", partitionB); + CommitResult commit1 = commit( + currentJobId.toString(), + ImmutableList.of(newDataFileA11, newDataFileA12, newDataFileB11, newDataFileB12), + formatVersion > 1 ? ImmutableList.of(newDeleteFileA11) : ImmutableList.of(), + formatVersion > 1 ? ImmutableList.of(newDataFileA11.path()) : ImmutableList.of() + ); + + harness.processElement(commit1, ++timestamp); + assertFlinkManifests(formatVersion > 1 ? 3 : 2); + Assert.assertTrue(harness.extractOutputValues().isEmpty()); + + harness.snapshot(++checkpoint, timestamp); + harness.notifyOfCompletedCheckpoint(checkpoint); + assertFlinkManifests(formatVersion > 1 ? 3 : 2); // no rewritten file groups manifests should be deleted. + + // add other files to table between two txn, only the added eq-delete files should be appended + DataFile otherDataFileA = createDataFile("data-other-1", partitionA); + DeleteFile otherPosDeleteFileA = createPosDeleteFile("pos-delete-other-1", partitionA); + DeleteFile otherEqDeleteFileA = createEqDeleteFile("eq-delete-other-1", partitionA); + DataFile otherDataFileB = createDataFile("data-other-1", partitionB); + commit( + otherJobId.toString(), + ImmutableList.of(otherDataFileA, otherDataFileB), + formatVersion > 1 ? ImmutableList.of(otherPosDeleteFileA, otherEqDeleteFileA) : ImmutableList.of(), + formatVersion > 1 ? ImmutableList.of(otherDataFileA.path()) : ImmutableList.of() + ); + + // Txn#2: only append last committed files + DataFile newDataFileA21 = createDataFile("data-txn2-1", partitionA); + DeleteFile newDeleteFileA21 = createEqDeleteFile("eq-delete-txn2-1", partitionA); + DataFile newDataFileB21 = createDataFile("data-txn2-1", partitionB); + DeleteFile newDeleteFileB21 = createEqDeleteFile("eq-delete-txn2-1", partitionB); + CommitResult commit2 = commit( + currentJobId.toString(), + ImmutableList.of(newDataFileA21, newDataFileB21), + formatVersion > 1 ? ImmutableList.of(newDeleteFileA21, newDeleteFileB21) : ImmutableList.of(), + ImmutableList.of() + ); + harness.processElement(commit2, ++timestamp); + assertFlinkManifests(formatVersion > 1 ? 8 : 4); // write delta manifests for pre snapshot and pre partition. + Assert.assertEquals(2, harness.extractOutputValues().size()); + + harness.snapshot(++checkpoint, timestamp); + harness.notifyOfCompletedCheckpoint(checkpoint); + assertFlinkManifests(0); // delete rewritten file groups manifests after checkpoint. + + List expected = formatVersion > 1 ? Lists.newArrayList( + createRewriteTask(commit2.snapshotId(), partitionA, + combined(newDataFileA11, newDeleteFileA11, otherEqDeleteFileA, newDeleteFileA21), + combined(newDataFileA12, newDeleteFileA11, otherEqDeleteFileA, newDeleteFileA21), + combined(newDataFileA21) + ), + createRewriteTask(commit2.snapshotId(), partitionB, + combined(newDataFileB11, newDeleteFileB21), + combined(newDataFileB12, newDeleteFileB21), + combined(newDataFileB21) + ) + ) : Lists.newArrayList( + createRewriteTask(commit2.snapshotId(), partitionA, + combined(newDataFileA11), + combined(newDataFileA12), + combined(newDataFileA21) + ), + createRewriteTask(commit2.snapshotId(), partitionB, + combined(newDataFileB11), + combined(newDataFileB12), + combined(newDataFileB21) + ) + ); + List actual = harness.extractOutputValues(); + assertRewriteTasks(expected, actual); + } + } + + private StructLikeWrapper wrap(StructLike partition) { + return StructLikeWrapper.forType(table.spec().partitionType()).set(partition); + } + + private String getFilePath(String filename, StructLike partition) { + return partition == null ? table.locationProvider().newDataLocation(filename) : + table.locationProvider().newDataLocation(table.spec(), partition, filename); + } + + private DataFile createDataFile(String filename, StructLike partition) { + return createDataFile(filename, partition, 10); + } + + private DataFile createDataFile(String filename, StructLike partition, long fileSize) { + return DataFiles.builder(table.spec()) + .withFormat(format) + .withPath(getFilePath(filename, partition)) + .withFileSizeInBytes(fileSize) + .withPartition(partition) + .withRecordCount(1) + .build(); + } + + private DeleteFile createPosDeleteFile(String filename, StructLike partition) { + return FileMetadata.deleteFileBuilder(table.spec()) + .ofPositionDeletes() + .withFormat(format) + .withPath(getFilePath(filename, partition)) + .withFileSizeInBytes(10) + .withPartition(partition) + .withRecordCount(1) + .build(); + } + + private DeleteFile createEqDeleteFile(String filename, StructLike partition) { + return FileMetadata.deleteFileBuilder(table.spec()) + .ofEqualityDeletes() + .withFormat(format) + .withPath(getFilePath(filename, partition)) + .withFileSizeInBytes(10) + .withPartition(partition) + .withRecordCount(1) + .build(); + } + + private CommitResult commit(String flinkJobId, List dataFiles, List deleteFiles, + List referencedFiles) { + RowDelta rowDelta = table.newRowDelta(); + dataFiles.forEach(rowDelta::addRows); + deleteFiles.forEach(rowDelta::addDeletes); + if (flinkJobId != null) { + rowDelta.set(FLINK_JOB_ID, flinkJobId); + } + rowDelta.commit(); + + CreateSnapshotEvent event = (CreateSnapshotEvent) rowDelta.updateEvent(); + WriteResult writeResult = WriteResult.builder() + .addDataFiles(dataFiles) + .addDeleteFiles(deleteFiles) + .addReferencedDataFiles(referencedFiles) + .build(); + return CommitResult.builder(event.sequenceNumber(), event.snapshotId()).add(writeResult).build(); + } + + private Pair> combined(DataFile dataFile, DeleteFile... deleteFiles) { + return Pair.of(dataFile, Lists.newArrayList(deleteFiles)); + } + + @SafeVarargs + private final RewriteTask createRewriteTask(long snapshotId, StructLike partition, + Pair>... scanFiles) { + List fileScanTasks = Arrays.stream(scanFiles) + .map(pair -> new MockFileScanTask(pair.first(), pair.second().toArray(new DeleteFile[0]))) + .collect(Collectors.toList()); + + CombinedScanTask combinedScanTask = new BaseCombinedScanTask(fileScanTasks); + return new RewriteTask(snapshotId, partition, combinedScanTask); + } + + private void assertRewriteTasks(List expectedFileGroups, List actualFileGroups) { + Assert.assertEquals("expected RewriteTasks and actual RewriteTasks should have same size", + expectedFileGroups.size(), actualFileGroups.size()); + + if (!table.spec().isUnpartitioned()) { + expectedFileGroups.sort(Comparator.comparing(o -> o.partition().toString())); + actualFileGroups.sort(Comparator.comparing(o -> o.partition().toString())); + } + + for (int i = 0; i < actualFileGroups.size(); i++) { + RewriteTask expected = expectedFileGroups.get(i); + RewriteTask actual = actualFileGroups.get(i); + Assert.assertEquals("Snapshot id should match", expected.snapshotId(), actual.snapshotId()); + Assert.assertEquals("Partition should match", wrap(expected.partition()), wrap(actual.partition())); + Assert.assertEquals( + "Data files should match", + expected.task().files().stream().map(t -> t.file().path()).collect(Collectors.toSet()), + actual.task().files().stream().map(t -> t.file().path()).collect(Collectors.toSet()) + ); + Assert.assertEquals( + "Delete files should match", + Sets.newHashSet(Iterables.transform(Iterables.concat(Iterables.transform( + expected.task().files(), FileScanTask::deletes)), ContentFile::path)), + Sets.newHashSet(Iterables.transform(Iterables.concat(Iterables.transform( + actual.task().files(), FileScanTask::deletes)), ContentFile::path)) + ); + } + } + + private void assertFlinkManifests(int expectedCount) throws IOException { + List manifests = Files.list(flinkManifestFolder.toPath()) + .filter(p -> !p.toString().endsWith(".crc")) + .collect(Collectors.toList()); + Assert.assertEquals(String.format("Expected %s flink manifests, but the list is: %s", expectedCount, manifests), + expectedCount, manifests.size()); + } + + private OneInputStreamOperatorTestHarness createStreamOpr(JobID jobID) + throws Exception { + TestOperatorFactory factory = TestOperatorFactory.of(tablePath); + return new OneInputStreamOperatorTestHarness<>(factory, createEnvironment(jobID)); + } + + private static MockEnvironment createEnvironment(JobID jobID) { + return new MockEnvironmentBuilder() + .setTaskName("test task") + .setManagedMemorySize(32 * 1024) + .setInputSplitProvider(new MockInputSplitProvider()) + .setBufferSize(256) + .setTaskConfiguration(new org.apache.flink.configuration.Configuration()) + .setExecutionConfig(new ExecutionConfig()) + .setMaxParallelism(16) + .setJobID(jobID) + .build(); + } + + private static class TestOperatorFactory extends AbstractStreamOperatorFactory + implements OneInputStreamOperatorFactory { + private final String tablePath; + + private TestOperatorFactory(String tablePath) { + this.tablePath = tablePath; + } + + private static TestOperatorFactory of(String tablePath) { + return new TestOperatorFactory(tablePath); + } + + @Override + @SuppressWarnings("unchecked") + public > T createStreamOperator( + StreamOperatorParameters param) { + IcebergRewriteTaskEmitter operator = new IcebergRewriteTaskEmitter(TableLoader.fromHadoopTable(tablePath)); + operator.setup(param.getContainingTask(), param.getStreamConfig(), param.getOutput()); + return (T) operator; + } + + @Override + public Class getStreamOperatorClass(ClassLoader classLoader) { + return IcebergRewriteTaskEmitter.class; + } + } +} diff --git a/flink/v1.14/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamRewriter.java b/flink/v1.14/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamRewriter.java new file mode 100644 index 000000000000..bb5a9322300e --- /dev/null +++ b/flink/v1.14/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamRewriter.java @@ -0,0 +1,323 @@ +/* + * 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.flink.sink; + +import java.io.File; +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.stream.Collectors; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.JobID; +import org.apache.flink.runtime.operators.testutils.MockEnvironment; +import org.apache.flink.runtime.operators.testutils.MockEnvironmentBuilder; +import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider; +import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.BaseCombinedScanTask; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.MockFileScanTask; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.PartitionSpecParser; +import org.apache.iceberg.RewriteFiles; +import org.apache.iceberg.RowDelta; +import org.apache.iceberg.SchemaParser; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.events.CreateSnapshotEvent; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.expressions.ResidualEvaluator; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.io.FileAppenderFactory; +import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +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.Pair; +import org.apache.iceberg.util.StructLikeWrapper; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +@RunWith(Parameterized.class) +public class TestIcebergStreamRewriter { + + @Rule + public TemporaryFolder temp = new TemporaryFolder(); + + private Table table; + private String tablePath; + private FileAppenderFactory appenderFactory; + + private final FileFormat format; + private final int formatVersion; + + @Parameterized.Parameters(name = "FileFormat = {0}, FormatVersion={1}") + public static Object[][] parameters() { + return new Object[][] { + new Object[] {"avro", 1}, + new Object[] {"avro", 2}, + new Object[] {"parquet", 1}, + new Object[] {"parquet", 2}, + new Object[] {"orc", 1}, + new Object[] {"orc", 2}, + }; + } + + public TestIcebergStreamRewriter(String format, int formatVersion) { + this.format = FileFormat.valueOf(format.toUpperCase(Locale.ENGLISH)); + this.formatVersion = formatVersion; + } + + @Before + public void setupTable() throws IOException { + File tableDir = temp.newFolder(); + Assert.assertTrue(tableDir.delete()); + + tablePath = tableDir.getAbsolutePath(); + File flinkManifestFolder = temp.newFolder(); + + // Construct the iceberg table. + Map props = ImmutableMap.of( + TableProperties.DEFAULT_FILE_FORMAT, format.name(), + TableProperties.FORMAT_VERSION, String.valueOf(formatVersion), + ManifestOutputFileFactory.FLINK_MANIFEST_LOCATION, flinkManifestFolder.getAbsolutePath() + ); + // partitioned must be false for `createRewriteTask` + table = SimpleDataUtil.createTable(tablePath, props, false); + + appenderFactory = createDeletableAppenderFactory(); + } + + @Test + public void testRewriteFiles() throws Exception { + long timestamp = 0; + JobID jobID = new JobID(); + try (OneInputStreamOperatorTestHarness harness = createStreamOpr(jobID)) { + harness.setup(); + harness.open(); + + StructLike partition = SimpleDataUtil.createPartition(null); + + DataFile dataFile1 = writeDataFile("data-1", partition, ImmutableList.of( + SimpleDataUtil.createRowData(1, "aaa"), + SimpleDataUtil.createRowData(2, "bbb"), + SimpleDataUtil.createRowData(3, "ccc") + )); + DataFile dataFile2 = writeDataFile("data-2", partition, ImmutableList.of( + SimpleDataUtil.createRowData(4, "ddd"), + SimpleDataUtil.createRowData(5, "eee") + )); + DeleteFile posDeleteFile = writePosDeleteFile("pos-delete-txn1-1", partition, ImmutableList.of( + Pair.of(dataFile1.path(), 1L)) + ); + commit( + ImmutableList.of(dataFile1, dataFile2), + formatVersion > 1 ? ImmutableList.of(posDeleteFile) : ImmutableList.of(), + formatVersion > 1 ? ImmutableList.of(dataFile1.path()) : ImmutableList.of() + ); + + DataFile dataFile3 = writeDataFile("data-3", partition, ImmutableList.of( + SimpleDataUtil.createRowData(6, "fff"), + SimpleDataUtil.createRowData(7, "ggg") + )); + DeleteFile eqDeleteFile = writeEqDeleteFile("eq-delete-txn2-1", partition, ImmutableList.of( + SimpleDataUtil.createRowData(3, "ccc"), + SimpleDataUtil.createRowData(4, "ddd") + )); + CommitResult commit = commit( + ImmutableList.of(dataFile3), + formatVersion > 1 ? ImmutableList.of(eqDeleteFile) : ImmutableList.of(), + ImmutableList.of() + ); + + RewriteTask rewriteTask = formatVersion > 1 ? createRewriteTask(commit.snapshotId(), partition, + combined(dataFile1, posDeleteFile, eqDeleteFile), + combined(dataFile2, eqDeleteFile), + combined(dataFile3) + ) : createRewriteTask(commit.snapshotId(), partition, + combined(dataFile1), + combined(dataFile2), + combined(dataFile3) + ); + + harness.processElement(rewriteTask, ++timestamp); + Assert.assertEquals(1, harness.extractOutputValues().size()); + + RewriteResult rewriteResult = harness.extractOutputValues().get(0); + Assert.assertEquals(1, rewriteResult.partitions().size()); + Assert.assertEquals(wrap(partition), wrap(rewriteResult.partitions().iterator().next())); + Assert.assertEquals(rewriteTask.snapshotId(), rewriteResult.startingSnapshotId()); + Assert.assertEquals(3, Iterables.size(rewriteResult.rewrittenDataFiles())); + Assert.assertEquals(1, Iterables.size(rewriteResult.addedDataFiles())); + + commitRewrite(rewriteResult); + List expected = formatVersion > 1 ? ImmutableList.of( + SimpleDataUtil.createRowData(1, "aaa"), + SimpleDataUtil.createRowData(5, "eee"), + SimpleDataUtil.createRowData(6, "fff"), + SimpleDataUtil.createRowData(7, "ggg") + ) : ImmutableList.of( + SimpleDataUtil.createRowData(1, "aaa"), + SimpleDataUtil.createRowData(2, "bbb"), + SimpleDataUtil.createRowData(3, "ccc"), + SimpleDataUtil.createRowData(4, "ddd"), + SimpleDataUtil.createRowData(5, "eee"), + SimpleDataUtil.createRowData(6, "fff"), + SimpleDataUtil.createRowData(7, "ggg") + ); + SimpleDataUtil.assertTableRows(table, ImmutableList.copyOf(expected)); + } + } + private StructLikeWrapper wrap(StructLike partition) { + return StructLikeWrapper.forType(table.spec().partitionType()).set(partition); + } + + private Pair> combined(DataFile dataFile, DeleteFile... deleteFiles) { + return Pair.of(dataFile, Lists.newArrayList(deleteFiles)); + } + + @SafeVarargs + private final RewriteTask createRewriteTask(long snapshotId, StructLike partition, + Pair>... scanFiles) { + String schemaStr = SchemaParser.toJson(table.schema()); + String specStr = PartitionSpecParser.toJson(PartitionSpec.unpartitioned()); + ResidualEvaluator residuals = ResidualEvaluator.unpartitioned(Expressions.alwaysTrue()); + List fileScanTasks = Arrays.stream(scanFiles) + .map(p -> new MockFileScanTask(p.first(), p.second().toArray(new DeleteFile[0]), schemaStr, specStr, residuals)) + .collect(Collectors.toList()); + + CombinedScanTask combinedScanTask = new BaseCombinedScanTask(fileScanTasks); + return new RewriteTask(snapshotId, partition, combinedScanTask); + } + + private CommitResult commit(List dataFiles, List deleteFiles, + List referencedFiles) { + RowDelta rowDelta = table.newRowDelta(); + dataFiles.forEach(rowDelta::addRows); + deleteFiles.forEach(rowDelta::addDeletes); + rowDelta.commit(); + + CreateSnapshotEvent event = (CreateSnapshotEvent) rowDelta.updateEvent(); + WriteResult writeResult = WriteResult.builder() + .addDataFiles(dataFiles) + .addDeleteFiles(deleteFiles) + .addReferencedDataFiles(referencedFiles) + .build(); + return CommitResult.builder(event.sequenceNumber(), event.snapshotId()).add(writeResult).build(); + } + + private void commitRewrite(RewriteResult result) { + RewriteFiles rewriteFiles = table.newRewrite() + .validateFromSnapshot(result.startingSnapshotId()) + .rewriteFiles(Sets.newHashSet(result.rewrittenDataFiles()), Sets.newHashSet(result.addedDataFiles())); + rewriteFiles.commit(); + } + + private DataFile writeDataFile(String filename, StructLike partition, List rows) + throws IOException { + return SimpleDataUtil.writeDataFile(table, format, filename, appenderFactory, partition, rows); + } + + private DeleteFile writeEqDeleteFile(String filename, StructLike partition, List deletes) + throws IOException { + return SimpleDataUtil.writeEqDeleteFile(table, format, filename, appenderFactory, partition, deletes); + } + + private DeleteFile writePosDeleteFile(String filename, StructLike partition, List> positions) + throws IOException { + return SimpleDataUtil.writePosDeleteFile(table, format, filename, appenderFactory, partition, positions); + } + + private FileAppenderFactory createDeletableAppenderFactory() { + int[] equalityFieldIds = new int[] { + table.schema().findField("id").fieldId(), + table.schema().findField("data").fieldId() + }; + return new FlinkAppenderFactory(table.schema(), + FlinkSchemaUtil.convert(table.schema()), table.properties(), table.spec(), equalityFieldIds, + table.schema(), null); + } + + private OneInputStreamOperatorTestHarness createStreamOpr(JobID jobID) + throws Exception { + TestOperatorFactory factory = TestOperatorFactory.of(tablePath); + return new OneInputStreamOperatorTestHarness<>(factory, createEnvironment(jobID)); + } + + private static MockEnvironment createEnvironment(JobID jobID) { + return new MockEnvironmentBuilder() + .setTaskName("test task") + .setManagedMemorySize(32 * 1024) + .setInputSplitProvider(new MockInputSplitProvider()) + .setBufferSize(256) + .setTaskConfiguration(new org.apache.flink.configuration.Configuration()) + .setExecutionConfig(new ExecutionConfig()) + .setMaxParallelism(16) + .setJobID(jobID) + .build(); + } + + private static class TestOperatorFactory extends AbstractStreamOperatorFactory + implements OneInputStreamOperatorFactory { + private final String tablePath; + + private TestOperatorFactory(String tablePath) { + this.tablePath = tablePath; + } + + private static TestOperatorFactory of(String tablePath) { + return new TestOperatorFactory(tablePath); + } + + @Override + @SuppressWarnings("unchecked") + public > T createStreamOperator( + StreamOperatorParameters param) { + IcebergStreamRewriter operator = new IcebergStreamRewriter(TableLoader.fromHadoopTable(tablePath)); + operator.setup(param.getContainingTask(), param.getStreamConfig(), param.getOutput()); + return (T) operator; + } + + @Override + public Class getStreamOperatorClass(ClassLoader classLoader) { + return IcebergStreamRewriter.class; + } + } +}