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

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,8 @@
import org.apache.flink.streaming.api.watermark.Watermark;
import org.apache.iceberg.actions.BinPackRewriteFilePlanner;
import org.apache.iceberg.actions.SizeBasedFileRewritePlanner;
import org.apache.iceberg.expressions.Expression;
import org.apache.iceberg.expressions.Expressions;
import org.apache.iceberg.flink.maintenance.operator.DataFileRewriteCommitter;
import org.apache.iceberg.flink.maintenance.operator.DataFileRewritePlanner;
import org.apache.iceberg.flink.maintenance.operator.DataFileRewriteRunner;
Expand Down Expand Up @@ -56,6 +58,7 @@ public static class Builder extends MaintenanceTaskBuilder<RewriteDataFiles.Buil
org.apache.iceberg.actions.RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT;
private final Map<String, String> rewriteOptions = Maps.newHashMapWithExpectedSize(6);
private long maxRewriteBytes = Long.MAX_VALUE;
private Expression filter = Expressions.alwaysTrue();

@Override
String maintenanceTaskName() {
Expand Down Expand Up @@ -190,6 +193,18 @@ public Builder maxFilesToRewrite(int maxFilesToRewrite) {
return this;
}

/**
* A user provided filter for determining which files will be considered by the rewrite
* strategy.
*
* @param newFilter the filter expression to apply
* @return this for method chaining
*/
public Builder filter(Expression newFilter) {
this.filter = newFilter;
return this;
}

/**
* Configures the properties for the rewriter.
*
Expand Down Expand Up @@ -233,7 +248,8 @@ DataStream<TaskResult> append(DataStream<Trigger> trigger) {
tableLoader(),
partialProgressEnabled ? partialProgressMaxCommits : 1,
maxRewriteBytes,
rewriteOptions))
rewriteOptions,
filter))
.name(operatorName(PLANNER_TASK_NAME))
.uid(PLANNER_TASK_NAME + uidSuffix())
.slotSharingGroup(slotSharingGroup())
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@
import org.apache.iceberg.actions.FileRewritePlan;
import org.apache.iceberg.actions.RewriteDataFiles;
import org.apache.iceberg.actions.RewriteFileGroup;
import org.apache.iceberg.expressions.Expression;
import org.apache.iceberg.flink.TableLoader;
import org.apache.iceberg.flink.maintenance.api.Trigger;
import org.apache.iceberg.io.CloseableIterator;
Expand Down Expand Up @@ -60,6 +61,7 @@ public class DataFileRewritePlanner
private final long maxRewriteBytes;
private final Map<String, String> rewriterOptions;
private transient Counter errorCounter;
private final Expression filter;

public DataFileRewritePlanner(
String tableName,
Expand All @@ -68,7 +70,9 @@ public DataFileRewritePlanner(
TableLoader tableLoader,
int newPartialProgressMaxCommits,
long maxRewriteBytes,
Map<String, String> rewriterOptions) {
Map<String, String> rewriterOptions,
Expression filter) {

Preconditions.checkNotNull(tableName, "Table name should no be null");
Preconditions.checkNotNull(taskName, "Task name should no be null");
Preconditions.checkNotNull(tableLoader, "Table loader should no be null");
Expand All @@ -81,6 +85,7 @@ public DataFileRewritePlanner(
this.partialProgressMaxCommits = newPartialProgressMaxCommits;
this.maxRewriteBytes = maxRewriteBytes;
this.rewriterOptions = rewriterOptions;
this.filter = filter;
}

@Override
Expand Down Expand Up @@ -113,7 +118,7 @@ public void processElement(Trigger value, Context ctx, Collector<PlannedGroup> o
return;
}

BinPackRewriteFilePlanner planner = new BinPackRewriteFilePlanner(table);
BinPackRewriteFilePlanner planner = new BinPackRewriteFilePlanner(table, filter);
planner.init(rewriterOptions);

FileRewritePlan<RewriteDataFiles.FileGroupInfo, FileScanTask, DataFile, RewriteFileGroup>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@
import org.apache.flink.streaming.api.graph.StreamGraphGenerator;
import org.apache.iceberg.ManifestFiles;
import org.apache.iceberg.Table;
import org.apache.iceberg.expressions.Expressions;
import org.apache.iceberg.flink.SimpleDataUtil;
import org.apache.iceberg.flink.maintenance.operator.MetricsReporterFactoryForTests;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
Expand Down Expand Up @@ -373,6 +374,46 @@ void testV2Table() throws Exception {
.build());
}

@Test
void testRewriteWithFilter() throws Exception {
Table table = createTable();
insert(table, 1, "a");
insert(table, 2, "b");
insert(table, 3, "c");
insert(table, 4, "d");

assertFileNum(table, 4, 0);

appendRewriteDataFiles(
RewriteDataFiles.builder()
.parallelism(2)
.deleteFileThreshold(10)
.targetFileSizeBytes(1_000_000L)
.maxFileGroupSizeBytes(10_000_000L)
.maxFileSizeBytes(2_000_000L)
.minFileSizeBytes(500_000L)
.minInputFiles(2)
// Only rewrite data files where id is 1 or 2 for testing rewrite
.filter(Expressions.in("id", 1, 2))
Copy link
Contributor

Choose a reason for hiding this comment

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

It's always nice to add a comment at the test-relevant parameters (here: the filter).

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Ok, I have add it .

.partialProgressEnabled(true)
.partialProgressMaxCommits(1)
.maxRewriteBytes(100_000L)
.rewriteAll(false));

runAndWaitForSuccess(infra.env(), infra.source(), infra.sink());

// There is four files, only id is 1 and 2 will be rewritten. so expect 3 files.
assertFileNum(table, 3, 0);

SimpleDataUtil.assertTableRecords(
table,
ImmutableList.of(
createRecord(1, "a"),
createRecord(2, "b"),
createRecord(3, "c"),
createRecord(4, "d")));
}

private void appendRewriteDataFiles() {
appendRewriteDataFiles(RewriteDataFiles.builder().rewriteAll(true));
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@
import org.apache.flink.streaming.util.ProcessFunctionTestHarnesses;
import org.apache.iceberg.DataFile;
import org.apache.iceberg.Table;
import org.apache.iceberg.expressions.Expressions;
import org.apache.iceberg.flink.TableLoader;
import org.apache.iceberg.flink.maintenance.api.Trigger;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
Expand All @@ -47,7 +48,8 @@ static List<DataFileRewritePlanner.PlannedGroup> planDataFileRewrite(TableLoader
tableLoader,
11,
10_000_000L,
ImmutableMap.of(MIN_INPUT_FILES, "2")))) {
ImmutableMap.of(MIN_INPUT_FILES, "2"),
Expressions.alwaysTrue()))) {
testHarness.open();

OperatorTestBase.trigger(testHarness);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@
import org.apache.iceberg.FileContent;
import org.apache.iceberg.FileScanTask;
import org.apache.iceberg.Table;
import org.apache.iceberg.expressions.Expressions;
import org.apache.iceberg.flink.maintenance.api.Trigger;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
import org.apache.iceberg.relocated.com.google.common.collect.Sets;
Expand Down Expand Up @@ -102,7 +103,8 @@ void testError() throws Exception {
tableLoader(),
11,
1L,
ImmutableMap.of(MIN_INPUT_FILES, "2")))) {
ImmutableMap.of(MIN_INPUT_FILES, "2"),
Expressions.alwaysTrue()))) {
testHarness.open();

// Cause an exception
Expand Down Expand Up @@ -167,7 +169,8 @@ void testMaxRewriteBytes() throws Exception {
tableLoader(),
11,
maxRewriteBytes,
ImmutableMap.of(MIN_INPUT_FILES, "2")))) {
ImmutableMap.of(MIN_INPUT_FILES, "2"),
Expressions.alwaysTrue()))) {
testHarness.open();

OperatorTestBase.trigger(testHarness);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -45,6 +45,7 @@
import org.apache.iceberg.data.RandomGenericData;
import org.apache.iceberg.data.Record;
import org.apache.iceberg.data.parquet.GenericParquetReaders;
import org.apache.iceberg.expressions.Expressions;
import org.apache.iceberg.flink.maintenance.api.Trigger;
import org.apache.iceberg.io.CloseableIterable;
import org.apache.iceberg.parquet.Parquet;
Expand Down Expand Up @@ -286,7 +287,8 @@ void testSplitSize() throws Exception {
MIN_INPUT_FILES,
"2",
TARGET_FILE_SIZE_BYTES,
String.valueOf(targetFileSize))))) {
String.valueOf(targetFileSize)),
Expressions.alwaysTrue()))) {
testHarness.open();

OperatorTestBase.trigger(testHarness);
Expand Down