pvary commented on code in PR #14435: URL: https://github.com/apache/iceberg/pull/14435#discussion_r2556270586
########## parquet/src/main/java/org/apache/iceberg/parquet/ParquetFileMerger.java: ########## @@ -0,0 +1,540 @@ +/* + * 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.nio.ByteOrder; +import java.util.List; +import java.util.Map; +import org.apache.iceberg.MetadataColumns; +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.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.parquet.bytes.BytesInput; +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.column.Encoding; +import org.apache.parquet.column.ParquetProperties; +import org.apache.parquet.column.statistics.LongStatistics; +import org.apache.parquet.hadoop.ParquetFileReader; +import org.apache.parquet.hadoop.ParquetFileWriter; +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. + * + * <p>This class ensures that all input files have identical Parquet schemas before merging. The + * merge operation is performed by copying row groups directly without + * serialization/deserialization, providing significant performance benefits over traditional + * read-rewrite approaches. + * + * <p>This class works with any Iceberg FileIO implementation (HadoopFileIO, S3FileIO, GCSFileIO, + * etc.), making it cloud-agnostic. + * + * <p>TODO: Encrypted tables are not supported + * + * <p>Key features: + * + * <ul> + * <li>Row group merging without deserialization using {@link ParquetFileWriter#appendFile} + * <li>Strict schema validation - all files must have identical {@link MessageType} + * <li>Metadata merging for Iceberg-specific footer data + * <li>Works with any FileIO implementation (local, S3, GCS, Azure, etc.) + * </ul> + * + * <p>Restrictions: + * + * <ul> + * <li>All files must have compatible schemas (identical {@link MessageType}) + * <li>Files must not be encrypted + * <li>Files must not have associated delete files or delete vectors + * <li>Table must not have a sort order (including z-ordered tables) + * </ul> + * + * <p>Typical usage: + * + * <pre> + * FileIO fileIO = table.io(); + * List<InputFile> inputFiles = Arrays.asList( + * fileIO.newInputFile("s3://bucket/file1.parquet"), + * fileIO.newInputFile("s3://bucket/file2.parquet") + * ); + * OutputFile outputFile = fileIO.newOutputFile("s3://bucket/merged.parquet"); + * long rowGroupSize = 128 * 1024 * 1024; // 128 MB + * int columnIndexTruncateLength = 64; // Default truncation length + * ParquetFileMerger.mergeFiles(inputFiles, outputFile, rowGroupSize, columnIndexTruncateLength, null); + * </pre> + */ +public class ParquetFileMerger { + + private ParquetFileMerger() { + // Utility class - prevent instantiation + } + + /** + * Merges multiple Parquet files into a single output file at the row-group level using Iceberg + * FileIO. + * + * <p>This method works with any Iceberg FileIO implementation (S3FileIO, GCSFileIO, etc.), not + * just HadoopFileIO. + * + * <p>All input files must have identical Parquet schemas ({@link MessageType}), otherwise an + * exception is thrown. The merge is performed by copying row groups directly without + * serialization/deserialization. + * + * @param inputFiles List of Iceberg input files to merge + * @param outputFile Iceberg output file for the merged result + * @param rowGroupSize Target row group size in bytes + * @param columnIndexTruncateLength Maximum length for min/max values in column index + * @param extraMetadata Additional metadata to include in the output file footer (can be null) + * @throws IOException if I/O error occurs during merge operation + * @throws IllegalArgumentException if no input files provided or schemas don't match + */ + public static void mergeFiles( + List<InputFile> inputFiles, + OutputFile outputFile, + long rowGroupSize, + int columnIndexTruncateLength, + Map<String, String> extraMetadata) + throws IOException { + Preconditions.checkArgument( + inputFiles != null && !inputFiles.isEmpty(), "No input files provided for merging"); + + // Validate and get the common schema from the first file + MessageType schema = readSchema(inputFiles.get(0)); + + // Validate all files have the same schema + for (int i = 1; i < inputFiles.size(); i++) { + MessageType currentSchema = readSchema(inputFiles.get(i)); + + if (!schema.equals(currentSchema)) { + throw new IllegalArgumentException( + String.format( + "Schema mismatch detected: file '%s' has schema %s but file '%s' has schema %s. " + + "All files must have identical Parquet schemas for row-group level merging.", + inputFiles.get(0).location(), schema, inputFiles.get(i).location(), currentSchema)); + } + } + + // Create the output Parquet file writer + try (ParquetFileWriter writer = + 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)) { + + writer.start(); + + // Append each input file's row groups to the output + for (InputFile inputFile : inputFiles) { + writer.appendFile(ParquetIO.file(inputFile)); + } + + // End writing with optional metadata + if (extraMetadata != null && !extraMetadata.isEmpty()) { + writer.end(extraMetadata); + } else { + writer.end(emptyMap()); + } + } + } + + /** + * Checks if a list of Iceberg InputFiles can be merged (i.e., they all have identical schemas). + * + * <p>This method works with any Iceberg FileIO implementation (S3FileIO, GCSFileIO, etc.). + * + * @param inputFiles List of Iceberg input files to check + * @return true if all files have identical schemas and can be merged, false otherwise + */ + public static boolean canMerge(List<InputFile> inputFiles) { + try { + if (inputFiles == null || inputFiles.isEmpty()) { + return false; + } + + // Read schema from the first file + MessageType firstSchema = readSchema(inputFiles.get(0)); + + // Validate all remaining files have the same schema + for (int i = 1; i < inputFiles.size(); i++) { + MessageType currentSchema = readSchema(inputFiles.get(i)); + + if (!firstSchema.equals(currentSchema)) { + return false; + } + } + + return true; + } catch (IllegalArgumentException | IOException e) { + return false; + } + } + + /** + * Checks if a list of Iceberg InputFiles can be merged with explicit firstRowIds. + * + * <p>This method performs additional validation beyond {@link #canMerge(List)} to ensure that + * files can be merged when explicit firstRowIds are provided. + * + * <p>Returns false if files already have a physical _row_id column and firstRowIds are provided, + * since we cannot override existing row IDs. + * + * @param inputFiles List of Iceberg input files to check + * @param firstRowIds Optional list of starting row IDs for each input file (null if no lineage + * needed) + * @return true if files can be merged with the given firstRowIds, false otherwise + */ + public static boolean canMergeWithRowIds(List<InputFile> inputFiles, List<Long> firstRowIds) { + // First check normal merge compatibility (schemas, etc.) + if (!canMerge(inputFiles)) { + return false; + } + + // If firstRowIds provided, ensure files don't already have physical _row_id + boolean needsRowLineageProcessing = firstRowIds != null && !firstRowIds.isEmpty(); + if (needsRowLineageProcessing) { + try { + MessageType schema = readSchema(inputFiles.get(0)); + if (schema.containsField(MetadataColumns.ROW_ID.name())) { + // Files already have physical _row_id - can't override with new firstRowIds + return false; + } + } catch (IOException e) { + return false; + } + } + + return true; + } + + /** + * Reads the Parquet schema from an Iceberg InputFile. + * + * @param inputFile Iceberg input file to read schema from + * @return MessageType schema of the Parquet file + * @throws IOException if reading fails + */ + private static MessageType readSchema(InputFile inputFile) throws IOException { + return ParquetFileReader.open(ParquetIO.file(inputFile)) + .getFooter() + .getFileMetaData() + .getSchema(); + } + + /** + * Validates that all input files have the same Parquet schema. + * + * @param inputFiles List of files to validate + * @param firstSchema Schema from the first file to compare against + * @throws IOException if reading fails + * @throws IllegalArgumentException if schemas don't match + */ + private static void validateSchemasMatch(List<InputFile> inputFiles, MessageType firstSchema) + throws IOException { + for (int i = 1; i < inputFiles.size(); i++) { + MessageType currentSchema = readSchema(inputFiles.get(i)); + if (!firstSchema.equals(currentSchema)) { + throw new IllegalArgumentException( + String.format( + "Schema mismatch detected: file '%s' has schema %s but file '%s' has schema %s. " + + "All files must have identical Parquet schemas for row-group level merging.", + inputFiles.get(0).location(), + firstSchema, + inputFiles.get(i).location(), + currentSchema)); + } + } + } + + /** Internal method to merge files when schema is already known. */ + private static void mergeFilesWithSchema( + List<InputFile> inputFiles, + OutputFile outputFile, + MessageType schema, + long rowGroupSize, + int columnIndexTruncateLength, + Map<String, String> extraMetadata) + throws IOException { + try (ParquetFileWriter writer = + 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)) { + + writer.start(); + for (InputFile inputFile : inputFiles) { + writer.appendFile(ParquetIO.file(inputFile)); + } Review Comment: nit: newline -- 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]
