hsiang-c commented on code in PR #1987: URL: https://github.com/apache/datafusion-comet/pull/1987#discussion_r2286152103
########## dev/diffs/iceberg/1.8.1.diff: ########## @@ -1,186 +1,721 @@ +diff --git a/build.gradle b/build.gradle +index 7327b38..7967109 100644 +--- a/build.gradle ++++ b/build.gradle +@@ -780,6 +780,13 @@ project(':iceberg-parquet') { + implementation project(':iceberg-core') + implementation project(':iceberg-common') + ++ implementation("org.apache.datafusion:comet-spark-spark${sparkVersionsString}_${scalaVersion}:${libs.versions.comet.get()}") { ++ exclude group: 'org.apache.arrow' ++ exclude group: 'org.apache.parquet' ++ exclude group: 'org.apache.spark' ++ exclude group: 'org.apache.iceberg' ++ } ++ + implementation(libs.parquet.avro) { + exclude group: 'org.apache.avro', module: 'avro' + // already shaded by Parquet diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml -index 04ffa8f..d4107be 100644 +index 04ffa8f..cc0099c 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml -@@ -81,7 +81,7 @@ slf4j = "2.0.16" +@@ -34,6 +34,7 @@ azuresdk-bom = "1.2.31" + awssdk-s3accessgrants = "2.3.0" + caffeine = "2.9.3" + calcite = "1.10.0" ++comet = "0.10.0-SNAPSHOT" + datasketches = "6.2.0" + delta-standalone = "3.3.0" + delta-spark = "3.3.0" +@@ -81,7 +82,7 @@ slf4j = "2.0.16" snowflake-jdbc = "3.22.0" spark-hive33 = "3.3.4" spark-hive34 = "3.4.4" -spark-hive35 = "3.5.4" -+spark-hive35 = "3.5.6-SNAPSHOT" ++spark-hive35 = "3.5.6" sqlite-jdbc = "3.48.0.0" testcontainers = "1.20.4" tez010 = "0.10.4" -diff --git a/spark/v3.4/build.gradle b/spark/v3.4/build.gradle -index 6eb26e8..50cefce 100644 ---- a/spark/v3.4/build.gradle -+++ b/spark/v3.4/build.gradle -@@ -75,7 +75,7 @@ project(":iceberg-spark:iceberg-spark-${sparkMajorVersion}_${scalaVersion}") { - exclude group: 'org.roaringbitmap' - } - -- compileOnly "org.apache.datafusion:comet-spark-spark${sparkMajorVersion}_${scalaVersion}:0.5.0" -+ compileOnly "org.apache.datafusion:comet-spark-spark${sparkMajorVersion}_${scalaVersion}:0.10.0-SNAPSHOT" - - implementation libs.parquet.column - implementation libs.parquet.hadoop -@@ -185,7 +185,7 @@ project(":iceberg-spark:iceberg-spark-extensions-${sparkMajorVersion}_${scalaVer - testImplementation libs.avro.avro - testImplementation libs.parquet.hadoop - testImplementation libs.junit.vintage.engine -- testImplementation "org.apache.datafusion:comet-spark-spark${sparkMajorVersion}_${scalaVersion}:0.5.0" -+ testImplementation "org.apache.datafusion:comet-spark-spark${sparkMajorVersion}_${scalaVersion}:0.10.0-SNAPSHOT" - - // Required because we remove antlr plugin dependencies from the compile configuration, see note above - runtimeOnly libs.antlr.runtime -@@ -260,6 +260,8 @@ project(":iceberg-spark:iceberg-spark-runtime-${sparkMajorVersion}_${scalaVersio - integrationImplementation project(path: ':iceberg-hive-metastore', configuration: 'testArtifacts') - integrationImplementation project(path: ":iceberg-spark:iceberg-spark-${sparkMajorVersion}_${scalaVersion}", configuration: 'testArtifacts') - integrationImplementation project(path: ":iceberg-spark:iceberg-spark-extensions-${sparkMajorVersion}_${scalaVersion}", configuration: 'testArtifacts') -+ integrationImplementation project(path: ':iceberg-parquet') -+ integrationImplementation "org.apache.datafusion:comet-spark-spark${sparkMajorVersion}_${scalaVersion}:0.10.0-SNAPSHOT" - - // runtime dependencies for running Hive Catalog based integration test - integrationRuntimeOnly project(':iceberg-hive-metastore') -@@ -297,8 +299,8 @@ project(":iceberg-spark:iceberg-spark-runtime-${sparkMajorVersion}_${scalaVersio - relocate 'org.apache.avro', 'org.apache.iceberg.shaded.org.apache.avro' - relocate 'avro.shaded', 'org.apache.iceberg.shaded.org.apache.avro.shaded' - relocate 'com.thoughtworks.paranamer', 'org.apache.iceberg.shaded.com.thoughtworks.paranamer' -- relocate 'org.apache.parquet', 'org.apache.iceberg.shaded.org.apache.parquet' -- relocate 'shaded.parquet', 'org.apache.iceberg.shaded.org.apache.parquet.shaded' -+// relocate 'org.apache.parquet', 'org.apache.iceberg.shaded.org.apache.parquet' -+// relocate 'shaded.parquet', 'org.apache.iceberg.shaded.org.apache.parquet.shaded' - relocate 'org.apache.orc', 'org.apache.iceberg.shaded.org.apache.orc' - relocate 'io.airlift', 'org.apache.iceberg.shaded.io.airlift' - relocate 'org.apache.hc.client5', 'org.apache.iceberg.shaded.org.apache.hc.client5' -diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java -index 0ca1236..87daef4 100644 ---- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java -+++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java -@@ -29,7 +29,7 @@ public class SparkSQLProperties { - - // Controls which Parquet reader implementation to use - public static final String PARQUET_READER_TYPE = "spark.sql.iceberg.parquet.reader-type"; -- public static final ParquetReaderType PARQUET_READER_TYPE_DEFAULT = ParquetReaderType.ICEBERG; -+ public static final ParquetReaderType PARQUET_READER_TYPE_DEFAULT = ParquetReaderType.COMET; - - // Controls whether reading/writing timestamps without timezones is allowed - @Deprecated -diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnReader.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnReader.java -index 4794863..0be31c1 100644 ---- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnReader.java -+++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnReader.java -@@ -20,11 +20,11 @@ package org.apache.iceberg.spark.data.vectorized; +diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/CometTypeUtils.java b/parquet/src/main/java/org/apache/iceberg/parquet/CometTypeUtils.java +new file mode 100644 +index 0000000..ddf6c7d +--- /dev/null ++++ b/parquet/src/main/java/org/apache/iceberg/parquet/CometTypeUtils.java +@@ -0,0 +1,255 @@ ++/* ++ * 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 java.util.Map; ++import org.apache.comet.parquet.ParquetColumnSpec; ++import org.apache.iceberg.relocated.com.google.common.collect.Maps; ++import org.apache.parquet.column.ColumnDescriptor; ++import org.apache.parquet.schema.LogicalTypeAnnotation; ++import org.apache.parquet.schema.PrimitiveType; ++import org.apache.parquet.schema.Type; ++import org.apache.parquet.schema.Types; ++ ++public class CometTypeUtils { ++ ++ private CometTypeUtils() {} ++ ++ public static ParquetColumnSpec descriptorToParquetColumnSpec(ColumnDescriptor descriptor) { ++ ++ String[] path = descriptor.getPath(); ++ PrimitiveType primitiveType = descriptor.getPrimitiveType(); ++ String physicalType = primitiveType.getPrimitiveTypeName().name(); ++ ++ int typeLength = ++ primitiveType.getPrimitiveTypeName() == PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY ++ ? primitiveType.getTypeLength() ++ : 0; ++ ++ boolean isRepeated = primitiveType.getRepetition() == Type.Repetition.REPEATED; ++ ++ // ToDo: extract this into a Util method ++ String logicalTypeName = null; ++ Map<String, String> logicalTypeParams = Maps.newHashMap(); ++ LogicalTypeAnnotation logicalType = primitiveType.getLogicalTypeAnnotation(); ++ ++ if (logicalType != null) { ++ logicalTypeName = logicalType.getClass().getSimpleName(); ++ ++ // Handle specific logical types ++ if (logicalType instanceof LogicalTypeAnnotation.DecimalLogicalTypeAnnotation) { ++ LogicalTypeAnnotation.DecimalLogicalTypeAnnotation decimal = ++ (LogicalTypeAnnotation.DecimalLogicalTypeAnnotation) logicalType; ++ logicalTypeParams.put("precision", String.valueOf(decimal.getPrecision())); ++ logicalTypeParams.put("scale", String.valueOf(decimal.getScale())); ++ } else if (logicalType instanceof LogicalTypeAnnotation.TimestampLogicalTypeAnnotation) { ++ LogicalTypeAnnotation.TimestampLogicalTypeAnnotation timestamp = ++ (LogicalTypeAnnotation.TimestampLogicalTypeAnnotation) logicalType; ++ logicalTypeParams.put("isAdjustedToUTC", String.valueOf(timestamp.isAdjustedToUTC())); ++ logicalTypeParams.put("unit", timestamp.getUnit().name()); ++ } else if (logicalType instanceof LogicalTypeAnnotation.TimeLogicalTypeAnnotation) { ++ LogicalTypeAnnotation.TimeLogicalTypeAnnotation time = ++ (LogicalTypeAnnotation.TimeLogicalTypeAnnotation) logicalType; ++ logicalTypeParams.put("isAdjustedToUTC", String.valueOf(time.isAdjustedToUTC())); ++ logicalTypeParams.put("unit", time.getUnit().name()); ++ } else if (logicalType instanceof LogicalTypeAnnotation.IntLogicalTypeAnnotation) { ++ LogicalTypeAnnotation.IntLogicalTypeAnnotation intType = ++ (LogicalTypeAnnotation.IntLogicalTypeAnnotation) logicalType; ++ logicalTypeParams.put("isSigned", String.valueOf(intType.isSigned())); ++ logicalTypeParams.put("bitWidth", String.valueOf(intType.getBitWidth())); ++ } ++ } ++ ++ return new ParquetColumnSpec( ++ 1, // ToDo: pass in the correct id ++ path, ++ physicalType, ++ typeLength, ++ isRepeated, ++ descriptor.getMaxDefinitionLevel(), ++ descriptor.getMaxRepetitionLevel(), ++ logicalTypeName, ++ logicalTypeParams); ++ } ++ ++ public static ColumnDescriptor buildColumnDescriptor(ParquetColumnSpec columnSpec) { ++ PrimitiveType.PrimitiveTypeName primType = ++ PrimitiveType.PrimitiveTypeName.valueOf(columnSpec.getPhysicalType()); ++ ++ Type.Repetition repetition; ++ if (columnSpec.getMaxRepetitionLevel() > 0) { ++ repetition = Type.Repetition.REPEATED; ++ } else if (columnSpec.getMaxDefinitionLevel() > 0) { ++ repetition = Type.Repetition.OPTIONAL; ++ } else { ++ repetition = Type.Repetition.REQUIRED; ++ } ++ ++ String name = columnSpec.getPath()[columnSpec.getPath().length - 1]; ++ // Reconstruct the logical type from parameters ++ LogicalTypeAnnotation logicalType = null; ++ if (columnSpec.getLogicalTypeName() != null) { ++ logicalType = ++ reconstructLogicalType( ++ columnSpec.getLogicalTypeName(), columnSpec.getLogicalTypeParams()); ++ } ++ ++ PrimitiveType primitiveType; ++ if (primType == PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY) { ++ primitiveType = ++ org.apache.parquet.schema.Types.primitive(primType, repetition) ++ .length(columnSpec.getTypeLength()) ++ .as(logicalType) ++ .id(columnSpec.getFieldId()) ++ .named(name); ++ } else { ++ primitiveType = ++ Types.primitive(primType, repetition) ++ .as(logicalType) ++ .id(columnSpec.getFieldId()) ++ .named(name); ++ } ++ ++ return new ColumnDescriptor( ++ columnSpec.getPath(), ++ primitiveType, ++ columnSpec.getMaxRepetitionLevel(), ++ columnSpec.getMaxDefinitionLevel()); ++ } ++ ++ private static LogicalTypeAnnotation reconstructLogicalType( ++ String logicalTypeName, java.util.Map<String, String> params) { ++ ++ switch (logicalTypeName) { ++ // MAP ++ case "MapLogicalTypeAnnotation": ++ return LogicalTypeAnnotation.mapType(); ++ ++ // LIST ++ case "ListLogicalTypeAnnotation": ++ return LogicalTypeAnnotation.listType(); ++ ++ // STRING ++ case "StringLogicalTypeAnnotation": ++ return LogicalTypeAnnotation.stringType(); ++ ++ // MAP_KEY_VALUE ++ case "MapKeyValueLogicalTypeAnnotation": ++ return LogicalTypeAnnotation.MapKeyValueTypeAnnotation.getInstance(); ++ ++ // ENUM ++ case "EnumLogicalTypeAnnotation": ++ return LogicalTypeAnnotation.enumType(); ++ ++ // DECIMAL ++ case "DecimalLogicalTypeAnnotation": ++ if (!params.containsKey("scale") || !params.containsKey("precision")) { ++ throw new IllegalArgumentException( ++ "Missing required parameters for DecimalLogicalTypeAnnotation: " + params); ++ } ++ int scale = Integer.parseInt(params.get("scale")); ++ int precision = Integer.parseInt(params.get("precision")); ++ return LogicalTypeAnnotation.decimalType(scale, precision); ++ ++ // DATE ++ case "DateLogicalTypeAnnotation": ++ return LogicalTypeAnnotation.dateType(); ++ ++ // TIME ++ case "TimeLogicalTypeAnnotation": ++ if (!params.containsKey("isAdjustedToUTC") || !params.containsKey("unit")) { ++ throw new IllegalArgumentException( ++ "Missing required parameters for TimeLogicalTypeAnnotation: " + params); ++ } ++ ++ boolean isUTC = Boolean.parseBoolean(params.get("isAdjustedToUTC")); ++ String timeUnitStr = params.get("unit"); ++ ++ LogicalTypeAnnotation.TimeUnit timeUnit; ++ switch (timeUnitStr) { ++ case "MILLIS": ++ timeUnit = LogicalTypeAnnotation.TimeUnit.MILLIS; ++ break; ++ case "MICROS": ++ timeUnit = LogicalTypeAnnotation.TimeUnit.MICROS; ++ break; ++ case "NANOS": ++ timeUnit = LogicalTypeAnnotation.TimeUnit.NANOS; ++ break; ++ default: ++ throw new IllegalArgumentException("Unknown time unit: " + timeUnitStr); ++ } ++ return LogicalTypeAnnotation.timeType(isUTC, timeUnit); ++ ++ // TIMESTAMP ++ case "TimestampLogicalTypeAnnotation": ++ if (!params.containsKey("isAdjustedToUTC") || !params.containsKey("unit")) { ++ throw new IllegalArgumentException( ++ "Missing required parameters for TimestampLogicalTypeAnnotation: " + params); ++ } ++ boolean isAdjustedToUTC = Boolean.parseBoolean(params.get("isAdjustedToUTC")); ++ String unitStr = params.get("unit"); ++ ++ LogicalTypeAnnotation.TimeUnit unit; ++ switch (unitStr) { ++ case "MILLIS": ++ unit = LogicalTypeAnnotation.TimeUnit.MILLIS; ++ break; ++ case "MICROS": ++ unit = LogicalTypeAnnotation.TimeUnit.MICROS; ++ break; ++ case "NANOS": ++ unit = LogicalTypeAnnotation.TimeUnit.NANOS; ++ break; ++ default: ++ throw new IllegalArgumentException("Unknown timestamp unit: " + unitStr); ++ } ++ return LogicalTypeAnnotation.timestampType(isAdjustedToUTC, unit); ++ ++ // INTEGER ++ case "IntLogicalTypeAnnotation": ++ if (!params.containsKey("isSigned") || !params.containsKey("bitWidth")) { ++ throw new IllegalArgumentException( ++ "Missing required parameters for IntLogicalTypeAnnotation: " + params); ++ } ++ boolean isSigned = Boolean.parseBoolean(params.get("isSigned")); ++ int bitWidth = Integer.parseInt(params.get("bitWidth")); ++ return LogicalTypeAnnotation.intType(bitWidth, isSigned); ++ ++ // JSON ++ case "JsonLogicalTypeAnnotation": ++ return LogicalTypeAnnotation.jsonType(); ++ ++ // BSON ++ case "BsonLogicalTypeAnnotation": ++ return LogicalTypeAnnotation.bsonType(); ++ ++ // UUID ++ case "UUIDLogicalTypeAnnotation": ++ return LogicalTypeAnnotation.uuidType(); ++ ++ // INTERVAL ++ case "IntervalLogicalTypeAnnotation": ++ return LogicalTypeAnnotation.IntervalLogicalTypeAnnotation.getInstance(); ++ ++ default: ++ throw new IllegalArgumentException("Unknown logical type: " + logicalTypeName); ++ } ++ } ++} +diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/CometVectorizedParquetReader.java b/parquet/src/main/java/org/apache/iceberg/parquet/CometVectorizedParquetReader.java +new file mode 100644 +index 0000000..88b195b +--- /dev/null ++++ b/parquet/src/main/java/org/apache/iceberg/parquet/CometVectorizedParquetReader.java +@@ -0,0 +1,255 @@ ++/* ++ * 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 java.io.IOException; ++import java.io.UncheckedIOException; ++import java.nio.ByteBuffer; ++import java.util.List; ++import java.util.Map; ++import java.util.NoSuchElementException; ++import java.util.function.Function; ++import org.apache.comet.parquet.FileReader; ++import org.apache.comet.parquet.ParquetColumnSpec; ++import org.apache.comet.parquet.ReadOptions; ++import org.apache.comet.parquet.RowGroupReader; ++import org.apache.comet.parquet.WrappedInputFile; ++import org.apache.hadoop.conf.Configuration; ++import org.apache.iceberg.Schema; ++import org.apache.iceberg.exceptions.RuntimeIOException; ++import org.apache.iceberg.expressions.Expression; ++import org.apache.iceberg.expressions.Expressions; ++import org.apache.iceberg.io.CloseableGroup; ++import org.apache.iceberg.io.CloseableIterable; ++import org.apache.iceberg.io.CloseableIterator; ++import org.apache.iceberg.io.InputFile; ++import org.apache.iceberg.mapping.NameMapping; ++import org.apache.iceberg.relocated.com.google.common.collect.Lists; ++import org.apache.iceberg.util.ByteBuffers; ++import org.apache.parquet.ParquetReadOptions; ++import org.apache.parquet.column.ColumnDescriptor; ++import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; ++import org.apache.parquet.hadoop.metadata.ColumnPath; ++import org.apache.parquet.schema.MessageType; ++ ++public class CometVectorizedParquetReader<T> extends CloseableGroup ++ implements CloseableIterable<T> { ++ private final InputFile input; ++ private final ParquetReadOptions options; ++ private final Schema expectedSchema; ++ private final Function<MessageType, VectorizedReader<?>> batchReaderFunc; ++ private final Expression filter; ++ private final boolean reuseContainers; ++ private final boolean caseSensitive; ++ private final int batchSize; ++ private final NameMapping nameMapping; ++ private final Map<String, String> properties; ++ private Long start = null; ++ private Long length = null; ++ private ByteBuffer fileEncryptionKey = null; ++ private ByteBuffer fileAADPrefix = null; ++ ++ public CometVectorizedParquetReader( ++ InputFile input, ++ Schema expectedSchema, ++ ParquetReadOptions options, ++ Function<MessageType, VectorizedReader<?>> readerFunc, ++ NameMapping nameMapping, ++ Expression filter, ++ boolean reuseContainers, ++ boolean caseSensitive, ++ int maxRecordsPerBatch, ++ Map<String, String> properties, ++ Long start, ++ Long length, ++ ByteBuffer fileEncryptionKey, ++ ByteBuffer fileAADPrefix) { ++ this.input = input; ++ this.expectedSchema = expectedSchema; ++ this.options = options; ++ this.batchReaderFunc = readerFunc; ++ // replace alwaysTrue with null to avoid extra work evaluating a trivial filter ++ this.filter = filter == Expressions.alwaysTrue() ? null : filter; ++ this.reuseContainers = reuseContainers; ++ this.caseSensitive = caseSensitive; ++ this.batchSize = maxRecordsPerBatch; ++ this.nameMapping = nameMapping; ++ this.properties = properties; ++ this.start = start; ++ this.length = length; ++ this.fileEncryptionKey = fileEncryptionKey; ++ this.fileAADPrefix = fileAADPrefix; ++ } ++ ++ private ReadConf conf = null; ++ ++ private ReadConf init() { ++ if (conf == null) { ++ ReadConf readConf = ++ new ReadConf( ++ input, ++ options, ++ expectedSchema, ++ filter, ++ null, ++ batchReaderFunc, ++ nameMapping, ++ reuseContainers, ++ caseSensitive, ++ batchSize); ++ this.conf = readConf.copy(); ++ return readConf; ++ } ++ return conf; ++ } ++ ++ @Override ++ public CloseableIterator<T> iterator() { ++ FileIterator<T> iter = ++ new FileIterator<>(init(), properties, start, length, fileEncryptionKey, fileAADPrefix); ++ addCloseable(iter); ++ return iter; ++ } ++ ++ private static class FileIterator<T> implements CloseableIterator<T> { ++ // private final ParquetFileReader reader; ++ private final boolean[] shouldSkip; ++ private final VectorizedReader<T> model; ++ private final long totalValues; ++ private final int batchSize; ++ private final List<Map<ColumnPath, ColumnChunkMetaData>> columnChunkMetadata; ++ private final boolean reuseContainers; ++ private int nextRowGroup = 0; ++ private long nextRowGroupStart = 0; ++ private long valuesRead = 0; ++ private T last = null; ++ private final FileReader cometReader; ++ ++ FileIterator( ++ ReadConf conf, ++ Map<String, String> properties, ++ Long start, ++ Long length, ++ ByteBuffer fileEncryptionKey, ++ ByteBuffer fileAADPrefix) { ++ this.shouldSkip = conf.shouldSkip(); ++ this.totalValues = conf.totalValues(); ++ this.reuseContainers = conf.reuseContainers(); ++ this.model = conf.vectorizedModel(); ++ this.batchSize = conf.batchSize(); ++ this.model.setBatchSize(this.batchSize); ++ this.columnChunkMetadata = conf.columnChunkMetadataForRowGroups(); ++ this.cometReader = ++ newCometReader( ++ conf.file(), ++ conf.projection(), ++ properties, ++ start, ++ length, ++ fileEncryptionKey, ++ fileAADPrefix); ++ } ++ ++ private FileReader newCometReader( ++ InputFile file, ++ MessageType projection, ++ Map<String, String> properties, ++ Long start, ++ Long length, ++ ByteBuffer fileEncryptionKey, ++ ByteBuffer fileAADPrefix) { ++ try { ++ ReadOptions cometOptions = ReadOptions.builder(new Configuration()).build(); ++ ++ FileReader fileReader = ++ new FileReader( ++ new WrappedInputFile(file), ++ cometOptions, ++ properties, ++ start, ++ length, ++ ByteBuffers.toByteArray(fileEncryptionKey), ++ ByteBuffers.toByteArray(fileAADPrefix)); ++ ++ List<ColumnDescriptor> columnDescriptors = projection.getColumns(); ++ ++ List<ParquetColumnSpec> specs = Lists.newArrayList(); ++ ++ for (ColumnDescriptor descriptor : columnDescriptors) { ++ ParquetColumnSpec spec = CometTypeUtils.descriptorToParquetColumnSpec(descriptor); ++ specs.add(spec); ++ } ++ ++ fileReader.setRequestedSchemaFromSpecs(specs); ++ return fileReader; ++ } catch (IOException e) { ++ throw new UncheckedIOException("Failed to open Parquet file: " + file.location(), e); ++ } ++ } ++ ++ @Override ++ public boolean hasNext() { ++ return valuesRead < totalValues; ++ } ++ ++ @Override ++ public T next() { ++ if (!hasNext()) { ++ throw new NoSuchElementException(); ++ } ++ if (valuesRead >= nextRowGroupStart) { ++ advance(); ++ } ++ ++ // batchSize is an integer, so casting to integer is safe ++ int numValuesToRead = (int) Math.min(nextRowGroupStart - valuesRead, batchSize); ++ if (reuseContainers) { ++ this.last = model.read(last, numValuesToRead); ++ } else { ++ this.last = model.read(null, numValuesToRead); ++ } ++ valuesRead += numValuesToRead; ++ ++ return last; ++ } ++ ++ private void advance() { ++ while (shouldSkip[nextRowGroup]) { ++ nextRowGroup += 1; ++ cometReader.skipNextRowGroup(); ++ } ++ RowGroupReader pages; ++ try { ++ pages = cometReader.readNextRowGroup(); ++ } catch (IOException e) { ++ throw new RuntimeIOException(e); ++ } ++ ++ model.setRowGroupInfo(pages, columnChunkMetadata.get(nextRowGroup)); ++ nextRowGroupStart += pages.getRowCount(); ++ nextRowGroup += 1; ++ } ++ ++ @Override ++ public void close() throws IOException { ++ model.close(); ++ cometReader.close(); ++ } ++ } ++} +diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java b/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java +<<<<<<< Updated upstream +index 2c37a52..503dbd6 100644 +======= +index 2c37a52..3442cfc 100644 +>>>>>>> Stashed changes Review Comment: Sorry about that, fixed now. -- 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: github-unsubscr...@datafusion.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: github-unsubscr...@datafusion.apache.org For additional commands, e-mail: github-h...@datafusion.apache.org