pvary commented on code in PR #14435: URL: https://github.com/apache/iceberg/pull/14435#discussion_r2559729479
########## spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/actions/SparkParquetFileMergeRunner.java: ########## @@ -0,0 +1,474 @@ +/* + * 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.Set; +import java.util.stream.Collectors; +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.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.TableUtil; +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.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.parquet.crypto.ParquetCryptoRuntimeException; +import org.apache.parquet.schema.MessageType; +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 = validateAndGetSchema(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.warn( + "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 and returns the schema. + * + * <p>Requirements: + * + * <ul> + * <li>All files must be Parquet format + * <li>Table must not have a sort order (no sorting or z-ordering) + * <li>Files must not have delete files or delete vectors + * <li>Files must have compatible schemas (verified by ParquetFileMerger.readAndValidateSchema) + * <li>Files must not be encrypted (detected by ParquetCryptoRuntimeException) + * </ul> + * + * @param group the file group to check + * @return the Parquet schema if row-group merging can be used, null otherwise + */ + private MessageType validateAndGetSchema(RewriteFileGroup group) { + // Check if all files are Parquet format + boolean allParquet = + group.rewrittenFiles().stream().allMatch(file -> file.format() == FileFormat.PARQUET); + + if (!allParquet) { + LOG.debug("Cannot use row-group merge: not all files are Parquet format"); + return null; + } + + // 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 null; + } + + // 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 null; + } + + // Validate schema compatibility and check for encryption using Iceberg InputFile API + try { + List<InputFile> inputFiles = + group.rewrittenFiles().stream() + .map(f -> table().io().newInputFile(f.path().toString())) + .collect(Collectors.toList()); + + // Validate files can be merged (returns schema if valid, null otherwise) + MessageType schema = ParquetFileMerger.readAndValidateSchema(inputFiles); + + if (schema == null) { + LOG.warn( + "Cannot use row-group merge for {} files. Falling back to standard rewrite. " + + "Reason: Schema validation failed", + group.rewrittenFiles().size()); + } + + return schema; + } catch (ParquetCryptoRuntimeException e) { + // ParquetFileReader.open() throws this when trying to read an encrypted footer without keys. + // This happens in ParquetFileMerger.readAndValidateSchema() when validating schemas. + // Exception message: "Trying to read file with encrypted footer. No keys available" + LOG.debug("Cannot use row-group merge: encrypted files detected", e); + return null; + } catch (Exception e) { + LOG.warn("Cannot use row-group merge: validation failed", e); + return null; + } + } + + /** + * Merges Parquet files in the group, respecting the expected output file count determined by the + * planner. Files are distributed evenly across the expected number of output files. + */ + private void mergeParquetFilesDistributed( + String groupId, RewriteFileGroup group, MessageType schema) { + PartitionSpec spec = table().specs().get(group.outputSpecId()); + StructLike partition = group.info().partition(); + long maxOutputFileSize = group.maxOutputFileSize(); + int expectedOutputFiles = group.expectedOutputFiles(); + + LOG.info( + "Merging {} Parquet files into {} expected output files (group: {})", + group.rewrittenFiles().size(), + expectedOutputFiles, + groupId); + + // Check if table supports row lineage + boolean preserveRowLineage = TableUtil.supportsRowLineage(table()); + + // Distribute files evenly across expected output files (planner already determined the count) + List<List<DataFile>> fileBatches = + distributeFilesEvenly(group.rewrittenFiles(), expectedOutputFiles); + + // 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); + + // Get column index truncate length from Hadoop Configuration (same as ParquetWriter) + Configuration hadoopConf = spark().sessionState().newHadoopConf(); + int columnIndexTruncateLength = hadoopConf.getInt("parquet.columnindex.truncate.length", 64); + + // Create OutputFileFactory for generating output files with proper naming + OutputFileFactory fileFactory = + OutputFileFactory.builderFor(table(), spec.specId(), 1).format(FileFormat.PARQUET).build(); Review Comment: We should create a different `OutputFileFactory` for every Executor node to ensure there are no duplication. In this case we don't have to magically create `taskId`s -- 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]
