From e8d15cc082ee20f353b50a964bdb27440634679b Mon Sep 17 00:00:00 2001 From: Guosmilesmile <511955993@qq.com> Date: Tue, 25 Nov 2025 16:41:48 +0800 Subject: [PATCH 1/4] Flink:Rewrite DataFile Support Parquet Merge --- .../maintenance/api/RewriteDataFiles.java | 14 +- .../operator/DataFileRewriteRunner.java | 115 +++- .../maintenance/api/TestRewriteDataFiles.java | 41 +- .../maintenance/operator/RewriteUtil.java | 8 +- .../TestDataFileRewriteCommitter.java | 35 +- .../operator/TestDataFileRewriteRunner.java | 97 ++- .../iceberg/parquet/ParquetFileMerger.java | 568 ++++++++++++++++++ .../apache/iceberg/parquet/ParquetWriter.java | 8 +- 8 files changed, 823 insertions(+), 63 deletions(-) create mode 100644 parquet/src/main/java/org/apache/iceberg/parquet/ParquetFileMerger.java diff --git a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/RewriteDataFiles.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/RewriteDataFiles.java index 3b64a79eee89..3267f191a3d0 100644 --- a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/RewriteDataFiles.java +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/api/RewriteDataFiles.java @@ -59,6 +59,7 @@ public static class Builder extends MaintenanceTaskBuilder rewriteOptions = Maps.newHashMapWithExpectedSize(6); private long maxRewriteBytes = Long.MAX_VALUE; private Expression filter = Expressions.alwaysTrue(); + private boolean openParquetMerge = false; @Override String maintenanceTaskName() { @@ -218,6 +219,16 @@ public Builder filter(Expression newFilter) { return this; } + /** + * Configures whether to open parquet merge. + * + * @param newOpenParquetMerge whether to open parquet merge + */ + public Builder openParquetMerge(boolean newOpenParquetMerge) { + this.openParquetMerge = newOpenParquetMerge; + return this; + } + /** * Configures the properties for the rewriter. * @@ -271,7 +282,8 @@ DataStream append(DataStream trigger) { SingleOutputStreamOperator rewritten = planned .rebalance() - .process(new DataFileRewriteRunner(tableName(), taskName(), index())) + .process( + new DataFileRewriteRunner(tableName(), taskName(), index(), openParquetMerge)) .name(operatorName(REWRITE_TASK_NAME)) .uid(REWRITE_TASK_NAME + uidSuffix()) .slotSharingGroup(slotSharingGroup()) diff --git a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DataFileRewriteRunner.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DataFileRewriteRunner.java index 57b0e53d86e6..e956d4a7eec4 100644 --- a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DataFileRewriteRunner.java +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DataFileRewriteRunner.java @@ -20,6 +20,7 @@ import static org.apache.iceberg.TableProperties.DEFAULT_NAME_MAPPING; +import java.io.IOException; import java.util.Collections; import java.util.Set; import org.apache.flink.annotation.Internal; @@ -33,8 +34,11 @@ import org.apache.iceberg.BaseCombinedScanTask; import org.apache.iceberg.DataFile; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.FileScanTask; import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.TableUtil; import org.apache.iceberg.actions.RewriteFileGroup; @@ -45,10 +49,15 @@ import org.apache.iceberg.flink.source.DataIterator; import org.apache.iceberg.flink.source.FileScanTaskReader; import org.apache.iceberg.flink.source.RowDataFileScanTaskReader; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.io.OutputFileFactory; import org.apache.iceberg.io.TaskWriter; +import org.apache.iceberg.parquet.ParquetFileMerger; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +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.PropertyUtil; +import org.apache.parquet.schema.MessageType; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -65,17 +74,20 @@ public class DataFileRewriteRunner private final String tableName; private final String taskName; private final int taskIndex; + private final boolean openParquetMerge; private transient int subTaskId; private transient int attemptId; private transient Counter errorCounter; - public DataFileRewriteRunner(String tableName, String taskName, int taskIndex) { + public DataFileRewriteRunner( + String tableName, String taskName, int taskIndex, boolean openParquetMerge) { Preconditions.checkNotNull(tableName, "Table name should no be null"); Preconditions.checkNotNull(taskName, "Task name should no be null"); this.tableName = tableName; this.taskName = taskName; this.taskIndex = taskIndex; + this.openParquetMerge = openParquetMerge; } @Override @@ -112,6 +124,54 @@ public void processElement(PlannedGroup value, Context ctx, Collector writer = writerFor(value, preserveRowId)) { @@ -175,6 +235,59 @@ public void processElement(PlannedGroup value, Context ctx, Collector !task.deletes().isEmpty()); + if (hasDeletes) { + return null; + } + + // Validate Parquet-specific requirements and get schema + return ParquetFileMerger.canMergeAndGetSchema( + Lists.newArrayList(group.rewrittenFiles()), table.io(), group.maxOutputFileSize()); + } + private TaskWriter writerFor(PlannedGroup value, boolean preserveRowId) { String formatString = PropertyUtil.propertyAsString( diff --git a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestRewriteDataFiles.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestRewriteDataFiles.java index 707038c925d5..3f1fb470bcc6 100644 --- a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestRewriteDataFiles.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/api/TestRewriteDataFiles.java @@ -34,6 +34,9 @@ import org.apache.flink.streaming.api.graph.StreamGraphGenerator; import org.apache.iceberg.ManifestFiles; import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.Schema; import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.Table; @@ -43,9 +46,21 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +@ExtendWith(ParameterizedTestExtension.class) class TestRewriteDataFiles extends MaintenanceTaskTestBase { - @Test + + @Parameter(index = 0) + private boolean openParquetMerge; + + @Parameters(name = "openParquetMerge = {0}") + private static Object[][] parameters() { + return new Object[][] {{true}, {false}}; + } + + @TestTemplate void testRewriteUnpartitioned() throws Exception { Table table = createTable(); insert(table, 1, "a"); @@ -64,6 +79,7 @@ void testRewriteUnpartitioned() throws Exception { .maxFileSizeBytes(2_000_000L) .minFileSizeBytes(500_000L) .minInputFiles(2) + .openParquetMerge(openParquetMerge) .partialProgressEnabled(true) .partialProgressMaxCommits(1) .maxRewriteBytes(100_000L) @@ -82,7 +98,7 @@ void testRewriteUnpartitioned() throws Exception { createRecord(4, "d"))); } - @Test + @TestTemplate void testRewriteUnpartitionedPreserveLineage() throws Exception { Table table = createTable(3); insert(table, 1, "a"); @@ -101,6 +117,7 @@ void testRewriteUnpartitionedPreserveLineage() throws Exception { .maxFileSizeBytes(2_000_000L) .minFileSizeBytes(500_000L) .minInputFiles(2) + .openParquetMerge(openParquetMerge) .partialProgressEnabled(true) .partialProgressMaxCommits(1) .maxRewriteBytes(100_000L) @@ -122,7 +139,7 @@ void testRewriteUnpartitionedPreserveLineage() throws Exception { schema); } - @Test + @TestTemplate void testRewriteTheSameFilePreserveLineage() throws Exception { Table table = createTable(3); insert(table, 1, "a"); @@ -143,6 +160,7 @@ void testRewriteTheSameFilePreserveLineage() throws Exception { .maxFileSizeBytes(2_000_000L) .minFileSizeBytes(500_000L) .minInputFiles(2) + .openParquetMerge(openParquetMerge) .partialProgressEnabled(true) .partialProgressMaxCommits(1) .maxRewriteBytes(100_000L) @@ -166,7 +184,7 @@ void testRewriteTheSameFilePreserveLineage() throws Exception { schema); } - @Test + @TestTemplate void testRewritePartitionedPreserveLineage() throws Exception { Table table = createPartitionedTable(3); insertPartitioned(table, 1, "p1"); @@ -194,7 +212,7 @@ void testRewritePartitionedPreserveLineage() throws Exception { schema); } - @Test + @TestTemplate void testRewritePartitioned() throws Exception { Table table = createPartitionedTable(); insertPartitioned(table, 1, "p1"); @@ -294,7 +312,7 @@ void testPlannerFailure() throws Exception { .build()); } - @Test + @TestTemplate void testUidAndSlotSharingGroup() { createTable(); @@ -316,7 +334,7 @@ void testUidAndSlotSharingGroup() { checkSlotSharingGroupsAreSet(infra.env(), SLOT_SHARING_GROUP); } - @Test + @TestTemplate void testUidAndSlotSharingGroupUnset() { createTable(); @@ -336,7 +354,7 @@ void testUidAndSlotSharingGroupUnset() { checkSlotSharingGroupsAreSet(infra.env(), null); } - @Test + @TestTemplate void testMetrics() throws Exception { Table table = createTable(); insert(table, 1, "a"); @@ -410,7 +428,7 @@ void testMetrics() throws Exception { .build()); } - @Test + @TestTemplate void testV2Table() throws Exception { Table table = createTableWithDelete(); update(table, 1, null, "a", "b"); @@ -489,7 +507,7 @@ void testV2Table() throws Exception { .build()); } - @Test + @TestTemplate void testRewriteWithFilter() throws Exception { Table table = createTable(); insert(table, 1, "a"); @@ -530,7 +548,8 @@ void testRewriteWithFilter() throws Exception { } private void appendRewriteDataFiles() { - appendRewriteDataFiles(RewriteDataFiles.builder().rewriteAll(true)); + appendRewriteDataFiles( + RewriteDataFiles.builder().openParquetMerge(openParquetMerge).rewriteAll(true)); } private void appendRewriteDataFiles(RewriteDataFiles.Builder builder) { diff --git a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/RewriteUtil.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/RewriteUtil.java index 95992ccd979a..ada7a48a688c 100644 --- a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/RewriteUtil.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/RewriteUtil.java @@ -66,13 +66,17 @@ static List planDataFileRewrite( } static List executeRewrite( - List elements) throws Exception { + List elements, boolean openParquetMerge) + throws Exception { try (OneInputStreamOperatorTestHarness< DataFileRewritePlanner.PlannedGroup, DataFileRewriteRunner.ExecutedGroup> testHarness = ProcessFunctionTestHarnesses.forProcessFunction( new DataFileRewriteRunner( - OperatorTestBase.DUMMY_TABLE_NAME, OperatorTestBase.DUMMY_TABLE_NAME, 0))) { + OperatorTestBase.DUMMY_TABLE_NAME, + OperatorTestBase.DUMMY_TABLE_NAME, + 0, + openParquetMerge))) { testHarness.open(); for (DataFileRewritePlanner.PlannedGroup element : elements) { diff --git a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestDataFileRewriteCommitter.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestDataFileRewriteCommitter.java index 9e8f2ec92162..a35721b2ff60 100644 --- a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestDataFileRewriteCommitter.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestDataFileRewriteCommitter.java @@ -31,13 +31,26 @@ import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; import org.apache.iceberg.ContentFile; import org.apache.iceberg.DataFile; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.Table; import org.apache.iceberg.flink.maintenance.api.Trigger; import org.apache.iceberg.relocated.com.google.common.collect.Sets; -import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +@ExtendWith(ParameterizedTestExtension.class) class TestDataFileRewriteCommitter extends OperatorTestBase { - @Test + @Parameter(index = 0) + private boolean openParquetMerge; + + @Parameters(name = "openParquetMerge = {0}") + private static Object[][] parameters() { + return new Object[][] {{false}, {true}}; + } + + @TestTemplate void testUnpartitioned() throws Exception { Table table = createTable(); insert(table, 1, "p1"); @@ -46,7 +59,7 @@ void testUnpartitioned() throws Exception { List planned = planDataFileRewrite(tableLoader()); assertThat(planned).hasSize(1); - List rewritten = executeRewrite(planned); + List rewritten = executeRewrite(planned, openParquetMerge); assertThat(rewritten).hasSize(1); try (OneInputStreamOperatorTestHarness @@ -64,7 +77,7 @@ void testUnpartitioned() throws Exception { table, rewritten.get(0).group().addedFiles(), rewritten.get(0).group().rewrittenFiles(), 1); } - @Test + @TestTemplate void testPartitioned() throws Exception { Table table = createPartitionedTable(); insertPartitioned(table, 1, "p1"); @@ -74,7 +87,7 @@ void testPartitioned() throws Exception { List planned = planDataFileRewrite(tableLoader()); assertThat(planned).hasSize(2); - List rewritten = executeRewrite(planned); + List rewritten = executeRewrite(planned, openParquetMerge); assertThat(rewritten).hasSize(2); assertThat(rewritten.get(0).groupsPerCommit()).isEqualTo(1); assertThat(rewritten.get(1).groupsPerCommit()).isEqualTo(1); @@ -105,7 +118,7 @@ void testPartitioned() throws Exception { } } - @Test + @TestTemplate void testNewTable() throws Exception { Table table = createTable(); List rewritten; @@ -120,7 +133,7 @@ void testNewTable() throws Exception { List planned = planDataFileRewrite(tableLoader()); assertThat(planned).hasSize(1); - rewritten = executeRewrite(planned); + rewritten = executeRewrite(planned, openParquetMerge); assertThat(rewritten).hasSize(1); testHarness.processElement(rewritten.get(0), EVENT_TIME); @@ -134,7 +147,7 @@ void testNewTable() throws Exception { table, rewritten.get(0).group().addedFiles(), rewritten.get(0).group().rewrittenFiles(), 1); } - @Test + @TestTemplate void testBatchSize() throws Exception { Table table = createPartitionedTable(); insertPartitioned(table, 1, "p1"); @@ -146,7 +159,7 @@ void testBatchSize() throws Exception { List planned = planDataFileRewrite(tableLoader()); assertThat(planned).hasSize(3); - List rewritten = executeRewrite(planned); + List rewritten = executeRewrite(planned, openParquetMerge); assertThat(rewritten).hasSize(3); ensureDifferentGroups(rewritten); @@ -178,7 +191,7 @@ void testBatchSize() throws Exception { table, rewritten.get(2).group().addedFiles(), rewritten.get(2).group().rewrittenFiles(), 3); } - @Test + @TestTemplate void testError() throws Exception { Table table = createPartitionedTable(); insertPartitioned(table, 1, "p1"); @@ -192,7 +205,7 @@ void testError() throws Exception { List planned = planDataFileRewrite(tableLoader()); assertThat(planned).hasSize(4); - List rewritten = executeRewrite(planned); + List rewritten = executeRewrite(planned, openParquetMerge); assertThat(rewritten).hasSize(4); try (OneInputStreamOperatorTestHarness diff --git a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestDataFileRewriteRunner.java b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestDataFileRewriteRunner.java index 4e21c7a956e4..8cc9584e60e3 100644 --- a/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestDataFileRewriteRunner.java +++ b/flink/v2.1/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestDataFileRewriteRunner.java @@ -35,6 +35,9 @@ import org.apache.iceberg.DataFile; import org.apache.iceberg.FileContent; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionData; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; @@ -54,34 +57,56 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Sets; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.ValueSource; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +@ExtendWith(ParameterizedTestExtension.class) class TestDataFileRewriteRunner extends OperatorTestBase { - @ParameterizedTest - @ValueSource(booleans = {true, false}) - void testExecute(boolean partitioned) throws Exception { - Table table; - PartitionData partition; - if (partitioned) { - table = createPartitionedTable(); - partition = new PartitionData(table.spec().partitionType()); - partition.set(0, "p1"); - insertPartitioned(table, 1, "p1"); - insertPartitioned(table, 2, "p1"); - insertPartitioned(table, 3, "p1"); - } else { - table = createTable(); - partition = new PartitionData(PartitionSpec.unpartitioned().partitionType()); - insert(table, 1, "p1"); - insert(table, 2, "p1"); - insert(table, 3, "p1"); - } + + @Parameter(index = 0) + private boolean openParquetMerge; + + @Parameters(name = "openParquetMerge = {0}") + private static Object[][] parameters() { + return new Object[][] {{true}, {false}}; + } + + @TestTemplate + void testExecuteUnpartitioned() throws Exception { + Table table = createTable(); + PartitionData partition = new PartitionData(PartitionSpec.unpartitioned().partitionType()); + insert(table, 1, "p1"); + insert(table, 2, "p1"); + insert(table, 3, "p1"); + + List planned = planDataFileRewrite(tableLoader()); + assertThat(planned).hasSize(1); + List actual = executeRewrite(planned, openParquetMerge); + assertThat(actual).hasSize(1); + + assertRewriteFileGroup( + actual.get(0), + table, + records( + table.schema(), + ImmutableSet.of( + ImmutableList.of(1, "p1"), ImmutableList.of(2, "p1"), ImmutableList.of(3, "p1"))), + 1, + ImmutableSet.of(partition)); + } + + @TestTemplate + void testExecutePartitioned() throws Exception { + Table table = createPartitionedTable(); + PartitionData partition = new PartitionData(table.spec().partitionType()); + partition.set(0, "p1"); + insertPartitioned(table, 1, "p1"); + insertPartitioned(table, 2, "p1"); + insertPartitioned(table, 3, "p1"); List planned = planDataFileRewrite(tableLoader()); assertThat(planned).hasSize(1); - List actual = executeRewrite(planned); + List actual = executeRewrite(planned, openParquetMerge); assertThat(actual).hasSize(1); assertRewriteFileGroup( @@ -95,7 +120,7 @@ void testExecute(boolean partitioned) throws Exception { ImmutableSet.of(partition)); } - @Test + @TestTemplate void testPartitionSpecChange() throws Exception { Table table = createPartitionedTable(); insertPartitioned(table, 1, "p1"); @@ -108,7 +133,10 @@ void testPartitionSpecChange() throws Exception { testHarness = ProcessFunctionTestHarnesses.forProcessFunction( new DataFileRewriteRunner( - OperatorTestBase.DUMMY_TABLE_NAME, OperatorTestBase.DUMMY_TABLE_NAME, 0))) { + OperatorTestBase.DUMMY_TABLE_NAME, + OperatorTestBase.DUMMY_TABLE_NAME, + 0, + openParquetMerge))) { testHarness.open(); List planned = planDataFileRewrite(tableLoader()); @@ -196,7 +224,7 @@ void testPartitionSpecChange() throws Exception { } } - @Test + @TestTemplate void testError() throws Exception { Table table = createTable(); insert(table, 1, "a"); @@ -207,7 +235,10 @@ void testError() throws Exception { testHarness = ProcessFunctionTestHarnesses.forProcessFunction( new DataFileRewriteRunner( - OperatorTestBase.DUMMY_TABLE_NAME, OperatorTestBase.DUMMY_TABLE_NAME, 0))) { + OperatorTestBase.DUMMY_TABLE_NAME, + OperatorTestBase.DUMMY_TABLE_NAME, + 0, + openParquetMerge))) { testHarness.open(); List planned = planDataFileRewrite(tableLoader()); @@ -228,7 +259,7 @@ void testError() throws Exception { } } - @Test + @TestTemplate void testV2Table() throws Exception { Table table = createTableWithDelete(); update(table, 1, null, "a", "b"); @@ -237,7 +268,7 @@ void testV2Table() throws Exception { List planned = planDataFileRewrite(tableLoader()); assertThat(planned).hasSize(1); - List actual = executeRewrite(planned); + List actual = executeRewrite(planned, openParquetMerge); assertThat(actual).hasSize(1); assertRewriteFileGroup( @@ -248,7 +279,7 @@ void testV2Table() throws Exception { ImmutableSet.of(new PartitionData(PartitionSpec.unpartitioned().partitionType()))); } - @Test + @TestTemplate void testV3Table() throws Exception { Table table = createTableWithDelete(3); update(table, 1, null, "a", "b", 3); @@ -257,7 +288,7 @@ void testV3Table() throws Exception { List planned = planDataFileRewrite(tableLoader()); assertThat(planned).hasSize(1); - List actual = executeRewrite(planned); + List actual = executeRewrite(planned, openParquetMerge); assertThat(actual).hasSize(1); assertRewriteFileGroup( @@ -268,7 +299,7 @@ void testV3Table() throws Exception { ImmutableSet.of(new PartitionData(PartitionSpec.unpartitioned().partitionType()))); } - @Test + @TestTemplate void testSplitSize() throws Exception { Table table = createTable(); @@ -318,7 +349,7 @@ void testSplitSize() throws Exception { assertThat(planned).hasSize(1); } - List actual = executeRewrite(planned); + List actual = executeRewrite(planned, openParquetMerge); assertThat(actual).hasSize(1); assertRewriteFileGroup( diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetFileMerger.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetFileMerger.java new file mode 100644 index 000000000000..0f8eb9058844 --- /dev/null +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetFileMerger.java @@ -0,0 +1,568 @@ +/* + * 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.parquet; + +import static java.util.Collections.emptyMap; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.List; +import java.util.Map; +import java.util.function.LongUnaryOperator; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.Metrics; +import org.apache.iceberg.MetricsConfig; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.hadoop.HadoopOutputFile; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.io.SeekableInputStream; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Conversions; +import org.apache.iceberg.types.Types.LongType; +import org.apache.parquet.bytes.BytesInput; +import org.apache.parquet.bytes.BytesUtils; +import org.apache.parquet.bytes.DirectByteBufferAllocator; +import org.apache.parquet.bytes.HeapByteBufferAllocator; +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.column.Encoding; +import org.apache.parquet.column.ParquetProperties; +import org.apache.parquet.column.statistics.Statistics; +import org.apache.parquet.column.values.ValuesWriter; +import org.apache.parquet.column.values.delta.DeltaBinaryPackingValuesWriterForLong; +import org.apache.parquet.hadoop.CodecFactory; +import org.apache.parquet.hadoop.ParquetFileReader; +import org.apache.parquet.hadoop.ParquetFileWriter; +import org.apache.parquet.hadoop.ParquetOutputFormat; +import org.apache.parquet.hadoop.metadata.BlockMetaData; +import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; +import org.apache.parquet.hadoop.metadata.CompressionCodecName; +import org.apache.parquet.io.DelegatingSeekableInputStream; +import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.PrimitiveType; +import org.apache.parquet.schema.Type; +import org.apache.parquet.schema.Types; + +/** + * Utility class for performing strict schema validation and merging of Parquet files at the + * row-group level. + */ +public class ParquetFileMerger { + // Default buffer sizes for DeltaBinaryPackingValuesWriter + private static final int DEFAULT_INITIAL_BUFFER_SIZE = 64 * 1024; // 64KB + private static final int DEFAULT_PAGE_SIZE_FOR_ENCODING = 64 * 1024; // 64KB + private static final PrimitiveType ROW_ID_TYPE = + Types.required(PrimitiveType.PrimitiveTypeName.INT64) + .id(MetadataColumns.ROW_ID.fieldId()) + .named(MetadataColumns.ROW_ID.name()); + + private static final PrimitiveType LAST_UPDATED_SEQUENCE_NUMBER_TYPE = + Types.required(PrimitiveType.PrimitiveTypeName.INT64) + .id(MetadataColumns.LAST_UPDATED_SEQUENCE_NUMBER.fieldId()) + .named(MetadataColumns.LAST_UPDATED_SEQUENCE_NUMBER.name()); + + private static final ColumnDescriptor ROW_ID_DESCRIPTOR = + new ColumnDescriptor(new String[] {MetadataColumns.ROW_ID.name()}, ROW_ID_TYPE, 0, 0); + private static final ColumnDescriptor LAST_UPDATED_SEQUENCE_NUMBER_DESCRIPTOR = + new ColumnDescriptor( + new String[] {MetadataColumns.LAST_UPDATED_SEQUENCE_NUMBER.name()}, + LAST_UPDATED_SEQUENCE_NUMBER_TYPE, + 0, + 0); + + private ParquetFileMerger() { + // Utility class - prevent instantiation + } + + /** + * Checks whether the provided DataFiles are eligible for merging and, if successful, returns the + * common Parquet schema. + * + *

This method validates: + * + *

    + *
  • All files must have compatible schemas (identical {@link MessageType}) + *
  • Files must not be encrypted + *
  • All files have the same partition spec + *
  • No files exceed the target output size (not splitting large files) + *
+ * + *

This validation is useful for compaction operations in Spark, Flink, or other engines that + * need to ensure files can be safely merged. The returned MessageType can be passed to {@link + * #binaryMerge} to avoid re-reading the schema. + * + * @param dataFiles List of DataFiles to validate + * @param fileIO FileIO to use for reading files + * @param targetOutputSize Maximum size for output file (files larger than this cannot be merged) + * @return MessageType schema if files can be merged, null otherwise + */ + public static MessageType canMergeAndGetSchema( + List dataFiles, FileIO fileIO, long targetOutputSize) { + Preconditions.checkArgument( + dataFiles != null && !dataFiles.isEmpty(), "dataFiles cannot be null or empty"); + + // Single loop to check partition spec consistency, file sizes, and build InputFile list + int firstSpecId = dataFiles.get(0).specId(); + List inputFiles = Lists.newArrayListWithCapacity(dataFiles.size()); + for (DataFile dataFile : dataFiles) { + if (dataFile.specId() != firstSpecId) { + return null; + } + + if (dataFile.fileSizeInBytes() > targetOutputSize) { + return null; + } + + inputFiles.add(fileIO.newInputFile(dataFile.location())); + } + + return canMergeAndGetSchema(inputFiles); + } + + private static MessageType readSchema(InputFile inputFile) throws IOException { + try (ParquetFileReader reader = ParquetFileReader.open(ParquetIO.file(inputFile))) { + return reader.getFooter().getFileMetaData().getSchema(); + } + } + + /** + * Checks whether all row lineage columns in the given input files are guaranteed to have non-null + * values based on available statistics. + * + * @param inputFiles the files to validate + * @return {@code true} if statistics exist for all row lineage columns and indicate that no null + * values are present; {@code false} otherwise + */ + private static boolean allRowLineageColumnsNonNull(List inputFiles) { + try { + for (InputFile inputFile : inputFiles) { + try (ParquetFileReader reader = ParquetFileReader.open(ParquetIO.file(inputFile))) { + List rowGroups = reader.getFooter().getBlocks(); + + for (BlockMetaData rowGroup : rowGroups) { + for (ColumnChunkMetaData columnChunk : rowGroup.getColumns()) { + String columnPath = columnChunk.getPath().toDotString(); + + if (columnPath.equals(MetadataColumns.ROW_ID.name())) { + Statistics stats = columnChunk.getStatistics(); + if (stats == null || stats.getNumNulls() > 0) { + return false; + } + } + + if (columnPath.equals(MetadataColumns.LAST_UPDATED_SEQUENCE_NUMBER.name())) { + Statistics stats = columnChunk.getStatistics(); + if (stats == null || stats.getNumNulls() > 0) { + return false; + } + } + } + } + } + } + + return true; + } catch (IOException e) { + // If we can't read the file metadata, we can't validate + return false; + } + } + + private static ParquetFileWriter writer( + OutputFile outputFile, MessageType schema, long rowGroupSize, int columnIndexTruncateLength) + throws IOException { + return new ParquetFileWriter( + ParquetIO.file(outputFile), + schema, + ParquetFileWriter.Mode.CREATE, + rowGroupSize, + 0, // maxPaddingSize - hardcoded to 0 (same as ParquetWriter) + columnIndexTruncateLength, + ParquetProperties.DEFAULT_STATISTICS_TRUNCATE_LENGTH, + ParquetProperties.DEFAULT_PAGE_WRITE_CHECKSUM_ENABLED); + } + + /** Internal method to merge files when schema is already known. */ + private static void binaryMerge( + List inputFiles, + OutputFile outputFile, + MessageType schema, + long rowGroupSize, + int columnIndexTruncateLength) + throws IOException { + try (ParquetFileWriter writer = + writer(outputFile, schema, rowGroupSize, columnIndexTruncateLength)) { + + Map extraMetadata = null; + writer.start(); + for (InputFile inputFile : inputFiles) { + try (ParquetFileReader reader = ParquetFileReader.open(ParquetIO.file(inputFile))) { + // Read metadata from the first file + if (extraMetadata == null) { + extraMetadata = reader.getFooter().getFileMetaData().getKeyValueMetaData(); + } + + reader.appendTo(writer); + } + } + + writer.end(extraMetadata != null ? extraMetadata : emptyMap()); + } + } + + /** + * Internal method for merging files when row lineage columns must be generated. Adds both {@code + * _row_id} and {@code _last_updated_sequence_number} to the output file, and populates their + * values using the provided {@code firstRowIds} and {@code dataSequenceNumbers}. + */ + private static void generateRowLineageAndMerge( + List inputFiles, + OutputFile outputFile, + List firstRowIds, + List dataSequenceNumbers, + MessageType baseSchema, + long rowGroupSize, + int columnIndexTruncateLength) + throws IOException { + MessageType extendedSchema = addRowLineageColumns(baseSchema); + + try (ParquetFileWriter writer = + writer(outputFile, extendedSchema, rowGroupSize, columnIndexTruncateLength)) { + + writer.start(); + + Map extraMetadata = null; + + for (int fileIdx = 0; fileIdx < inputFiles.size(); fileIdx++) { + InputFile inputFile = inputFiles.get(fileIdx); + long currentRowId = firstRowIds.get(fileIdx); + long dataSequenceNumber = dataSequenceNumbers.get(fileIdx); + + try (ParquetFileReader reader = ParquetFileReader.open(ParquetIO.file(inputFile))) { + // Read metadata from first file + if (extraMetadata == null) { + extraMetadata = reader.getFooter().getFileMetaData().getKeyValueMetaData(); + } + + List rowGroups = reader.getFooter().getBlocks(); + + for (BlockMetaData rowGroup : rowGroups) { + long rowCount = rowGroup.getRowCount(); + writer.startBlock(rowCount); + + // Copy all existing column chunks (binary copy) + copyColumnChunks(writer, baseSchema, inputFile, rowGroup); + + // Extract compression codec from existing columns to use for row lineage columns + CompressionCodecName codec = rowGroup.getColumns().get(0).getCodec(); + + // Write new _row_id column chunk (DELTA_BINARY_PACKED encoded, then compressed with + // codec) with sequential values: currentRowId, currentRowId+1, currentRowId+2, ... + long startRowId = currentRowId; + writeLongColumnChunk( + writer, + ROW_ID_DESCRIPTOR, + rowCount, + codec, + startRowId, + startRowId + rowCount - 1, + i -> startRowId + i); + currentRowId += rowCount; + + // Write new _last_updated_sequence_number column chunk with constant value for all + // rows: dataSequenceNumber, dataSequenceNumber, ... + writeLongColumnChunk( + writer, + LAST_UPDATED_SEQUENCE_NUMBER_DESCRIPTOR, + rowCount, + codec, + dataSequenceNumber, + dataSequenceNumber, + i -> dataSequenceNumber); + + writer.endBlock(); + } + } + } + + writer.end(extraMetadata != null ? extraMetadata : emptyMap()); + } + } + + /** + * Merges multiple Parquet data files with optional row lineage preservation. + * + *

This method intelligently handles row lineage based on the input files and metadata: + * + *

    + *
  • If row lineage is not needed (null firstRowId/dataSequenceNumber): binary copy merge + *
  • If row lineage is needed AND already present (physical columns exist): binary copy merge + *
  • If row lineage is needed AND not present: synthesizes physical row lineage columns from + * DataFile metadata + *
+ * + *

All input files must satisfy the conditions verified by {@link #canMergeAndGetSchema(List, + * FileIO, long)}. The {@code schema} parameter should also be obtained from this method to + * prevent redundant file reads. + * + * @param dataFiles List of Iceberg DataFiles to merge + * @param fileIO FileIO to use for reading input files + * @param outputFile Output file for the merged result + * @param schema Parquet schema from {@link #canMergeAndGetSchema(List, FileIO, long)} + * @param rowGroupSize Target row group size in bytes + * @param spec PartitionSpec for the output file + * @param partition Partition data for the output file (null for unpartitioned tables) + * @return DataFile representing the merged output file with complete metadata + * @throws IOException if I/O error occurs during merge operation + */ + public static DataFile mergeFiles( + List dataFiles, + FileIO fileIO, + OutputFile outputFile, + MessageType schema, + long rowGroupSize, + PartitionSpec spec, + StructLike partition) + throws IOException { + // Convert DataFiles to InputFiles and extract row lineage metadata + List inputFiles = Lists.newArrayListWithCapacity(dataFiles.size()); + List firstRowIds = Lists.newArrayListWithCapacity(dataFiles.size()); + List dataSequenceNumbers = Lists.newArrayListWithCapacity(dataFiles.size()); + boolean hasRowLineage = false; + + for (DataFile dataFile : dataFiles) { + inputFiles.add(fileIO.newInputFile(dataFile.location())); + firstRowIds.add(dataFile.firstRowId()); + dataSequenceNumbers.add(dataFile.dataSequenceNumber()); + + if (dataFile.firstRowId() != null && dataFile.dataSequenceNumber() != null) { + hasRowLineage = true; + } + } + + // Initialize columnIndexTruncateLength following the same pattern as Parquet.java + Configuration conf = + outputFile instanceof HadoopOutputFile + ? new Configuration(((HadoopOutputFile) outputFile).getConf()) + : new Configuration(); + int columnIndexTruncateLength = + conf.getInt( + ParquetOutputFormat.COLUMN_INDEX_TRUNCATE_LENGTH, + ParquetProperties.DEFAULT_COLUMN_INDEX_TRUNCATE_LENGTH); + + // Check if we need to synthesize physical row lineage columns from virtual metadata + boolean shouldSynthesizeRowLineage = + hasRowLineage + && !schema.containsField(MetadataColumns.ROW_ID.name()) + && !schema.containsField(MetadataColumns.LAST_UPDATED_SEQUENCE_NUMBER.name()); + + if (shouldSynthesizeRowLineage) { + // Files have virtual row lineage - synthesize physical columns + generateRowLineageAndMerge( + inputFiles, + outputFile, + firstRowIds, + dataSequenceNumbers, + schema, + rowGroupSize, + columnIndexTruncateLength); + } else { + // Use simple binary copy (either no row lineage, or files already have physical columns) + binaryMerge(inputFiles, outputFile, schema, rowGroupSize, columnIndexTruncateLength); + } + + InputFile compactedFile = fileIO.newInputFile(outputFile.location()); + Metrics metrics = ParquetUtil.fileMetrics(compactedFile, MetricsConfig.getDefault()); + + DataFiles.Builder builder = + DataFiles.builder(spec) + .withPath(compactedFile.location()) + .withFormat(FileFormat.PARQUET) + .withFileSizeInBytes(compactedFile.getLength()) + .withMetrics(metrics); + + if (partition != null) { + builder.withPartition(partition); + } + + // Extract firstRowId from Parquet column statistics (for V3+ tables with row lineage) + // The min value of _row_id column becomes firstRowId + if (metrics.lowerBounds() != null) { + ByteBuffer rowIdLowerBound = metrics.lowerBounds().get(MetadataColumns.ROW_ID.fieldId()); + if (rowIdLowerBound != null) { + Long firstRowId = Conversions.fromByteBuffer(LongType.get(), rowIdLowerBound); + builder.withFirstRowId(firstRowId); + } + } + + return builder.build(); + } + + @VisibleForTesting + static MessageType canMergeAndGetSchema(List inputFiles) { + Preconditions.checkArgument( + inputFiles != null && !inputFiles.isEmpty(), "inputFiles cannot be null or empty"); + + try { + MessageType firstSchema = readSchema(inputFiles.get(0)); + + for (int i = 1; i < inputFiles.size(); i++) { + MessageType currentSchema = readSchema(inputFiles.get(i)); + if (!firstSchema.equals(currentSchema)) { + return null; + } + } + + boolean hasPhysicalRowLineageColumns = + firstSchema.containsField(MetadataColumns.ROW_ID.name()) + || firstSchema.containsField(MetadataColumns.LAST_UPDATED_SEQUENCE_NUMBER.name()); + + if (hasPhysicalRowLineageColumns && !allRowLineageColumnsNonNull(inputFiles)) { + return null; + } + + return firstSchema; + } catch (RuntimeException | IOException e) { + // Returns null for: + // - Non-Parquet files (IOException when reading Parquet footer) + // - Encrypted files (ParquetCryptoRuntimeException extends RuntimeException) + // - Any other validation failures + return null; + } + } + + /** + * Extends a Parquet schema by adding the row lineage metadata columns: _row_id, + * _last_updated_sequence_number. + */ + private static MessageType addRowLineageColumns(MessageType baseSchema) { + List fields = Lists.newArrayList(baseSchema.getFields()); + fields.add(ROW_ID_TYPE); + fields.add(LAST_UPDATED_SEQUENCE_NUMBER_TYPE); + + return new MessageType(baseSchema.getName(), fields); + } + + /** + * Utility helper to write a long column chunk with DELTA_BINARY_PACKED encoding. The encoded data + * is then compressed using the specified codec. This method handles the common pattern of writing + * {@code _row_id} and {@code _last_updated_sequence_number} columns. + * + * @param writer ParquetFileWriter to write to + * @param descriptor Column descriptor for the column + * @param rowCount Number of rows in this row group + * @param codec Compression codec to use (should match the file's codec) + * @param minValue Minimum value for statistics + * @param maxValue Maximum value for statistics + * @param valueGenerator Function that generates the value for row i (0-based index) + * @throws IOException if writing fails + */ + private static void writeLongColumnChunk( + ParquetFileWriter writer, + ColumnDescriptor descriptor, + long rowCount, + CompressionCodecName codec, + long minValue, + long maxValue, + LongUnaryOperator valueGenerator) + throws IOException { + + writer.startColumn(descriptor, rowCount, codec); + + int uncompressedSize; + BytesInput compressedData; + + try (ValuesWriter valuesWriter = + new DeltaBinaryPackingValuesWriterForLong( + DEFAULT_INITIAL_BUFFER_SIZE, + DEFAULT_PAGE_SIZE_FOR_ENCODING, + HeapByteBufferAllocator.getInstance())) { + + for (long i = 0; i < rowCount; i++) { + valuesWriter.writeLong(valueGenerator.applyAsLong(i)); + } + + BytesInput encodedData = valuesWriter.getBytes(); + uncompressedSize = (int) encodedData.size(); + + if (codec != CompressionCodecName.UNCOMPRESSED) { + CodecFactory codecFactory = + CodecFactory.createDirectCodecFactory( + new Configuration(), DirectByteBufferAllocator.getInstance(), 0); + compressedData = codecFactory.getCompressor(codec).compress(encodedData); + } else { + compressedData = encodedData; + } + } + + Statistics stats = + Statistics.getBuilderForReading(descriptor.getPrimitiveType()) + .withMax(BytesUtils.longToBytes(maxValue)) + .withMin(BytesUtils.longToBytes(minValue)) + .withNumNulls(0) + .build(); + + // For required column (no nulls), we don't need repetition/definition level encoding + writer.writeDataPage( + (int) rowCount, + uncompressedSize, + compressedData, + stats, + rowCount, + Encoding.RLE, + Encoding.RLE, + Encoding.DELTA_BINARY_PACKED); + + writer.endColumn(); + } + + /** Copies all column chunks from a row group using binary copy. */ + private static void copyColumnChunks( + ParquetFileWriter writer, MessageType baseSchema, InputFile inputFile, BlockMetaData rowGroup) + throws IOException { + try (SeekableInputStream icebergStream = inputFile.newStream()) { + org.apache.parquet.io.SeekableInputStream parquetStream = + new DelegatingSeekableInputStream(icebergStream) { + @Override + public long getPos() throws IOException { + return icebergStream.getPos(); + } + + @Override + public void seek(long newPos) throws IOException { + icebergStream.seek(newPos); + } + }; + + for (ColumnChunkMetaData columnChunk : rowGroup.getColumns()) { + ColumnDescriptor columnDescriptor = + baseSchema.getColumnDescription(columnChunk.getPath().toArray()); + writer.appendColumnChunk(columnDescriptor, parquetStream, columnChunk, null, null, null); + } + } + } +} diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetWriter.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetWriter.java index e31df97c2bad..c126acd48595 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetWriter.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetWriter.java @@ -39,6 +39,7 @@ import org.apache.parquet.hadoop.CodecFactory; import org.apache.parquet.hadoop.ColumnChunkPageWriteStore; import org.apache.parquet.hadoop.ParquetFileWriter; +import org.apache.parquet.hadoop.ParquetOutputFormat; import org.apache.parquet.hadoop.metadata.CompressionCodecName; import org.apache.parquet.schema.MessageType; @@ -68,9 +69,6 @@ class ParquetWriter implements FileAppender, Closeable { private ParquetFileWriter writer; private int rowGroupOrdinal; - private static final String COLUMN_INDEX_TRUNCATE_LENGTH = "parquet.columnindex.truncate.length"; - private static final int DEFAULT_COLUMN_INDEX_TRUNCATE_LENGTH = 64; - @SuppressWarnings("unchecked") ParquetWriter( Configuration conf, @@ -95,7 +93,9 @@ class ParquetWriter implements FileAppender, Closeable { this.model = (ParquetValueWriter) createWriterFunc.apply(schema, parquetSchema); this.metricsConfig = metricsConfig; this.columnIndexTruncateLength = - conf.getInt(COLUMN_INDEX_TRUNCATE_LENGTH, DEFAULT_COLUMN_INDEX_TRUNCATE_LENGTH); + conf.getInt( + ParquetOutputFormat.COLUMN_INDEX_TRUNCATE_LENGTH, + ParquetProperties.DEFAULT_COLUMN_INDEX_TRUNCATE_LENGTH); this.writeMode = writeMode; this.output = output; this.conf = conf; From 3e013a32f363f39947a7648afddab0110cd0ad53 Mon Sep 17 00:00:00 2001 From: Guosmilesmile <511955993@qq.com> Date: Wed, 14 Jan 2026 14:26:11 +0800 Subject: [PATCH 2/4] add partition check --- .../flink/maintenance/operator/DataFileRewriteRunner.java | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DataFileRewriteRunner.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DataFileRewriteRunner.java index e956d4a7eec4..204252600567 100644 --- a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DataFileRewriteRunner.java +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DataFileRewriteRunner.java @@ -283,6 +283,12 @@ private MessageType canMergeAndGetSchema(RewriteFileGroup group, Table table) { return null; } + boolean allTheSamePartition = + group.rewrittenFiles().stream().anyMatch(file -> file.specId() != table.spec().specId()); + if (allTheSamePartition) { + return null; + } + // Validate Parquet-specific requirements and get schema return ParquetFileMerger.canMergeAndGetSchema( Lists.newArrayList(group.rewrittenFiles()), table.io(), group.maxOutputFileSize()); From a0cf0128cbeb9dc3aef596f60b58367bd46aa634 Mon Sep 17 00:00:00 2001 From: Guosmilesmile <511955993@qq.com> Date: Wed, 14 Jan 2026 15:08:47 +0800 Subject: [PATCH 3/4] add doc --- docs/docs/flink-maintenance.md | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/docs/docs/flink-maintenance.md b/docs/docs/flink-maintenance.md index 37fb9a7a84c4..82f259963b2e 100644 --- a/docs/docs/flink-maintenance.md +++ b/docs/docs/flink-maintenance.md @@ -219,6 +219,7 @@ env.execute("Table Maintenance Job"); | `maxRewriteBytes(long)` | Maximum bytes to rewrite per execution | Long.MAX_VALUE | long | | `filter(Expression)` | Filter expression for selecting files to rewrite | Expressions.alwaysTrue() | Expression | | `maxFileGroupInputFiles(long)` | Maximum allowed number of input files within a file group | Long.MAX_VALUE | long | +| `openParquetMerge(boolean)` | For Parquet tables, `rewriteDataFiles` can use an optimized row-group level merge strategy that is significantly faster than the standard read-rewrite approach. This optimization directly copies row groups without deserialization and re-serialization. | false | boolean | #### DeleteOrphanFiles Configuration @@ -398,6 +399,12 @@ These keys are used in SQL (SET or table WITH options) and are applicable when w - Enable `partialProgressEnabled` for large rewrite operations - Set reasonable `maxRewriteBytes` limits - Setting an appropriate `maxFileGroupSizeBytes` can break down large FileGroups into smaller ones, thereby increasing the speed of parallel processing +- For Parquet tables, `rewriteDataFiles` can open parquet merge, use an optimized row-group level merge strategy that is significantly faster than the standard read-rewrite approach. This optimization is applied when the following requirements are met: +- * All files are in Parquet format +- * Files have compatible schemas +- * Files are not encrypted +- * Files do not have associated delete files or delete vectors +- * Table does not have a sort order (including z-ordered tables) ### Troubleshooting From d1717734e9aa1b41d9cee21abfb1467c4d3bc536 Mon Sep 17 00:00:00 2001 From: Guosmilesmile <511955993@qq.com> Date: Thu, 22 Jan 2026 21:40:01 +0800 Subject: [PATCH 4/4] change table spec to group output spec --- .../flink/maintenance/operator/DataFileRewriteRunner.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DataFileRewriteRunner.java b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DataFileRewriteRunner.java index 204252600567..3d50b0c7e678 100644 --- a/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DataFileRewriteRunner.java +++ b/flink/v2.1/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/DataFileRewriteRunner.java @@ -284,7 +284,7 @@ private MessageType canMergeAndGetSchema(RewriteFileGroup group, Table table) { } boolean allTheSamePartition = - group.rewrittenFiles().stream().anyMatch(file -> file.specId() != table.spec().specId()); + group.rewrittenFiles().stream().anyMatch(file -> file.specId() != group.outputSpecId()); if (allTheSamePartition) { return null; }