Guosmilesmile commented on code in PR #14435: URL: https://github.com/apache/iceberg/pull/14435#discussion_r2588330120
########## spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SparkParquetFileMergeRunner.java: ########## @@ -0,0 +1,271 @@ +/* + * 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.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 (checked via file scan tasks) + * </ul> + * + * <p>Additional requirements checked by {@link ParquetFileMerger#canMerge(List, FileIO, long)}: + * + * <ul> + * <li>All files must be valid Parquet format + * <li>Files must have compatible schemas + * <li>Files must not be encrypted + * <li>Files must not have associated delete files (via DataFile metadata) + * <li>All files must have the same partition spec + * <li>Files must not exceed target size (not splitting large files) + * <li>If 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) { Review Comment: Yes - Flink also needs to verify these things, for example whether all files are Parquet, whether the output is a single file, whether there are delete files, etc. So we should move the duplicate code into ParquetFileMerger. -- 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]
