From 8df3ab41dae529c88840dce747c41210a210effc Mon Sep 17 00:00:00 2001 From: huaxingao Date: Tue, 24 Jun 2025 15:20:26 -0700 Subject: [PATCH 01/15] Spark:3.4 Encapsulate parquet objects for Comet --- build.gradle | 8 + .../parquet/CometVectorizedParquetReader.java | 315 ++++++++++++++++++ .../org/apache/iceberg/parquet/Parquet.java | 47 ++- .../org/apache/iceberg/parquet/ReadConf.java | 8 + .../data/vectorized/CometColumnReader.java | 92 ++++- .../vectorized/CometColumnarBatchReader.java | 19 +- .../CometVectorizedReaderBuilder.java | 3 +- .../iceberg/spark/source/BaseBatchReader.java | 1 + 8 files changed, 467 insertions(+), 26 deletions(-) create mode 100644 parquet/src/main/java/org/apache/iceberg/parquet/CometVectorizedParquetReader.java diff --git a/build.gradle b/build.gradle index bf09d1b492ff..79603391c2ea 100644 --- a/build.gradle +++ b/build.gradle @@ -845,6 +845,7 @@ project(':iceberg-orc') { } project(':iceberg-parquet') { + test { useJUnitPlatform() } @@ -854,6 +855,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/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 000000000000..5b5555779e4d --- /dev/null +++ b/parquet/src/main/java/org/apache/iceberg/parquet/CometVectorizedParquetReader.java @@ -0,0 +1,315 @@ +/* + * 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.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.hadoop.HadoopInputFile; +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.relocated.com.google.common.collect.Maps; +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.LogicalTypeAnnotation; +import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.PrimitiveType; +import org.apache.parquet.schema.Type; + +public class CometVectorizedParquetReader extends CloseableGroup + implements CloseableIterable { + private final InputFile input; + private final ParquetReadOptions options; + private final Schema expectedSchema; + private final Function> batchReaderFunc; + private final Expression filter; + private final boolean reuseContainers; + private final boolean caseSensitive; + private final int batchSize; + private final NameMapping nameMapping; + private final Map 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> readerFunc, + NameMapping nameMapping, + Expression filter, + boolean reuseContainers, + boolean caseSensitive, + int maxRecordsPerBatch, + Map 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 iterator() { + FileIterator iter = + new FileIterator<>(init(), properties, start, length, fileEncryptionKey, fileAADPrefix); + addCloseable(iter); + return iter; + } + + private static class FileIterator implements CloseableIterator { + // private final ParquetFileReader reader; + private final boolean[] shouldSkip; + private final VectorizedReader model; + private final long totalValues; + private final int batchSize; + private final List> 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 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 properties, + Long start, + Long length, + ByteBuffer fileEncryptionKey, + ByteBuffer fileAADPrefix) { + try { + ReadOptions cometOptions = ReadOptions.builder(new Configuration()).build(); + + FileReader fileReader = + new FileReader( + ((HadoopInputFile) file).getPath(), + new Configuration(((HadoopInputFile) file).getConf()), + cometOptions, + properties, + start, + length, + ByteBuffers.toByteArray(fileEncryptionKey), + ByteBuffers.toByteArray(fileAADPrefix)); + + List columnDescriptors = projection.getColumns(); + + List specs = Lists.newArrayList(); + + for (ColumnDescriptor descriptor : columnDescriptors) { + 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 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())); + } + } + + ParquetColumnSpec spec = + new ParquetColumnSpec( + 1, // ToDo: pass in the correct id + path, + physicalType, + typeLength, + isRepeated, + descriptor.getMaxDefinitionLevel(), + descriptor.getMaxRepetitionLevel(), + logicalTypeName, + logicalTypeParams); + 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 index 6f68fbe150ff..f9df53b038b8 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java @@ -1161,6 +1161,7 @@ public static class ReadBuilder implements InternalData.ReadBuilder { private NameMapping nameMapping = null; private ByteBuffer fileEncryptionKey = null; private ByteBuffer fileAADPrefix = null; + private boolean isComet; private ReadBuilder(InputFile file) { this.file = file; @@ -1289,6 +1290,11 @@ public ReadBuilder setCustomType(int fieldId, Class struct throw new UnsupportedOperationException("Custom types are not yet supported"); } + public ReadBuilder enableComet(boolean enableComet) { + this.isComet = enableComet; + return this; + } + public ReadBuilder withFileEncryptionKey(ByteBuffer encryptionKey) { this.fileEncryptionKey = encryptionKey; return this; @@ -1300,7 +1306,7 @@ public ReadBuilder withAADPrefix(ByteBuffer aadPrefix) { } @Override - @SuppressWarnings({"unchecked", "checkstyle:CyclomaticComplexity"}) + @SuppressWarnings({"unchecked", "checkstyle:CyclomaticComplexity", "MethodLength"}) public CloseableIterable build() { FileDecryptionProperties fileDecryptionProperties = null; if (fileEncryptionKey != null) { @@ -1352,16 +1358,35 @@ public CloseableIterable build() { } if (batchedReaderFunc != null) { - return new VectorizedParquetReader<>( - file, - schema, - options, - batchedReaderFunc, - mapping, - filter, - reuseContainers, - caseSensitive, - maxRecordsPerBatch); + if (isComet) { + LOG.info("Comet enabled"); + return new CometVectorizedParquetReader<>( + file, + schema, + options, + batchedReaderFunc, + mapping, + filter, + reuseContainers, + caseSensitive, + maxRecordsPerBatch, + properties, + start, + length, + fileEncryptionKey, + fileAADPrefix); + } else { + return new VectorizedParquetReader<>( + file, + schema, + options, + batchedReaderFunc, + mapping, + filter, + reuseContainers, + caseSensitive, + maxRecordsPerBatch); + } } else { Function> readBuilder = readerFuncWithSchema != null diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ReadConf.java b/parquet/src/main/java/org/apache/iceberg/parquet/ReadConf.java index 1fb2372ba568..142e5fbadf1f 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ReadConf.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ReadConf.java @@ -157,6 +157,14 @@ ParquetFileReader reader() { return newReader; } + InputFile file() { + return file; + } + + MessageType projection() { + return projection; + } + ParquetValueReader model() { return model; } 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 81b7d83a7077..1ddd4d0dfd4d 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 @@ -19,18 +19,26 @@ package org.apache.iceberg.spark.data.vectorized; import java.io.IOException; +import java.util.Map; +import org.apache.comet.CometConf; import org.apache.comet.CometSchemaImporter; import org.apache.comet.parquet.AbstractColumnReader; import org.apache.comet.parquet.ColumnReader; +import org.apache.comet.parquet.ParquetColumnSpec; +import org.apache.comet.parquet.RowGroupReader; import org.apache.comet.parquet.TypeUtil; import org.apache.comet.parquet.Utils; import org.apache.comet.shaded.arrow.memory.RootAllocator; import org.apache.iceberg.parquet.VectorizedReader; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.spark.SparkSchemaUtil; import org.apache.iceberg.types.Types; import org.apache.parquet.column.ColumnDescriptor; -import org.apache.parquet.column.page.PageReader; +import org.apache.parquet.schema.LogicalTypeAnnotation; +import org.apache.parquet.schema.PrimitiveType; +import org.apache.parquet.schema.Type; +import org.apache.spark.sql.internal.SQLConf; import org.apache.spark.sql.types.DataType; import org.apache.spark.sql.types.Metadata; import org.apache.spark.sql.types.StructField; @@ -42,16 +50,19 @@ class CometColumnReader implements VectorizedReader { private final ColumnDescriptor descriptor; private final DataType sparkType; + private final int fieldId; // The delegated ColumnReader from Comet side private AbstractColumnReader delegate; private boolean initialized = false; private int batchSize = DEFAULT_BATCH_SIZE; private CometSchemaImporter importer; + private ParquetColumnSpec spec; - CometColumnReader(DataType sparkType, ColumnDescriptor descriptor) { + CometColumnReader(DataType sparkType, ColumnDescriptor descriptor, int fieldId) { this.sparkType = sparkType; this.descriptor = descriptor; + this.fieldId = fieldId; } CometColumnReader(Types.NestedField field) { @@ -59,6 +70,7 @@ class CometColumnReader implements VectorizedReader { StructField structField = new StructField(field.name(), dataType, false, Metadata.empty()); this.sparkType = dataType; this.descriptor = TypeUtil.convertToParquet(structField); + this.fieldId = field.fieldId(); } public AbstractColumnReader delegate() { @@ -92,7 +104,77 @@ public void reset() { } this.importer = new CometSchemaImporter(new RootAllocator()); - this.delegate = Utils.getColumnReader(sparkType, descriptor, importer, batchSize, false, false); + + String[] path = descriptor.getPath(); + PrimitiveType primitiveType = descriptor.getPrimitiveType(); + String physicalType = primitiveType.getPrimitiveTypeName().name(); + + // ToDo: extract this into a Util method + String logicalTypeName = null; + Map 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())); + } + } + + int typeLength = + primitiveType.getPrimitiveTypeName() == PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY + ? primitiveType.getTypeLength() + : 0; + boolean isRepeated = primitiveType.getRepetition() == Type.Repetition.REPEATED; + spec = + new ParquetColumnSpec( + fieldId, + path, + physicalType, + typeLength, + isRepeated, + descriptor.getMaxDefinitionLevel(), + descriptor.getMaxRepetitionLevel(), + logicalTypeName, + logicalTypeParams); + + boolean useLegacyTime = + Boolean.parseBoolean( + SQLConf.get() + .getConfString( + CometConf.COMET_EXCEPTION_ON_LEGACY_DATE_TIMESTAMP().key(), "false")); + boolean useLazyMaterialization = + Boolean.parseBoolean( + SQLConf.get().getConfString(CometConf.COMET_USE_LAZY_MATERIALIZATION().key(), "true")); + this.delegate = + Utils.getColumnReader( + sparkType, + spec, + importer, + batchSize, + true, // Comet sets this to true for native execution + useLazyMaterialization, + useLegacyTime); this.initialized = true; } @@ -111,9 +193,9 @@ public DataType sparkType() { *

NOTE: this should be called before reading a new Parquet column chunk, and after {@link * CometColumnReader#reset} is called. */ - public void setPageReader(PageReader pageReader) throws IOException { + public void setPageReader(RowGroupReader pageStore) throws IOException { Preconditions.checkState(initialized, "Invalid state: 'reset' should be called first"); - ((ColumnReader) delegate).setPageReader(pageReader); + ((ColumnReader) delegate).setRowGroupReader(pageStore, spec); } @Override diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnarBatchReader.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnarBatchReader.java index 04ac69476add..52d19ec2daf7 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnarBatchReader.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnarBatchReader.java @@ -23,7 +23,8 @@ import java.util.List; import java.util.Map; import org.apache.comet.parquet.AbstractColumnReader; -import org.apache.comet.parquet.BatchReader; +import org.apache.comet.parquet.IcebergCometBatchReader; +import org.apache.comet.parquet.RowGroupReader; import org.apache.iceberg.Schema; import org.apache.iceberg.data.DeleteFilter; import org.apache.iceberg.parquet.VectorizedReader; @@ -55,7 +56,7 @@ class CometColumnarBatchReader implements VectorizedReader { // calling BatchReader.nextBatch, the isDeleted value is not yet available, so // DeleteColumnReader.readBatch must be called explicitly later, after the isDeleted value is // available. - private final BatchReader delegate; + private final IcebergCometBatchReader delegate; private DeleteFilter deletes = null; private long rowStartPosInBatch = 0; @@ -64,10 +65,7 @@ class CometColumnarBatchReader implements VectorizedReader { readers.stream().map(CometColumnReader.class::cast).toArray(CometColumnReader[]::new); this.hasIsDeletedColumn = readers.stream().anyMatch(reader -> reader instanceof CometDeleteColumnReader); - - AbstractColumnReader[] abstractColumnReaders = new AbstractColumnReader[readers.size()]; - this.delegate = new BatchReader(abstractColumnReaders); - delegate.setSparkSchema(SparkSchemaUtil.convert(schema)); + this.delegate = new IcebergCometBatchReader(readers.size(), SparkSchemaUtil.convert(schema)); } @Override @@ -79,19 +77,22 @@ public void setRowGroupInfo( && !(readers[i] instanceof CometPositionColumnReader) && !(readers[i] instanceof CometDeleteColumnReader)) { readers[i].reset(); - readers[i].setPageReader(pageStore.getPageReader(readers[i].descriptor())); + readers[i].setPageReader((RowGroupReader) pageStore); } } catch (IOException e) { throw new UncheckedIOException("Failed to setRowGroupInfo for Comet vectorization", e); } } + AbstractColumnReader[] delegateReaders = new AbstractColumnReader[readers.length]; for (int i = 0; i < readers.length; i++) { - delegate.getColumnReaders()[i] = this.readers[i].delegate(); + delegateReaders[i] = readers[i].delegate(); } + delegate.init(delegateReaders); + this.rowStartPosInBatch = - pageStore + ((RowGroupReader) pageStore) .getRowIndexOffset() .orElseThrow( () -> diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometVectorizedReaderBuilder.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometVectorizedReaderBuilder.java index d36f1a727477..56f8c9bff933 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometVectorizedReaderBuilder.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometVectorizedReaderBuilder.java @@ -142,6 +142,7 @@ public VectorizedReader primitive( return null; } - return new CometColumnReader(SparkSchemaUtil.convert(icebergField.type()), desc); + return new CometColumnReader( + SparkSchemaUtil.convert(icebergField.type()), desc, parquetFieldId); } } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java index 780e1750a52e..57892ac4c59d 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java @@ -109,6 +109,7 @@ private CloseableIterable newParquetIterable( // read performance as every batch read doesn't have to pay the cost of allocating memory. .reuseContainers() .withNameMapping(nameMapping()) + .enableComet(parquetConf.readerType() == ParquetReaderType.COMET) .build(); } From ece26bb1493010c9b39f7f6687dc81ac302c1c0b Mon Sep 17 00:00:00 2001 From: Parth Chandra Date: Tue, 5 Aug 2025 17:44:31 -0700 Subject: [PATCH 02/15] Update for changes to Comet apis. better message Pass correct field id update to use comet 0.10.0 fix typo remove changes for Spark 4.0 Revert "remove changes for Spark 4.0" This reverts commit 1fe55674fd29635ccb38b9b1318168b279237b7a. --- gradle/libs.versions.toml | 2 +- .../iceberg/parquet/CometTypeUtils.java | 260 ++++++++++++++++++ .../parquet/CometVectorizedParquetReader.java | 66 +---- .../org/apache/iceberg/parquet/Parquet.java | 2 +- spark/v3.4/build.gradle | 2 + .../data/vectorized/CometColumnReader.java | 64 +---- .../vectorized/CometConstantColumnReader.java | 7 +- .../vectorized/CometDeleteColumnReader.java | 2 +- .../vectorized/CometPositionColumnReader.java | 3 +- spark/v3.5/build.gradle | 2 + .../data/vectorized/CometColumnReader.java | 40 ++- .../vectorized/CometColumnarBatchReader.java | 19 +- .../vectorized/CometConstantColumnReader.java | 7 +- .../vectorized/CometDeleteColumnReader.java | 4 +- .../vectorized/CometPositionColumnReader.java | 3 +- .../CometVectorizedReaderBuilder.java | 3 +- .../iceberg/spark/source/BaseBatchReader.java | 1 + spark/v4.0/build.gradle | 2 + .../data/vectorized/CometColumnReader.java | 40 ++- .../vectorized/CometColumnarBatchReader.java | 18 +- .../vectorized/CometConstantColumnReader.java | 7 +- .../vectorized/CometDeleteColumnReader.java | 2 +- .../vectorized/CometPositionColumnReader.java | 3 +- .../CometVectorizedReaderBuilder.java | 3 +- .../iceberg/spark/source/BaseBatchReader.java | 1 + 25 files changed, 398 insertions(+), 165 deletions(-) create mode 100644 parquet/src/main/java/org/apache/iceberg/parquet/CometTypeUtils.java diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index e4555ff70942..3419e708edd7 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -37,7 +37,7 @@ awssdk-s3accessgrants = "2.3.0" bson-ver = "4.11.5" caffeine = "2.9.3" calcite = "1.40.0" -comet = "0.8.1" +comet = "0.10.0" datasketches = "6.2.0" delta-standalone = "3.3.2" delta-spark = "3.3.2" 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 000000000000..8f4305ce0681 --- /dev/null +++ b/parquet/src/main/java/org/apache/iceberg/parquet/CometTypeUtils.java @@ -0,0 +1,260 @@ +/* + * 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; + + String logicalTypeName = null; + Map 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())); + } + } + + int id = -1; + Type type = descriptor.getPrimitiveType(); + if (type != null && type.getId() != null) { + id = type.getId().intValue(); + } + + return new ParquetColumnSpec( + 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 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 index 5b5555779e4d..88b195b76a2d 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/CometVectorizedParquetReader.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/CometVectorizedParquetReader.java @@ -29,28 +29,24 @@ 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.hadoop.HadoopInputFile; 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.relocated.com.google.common.collect.Maps; 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.LogicalTypeAnnotation; import org.apache.parquet.schema.MessageType; -import org.apache.parquet.schema.PrimitiveType; -import org.apache.parquet.schema.Type; public class CometVectorizedParquetReader extends CloseableGroup implements CloseableIterable { @@ -183,8 +179,7 @@ private FileReader newCometReader( FileReader fileReader = new FileReader( - ((HadoopInputFile) file).getPath(), - new Configuration(((HadoopInputFile) file).getConf()), + new WrappedInputFile(file), cometOptions, properties, start, @@ -197,62 +192,7 @@ private FileReader newCometReader( List specs = Lists.newArrayList(); for (ColumnDescriptor descriptor : columnDescriptors) { - 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 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())); - } - } - - ParquetColumnSpec spec = - new ParquetColumnSpec( - 1, // ToDo: pass in the correct id - path, - physicalType, - typeLength, - isRepeated, - descriptor.getMaxDefinitionLevel(), - descriptor.getMaxRepetitionLevel(), - logicalTypeName, - logicalTypeParams); + ParquetColumnSpec spec = CometTypeUtils.descriptorToParquetColumnSpec(descriptor); specs.add(spec); } diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java b/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java index f9df53b038b8..bc173c7b6411 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java @@ -1359,7 +1359,7 @@ public CloseableIterable build() { if (batchedReaderFunc != null) { if (isComet) { - LOG.info("Comet enabled"); + LOG.info("Comet vectorized reader enabled"); return new CometVectorizedParquetReader<>( file, schema, diff --git a/spark/v3.4/build.gradle b/spark/v3.4/build.gradle index 714be0831d8e..ca3b9dc41141 100644 --- a/spark/v3.4/build.gradle +++ b/spark/v3.4/build.gradle @@ -264,6 +264,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 "org.apache.datafusion:comet-spark-spark${sparkMajorVersion}_${scalaVersion}:${libs.versions.comet.get()}" + // runtime dependencies for running Hive Catalog based integration test integrationRuntimeOnly project(':iceberg-hive-metastore') 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 1ddd4d0dfd4d..eba1a2a0fb15 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 @@ -19,7 +19,6 @@ package org.apache.iceberg.spark.data.vectorized; import java.io.IOException; -import java.util.Map; import org.apache.comet.CometConf; import org.apache.comet.CometSchemaImporter; import org.apache.comet.parquet.AbstractColumnReader; @@ -29,15 +28,12 @@ import org.apache.comet.parquet.TypeUtil; import org.apache.comet.parquet.Utils; import org.apache.comet.shaded.arrow.memory.RootAllocator; +import org.apache.iceberg.parquet.CometTypeUtils; import org.apache.iceberg.parquet.VectorizedReader; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.spark.SparkSchemaUtil; import org.apache.iceberg.types.Types; 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.spark.sql.internal.SQLConf; import org.apache.spark.sql.types.DataType; import org.apache.spark.sql.types.Metadata; @@ -69,7 +65,8 @@ class CometColumnReader implements VectorizedReader { DataType dataType = SparkSchemaUtil.convert(field.type()); StructField structField = new StructField(field.name(), dataType, false, Metadata.empty()); this.sparkType = dataType; - this.descriptor = TypeUtil.convertToParquet(structField); + this.descriptor = + CometTypeUtils.buildColumnDescriptor(TypeUtil.convertToParquetSpec(structField)); this.fieldId = field.fieldId(); } @@ -105,58 +102,7 @@ public void reset() { this.importer = new CometSchemaImporter(new RootAllocator()); - String[] path = descriptor.getPath(); - PrimitiveType primitiveType = descriptor.getPrimitiveType(); - String physicalType = primitiveType.getPrimitiveTypeName().name(); - - // ToDo: extract this into a Util method - String logicalTypeName = null; - Map 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())); - } - } - - int typeLength = - primitiveType.getPrimitiveTypeName() == PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY - ? primitiveType.getTypeLength() - : 0; - boolean isRepeated = primitiveType.getRepetition() == Type.Repetition.REPEATED; - spec = - new ParquetColumnSpec( - fieldId, - path, - physicalType, - typeLength, - isRepeated, - descriptor.getMaxDefinitionLevel(), - descriptor.getMaxRepetitionLevel(), - logicalTypeName, - logicalTypeParams); + spec = CometTypeUtils.descriptorToParquetColumnSpec(descriptor); boolean useLegacyTime = Boolean.parseBoolean( @@ -165,7 +111,7 @@ public void reset() { CometConf.COMET_EXCEPTION_ON_LEGACY_DATE_TIMESTAMP().key(), "false")); boolean useLazyMaterialization = Boolean.parseBoolean( - SQLConf.get().getConfString(CometConf.COMET_USE_LAZY_MATERIALIZATION().key(), "true")); + SQLConf.get().getConfString(CometConf.COMET_USE_LAZY_MATERIALIZATION().key(), "false")); this.delegate = Utils.getColumnReader( sparkType, diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometConstantColumnReader.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometConstantColumnReader.java index c665002e8f66..6201988ee2b5 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometConstantColumnReader.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometConstantColumnReader.java @@ -21,6 +21,7 @@ import java.math.BigDecimal; import java.nio.ByteBuffer; import org.apache.comet.parquet.ConstantColumnReader; +import org.apache.iceberg.parquet.CometTypeUtils; import org.apache.iceberg.types.Types; import org.apache.spark.sql.types.DataType; import org.apache.spark.sql.types.DataTypes; @@ -34,7 +35,11 @@ class CometConstantColumnReader extends CometColumnReader { super(field); // use delegate to set constant value on the native side to be consumed by native execution. setDelegate( - new ConstantColumnReader(sparkType(), descriptor(), convertToSparkValue(value), false)); + new ConstantColumnReader( + sparkType(), + CometTypeUtils.descriptorToParquetColumnSpec(descriptor()), + convertToSparkValue(value), + false)); } @Override diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeleteColumnReader.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeleteColumnReader.java index 4a28fc51da9b..cba108e4326e 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeleteColumnReader.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeleteColumnReader.java @@ -51,7 +51,7 @@ private static class DeleteColumnReader extends MetadataColumnReader { DeleteColumnReader() { super( DataTypes.BooleanType, - TypeUtil.convertToParquet( + TypeUtil.convertToParquetSpec( new StructField("_deleted", DataTypes.BooleanType, false, Metadata.empty())), false /* useDecimal128 = false */, false /* isConstant */); diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometPositionColumnReader.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometPositionColumnReader.java index 1949a717982a..3416d47bd8b4 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometPositionColumnReader.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometPositionColumnReader.java @@ -20,6 +20,7 @@ import org.apache.comet.parquet.MetadataColumnReader; import org.apache.comet.parquet.Native; +import org.apache.iceberg.parquet.CometTypeUtils; import org.apache.iceberg.types.Types; import org.apache.parquet.column.ColumnDescriptor; import org.apache.spark.sql.types.DataTypes; @@ -44,7 +45,7 @@ private static class PositionColumnReader extends MetadataColumnReader { PositionColumnReader(ColumnDescriptor descriptor) { super( DataTypes.LongType, - descriptor, + CometTypeUtils.descriptorToParquetColumnSpec(descriptor), false /* useDecimal128 = false */, false /* isConstant */); } diff --git a/spark/v3.5/build.gradle b/spark/v3.5/build.gradle index 69700d84366d..138240f566c2 100644 --- a/spark/v3.5/build.gradle +++ b/spark/v3.5/build.gradle @@ -264,6 +264,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 "org.apache.datafusion:comet-spark-spark${sparkMajorVersion}_${scalaVersion}:${libs.versions.comet.get()}" + // runtime dependencies for running Hive Catalog based integration test integrationRuntimeOnly project(':iceberg-hive-metastore') diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnReader.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnReader.java index 81b7d83a7077..eba1a2a0fb15 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnReader.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnReader.java @@ -19,18 +19,22 @@ package org.apache.iceberg.spark.data.vectorized; import java.io.IOException; +import org.apache.comet.CometConf; import org.apache.comet.CometSchemaImporter; import org.apache.comet.parquet.AbstractColumnReader; import org.apache.comet.parquet.ColumnReader; +import org.apache.comet.parquet.ParquetColumnSpec; +import org.apache.comet.parquet.RowGroupReader; import org.apache.comet.parquet.TypeUtil; import org.apache.comet.parquet.Utils; import org.apache.comet.shaded.arrow.memory.RootAllocator; +import org.apache.iceberg.parquet.CometTypeUtils; import org.apache.iceberg.parquet.VectorizedReader; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.spark.SparkSchemaUtil; import org.apache.iceberg.types.Types; import org.apache.parquet.column.ColumnDescriptor; -import org.apache.parquet.column.page.PageReader; +import org.apache.spark.sql.internal.SQLConf; import org.apache.spark.sql.types.DataType; import org.apache.spark.sql.types.Metadata; import org.apache.spark.sql.types.StructField; @@ -42,23 +46,28 @@ class CometColumnReader implements VectorizedReader { private final ColumnDescriptor descriptor; private final DataType sparkType; + private final int fieldId; // The delegated ColumnReader from Comet side private AbstractColumnReader delegate; private boolean initialized = false; private int batchSize = DEFAULT_BATCH_SIZE; private CometSchemaImporter importer; + private ParquetColumnSpec spec; - CometColumnReader(DataType sparkType, ColumnDescriptor descriptor) { + CometColumnReader(DataType sparkType, ColumnDescriptor descriptor, int fieldId) { this.sparkType = sparkType; this.descriptor = descriptor; + this.fieldId = fieldId; } CometColumnReader(Types.NestedField field) { DataType dataType = SparkSchemaUtil.convert(field.type()); StructField structField = new StructField(field.name(), dataType, false, Metadata.empty()); this.sparkType = dataType; - this.descriptor = TypeUtil.convertToParquet(structField); + this.descriptor = + CometTypeUtils.buildColumnDescriptor(TypeUtil.convertToParquetSpec(structField)); + this.fieldId = field.fieldId(); } public AbstractColumnReader delegate() { @@ -92,7 +101,26 @@ public void reset() { } this.importer = new CometSchemaImporter(new RootAllocator()); - this.delegate = Utils.getColumnReader(sparkType, descriptor, importer, batchSize, false, false); + + spec = CometTypeUtils.descriptorToParquetColumnSpec(descriptor); + + boolean useLegacyTime = + Boolean.parseBoolean( + SQLConf.get() + .getConfString( + CometConf.COMET_EXCEPTION_ON_LEGACY_DATE_TIMESTAMP().key(), "false")); + boolean useLazyMaterialization = + Boolean.parseBoolean( + SQLConf.get().getConfString(CometConf.COMET_USE_LAZY_MATERIALIZATION().key(), "false")); + this.delegate = + Utils.getColumnReader( + sparkType, + spec, + importer, + batchSize, + true, // Comet sets this to true for native execution + useLazyMaterialization, + useLegacyTime); this.initialized = true; } @@ -111,9 +139,9 @@ public DataType sparkType() { *

NOTE: this should be called before reading a new Parquet column chunk, and after {@link * CometColumnReader#reset} is called. */ - public void setPageReader(PageReader pageReader) throws IOException { + public void setPageReader(RowGroupReader pageStore) throws IOException { Preconditions.checkState(initialized, "Invalid state: 'reset' should be called first"); - ((ColumnReader) delegate).setPageReader(pageReader); + ((ColumnReader) delegate).setRowGroupReader(pageStore, spec); } @Override diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnarBatchReader.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnarBatchReader.java index 04ac69476add..52d19ec2daf7 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnarBatchReader.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnarBatchReader.java @@ -23,7 +23,8 @@ import java.util.List; import java.util.Map; import org.apache.comet.parquet.AbstractColumnReader; -import org.apache.comet.parquet.BatchReader; +import org.apache.comet.parquet.IcebergCometBatchReader; +import org.apache.comet.parquet.RowGroupReader; import org.apache.iceberg.Schema; import org.apache.iceberg.data.DeleteFilter; import org.apache.iceberg.parquet.VectorizedReader; @@ -55,7 +56,7 @@ class CometColumnarBatchReader implements VectorizedReader { // calling BatchReader.nextBatch, the isDeleted value is not yet available, so // DeleteColumnReader.readBatch must be called explicitly later, after the isDeleted value is // available. - private final BatchReader delegate; + private final IcebergCometBatchReader delegate; private DeleteFilter deletes = null; private long rowStartPosInBatch = 0; @@ -64,10 +65,7 @@ class CometColumnarBatchReader implements VectorizedReader { readers.stream().map(CometColumnReader.class::cast).toArray(CometColumnReader[]::new); this.hasIsDeletedColumn = readers.stream().anyMatch(reader -> reader instanceof CometDeleteColumnReader); - - AbstractColumnReader[] abstractColumnReaders = new AbstractColumnReader[readers.size()]; - this.delegate = new BatchReader(abstractColumnReaders); - delegate.setSparkSchema(SparkSchemaUtil.convert(schema)); + this.delegate = new IcebergCometBatchReader(readers.size(), SparkSchemaUtil.convert(schema)); } @Override @@ -79,19 +77,22 @@ public void setRowGroupInfo( && !(readers[i] instanceof CometPositionColumnReader) && !(readers[i] instanceof CometDeleteColumnReader)) { readers[i].reset(); - readers[i].setPageReader(pageStore.getPageReader(readers[i].descriptor())); + readers[i].setPageReader((RowGroupReader) pageStore); } } catch (IOException e) { throw new UncheckedIOException("Failed to setRowGroupInfo for Comet vectorization", e); } } + AbstractColumnReader[] delegateReaders = new AbstractColumnReader[readers.length]; for (int i = 0; i < readers.length; i++) { - delegate.getColumnReaders()[i] = this.readers[i].delegate(); + delegateReaders[i] = readers[i].delegate(); } + delegate.init(delegateReaders); + this.rowStartPosInBatch = - pageStore + ((RowGroupReader) pageStore) .getRowIndexOffset() .orElseThrow( () -> diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometConstantColumnReader.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometConstantColumnReader.java index 047c96314b13..88d691a607a7 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometConstantColumnReader.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometConstantColumnReader.java @@ -21,6 +21,7 @@ import java.math.BigDecimal; import java.nio.ByteBuffer; import org.apache.comet.parquet.ConstantColumnReader; +import org.apache.iceberg.parquet.CometTypeUtils; import org.apache.iceberg.types.Types; import org.apache.spark.sql.types.DataType; import org.apache.spark.sql.types.DataTypes; @@ -34,7 +35,11 @@ class CometConstantColumnReader extends CometColumnReader { super(field); // use delegate to set constant value on the native side to be consumed by native execution. setDelegate( - new ConstantColumnReader(sparkType(), descriptor(), convertToSparkValue(value), false)); + new ConstantColumnReader( + sparkType(), + CometTypeUtils.descriptorToParquetColumnSpec(descriptor()), + convertToSparkValue(value), + false)); } @Override diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeleteColumnReader.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeleteColumnReader.java index 6235bfe4865e..cba108e4326e 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeleteColumnReader.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeleteColumnReader.java @@ -51,10 +51,10 @@ private static class DeleteColumnReader extends MetadataColumnReader { DeleteColumnReader() { super( DataTypes.BooleanType, - TypeUtil.convertToParquet( + TypeUtil.convertToParquetSpec( new StructField("_deleted", DataTypes.BooleanType, false, Metadata.empty())), false /* useDecimal128 = false */, - false /* isConstant = false */); + false /* isConstant */); this.isDeleted = new boolean[0]; } diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometPositionColumnReader.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometPositionColumnReader.java index bcc0e514c28d..98e80068c519 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometPositionColumnReader.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometPositionColumnReader.java @@ -20,6 +20,7 @@ import org.apache.comet.parquet.MetadataColumnReader; import org.apache.comet.parquet.Native; +import org.apache.iceberg.parquet.CometTypeUtils; import org.apache.iceberg.types.Types; import org.apache.parquet.column.ColumnDescriptor; import org.apache.spark.sql.types.DataTypes; @@ -44,7 +45,7 @@ private static class PositionColumnReader extends MetadataColumnReader { PositionColumnReader(ColumnDescriptor descriptor) { super( DataTypes.LongType, - descriptor, + CometTypeUtils.descriptorToParquetColumnSpec(descriptor), false /* useDecimal128 = false */, false /* isConstant = false */); } diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometVectorizedReaderBuilder.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometVectorizedReaderBuilder.java index d36f1a727477..56f8c9bff933 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometVectorizedReaderBuilder.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometVectorizedReaderBuilder.java @@ -142,6 +142,7 @@ public VectorizedReader primitive( return null; } - return new CometColumnReader(SparkSchemaUtil.convert(icebergField.type()), desc); + return new CometColumnReader( + SparkSchemaUtil.convert(icebergField.type()), desc, parquetFieldId); } } diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java index 780e1750a52e..57892ac4c59d 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java @@ -109,6 +109,7 @@ private CloseableIterable newParquetIterable( // read performance as every batch read doesn't have to pay the cost of allocating memory. .reuseContainers() .withNameMapping(nameMapping()) + .enableComet(parquetConf.readerType() == ParquetReaderType.COMET) .build(); } diff --git a/spark/v4.0/build.gradle b/spark/v4.0/build.gradle index 9c7ea06f9938..512b80b695b0 100644 --- a/spark/v4.0/build.gradle +++ b/spark/v4.0/build.gradle @@ -269,6 +269,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 "org.apache.datafusion:comet-spark-spark3.5_2.13:${libs.versions.comet.get()}" + // runtime dependencies for running Hive Catalog based integration test integrationRuntimeOnly project(':iceberg-hive-metastore') diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnReader.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnReader.java index 81b7d83a7077..eba1a2a0fb15 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnReader.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnReader.java @@ -19,18 +19,22 @@ package org.apache.iceberg.spark.data.vectorized; import java.io.IOException; +import org.apache.comet.CometConf; import org.apache.comet.CometSchemaImporter; import org.apache.comet.parquet.AbstractColumnReader; import org.apache.comet.parquet.ColumnReader; +import org.apache.comet.parquet.ParquetColumnSpec; +import org.apache.comet.parquet.RowGroupReader; import org.apache.comet.parquet.TypeUtil; import org.apache.comet.parquet.Utils; import org.apache.comet.shaded.arrow.memory.RootAllocator; +import org.apache.iceberg.parquet.CometTypeUtils; import org.apache.iceberg.parquet.VectorizedReader; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.spark.SparkSchemaUtil; import org.apache.iceberg.types.Types; import org.apache.parquet.column.ColumnDescriptor; -import org.apache.parquet.column.page.PageReader; +import org.apache.spark.sql.internal.SQLConf; import org.apache.spark.sql.types.DataType; import org.apache.spark.sql.types.Metadata; import org.apache.spark.sql.types.StructField; @@ -42,23 +46,28 @@ class CometColumnReader implements VectorizedReader { private final ColumnDescriptor descriptor; private final DataType sparkType; + private final int fieldId; // The delegated ColumnReader from Comet side private AbstractColumnReader delegate; private boolean initialized = false; private int batchSize = DEFAULT_BATCH_SIZE; private CometSchemaImporter importer; + private ParquetColumnSpec spec; - CometColumnReader(DataType sparkType, ColumnDescriptor descriptor) { + CometColumnReader(DataType sparkType, ColumnDescriptor descriptor, int fieldId) { this.sparkType = sparkType; this.descriptor = descriptor; + this.fieldId = fieldId; } CometColumnReader(Types.NestedField field) { DataType dataType = SparkSchemaUtil.convert(field.type()); StructField structField = new StructField(field.name(), dataType, false, Metadata.empty()); this.sparkType = dataType; - this.descriptor = TypeUtil.convertToParquet(structField); + this.descriptor = + CometTypeUtils.buildColumnDescriptor(TypeUtil.convertToParquetSpec(structField)); + this.fieldId = field.fieldId(); } public AbstractColumnReader delegate() { @@ -92,7 +101,26 @@ public void reset() { } this.importer = new CometSchemaImporter(new RootAllocator()); - this.delegate = Utils.getColumnReader(sparkType, descriptor, importer, batchSize, false, false); + + spec = CometTypeUtils.descriptorToParquetColumnSpec(descriptor); + + boolean useLegacyTime = + Boolean.parseBoolean( + SQLConf.get() + .getConfString( + CometConf.COMET_EXCEPTION_ON_LEGACY_DATE_TIMESTAMP().key(), "false")); + boolean useLazyMaterialization = + Boolean.parseBoolean( + SQLConf.get().getConfString(CometConf.COMET_USE_LAZY_MATERIALIZATION().key(), "false")); + this.delegate = + Utils.getColumnReader( + sparkType, + spec, + importer, + batchSize, + true, // Comet sets this to true for native execution + useLazyMaterialization, + useLegacyTime); this.initialized = true; } @@ -111,9 +139,9 @@ public DataType sparkType() { *

NOTE: this should be called before reading a new Parquet column chunk, and after {@link * CometColumnReader#reset} is called. */ - public void setPageReader(PageReader pageReader) throws IOException { + public void setPageReader(RowGroupReader pageStore) throws IOException { Preconditions.checkState(initialized, "Invalid state: 'reset' should be called first"); - ((ColumnReader) delegate).setPageReader(pageReader); + ((ColumnReader) delegate).setRowGroupReader(pageStore, spec); } @Override diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnarBatchReader.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnarBatchReader.java index 04ac69476add..9ebe4b6396c1 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnarBatchReader.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnarBatchReader.java @@ -23,7 +23,8 @@ import java.util.List; import java.util.Map; import org.apache.comet.parquet.AbstractColumnReader; -import org.apache.comet.parquet.BatchReader; +import org.apache.comet.parquet.IcebergCometBatchReader; +import org.apache.comet.parquet.RowGroupReader; import org.apache.iceberg.Schema; import org.apache.iceberg.data.DeleteFilter; import org.apache.iceberg.parquet.VectorizedReader; @@ -55,7 +56,7 @@ class CometColumnarBatchReader implements VectorizedReader { // calling BatchReader.nextBatch, the isDeleted value is not yet available, so // DeleteColumnReader.readBatch must be called explicitly later, after the isDeleted value is // available. - private final BatchReader delegate; + private final IcebergCometBatchReader delegate; private DeleteFilter deletes = null; private long rowStartPosInBatch = 0; @@ -65,9 +66,7 @@ class CometColumnarBatchReader implements VectorizedReader { this.hasIsDeletedColumn = readers.stream().anyMatch(reader -> reader instanceof CometDeleteColumnReader); - AbstractColumnReader[] abstractColumnReaders = new AbstractColumnReader[readers.size()]; - this.delegate = new BatchReader(abstractColumnReaders); - delegate.setSparkSchema(SparkSchemaUtil.convert(schema)); + this.delegate = new IcebergCometBatchReader(readers.size(), SparkSchemaUtil.convert(schema)); } @Override @@ -79,19 +78,22 @@ public void setRowGroupInfo( && !(readers[i] instanceof CometPositionColumnReader) && !(readers[i] instanceof CometDeleteColumnReader)) { readers[i].reset(); - readers[i].setPageReader(pageStore.getPageReader(readers[i].descriptor())); + readers[i].setPageReader((RowGroupReader) pageStore); } } catch (IOException e) { throw new UncheckedIOException("Failed to setRowGroupInfo for Comet vectorization", e); } } + AbstractColumnReader[] delegateReaders = new AbstractColumnReader[readers.length]; for (int i = 0; i < readers.length; i++) { - delegate.getColumnReaders()[i] = this.readers[i].delegate(); + delegateReaders[i] = readers[i].delegate(); } + delegate.init(delegateReaders); + this.rowStartPosInBatch = - pageStore + ((RowGroupReader) pageStore) .getRowIndexOffset() .orElseThrow( () -> diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometConstantColumnReader.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometConstantColumnReader.java index 047c96314b13..88d691a607a7 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometConstantColumnReader.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometConstantColumnReader.java @@ -21,6 +21,7 @@ import java.math.BigDecimal; import java.nio.ByteBuffer; import org.apache.comet.parquet.ConstantColumnReader; +import org.apache.iceberg.parquet.CometTypeUtils; import org.apache.iceberg.types.Types; import org.apache.spark.sql.types.DataType; import org.apache.spark.sql.types.DataTypes; @@ -34,7 +35,11 @@ class CometConstantColumnReader extends CometColumnReader { super(field); // use delegate to set constant value on the native side to be consumed by native execution. setDelegate( - new ConstantColumnReader(sparkType(), descriptor(), convertToSparkValue(value), false)); + new ConstantColumnReader( + sparkType(), + CometTypeUtils.descriptorToParquetColumnSpec(descriptor()), + convertToSparkValue(value), + false)); } @Override diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeleteColumnReader.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeleteColumnReader.java index 6235bfe4865e..721279786cd2 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeleteColumnReader.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeleteColumnReader.java @@ -51,7 +51,7 @@ private static class DeleteColumnReader extends MetadataColumnReader { DeleteColumnReader() { super( DataTypes.BooleanType, - TypeUtil.convertToParquet( + TypeUtil.convertToParquetSpec( new StructField("_deleted", DataTypes.BooleanType, false, Metadata.empty())), false /* useDecimal128 = false */, false /* isConstant = false */); diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometPositionColumnReader.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometPositionColumnReader.java index bcc0e514c28d..98e80068c519 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometPositionColumnReader.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometPositionColumnReader.java @@ -20,6 +20,7 @@ import org.apache.comet.parquet.MetadataColumnReader; import org.apache.comet.parquet.Native; +import org.apache.iceberg.parquet.CometTypeUtils; import org.apache.iceberg.types.Types; import org.apache.parquet.column.ColumnDescriptor; import org.apache.spark.sql.types.DataTypes; @@ -44,7 +45,7 @@ private static class PositionColumnReader extends MetadataColumnReader { PositionColumnReader(ColumnDescriptor descriptor) { super( DataTypes.LongType, - descriptor, + CometTypeUtils.descriptorToParquetColumnSpec(descriptor), false /* useDecimal128 = false */, false /* isConstant = false */); } diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometVectorizedReaderBuilder.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometVectorizedReaderBuilder.java index d36f1a727477..56f8c9bff933 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometVectorizedReaderBuilder.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometVectorizedReaderBuilder.java @@ -142,6 +142,7 @@ public VectorizedReader primitive( return null; } - return new CometColumnReader(SparkSchemaUtil.convert(icebergField.type()), desc); + return new CometColumnReader( + SparkSchemaUtil.convert(icebergField.type()), desc, parquetFieldId); } } diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java index 780e1750a52e..57892ac4c59d 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java @@ -109,6 +109,7 @@ private CloseableIterable newParquetIterable( // read performance as every batch read doesn't have to pay the cost of allocating memory. .reuseContainers() .withNameMapping(nameMapping()) + .enableComet(parquetConf.readerType() == ParquetReaderType.COMET) .build(); } From ba75958d5a57a804a66ed5d09eda325c9c56d690 Mon Sep 17 00:00:00 2001 From: Parth Chandra Date: Thu, 30 Oct 2025 17:26:34 -0700 Subject: [PATCH 03/15] move CometVectorizedParquetReader to spark module --- build.gradle | 7 - .../org/apache/iceberg/parquet/Parquet.java | 90 ++++-- .../org/apache/iceberg/parquet/ReadConf.java | 26 +- .../VectorizedParquetReaderFactory.java | 90 ++++++ .../data/vectorized/CometColumnReader.java | 2 +- .../vectorized/CometConstantColumnReader.java | 2 +- .../vectorized/CometPositionColumnReader.java | 2 +- .../spark}/parquet/CometTypeUtils.java | 2 +- .../parquet/CometVectorizedParquetReader.java | 4 +- .../CometVectorizedParquetReaderFactory.java | 80 ++++++ ...erg.parquet.VectorizedParquetReaderFactory | 1 + .../data/vectorized/CometColumnReader.java | 2 +- .../vectorized/CometConstantColumnReader.java | 2 +- .../vectorized/CometPositionColumnReader.java | 2 +- .../iceberg/spark/parquet/CometTypeUtils.java | 260 ++++++++++++++++++ .../parquet/CometVectorizedParquetReader.java | 257 +++++++++++++++++ .../CometVectorizedParquetReaderFactory.java | 80 ++++++ ...erg.parquet.VectorizedParquetReaderFactory | 1 + .../data/vectorized/CometColumnReader.java | 2 +- .../vectorized/CometConstantColumnReader.java | 2 +- .../vectorized/CometPositionColumnReader.java | 2 +- .../iceberg/spark/parquet/CometTypeUtils.java | 260 ++++++++++++++++++ .../parquet/CometVectorizedParquetReader.java | 257 +++++++++++++++++ .../CometVectorizedParquetReaderFactory.java | 80 ++++++ ...erg.parquet.VectorizedParquetReaderFactory | 1 + 25 files changed, 1453 insertions(+), 61 deletions(-) create mode 100644 parquet/src/main/java/org/apache/iceberg/parquet/VectorizedParquetReaderFactory.java rename {parquet/src/main/java/org/apache/iceberg => spark/v3.4/spark/src/main/java/org/apache/iceberg/spark}/parquet/CometTypeUtils.java (99%) rename {parquet/src/main/java/org/apache/iceberg => spark/v3.4/spark/src/main/java/org/apache/iceberg/spark}/parquet/CometVectorizedParquetReader.java (98%) create mode 100644 spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/parquet/CometVectorizedParquetReaderFactory.java create mode 100644 spark/v3.4/spark/src/main/resources/META-INF/services/org.apache.iceberg.parquet.VectorizedParquetReaderFactory create mode 100644 spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/parquet/CometTypeUtils.java create mode 100644 spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/parquet/CometVectorizedParquetReader.java create mode 100644 spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/parquet/CometVectorizedParquetReaderFactory.java create mode 100644 spark/v3.5/spark/src/main/resources/META-INF/services/org.apache.iceberg.parquet.VectorizedParquetReaderFactory create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/parquet/CometTypeUtils.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/parquet/CometVectorizedParquetReader.java create mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/parquet/CometVectorizedParquetReaderFactory.java create mode 100644 spark/v4.0/spark/src/main/resources/META-INF/services/org.apache.iceberg.parquet.VectorizedParquetReaderFactory diff --git a/build.gradle b/build.gradle index 79603391c2ea..cde2940ffd1f 100644 --- a/build.gradle +++ b/build.gradle @@ -855,13 +855,6 @@ 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/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java b/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java index bc173c7b6411..976b2d7c5992 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java @@ -56,6 +56,7 @@ import java.util.Locale; import java.util.Map; import java.util.Objects; +import java.util.ServiceLoader; import java.util.function.BiConsumer; import java.util.function.BiFunction; import java.util.function.Function; @@ -1161,7 +1162,6 @@ public static class ReadBuilder implements InternalData.ReadBuilder { private NameMapping nameMapping = null; private ByteBuffer fileEncryptionKey = null; private ByteBuffer fileAADPrefix = null; - private boolean isComet; private ReadBuilder(InputFile file) { this.file = file; @@ -1290,8 +1290,17 @@ public ReadBuilder setCustomType(int fieldId, Class struct throw new UnsupportedOperationException("Custom types are not yet supported"); } + /** + * @deprecated Use {@link #set(String, String)} with "read.parquet.vectorized-reader.factory" = + * "comet" instead + */ + @Deprecated public ReadBuilder enableComet(boolean enableComet) { - this.isComet = enableComet; + if (enableComet) { + this.properties.put("read.parquet.vectorized-reader.factory", "comet"); + } else { + this.properties.remove("read.parquet.vectorized-reader.factory"); + } return this; } @@ -1358,35 +1367,42 @@ public CloseableIterable build() { } if (batchedReaderFunc != null) { - if (isComet) { - LOG.info("Comet vectorized reader enabled"); - return new CometVectorizedParquetReader<>( - file, - schema, - options, - batchedReaderFunc, - mapping, - filter, - reuseContainers, - caseSensitive, - maxRecordsPerBatch, - properties, - start, - length, - fileEncryptionKey, - fileAADPrefix); - } else { - return new VectorizedParquetReader<>( - file, - schema, - options, - batchedReaderFunc, - mapping, - filter, - reuseContainers, - caseSensitive, - maxRecordsPerBatch); + // Try to load custom vectorized reader factory from properties + String readerName = properties.get("read.parquet.vectorized-reader.factory"); + + if (readerName != null) { + LOG.info("Loading custom vectorized reader factory: {}", readerName); + VectorizedParquetReaderFactory factory = loadReaderFactory(readerName); + if (factory != null) { + return factory.createReader( + file, + schema, + options, + batchedReaderFunc, + mapping, + filter, + reuseContainers, + caseSensitive, + maxRecordsPerBatch, + properties, + start, + length, + fileEncryptionKey, + fileAADPrefix); + } } + + // Fall back to default VectorizedParquetReader + return new VectorizedParquetReader<>( + file, + schema, + options, + batchedReaderFunc, + mapping, + filter, + reuseContainers, + caseSensitive, + maxRecordsPerBatch); } else { Function> readBuilder = readerFuncWithSchema != null @@ -1469,6 +1485,20 @@ public CloseableIterable build() { } } + private static VectorizedParquetReaderFactory loadReaderFactory(String name) { + ServiceLoader loader = + ServiceLoader.load(VectorizedParquetReaderFactory.class); + + for (VectorizedParquetReaderFactory factory : loader) { + if (factory.name().equalsIgnoreCase(name)) { + return factory; + } + } + + LOG.warn("Could not find vectorized reader factory: {}", name); + return null; + } + private static class ParquetReadBuilder extends ParquetReader.Builder { private Schema schema = null; private ReadSupport readSupport = null; diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ReadConf.java b/parquet/src/main/java/org/apache/iceberg/parquet/ReadConf.java index 142e5fbadf1f..b82472c0639a 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ReadConf.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ReadConf.java @@ -43,7 +43,7 @@ * * @param type of value to read */ -class ReadConf { +public class ReadConf { private final ParquetFileReader reader; private final InputFile file; private final ParquetReadOptions options; @@ -60,7 +60,7 @@ class ReadConf { private final List> columnChunkMetaDataForRowGroups; @SuppressWarnings("unchecked") - ReadConf( + public ReadConf( InputFile file, ParquetReadOptions options, Schema expectedSchema, @@ -146,7 +146,7 @@ private ReadConf(ReadConf toCopy) { this.columnChunkMetaDataForRowGroups = toCopy.columnChunkMetaDataForRowGroups; } - ParquetFileReader reader() { + public ParquetFileReader reader() { if (reader != null) { reader.setRequestedSchema(projection); return reader; @@ -157,43 +157,43 @@ ParquetFileReader reader() { return newReader; } - InputFile file() { + public InputFile file() { return file; } - MessageType projection() { + public MessageType projection() { return projection; } - ParquetValueReader model() { + public ParquetValueReader model() { return model; } - VectorizedReader vectorizedModel() { + public VectorizedReader vectorizedModel() { return vectorizedModel; } - boolean[] shouldSkip() { + public boolean[] shouldSkip() { return shouldSkip; } - long totalValues() { + public long totalValues() { return totalValues; } - boolean reuseContainers() { + public boolean reuseContainers() { return reuseContainers; } - Integer batchSize() { + public Integer batchSize() { return batchSize; } - List> columnChunkMetadataForRowGroups() { + public List> columnChunkMetadataForRowGroups() { return columnChunkMetaDataForRowGroups; } - ReadConf copy() { + public ReadConf copy() { return new ReadConf<>(this); } diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/VectorizedParquetReaderFactory.java b/parquet/src/main/java/org/apache/iceberg/parquet/VectorizedParquetReaderFactory.java new file mode 100644 index 000000000000..3c3702f56c67 --- /dev/null +++ b/parquet/src/main/java/org/apache/iceberg/parquet/VectorizedParquetReaderFactory.java @@ -0,0 +1,90 @@ +/* + * 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.nio.ByteBuffer; +import java.util.Map; +import java.util.function.Function; +import org.apache.iceberg.Schema; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.mapping.NameMapping; +import org.apache.parquet.ParquetReadOptions; +import org.apache.parquet.schema.MessageType; + +/** + * Service Provider Interface (SPI) for creating custom vectorized Parquet readers. + * + *

Implementations of this interface can be loaded at runtime using Java's {@link + * java.util.ServiceLoader} mechanism. To register an implementation, create a file named {@code + * META-INF/services/org.apache.iceberg.parquet.VectorizedParquetReaderFactory} containing the fully + * qualified class name of the implementation. + * + *

This allows for pluggable vectorized reader implementations (e.g., Comet, Arrow, Velox) + * without requiring the core parquet module to depend on specific execution engines. + */ +public interface VectorizedParquetReaderFactory { + + /** + * Returns the unique identifier for this reader factory. + * + *

This name is used to select the reader factory via configuration. For example, "comet" for + * the Comet vectorized reader. + * + * @return the unique name for this factory + */ + String name(); + + /** + * Creates a vectorized parquet reader with the given configuration. + * + * @param file the input file to read + * @param schema the expected schema for the data + * @param options parquet read options + * @param batchedReaderFunc function to create a VectorizedReader from a MessageType + * @param mapping name mapping for schema evolution + * @param filter filter expression to apply during reading + * @param reuseContainers whether to reuse containers for records + * @param caseSensitive whether column name matching should be case-sensitive + * @param maxRecordsPerBatch maximum number of records per batch + * @param properties additional properties for reader configuration + * @param start optional start position for reading + * @param length optional length to read + * @param fileEncryptionKey optional encryption key for the file + * @param fileAADPrefix optional AAD prefix for encryption + * @param the type of records returned by the reader + * @return a closeable iterable of records + */ + CloseableIterable createReader( + InputFile file, + Schema schema, + ParquetReadOptions options, + Function> batchedReaderFunc, + NameMapping mapping, + Expression filter, + boolean reuseContainers, + boolean caseSensitive, + int maxRecordsPerBatch, + Map properties, + Long start, + Long length, + ByteBuffer fileEncryptionKey, + ByteBuffer fileAADPrefix); +} 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 eba1a2a0fb15..d6350af7a273 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 @@ -28,10 +28,10 @@ import org.apache.comet.parquet.TypeUtil; import org.apache.comet.parquet.Utils; import org.apache.comet.shaded.arrow.memory.RootAllocator; -import org.apache.iceberg.parquet.CometTypeUtils; import org.apache.iceberg.parquet.VectorizedReader; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.spark.SparkSchemaUtil; +import org.apache.iceberg.spark.parquet.CometTypeUtils; import org.apache.iceberg.types.Types; import org.apache.parquet.column.ColumnDescriptor; import org.apache.spark.sql.internal.SQLConf; diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometConstantColumnReader.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometConstantColumnReader.java index 6201988ee2b5..76e3c2895558 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometConstantColumnReader.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometConstantColumnReader.java @@ -21,7 +21,7 @@ import java.math.BigDecimal; import java.nio.ByteBuffer; import org.apache.comet.parquet.ConstantColumnReader; -import org.apache.iceberg.parquet.CometTypeUtils; +import org.apache.iceberg.spark.parquet.CometTypeUtils; import org.apache.iceberg.types.Types; import org.apache.spark.sql.types.DataType; import org.apache.spark.sql.types.DataTypes; diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometPositionColumnReader.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometPositionColumnReader.java index 3416d47bd8b4..4c22edbb857e 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometPositionColumnReader.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometPositionColumnReader.java @@ -20,7 +20,7 @@ import org.apache.comet.parquet.MetadataColumnReader; import org.apache.comet.parquet.Native; -import org.apache.iceberg.parquet.CometTypeUtils; +import org.apache.iceberg.spark.parquet.CometTypeUtils; import org.apache.iceberg.types.Types; import org.apache.parquet.column.ColumnDescriptor; import org.apache.spark.sql.types.DataTypes; diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/CometTypeUtils.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/parquet/CometTypeUtils.java similarity index 99% rename from parquet/src/main/java/org/apache/iceberg/parquet/CometTypeUtils.java rename to spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/parquet/CometTypeUtils.java index 8f4305ce0681..5c48464f896b 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/CometTypeUtils.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/parquet/CometTypeUtils.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.iceberg.parquet; +package org.apache.iceberg.spark.parquet; import java.util.Map; import org.apache.comet.parquet.ParquetColumnSpec; diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/CometVectorizedParquetReader.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/parquet/CometVectorizedParquetReader.java similarity index 98% rename from parquet/src/main/java/org/apache/iceberg/parquet/CometVectorizedParquetReader.java rename to spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/parquet/CometVectorizedParquetReader.java index 88b195b76a2d..2218b0a030d4 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/CometVectorizedParquetReader.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/parquet/CometVectorizedParquetReader.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.iceberg.parquet; +package org.apache.iceberg.spark.parquet; import java.io.IOException; import java.io.UncheckedIOException; @@ -40,6 +40,8 @@ import org.apache.iceberg.io.CloseableIterator; import org.apache.iceberg.io.InputFile; import org.apache.iceberg.mapping.NameMapping; +import org.apache.iceberg.parquet.ReadConf; +import org.apache.iceberg.parquet.VectorizedReader; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.util.ByteBuffers; import org.apache.parquet.ParquetReadOptions; diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/parquet/CometVectorizedParquetReaderFactory.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/parquet/CometVectorizedParquetReaderFactory.java new file mode 100644 index 000000000000..934608347bc5 --- /dev/null +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/parquet/CometVectorizedParquetReaderFactory.java @@ -0,0 +1,80 @@ +/* + * 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.parquet; + +import java.nio.ByteBuffer; +import java.util.Map; +import java.util.function.Function; +import org.apache.iceberg.Schema; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.mapping.NameMapping; +import org.apache.iceberg.parquet.VectorizedParquetReaderFactory; +import org.apache.iceberg.parquet.VectorizedReader; +import org.apache.parquet.ParquetReadOptions; +import org.apache.parquet.schema.MessageType; + +/** + * Factory for creating Comet-based vectorized Parquet readers. + * + *

This factory is loaded via Java's ServiceLoader mechanism and provides Comet's native + * vectorized Parquet reader implementation for Iceberg tables. + */ +public class CometVectorizedParquetReaderFactory implements VectorizedParquetReaderFactory { + + @Override + public String name() { + return "comet"; + } + + @Override + public CloseableIterable createReader( + InputFile file, + Schema schema, + ParquetReadOptions options, + Function> batchedReaderFunc, + NameMapping mapping, + Expression filter, + boolean reuseContainers, + boolean caseSensitive, + int maxRecordsPerBatch, + Map properties, + Long start, + Long length, + ByteBuffer fileEncryptionKey, + ByteBuffer fileAADPrefix) { + + return new CometVectorizedParquetReader<>( + file, + schema, + options, + batchedReaderFunc, + mapping, + filter, + reuseContainers, + caseSensitive, + maxRecordsPerBatch, + properties, + start, + length, + fileEncryptionKey, + fileAADPrefix); + } +} diff --git a/spark/v3.4/spark/src/main/resources/META-INF/services/org.apache.iceberg.parquet.VectorizedParquetReaderFactory b/spark/v3.4/spark/src/main/resources/META-INF/services/org.apache.iceberg.parquet.VectorizedParquetReaderFactory new file mode 100644 index 000000000000..f31ba9dd5c02 --- /dev/null +++ b/spark/v3.4/spark/src/main/resources/META-INF/services/org.apache.iceberg.parquet.VectorizedParquetReaderFactory @@ -0,0 +1 @@ +org.apache.iceberg.spark.parquet.CometVectorizedParquetReaderFactory diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnReader.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnReader.java index eba1a2a0fb15..d6350af7a273 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnReader.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnReader.java @@ -28,10 +28,10 @@ import org.apache.comet.parquet.TypeUtil; import org.apache.comet.parquet.Utils; import org.apache.comet.shaded.arrow.memory.RootAllocator; -import org.apache.iceberg.parquet.CometTypeUtils; import org.apache.iceberg.parquet.VectorizedReader; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.spark.SparkSchemaUtil; +import org.apache.iceberg.spark.parquet.CometTypeUtils; import org.apache.iceberg.types.Types; import org.apache.parquet.column.ColumnDescriptor; import org.apache.spark.sql.internal.SQLConf; diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometConstantColumnReader.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometConstantColumnReader.java index 88d691a607a7..bb3eca5aa45f 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometConstantColumnReader.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometConstantColumnReader.java @@ -21,7 +21,7 @@ import java.math.BigDecimal; import java.nio.ByteBuffer; import org.apache.comet.parquet.ConstantColumnReader; -import org.apache.iceberg.parquet.CometTypeUtils; +import org.apache.iceberg.spark.parquet.CometTypeUtils; import org.apache.iceberg.types.Types; import org.apache.spark.sql.types.DataType; import org.apache.spark.sql.types.DataTypes; diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometPositionColumnReader.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometPositionColumnReader.java index 98e80068c519..017318b13872 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometPositionColumnReader.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometPositionColumnReader.java @@ -20,7 +20,7 @@ import org.apache.comet.parquet.MetadataColumnReader; import org.apache.comet.parquet.Native; -import org.apache.iceberg.parquet.CometTypeUtils; +import org.apache.iceberg.spark.parquet.CometTypeUtils; import org.apache.iceberg.types.Types; import org.apache.parquet.column.ColumnDescriptor; import org.apache.spark.sql.types.DataTypes; diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/parquet/CometTypeUtils.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/parquet/CometTypeUtils.java new file mode 100644 index 000000000000..5c48464f896b --- /dev/null +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/parquet/CometTypeUtils.java @@ -0,0 +1,260 @@ +/* + * 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.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; + + String logicalTypeName = null; + Map 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())); + } + } + + int id = -1; + Type type = descriptor.getPrimitiveType(); + if (type != null && type.getId() != null) { + id = type.getId().intValue(); + } + + return new ParquetColumnSpec( + 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 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/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/parquet/CometVectorizedParquetReader.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/parquet/CometVectorizedParquetReader.java new file mode 100644 index 000000000000..2218b0a030d4 --- /dev/null +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/parquet/CometVectorizedParquetReader.java @@ -0,0 +1,257 @@ +/* + * 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.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.parquet.ReadConf; +import org.apache.iceberg.parquet.VectorizedReader; +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 extends CloseableGroup + implements CloseableIterable { + private final InputFile input; + private final ParquetReadOptions options; + private final Schema expectedSchema; + private final Function> batchReaderFunc; + private final Expression filter; + private final boolean reuseContainers; + private final boolean caseSensitive; + private final int batchSize; + private final NameMapping nameMapping; + private final Map 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> readerFunc, + NameMapping nameMapping, + Expression filter, + boolean reuseContainers, + boolean caseSensitive, + int maxRecordsPerBatch, + Map 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 iterator() { + FileIterator iter = + new FileIterator<>(init(), properties, start, length, fileEncryptionKey, fileAADPrefix); + addCloseable(iter); + return iter; + } + + private static class FileIterator implements CloseableIterator { + // private final ParquetFileReader reader; + private final boolean[] shouldSkip; + private final VectorizedReader model; + private final long totalValues; + private final int batchSize; + private final List> 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 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 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 columnDescriptors = projection.getColumns(); + + List 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/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/parquet/CometVectorizedParquetReaderFactory.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/parquet/CometVectorizedParquetReaderFactory.java new file mode 100644 index 000000000000..934608347bc5 --- /dev/null +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/parquet/CometVectorizedParquetReaderFactory.java @@ -0,0 +1,80 @@ +/* + * 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.parquet; + +import java.nio.ByteBuffer; +import java.util.Map; +import java.util.function.Function; +import org.apache.iceberg.Schema; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.mapping.NameMapping; +import org.apache.iceberg.parquet.VectorizedParquetReaderFactory; +import org.apache.iceberg.parquet.VectorizedReader; +import org.apache.parquet.ParquetReadOptions; +import org.apache.parquet.schema.MessageType; + +/** + * Factory for creating Comet-based vectorized Parquet readers. + * + *

This factory is loaded via Java's ServiceLoader mechanism and provides Comet's native + * vectorized Parquet reader implementation for Iceberg tables. + */ +public class CometVectorizedParquetReaderFactory implements VectorizedParquetReaderFactory { + + @Override + public String name() { + return "comet"; + } + + @Override + public CloseableIterable createReader( + InputFile file, + Schema schema, + ParquetReadOptions options, + Function> batchedReaderFunc, + NameMapping mapping, + Expression filter, + boolean reuseContainers, + boolean caseSensitive, + int maxRecordsPerBatch, + Map properties, + Long start, + Long length, + ByteBuffer fileEncryptionKey, + ByteBuffer fileAADPrefix) { + + return new CometVectorizedParquetReader<>( + file, + schema, + options, + batchedReaderFunc, + mapping, + filter, + reuseContainers, + caseSensitive, + maxRecordsPerBatch, + properties, + start, + length, + fileEncryptionKey, + fileAADPrefix); + } +} diff --git a/spark/v3.5/spark/src/main/resources/META-INF/services/org.apache.iceberg.parquet.VectorizedParquetReaderFactory b/spark/v3.5/spark/src/main/resources/META-INF/services/org.apache.iceberg.parquet.VectorizedParquetReaderFactory new file mode 100644 index 000000000000..f31ba9dd5c02 --- /dev/null +++ b/spark/v3.5/spark/src/main/resources/META-INF/services/org.apache.iceberg.parquet.VectorizedParquetReaderFactory @@ -0,0 +1 @@ +org.apache.iceberg.spark.parquet.CometVectorizedParquetReaderFactory diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnReader.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnReader.java index eba1a2a0fb15..d6350af7a273 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnReader.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnReader.java @@ -28,10 +28,10 @@ import org.apache.comet.parquet.TypeUtil; import org.apache.comet.parquet.Utils; import org.apache.comet.shaded.arrow.memory.RootAllocator; -import org.apache.iceberg.parquet.CometTypeUtils; import org.apache.iceberg.parquet.VectorizedReader; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.spark.SparkSchemaUtil; +import org.apache.iceberg.spark.parquet.CometTypeUtils; import org.apache.iceberg.types.Types; import org.apache.parquet.column.ColumnDescriptor; import org.apache.spark.sql.internal.SQLConf; diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometConstantColumnReader.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometConstantColumnReader.java index 88d691a607a7..bb3eca5aa45f 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometConstantColumnReader.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometConstantColumnReader.java @@ -21,7 +21,7 @@ import java.math.BigDecimal; import java.nio.ByteBuffer; import org.apache.comet.parquet.ConstantColumnReader; -import org.apache.iceberg.parquet.CometTypeUtils; +import org.apache.iceberg.spark.parquet.CometTypeUtils; import org.apache.iceberg.types.Types; import org.apache.spark.sql.types.DataType; import org.apache.spark.sql.types.DataTypes; diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometPositionColumnReader.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometPositionColumnReader.java index 98e80068c519..017318b13872 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometPositionColumnReader.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometPositionColumnReader.java @@ -20,7 +20,7 @@ import org.apache.comet.parquet.MetadataColumnReader; import org.apache.comet.parquet.Native; -import org.apache.iceberg.parquet.CometTypeUtils; +import org.apache.iceberg.spark.parquet.CometTypeUtils; import org.apache.iceberg.types.Types; import org.apache.parquet.column.ColumnDescriptor; import org.apache.spark.sql.types.DataTypes; diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/parquet/CometTypeUtils.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/parquet/CometTypeUtils.java new file mode 100644 index 000000000000..5c48464f896b --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/parquet/CometTypeUtils.java @@ -0,0 +1,260 @@ +/* + * 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.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; + + String logicalTypeName = null; + Map 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())); + } + } + + int id = -1; + Type type = descriptor.getPrimitiveType(); + if (type != null && type.getId() != null) { + id = type.getId().intValue(); + } + + return new ParquetColumnSpec( + 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 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/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/parquet/CometVectorizedParquetReader.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/parquet/CometVectorizedParquetReader.java new file mode 100644 index 000000000000..2218b0a030d4 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/parquet/CometVectorizedParquetReader.java @@ -0,0 +1,257 @@ +/* + * 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.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.parquet.ReadConf; +import org.apache.iceberg.parquet.VectorizedReader; +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 extends CloseableGroup + implements CloseableIterable { + private final InputFile input; + private final ParquetReadOptions options; + private final Schema expectedSchema; + private final Function> batchReaderFunc; + private final Expression filter; + private final boolean reuseContainers; + private final boolean caseSensitive; + private final int batchSize; + private final NameMapping nameMapping; + private final Map 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> readerFunc, + NameMapping nameMapping, + Expression filter, + boolean reuseContainers, + boolean caseSensitive, + int maxRecordsPerBatch, + Map 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 iterator() { + FileIterator iter = + new FileIterator<>(init(), properties, start, length, fileEncryptionKey, fileAADPrefix); + addCloseable(iter); + return iter; + } + + private static class FileIterator implements CloseableIterator { + // private final ParquetFileReader reader; + private final boolean[] shouldSkip; + private final VectorizedReader model; + private final long totalValues; + private final int batchSize; + private final List> 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 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 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 columnDescriptors = projection.getColumns(); + + List 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/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/parquet/CometVectorizedParquetReaderFactory.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/parquet/CometVectorizedParquetReaderFactory.java new file mode 100644 index 000000000000..934608347bc5 --- /dev/null +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/parquet/CometVectorizedParquetReaderFactory.java @@ -0,0 +1,80 @@ +/* + * 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.parquet; + +import java.nio.ByteBuffer; +import java.util.Map; +import java.util.function.Function; +import org.apache.iceberg.Schema; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.mapping.NameMapping; +import org.apache.iceberg.parquet.VectorizedParquetReaderFactory; +import org.apache.iceberg.parquet.VectorizedReader; +import org.apache.parquet.ParquetReadOptions; +import org.apache.parquet.schema.MessageType; + +/** + * Factory for creating Comet-based vectorized Parquet readers. + * + *

This factory is loaded via Java's ServiceLoader mechanism and provides Comet's native + * vectorized Parquet reader implementation for Iceberg tables. + */ +public class CometVectorizedParquetReaderFactory implements VectorizedParquetReaderFactory { + + @Override + public String name() { + return "comet"; + } + + @Override + public CloseableIterable createReader( + InputFile file, + Schema schema, + ParquetReadOptions options, + Function> batchedReaderFunc, + NameMapping mapping, + Expression filter, + boolean reuseContainers, + boolean caseSensitive, + int maxRecordsPerBatch, + Map properties, + Long start, + Long length, + ByteBuffer fileEncryptionKey, + ByteBuffer fileAADPrefix) { + + return new CometVectorizedParquetReader<>( + file, + schema, + options, + batchedReaderFunc, + mapping, + filter, + reuseContainers, + caseSensitive, + maxRecordsPerBatch, + properties, + start, + length, + fileEncryptionKey, + fileAADPrefix); + } +} diff --git a/spark/v4.0/spark/src/main/resources/META-INF/services/org.apache.iceberg.parquet.VectorizedParquetReaderFactory b/spark/v4.0/spark/src/main/resources/META-INF/services/org.apache.iceberg.parquet.VectorizedParquetReaderFactory new file mode 100644 index 000000000000..4bd71003032d --- /dev/null +++ b/spark/v4.0/spark/src/main/resources/META-INF/services/org.apache.iceberg.parquet.VectorizedParquetReaderFactory @@ -0,0 +1 @@ +org.apache.iceberg.spark.parquet.CometVectorizedParquetReaderFactory \ No newline at end of file From e77a63b7b0612f34290e9a9e4492b0ff84b6aa32 Mon Sep 17 00:00:00 2001 From: Parth Chandra Date: Fri, 31 Oct 2025 15:24:40 -0700 Subject: [PATCH 04/15] Use comet 0.10.1 --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 3419e708edd7..c8fa432a3020 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -37,7 +37,7 @@ awssdk-s3accessgrants = "2.3.0" bson-ver = "4.11.5" caffeine = "2.9.3" calcite = "1.40.0" -comet = "0.10.0" +comet = "0.10.1" datasketches = "6.2.0" delta-standalone = "3.3.2" delta-spark = "3.3.2" From c52269302ac7803740e7e657745164bc9fbfc41a Mon Sep 17 00:00:00 2001 From: Parth Chandra Date: Mon, 3 Nov 2025 09:14:36 -0800 Subject: [PATCH 05/15] Address review comments --- .../main/java/org/apache/iceberg/parquet/Parquet.java | 11 +++++------ .../apache/iceberg/spark/parquet/CometTypeUtils.java | 4 ++-- .../spark/parquet/CometVectorizedParquetReader.java | 1 - .../apache/iceberg/spark/parquet/CometTypeUtils.java | 4 ++-- .../spark/parquet/CometVectorizedParquetReader.java | 1 - .../apache/iceberg/spark/parquet/CometTypeUtils.java | 4 ++-- .../spark/parquet/CometVectorizedParquetReader.java | 1 - 7 files changed, 11 insertions(+), 15 deletions(-) diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java b/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java index 6933a855aa89..c08d173c391c 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java @@ -128,6 +128,7 @@ public class Parquet { private static final Logger LOG = LoggerFactory.getLogger(Parquet.class); + private static final String VECTORIZED_READER_FACTORY = "read.parquet.vectorized-reader.factory"; private Parquet() {} @@ -1380,15 +1381,13 @@ public ReadBuilder setCustomType(int fieldId, Class struct } /** - * @deprecated Use {@link #set(String, String)} with "read.parquet.vectorized-reader.factory" = - * "comet" instead + * Convenience method to enable comet */ - @Deprecated public ReadBuilder enableComet(boolean enableComet) { if (enableComet) { - this.properties.put("read.parquet.vectorized-reader.factory", "comet"); + this.properties.put(VECTORIZED_READER_FACTORY, "comet"); } else { - this.properties.remove("read.parquet.vectorized-reader.factory"); + this.properties.remove(VECTORIZED_READER_FACTORY); } return this; } @@ -1458,7 +1457,7 @@ public CloseableIterable build() { if (batchedReaderFunc != null) { // Try to load custom vectorized reader factory from properties - String readerName = properties.get("read.parquet.vectorized-reader.factory"); + String readerName = properties.get(VECTORIZED_READER_FACTORY); if (readerName != null) { LOG.info("Loading custom vectorized reader factory: {}", readerName); diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/parquet/CometTypeUtils.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/parquet/CometTypeUtils.java index 5c48464f896b..b94058842510 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/parquet/CometTypeUtils.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/parquet/CometTypeUtils.java @@ -118,7 +118,7 @@ public static ColumnDescriptor buildColumnDescriptor(ParquetColumnSpec columnSpe PrimitiveType primitiveType; if (primType == PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY) { primitiveType = - org.apache.parquet.schema.Types.primitive(primType, repetition) + Types.primitive(primType, repetition) .length(columnSpec.getTypeLength()) .as(logicalType) .id(columnSpec.getFieldId()) @@ -139,7 +139,7 @@ public static ColumnDescriptor buildColumnDescriptor(ParquetColumnSpec columnSpe } private static LogicalTypeAnnotation reconstructLogicalType( - String logicalTypeName, java.util.Map params) { + String logicalTypeName, Map params) { switch (logicalTypeName) { // MAP diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/parquet/CometVectorizedParquetReader.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/parquet/CometVectorizedParquetReader.java index 2218b0a030d4..f133896b1201 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/parquet/CometVectorizedParquetReader.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/parquet/CometVectorizedParquetReader.java @@ -130,7 +130,6 @@ public CloseableIterator iterator() { } private static class FileIterator implements CloseableIterator { - // private final ParquetFileReader reader; private final boolean[] shouldSkip; private final VectorizedReader model; private final long totalValues; diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/parquet/CometTypeUtils.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/parquet/CometTypeUtils.java index 5c48464f896b..b94058842510 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/parquet/CometTypeUtils.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/parquet/CometTypeUtils.java @@ -118,7 +118,7 @@ public static ColumnDescriptor buildColumnDescriptor(ParquetColumnSpec columnSpe PrimitiveType primitiveType; if (primType == PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY) { primitiveType = - org.apache.parquet.schema.Types.primitive(primType, repetition) + Types.primitive(primType, repetition) .length(columnSpec.getTypeLength()) .as(logicalType) .id(columnSpec.getFieldId()) @@ -139,7 +139,7 @@ public static ColumnDescriptor buildColumnDescriptor(ParquetColumnSpec columnSpe } private static LogicalTypeAnnotation reconstructLogicalType( - String logicalTypeName, java.util.Map params) { + String logicalTypeName, Map params) { switch (logicalTypeName) { // MAP diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/parquet/CometVectorizedParquetReader.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/parquet/CometVectorizedParquetReader.java index 2218b0a030d4..f133896b1201 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/parquet/CometVectorizedParquetReader.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/parquet/CometVectorizedParquetReader.java @@ -130,7 +130,6 @@ public CloseableIterator iterator() { } private static class FileIterator implements CloseableIterator { - // private final ParquetFileReader reader; private final boolean[] shouldSkip; private final VectorizedReader model; private final long totalValues; diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/parquet/CometTypeUtils.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/parquet/CometTypeUtils.java index 5c48464f896b..b94058842510 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/parquet/CometTypeUtils.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/parquet/CometTypeUtils.java @@ -118,7 +118,7 @@ public static ColumnDescriptor buildColumnDescriptor(ParquetColumnSpec columnSpe PrimitiveType primitiveType; if (primType == PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY) { primitiveType = - org.apache.parquet.schema.Types.primitive(primType, repetition) + Types.primitive(primType, repetition) .length(columnSpec.getTypeLength()) .as(logicalType) .id(columnSpec.getFieldId()) @@ -139,7 +139,7 @@ public static ColumnDescriptor buildColumnDescriptor(ParquetColumnSpec columnSpe } private static LogicalTypeAnnotation reconstructLogicalType( - String logicalTypeName, java.util.Map params) { + String logicalTypeName, Map params) { switch (logicalTypeName) { // MAP diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/parquet/CometVectorizedParquetReader.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/parquet/CometVectorizedParquetReader.java index 2218b0a030d4..f133896b1201 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/parquet/CometVectorizedParquetReader.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/parquet/CometVectorizedParquetReader.java @@ -130,7 +130,6 @@ public CloseableIterator iterator() { } private static class FileIterator implements CloseableIterator { - // private final ParquetFileReader reader; private final boolean[] shouldSkip; private final VectorizedReader model; private final long totalValues; From 2cc462260b01c2493eb1d1e64307b11243eda9b5 Mon Sep 17 00:00:00 2001 From: Parth Chandra Date: Mon, 3 Nov 2025 09:19:37 -0800 Subject: [PATCH 06/15] spotless --- parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java b/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java index c08d173c391c..fdcf69571792 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java @@ -1380,9 +1380,7 @@ public ReadBuilder setCustomType(int fieldId, Class struct return this; } - /** - * Convenience method to enable comet - */ + /** Convenience method to enable comet */ public ReadBuilder enableComet(boolean enableComet) { if (enableComet) { this.properties.put(VECTORIZED_READER_FACTORY, "comet"); From c093598b05380374b7e93b19b2bc7539aa310804 Mon Sep 17 00:00:00 2001 From: Parth Chandra Date: Mon, 3 Nov 2025 12:17:57 -0800 Subject: [PATCH 07/15] Add apache header --- ...berg.parquet.VectorizedParquetReaderFactory | 18 ++++++++++++++++++ ...berg.parquet.VectorizedParquetReaderFactory | 18 ++++++++++++++++++ ...berg.parquet.VectorizedParquetReaderFactory | 18 ++++++++++++++++++ 3 files changed, 54 insertions(+) diff --git a/spark/v3.4/spark/src/main/resources/META-INF/services/org.apache.iceberg.parquet.VectorizedParquetReaderFactory b/spark/v3.4/spark/src/main/resources/META-INF/services/org.apache.iceberg.parquet.VectorizedParquetReaderFactory index f31ba9dd5c02..2dec84408d64 100644 --- a/spark/v3.4/spark/src/main/resources/META-INF/services/org.apache.iceberg.parquet.VectorizedParquetReaderFactory +++ b/spark/v3.4/spark/src/main/resources/META-INF/services/org.apache.iceberg.parquet.VectorizedParquetReaderFactory @@ -1 +1,19 @@ +# +# 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. +# org.apache.iceberg.spark.parquet.CometVectorizedParquetReaderFactory diff --git a/spark/v3.5/spark/src/main/resources/META-INF/services/org.apache.iceberg.parquet.VectorizedParquetReaderFactory b/spark/v3.5/spark/src/main/resources/META-INF/services/org.apache.iceberg.parquet.VectorizedParquetReaderFactory index f31ba9dd5c02..2dec84408d64 100644 --- a/spark/v3.5/spark/src/main/resources/META-INF/services/org.apache.iceberg.parquet.VectorizedParquetReaderFactory +++ b/spark/v3.5/spark/src/main/resources/META-INF/services/org.apache.iceberg.parquet.VectorizedParquetReaderFactory @@ -1 +1,19 @@ +# +# 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. +# org.apache.iceberg.spark.parquet.CometVectorizedParquetReaderFactory diff --git a/spark/v4.0/spark/src/main/resources/META-INF/services/org.apache.iceberg.parquet.VectorizedParquetReaderFactory b/spark/v4.0/spark/src/main/resources/META-INF/services/org.apache.iceberg.parquet.VectorizedParquetReaderFactory index 4bd71003032d..8ce8a1ba7307 100644 --- a/spark/v4.0/spark/src/main/resources/META-INF/services/org.apache.iceberg.parquet.VectorizedParquetReaderFactory +++ b/spark/v4.0/spark/src/main/resources/META-INF/services/org.apache.iceberg.parquet.VectorizedParquetReaderFactory @@ -1 +1,19 @@ +# +# 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. +# org.apache.iceberg.spark.parquet.CometVectorizedParquetReaderFactory \ No newline at end of file From 461bd2354cd3bb94cd07e9ad92ee0a8518b11a5c Mon Sep 17 00:00:00 2001 From: Parth Chandra Date: Mon, 10 Nov 2025 11:04:53 -0800 Subject: [PATCH 08/15] do not use ServiceLoader --- .../org/apache/iceberg/parquet/Parquet.java | 48 ++++++++++++++----- .../VectorizedParquetReaderFactory.java | 9 ++-- .../iceberg/spark/ParquetBatchReadConf.java | 3 ++ .../apache/iceberg/spark/SparkReadConf.java | 7 +++ .../iceberg/spark/SparkSQLProperties.java | 8 ++++ .../CometVectorizedParquetReaderFactory.java | 5 +- .../iceberg/spark/source/BaseBatchReader.java | 2 +- .../iceberg/spark/source/SparkBatch.java | 9 ++++ ...erg.parquet.VectorizedParquetReaderFactory | 19 -------- .../iceberg/spark/ParquetBatchReadConf.java | 3 ++ .../apache/iceberg/spark/SparkReadConf.java | 7 +++ .../iceberg/spark/SparkSQLProperties.java | 9 ++++ .../CometVectorizedParquetReaderFactory.java | 5 +- .../iceberg/spark/source/BaseBatchReader.java | 2 +- .../iceberg/spark/source/SparkBatch.java | 9 ++++ ...erg.parquet.VectorizedParquetReaderFactory | 19 -------- .../iceberg/spark/ParquetBatchReadConf.java | 3 ++ .../apache/iceberg/spark/SparkReadConf.java | 7 +++ .../iceberg/spark/SparkSQLProperties.java | 9 ++++ .../CometVectorizedParquetReaderFactory.java | 5 +- .../iceberg/spark/source/BaseBatchReader.java | 2 +- .../iceberg/spark/source/SparkBatch.java | 9 ++++ ...erg.parquet.VectorizedParquetReaderFactory | 19 -------- 23 files changed, 135 insertions(+), 83 deletions(-) delete mode 100644 spark/v3.4/spark/src/main/resources/META-INF/services/org.apache.iceberg.parquet.VectorizedParquetReaderFactory delete mode 100644 spark/v3.5/spark/src/main/resources/META-INF/services/org.apache.iceberg.parquet.VectorizedParquetReaderFactory delete mode 100644 spark/v4.0/spark/src/main/resources/META-INF/services/org.apache.iceberg.parquet.VectorizedParquetReaderFactory diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java b/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java index fdcf69571792..59c607cfce45 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java @@ -57,7 +57,6 @@ import java.util.Locale; import java.util.Map; import java.util.Objects; -import java.util.ServiceLoader; import java.util.function.BiConsumer; import java.util.function.BiFunction; import java.util.function.Function; @@ -1383,7 +1382,25 @@ public ReadBuilder setCustomType(int fieldId, Class struct /** Convenience method to enable comet */ public ReadBuilder enableComet(boolean enableComet) { if (enableComet) { - this.properties.put(VECTORIZED_READER_FACTORY, "comet"); + this.properties.put( + VECTORIZED_READER_FACTORY, + "org.apache.iceberg.spark.parquet.CometVectorizedParquetReaderFactory"); + } else { + this.properties.remove(VECTORIZED_READER_FACTORY); + } + return this; + } + + /** + * Sets the vectorized reader factory class to use for reading Parquet files. + * + * @param factoryClassName fully qualified class name of the VectorizedParquetReaderFactory + * implementation, or null to use the default reader + * @return this builder for method chaining + */ + public ReadBuilder vectorizedReaderFactory(String factoryClassName) { + if (factoryClassName != null) { + this.properties.put(VECTORIZED_READER_FACTORY, factoryClassName); } else { this.properties.remove(VECTORIZED_READER_FACTORY); } @@ -1575,18 +1592,25 @@ public CloseableIterable build() { } } - private static VectorizedParquetReaderFactory loadReaderFactory(String name) { - ServiceLoader loader = - ServiceLoader.load(VectorizedParquetReaderFactory.class); - - for (VectorizedParquetReaderFactory factory : loader) { - if (factory.name().equalsIgnoreCase(name)) { - return factory; + private static VectorizedParquetReaderFactory loadReaderFactory(String className) { + try { + Class factoryClass = Class.forName(className); + if (!VectorizedParquetReaderFactory.class.isAssignableFrom(factoryClass)) { + LOG.warn("Class {} does not implement VectorizedParquetReaderFactory interface", className); + return null; } + return (VectorizedParquetReaderFactory) factoryClass.getDeclaredConstructor().newInstance(); + } catch (ClassNotFoundException e) { + LOG.warn("Could not find vectorized reader factory class: {}", className, e); + return null; + } catch (NoSuchMethodException e) { + LOG.warn( + "Vectorized reader factory class {} does not have a no-arg constructor", className, e); + return null; + } catch (Exception e) { + LOG.warn("Failed to instantiate vectorized reader factory: {}", className, e); + return null; } - - LOG.warn("Could not find vectorized reader factory: {}", name); - return null; } private static class ParquetReadBuilder extends ParquetReader.Builder { diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/VectorizedParquetReaderFactory.java b/parquet/src/main/java/org/apache/iceberg/parquet/VectorizedParquetReaderFactory.java index 3c3702f56c67..fb78affdcf3b 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/VectorizedParquetReaderFactory.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/VectorizedParquetReaderFactory.java @@ -30,12 +30,11 @@ import org.apache.parquet.schema.MessageType; /** - * Service Provider Interface (SPI) for creating custom vectorized Parquet readers. + * Interface for creating custom vectorized Parquet readers. * - *

Implementations of this interface can be loaded at runtime using Java's {@link - * java.util.ServiceLoader} mechanism. To register an implementation, create a file named {@code - * META-INF/services/org.apache.iceberg.parquet.VectorizedParquetReaderFactory} containing the fully - * qualified class name of the implementation. + *

Implementations of this interface are loaded at runtime via reflection by specifying the fully + * qualified class name in the {@code read.parquet.vectorized-reader.factory} configuration + * property. * *

This allows for pluggable vectorized reader implementations (e.g., Comet, Arrow, Velox) * without requiring the core parquet module to depend on specific execution engines. diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/ParquetBatchReadConf.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/ParquetBatchReadConf.java index 442d728d4d69..dc84485e17f4 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/ParquetBatchReadConf.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/ParquetBatchReadConf.java @@ -19,6 +19,7 @@ package org.apache.iceberg.spark; import java.io.Serializable; +import java.util.Optional; import org.immutables.value.Value; @Value.Immutable @@ -26,4 +27,6 @@ public interface ParquetBatchReadConf extends Serializable { int batchSize(); ParquetReaderType readerType(); + + Optional factoryClassName(); } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java index dd7e2c20c1b9..62465f46f435 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java @@ -379,4 +379,11 @@ public ParquetReaderType parquetReaderType() { .defaultValue(SparkSQLProperties.PARQUET_READER_TYPE_DEFAULT) .parse(); } + + public String parquetVectorizedReaderFactory() { + return confParser + .stringConf() + .sessionConf(SparkSQLProperties.PARQUET_VECTORIZED_READER_FACTORY) + .parseOptional(); + } } 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 fa7d4a4b185a..d13b4d572029 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 @@ -31,6 +31,14 @@ private SparkSQLProperties() {} public static final String PARQUET_READER_TYPE = "spark.sql.iceberg.parquet.reader-type"; public static final ParquetReaderType PARQUET_READER_TYPE_DEFAULT = ParquetReaderType.ICEBERG; + // Controls the fully qualified class name of the vectorized Parquet reader factory + public static final String PARQUET_VECTORIZED_READER_FACTORY = + "spark.sql.iceberg.parquet.vectorized-reader.factory"; + + // Comet vectorized reader factory class name + public static final String COMET_VECTORIZED_READER_FACTORY_CLASS = + "org.apache.iceberg.spark.parquet.CometVectorizedParquetReaderFactory"; + // Controls whether reading/writing timestamps without timezones is allowed @Deprecated public static final String HANDLE_TIMESTAMP_WITHOUT_TIMEZONE = diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/parquet/CometVectorizedParquetReaderFactory.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/parquet/CometVectorizedParquetReaderFactory.java index 934608347bc5..68c8d9a04034 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/parquet/CometVectorizedParquetReaderFactory.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/parquet/CometVectorizedParquetReaderFactory.java @@ -34,8 +34,9 @@ /** * Factory for creating Comet-based vectorized Parquet readers. * - *

This factory is loaded via Java's ServiceLoader mechanism and provides Comet's native - * vectorized Parquet reader implementation for Iceberg tables. + *

This factory is loaded via reflection when the {@code read.parquet.vectorized-reader.factory} + * property is set to {@code org.apache.iceberg.spark.parquet.CometVectorizedParquetReaderFactory}. + * It provides Comet's native vectorized Parquet reader implementation for Iceberg tables. */ public class CometVectorizedParquetReaderFactory implements VectorizedParquetReaderFactory { diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java index 473b34bb0f3e..4d6ceb5e7f9d 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java @@ -111,7 +111,7 @@ private CloseableIterable newParquetIterable( // read performance as every batch read doesn't have to pay the cost of allocating memory. .reuseContainers() .withNameMapping(nameMapping()) - .enableComet(parquetConf.readerType() == ParquetReaderType.COMET) + .vectorizedReaderFactory(parquetConf.factoryClassName().orElse(null)) .build(); } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatch.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatch.java index 0626d0b43985..8a9ae2c63df5 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatch.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatch.java @@ -137,9 +137,18 @@ public PartitionReaderFactory createReaderFactory() { } private ParquetBatchReadConf parquetBatchReadConf(ParquetReaderType readerType) { + String factoryClassName = readConf.parquetVectorizedReaderFactory(); + + // If no explicit factory is set and reader type is COMET, use the default Comet factory + if (factoryClassName == null && readerType == ParquetReaderType.COMET) { + factoryClassName = + org.apache.iceberg.spark.SparkSQLProperties.COMET_VECTORIZED_READER_FACTORY_CLASS; + } + return ImmutableParquetBatchReadConf.builder() .batchSize(readConf.parquetBatchSize()) .readerType(readerType) + .factoryClassName(factoryClassName) .build(); } diff --git a/spark/v3.4/spark/src/main/resources/META-INF/services/org.apache.iceberg.parquet.VectorizedParquetReaderFactory b/spark/v3.4/spark/src/main/resources/META-INF/services/org.apache.iceberg.parquet.VectorizedParquetReaderFactory deleted file mode 100644 index 2dec84408d64..000000000000 --- a/spark/v3.4/spark/src/main/resources/META-INF/services/org.apache.iceberg.parquet.VectorizedParquetReaderFactory +++ /dev/null @@ -1,19 +0,0 @@ -# -# 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. -# -org.apache.iceberg.spark.parquet.CometVectorizedParquetReaderFactory diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/ParquetBatchReadConf.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/ParquetBatchReadConf.java index 442d728d4d69..dc84485e17f4 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/ParquetBatchReadConf.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/ParquetBatchReadConf.java @@ -19,6 +19,7 @@ package org.apache.iceberg.spark; import java.io.Serializable; +import java.util.Optional; import org.immutables.value.Value; @Value.Immutable @@ -26,4 +27,6 @@ public interface ParquetBatchReadConf extends Serializable { int batchSize(); ParquetReaderType readerType(); + + Optional factoryClassName(); } diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java index 9ea08c316919..e0b017d6c6b1 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java @@ -375,4 +375,11 @@ public ParquetReaderType parquetReaderType() { .defaultValue(SparkSQLProperties.PARQUET_READER_TYPE_DEFAULT) .parse(); } + + public String parquetVectorizedReaderFactory() { + return confParser + .stringConf() + .sessionConf(SparkSQLProperties.PARQUET_VECTORIZED_READER_FACTORY) + .parseOptional(); + } } diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java index 5e76123cab42..8f516c1b5fdc 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java @@ -30,6 +30,15 @@ private 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; + + // Controls the fully qualified class name of the vectorized Parquet reader factory + public static final String PARQUET_VECTORIZED_READER_FACTORY = + "spark.sql.iceberg.parquet.vectorized-reader.factory"; + + // Comet vectorized reader factory class name + public static final String COMET_VECTORIZED_READER_FACTORY_CLASS = + "org.apache.iceberg.spark.parquet.CometVectorizedParquetReaderFactory"; + // Controls whether to perform the nullability check during writes public static final String CHECK_NULLABILITY = "spark.sql.iceberg.check-nullability"; public static final boolean CHECK_NULLABILITY_DEFAULT = true; diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/parquet/CometVectorizedParquetReaderFactory.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/parquet/CometVectorizedParquetReaderFactory.java index 934608347bc5..68c8d9a04034 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/parquet/CometVectorizedParquetReaderFactory.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/parquet/CometVectorizedParquetReaderFactory.java @@ -34,8 +34,9 @@ /** * Factory for creating Comet-based vectorized Parquet readers. * - *

This factory is loaded via Java's ServiceLoader mechanism and provides Comet's native - * vectorized Parquet reader implementation for Iceberg tables. + *

This factory is loaded via reflection when the {@code read.parquet.vectorized-reader.factory} + * property is set to {@code org.apache.iceberg.spark.parquet.CometVectorizedParquetReaderFactory}. + * It provides Comet's native vectorized Parquet reader implementation for Iceberg tables. */ public class CometVectorizedParquetReaderFactory implements VectorizedParquetReaderFactory { diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java index 473b34bb0f3e..4d6ceb5e7f9d 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java @@ -111,7 +111,7 @@ private CloseableIterable newParquetIterable( // read performance as every batch read doesn't have to pay the cost of allocating memory. .reuseContainers() .withNameMapping(nameMapping()) - .enableComet(parquetConf.readerType() == ParquetReaderType.COMET) + .vectorizedReaderFactory(parquetConf.factoryClassName().orElse(null)) .build(); } diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatch.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatch.java index 0626d0b43985..8a9ae2c63df5 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatch.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatch.java @@ -137,9 +137,18 @@ public PartitionReaderFactory createReaderFactory() { } private ParquetBatchReadConf parquetBatchReadConf(ParquetReaderType readerType) { + String factoryClassName = readConf.parquetVectorizedReaderFactory(); + + // If no explicit factory is set and reader type is COMET, use the default Comet factory + if (factoryClassName == null && readerType == ParquetReaderType.COMET) { + factoryClassName = + org.apache.iceberg.spark.SparkSQLProperties.COMET_VECTORIZED_READER_FACTORY_CLASS; + } + return ImmutableParquetBatchReadConf.builder() .batchSize(readConf.parquetBatchSize()) .readerType(readerType) + .factoryClassName(factoryClassName) .build(); } diff --git a/spark/v3.5/spark/src/main/resources/META-INF/services/org.apache.iceberg.parquet.VectorizedParquetReaderFactory b/spark/v3.5/spark/src/main/resources/META-INF/services/org.apache.iceberg.parquet.VectorizedParquetReaderFactory deleted file mode 100644 index 2dec84408d64..000000000000 --- a/spark/v3.5/spark/src/main/resources/META-INF/services/org.apache.iceberg.parquet.VectorizedParquetReaderFactory +++ /dev/null @@ -1,19 +0,0 @@ -# -# 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. -# -org.apache.iceberg.spark.parquet.CometVectorizedParquetReaderFactory diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/ParquetBatchReadConf.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/ParquetBatchReadConf.java index 442d728d4d69..dc84485e17f4 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/ParquetBatchReadConf.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/ParquetBatchReadConf.java @@ -19,6 +19,7 @@ package org.apache.iceberg.spark; import java.io.Serializable; +import java.util.Optional; import org.immutables.value.Value; @Value.Immutable @@ -26,4 +27,6 @@ public interface ParquetBatchReadConf extends Serializable { int batchSize(); ParquetReaderType readerType(); + + Optional factoryClassName(); } diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java index 2788e160d526..826d5b0ee095 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java @@ -375,4 +375,11 @@ public ParquetReaderType parquetReaderType() { .defaultValue(SparkSQLProperties.PARQUET_READER_TYPE_DEFAULT) .parse(); } + + public String parquetVectorizedReaderFactory() { + return confParser + .stringConf() + .sessionConf(SparkSQLProperties.PARQUET_VECTORIZED_READER_FACTORY) + .parseOptional(); + } } diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java index ba19d241f64e..d346884e94dc 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java @@ -30,6 +30,15 @@ private 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; + + // Controls the fully qualified class name of the vectorized Parquet reader factory + public static final String PARQUET_VECTORIZED_READER_FACTORY = + "spark.sql.iceberg.parquet.vectorized-reader.factory"; + + // Comet vectorized reader factory class name + public static final String COMET_VECTORIZED_READER_FACTORY_CLASS = + "org.apache.iceberg.spark.parquet.CometVectorizedParquetReaderFactory"; + // Controls whether to perform the nullability check during writes public static final String CHECK_NULLABILITY = "spark.sql.iceberg.check-nullability"; public static final boolean CHECK_NULLABILITY_DEFAULT = true; diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/parquet/CometVectorizedParquetReaderFactory.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/parquet/CometVectorizedParquetReaderFactory.java index 934608347bc5..68c8d9a04034 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/parquet/CometVectorizedParquetReaderFactory.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/parquet/CometVectorizedParquetReaderFactory.java @@ -34,8 +34,9 @@ /** * Factory for creating Comet-based vectorized Parquet readers. * - *

This factory is loaded via Java's ServiceLoader mechanism and provides Comet's native - * vectorized Parquet reader implementation for Iceberg tables. + *

This factory is loaded via reflection when the {@code read.parquet.vectorized-reader.factory} + * property is set to {@code org.apache.iceberg.spark.parquet.CometVectorizedParquetReaderFactory}. + * It provides Comet's native vectorized Parquet reader implementation for Iceberg tables. */ public class CometVectorizedParquetReaderFactory implements VectorizedParquetReaderFactory { diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java index 473b34bb0f3e..4d6ceb5e7f9d 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java @@ -111,7 +111,7 @@ private CloseableIterable newParquetIterable( // read performance as every batch read doesn't have to pay the cost of allocating memory. .reuseContainers() .withNameMapping(nameMapping()) - .enableComet(parquetConf.readerType() == ParquetReaderType.COMET) + .vectorizedReaderFactory(parquetConf.factoryClassName().orElse(null)) .build(); } diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatch.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatch.java index 0626d0b43985..8a9ae2c63df5 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatch.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatch.java @@ -137,9 +137,18 @@ public PartitionReaderFactory createReaderFactory() { } private ParquetBatchReadConf parquetBatchReadConf(ParquetReaderType readerType) { + String factoryClassName = readConf.parquetVectorizedReaderFactory(); + + // If no explicit factory is set and reader type is COMET, use the default Comet factory + if (factoryClassName == null && readerType == ParquetReaderType.COMET) { + factoryClassName = + org.apache.iceberg.spark.SparkSQLProperties.COMET_VECTORIZED_READER_FACTORY_CLASS; + } + return ImmutableParquetBatchReadConf.builder() .batchSize(readConf.parquetBatchSize()) .readerType(readerType) + .factoryClassName(factoryClassName) .build(); } diff --git a/spark/v4.0/spark/src/main/resources/META-INF/services/org.apache.iceberg.parquet.VectorizedParquetReaderFactory b/spark/v4.0/spark/src/main/resources/META-INF/services/org.apache.iceberg.parquet.VectorizedParquetReaderFactory deleted file mode 100644 index 8ce8a1ba7307..000000000000 --- a/spark/v4.0/spark/src/main/resources/META-INF/services/org.apache.iceberg.parquet.VectorizedParquetReaderFactory +++ /dev/null @@ -1,19 +0,0 @@ -# -# 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. -# -org.apache.iceberg.spark.parquet.CometVectorizedParquetReaderFactory \ No newline at end of file From 43ea52059d97d79ea150337d33f79cafcdd62079 Mon Sep 17 00:00:00 2001 From: Parth Chandra Date: Tue, 11 Nov 2025 13:01:18 -0800 Subject: [PATCH 09/15] use builder --- .../org/apache/iceberg/parquet/Parquet.java | 25 +- .../VectorizedParquetReaderFactory.java | 278 ++++++++++++++++-- .../parquet/CometVectorizedParquetReader.java | 106 ++++++- .../CometVectorizedParquetReaderFactory.java | 53 +--- .../parquet/CometVectorizedParquetReader.java | 106 ++++++- .../CometVectorizedParquetReaderFactory.java | 53 +--- .../parquet/CometVectorizedParquetReader.java | 106 ++++++- .../CometVectorizedParquetReaderFactory.java | 53 +--- 8 files changed, 599 insertions(+), 181 deletions(-) diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java b/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java index 59c607cfce45..07eba1a0282e 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java @@ -1479,20 +1479,17 @@ public CloseableIterable build() { VectorizedParquetReaderFactory factory = loadReaderFactory(readerName); if (factory != null) { return factory.createReader( - file, - schema, - options, - batchedReaderFunc, - mapping, - filter, - reuseContainers, - caseSensitive, - maxRecordsPerBatch, - properties, - start, - length, - fileEncryptionKey, - fileAADPrefix); + VectorizedParquetReaderFactory.ReaderParams.builder( + file, schema, options, batchedReaderFunc) + .nameMapping(mapping) + .filter(filter) + .reuseContainers(reuseContainers) + .caseSensitive(caseSensitive) + .maxRecordsPerBatch(maxRecordsPerBatch) + .properties(properties) + .split(start, length) + .encryption(fileEncryptionKey, fileAADPrefix) + .build()); } } diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/VectorizedParquetReaderFactory.java b/parquet/src/main/java/org/apache/iceberg/parquet/VectorizedParquetReaderFactory.java index fb78affdcf3b..6ea4e2b81408 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/VectorizedParquetReaderFactory.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/VectorizedParquetReaderFactory.java @@ -54,36 +54,256 @@ public interface VectorizedParquetReaderFactory { /** * Creates a vectorized parquet reader with the given configuration. * - * @param file the input file to read - * @param schema the expected schema for the data - * @param options parquet read options - * @param batchedReaderFunc function to create a VectorizedReader from a MessageType - * @param mapping name mapping for schema evolution - * @param filter filter expression to apply during reading - * @param reuseContainers whether to reuse containers for records - * @param caseSensitive whether column name matching should be case-sensitive - * @param maxRecordsPerBatch maximum number of records per batch - * @param properties additional properties for reader configuration - * @param start optional start position for reading - * @param length optional length to read - * @param fileEncryptionKey optional encryption key for the file - * @param fileAADPrefix optional AAD prefix for encryption + * @param params reader parameters encapsulating all configuration options * @param the type of records returned by the reader * @return a closeable iterable of records */ - CloseableIterable createReader( - InputFile file, - Schema schema, - ParquetReadOptions options, - Function> batchedReaderFunc, - NameMapping mapping, - Expression filter, - boolean reuseContainers, - boolean caseSensitive, - int maxRecordsPerBatch, - Map properties, - Long start, - Long length, - ByteBuffer fileEncryptionKey, - ByteBuffer fileAADPrefix); + CloseableIterable createReader(ReaderParams params); + + /** Parameters for creating a vectorized parquet reader. */ + interface ReaderParams { + InputFile file(); + + Schema schema(); + + ParquetReadOptions options(); + + Function> batchedReaderFunc(); + + NameMapping mapping(); + + Expression filter(); + + boolean reuseContainers(); + + boolean caseSensitive(); + + int maxRecordsPerBatch(); + + Map properties(); + + Long start(); + + Long length(); + + ByteBuffer fileEncryptionKey(); + + ByteBuffer fileAADPrefix(); + + static Builder builder( + InputFile file, + Schema schema, + ParquetReadOptions options, + Function> batchedReaderFunc) { + return new Builder(file, schema, options, batchedReaderFunc); + } + + /** Builder for ReaderParams. */ + class Builder { + private final InputFile file; + private final Schema schema; + private final ParquetReadOptions options; + private final Function> batchedReaderFunc; + private NameMapping mapping = null; + private Expression filter = null; + private boolean reuseContainers = false; + private boolean caseSensitive = true; + private int maxRecordsPerBatch = 10000; + private Map properties = null; + private Long start = null; + private Long length = null; + private ByteBuffer fileEncryptionKey = null; + private ByteBuffer fileAADPrefix = null; + + private Builder( + InputFile file, + Schema schema, + ParquetReadOptions options, + Function> batchedReaderFunc) { + this.file = file; + this.schema = schema; + this.options = options; + this.batchedReaderFunc = batchedReaderFunc; + } + + public Builder nameMapping(NameMapping nameMapping) { + this.mapping = nameMapping; + return this; + } + + public Builder filter(Expression filterExpr) { + this.filter = filterExpr; + return this; + } + + public Builder reuseContainers(boolean reuse) { + this.reuseContainers = reuse; + return this; + } + + public Builder caseSensitive(boolean sensitive) { + this.caseSensitive = sensitive; + return this; + } + + public Builder maxRecordsPerBatch(int maxRecords) { + this.maxRecordsPerBatch = maxRecords; + return this; + } + + public Builder properties(Map props) { + this.properties = props; + return this; + } + + public Builder split(Long splitStart, Long splitLength) { + this.start = splitStart; + this.length = splitLength; + return this; + } + + public Builder encryption(ByteBuffer encryptionKey, ByteBuffer aadPrefix) { + this.fileEncryptionKey = encryptionKey; + this.fileAADPrefix = aadPrefix; + return this; + } + + public ReaderParams build() { + return new ReaderParamsImpl( + file, + schema, + options, + batchedReaderFunc, + mapping, + filter, + reuseContainers, + caseSensitive, + maxRecordsPerBatch, + properties, + start, + length, + fileEncryptionKey, + fileAADPrefix); + } + } + } + + /** Implementation of ReaderParams. */ + class ReaderParamsImpl implements ReaderParams { + private final InputFile file; + private final Schema schema; + private final ParquetReadOptions options; + private final Function> batchedReaderFunc; + private final NameMapping mapping; + private final Expression filter; + private final boolean reuseContainers; + private final boolean caseSensitive; + private final int maxRecordsPerBatch; + private final Map properties; + private final Long start; + private final Long length; + private final ByteBuffer fileEncryptionKey; + private final ByteBuffer fileAADPrefix; + + private ReaderParamsImpl( + InputFile file, + Schema schema, + ParquetReadOptions options, + Function> batchedReaderFunc, + NameMapping mapping, + Expression filter, + boolean reuseContainers, + boolean caseSensitive, + int maxRecordsPerBatch, + Map properties, + Long start, + Long length, + ByteBuffer fileEncryptionKey, + ByteBuffer fileAADPrefix) { + this.file = file; + this.schema = schema; + this.options = options; + this.batchedReaderFunc = batchedReaderFunc; + this.mapping = mapping; + this.filter = filter; + this.reuseContainers = reuseContainers; + this.caseSensitive = caseSensitive; + this.maxRecordsPerBatch = maxRecordsPerBatch; + this.properties = properties; + this.start = start; + this.length = length; + this.fileEncryptionKey = fileEncryptionKey; + this.fileAADPrefix = fileAADPrefix; + } + + @Override + public InputFile file() { + return file; + } + + @Override + public Schema schema() { + return schema; + } + + @Override + public ParquetReadOptions options() { + return options; + } + + @Override + public Function> batchedReaderFunc() { + return batchedReaderFunc; + } + + @Override + public NameMapping mapping() { + return mapping; + } + + @Override + public Expression filter() { + return filter; + } + + @Override + public boolean reuseContainers() { + return reuseContainers; + } + + @Override + public boolean caseSensitive() { + return caseSensitive; + } + + @Override + public int maxRecordsPerBatch() { + return maxRecordsPerBatch; + } + + @Override + public Map properties() { + return properties; + } + + @Override + public Long start() { + return start; + } + + @Override + public Long length() { + return length; + } + + @Override + public ByteBuffer fileEncryptionKey() { + return fileEncryptionKey; + } + + @Override + public ByteBuffer fileAADPrefix() { + return fileAADPrefix; + } + } } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/parquet/CometVectorizedParquetReader.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/parquet/CometVectorizedParquetReader.java index f133896b1201..dee3fab05ff6 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/parquet/CometVectorizedParquetReader.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/parquet/CometVectorizedParquetReader.java @@ -62,12 +62,12 @@ public class CometVectorizedParquetReader extends CloseableGroup private final int batchSize; private final NameMapping nameMapping; private final Map properties; - private Long start = null; - private Long length = null; - private ByteBuffer fileEncryptionKey = null; - private ByteBuffer fileAADPrefix = null; + private final Long start; + private final Long length; + private final ByteBuffer fileEncryptionKey; + private final ByteBuffer fileAADPrefix; - public CometVectorizedParquetReader( + private CometVectorizedParquetReader( InputFile input, Schema expectedSchema, ParquetReadOptions options, @@ -99,6 +99,102 @@ public CometVectorizedParquetReader( this.fileAADPrefix = fileAADPrefix; } + public static Builder builder( + InputFile file, + Schema schema, + ParquetReadOptions options, + Function> batchedReaderFunc) { + return new Builder(file, schema, options, batchedReaderFunc); + } + + public static class Builder { + private final InputFile file; + private final Schema schema; + private final ParquetReadOptions options; + private final Function> batchedReaderFunc; + private NameMapping nameMapping = null; + private Expression filter = null; + private boolean reuseContainers = false; + private boolean caseSensitive = true; + private int maxRecordsPerBatch = 10000; + private Map properties = null; + private Long start = null; + private Long length = null; + private ByteBuffer fileEncryptionKey = null; + private ByteBuffer fileAADPrefix = null; + + private Builder( + InputFile file, + Schema schema, + ParquetReadOptions options, + Function> batchedReaderFunc) { + this.file = file; + this.schema = schema; + this.options = options; + this.batchedReaderFunc = batchedReaderFunc; + } + + public Builder nameMapping(NameMapping mapping) { + this.nameMapping = mapping; + return this; + } + + public Builder filter(Expression filterExpr) { + this.filter = filterExpr; + return this; + } + + public Builder reuseContainers(boolean reuse) { + this.reuseContainers = reuse; + return this; + } + + public Builder caseSensitive(boolean sensitive) { + this.caseSensitive = sensitive; + return this; + } + + public Builder maxRecordsPerBatch(int maxRecords) { + this.maxRecordsPerBatch = maxRecords; + return this; + } + + public Builder properties(Map props) { + this.properties = props; + return this; + } + + public Builder split(Long splitStart, Long splitLength) { + this.start = splitStart; + this.length = splitLength; + return this; + } + + public Builder encryption(ByteBuffer encryptionKey, ByteBuffer aadPrefix) { + this.fileEncryptionKey = encryptionKey; + this.fileAADPrefix = aadPrefix; + return this; + } + + public CometVectorizedParquetReader build() { + return new CometVectorizedParquetReader<>( + file, + schema, + options, + batchedReaderFunc, + nameMapping, + filter, + reuseContainers, + caseSensitive, + maxRecordsPerBatch, + properties, + start, + length, + fileEncryptionKey, + fileAADPrefix); + } + } + private ReadConf conf = null; private ReadConf init() { diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/parquet/CometVectorizedParquetReaderFactory.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/parquet/CometVectorizedParquetReaderFactory.java index 68c8d9a04034..25e7616ac373 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/parquet/CometVectorizedParquetReaderFactory.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/parquet/CometVectorizedParquetReaderFactory.java @@ -18,18 +18,8 @@ */ package org.apache.iceberg.spark.parquet; -import java.nio.ByteBuffer; -import java.util.Map; -import java.util.function.Function; -import org.apache.iceberg.Schema; -import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.io.CloseableIterable; -import org.apache.iceberg.io.InputFile; -import org.apache.iceberg.mapping.NameMapping; import org.apache.iceberg.parquet.VectorizedParquetReaderFactory; -import org.apache.iceberg.parquet.VectorizedReader; -import org.apache.parquet.ParquetReadOptions; -import org.apache.parquet.schema.MessageType; /** * Factory for creating Comet-based vectorized Parquet readers. @@ -46,36 +36,17 @@ public String name() { } @Override - public CloseableIterable createReader( - InputFile file, - Schema schema, - ParquetReadOptions options, - Function> batchedReaderFunc, - NameMapping mapping, - Expression filter, - boolean reuseContainers, - boolean caseSensitive, - int maxRecordsPerBatch, - Map properties, - Long start, - Long length, - ByteBuffer fileEncryptionKey, - ByteBuffer fileAADPrefix) { - - return new CometVectorizedParquetReader<>( - file, - schema, - options, - batchedReaderFunc, - mapping, - filter, - reuseContainers, - caseSensitive, - maxRecordsPerBatch, - properties, - start, - length, - fileEncryptionKey, - fileAADPrefix); + public CloseableIterable createReader(ReaderParams params) { + return CometVectorizedParquetReader.builder( + params.file(), params.schema(), params.options(), params.batchedReaderFunc()) + .nameMapping(params.mapping()) + .filter(params.filter()) + .reuseContainers(params.reuseContainers()) + .caseSensitive(params.caseSensitive()) + .maxRecordsPerBatch(params.maxRecordsPerBatch()) + .properties(params.properties()) + .split(params.start(), params.length()) + .encryption(params.fileEncryptionKey(), params.fileAADPrefix()) + .build(); } } diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/parquet/CometVectorizedParquetReader.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/parquet/CometVectorizedParquetReader.java index f133896b1201..dee3fab05ff6 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/parquet/CometVectorizedParquetReader.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/parquet/CometVectorizedParquetReader.java @@ -62,12 +62,12 @@ public class CometVectorizedParquetReader extends CloseableGroup private final int batchSize; private final NameMapping nameMapping; private final Map properties; - private Long start = null; - private Long length = null; - private ByteBuffer fileEncryptionKey = null; - private ByteBuffer fileAADPrefix = null; + private final Long start; + private final Long length; + private final ByteBuffer fileEncryptionKey; + private final ByteBuffer fileAADPrefix; - public CometVectorizedParquetReader( + private CometVectorizedParquetReader( InputFile input, Schema expectedSchema, ParquetReadOptions options, @@ -99,6 +99,102 @@ public CometVectorizedParquetReader( this.fileAADPrefix = fileAADPrefix; } + public static Builder builder( + InputFile file, + Schema schema, + ParquetReadOptions options, + Function> batchedReaderFunc) { + return new Builder(file, schema, options, batchedReaderFunc); + } + + public static class Builder { + private final InputFile file; + private final Schema schema; + private final ParquetReadOptions options; + private final Function> batchedReaderFunc; + private NameMapping nameMapping = null; + private Expression filter = null; + private boolean reuseContainers = false; + private boolean caseSensitive = true; + private int maxRecordsPerBatch = 10000; + private Map properties = null; + private Long start = null; + private Long length = null; + private ByteBuffer fileEncryptionKey = null; + private ByteBuffer fileAADPrefix = null; + + private Builder( + InputFile file, + Schema schema, + ParquetReadOptions options, + Function> batchedReaderFunc) { + this.file = file; + this.schema = schema; + this.options = options; + this.batchedReaderFunc = batchedReaderFunc; + } + + public Builder nameMapping(NameMapping mapping) { + this.nameMapping = mapping; + return this; + } + + public Builder filter(Expression filterExpr) { + this.filter = filterExpr; + return this; + } + + public Builder reuseContainers(boolean reuse) { + this.reuseContainers = reuse; + return this; + } + + public Builder caseSensitive(boolean sensitive) { + this.caseSensitive = sensitive; + return this; + } + + public Builder maxRecordsPerBatch(int maxRecords) { + this.maxRecordsPerBatch = maxRecords; + return this; + } + + public Builder properties(Map props) { + this.properties = props; + return this; + } + + public Builder split(Long splitStart, Long splitLength) { + this.start = splitStart; + this.length = splitLength; + return this; + } + + public Builder encryption(ByteBuffer encryptionKey, ByteBuffer aadPrefix) { + this.fileEncryptionKey = encryptionKey; + this.fileAADPrefix = aadPrefix; + return this; + } + + public CometVectorizedParquetReader build() { + return new CometVectorizedParquetReader<>( + file, + schema, + options, + batchedReaderFunc, + nameMapping, + filter, + reuseContainers, + caseSensitive, + maxRecordsPerBatch, + properties, + start, + length, + fileEncryptionKey, + fileAADPrefix); + } + } + private ReadConf conf = null; private ReadConf init() { diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/parquet/CometVectorizedParquetReaderFactory.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/parquet/CometVectorizedParquetReaderFactory.java index 68c8d9a04034..25e7616ac373 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/parquet/CometVectorizedParquetReaderFactory.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/parquet/CometVectorizedParquetReaderFactory.java @@ -18,18 +18,8 @@ */ package org.apache.iceberg.spark.parquet; -import java.nio.ByteBuffer; -import java.util.Map; -import java.util.function.Function; -import org.apache.iceberg.Schema; -import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.io.CloseableIterable; -import org.apache.iceberg.io.InputFile; -import org.apache.iceberg.mapping.NameMapping; import org.apache.iceberg.parquet.VectorizedParquetReaderFactory; -import org.apache.iceberg.parquet.VectorizedReader; -import org.apache.parquet.ParquetReadOptions; -import org.apache.parquet.schema.MessageType; /** * Factory for creating Comet-based vectorized Parquet readers. @@ -46,36 +36,17 @@ public String name() { } @Override - public CloseableIterable createReader( - InputFile file, - Schema schema, - ParquetReadOptions options, - Function> batchedReaderFunc, - NameMapping mapping, - Expression filter, - boolean reuseContainers, - boolean caseSensitive, - int maxRecordsPerBatch, - Map properties, - Long start, - Long length, - ByteBuffer fileEncryptionKey, - ByteBuffer fileAADPrefix) { - - return new CometVectorizedParquetReader<>( - file, - schema, - options, - batchedReaderFunc, - mapping, - filter, - reuseContainers, - caseSensitive, - maxRecordsPerBatch, - properties, - start, - length, - fileEncryptionKey, - fileAADPrefix); + public CloseableIterable createReader(ReaderParams params) { + return CometVectorizedParquetReader.builder( + params.file(), params.schema(), params.options(), params.batchedReaderFunc()) + .nameMapping(params.mapping()) + .filter(params.filter()) + .reuseContainers(params.reuseContainers()) + .caseSensitive(params.caseSensitive()) + .maxRecordsPerBatch(params.maxRecordsPerBatch()) + .properties(params.properties()) + .split(params.start(), params.length()) + .encryption(params.fileEncryptionKey(), params.fileAADPrefix()) + .build(); } } diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/parquet/CometVectorizedParquetReader.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/parquet/CometVectorizedParquetReader.java index f133896b1201..dee3fab05ff6 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/parquet/CometVectorizedParquetReader.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/parquet/CometVectorizedParquetReader.java @@ -62,12 +62,12 @@ public class CometVectorizedParquetReader extends CloseableGroup private final int batchSize; private final NameMapping nameMapping; private final Map properties; - private Long start = null; - private Long length = null; - private ByteBuffer fileEncryptionKey = null; - private ByteBuffer fileAADPrefix = null; + private final Long start; + private final Long length; + private final ByteBuffer fileEncryptionKey; + private final ByteBuffer fileAADPrefix; - public CometVectorizedParquetReader( + private CometVectorizedParquetReader( InputFile input, Schema expectedSchema, ParquetReadOptions options, @@ -99,6 +99,102 @@ public CometVectorizedParquetReader( this.fileAADPrefix = fileAADPrefix; } + public static Builder builder( + InputFile file, + Schema schema, + ParquetReadOptions options, + Function> batchedReaderFunc) { + return new Builder(file, schema, options, batchedReaderFunc); + } + + public static class Builder { + private final InputFile file; + private final Schema schema; + private final ParquetReadOptions options; + private final Function> batchedReaderFunc; + private NameMapping nameMapping = null; + private Expression filter = null; + private boolean reuseContainers = false; + private boolean caseSensitive = true; + private int maxRecordsPerBatch = 10000; + private Map properties = null; + private Long start = null; + private Long length = null; + private ByteBuffer fileEncryptionKey = null; + private ByteBuffer fileAADPrefix = null; + + private Builder( + InputFile file, + Schema schema, + ParquetReadOptions options, + Function> batchedReaderFunc) { + this.file = file; + this.schema = schema; + this.options = options; + this.batchedReaderFunc = batchedReaderFunc; + } + + public Builder nameMapping(NameMapping mapping) { + this.nameMapping = mapping; + return this; + } + + public Builder filter(Expression filterExpr) { + this.filter = filterExpr; + return this; + } + + public Builder reuseContainers(boolean reuse) { + this.reuseContainers = reuse; + return this; + } + + public Builder caseSensitive(boolean sensitive) { + this.caseSensitive = sensitive; + return this; + } + + public Builder maxRecordsPerBatch(int maxRecords) { + this.maxRecordsPerBatch = maxRecords; + return this; + } + + public Builder properties(Map props) { + this.properties = props; + return this; + } + + public Builder split(Long splitStart, Long splitLength) { + this.start = splitStart; + this.length = splitLength; + return this; + } + + public Builder encryption(ByteBuffer encryptionKey, ByteBuffer aadPrefix) { + this.fileEncryptionKey = encryptionKey; + this.fileAADPrefix = aadPrefix; + return this; + } + + public CometVectorizedParquetReader build() { + return new CometVectorizedParquetReader<>( + file, + schema, + options, + batchedReaderFunc, + nameMapping, + filter, + reuseContainers, + caseSensitive, + maxRecordsPerBatch, + properties, + start, + length, + fileEncryptionKey, + fileAADPrefix); + } + } + private ReadConf conf = null; private ReadConf init() { diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/parquet/CometVectorizedParquetReaderFactory.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/parquet/CometVectorizedParquetReaderFactory.java index 68c8d9a04034..25e7616ac373 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/parquet/CometVectorizedParquetReaderFactory.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/parquet/CometVectorizedParquetReaderFactory.java @@ -18,18 +18,8 @@ */ package org.apache.iceberg.spark.parquet; -import java.nio.ByteBuffer; -import java.util.Map; -import java.util.function.Function; -import org.apache.iceberg.Schema; -import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.io.CloseableIterable; -import org.apache.iceberg.io.InputFile; -import org.apache.iceberg.mapping.NameMapping; import org.apache.iceberg.parquet.VectorizedParquetReaderFactory; -import org.apache.iceberg.parquet.VectorizedReader; -import org.apache.parquet.ParquetReadOptions; -import org.apache.parquet.schema.MessageType; /** * Factory for creating Comet-based vectorized Parquet readers. @@ -46,36 +36,17 @@ public String name() { } @Override - public CloseableIterable createReader( - InputFile file, - Schema schema, - ParquetReadOptions options, - Function> batchedReaderFunc, - NameMapping mapping, - Expression filter, - boolean reuseContainers, - boolean caseSensitive, - int maxRecordsPerBatch, - Map properties, - Long start, - Long length, - ByteBuffer fileEncryptionKey, - ByteBuffer fileAADPrefix) { - - return new CometVectorizedParquetReader<>( - file, - schema, - options, - batchedReaderFunc, - mapping, - filter, - reuseContainers, - caseSensitive, - maxRecordsPerBatch, - properties, - start, - length, - fileEncryptionKey, - fileAADPrefix); + public CloseableIterable createReader(ReaderParams params) { + return CometVectorizedParquetReader.builder( + params.file(), params.schema(), params.options(), params.batchedReaderFunc()) + .nameMapping(params.mapping()) + .filter(params.filter()) + .reuseContainers(params.reuseContainers()) + .caseSensitive(params.caseSensitive()) + .maxRecordsPerBatch(params.maxRecordsPerBatch()) + .properties(params.properties()) + .split(params.start(), params.length()) + .encryption(params.fileEncryptionKey(), params.fileAADPrefix()) + .build(); } } From 22d5de10b2760f7edf9e2bead3741ad97370937c Mon Sep 17 00:00:00 2001 From: Parth Chandra Date: Tue, 11 Nov 2025 17:51:52 -0800 Subject: [PATCH 10/15] Fix NPE --- .../apache/iceberg/spark/source/SparkBatch.java | 15 ++++++++++----- .../apache/iceberg/spark/source/SparkBatch.java | 15 ++++++++++----- .../apache/iceberg/spark/source/SparkBatch.java | 15 ++++++++++----- 3 files changed, 30 insertions(+), 15 deletions(-) diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatch.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatch.java index 8a9ae2c63df5..dce54167e224 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatch.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatch.java @@ -145,11 +145,16 @@ private ParquetBatchReadConf parquetBatchReadConf(ParquetReaderType readerType) org.apache.iceberg.spark.SparkSQLProperties.COMET_VECTORIZED_READER_FACTORY_CLASS; } - return ImmutableParquetBatchReadConf.builder() - .batchSize(readConf.parquetBatchSize()) - .readerType(readerType) - .factoryClassName(factoryClassName) - .build(); + ImmutableParquetBatchReadConf.Builder builder = + ImmutableParquetBatchReadConf.builder() + .batchSize(readConf.parquetBatchSize()) + .readerType(readerType); + + if (factoryClassName != null) { + builder.factoryClassName(factoryClassName); + } + + return builder.build(); } private OrcBatchReadConf orcBatchReadConf() { diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatch.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatch.java index 8a9ae2c63df5..dce54167e224 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatch.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatch.java @@ -145,11 +145,16 @@ private ParquetBatchReadConf parquetBatchReadConf(ParquetReaderType readerType) org.apache.iceberg.spark.SparkSQLProperties.COMET_VECTORIZED_READER_FACTORY_CLASS; } - return ImmutableParquetBatchReadConf.builder() - .batchSize(readConf.parquetBatchSize()) - .readerType(readerType) - .factoryClassName(factoryClassName) - .build(); + ImmutableParquetBatchReadConf.Builder builder = + ImmutableParquetBatchReadConf.builder() + .batchSize(readConf.parquetBatchSize()) + .readerType(readerType); + + if (factoryClassName != null) { + builder.factoryClassName(factoryClassName); + } + + return builder.build(); } private OrcBatchReadConf orcBatchReadConf() { diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatch.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatch.java index 8a9ae2c63df5..dce54167e224 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatch.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatch.java @@ -145,11 +145,16 @@ private ParquetBatchReadConf parquetBatchReadConf(ParquetReaderType readerType) org.apache.iceberg.spark.SparkSQLProperties.COMET_VECTORIZED_READER_FACTORY_CLASS; } - return ImmutableParquetBatchReadConf.builder() - .batchSize(readConf.parquetBatchSize()) - .readerType(readerType) - .factoryClassName(factoryClassName) - .build(); + ImmutableParquetBatchReadConf.Builder builder = + ImmutableParquetBatchReadConf.builder() + .batchSize(readConf.parquetBatchSize()) + .readerType(readerType); + + if (factoryClassName != null) { + builder.factoryClassName(factoryClassName); + } + + return builder.build(); } private OrcBatchReadConf orcBatchReadConf() { From bc0b83ef7c119cf28a1b262172562b993ed5c6d6 Mon Sep 17 00:00:00 2001 From: Parth Chandra Date: Tue, 18 Nov 2025 13:56:17 -0800 Subject: [PATCH 11/15] remove unnecessary blank line --- .../main/java/org/apache/iceberg/spark/SparkSQLProperties.java | 1 - .../main/java/org/apache/iceberg/spark/SparkSQLProperties.java | 1 - .../main/java/org/apache/iceberg/spark/SparkSQLProperties.java | 1 - 3 files changed, 3 deletions(-) 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 d13b4d572029..abeb29da3a30 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 @@ -30,7 +30,6 @@ private 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; - // Controls the fully qualified class name of the vectorized Parquet reader factory public static final String PARQUET_VECTORIZED_READER_FACTORY = "spark.sql.iceberg.parquet.vectorized-reader.factory"; diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java index 8f516c1b5fdc..5aae7fd5cbeb 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java @@ -30,7 +30,6 @@ private 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; - // Controls the fully qualified class name of the vectorized Parquet reader factory public static final String PARQUET_VECTORIZED_READER_FACTORY = "spark.sql.iceberg.parquet.vectorized-reader.factory"; diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java index d346884e94dc..33e6ab1180df 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java @@ -30,7 +30,6 @@ private 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; - // Controls the fully qualified class name of the vectorized Parquet reader factory public static final String PARQUET_VECTORIZED_READER_FACTORY = "spark.sql.iceberg.parquet.vectorized-reader.factory"; From e65888ec3df6476e64f2faa8349713280a9d616a Mon Sep 17 00:00:00 2001 From: Parth Chandra Date: Tue, 18 Nov 2025 14:23:37 -0800 Subject: [PATCH 12/15] fix after updating from main --- spark/v4.0/build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/spark/v4.0/build.gradle b/spark/v4.0/build.gradle index 4353b0fa49a0..552fb4021f1c 100644 --- a/spark/v4.0/build.gradle +++ b/spark/v4.0/build.gradle @@ -276,7 +276,7 @@ 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 "org.apache.datafusion:comet-spark-spark3.5_2.13:${libs.versions.comet.get()}" + integrationImplementation "org.apache.datafusion:comet-spark-spark${sparkMajorVersion}_2.13:${libs.versions.comet.get()}" // runtime dependencies for running Hive Catalog based integration test From 6d74b808f6718928abd878a1f5f8b1e969bedcc8 Mon Sep 17 00:00:00 2001 From: Parth Chandra Date: Thu, 4 Dec 2025 17:14:36 -0800 Subject: [PATCH 13/15] revert inadvertent version change --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index fd83b9e92358..3ba051c8b897 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -37,7 +37,7 @@ awssdk-s3accessgrants = "2.3.0" bson-ver = "4.11.5" caffeine = "2.9.3" calcite = "1.41.0" -comet = "0.12.0" +comet = "0.10.1" datasketches = "6.2.0" delta-standalone = "3.3.2" delta-spark = "3.3.2" From efb0f9e1325d675391caef60473687f5594c443e Mon Sep 17 00:00:00 2001 From: Parth Chandra Date: Fri, 5 Dec 2025 11:41:52 -0800 Subject: [PATCH 14/15] address review comments --- .../org/apache/iceberg/parquet/Parquet.java | 32 +-- .../apache/iceberg/parquet/TestParquet.java | 241 ++++++++++++++++++ .../iceberg/spark/source/SparkBatch.java | 6 - .../iceberg/spark/source/SparkBatch.java | 6 - .../apache/iceberg/spark/SparkReadConf.java | 1 + .../iceberg/spark/source/SparkBatch.java | 6 - 6 files changed, 249 insertions(+), 43 deletions(-) diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java b/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java index 07eba1a0282e..81c6d436baed 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java @@ -1379,18 +1379,6 @@ public ReadBuilder setCustomType(int fieldId, Class struct return this; } - /** Convenience method to enable comet */ - public ReadBuilder enableComet(boolean enableComet) { - if (enableComet) { - this.properties.put( - VECTORIZED_READER_FACTORY, - "org.apache.iceberg.spark.parquet.CometVectorizedParquetReaderFactory"); - } else { - this.properties.remove(VECTORIZED_READER_FACTORY); - } - return this; - } - /** * Sets the vectorized reader factory class to use for reading Parquet files. * @@ -1472,11 +1460,11 @@ public CloseableIterable build() { if (batchedReaderFunc != null) { // Try to load custom vectorized reader factory from properties - String readerName = properties.get(VECTORIZED_READER_FACTORY); + String factoryName = properties.get(VECTORIZED_READER_FACTORY); - if (readerName != null) { - LOG.info("Loading custom vectorized reader factory: {}", readerName); - VectorizedParquetReaderFactory factory = loadReaderFactory(readerName); + if (factoryName != null) { + LOG.info("Loading custom vectorized reader factory: {}", factoryName); + VectorizedParquetReaderFactory factory = loadReaderFactory(factoryName); if (factory != null) { return factory.createReader( VectorizedParquetReaderFactory.ReaderParams.builder( @@ -1592,18 +1580,12 @@ public CloseableIterable build() { private static VectorizedParquetReaderFactory loadReaderFactory(String className) { try { Class factoryClass = Class.forName(className); - if (!VectorizedParquetReaderFactory.class.isAssignableFrom(factoryClass)) { + if (VectorizedParquetReaderFactory.class.isAssignableFrom(factoryClass)) { + return (VectorizedParquetReaderFactory) factoryClass.getDeclaredConstructor().newInstance(); + } else { LOG.warn("Class {} does not implement VectorizedParquetReaderFactory interface", className); return null; } - return (VectorizedParquetReaderFactory) factoryClass.getDeclaredConstructor().newInstance(); - } catch (ClassNotFoundException e) { - LOG.warn("Could not find vectorized reader factory class: {}", className, e); - return null; - } catch (NoSuchMethodException e) { - LOG.warn( - "Vectorized reader factory class {} does not have a no-arg constructor", className, e); - return null; } catch (Exception e) { LOG.warn("Failed to instantiate vectorized reader factory: {}", className, e); return null; diff --git a/parquet/src/test/java/org/apache/iceberg/parquet/TestParquet.java b/parquet/src/test/java/org/apache/iceberg/parquet/TestParquet.java index 58850ec7c9f4..531c89c8c678 100644 --- a/parquet/src/test/java/org/apache/iceberg/parquet/TestParquet.java +++ b/parquet/src/test/java/org/apache/iceberg/parquet/TestParquet.java @@ -36,6 +36,7 @@ import java.nio.ByteBuffer; import java.nio.file.Path; import java.util.Collections; +import java.util.Iterator; import java.util.List; import java.util.function.Function; import java.util.stream.Stream; @@ -314,6 +315,176 @@ public void testFooterMetricsWithNameMappingForFileWithoutIds() throws IOExcepti } } + @Test + public void testVectorizedReaderFactoryConfiguration() throws IOException { + Schema schema = new Schema(optional(1, "intCol", IntegerType.get())); + File file = createTempFile(temp); + + // Write test data + List records = Lists.newArrayList(); + org.apache.avro.Schema avroSchema = AvroSchemaUtil.convert(schema.asStruct()); + GenericData.Record record = new GenericData.Record(avroSchema); + record.put("intCol", 42); + records.add(record); + + write(file, schema, Collections.emptyMap(), null, records.toArray(new GenericData.Record[] {})); + + // Reset the flag + TestMockVectorizedReaderFactory.wasCalled = false; + + // Test setting vectorized reader factory + Parquet.ReadBuilder readBuilder = + Parquet.read(Files.localInput(file)) + .project(schema) + .createBatchedReaderFunc(fileSchema -> new MockVectorizedReader()) + .vectorizedReaderFactory(MockVectorizedReaderFactory.class.getName()); + + // We can't easily verify the property directly since it's private, + // but we can verify the build succeeds + readBuilder.build().iterator(); // Should not throw + + // Verify our mock factory was NOT used (because MockVectorizedReaderFactory is not a valid factory) + assertThat(TestMockVectorizedReaderFactory.wasCalled) + .as("TestMockVectorizedReaderFactory should not have been called") + .isFalse(); + } + + @Test + public void testVectorizedReaderFactoryRemoveWithNull() throws IOException { + Schema schema = new Schema(optional(1, "intCol", IntegerType.get())); + File file = createTempFile(temp); + + // Write test data + List records = Lists.newArrayList(); + org.apache.avro.Schema avroSchema = AvroSchemaUtil.convert(schema.asStruct()); + GenericData.Record record = new GenericData.Record(avroSchema); + record.put("intCol", 42); + records.add(record); + + write(file, schema, Collections.emptyMap(), null, records.toArray(new GenericData.Record[] {})); + + // Test removing vectorized reader factory with null + Parquet.ReadBuilder readBuilder = + Parquet.read(Files.localInput(file)) + .project(schema) + .createBatchedReaderFunc(fileSchema -> new MockVectorizedReader()) + .vectorizedReaderFactory(MockVectorizedReaderFactory.class.getName()) + .vectorizedReaderFactory(null); // Remove it + + // Build should succeed and use default reader + readBuilder.build().iterator(); // Should not throw + } + + @Test + public void testVectorizedReaderFactoryMissingClass() throws IOException { + Schema schema = new Schema(optional(1, "intCol", IntegerType.get())); + File file = createTempFile(temp); + + // Write test data + List records = Lists.newArrayList(); + org.apache.avro.Schema avroSchema = AvroSchemaUtil.convert(schema.asStruct()); + GenericData.Record record = new GenericData.Record(avroSchema); + record.put("intCol", 42); + records.add(record); + + write(file, schema, Collections.emptyMap(), null, records.toArray(new GenericData.Record[] {})); + + // Test with non-existent class - should fall back to default reader + Parquet.ReadBuilder readBuilder = + Parquet.read(Files.localInput(file)) + .project(schema) + .createBatchedReaderFunc(fileSchema -> new MockVectorizedReader()) + .vectorizedReaderFactory("com.example.NonExistentFactory"); + + // Should not throw - falls back to default reader + readBuilder.build().iterator(); + } + + @Test + public void testVectorizedReaderFactoryInvalidClass() throws IOException { + Schema schema = new Schema(optional(1, "intCol", IntegerType.get())); + File file = createTempFile(temp); + + // Write test data + List records = Lists.newArrayList(); + org.apache.avro.Schema avroSchema = AvroSchemaUtil.convert(schema.asStruct()); + GenericData.Record record = new GenericData.Record(avroSchema); + record.put("intCol", 42); + records.add(record); + + write(file, schema, Collections.emptyMap(), null, records.toArray(new GenericData.Record[] {})); + + // Test with a class that doesn't implement VectorizedParquetReaderFactory + Parquet.ReadBuilder readBuilder = + Parquet.read(Files.localInput(file)) + .project(schema) + .createBatchedReaderFunc(fileSchema -> new MockVectorizedReader()) + .vectorizedReaderFactory(InvalidReaderFactory.class.getName()); + + // Should not throw - falls back to default reader + readBuilder.build().iterator(); + } + + @Test + public void testVectorizedReaderFactoryNoDefaultConstructor() throws IOException { + Schema schema = new Schema(optional(1, "intCol", IntegerType.get())); + File file = createTempFile(temp); + + // Write test data + List records = Lists.newArrayList(); + org.apache.avro.Schema avroSchema = AvroSchemaUtil.convert(schema.asStruct()); + GenericData.Record record = new GenericData.Record(avroSchema); + record.put("intCol", 42); + records.add(record); + + write(file, schema, Collections.emptyMap(), null, records.toArray(new GenericData.Record[] {})); + + // Test with a class that has no default constructor + Parquet.ReadBuilder readBuilder = + Parquet.read(Files.localInput(file)) + .project(schema) + .createBatchedReaderFunc(fileSchema -> new MockVectorizedReader()) + .vectorizedReaderFactory(NoDefaultConstructorFactory.class.getName()); + + // Should not throw - falls back to default reader + readBuilder.build().iterator(); + } + + @Test + public void testVectorizedReaderFactorySuccessfulLoad() throws IOException { + Schema schema = new Schema(optional(1, "intCol", IntegerType.get())); + File file = createTempFile(temp); + + // Write test data + List records = Lists.newArrayList(); + org.apache.avro.Schema avroSchema = AvroSchemaUtil.convert(schema.asStruct()); + GenericData.Record record = new GenericData.Record(avroSchema); + record.put("intCol", 42); + records.add(record); + + write(file, schema, Collections.emptyMap(), null, records.toArray(new GenericData.Record[] {})); + + // Reset the flag + TestMockVectorizedReaderFactory.wasCalled = false; + + // Test successful factory loading + Parquet.ReadBuilder readBuilder = + Parquet.read(Files.localInput(file)) + .project(schema) + .createBatchedReaderFunc(fileSchema -> new MockVectorizedReader()) + .vectorizedReaderFactory(TestMockVectorizedReaderFactory.class.getName()); + + // Build and consume the reader + Iterator iterator = readBuilder.build().iterator(); + assertThat(iterator.hasNext()).isTrue(); + iterator.next(); + + // Verify our mock factory was actually used + assertThat(TestMockVectorizedReaderFactory.wasCalled) + .as("Mock factory should have been called") + .isTrue(); + } + private Pair generateFile( Function> createWriterFunc, int desiredRecordCount, @@ -354,4 +525,74 @@ private Pair generateFile( records.toArray(new GenericData.Record[] {})); return Pair.of(file, size); } + + // Test helper classes + + /** A mock VectorizedReader for testing. */ + public static class MockVectorizedReader implements VectorizedReader { + @Override + public Object read(Object reuse, int numRows) { + return null; + } + + @Override + public void setBatchSize(int batchSize) { + // No-op + } + + @Override + public void close() { + // No-op + } + } + + /** A mock factory class that implements VectorizedParquetReaderFactory for testing. */ + public static class TestMockVectorizedReaderFactory implements VectorizedParquetReaderFactory { + static boolean wasCalled = false; + + @Override + public String name() { + return "test-mock"; + } + + @Override + @SuppressWarnings("unchecked") + public org.apache.iceberg.io.CloseableIterable createReader(ReaderParams params) { + wasCalled = true; + // Return a simple iterable that provides the mock data + GenericData.Record record = + new GenericData.Record(AvroSchemaUtil.convert(params.schema().asStruct(), "table")); + record.put(0, 42); + return (org.apache.iceberg.io.CloseableIterable) + org.apache.iceberg.io.CloseableIterable.withNoopClose(Collections.singletonList(record)); + } + } + + /** A mock factory class without implementing the interface. */ + public static class InvalidReaderFactory { + public InvalidReaderFactory() {} + + public String name() { + return "invalid"; + } + } + + /** A mock factory class with no default constructor. */ + public static class NoDefaultConstructorFactory implements VectorizedParquetReaderFactory { + @SuppressWarnings("unused") + public NoDefaultConstructorFactory(String unusedParam) {} + + @Override + public String name() { + return "no-default"; + } + + @Override + public org.apache.iceberg.io.CloseableIterable createReader(ReaderParams params) { + return null; + } + } + + /** A simple reference class that can be loaded. */ + public static class MockVectorizedReaderFactory {} } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatch.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatch.java index dce54167e224..764bbc47d923 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatch.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatch.java @@ -139,12 +139,6 @@ public PartitionReaderFactory createReaderFactory() { private ParquetBatchReadConf parquetBatchReadConf(ParquetReaderType readerType) { String factoryClassName = readConf.parquetVectorizedReaderFactory(); - // If no explicit factory is set and reader type is COMET, use the default Comet factory - if (factoryClassName == null && readerType == ParquetReaderType.COMET) { - factoryClassName = - org.apache.iceberg.spark.SparkSQLProperties.COMET_VECTORIZED_READER_FACTORY_CLASS; - } - ImmutableParquetBatchReadConf.Builder builder = ImmutableParquetBatchReadConf.builder() .batchSize(readConf.parquetBatchSize()) diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatch.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatch.java index dce54167e224..764bbc47d923 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatch.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatch.java @@ -139,12 +139,6 @@ public PartitionReaderFactory createReaderFactory() { private ParquetBatchReadConf parquetBatchReadConf(ParquetReaderType readerType) { String factoryClassName = readConf.parquetVectorizedReaderFactory(); - // If no explicit factory is set and reader type is COMET, use the default Comet factory - if (factoryClassName == null && readerType == ParquetReaderType.COMET) { - factoryClassName = - org.apache.iceberg.spark.SparkSQLProperties.COMET_VECTORIZED_READER_FACTORY_CLASS; - } - ImmutableParquetBatchReadConf.Builder builder = ImmutableParquetBatchReadConf.builder() .batchSize(readConf.parquetBatchSize()) diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java index 826d5b0ee095..6e8331c0bcac 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java @@ -380,6 +380,7 @@ public String parquetVectorizedReaderFactory() { return confParser .stringConf() .sessionConf(SparkSQLProperties.PARQUET_VECTORIZED_READER_FACTORY) + .defaultValue(SparkSQLProperties.COMET_VECTORIZED_READER_FACTORY_CLASS) .parseOptional(); } } diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatch.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatch.java index dce54167e224..764bbc47d923 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatch.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatch.java @@ -139,12 +139,6 @@ public PartitionReaderFactory createReaderFactory() { private ParquetBatchReadConf parquetBatchReadConf(ParquetReaderType readerType) { String factoryClassName = readConf.parquetVectorizedReaderFactory(); - // If no explicit factory is set and reader type is COMET, use the default Comet factory - if (factoryClassName == null && readerType == ParquetReaderType.COMET) { - factoryClassName = - org.apache.iceberg.spark.SparkSQLProperties.COMET_VECTORIZED_READER_FACTORY_CLASS; - } - ImmutableParquetBatchReadConf.Builder builder = ImmutableParquetBatchReadConf.builder() .batchSize(readConf.parquetBatchSize()) From 4e29904e6742ea84236f9806f2fc046dc49dfc35 Mon Sep 17 00:00:00 2001 From: Parth Chandra Date: Fri, 5 Dec 2025 13:02:09 -0800 Subject: [PATCH 15/15] spotless --- .../src/test/java/org/apache/iceberg/parquet/TestParquet.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/parquet/src/test/java/org/apache/iceberg/parquet/TestParquet.java b/parquet/src/test/java/org/apache/iceberg/parquet/TestParquet.java index 531c89c8c678..d31e844ae993 100644 --- a/parquet/src/test/java/org/apache/iceberg/parquet/TestParquet.java +++ b/parquet/src/test/java/org/apache/iceberg/parquet/TestParquet.java @@ -343,7 +343,8 @@ public void testVectorizedReaderFactoryConfiguration() throws IOException { // but we can verify the build succeeds readBuilder.build().iterator(); // Should not throw - // Verify our mock factory was NOT used (because MockVectorizedReaderFactory is not a valid factory) + // Verify our mock factory was NOT used (because MockVectorizedReaderFactory is not a valid + // factory) assertThat(TestMockVectorizedReaderFactory.wasCalled) .as("TestMockVectorizedReaderFactory should not have been called") .isFalse();