shangxinli commented on code in PR #14435: URL: https://github.com/apache/iceberg/pull/14435#discussion_r2585421074
########## spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SparkParquetFileMergeRunner.java: ########## @@ -0,0 +1,404 @@ +/* + * 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.io.Serializable; +import java.nio.ByteBuffer; +import java.util.List; +import java.util.stream.Collectors; +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.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.InputFile; +import org.apache.iceberg.io.OutputFileFactory; +import org.apache.iceberg.parquet.ParquetFileMerger; +import org.apache.iceberg.parquet.ParquetUtil; +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.types.Conversions; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.PropertyUtil; +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); + } + } + + /** + * Validates if the file group can use row-group level merging. + * + * <p>Requirements checked here: + * + * <ul> + * <li>Group must expect exactly 1 output file (ParquetFileMerger cannot split into multiple + * files) + * <li>Table must not have a sort order (no sorting or z-ordering) + * <li>Files must not have delete files or delete vectors + * <li>Partition spec must not be changing (row-group merge cannot repartition data) + * <li>Must not be splitting large files (row-group merge is optimized for merging, not + * splitting) + * </ul> + * + * <p>Additional requirements checked by ParquetFileMerger.canMerge: + * + * <ul> + * <li>All files must be valid Parquet format + * <li>Files must have compatible schemas + * <li>Files must not be encrypted + * <li>If a physical _row_id column exists, all values must be non-null + * </ul> + * + * <p>To ensure groups produce single output files, configure: {@code + * rewrite-data-files.max-file-group-size-bytes = rewrite-data-files.target-file-size-bytes} + * + * @param group the file group to check + * @return true if row-group merging can be used, false otherwise + */ + @VisibleForTesting + boolean canMerge(RewriteFileGroup group) { + // Check if group expects exactly one output file + // ParquetFileMerger only supports merging to a single output file + if (group.expectedOutputFiles() != 1) { + LOG.debug( + "Cannot use row-group merge: group expects {} output files but ParquetFileMerger " + + "only supports 1 output file. Configure " + + "rewrite-data-files.max-file-group-size-bytes = rewrite-data-files.target-file-size-bytes " + + "to ensure groups produce single output files.", + group.expectedOutputFiles()); + return false; + } + + // Check if table has a sort order - row-group merge cannot preserve sort order + if (table().sortOrder().isSorted()) { + LOG.debug( + "Cannot use row-group merge: table has a sort order ({}). " + + "Row-group merging would not preserve the sort order.", + table().sortOrder()); + return false; + } + + // Check for delete files - row-group merge cannot apply deletes + boolean hasDeletes = group.fileScanTasks().stream().anyMatch(task -> !task.deletes().isEmpty()); + + if (hasDeletes) { + LOG.debug( + "Cannot use row-group merge: files have delete files or delete vectors. " + + "Row-group merging cannot apply deletes."); + return false; + } + + // Check if partition spec is being changed - row-group merge cannot repartition data + int outputSpecId = group.outputSpecId(); + boolean hasPartitionSpecChange = + group.rewrittenFiles().stream().anyMatch(file -> file.specId() != outputSpecId); + + if (hasPartitionSpecChange) { + LOG.debug( + "Cannot use row-group merge: partition spec is changing. " + + "Row-group merging cannot change data partitioning."); + return false; + } + + // Check if we're splitting large files - row-group merge is designed for file merging, not + // splitting + long maxOutputFileSize = group.maxOutputFileSize(); + boolean isSplittingFiles = + group.rewrittenFiles().stream() + .anyMatch(file -> file.fileSizeInBytes() > maxOutputFileSize); + + if (isSplittingFiles) { + LOG.debug( + "Cannot use row-group merge: compaction is splitting large files. " + + "Row-group merging is optimized for merging files, not splitting them."); + return false; + } + + // Validate schema compatibility and other Parquet-specific requirements + try { + List<InputFile> inputFiles = + group.rewrittenFiles().stream() + .map(f -> table().io().newInputFile(f.path().toString())) + .collect(Collectors.toList()); + + // Validate files can be merged + boolean canMerge = ParquetFileMerger.canMerge(inputFiles); + + if (!canMerge) { + LOG.info( + "Cannot use row-group merge for {} files. Falling back to standard rewrite. " + + "Reason: Parquet validation failed", + group.rewrittenFiles().size()); + } + + return canMerge; + } catch (Exception e) { + LOG.info("Cannot use row-group merge: validation failed", e); + return false; + } + } + + /** + * 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 row group size from table properties + long rowGroupSize = + PropertyUtil.propertyAsLong( + table().properties(), + TableProperties.PARQUET_ROW_GROUP_SIZE_BYTES, + TableProperties.PARQUET_ROW_GROUP_SIZE_BYTES_DEFAULT); + + // Create single merge task with all files + List<DataFile> dataFiles = Lists.newArrayList(group.rewrittenFiles()); + MergeTaskInfo mergeTask = new MergeTaskInfo(dataFiles, rowGroupSize, 0, partition, spec); + + // Get FileIO for executors - table().io() is serializable + FileIO fileIO = table().io(); + + // Serialize table for executors (needed to create OutputFileFactory) + Table serializableTable = table(); + + // Execute merge on an executor + JavaSparkContext jsc = JavaSparkContext.fromSparkContext(spark().sparkContext()); + JavaRDD<MergeTaskInfo> taskRDD = jsc.parallelize(Lists.newArrayList(mergeTask), 1); + MergeResult mergeResult = + taskRDD.map(task -> mergeFilesForTask(task, fileIO, serializableTable)).collect().get(0); + + // Driver constructs DataFile from metadata + MetricsConfig metricsConfig = MetricsConfig.getDefault(); + Metrics metrics = + ParquetUtil.fileMetrics(table().io().newInputFile(mergeResult.path()), metricsConfig); + + DataFiles.Builder builder = + DataFiles.builder(spec) + .withPath(mergeResult.path()) + .withFormat(FileFormat.PARQUET) + .withPartition(partition) + .withFileSizeInBytes(mergeResult.fileSize()) + .withMetrics(metrics); + + // Extract firstRowId from Parquet column statistics (same as binpack approach) + // 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(Types.LongType.get(), rowIdLowerBound); + builder.withFirstRowId(firstRowId); + } + } + + DataFile newFile = builder.build(); + + // 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. Returns only metadata + * (file path and size); DataFile construction happens on the driver. + * + * <p>IMPORTANT: OutputFileFactory is created here on the executor (not serialized from driver) + * using TaskContext.taskAttemptId() to ensure unique filenames across task retry attempts. + */ + private static MergeResult mergeFilesForTask(MergeTaskInfo task, FileIO fileIO, Table table) Review Comment: Good catch! You're right - we don't need MergeTaskInfo. I've simplified this by removing the wrapper class. All parameters are now explicitly visible in the method signature: private static MergeResult mergeFilesForTask( List<DataFile> dataFiles, long rowGroupSize, StructLike partition, PartitionSpec spec, FileIO fileIO, Table table) -- 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]
