diff --git a/lib/trino-parquet/src/main/java/io/trino/parquet/metadata/ParquetMetadata.java b/lib/trino-parquet/src/main/java/io/trino/parquet/metadata/ParquetMetadata.java index 9363add2ca6c0..cfc474c3b8218 100644 --- a/lib/trino-parquet/src/main/java/io/trino/parquet/metadata/ParquetMetadata.java +++ b/lib/trino-parquet/src/main/java/io/trino/parquet/metadata/ParquetMetadata.java @@ -13,32 +13,268 @@ */ package io.trino.parquet.metadata; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import io.airlift.log.Logger; +import io.trino.parquet.ParquetCorruptionException; +import io.trino.parquet.ParquetDataSourceId; +import io.trino.spi.TrinoException; +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.format.ColumnChunk; +import org.apache.parquet.format.ColumnMetaData; +import org.apache.parquet.format.Encoding; +import org.apache.parquet.format.FileMetaData; +import org.apache.parquet.format.KeyValue; +import org.apache.parquet.format.RowGroup; +import org.apache.parquet.format.SchemaElement; +import org.apache.parquet.hadoop.metadata.ColumnPath; +import org.apache.parquet.hadoop.metadata.CompressionCodecName; +import org.apache.parquet.schema.LogicalTypeAnnotation; +import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.PrimitiveType; +import org.apache.parquet.schema.Type; +import org.apache.parquet.schema.Types; + +import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; +import java.util.Iterator; import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.Optional; +import java.util.Set; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.collect.ImmutableSet.toImmutableSet; +import static io.trino.parquet.ParquetMetadataConverter.convertEncodingStats; +import static io.trino.parquet.ParquetMetadataConverter.getEncoding; +import static io.trino.parquet.ParquetMetadataConverter.getLogicalTypeAnnotation; +import static io.trino.parquet.ParquetMetadataConverter.getPrimitive; +import static io.trino.parquet.ParquetMetadataConverter.toColumnIndexReference; +import static io.trino.parquet.ParquetMetadataConverter.toOffsetIndexReference; +import static io.trino.parquet.ParquetValidationUtils.validateParquet; +import static io.trino.parquet.reader.MetadataReader.readStats; +import static io.trino.spi.StandardErrorCode.CORRUPT_PAGE; +import static java.util.Objects.requireNonNull; +import static java.util.stream.Collectors.toMap; public class ParquetMetadata { - private final FileMetadata fileMetaData; - private final List blocks; + private static final Logger log = Logger.get(ParquetMetadata.class); + + private final FileMetaData fileMetaData; + private final ParquetDataSourceId dataSourceId; + private final Optional offset; + private final Optional length; + private final FileMetadata parquetMetadata; - public ParquetMetadata(FileMetadata fileMetaData, List blocks) + public ParquetMetadata(FileMetaData fileMetaData, ParquetDataSourceId dataSourceId, Optional offset, Optional length) + throws ParquetCorruptionException { - this.fileMetaData = fileMetaData; - this.blocks = blocks; + this.fileMetaData = requireNonNull(fileMetaData, "fileMetaData is null"); + this.dataSourceId = requireNonNull(dataSourceId, "dataSourceId is null"); + this.offset = requireNonNull(offset, "offset is null"); + this.length = requireNonNull(length, "length is null"); + checkArgument(offset.isEmpty() && length.isEmpty() || (offset.isPresent() && length.isPresent()), + "Both offset and length must be present or absent"); + this.parquetMetadata = new FileMetadata(readMessageType(), keyValueMetaData(fileMetaData), fileMetaData.getCreated_by()); } - public List getBlocks() + private static Map keyValueMetaData(FileMetaData fileMetaData) { - return blocks; + if (fileMetaData.getKey_value_metadata() == null) { + return ImmutableMap.of(); + } + return fileMetaData.getKey_value_metadata().stream().collect(toMap(KeyValue::getKey, KeyValue::getValue)); + } + + private MessageType readMessageType() + throws ParquetCorruptionException + { + List schema = fileMetaData.getSchema(); + validateParquet(!schema.isEmpty(), dataSourceId, "Schema is empty"); + + Iterator schemaIterator = schema.iterator(); + SchemaElement rootSchema = schemaIterator.next(); + Types.MessageTypeBuilder builder = Types.buildMessage(); + readTypeSchema(builder, schemaIterator, rootSchema.getNum_children()); + return builder.named(rootSchema.name); } public FileMetadata getFileMetaData() { - return fileMetaData; + return parquetMetadata; + } + + public List getBlocks(Collection columnDescriptors) + { + Set paths = columnDescriptors.stream() + .map(ColumnDescriptor::getPath) + .map(ColumnPath::get) + .collect(toImmutableSet()); + + try { + return buildBlocks(paths); + } + catch (ParquetCorruptionException e) { + throw new TrinoException(CORRUPT_PAGE, e); + } + } + + public List getBlocks() + { + return getBlocks(ImmutableSet.of()); + } + + private List buildBlocks(Set paths) + throws ParquetCorruptionException + { + List schema = fileMetaData.getSchema(); + validateParquet(!schema.isEmpty(), dataSourceId, "Schema is empty"); + + MessageType messageType = readParquetSchema(schema); + ImmutableList.Builder builder = ImmutableList.builder(); + List rowGroups = fileMetaData.getRow_groups(); + if (rowGroups != null) { + for (int i = 0; i < rowGroups.size(); i++) { + RowGroup rowGroup = rowGroups.get(i); + List columns = rowGroup.getColumns(); + validateParquet(!columns.isEmpty(), dataSourceId, "No columns in row group: %s", rowGroup); + String filePath = columns.getFirst().getFile_path(); + + if (offset.isPresent() && length.isPresent() && rowGroup.isSetFile_offset()) { + if (rowGroup.file_offset >= offset.get() + length.get()) { + break; + } + if (i < rowGroups.size() - 1 && offset.get() >= rowGroups.get(i + 1).file_offset) { + continue; + } + } + + ImmutableList.Builder columnMetadataBuilder = ImmutableList.builderWithExpectedSize(columns.size()); + for (ColumnChunk columnChunk : columns) { + validateParquet( + (filePath == null && columnChunk.getFile_path() == null) + || (filePath != null && filePath.equals(columnChunk.getFile_path())), + dataSourceId, + "all column chunks of the same row group must be in the same file"); + ColumnMetaData metaData = columnChunk.meta_data; + String[] path = metaData.path_in_schema.stream() + .map(value -> value.toLowerCase(Locale.ENGLISH)) + .toArray(String[]::new); + ColumnPath columnPath = ColumnPath.get(path); + if (!paths.isEmpty() && !paths.contains(columnPath)) { + continue; + } + PrimitiveType primitiveType = messageType.getType(columnPath.toArray()).asPrimitiveType(); + ColumnChunkMetadata column = ColumnChunkMetadata.get( + columnPath, + primitiveType, + CompressionCodecName.fromParquet(metaData.codec), + convertEncodingStats(metaData.encoding_stats), + readEncodings(metaData.encodings), + readStats(Optional.ofNullable(fileMetaData.getCreated_by()), Optional.ofNullable(metaData.statistics), primitiveType), + metaData.data_page_offset, + metaData.dictionary_page_offset, + metaData.num_values, + metaData.total_compressed_size, + metaData.total_uncompressed_size); + column.setColumnIndexReference(toColumnIndexReference(columnChunk)); + column.setOffsetIndexReference(toOffsetIndexReference(columnChunk)); + column.setBloomFilterOffset(metaData.bloom_filter_offset); + columnMetadataBuilder.add(column); + } + builder.add(new BlockMetadata(rowGroup.getNum_rows(), columnMetadataBuilder.build())); + } + } + return builder.build(); + } + + private static Set readEncodings(List encodings) + { + Set columnEncodings = new HashSet<>(); + for (Encoding encoding : encodings) { + columnEncodings.add(getEncoding(encoding)); + } + return Collections.unmodifiableSet(columnEncodings); + } + + private static MessageType readParquetSchema(List schema) + { + Iterator schemaIterator = schema.iterator(); + SchemaElement rootSchema = schemaIterator.next(); + Types.MessageTypeBuilder builder = Types.buildMessage(); + readTypeSchema(builder, schemaIterator, rootSchema.getNum_children()); + return builder.named(rootSchema.name); + } + + private static void readTypeSchema(Types.GroupBuilder builder, Iterator schemaIterator, int typeCount) + { + for (int i = 0; i < typeCount; i++) { + SchemaElement element = schemaIterator.next(); + Types.Builder typeBuilder; + if (element.type == null) { + typeBuilder = builder.group(Type.Repetition.valueOf(element.repetition_type.name())); + readTypeSchema((Types.GroupBuilder) typeBuilder, schemaIterator, element.num_children); + } + else { + Types.PrimitiveBuilder primitiveBuilder = builder.primitive(getPrimitive(element.type), Type.Repetition.valueOf(element.repetition_type.name())); + if (element.isSetType_length()) { + primitiveBuilder.length(element.type_length); + } + if (element.isSetPrecision()) { + primitiveBuilder.precision(element.precision); + } + if (element.isSetScale()) { + primitiveBuilder.scale(element.scale); + } + typeBuilder = primitiveBuilder; + } + + // Reading of element.logicalType and element.converted_type corresponds to parquet-mr's code at + // https://github.com/apache/parquet-mr/blob/apache-parquet-1.12.0/parquet-hadoop/src/main/java/org/apache/parquet/format/converter/ParquetMetadataConverter.java#L1568-L1582 + LogicalTypeAnnotation annotationFromLogicalType = null; + if (element.isSetLogicalType()) { + annotationFromLogicalType = getLogicalTypeAnnotation(element.logicalType); + typeBuilder.as(annotationFromLogicalType); + } + if (element.isSetConverted_type()) { + LogicalTypeAnnotation annotationFromConvertedType = getLogicalTypeAnnotation(element.converted_type, element); + if (annotationFromLogicalType != null) { + // Both element.logicalType and element.converted_type set + if (annotationFromLogicalType.toOriginalType() == annotationFromConvertedType.toOriginalType()) { + // element.converted_type matches element.logicalType, even though annotationFromLogicalType may differ from annotationFromConvertedType + // Following parquet-mr behavior, we favor LogicalTypeAnnotation derived from element.logicalType, as potentially containing more information. + } + else { + // Following parquet-mr behavior, issue warning and let converted_type take precedence. + log.warn("Converted type and logical type metadata map to different OriginalType (convertedType: %s, logical type: %s). Using value in converted type.", + element.converted_type, element.logicalType); + // parquet-mr reads only OriginalType from converted_type. We retain full LogicalTypeAnnotation + // 1. for compatibility, as previous Trino reader code would read LogicalTypeAnnotation from element.converted_type and some additional fields. + // 2. so that we override LogicalTypeAnnotation annotation read from element.logicalType in case of mismatch detected. + typeBuilder.as(annotationFromConvertedType); + } + } + else { + // parquet-mr reads only OriginalType from converted_type. We retain full LogicalTypeAnnotation for compatibility, as previous + // Trino reader code would read LogicalTypeAnnotation from element.converted_type and some additional fields. + typeBuilder.as(annotationFromConvertedType); + } + } + + if (element.isSetField_id()) { + typeBuilder.id(element.field_id); + } + typeBuilder.named(element.name.toLowerCase(Locale.ENGLISH)); + } } @Override public String toString() { - return "ParquetMetaData{" + fileMetaData + ", blocks: " + blocks + "}"; + return "ParquetMetaData{" + fileMetaData + "}"; } } diff --git a/lib/trino-parquet/src/main/java/io/trino/parquet/reader/MetadataReader.java b/lib/trino-parquet/src/main/java/io/trino/parquet/reader/MetadataReader.java index fe0635646f989..29ca1adc8c263 100644 --- a/lib/trino-parquet/src/main/java/io/trino/parquet/reader/MetadataReader.java +++ b/lib/trino-parquet/src/main/java/io/trino/parquet/reader/MetadataReader.java @@ -13,57 +13,27 @@ */ package io.trino.parquet.reader; -import com.google.common.collect.ImmutableList; -import io.airlift.log.Logger; import io.airlift.slice.Slice; import io.airlift.slice.Slices; import io.trino.parquet.ParquetCorruptionException; import io.trino.parquet.ParquetDataSource; import io.trino.parquet.ParquetDataSourceId; import io.trino.parquet.ParquetWriteValidation; -import io.trino.parquet.metadata.BlockMetadata; -import io.trino.parquet.metadata.ColumnChunkMetadata; import io.trino.parquet.metadata.FileMetadata; import io.trino.parquet.metadata.ParquetMetadata; import org.apache.parquet.CorruptStatistics; import org.apache.parquet.column.statistics.BinaryStatistics; -import org.apache.parquet.format.ColumnChunk; -import org.apache.parquet.format.ColumnMetaData; -import org.apache.parquet.format.Encoding; import org.apache.parquet.format.FileMetaData; -import org.apache.parquet.format.KeyValue; -import org.apache.parquet.format.RowGroup; -import org.apache.parquet.format.SchemaElement; import org.apache.parquet.format.Statistics; -import org.apache.parquet.hadoop.metadata.ColumnPath; -import org.apache.parquet.hadoop.metadata.CompressionCodecName; import org.apache.parquet.schema.LogicalTypeAnnotation; -import org.apache.parquet.schema.MessageType; import org.apache.parquet.schema.PrimitiveType; -import org.apache.parquet.schema.Type.Repetition; -import org.apache.parquet.schema.Types; import java.io.IOException; import java.io.InputStream; -import java.util.ArrayList; import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Iterator; -import java.util.List; -import java.util.Locale; -import java.util.Map; import java.util.Optional; -import java.util.Set; -import static io.trino.parquet.ParquetMetadataConverter.convertEncodingStats; import static io.trino.parquet.ParquetMetadataConverter.fromParquetStatistics; -import static io.trino.parquet.ParquetMetadataConverter.getEncoding; -import static io.trino.parquet.ParquetMetadataConverter.getLogicalTypeAnnotation; -import static io.trino.parquet.ParquetMetadataConverter.getPrimitive; -import static io.trino.parquet.ParquetMetadataConverter.toColumnIndexReference; -import static io.trino.parquet.ParquetMetadataConverter.toOffsetIndexReference; import static io.trino.parquet.ParquetValidationUtils.validateParquet; import static java.lang.Boolean.FALSE; import static java.lang.Boolean.TRUE; @@ -73,8 +43,6 @@ public final class MetadataReader { - private static final Logger log = Logger.get(MetadataReader.class); - private static final Slice MAGIC = Slices.utf8Slice("PAR1"); private static final int POST_SCRIPT_SIZE = Integer.BYTES + MAGIC.length(); // Typical 1GB files produced by Trino were found to have footer size between 30-40KB @@ -82,7 +50,7 @@ public final class MetadataReader private MetadataReader() {} - public static ParquetMetadata readFooter(ParquetDataSource dataSource, Optional parquetWriteValidation) + public static ParquetMetadata readFooter(ParquetDataSource dataSource, Optional parquetWriteValidation, Optional offset, Optional length) throws IOException { // Parquet File Layout: @@ -119,144 +87,11 @@ public static ParquetMetadata readFooter(ParquetDataSource dataSource, Optional< InputStream metadataStream = buffer.slice(buffer.length() - completeFooterSize, metadataLength).getInput(); FileMetaData fileMetaData = readFileMetaData(metadataStream); - ParquetMetadata parquetMetadata = createParquetMetadata(fileMetaData, dataSource.getId()); + ParquetMetadata parquetMetadata = new ParquetMetadata(fileMetaData, dataSource.getId(), offset, length); validateFileMetadata(dataSource.getId(), parquetMetadata.getFileMetaData(), parquetWriteValidation); return parquetMetadata; } - public static ParquetMetadata createParquetMetadata(FileMetaData fileMetaData, ParquetDataSourceId dataSourceId) - throws ParquetCorruptionException - { - List schema = fileMetaData.getSchema(); - validateParquet(!schema.isEmpty(), dataSourceId, "Schema is empty"); - - MessageType messageType = readParquetSchema(schema); - List blocks = new ArrayList<>(); - List rowGroups = fileMetaData.getRow_groups(); - if (rowGroups != null) { - for (RowGroup rowGroup : rowGroups) { - List columns = rowGroup.getColumns(); - validateParquet(!columns.isEmpty(), dataSourceId, "No columns in row group: %s", rowGroup); - String filePath = columns.get(0).getFile_path(); - ImmutableList.Builder columnMetadataBuilder = ImmutableList.builderWithExpectedSize(columns.size()); - for (ColumnChunk columnChunk : columns) { - validateParquet( - (filePath == null && columnChunk.getFile_path() == null) - || (filePath != null && filePath.equals(columnChunk.getFile_path())), - dataSourceId, - "all column chunks of the same row group must be in the same file"); - ColumnMetaData metaData = columnChunk.meta_data; - String[] path = metaData.path_in_schema.stream() - .map(value -> value.toLowerCase(Locale.ENGLISH)) - .toArray(String[]::new); - ColumnPath columnPath = ColumnPath.get(path); - PrimitiveType primitiveType = messageType.getType(columnPath.toArray()).asPrimitiveType(); - ColumnChunkMetadata column = ColumnChunkMetadata.get( - columnPath, - primitiveType, - CompressionCodecName.fromParquet(metaData.codec), - convertEncodingStats(metaData.encoding_stats), - readEncodings(metaData.encodings), - readStats(Optional.ofNullable(fileMetaData.getCreated_by()), Optional.ofNullable(metaData.statistics), primitiveType), - metaData.data_page_offset, - metaData.dictionary_page_offset, - metaData.num_values, - metaData.total_compressed_size, - metaData.total_uncompressed_size); - column.setColumnIndexReference(toColumnIndexReference(columnChunk)); - column.setOffsetIndexReference(toOffsetIndexReference(columnChunk)); - column.setBloomFilterOffset(metaData.bloom_filter_offset); - columnMetadataBuilder.add(column); - } - blocks.add(new BlockMetadata(rowGroup.getNum_rows(), columnMetadataBuilder.build())); - } - } - - Map keyValueMetaData = new HashMap<>(); - List keyValueList = fileMetaData.getKey_value_metadata(); - if (keyValueList != null) { - for (KeyValue keyValue : keyValueList) { - keyValueMetaData.put(keyValue.key, keyValue.value); - } - } - FileMetadata parquetFileMetadata = new FileMetadata( - messageType, - keyValueMetaData, - fileMetaData.getCreated_by()); - return new ParquetMetadata(parquetFileMetadata, blocks); - } - - private static MessageType readParquetSchema(List schema) - { - Iterator schemaIterator = schema.iterator(); - SchemaElement rootSchema = schemaIterator.next(); - Types.MessageTypeBuilder builder = Types.buildMessage(); - readTypeSchema(builder, schemaIterator, rootSchema.getNum_children()); - return builder.named(rootSchema.name); - } - - private static void readTypeSchema(Types.GroupBuilder builder, Iterator schemaIterator, int typeCount) - { - for (int i = 0; i < typeCount; i++) { - SchemaElement element = schemaIterator.next(); - Types.Builder typeBuilder; - if (element.type == null) { - typeBuilder = builder.group(Repetition.valueOf(element.repetition_type.name())); - readTypeSchema((Types.GroupBuilder) typeBuilder, schemaIterator, element.num_children); - } - else { - Types.PrimitiveBuilder primitiveBuilder = builder.primitive(getPrimitive(element.type), Repetition.valueOf(element.repetition_type.name())); - if (element.isSetType_length()) { - primitiveBuilder.length(element.type_length); - } - if (element.isSetPrecision()) { - primitiveBuilder.precision(element.precision); - } - if (element.isSetScale()) { - primitiveBuilder.scale(element.scale); - } - typeBuilder = primitiveBuilder; - } - - // Reading of element.logicalType and element.converted_type corresponds to parquet-mr's code at - // https://github.com/apache/parquet-mr/blob/apache-parquet-1.12.0/parquet-hadoop/src/main/java/org/apache/parquet/format/converter/ParquetMetadataConverter.java#L1568-L1582 - LogicalTypeAnnotation annotationFromLogicalType = null; - if (element.isSetLogicalType()) { - annotationFromLogicalType = getLogicalTypeAnnotation(element.logicalType); - typeBuilder.as(annotationFromLogicalType); - } - if (element.isSetConverted_type()) { - LogicalTypeAnnotation annotationFromConvertedType = getLogicalTypeAnnotation(element.converted_type, element); - if (annotationFromLogicalType != null) { - // Both element.logicalType and element.converted_type set - if (annotationFromLogicalType.toOriginalType() == annotationFromConvertedType.toOriginalType()) { - // element.converted_type matches element.logicalType, even though annotationFromLogicalType may differ from annotationFromConvertedType - // Following parquet-mr behavior, we favor LogicalTypeAnnotation derived from element.logicalType, as potentially containing more information. - } - else { - // Following parquet-mr behavior, issue warning and let converted_type take precedence. - log.warn("Converted type and logical type metadata map to different OriginalType (convertedType: %s, logical type: %s). Using value in converted type.", - element.converted_type, element.logicalType); - // parquet-mr reads only OriginalType from converted_type. We retain full LogicalTypeAnnotation - // 1. for compatibility, as previous Trino reader code would read LogicalTypeAnnotation from element.converted_type and some additional fields. - // 2. so that we override LogicalTypeAnnotation annotation read from element.logicalType in case of mismatch detected. - typeBuilder.as(annotationFromConvertedType); - } - } - else { - // parquet-mr reads only OriginalType from converted_type. We retain full LogicalTypeAnnotation for compatibility, as previous - // Trino reader code would read LogicalTypeAnnotation from element.converted_type and some additional fields. - typeBuilder.as(annotationFromConvertedType); - } - } - - if (element.isSetField_id()) { - typeBuilder.id(element.field_id); - } - typeBuilder.named(element.name.toLowerCase(Locale.ENGLISH)); - } - } - public static org.apache.parquet.column.statistics.Statistics readStats(Optional fileCreatedBy, Optional statisticsFromFile, PrimitiveType type) { Statistics statistics = statisticsFromFile.orElse(null); @@ -352,15 +187,6 @@ private static int commonPrefix(byte[] a, byte[] b) return commonPrefixLength; } - private static Set readEncodings(List encodings) - { - Set columnEncodings = new HashSet<>(); - for (Encoding encoding : encodings) { - columnEncodings.add(getEncoding(encoding)); - } - return Collections.unmodifiableSet(columnEncodings); - } - private static void validateFileMetadata(ParquetDataSourceId dataSourceId, FileMetadata fileMetaData, Optional parquetWriteValidation) throws ParquetCorruptionException { diff --git a/lib/trino-parquet/src/main/java/io/trino/parquet/writer/ParquetWriter.java b/lib/trino-parquet/src/main/java/io/trino/parquet/writer/ParquetWriter.java index 1eed8ba73ef10..81f230435b94d 100644 --- a/lib/trino-parquet/src/main/java/io/trino/parquet/writer/ParquetWriter.java +++ b/lib/trino-parquet/src/main/java/io/trino/parquet/writer/ParquetWriter.java @@ -240,7 +240,7 @@ public void validate(ParquetDataSource input) checkState(validationBuilder.isPresent(), "validation is not enabled"); ParquetWriteValidation writeValidation = validationBuilder.get().build(); try { - ParquetMetadata parquetMetadata = MetadataReader.readFooter(input, Optional.of(writeValidation)); + ParquetMetadata parquetMetadata = MetadataReader.readFooter(input, Optional.of(writeValidation), Optional.empty(), Optional.empty()); try (ParquetReader parquetReader = createParquetReader(input, parquetMetadata, writeValidation)) { for (Page page = parquetReader.nextPage(); page != null; page = parquetReader.nextPage()) { // fully load the page diff --git a/lib/trino-parquet/src/test/java/io/trino/parquet/BenchmarkColumnarFilterParquetData.java b/lib/trino-parquet/src/test/java/io/trino/parquet/BenchmarkColumnarFilterParquetData.java index 9f7918115838f..f7228cf4580d8 100644 --- a/lib/trino-parquet/src/test/java/io/trino/parquet/BenchmarkColumnarFilterParquetData.java +++ b/lib/trino-parquet/src/test/java/io/trino/parquet/BenchmarkColumnarFilterParquetData.java @@ -225,7 +225,7 @@ public void setup() testData.getColumnNames(), testData.getPages()), new ParquetReaderOptions()); - parquetMetadata = MetadataReader.readFooter(dataSource, Optional.empty()); + parquetMetadata = MetadataReader.readFooter(dataSource, Optional.empty(), Optional.empty(), Optional.empty()); columnNames = columns.stream() .map(TpchColumn::getColumnName) .collect(toImmutableList()); diff --git a/lib/trino-parquet/src/test/java/io/trino/parquet/reader/TestByteStreamSplitEncoding.java b/lib/trino-parquet/src/test/java/io/trino/parquet/reader/TestByteStreamSplitEncoding.java index d42725e5acb2f..47028cbb5e915 100644 --- a/lib/trino-parquet/src/test/java/io/trino/parquet/reader/TestByteStreamSplitEncoding.java +++ b/lib/trino-parquet/src/test/java/io/trino/parquet/reader/TestByteStreamSplitEncoding.java @@ -50,7 +50,7 @@ public void testReadFloatDouble() ParquetDataSource dataSource = new FileParquetDataSource( new File(Resources.getResource("byte_stream_split_float_and_double.parquet").toURI()), new ParquetReaderOptions()); - ParquetMetadata parquetMetadata = MetadataReader.readFooter(dataSource, Optional.empty()); + ParquetMetadata parquetMetadata = MetadataReader.readFooter(dataSource, Optional.empty(), Optional.empty(), Optional.empty()); ParquetReader reader = createParquetReader(dataSource, parquetMetadata, newSimpleAggregatedMemoryContext(), types, columnNames); readAndCompare(reader, getExpectedValues()); diff --git a/lib/trino-parquet/src/test/java/io/trino/parquet/reader/TestInt96Timestamp.java b/lib/trino-parquet/src/test/java/io/trino/parquet/reader/TestInt96Timestamp.java index aabb734e5b0c8..ead68a9448803 100644 --- a/lib/trino-parquet/src/test/java/io/trino/parquet/reader/TestInt96Timestamp.java +++ b/lib/trino-parquet/src/test/java/io/trino/parquet/reader/TestInt96Timestamp.java @@ -112,7 +112,7 @@ public void testNanosOutsideDayRange() ParquetDataSource dataSource = new FileParquetDataSource( new File(Resources.getResource("int96_timestamps_nanos_outside_day_range.parquet").toURI()), new ParquetReaderOptions()); - ParquetMetadata parquetMetadata = MetadataReader.readFooter(dataSource, Optional.empty()); + ParquetMetadata parquetMetadata = MetadataReader.readFooter(dataSource, Optional.empty(), Optional.empty(), Optional.empty()); ParquetReader reader = createParquetReader(dataSource, parquetMetadata, newSimpleAggregatedMemoryContext(), types, columnNames); Page page = reader.nextPage(); diff --git a/lib/trino-parquet/src/test/java/io/trino/parquet/reader/TestParquetReader.java b/lib/trino-parquet/src/test/java/io/trino/parquet/reader/TestParquetReader.java index 00ecd8388857b..f73a163763461 100644 --- a/lib/trino-parquet/src/test/java/io/trino/parquet/reader/TestParquetReader.java +++ b/lib/trino-parquet/src/test/java/io/trino/parquet/reader/TestParquetReader.java @@ -79,7 +79,7 @@ public void testColumnReaderMemoryUsage() columnNames, generateInputPages(types, 100, 5)), new ParquetReaderOptions()); - ParquetMetadata parquetMetadata = MetadataReader.readFooter(dataSource, Optional.empty()); + ParquetMetadata parquetMetadata = MetadataReader.readFooter(dataSource, Optional.empty(), Optional.empty(), Optional.empty()); assertThat(parquetMetadata.getBlocks().size()).isGreaterThan(1); // Verify file has only non-dictionary encodings as dictionary memory usage is already tested in TestFlatColumnReader#testMemoryUsage parquetMetadata.getBlocks().forEach(block -> { @@ -132,7 +132,7 @@ public void testEmptyRowRangesWithColumnIndex() ParquetDataSource dataSource = new FileParquetDataSource( new File(Resources.getResource("lineitem_sorted_by_shipdate/data.parquet").toURI()), new ParquetReaderOptions()); - ParquetMetadata parquetMetadata = MetadataReader.readFooter(dataSource, Optional.empty()); + ParquetMetadata parquetMetadata = MetadataReader.readFooter(dataSource, Optional.empty(), Optional.empty(), Optional.empty()); assertThat(parquetMetadata.getBlocks()).hasSize(2); // The predicate and the file are prepared so that page indexes will result in non-overlapping row ranges and eliminate the entire first row group // while the second row group still has to be read @@ -193,7 +193,7 @@ private void testReadingOldParquetFiles(File file, List columnNames, Typ file, new ParquetReaderOptions()); ConnectorSession session = TestingConnectorSession.builder().build(); - ParquetMetadata parquetMetadata = MetadataReader.readFooter(dataSource, Optional.empty()); + ParquetMetadata parquetMetadata = MetadataReader.readFooter(dataSource, Optional.empty(), Optional.empty(), Optional.empty()); try (ParquetReader reader = createParquetReader(dataSource, parquetMetadata, newSimpleAggregatedMemoryContext(), ImmutableList.of(columnType), columnNames)) { Page page = reader.nextPage(); Iterator expected = expectedValues.iterator(); diff --git a/lib/trino-parquet/src/test/java/io/trino/parquet/reader/TestTimeMillis.java b/lib/trino-parquet/src/test/java/io/trino/parquet/reader/TestTimeMillis.java index 390608f445a97..08a2fd5952e34 100644 --- a/lib/trino-parquet/src/test/java/io/trino/parquet/reader/TestTimeMillis.java +++ b/lib/trino-parquet/src/test/java/io/trino/parquet/reader/TestTimeMillis.java @@ -60,7 +60,7 @@ private void testTimeMillsInt32(TimeType timeType) ParquetDataSource dataSource = new FileParquetDataSource( new File(Resources.getResource("time_millis_int32.snappy.parquet").toURI()), new ParquetReaderOptions()); - ParquetMetadata parquetMetadata = MetadataReader.readFooter(dataSource, Optional.empty()); + ParquetMetadata parquetMetadata = MetadataReader.readFooter(dataSource, Optional.empty(), Optional.empty(), Optional.empty()); ParquetReader reader = createParquetReader(dataSource, parquetMetadata, newSimpleAggregatedMemoryContext(), types, columnNames); Page page = reader.nextPage(); diff --git a/lib/trino-parquet/src/test/java/io/trino/parquet/writer/TestParquetWriter.java b/lib/trino-parquet/src/test/java/io/trino/parquet/writer/TestParquetWriter.java index a80cbbcd00d73..ef995ce4b753d 100644 --- a/lib/trino-parquet/src/test/java/io/trino/parquet/writer/TestParquetWriter.java +++ b/lib/trino-parquet/src/test/java/io/trino/parquet/writer/TestParquetWriter.java @@ -128,7 +128,7 @@ public void testWrittenPageSize() columnNames, generateInputPages(types, 100, 1000)), new ParquetReaderOptions()); - ParquetMetadata parquetMetadata = MetadataReader.readFooter(dataSource, Optional.empty()); + ParquetMetadata parquetMetadata = MetadataReader.readFooter(dataSource, Optional.empty(), Optional.empty(), Optional.empty()); assertThat(parquetMetadata.getBlocks()).hasSize(1); assertThat(parquetMetadata.getBlocks().get(0).rowCount()).isEqualTo(100 * 1000); @@ -177,7 +177,7 @@ public void testWrittenPageValueCount() columnNames, generateInputPages(types, 100, 1000)), new ParquetReaderOptions()); - ParquetMetadata parquetMetadata = MetadataReader.readFooter(dataSource, Optional.empty()); + ParquetMetadata parquetMetadata = MetadataReader.readFooter(dataSource, Optional.empty(), Optional.empty(), Optional.empty()); assertThat(parquetMetadata.getBlocks()).hasSize(1); assertThat(parquetMetadata.getBlocks().get(0).rowCount()).isEqualTo(100 * 1000); @@ -258,7 +258,7 @@ public void testLargeStringTruncation() ImmutableList.of(new Page(2, blockA, blockB))), new ParquetReaderOptions()); - ParquetMetadata parquetMetadata = MetadataReader.readFooter(dataSource, Optional.empty()); + ParquetMetadata parquetMetadata = MetadataReader.readFooter(dataSource, Optional.empty(), Optional.empty(), Optional.empty()); BlockMetadata blockMetaData = getOnlyElement(parquetMetadata.getBlocks()); ColumnChunkMetadata chunkMetaData = blockMetaData.columns().get(0); @@ -291,7 +291,7 @@ public void testColumnReordering() generateInputPages(types, 100, 100)), new ParquetReaderOptions()); - ParquetMetadata parquetMetadata = MetadataReader.readFooter(dataSource, Optional.empty()); + ParquetMetadata parquetMetadata = MetadataReader.readFooter(dataSource, Optional.empty(), Optional.empty(), Optional.empty()); assertThat(parquetMetadata.getBlocks().size()).isGreaterThanOrEqualTo(10); for (BlockMetadata blockMetaData : parquetMetadata.getBlocks()) { // Verify that the columns are stored in the same order as the metadata @@ -348,7 +348,7 @@ public void testDictionaryPageOffset() generateInputPages(types, 100, 100)), new ParquetReaderOptions()); - ParquetMetadata parquetMetadata = MetadataReader.readFooter(dataSource, Optional.empty()); + ParquetMetadata parquetMetadata = MetadataReader.readFooter(dataSource, Optional.empty(), Optional.empty(), Optional.empty()); assertThat(parquetMetadata.getBlocks().size()).isGreaterThanOrEqualTo(1); for (BlockMetadata blockMetaData : parquetMetadata.getBlocks()) { ColumnChunkMetadata chunkMetaData = getOnlyElement(blockMetaData.columns()); @@ -397,7 +397,7 @@ public void testWriteBloomFilters(Type type, List data) generateInputPages(types, 100, data)), new ParquetReaderOptions()); - ParquetMetadata parquetMetadata = MetadataReader.readFooter(dataSource, Optional.empty()); + ParquetMetadata parquetMetadata = MetadataReader.readFooter(dataSource, Optional.empty(), Optional.empty(), Optional.empty()); // Check that bloom filters are right after each other int bloomFilterSize = Integer.highestOneBit(BlockSplitBloomFilter.optimalNumOfBits(BLOOM_FILTER_EXPECTED_ENTRIES, DEFAULT_BLOOM_FILTER_FPP) / 8) << 1; for (BlockMetadata block : parquetMetadata.getBlocks()) { @@ -462,7 +462,7 @@ void testBloomFilterWithDictionaryFallback() .build()), new ParquetReaderOptions()); - ParquetMetadata parquetMetadata = MetadataReader.readFooter(dataSource, Optional.empty()); + ParquetMetadata parquetMetadata = MetadataReader.readFooter(dataSource, Optional.empty(), Optional.empty(), Optional.empty()); BlockMetadata blockMetaData = getOnlyElement(parquetMetadata.getBlocks()); ColumnChunkMetadata chunkMetaData = getOnlyElement(blockMetaData.columns()); assertThat(chunkMetaData.getEncodingStats().hasDictionaryPages()).isTrue(); diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeMergeSink.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeMergeSink.java index 0ad393f8644e6..05bc022eacf25 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeMergeSink.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeMergeSink.java @@ -378,7 +378,7 @@ private Slice writeMergeResult(Slice path, FileDeletion deletion) } TrinoInputFile inputFile = fileSystem.newInputFile(Location.of(path.toStringUtf8())); try (ParquetDataSource dataSource = new TrinoParquetDataSource(inputFile, parquetReaderOptions, fileFormatDataSourceStats)) { - ParquetMetadata parquetMetadata = MetadataReader.readFooter(dataSource, Optional.empty()); + ParquetMetadata parquetMetadata = MetadataReader.readFooter(dataSource, Optional.empty(), Optional.empty(), Optional.empty()); long rowCount = parquetMetadata.getBlocks().stream().map(BlockMetadata::rowCount).mapToLong(Long::longValue).sum(); RoaringBitmapArray rowsRetained = new RoaringBitmapArray(); rowsRetained.addRange(0, rowCount - 1); diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakePageSourceProvider.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakePageSourceProvider.java index 3754810240b79..2f02ff987d5e4 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakePageSourceProvider.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakePageSourceProvider.java @@ -303,7 +303,7 @@ private PositionDeleteFilter readDeletes( public Map loadParquetIdAndNameMapping(TrinoInputFile inputFile, ParquetReaderOptions options) { try (ParquetDataSource dataSource = new TrinoParquetDataSource(inputFile, options, fileFormatDataSourceStats)) { - ParquetMetadata parquetMetadata = MetadataReader.readFooter(dataSource, Optional.empty()); + ParquetMetadata parquetMetadata = MetadataReader.readFooter(dataSource, Optional.empty(), Optional.empty(), Optional.empty()); FileMetadata fileMetaData = parquetMetadata.getFileMetaData(); MessageType fileSchema = fileMetaData.getSchema(); diff --git a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeWriter.java b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeWriter.java index 8f686205e2395..5d7e28a3d4458 100644 --- a/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeWriter.java +++ b/plugin/trino-delta-lake/src/main/java/io/trino/plugin/deltalake/DeltaLakeWriter.java @@ -22,7 +22,6 @@ import io.trino.parquet.metadata.BlockMetadata; import io.trino.parquet.metadata.ColumnChunkMetadata; import io.trino.parquet.metadata.ParquetMetadata; -import io.trino.parquet.reader.MetadataReader; import io.trino.plugin.deltalake.DataFileInfo.DataFileType; import io.trino.plugin.deltalake.transactionlog.statistics.DeltaLakeJsonFileStatistics; import io.trino.plugin.hive.FileWriter; @@ -184,7 +183,7 @@ public DataFileInfo getDataFileInfo() { Location path = rootTableLocation.appendPath(relativeFilePath); FileMetaData fileMetaData = fileWriter.getFileMetadata(); - ParquetMetadata parquetMetadata = MetadataReader.createParquetMetadata(fileMetaData, new ParquetDataSourceId(path.toString())); + ParquetMetadata parquetMetadata = new ParquetMetadata(fileMetaData, new ParquetDataSourceId(path.toString()), Optional.empty(), Optional.empty()); return new DataFileInfo( relativeFilePath, diff --git a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeBasic.java b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeBasic.java index c1535cc40ffa0..9f376ce0f4c78 100644 --- a/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeBasic.java +++ b/plugin/trino-delta-lake/src/test/java/io/trino/plugin/deltalake/TestDeltaLakeBasic.java @@ -329,7 +329,7 @@ private void testOptimizeWithColumnMappingMode(String columnMappingMode) TrinoInputFile inputFile = new LocalInputFile(tableLocation.resolve(addFileEntry.getPath()).toFile()); ParquetMetadata parquetMetadata = MetadataReader.readFooter( new TrinoParquetDataSource(inputFile, new ParquetReaderOptions(), new FileFormatDataSourceStats()), - Optional.empty()); + Optional.empty(), Optional.empty(), Optional.empty()); FileMetadata fileMetaData = parquetMetadata.getFileMetaData(); PrimitiveType physicalType = getOnlyElement(fileMetaData.getSchema().getColumns().iterator()).getPrimitiveType(); assertThat(physicalType.getName()).isEqualTo(physicalName); diff --git a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/parquet/ParquetPageSourceFactory.java b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/parquet/ParquetPageSourceFactory.java index f3481ac4e253d..2c69a68511575 100644 --- a/plugin/trino-hive/src/main/java/io/trino/plugin/hive/parquet/ParquetPageSourceFactory.java +++ b/plugin/trino-hive/src/main/java/io/trino/plugin/hive/parquet/ParquetPageSourceFactory.java @@ -221,7 +221,7 @@ public static ReaderPageSource createPageSource( AggregatedMemoryContext memoryContext = newSimpleAggregatedMemoryContext(); dataSource = createDataSource(inputFile, estimatedFileSize, options, memoryContext, stats); - ParquetMetadata parquetMetadata = MetadataReader.readFooter(dataSource, parquetWriteValidation); + ParquetMetadata parquetMetadata = MetadataReader.readFooter(dataSource, parquetWriteValidation, Optional.empty(), Optional.empty()); FileMetadata fileMetaData = parquetMetadata.getFileMetaData(); fileSchema = fileMetaData.getSchema(); @@ -253,7 +253,7 @@ public static ReaderPageSource createPageSource( start, length, dataSource, - parquetMetadata.getBlocks(), + parquetMetadata.getBlocks(descriptorsByPath.values()), parquetTupleDomains, parquetPredicates, descriptorsByPath, diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/parquet/TestBloomFilterStore.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/parquet/TestBloomFilterStore.java index 9e92b87782d20..443ae8fea1066 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/parquet/TestBloomFilterStore.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/parquet/TestBloomFilterStore.java @@ -308,7 +308,7 @@ private static BloomFilterStore generateBloomFilterStore(ParquetTester.TempFile TrinoInputFile inputFile = new LocalInputFile(tempFile.getFile()); TrinoParquetDataSource dataSource = new TrinoParquetDataSource(inputFile, new ParquetReaderOptions(), new FileFormatDataSourceStats()); - ParquetMetadata parquetMetadata = MetadataReader.readFooter(dataSource, Optional.empty()); + ParquetMetadata parquetMetadata = MetadataReader.readFooter(dataSource, Optional.empty(), Optional.empty(), Optional.empty()); ColumnChunkMetadata columnChunkMetaData = getOnlyElement(getOnlyElement(parquetMetadata.getBlocks()).columns()); return new BloomFilterStore(dataSource, getOnlyElement(parquetMetadata.getBlocks()), Set.of(columnChunkMetaData.getPath())); diff --git a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiPageSourceProvider.java b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiPageSourceProvider.java index 15129f4ee17d9..8860248ab208f 100644 --- a/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiPageSourceProvider.java +++ b/plugin/trino-hudi/src/main/java/io/trino/plugin/hudi/HudiPageSourceProvider.java @@ -198,7 +198,7 @@ private static ConnectorPageSource createPageSource( try { AggregatedMemoryContext memoryContext = newSimpleAggregatedMemoryContext(); dataSource = createDataSource(inputFile, OptionalLong.of(hudiSplit.getFileSize()), options, memoryContext, dataSourceStats); - ParquetMetadata parquetMetadata = MetadataReader.readFooter(dataSource, Optional.empty()); + ParquetMetadata parquetMetadata = MetadataReader.readFooter(dataSource, Optional.empty(), Optional.empty(), Optional.empty()); FileMetadata fileMetaData = parquetMetadata.getFileMetaData(); MessageType fileSchema = fileMetaData.getSchema(); diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergPageSourceProvider.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergPageSourceProvider.java index d937f5c57133f..ac1902b9ade93 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergPageSourceProvider.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergPageSourceProvider.java @@ -912,7 +912,7 @@ private static ReaderPageSourceWithRowPositions createParquetPageSource( ParquetDataSource dataSource = null; try { dataSource = createDataSource(inputFile, OptionalLong.of(fileSize), options, memoryContext, fileFormatDataSourceStats); - ParquetMetadata parquetMetadata = MetadataReader.readFooter(dataSource, Optional.empty()); + ParquetMetadata parquetMetadata = MetadataReader.readFooter(dataSource, Optional.empty(), Optional.of(start), Optional.of(length)); FileMetadata fileMetaData = parquetMetadata.getFileMetaData(); MessageType fileSchema = fileMetaData.getSchema(); if (nameMapping.isPresent() && !ParquetSchemaUtil.hasIds(fileSchema)) { @@ -928,9 +928,10 @@ private static ReaderPageSourceWithRowPositions createParquetPageSource( .map(readerColumns -> (List) readerColumns.get().stream().map(IcebergColumnHandle.class::cast).collect(toImmutableList())) .orElse(regularColumns); - List parquetFields = readBaseColumns.stream() + List> parquetFields = readBaseColumns.stream() .map(column -> parquetIdToField.get(column.getId())) - .collect(toList()); + .map(Optional::ofNullable) + .collect(toImmutableList()); MessageType requestedSchema = getMessageType(regularColumns, fileSchema.getName(), parquetIdToField); Map, ColumnDescriptor> descriptorsByPath = getDescriptors(fileSchema, requestedSchema); @@ -941,7 +942,7 @@ private static ReaderPageSourceWithRowPositions createParquetPageSource( start, length, dataSource, - parquetMetadata.getBlocks(), + parquetMetadata.getBlocks(descriptorsByPath.values()), ImmutableList.of(parquetTupleDomain), ImmutableList.of(parquetPredicate), descriptorsByPath, @@ -993,20 +994,20 @@ else if (column.getId() == TRINO_MERGE_PARTITION_DATA) { pageSourceBuilder.addConstantColumn(nativeValueToBlock(column.getType(), utf8Slice(partitionData))); } else { - org.apache.parquet.schema.Type parquetField = parquetFields.get(columnIndex); + Optional parquetField = parquetFields.get(columnIndex); Type trinoType = column.getBaseType(); - if (parquetField == null) { + if (parquetField.isEmpty()) { pageSourceBuilder.addNullColumn(trinoType); continue; } // The top level columns are already mapped by name/id appropriately. - ColumnIO columnIO = messageColumnIO.getChild(parquetField.getName()); + ColumnIO columnIO = messageColumnIO.getChild(parquetField.get().getName()); Optional field = IcebergParquetColumnIOConverter.constructField(new FieldContext(trinoType, column.getColumnIdentity()), columnIO); if (field.isEmpty()) { pageSourceBuilder.addNullColumn(trinoType); continue; } - parquetColumnFieldsBuilder.add(new Column(parquetField.getName(), field.get())); + parquetColumnFieldsBuilder.add(new Column(parquetField.get().getName(), field.get())); pageSourceBuilder.addSourceColumn(parquetSourceChannel); parquetSourceChannel++; } diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergParquetFileWriter.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergParquetFileWriter.java index 7f0716b66188a..a3b9da29bd15b 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergParquetFileWriter.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergParquetFileWriter.java @@ -33,7 +33,6 @@ import java.util.Optional; import java.util.stream.Stream; -import static io.trino.parquet.reader.MetadataReader.createParquetMetadata; import static io.trino.plugin.iceberg.util.ParquetUtil.footerMetrics; import static io.trino.plugin.iceberg.util.ParquetUtil.getSplitOffsets; import static io.trino.spi.StandardErrorCode.GENERIC_INTERNAL_ERROR; @@ -83,7 +82,7 @@ public FileMetrics getFileMetrics() { ParquetMetadata parquetMetadata; try { - parquetMetadata = createParquetMetadata(parquetFileWriter.getFileMetadata(), new ParquetDataSourceId(location.toString())); + parquetMetadata = new ParquetMetadata(parquetFileWriter.getFileMetadata(), new ParquetDataSourceId(location.toString()), Optional.empty(), Optional.empty()); } catch (IOException e) { throw new TrinoException(GENERIC_INTERNAL_ERROR, format("Error creating metadata for Parquet file %s", location), e); diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/procedure/MigrationUtils.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/procedure/MigrationUtils.java index 9af0647f84e5d..c5eff09ece1ba 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/procedure/MigrationUtils.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/procedure/MigrationUtils.java @@ -148,7 +148,7 @@ public static Metrics loadMetrics(TrinoInputFile file, HiveStorageFormat storage private static Metrics parquetMetrics(TrinoInputFile file, MetricsConfig metricsConfig, NameMapping nameMapping) { try (ParquetDataSource dataSource = new TrinoParquetDataSource(file, new ParquetReaderOptions(), new FileFormatDataSourceStats())) { - ParquetMetadata metadata = MetadataReader.readFooter(dataSource, Optional.empty()); + ParquetMetadata metadata = MetadataReader.readFooter(dataSource, Optional.empty(), Optional.empty(), Optional.empty()); return ParquetUtil.footerMetrics(metadata, Stream.empty(), metricsConfig, nameMapping); } catch (IOException e) { diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/IcebergTestUtils.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/IcebergTestUtils.java index 24c18d4162ec2..493a6ddce569a 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/IcebergTestUtils.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/IcebergTestUtils.java @@ -131,7 +131,7 @@ public static boolean checkParquetFileSorting(TrinoInputFile inputFile, String s try { parquetMetadata = MetadataReader.readFooter( new TrinoParquetDataSource(inputFile, new ParquetReaderOptions(), new FileFormatDataSourceStats()), - Optional.empty()); + Optional.empty(), Optional.empty(), Optional.empty()); } catch (IOException e) { throw new UncheckedIOException(e);