parthchandra commented on code in PR #9841: URL: https://github.com/apache/iceberg/pull/9841#discussion_r1934889396
########## spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnReader.java: ########## @@ -0,0 +1,146 @@ +/* + * 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.util.Map; +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.c.CometSchemaImporter; +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.PageReadStore; +import org.apache.parquet.column.page.PageReader; +import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; +import org.apache.parquet.hadoop.metadata.ColumnPath; +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<ColumnVector> { + // use the Comet default batch size + public static final int DEFAULT_BATCH_SIZE = 8192; Review Comment: +1. In some follow up PR we can try to pass in the configured value (not just the default). ########## spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnReader.java: ########## @@ -0,0 +1,149 @@ +/* + * 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.util.Map; +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.c.CometSchemaImporter; +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.PageReadStore; +import org.apache.parquet.column.page.PageReader; +import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; +import org.apache.parquet.hadoop.metadata.ColumnPath; +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.Metadata; +import org.apache.spark.sql.types.StructField; + +class CometColumnReader implements VectorizedReader<CometVector> { + public static final int DEFAULT_BATCH_SIZE = 5000; + + private final DataType sparkType; + // the delegated column reader from Comet side + private AbstractColumnReader delegate; + private final CometVector vector; + private final ColumnDescriptor descriptor; + private boolean initialized = false; + private int batchSize = DEFAULT_BATCH_SIZE; Review Comment: +1 ########## spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnarBatchReader.java: ########## @@ -0,0 +1,199 @@ +/* + * 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.data.DeleteFilter; +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.util.Pair; +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.catalyst.InternalRow; +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<ColumnarBatch> { + + private final CometColumnReader[] readers; + private final boolean hasIsDeletedColumn; + // The delegated batch reader on Comet side + private final BatchReader delegate; Review Comment: Thank you for adding the comment ! ########## spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/CometColumnReader.java: ########## @@ -0,0 +1,146 @@ +/* + * 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.util.Map; +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.c.CometSchemaImporter; +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.PageReadStore; +import org.apache.parquet.column.page.PageReader; +import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; +import org.apache.parquet.hadoop.metadata.ColumnPath; +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<ColumnVector> { + // 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 (delegate != null) { Review Comment: If `delegate` is not null here `importer` will also be not null. I think the importer can be reused, so there is no need to close it here, but we overwrite it in the next line, so either we should reuse it, or close it. (Closing it is safer) -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org