shangxinli commented on code in PR #14435: URL: https://github.com/apache/iceberg/pull/14435#discussion_r2604248262
########## spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SparkParquetFileMergeRunner.java: ########## @@ -0,0 +1,203 @@ +/* + * 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.spark.actions; + +import java.io.IOException; +import java.util.List; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.actions.RewriteFileGroup; +import org.apache.iceberg.encryption.EncryptedOutputFile; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.io.OutputFileFactory; +import org.apache.iceberg.parquet.ParquetFileMerger; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.spark.FileRewriteCoordinator; +import org.apache.iceberg.util.PropertyUtil; +import org.apache.parquet.schema.MessageType; +import org.apache.spark.TaskContext; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.SparkSession; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Extension of SparkBinPackFileRewriteRunner that uses ParquetFileMerger for efficient row-group + * level merging of Parquet files when applicable. + * + * <p>This runner uses {@link ParquetFileMerger} to merge Parquet files at the row-group level + * without full deserialization, which is significantly faster than the standard Spark rewrite + * approach for Parquet files. + * + * <p>The decision to use this runner vs. SparkBinPackFileRewriteRunner is controlled by the + * configuration option {@code use-parquet-file-merger}. + */ +public class SparkParquetFileMergeRunner extends SparkBinPackFileRewriteRunner { + private static final Logger LOG = LoggerFactory.getLogger(SparkParquetFileMergeRunner.class); + private final FileRewriteCoordinator coordinator = FileRewriteCoordinator.get(); + + public SparkParquetFileMergeRunner(SparkSession spark, Table table) { + super(spark, table); + } + + @Override + public String description() { + return "PARQUET-MERGE"; + } + + @Override + protected void doRewrite(String groupId, RewriteFileGroup group) { + // Early validation: check if requirements are met + if (!canMerge(group)) { + LOG.info( + "Row-group merge requirements not met for group {}. Using standard Spark rewrite.", + groupId); + super.doRewrite(groupId, group); + return; + } + + // Requirements met - attempt row-group level merge + try { + LOG.info( + "Merging {} Parquet files using row-group level merge (group: {})", + group.rewrittenFiles().size(), + groupId); + mergeParquetFilesDistributed(groupId, group); + } catch (Exception e) { + LOG.info( + "Row-group merge failed for group {}, falling back to standard Spark rewrite: {}", + groupId, + e.getMessage(), + e); + // Fallback to standard rewrite + super.doRewrite(groupId, group); + } + } + + @VisibleForTesting + boolean canMerge(RewriteFileGroup group) { + return ParquetFileMerger.canMerge(group, table().sortOrder(), table().io()); + } + + /** + * Merges all input files in a group into a single output file. + * + * <p>This method assumes the group has been validated by {@link #canMerge(RewriteFileGroup)} to + * have exactly one expected output file. + */ + private void mergeParquetFilesDistributed(String groupId, RewriteFileGroup group) { + PartitionSpec spec = table().specs().get(group.outputSpecId()); + StructLike partition = group.info().partition(); + + LOG.info( + "Merging {} Parquet files into 1 output file (group: {})", + group.rewrittenFiles().size(), + groupId); + + // Get parameters needed for merge + long rowGroupSize = + PropertyUtil.propertyAsLong( + table().properties(), + TableProperties.PARQUET_ROW_GROUP_SIZE_BYTES, + TableProperties.PARQUET_ROW_GROUP_SIZE_BYTES_DEFAULT); + List<DataFile> dataFiles = Lists.newArrayList(group.rewrittenFiles()); + FileIO fileIO = table().io(); + Table serializableTable = table(); + + // Execute merge on an executor + // Use parallelize with a single dummy element to run the merge on an executor + JavaSparkContext jsc = JavaSparkContext.fromSparkContext(spark().sparkContext()); + JavaRDD<Integer> taskRDD = jsc.parallelize(Lists.newArrayList(1), 1); + DataFile newFile = + taskRDD + .map( + ignored -> + mergeFilesForTask( + dataFiles, rowGroupSize, partition, spec, fileIO, serializableTable)) + .collect() + .get(0); + + // Register merged file with coordinator + coordinator.stageRewrite(table(), groupId, Sets.newHashSet(newFile)); + + LOG.info( + "Successfully merged {} Parquet files into 1 output file (group: {})", + group.rewrittenFiles().size(), + groupId); + } + + /** + * Performs the actual merge operation for a single task on an executor. + * + * <p>IMPORTANT: OutputFileFactory is created here on the executor (not serialized from driver) + * using TaskContext.taskAttemptId() to ensure unique filenames across task retry attempts. + * + * <p>This method first validates and gets the schema using {@link + * ParquetFileMerger#canMergeAndGetMessageType}, then passes it to {@link + * ParquetFileMerger#mergeFiles} to avoid redundant file reads. + */ + private static DataFile mergeFilesForTask( + List<DataFile> dataFiles, + long rowGroupSize, + StructLike partition, + PartitionSpec spec, + FileIO fileIO, + Table table) + throws IOException { + // Get schema and validate (this reads the first file once) + MessageType schema = + ParquetFileMerger.canMergeAndGetMessageType( + dataFiles, fileIO, Long.MAX_VALUE /* no size check here */); Review Comment: Just did a refactoring of the code. Here are what the changes. 1. Removed canMerge() method - Completely removed from both ParquetFileMerger and SparkParquetFileMergeRunner. No more canMerge() anywhere in the codebase. 2. Unified API with canMergeAndGetSchema() - All validation methods now use the single naming convention canMergeAndGetSchema(): - ParquetFileMerger.canMergeAndGetSchema(List<DataFile>, FileIO, long) - public API for file-level validation (partition specs, file sizes, Parquet schema) - SparkParquetFileMergeRunner.canMergeAndGetSchema(RewriteFileGroup) - private method for group-level validation (expectedOutputFiles, sortOrder, deletes) 3. The MessageType schema is returned from validation and passed down to executor tasks to avoid reopening the file for getting schema. ########## spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SparkParquetFileMergeRunner.java: ########## @@ -0,0 +1,234 @@ +/* + * 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.spark.actions; + +import java.io.IOException; +import java.util.List; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.actions.RewriteFileGroup; +import org.apache.iceberg.encryption.EncryptedOutputFile; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.io.OutputFileFactory; +import org.apache.iceberg.parquet.ParquetFileMerger; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.spark.FileRewriteCoordinator; +import org.apache.iceberg.util.PropertyUtil; +import org.apache.parquet.schema.MessageType; +import org.apache.spark.TaskContext; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.SparkSession; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Extension of SparkBinPackFileRewriteRunner that uses ParquetFileMerger for efficient row-group + * level merging of Parquet files when applicable. + * + * <p>This runner uses {@link ParquetFileMerger} to merge Parquet files at the row-group level + * without full deserialization, which is significantly faster than the standard Spark rewrite + * approach for Parquet files. + * + * <p>The decision to use this runner vs. SparkBinPackFileRewriteRunner is controlled by the + * configuration option {@code use-parquet-file-merger}. + */ +public class SparkParquetFileMergeRunner extends SparkBinPackFileRewriteRunner { + private static final Logger LOG = LoggerFactory.getLogger(SparkParquetFileMergeRunner.class); + private final FileRewriteCoordinator coordinator = FileRewriteCoordinator.get(); + + public SparkParquetFileMergeRunner(SparkSession spark, Table table) { + super(spark, table); + } + + @Override + public String description() { + return "PARQUET-MERGE"; + } + + @Override + protected void doRewrite(String groupId, RewriteFileGroup group) { + // Early validation: check if requirements are met and get schema + MessageType schema = canMergeAndGetSchema(group); + if (schema == null) { + LOG.info( + "Row-group merge requirements not met for group {}. Using standard Spark rewrite.", + groupId); + super.doRewrite(groupId, group); + return; + } + + // Requirements met - attempt row-group level merge + try { + LOG.info( + "Merging {} Parquet files using row-group level merge (group: {})", + group.rewrittenFiles().size(), + groupId); + mergeParquetFilesDistributed(groupId, group, schema); + } catch (Exception e) { + LOG.info( + "Row-group merge failed for group {}, falling back to standard Spark rewrite: {}", + groupId, + e.getMessage(), + e); + // Fallback to standard rewrite + super.doRewrite(groupId, group); + } + } + + @VisibleForTesting + boolean canMerge(RewriteFileGroup group) { + return canMergeAndGetSchema(group) != null; + } + + /** + * Validates if a group can be merged and returns the Parquet schema if successful. + * + * <p>This method checks all requirements for row-group merging and returns the schema if all + * checks pass. The returned schema can be reused to avoid redundant file reads. + * + * @param group the file group to validate + * @return MessageType schema if files can be merged, null otherwise + */ + private MessageType canMergeAndGetSchema(RewriteFileGroup group) { Review Comment: Addressed in above comment. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
