From 0c24239d50f6db7295814923ce3e16595ef922b6 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Wed, 18 Mar 2026 09:15:52 -0600 Subject: [PATCH 1/3] Spark: Remove Apache DataFusion Comet integration Removes the datafusion-comet dependency and all related code across Spark versions 3.4, 3.5, 4.0, and 4.1. This includes the Comet-specific vectorized reader classes, ParquetReaderType enum, the spark.sql.iceberg.parquet.reader-type configuration property, and associated build/test infrastructure. --- .../checkstyle/checkstyle-suppressions.xml | 5 - docs/docs/spark-configuration.md | 1 - gradle/libs.versions.toml | 1 - spark/v3.4/build.gradle | 3 - .../iceberg/spark/ParquetBatchReadConf.java | 2 - .../iceberg/spark/ParquetReaderType.java | 47 ------ .../apache/iceberg/spark/SparkReadConf.java | 8 - .../iceberg/spark/SparkSQLProperties.java | 4 - .../data/vectorized/CometColumnReader.java | 140 ---------------- .../vectorized/CometColumnarBatchReader.java | 137 ---------------- .../vectorized/CometConstantColumnReader.java | 65 -------- .../vectorized/CometDeleteColumnReader.java | 74 --------- .../vectorized/CometDeletedColumnVector.java | 155 ------------------ .../vectorized/CometPositionColumnReader.java | 62 ------- .../CometVectorizedReaderBuilder.java | 138 ---------------- .../VectorizedSparkParquetReaders.java | 21 --- .../iceberg/spark/source/BaseBatchReader.java | 10 +- .../iceberg/spark/source/SparkBatch.java | 26 +-- .../spark/source/SparkFormatModels.java | 8 - .../TestParquetCometVectorizedScan.java | 33 ---- spark/v3.5/build.gradle | 3 - .../iceberg/spark/ParquetBatchReadConf.java | 2 - .../iceberg/spark/ParquetReaderType.java | 47 ------ .../apache/iceberg/spark/SparkReadConf.java | 8 - .../iceberg/spark/SparkSQLProperties.java | 3 - .../data/vectorized/CometColumnReader.java | 140 ---------------- .../vectorized/CometColumnarBatchReader.java | 137 ---------------- .../vectorized/CometConstantColumnReader.java | 65 -------- .../vectorized/CometDeleteColumnReader.java | 74 --------- .../vectorized/CometDeletedColumnVector.java | 155 ------------------ .../vectorized/CometPositionColumnReader.java | 62 ------- .../CometVectorizedReaderBuilder.java | 138 ---------------- .../VectorizedSparkParquetReaders.java | 21 --- .../iceberg/spark/source/BaseBatchReader.java | 10 +- .../iceberg/spark/source/SparkBatch.java | 26 +-- .../spark/source/SparkFormatModels.java | 8 - .../TestParquetCometVectorizedScan.java | 33 ---- spark/v4.0/build.gradle | 3 - .../iceberg/spark/ParquetBatchReadConf.java | 2 - .../iceberg/spark/ParquetReaderType.java | 47 ------ .../apache/iceberg/spark/SparkReadConf.java | 8 - .../iceberg/spark/SparkSQLProperties.java | 3 - .../data/vectorized/CometColumnReader.java | 140 ---------------- .../vectorized/CometColumnarBatchReader.java | 137 ---------------- .../vectorized/CometConstantColumnReader.java | 65 -------- .../vectorized/CometDeleteColumnReader.java | 74 --------- .../vectorized/CometDeletedColumnVector.java | 155 ------------------ .../vectorized/CometPositionColumnReader.java | 62 ------- .../CometVectorizedReaderBuilder.java | 138 ---------------- .../VectorizedSparkParquetReaders.java | 21 --- .../iceberg/spark/source/BaseBatchReader.java | 10 +- .../iceberg/spark/source/SparkBatch.java | 26 +-- .../spark/source/SparkFormatModels.java | 8 - .../TestParquetCometVectorizedScan.java | 33 ---- spark/v4.1/build.gradle | 5 - .../iceberg/spark/ParquetBatchReadConf.java | 2 - .../iceberg/spark/ParquetReaderType.java | 47 ------ .../apache/iceberg/spark/SparkReadConf.java | 8 - .../iceberg/spark/SparkSQLProperties.java | 3 - .../data/vectorized/CometColumnReader.java | 140 ---------------- .../vectorized/CometColumnarBatchReader.java | 137 ---------------- .../vectorized/CometConstantColumnReader.java | 65 -------- .../vectorized/CometDeleteColumnReader.java | 74 --------- .../vectorized/CometDeletedColumnVector.java | 155 ------------------ .../vectorized/CometPositionColumnReader.java | 62 ------- .../CometVectorizedReaderBuilder.java | 138 ---------------- .../VectorizedSparkParquetReaders.java | 22 --- .../iceberg/spark/source/BaseBatchReader.java | 10 +- .../iceberg/spark/source/SparkBatch.java | 30 +--- .../spark/source/SparkFormatModels.java | 8 - .../TestParquetCometVectorizedScan.java | 33 ---- 71 files changed, 17 insertions(+), 3726 deletions(-) delete mode 100644 spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/ParquetReaderType.java delete mode 100644 spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnReader.java delete mode 100644 spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnarBatchReader.java delete mode 100644 spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometConstantColumnReader.java delete mode 100644 spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeleteColumnReader.java delete mode 100644 spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeletedColumnVector.java delete mode 100644 spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometPositionColumnReader.java delete mode 100644 spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometVectorizedReaderBuilder.java delete mode 100644 spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetCometVectorizedScan.java delete mode 100644 spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/ParquetReaderType.java delete mode 100644 spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnReader.java delete mode 100644 spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnarBatchReader.java delete mode 100644 spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometConstantColumnReader.java delete mode 100644 spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeleteColumnReader.java delete mode 100644 spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeletedColumnVector.java delete mode 100644 spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometPositionColumnReader.java delete mode 100644 spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometVectorizedReaderBuilder.java delete mode 100644 spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetCometVectorizedScan.java delete mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/ParquetReaderType.java delete mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnReader.java delete mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnarBatchReader.java delete mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometConstantColumnReader.java delete mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeleteColumnReader.java delete mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeletedColumnVector.java delete mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometPositionColumnReader.java delete mode 100644 spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometVectorizedReaderBuilder.java delete mode 100644 spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetCometVectorizedScan.java delete mode 100644 spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/ParquetReaderType.java delete mode 100644 spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnReader.java delete mode 100644 spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnarBatchReader.java delete mode 100644 spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometConstantColumnReader.java delete mode 100644 spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeleteColumnReader.java delete mode 100644 spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeletedColumnVector.java delete mode 100644 spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometPositionColumnReader.java delete mode 100644 spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometVectorizedReaderBuilder.java delete mode 100644 spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetCometVectorizedScan.java diff --git a/.baseline/checkstyle/checkstyle-suppressions.xml b/.baseline/checkstyle/checkstyle-suppressions.xml index 1e79b1a7aa4b..c3a07bf6453f 100644 --- a/.baseline/checkstyle/checkstyle-suppressions.xml +++ b/.baseline/checkstyle/checkstyle-suppressions.xml @@ -53,11 +53,6 @@ - - - - - diff --git a/docs/docs/spark-configuration.md b/docs/docs/spark-configuration.md index 8193d1dc352b..01bb773680fd 100644 --- a/docs/docs/spark-configuration.md +++ b/docs/docs/spark-configuration.md @@ -174,7 +174,6 @@ val spark = SparkSession.builder() | Spark option | Default | Description | |--------------------------------------------------------|----------------------------------------------------------------|---------------------------------------------------------------------------------------------------------------------------------| | spark.sql.iceberg.vectorization.enabled | Table default | Enables vectorized reads of data files | -| spark.sql.iceberg.parquet.reader-type | ICEBERG | Sets Parquet reader implementation (`ICEBERG`,`COMET`) | | spark.sql.iceberg.check-nullability | true | Validate that the write schema's nullability matches the table's nullability | | spark.sql.iceberg.check-ordering | true | Validates the write schema column order matches the table schema order | | spark.sql.iceberg.planning.preserve-data-grouping | false | When true, co-locate scan tasks for the same partition in the same read split, used in Storage Partitioned Joins | diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index abe2c3f543eb..b16cd5349030 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -39,7 +39,6 @@ awssdk-s3accessgrants = "2.4.1" bson-ver = "4.11.5" caffeine = "2.9.3" calcite = "1.41.0" -comet = "0.12.0" datasketches = "6.2.0" delta-standalone = "3.3.2" delta-spark = "3.3.2" diff --git a/spark/v3.4/build.gradle b/spark/v3.4/build.gradle index 714ee703b842..bfe84b08dfa4 100644 --- a/spark/v3.4/build.gradle +++ b/spark/v3.4/build.gradle @@ -75,8 +75,6 @@ project(":iceberg-spark:iceberg-spark-${sparkMajorVersion}_${scalaVersion}") { exclude group: 'org.roaringbitmap' } - compileOnly "org.apache.datafusion:comet-spark-spark${sparkMajorVersion}_${scalaVersion}:${libs.versions.comet.get()}" - implementation libs.parquet.column implementation libs.parquet.hadoop @@ -184,7 +182,6 @@ project(":iceberg-spark:iceberg-spark-extensions-${sparkMajorVersion}_${scalaVer testImplementation libs.avro.avro testImplementation libs.parquet.hadoop testImplementation libs.awaitility - testImplementation "org.apache.datafusion:comet-spark-spark${sparkMajorVersion}_${scalaVersion}:${libs.versions.comet.get()}" // Required because we remove antlr plugin dependencies from the compile configuration, see note above runtimeOnly libs.antlr.runtime 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..b7cd441c9642 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 @@ -24,6 +24,4 @@ @Value.Immutable public interface ParquetBatchReadConf extends Serializable { int batchSize(); - - ParquetReaderType readerType(); } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/ParquetReaderType.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/ParquetReaderType.java deleted file mode 100644 index d9742c048251..000000000000 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/ParquetReaderType.java +++ /dev/null @@ -1,47 +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. - */ -package org.apache.iceberg.spark; - -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; - -/** Enumerates the types of Parquet readers. */ -public enum ParquetReaderType { - /** ICEBERG type utilizes the built-in Parquet reader. */ - ICEBERG, - - /** - * COMET type changes the Parquet reader to the Apache DataFusion Comet Parquet reader. Comet - * Parquet reader performs I/O and decompression in the JVM but decodes in native to improve - * performance. Additionally, Comet will convert Spark's physical plan into a native physical plan - * and execute this plan natively. - * - *

TODO: Implement {@link org.apache.comet.parquet.SupportsComet} in SparkScan to convert Spark - * physical plan to native physical plan for native execution. - */ - COMET; - - public static ParquetReaderType fromString(String typeAsString) { - Preconditions.checkArgument(typeAsString != null, "Parquet reader type is null"); - try { - return ParquetReaderType.valueOf(typeAsString.toUpperCase()); - } catch (IllegalArgumentException e) { - throw new IllegalArgumentException("Unknown parquet reader type: " + typeAsString); - } - } -} 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 ff2b10ef2143..3c8556731f74 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 @@ -374,12 +374,4 @@ public boolean reportColumnStats() { .defaultValue(SparkSQLProperties.REPORT_COLUMN_STATS_DEFAULT) .parse(); } - - public ParquetReaderType parquetReaderType() { - return confParser - .enumConf(ParquetReaderType::fromString) - .sessionConf(SparkSQLProperties.PARQUET_READER_TYPE) - .defaultValue(SparkSQLProperties.PARQUET_READER_TYPE_DEFAULT) - .parse(); - } } 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..41dc73add3b4 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 @@ -27,10 +27,6 @@ private SparkSQLProperties() {} // Controls whether vectorized reads are enabled public static final String VECTORIZATION_ENABLED = "spark.sql.iceberg.vectorization.enabled"; - // 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 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/data/vectorized/CometColumnReader.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnReader.java deleted file mode 100644 index 81b7d83a7077..000000000000 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnReader.java +++ /dev/null @@ -1,140 +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. - */ -package org.apache.iceberg.spark.data.vectorized; - -import java.io.IOException; -import org.apache.comet.CometSchemaImporter; -import org.apache.comet.parquet.AbstractColumnReader; -import org.apache.comet.parquet.ColumnReader; -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.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.types.DataType; -import org.apache.spark.sql.types.Metadata; -import org.apache.spark.sql.types.StructField; -import org.apache.spark.sql.vectorized.ColumnVector; - -class CometColumnReader implements VectorizedReader { - // use the Comet default batch size - public static final int DEFAULT_BATCH_SIZE = 8192; - - private final ColumnDescriptor descriptor; - private final DataType sparkType; - - // The delegated ColumnReader from Comet side - private AbstractColumnReader delegate; - private boolean initialized = false; - private int batchSize = DEFAULT_BATCH_SIZE; - private CometSchemaImporter importer; - - CometColumnReader(DataType sparkType, ColumnDescriptor descriptor) { - this.sparkType = sparkType; - this.descriptor = descriptor; - } - - 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); - } - - public AbstractColumnReader delegate() { - return delegate; - } - - void setDelegate(AbstractColumnReader delegate) { - this.delegate = delegate; - } - - void setInitialized(boolean initialized) { - this.initialized = initialized; - } - - public int batchSize() { - return batchSize; - } - - /** - * This method is to initialized/reset the CometColumnReader. This needs to be called for each row - * group after readNextRowGroup, so a new dictionary encoding can be set for each of the new row - * groups. - */ - public void reset() { - if (importer != null) { - importer.close(); - } - - if (delegate != null) { - delegate.close(); - } - - this.importer = new CometSchemaImporter(new RootAllocator()); - this.delegate = Utils.getColumnReader(sparkType, descriptor, importer, batchSize, false, false); - this.initialized = true; - } - - public ColumnDescriptor descriptor() { - return descriptor; - } - - /** Returns the Spark data type for this column. */ - public DataType sparkType() { - return sparkType; - } - - /** - * Set the page reader to be 'pageReader'. - * - *

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 { - Preconditions.checkState(initialized, "Invalid state: 'reset' should be called first"); - ((ColumnReader) delegate).setPageReader(pageReader); - } - - @Override - public void close() { - // close resources on native side - if (importer != null) { - importer.close(); - } - - if (delegate != null) { - delegate.close(); - } - } - - @Override - public void setBatchSize(int size) { - this.batchSize = size; - } - - @Override - public ColumnVector read(ColumnVector reuse, int numRowsToRead) { - throw new UnsupportedOperationException("Not supported"); - } -} 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 deleted file mode 100644 index 3d3e9aca24de..000000000000 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnarBatchReader.java +++ /dev/null @@ -1,137 +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. - */ -package org.apache.iceberg.spark.data.vectorized; - -import java.io.IOException; -import java.io.UncheckedIOException; -import java.util.List; -import java.util.Map; -import org.apache.comet.parquet.AbstractColumnReader; -import org.apache.comet.parquet.BatchReader; -import org.apache.iceberg.Schema; -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.parquet.column.page.PageReadStore; -import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; -import org.apache.parquet.hadoop.metadata.ColumnPath; -import org.apache.spark.sql.vectorized.ColumnVector; -import org.apache.spark.sql.vectorized.ColumnarBatch; - -/** - * {@link VectorizedReader} that returns Spark's {@link ColumnarBatch} to support Spark's vectorized - * read path. The {@link ColumnarBatch} returned is created by passing in the Arrow vectors - * populated via delegated read calls to {@link CometColumnReader VectorReader(s)}. - */ -@SuppressWarnings("checkstyle:VisibilityModifier") -class CometColumnarBatchReader implements VectorizedReader { - - private final CometColumnReader[] readers; - - // The delegated BatchReader on the Comet side does the real work of loading a batch of rows. - // The Comet BatchReader contains an array of ColumnReader. There is no need to explicitly call - // ColumnReader.readBatch; instead, BatchReader.nextBatch will be called, which underneath calls - // ColumnReader.readBatch. The only exception is DeleteColumnReader, because at the time of - // 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; - - CometColumnarBatchReader(List> readers, Schema schema) { - this.readers = - readers.stream().map(CometColumnReader.class::cast).toArray(CometColumnReader[]::new); - - AbstractColumnReader[] abstractColumnReaders = new AbstractColumnReader[readers.size()]; - this.delegate = new BatchReader(abstractColumnReaders); - delegate.setSparkSchema(SparkSchemaUtil.convert(schema)); - } - - @Override - public void setRowGroupInfo( - PageReadStore pageStore, Map metaData) { - for (int i = 0; i < readers.length; i++) { - try { - if (!(readers[i] instanceof CometConstantColumnReader) - && !(readers[i] instanceof CometPositionColumnReader) - && !(readers[i] instanceof CometDeleteColumnReader)) { - readers[i].reset(); - readers[i].setPageReader(pageStore.getPageReader(readers[i].descriptor())); - } - } catch (IOException e) { - throw new UncheckedIOException("Failed to setRowGroupInfo for Comet vectorization", e); - } - } - - for (int i = 0; i < readers.length; i++) { - delegate.getColumnReaders()[i] = this.readers[i].delegate(); - } - } - - @Override - public final ColumnarBatch read(ColumnarBatch reuse, int numRowsToRead) { - return new ColumnBatchLoader(numRowsToRead).loadDataToColumnBatch(); - } - - @Override - public void setBatchSize(int batchSize) { - for (CometColumnReader reader : readers) { - if (reader != null) { - reader.setBatchSize(batchSize); - } - } - } - - @Override - public void close() { - for (CometColumnReader reader : readers) { - if (reader != null) { - reader.close(); - } - } - } - - private class ColumnBatchLoader { - private final int batchSize; - - ColumnBatchLoader(int numRowsToRead) { - Preconditions.checkArgument( - numRowsToRead > 0, "Invalid number of rows to read: %s", numRowsToRead); - this.batchSize = numRowsToRead; - } - - ColumnarBatch loadDataToColumnBatch() { - ColumnVector[] vectors = readDataToColumnVectors(); - - ColumnarBatch batch = new ColumnarBatch(vectors); - batch.setNumRows(batchSize); - return batch; - } - - ColumnVector[] readDataToColumnVectors() { - ColumnVector[] columnVectors = new ColumnVector[readers.length]; - // Fetch rows for all readers in the delegate - delegate.nextBatch(batchSize); - for (int i = 0; i < readers.length; i++) { - columnVectors[i] = readers[i].delegate().currentBatch(); - } - - return columnVectors; - } - } -} 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 deleted file mode 100644 index c665002e8f66..000000000000 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometConstantColumnReader.java +++ /dev/null @@ -1,65 +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. - */ -package org.apache.iceberg.spark.data.vectorized; - -import java.math.BigDecimal; -import java.nio.ByteBuffer; -import org.apache.comet.parquet.ConstantColumnReader; -import org.apache.iceberg.types.Types; -import org.apache.spark.sql.types.DataType; -import org.apache.spark.sql.types.DataTypes; -import org.apache.spark.sql.types.Decimal; -import org.apache.spark.sql.types.DecimalType; -import org.apache.spark.unsafe.types.UTF8String; - -class CometConstantColumnReader extends CometColumnReader { - - CometConstantColumnReader(T value, Types.NestedField field) { - 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)); - } - - @Override - public void setBatchSize(int batchSize) { - super.setBatchSize(batchSize); - delegate().setBatchSize(batchSize); - setInitialized(true); - } - - private Object convertToSparkValue(T value) { - DataType dataType = sparkType(); - // Match the value to Spark internal type if necessary - if (dataType == DataTypes.StringType && value instanceof String) { - // the internal type for StringType is UTF8String - return UTF8String.fromString((String) value); - } else if (dataType instanceof DecimalType && value instanceof BigDecimal) { - // the internal type for DecimalType is Decimal - return Decimal.apply((BigDecimal) value); - } else if (dataType == DataTypes.BinaryType && value instanceof ByteBuffer) { - // the internal type for DecimalType is byte[] - // Iceberg default value should always use HeapBufferBuffer, so calling ByteBuffer.array() - // should be safe. - return ((java.nio.ByteBuffer) value).array(); - } else { - return value; - } - } -} 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 deleted file mode 100644 index 26219014f777..000000000000 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeleteColumnReader.java +++ /dev/null @@ -1,74 +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. - */ -package org.apache.iceberg.spark.data.vectorized; - -import org.apache.comet.parquet.MetadataColumnReader; -import org.apache.comet.parquet.Native; -import org.apache.comet.parquet.TypeUtil; -import org.apache.comet.vector.CometVector; -import org.apache.iceberg.types.Types; -import org.apache.spark.sql.types.DataTypes; -import org.apache.spark.sql.types.Metadata; -import org.apache.spark.sql.types.StructField; - -class CometDeleteColumnReader extends CometColumnReader { - CometDeleteColumnReader(Types.NestedField field) { - super(field); - setDelegate(new DeleteColumnReader()); - } - - @Override - public void setBatchSize(int batchSize) { - super.setBatchSize(batchSize); - delegate().setBatchSize(batchSize); - setInitialized(true); - } - - private static class DeleteColumnReader extends MetadataColumnReader { - private final CometDeletedColumnVector deletedVector; - - DeleteColumnReader() { - this(new boolean[0]); - } - - DeleteColumnReader(boolean[] isDeleted) { - super( - DataTypes.BooleanType, - TypeUtil.convertToParquet( - new StructField("_deleted", DataTypes.BooleanType, false, Metadata.empty())), - false /* useDecimal128 = false */, - false /* isConstant = false */); - this.deletedVector = new CometDeletedColumnVector(isDeleted); - } - - @Override - public void readBatch(int total) { - Native.resetBatch(nativeHandle); - // set isDeleted on the native side to be consumed by native execution - Native.setIsDeleted(nativeHandle, deletedVector.isDeleted()); - - super.readBatch(total); - } - - @Override - public CometVector currentBatch() { - return deletedVector; - } - } -} diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeletedColumnVector.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeletedColumnVector.java deleted file mode 100644 index 5817f2c20a4f..000000000000 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeletedColumnVector.java +++ /dev/null @@ -1,155 +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. - */ -package org.apache.iceberg.spark.data.vectorized; - -import org.apache.comet.shaded.arrow.vector.ValueVector; -import org.apache.comet.vector.CometVector; -import org.apache.iceberg.spark.SparkSchemaUtil; -import org.apache.iceberg.types.Types; -import org.apache.spark.sql.types.Decimal; -import org.apache.spark.sql.vectorized.ColumnVector; -import org.apache.spark.sql.vectorized.ColumnarArray; -import org.apache.spark.sql.vectorized.ColumnarMap; -import org.apache.spark.unsafe.types.UTF8String; - -public class CometDeletedColumnVector extends CometVector implements UpdatableDeletedColumnVector { - private boolean[] isDeleted; - - public CometDeletedColumnVector(boolean[] isDeleted) { - super(SparkSchemaUtil.convert(Types.BooleanType.get()), false); - this.isDeleted = isDeleted; - } - - @Override - public void setValue(boolean[] deleted) { - this.isDeleted = deleted; - } - - boolean[] isDeleted() { - return isDeleted; - } - - @Override - public void setNumNulls(int numNulls) { - throw new UnsupportedOperationException("Not implemented"); - } - - @Override - public void setNumValues(int numValues) { - throw new UnsupportedOperationException("Not implemented"); - } - - @Override - public int numValues() { - throw new UnsupportedOperationException("Not implemented"); - } - - @Override - public ValueVector getValueVector() { - throw new UnsupportedOperationException("Not implemented"); - } - - @Override - public CometVector slice(int offset, int length) { - throw new UnsupportedOperationException("Not implemented"); - } - - @Override - public void close() {} - - @Override - public boolean hasNull() { - return false; - } - - @Override - public int numNulls() { - return 0; - } - - @Override - public boolean isNullAt(int rowId) { - return false; - } - - @Override - public boolean getBoolean(int rowId) { - return isDeleted[rowId]; - } - - @Override - public byte getByte(int rowId) { - throw new UnsupportedOperationException(); - } - - @Override - public short getShort(int rowId) { - throw new UnsupportedOperationException(); - } - - @Override - public int getInt(int rowId) { - throw new UnsupportedOperationException(); - } - - @Override - public long getLong(int rowId) { - throw new UnsupportedOperationException(); - } - - @Override - public float getFloat(int rowId) { - throw new UnsupportedOperationException(); - } - - @Override - public double getDouble(int rowId) { - throw new UnsupportedOperationException(); - } - - @Override - public ColumnarArray getArray(int rowId) { - throw new UnsupportedOperationException(); - } - - @Override - public ColumnarMap getMap(int ordinal) { - throw new UnsupportedOperationException(); - } - - @Override - public Decimal getDecimal(int rowId, int precision, int scale) { - throw new UnsupportedOperationException(); - } - - @Override - public UTF8String getUTF8String(int rowId) { - throw new UnsupportedOperationException(); - } - - @Override - public byte[] getBinary(int rowId) { - throw new UnsupportedOperationException(); - } - - @Override - public ColumnVector getChild(int ordinal) { - throw new UnsupportedOperationException(); - } -} 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 deleted file mode 100644 index 1949a717982a..000000000000 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometPositionColumnReader.java +++ /dev/null @@ -1,62 +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. - */ -package org.apache.iceberg.spark.data.vectorized; - -import org.apache.comet.parquet.MetadataColumnReader; -import org.apache.comet.parquet.Native; -import org.apache.iceberg.types.Types; -import org.apache.parquet.column.ColumnDescriptor; -import org.apache.spark.sql.types.DataTypes; - -class CometPositionColumnReader extends CometColumnReader { - CometPositionColumnReader(Types.NestedField field) { - super(field); - setDelegate(new PositionColumnReader(descriptor())); - } - - @Override - public void setBatchSize(int batchSize) { - super.setBatchSize(batchSize); - delegate().setBatchSize(batchSize); - setInitialized(true); - } - - private static class PositionColumnReader extends MetadataColumnReader { - /** The current position value of the column that are used to initialize this column reader. */ - private long position; - - PositionColumnReader(ColumnDescriptor descriptor) { - super( - DataTypes.LongType, - descriptor, - false /* useDecimal128 = false */, - false /* isConstant */); - } - - @Override - public void readBatch(int total) { - Native.resetBatch(nativeHandle); - // set position on the native side to be consumed by native execution - Native.setPosition(nativeHandle, position, total); - position += total; - - super.readBatch(total); - } - } -} 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 deleted file mode 100644 index 779dc240d4f6..000000000000 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometVectorizedReaderBuilder.java +++ /dev/null @@ -1,138 +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. - */ -package org.apache.iceberg.spark.data.vectorized; - -import java.util.List; -import java.util.Map; -import java.util.function.Function; -import java.util.stream.IntStream; -import org.apache.iceberg.MetadataColumns; -import org.apache.iceberg.Schema; -import org.apache.iceberg.parquet.TypeWithSchemaVisitor; -import org.apache.iceberg.parquet.VectorizedReader; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -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.GroupType; -import org.apache.parquet.schema.MessageType; -import org.apache.parquet.schema.PrimitiveType; -import org.apache.parquet.schema.Type; - -class CometVectorizedReaderBuilder extends TypeWithSchemaVisitor> { - - private final MessageType parquetSchema; - private final Schema icebergSchema; - private final Map idToConstant; - private final Function>, VectorizedReader> readerFactory; - - CometVectorizedReaderBuilder( - Schema expectedSchema, - MessageType parquetSchema, - Map idToConstant, - Function>, VectorizedReader> readerFactory) { - this.parquetSchema = parquetSchema; - this.icebergSchema = expectedSchema; - this.idToConstant = idToConstant; - this.readerFactory = readerFactory; - } - - @Override - public VectorizedReader message( - Types.StructType expected, MessageType message, List> fieldReaders) { - GroupType groupType = message.asGroupType(); - Map> readersById = Maps.newHashMap(); - List fields = groupType.getFields(); - - IntStream.range(0, fields.size()) - .filter(pos -> fields.get(pos).getId() != null) - .forEach(pos -> readersById.put(fields.get(pos).getId().intValue(), fieldReaders.get(pos))); - - List icebergFields = - expected != null ? expected.fields() : ImmutableList.of(); - - List> reorderedFields = - Lists.newArrayListWithExpectedSize(icebergFields.size()); - - for (Types.NestedField field : icebergFields) { - int id = field.fieldId(); - VectorizedReader reader = readersById.get(id); - if (idToConstant.containsKey(id)) { - CometConstantColumnReader constantReader = - new CometConstantColumnReader<>(idToConstant.get(id), field); - reorderedFields.add(constantReader); - } else if (id == MetadataColumns.ROW_POSITION.fieldId()) { - reorderedFields.add(new CometPositionColumnReader(field)); - } else if (id == MetadataColumns.IS_DELETED.fieldId()) { - CometColumnReader deleteReader = new CometDeleteColumnReader<>(field); - reorderedFields.add(deleteReader); - } else if (reader != null) { - reorderedFields.add(reader); - } else if (field.initialDefault() != null) { - CometColumnReader constantReader = - new CometConstantColumnReader<>(field.initialDefault(), field); - reorderedFields.add(constantReader); - } else if (field.isOptional()) { - CometColumnReader constantReader = new CometConstantColumnReader<>(null, field); - reorderedFields.add(constantReader); - } else { - throw new IllegalArgumentException( - String.format("Missing required field: %s", field.name())); - } - } - return vectorizedReader(reorderedFields); - } - - protected VectorizedReader vectorizedReader(List> reorderedFields) { - return readerFactory.apply(reorderedFields); - } - - @Override - public VectorizedReader struct( - Types.StructType expected, GroupType groupType, List> fieldReaders) { - if (expected != null) { - throw new UnsupportedOperationException( - "Vectorized reads are not supported yet for struct fields"); - } - return null; - } - - @Override - public VectorizedReader primitive( - org.apache.iceberg.types.Type.PrimitiveType expected, PrimitiveType primitive) { - - if (primitive.getId() == null) { - return null; - } - int parquetFieldId = primitive.getId().intValue(); - ColumnDescriptor desc = parquetSchema.getColumnDescription(currentPath()); - // Nested types not yet supported for vectorized reads - if (desc.getMaxRepetitionLevel() > 0) { - return null; - } - Types.NestedField icebergField = icebergSchema.findField(parquetFieldId); - if (icebergField == null) { - return null; - } - - return new CometColumnReader(SparkSchemaUtil.convert(icebergField.type()), desc); - } -} diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkParquetReaders.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkParquetReaders.java index 55f9fc1768a3..e959bbc2d542 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkParquetReaders.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkParquetReaders.java @@ -30,7 +30,6 @@ import org.apache.iceberg.parquet.VectorizedReader; import org.apache.iceberg.spark.SparkUtil; import org.apache.parquet.schema.MessageType; -import org.apache.spark.sql.vectorized.ColumnVector; import org.apache.spark.sql.vectorized.ColumnarBatch; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -77,26 +76,6 @@ public static ColumnarBatchReader buildReader( return buildReader(expectedSchema, fileSchema, idToConstant, ArrowAllocation.rootAllocator()); } - public static VectorizedReader buildCometReader( - Schema expectedSchema, MessageType fileSchema, Map idToConstant) { - return (CometColumnarBatchReader) - TypeWithSchemaVisitor.visit( - expectedSchema.asStruct(), - fileSchema, - new CometVectorizedReaderBuilder( - expectedSchema, - fileSchema, - idToConstant, - readers -> new CometColumnarBatchReader(readers, expectedSchema))); - } - - /** A subclass of ColumnarBatch to identify Comet readers. */ - public static class CometColumnarBatch extends ColumnarBatch { - public CometColumnarBatch(ColumnVector[] columns) { - super(columns); - } - } - // enables unsafe memory access to avoid costly checks to see if index is within bounds // as long as it is not configured explicitly (see BoundsChecking in Arrow) private static void enableUnsafeMemoryAccess() { 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 89c03a4c2b72..fe062f9d7357 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 @@ -35,11 +35,9 @@ import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.spark.OrcBatchReadConf; import org.apache.iceberg.spark.ParquetBatchReadConf; -import org.apache.iceberg.spark.ParquetReaderType; import org.apache.iceberg.spark.data.vectorized.ColumnVectorWithFilter; import org.apache.iceberg.spark.data.vectorized.ColumnarBatchUtil; import org.apache.iceberg.spark.data.vectorized.UpdatableDeletedColumnVector; -import org.apache.iceberg.spark.data.vectorized.VectorizedSparkParquetReaders; import org.apache.iceberg.util.Pair; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.vectorized.ColumnVector; @@ -72,10 +70,8 @@ protected CloseableIterable newBatchIterable( Expression residual, Map idToConstant, @Nonnull SparkDeleteFilter deleteFilter) { - Class readType = - useComet() ? VectorizedSparkParquetReaders.CometColumnarBatch.class : ColumnarBatch.class; ReadBuilder readBuilder = - FormatModelRegistry.readBuilder(format, readType, inputFile); + FormatModelRegistry.readBuilder(format, ColumnarBatch.class, inputFile); if (parquetConf != null) { readBuilder = readBuilder.recordsPerBatch(parquetConf.batchSize()); @@ -101,10 +97,6 @@ protected CloseableIterable newBatchIterable( return CloseableIterable.transform(iterable, new BatchDeleteFilter(deleteFilter)::filterBatch); } - private boolean useComet() { - return parquetConf != null && parquetConf.readerType() == ParquetReaderType.COMET; - } - @VisibleForTesting static class BatchDeleteFilter { private final DeleteFilter deletes; 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..6bbf3ca3bab2 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 @@ -32,10 +32,8 @@ import org.apache.iceberg.spark.ImmutableParquetBatchReadConf; import org.apache.iceberg.spark.OrcBatchReadConf; import org.apache.iceberg.spark.ParquetBatchReadConf; -import org.apache.iceberg.spark.ParquetReaderType; import org.apache.iceberg.spark.SparkReadConf; import org.apache.iceberg.spark.SparkUtil; -import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.broadcast.Broadcast; @@ -122,11 +120,8 @@ private String[][] computePreferredLocations() { @Override public PartitionReaderFactory createReaderFactory() { - if (useCometBatchReads()) { - return new SparkColumnarReaderFactory(parquetBatchReadConf(ParquetReaderType.COMET)); - - } else if (useParquetBatchReads()) { - return new SparkColumnarReaderFactory(parquetBatchReadConf(ParquetReaderType.ICEBERG)); + if (useParquetBatchReads()) { + return new SparkColumnarReaderFactory(parquetBatchReadConf()); } else if (useOrcBatchReads()) { return new SparkColumnarReaderFactory(orcBatchReadConf()); @@ -136,10 +131,9 @@ public PartitionReaderFactory createReaderFactory() { } } - private ParquetBatchReadConf parquetBatchReadConf(ParquetReaderType readerType) { + private ParquetBatchReadConf parquetBatchReadConf() { return ImmutableParquetBatchReadConf.builder() .batchSize(readConf.parquetBatchSize()) - .readerType(readerType) .build(); } @@ -175,20 +169,6 @@ private boolean supportsParquetBatchReads(Types.NestedField field) { return field.type().isPrimitiveType() || MetadataColumns.isMetadataColumn(field.fieldId()); } - private boolean useCometBatchReads() { - return readConf.parquetVectorizationEnabled() - && readConf.parquetReaderType() == ParquetReaderType.COMET - && expectedSchema.columns().stream().allMatch(this::supportsCometBatchReads) - && taskGroups.stream().allMatch(this::supportsParquetBatchReads); - } - - private boolean supportsCometBatchReads(Types.NestedField field) { - return field.type().isPrimitiveType() - && !field.type().typeId().equals(Type.TypeID.UUID) - && field.fieldId() != MetadataColumns.ROW_ID.fieldId() - && field.fieldId() != MetadataColumns.LAST_UPDATED_SEQUENCE_NUMBER.fieldId(); - } - // conditions for using ORC batch reads: // - ORC vectorization is enabled // - all tasks are of type FileScanTask and read only ORC files with no delete files diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkFormatModels.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkFormatModels.java index 677f2e950b44..a1e8a82b4d80 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkFormatModels.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkFormatModels.java @@ -60,14 +60,6 @@ public static void register() { VectorizedSparkParquetReaders.buildReader( icebergSchema, fileSchema, idToConstant))); - FormatModelRegistry.register( - ParquetFormatModel.create( - VectorizedSparkParquetReaders.CometColumnarBatch.class, - StructType.class, - (icebergSchema, fileSchema, engineSchema, idToConstant) -> - VectorizedSparkParquetReaders.buildCometReader( - icebergSchema, fileSchema, idToConstant))); - FormatModelRegistry.register( ORCFormatModel.create( InternalRow.class, diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetCometVectorizedScan.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetCometVectorizedScan.java deleted file mode 100644 index 5e1d88f8bcd4..000000000000 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetCometVectorizedScan.java +++ /dev/null @@ -1,33 +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. - */ -package org.apache.iceberg.spark.source; - -import org.junit.jupiter.api.BeforeAll; - -public class TestParquetCometVectorizedScan extends TestParquetScan { - @BeforeAll - public static void setComet() { - ScanTestBase.spark.conf().set("spark.sql.iceberg.parquet.reader-type", "COMET"); - } - - @Override - protected boolean vectorized() { - return true; - } -} diff --git a/spark/v3.5/build.gradle b/spark/v3.5/build.gradle index 20a9cfb007bd..2fe3deb0ce5e 100644 --- a/spark/v3.5/build.gradle +++ b/spark/v3.5/build.gradle @@ -75,8 +75,6 @@ project(":iceberg-spark:iceberg-spark-${sparkMajorVersion}_${scalaVersion}") { exclude group: 'org.roaringbitmap' } - compileOnly "org.apache.datafusion:comet-spark-spark${sparkMajorVersion}_${scalaVersion}:${libs.versions.comet.get()}" - implementation libs.parquet.column implementation libs.parquet.hadoop @@ -185,7 +183,6 @@ project(":iceberg-spark:iceberg-spark-extensions-${sparkMajorVersion}_${scalaVer testImplementation libs.avro.avro testImplementation libs.parquet.hadoop testImplementation libs.awaitility - testImplementation "org.apache.datafusion:comet-spark-spark${sparkMajorVersion}_${scalaVersion}:${libs.versions.comet.get()}" testImplementation(testFixtures(project(':iceberg-parquet'))) // Required because we remove antlr plugin dependencies from the compile configuration, see note above 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..b7cd441c9642 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 @@ -24,6 +24,4 @@ @Value.Immutable public interface ParquetBatchReadConf extends Serializable { int batchSize(); - - ParquetReaderType readerType(); } diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/ParquetReaderType.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/ParquetReaderType.java deleted file mode 100644 index d9742c048251..000000000000 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/ParquetReaderType.java +++ /dev/null @@ -1,47 +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. - */ -package org.apache.iceberg.spark; - -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; - -/** Enumerates the types of Parquet readers. */ -public enum ParquetReaderType { - /** ICEBERG type utilizes the built-in Parquet reader. */ - ICEBERG, - - /** - * COMET type changes the Parquet reader to the Apache DataFusion Comet Parquet reader. Comet - * Parquet reader performs I/O and decompression in the JVM but decodes in native to improve - * performance. Additionally, Comet will convert Spark's physical plan into a native physical plan - * and execute this plan natively. - * - *

TODO: Implement {@link org.apache.comet.parquet.SupportsComet} in SparkScan to convert Spark - * physical plan to native physical plan for native execution. - */ - COMET; - - public static ParquetReaderType fromString(String typeAsString) { - Preconditions.checkArgument(typeAsString != null, "Parquet reader type is null"); - try { - return ParquetReaderType.valueOf(typeAsString.toUpperCase()); - } catch (IllegalArgumentException e) { - throw new IllegalArgumentException("Unknown parquet reader type: " + typeAsString); - } - } -} 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 e9d28ca8538c..b38c041507bb 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 @@ -370,12 +370,4 @@ public boolean reportColumnStats() { .defaultValue(SparkSQLProperties.REPORT_COLUMN_STATS_DEFAULT) .parse(); } - - public ParquetReaderType parquetReaderType() { - return confParser - .enumConf(ParquetReaderType::fromString) - .sessionConf(SparkSQLProperties.PARQUET_READER_TYPE) - .defaultValue(SparkSQLProperties.PARQUET_READER_TYPE_DEFAULT) - .parse(); - } } 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..e3ee288affbe 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 @@ -27,9 +27,6 @@ private SparkSQLProperties() {} // Controls whether vectorized reads are enabled public static final String VECTORIZATION_ENABLED = "spark.sql.iceberg.vectorization.enabled"; - // 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 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/data/vectorized/CometColumnReader.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnReader.java deleted file mode 100644 index 81b7d83a7077..000000000000 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnReader.java +++ /dev/null @@ -1,140 +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. - */ -package org.apache.iceberg.spark.data.vectorized; - -import java.io.IOException; -import org.apache.comet.CometSchemaImporter; -import org.apache.comet.parquet.AbstractColumnReader; -import org.apache.comet.parquet.ColumnReader; -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.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.types.DataType; -import org.apache.spark.sql.types.Metadata; -import org.apache.spark.sql.types.StructField; -import org.apache.spark.sql.vectorized.ColumnVector; - -class CometColumnReader implements VectorizedReader { - // use the Comet default batch size - public static final int DEFAULT_BATCH_SIZE = 8192; - - private final ColumnDescriptor descriptor; - private final DataType sparkType; - - // The delegated ColumnReader from Comet side - private AbstractColumnReader delegate; - private boolean initialized = false; - private int batchSize = DEFAULT_BATCH_SIZE; - private CometSchemaImporter importer; - - CometColumnReader(DataType sparkType, ColumnDescriptor descriptor) { - this.sparkType = sparkType; - this.descriptor = descriptor; - } - - 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); - } - - public AbstractColumnReader delegate() { - return delegate; - } - - void setDelegate(AbstractColumnReader delegate) { - this.delegate = delegate; - } - - void setInitialized(boolean initialized) { - this.initialized = initialized; - } - - public int batchSize() { - return batchSize; - } - - /** - * This method is to initialized/reset the CometColumnReader. This needs to be called for each row - * group after readNextRowGroup, so a new dictionary encoding can be set for each of the new row - * groups. - */ - public void reset() { - if (importer != null) { - importer.close(); - } - - if (delegate != null) { - delegate.close(); - } - - this.importer = new CometSchemaImporter(new RootAllocator()); - this.delegate = Utils.getColumnReader(sparkType, descriptor, importer, batchSize, false, false); - this.initialized = true; - } - - public ColumnDescriptor descriptor() { - return descriptor; - } - - /** Returns the Spark data type for this column. */ - public DataType sparkType() { - return sparkType; - } - - /** - * Set the page reader to be 'pageReader'. - * - *

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 { - Preconditions.checkState(initialized, "Invalid state: 'reset' should be called first"); - ((ColumnReader) delegate).setPageReader(pageReader); - } - - @Override - public void close() { - // close resources on native side - if (importer != null) { - importer.close(); - } - - if (delegate != null) { - delegate.close(); - } - } - - @Override - public void setBatchSize(int size) { - this.batchSize = size; - } - - @Override - public ColumnVector read(ColumnVector reuse, int numRowsToRead) { - throw new UnsupportedOperationException("Not supported"); - } -} 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 deleted file mode 100644 index 3d3e9aca24de..000000000000 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnarBatchReader.java +++ /dev/null @@ -1,137 +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. - */ -package org.apache.iceberg.spark.data.vectorized; - -import java.io.IOException; -import java.io.UncheckedIOException; -import java.util.List; -import java.util.Map; -import org.apache.comet.parquet.AbstractColumnReader; -import org.apache.comet.parquet.BatchReader; -import org.apache.iceberg.Schema; -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.parquet.column.page.PageReadStore; -import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; -import org.apache.parquet.hadoop.metadata.ColumnPath; -import org.apache.spark.sql.vectorized.ColumnVector; -import org.apache.spark.sql.vectorized.ColumnarBatch; - -/** - * {@link VectorizedReader} that returns Spark's {@link ColumnarBatch} to support Spark's vectorized - * read path. The {@link ColumnarBatch} returned is created by passing in the Arrow vectors - * populated via delegated read calls to {@link CometColumnReader VectorReader(s)}. - */ -@SuppressWarnings("checkstyle:VisibilityModifier") -class CometColumnarBatchReader implements VectorizedReader { - - private final CometColumnReader[] readers; - - // The delegated BatchReader on the Comet side does the real work of loading a batch of rows. - // The Comet BatchReader contains an array of ColumnReader. There is no need to explicitly call - // ColumnReader.readBatch; instead, BatchReader.nextBatch will be called, which underneath calls - // ColumnReader.readBatch. The only exception is DeleteColumnReader, because at the time of - // 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; - - CometColumnarBatchReader(List> readers, Schema schema) { - this.readers = - readers.stream().map(CometColumnReader.class::cast).toArray(CometColumnReader[]::new); - - AbstractColumnReader[] abstractColumnReaders = new AbstractColumnReader[readers.size()]; - this.delegate = new BatchReader(abstractColumnReaders); - delegate.setSparkSchema(SparkSchemaUtil.convert(schema)); - } - - @Override - public void setRowGroupInfo( - PageReadStore pageStore, Map metaData) { - for (int i = 0; i < readers.length; i++) { - try { - if (!(readers[i] instanceof CometConstantColumnReader) - && !(readers[i] instanceof CometPositionColumnReader) - && !(readers[i] instanceof CometDeleteColumnReader)) { - readers[i].reset(); - readers[i].setPageReader(pageStore.getPageReader(readers[i].descriptor())); - } - } catch (IOException e) { - throw new UncheckedIOException("Failed to setRowGroupInfo for Comet vectorization", e); - } - } - - for (int i = 0; i < readers.length; i++) { - delegate.getColumnReaders()[i] = this.readers[i].delegate(); - } - } - - @Override - public final ColumnarBatch read(ColumnarBatch reuse, int numRowsToRead) { - return new ColumnBatchLoader(numRowsToRead).loadDataToColumnBatch(); - } - - @Override - public void setBatchSize(int batchSize) { - for (CometColumnReader reader : readers) { - if (reader != null) { - reader.setBatchSize(batchSize); - } - } - } - - @Override - public void close() { - for (CometColumnReader reader : readers) { - if (reader != null) { - reader.close(); - } - } - } - - private class ColumnBatchLoader { - private final int batchSize; - - ColumnBatchLoader(int numRowsToRead) { - Preconditions.checkArgument( - numRowsToRead > 0, "Invalid number of rows to read: %s", numRowsToRead); - this.batchSize = numRowsToRead; - } - - ColumnarBatch loadDataToColumnBatch() { - ColumnVector[] vectors = readDataToColumnVectors(); - - ColumnarBatch batch = new ColumnarBatch(vectors); - batch.setNumRows(batchSize); - return batch; - } - - ColumnVector[] readDataToColumnVectors() { - ColumnVector[] columnVectors = new ColumnVector[readers.length]; - // Fetch rows for all readers in the delegate - delegate.nextBatch(batchSize); - for (int i = 0; i < readers.length; i++) { - columnVectors[i] = readers[i].delegate().currentBatch(); - } - - return columnVectors; - } - } -} 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 deleted file mode 100644 index 047c96314b13..000000000000 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometConstantColumnReader.java +++ /dev/null @@ -1,65 +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. - */ -package org.apache.iceberg.spark.data.vectorized; - -import java.math.BigDecimal; -import java.nio.ByteBuffer; -import org.apache.comet.parquet.ConstantColumnReader; -import org.apache.iceberg.types.Types; -import org.apache.spark.sql.types.DataType; -import org.apache.spark.sql.types.DataTypes; -import org.apache.spark.sql.types.Decimal; -import org.apache.spark.sql.types.DecimalType; -import org.apache.spark.unsafe.types.UTF8String; - -class CometConstantColumnReader extends CometColumnReader { - - CometConstantColumnReader(T value, Types.NestedField field) { - 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)); - } - - @Override - public void setBatchSize(int batchSize) { - super.setBatchSize(batchSize); - delegate().setBatchSize(batchSize); - setInitialized(true); - } - - private Object convertToSparkValue(T value) { - DataType dataType = sparkType(); - // Match the value to Spark internal type if necessary - if (dataType == DataTypes.StringType && value instanceof String) { - // the internal type for StringType is UTF8String - return UTF8String.fromString((String) value); - } else if (dataType instanceof DecimalType && value instanceof BigDecimal) { - // the internal type for DecimalType is Decimal - return Decimal.apply((BigDecimal) value); - } else if (dataType == DataTypes.BinaryType && value instanceof ByteBuffer) { - // the internal type for DecimalType is byte[] - // Iceberg default value should always use HeapBufferBuffer, so calling ByteBuffer.array() - // should be safe. - return ((ByteBuffer) value).array(); - } else { - return value; - } - } -} 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 deleted file mode 100644 index 26219014f777..000000000000 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeleteColumnReader.java +++ /dev/null @@ -1,74 +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. - */ -package org.apache.iceberg.spark.data.vectorized; - -import org.apache.comet.parquet.MetadataColumnReader; -import org.apache.comet.parquet.Native; -import org.apache.comet.parquet.TypeUtil; -import org.apache.comet.vector.CometVector; -import org.apache.iceberg.types.Types; -import org.apache.spark.sql.types.DataTypes; -import org.apache.spark.sql.types.Metadata; -import org.apache.spark.sql.types.StructField; - -class CometDeleteColumnReader extends CometColumnReader { - CometDeleteColumnReader(Types.NestedField field) { - super(field); - setDelegate(new DeleteColumnReader()); - } - - @Override - public void setBatchSize(int batchSize) { - super.setBatchSize(batchSize); - delegate().setBatchSize(batchSize); - setInitialized(true); - } - - private static class DeleteColumnReader extends MetadataColumnReader { - private final CometDeletedColumnVector deletedVector; - - DeleteColumnReader() { - this(new boolean[0]); - } - - DeleteColumnReader(boolean[] isDeleted) { - super( - DataTypes.BooleanType, - TypeUtil.convertToParquet( - new StructField("_deleted", DataTypes.BooleanType, false, Metadata.empty())), - false /* useDecimal128 = false */, - false /* isConstant = false */); - this.deletedVector = new CometDeletedColumnVector(isDeleted); - } - - @Override - public void readBatch(int total) { - Native.resetBatch(nativeHandle); - // set isDeleted on the native side to be consumed by native execution - Native.setIsDeleted(nativeHandle, deletedVector.isDeleted()); - - super.readBatch(total); - } - - @Override - public CometVector currentBatch() { - return deletedVector; - } - } -} diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeletedColumnVector.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeletedColumnVector.java deleted file mode 100644 index 5817f2c20a4f..000000000000 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeletedColumnVector.java +++ /dev/null @@ -1,155 +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. - */ -package org.apache.iceberg.spark.data.vectorized; - -import org.apache.comet.shaded.arrow.vector.ValueVector; -import org.apache.comet.vector.CometVector; -import org.apache.iceberg.spark.SparkSchemaUtil; -import org.apache.iceberg.types.Types; -import org.apache.spark.sql.types.Decimal; -import org.apache.spark.sql.vectorized.ColumnVector; -import org.apache.spark.sql.vectorized.ColumnarArray; -import org.apache.spark.sql.vectorized.ColumnarMap; -import org.apache.spark.unsafe.types.UTF8String; - -public class CometDeletedColumnVector extends CometVector implements UpdatableDeletedColumnVector { - private boolean[] isDeleted; - - public CometDeletedColumnVector(boolean[] isDeleted) { - super(SparkSchemaUtil.convert(Types.BooleanType.get()), false); - this.isDeleted = isDeleted; - } - - @Override - public void setValue(boolean[] deleted) { - this.isDeleted = deleted; - } - - boolean[] isDeleted() { - return isDeleted; - } - - @Override - public void setNumNulls(int numNulls) { - throw new UnsupportedOperationException("Not implemented"); - } - - @Override - public void setNumValues(int numValues) { - throw new UnsupportedOperationException("Not implemented"); - } - - @Override - public int numValues() { - throw new UnsupportedOperationException("Not implemented"); - } - - @Override - public ValueVector getValueVector() { - throw new UnsupportedOperationException("Not implemented"); - } - - @Override - public CometVector slice(int offset, int length) { - throw new UnsupportedOperationException("Not implemented"); - } - - @Override - public void close() {} - - @Override - public boolean hasNull() { - return false; - } - - @Override - public int numNulls() { - return 0; - } - - @Override - public boolean isNullAt(int rowId) { - return false; - } - - @Override - public boolean getBoolean(int rowId) { - return isDeleted[rowId]; - } - - @Override - public byte getByte(int rowId) { - throw new UnsupportedOperationException(); - } - - @Override - public short getShort(int rowId) { - throw new UnsupportedOperationException(); - } - - @Override - public int getInt(int rowId) { - throw new UnsupportedOperationException(); - } - - @Override - public long getLong(int rowId) { - throw new UnsupportedOperationException(); - } - - @Override - public float getFloat(int rowId) { - throw new UnsupportedOperationException(); - } - - @Override - public double getDouble(int rowId) { - throw new UnsupportedOperationException(); - } - - @Override - public ColumnarArray getArray(int rowId) { - throw new UnsupportedOperationException(); - } - - @Override - public ColumnarMap getMap(int ordinal) { - throw new UnsupportedOperationException(); - } - - @Override - public Decimal getDecimal(int rowId, int precision, int scale) { - throw new UnsupportedOperationException(); - } - - @Override - public UTF8String getUTF8String(int rowId) { - throw new UnsupportedOperationException(); - } - - @Override - public byte[] getBinary(int rowId) { - throw new UnsupportedOperationException(); - } - - @Override - public ColumnVector getChild(int ordinal) { - throw new UnsupportedOperationException(); - } -} 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 deleted file mode 100644 index bcc0e514c28d..000000000000 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometPositionColumnReader.java +++ /dev/null @@ -1,62 +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. - */ -package org.apache.iceberg.spark.data.vectorized; - -import org.apache.comet.parquet.MetadataColumnReader; -import org.apache.comet.parquet.Native; -import org.apache.iceberg.types.Types; -import org.apache.parquet.column.ColumnDescriptor; -import org.apache.spark.sql.types.DataTypes; - -class CometPositionColumnReader extends CometColumnReader { - CometPositionColumnReader(Types.NestedField field) { - super(field); - setDelegate(new PositionColumnReader(descriptor())); - } - - @Override - public void setBatchSize(int batchSize) { - super.setBatchSize(batchSize); - delegate().setBatchSize(batchSize); - setInitialized(true); - } - - private static class PositionColumnReader extends MetadataColumnReader { - /** The current position value of the column that are used to initialize this column reader. */ - private long position; - - PositionColumnReader(ColumnDescriptor descriptor) { - super( - DataTypes.LongType, - descriptor, - false /* useDecimal128 = false */, - false /* isConstant = false */); - } - - @Override - public void readBatch(int total) { - Native.resetBatch(nativeHandle); - // set position on the native side to be consumed by native execution - Native.setPosition(nativeHandle, position, total); - position += total; - - super.readBatch(total); - } - } -} 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 deleted file mode 100644 index 779dc240d4f6..000000000000 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometVectorizedReaderBuilder.java +++ /dev/null @@ -1,138 +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. - */ -package org.apache.iceberg.spark.data.vectorized; - -import java.util.List; -import java.util.Map; -import java.util.function.Function; -import java.util.stream.IntStream; -import org.apache.iceberg.MetadataColumns; -import org.apache.iceberg.Schema; -import org.apache.iceberg.parquet.TypeWithSchemaVisitor; -import org.apache.iceberg.parquet.VectorizedReader; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -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.GroupType; -import org.apache.parquet.schema.MessageType; -import org.apache.parquet.schema.PrimitiveType; -import org.apache.parquet.schema.Type; - -class CometVectorizedReaderBuilder extends TypeWithSchemaVisitor> { - - private final MessageType parquetSchema; - private final Schema icebergSchema; - private final Map idToConstant; - private final Function>, VectorizedReader> readerFactory; - - CometVectorizedReaderBuilder( - Schema expectedSchema, - MessageType parquetSchema, - Map idToConstant, - Function>, VectorizedReader> readerFactory) { - this.parquetSchema = parquetSchema; - this.icebergSchema = expectedSchema; - this.idToConstant = idToConstant; - this.readerFactory = readerFactory; - } - - @Override - public VectorizedReader message( - Types.StructType expected, MessageType message, List> fieldReaders) { - GroupType groupType = message.asGroupType(); - Map> readersById = Maps.newHashMap(); - List fields = groupType.getFields(); - - IntStream.range(0, fields.size()) - .filter(pos -> fields.get(pos).getId() != null) - .forEach(pos -> readersById.put(fields.get(pos).getId().intValue(), fieldReaders.get(pos))); - - List icebergFields = - expected != null ? expected.fields() : ImmutableList.of(); - - List> reorderedFields = - Lists.newArrayListWithExpectedSize(icebergFields.size()); - - for (Types.NestedField field : icebergFields) { - int id = field.fieldId(); - VectorizedReader reader = readersById.get(id); - if (idToConstant.containsKey(id)) { - CometConstantColumnReader constantReader = - new CometConstantColumnReader<>(idToConstant.get(id), field); - reorderedFields.add(constantReader); - } else if (id == MetadataColumns.ROW_POSITION.fieldId()) { - reorderedFields.add(new CometPositionColumnReader(field)); - } else if (id == MetadataColumns.IS_DELETED.fieldId()) { - CometColumnReader deleteReader = new CometDeleteColumnReader<>(field); - reorderedFields.add(deleteReader); - } else if (reader != null) { - reorderedFields.add(reader); - } else if (field.initialDefault() != null) { - CometColumnReader constantReader = - new CometConstantColumnReader<>(field.initialDefault(), field); - reorderedFields.add(constantReader); - } else if (field.isOptional()) { - CometColumnReader constantReader = new CometConstantColumnReader<>(null, field); - reorderedFields.add(constantReader); - } else { - throw new IllegalArgumentException( - String.format("Missing required field: %s", field.name())); - } - } - return vectorizedReader(reorderedFields); - } - - protected VectorizedReader vectorizedReader(List> reorderedFields) { - return readerFactory.apply(reorderedFields); - } - - @Override - public VectorizedReader struct( - Types.StructType expected, GroupType groupType, List> fieldReaders) { - if (expected != null) { - throw new UnsupportedOperationException( - "Vectorized reads are not supported yet for struct fields"); - } - return null; - } - - @Override - public VectorizedReader primitive( - org.apache.iceberg.types.Type.PrimitiveType expected, PrimitiveType primitive) { - - if (primitive.getId() == null) { - return null; - } - int parquetFieldId = primitive.getId().intValue(); - ColumnDescriptor desc = parquetSchema.getColumnDescription(currentPath()); - // Nested types not yet supported for vectorized reads - if (desc.getMaxRepetitionLevel() > 0) { - return null; - } - Types.NestedField icebergField = icebergSchema.findField(parquetFieldId); - if (icebergField == null) { - return null; - } - - return new CometColumnReader(SparkSchemaUtil.convert(icebergField.type()), desc); - } -} diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkParquetReaders.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkParquetReaders.java index 55f9fc1768a3..e959bbc2d542 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkParquetReaders.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkParquetReaders.java @@ -30,7 +30,6 @@ import org.apache.iceberg.parquet.VectorizedReader; import org.apache.iceberg.spark.SparkUtil; import org.apache.parquet.schema.MessageType; -import org.apache.spark.sql.vectorized.ColumnVector; import org.apache.spark.sql.vectorized.ColumnarBatch; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -77,26 +76,6 @@ public static ColumnarBatchReader buildReader( return buildReader(expectedSchema, fileSchema, idToConstant, ArrowAllocation.rootAllocator()); } - public static VectorizedReader buildCometReader( - Schema expectedSchema, MessageType fileSchema, Map idToConstant) { - return (CometColumnarBatchReader) - TypeWithSchemaVisitor.visit( - expectedSchema.asStruct(), - fileSchema, - new CometVectorizedReaderBuilder( - expectedSchema, - fileSchema, - idToConstant, - readers -> new CometColumnarBatchReader(readers, expectedSchema))); - } - - /** A subclass of ColumnarBatch to identify Comet readers. */ - public static class CometColumnarBatch extends ColumnarBatch { - public CometColumnarBatch(ColumnVector[] columns) { - super(columns); - } - } - // enables unsafe memory access to avoid costly checks to see if index is within bounds // as long as it is not configured explicitly (see BoundsChecking in Arrow) private static void enableUnsafeMemoryAccess() { 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 89c03a4c2b72..fe062f9d7357 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 @@ -35,11 +35,9 @@ import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.spark.OrcBatchReadConf; import org.apache.iceberg.spark.ParquetBatchReadConf; -import org.apache.iceberg.spark.ParquetReaderType; import org.apache.iceberg.spark.data.vectorized.ColumnVectorWithFilter; import org.apache.iceberg.spark.data.vectorized.ColumnarBatchUtil; import org.apache.iceberg.spark.data.vectorized.UpdatableDeletedColumnVector; -import org.apache.iceberg.spark.data.vectorized.VectorizedSparkParquetReaders; import org.apache.iceberg.util.Pair; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.vectorized.ColumnVector; @@ -72,10 +70,8 @@ protected CloseableIterable newBatchIterable( Expression residual, Map idToConstant, @Nonnull SparkDeleteFilter deleteFilter) { - Class readType = - useComet() ? VectorizedSparkParquetReaders.CometColumnarBatch.class : ColumnarBatch.class; ReadBuilder readBuilder = - FormatModelRegistry.readBuilder(format, readType, inputFile); + FormatModelRegistry.readBuilder(format, ColumnarBatch.class, inputFile); if (parquetConf != null) { readBuilder = readBuilder.recordsPerBatch(parquetConf.batchSize()); @@ -101,10 +97,6 @@ protected CloseableIterable newBatchIterable( return CloseableIterable.transform(iterable, new BatchDeleteFilter(deleteFilter)::filterBatch); } - private boolean useComet() { - return parquetConf != null && parquetConf.readerType() == ParquetReaderType.COMET; - } - @VisibleForTesting static class BatchDeleteFilter { private final DeleteFilter deletes; 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..6bbf3ca3bab2 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 @@ -32,10 +32,8 @@ import org.apache.iceberg.spark.ImmutableParquetBatchReadConf; import org.apache.iceberg.spark.OrcBatchReadConf; import org.apache.iceberg.spark.ParquetBatchReadConf; -import org.apache.iceberg.spark.ParquetReaderType; import org.apache.iceberg.spark.SparkReadConf; import org.apache.iceberg.spark.SparkUtil; -import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.broadcast.Broadcast; @@ -122,11 +120,8 @@ private String[][] computePreferredLocations() { @Override public PartitionReaderFactory createReaderFactory() { - if (useCometBatchReads()) { - return new SparkColumnarReaderFactory(parquetBatchReadConf(ParquetReaderType.COMET)); - - } else if (useParquetBatchReads()) { - return new SparkColumnarReaderFactory(parquetBatchReadConf(ParquetReaderType.ICEBERG)); + if (useParquetBatchReads()) { + return new SparkColumnarReaderFactory(parquetBatchReadConf()); } else if (useOrcBatchReads()) { return new SparkColumnarReaderFactory(orcBatchReadConf()); @@ -136,10 +131,9 @@ public PartitionReaderFactory createReaderFactory() { } } - private ParquetBatchReadConf parquetBatchReadConf(ParquetReaderType readerType) { + private ParquetBatchReadConf parquetBatchReadConf() { return ImmutableParquetBatchReadConf.builder() .batchSize(readConf.parquetBatchSize()) - .readerType(readerType) .build(); } @@ -175,20 +169,6 @@ private boolean supportsParquetBatchReads(Types.NestedField field) { return field.type().isPrimitiveType() || MetadataColumns.isMetadataColumn(field.fieldId()); } - private boolean useCometBatchReads() { - return readConf.parquetVectorizationEnabled() - && readConf.parquetReaderType() == ParquetReaderType.COMET - && expectedSchema.columns().stream().allMatch(this::supportsCometBatchReads) - && taskGroups.stream().allMatch(this::supportsParquetBatchReads); - } - - private boolean supportsCometBatchReads(Types.NestedField field) { - return field.type().isPrimitiveType() - && !field.type().typeId().equals(Type.TypeID.UUID) - && field.fieldId() != MetadataColumns.ROW_ID.fieldId() - && field.fieldId() != MetadataColumns.LAST_UPDATED_SEQUENCE_NUMBER.fieldId(); - } - // conditions for using ORC batch reads: // - ORC vectorization is enabled // - all tasks are of type FileScanTask and read only ORC files with no delete files diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkFormatModels.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkFormatModels.java index 677f2e950b44..a1e8a82b4d80 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkFormatModels.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkFormatModels.java @@ -60,14 +60,6 @@ public static void register() { VectorizedSparkParquetReaders.buildReader( icebergSchema, fileSchema, idToConstant))); - FormatModelRegistry.register( - ParquetFormatModel.create( - VectorizedSparkParquetReaders.CometColumnarBatch.class, - StructType.class, - (icebergSchema, fileSchema, engineSchema, idToConstant) -> - VectorizedSparkParquetReaders.buildCometReader( - icebergSchema, fileSchema, idToConstant))); - FormatModelRegistry.register( ORCFormatModel.create( InternalRow.class, diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetCometVectorizedScan.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetCometVectorizedScan.java deleted file mode 100644 index 5e1d88f8bcd4..000000000000 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetCometVectorizedScan.java +++ /dev/null @@ -1,33 +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. - */ -package org.apache.iceberg.spark.source; - -import org.junit.jupiter.api.BeforeAll; - -public class TestParquetCometVectorizedScan extends TestParquetScan { - @BeforeAll - public static void setComet() { - ScanTestBase.spark.conf().set("spark.sql.iceberg.parquet.reader-type", "COMET"); - } - - @Override - protected boolean vectorized() { - return true; - } -} diff --git a/spark/v4.0/build.gradle b/spark/v4.0/build.gradle index fb9df4e0e0f2..90996f8c6f5a 100644 --- a/spark/v4.0/build.gradle +++ b/spark/v4.0/build.gradle @@ -84,8 +84,6 @@ project(":iceberg-spark:iceberg-spark-${sparkMajorVersion}_${scalaVersion}") { exclude group: 'org.roaringbitmap' } - compileOnly "org.apache.datafusion:comet-spark-spark${sparkMajorVersion}_2.13:${libs.versions.comet.get()}" - implementation libs.parquet.column implementation libs.parquet.hadoop @@ -194,7 +192,6 @@ project(":iceberg-spark:iceberg-spark-extensions-${sparkMajorVersion}_${scalaVer testImplementation libs.avro.avro testImplementation libs.parquet.hadoop testImplementation libs.awaitility - testImplementation "org.apache.datafusion:comet-spark-spark${sparkMajorVersion}_2.13:${libs.versions.comet.get()}" testImplementation(testFixtures(project(':iceberg-parquet'))) // Required because we remove antlr plugin dependencies from the compile configuration, see note above 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..b7cd441c9642 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 @@ -24,6 +24,4 @@ @Value.Immutable public interface ParquetBatchReadConf extends Serializable { int batchSize(); - - ParquetReaderType readerType(); } diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/ParquetReaderType.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/ParquetReaderType.java deleted file mode 100644 index d9742c048251..000000000000 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/ParquetReaderType.java +++ /dev/null @@ -1,47 +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. - */ -package org.apache.iceberg.spark; - -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; - -/** Enumerates the types of Parquet readers. */ -public enum ParquetReaderType { - /** ICEBERG type utilizes the built-in Parquet reader. */ - ICEBERG, - - /** - * COMET type changes the Parquet reader to the Apache DataFusion Comet Parquet reader. Comet - * Parquet reader performs I/O and decompression in the JVM but decodes in native to improve - * performance. Additionally, Comet will convert Spark's physical plan into a native physical plan - * and execute this plan natively. - * - *

TODO: Implement {@link org.apache.comet.parquet.SupportsComet} in SparkScan to convert Spark - * physical plan to native physical plan for native execution. - */ - COMET; - - public static ParquetReaderType fromString(String typeAsString) { - Preconditions.checkArgument(typeAsString != null, "Parquet reader type is null"); - try { - return ParquetReaderType.valueOf(typeAsString.toUpperCase()); - } catch (IllegalArgumentException e) { - throw new IllegalArgumentException("Unknown parquet reader type: " + typeAsString); - } - } -} 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 420c3517ffb5..9a19aa7d1e62 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 @@ -370,12 +370,4 @@ public boolean reportColumnStats() { .defaultValue(SparkSQLProperties.REPORT_COLUMN_STATS_DEFAULT) .parse(); } - - public ParquetReaderType parquetReaderType() { - return confParser - .enumConf(ParquetReaderType::fromString) - .sessionConf(SparkSQLProperties.PARQUET_READER_TYPE) - .defaultValue(SparkSQLProperties.PARQUET_READER_TYPE_DEFAULT) - .parse(); - } } 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 81139969f746..735ee4efbc35 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 @@ -27,9 +27,6 @@ private SparkSQLProperties() {} // Controls whether vectorized reads are enabled public static final String VECTORIZATION_ENABLED = "spark.sql.iceberg.vectorization.enabled"; - // 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 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/data/vectorized/CometColumnReader.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnReader.java deleted file mode 100644 index 81b7d83a7077..000000000000 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnReader.java +++ /dev/null @@ -1,140 +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. - */ -package org.apache.iceberg.spark.data.vectorized; - -import java.io.IOException; -import org.apache.comet.CometSchemaImporter; -import org.apache.comet.parquet.AbstractColumnReader; -import org.apache.comet.parquet.ColumnReader; -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.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.types.DataType; -import org.apache.spark.sql.types.Metadata; -import org.apache.spark.sql.types.StructField; -import org.apache.spark.sql.vectorized.ColumnVector; - -class CometColumnReader implements VectorizedReader { - // use the Comet default batch size - public static final int DEFAULT_BATCH_SIZE = 8192; - - private final ColumnDescriptor descriptor; - private final DataType sparkType; - - // The delegated ColumnReader from Comet side - private AbstractColumnReader delegate; - private boolean initialized = false; - private int batchSize = DEFAULT_BATCH_SIZE; - private CometSchemaImporter importer; - - CometColumnReader(DataType sparkType, ColumnDescriptor descriptor) { - this.sparkType = sparkType; - this.descriptor = descriptor; - } - - 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); - } - - public AbstractColumnReader delegate() { - return delegate; - } - - void setDelegate(AbstractColumnReader delegate) { - this.delegate = delegate; - } - - void setInitialized(boolean initialized) { - this.initialized = initialized; - } - - public int batchSize() { - return batchSize; - } - - /** - * This method is to initialized/reset the CometColumnReader. This needs to be called for each row - * group after readNextRowGroup, so a new dictionary encoding can be set for each of the new row - * groups. - */ - public void reset() { - if (importer != null) { - importer.close(); - } - - if (delegate != null) { - delegate.close(); - } - - this.importer = new CometSchemaImporter(new RootAllocator()); - this.delegate = Utils.getColumnReader(sparkType, descriptor, importer, batchSize, false, false); - this.initialized = true; - } - - public ColumnDescriptor descriptor() { - return descriptor; - } - - /** Returns the Spark data type for this column. */ - public DataType sparkType() { - return sparkType; - } - - /** - * Set the page reader to be 'pageReader'. - * - *

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 { - Preconditions.checkState(initialized, "Invalid state: 'reset' should be called first"); - ((ColumnReader) delegate).setPageReader(pageReader); - } - - @Override - public void close() { - // close resources on native side - if (importer != null) { - importer.close(); - } - - if (delegate != null) { - delegate.close(); - } - } - - @Override - public void setBatchSize(int size) { - this.batchSize = size; - } - - @Override - public ColumnVector read(ColumnVector reuse, int numRowsToRead) { - throw new UnsupportedOperationException("Not supported"); - } -} 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 deleted file mode 100644 index 3d3e9aca24de..000000000000 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnarBatchReader.java +++ /dev/null @@ -1,137 +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. - */ -package org.apache.iceberg.spark.data.vectorized; - -import java.io.IOException; -import java.io.UncheckedIOException; -import java.util.List; -import java.util.Map; -import org.apache.comet.parquet.AbstractColumnReader; -import org.apache.comet.parquet.BatchReader; -import org.apache.iceberg.Schema; -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.parquet.column.page.PageReadStore; -import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; -import org.apache.parquet.hadoop.metadata.ColumnPath; -import org.apache.spark.sql.vectorized.ColumnVector; -import org.apache.spark.sql.vectorized.ColumnarBatch; - -/** - * {@link VectorizedReader} that returns Spark's {@link ColumnarBatch} to support Spark's vectorized - * read path. The {@link ColumnarBatch} returned is created by passing in the Arrow vectors - * populated via delegated read calls to {@link CometColumnReader VectorReader(s)}. - */ -@SuppressWarnings("checkstyle:VisibilityModifier") -class CometColumnarBatchReader implements VectorizedReader { - - private final CometColumnReader[] readers; - - // The delegated BatchReader on the Comet side does the real work of loading a batch of rows. - // The Comet BatchReader contains an array of ColumnReader. There is no need to explicitly call - // ColumnReader.readBatch; instead, BatchReader.nextBatch will be called, which underneath calls - // ColumnReader.readBatch. The only exception is DeleteColumnReader, because at the time of - // 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; - - CometColumnarBatchReader(List> readers, Schema schema) { - this.readers = - readers.stream().map(CometColumnReader.class::cast).toArray(CometColumnReader[]::new); - - AbstractColumnReader[] abstractColumnReaders = new AbstractColumnReader[readers.size()]; - this.delegate = new BatchReader(abstractColumnReaders); - delegate.setSparkSchema(SparkSchemaUtil.convert(schema)); - } - - @Override - public void setRowGroupInfo( - PageReadStore pageStore, Map metaData) { - for (int i = 0; i < readers.length; i++) { - try { - if (!(readers[i] instanceof CometConstantColumnReader) - && !(readers[i] instanceof CometPositionColumnReader) - && !(readers[i] instanceof CometDeleteColumnReader)) { - readers[i].reset(); - readers[i].setPageReader(pageStore.getPageReader(readers[i].descriptor())); - } - } catch (IOException e) { - throw new UncheckedIOException("Failed to setRowGroupInfo for Comet vectorization", e); - } - } - - for (int i = 0; i < readers.length; i++) { - delegate.getColumnReaders()[i] = this.readers[i].delegate(); - } - } - - @Override - public final ColumnarBatch read(ColumnarBatch reuse, int numRowsToRead) { - return new ColumnBatchLoader(numRowsToRead).loadDataToColumnBatch(); - } - - @Override - public void setBatchSize(int batchSize) { - for (CometColumnReader reader : readers) { - if (reader != null) { - reader.setBatchSize(batchSize); - } - } - } - - @Override - public void close() { - for (CometColumnReader reader : readers) { - if (reader != null) { - reader.close(); - } - } - } - - private class ColumnBatchLoader { - private final int batchSize; - - ColumnBatchLoader(int numRowsToRead) { - Preconditions.checkArgument( - numRowsToRead > 0, "Invalid number of rows to read: %s", numRowsToRead); - this.batchSize = numRowsToRead; - } - - ColumnarBatch loadDataToColumnBatch() { - ColumnVector[] vectors = readDataToColumnVectors(); - - ColumnarBatch batch = new ColumnarBatch(vectors); - batch.setNumRows(batchSize); - return batch; - } - - ColumnVector[] readDataToColumnVectors() { - ColumnVector[] columnVectors = new ColumnVector[readers.length]; - // Fetch rows for all readers in the delegate - delegate.nextBatch(batchSize); - for (int i = 0; i < readers.length; i++) { - columnVectors[i] = readers[i].delegate().currentBatch(); - } - - return columnVectors; - } - } -} 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 deleted file mode 100644 index 047c96314b13..000000000000 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometConstantColumnReader.java +++ /dev/null @@ -1,65 +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. - */ -package org.apache.iceberg.spark.data.vectorized; - -import java.math.BigDecimal; -import java.nio.ByteBuffer; -import org.apache.comet.parquet.ConstantColumnReader; -import org.apache.iceberg.types.Types; -import org.apache.spark.sql.types.DataType; -import org.apache.spark.sql.types.DataTypes; -import org.apache.spark.sql.types.Decimal; -import org.apache.spark.sql.types.DecimalType; -import org.apache.spark.unsafe.types.UTF8String; - -class CometConstantColumnReader extends CometColumnReader { - - CometConstantColumnReader(T value, Types.NestedField field) { - 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)); - } - - @Override - public void setBatchSize(int batchSize) { - super.setBatchSize(batchSize); - delegate().setBatchSize(batchSize); - setInitialized(true); - } - - private Object convertToSparkValue(T value) { - DataType dataType = sparkType(); - // Match the value to Spark internal type if necessary - if (dataType == DataTypes.StringType && value instanceof String) { - // the internal type for StringType is UTF8String - return UTF8String.fromString((String) value); - } else if (dataType instanceof DecimalType && value instanceof BigDecimal) { - // the internal type for DecimalType is Decimal - return Decimal.apply((BigDecimal) value); - } else if (dataType == DataTypes.BinaryType && value instanceof ByteBuffer) { - // the internal type for DecimalType is byte[] - // Iceberg default value should always use HeapBufferBuffer, so calling ByteBuffer.array() - // should be safe. - return ((ByteBuffer) value).array(); - } else { - return value; - } - } -} 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 deleted file mode 100644 index 26219014f777..000000000000 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeleteColumnReader.java +++ /dev/null @@ -1,74 +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. - */ -package org.apache.iceberg.spark.data.vectorized; - -import org.apache.comet.parquet.MetadataColumnReader; -import org.apache.comet.parquet.Native; -import org.apache.comet.parquet.TypeUtil; -import org.apache.comet.vector.CometVector; -import org.apache.iceberg.types.Types; -import org.apache.spark.sql.types.DataTypes; -import org.apache.spark.sql.types.Metadata; -import org.apache.spark.sql.types.StructField; - -class CometDeleteColumnReader extends CometColumnReader { - CometDeleteColumnReader(Types.NestedField field) { - super(field); - setDelegate(new DeleteColumnReader()); - } - - @Override - public void setBatchSize(int batchSize) { - super.setBatchSize(batchSize); - delegate().setBatchSize(batchSize); - setInitialized(true); - } - - private static class DeleteColumnReader extends MetadataColumnReader { - private final CometDeletedColumnVector deletedVector; - - DeleteColumnReader() { - this(new boolean[0]); - } - - DeleteColumnReader(boolean[] isDeleted) { - super( - DataTypes.BooleanType, - TypeUtil.convertToParquet( - new StructField("_deleted", DataTypes.BooleanType, false, Metadata.empty())), - false /* useDecimal128 = false */, - false /* isConstant = false */); - this.deletedVector = new CometDeletedColumnVector(isDeleted); - } - - @Override - public void readBatch(int total) { - Native.resetBatch(nativeHandle); - // set isDeleted on the native side to be consumed by native execution - Native.setIsDeleted(nativeHandle, deletedVector.isDeleted()); - - super.readBatch(total); - } - - @Override - public CometVector currentBatch() { - return deletedVector; - } - } -} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeletedColumnVector.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeletedColumnVector.java deleted file mode 100644 index 5817f2c20a4f..000000000000 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeletedColumnVector.java +++ /dev/null @@ -1,155 +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. - */ -package org.apache.iceberg.spark.data.vectorized; - -import org.apache.comet.shaded.arrow.vector.ValueVector; -import org.apache.comet.vector.CometVector; -import org.apache.iceberg.spark.SparkSchemaUtil; -import org.apache.iceberg.types.Types; -import org.apache.spark.sql.types.Decimal; -import org.apache.spark.sql.vectorized.ColumnVector; -import org.apache.spark.sql.vectorized.ColumnarArray; -import org.apache.spark.sql.vectorized.ColumnarMap; -import org.apache.spark.unsafe.types.UTF8String; - -public class CometDeletedColumnVector extends CometVector implements UpdatableDeletedColumnVector { - private boolean[] isDeleted; - - public CometDeletedColumnVector(boolean[] isDeleted) { - super(SparkSchemaUtil.convert(Types.BooleanType.get()), false); - this.isDeleted = isDeleted; - } - - @Override - public void setValue(boolean[] deleted) { - this.isDeleted = deleted; - } - - boolean[] isDeleted() { - return isDeleted; - } - - @Override - public void setNumNulls(int numNulls) { - throw new UnsupportedOperationException("Not implemented"); - } - - @Override - public void setNumValues(int numValues) { - throw new UnsupportedOperationException("Not implemented"); - } - - @Override - public int numValues() { - throw new UnsupportedOperationException("Not implemented"); - } - - @Override - public ValueVector getValueVector() { - throw new UnsupportedOperationException("Not implemented"); - } - - @Override - public CometVector slice(int offset, int length) { - throw new UnsupportedOperationException("Not implemented"); - } - - @Override - public void close() {} - - @Override - public boolean hasNull() { - return false; - } - - @Override - public int numNulls() { - return 0; - } - - @Override - public boolean isNullAt(int rowId) { - return false; - } - - @Override - public boolean getBoolean(int rowId) { - return isDeleted[rowId]; - } - - @Override - public byte getByte(int rowId) { - throw new UnsupportedOperationException(); - } - - @Override - public short getShort(int rowId) { - throw new UnsupportedOperationException(); - } - - @Override - public int getInt(int rowId) { - throw new UnsupportedOperationException(); - } - - @Override - public long getLong(int rowId) { - throw new UnsupportedOperationException(); - } - - @Override - public float getFloat(int rowId) { - throw new UnsupportedOperationException(); - } - - @Override - public double getDouble(int rowId) { - throw new UnsupportedOperationException(); - } - - @Override - public ColumnarArray getArray(int rowId) { - throw new UnsupportedOperationException(); - } - - @Override - public ColumnarMap getMap(int ordinal) { - throw new UnsupportedOperationException(); - } - - @Override - public Decimal getDecimal(int rowId, int precision, int scale) { - throw new UnsupportedOperationException(); - } - - @Override - public UTF8String getUTF8String(int rowId) { - throw new UnsupportedOperationException(); - } - - @Override - public byte[] getBinary(int rowId) { - throw new UnsupportedOperationException(); - } - - @Override - public ColumnVector getChild(int ordinal) { - throw new UnsupportedOperationException(); - } -} 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 deleted file mode 100644 index bcc0e514c28d..000000000000 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometPositionColumnReader.java +++ /dev/null @@ -1,62 +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. - */ -package org.apache.iceberg.spark.data.vectorized; - -import org.apache.comet.parquet.MetadataColumnReader; -import org.apache.comet.parquet.Native; -import org.apache.iceberg.types.Types; -import org.apache.parquet.column.ColumnDescriptor; -import org.apache.spark.sql.types.DataTypes; - -class CometPositionColumnReader extends CometColumnReader { - CometPositionColumnReader(Types.NestedField field) { - super(field); - setDelegate(new PositionColumnReader(descriptor())); - } - - @Override - public void setBatchSize(int batchSize) { - super.setBatchSize(batchSize); - delegate().setBatchSize(batchSize); - setInitialized(true); - } - - private static class PositionColumnReader extends MetadataColumnReader { - /** The current position value of the column that are used to initialize this column reader. */ - private long position; - - PositionColumnReader(ColumnDescriptor descriptor) { - super( - DataTypes.LongType, - descriptor, - false /* useDecimal128 = false */, - false /* isConstant = false */); - } - - @Override - public void readBatch(int total) { - Native.resetBatch(nativeHandle); - // set position on the native side to be consumed by native execution - Native.setPosition(nativeHandle, position, total); - position += total; - - super.readBatch(total); - } - } -} 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 deleted file mode 100644 index 779dc240d4f6..000000000000 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometVectorizedReaderBuilder.java +++ /dev/null @@ -1,138 +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. - */ -package org.apache.iceberg.spark.data.vectorized; - -import java.util.List; -import java.util.Map; -import java.util.function.Function; -import java.util.stream.IntStream; -import org.apache.iceberg.MetadataColumns; -import org.apache.iceberg.Schema; -import org.apache.iceberg.parquet.TypeWithSchemaVisitor; -import org.apache.iceberg.parquet.VectorizedReader; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -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.GroupType; -import org.apache.parquet.schema.MessageType; -import org.apache.parquet.schema.PrimitiveType; -import org.apache.parquet.schema.Type; - -class CometVectorizedReaderBuilder extends TypeWithSchemaVisitor> { - - private final MessageType parquetSchema; - private final Schema icebergSchema; - private final Map idToConstant; - private final Function>, VectorizedReader> readerFactory; - - CometVectorizedReaderBuilder( - Schema expectedSchema, - MessageType parquetSchema, - Map idToConstant, - Function>, VectorizedReader> readerFactory) { - this.parquetSchema = parquetSchema; - this.icebergSchema = expectedSchema; - this.idToConstant = idToConstant; - this.readerFactory = readerFactory; - } - - @Override - public VectorizedReader message( - Types.StructType expected, MessageType message, List> fieldReaders) { - GroupType groupType = message.asGroupType(); - Map> readersById = Maps.newHashMap(); - List fields = groupType.getFields(); - - IntStream.range(0, fields.size()) - .filter(pos -> fields.get(pos).getId() != null) - .forEach(pos -> readersById.put(fields.get(pos).getId().intValue(), fieldReaders.get(pos))); - - List icebergFields = - expected != null ? expected.fields() : ImmutableList.of(); - - List> reorderedFields = - Lists.newArrayListWithExpectedSize(icebergFields.size()); - - for (Types.NestedField field : icebergFields) { - int id = field.fieldId(); - VectorizedReader reader = readersById.get(id); - if (idToConstant.containsKey(id)) { - CometConstantColumnReader constantReader = - new CometConstantColumnReader<>(idToConstant.get(id), field); - reorderedFields.add(constantReader); - } else if (id == MetadataColumns.ROW_POSITION.fieldId()) { - reorderedFields.add(new CometPositionColumnReader(field)); - } else if (id == MetadataColumns.IS_DELETED.fieldId()) { - CometColumnReader deleteReader = new CometDeleteColumnReader<>(field); - reorderedFields.add(deleteReader); - } else if (reader != null) { - reorderedFields.add(reader); - } else if (field.initialDefault() != null) { - CometColumnReader constantReader = - new CometConstantColumnReader<>(field.initialDefault(), field); - reorderedFields.add(constantReader); - } else if (field.isOptional()) { - CometColumnReader constantReader = new CometConstantColumnReader<>(null, field); - reorderedFields.add(constantReader); - } else { - throw new IllegalArgumentException( - String.format("Missing required field: %s", field.name())); - } - } - return vectorizedReader(reorderedFields); - } - - protected VectorizedReader vectorizedReader(List> reorderedFields) { - return readerFactory.apply(reorderedFields); - } - - @Override - public VectorizedReader struct( - Types.StructType expected, GroupType groupType, List> fieldReaders) { - if (expected != null) { - throw new UnsupportedOperationException( - "Vectorized reads are not supported yet for struct fields"); - } - return null; - } - - @Override - public VectorizedReader primitive( - org.apache.iceberg.types.Type.PrimitiveType expected, PrimitiveType primitive) { - - if (primitive.getId() == null) { - return null; - } - int parquetFieldId = primitive.getId().intValue(); - ColumnDescriptor desc = parquetSchema.getColumnDescription(currentPath()); - // Nested types not yet supported for vectorized reads - if (desc.getMaxRepetitionLevel() > 0) { - return null; - } - Types.NestedField icebergField = icebergSchema.findField(parquetFieldId); - if (icebergField == null) { - return null; - } - - return new CometColumnReader(SparkSchemaUtil.convert(icebergField.type()), desc); - } -} diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkParquetReaders.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkParquetReaders.java index 55f9fc1768a3..e959bbc2d542 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkParquetReaders.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkParquetReaders.java @@ -30,7 +30,6 @@ import org.apache.iceberg.parquet.VectorizedReader; import org.apache.iceberg.spark.SparkUtil; import org.apache.parquet.schema.MessageType; -import org.apache.spark.sql.vectorized.ColumnVector; import org.apache.spark.sql.vectorized.ColumnarBatch; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -77,26 +76,6 @@ public static ColumnarBatchReader buildReader( return buildReader(expectedSchema, fileSchema, idToConstant, ArrowAllocation.rootAllocator()); } - public static VectorizedReader buildCometReader( - Schema expectedSchema, MessageType fileSchema, Map idToConstant) { - return (CometColumnarBatchReader) - TypeWithSchemaVisitor.visit( - expectedSchema.asStruct(), - fileSchema, - new CometVectorizedReaderBuilder( - expectedSchema, - fileSchema, - idToConstant, - readers -> new CometColumnarBatchReader(readers, expectedSchema))); - } - - /** A subclass of ColumnarBatch to identify Comet readers. */ - public static class CometColumnarBatch extends ColumnarBatch { - public CometColumnarBatch(ColumnVector[] columns) { - super(columns); - } - } - // enables unsafe memory access to avoid costly checks to see if index is within bounds // as long as it is not configured explicitly (see BoundsChecking in Arrow) private static void enableUnsafeMemoryAccess() { 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 89c03a4c2b72..fe062f9d7357 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 @@ -35,11 +35,9 @@ import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.spark.OrcBatchReadConf; import org.apache.iceberg.spark.ParquetBatchReadConf; -import org.apache.iceberg.spark.ParquetReaderType; import org.apache.iceberg.spark.data.vectorized.ColumnVectorWithFilter; import org.apache.iceberg.spark.data.vectorized.ColumnarBatchUtil; import org.apache.iceberg.spark.data.vectorized.UpdatableDeletedColumnVector; -import org.apache.iceberg.spark.data.vectorized.VectorizedSparkParquetReaders; import org.apache.iceberg.util.Pair; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.vectorized.ColumnVector; @@ -72,10 +70,8 @@ protected CloseableIterable newBatchIterable( Expression residual, Map idToConstant, @Nonnull SparkDeleteFilter deleteFilter) { - Class readType = - useComet() ? VectorizedSparkParquetReaders.CometColumnarBatch.class : ColumnarBatch.class; ReadBuilder readBuilder = - FormatModelRegistry.readBuilder(format, readType, inputFile); + FormatModelRegistry.readBuilder(format, ColumnarBatch.class, inputFile); if (parquetConf != null) { readBuilder = readBuilder.recordsPerBatch(parquetConf.batchSize()); @@ -101,10 +97,6 @@ protected CloseableIterable newBatchIterable( return CloseableIterable.transform(iterable, new BatchDeleteFilter(deleteFilter)::filterBatch); } - private boolean useComet() { - return parquetConf != null && parquetConf.readerType() == ParquetReaderType.COMET; - } - @VisibleForTesting static class BatchDeleteFilter { private final DeleteFilter deletes; 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..6bbf3ca3bab2 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 @@ -32,10 +32,8 @@ import org.apache.iceberg.spark.ImmutableParquetBatchReadConf; import org.apache.iceberg.spark.OrcBatchReadConf; import org.apache.iceberg.spark.ParquetBatchReadConf; -import org.apache.iceberg.spark.ParquetReaderType; import org.apache.iceberg.spark.SparkReadConf; import org.apache.iceberg.spark.SparkUtil; -import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.broadcast.Broadcast; @@ -122,11 +120,8 @@ private String[][] computePreferredLocations() { @Override public PartitionReaderFactory createReaderFactory() { - if (useCometBatchReads()) { - return new SparkColumnarReaderFactory(parquetBatchReadConf(ParquetReaderType.COMET)); - - } else if (useParquetBatchReads()) { - return new SparkColumnarReaderFactory(parquetBatchReadConf(ParquetReaderType.ICEBERG)); + if (useParquetBatchReads()) { + return new SparkColumnarReaderFactory(parquetBatchReadConf()); } else if (useOrcBatchReads()) { return new SparkColumnarReaderFactory(orcBatchReadConf()); @@ -136,10 +131,9 @@ public PartitionReaderFactory createReaderFactory() { } } - private ParquetBatchReadConf parquetBatchReadConf(ParquetReaderType readerType) { + private ParquetBatchReadConf parquetBatchReadConf() { return ImmutableParquetBatchReadConf.builder() .batchSize(readConf.parquetBatchSize()) - .readerType(readerType) .build(); } @@ -175,20 +169,6 @@ private boolean supportsParquetBatchReads(Types.NestedField field) { return field.type().isPrimitiveType() || MetadataColumns.isMetadataColumn(field.fieldId()); } - private boolean useCometBatchReads() { - return readConf.parquetVectorizationEnabled() - && readConf.parquetReaderType() == ParquetReaderType.COMET - && expectedSchema.columns().stream().allMatch(this::supportsCometBatchReads) - && taskGroups.stream().allMatch(this::supportsParquetBatchReads); - } - - private boolean supportsCometBatchReads(Types.NestedField field) { - return field.type().isPrimitiveType() - && !field.type().typeId().equals(Type.TypeID.UUID) - && field.fieldId() != MetadataColumns.ROW_ID.fieldId() - && field.fieldId() != MetadataColumns.LAST_UPDATED_SEQUENCE_NUMBER.fieldId(); - } - // conditions for using ORC batch reads: // - ORC vectorization is enabled // - all tasks are of type FileScanTask and read only ORC files with no delete files diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkFormatModels.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkFormatModels.java index 677f2e950b44..a1e8a82b4d80 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkFormatModels.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/source/SparkFormatModels.java @@ -60,14 +60,6 @@ public static void register() { VectorizedSparkParquetReaders.buildReader( icebergSchema, fileSchema, idToConstant))); - FormatModelRegistry.register( - ParquetFormatModel.create( - VectorizedSparkParquetReaders.CometColumnarBatch.class, - StructType.class, - (icebergSchema, fileSchema, engineSchema, idToConstant) -> - VectorizedSparkParquetReaders.buildCometReader( - icebergSchema, fileSchema, idToConstant))); - FormatModelRegistry.register( ORCFormatModel.create( InternalRow.class, diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetCometVectorizedScan.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetCometVectorizedScan.java deleted file mode 100644 index 5e1d88f8bcd4..000000000000 --- a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetCometVectorizedScan.java +++ /dev/null @@ -1,33 +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. - */ -package org.apache.iceberg.spark.source; - -import org.junit.jupiter.api.BeforeAll; - -public class TestParquetCometVectorizedScan extends TestParquetScan { - @BeforeAll - public static void setComet() { - ScanTestBase.spark.conf().set("spark.sql.iceberg.parquet.reader-type", "COMET"); - } - - @Override - protected boolean vectorized() { - return true; - } -} diff --git a/spark/v4.1/build.gradle b/spark/v4.1/build.gradle index a342a0b59c92..a6b03f7e37a3 100644 --- a/spark/v4.1/build.gradle +++ b/spark/v4.1/build.gradle @@ -84,9 +84,6 @@ project(":iceberg-spark:iceberg-spark-${sparkMajorVersion}_${scalaVersion}") { exclude group: 'org.roaringbitmap' } - // TODO: datafusion-comet Spark 4.1 support - compileOnly "org.apache.datafusion:comet-spark-spark4.0_2.13:${libs.versions.comet.get()}" - implementation libs.parquet.column implementation libs.parquet.hadoop @@ -195,8 +192,6 @@ project(":iceberg-spark:iceberg-spark-extensions-${sparkMajorVersion}_${scalaVer testImplementation libs.avro.avro testImplementation libs.parquet.hadoop testImplementation libs.awaitility - // TODO: datafusion-comet Spark 4.1 support - testImplementation "org.apache.datafusion:comet-spark-spark4.0_2.13:${libs.versions.comet.get()}" testImplementation(testFixtures(project(':iceberg-parquet'))) // Required because we remove antlr plugin dependencies from the compile configuration, see note above diff --git a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/ParquetBatchReadConf.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/ParquetBatchReadConf.java index 442d728d4d69..b7cd441c9642 100644 --- a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/ParquetBatchReadConf.java +++ b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/ParquetBatchReadConf.java @@ -24,6 +24,4 @@ @Value.Immutable public interface ParquetBatchReadConf extends Serializable { int batchSize(); - - ParquetReaderType readerType(); } diff --git a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/ParquetReaderType.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/ParquetReaderType.java deleted file mode 100644 index d9742c048251..000000000000 --- a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/ParquetReaderType.java +++ /dev/null @@ -1,47 +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. - */ -package org.apache.iceberg.spark; - -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; - -/** Enumerates the types of Parquet readers. */ -public enum ParquetReaderType { - /** ICEBERG type utilizes the built-in Parquet reader. */ - ICEBERG, - - /** - * COMET type changes the Parquet reader to the Apache DataFusion Comet Parquet reader. Comet - * Parquet reader performs I/O and decompression in the JVM but decodes in native to improve - * performance. Additionally, Comet will convert Spark's physical plan into a native physical plan - * and execute this plan natively. - * - *

TODO: Implement {@link org.apache.comet.parquet.SupportsComet} in SparkScan to convert Spark - * physical plan to native physical plan for native execution. - */ - COMET; - - public static ParquetReaderType fromString(String typeAsString) { - Preconditions.checkArgument(typeAsString != null, "Parquet reader type is null"); - try { - return ParquetReaderType.valueOf(typeAsString.toUpperCase()); - } catch (IllegalArgumentException e) { - throw new IllegalArgumentException("Unknown parquet reader type: " + typeAsString); - } - } -} diff --git a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java index d5ec726f5b6f..36c34251c317 100644 --- a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java +++ b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java @@ -369,14 +369,6 @@ public boolean reportColumnStats() { .parse(); } - public ParquetReaderType parquetReaderType() { - return confParser - .enumConf(ParquetReaderType::fromString) - .sessionConf(SparkSQLProperties.PARQUET_READER_TYPE) - .defaultValue(SparkSQLProperties.PARQUET_READER_TYPE_DEFAULT) - .parse(); - } - public boolean identifierFieldsRely() { return confParser .booleanConf() diff --git a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java index 867370ea447e..161f09d53e2c 100644 --- a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java +++ b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java @@ -27,9 +27,6 @@ private SparkSQLProperties() {} // Controls whether vectorized reads are enabled public static final String VECTORIZATION_ENABLED = "spark.sql.iceberg.vectorization.enabled"; - // 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 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.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnReader.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnReader.java deleted file mode 100644 index 81b7d83a7077..000000000000 --- a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnReader.java +++ /dev/null @@ -1,140 +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. - */ -package org.apache.iceberg.spark.data.vectorized; - -import java.io.IOException; -import org.apache.comet.CometSchemaImporter; -import org.apache.comet.parquet.AbstractColumnReader; -import org.apache.comet.parquet.ColumnReader; -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.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.types.DataType; -import org.apache.spark.sql.types.Metadata; -import org.apache.spark.sql.types.StructField; -import org.apache.spark.sql.vectorized.ColumnVector; - -class CometColumnReader implements VectorizedReader { - // use the Comet default batch size - public static final int DEFAULT_BATCH_SIZE = 8192; - - private final ColumnDescriptor descriptor; - private final DataType sparkType; - - // The delegated ColumnReader from Comet side - private AbstractColumnReader delegate; - private boolean initialized = false; - private int batchSize = DEFAULT_BATCH_SIZE; - private CometSchemaImporter importer; - - CometColumnReader(DataType sparkType, ColumnDescriptor descriptor) { - this.sparkType = sparkType; - this.descriptor = descriptor; - } - - 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); - } - - public AbstractColumnReader delegate() { - return delegate; - } - - void setDelegate(AbstractColumnReader delegate) { - this.delegate = delegate; - } - - void setInitialized(boolean initialized) { - this.initialized = initialized; - } - - public int batchSize() { - return batchSize; - } - - /** - * This method is to initialized/reset the CometColumnReader. This needs to be called for each row - * group after readNextRowGroup, so a new dictionary encoding can be set for each of the new row - * groups. - */ - public void reset() { - if (importer != null) { - importer.close(); - } - - if (delegate != null) { - delegate.close(); - } - - this.importer = new CometSchemaImporter(new RootAllocator()); - this.delegate = Utils.getColumnReader(sparkType, descriptor, importer, batchSize, false, false); - this.initialized = true; - } - - public ColumnDescriptor descriptor() { - return descriptor; - } - - /** Returns the Spark data type for this column. */ - public DataType sparkType() { - return sparkType; - } - - /** - * Set the page reader to be 'pageReader'. - * - *

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 { - Preconditions.checkState(initialized, "Invalid state: 'reset' should be called first"); - ((ColumnReader) delegate).setPageReader(pageReader); - } - - @Override - public void close() { - // close resources on native side - if (importer != null) { - importer.close(); - } - - if (delegate != null) { - delegate.close(); - } - } - - @Override - public void setBatchSize(int size) { - this.batchSize = size; - } - - @Override - public ColumnVector read(ColumnVector reuse, int numRowsToRead) { - throw new UnsupportedOperationException("Not supported"); - } -} diff --git a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnarBatchReader.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnarBatchReader.java deleted file mode 100644 index 3d3e9aca24de..000000000000 --- a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnarBatchReader.java +++ /dev/null @@ -1,137 +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. - */ -package org.apache.iceberg.spark.data.vectorized; - -import java.io.IOException; -import java.io.UncheckedIOException; -import java.util.List; -import java.util.Map; -import org.apache.comet.parquet.AbstractColumnReader; -import org.apache.comet.parquet.BatchReader; -import org.apache.iceberg.Schema; -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.parquet.column.page.PageReadStore; -import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; -import org.apache.parquet.hadoop.metadata.ColumnPath; -import org.apache.spark.sql.vectorized.ColumnVector; -import org.apache.spark.sql.vectorized.ColumnarBatch; - -/** - * {@link VectorizedReader} that returns Spark's {@link ColumnarBatch} to support Spark's vectorized - * read path. The {@link ColumnarBatch} returned is created by passing in the Arrow vectors - * populated via delegated read calls to {@link CometColumnReader VectorReader(s)}. - */ -@SuppressWarnings("checkstyle:VisibilityModifier") -class CometColumnarBatchReader implements VectorizedReader { - - private final CometColumnReader[] readers; - - // The delegated BatchReader on the Comet side does the real work of loading a batch of rows. - // The Comet BatchReader contains an array of ColumnReader. There is no need to explicitly call - // ColumnReader.readBatch; instead, BatchReader.nextBatch will be called, which underneath calls - // ColumnReader.readBatch. The only exception is DeleteColumnReader, because at the time of - // 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; - - CometColumnarBatchReader(List> readers, Schema schema) { - this.readers = - readers.stream().map(CometColumnReader.class::cast).toArray(CometColumnReader[]::new); - - AbstractColumnReader[] abstractColumnReaders = new AbstractColumnReader[readers.size()]; - this.delegate = new BatchReader(abstractColumnReaders); - delegate.setSparkSchema(SparkSchemaUtil.convert(schema)); - } - - @Override - public void setRowGroupInfo( - PageReadStore pageStore, Map metaData) { - for (int i = 0; i < readers.length; i++) { - try { - if (!(readers[i] instanceof CometConstantColumnReader) - && !(readers[i] instanceof CometPositionColumnReader) - && !(readers[i] instanceof CometDeleteColumnReader)) { - readers[i].reset(); - readers[i].setPageReader(pageStore.getPageReader(readers[i].descriptor())); - } - } catch (IOException e) { - throw new UncheckedIOException("Failed to setRowGroupInfo for Comet vectorization", e); - } - } - - for (int i = 0; i < readers.length; i++) { - delegate.getColumnReaders()[i] = this.readers[i].delegate(); - } - } - - @Override - public final ColumnarBatch read(ColumnarBatch reuse, int numRowsToRead) { - return new ColumnBatchLoader(numRowsToRead).loadDataToColumnBatch(); - } - - @Override - public void setBatchSize(int batchSize) { - for (CometColumnReader reader : readers) { - if (reader != null) { - reader.setBatchSize(batchSize); - } - } - } - - @Override - public void close() { - for (CometColumnReader reader : readers) { - if (reader != null) { - reader.close(); - } - } - } - - private class ColumnBatchLoader { - private final int batchSize; - - ColumnBatchLoader(int numRowsToRead) { - Preconditions.checkArgument( - numRowsToRead > 0, "Invalid number of rows to read: %s", numRowsToRead); - this.batchSize = numRowsToRead; - } - - ColumnarBatch loadDataToColumnBatch() { - ColumnVector[] vectors = readDataToColumnVectors(); - - ColumnarBatch batch = new ColumnarBatch(vectors); - batch.setNumRows(batchSize); - return batch; - } - - ColumnVector[] readDataToColumnVectors() { - ColumnVector[] columnVectors = new ColumnVector[readers.length]; - // Fetch rows for all readers in the delegate - delegate.nextBatch(batchSize); - for (int i = 0; i < readers.length; i++) { - columnVectors[i] = readers[i].delegate().currentBatch(); - } - - return columnVectors; - } - } -} diff --git a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometConstantColumnReader.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometConstantColumnReader.java deleted file mode 100644 index 047c96314b13..000000000000 --- a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometConstantColumnReader.java +++ /dev/null @@ -1,65 +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. - */ -package org.apache.iceberg.spark.data.vectorized; - -import java.math.BigDecimal; -import java.nio.ByteBuffer; -import org.apache.comet.parquet.ConstantColumnReader; -import org.apache.iceberg.types.Types; -import org.apache.spark.sql.types.DataType; -import org.apache.spark.sql.types.DataTypes; -import org.apache.spark.sql.types.Decimal; -import org.apache.spark.sql.types.DecimalType; -import org.apache.spark.unsafe.types.UTF8String; - -class CometConstantColumnReader extends CometColumnReader { - - CometConstantColumnReader(T value, Types.NestedField field) { - 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)); - } - - @Override - public void setBatchSize(int batchSize) { - super.setBatchSize(batchSize); - delegate().setBatchSize(batchSize); - setInitialized(true); - } - - private Object convertToSparkValue(T value) { - DataType dataType = sparkType(); - // Match the value to Spark internal type if necessary - if (dataType == DataTypes.StringType && value instanceof String) { - // the internal type for StringType is UTF8String - return UTF8String.fromString((String) value); - } else if (dataType instanceof DecimalType && value instanceof BigDecimal) { - // the internal type for DecimalType is Decimal - return Decimal.apply((BigDecimal) value); - } else if (dataType == DataTypes.BinaryType && value instanceof ByteBuffer) { - // the internal type for DecimalType is byte[] - // Iceberg default value should always use HeapBufferBuffer, so calling ByteBuffer.array() - // should be safe. - return ((ByteBuffer) value).array(); - } else { - return value; - } - } -} diff --git a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeleteColumnReader.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeleteColumnReader.java deleted file mode 100644 index 26219014f777..000000000000 --- a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeleteColumnReader.java +++ /dev/null @@ -1,74 +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. - */ -package org.apache.iceberg.spark.data.vectorized; - -import org.apache.comet.parquet.MetadataColumnReader; -import org.apache.comet.parquet.Native; -import org.apache.comet.parquet.TypeUtil; -import org.apache.comet.vector.CometVector; -import org.apache.iceberg.types.Types; -import org.apache.spark.sql.types.DataTypes; -import org.apache.spark.sql.types.Metadata; -import org.apache.spark.sql.types.StructField; - -class CometDeleteColumnReader extends CometColumnReader { - CometDeleteColumnReader(Types.NestedField field) { - super(field); - setDelegate(new DeleteColumnReader()); - } - - @Override - public void setBatchSize(int batchSize) { - super.setBatchSize(batchSize); - delegate().setBatchSize(batchSize); - setInitialized(true); - } - - private static class DeleteColumnReader extends MetadataColumnReader { - private final CometDeletedColumnVector deletedVector; - - DeleteColumnReader() { - this(new boolean[0]); - } - - DeleteColumnReader(boolean[] isDeleted) { - super( - DataTypes.BooleanType, - TypeUtil.convertToParquet( - new StructField("_deleted", DataTypes.BooleanType, false, Metadata.empty())), - false /* useDecimal128 = false */, - false /* isConstant = false */); - this.deletedVector = new CometDeletedColumnVector(isDeleted); - } - - @Override - public void readBatch(int total) { - Native.resetBatch(nativeHandle); - // set isDeleted on the native side to be consumed by native execution - Native.setIsDeleted(nativeHandle, deletedVector.isDeleted()); - - super.readBatch(total); - } - - @Override - public CometVector currentBatch() { - return deletedVector; - } - } -} diff --git a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeletedColumnVector.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeletedColumnVector.java deleted file mode 100644 index 5817f2c20a4f..000000000000 --- a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometDeletedColumnVector.java +++ /dev/null @@ -1,155 +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. - */ -package org.apache.iceberg.spark.data.vectorized; - -import org.apache.comet.shaded.arrow.vector.ValueVector; -import org.apache.comet.vector.CometVector; -import org.apache.iceberg.spark.SparkSchemaUtil; -import org.apache.iceberg.types.Types; -import org.apache.spark.sql.types.Decimal; -import org.apache.spark.sql.vectorized.ColumnVector; -import org.apache.spark.sql.vectorized.ColumnarArray; -import org.apache.spark.sql.vectorized.ColumnarMap; -import org.apache.spark.unsafe.types.UTF8String; - -public class CometDeletedColumnVector extends CometVector implements UpdatableDeletedColumnVector { - private boolean[] isDeleted; - - public CometDeletedColumnVector(boolean[] isDeleted) { - super(SparkSchemaUtil.convert(Types.BooleanType.get()), false); - this.isDeleted = isDeleted; - } - - @Override - public void setValue(boolean[] deleted) { - this.isDeleted = deleted; - } - - boolean[] isDeleted() { - return isDeleted; - } - - @Override - public void setNumNulls(int numNulls) { - throw new UnsupportedOperationException("Not implemented"); - } - - @Override - public void setNumValues(int numValues) { - throw new UnsupportedOperationException("Not implemented"); - } - - @Override - public int numValues() { - throw new UnsupportedOperationException("Not implemented"); - } - - @Override - public ValueVector getValueVector() { - throw new UnsupportedOperationException("Not implemented"); - } - - @Override - public CometVector slice(int offset, int length) { - throw new UnsupportedOperationException("Not implemented"); - } - - @Override - public void close() {} - - @Override - public boolean hasNull() { - return false; - } - - @Override - public int numNulls() { - return 0; - } - - @Override - public boolean isNullAt(int rowId) { - return false; - } - - @Override - public boolean getBoolean(int rowId) { - return isDeleted[rowId]; - } - - @Override - public byte getByte(int rowId) { - throw new UnsupportedOperationException(); - } - - @Override - public short getShort(int rowId) { - throw new UnsupportedOperationException(); - } - - @Override - public int getInt(int rowId) { - throw new UnsupportedOperationException(); - } - - @Override - public long getLong(int rowId) { - throw new UnsupportedOperationException(); - } - - @Override - public float getFloat(int rowId) { - throw new UnsupportedOperationException(); - } - - @Override - public double getDouble(int rowId) { - throw new UnsupportedOperationException(); - } - - @Override - public ColumnarArray getArray(int rowId) { - throw new UnsupportedOperationException(); - } - - @Override - public ColumnarMap getMap(int ordinal) { - throw new UnsupportedOperationException(); - } - - @Override - public Decimal getDecimal(int rowId, int precision, int scale) { - throw new UnsupportedOperationException(); - } - - @Override - public UTF8String getUTF8String(int rowId) { - throw new UnsupportedOperationException(); - } - - @Override - public byte[] getBinary(int rowId) { - throw new UnsupportedOperationException(); - } - - @Override - public ColumnVector getChild(int ordinal) { - throw new UnsupportedOperationException(); - } -} diff --git a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometPositionColumnReader.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometPositionColumnReader.java deleted file mode 100644 index bcc0e514c28d..000000000000 --- a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometPositionColumnReader.java +++ /dev/null @@ -1,62 +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. - */ -package org.apache.iceberg.spark.data.vectorized; - -import org.apache.comet.parquet.MetadataColumnReader; -import org.apache.comet.parquet.Native; -import org.apache.iceberg.types.Types; -import org.apache.parquet.column.ColumnDescriptor; -import org.apache.spark.sql.types.DataTypes; - -class CometPositionColumnReader extends CometColumnReader { - CometPositionColumnReader(Types.NestedField field) { - super(field); - setDelegate(new PositionColumnReader(descriptor())); - } - - @Override - public void setBatchSize(int batchSize) { - super.setBatchSize(batchSize); - delegate().setBatchSize(batchSize); - setInitialized(true); - } - - private static class PositionColumnReader extends MetadataColumnReader { - /** The current position value of the column that are used to initialize this column reader. */ - private long position; - - PositionColumnReader(ColumnDescriptor descriptor) { - super( - DataTypes.LongType, - descriptor, - false /* useDecimal128 = false */, - false /* isConstant = false */); - } - - @Override - public void readBatch(int total) { - Native.resetBatch(nativeHandle); - // set position on the native side to be consumed by native execution - Native.setPosition(nativeHandle, position, total); - position += total; - - super.readBatch(total); - } - } -} diff --git a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometVectorizedReaderBuilder.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometVectorizedReaderBuilder.java deleted file mode 100644 index 779dc240d4f6..000000000000 --- a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometVectorizedReaderBuilder.java +++ /dev/null @@ -1,138 +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. - */ -package org.apache.iceberg.spark.data.vectorized; - -import java.util.List; -import java.util.Map; -import java.util.function.Function; -import java.util.stream.IntStream; -import org.apache.iceberg.MetadataColumns; -import org.apache.iceberg.Schema; -import org.apache.iceberg.parquet.TypeWithSchemaVisitor; -import org.apache.iceberg.parquet.VectorizedReader; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -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.GroupType; -import org.apache.parquet.schema.MessageType; -import org.apache.parquet.schema.PrimitiveType; -import org.apache.parquet.schema.Type; - -class CometVectorizedReaderBuilder extends TypeWithSchemaVisitor> { - - private final MessageType parquetSchema; - private final Schema icebergSchema; - private final Map idToConstant; - private final Function>, VectorizedReader> readerFactory; - - CometVectorizedReaderBuilder( - Schema expectedSchema, - MessageType parquetSchema, - Map idToConstant, - Function>, VectorizedReader> readerFactory) { - this.parquetSchema = parquetSchema; - this.icebergSchema = expectedSchema; - this.idToConstant = idToConstant; - this.readerFactory = readerFactory; - } - - @Override - public VectorizedReader message( - Types.StructType expected, MessageType message, List> fieldReaders) { - GroupType groupType = message.asGroupType(); - Map> readersById = Maps.newHashMap(); - List fields = groupType.getFields(); - - IntStream.range(0, fields.size()) - .filter(pos -> fields.get(pos).getId() != null) - .forEach(pos -> readersById.put(fields.get(pos).getId().intValue(), fieldReaders.get(pos))); - - List icebergFields = - expected != null ? expected.fields() : ImmutableList.of(); - - List> reorderedFields = - Lists.newArrayListWithExpectedSize(icebergFields.size()); - - for (Types.NestedField field : icebergFields) { - int id = field.fieldId(); - VectorizedReader reader = readersById.get(id); - if (idToConstant.containsKey(id)) { - CometConstantColumnReader constantReader = - new CometConstantColumnReader<>(idToConstant.get(id), field); - reorderedFields.add(constantReader); - } else if (id == MetadataColumns.ROW_POSITION.fieldId()) { - reorderedFields.add(new CometPositionColumnReader(field)); - } else if (id == MetadataColumns.IS_DELETED.fieldId()) { - CometColumnReader deleteReader = new CometDeleteColumnReader<>(field); - reorderedFields.add(deleteReader); - } else if (reader != null) { - reorderedFields.add(reader); - } else if (field.initialDefault() != null) { - CometColumnReader constantReader = - new CometConstantColumnReader<>(field.initialDefault(), field); - reorderedFields.add(constantReader); - } else if (field.isOptional()) { - CometColumnReader constantReader = new CometConstantColumnReader<>(null, field); - reorderedFields.add(constantReader); - } else { - throw new IllegalArgumentException( - String.format("Missing required field: %s", field.name())); - } - } - return vectorizedReader(reorderedFields); - } - - protected VectorizedReader vectorizedReader(List> reorderedFields) { - return readerFactory.apply(reorderedFields); - } - - @Override - public VectorizedReader struct( - Types.StructType expected, GroupType groupType, List> fieldReaders) { - if (expected != null) { - throw new UnsupportedOperationException( - "Vectorized reads are not supported yet for struct fields"); - } - return null; - } - - @Override - public VectorizedReader primitive( - org.apache.iceberg.types.Type.PrimitiveType expected, PrimitiveType primitive) { - - if (primitive.getId() == null) { - return null; - } - int parquetFieldId = primitive.getId().intValue(); - ColumnDescriptor desc = parquetSchema.getColumnDescription(currentPath()); - // Nested types not yet supported for vectorized reads - if (desc.getMaxRepetitionLevel() > 0) { - return null; - } - Types.NestedField icebergField = icebergSchema.findField(parquetFieldId); - if (icebergField == null) { - return null; - } - - return new CometColumnReader(SparkSchemaUtil.convert(icebergField.type()), desc); - } -} diff --git a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkParquetReaders.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkParquetReaders.java index 55f9fc1768a3..a80249ee7c36 100644 --- a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkParquetReaders.java +++ b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkParquetReaders.java @@ -30,8 +30,6 @@ import org.apache.iceberg.parquet.VectorizedReader; import org.apache.iceberg.spark.SparkUtil; import org.apache.parquet.schema.MessageType; -import org.apache.spark.sql.vectorized.ColumnVector; -import org.apache.spark.sql.vectorized.ColumnarBatch; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -77,26 +75,6 @@ public static ColumnarBatchReader buildReader( return buildReader(expectedSchema, fileSchema, idToConstant, ArrowAllocation.rootAllocator()); } - public static VectorizedReader buildCometReader( - Schema expectedSchema, MessageType fileSchema, Map idToConstant) { - return (CometColumnarBatchReader) - TypeWithSchemaVisitor.visit( - expectedSchema.asStruct(), - fileSchema, - new CometVectorizedReaderBuilder( - expectedSchema, - fileSchema, - idToConstant, - readers -> new CometColumnarBatchReader(readers, expectedSchema))); - } - - /** A subclass of ColumnarBatch to identify Comet readers. */ - public static class CometColumnarBatch extends ColumnarBatch { - public CometColumnarBatch(ColumnVector[] columns) { - super(columns); - } - } - // enables unsafe memory access to avoid costly checks to see if index is within bounds // as long as it is not configured explicitly (see BoundsChecking in Arrow) private static void enableUnsafeMemoryAccess() { diff --git a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java index 0acd8bc24476..2dd6a52c2607 100644 --- a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java +++ b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/BaseBatchReader.java @@ -35,11 +35,9 @@ import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.spark.OrcBatchReadConf; import org.apache.iceberg.spark.ParquetBatchReadConf; -import org.apache.iceberg.spark.ParquetReaderType; import org.apache.iceberg.spark.data.vectorized.ColumnVectorWithFilter; import org.apache.iceberg.spark.data.vectorized.ColumnarBatchUtil; import org.apache.iceberg.spark.data.vectorized.UpdatableDeletedColumnVector; -import org.apache.iceberg.spark.data.vectorized.VectorizedSparkParquetReaders; import org.apache.iceberg.util.Pair; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.vectorized.ColumnVector; @@ -70,10 +68,8 @@ protected CloseableIterable newBatchIterable( Expression residual, Map idToConstant, @Nonnull SparkDeleteFilter deleteFilter) { - Class readType = - useComet() ? VectorizedSparkParquetReaders.CometColumnarBatch.class : ColumnarBatch.class; ReadBuilder readBuilder = - FormatModelRegistry.readBuilder(format, readType, inputFile); + FormatModelRegistry.readBuilder(format, ColumnarBatch.class, inputFile); if (parquetConf != null) { readBuilder = readBuilder.recordsPerBatch(parquetConf.batchSize()); @@ -99,10 +95,6 @@ protected CloseableIterable newBatchIterable( return CloseableIterable.transform(iterable, new BatchDeleteFilter(deleteFilter)::filterBatch); } - private boolean useComet() { - return parquetConf != null && parquetConf.readerType() == ParquetReaderType.COMET; - } - @VisibleForTesting static class BatchDeleteFilter { private final DeleteFilter deletes; diff --git a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatch.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatch.java index a4d143fe9321..f4946c64044b 100644 --- a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatch.java +++ b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkBatch.java @@ -32,10 +32,8 @@ import org.apache.iceberg.spark.ImmutableParquetBatchReadConf; import org.apache.iceberg.spark.OrcBatchReadConf; import org.apache.iceberg.spark.ParquetBatchReadConf; -import org.apache.iceberg.spark.ParquetReaderType; import org.apache.iceberg.spark.SparkReadConf; import org.apache.iceberg.spark.SparkUtil; -import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.broadcast.Broadcast; @@ -119,11 +117,8 @@ private String[][] computePreferredLocations() { @Override public PartitionReaderFactory createReaderFactory() { - if (useCometBatchReads()) { - return new SparkColumnarReaderFactory(parquetBatchReadConf(ParquetReaderType.COMET)); - - } else if (useParquetBatchReads()) { - return new SparkColumnarReaderFactory(parquetBatchReadConf(ParquetReaderType.ICEBERG)); + if (useParquetBatchReads()) { + return new SparkColumnarReaderFactory(parquetBatchReadConf()); } else if (useOrcBatchReads()) { return new SparkColumnarReaderFactory(orcBatchReadConf()); @@ -133,11 +128,8 @@ public PartitionReaderFactory createReaderFactory() { } } - private ParquetBatchReadConf parquetBatchReadConf(ParquetReaderType readerType) { - return ImmutableParquetBatchReadConf.builder() - .batchSize(readConf.parquetBatchSize()) - .readerType(readerType) - .build(); + private ParquetBatchReadConf parquetBatchReadConf() { + return ImmutableParquetBatchReadConf.builder().batchSize(readConf.parquetBatchSize()).build(); } private OrcBatchReadConf orcBatchReadConf() { @@ -172,20 +164,6 @@ private boolean supportsParquetBatchReads(Types.NestedField field) { return field.type().isPrimitiveType() || MetadataColumns.isMetadataColumn(field.fieldId()); } - private boolean useCometBatchReads() { - return readConf.parquetVectorizationEnabled() - && readConf.parquetReaderType() == ParquetReaderType.COMET - && projection.columns().stream().allMatch(this::supportsCometBatchReads) - && taskGroups.stream().allMatch(this::supportsParquetBatchReads); - } - - private boolean supportsCometBatchReads(Types.NestedField field) { - return field.type().isPrimitiveType() - && !field.type().typeId().equals(Type.TypeID.UUID) - && field.fieldId() != MetadataColumns.ROW_ID.fieldId() - && field.fieldId() != MetadataColumns.LAST_UPDATED_SEQUENCE_NUMBER.fieldId(); - } - // conditions for using ORC batch reads: // - ORC vectorization is enabled // - all tasks are of type FileScanTask and read only ORC files with no delete files diff --git a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkFormatModels.java b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkFormatModels.java index 677f2e950b44..a1e8a82b4d80 100644 --- a/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkFormatModels.java +++ b/spark/v4.1/spark/src/main/java/org/apache/iceberg/spark/source/SparkFormatModels.java @@ -60,14 +60,6 @@ public static void register() { VectorizedSparkParquetReaders.buildReader( icebergSchema, fileSchema, idToConstant))); - FormatModelRegistry.register( - ParquetFormatModel.create( - VectorizedSparkParquetReaders.CometColumnarBatch.class, - StructType.class, - (icebergSchema, fileSchema, engineSchema, idToConstant) -> - VectorizedSparkParquetReaders.buildCometReader( - icebergSchema, fileSchema, idToConstant))); - FormatModelRegistry.register( ORCFormatModel.create( InternalRow.class, diff --git a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetCometVectorizedScan.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetCometVectorizedScan.java deleted file mode 100644 index 5e1d88f8bcd4..000000000000 --- a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestParquetCometVectorizedScan.java +++ /dev/null @@ -1,33 +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. - */ -package org.apache.iceberg.spark.source; - -import org.junit.jupiter.api.BeforeAll; - -public class TestParquetCometVectorizedScan extends TestParquetScan { - @BeforeAll - public static void setComet() { - ScanTestBase.spark.conf().set("spark.sql.iceberg.parquet.reader-type", "COMET"); - } - - @Override - protected boolean vectorized() { - return true; - } -} From a91365f223b2a164353661cf6a0957ad827864d3 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Wed, 18 Mar 2026 09:43:42 -0600 Subject: [PATCH 2/3] spotless --- .../spark/data/vectorized/VectorizedSparkParquetReaders.java | 1 - .../main/java/org/apache/iceberg/spark/source/SparkBatch.java | 4 +--- .../spark/data/vectorized/VectorizedSparkParquetReaders.java | 1 - .../main/java/org/apache/iceberg/spark/source/SparkBatch.java | 4 +--- .../spark/data/vectorized/VectorizedSparkParquetReaders.java | 1 - .../main/java/org/apache/iceberg/spark/source/SparkBatch.java | 4 +--- 6 files changed, 3 insertions(+), 12 deletions(-) diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkParquetReaders.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkParquetReaders.java index e959bbc2d542..a80249ee7c36 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkParquetReaders.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkParquetReaders.java @@ -30,7 +30,6 @@ import org.apache.iceberg.parquet.VectorizedReader; import org.apache.iceberg.spark.SparkUtil; import org.apache.parquet.schema.MessageType; -import org.apache.spark.sql.vectorized.ColumnarBatch; import org.slf4j.Logger; import org.slf4j.LoggerFactory; 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 6bbf3ca3bab2..261d5fa227a5 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 @@ -132,9 +132,7 @@ public PartitionReaderFactory createReaderFactory() { } private ParquetBatchReadConf parquetBatchReadConf() { - return ImmutableParquetBatchReadConf.builder() - .batchSize(readConf.parquetBatchSize()) - .build(); + return ImmutableParquetBatchReadConf.builder().batchSize(readConf.parquetBatchSize()).build(); } private OrcBatchReadConf orcBatchReadConf() { diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkParquetReaders.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkParquetReaders.java index e959bbc2d542..a80249ee7c36 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkParquetReaders.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkParquetReaders.java @@ -30,7 +30,6 @@ import org.apache.iceberg.parquet.VectorizedReader; import org.apache.iceberg.spark.SparkUtil; import org.apache.parquet.schema.MessageType; -import org.apache.spark.sql.vectorized.ColumnarBatch; import org.slf4j.Logger; import org.slf4j.LoggerFactory; 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 6bbf3ca3bab2..261d5fa227a5 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 @@ -132,9 +132,7 @@ public PartitionReaderFactory createReaderFactory() { } private ParquetBatchReadConf parquetBatchReadConf() { - return ImmutableParquetBatchReadConf.builder() - .batchSize(readConf.parquetBatchSize()) - .build(); + return ImmutableParquetBatchReadConf.builder().batchSize(readConf.parquetBatchSize()).build(); } private OrcBatchReadConf orcBatchReadConf() { diff --git a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkParquetReaders.java b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkParquetReaders.java index e959bbc2d542..a80249ee7c36 100644 --- a/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkParquetReaders.java +++ b/spark/v4.0/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkParquetReaders.java @@ -30,7 +30,6 @@ import org.apache.iceberg.parquet.VectorizedReader; import org.apache.iceberg.spark.SparkUtil; import org.apache.parquet.schema.MessageType; -import org.apache.spark.sql.vectorized.ColumnarBatch; import org.slf4j.Logger; import org.slf4j.LoggerFactory; 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 6bbf3ca3bab2..261d5fa227a5 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 @@ -132,9 +132,7 @@ public PartitionReaderFactory createReaderFactory() { } private ParquetBatchReadConf parquetBatchReadConf() { - return ImmutableParquetBatchReadConf.builder() - .batchSize(readConf.parquetBatchSize()) - .build(); + return ImmutableParquetBatchReadConf.builder().batchSize(readConf.parquetBatchSize()).build(); } private OrcBatchReadConf orcBatchReadConf() { From 3d850381005f2849f5a2ac1fd24f8365ecf0eec7 Mon Sep 17 00:00:00 2001 From: Andy Grove Date: Wed, 18 Mar 2026 09:54:11 -0600 Subject: [PATCH 3/3] update test --- .../iceberg/spark/source/TestSparkReaderDeletes.java | 7 +------ .../iceberg/spark/source/TestSparkReaderDeletes.java | 7 +------ .../iceberg/spark/source/TestSparkReaderDeletes.java | 7 +------ 3 files changed, 3 insertions(+), 18 deletions(-) diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java index 076c476ac8ec..e16c9c2176eb 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java @@ -73,7 +73,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.spark.ImmutableParquetBatchReadConf; import org.apache.iceberg.spark.ParquetBatchReadConf; -import org.apache.iceberg.spark.ParquetReaderType; import org.apache.iceberg.spark.SparkSchemaUtil; import org.apache.iceberg.spark.SparkStructLike; import org.apache.iceberg.spark.data.RandomData; @@ -669,11 +668,7 @@ public void testEqualityDeleteWithDifferentScanAndDeleteColumns() throws IOExcep TableProperties.SPLIT_LOOKBACK_DEFAULT, TableProperties.SPLIT_OPEN_FILE_COST_DEFAULT); - ParquetBatchReadConf conf = - ImmutableParquetBatchReadConf.builder() - .batchSize(7) - .readerType(ParquetReaderType.ICEBERG) - .build(); + ParquetBatchReadConf conf = ImmutableParquetBatchReadConf.builder().batchSize(7).build(); for (CombinedScanTask task : tasks) { try (BatchDataReader reader = diff --git a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java index 076c476ac8ec..e16c9c2176eb 100644 --- a/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java +++ b/spark/v4.0/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java @@ -73,7 +73,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.spark.ImmutableParquetBatchReadConf; import org.apache.iceberg.spark.ParquetBatchReadConf; -import org.apache.iceberg.spark.ParquetReaderType; import org.apache.iceberg.spark.SparkSchemaUtil; import org.apache.iceberg.spark.SparkStructLike; import org.apache.iceberg.spark.data.RandomData; @@ -669,11 +668,7 @@ public void testEqualityDeleteWithDifferentScanAndDeleteColumns() throws IOExcep TableProperties.SPLIT_LOOKBACK_DEFAULT, TableProperties.SPLIT_OPEN_FILE_COST_DEFAULT); - ParquetBatchReadConf conf = - ImmutableParquetBatchReadConf.builder() - .batchSize(7) - .readerType(ParquetReaderType.ICEBERG) - .build(); + ParquetBatchReadConf conf = ImmutableParquetBatchReadConf.builder().batchSize(7).build(); for (CombinedScanTask task : tasks) { try (BatchDataReader reader = diff --git a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java index eea7ac6e5948..a58fc8bdb955 100644 --- a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java +++ b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java @@ -73,7 +73,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.spark.ImmutableParquetBatchReadConf; import org.apache.iceberg.spark.ParquetBatchReadConf; -import org.apache.iceberg.spark.ParquetReaderType; import org.apache.iceberg.spark.SparkSchemaUtil; import org.apache.iceberg.spark.SparkStructLike; import org.apache.iceberg.spark.data.RandomData; @@ -672,11 +671,7 @@ public void testEqualityDeleteWithDifferentScanAndDeleteColumns() throws IOExcep TableProperties.SPLIT_LOOKBACK_DEFAULT, TableProperties.SPLIT_OPEN_FILE_COST_DEFAULT); - ParquetBatchReadConf conf = - ImmutableParquetBatchReadConf.builder() - .batchSize(7) - .readerType(ParquetReaderType.ICEBERG) - .build(); + ParquetBatchReadConf conf = ImmutableParquetBatchReadConf.builder().batchSize(7).build(); for (CombinedScanTask task : tasks) { try (BatchDataReader reader =