You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2021/03/01 15:12:50 UTC

[GitHub] [iceberg] mayursrivastava opened a new pull request #2286: Add Arrow vectorized reader

mayursrivastava opened a new pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286


   This is my first Iceberg pull request - I'm not fully aware of the contributing conventions of this repo, so let me know if any changes are needed in the pr. 
   
   I've refactored some code from the Spark vectorized reader and added an ArrowReader which is a vectorized reader in Iceberg core. This is a follow up on the following discussion on the mailing list: https://lists.apache.org/thread.html/r4162810433a81a751ce15d5d82b98c5be698790176dc686c93fc3b81%40%3Cdev.iceberg.apache.org%3E 
   
   About the ArrowReader:
   
   1. I’ve put the ArrowReader in the iceberg-data module because it needed to access the Iceberg table scan. Let me know if the reader needs to be moved.
   
   2. I had to make a dependency addition of ‘iceberg-arrow’ for the iceberg-data module. Specially for the ArrowReaderTest, I had to add the following. Let me know if there is a better way for doing this.
       compileOnly("org.apache.arrow:arrow-vector") {
         exclude group: 'io.netty', module: 'netty-buffer'
         exclude group: 'com.google.code.findbugs', module: 'jsr305'
       }
   
   3. Most of the code in ArrowReader is taken from the spark vectorized reader. I think there is a potential to share ‘BaseArrowFileScanTaskReader’ in both versions, but I did not attempt to do it yet.
   
   4. ArrowReader returns an iterator of VectorSchemaRoot and the behavior is explained in the Javadoc.
   
   5. Some small changes were needed in IcebergGenerics to expose the table scan object and VectorizedArrowReader to allocate different timestamp Arrow vectors based on with/without timezone.
   
   6. All prepush gradle tests pass except one which is still running (and it seems very slow - TestFlinkIcebergSink).
   
   7. I've not performed any performance tests with the implementation yet. I'm planning to do so this week.
   
   Following are some limitations/questions for this implementation:
   
   1. The arrow vector type is coupled with the physical data type in the parquet file: When column data contains a constant value, the column is dictionary encoded and the returned Arrow type is int32 irrespective of the Iceberg data type. I think that the Arrow vector type should be consistent with the logical Iceberg data type (and not change due to the physical data type). There is a test ArrowReaderTest.testReadAllWithConstantRecords() that is currently ignored.
   
   2. Type promotion does not work: In the ArrowReaderTest, the data for column ‘int_promotion’ was written as int, and then type was promoted from int to long, but the Arrow reader still returns IntVector. I think that the Arrow vector type should be consistent with the promoted logical Iceberg data type.
   
   3. Data type limitations:
     a. Types not tested: UUID, FixedType, DecimalType. In the ArrowReaderTest, the parquet write was failing for these data types (due to a null pointer exception in ParquetMetadataConverter.addRowGroup: columnMetadata.getStatistics() was null). Are there unit tests with these types that write to parquet?
     b. Types not supported: TimeType, ListType, MapType, StructType. What is the path to add Arrow support for these data types?
   


----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] mayursrivastava commented on a change in pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
mayursrivastava commented on a change in pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#discussion_r680986832



##########
File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowReader.java
##########
@@ -0,0 +1,325 @@
+/*
+ * 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.arrow.vectorized;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import java.util.stream.StreamSupport;
+import org.apache.arrow.vector.NullCheckingForGet;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.arrow.vector.types.Types.MinorType;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.encryption.EncryptedFiles;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.CloseableIterator;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.types.Types;
+import org.apache.parquet.schema.MessageType;
+
+/**
+ * Vectorized reader that returns an iterator of {@link ColumnarBatch}.
+ * See {@link #open(CloseableIterable)} ()} to learn about the
+ * behavior of the iterator.
+ *
+ * <p>The following Iceberg data types are supported and have been tested:
+ * <ul>
+ *     <li>Iceberg: {@link Types.BooleanType}, Arrow: {@link MinorType#BIT}</li>
+ *     <li>Iceberg: {@link Types.IntegerType}, Arrow: {@link MinorType#INT}</li>
+ *     <li>Iceberg: {@link Types.LongType}, Arrow: {@link MinorType#BIGINT}</li>
+ *     <li>Iceberg: {@link Types.FloatType}, Arrow: {@link MinorType#FLOAT4}</li>
+ *     <li>Iceberg: {@link Types.DoubleType}, Arrow: {@link MinorType#FLOAT8}</li>
+ *     <li>Iceberg: {@link Types.StringType}, Arrow: {@link MinorType#VARCHAR}</li>
+ *     <li>Iceberg: {@link Types.TimestampType} (both with and without timezone),
+ *         Arrow: {@link MinorType#TIMEMICRO}</li>
+ *     <li>Iceberg: {@link Types.BinaryType}, Arrow: {@link MinorType#VARBINARY}</li>
+ *     <li>Iceberg: {@link Types.DateType}, Arrow: {@link MinorType#DATEDAY}</li>
+ * </ul>
+ *
+ * <p>Features that don't work in this implementation:
+ * <ul>
+ *     <li>Type promotion: In case of type promotion, the Arrow vector corresponding to
+ *     the data type in the parquet file is returned instead of the data type in the latest schema.
+ *     See https://github.com/apache/iceberg/issues/2483.</li>
+ *     <li>Columns with constant values are physically encoded as a dictionary. The Arrow vector
+ *     type is int32 instead of the type as per the schema.
+ *     See https://github.com/apache/iceberg/issues/2484.</li>
+ *     <li>Data types: {@link Types.TimeType}, {@link Types.ListType}, {@link Types.MapType},
+ *     {@link Types.StructType}, {@link Types.UUIDType}, {@link Types.FixedType} and
+ *     {@link Types.DecimalType}
+ *     See https://github.com/apache/iceberg/issues/2485 and https://github.com/apache/iceberg/issues/2486.</li>
+ *     <li>Iceberg v2 spec is not supported.
+ *     See https://github.com/apache/iceberg/issues/2487.</li>

Review comment:
       Hi @rdblue, apologies for the delay in getting back to it. How can we check whether the tableScan or table is following the v2 spec? 




-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] mayursrivastava commented on a change in pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
mayursrivastava commented on a change in pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#discussion_r638324681



##########
File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowReader.java
##########
@@ -0,0 +1,325 @@
+/*
+ * 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.arrow.vectorized;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import java.util.stream.StreamSupport;
+import org.apache.arrow.vector.NullCheckingForGet;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.arrow.vector.types.Types.MinorType;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.encryption.EncryptedFiles;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.CloseableIterator;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.types.Types;
+import org.apache.parquet.schema.MessageType;
+
+/**
+ * Vectorized reader that returns an iterator of {@link ColumnarBatch}.
+ * See {@link #open(CloseableIterable)} ()} to learn about the
+ * behavior of the iterator.
+ *
+ * <p>The following Iceberg data types are supported and have been tested:
+ * <ul>
+ *     <li>Iceberg: {@link Types.BooleanType}, Arrow: {@link MinorType#BIT}</li>
+ *     <li>Iceberg: {@link Types.IntegerType}, Arrow: {@link MinorType#INT}</li>
+ *     <li>Iceberg: {@link Types.LongType}, Arrow: {@link MinorType#BIGINT}</li>
+ *     <li>Iceberg: {@link Types.FloatType}, Arrow: {@link MinorType#FLOAT4}</li>
+ *     <li>Iceberg: {@link Types.DoubleType}, Arrow: {@link MinorType#FLOAT8}</li>
+ *     <li>Iceberg: {@link Types.StringType}, Arrow: {@link MinorType#VARCHAR}</li>
+ *     <li>Iceberg: {@link Types.TimestampType} (both with and without timezone),
+ *         Arrow: {@link MinorType#TIMEMICRO}</li>
+ *     <li>Iceberg: {@link Types.BinaryType}, Arrow: {@link MinorType#VARBINARY}</li>
+ *     <li>Iceberg: {@link Types.DateType}, Arrow: {@link MinorType#DATEDAY}</li>
+ * </ul>
+ *
+ * <p>Features that don't work in this implementation:
+ * <ul>
+ *     <li>Type promotion: In case of type promotion, the Arrow vector corresponding to
+ *     the data type in the parquet file is returned instead of the data type in the latest schema.
+ *     See https://github.com/apache/iceberg/issues/2483.</li>
+ *     <li>Columns with constant values are physically encoded as a dictionary. The Arrow vector
+ *     type is int32 instead of the type as per the schema.
+ *     See https://github.com/apache/iceberg/issues/2484.</li>
+ *     <li>Data types: {@link Types.TimeType}, {@link Types.ListType}, {@link Types.MapType},
+ *     {@link Types.StructType}, {@link Types.UUIDType}, {@link Types.FixedType} and
+ *     {@link Types.DecimalType}
+ *     See https://github.com/apache/iceberg/issues/2485 and https://github.com/apache/iceberg/issues/2486.</li>
+ *     <li>Iceberg v2 spec is not supported.
+ *     See https://github.com/apache/iceberg/issues/2487.</li>
+ * </ul>
+ */
+public class ArrowReader extends CloseableGroup {
+
+  private final Schema schema;
+  private final FileIO io;
+  private final EncryptionManager encryption;
+  private final int batchSize;
+  private final boolean reuseContainers;
+
+  /**
+   * Create a new instance of the reader.
+   *
+   * @param scan the table scan object.
+   * @param batchSize the maximum number of rows per Arrow batch.
+   * @param reuseContainers whether to reuse Arrow vectors when iterating through the data.
+   *                        If set to {@code false}, every {@link Iterator#next()} call creates
+   *                        new instances of Arrow vectors.
+   *                        If set to {@code true}, the Arrow vectors in the previous
+   *                        {@link Iterator#next()} may be reused for the data returned
+   *                        in the current {@link Iterator#next()}.
+   *                        This option avoids allocating memory again and again.
+   *                        Irrespective of the value of {@code reuseContainers}, the Arrow vectors
+   *                        in the previous {@link Iterator#next()} call are closed before creating
+   *                        new instances if the current {@link Iterator#next()}.
+   */
+  public ArrowReader(TableScan scan, int batchSize, boolean reuseContainers) {
+    this.schema = scan.schema();
+    this.io = scan.table().io();
+    this.encryption = scan.table().encryption();
+    this.batchSize = batchSize;
+    // start planning tasks in the background
+    this.reuseContainers = reuseContainers;
+  }
+
+  /**
+   * Returns a new iterator of {@link ColumnarBatch} objects.
+   * <p>
+   * Note that the reader owns the {@link ColumnarBatch} objects and takes care of closing them.
+   * The caller should not hold onto a {@link ColumnarBatch} or try to close them.
+   *
+   * <p>If {@code reuseContainers} is {@code false}, the Arrow vectors in the
+   * previous {@link ColumnarBatch} are closed before returning the next {@link ColumnarBatch} object.
+   * This implies that the caller should either use the {@link ColumnarBatch} or transfer the ownership of
+   * {@link ColumnarBatch} before getting the next {@link ColumnarBatch}.
+   *
+   * <p>If {@code reuseContainers} is {@code true}, the Arrow vectors in the
+   * previous {@link ColumnarBatch} may be reused for the next {@link ColumnarBatch}.
+   * This implies that the caller should either use the {@link ColumnarBatch} or deep copy the
+   * {@link ColumnarBatch} before getting the next {@link ColumnarBatch}.
+   */
+  public CloseableIterator<ColumnarBatch> open(CloseableIterable<CombinedScanTask> tasks) {
+    CloseableIterator<ColumnarBatch> itr = new VectorizedCombinedScanIterator(
+        tasks,
+        schema,
+        null,
+        io,
+        encryption,
+        true,
+        batchSize,
+        reuseContainers
+    );
+    addCloseable(itr);
+    return itr;
+  }
+
+  @Override
+  public void close() throws IOException {
+    super.close(); // close data files
+  }
+
+  /**
+   * Reads the data file and returns an iterator of {@link VectorSchemaRoot}.
+   * Only Parquet data file format is supported.
+   */
+  private static final class VectorizedCombinedScanIterator implements CloseableIterator<ColumnarBatch> {
+
+    private final List<FileScanTask> fileTasks;
+    private final Iterator<FileScanTask> fileItr;
+    private final Map<String, InputFile> inputFiles;
+    private final Schema expectedSchema;
+    private final String nameMapping;
+    private final boolean caseSensitive;
+    private final int batchSize;
+    private final boolean reuseContainers;
+    private CloseableIterator<ColumnarBatch> currentIterator;
+    private ColumnarBatch current;
+    private FileScanTask currentTask;
+
+    /**
+     * Create a new instance.
+     *
+     * @param tasks             Combined file scan tasks.
+     * @param expectedSchema    Read schema. The returned data will have this schema.
+     * @param nameMapping       Mapping from external schema names to Iceberg type IDs.
+     * @param io                File I/O.
+     * @param encryptionManager Encryption manager.
+     * @param caseSensitive     If {@code true}, column names are case sensitive.
+     *                          If {@code false}, column names are not case sensitive.
+     * @param batchSize         Batch size in number of rows. Each Arrow batch contains
+     *                          a maximum of {@code batchSize} rows.
+     * @param reuseContainers   If set to {@code false}, every {@link Iterator#next()} call creates
+     *                          new instances of Arrow vectors.
+     *                          If set to {@code true}, the Arrow vectors in the previous
+     *                          {@link Iterator#next()} may be reused for the data returned
+     *                          in the current {@link Iterator#next()}.
+     *                          This option avoids allocating memory again and again.
+     *                          Irrespective of the value of {@code reuseContainers}, the Arrow vectors
+     *                          in the previous {@link Iterator#next()} call are closed before creating
+     *                          new instances if the current {@link Iterator#next()}.
+     */
+    VectorizedCombinedScanIterator(
+        CloseableIterable<CombinedScanTask> tasks,
+        Schema expectedSchema,
+        String nameMapping,
+        FileIO io,
+        EncryptionManager encryptionManager,
+        boolean caseSensitive,
+        int batchSize,
+        boolean reuseContainers) {
+      this.fileTasks = StreamSupport.stream(tasks.spliterator(), false)
+          .map(CombinedScanTask::files)
+          .flatMap(Collection::stream)
+          .collect(Collectors.toList());
+      this.fileItr = fileTasks.iterator();
+      this.inputFiles = Collections.unmodifiableMap(fileTasks.stream()
+          .flatMap(fileScanTask -> Stream.concat(Stream.of(fileScanTask.file()), fileScanTask.deletes().stream()))
+          .map(file -> EncryptedFiles.encryptedInput(io.newInputFile(file.path().toString()), file.keyMetadata()))
+          .map(encryptionManager::decrypt)

Review comment:
       Thanks for looking into this pr @rdblue.
   I'll look into the suggestion and get back to you. A new pr will be cleaner for the suggested changes. Is that fine?




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] mayursrivastava commented on a change in pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
mayursrivastava commented on a change in pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#discussion_r617715498



##########
File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowReader.java
##########
@@ -0,0 +1,331 @@
+/*
+ * 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.arrow.vectorized;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import java.util.stream.StreamSupport;
+import org.apache.arrow.vector.NullCheckingForGet;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.arrow.vector.types.Types.MinorType;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.encryption.EncryptedFiles;
+import org.apache.iceberg.encryption.EncryptedInputFile;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.CloseableIterator;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.types.Types;
+import org.apache.parquet.schema.MessageType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Vectorized reader that returns an iterator of {@link ColumnarBatch}.
+ * See {@link #open(CloseableIterable)} ()} to learn about the
+ * behavior of the iterator.
+ *
+ * <p>The following Iceberg data types are supported and have been tested:
+ * <ul>
+ *     <li>Iceberg: {@link Types.BooleanType}, Arrow: {@link MinorType#BIT}</li>
+ *     <li>Iceberg: {@link Types.IntegerType}, Arrow: {@link MinorType#INT}</li>
+ *     <li>Iceberg: {@link Types.LongType}, Arrow: {@link MinorType#BIGINT}</li>
+ *     <li>Iceberg: {@link Types.FloatType}, Arrow: {@link MinorType#FLOAT4}</li>
+ *     <li>Iceberg: {@link Types.DoubleType}, Arrow: {@link MinorType#FLOAT8}</li>
+ *     <li>Iceberg: {@link Types.StringType}, Arrow: {@link MinorType#VARCHAR}</li>
+ *     <li>Iceberg: {@link Types.TimestampType} (both with and without timezone),
+ *         Arrow: {@link MinorType#TIMEMICRO}</li>
+ *     <li>Iceberg: {@link Types.BinaryType}, Arrow: {@link MinorType#VARBINARY}</li>
+ *     <li>Iceberg: {@link Types.DateType}, Arrow: {@link MinorType#DATEDAY}</li>
+ * </ul>
+ *
+ * <p>Features that don't work in this implementation:
+ * <ul>
+ *     <li>Type promotion: In case of type promotion, the Arrow vector corresponding to
+ *     the data type in the parquet file is returned instead of the data type in the latest schema.
+ *     See https://github.com/apache/iceberg/issues/2483.</li>
+ *     <li>Columns with constant values are physically encoded as a dictionary. The Arrow vector
+ *     type is int32 instead of the type as per the schema.
+ *     See https://github.com/apache/iceberg/issues/2484.</li>
+ *     <li>Data types: {@link Types.TimeType}, {@link Types.ListType}, {@link Types.MapType},
+ *     {@link Types.StructType}, {@link Types.UUIDType}, {@link Types.FixedType} and
+ *     {@link Types.DecimalType}
+ *     See https://github.com/apache/iceberg/issues/2485 and https://github.com/apache/iceberg/issues/2486.</li>
+ *     <li>Iceberg v2 spec is not supported.
+ *     See https://github.com/apache/iceberg/issues/2487.</li>
+ * </ul>
+ */
+public class ArrowReader extends CloseableGroup {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ArrowReader.class);
+
+  private final Schema schema;
+  private final FileIO io;
+  private final EncryptionManager encryption;
+  private final int batchSize;
+  private final boolean reuseContainers;
+
+  /**
+   * Create a new instance of the reader.
+   *
+   * @param scan the table scan object.
+   * @param batchSize the maximum number of rows per Arrow batch.
+   * @param reuseContainers whether to reuse Arrow vectors when iterating through the data.
+   *                        If set to {@code false}, every {@link Iterator#next()} call creates
+   *                        new instances of Arrow vectors.
+   *                        If set to {@code true}, the Arrow vectors in the previous
+   *                        {@link Iterator#next()} may be reused for the data returned
+   *                        in the current {@link Iterator#next()}.
+   *                        This option avoids allocating memory again and again.
+   *                        Irrespective of the value of {@code reuseContainers}, the Arrow vectors
+   *                        in the previous {@link Iterator#next()} call are closed before creating
+   *                        new instances if the current {@link Iterator#next()}.
+   */
+  public ArrowReader(TableScan scan, int batchSize, boolean reuseContainers) {
+    this.schema = scan.schema();
+    this.io = scan.table().io();
+    this.encryption = scan.table().encryption();
+    this.batchSize = batchSize;
+    // start planning tasks in the background
+    this.reuseContainers = reuseContainers;
+  }
+
+  /**
+   * Returns a new iterator of {@link ColumnarBatch} objects.
+   * <p>
+   * Note that the reader owns the {@link ColumnarBatch} objects and takes care of closing them.
+   * The caller should not hold onto a {@link ColumnarBatch} or try to close them.
+   *
+   * <p>If {@code reuseContainers} is {@code false}, the Arrow vectors in the
+   * previous {@link ColumnarBatch} are closed before returning the next {@link ColumnarBatch} object.
+   * This implies that the caller should either use the {@link ColumnarBatch} or transfer the ownership of
+   * {@link ColumnarBatch} before getting the next {@link ColumnarBatch}.
+   *
+   * <p>If {@code reuseContainers} is {@code true}, the Arrow vectors in the
+   * previous {@link ColumnarBatch} may be reused for the next {@link ColumnarBatch}.
+   * This implies that the caller should either use the {@link ColumnarBatch} or deep copy the
+   * {@link ColumnarBatch} before getting the next {@link ColumnarBatch}.
+   */
+  public CloseableIterator<ColumnarBatch> open(CloseableIterable<CombinedScanTask> tasks) {
+    CloseableIterator<ColumnarBatch> itr = new VectorizedCombinedScanIterator(
+        tasks,
+        schema,
+        null,
+        io,
+        encryption,
+        true,
+        batchSize,
+        reuseContainers
+    );
+    addCloseable(itr);
+    return itr;
+  }
+
+  @Override
+  public void close() throws IOException {
+    super.close(); // close data files
+  }
+
+  /**
+   * Reads the data file and returns an iterator of {@link VectorSchemaRoot}.
+   * Only Parquet data file format is supported.
+   */
+  private static final class VectorizedCombinedScanIterator implements CloseableIterator<ColumnarBatch> {
+
+    private final Iterator<FileScanTask> fileItr;
+    private final Map<String, InputFile> inputFiles;
+    private final Schema expectedSchema;
+    private final String nameMapping;
+    private final boolean caseSensitive;
+    private final int batchSize;
+    private final boolean reuseContainers;
+    private CloseableIterator<ColumnarBatch> currentIterator;
+    private ColumnarBatch current;
+    private FileScanTask currentTask;
+
+    /**
+     * Create a new instance.
+     *
+     * @param tasks             Combined file scan tasks.
+     * @param expectedSchema    Read schema. The returned data will have this schema.
+     * @param nameMapping       Mapping from external schema names to Iceberg type IDs.
+     * @param io                File I/O.
+     * @param encryptionManager Encryption manager.
+     * @param caseSensitive     If {@code true}, column names are case sensitive.
+     *                          If {@code false}, column names are not case sensitive.
+     * @param batchSize         Batch size in number of rows. Each Arrow batch contains
+     *                          a maximum of {@code batchSize} rows.
+     * @param reuseContainers   If set to {@code false}, every {@link Iterator#next()} call creates
+     *                          new instances of Arrow vectors.
+     *                          If set to {@code true}, the Arrow vectors in the previous
+     *                          {@link Iterator#next()} may be reused for the data returned
+     *                          in the current {@link Iterator#next()}.
+     *                          This option avoids allocating memory again and again.
+     *                          Irrespective of the value of {@code reuseContainers}, the Arrow vectors
+     *                          in the previous {@link Iterator#next()} call are closed before creating
+     *                          new instances if the current {@link Iterator#next()}.
+     */
+    VectorizedCombinedScanIterator(
+        CloseableIterable<CombinedScanTask> tasks,
+        Schema expectedSchema,
+        String nameMapping,
+        FileIO io,
+        EncryptionManager encryptionManager,
+        boolean caseSensitive,
+        int batchSize,
+        boolean reuseContainers) {
+      List<FileScanTask> fileTasks = StreamSupport.stream(tasks.spliterator(), false)
+          .map(CombinedScanTask::files)
+          .flatMap(Collection::stream)
+          .collect(Collectors.toList());
+      this.fileItr = fileTasks.iterator();
+      Map<String, ByteBuffer> keyMetadata = Maps.newHashMap();
+      fileTasks.stream()
+          .flatMap(fileScanTask -> Stream.concat(Stream.of(fileScanTask.file()), fileScanTask.deletes().stream()))
+          .forEach(file -> keyMetadata.put(file.path().toString(), file.keyMetadata()));
+      Stream<EncryptedInputFile> encrypted = keyMetadata.entrySet().stream()
+          .map(entry -> EncryptedFiles.encryptedInput(io.newInputFile(entry.getKey()), entry.getValue()));
+
+      // decrypt with the batch call to avoid multiple RPCs to a key server, if possible
+      Iterable<InputFile> decryptedFiles = encryptionManager.decrypt(encrypted::iterator);
+
+      Map<String, InputFile> files = Maps.newHashMapWithExpectedSize(fileTasks.size());
+      decryptedFiles.forEach(decrypted -> files.putIfAbsent(decrypted.location(), decrypted));
+      this.inputFiles = Collections.unmodifiableMap(files);
+
+      this.currentIterator = CloseableIterator.empty();
+      this.expectedSchema = expectedSchema;
+      this.nameMapping = nameMapping;
+      this.caseSensitive = caseSensitive;
+      this.batchSize = batchSize;
+      this.reuseContainers = reuseContainers;
+    }
+
+    @Override
+    public boolean hasNext() {
+      try {
+        while (true) {
+          if (currentIterator.hasNext()) {
+            this.current = currentIterator.next();
+            return true;
+          } else if (fileItr.hasNext()) {
+            this.currentIterator.close();
+            this.currentTask = fileItr.next();
+            this.currentIterator = open(currentTask);
+          } else {
+            this.currentIterator.close();
+            return false;
+          }
+        }
+      } catch (IOException | RuntimeException e) {
+        if (currentTask != null && !currentTask.isDataTask()) {
+          LOG.error("Error reading file: {}", getInputFile(currentTask).location(), e);

Review comment:
       Updated the exception message. Removed LOG message because the exception is already capturing the message.

##########
File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowReader.java
##########
@@ -0,0 +1,331 @@
+/*
+ * 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.arrow.vectorized;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import java.util.stream.StreamSupport;
+import org.apache.arrow.vector.NullCheckingForGet;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.arrow.vector.types.Types.MinorType;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.encryption.EncryptedFiles;
+import org.apache.iceberg.encryption.EncryptedInputFile;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.CloseableIterator;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.types.Types;
+import org.apache.parquet.schema.MessageType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Vectorized reader that returns an iterator of {@link ColumnarBatch}.
+ * See {@link #open(CloseableIterable)} ()} to learn about the
+ * behavior of the iterator.
+ *
+ * <p>The following Iceberg data types are supported and have been tested:
+ * <ul>
+ *     <li>Iceberg: {@link Types.BooleanType}, Arrow: {@link MinorType#BIT}</li>
+ *     <li>Iceberg: {@link Types.IntegerType}, Arrow: {@link MinorType#INT}</li>
+ *     <li>Iceberg: {@link Types.LongType}, Arrow: {@link MinorType#BIGINT}</li>
+ *     <li>Iceberg: {@link Types.FloatType}, Arrow: {@link MinorType#FLOAT4}</li>
+ *     <li>Iceberg: {@link Types.DoubleType}, Arrow: {@link MinorType#FLOAT8}</li>
+ *     <li>Iceberg: {@link Types.StringType}, Arrow: {@link MinorType#VARCHAR}</li>
+ *     <li>Iceberg: {@link Types.TimestampType} (both with and without timezone),
+ *         Arrow: {@link MinorType#TIMEMICRO}</li>
+ *     <li>Iceberg: {@link Types.BinaryType}, Arrow: {@link MinorType#VARBINARY}</li>
+ *     <li>Iceberg: {@link Types.DateType}, Arrow: {@link MinorType#DATEDAY}</li>
+ * </ul>
+ *
+ * <p>Features that don't work in this implementation:
+ * <ul>
+ *     <li>Type promotion: In case of type promotion, the Arrow vector corresponding to
+ *     the data type in the parquet file is returned instead of the data type in the latest schema.
+ *     See https://github.com/apache/iceberg/issues/2483.</li>
+ *     <li>Columns with constant values are physically encoded as a dictionary. The Arrow vector
+ *     type is int32 instead of the type as per the schema.
+ *     See https://github.com/apache/iceberg/issues/2484.</li>
+ *     <li>Data types: {@link Types.TimeType}, {@link Types.ListType}, {@link Types.MapType},
+ *     {@link Types.StructType}, {@link Types.UUIDType}, {@link Types.FixedType} and
+ *     {@link Types.DecimalType}
+ *     See https://github.com/apache/iceberg/issues/2485 and https://github.com/apache/iceberg/issues/2486.</li>
+ *     <li>Iceberg v2 spec is not supported.
+ *     See https://github.com/apache/iceberg/issues/2487.</li>
+ * </ul>
+ */
+public class ArrowReader extends CloseableGroup {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ArrowReader.class);
+
+  private final Schema schema;
+  private final FileIO io;
+  private final EncryptionManager encryption;
+  private final int batchSize;
+  private final boolean reuseContainers;
+
+  /**
+   * Create a new instance of the reader.
+   *
+   * @param scan the table scan object.
+   * @param batchSize the maximum number of rows per Arrow batch.
+   * @param reuseContainers whether to reuse Arrow vectors when iterating through the data.
+   *                        If set to {@code false}, every {@link Iterator#next()} call creates
+   *                        new instances of Arrow vectors.
+   *                        If set to {@code true}, the Arrow vectors in the previous
+   *                        {@link Iterator#next()} may be reused for the data returned
+   *                        in the current {@link Iterator#next()}.
+   *                        This option avoids allocating memory again and again.
+   *                        Irrespective of the value of {@code reuseContainers}, the Arrow vectors
+   *                        in the previous {@link Iterator#next()} call are closed before creating
+   *                        new instances if the current {@link Iterator#next()}.
+   */
+  public ArrowReader(TableScan scan, int batchSize, boolean reuseContainers) {
+    this.schema = scan.schema();
+    this.io = scan.table().io();
+    this.encryption = scan.table().encryption();
+    this.batchSize = batchSize;
+    // start planning tasks in the background
+    this.reuseContainers = reuseContainers;
+  }
+
+  /**
+   * Returns a new iterator of {@link ColumnarBatch} objects.
+   * <p>
+   * Note that the reader owns the {@link ColumnarBatch} objects and takes care of closing them.
+   * The caller should not hold onto a {@link ColumnarBatch} or try to close them.
+   *
+   * <p>If {@code reuseContainers} is {@code false}, the Arrow vectors in the
+   * previous {@link ColumnarBatch} are closed before returning the next {@link ColumnarBatch} object.
+   * This implies that the caller should either use the {@link ColumnarBatch} or transfer the ownership of
+   * {@link ColumnarBatch} before getting the next {@link ColumnarBatch}.
+   *
+   * <p>If {@code reuseContainers} is {@code true}, the Arrow vectors in the
+   * previous {@link ColumnarBatch} may be reused for the next {@link ColumnarBatch}.
+   * This implies that the caller should either use the {@link ColumnarBatch} or deep copy the
+   * {@link ColumnarBatch} before getting the next {@link ColumnarBatch}.
+   */
+  public CloseableIterator<ColumnarBatch> open(CloseableIterable<CombinedScanTask> tasks) {
+    CloseableIterator<ColumnarBatch> itr = new VectorizedCombinedScanIterator(
+        tasks,
+        schema,
+        null,
+        io,
+        encryption,
+        true,
+        batchSize,
+        reuseContainers
+    );
+    addCloseable(itr);
+    return itr;
+  }
+
+  @Override
+  public void close() throws IOException {
+    super.close(); // close data files
+  }
+
+  /**
+   * Reads the data file and returns an iterator of {@link VectorSchemaRoot}.
+   * Only Parquet data file format is supported.
+   */
+  private static final class VectorizedCombinedScanIterator implements CloseableIterator<ColumnarBatch> {
+
+    private final Iterator<FileScanTask> fileItr;
+    private final Map<String, InputFile> inputFiles;
+    private final Schema expectedSchema;
+    private final String nameMapping;
+    private final boolean caseSensitive;
+    private final int batchSize;
+    private final boolean reuseContainers;
+    private CloseableIterator<ColumnarBatch> currentIterator;
+    private ColumnarBatch current;
+    private FileScanTask currentTask;
+
+    /**
+     * Create a new instance.
+     *
+     * @param tasks             Combined file scan tasks.
+     * @param expectedSchema    Read schema. The returned data will have this schema.
+     * @param nameMapping       Mapping from external schema names to Iceberg type IDs.
+     * @param io                File I/O.
+     * @param encryptionManager Encryption manager.
+     * @param caseSensitive     If {@code true}, column names are case sensitive.
+     *                          If {@code false}, column names are not case sensitive.
+     * @param batchSize         Batch size in number of rows. Each Arrow batch contains
+     *                          a maximum of {@code batchSize} rows.
+     * @param reuseContainers   If set to {@code false}, every {@link Iterator#next()} call creates
+     *                          new instances of Arrow vectors.
+     *                          If set to {@code true}, the Arrow vectors in the previous
+     *                          {@link Iterator#next()} may be reused for the data returned
+     *                          in the current {@link Iterator#next()}.
+     *                          This option avoids allocating memory again and again.
+     *                          Irrespective of the value of {@code reuseContainers}, the Arrow vectors
+     *                          in the previous {@link Iterator#next()} call are closed before creating
+     *                          new instances if the current {@link Iterator#next()}.
+     */
+    VectorizedCombinedScanIterator(
+        CloseableIterable<CombinedScanTask> tasks,
+        Schema expectedSchema,
+        String nameMapping,
+        FileIO io,
+        EncryptionManager encryptionManager,
+        boolean caseSensitive,
+        int batchSize,
+        boolean reuseContainers) {
+      List<FileScanTask> fileTasks = StreamSupport.stream(tasks.spliterator(), false)
+          .map(CombinedScanTask::files)
+          .flatMap(Collection::stream)
+          .collect(Collectors.toList());
+      this.fileItr = fileTasks.iterator();
+      Map<String, ByteBuffer> keyMetadata = Maps.newHashMap();
+      fileTasks.stream()
+          .flatMap(fileScanTask -> Stream.concat(Stream.of(fileScanTask.file()), fileScanTask.deletes().stream()))
+          .forEach(file -> keyMetadata.put(file.path().toString(), file.keyMetadata()));
+      Stream<EncryptedInputFile> encrypted = keyMetadata.entrySet().stream()
+          .map(entry -> EncryptedFiles.encryptedInput(io.newInputFile(entry.getKey()), entry.getValue()));
+
+      // decrypt with the batch call to avoid multiple RPCs to a key server, if possible
+      Iterable<InputFile> decryptedFiles = encryptionManager.decrypt(encrypted::iterator);
+
+      Map<String, InputFile> files = Maps.newHashMapWithExpectedSize(fileTasks.size());
+      decryptedFiles.forEach(decrypted -> files.putIfAbsent(decrypted.location(), decrypted));
+      this.inputFiles = Collections.unmodifiableMap(files);
+
+      this.currentIterator = CloseableIterator.empty();
+      this.expectedSchema = expectedSchema;
+      this.nameMapping = nameMapping;
+      this.caseSensitive = caseSensitive;
+      this.batchSize = batchSize;
+      this.reuseContainers = reuseContainers;
+    }
+
+    @Override
+    public boolean hasNext() {
+      try {
+        while (true) {
+          if (currentIterator.hasNext()) {
+            this.current = currentIterator.next();
+            return true;
+          } else if (fileItr.hasNext()) {
+            this.currentIterator.close();
+            this.currentTask = fileItr.next();
+            this.currentIterator = open(currentTask);
+          } else {
+            this.currentIterator.close();
+            return false;
+          }
+        }
+      } catch (IOException | RuntimeException e) {
+        if (currentTask != null && !currentTask.isDataTask()) {
+          LOG.error("Error reading file: {}", getInputFile(currentTask).location(), e);
+        }
+        throw new RuntimeException(e);

Review comment:
       Updated

##########
File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowVectorAccessor.java
##########
@@ -17,79 +17,82 @@
  * under the License.
  */
 
-package org.apache.iceberg.spark.data.vectorized;
+package org.apache.iceberg.arrow.vectorized;
 
 import org.apache.arrow.vector.ValueVector;
-import org.apache.spark.sql.types.Decimal;
-import org.apache.spark.sql.vectorized.ArrowColumnVector;
-import org.apache.spark.sql.vectorized.ColumnarArray;
-import org.apache.spark.unsafe.types.UTF8String;
-
-@SuppressWarnings("checkstyle:VisibilityModifier")
-public abstract class ArrowVectorAccessor {
 
+public class ArrowVectorAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT extends AutoCloseable> {
   private final ValueVector vector;
-  private final ArrowColumnVector[] childColumns;
+  private final ChildVectorT[] childColumns;
 
-  ArrowVectorAccessor(ValueVector vector) {
-    this.vector = vector;
-    this.childColumns = new ArrowColumnVector[0];
+  protected ArrowVectorAccessor(ValueVector vector) {
+    this(vector, null);
   }
 
-  ArrowVectorAccessor(ValueVector vector, ArrowColumnVector[] children) {
+  protected ArrowVectorAccessor(ValueVector vector, ChildVectorT[] children) {
     this.vector = vector;
     this.childColumns = children;
   }
 
-  final void close() {
-    for (ArrowColumnVector column : childColumns) {
-      // Closing an ArrowColumnVector is expected to not throw any exception
-      column.close();
+  public final void close() {

Review comment:
       Fixed

##########
File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowVectorAccessors.java
##########
@@ -0,0 +1,71 @@
+/*
+ * 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.arrow.vectorized;
+
+import java.nio.charset.StandardCharsets;
+import java.util.function.Supplier;
+import org.apache.arrow.vector.VarCharVector;
+import org.apache.iceberg.arrow.vectorized.GenericArrowVectorAccessorFactory.StringFactory;
+
+final class ArrowVectorAccessors {
+
+  private static final GenericArrowVectorAccessorFactory<?, String, ?, ?> factory;
+
+  static {
+    factory = new GenericArrowVectorAccessorFactory<>(
+        throwingSupplier("Decimal type is not supported"),
+        JavaStringFactory::new,
+        throwingSupplier("Struct type is not supported"),
+        throwingSupplier("List type is not supported")
+    );
+  }
+
+  private static <T> Supplier<T> throwingSupplier(String message) {
+    return () -> {
+      throw new UnsupportedOperationException(message);
+    };
+  }
+
+  private ArrowVectorAccessors() {
+    throw new UnsupportedOperationException();

Review comment:
       Fixed

##########
File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/GenericArrowVectorAccessorFactory.java
##########
@@ -0,0 +1,693 @@
+/*
+ * 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.arrow.vectorized;
+
+import java.lang.reflect.Array;
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.util.function.IntFunction;
+import java.util.function.Supplier;
+import java.util.stream.IntStream;
+import org.apache.arrow.vector.BigIntVector;
+import org.apache.arrow.vector.BitVector;
+import org.apache.arrow.vector.DateDayVector;
+import org.apache.arrow.vector.DecimalVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.Float4Vector;
+import org.apache.arrow.vector.Float8Vector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.TimeStampMicroTZVector;
+import org.apache.arrow.vector.TimeStampMicroVector;
+import org.apache.arrow.vector.ValueVector;
+import org.apache.arrow.vector.VarBinaryVector;
+import org.apache.arrow.vector.VarCharVector;
+import org.apache.arrow.vector.complex.ListVector;
+import org.apache.arrow.vector.complex.StructVector;
+import org.apache.arrow.vector.util.DecimalUtility;
+import org.apache.parquet.Preconditions;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.PrimitiveType;
+
+/**
+ * This class is creates typed {@link ArrowVectorAccessor} from {@link VectorHolder}.
+ * It provides a generic implementation for following Arrow types:
+ * <ul>
+ *   <li>Decimal type can be deserialized to a type that supports decimal,
+ *   e.g. BigDecimal or Spark's Decimal.</li>
+ *   <li>UTF8 String type can deserialized to a Java String or Spark's UTF8String.</li>
+ *   <li>List type: the child elements of a list can be deserialized to Spark's ColumnarArray or similar type.</li>
+ *   <li>Struct type: the child elements of a struct can be deserialized to a Spark's ArrowColumnVector
+ *   or similar type.</li>
+ * </ul>
+ * @param <DecimalT> A concrete type that can represent a decimal.
+ * @param <Utf8StringT> A concrete type that can represent a UTF8 string.
+ * @param <ArrayT> A concrete type that can represent an array value in a list vector, e.g. Spark's ColumnarArray.
+ * @param <ChildVectorT> A concrete type that can represent a child vector in a struct, e.g. Spark's ArrowColumnVector.
+ */
+public class GenericArrowVectorAccessorFactory<DecimalT, Utf8StringT, ArrayT, ChildVectorT extends AutoCloseable> {
+
+  private final Supplier<DecimalFactory<DecimalT>> decimalFactorySupplier;
+  private final Supplier<StringFactory<Utf8StringT>> stringFactorySupplier;
+  private final Supplier<StructChildFactory<ChildVectorT>> structChildFactorySupplier;
+  private final Supplier<ArrayFactory<ChildVectorT, ArrayT>> arrayFactorySupplier;
+
+  /**
+   * The constructor is parameterized using the decimal, string, struct and array factories.
+   * If a specific type is not supported, the factory supplier can raise an
+   * {@link UnsupportedOperationException}.
+   */
+  protected GenericArrowVectorAccessorFactory(
+          Supplier<DecimalFactory<DecimalT>> decimalFactorySupplier,
+          Supplier<StringFactory<Utf8StringT>> stringFactorySupplier,
+          Supplier<StructChildFactory<ChildVectorT>> structChildFactorySupplier,
+          Supplier<ArrayFactory<ChildVectorT, ArrayT>> arrayFactorySupplier) {
+    this.decimalFactorySupplier = decimalFactorySupplier;
+    this.stringFactorySupplier = stringFactorySupplier;
+    this.structChildFactorySupplier = structChildFactorySupplier;
+    this.arrayFactorySupplier = arrayFactorySupplier;
+  }
+
+  public ArrowVectorAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT> getVectorAccessor(VectorHolder holder) {
+    Dictionary dictionary = holder.dictionary();
+    boolean isVectorDictEncoded = holder.isDictionaryEncoded();
+    FieldVector vector = holder.vector();
+    if (isVectorDictEncoded) {
+      ColumnDescriptor desc = holder.descriptor();
+      PrimitiveType primitive = desc.getPrimitiveType();
+      return getDictionaryVectorAccessor(dictionary, desc, vector, primitive);
+    } else {
+      return getPlainVectorAccessor(vector);
+    }
+  }
+
+  private ArrowVectorAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT> getDictionaryVectorAccessor(
+      Dictionary dictionary,
+      ColumnDescriptor desc,
+      FieldVector vector, PrimitiveType primitive) {
+    Preconditions.checkState(vector instanceof IntVector, "Dictionary ids should be stored in IntVectors only");
+    if (primitive.getOriginalType() != null) {
+      switch (desc.getPrimitiveType().getOriginalType()) {
+        case ENUM:
+        case JSON:
+        case UTF8:
+        case BSON:
+          return new DictionaryStringAccessor<>((IntVector) vector, dictionary, stringFactorySupplier.get());
+        case INT_64:
+        case TIMESTAMP_MILLIS:
+        case TIMESTAMP_MICROS:
+          return new DictionaryLongAccessor<>((IntVector) vector, dictionary);
+        case DECIMAL:
+          switch (primitive.getPrimitiveTypeName()) {
+            case BINARY:
+            case FIXED_LEN_BYTE_ARRAY:
+              return new DictionaryDecimalBinaryAccessor<>(
+                  (IntVector) vector,
+                  dictionary,
+                  decimalFactorySupplier.get());
+            case INT64:
+              return new DictionaryDecimalLongAccessor<>(
+                  (IntVector) vector,
+                  dictionary,
+                  decimalFactorySupplier.get());
+            case INT32:
+              return new DictionaryDecimalIntAccessor<>(
+                  (IntVector) vector,
+                  dictionary,
+                  decimalFactorySupplier.get());
+            default:
+              throw new UnsupportedOperationException(
+                  "Unsupported base type for decimal: " + primitive.getPrimitiveTypeName());
+          }
+        default:
+          throw new UnsupportedOperationException(
+              "Unsupported logical type: " + primitive.getOriginalType());
+      }
+    } else {
+      switch (primitive.getPrimitiveTypeName()) {
+        case FIXED_LEN_BYTE_ARRAY:
+        case BINARY:
+          return new DictionaryBinaryAccessor<>((IntVector) vector, dictionary);
+        case FLOAT:
+          return new DictionaryFloatAccessor<>((IntVector) vector, dictionary);
+        case INT64:
+          return new DictionaryLongAccessor<>((IntVector) vector, dictionary);
+        case DOUBLE:
+          return new DictionaryDoubleAccessor<>((IntVector) vector, dictionary);
+        default:
+          throw new UnsupportedOperationException("Unsupported type: " + primitive);
+      }
+    }
+  }
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  private ArrowVectorAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT>
+      getPlainVectorAccessor(FieldVector vector) {
+    if (vector instanceof BitVector) {
+      return new BooleanAccessor<>((BitVector) vector);
+    } else if (vector instanceof IntVector) {
+      return new IntAccessor<>((IntVector) vector);
+    } else if (vector instanceof BigIntVector) {
+      return new LongAccessor<>((BigIntVector) vector);
+    } else if (vector instanceof Float4Vector) {
+      return new FloatAccessor<>((Float4Vector) vector);
+    } else if (vector instanceof Float8Vector) {
+      return new DoubleAccessor<>((Float8Vector) vector);
+    } else if (vector instanceof DecimalVector) {
+      return new DecimalAccessor<>((DecimalVector) vector, decimalFactorySupplier.get());
+    } else if (vector instanceof VarCharVector) {
+      return new StringAccessor<>((VarCharVector) vector, stringFactorySupplier.get());
+    } else if (vector instanceof VarBinaryVector) {
+      return new BinaryAccessor<>((VarBinaryVector) vector);
+    } else if (vector instanceof DateDayVector) {
+      return new DateAccessor<>((DateDayVector) vector);
+    } else if (vector instanceof TimeStampMicroTZVector) {
+      return new TimestampMicroTzAccessor<>((TimeStampMicroTZVector) vector);
+    } else if (vector instanceof TimeStampMicroVector) {
+      return new TimestampMicroAccessor<>((TimeStampMicroVector) vector);
+    } else if (vector instanceof ListVector) {
+      ListVector listVector = (ListVector) vector;
+      return new ArrayAccessor<>(listVector, arrayFactorySupplier.get());
+    } else if (vector instanceof StructVector) {
+      StructVector structVector = (StructVector) vector;
+      return new StructAccessor<>(structVector, structChildFactorySupplier.get());
+    }
+    throw new UnsupportedOperationException("Unsupported vector: " + vector.getClass());
+  }
+
+  private static class BooleanAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT extends AutoCloseable>
+          extends ArrowVectorAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT> {
+    private final BitVector vector;
+
+    BooleanAccessor(BitVector vector) {
+      super(vector);
+      this.vector = vector;
+    }
+
+    @Override
+    public final boolean getBoolean(int rowId) {
+      return vector.get(rowId) == 1;
+    }
+  }
+
+  private static class IntAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT extends AutoCloseable>
+          extends ArrowVectorAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT> {
+
+    private final IntVector vector;
+
+    IntAccessor(IntVector vector) {
+      super(vector);
+      this.vector = vector;
+    }
+
+    @Override
+    public final int getInt(int rowId) {
+      return vector.get(rowId);
+    }
+
+    @Override
+    public final long getLong(int rowId) {
+      return getInt(rowId);
+    }
+  }
+
+  private static class LongAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT extends AutoCloseable>
+          extends ArrowVectorAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT> {
+
+    private final BigIntVector vector;
+
+    LongAccessor(BigIntVector vector) {
+      super(vector);
+      this.vector = vector;
+    }
+
+    @Override
+    public final long getLong(int rowId) {
+      return vector.get(rowId);
+    }
+  }
+
+  private static class DictionaryLongAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT extends AutoCloseable>
+          extends ArrowVectorAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT> {
+    private final IntVector offsetVector;
+    private final long[] decodedDictionary;
+
+    DictionaryLongAccessor(IntVector vector, Dictionary dictionary) {
+      super(vector);
+      this.offsetVector = vector;
+      this.decodedDictionary = IntStream.rangeClosed(0, dictionary.getMaxId())
+          .mapToLong(dictionary::decodeToLong)
+          .toArray();
+    }
+
+    @Override
+    public final long getLong(int rowId) {
+      return decodedDictionary[offsetVector.get(rowId)];
+    }
+  }
+
+  private static class FloatAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT extends AutoCloseable>
+          extends ArrowVectorAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT> {
+
+    private final Float4Vector vector;
+
+    FloatAccessor(Float4Vector vector) {
+      super(vector);
+      this.vector = vector;
+    }
+
+    @Override
+    public final float getFloat(int rowId) {
+      return vector.get(rowId);
+    }
+
+    @Override
+    public final double getDouble(int rowId) {
+      return getFloat(rowId);
+    }
+  }
+
+  private static class DictionaryFloatAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT extends AutoCloseable>
+          extends ArrowVectorAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT> {
+    private final IntVector offsetVector;
+    private final float[] decodedDictionary;
+
+    DictionaryFloatAccessor(IntVector vector, Dictionary dictionary) {
+      super(vector);
+      this.offsetVector = vector;
+      this.decodedDictionary = new float[dictionary.getMaxId() + 1];
+      for (int i = 0; i <= dictionary.getMaxId(); i++) {
+        decodedDictionary[i] = dictionary.decodeToFloat(i);
+      }
+    }
+
+    @Override
+    public final float getFloat(int rowId) {
+      return decodedDictionary[offsetVector.get(rowId)];
+    }
+
+    @Override
+    public final double getDouble(int rowId) {
+      return getFloat(rowId);
+    }
+  }
+
+  private static class DoubleAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT extends AutoCloseable>
+          extends ArrowVectorAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT> {
+
+    private final Float8Vector vector;
+
+    DoubleAccessor(Float8Vector vector) {
+      super(vector);
+      this.vector = vector;
+    }
+
+    @Override
+    public final double getDouble(int rowId) {
+      return vector.get(rowId);
+    }
+  }
+
+  private static class DictionaryDoubleAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT extends AutoCloseable>
+          extends ArrowVectorAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT> {
+    private final IntVector offsetVector;
+    private final double[] decodedDictionary;
+
+    DictionaryDoubleAccessor(IntVector vector, Dictionary dictionary) {
+      super(vector);
+      this.offsetVector = vector;
+      this.decodedDictionary = IntStream.rangeClosed(0, dictionary.getMaxId())
+          .mapToDouble(dictionary::decodeToDouble)
+          .toArray();
+    }
+
+    @Override
+    public final double getDouble(int rowId) {
+      return decodedDictionary[offsetVector.get(rowId)];
+    }
+  }
+
+  private static class StringAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT extends AutoCloseable>
+          extends ArrowVectorAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT> {
+
+    private final VarCharVector vector;
+    private final StringFactory<Utf8StringT> stringFactory;
+
+    StringAccessor(VarCharVector vector, StringFactory<Utf8StringT> stringFactory) {
+      super(vector);
+      this.vector = vector;
+      this.stringFactory = stringFactory;
+    }
+
+    @Override
+    public final Utf8StringT getUTF8String(int rowId) {
+      return stringFactory.ofRow(vector, rowId);
+    }
+  }
+
+  private static class DictionaryStringAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT extends AutoCloseable>
+          extends ArrowVectorAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT> {
+    private final Utf8StringT[] decodedDictionary;
+    private final IntVector offsetVector;
+
+    DictionaryStringAccessor(IntVector vector, Dictionary dictionary, StringFactory<Utf8StringT> stringFactory) {
+      super(vector);
+      this.offsetVector = vector;
+      this.decodedDictionary = IntStream.rangeClosed(0, dictionary.getMaxId())
+          .mapToObj(dictionary::decodeToBinary)
+          .map(binary -> stringFactory.ofBytes(binary.getBytes()))
+          .toArray(genericArray(stringFactory.getGenericClass()));
+    }
+
+    @Override
+    public final Utf8StringT getUTF8String(int rowId) {
+      int offset = offsetVector.get(rowId);
+      return decodedDictionary[offset];
+    }
+  }
+
+  private static class BinaryAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT extends AutoCloseable>
+          extends ArrowVectorAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT> {
+
+    private final VarBinaryVector vector;
+
+    BinaryAccessor(VarBinaryVector vector) {
+      super(vector);
+      this.vector = vector;
+    }
+
+    @Override
+    public final byte[] getBinary(int rowId) {
+      return vector.get(rowId);
+    }
+  }
+
+  private static class DictionaryBinaryAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT extends AutoCloseable>
+          extends ArrowVectorAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT> {
+    private final IntVector offsetVector;
+    private final byte[][] decodedDictionary;
+
+    DictionaryBinaryAccessor(IntVector vector, Dictionary dictionary) {
+      super(vector);
+      this.offsetVector = vector;
+      this.decodedDictionary = IntStream.rangeClosed(0, dictionary.getMaxId())
+          .mapToObj(dictionary::decodeToBinary)
+          .map(Binary::getBytes)
+          .toArray(byte[][]::new);
+    }
+
+    @Override
+    public final byte[] getBinary(int rowId) {
+      int offset = offsetVector.get(rowId);
+      return decodedDictionary[offset];
+    }
+  }
+
+  private static class DateAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT extends AutoCloseable>
+          extends ArrowVectorAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT> {
+
+    private final DateDayVector vector;
+
+    DateAccessor(DateDayVector vector) {
+      super(vector);
+      this.vector = vector;
+    }
+
+    @Override
+    public final int getInt(int rowId) {
+      return vector.get(rowId);
+    }
+  }
+
+  private static class TimestampMicroTzAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT extends AutoCloseable>
+          extends ArrowVectorAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT> {
+
+    private final TimeStampMicroTZVector vector;
+
+    TimestampMicroTzAccessor(TimeStampMicroTZVector vector) {
+      super(vector);
+      this.vector = vector;
+    }
+
+    @Override
+    public final long getLong(int rowId) {
+      return vector.get(rowId);
+    }
+  }
+
+  private static class TimestampMicroAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT extends AutoCloseable>
+      extends ArrowVectorAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT> {
+
+    private final TimeStampMicroVector vector;
+
+    TimestampMicroAccessor(TimeStampMicroVector vector) {
+      super(vector);
+      this.vector = vector;
+    }
+
+    @Override
+    public final long getLong(int rowId) {
+      return vector.get(rowId);
+    }
+  }
+
+  private static class ArrayAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT extends AutoCloseable>
+          extends ArrowVectorAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT> {
+
+    private final ListVector vector;
+    private final ChildVectorT arrayData;
+    private final ArrayFactory<ChildVectorT, ArrayT> arrayFactory;
+
+    ArrayAccessor(ListVector vector, ArrayFactory<ChildVectorT, ArrayT> arrayFactory) {
+      super(vector);
+      this.vector = vector;
+      this.arrayFactory = arrayFactory;
+      this.arrayData = arrayFactory.ofChild(vector.getDataVector());
+    }
+
+    @Override
+    public final ArrayT getArray(int rowId) {
+      return arrayFactory.ofRow(vector, arrayData, rowId);
+    }
+  }
+
+  private static class StructAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT extends AutoCloseable>
+          extends ArrowVectorAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT> {
+    StructAccessor(StructVector structVector, StructChildFactory<ChildVectorT> structChildFactory) {
+      super(structVector, IntStream.range(0, structVector.size())
+              .mapToObj(structVector::getVectorById)
+              .map(structChildFactory::of)
+              .toArray(genericArray(structChildFactory.getGenericClass())));
+    }
+  }
+
+  private static class DecimalAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT extends AutoCloseable>
+          extends ArrowVectorAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT> {
+
+    private final DecimalVector vector;
+    private final DecimalFactory<DecimalT> decimalFactory;
+
+    DecimalAccessor(DecimalVector vector, DecimalFactory<DecimalT> decimalFactory) {
+      super(vector);
+      this.vector = vector;
+      this.decimalFactory = decimalFactory;
+    }
+
+    @Override
+    public final DecimalT getDecimal(int rowId, int precision, int scale) {
+      return decimalFactory.ofBigDecimal(
+              DecimalUtility.getBigDecimalFromArrowBuf(vector.getDataBuffer(), rowId, scale),
+              precision, scale);
+    }
+  }
+
+  @SuppressWarnings("checkstyle:VisibilityModifier")
+  private abstract static class
+      DictionaryDecimalAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT extends AutoCloseable>
+          extends ArrowVectorAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT> {
+    final DecimalT[] cache;

Review comment:
       Refactored




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] mayursrivastava commented on pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
mayursrivastava commented on pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#issuecomment-828391883


   Hi @rymurr, I recorded results from master branch as well. Here's the result:
   
   ```
   # JMH version: 1.21
   # VM version: JDK 1.8.0_282, OpenJDK 64-Bit Server VM, 25.282-b08
   # VM invoker: /usr/lib/jvm/java-8-openjdk-amd64/jre/bin/java
   # VM options: <none>
   # Warmup: 3 iterations, single-shot each
   # Measurement: 5 iterations, single-shot each
   # Timeout: 10 min per iteration
   # Threads: 1 thread
   # Benchmark mode: Single shot invocation time
   # Benchmark: org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadFlatParquetDataBenchmark.readDatesIcebergVectorized5k
   
   # Run progress: 0.00% complete, ETA 00:00:00
   # Fork: 1 of 1
   # Warmup Iteration   1: 1.761 s/op
   # Warmup Iteration   2: 1.464 s/op
   # Warmup Iteration   3: 1.621 s/op
   Iteration   1: 1.545 s/op
   Iteration   2: 1.493 s/op
   Iteration   3: 1.479 s/op
   Iteration   4: 1.460 s/op
   Iteration   5: 1.471 s/op
   
   
   Result "org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadFlatParquetDataBenchmark.readDatesIcebergVectorized5k":
     N = 5
     mean =      1.490 ±(99.9%) 0.128 s/op
   
     Histogram, s/op:
       [1.460, 1.465) = 1 
       [1.465, 1.470) = 0 
       [1.470, 1.475) = 1 
       [1.475, 1.480) = 1 
       [1.480, 1.485) = 0 
       [1.485, 1.490) = 0 
       [1.490, 1.495) = 1 
       [1.495, 1.500) = 0 
       [1.500, 1.505) = 0 
       [1.505, 1.510) = 0 
       [1.510, 1.515) = 0 
       [1.515, 1.520) = 0 
       [1.520, 1.525) = 0 
       [1.525, 1.530) = 0 
       [1.530, 1.535) = 0 
       [1.535, 1.540) = 0 
       [1.540, 1.545) = 0 
       [1.545, 1.550) = 1 
   
     Percentiles, s/op:
         p(0.0000) =      1.460 s/op
        p(50.0000) =      1.479 s/op
        p(90.0000) =      1.545 s/op
        p(95.0000) =      1.545 s/op
        p(99.0000) =      1.545 s/op
        p(99.9000) =      1.545 s/op
        p(99.9900) =      1.545 s/op
        p(99.9990) =      1.545 s/op
        p(99.9999) =      1.545 s/op
       p(100.0000) =      1.545 s/op
   
   
   # JMH version: 1.21
   # VM version: JDK 1.8.0_282, OpenJDK 64-Bit Server VM, 25.282-b08
   # VM invoker: /usr/lib/jvm/java-8-openjdk-amd64/jre/bin/java
   # VM options: <none>
   # Warmup: 3 iterations, single-shot each
   # Measurement: 5 iterations, single-shot each
   # Timeout: 10 min per iteration
   # Threads: 1 thread
   # Benchmark mode: Single shot invocation time
   # Benchmark: org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadFlatParquetDataBenchmark.readDatesSparkVectorized5k
   
   # Run progress: 6.25% complete, ETA 00:49:08
   # Fork: 1 of 1
   # Warmup Iteration   1: 1.683 s/op
   # Warmup Iteration   2: 1.314 s/op
   # Warmup Iteration   3: 1.309 s/op
   Iteration   1: 1.329 s/op
   Iteration   2: 1.319 s/op
   Iteration   3: 1.289 s/op
   Iteration   4: 1.264 s/op
   Iteration   5: 1.271 s/op
   
   
   Result "org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadFlatParquetDataBenchmark.readDatesSparkVectorized5k":
     N = 5
     mean =      1.294 ±(99.9%) 0.111 s/op
   
     Histogram, s/op:
       [1.260, 1.265) = 1 
       [1.265, 1.270) = 0 
       [1.270, 1.275) = 1 
       [1.275, 1.280) = 0 
       [1.280, 1.285) = 0 
       [1.285, 1.290) = 1 
       [1.290, 1.295) = 0 
       [1.295, 1.300) = 0 
       [1.300, 1.305) = 0 
       [1.305, 1.310) = 0 
       [1.310, 1.315) = 0 
       [1.315, 1.320) = 1 
       [1.320, 1.325) = 0 
       [1.325, 1.330) = 1 
   
     Percentiles, s/op:
         p(0.0000) =      1.264 s/op
        p(50.0000) =      1.289 s/op
        p(90.0000) =      1.329 s/op
        p(95.0000) =      1.329 s/op
        p(99.0000) =      1.329 s/op
        p(99.9000) =      1.329 s/op
        p(99.9900) =      1.329 s/op
        p(99.9990) =      1.329 s/op
        p(99.9999) =      1.329 s/op
       p(100.0000) =      1.329 s/op
   
   
   # JMH version: 1.21
   # VM version: JDK 1.8.0_282, OpenJDK 64-Bit Server VM, 25.282-b08
   # VM invoker: /usr/lib/jvm/java-8-openjdk-amd64/jre/bin/java
   # VM options: <none>
   # Warmup: 3 iterations, single-shot each
   # Measurement: 5 iterations, single-shot each
   # Timeout: 10 min per iteration
   # Threads: 1 thread
   # Benchmark mode: Single shot invocation time
   # Benchmark: org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadFlatParquetDataBenchmark.readDecimalsIcebergVectorized5k
   
   # Run progress: 12.50% complete, ETA 00:45:16
   # Fork: 1 of 1
   # Warmup Iteration   1: 8.414 s/op
   # Warmup Iteration   2: 7.756 s/op
   # Warmup Iteration   3: 7.869 s/op
   Iteration   1: 8.740 s/op
   Iteration   2: 8.667 s/op
   Iteration   3: 8.597 s/op
   Iteration   4: 8.597 s/op
   Iteration   5: 8.610 s/op
   
   
   Result "org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadFlatParquetDataBenchmark.readDecimalsIcebergVectorized5k":
     N = 5
     mean =      8.642 ±(99.9%) 0.239 s/op
   
     Histogram, s/op:
       [8.500, 8.525) = 0 
       [8.525, 8.550) = 0 
       [8.550, 8.575) = 0 
       [8.575, 8.600) = 2 
       [8.600, 8.625) = 1 
       [8.625, 8.650) = 0 
       [8.650, 8.675) = 1 
       [8.675, 8.700) = 0 
       [8.700, 8.725) = 0 
       [8.725, 8.750) = 1 
       [8.750, 8.775) = 0 
       [8.775, 8.800) = 0 
   
     Percentiles, s/op:
         p(0.0000) =      8.597 s/op
        p(50.0000) =      8.610 s/op
        p(90.0000) =      8.740 s/op
        p(95.0000) =      8.740 s/op
        p(99.0000) =      8.740 s/op
        p(99.9000) =      8.740 s/op
        p(99.9900) =      8.740 s/op
        p(99.9990) =      8.740 s/op
        p(99.9999) =      8.740 s/op
       p(100.0000) =      8.740 s/op
   
   
   # JMH version: 1.21
   # VM version: JDK 1.8.0_282, OpenJDK 64-Bit Server VM, 25.282-b08
   # VM invoker: /usr/lib/jvm/java-8-openjdk-amd64/jre/bin/java
   # VM options: <none>
   # Warmup: 3 iterations, single-shot each
   # Measurement: 5 iterations, single-shot each
   # Timeout: 10 min per iteration
   # Threads: 1 thread
   # Benchmark mode: Single shot invocation time
   # Benchmark: org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadFlatParquetDataBenchmark.readDecimalsSparkVectorized5k
   
   # Run progress: 18.75% complete, ETA 00:45:51
   # Fork: 1 of 1
   # Warmup Iteration   1: 9.147 s/op
   # Warmup Iteration   2: 8.526 s/op
   # Warmup Iteration   3: 8.513 s/op
   Iteration   1: 8.478 s/op
   Iteration   2: 8.419 s/op
   Iteration   3: 8.455 s/op
   Iteration   4: 8.421 s/op
   Iteration   5: 8.450 s/op
   
   
   Result "org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadFlatParquetDataBenchmark.readDecimalsSparkVectorized5k":
     N = 5
     mean =      8.444 ±(99.9%) 0.096 s/op
   
     Histogram, s/op:
       [8.410, 8.415) = 0 
       [8.415, 8.420) = 1 
       [8.420, 8.425) = 1 
       [8.425, 8.430) = 0 
       [8.430, 8.435) = 0 
       [8.435, 8.440) = 0 
       [8.440, 8.445) = 0 
       [8.445, 8.450) = 1 
       [8.450, 8.455) = 0 
       [8.455, 8.460) = 1 
       [8.460, 8.465) = 0 
       [8.465, 8.470) = 0 
       [8.470, 8.475) = 0 
       [8.475, 8.480) = 1 
   
     Percentiles, s/op:
         p(0.0000) =      8.419 s/op
        p(50.0000) =      8.450 s/op
        p(90.0000) =      8.478 s/op
        p(95.0000) =      8.478 s/op
        p(99.0000) =      8.478 s/op
        p(99.9000) =      8.478 s/op
        p(99.9900) =      8.478 s/op
        p(99.9990) =      8.478 s/op
        p(99.9999) =      8.478 s/op
       p(100.0000) =      8.478 s/op
   
   
   # JMH version: 1.21
   # VM version: JDK 1.8.0_282, OpenJDK 64-Bit Server VM, 25.282-b08
   # VM invoker: /usr/lib/jvm/java-8-openjdk-amd64/jre/bin/java
   # VM options: <none>
   # Warmup: 3 iterations, single-shot each
   # Measurement: 5 iterations, single-shot each
   # Timeout: 10 min per iteration
   # Threads: 1 thread
   # Benchmark mode: Single shot invocation time
   # Benchmark: org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadFlatParquetDataBenchmark.readDoublesIcebergVectorized5k
   
   # Run progress: 25.00% complete, ETA 00:45:34
   # Fork: 1 of 1
   # Warmup Iteration   1: 3.148 s/op
   # Warmup Iteration   2: 2.701 s/op
   # Warmup Iteration   3: 2.624 s/op
   Iteration   1: 2.559 s/op
   Iteration   2: 2.559 s/op
   Iteration   3: 2.532 s/op
   Iteration   4: 2.481 s/op
   Iteration   5: 2.658 s/op
   
   
   Result "org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadFlatParquetDataBenchmark.readDoublesIcebergVectorized5k":
     N = 5
     mean =      2.558 ±(99.9%) 0.248 s/op
   
     Histogram, s/op:
       [2.400, 2.425) = 0 
       [2.425, 2.450) = 0 
       [2.450, 2.475) = 0 
       [2.475, 2.500) = 1 
       [2.500, 2.525) = 0 
       [2.525, 2.550) = 1 
       [2.550, 2.575) = 2 
       [2.575, 2.600) = 0 
       [2.600, 2.625) = 0 
       [2.625, 2.650) = 0 
       [2.650, 2.675) = 1 
   
     Percentiles, s/op:
         p(0.0000) =      2.481 s/op
        p(50.0000) =      2.559 s/op
        p(90.0000) =      2.658 s/op
        p(95.0000) =      2.658 s/op
        p(99.0000) =      2.658 s/op
        p(99.9000) =      2.658 s/op
        p(99.9900) =      2.658 s/op
        p(99.9990) =      2.658 s/op
        p(99.9999) =      2.658 s/op
       p(100.0000) =      2.658 s/op
   
   
   # JMH version: 1.21
   # VM version: JDK 1.8.0_282, OpenJDK 64-Bit Server VM, 25.282-b08
   # VM invoker: /usr/lib/jvm/java-8-openjdk-amd64/jre/bin/java
   # VM options: <none>
   # Warmup: 3 iterations, single-shot each
   # Measurement: 5 iterations, single-shot each
   # Timeout: 10 min per iteration
   # Threads: 1 thread
   # Benchmark mode: Single shot invocation time
   # Benchmark: org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadFlatParquetDataBenchmark.readDoublesSparkVectorized5k
   
   # Run progress: 31.25% complete, ETA 00:41:41
   # Fork: 1 of 1
   # Warmup Iteration   1: 3.058 s/op
   # Warmup Iteration   2: 2.484 s/op
   # Warmup Iteration   3: 2.453 s/op
   Iteration   1: 2.407 s/op
   Iteration   2: 2.359 s/op
   Iteration   3: 2.380 s/op
   Iteration   4: 2.342 s/op
   Iteration   5: 2.373 s/op
   
   
   Result "org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadFlatParquetDataBenchmark.readDoublesSparkVectorized5k":
     N = 5
     mean =      2.372 ±(99.9%) 0.093 s/op
   
     Histogram, s/op:
       [2.340, 2.345) = 1 
       [2.345, 2.350) = 0 
       [2.350, 2.355) = 0 
       [2.355, 2.360) = 1 
       [2.360, 2.365) = 0 
       [2.365, 2.370) = 0 
       [2.370, 2.375) = 1 
       [2.375, 2.380) = 0 
       [2.380, 2.385) = 1 
       [2.385, 2.390) = 0 
       [2.390, 2.395) = 0 
       [2.395, 2.400) = 0 
       [2.400, 2.405) = 0 
       [2.405, 2.410) = 1 
   
     Percentiles, s/op:
         p(0.0000) =      2.342 s/op
        p(50.0000) =      2.373 s/op
        p(90.0000) =      2.407 s/op
        p(95.0000) =      2.407 s/op
        p(99.0000) =      2.407 s/op
        p(99.9000) =      2.407 s/op
        p(99.9900) =      2.407 s/op
        p(99.9990) =      2.407 s/op
        p(99.9999) =      2.407 s/op
       p(100.0000) =      2.407 s/op
   
   
   # JMH version: 1.21
   # VM version: JDK 1.8.0_282, OpenJDK 64-Bit Server VM, 25.282-b08
   # VM invoker: /usr/lib/jvm/java-8-openjdk-amd64/jre/bin/java
   # VM options: <none>
   # Warmup: 3 iterations, single-shot each
   # Measurement: 5 iterations, single-shot each
   # Timeout: 10 min per iteration
   # Threads: 1 thread
   # Benchmark mode: Single shot invocation time
   # Benchmark: org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadFlatParquetDataBenchmark.readFloatsIcebergVectorized5k
   
   # Run progress: 37.50% complete, ETA 00:37:45
   # Fork: 1 of 1
   # Warmup Iteration   1: 3.058 s/op
   # Warmup Iteration   2: 2.513 s/op
   # Warmup Iteration   3: 2.499 s/op
   Iteration   1: 2.392 s/op
   Iteration   2: 2.402 s/op
   Iteration   3: 2.346 s/op
   Iteration   4: 2.378 s/op
   Iteration   5: 2.371 s/op
   
   
   Result "org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadFlatParquetDataBenchmark.readFloatsIcebergVectorized5k":
     N = 5
     mean =      2.378 ±(99.9%) 0.083 s/op
   
     Histogram, s/op:
       [2.340, 2.345) = 0 
       [2.345, 2.350) = 1 
       [2.350, 2.355) = 0 
       [2.355, 2.360) = 0 
       [2.360, 2.365) = 0 
       [2.365, 2.370) = 0 
       [2.370, 2.375) = 1 
       [2.375, 2.380) = 1 
       [2.380, 2.385) = 0 
       [2.385, 2.390) = 0 
       [2.390, 2.395) = 1 
       [2.395, 2.400) = 0 
       [2.400, 2.405) = 1 
       [2.405, 2.410) = 0 
   
     Percentiles, s/op:
         p(0.0000) =      2.346 s/op
        p(50.0000) =      2.378 s/op
        p(90.0000) =      2.402 s/op
        p(95.0000) =      2.402 s/op
        p(99.0000) =      2.402 s/op
        p(99.9000) =      2.402 s/op
        p(99.9900) =      2.402 s/op
        p(99.9990) =      2.402 s/op
        p(99.9999) =      2.402 s/op
       p(100.0000) =      2.402 s/op
   
   
   # JMH version: 1.21
   # VM version: JDK 1.8.0_282, OpenJDK 64-Bit Server VM, 25.282-b08
   # VM invoker: /usr/lib/jvm/java-8-openjdk-amd64/jre/bin/java
   # VM options: <none>
   # Warmup: 3 iterations, single-shot each
   # Measurement: 5 iterations, single-shot each
   # Timeout: 10 min per iteration
   # Threads: 1 thread
   # Benchmark mode: Single shot invocation time
   # Benchmark: org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadFlatParquetDataBenchmark.readFloatsSparkVectorized5k
   
   # Run progress: 43.75% complete, ETA 00:33:53
   # Fork: 1 of 1
   # Warmup Iteration   1: 2.855 s/op
   # Warmup Iteration   2: 2.304 s/op
   # Warmup Iteration   3: 2.241 s/op
   Iteration   1: 2.252 s/op
   Iteration   2: 2.181 s/op
   Iteration   3: 2.219 s/op
   Iteration   4: 2.197 s/op
   Iteration   5: 2.190 s/op
   
   
   Result "org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadFlatParquetDataBenchmark.readFloatsSparkVectorized5k":
     N = 5
     mean =      2.208 ±(99.9%) 0.110 s/op
   
     Histogram, s/op:
       [2.180, 2.185) = 1 
       [2.185, 2.190) = 1 
       [2.190, 2.195) = 0 
       [2.195, 2.200) = 1 
       [2.200, 2.205) = 0 
       [2.205, 2.210) = 0 
       [2.210, 2.215) = 0 
       [2.215, 2.220) = 1 
       [2.220, 2.225) = 0 
       [2.225, 2.230) = 0 
       [2.230, 2.235) = 0 
       [2.235, 2.240) = 0 
       [2.240, 2.245) = 0 
       [2.245, 2.250) = 0 
       [2.250, 2.255) = 1 
       [2.255, 2.260) = 0 
   
     Percentiles, s/op:
         p(0.0000) =      2.181 s/op
        p(50.0000) =      2.197 s/op
        p(90.0000) =      2.252 s/op
        p(95.0000) =      2.252 s/op
        p(99.0000) =      2.252 s/op
        p(99.9000) =      2.252 s/op
        p(99.9900) =      2.252 s/op
        p(99.9990) =      2.252 s/op
        p(99.9999) =      2.252 s/op
       p(100.0000) =      2.252 s/op
   
   
   # JMH version: 1.21
   # VM version: JDK 1.8.0_282, OpenJDK 64-Bit Server VM, 25.282-b08
   # VM invoker: /usr/lib/jvm/java-8-openjdk-amd64/jre/bin/java
   # VM options: <none>
   # Warmup: 3 iterations, single-shot each
   # Measurement: 5 iterations, single-shot each
   # Timeout: 10 min per iteration
   # Threads: 1 thread
   # Benchmark mode: Single shot invocation time
   # Benchmark: org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadFlatParquetDataBenchmark.readIntegersIcebergVectorized5k
   
   # Run progress: 50.00% complete, ETA 00:30:02
   # Fork: 1 of 1
   # Warmup Iteration   1: 3.129 s/op
   # Warmup Iteration   2: 2.522 s/op
   # Warmup Iteration   3: 2.564 s/op
   Iteration   1: 2.434 s/op
   Iteration   2: 2.478 s/op
   Iteration   3: 2.394 s/op
   Iteration   4: 2.419 s/op
   Iteration   5: 2.425 s/op
   
   
   Result "org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadFlatParquetDataBenchmark.readIntegersIcebergVectorized5k":
     N = 5
     mean =      2.430 ±(99.9%) 0.118 s/op
   
     Histogram, s/op:
       [2.390, 2.395) = 1 
       [2.395, 2.400) = 0 
       [2.400, 2.405) = 0 
       [2.405, 2.410) = 0 
       [2.410, 2.415) = 0 
       [2.415, 2.420) = 1 
       [2.420, 2.425) = 0 
       [2.425, 2.430) = 1 
       [2.430, 2.435) = 1 
       [2.435, 2.440) = 0 
       [2.440, 2.445) = 0 
       [2.445, 2.450) = 0 
       [2.450, 2.455) = 0 
       [2.455, 2.460) = 0 
       [2.460, 2.465) = 0 
       [2.465, 2.470) = 0 
       [2.470, 2.475) = 0 
   
     Percentiles, s/op:
         p(0.0000) =      2.394 s/op
        p(50.0000) =      2.425 s/op
        p(90.0000) =      2.478 s/op
        p(95.0000) =      2.478 s/op
        p(99.0000) =      2.478 s/op
        p(99.9000) =      2.478 s/op
        p(99.9900) =      2.478 s/op
        p(99.9990) =      2.478 s/op
        p(99.9999) =      2.478 s/op
       p(100.0000) =      2.478 s/op
   
   
   # JMH version: 1.21
   # VM version: JDK 1.8.0_282, OpenJDK 64-Bit Server VM, 25.282-b08
   # VM invoker: /usr/lib/jvm/java-8-openjdk-amd64/jre/bin/java
   # VM options: <none>
   # Warmup: 3 iterations, single-shot each
   # Measurement: 5 iterations, single-shot each
   # Timeout: 10 min per iteration
   # Threads: 1 thread
   # Benchmark mode: Single shot invocation time
   # Benchmark: org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadFlatParquetDataBenchmark.readIntegersSparkVectorized5k
   
   # Run progress: 56.25% complete, ETA 00:26:23
   # Fork: 1 of 1
   # Warmup Iteration   1: 3.047 s/op
   # Warmup Iteration   2: 2.405 s/op
   # Warmup Iteration   3: 2.374 s/op
   Iteration   1: 2.341 s/op
   Iteration   2: 2.271 s/op
   Iteration   3: 2.329 s/op
   Iteration   4: 2.268 s/op
   Iteration   5: 2.300 s/op
   
   
   Result "org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadFlatParquetDataBenchmark.readIntegersSparkVectorized5k":
     N = 5
     mean =      2.302 ±(99.9%) 0.126 s/op
   
     Histogram, s/op:
       [2.260, 2.265) = 0 
       [2.265, 2.270) = 1 
       [2.270, 2.275) = 1 
       [2.275, 2.280) = 0 
       [2.280, 2.285) = 0 
       [2.285, 2.290) = 0 
       [2.290, 2.295) = 0 
       [2.295, 2.300) = 0 
       [2.300, 2.305) = 1 
       [2.305, 2.310) = 0 
       [2.310, 2.315) = 0 
       [2.315, 2.320) = 0 
       [2.320, 2.325) = 0 
       [2.325, 2.330) = 1 
       [2.330, 2.335) = 0 
       [2.335, 2.340) = 0 
       [2.340, 2.345) = 1 
   
     Percentiles, s/op:
         p(0.0000) =      2.268 s/op
        p(50.0000) =      2.300 s/op
        p(90.0000) =      2.341 s/op
        p(95.0000) =      2.341 s/op
        p(99.0000) =      2.341 s/op
        p(99.9000) =      2.341 s/op
        p(99.9900) =      2.341 s/op
        p(99.9990) =      2.341 s/op
        p(99.9999) =      2.341 s/op
       p(100.0000) =      2.341 s/op
   
   
   # JMH version: 1.21
   # VM version: JDK 1.8.0_282, OpenJDK 64-Bit Server VM, 25.282-b08
   # VM invoker: /usr/lib/jvm/java-8-openjdk-amd64/jre/bin/java
   # VM options: <none>
   # Warmup: 3 iterations, single-shot each
   # Measurement: 5 iterations, single-shot each
   # Timeout: 10 min per iteration
   # Threads: 1 thread
   # Benchmark mode: Single shot invocation time
   # Benchmark: org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadFlatParquetDataBenchmark.readLongsIcebergVectorized5k
   
   # Run progress: 62.50% complete, ETA 00:22:36
   # Fork: 1 of 1
   # Warmup Iteration   1: 3.328 s/op
   # Warmup Iteration   2: 2.795 s/op
   # Warmup Iteration   3: 2.839 s/op
   Iteration   1: 2.648 s/op
   Iteration   2: 3.041 s/op
   Iteration   3: 2.635 s/op
   Iteration   4: 2.645 s/op
   Iteration   5: 2.689 s/op
   
   
   Result "org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadFlatParquetDataBenchmark.readLongsIcebergVectorized5k":
     N = 5
     mean =      2.732 ±(99.9%) 0.671 s/op
   
     Histogram, s/op:
       [2.600, 2.650) = 3 
       [2.650, 2.700) = 1 
       [2.700, 2.750) = 0 
       [2.750, 2.800) = 0 
       [2.800, 2.850) = 0 
       [2.850, 2.900) = 0 
       [2.900, 2.950) = 0 
       [2.950, 3.000) = 0 
       [3.000, 3.050) = 1 
   
     Percentiles, s/op:
         p(0.0000) =      2.635 s/op
        p(50.0000) =      2.648 s/op
        p(90.0000) =      3.041 s/op
        p(95.0000) =      3.041 s/op
        p(99.0000) =      3.041 s/op
        p(99.9000) =      3.041 s/op
        p(99.9900) =      3.041 s/op
        p(99.9990) =      3.041 s/op
        p(99.9999) =      3.041 s/op
       p(100.0000) =      3.041 s/op
   
   
   # JMH version: 1.21
   # VM version: JDK 1.8.0_282, OpenJDK 64-Bit Server VM, 25.282-b08
   # VM invoker: /usr/lib/jvm/java-8-openjdk-amd64/jre/bin/java
   # VM options: <none>
   # Warmup: 3 iterations, single-shot each
   # Measurement: 5 iterations, single-shot each
   # Timeout: 10 min per iteration
   # Threads: 1 thread
   # Benchmark mode: Single shot invocation time
   # Benchmark: org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadFlatParquetDataBenchmark.readLongsSparkVectorized5k
   
   # Run progress: 68.75% complete, ETA 00:18:52
   # Fork: 1 of 1
   # Warmup Iteration   1: 3.161 s/op
   # Warmup Iteration   2: 2.457 s/op
   # Warmup Iteration   3: 2.408 s/op
   Iteration   1: 2.344 s/op
   Iteration   2: 2.322 s/op
   Iteration   3: 2.351 s/op
   Iteration   4: 2.307 s/op
   Iteration   5: 2.310 s/op
   
   
   Result "org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadFlatParquetDataBenchmark.readLongsSparkVectorized5k":
     N = 5
     mean =      2.327 ±(99.9%) 0.076 s/op
   
     Histogram, s/op:
       [2.300, 2.305) = 0 
       [2.305, 2.310) = 1 
       [2.310, 2.315) = 1 
       [2.315, 2.320) = 0 
       [2.320, 2.325) = 1 
       [2.325, 2.330) = 0 
       [2.330, 2.335) = 0 
       [2.335, 2.340) = 0 
       [2.340, 2.345) = 1 
       [2.345, 2.350) = 0 
       [2.350, 2.355) = 1 
   
     Percentiles, s/op:
         p(0.0000) =      2.307 s/op
        p(50.0000) =      2.322 s/op
        p(90.0000) =      2.351 s/op
        p(95.0000) =      2.351 s/op
        p(99.0000) =      2.351 s/op
        p(99.9000) =      2.351 s/op
        p(99.9900) =      2.351 s/op
        p(99.9990) =      2.351 s/op
        p(99.9999) =      2.351 s/op
       p(100.0000) =      2.351 s/op
   
   
   # JMH version: 1.21
   # VM version: JDK 1.8.0_282, OpenJDK 64-Bit Server VM, 25.282-b08
   # VM invoker: /usr/lib/jvm/java-8-openjdk-amd64/jre/bin/java
   # VM options: <none>
   # Warmup: 3 iterations, single-shot each
   # Measurement: 5 iterations, single-shot each
   # Timeout: 10 min per iteration
   # Threads: 1 thread
   # Benchmark mode: Single shot invocation time
   # Benchmark: org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadFlatParquetDataBenchmark.readStringsIcebergVectorized5k
   
   # Run progress: 75.00% complete, ETA 00:15:07
   # Fork: 1 of 1
   # Warmup Iteration   1: 4.964 s/op
   # Warmup Iteration   2: 4.274 s/op
   # Warmup Iteration   3: 4.244 s/op
   Iteration   1: 4.167 s/op
   Iteration   2: 4.172 s/op
   Iteration   3: 4.127 s/op
   Iteration   4: 4.146 s/op
   Iteration   5: 4.169 s/op
   
   
   Result "org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadFlatParquetDataBenchmark.readStringsIcebergVectorized5k":
     N = 5
     mean =      4.156 ±(99.9%) 0.075 s/op
   
     Histogram, s/op:
       [4.120, 4.125) = 0 
       [4.125, 4.130) = 1 
       [4.130, 4.135) = 0 
       [4.135, 4.140) = 0 
       [4.140, 4.145) = 0 
       [4.145, 4.150) = 1 
       [4.150, 4.155) = 0 
       [4.155, 4.160) = 0 
       [4.160, 4.165) = 0 
       [4.165, 4.170) = 2 
       [4.170, 4.175) = 1 
   
     Percentiles, s/op:
         p(0.0000) =      4.127 s/op
        p(50.0000) =      4.167 s/op
        p(90.0000) =      4.172 s/op
        p(95.0000) =      4.172 s/op
        p(99.0000) =      4.172 s/op
        p(99.9000) =      4.172 s/op
        p(99.9900) =      4.172 s/op
        p(99.9990) =      4.172 s/op
        p(99.9999) =      4.172 s/op
       p(100.0000) =      4.172 s/op
   
   
   # JMH version: 1.21
   # VM version: JDK 1.8.0_282, OpenJDK 64-Bit Server VM, 25.282-b08
   # VM invoker: /usr/lib/jvm/java-8-openjdk-amd64/jre/bin/java
   # VM options: <none>
   # Warmup: 3 iterations, single-shot each
   # Measurement: 5 iterations, single-shot each
   # Timeout: 10 min per iteration
   # Threads: 1 thread
   # Benchmark mode: Single shot invocation time
   # Benchmark: org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadFlatParquetDataBenchmark.readStringsSparkVectorized5k
   
   # Run progress: 81.25% complete, ETA 00:11:22
   # Fork: 1 of 1
   # Warmup Iteration   1: 4.747 s/op
   # Warmup Iteration   2: 4.092 s/op
   # Warmup Iteration   3: 4.072 s/op
   Iteration   1: 4.013 s/op
   Iteration   2: 3.928 s/op
   Iteration   3: 4.017 s/op
   Iteration   4: 3.945 s/op
   Iteration   5: 3.990 s/op
   
   
   Result "org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadFlatParquetDataBenchmark.readStringsSparkVectorized5k":
     N = 5
     mean =      3.978 ±(99.9%) 0.155 s/op
   
     Histogram, s/op:
       [3.920, 3.930) = 1 
       [3.930, 3.940) = 0 
       [3.940, 3.950) = 1 
       [3.950, 3.960) = 0 
       [3.960, 3.970) = 0 
       [3.970, 3.980) = 0 
       [3.980, 3.990) = 0 
       [3.990, 4.000) = 1 
       [4.000, 4.010) = 0 
       [4.010, 4.020) = 2 
   
     Percentiles, s/op:
         p(0.0000) =      3.928 s/op
        p(50.0000) =      3.990 s/op
        p(90.0000) =      4.017 s/op
        p(95.0000) =      4.017 s/op
        p(99.0000) =      4.017 s/op
        p(99.9000) =      4.017 s/op
        p(99.9900) =      4.017 s/op
        p(99.9990) =      4.017 s/op
        p(99.9999) =      4.017 s/op
       p(100.0000) =      4.017 s/op
   
   
   # JMH version: 1.21
   # VM version: JDK 1.8.0_282, OpenJDK 64-Bit Server VM, 25.282-b08
   # VM invoker: /usr/lib/jvm/java-8-openjdk-amd64/jre/bin/java
   # VM options: <none>
   # Warmup: 3 iterations, single-shot each
   # Measurement: 5 iterations, single-shot each
   # Timeout: 10 min per iteration
   # Threads: 1 thread
   # Benchmark mode: Single shot invocation time
   # Benchmark: org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadFlatParquetDataBenchmark.readTimestampsIcebergVectorized5k
   
   # Run progress: 87.50% complete, ETA 00:07:36
   # Fork: 1 of 1
   # Warmup Iteration   1: 2.066 s/op
   # Warmup Iteration   2: 1.622 s/op
   # Warmup Iteration   3: 1.602 s/op
   Iteration   1: 1.546 s/op
   Iteration   2: 1.503 s/op
   Iteration   3: 1.542 s/op
   Iteration   4: 1.507 s/op
   Iteration   5: 1.599 s/op
   
   
   Result "org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadFlatParquetDataBenchmark.readTimestampsIcebergVectorized5k":
     N = 5
     mean =      1.539 ±(99.9%) 0.150 s/op
   
     Histogram, s/op:
       [1.500, 1.510) = 2 
       [1.510, 1.520) = 0 
       [1.520, 1.530) = 0 
       [1.530, 1.540) = 0 
       [1.540, 1.550) = 2 
       [1.550, 1.560) = 0 
       [1.560, 1.570) = 0 
       [1.570, 1.580) = 0 
       [1.580, 1.590) = 0 
       [1.590, 1.600) = 1 
   
     Percentiles, s/op:
         p(0.0000) =      1.503 s/op
        p(50.0000) =      1.542 s/op
        p(90.0000) =      1.599 s/op
        p(95.0000) =      1.599 s/op
        p(99.0000) =      1.599 s/op
        p(99.9000) =      1.599 s/op
        p(99.9900) =      1.599 s/op
        p(99.9990) =      1.599 s/op
        p(99.9999) =      1.599 s/op
       p(100.0000) =      1.599 s/op
   
   
   # JMH version: 1.21
   # VM version: JDK 1.8.0_282, OpenJDK 64-Bit Server VM, 25.282-b08
   # VM invoker: /usr/lib/jvm/java-8-openjdk-amd64/jre/bin/java
   # VM options: <none>
   # Warmup: 3 iterations, single-shot each
   # Measurement: 5 iterations, single-shot each
   # Timeout: 10 min per iteration
   # Threads: 1 thread
   # Benchmark mode: Single shot invocation time
   # Benchmark: org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadFlatParquetDataBenchmark.readTimestampsSparkVectorized5k
   
   # Run progress: 93.75% complete, ETA 00:03:47
   # Fork: 1 of 1
   # Warmup Iteration   1: 2.126 s/op
   # Warmup Iteration   2: 1.573 s/op
   # Warmup Iteration   3: 1.538 s/op
   Iteration   1: 1.509 s/op
   Iteration   2: 1.482 s/op
   Iteration   3: 1.497 s/op
   Iteration   4: 1.487 s/op
   Iteration   5: 1.447 s/op
   
   
   Result "org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadFlatParquetDataBenchmark.readTimestampsSparkVectorized5k":
     N = 5
     mean =      1.485 ±(99.9%) 0.090 s/op
   
     Histogram, s/op:
       [1.440, 1.445) = 0 
       [1.445, 1.450) = 1 
       [1.450, 1.455) = 0 
       [1.455, 1.460) = 0 
       [1.460, 1.465) = 0 
       [1.465, 1.470) = 0 
       [1.470, 1.475) = 0 
       [1.475, 1.480) = 0 
       [1.480, 1.485) = 1 
       [1.485, 1.490) = 1 
       [1.490, 1.495) = 0 
       [1.495, 1.500) = 1 
       [1.500, 1.505) = 0 
       [1.505, 1.510) = 1 
   
     Percentiles, s/op:
         p(0.0000) =      1.447 s/op
        p(50.0000) =      1.487 s/op
        p(90.0000) =      1.509 s/op
        p(95.0000) =      1.509 s/op
        p(99.0000) =      1.509 s/op
        p(99.9000) =      1.509 s/op
        p(99.9900) =      1.509 s/op
        p(99.9990) =      1.509 s/op
        p(99.9999) =      1.509 s/op
       p(100.0000) =      1.509 s/op
   
   
   # Run complete. Total time: 01:00:29
   
   REMEMBER: The numbers below are just data. To gain reusable insights, you need to follow up on
   why the numbers are the way they are. Use profilers (see -prof, -lprof), design factorial
   experiments, perform baseline and negative tests that provide experimental control, make sure
   the benchmarking environment is safe on JVM/OS/HW level, ask for reviews from the domain experts.
   Do not assume the numbers tell you what you want them to tell.
   
   Benchmark                                                                 Mode  Cnt  Score   Error  Units
   VectorizedReadFlatParquetDataBenchmark.readDatesIcebergVectorized5k         ss    5  1.490 ± 0.128   s/op
   VectorizedReadFlatParquetDataBenchmark.readDatesSparkVectorized5k           ss    5  1.294 ± 0.111   s/op
   VectorizedReadFlatParquetDataBenchmark.readDecimalsIcebergVectorized5k      ss    5  8.642 ± 0.239   s/op
   VectorizedReadFlatParquetDataBenchmark.readDecimalsSparkVectorized5k        ss    5  8.444 ± 0.096   s/op
   VectorizedReadFlatParquetDataBenchmark.readDoublesIcebergVectorized5k       ss    5  2.558 ± 0.248   s/op
   VectorizedReadFlatParquetDataBenchmark.readDoublesSparkVectorized5k         ss    5  2.372 ± 0.093   s/op
   VectorizedReadFlatParquetDataBenchmark.readFloatsIcebergVectorized5k        ss    5  2.378 ± 0.083   s/op
   VectorizedReadFlatParquetDataBenchmark.readFloatsSparkVectorized5k          ss    5  2.208 ± 0.110   s/op
   VectorizedReadFlatParquetDataBenchmark.readIntegersIcebergVectorized5k      ss    5  2.430 ± 0.118   s/op
   VectorizedReadFlatParquetDataBenchmark.readIntegersSparkVectorized5k        ss    5  2.302 ± 0.126   s/op
   VectorizedReadFlatParquetDataBenchmark.readLongsIcebergVectorized5k         ss    5  2.732 ± 0.671   s/op
   VectorizedReadFlatParquetDataBenchmark.readLongsSparkVectorized5k           ss    5  2.327 ± 0.076   s/op
   VectorizedReadFlatParquetDataBenchmark.readStringsIcebergVectorized5k       ss    5  4.156 ± 0.075   s/op
   VectorizedReadFlatParquetDataBenchmark.readStringsSparkVectorized5k         ss    5  3.978 ± 0.155   s/op
   VectorizedReadFlatParquetDataBenchmark.readTimestampsIcebergVectorized5k    ss    5  1.539 ± 0.150   s/op
   VectorizedReadFlatParquetDataBenchmark.readTimestampsSparkVectorized5k      ss    5  1.485 ± 0.090   s/op
   
   Benchmark result is saved to XXX/iceberg_master/spark2/build/reports/jmh/results.txt
   
   ```
   


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] mayursrivastava commented on pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
mayursrivastava commented on pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#issuecomment-818977125


   Hi, @rymurr I've updated the pr with your suggestions. Please take a look. 
   Some Java CI tests, however, timed out. I read in the community mailing list that CI has some issues. Is this related?


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] mayursrivastava commented on a change in pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
mayursrivastava commented on a change in pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#discussion_r591983868



##########
File path: data/src/main/java/org/apache/iceberg/data/IcebergGenerics.java
##########
@@ -47,6 +47,10 @@ public ScanBuilder(Table table) {
       this.tableScan = table.newScan();
     }
 
+    public TableScan getTableScan() {

Review comment:
       I'll revert this change

##########
File path: data/src/main/java/org/apache/iceberg/data/arrow/ArrowFileScanTaskReader.java
##########
@@ -0,0 +1,102 @@
+/*
+ * 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.data.arrow;
+
+import org.apache.arrow.vector.NullCheckingForGet;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.CloseableIterator;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Reads the data file and returns an iterator of {@link VectorSchemaRoot}. Only Parquet data file format is supported.
+ */
+public class ArrowFileScanTaskReader extends BaseArrowFileScanTaskReader<VectorSchemaRoot> {
+
+  private final Schema expectedSchema;
+  private final String nameMapping;
+  private final boolean caseSensitive;
+  private final int batchSize;
+  private final boolean reuseContainers;
+
+  /**
+   * Create a new instance.
+   *
+   * @param task              Combined file scan task.
+   * @param expectedSchema    Read schema. The returned data will have this schema.
+   * @param nameMapping       Mapping from external schema names to Iceberg type IDs.
+   * @param fileIo            File I/O.
+   * @param encryptionManager Encryption manager.
+   * @param caseSensitive     Indicates whether column names are case sensitive.
+   * @param batchSize         Batch size in number of rows. Each Arrow batch ({@link VectorSchemaRoot}) contains a
+   *                          maximum of {@code batchSize} rows.
+   * @param reuseContainers   Reuse Arrow vectors from the previous {@link VectorSchemaRoot} in the next {@link

Review comment:
       will do




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] mayursrivastava commented on a change in pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
mayursrivastava commented on a change in pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#discussion_r609627652



##########
File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowVector.java
##########
@@ -0,0 +1,513 @@
+/*
+ * 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.arrow.vectorized;
+
+import java.nio.charset.StandardCharsets;
+import java.util.stream.IntStream;
+import org.apache.arrow.vector.BigIntVector;
+import org.apache.arrow.vector.BitVector;
+import org.apache.arrow.vector.DateDayVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.Float4Vector;
+import org.apache.arrow.vector.Float8Vector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.TimeStampMicroTZVector;
+import org.apache.arrow.vector.TimeStampMicroVector;
+import org.apache.arrow.vector.ValueVector;
+import org.apache.arrow.vector.VarBinaryVector;
+import org.apache.arrow.vector.VarCharVector;
+import org.apache.arrow.vector.holders.NullableVarCharHolder;
+import org.apache.iceberg.types.Types;
+import org.apache.parquet.Preconditions;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.PrimitiveType;
+
+/**
+ * This class is inspired by Spark's {@code ColumnVector}.
+ * This class represents the column data for an Iceberg table query.
+ * It wraps an arrow {@link FieldVector} and provides simple
+ * accessors for the row values. Advanced users can access
+ * the {@link FieldVector}.
+ * <p>
+ *   Supported Iceberg data types:
+ *   <ul>
+ *     <li>{@link Types.BooleanType}</li>
+ *     <li>{@link Types.IntegerType}</li>
+ *     <li>{@link Types.LongType}</li>
+ *     <li>{@link Types.FloatType}</li>
+ *     <li>{@link Types.DoubleType}</li>
+ *     <li>{@link Types.StringType}</li>
+ *     <li>{@link Types.BinaryType}</li>
+ *     <li>{@link Types.TimestampType} (with and without timezone)</li>
+ *     <li>{@link Types.DateType}</li>
+ *   </ul>
+ */
+public class ArrowVector implements AutoCloseable {

Review comment:
       I guess you are suggesting to move the ColumnarBatch and related logic with some changes to arrow module. I can give it a try.




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
rymurr commented on pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#issuecomment-804361221


   Hey @mayursrivastava I am on holiday this week but will get to it first
   thing when I get back. Apologies for the delay!
   
   On Mon, 22 Mar 2021, 21:00 mayursrivastava, ***@***.***>
   wrote:
   
   > Hi @rymurr <https://github.com/rymurr>, I wanted to check if you got a
   > chance to take a look at the updates to this pr. I'm happy to make any more
   > changes. Thanks, Mayur
   >
   > —
   > You are receiving this because you were mentioned.
   > Reply to this email directly, view it on GitHub
   > <https://github.com/apache/iceberg/pull/2286#issuecomment-804354522>, or
   > unsubscribe
   > <https://github.com/notifications/unsubscribe-auth/AAPNXIJ7ACUXMVVSRIPX6MLTE6OVRANCNFSM4YMW4KAQ>
   > .
   >
   


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] mayursrivastava commented on a change in pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
mayursrivastava commented on a change in pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#discussion_r591984042



##########
File path: build.gradle
##########
@@ -250,13 +252,25 @@ project(':iceberg-data') {
       exclude group: 'org.slf4j', module: 'slf4j-log4j12'
     }
 
+    compileOnly("org.apache.arrow:arrow-vector") {

Review comment:
       will move everything to the arrow module




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#discussion_r638280456



##########
File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowReader.java
##########
@@ -0,0 +1,325 @@
+/*
+ * 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.arrow.vectorized;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import java.util.stream.StreamSupport;
+import org.apache.arrow.vector.NullCheckingForGet;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.arrow.vector.types.Types.MinorType;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.encryption.EncryptedFiles;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.CloseableIterator;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.types.Types;
+import org.apache.parquet.schema.MessageType;
+
+/**
+ * Vectorized reader that returns an iterator of {@link ColumnarBatch}.
+ * See {@link #open(CloseableIterable)} ()} to learn about the
+ * behavior of the iterator.
+ *
+ * <p>The following Iceberg data types are supported and have been tested:
+ * <ul>
+ *     <li>Iceberg: {@link Types.BooleanType}, Arrow: {@link MinorType#BIT}</li>
+ *     <li>Iceberg: {@link Types.IntegerType}, Arrow: {@link MinorType#INT}</li>
+ *     <li>Iceberg: {@link Types.LongType}, Arrow: {@link MinorType#BIGINT}</li>
+ *     <li>Iceberg: {@link Types.FloatType}, Arrow: {@link MinorType#FLOAT4}</li>
+ *     <li>Iceberg: {@link Types.DoubleType}, Arrow: {@link MinorType#FLOAT8}</li>
+ *     <li>Iceberg: {@link Types.StringType}, Arrow: {@link MinorType#VARCHAR}</li>
+ *     <li>Iceberg: {@link Types.TimestampType} (both with and without timezone),
+ *         Arrow: {@link MinorType#TIMEMICRO}</li>
+ *     <li>Iceberg: {@link Types.BinaryType}, Arrow: {@link MinorType#VARBINARY}</li>
+ *     <li>Iceberg: {@link Types.DateType}, Arrow: {@link MinorType#DATEDAY}</li>
+ * </ul>
+ *
+ * <p>Features that don't work in this implementation:
+ * <ul>
+ *     <li>Type promotion: In case of type promotion, the Arrow vector corresponding to
+ *     the data type in the parquet file is returned instead of the data type in the latest schema.
+ *     See https://github.com/apache/iceberg/issues/2483.</li>
+ *     <li>Columns with constant values are physically encoded as a dictionary. The Arrow vector
+ *     type is int32 instead of the type as per the schema.
+ *     See https://github.com/apache/iceberg/issues/2484.</li>
+ *     <li>Data types: {@link Types.TimeType}, {@link Types.ListType}, {@link Types.MapType},
+ *     {@link Types.StructType}, {@link Types.UUIDType}, {@link Types.FixedType} and
+ *     {@link Types.DecimalType}
+ *     See https://github.com/apache/iceberg/issues/2485 and https://github.com/apache/iceberg/issues/2486.</li>
+ *     <li>Iceberg v2 spec is not supported.
+ *     See https://github.com/apache/iceberg/issues/2487.</li>
+ * </ul>
+ */
+public class ArrowReader extends CloseableGroup {
+
+  private final Schema schema;
+  private final FileIO io;
+  private final EncryptionManager encryption;
+  private final int batchSize;
+  private final boolean reuseContainers;
+
+  /**
+   * Create a new instance of the reader.
+   *
+   * @param scan the table scan object.
+   * @param batchSize the maximum number of rows per Arrow batch.
+   * @param reuseContainers whether to reuse Arrow vectors when iterating through the data.
+   *                        If set to {@code false}, every {@link Iterator#next()} call creates
+   *                        new instances of Arrow vectors.
+   *                        If set to {@code true}, the Arrow vectors in the previous
+   *                        {@link Iterator#next()} may be reused for the data returned
+   *                        in the current {@link Iterator#next()}.
+   *                        This option avoids allocating memory again and again.
+   *                        Irrespective of the value of {@code reuseContainers}, the Arrow vectors
+   *                        in the previous {@link Iterator#next()} call are closed before creating
+   *                        new instances if the current {@link Iterator#next()}.
+   */
+  public ArrowReader(TableScan scan, int batchSize, boolean reuseContainers) {
+    this.schema = scan.schema();
+    this.io = scan.table().io();
+    this.encryption = scan.table().encryption();
+    this.batchSize = batchSize;
+    // start planning tasks in the background
+    this.reuseContainers = reuseContainers;
+  }
+
+  /**
+   * Returns a new iterator of {@link ColumnarBatch} objects.
+   * <p>
+   * Note that the reader owns the {@link ColumnarBatch} objects and takes care of closing them.
+   * The caller should not hold onto a {@link ColumnarBatch} or try to close them.
+   *
+   * <p>If {@code reuseContainers} is {@code false}, the Arrow vectors in the
+   * previous {@link ColumnarBatch} are closed before returning the next {@link ColumnarBatch} object.
+   * This implies that the caller should either use the {@link ColumnarBatch} or transfer the ownership of
+   * {@link ColumnarBatch} before getting the next {@link ColumnarBatch}.
+   *
+   * <p>If {@code reuseContainers} is {@code true}, the Arrow vectors in the
+   * previous {@link ColumnarBatch} may be reused for the next {@link ColumnarBatch}.
+   * This implies that the caller should either use the {@link ColumnarBatch} or deep copy the
+   * {@link ColumnarBatch} before getting the next {@link ColumnarBatch}.
+   */
+  public CloseableIterator<ColumnarBatch> open(CloseableIterable<CombinedScanTask> tasks) {
+    CloseableIterator<ColumnarBatch> itr = new VectorizedCombinedScanIterator(
+        tasks,
+        schema,
+        null,
+        io,
+        encryption,
+        true,
+        batchSize,
+        reuseContainers
+    );
+    addCloseable(itr);
+    return itr;
+  }
+
+  @Override
+  public void close() throws IOException {
+    super.close(); // close data files
+  }
+
+  /**
+   * Reads the data file and returns an iterator of {@link VectorSchemaRoot}.
+   * Only Parquet data file format is supported.
+   */
+  private static final class VectorizedCombinedScanIterator implements CloseableIterator<ColumnarBatch> {
+
+    private final List<FileScanTask> fileTasks;
+    private final Iterator<FileScanTask> fileItr;
+    private final Map<String, InputFile> inputFiles;
+    private final Schema expectedSchema;
+    private final String nameMapping;
+    private final boolean caseSensitive;
+    private final int batchSize;
+    private final boolean reuseContainers;
+    private CloseableIterator<ColumnarBatch> currentIterator;
+    private ColumnarBatch current;
+    private FileScanTask currentTask;
+
+    /**
+     * Create a new instance.
+     *
+     * @param tasks             Combined file scan tasks.
+     * @param expectedSchema    Read schema. The returned data will have this schema.
+     * @param nameMapping       Mapping from external schema names to Iceberg type IDs.
+     * @param io                File I/O.
+     * @param encryptionManager Encryption manager.
+     * @param caseSensitive     If {@code true}, column names are case sensitive.
+     *                          If {@code false}, column names are not case sensitive.
+     * @param batchSize         Batch size in number of rows. Each Arrow batch contains
+     *                          a maximum of {@code batchSize} rows.
+     * @param reuseContainers   If set to {@code false}, every {@link Iterator#next()} call creates
+     *                          new instances of Arrow vectors.
+     *                          If set to {@code true}, the Arrow vectors in the previous
+     *                          {@link Iterator#next()} may be reused for the data returned
+     *                          in the current {@link Iterator#next()}.
+     *                          This option avoids allocating memory again and again.
+     *                          Irrespective of the value of {@code reuseContainers}, the Arrow vectors
+     *                          in the previous {@link Iterator#next()} call are closed before creating
+     *                          new instances if the current {@link Iterator#next()}.
+     */
+    VectorizedCombinedScanIterator(
+        CloseableIterable<CombinedScanTask> tasks,
+        Schema expectedSchema,
+        String nameMapping,
+        FileIO io,
+        EncryptionManager encryptionManager,
+        boolean caseSensitive,
+        int batchSize,
+        boolean reuseContainers) {
+      this.fileTasks = StreamSupport.stream(tasks.spliterator(), false)
+          .map(CombinedScanTask::files)
+          .flatMap(Collection::stream)
+          .collect(Collectors.toList());
+      this.fileItr = fileTasks.iterator();
+      this.inputFiles = Collections.unmodifiableMap(fileTasks.stream()
+          .flatMap(fileScanTask -> Stream.concat(Stream.of(fileScanTask.file()), fileScanTask.deletes().stream()))
+          .map(file -> EncryptedFiles.encryptedInput(io.newInputFile(file.path().toString()), file.keyMetadata()))
+          .map(encryptionManager::decrypt)

Review comment:
       I think this should use the same logic that's in the [Spark and Flink read paths](https://github.com/apache/iceberg/blob/e8b3d1e8fb50df7ed2662a31d7a212fae87ad2a5/spark/src/main/java/org/apache/iceberg/spark/source/BaseDataReader.java#L68-L72). That uses the bulk decryption interface to avoid extra calls to a key manager and avoids problems with duplicate file entries that have been bugs in the past.




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] mayursrivastava commented on a change in pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
mayursrivastava commented on a change in pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#discussion_r617717549



##########
File path: build.gradle
##########
@@ -735,9 +735,12 @@ project(':iceberg-arrow') {
       exclude group: 'com.google.code.findbugs', module: 'jsr305'
     }
     compile("org.apache.arrow:arrow-memory-netty") {
-      exclude group: 'io.netty', module: 'netty-common'
       exclude group: 'com.google.code.findbugs', module: 'jsr305'

Review comment:
       ArrowReaderTest fails with the following exception if we don't include netty-common. I've changed it to testCompile(). Is there a better way to include netty dependencies for tests?
   Somehow, Spark is importing netty-all transitively (which includes the runtime netty dependencies).
   
   io/netty/util/ReferenceCounted
   java.lang.NoClassDefFoundError: io/netty/util/ReferenceCounted
   	at java.base/java.lang.ClassLoader.defineClass1(Native Method)
   	at java.base/java.lang.ClassLoader.defineClass(ClassLoader.java:1017)
   	at java.base/java.security.SecureClassLoader.defineClass(SecureClassLoader.java:174)
   	at java.base/jdk.internal.loader.BuiltinClassLoader.defineClass(BuiltinClassLoader.java:800)
   	at java.base/jdk.internal.loader.BuiltinClassLoader.findClassOnClassPathOrNull(BuiltinClassLoader.java:698)
   	at java.base/jdk.internal.loader.BuiltinClassLoader.loadClassOrNull(BuiltinClassLoader.java:621)
   	at java.base/jdk.internal.loader.BuiltinClassLoader.loadClass(BuiltinClassLoader.java:579)
   	at java.base/jdk.internal.loader.ClassLoaders$AppClassLoader.loadClass(ClassLoaders.java:178)
   	at java.base/java.lang.ClassLoader.loadClass(ClassLoader.java:522)
   	at io.netty.buffer.PooledByteBufAllocatorL.<init>(PooledByteBufAllocatorL.java:49)
   	at org.apache.arrow.memory.NettyAllocationManager.<clinit>(NettyAllocationManager.java:51)
   	at org.apache.arrow.memory.DefaultAllocationManagerFactory.<clinit>(DefaultAllocationManagerFactory.java:26)
   	at java.base/java.lang.Class.forName0(Native Method)
   	at java.base/java.lang.Class.forName(Class.java:315)
   	at org.apache.arrow.memory.DefaultAllocationManagerOption.getFactory(DefaultAllocationManagerOption.java:108)
   	at org.apache.arrow.memory.DefaultAllocationManagerOption.getDefaultAllocationManagerFactory(DefaultAllocationManagerOption.java:98)
   	at org.apache.arrow.memory.BaseAllocator$Config.getAllocationManagerFactory(BaseAllocator.java:722)
   	at org.apache.arrow.memory.ImmutableConfig.access$801(ImmutableConfig.java:24)
   	at org.apache.arrow.memory.ImmutableConfig$InitShim.getAllocationManagerFactory(ImmutableConfig.java:83)
   	at org.apache.arrow.memory.ImmutableConfig.<init>(ImmutableConfig.java:47)
   	at org.apache.arrow.memory.ImmutableConfig.<init>(ImmutableConfig.java:24)
   	at org.apache.arrow.memory.ImmutableConfig$Builder.build(ImmutableConfig.java:485)
   	at org.apache.arrow.memory.BaseAllocator.<clinit>(BaseAllocator.java:61)
   	at org.apache.iceberg.arrow.ArrowAllocation.<clinit>(ArrowAllocation.java:26)
   	at org.apache.iceberg.arrow.vectorized.VectorizedReaderBuilder.<init>(VectorizedReaderBuilder.java:57)
   	at org.apache.iceberg.arrow.vectorized.ArrowReader$VectorizedCombinedScanIterator.buildReader(ArrowReader.java:324)
   	at org.apache.iceberg.arrow.vectorized.ArrowReader$VectorizedCombinedScanIterator.lambda$open$4(ArrowReader.java:273)
   	at org.apache.iceberg.parquet.ReadConf.<init>(ReadConf.java:123)
   	at org.apache.iceberg.parquet.VectorizedParquetReader.init(VectorizedParquetReader.java:76)
   	at org.apache.iceberg.parquet.VectorizedParquetReader.iterator(VectorizedParquetReader.java:85)
   	at org.apache.iceberg.arrow.vectorized.ArrowReader$VectorizedCombinedScanIterator.open(ArrowReader.java:291)
   	at org.apache.iceberg.arrow.vectorized.ArrowReader$VectorizedCombinedScanIterator.hasNext(ArrowReader.java:246)
   	at org.apache.iceberg.arrow.vectorized.ArrowReaderTest.readAndCheckColumnarBatch(ArrowReaderTest.java:316)
   	at org.apache.iceberg.arrow.vectorized.ArrowReaderTest.readAndCheckQueryResult(ArrowReaderTest.java:305)
   	at org.apache.iceberg.arrow.vectorized.ArrowReaderTest.testReadRangeFilter(ArrowReaderTest.java:234)
   	at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
   	at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
   	at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
   	at java.base/java.lang.reflect.Method.invoke(Method.java:566)
   	at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:50)
   	at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
   	at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:47)
   	at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
   	at org.junit.rules.ExternalResource$1.evaluate(ExternalResource.java:48)
   	at org.junit.rules.RunRules.evaluate(RunRules.java:20)
   	at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:325)
   	at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:78)
   	at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:57)
   	at org.junit.runners.ParentRunner$3.run(ParentRunner.java:290)
   	at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:71)
   	at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:288)
   	at org.junit.runners.ParentRunner.access$000(ParentRunner.java:58)
   	at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:268)
   	at org.junit.runners.ParentRunner.run(ParentRunner.java:363)
   	at org.gradle.api.internal.tasks.testing.junit.JUnitTestClassExecutor.runTestClass(JUnitTestClassExecutor.java:110)
   	at org.gradle.api.internal.tasks.testing.junit.JUnitTestClassExecutor.execute(JUnitTestClassExecutor.java:58)
   	at org.gradle.api.internal.tasks.testing.junit.JUnitTestClassExecutor.execute(JUnitTestClassExecutor.java:38)
   	at org.gradle.api.internal.tasks.testing.junit.AbstractJUnitTestClassProcessor.processTestClass(AbstractJUnitTestClassProcessor.java:62)
   	at org.gradle.api.internal.tasks.testing.SuiteTestClassProcessor.processTestClass(SuiteTestClassProcessor.java:51)
   	at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
   	at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
   	at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
   	at java.base/java.lang.reflect.Method.invoke(Method.java:566)
   	at org.gradle.internal.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:35)
   	at org.gradle.internal.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:24)
   	at org.gradle.internal.dispatch.ContextClassLoaderDispatch.dispatch(ContextClassLoaderDispatch.java:32)
   	at org.gradle.internal.dispatch.ProxyDispatchAdapter$DispatchingInvocationHandler.invoke(ProxyDispatchAdapter.java:93)
   	at com.sun.proxy.$Proxy2.processTestClass(Unknown Source)
   	at org.gradle.api.internal.tasks.testing.worker.TestWorker.processTestClass(TestWorker.java:118)
   	at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
   	at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
   	at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
   	at java.base/java.lang.reflect.Method.invoke(Method.java:566)
   	at org.gradle.internal.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:35)
   	at org.gradle.internal.dispatch.ReflectionDispatch.dispatch(ReflectionDispatch.java:24)
   	at org.gradle.internal.remote.internal.hub.MessageHubBackedObjectConnection$DispatchWrapper.dispatch(MessageHubBackedObjectConnection.java:175)
   	at org.gradle.internal.remote.internal.hub.MessageHubBackedObjectConnection$DispatchWrapper.dispatch(MessageHubBackedObjectConnection.java:157)
   	at org.gradle.internal.remote.internal.hub.MessageHub$Handler.run(MessageHub.java:404)
   	at org.gradle.internal.concurrent.ExecutorPolicy$CatchAndRecordFailures.onExecute(ExecutorPolicy.java:63)
   	at org.gradle.internal.concurrent.ManagedExecutorImpl$1.run(ManagedExecutorImpl.java:46)
   	at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
   	at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
   	at org.gradle.internal.concurrent.ThreadFactoryImpl$ManagedThreadRunnable.run(ThreadFactoryImpl.java:55)
   	at java.base/java.lang.Thread.run(Thread.java:834)
   Caused by: java.lang.ClassNotFoundException: io.netty.util.ReferenceCounted
   	at java.base/jdk.internal.loader.BuiltinClassLoader.loadClass(BuiltinClassLoader.java:581)
   	at java.base/jdk.internal.loader.ClassLoaders$AppClassLoader.loadClass(ClassLoaders.java:178)
   	at java.base/java.lang.ClassLoader.loadClass(ClassLoader.java:522)
   	... 84 more
   
   




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] mayursrivastava commented on a change in pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
mayursrivastava commented on a change in pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#discussion_r682568668



##########
File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowReader.java
##########
@@ -0,0 +1,325 @@
+/*
+ * 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.arrow.vectorized;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import java.util.stream.StreamSupport;
+import org.apache.arrow.vector.NullCheckingForGet;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.arrow.vector.types.Types.MinorType;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.encryption.EncryptedFiles;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.CloseableIterator;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.types.Types;
+import org.apache.parquet.schema.MessageType;
+
+/**
+ * Vectorized reader that returns an iterator of {@link ColumnarBatch}.
+ * See {@link #open(CloseableIterable)} ()} to learn about the
+ * behavior of the iterator.
+ *
+ * <p>The following Iceberg data types are supported and have been tested:
+ * <ul>
+ *     <li>Iceberg: {@link Types.BooleanType}, Arrow: {@link MinorType#BIT}</li>
+ *     <li>Iceberg: {@link Types.IntegerType}, Arrow: {@link MinorType#INT}</li>
+ *     <li>Iceberg: {@link Types.LongType}, Arrow: {@link MinorType#BIGINT}</li>
+ *     <li>Iceberg: {@link Types.FloatType}, Arrow: {@link MinorType#FLOAT4}</li>
+ *     <li>Iceberg: {@link Types.DoubleType}, Arrow: {@link MinorType#FLOAT8}</li>
+ *     <li>Iceberg: {@link Types.StringType}, Arrow: {@link MinorType#VARCHAR}</li>
+ *     <li>Iceberg: {@link Types.TimestampType} (both with and without timezone),
+ *         Arrow: {@link MinorType#TIMEMICRO}</li>
+ *     <li>Iceberg: {@link Types.BinaryType}, Arrow: {@link MinorType#VARBINARY}</li>
+ *     <li>Iceberg: {@link Types.DateType}, Arrow: {@link MinorType#DATEDAY}</li>
+ * </ul>
+ *
+ * <p>Features that don't work in this implementation:
+ * <ul>
+ *     <li>Type promotion: In case of type promotion, the Arrow vector corresponding to
+ *     the data type in the parquet file is returned instead of the data type in the latest schema.
+ *     See https://github.com/apache/iceberg/issues/2483.</li>
+ *     <li>Columns with constant values are physically encoded as a dictionary. The Arrow vector
+ *     type is int32 instead of the type as per the schema.
+ *     See https://github.com/apache/iceberg/issues/2484.</li>
+ *     <li>Data types: {@link Types.TimeType}, {@link Types.ListType}, {@link Types.MapType},
+ *     {@link Types.StructType}, {@link Types.UUIDType}, {@link Types.FixedType} and
+ *     {@link Types.DecimalType}
+ *     See https://github.com/apache/iceberg/issues/2485 and https://github.com/apache/iceberg/issues/2486.</li>
+ *     <li>Iceberg v2 spec is not supported.
+ *     See https://github.com/apache/iceberg/issues/2487.</li>
+ * </ul>
+ */
+public class ArrowReader extends CloseableGroup {
+
+  private final Schema schema;
+  private final FileIO io;
+  private final EncryptionManager encryption;
+  private final int batchSize;
+  private final boolean reuseContainers;
+
+  /**
+   * Create a new instance of the reader.
+   *
+   * @param scan the table scan object.
+   * @param batchSize the maximum number of rows per Arrow batch.
+   * @param reuseContainers whether to reuse Arrow vectors when iterating through the data.
+   *                        If set to {@code false}, every {@link Iterator#next()} call creates
+   *                        new instances of Arrow vectors.
+   *                        If set to {@code true}, the Arrow vectors in the previous
+   *                        {@link Iterator#next()} may be reused for the data returned
+   *                        in the current {@link Iterator#next()}.
+   *                        This option avoids allocating memory again and again.
+   *                        Irrespective of the value of {@code reuseContainers}, the Arrow vectors
+   *                        in the previous {@link Iterator#next()} call are closed before creating
+   *                        new instances if the current {@link Iterator#next()}.
+   */
+  public ArrowReader(TableScan scan, int batchSize, boolean reuseContainers) {
+    this.schema = scan.schema();
+    this.io = scan.table().io();
+    this.encryption = scan.table().encryption();
+    this.batchSize = batchSize;
+    // start planning tasks in the background
+    this.reuseContainers = reuseContainers;
+  }
+
+  /**
+   * Returns a new iterator of {@link ColumnarBatch} objects.
+   * <p>
+   * Note that the reader owns the {@link ColumnarBatch} objects and takes care of closing them.
+   * The caller should not hold onto a {@link ColumnarBatch} or try to close them.
+   *
+   * <p>If {@code reuseContainers} is {@code false}, the Arrow vectors in the
+   * previous {@link ColumnarBatch} are closed before returning the next {@link ColumnarBatch} object.
+   * This implies that the caller should either use the {@link ColumnarBatch} or transfer the ownership of
+   * {@link ColumnarBatch} before getting the next {@link ColumnarBatch}.
+   *
+   * <p>If {@code reuseContainers} is {@code true}, the Arrow vectors in the
+   * previous {@link ColumnarBatch} may be reused for the next {@link ColumnarBatch}.
+   * This implies that the caller should either use the {@link ColumnarBatch} or deep copy the
+   * {@link ColumnarBatch} before getting the next {@link ColumnarBatch}.
+   */
+  public CloseableIterator<ColumnarBatch> open(CloseableIterable<CombinedScanTask> tasks) {
+    CloseableIterator<ColumnarBatch> itr = new VectorizedCombinedScanIterator(
+        tasks,
+        schema,
+        null,
+        io,
+        encryption,
+        true,
+        batchSize,
+        reuseContainers
+    );
+    addCloseable(itr);
+    return itr;
+  }
+
+  @Override
+  public void close() throws IOException {
+    super.close(); // close data files
+  }
+
+  /**
+   * Reads the data file and returns an iterator of {@link VectorSchemaRoot}.
+   * Only Parquet data file format is supported.
+   */
+  private static final class VectorizedCombinedScanIterator implements CloseableIterator<ColumnarBatch> {
+
+    private final List<FileScanTask> fileTasks;
+    private final Iterator<FileScanTask> fileItr;
+    private final Map<String, InputFile> inputFiles;
+    private final Schema expectedSchema;
+    private final String nameMapping;
+    private final boolean caseSensitive;
+    private final int batchSize;
+    private final boolean reuseContainers;
+    private CloseableIterator<ColumnarBatch> currentIterator;
+    private ColumnarBatch current;
+    private FileScanTask currentTask;
+
+    /**
+     * Create a new instance.
+     *
+     * @param tasks             Combined file scan tasks.
+     * @param expectedSchema    Read schema. The returned data will have this schema.
+     * @param nameMapping       Mapping from external schema names to Iceberg type IDs.
+     * @param io                File I/O.
+     * @param encryptionManager Encryption manager.
+     * @param caseSensitive     If {@code true}, column names are case sensitive.
+     *                          If {@code false}, column names are not case sensitive.
+     * @param batchSize         Batch size in number of rows. Each Arrow batch contains
+     *                          a maximum of {@code batchSize} rows.
+     * @param reuseContainers   If set to {@code false}, every {@link Iterator#next()} call creates
+     *                          new instances of Arrow vectors.
+     *                          If set to {@code true}, the Arrow vectors in the previous
+     *                          {@link Iterator#next()} may be reused for the data returned
+     *                          in the current {@link Iterator#next()}.
+     *                          This option avoids allocating memory again and again.
+     *                          Irrespective of the value of {@code reuseContainers}, the Arrow vectors
+     *                          in the previous {@link Iterator#next()} call are closed before creating
+     *                          new instances if the current {@link Iterator#next()}.
+     */
+    VectorizedCombinedScanIterator(
+        CloseableIterable<CombinedScanTask> tasks,
+        Schema expectedSchema,
+        String nameMapping,
+        FileIO io,
+        EncryptionManager encryptionManager,
+        boolean caseSensitive,
+        int batchSize,
+        boolean reuseContainers) {
+      this.fileTasks = StreamSupport.stream(tasks.spliterator(), false)
+          .map(CombinedScanTask::files)
+          .flatMap(Collection::stream)
+          .collect(Collectors.toList());
+      this.fileItr = fileTasks.iterator();
+      this.inputFiles = Collections.unmodifiableMap(fileTasks.stream()
+          .flatMap(fileScanTask -> Stream.concat(Stream.of(fileScanTask.file()), fileScanTask.deletes().stream()))
+          .map(file -> EncryptedFiles.encryptedInput(io.newInputFile(file.path().toString()), file.keyMetadata()))
+          .map(encryptionManager::decrypt)
+          .collect(Collectors.toMap(InputFile::location, Function.identity())));
+      this.currentIterator = CloseableIterator.empty();
+      this.expectedSchema = expectedSchema;
+      this.nameMapping = nameMapping;
+      this.caseSensitive = caseSensitive;
+      this.batchSize = batchSize;
+      this.reuseContainers = reuseContainers;
+    }
+
+    @Override
+    public boolean hasNext() {
+      try {
+        while (true) {
+          if (currentIterator.hasNext()) {
+            this.current = currentIterator.next();
+            return true;
+          } else if (fileItr.hasNext()) {
+            this.currentIterator.close();
+            this.currentTask = fileItr.next();
+            this.currentIterator = open(currentTask);
+          } else {
+            this.currentIterator.close();
+            return false;
+          }
+        }
+      } catch (IOException | RuntimeException e) {
+        if (currentTask != null && !currentTask.isDataTask()) {
+          throw new RuntimeException(
+              "Error reading file: " + getInputFile(currentTask).location() +
+                  ". Reason: the current task is not a data task, i.e. it cannot read data rows. " +

Review comment:
       Falling back to the same logic as in the BaseDataReader.
   But, the Iterator doesn't support throwing checked exceptions, so I still have to wrap the exception as a RuntimeException.
   Fixes present in https://github.com/apache/iceberg/pull/2933.
   




-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#discussion_r585433534



##########
File path: data/src/main/java/org/apache/iceberg/data/IcebergGenerics.java
##########
@@ -47,6 +47,10 @@ public ScanBuilder(Table table) {
       this.tableScan = table.newScan();
     }
 
+    public TableScan getTableScan() {

Review comment:
       This seems odd to me at first glance. What is the motivation for this change? Apologies if it becomes obvious later, i am early in my review

##########
File path: data/src/main/java/org/apache/iceberg/data/arrow/ArrowFileScanTaskReader.java
##########
@@ -0,0 +1,102 @@
+/*
+ * 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.data.arrow;
+
+import org.apache.arrow.vector.NullCheckingForGet;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.CloseableIterator;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Reads the data file and returns an iterator of {@link VectorSchemaRoot}. Only Parquet data file format is supported.
+ */
+public class ArrowFileScanTaskReader extends BaseArrowFileScanTaskReader<VectorSchemaRoot> {
+
+  private final Schema expectedSchema;
+  private final String nameMapping;
+  private final boolean caseSensitive;
+  private final int batchSize;
+  private final boolean reuseContainers;
+
+  /**
+   * Create a new instance.
+   *
+   * @param task              Combined file scan task.
+   * @param expectedSchema    Read schema. The returned data will have this schema.
+   * @param nameMapping       Mapping from external schema names to Iceberg type IDs.
+   * @param fileIo            File I/O.
+   * @param encryptionManager Encryption manager.
+   * @param caseSensitive     Indicates whether column names are case sensitive.
+   * @param batchSize         Batch size in number of rows. Each Arrow batch ({@link VectorSchemaRoot}) contains a
+   *                          maximum of {@code batchSize} rows.
+   * @param reuseContainers   Reuse Arrow vectors from the previous {@link VectorSchemaRoot} in the next {@link

Review comment:
       when should this boolean be flipped and what are the consequences of doing so? Maybe just a bit more documentation.

##########
File path: data/src/main/java/org/apache/iceberg/data/arrow/BaseArrowFileScanTaskReader.java
##########
@@ -0,0 +1,124 @@
+/*
+ * 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.data.arrow;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.stream.Stream;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.encryption.EncryptedFiles;
+import org.apache.iceberg.encryption.EncryptedInputFile;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.io.CloseableIterator;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Base class for vectorized Arrow reader.
+ */
+abstract class BaseArrowFileScanTaskReader<T> implements CloseableIterator<T> {

Review comment:
       I think this should be called `VectorizedTableScanIterable` similar to `TableScanIterable` but for vectorized reads. I am not sure it should be parameterised. As discussed before I would prefer to expose a simpler interface than `VectorSchemaRoot`. Rather similar to `ColumnBatch` but not using the actual spark code. The idea being it can be a non-arrow specific wrapper around `VectorSchemaRoot` that deals with lifetime etc for 'regular' users and exposes a getter for `VectorSchemaRoot` for expert users.

##########
File path: build.gradle
##########
@@ -239,6 +239,8 @@ project(':iceberg-data') {
     compile project(':iceberg-core')
     compileOnly project(':iceberg-parquet')
     compileOnly project(':iceberg-orc')
+    compileOnly project(':iceberg-arrow')

Review comment:
       I am not sure how this works as compile only? This means that it won't be added to downstream  modules (eg spark) where there may be different arrow versions but that also means no one can use it unless they also explicitly pull in all the arrow deps.
   
   Perhaps everything can go into the arrow module? If required here we can instantiate by reflection if the arrow module is present on teh classpath?

##########
File path: build.gradle
##########
@@ -250,13 +252,25 @@ project(':iceberg-data') {
       exclude group: 'org.slf4j', module: 'slf4j-log4j12'
     }
 
+    compileOnly("org.apache.arrow:arrow-vector") {

Review comment:
       What is this needed for? I think we should keep everything `arrow` in the arrow module.

##########
File path: data/src/main/java/org/apache/iceberg/data/arrow/VectorSchemaRootReader.java
##########
@@ -0,0 +1,112 @@
+/*
+ * 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.data.arrow;
+
+import java.util.List;
+import java.util.Map;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.iceberg.arrow.vectorized.VectorHolder;
+import org.apache.iceberg.arrow.vectorized.VectorizedArrowReader;
+import org.apache.iceberg.parquet.VectorizedReader;
+import org.apache.parquet.Preconditions;
+import org.apache.parquet.column.page.PageReadStore;
+import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
+import org.apache.parquet.hadoop.metadata.ColumnPath;
+
+/**
+ * A collection of vectorized readers per column (in the expected read schema) and Arrow Vector holders. This class owns
+ * the Arrow vectors and is responsible for closing the Arrow vectors.
+ */
+public class VectorSchemaRootReader implements VectorizedReader<VectorSchemaRoot> {

Review comment:
       Rather than reimplementing this perhaps it can use the existing `VectorizedArrowReader`?

##########
File path: data/src/main/java/org/apache/iceberg/data/arrow/ArrowFileScanTaskReader.java
##########
@@ -0,0 +1,102 @@
+/*
+ * 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.data.arrow;
+
+import org.apache.arrow.vector.NullCheckingForGet;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.CloseableIterator;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Reads the data file and returns an iterator of {@link VectorSchemaRoot}. Only Parquet data file format is supported.
+ */
+public class ArrowFileScanTaskReader extends BaseArrowFileScanTaskReader<VectorSchemaRoot> {

Review comment:
       I think this can be folded into the base class

##########
File path: data/src/main/java/org/apache/iceberg/data/arrow/VectorizedParquetReaders.java
##########
@@ -0,0 +1,155 @@
+/*
+ * 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.data.arrow;
+
+import java.util.List;
+import java.util.Map;
+import java.util.stream.IntStream;
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.arrow.ArrowAllocation;
+import org.apache.iceberg.arrow.vectorized.VectorizedArrowReader;
+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.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+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;
+
+/**
+ * Builds an {@link VectorSchemaRootReader}.
+ */
+public class VectorizedParquetReaders {

Review comment:
       What is the difference between this and the spark one? Can we just merge them? I would think that this could go in teh `arrow` module and be called from spark?

##########
File path: data/src/main/java/org/apache/iceberg/data/arrow/ArrowReader.java
##########
@@ -0,0 +1,219 @@
+/*
+ * 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.data.arrow;
+
+import java.io.IOException;
+import java.util.Iterator;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.arrow.vector.types.Types.MinorType;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.CloseableIterator;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.types.Types;
+
+/**
+ * Vectorized reader that returns an iterator of {@link VectorSchemaRoot}. See {@link #iterator()} to learn about the
+ * behavior of the iterator.
+ *
+ * <p>The following Iceberg data types are supported and have been tested:
+ * <ul>
+ *     <li>Iceberg: {@link Types.BooleanType}, Arrow: {@link MinorType#BIT}</li>
+ *     <li>Iceberg: {@link Types.IntegerType}, Arrow: {@link MinorType#INT}</li>
+ *     <li>Iceberg: {@link Types.LongType}, Arrow: {@link MinorType#BIGINT}</li>
+ *     <li>Iceberg: {@link Types.FloatType}, Arrow: {@link MinorType#FLOAT4}</li>
+ *     <li>Iceberg: {@link Types.DoubleType}, Arrow: {@link MinorType#FLOAT8}</li>
+ *     <li>Iceberg: {@link Types.StringType}, Arrow: {@link MinorType#VARCHAR}</li>
+ *     <li>Iceberg: {@link Types.TimestampType} (both with and without timezone),
+ *         Arrow: {@link MinorType#TIMEMICRO}</li>
+ *     <li>Iceberg: {@link Types.BinaryType}, Arrow: {@link MinorType#VARBINARY}</li>
+ *     <li>Iceberg: {@link Types.DateType}, Arrow: {@link MinorType#DATEDAY}</li>
+ * </ul>
+ *
+ * <p>Features that don't work in this implementation:
+ * <ul>
+ *     <li>Type promotion: In case of type promotion, the Arrow vector corresponding to
+ *     the data type in the parquet file is returned instead of the data type in the latest schema.</li>
+ *     <li>Columns with constant values are physically encoded as a dictionary. The Arrow vector
+ *     type is int32 instead of the type as per the schema.</li>
+ *     <li>Data types: {@link Types.TimeType}, {@link Types.ListType}, {@link Types.MapType},
+ *     {@link Types.StructType}</li>
+ * </ul>
+ *
+ * <p>Data types not tested: {@link Types.UUIDType}, {@link Types.FixedType}, {@link Types.DecimalType}.
+ */
+public class ArrowReader extends CloseableGroup implements CloseableIterable<VectorSchemaRoot> {

Review comment:
       Can this follow the same pattern as `GenericReader`? Does it need to be iterable or can it serve iterables




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
rymurr commented on pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#issuecomment-819363467


   > Hi, @rymurr I've updated the pr with your suggestions. Please take a look.
   > Some Java CI tests, however, timed out. I read in the community mailing list that CI has some issues. Is this related?
   
   Thanks a lot @mayursrivastava I will take a look first thing tomorrow.
   
   The build does indeed appear to be unstable, we can rebase this once its back again. Lets see what #2475 and #2474 turn up


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] mayursrivastava commented on a change in pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
mayursrivastava commented on a change in pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#discussion_r618416064



##########
File path: build.gradle
##########
@@ -735,9 +735,12 @@ project(':iceberg-arrow') {
       exclude group: 'com.google.code.findbugs', module: 'jsr305'
     }
     compile("org.apache.arrow:arrow-memory-netty") {
-      exclude group: 'io.netty', module: 'netty-common'
       exclude group: 'com.google.code.findbugs', module: 'jsr305'

Review comment:
       That works but the version needs to be added in the versions.props. If this is fine, I'll make the change.
   
   $ git diff
   diff --git a/build.gradle b/build.gradle
   index 75c69935..14f9bfae 100644
   --- a/build.gradle
   +++ b/build.gradle
   @@ -741,7 +741,7 @@ project(':iceberg-arrow') {
    
        testCompile project(path: ':iceberg-core', configuration: 'testArtifacts')
        // netty-common is needed to run ArrowReaderTest test cases.
   -    testCompile("org.apache.arrow:arrow-memory-netty")
   +    testCompile("io.netty:netty-common")
        testCompile("org.apache.hadoop:hadoop-common")
        testCompile("org.apache.hadoop:hadoop-mapreduce-client-core")
      }
   diff --git a/versions.props b/versions.props
   index dc4caddd..a147f50c 100644
   --- a/versions.props
   +++ b/versions.props
   @@ -31,3 +31,4 @@ org.apache.hive:hive-service = 2.3.8
    org.apache.tez:tez-dag = 0.8.4
    org.apache.tez:tez-mapreduce = 0.8.4
    com.adobe.testing:s3mock-junit4 = 2.1.28
   +io.netty:netty-common = 4.1.48.Final
   




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] mayursrivastava commented on a change in pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
mayursrivastava commented on a change in pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#discussion_r617715119



##########
File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowReader.java
##########
@@ -0,0 +1,331 @@
+/*
+ * 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.arrow.vectorized;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import java.util.stream.StreamSupport;
+import org.apache.arrow.vector.NullCheckingForGet;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.arrow.vector.types.Types.MinorType;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.encryption.EncryptedFiles;
+import org.apache.iceberg.encryption.EncryptedInputFile;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.CloseableIterator;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.types.Types;
+import org.apache.parquet.schema.MessageType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Vectorized reader that returns an iterator of {@link ColumnarBatch}.
+ * See {@link #open(CloseableIterable)} ()} to learn about the
+ * behavior of the iterator.
+ *
+ * <p>The following Iceberg data types are supported and have been tested:
+ * <ul>
+ *     <li>Iceberg: {@link Types.BooleanType}, Arrow: {@link MinorType#BIT}</li>
+ *     <li>Iceberg: {@link Types.IntegerType}, Arrow: {@link MinorType#INT}</li>
+ *     <li>Iceberg: {@link Types.LongType}, Arrow: {@link MinorType#BIGINT}</li>
+ *     <li>Iceberg: {@link Types.FloatType}, Arrow: {@link MinorType#FLOAT4}</li>
+ *     <li>Iceberg: {@link Types.DoubleType}, Arrow: {@link MinorType#FLOAT8}</li>
+ *     <li>Iceberg: {@link Types.StringType}, Arrow: {@link MinorType#VARCHAR}</li>
+ *     <li>Iceberg: {@link Types.TimestampType} (both with and without timezone),
+ *         Arrow: {@link MinorType#TIMEMICRO}</li>
+ *     <li>Iceberg: {@link Types.BinaryType}, Arrow: {@link MinorType#VARBINARY}</li>
+ *     <li>Iceberg: {@link Types.DateType}, Arrow: {@link MinorType#DATEDAY}</li>
+ * </ul>
+ *
+ * <p>Features that don't work in this implementation:
+ * <ul>
+ *     <li>Type promotion: In case of type promotion, the Arrow vector corresponding to
+ *     the data type in the parquet file is returned instead of the data type in the latest schema.
+ *     See https://github.com/apache/iceberg/issues/2483.</li>
+ *     <li>Columns with constant values are physically encoded as a dictionary. The Arrow vector
+ *     type is int32 instead of the type as per the schema.
+ *     See https://github.com/apache/iceberg/issues/2484.</li>
+ *     <li>Data types: {@link Types.TimeType}, {@link Types.ListType}, {@link Types.MapType},
+ *     {@link Types.StructType}, {@link Types.UUIDType}, {@link Types.FixedType} and
+ *     {@link Types.DecimalType}
+ *     See https://github.com/apache/iceberg/issues/2485 and https://github.com/apache/iceberg/issues/2486.</li>
+ *     <li>Iceberg v2 spec is not supported.
+ *     See https://github.com/apache/iceberg/issues/2487.</li>
+ * </ul>
+ */
+public class ArrowReader extends CloseableGroup {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ArrowReader.class);
+
+  private final Schema schema;
+  private final FileIO io;
+  private final EncryptionManager encryption;
+  private final int batchSize;
+  private final boolean reuseContainers;
+
+  /**
+   * Create a new instance of the reader.
+   *
+   * @param scan the table scan object.
+   * @param batchSize the maximum number of rows per Arrow batch.
+   * @param reuseContainers whether to reuse Arrow vectors when iterating through the data.
+   *                        If set to {@code false}, every {@link Iterator#next()} call creates
+   *                        new instances of Arrow vectors.
+   *                        If set to {@code true}, the Arrow vectors in the previous
+   *                        {@link Iterator#next()} may be reused for the data returned
+   *                        in the current {@link Iterator#next()}.
+   *                        This option avoids allocating memory again and again.
+   *                        Irrespective of the value of {@code reuseContainers}, the Arrow vectors
+   *                        in the previous {@link Iterator#next()} call are closed before creating
+   *                        new instances if the current {@link Iterator#next()}.
+   */
+  public ArrowReader(TableScan scan, int batchSize, boolean reuseContainers) {
+    this.schema = scan.schema();
+    this.io = scan.table().io();
+    this.encryption = scan.table().encryption();
+    this.batchSize = batchSize;
+    // start planning tasks in the background
+    this.reuseContainers = reuseContainers;
+  }
+
+  /**
+   * Returns a new iterator of {@link ColumnarBatch} objects.
+   * <p>
+   * Note that the reader owns the {@link ColumnarBatch} objects and takes care of closing them.
+   * The caller should not hold onto a {@link ColumnarBatch} or try to close them.
+   *
+   * <p>If {@code reuseContainers} is {@code false}, the Arrow vectors in the
+   * previous {@link ColumnarBatch} are closed before returning the next {@link ColumnarBatch} object.
+   * This implies that the caller should either use the {@link ColumnarBatch} or transfer the ownership of
+   * {@link ColumnarBatch} before getting the next {@link ColumnarBatch}.
+   *
+   * <p>If {@code reuseContainers} is {@code true}, the Arrow vectors in the
+   * previous {@link ColumnarBatch} may be reused for the next {@link ColumnarBatch}.
+   * This implies that the caller should either use the {@link ColumnarBatch} or deep copy the
+   * {@link ColumnarBatch} before getting the next {@link ColumnarBatch}.
+   */
+  public CloseableIterator<ColumnarBatch> open(CloseableIterable<CombinedScanTask> tasks) {
+    CloseableIterator<ColumnarBatch> itr = new VectorizedCombinedScanIterator(
+        tasks,
+        schema,
+        null,
+        io,
+        encryption,
+        true,
+        batchSize,
+        reuseContainers
+    );
+    addCloseable(itr);
+    return itr;
+  }
+
+  @Override
+  public void close() throws IOException {
+    super.close(); // close data files
+  }
+
+  /**
+   * Reads the data file and returns an iterator of {@link VectorSchemaRoot}.
+   * Only Parquet data file format is supported.
+   */
+  private static final class VectorizedCombinedScanIterator implements CloseableIterator<ColumnarBatch> {
+
+    private final Iterator<FileScanTask> fileItr;
+    private final Map<String, InputFile> inputFiles;
+    private final Schema expectedSchema;
+    private final String nameMapping;
+    private final boolean caseSensitive;
+    private final int batchSize;
+    private final boolean reuseContainers;
+    private CloseableIterator<ColumnarBatch> currentIterator;
+    private ColumnarBatch current;
+    private FileScanTask currentTask;
+
+    /**
+     * Create a new instance.
+     *
+     * @param tasks             Combined file scan tasks.
+     * @param expectedSchema    Read schema. The returned data will have this schema.
+     * @param nameMapping       Mapping from external schema names to Iceberg type IDs.
+     * @param io                File I/O.
+     * @param encryptionManager Encryption manager.
+     * @param caseSensitive     If {@code true}, column names are case sensitive.
+     *                          If {@code false}, column names are not case sensitive.
+     * @param batchSize         Batch size in number of rows. Each Arrow batch contains
+     *                          a maximum of {@code batchSize} rows.
+     * @param reuseContainers   If set to {@code false}, every {@link Iterator#next()} call creates
+     *                          new instances of Arrow vectors.
+     *                          If set to {@code true}, the Arrow vectors in the previous
+     *                          {@link Iterator#next()} may be reused for the data returned
+     *                          in the current {@link Iterator#next()}.
+     *                          This option avoids allocating memory again and again.
+     *                          Irrespective of the value of {@code reuseContainers}, the Arrow vectors
+     *                          in the previous {@link Iterator#next()} call are closed before creating
+     *                          new instances if the current {@link Iterator#next()}.
+     */
+    VectorizedCombinedScanIterator(
+        CloseableIterable<CombinedScanTask> tasks,
+        Schema expectedSchema,
+        String nameMapping,
+        FileIO io,
+        EncryptionManager encryptionManager,
+        boolean caseSensitive,
+        int batchSize,
+        boolean reuseContainers) {
+      List<FileScanTask> fileTasks = StreamSupport.stream(tasks.spliterator(), false)
+          .map(CombinedScanTask::files)
+          .flatMap(Collection::stream)
+          .collect(Collectors.toList());
+      this.fileItr = fileTasks.iterator();
+      Map<String, ByteBuffer> keyMetadata = Maps.newHashMap();
+      fileTasks.stream()
+          .flatMap(fileScanTask -> Stream.concat(Stream.of(fileScanTask.file()), fileScanTask.deletes().stream()))
+          .forEach(file -> keyMetadata.put(file.path().toString(), file.keyMetadata()));

Review comment:
       done

##########
File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowReader.java
##########
@@ -0,0 +1,331 @@
+/*
+ * 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.arrow.vectorized;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import java.util.stream.StreamSupport;
+import org.apache.arrow.vector.NullCheckingForGet;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.arrow.vector.types.Types.MinorType;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.encryption.EncryptedFiles;
+import org.apache.iceberg.encryption.EncryptedInputFile;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.CloseableIterator;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.types.Types;
+import org.apache.parquet.schema.MessageType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Vectorized reader that returns an iterator of {@link ColumnarBatch}.
+ * See {@link #open(CloseableIterable)} ()} to learn about the
+ * behavior of the iterator.
+ *
+ * <p>The following Iceberg data types are supported and have been tested:
+ * <ul>
+ *     <li>Iceberg: {@link Types.BooleanType}, Arrow: {@link MinorType#BIT}</li>
+ *     <li>Iceberg: {@link Types.IntegerType}, Arrow: {@link MinorType#INT}</li>
+ *     <li>Iceberg: {@link Types.LongType}, Arrow: {@link MinorType#BIGINT}</li>
+ *     <li>Iceberg: {@link Types.FloatType}, Arrow: {@link MinorType#FLOAT4}</li>
+ *     <li>Iceberg: {@link Types.DoubleType}, Arrow: {@link MinorType#FLOAT8}</li>
+ *     <li>Iceberg: {@link Types.StringType}, Arrow: {@link MinorType#VARCHAR}</li>
+ *     <li>Iceberg: {@link Types.TimestampType} (both with and without timezone),
+ *         Arrow: {@link MinorType#TIMEMICRO}</li>
+ *     <li>Iceberg: {@link Types.BinaryType}, Arrow: {@link MinorType#VARBINARY}</li>
+ *     <li>Iceberg: {@link Types.DateType}, Arrow: {@link MinorType#DATEDAY}</li>
+ * </ul>
+ *
+ * <p>Features that don't work in this implementation:
+ * <ul>
+ *     <li>Type promotion: In case of type promotion, the Arrow vector corresponding to
+ *     the data type in the parquet file is returned instead of the data type in the latest schema.
+ *     See https://github.com/apache/iceberg/issues/2483.</li>
+ *     <li>Columns with constant values are physically encoded as a dictionary. The Arrow vector
+ *     type is int32 instead of the type as per the schema.
+ *     See https://github.com/apache/iceberg/issues/2484.</li>
+ *     <li>Data types: {@link Types.TimeType}, {@link Types.ListType}, {@link Types.MapType},
+ *     {@link Types.StructType}, {@link Types.UUIDType}, {@link Types.FixedType} and
+ *     {@link Types.DecimalType}
+ *     See https://github.com/apache/iceberg/issues/2485 and https://github.com/apache/iceberg/issues/2486.</li>
+ *     <li>Iceberg v2 spec is not supported.
+ *     See https://github.com/apache/iceberg/issues/2487.</li>
+ * </ul>
+ */
+public class ArrowReader extends CloseableGroup {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ArrowReader.class);
+
+  private final Schema schema;
+  private final FileIO io;
+  private final EncryptionManager encryption;
+  private final int batchSize;
+  private final boolean reuseContainers;
+
+  /**
+   * Create a new instance of the reader.
+   *
+   * @param scan the table scan object.
+   * @param batchSize the maximum number of rows per Arrow batch.
+   * @param reuseContainers whether to reuse Arrow vectors when iterating through the data.
+   *                        If set to {@code false}, every {@link Iterator#next()} call creates
+   *                        new instances of Arrow vectors.
+   *                        If set to {@code true}, the Arrow vectors in the previous
+   *                        {@link Iterator#next()} may be reused for the data returned
+   *                        in the current {@link Iterator#next()}.
+   *                        This option avoids allocating memory again and again.
+   *                        Irrespective of the value of {@code reuseContainers}, the Arrow vectors
+   *                        in the previous {@link Iterator#next()} call are closed before creating
+   *                        new instances if the current {@link Iterator#next()}.
+   */
+  public ArrowReader(TableScan scan, int batchSize, boolean reuseContainers) {
+    this.schema = scan.schema();
+    this.io = scan.table().io();
+    this.encryption = scan.table().encryption();
+    this.batchSize = batchSize;
+    // start planning tasks in the background
+    this.reuseContainers = reuseContainers;
+  }
+
+  /**
+   * Returns a new iterator of {@link ColumnarBatch} objects.
+   * <p>
+   * Note that the reader owns the {@link ColumnarBatch} objects and takes care of closing them.
+   * The caller should not hold onto a {@link ColumnarBatch} or try to close them.
+   *
+   * <p>If {@code reuseContainers} is {@code false}, the Arrow vectors in the
+   * previous {@link ColumnarBatch} are closed before returning the next {@link ColumnarBatch} object.
+   * This implies that the caller should either use the {@link ColumnarBatch} or transfer the ownership of
+   * {@link ColumnarBatch} before getting the next {@link ColumnarBatch}.
+   *
+   * <p>If {@code reuseContainers} is {@code true}, the Arrow vectors in the
+   * previous {@link ColumnarBatch} may be reused for the next {@link ColumnarBatch}.
+   * This implies that the caller should either use the {@link ColumnarBatch} or deep copy the
+   * {@link ColumnarBatch} before getting the next {@link ColumnarBatch}.
+   */
+  public CloseableIterator<ColumnarBatch> open(CloseableIterable<CombinedScanTask> tasks) {
+    CloseableIterator<ColumnarBatch> itr = new VectorizedCombinedScanIterator(
+        tasks,
+        schema,
+        null,
+        io,
+        encryption,
+        true,
+        batchSize,
+        reuseContainers
+    );
+    addCloseable(itr);
+    return itr;
+  }
+
+  @Override
+  public void close() throws IOException {
+    super.close(); // close data files
+  }
+
+  /**
+   * Reads the data file and returns an iterator of {@link VectorSchemaRoot}.
+   * Only Parquet data file format is supported.
+   */
+  private static final class VectorizedCombinedScanIterator implements CloseableIterator<ColumnarBatch> {
+
+    private final Iterator<FileScanTask> fileItr;
+    private final Map<String, InputFile> inputFiles;
+    private final Schema expectedSchema;
+    private final String nameMapping;
+    private final boolean caseSensitive;
+    private final int batchSize;
+    private final boolean reuseContainers;
+    private CloseableIterator<ColumnarBatch> currentIterator;
+    private ColumnarBatch current;
+    private FileScanTask currentTask;
+
+    /**
+     * Create a new instance.
+     *
+     * @param tasks             Combined file scan tasks.
+     * @param expectedSchema    Read schema. The returned data will have this schema.
+     * @param nameMapping       Mapping from external schema names to Iceberg type IDs.
+     * @param io                File I/O.
+     * @param encryptionManager Encryption manager.
+     * @param caseSensitive     If {@code true}, column names are case sensitive.
+     *                          If {@code false}, column names are not case sensitive.
+     * @param batchSize         Batch size in number of rows. Each Arrow batch contains
+     *                          a maximum of {@code batchSize} rows.
+     * @param reuseContainers   If set to {@code false}, every {@link Iterator#next()} call creates
+     *                          new instances of Arrow vectors.
+     *                          If set to {@code true}, the Arrow vectors in the previous
+     *                          {@link Iterator#next()} may be reused for the data returned
+     *                          in the current {@link Iterator#next()}.
+     *                          This option avoids allocating memory again and again.
+     *                          Irrespective of the value of {@code reuseContainers}, the Arrow vectors
+     *                          in the previous {@link Iterator#next()} call are closed before creating
+     *                          new instances if the current {@link Iterator#next()}.
+     */
+    VectorizedCombinedScanIterator(
+        CloseableIterable<CombinedScanTask> tasks,
+        Schema expectedSchema,
+        String nameMapping,
+        FileIO io,
+        EncryptionManager encryptionManager,
+        boolean caseSensitive,
+        int batchSize,
+        boolean reuseContainers) {
+      List<FileScanTask> fileTasks = StreamSupport.stream(tasks.spliterator(), false)
+          .map(CombinedScanTask::files)
+          .flatMap(Collection::stream)
+          .collect(Collectors.toList());
+      this.fileItr = fileTasks.iterator();
+      Map<String, ByteBuffer> keyMetadata = Maps.newHashMap();
+      fileTasks.stream()
+          .flatMap(fileScanTask -> Stream.concat(Stream.of(fileScanTask.file()), fileScanTask.deletes().stream()))
+          .forEach(file -> keyMetadata.put(file.path().toString(), file.keyMetadata()));
+      Stream<EncryptedInputFile> encrypted = keyMetadata.entrySet().stream()
+          .map(entry -> EncryptedFiles.encryptedInput(io.newInputFile(entry.getKey()), entry.getValue()));
+
+      // decrypt with the batch call to avoid multiple RPCs to a key server, if possible
+      Iterable<InputFile> decryptedFiles = encryptionManager.decrypt(encrypted::iterator);
+
+      Map<String, InputFile> files = Maps.newHashMapWithExpectedSize(fileTasks.size());

Review comment:
       done




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
rymurr commented on pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#issuecomment-789712836


   Hey @mayursrivastava Ive gotten part of the way through this. Expect a more complete review soon. Early comment:
   
   I am unsure of exposing the `VectorSchemaRoot` directly. Its not an easy object to work with for non-experts and creates potential lifetime/ownership issues. I think a wrapper object similar to Sparks `ColumnBatch` may be more accessible for people who just want the speed of vectorised reads w/o the effort of directly reading Arrow data structures. Though I could see the desire to still expose an advanced user mode to consume the `VectorSchemaRoot` directly with arrow friendly tools.
   
   What do you think? What were the use cases you were thinking of for this?


----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] mayursrivastava edited a comment on pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
mayursrivastava edited a comment on pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#issuecomment-824225827


   Hi @rymurr, for me the benchmark is failing on the master. Currently, I'm running on wsl2/ubuntu. I'll provision another linux machine to try it again - this may take a couple of days.


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] mayursrivastava commented on a change in pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
mayursrivastava commented on a change in pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#discussion_r618416064



##########
File path: build.gradle
##########
@@ -735,9 +735,12 @@ project(':iceberg-arrow') {
       exclude group: 'com.google.code.findbugs', module: 'jsr305'
     }
     compile("org.apache.arrow:arrow-memory-netty") {
-      exclude group: 'io.netty', module: 'netty-common'
       exclude group: 'com.google.code.findbugs', module: 'jsr305'

Review comment:
       That works but the version needs to be added in the versions.props. If this is fine, I'll make the change.
   
   $ git diff # showing only versions.props diff
   index dc4caddd..a147f50c 100644
   --- a/versions.props
   +++ b/versions.props
   @@ -31,3 +31,4 @@ org.apache.hive:hive-service = 2.3.8
    org.apache.tez:tez-dag = 0.8.4
    org.apache.tez:tez-mapreduce = 0.8.4
    com.adobe.testing:s3mock-junit4 = 2.1.28
   +io.netty:netty-common = 4.1.48.Final
   
   
   




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
rymurr commented on pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#issuecomment-796735616


   > I've made the changes you suggested. Please take a look again.
   
   Thanks @mayursrivastava I will get to this asap :+1: 
   


----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] mayursrivastava commented on pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
mayursrivastava commented on pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#issuecomment-796339527


   Hi @rymurr
   
   I've made the changes you suggested. Please take a look again.
   
   Thanks,
   Mayur
   


----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#discussion_r638283576



##########
File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowReader.java
##########
@@ -0,0 +1,325 @@
+/*
+ * 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.arrow.vectorized;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import java.util.stream.StreamSupport;
+import org.apache.arrow.vector.NullCheckingForGet;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.arrow.vector.types.Types.MinorType;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.encryption.EncryptedFiles;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.CloseableIterator;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.types.Types;
+import org.apache.parquet.schema.MessageType;
+
+/**
+ * Vectorized reader that returns an iterator of {@link ColumnarBatch}.
+ * See {@link #open(CloseableIterable)} ()} to learn about the
+ * behavior of the iterator.
+ *
+ * <p>The following Iceberg data types are supported and have been tested:
+ * <ul>
+ *     <li>Iceberg: {@link Types.BooleanType}, Arrow: {@link MinorType#BIT}</li>
+ *     <li>Iceberg: {@link Types.IntegerType}, Arrow: {@link MinorType#INT}</li>
+ *     <li>Iceberg: {@link Types.LongType}, Arrow: {@link MinorType#BIGINT}</li>
+ *     <li>Iceberg: {@link Types.FloatType}, Arrow: {@link MinorType#FLOAT4}</li>
+ *     <li>Iceberg: {@link Types.DoubleType}, Arrow: {@link MinorType#FLOAT8}</li>
+ *     <li>Iceberg: {@link Types.StringType}, Arrow: {@link MinorType#VARCHAR}</li>
+ *     <li>Iceberg: {@link Types.TimestampType} (both with and without timezone),
+ *         Arrow: {@link MinorType#TIMEMICRO}</li>
+ *     <li>Iceberg: {@link Types.BinaryType}, Arrow: {@link MinorType#VARBINARY}</li>
+ *     <li>Iceberg: {@link Types.DateType}, Arrow: {@link MinorType#DATEDAY}</li>
+ * </ul>
+ *
+ * <p>Features that don't work in this implementation:
+ * <ul>
+ *     <li>Type promotion: In case of type promotion, the Arrow vector corresponding to
+ *     the data type in the parquet file is returned instead of the data type in the latest schema.
+ *     See https://github.com/apache/iceberg/issues/2483.</li>
+ *     <li>Columns with constant values are physically encoded as a dictionary. The Arrow vector
+ *     type is int32 instead of the type as per the schema.
+ *     See https://github.com/apache/iceberg/issues/2484.</li>
+ *     <li>Data types: {@link Types.TimeType}, {@link Types.ListType}, {@link Types.MapType},
+ *     {@link Types.StructType}, {@link Types.UUIDType}, {@link Types.FixedType} and
+ *     {@link Types.DecimalType}
+ *     See https://github.com/apache/iceberg/issues/2485 and https://github.com/apache/iceberg/issues/2486.</li>
+ *     <li>Iceberg v2 spec is not supported.
+ *     See https://github.com/apache/iceberg/issues/2487.</li>
+ * </ul>
+ */
+public class ArrowReader extends CloseableGroup {
+
+  private final Schema schema;
+  private final FileIO io;
+  private final EncryptionManager encryption;
+  private final int batchSize;
+  private final boolean reuseContainers;
+
+  /**
+   * Create a new instance of the reader.
+   *
+   * @param scan the table scan object.
+   * @param batchSize the maximum number of rows per Arrow batch.
+   * @param reuseContainers whether to reuse Arrow vectors when iterating through the data.
+   *                        If set to {@code false}, every {@link Iterator#next()} call creates
+   *                        new instances of Arrow vectors.
+   *                        If set to {@code true}, the Arrow vectors in the previous
+   *                        {@link Iterator#next()} may be reused for the data returned
+   *                        in the current {@link Iterator#next()}.
+   *                        This option avoids allocating memory again and again.
+   *                        Irrespective of the value of {@code reuseContainers}, the Arrow vectors
+   *                        in the previous {@link Iterator#next()} call are closed before creating
+   *                        new instances if the current {@link Iterator#next()}.
+   */
+  public ArrowReader(TableScan scan, int batchSize, boolean reuseContainers) {
+    this.schema = scan.schema();
+    this.io = scan.table().io();
+    this.encryption = scan.table().encryption();
+    this.batchSize = batchSize;
+    // start planning tasks in the background
+    this.reuseContainers = reuseContainers;
+  }
+
+  /**
+   * Returns a new iterator of {@link ColumnarBatch} objects.
+   * <p>
+   * Note that the reader owns the {@link ColumnarBatch} objects and takes care of closing them.
+   * The caller should not hold onto a {@link ColumnarBatch} or try to close them.
+   *
+   * <p>If {@code reuseContainers} is {@code false}, the Arrow vectors in the
+   * previous {@link ColumnarBatch} are closed before returning the next {@link ColumnarBatch} object.
+   * This implies that the caller should either use the {@link ColumnarBatch} or transfer the ownership of
+   * {@link ColumnarBatch} before getting the next {@link ColumnarBatch}.
+   *
+   * <p>If {@code reuseContainers} is {@code true}, the Arrow vectors in the
+   * previous {@link ColumnarBatch} may be reused for the next {@link ColumnarBatch}.
+   * This implies that the caller should either use the {@link ColumnarBatch} or deep copy the
+   * {@link ColumnarBatch} before getting the next {@link ColumnarBatch}.
+   */
+  public CloseableIterator<ColumnarBatch> open(CloseableIterable<CombinedScanTask> tasks) {
+    CloseableIterator<ColumnarBatch> itr = new VectorizedCombinedScanIterator(
+        tasks,
+        schema,
+        null,
+        io,
+        encryption,
+        true,
+        batchSize,
+        reuseContainers
+    );
+    addCloseable(itr);
+    return itr;
+  }
+
+  @Override
+  public void close() throws IOException {
+    super.close(); // close data files
+  }
+
+  /**
+   * Reads the data file and returns an iterator of {@link VectorSchemaRoot}.
+   * Only Parquet data file format is supported.
+   */
+  private static final class VectorizedCombinedScanIterator implements CloseableIterator<ColumnarBatch> {
+
+    private final List<FileScanTask> fileTasks;
+    private final Iterator<FileScanTask> fileItr;
+    private final Map<String, InputFile> inputFiles;
+    private final Schema expectedSchema;
+    private final String nameMapping;
+    private final boolean caseSensitive;
+    private final int batchSize;
+    private final boolean reuseContainers;
+    private CloseableIterator<ColumnarBatch> currentIterator;
+    private ColumnarBatch current;
+    private FileScanTask currentTask;
+
+    /**
+     * Create a new instance.
+     *
+     * @param tasks             Combined file scan tasks.
+     * @param expectedSchema    Read schema. The returned data will have this schema.
+     * @param nameMapping       Mapping from external schema names to Iceberg type IDs.
+     * @param io                File I/O.
+     * @param encryptionManager Encryption manager.
+     * @param caseSensitive     If {@code true}, column names are case sensitive.
+     *                          If {@code false}, column names are not case sensitive.
+     * @param batchSize         Batch size in number of rows. Each Arrow batch contains
+     *                          a maximum of {@code batchSize} rows.
+     * @param reuseContainers   If set to {@code false}, every {@link Iterator#next()} call creates
+     *                          new instances of Arrow vectors.
+     *                          If set to {@code true}, the Arrow vectors in the previous
+     *                          {@link Iterator#next()} may be reused for the data returned
+     *                          in the current {@link Iterator#next()}.
+     *                          This option avoids allocating memory again and again.
+     *                          Irrespective of the value of {@code reuseContainers}, the Arrow vectors
+     *                          in the previous {@link Iterator#next()} call are closed before creating
+     *                          new instances if the current {@link Iterator#next()}.
+     */
+    VectorizedCombinedScanIterator(
+        CloseableIterable<CombinedScanTask> tasks,
+        Schema expectedSchema,
+        String nameMapping,
+        FileIO io,
+        EncryptionManager encryptionManager,
+        boolean caseSensitive,
+        int batchSize,
+        boolean reuseContainers) {

Review comment:
       Nit: this doesn't conform to the style used by the rest of the project.




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] mayursrivastava commented on a change in pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
mayursrivastava commented on a change in pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#discussion_r614909212



##########
File path: arrow/src/test/java/org/apache/iceberg/arrow/vectorized/ArrowReaderTest.java
##########
@@ -0,0 +1,904 @@
+/*
+ * 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.arrow.vectorized;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.time.temporal.ChronoUnit;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+import org.apache.arrow.vector.BigIntVector;
+import org.apache.arrow.vector.BitVector;
+import org.apache.arrow.vector.DateDayVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.Float4Vector;
+import org.apache.arrow.vector.Float8Vector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.TimeStampMicroTZVector;
+import org.apache.arrow.vector.TimeStampMicroVector;
+import org.apache.arrow.vector.VarBinaryVector;
+import org.apache.arrow.vector.VarCharVector;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.arrow.vector.types.Types.MinorType;
+import org.apache.arrow.vector.types.pojo.ArrowType;
+import org.apache.arrow.vector.types.pojo.Field;
+import org.apache.arrow.vector.types.pojo.FieldType;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.OverwriteFiles;
+import org.apache.iceberg.PartitionKey;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.data.parquet.GenericParquetWriter;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.types.Types;
+import org.junit.Ignore;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.Files.localInput;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Test cases for {@link ArrowReader}.
+ * <p>All tests create a table with monthly partitions and write 1 year of data to the table.
+ */
+public class ArrowReaderTest {
+
+  private static final int NUM_ROWS_PER_MONTH = 20;
+  private static final ImmutableList<String> ALL_COLUMNS =
+      ImmutableList.of(
+          "timestamp",
+          "timestamp_nullable",
+          "boolean",
+          "boolean_nullable",
+          "int",
+          "int_nullable",
+          "long",
+          "long_nullable",
+          "float",
+          "float_nullable",
+          "double",
+          "double_nullable",
+          "timestamp_tz",
+          "timestamp_tz_nullable",
+          "string",
+          "string_nullable",
+          "bytes",
+          "bytes_nullable",
+          "date",
+          "date_nullable",
+          "int_promotion"
+      );
+
+  @Rule
+  public final TemporaryFolder temp = new TemporaryFolder();
+
+  private HadoopTables tables;
+
+  private String tableLocation;
+  private List<GenericRecord> rowsWritten;
+
+  /**
+   * Read all rows and columns from the table without any filter. The test asserts that the Arrow {@link
+   * VectorSchemaRoot} contains the expected schema and expected vector types. Then the test asserts that the vectors
+   * contains expected values. The test also asserts the total number of rows match the expected value.
+   */
+  @Test
+  public void testReadAll() throws Exception {
+    writeTableWithIncrementalRecords();
+    Table table = tables.load(tableLocation);
+    readAndCheckVectorSchemaRoots(table.newScan(), NUM_ROWS_PER_MONTH, 12 * NUM_ROWS_PER_MONTH, ALL_COLUMNS);
+  }
+
+  /**
+   * This test writes each partition with constant value rows. The Arrow vectors returned are mostly of type int32
+   * which is unexpected. This is happening because of dictionary encoding at the storage level.
+   * <p>
+   * Following are the expected and actual Arrow schema:
+   * <pre>
+   * Expected Arrow Schema:
+   * timestamp: Timestamp(MICROSECOND, null) not null,
+   * timestamp_nullable: Timestamp(MICROSECOND, null),
+   * boolean: Bool not null,
+   * boolean_nullable: Bool,
+   * int: Int(32, true) not null,
+   * int_nullable: Int(32, true),
+   * long: Int(64, true) not null,
+   * long_nullable: Int(64, true),
+   * float: FloatingPoint(SINGLE) not null,
+   * float_nullable: FloatingPoint(SINGLE),
+   * double: FloatingPoint(DOUBLE) not null,
+   * double_nullable: FloatingPoint(DOUBLE),
+   * timestamp_tz: Timestamp(MICROSECOND, UTC) not null,
+   * timestamp_tz_nullable: Timestamp(MICROSECOND, UTC),
+   * string: Utf8 not null,
+   * string_nullable: Utf8,
+   * bytes: Binary not null,
+   * bytes_nullable: Binary,
+   * date: Date(DAY) not null,
+   * date_nullable: Date(DAY),
+   * int_promotion: Int(32, true) not null
+   *
+   * Actual Arrow Schema:
+   * timestamp: Int(32, true) not null,
+   * timestamp_nullable: Int(32, true),
+   * boolean: Bool not null,
+   * boolean_nullable: Bool,
+   * int: Int(32, true) not null,
+   * int_nullable: Int(32, true),
+   * long: Int(32, true) not null,
+   * long_nullable: Int(32, true),
+   * float: Int(32, true) not null,
+   * float_nullable: Int(32, true),
+   * double: Int(32, true) not null,
+   * double_nullable: Int(32, true),
+   * timestamp_tz: Int(32, true) not null,
+   * timestamp_tz_nullable: Int(32, true),
+   * string: Int(32, true) not null,
+   * string_nullable: Int(32, true),
+   * bytes: Int(32, true) not null,
+   * bytes_nullable: Int(32, true),
+   * date: Date(DAY) not null,
+   * date_nullable: Date(DAY),
+   * int_promotion: Int(32, true) not null
+   * </pre>
+   * <p>
+   * TODO: fix the returned Arrow vectors to have vector types consistent with Iceberg types.
+   * <p>
+   * Read all rows and columns from the table without any filter. The test asserts that the Arrow {@link
+   * VectorSchemaRoot} contains the expected schema and expected vector types. Then the test asserts that the vectors
+   * contains expected values. The test also asserts the total number of rows match the expected value.
+   */
+  @Test
+  @Ignore
+  public void testReadAllWithConstantRecords() throws Exception {
+    writeTableWithConstantRecords();
+    Table table = tables.load(tableLocation);
+    readAndCheckVectorSchemaRoots(table.newScan(), NUM_ROWS_PER_MONTH, 12 * NUM_ROWS_PER_MONTH, ALL_COLUMNS);
+  }
+
+  /**
+   * Read all rows and columns from the table without any filter. The test uses a batch size smaller than the number of
+   * rows in a partition. The test asserts that the Arrow {@link VectorSchemaRoot} contains the expected schema and
+   * expected vector types. Then the test asserts that the vectors contains expected values. The test also asserts the
+   * total number of rows match the expected value.
+   */
+  @Test
+  public void testReadAllWithSmallerBatchSize() throws Exception {
+    writeTableWithIncrementalRecords();
+    Table table = tables.load(tableLocation);
+    TableScan scan = table.newScan();
+    readAndCheckVectorSchemaRoots(scan, 10, 12 * NUM_ROWS_PER_MONTH, ALL_COLUMNS);
+  }
+
+  /**
+   * Read selected rows and all columns from the table using a time range row filter. The test asserts that the Arrow
+   * {@link VectorSchemaRoot} contains the expected schema and expected vector types. Then the test asserts that the
+   * vectors contains expected values. The test also asserts the total number of rows match the expected value.
+   */
+  @Test
+  public void testReadRangeFilter() throws Exception {
+    writeTableWithIncrementalRecords();
+    Table table = tables.load(tableLocation);
+    LocalDateTime beginTime = LocalDateTime.of(2020, 1, 1, 0, 0, 0);
+    LocalDateTime endTime = LocalDateTime.of(2020, 2, 1, 0, 0, 0);
+    TableScan scan = table.newScan()
+        .filter(Expressions.and(
+            Expressions.greaterThanOrEqual("timestamp", timestampToMicros(beginTime)),
+            Expressions.lessThan("timestamp", timestampToMicros(endTime))));
+    readAndCheckVectorSchemaRoots(scan, NUM_ROWS_PER_MONTH, NUM_ROWS_PER_MONTH, ALL_COLUMNS);
+  }
+
+  /**
+   * Read selected rows and all columns from the table using a time range row filter.
+   * The test asserts that the result is empty.
+   */
+  @Test
+  public void testReadRangeFilterEmptyResult() throws Exception {
+    writeTableWithIncrementalRecords();
+    Table table = tables.load(tableLocation);
+    LocalDateTime beginTime = LocalDateTime.of(2021, 1, 1, 0, 0, 0);
+    LocalDateTime endTime = LocalDateTime.of(2021, 2, 1, 0, 0, 0);
+    TableScan scan = table.newScan()
+            .filter(Expressions.and(
+                    Expressions.greaterThanOrEqual("timestamp", timestampToMicros(beginTime)),
+                    Expressions.lessThan("timestamp", timestampToMicros(endTime))));
+    int numRoots = 0;
+    try (VectorizedTableScanIterable itr = new VectorizedTableScanIterable(scan, NUM_ROWS_PER_MONTH, false)) {
+      for (ColumnarBatch batch : itr) {
+        numRoots++;
+      }
+    }
+    assertEquals(0, numRoots);
+  }
+
+  /**
+   * Read all rows and selected columns from the table with a column selection filter. The test asserts that the Arrow
+   * {@link VectorSchemaRoot} contains the expected schema and expected vector types. Then the test asserts that the
+   * vectors contains expected values. The test also asserts the total number of rows match the expected value.
+   */
+  @Test
+  public void testReadColumnFilter1() throws Exception {
+    writeTableWithIncrementalRecords();
+    Table table = tables.load(tableLocation);
+    TableScan scan = table.newScan()
+        .select("timestamp", "int", "string");
+    readAndCheckVectorSchemaRoots(
+        scan, NUM_ROWS_PER_MONTH, 12 * NUM_ROWS_PER_MONTH,
+        ImmutableList.of("timestamp", "int", "string"));
+  }
+
+  /**
+   * Read all rows and a single column from the table with a column selection filter. The test asserts that the Arrow
+   * {@link VectorSchemaRoot} contains the expected schema and expected vector types. Then the test asserts that the
+   * vectors contains expected values. The test also asserts the total number of rows match the expected value.
+   */
+  @Test
+  public void testReadColumnFilter2() throws Exception {
+    writeTableWithIncrementalRecords();
+    Table table = tables.load(tableLocation);
+    TableScan scan = table.newScan()
+        .select("timestamp");
+    readAndCheckVectorSchemaRoots(
+        scan, NUM_ROWS_PER_MONTH, 12 * NUM_ROWS_PER_MONTH,
+        ImmutableList.of("timestamp"));
+  }
+
+  private void readAndCheckVectorSchemaRoots(
+      TableScan scan,
+      int numRowsPerRoot,
+      int expectedTotalRows,
+      List<String> columns) throws IOException {
+    Set<String> columnSet = ImmutableSet.copyOf(columns);
+    int rowIndex = 0;
+    int totalRows = 0;
+    try (VectorizedTableScanIterable itr = new VectorizedTableScanIterable(scan, numRowsPerRoot, false)) {
+      for (ColumnarBatch batch : itr) {
+        List<GenericRecord> expectedRows = rowsWritten.subList(rowIndex, rowIndex + numRowsPerRoot);
+        checkColumnarBatch(numRowsPerRoot, expectedRows, batch, columns);

Review comment:
       I will split it up and add javadoc




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] mayursrivastava commented on a change in pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
mayursrivastava commented on a change in pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#discussion_r682569401



##########
File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowReader.java
##########
@@ -0,0 +1,325 @@
+/*
+ * 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.arrow.vectorized;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import java.util.stream.StreamSupport;
+import org.apache.arrow.vector.NullCheckingForGet;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.arrow.vector.types.Types.MinorType;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.encryption.EncryptedFiles;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.CloseableIterator;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.types.Types;
+import org.apache.parquet.schema.MessageType;
+
+/**
+ * Vectorized reader that returns an iterator of {@link ColumnarBatch}.
+ * See {@link #open(CloseableIterable)} ()} to learn about the
+ * behavior of the iterator.
+ *
+ * <p>The following Iceberg data types are supported and have been tested:
+ * <ul>
+ *     <li>Iceberg: {@link Types.BooleanType}, Arrow: {@link MinorType#BIT}</li>
+ *     <li>Iceberg: {@link Types.IntegerType}, Arrow: {@link MinorType#INT}</li>
+ *     <li>Iceberg: {@link Types.LongType}, Arrow: {@link MinorType#BIGINT}</li>
+ *     <li>Iceberg: {@link Types.FloatType}, Arrow: {@link MinorType#FLOAT4}</li>
+ *     <li>Iceberg: {@link Types.DoubleType}, Arrow: {@link MinorType#FLOAT8}</li>
+ *     <li>Iceberg: {@link Types.StringType}, Arrow: {@link MinorType#VARCHAR}</li>
+ *     <li>Iceberg: {@link Types.TimestampType} (both with and without timezone),
+ *         Arrow: {@link MinorType#TIMEMICRO}</li>
+ *     <li>Iceberg: {@link Types.BinaryType}, Arrow: {@link MinorType#VARBINARY}</li>
+ *     <li>Iceberg: {@link Types.DateType}, Arrow: {@link MinorType#DATEDAY}</li>
+ * </ul>
+ *
+ * <p>Features that don't work in this implementation:
+ * <ul>
+ *     <li>Type promotion: In case of type promotion, the Arrow vector corresponding to
+ *     the data type in the parquet file is returned instead of the data type in the latest schema.
+ *     See https://github.com/apache/iceberg/issues/2483.</li>
+ *     <li>Columns with constant values are physically encoded as a dictionary. The Arrow vector
+ *     type is int32 instead of the type as per the schema.
+ *     See https://github.com/apache/iceberg/issues/2484.</li>
+ *     <li>Data types: {@link Types.TimeType}, {@link Types.ListType}, {@link Types.MapType},
+ *     {@link Types.StructType}, {@link Types.UUIDType}, {@link Types.FixedType} and
+ *     {@link Types.DecimalType}
+ *     See https://github.com/apache/iceberg/issues/2485 and https://github.com/apache/iceberg/issues/2486.</li>
+ *     <li>Iceberg v2 spec is not supported.
+ *     See https://github.com/apache/iceberg/issues/2487.</li>
+ * </ul>
+ */
+public class ArrowReader extends CloseableGroup {
+
+  private final Schema schema;
+  private final FileIO io;
+  private final EncryptionManager encryption;
+  private final int batchSize;
+  private final boolean reuseContainers;
+
+  /**
+   * Create a new instance of the reader.
+   *
+   * @param scan the table scan object.
+   * @param batchSize the maximum number of rows per Arrow batch.
+   * @param reuseContainers whether to reuse Arrow vectors when iterating through the data.
+   *                        If set to {@code false}, every {@link Iterator#next()} call creates
+   *                        new instances of Arrow vectors.
+   *                        If set to {@code true}, the Arrow vectors in the previous
+   *                        {@link Iterator#next()} may be reused for the data returned
+   *                        in the current {@link Iterator#next()}.
+   *                        This option avoids allocating memory again and again.
+   *                        Irrespective of the value of {@code reuseContainers}, the Arrow vectors
+   *                        in the previous {@link Iterator#next()} call are closed before creating
+   *                        new instances if the current {@link Iterator#next()}.
+   */
+  public ArrowReader(TableScan scan, int batchSize, boolean reuseContainers) {
+    this.schema = scan.schema();
+    this.io = scan.table().io();
+    this.encryption = scan.table().encryption();
+    this.batchSize = batchSize;
+    // start planning tasks in the background
+    this.reuseContainers = reuseContainers;
+  }
+
+  /**
+   * Returns a new iterator of {@link ColumnarBatch} objects.
+   * <p>
+   * Note that the reader owns the {@link ColumnarBatch} objects and takes care of closing them.
+   * The caller should not hold onto a {@link ColumnarBatch} or try to close them.
+   *
+   * <p>If {@code reuseContainers} is {@code false}, the Arrow vectors in the
+   * previous {@link ColumnarBatch} are closed before returning the next {@link ColumnarBatch} object.
+   * This implies that the caller should either use the {@link ColumnarBatch} or transfer the ownership of
+   * {@link ColumnarBatch} before getting the next {@link ColumnarBatch}.
+   *
+   * <p>If {@code reuseContainers} is {@code true}, the Arrow vectors in the
+   * previous {@link ColumnarBatch} may be reused for the next {@link ColumnarBatch}.
+   * This implies that the caller should either use the {@link ColumnarBatch} or deep copy the
+   * {@link ColumnarBatch} before getting the next {@link ColumnarBatch}.
+   */
+  public CloseableIterator<ColumnarBatch> open(CloseableIterable<CombinedScanTask> tasks) {
+    CloseableIterator<ColumnarBatch> itr = new VectorizedCombinedScanIterator(
+        tasks,
+        schema,
+        null,
+        io,
+        encryption,
+        true,
+        batchSize,
+        reuseContainers
+    );
+    addCloseable(itr);
+    return itr;
+  }
+
+  @Override
+  public void close() throws IOException {
+    super.close(); // close data files
+  }
+
+  /**
+   * Reads the data file and returns an iterator of {@link VectorSchemaRoot}.
+   * Only Parquet data file format is supported.
+   */
+  private static final class VectorizedCombinedScanIterator implements CloseableIterator<ColumnarBatch> {
+
+    private final List<FileScanTask> fileTasks;
+    private final Iterator<FileScanTask> fileItr;
+    private final Map<String, InputFile> inputFiles;
+    private final Schema expectedSchema;
+    private final String nameMapping;
+    private final boolean caseSensitive;
+    private final int batchSize;
+    private final boolean reuseContainers;
+    private CloseableIterator<ColumnarBatch> currentIterator;
+    private ColumnarBatch current;
+    private FileScanTask currentTask;
+
+    /**
+     * Create a new instance.
+     *
+     * @param tasks             Combined file scan tasks.
+     * @param expectedSchema    Read schema. The returned data will have this schema.
+     * @param nameMapping       Mapping from external schema names to Iceberg type IDs.
+     * @param io                File I/O.
+     * @param encryptionManager Encryption manager.
+     * @param caseSensitive     If {@code true}, column names are case sensitive.
+     *                          If {@code false}, column names are not case sensitive.
+     * @param batchSize         Batch size in number of rows. Each Arrow batch contains
+     *                          a maximum of {@code batchSize} rows.
+     * @param reuseContainers   If set to {@code false}, every {@link Iterator#next()} call creates
+     *                          new instances of Arrow vectors.
+     *                          If set to {@code true}, the Arrow vectors in the previous
+     *                          {@link Iterator#next()} may be reused for the data returned
+     *                          in the current {@link Iterator#next()}.
+     *                          This option avoids allocating memory again and again.
+     *                          Irrespective of the value of {@code reuseContainers}, the Arrow vectors
+     *                          in the previous {@link Iterator#next()} call are closed before creating
+     *                          new instances if the current {@link Iterator#next()}.
+     */
+    VectorizedCombinedScanIterator(
+        CloseableIterable<CombinedScanTask> tasks,
+        Schema expectedSchema,
+        String nameMapping,
+        FileIO io,
+        EncryptionManager encryptionManager,
+        boolean caseSensitive,
+        int batchSize,
+        boolean reuseContainers) {
+      this.fileTasks = StreamSupport.stream(tasks.spliterator(), false)
+          .map(CombinedScanTask::files)
+          .flatMap(Collection::stream)
+          .collect(Collectors.toList());
+      this.fileItr = fileTasks.iterator();
+      this.inputFiles = Collections.unmodifiableMap(fileTasks.stream()
+          .flatMap(fileScanTask -> Stream.concat(Stream.of(fileScanTask.file()), fileScanTask.deletes().stream()))
+          .map(file -> EncryptedFiles.encryptedInput(io.newInputFile(file.path().toString()), file.keyMetadata()))
+          .map(encryptionManager::decrypt)

Review comment:
       This was already fixed in #2720




-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] mayursrivastava commented on a change in pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
mayursrivastava commented on a change in pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#discussion_r682567659



##########
File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowReader.java
##########
@@ -0,0 +1,325 @@
+/*
+ * 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.arrow.vectorized;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import java.util.stream.StreamSupport;
+import org.apache.arrow.vector.NullCheckingForGet;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.arrow.vector.types.Types.MinorType;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.encryption.EncryptedFiles;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.CloseableIterator;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.types.Types;
+import org.apache.parquet.schema.MessageType;
+
+/**
+ * Vectorized reader that returns an iterator of {@link ColumnarBatch}.
+ * See {@link #open(CloseableIterable)} ()} to learn about the
+ * behavior of the iterator.
+ *
+ * <p>The following Iceberg data types are supported and have been tested:
+ * <ul>
+ *     <li>Iceberg: {@link Types.BooleanType}, Arrow: {@link MinorType#BIT}</li>
+ *     <li>Iceberg: {@link Types.IntegerType}, Arrow: {@link MinorType#INT}</li>
+ *     <li>Iceberg: {@link Types.LongType}, Arrow: {@link MinorType#BIGINT}</li>
+ *     <li>Iceberg: {@link Types.FloatType}, Arrow: {@link MinorType#FLOAT4}</li>
+ *     <li>Iceberg: {@link Types.DoubleType}, Arrow: {@link MinorType#FLOAT8}</li>
+ *     <li>Iceberg: {@link Types.StringType}, Arrow: {@link MinorType#VARCHAR}</li>
+ *     <li>Iceberg: {@link Types.TimestampType} (both with and without timezone),
+ *         Arrow: {@link MinorType#TIMEMICRO}</li>
+ *     <li>Iceberg: {@link Types.BinaryType}, Arrow: {@link MinorType#VARBINARY}</li>
+ *     <li>Iceberg: {@link Types.DateType}, Arrow: {@link MinorType#DATEDAY}</li>
+ * </ul>
+ *
+ * <p>Features that don't work in this implementation:
+ * <ul>
+ *     <li>Type promotion: In case of type promotion, the Arrow vector corresponding to
+ *     the data type in the parquet file is returned instead of the data type in the latest schema.
+ *     See https://github.com/apache/iceberg/issues/2483.</li>
+ *     <li>Columns with constant values are physically encoded as a dictionary. The Arrow vector
+ *     type is int32 instead of the type as per the schema.
+ *     See https://github.com/apache/iceberg/issues/2484.</li>
+ *     <li>Data types: {@link Types.TimeType}, {@link Types.ListType}, {@link Types.MapType},
+ *     {@link Types.StructType}, {@link Types.UUIDType}, {@link Types.FixedType} and
+ *     {@link Types.DecimalType}
+ *     See https://github.com/apache/iceberg/issues/2485 and https://github.com/apache/iceberg/issues/2486.</li>
+ *     <li>Iceberg v2 spec is not supported.
+ *     See https://github.com/apache/iceberg/issues/2487.</li>
+ * </ul>
+ */
+public class ArrowReader extends CloseableGroup {
+
+  private final Schema schema;
+  private final FileIO io;
+  private final EncryptionManager encryption;
+  private final int batchSize;
+  private final boolean reuseContainers;
+
+  /**
+   * Create a new instance of the reader.
+   *
+   * @param scan the table scan object.
+   * @param batchSize the maximum number of rows per Arrow batch.
+   * @param reuseContainers whether to reuse Arrow vectors when iterating through the data.
+   *                        If set to {@code false}, every {@link Iterator#next()} call creates
+   *                        new instances of Arrow vectors.
+   *                        If set to {@code true}, the Arrow vectors in the previous
+   *                        {@link Iterator#next()} may be reused for the data returned
+   *                        in the current {@link Iterator#next()}.
+   *                        This option avoids allocating memory again and again.
+   *                        Irrespective of the value of {@code reuseContainers}, the Arrow vectors
+   *                        in the previous {@link Iterator#next()} call are closed before creating
+   *                        new instances if the current {@link Iterator#next()}.
+   */
+  public ArrowReader(TableScan scan, int batchSize, boolean reuseContainers) {
+    this.schema = scan.schema();
+    this.io = scan.table().io();
+    this.encryption = scan.table().encryption();
+    this.batchSize = batchSize;
+    // start planning tasks in the background
+    this.reuseContainers = reuseContainers;
+  }
+
+  /**
+   * Returns a new iterator of {@link ColumnarBatch} objects.
+   * <p>
+   * Note that the reader owns the {@link ColumnarBatch} objects and takes care of closing them.
+   * The caller should not hold onto a {@link ColumnarBatch} or try to close them.
+   *
+   * <p>If {@code reuseContainers} is {@code false}, the Arrow vectors in the
+   * previous {@link ColumnarBatch} are closed before returning the next {@link ColumnarBatch} object.
+   * This implies that the caller should either use the {@link ColumnarBatch} or transfer the ownership of
+   * {@link ColumnarBatch} before getting the next {@link ColumnarBatch}.
+   *
+   * <p>If {@code reuseContainers} is {@code true}, the Arrow vectors in the
+   * previous {@link ColumnarBatch} may be reused for the next {@link ColumnarBatch}.
+   * This implies that the caller should either use the {@link ColumnarBatch} or deep copy the
+   * {@link ColumnarBatch} before getting the next {@link ColumnarBatch}.
+   */
+  public CloseableIterator<ColumnarBatch> open(CloseableIterable<CombinedScanTask> tasks) {
+    CloseableIterator<ColumnarBatch> itr = new VectorizedCombinedScanIterator(
+        tasks,
+        schema,
+        null,
+        io,
+        encryption,
+        true,
+        batchSize,
+        reuseContainers
+    );
+    addCloseable(itr);
+    return itr;
+  }
+
+  @Override
+  public void close() throws IOException {
+    super.close(); // close data files
+  }
+
+  /**
+   * Reads the data file and returns an iterator of {@link VectorSchemaRoot}.
+   * Only Parquet data file format is supported.
+   */
+  private static final class VectorizedCombinedScanIterator implements CloseableIterator<ColumnarBatch> {
+
+    private final List<FileScanTask> fileTasks;
+    private final Iterator<FileScanTask> fileItr;
+    private final Map<String, InputFile> inputFiles;
+    private final Schema expectedSchema;
+    private final String nameMapping;
+    private final boolean caseSensitive;
+    private final int batchSize;
+    private final boolean reuseContainers;
+    private CloseableIterator<ColumnarBatch> currentIterator;
+    private ColumnarBatch current;
+    private FileScanTask currentTask;
+
+    /**
+     * Create a new instance.
+     *
+     * @param tasks             Combined file scan tasks.
+     * @param expectedSchema    Read schema. The returned data will have this schema.
+     * @param nameMapping       Mapping from external schema names to Iceberg type IDs.
+     * @param io                File I/O.
+     * @param encryptionManager Encryption manager.
+     * @param caseSensitive     If {@code true}, column names are case sensitive.
+     *                          If {@code false}, column names are not case sensitive.
+     * @param batchSize         Batch size in number of rows. Each Arrow batch contains
+     *                          a maximum of {@code batchSize} rows.
+     * @param reuseContainers   If set to {@code false}, every {@link Iterator#next()} call creates
+     *                          new instances of Arrow vectors.
+     *                          If set to {@code true}, the Arrow vectors in the previous
+     *                          {@link Iterator#next()} may be reused for the data returned
+     *                          in the current {@link Iterator#next()}.
+     *                          This option avoids allocating memory again and again.
+     *                          Irrespective of the value of {@code reuseContainers}, the Arrow vectors
+     *                          in the previous {@link Iterator#next()} call are closed before creating
+     *                          new instances if the current {@link Iterator#next()}.
+     */
+    VectorizedCombinedScanIterator(
+        CloseableIterable<CombinedScanTask> tasks,
+        Schema expectedSchema,
+        String nameMapping,
+        FileIO io,
+        EncryptionManager encryptionManager,
+        boolean caseSensitive,
+        int batchSize,
+        boolean reuseContainers) {

Review comment:
       Fixed in https://github.com/apache/iceberg/pull/2933




-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#discussion_r638343076



##########
File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowReader.java
##########
@@ -0,0 +1,325 @@
+/*
+ * 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.arrow.vectorized;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import java.util.stream.StreamSupport;
+import org.apache.arrow.vector.NullCheckingForGet;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.arrow.vector.types.Types.MinorType;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.encryption.EncryptedFiles;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.CloseableIterator;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.types.Types;
+import org.apache.parquet.schema.MessageType;
+
+/**
+ * Vectorized reader that returns an iterator of {@link ColumnarBatch}.
+ * See {@link #open(CloseableIterable)} ()} to learn about the
+ * behavior of the iterator.
+ *
+ * <p>The following Iceberg data types are supported and have been tested:
+ * <ul>
+ *     <li>Iceberg: {@link Types.BooleanType}, Arrow: {@link MinorType#BIT}</li>
+ *     <li>Iceberg: {@link Types.IntegerType}, Arrow: {@link MinorType#INT}</li>
+ *     <li>Iceberg: {@link Types.LongType}, Arrow: {@link MinorType#BIGINT}</li>
+ *     <li>Iceberg: {@link Types.FloatType}, Arrow: {@link MinorType#FLOAT4}</li>
+ *     <li>Iceberg: {@link Types.DoubleType}, Arrow: {@link MinorType#FLOAT8}</li>
+ *     <li>Iceberg: {@link Types.StringType}, Arrow: {@link MinorType#VARCHAR}</li>
+ *     <li>Iceberg: {@link Types.TimestampType} (both with and without timezone),
+ *         Arrow: {@link MinorType#TIMEMICRO}</li>
+ *     <li>Iceberg: {@link Types.BinaryType}, Arrow: {@link MinorType#VARBINARY}</li>
+ *     <li>Iceberg: {@link Types.DateType}, Arrow: {@link MinorType#DATEDAY}</li>
+ * </ul>
+ *
+ * <p>Features that don't work in this implementation:
+ * <ul>
+ *     <li>Type promotion: In case of type promotion, the Arrow vector corresponding to
+ *     the data type in the parquet file is returned instead of the data type in the latest schema.
+ *     See https://github.com/apache/iceberg/issues/2483.</li>
+ *     <li>Columns with constant values are physically encoded as a dictionary. The Arrow vector
+ *     type is int32 instead of the type as per the schema.
+ *     See https://github.com/apache/iceberg/issues/2484.</li>
+ *     <li>Data types: {@link Types.TimeType}, {@link Types.ListType}, {@link Types.MapType},
+ *     {@link Types.StructType}, {@link Types.UUIDType}, {@link Types.FixedType} and
+ *     {@link Types.DecimalType}
+ *     See https://github.com/apache/iceberg/issues/2485 and https://github.com/apache/iceberg/issues/2486.</li>
+ *     <li>Iceberg v2 spec is not supported.
+ *     See https://github.com/apache/iceberg/issues/2487.</li>
+ * </ul>
+ */
+public class ArrowReader extends CloseableGroup {
+
+  private final Schema schema;
+  private final FileIO io;
+  private final EncryptionManager encryption;
+  private final int batchSize;
+  private final boolean reuseContainers;
+
+  /**
+   * Create a new instance of the reader.
+   *
+   * @param scan the table scan object.
+   * @param batchSize the maximum number of rows per Arrow batch.
+   * @param reuseContainers whether to reuse Arrow vectors when iterating through the data.
+   *                        If set to {@code false}, every {@link Iterator#next()} call creates
+   *                        new instances of Arrow vectors.
+   *                        If set to {@code true}, the Arrow vectors in the previous
+   *                        {@link Iterator#next()} may be reused for the data returned
+   *                        in the current {@link Iterator#next()}.
+   *                        This option avoids allocating memory again and again.
+   *                        Irrespective of the value of {@code reuseContainers}, the Arrow vectors
+   *                        in the previous {@link Iterator#next()} call are closed before creating
+   *                        new instances if the current {@link Iterator#next()}.
+   */
+  public ArrowReader(TableScan scan, int batchSize, boolean reuseContainers) {
+    this.schema = scan.schema();
+    this.io = scan.table().io();
+    this.encryption = scan.table().encryption();
+    this.batchSize = batchSize;
+    // start planning tasks in the background
+    this.reuseContainers = reuseContainers;
+  }
+
+  /**
+   * Returns a new iterator of {@link ColumnarBatch} objects.
+   * <p>
+   * Note that the reader owns the {@link ColumnarBatch} objects and takes care of closing them.
+   * The caller should not hold onto a {@link ColumnarBatch} or try to close them.
+   *
+   * <p>If {@code reuseContainers} is {@code false}, the Arrow vectors in the
+   * previous {@link ColumnarBatch} are closed before returning the next {@link ColumnarBatch} object.
+   * This implies that the caller should either use the {@link ColumnarBatch} or transfer the ownership of
+   * {@link ColumnarBatch} before getting the next {@link ColumnarBatch}.
+   *
+   * <p>If {@code reuseContainers} is {@code true}, the Arrow vectors in the
+   * previous {@link ColumnarBatch} may be reused for the next {@link ColumnarBatch}.
+   * This implies that the caller should either use the {@link ColumnarBatch} or deep copy the
+   * {@link ColumnarBatch} before getting the next {@link ColumnarBatch}.
+   */
+  public CloseableIterator<ColumnarBatch> open(CloseableIterable<CombinedScanTask> tasks) {
+    CloseableIterator<ColumnarBatch> itr = new VectorizedCombinedScanIterator(
+        tasks,
+        schema,
+        null,
+        io,
+        encryption,
+        true,
+        batchSize,
+        reuseContainers
+    );
+    addCloseable(itr);
+    return itr;
+  }
+
+  @Override
+  public void close() throws IOException {
+    super.close(); // close data files
+  }
+
+  /**
+   * Reads the data file and returns an iterator of {@link VectorSchemaRoot}.
+   * Only Parquet data file format is supported.
+   */
+  private static final class VectorizedCombinedScanIterator implements CloseableIterator<ColumnarBatch> {
+
+    private final List<FileScanTask> fileTasks;
+    private final Iterator<FileScanTask> fileItr;
+    private final Map<String, InputFile> inputFiles;
+    private final Schema expectedSchema;
+    private final String nameMapping;
+    private final boolean caseSensitive;
+    private final int batchSize;
+    private final boolean reuseContainers;
+    private CloseableIterator<ColumnarBatch> currentIterator;
+    private ColumnarBatch current;
+    private FileScanTask currentTask;
+
+    /**
+     * Create a new instance.
+     *
+     * @param tasks             Combined file scan tasks.
+     * @param expectedSchema    Read schema. The returned data will have this schema.
+     * @param nameMapping       Mapping from external schema names to Iceberg type IDs.
+     * @param io                File I/O.
+     * @param encryptionManager Encryption manager.
+     * @param caseSensitive     If {@code true}, column names are case sensitive.
+     *                          If {@code false}, column names are not case sensitive.
+     * @param batchSize         Batch size in number of rows. Each Arrow batch contains
+     *                          a maximum of {@code batchSize} rows.
+     * @param reuseContainers   If set to {@code false}, every {@link Iterator#next()} call creates
+     *                          new instances of Arrow vectors.
+     *                          If set to {@code true}, the Arrow vectors in the previous
+     *                          {@link Iterator#next()} may be reused for the data returned
+     *                          in the current {@link Iterator#next()}.
+     *                          This option avoids allocating memory again and again.
+     *                          Irrespective of the value of {@code reuseContainers}, the Arrow vectors
+     *                          in the previous {@link Iterator#next()} call are closed before creating
+     *                          new instances if the current {@link Iterator#next()}.
+     */
+    VectorizedCombinedScanIterator(
+        CloseableIterable<CombinedScanTask> tasks,
+        Schema expectedSchema,
+        String nameMapping,
+        FileIO io,
+        EncryptionManager encryptionManager,
+        boolean caseSensitive,
+        int batchSize,
+        boolean reuseContainers) {
+      this.fileTasks = StreamSupport.stream(tasks.spliterator(), false)
+          .map(CombinedScanTask::files)
+          .flatMap(Collection::stream)
+          .collect(Collectors.toList());
+      this.fileItr = fileTasks.iterator();
+      this.inputFiles = Collections.unmodifiableMap(fileTasks.stream()
+          .flatMap(fileScanTask -> Stream.concat(Stream.of(fileScanTask.file()), fileScanTask.deletes().stream()))
+          .map(file -> EncryptedFiles.encryptedInput(io.newInputFile(file.path().toString()), file.keyMetadata()))
+          .map(encryptionManager::decrypt)
+          .collect(Collectors.toMap(InputFile::location, Function.identity())));
+      this.currentIterator = CloseableIterator.empty();
+      this.expectedSchema = expectedSchema;
+      this.nameMapping = nameMapping;
+      this.caseSensitive = caseSensitive;
+      this.batchSize = batchSize;
+      this.reuseContainers = reuseContainers;
+    }
+
+    @Override
+    public boolean hasNext() {
+      try {
+        while (true) {
+          if (currentIterator.hasNext()) {
+            this.current = currentIterator.next();
+            return true;
+          } else if (fileItr.hasNext()) {
+            this.currentIterator.close();
+            this.currentTask = fileItr.next();
+            this.currentIterator = open(currentTask);
+          } else {
+            this.currentIterator.close();
+            return false;
+          }
+        }
+      } catch (IOException | RuntimeException e) {
+        if (currentTask != null && !currentTask.isDataTask()) {
+          throw new RuntimeException(
+              "Error reading file: " + getInputFile(currentTask).location() +
+                  ". Reason: the current task is not a data task, i.e. it cannot read data rows. " +

Review comment:
       I think the problem is only the exception message. Let's get rid of the reason since it isn't accurate and go with the same thing that the `BaseDataReader` does. I think that this should also _not_ change the exception. Instead, I think it is a good idea to log the problem with the file location and then re-throw the original exception. Wrapping the exception makes it so any problem that occurs in the block can't be caught and handled.




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
rymurr commented on pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#issuecomment-829511260


   yeah, it looks like it was a flaky build. The build is passing this time so was prob just a flaky test in another module.


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] mayursrivastava commented on a change in pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
mayursrivastava commented on a change in pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#discussion_r612036564



##########
File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/VectorizedParquetReaders.java
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.arrow.vectorized;
+
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.parquet.schema.MessageType;
+
+/**
+ * Builds an {@link ArrowBatchReader}.
+ */
+class VectorizedParquetReaders {
+
+  private VectorizedParquetReaders() {
+  }
+
+  /**
+   * Build the {@link ArrowBatchReader} for the expected schema and file schema.
+   *
+   * @param expectedSchema         Expected schema of the data returned.
+   * @param fileSchema             Schema of the data file.
+   * @param setArrowValidityVector Indicates whether to set the validity vector in Arrow vectors.
+   */
+  public static ArrowBatchReader buildReader(

Review comment:
       removed this class.




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#discussion_r638281110



##########
File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowReader.java
##########
@@ -0,0 +1,325 @@
+/*
+ * 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.arrow.vectorized;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import java.util.stream.StreamSupport;
+import org.apache.arrow.vector.NullCheckingForGet;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.arrow.vector.types.Types.MinorType;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.encryption.EncryptedFiles;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.CloseableIterator;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.types.Types;
+import org.apache.parquet.schema.MessageType;
+
+/**
+ * Vectorized reader that returns an iterator of {@link ColumnarBatch}.
+ * See {@link #open(CloseableIterable)} ()} to learn about the
+ * behavior of the iterator.
+ *
+ * <p>The following Iceberg data types are supported and have been tested:
+ * <ul>
+ *     <li>Iceberg: {@link Types.BooleanType}, Arrow: {@link MinorType#BIT}</li>
+ *     <li>Iceberg: {@link Types.IntegerType}, Arrow: {@link MinorType#INT}</li>
+ *     <li>Iceberg: {@link Types.LongType}, Arrow: {@link MinorType#BIGINT}</li>
+ *     <li>Iceberg: {@link Types.FloatType}, Arrow: {@link MinorType#FLOAT4}</li>
+ *     <li>Iceberg: {@link Types.DoubleType}, Arrow: {@link MinorType#FLOAT8}</li>
+ *     <li>Iceberg: {@link Types.StringType}, Arrow: {@link MinorType#VARCHAR}</li>
+ *     <li>Iceberg: {@link Types.TimestampType} (both with and without timezone),
+ *         Arrow: {@link MinorType#TIMEMICRO}</li>
+ *     <li>Iceberg: {@link Types.BinaryType}, Arrow: {@link MinorType#VARBINARY}</li>
+ *     <li>Iceberg: {@link Types.DateType}, Arrow: {@link MinorType#DATEDAY}</li>
+ * </ul>
+ *
+ * <p>Features that don't work in this implementation:
+ * <ul>
+ *     <li>Type promotion: In case of type promotion, the Arrow vector corresponding to
+ *     the data type in the parquet file is returned instead of the data type in the latest schema.
+ *     See https://github.com/apache/iceberg/issues/2483.</li>
+ *     <li>Columns with constant values are physically encoded as a dictionary. The Arrow vector
+ *     type is int32 instead of the type as per the schema.
+ *     See https://github.com/apache/iceberg/issues/2484.</li>
+ *     <li>Data types: {@link Types.TimeType}, {@link Types.ListType}, {@link Types.MapType},
+ *     {@link Types.StructType}, {@link Types.UUIDType}, {@link Types.FixedType} and
+ *     {@link Types.DecimalType}
+ *     See https://github.com/apache/iceberg/issues/2485 and https://github.com/apache/iceberg/issues/2486.</li>
+ *     <li>Iceberg v2 spec is not supported.
+ *     See https://github.com/apache/iceberg/issues/2487.</li>

Review comment:
       Will this path fail if there is a delete file in one of the read tasks?




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#discussion_r638285987



##########
File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowReader.java
##########
@@ -0,0 +1,325 @@
+/*
+ * 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.arrow.vectorized;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import java.util.stream.StreamSupport;
+import org.apache.arrow.vector.NullCheckingForGet;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.arrow.vector.types.Types.MinorType;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.encryption.EncryptedFiles;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.CloseableIterator;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.types.Types;
+import org.apache.parquet.schema.MessageType;
+
+/**
+ * Vectorized reader that returns an iterator of {@link ColumnarBatch}.
+ * See {@link #open(CloseableIterable)} ()} to learn about the
+ * behavior of the iterator.
+ *
+ * <p>The following Iceberg data types are supported and have been tested:
+ * <ul>
+ *     <li>Iceberg: {@link Types.BooleanType}, Arrow: {@link MinorType#BIT}</li>
+ *     <li>Iceberg: {@link Types.IntegerType}, Arrow: {@link MinorType#INT}</li>
+ *     <li>Iceberg: {@link Types.LongType}, Arrow: {@link MinorType#BIGINT}</li>
+ *     <li>Iceberg: {@link Types.FloatType}, Arrow: {@link MinorType#FLOAT4}</li>
+ *     <li>Iceberg: {@link Types.DoubleType}, Arrow: {@link MinorType#FLOAT8}</li>
+ *     <li>Iceberg: {@link Types.StringType}, Arrow: {@link MinorType#VARCHAR}</li>
+ *     <li>Iceberg: {@link Types.TimestampType} (both with and without timezone),
+ *         Arrow: {@link MinorType#TIMEMICRO}</li>
+ *     <li>Iceberg: {@link Types.BinaryType}, Arrow: {@link MinorType#VARBINARY}</li>
+ *     <li>Iceberg: {@link Types.DateType}, Arrow: {@link MinorType#DATEDAY}</li>
+ * </ul>
+ *
+ * <p>Features that don't work in this implementation:
+ * <ul>
+ *     <li>Type promotion: In case of type promotion, the Arrow vector corresponding to
+ *     the data type in the parquet file is returned instead of the data type in the latest schema.
+ *     See https://github.com/apache/iceberg/issues/2483.</li>
+ *     <li>Columns with constant values are physically encoded as a dictionary. The Arrow vector
+ *     type is int32 instead of the type as per the schema.
+ *     See https://github.com/apache/iceberg/issues/2484.</li>
+ *     <li>Data types: {@link Types.TimeType}, {@link Types.ListType}, {@link Types.MapType},
+ *     {@link Types.StructType}, {@link Types.UUIDType}, {@link Types.FixedType} and
+ *     {@link Types.DecimalType}
+ *     See https://github.com/apache/iceberg/issues/2485 and https://github.com/apache/iceberg/issues/2486.</li>
+ *     <li>Iceberg v2 spec is not supported.
+ *     See https://github.com/apache/iceberg/issues/2487.</li>
+ * </ul>
+ */
+public class ArrowReader extends CloseableGroup {
+
+  private final Schema schema;
+  private final FileIO io;
+  private final EncryptionManager encryption;
+  private final int batchSize;
+  private final boolean reuseContainers;
+
+  /**
+   * Create a new instance of the reader.
+   *
+   * @param scan the table scan object.
+   * @param batchSize the maximum number of rows per Arrow batch.
+   * @param reuseContainers whether to reuse Arrow vectors when iterating through the data.
+   *                        If set to {@code false}, every {@link Iterator#next()} call creates
+   *                        new instances of Arrow vectors.
+   *                        If set to {@code true}, the Arrow vectors in the previous
+   *                        {@link Iterator#next()} may be reused for the data returned
+   *                        in the current {@link Iterator#next()}.
+   *                        This option avoids allocating memory again and again.
+   *                        Irrespective of the value of {@code reuseContainers}, the Arrow vectors
+   *                        in the previous {@link Iterator#next()} call are closed before creating
+   *                        new instances if the current {@link Iterator#next()}.
+   */
+  public ArrowReader(TableScan scan, int batchSize, boolean reuseContainers) {
+    this.schema = scan.schema();
+    this.io = scan.table().io();
+    this.encryption = scan.table().encryption();
+    this.batchSize = batchSize;
+    // start planning tasks in the background
+    this.reuseContainers = reuseContainers;
+  }
+
+  /**
+   * Returns a new iterator of {@link ColumnarBatch} objects.
+   * <p>
+   * Note that the reader owns the {@link ColumnarBatch} objects and takes care of closing them.
+   * The caller should not hold onto a {@link ColumnarBatch} or try to close them.
+   *
+   * <p>If {@code reuseContainers} is {@code false}, the Arrow vectors in the
+   * previous {@link ColumnarBatch} are closed before returning the next {@link ColumnarBatch} object.
+   * This implies that the caller should either use the {@link ColumnarBatch} or transfer the ownership of
+   * {@link ColumnarBatch} before getting the next {@link ColumnarBatch}.
+   *
+   * <p>If {@code reuseContainers} is {@code true}, the Arrow vectors in the
+   * previous {@link ColumnarBatch} may be reused for the next {@link ColumnarBatch}.
+   * This implies that the caller should either use the {@link ColumnarBatch} or deep copy the
+   * {@link ColumnarBatch} before getting the next {@link ColumnarBatch}.
+   */
+  public CloseableIterator<ColumnarBatch> open(CloseableIterable<CombinedScanTask> tasks) {
+    CloseableIterator<ColumnarBatch> itr = new VectorizedCombinedScanIterator(
+        tasks,
+        schema,
+        null,
+        io,
+        encryption,
+        true,
+        batchSize,
+        reuseContainers
+    );
+    addCloseable(itr);
+    return itr;
+  }
+
+  @Override
+  public void close() throws IOException {
+    super.close(); // close data files
+  }
+
+  /**
+   * Reads the data file and returns an iterator of {@link VectorSchemaRoot}.
+   * Only Parquet data file format is supported.
+   */
+  private static final class VectorizedCombinedScanIterator implements CloseableIterator<ColumnarBatch> {
+
+    private final List<FileScanTask> fileTasks;
+    private final Iterator<FileScanTask> fileItr;
+    private final Map<String, InputFile> inputFiles;
+    private final Schema expectedSchema;
+    private final String nameMapping;
+    private final boolean caseSensitive;
+    private final int batchSize;
+    private final boolean reuseContainers;
+    private CloseableIterator<ColumnarBatch> currentIterator;
+    private ColumnarBatch current;
+    private FileScanTask currentTask;
+
+    /**
+     * Create a new instance.
+     *
+     * @param tasks             Combined file scan tasks.
+     * @param expectedSchema    Read schema. The returned data will have this schema.
+     * @param nameMapping       Mapping from external schema names to Iceberg type IDs.
+     * @param io                File I/O.
+     * @param encryptionManager Encryption manager.
+     * @param caseSensitive     If {@code true}, column names are case sensitive.
+     *                          If {@code false}, column names are not case sensitive.
+     * @param batchSize         Batch size in number of rows. Each Arrow batch contains
+     *                          a maximum of {@code batchSize} rows.
+     * @param reuseContainers   If set to {@code false}, every {@link Iterator#next()} call creates
+     *                          new instances of Arrow vectors.
+     *                          If set to {@code true}, the Arrow vectors in the previous
+     *                          {@link Iterator#next()} may be reused for the data returned
+     *                          in the current {@link Iterator#next()}.
+     *                          This option avoids allocating memory again and again.
+     *                          Irrespective of the value of {@code reuseContainers}, the Arrow vectors
+     *                          in the previous {@link Iterator#next()} call are closed before creating
+     *                          new instances if the current {@link Iterator#next()}.
+     */
+    VectorizedCombinedScanIterator(
+        CloseableIterable<CombinedScanTask> tasks,
+        Schema expectedSchema,
+        String nameMapping,
+        FileIO io,
+        EncryptionManager encryptionManager,
+        boolean caseSensitive,
+        int batchSize,
+        boolean reuseContainers) {
+      this.fileTasks = StreamSupport.stream(tasks.spliterator(), false)
+          .map(CombinedScanTask::files)
+          .flatMap(Collection::stream)
+          .collect(Collectors.toList());
+      this.fileItr = fileTasks.iterator();
+      this.inputFiles = Collections.unmodifiableMap(fileTasks.stream()
+          .flatMap(fileScanTask -> Stream.concat(Stream.of(fileScanTask.file()), fileScanTask.deletes().stream()))
+          .map(file -> EncryptedFiles.encryptedInput(io.newInputFile(file.path().toString()), file.keyMetadata()))
+          .map(encryptionManager::decrypt)
+          .collect(Collectors.toMap(InputFile::location, Function.identity())));
+      this.currentIterator = CloseableIterator.empty();
+      this.expectedSchema = expectedSchema;
+      this.nameMapping = nameMapping;
+      this.caseSensitive = caseSensitive;
+      this.batchSize = batchSize;
+      this.reuseContainers = reuseContainers;
+    }
+
+    @Override
+    public boolean hasNext() {
+      try {
+        while (true) {
+          if (currentIterator.hasNext()) {
+            this.current = currentIterator.next();
+            return true;
+          } else if (fileItr.hasNext()) {
+            this.currentIterator.close();
+            this.currentTask = fileItr.next();
+            this.currentIterator = open(currentTask);
+          } else {
+            this.currentIterator.close();
+            return false;
+          }
+        }
+      } catch (IOException | RuntimeException e) {
+        if (currentTask != null && !currentTask.isDataTask()) {
+          throw new RuntimeException(
+              "Error reading file: " + getInputFile(currentTask).location() +
+                  ". Reason: the current task is not a data task, i.e. it cannot read data rows. " +

Review comment:
       This looks incorrect to me. A data task is one that produces its own rows. For example, a static task with version history for the history table is a data task. This description seems to say the opposite is a problem, that if a task is not a data task it can't be read. That's not correct.




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] mayursrivastava edited a comment on pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
mayursrivastava edited a comment on pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#issuecomment-824210180


   `
   ./gradlew :iceberg-spark:jmh -PjmhIncludeRegex=VectorizedReadFlatParquetDataBenchmark -PjmhOutputPath=benchmark/results.txt
   `
   
   @rymurr, is this :iceberg-spark2:jmh? Looks like this requires Java8 runtime. Am I right? I ran it on the master branch (without my changes), but it fails with the following error. Does it need a powerful machine?
   
   `
   JMH version: 1.21
   VM version: JDK 1.8.0_282, OpenJDK 64-Bit Server VM, 25.282-b08
   VM invoker: /usr/lib/jvm/java-8-openjdk-amd64/jre/bin/java
   VM options: <none>
   Warmup: 3 iterations, single-shot each
   Measurement: 5 iterations, single-shot each
   Timeout: 10 min per iteration
   Threads: 1 thread
   Benchmark mode: Single shot invocation time
   Benchmark: org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadDictionaryEncodedFlatParquetDataBenchmark.readDatesIcebergVectorized5k
   
   # Run progress: 0.00% complete, ETA 00:00:00
   # Fork: 1 of 1
   # Warmup Iteration   1: (*interrupt*) <failure>
   
   java.lang.InterruptedException
           at java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireSharedInterruptibly(AbstractQueuedSynchronizer.java:998)
           at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireSharedInterruptibly(AbstractQueuedSynchronizer.java:1304)
           at scala.concurrent.impl.Promise$DefaultPromise.tryAwait(Promise.scala:206)
           at scala.concurrent.impl.Promise$DefaultPromise.ready(Promise.scala:222)
           at scala.concurrent.impl.Promise$DefaultPromise.ready(Promise.scala:157)
           at org.apache.spark.util.ThreadUtils$.awaitReady(ThreadUtils.scala:243)
           at org.apache.spark.scheduler.DAGScheduler.runJob(DAGScheduler.scala:750)
           at org.apache.spark.SparkContext.runJob(SparkContext.scala:2061)
           at org.apache.spark.sql.execution.datasources.v2.WriteToDataSourceV2Exec.doExecute(WriteToDataSourceV2Exec.scala:64)
           at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:131)
           at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:127)
           at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:155)
           at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
           at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:152)
           at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:127)
           at org.apache.spark.sql.execution.QueryExecution.toRdd$lzycompute(QueryExecution.scala:83)
           at org.apache.spark.sql.execution.QueryExecution.toRdd(QueryExecution.scala:81)
           at org.apache.spark.sql.DataFrameWriter$$anonfun$runCommand$1.apply(DataFrameWriter.scala:696)
           at org.apache.spark.sql.DataFrameWriter$$anonfun$runCommand$1.apply(DataFrameWriter.scala:696)
           at org.apache.spark.sql.execution.SQLExecution$$anonfun$withNewExecutionId$1.apply(SQLExecution.scala:80)
           at org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:127)
           at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:75)
           at org.apache.spark.sql.DataFrameWriter.runCommand(DataFrameWriter.scala:696)
           at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:280)
           at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:249)
           at org.apache.iceberg.spark.source.IcebergSourceBenchmark.appendAsFile(IcebergSourceBenchmark.java:130)
           at org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadDictionaryEncodedFlatParquetDataBenchmark.appendData(VectorizedReadDictionaryEncodedFlatParquetDataBenchmark.java:82)
           at org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadDictionaryEncodedFlatParquetDataBenchmark.setupBenchmark(VectorizedReadDictionaryEncodedFlatParquetDataBenchmark.java:56)
           at org.apache.iceberg.spark.source.parquet.vectorized.generated.VectorizedReadDictionaryEncodedFlatParquetDataBenchmark_readDatesIcebergVectorized5k_jmhTest._jmh_tryInit_f_vectorizedreaddictionaryencodedflatparquetdatabenchmark0_G(VectorizedReadDictionaryEncodedFlatParquetDataBenchmark_readDatesIcebergVectorized5k_jmhTest.java:438)
           at org.apache.iceberg.spark.source.parquet.vectorized.generated.VectorizedReadDictionaryEncodedFlatParquetDataBenchmark_readDatesIcebergVectorized5k_jmhTest.readDatesIcebergVectorized5k_SingleShotTime(VectorizedReadDictionaryEncodedFlatParquetDataBenchmark_readDatesIcebergVectorized5k_jmhTest.java:363)
           at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
           at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
           at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
           at java.lang.reflect.Method.invoke(Method.java:498)
           at org.openjdk.jmh.runner.BenchmarkHandler$BenchmarkTask.call(BenchmarkHandler.java:453)
           at org.openjdk.jmh.runner.BenchmarkHandler$BenchmarkTask.call(BenchmarkHandler.java:437)
           at java.util.concurrent.FutureTask.run(FutureTask.java:266)
           at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
           at java.util.concurrent.FutureTask.run(FutureTask.java:266)
           at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
           at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
           at java.lang.Thread.run(Thread.java:748)
   
   
   Benchmark had encountered error, and fail on error was requested
   `


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] mayursrivastava commented on a change in pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
mayursrivastava commented on a change in pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#discussion_r617715020



##########
File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowBatchReader.java
##########
@@ -0,0 +1,106 @@
+/*
+ * 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.arrow.vectorized;
+
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.parquet.VectorizedReader;
+import org.apache.parquet.Preconditions;
+import org.apache.parquet.column.page.PageReadStore;
+import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
+import org.apache.parquet.hadoop.metadata.ColumnPath;
+
+/**
+ * A collection of vectorized readers per column (in the expected read schema) and Arrow Vector holders. This class owns
+ * the Arrow vectors and is responsible for closing the Arrow vectors.
+ */
+class ArrowBatchReader implements VectorizedReader<ColumnarBatch> {
+
+  private final VectorizedArrowReader[] readers;
+  private final VectorHolder[] vectorHolders;
+
+  ArrowBatchReader(List<VectorizedReader<?>> readers) {
+    this.readers = readers.stream()
+        .map(VectorizedArrowReader.class::cast)
+        .toArray(VectorizedArrowReader[]::new);
+    this.vectorHolders = new VectorHolder[readers.size()];
+  }
+
+  @Override
+  public final void setRowGroupInfo(
+      PageReadStore pageStore, Map<ColumnPath, ColumnChunkMetaData> metaData, long rowPosition) {
+    for (VectorizedArrowReader reader : readers) {

Review comment:
       It should not be null. I got this code from the ColumnarBatchReader. I've removed the null check from ArrowBatchReader.




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] mayursrivastava commented on a change in pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
mayursrivastava commented on a change in pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#discussion_r638324812



##########
File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowReader.java
##########
@@ -0,0 +1,325 @@
+/*
+ * 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.arrow.vectorized;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import java.util.stream.StreamSupport;
+import org.apache.arrow.vector.NullCheckingForGet;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.arrow.vector.types.Types.MinorType;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.encryption.EncryptedFiles;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.CloseableIterator;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.types.Types;
+import org.apache.parquet.schema.MessageType;
+
+/**
+ * Vectorized reader that returns an iterator of {@link ColumnarBatch}.
+ * See {@link #open(CloseableIterable)} ()} to learn about the
+ * behavior of the iterator.
+ *
+ * <p>The following Iceberg data types are supported and have been tested:
+ * <ul>
+ *     <li>Iceberg: {@link Types.BooleanType}, Arrow: {@link MinorType#BIT}</li>
+ *     <li>Iceberg: {@link Types.IntegerType}, Arrow: {@link MinorType#INT}</li>
+ *     <li>Iceberg: {@link Types.LongType}, Arrow: {@link MinorType#BIGINT}</li>
+ *     <li>Iceberg: {@link Types.FloatType}, Arrow: {@link MinorType#FLOAT4}</li>
+ *     <li>Iceberg: {@link Types.DoubleType}, Arrow: {@link MinorType#FLOAT8}</li>
+ *     <li>Iceberg: {@link Types.StringType}, Arrow: {@link MinorType#VARCHAR}</li>
+ *     <li>Iceberg: {@link Types.TimestampType} (both with and without timezone),
+ *         Arrow: {@link MinorType#TIMEMICRO}</li>
+ *     <li>Iceberg: {@link Types.BinaryType}, Arrow: {@link MinorType#VARBINARY}</li>
+ *     <li>Iceberg: {@link Types.DateType}, Arrow: {@link MinorType#DATEDAY}</li>
+ * </ul>
+ *
+ * <p>Features that don't work in this implementation:
+ * <ul>
+ *     <li>Type promotion: In case of type promotion, the Arrow vector corresponding to
+ *     the data type in the parquet file is returned instead of the data type in the latest schema.
+ *     See https://github.com/apache/iceberg/issues/2483.</li>
+ *     <li>Columns with constant values are physically encoded as a dictionary. The Arrow vector
+ *     type is int32 instead of the type as per the schema.
+ *     See https://github.com/apache/iceberg/issues/2484.</li>
+ *     <li>Data types: {@link Types.TimeType}, {@link Types.ListType}, {@link Types.MapType},
+ *     {@link Types.StructType}, {@link Types.UUIDType}, {@link Types.FixedType} and
+ *     {@link Types.DecimalType}
+ *     See https://github.com/apache/iceberg/issues/2485 and https://github.com/apache/iceberg/issues/2486.</li>
+ *     <li>Iceberg v2 spec is not supported.
+ *     See https://github.com/apache/iceberg/issues/2487.</li>

Review comment:
       Not currently, but I'll check for delete files and raise an exception.




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#discussion_r638344316



##########
File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ColumnVector.java
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.arrow.vectorized;
+
+import org.apache.arrow.vector.FieldVector;
+import org.apache.iceberg.types.Types;
+
+/**
+ * This class is inspired by Spark's {@code ColumnVector}.
+ * This class represents the column data for an Iceberg table query.
+ * It wraps an arrow {@link FieldVector} and provides simple
+ * accessors for the row values. Advanced users can access
+ * the {@link FieldVector}.
+ * <p>
+ *   Supported Iceberg data types:
+ *   <ul>
+ *     <li>{@link Types.BooleanType}</li>
+ *     <li>{@link Types.IntegerType}</li>
+ *     <li>{@link Types.LongType}</li>
+ *     <li>{@link Types.FloatType}</li>
+ *     <li>{@link Types.DoubleType}</li>
+ *     <li>{@link Types.StringType}</li>
+ *     <li>{@link Types.BinaryType}</li>
+ *     <li>{@link Types.TimestampType} (with and without timezone)</li>
+ *     <li>{@link Types.DateType}</li>
+ *   </ul>
+ */
+public class ColumnVector implements AutoCloseable {

Review comment:
       @mayursrivastava, maybe you could comment on this choice as well?




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] mayursrivastava commented on pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
mayursrivastava commented on pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#issuecomment-821308714


   Support for decimal types should not be difficult. If complex types are supported in the parquet to arrow reader, then it should require adding classes to represent child (similar to Spark) and adding tests.


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
rymurr commented on pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#issuecomment-829218210


   > Thanks @rymurr for your time to look into this. I have a question: how will this pr get merged?
   
   I will merge it today. Just wanted to give a few hours to see if anyone else wants to chime in


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#discussion_r638341789



##########
File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowReader.java
##########
@@ -0,0 +1,325 @@
+/*
+ * 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.arrow.vectorized;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import java.util.stream.StreamSupport;
+import org.apache.arrow.vector.NullCheckingForGet;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.arrow.vector.types.Types.MinorType;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.encryption.EncryptedFiles;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.CloseableIterator;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.types.Types;
+import org.apache.parquet.schema.MessageType;
+
+/**
+ * Vectorized reader that returns an iterator of {@link ColumnarBatch}.
+ * See {@link #open(CloseableIterable)} ()} to learn about the
+ * behavior of the iterator.
+ *
+ * <p>The following Iceberg data types are supported and have been tested:
+ * <ul>
+ *     <li>Iceberg: {@link Types.BooleanType}, Arrow: {@link MinorType#BIT}</li>
+ *     <li>Iceberg: {@link Types.IntegerType}, Arrow: {@link MinorType#INT}</li>
+ *     <li>Iceberg: {@link Types.LongType}, Arrow: {@link MinorType#BIGINT}</li>
+ *     <li>Iceberg: {@link Types.FloatType}, Arrow: {@link MinorType#FLOAT4}</li>
+ *     <li>Iceberg: {@link Types.DoubleType}, Arrow: {@link MinorType#FLOAT8}</li>
+ *     <li>Iceberg: {@link Types.StringType}, Arrow: {@link MinorType#VARCHAR}</li>
+ *     <li>Iceberg: {@link Types.TimestampType} (both with and without timezone),
+ *         Arrow: {@link MinorType#TIMEMICRO}</li>
+ *     <li>Iceberg: {@link Types.BinaryType}, Arrow: {@link MinorType#VARBINARY}</li>
+ *     <li>Iceberg: {@link Types.DateType}, Arrow: {@link MinorType#DATEDAY}</li>
+ * </ul>
+ *
+ * <p>Features that don't work in this implementation:
+ * <ul>
+ *     <li>Type promotion: In case of type promotion, the Arrow vector corresponding to
+ *     the data type in the parquet file is returned instead of the data type in the latest schema.
+ *     See https://github.com/apache/iceberg/issues/2483.</li>
+ *     <li>Columns with constant values are physically encoded as a dictionary. The Arrow vector
+ *     type is int32 instead of the type as per the schema.
+ *     See https://github.com/apache/iceberg/issues/2484.</li>
+ *     <li>Data types: {@link Types.TimeType}, {@link Types.ListType}, {@link Types.MapType},
+ *     {@link Types.StructType}, {@link Types.UUIDType}, {@link Types.FixedType} and
+ *     {@link Types.DecimalType}
+ *     See https://github.com/apache/iceberg/issues/2485 and https://github.com/apache/iceberg/issues/2486.</li>
+ *     <li>Iceberg v2 spec is not supported.
+ *     See https://github.com/apache/iceberg/issues/2487.</li>
+ * </ul>
+ */
+public class ArrowReader extends CloseableGroup {
+
+  private final Schema schema;
+  private final FileIO io;
+  private final EncryptionManager encryption;
+  private final int batchSize;
+  private final boolean reuseContainers;
+
+  /**
+   * Create a new instance of the reader.
+   *
+   * @param scan the table scan object.
+   * @param batchSize the maximum number of rows per Arrow batch.
+   * @param reuseContainers whether to reuse Arrow vectors when iterating through the data.
+   *                        If set to {@code false}, every {@link Iterator#next()} call creates
+   *                        new instances of Arrow vectors.
+   *                        If set to {@code true}, the Arrow vectors in the previous
+   *                        {@link Iterator#next()} may be reused for the data returned
+   *                        in the current {@link Iterator#next()}.
+   *                        This option avoids allocating memory again and again.
+   *                        Irrespective of the value of {@code reuseContainers}, the Arrow vectors
+   *                        in the previous {@link Iterator#next()} call are closed before creating
+   *                        new instances if the current {@link Iterator#next()}.
+   */
+  public ArrowReader(TableScan scan, int batchSize, boolean reuseContainers) {
+    this.schema = scan.schema();
+    this.io = scan.table().io();
+    this.encryption = scan.table().encryption();
+    this.batchSize = batchSize;
+    // start planning tasks in the background
+    this.reuseContainers = reuseContainers;
+  }
+
+  /**
+   * Returns a new iterator of {@link ColumnarBatch} objects.
+   * <p>
+   * Note that the reader owns the {@link ColumnarBatch} objects and takes care of closing them.
+   * The caller should not hold onto a {@link ColumnarBatch} or try to close them.
+   *
+   * <p>If {@code reuseContainers} is {@code false}, the Arrow vectors in the
+   * previous {@link ColumnarBatch} are closed before returning the next {@link ColumnarBatch} object.
+   * This implies that the caller should either use the {@link ColumnarBatch} or transfer the ownership of
+   * {@link ColumnarBatch} before getting the next {@link ColumnarBatch}.
+   *
+   * <p>If {@code reuseContainers} is {@code true}, the Arrow vectors in the
+   * previous {@link ColumnarBatch} may be reused for the next {@link ColumnarBatch}.
+   * This implies that the caller should either use the {@link ColumnarBatch} or deep copy the
+   * {@link ColumnarBatch} before getting the next {@link ColumnarBatch}.
+   */
+  public CloseableIterator<ColumnarBatch> open(CloseableIterable<CombinedScanTask> tasks) {
+    CloseableIterator<ColumnarBatch> itr = new VectorizedCombinedScanIterator(
+        tasks,
+        schema,
+        null,
+        io,
+        encryption,
+        true,
+        batchSize,
+        reuseContainers
+    );
+    addCloseable(itr);
+    return itr;
+  }
+
+  @Override
+  public void close() throws IOException {
+    super.close(); // close data files
+  }
+
+  /**
+   * Reads the data file and returns an iterator of {@link VectorSchemaRoot}.
+   * Only Parquet data file format is supported.
+   */
+  private static final class VectorizedCombinedScanIterator implements CloseableIterator<ColumnarBatch> {
+
+    private final List<FileScanTask> fileTasks;
+    private final Iterator<FileScanTask> fileItr;
+    private final Map<String, InputFile> inputFiles;
+    private final Schema expectedSchema;
+    private final String nameMapping;
+    private final boolean caseSensitive;
+    private final int batchSize;
+    private final boolean reuseContainers;
+    private CloseableIterator<ColumnarBatch> currentIterator;
+    private ColumnarBatch current;
+    private FileScanTask currentTask;
+
+    /**
+     * Create a new instance.
+     *
+     * @param tasks             Combined file scan tasks.
+     * @param expectedSchema    Read schema. The returned data will have this schema.
+     * @param nameMapping       Mapping from external schema names to Iceberg type IDs.
+     * @param io                File I/O.
+     * @param encryptionManager Encryption manager.
+     * @param caseSensitive     If {@code true}, column names are case sensitive.
+     *                          If {@code false}, column names are not case sensitive.
+     * @param batchSize         Batch size in number of rows. Each Arrow batch contains
+     *                          a maximum of {@code batchSize} rows.
+     * @param reuseContainers   If set to {@code false}, every {@link Iterator#next()} call creates
+     *                          new instances of Arrow vectors.
+     *                          If set to {@code true}, the Arrow vectors in the previous
+     *                          {@link Iterator#next()} may be reused for the data returned
+     *                          in the current {@link Iterator#next()}.
+     *                          This option avoids allocating memory again and again.
+     *                          Irrespective of the value of {@code reuseContainers}, the Arrow vectors
+     *                          in the previous {@link Iterator#next()} call are closed before creating
+     *                          new instances if the current {@link Iterator#next()}.
+     */
+    VectorizedCombinedScanIterator(
+        CloseableIterable<CombinedScanTask> tasks,
+        Schema expectedSchema,
+        String nameMapping,
+        FileIO io,
+        EncryptionManager encryptionManager,
+        boolean caseSensitive,
+        int batchSize,
+        boolean reuseContainers) {
+      this.fileTasks = StreamSupport.stream(tasks.spliterator(), false)
+          .map(CombinedScanTask::files)
+          .flatMap(Collection::stream)
+          .collect(Collectors.toList());
+      this.fileItr = fileTasks.iterator();
+      this.inputFiles = Collections.unmodifiableMap(fileTasks.stream()
+          .flatMap(fileScanTask -> Stream.concat(Stream.of(fileScanTask.file()), fileScanTask.deletes().stream()))
+          .map(file -> EncryptedFiles.encryptedInput(io.newInputFile(file.path().toString()), file.keyMetadata()))
+          .map(encryptionManager::decrypt)

Review comment:
       Yes, a follow-up is fine with me.




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] mayursrivastava commented on pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
mayursrivastava commented on pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#issuecomment-829176570


   Thanks @rymurr for your time to look into this. I have a question: how will this pr get merged?


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] mayursrivastava commented on a change in pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
mayursrivastava commented on a change in pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#discussion_r591990474



##########
File path: build.gradle
##########
@@ -738,6 +738,13 @@ project(':iceberg-arrow') {
       exclude group: 'io.netty', module: 'netty-common'
       exclude group: 'com.google.code.findbugs', module: 'jsr305'
     }
+
+    testCompile("org.apache.arrow:arrow-memory-netty") {

Review comment:
       these dependencies are needed for the ArrowReaderTest.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] mayursrivastava commented on a change in pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
mayursrivastava commented on a change in pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#discussion_r612382323



##########
File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowVector.java
##########
@@ -0,0 +1,513 @@
+/*
+ * 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.arrow.vectorized;
+
+import java.nio.charset.StandardCharsets;
+import java.util.stream.IntStream;
+import org.apache.arrow.vector.BigIntVector;
+import org.apache.arrow.vector.BitVector;
+import org.apache.arrow.vector.DateDayVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.Float4Vector;
+import org.apache.arrow.vector.Float8Vector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.TimeStampMicroTZVector;
+import org.apache.arrow.vector.TimeStampMicroVector;
+import org.apache.arrow.vector.ValueVector;
+import org.apache.arrow.vector.VarBinaryVector;
+import org.apache.arrow.vector.VarCharVector;
+import org.apache.arrow.vector.holders.NullableVarCharHolder;
+import org.apache.iceberg.types.Types;
+import org.apache.parquet.Preconditions;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.PrimitiveType;
+
+/**
+ * This class is inspired by Spark's {@code ColumnVector}.
+ * This class represents the column data for an Iceberg table query.
+ * It wraps an arrow {@link FieldVector} and provides simple
+ * accessors for the row values. Advanced users can access
+ * the {@link FieldVector}.
+ * <p>
+ *   Supported Iceberg data types:
+ *   <ul>
+ *     <li>{@link Types.BooleanType}</li>
+ *     <li>{@link Types.IntegerType}</li>
+ *     <li>{@link Types.LongType}</li>
+ *     <li>{@link Types.FloatType}</li>
+ *     <li>{@link Types.DoubleType}</li>
+ *     <li>{@link Types.StringType}</li>
+ *     <li>{@link Types.BinaryType}</li>
+ *     <li>{@link Types.TimestampType} (with and without timezone)</li>
+ *     <li>{@link Types.DateType}</li>
+ *   </ul>
+ */
+public class ArrowVector implements AutoCloseable {

Review comment:
       Updated the pr




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] mayursrivastava commented on a change in pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
mayursrivastava commented on a change in pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#discussion_r609628083



##########
File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowBatch.java
##########
@@ -0,0 +1,82 @@
+/*
+ * 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.arrow.vectorized;
+
+import java.util.Arrays;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.VectorSchemaRoot;
+
+/**
+ * This class is inspired by Spark's {@code ColumnarBatch}.
+ * This class wraps a columnar batch in the result set of an Iceberg table query.
+ */
+public class ArrowBatch implements AutoCloseable {

Review comment:
       I can try to move ColumnBatch to arrow module. 




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
rymurr commented on pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#issuecomment-824213194


   Yeah, tahts the right benchmark and right jvm. Haven't seen it encounter that error before. Does it run on `master` branch?


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] mayursrivastava commented on a change in pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
mayursrivastava commented on a change in pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#discussion_r638331286



##########
File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowReader.java
##########
@@ -0,0 +1,325 @@
+/*
+ * 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.arrow.vectorized;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import java.util.stream.StreamSupport;
+import org.apache.arrow.vector.NullCheckingForGet;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.arrow.vector.types.Types.MinorType;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.encryption.EncryptedFiles;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.CloseableIterator;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.types.Types;
+import org.apache.parquet.schema.MessageType;
+
+/**
+ * Vectorized reader that returns an iterator of {@link ColumnarBatch}.
+ * See {@link #open(CloseableIterable)} ()} to learn about the
+ * behavior of the iterator.
+ *
+ * <p>The following Iceberg data types are supported and have been tested:
+ * <ul>
+ *     <li>Iceberg: {@link Types.BooleanType}, Arrow: {@link MinorType#BIT}</li>
+ *     <li>Iceberg: {@link Types.IntegerType}, Arrow: {@link MinorType#INT}</li>
+ *     <li>Iceberg: {@link Types.LongType}, Arrow: {@link MinorType#BIGINT}</li>
+ *     <li>Iceberg: {@link Types.FloatType}, Arrow: {@link MinorType#FLOAT4}</li>
+ *     <li>Iceberg: {@link Types.DoubleType}, Arrow: {@link MinorType#FLOAT8}</li>
+ *     <li>Iceberg: {@link Types.StringType}, Arrow: {@link MinorType#VARCHAR}</li>
+ *     <li>Iceberg: {@link Types.TimestampType} (both with and without timezone),
+ *         Arrow: {@link MinorType#TIMEMICRO}</li>
+ *     <li>Iceberg: {@link Types.BinaryType}, Arrow: {@link MinorType#VARBINARY}</li>
+ *     <li>Iceberg: {@link Types.DateType}, Arrow: {@link MinorType#DATEDAY}</li>
+ * </ul>
+ *
+ * <p>Features that don't work in this implementation:
+ * <ul>
+ *     <li>Type promotion: In case of type promotion, the Arrow vector corresponding to
+ *     the data type in the parquet file is returned instead of the data type in the latest schema.
+ *     See https://github.com/apache/iceberg/issues/2483.</li>
+ *     <li>Columns with constant values are physically encoded as a dictionary. The Arrow vector
+ *     type is int32 instead of the type as per the schema.
+ *     See https://github.com/apache/iceberg/issues/2484.</li>
+ *     <li>Data types: {@link Types.TimeType}, {@link Types.ListType}, {@link Types.MapType},
+ *     {@link Types.StructType}, {@link Types.UUIDType}, {@link Types.FixedType} and
+ *     {@link Types.DecimalType}
+ *     See https://github.com/apache/iceberg/issues/2485 and https://github.com/apache/iceberg/issues/2486.</li>
+ *     <li>Iceberg v2 spec is not supported.
+ *     See https://github.com/apache/iceberg/issues/2487.</li>
+ * </ul>
+ */
+public class ArrowReader extends CloseableGroup {
+
+  private final Schema schema;
+  private final FileIO io;
+  private final EncryptionManager encryption;
+  private final int batchSize;
+  private final boolean reuseContainers;
+
+  /**
+   * Create a new instance of the reader.
+   *
+   * @param scan the table scan object.
+   * @param batchSize the maximum number of rows per Arrow batch.
+   * @param reuseContainers whether to reuse Arrow vectors when iterating through the data.
+   *                        If set to {@code false}, every {@link Iterator#next()} call creates
+   *                        new instances of Arrow vectors.
+   *                        If set to {@code true}, the Arrow vectors in the previous
+   *                        {@link Iterator#next()} may be reused for the data returned
+   *                        in the current {@link Iterator#next()}.
+   *                        This option avoids allocating memory again and again.
+   *                        Irrespective of the value of {@code reuseContainers}, the Arrow vectors
+   *                        in the previous {@link Iterator#next()} call are closed before creating
+   *                        new instances if the current {@link Iterator#next()}.
+   */
+  public ArrowReader(TableScan scan, int batchSize, boolean reuseContainers) {
+    this.schema = scan.schema();
+    this.io = scan.table().io();
+    this.encryption = scan.table().encryption();
+    this.batchSize = batchSize;
+    // start planning tasks in the background
+    this.reuseContainers = reuseContainers;
+  }
+
+  /**
+   * Returns a new iterator of {@link ColumnarBatch} objects.
+   * <p>
+   * Note that the reader owns the {@link ColumnarBatch} objects and takes care of closing them.
+   * The caller should not hold onto a {@link ColumnarBatch} or try to close them.
+   *
+   * <p>If {@code reuseContainers} is {@code false}, the Arrow vectors in the
+   * previous {@link ColumnarBatch} are closed before returning the next {@link ColumnarBatch} object.
+   * This implies that the caller should either use the {@link ColumnarBatch} or transfer the ownership of
+   * {@link ColumnarBatch} before getting the next {@link ColumnarBatch}.
+   *
+   * <p>If {@code reuseContainers} is {@code true}, the Arrow vectors in the
+   * previous {@link ColumnarBatch} may be reused for the next {@link ColumnarBatch}.
+   * This implies that the caller should either use the {@link ColumnarBatch} or deep copy the
+   * {@link ColumnarBatch} before getting the next {@link ColumnarBatch}.
+   */
+  public CloseableIterator<ColumnarBatch> open(CloseableIterable<CombinedScanTask> tasks) {
+    CloseableIterator<ColumnarBatch> itr = new VectorizedCombinedScanIterator(
+        tasks,
+        schema,
+        null,
+        io,
+        encryption,
+        true,
+        batchSize,
+        reuseContainers
+    );
+    addCloseable(itr);
+    return itr;
+  }
+
+  @Override
+  public void close() throws IOException {
+    super.close(); // close data files
+  }
+
+  /**
+   * Reads the data file and returns an iterator of {@link VectorSchemaRoot}.
+   * Only Parquet data file format is supported.
+   */
+  private static final class VectorizedCombinedScanIterator implements CloseableIterator<ColumnarBatch> {
+
+    private final List<FileScanTask> fileTasks;
+    private final Iterator<FileScanTask> fileItr;
+    private final Map<String, InputFile> inputFiles;
+    private final Schema expectedSchema;
+    private final String nameMapping;
+    private final boolean caseSensitive;
+    private final int batchSize;
+    private final boolean reuseContainers;
+    private CloseableIterator<ColumnarBatch> currentIterator;
+    private ColumnarBatch current;
+    private FileScanTask currentTask;
+
+    /**
+     * Create a new instance.
+     *
+     * @param tasks             Combined file scan tasks.
+     * @param expectedSchema    Read schema. The returned data will have this schema.
+     * @param nameMapping       Mapping from external schema names to Iceberg type IDs.
+     * @param io                File I/O.
+     * @param encryptionManager Encryption manager.
+     * @param caseSensitive     If {@code true}, column names are case sensitive.
+     *                          If {@code false}, column names are not case sensitive.
+     * @param batchSize         Batch size in number of rows. Each Arrow batch contains
+     *                          a maximum of {@code batchSize} rows.
+     * @param reuseContainers   If set to {@code false}, every {@link Iterator#next()} call creates
+     *                          new instances of Arrow vectors.
+     *                          If set to {@code true}, the Arrow vectors in the previous
+     *                          {@link Iterator#next()} may be reused for the data returned
+     *                          in the current {@link Iterator#next()}.
+     *                          This option avoids allocating memory again and again.
+     *                          Irrespective of the value of {@code reuseContainers}, the Arrow vectors
+     *                          in the previous {@link Iterator#next()} call are closed before creating
+     *                          new instances if the current {@link Iterator#next()}.
+     */
+    VectorizedCombinedScanIterator(
+        CloseableIterable<CombinedScanTask> tasks,
+        Schema expectedSchema,
+        String nameMapping,
+        FileIO io,
+        EncryptionManager encryptionManager,
+        boolean caseSensitive,
+        int batchSize,
+        boolean reuseContainers) {
+      this.fileTasks = StreamSupport.stream(tasks.spliterator(), false)
+          .map(CombinedScanTask::files)
+          .flatMap(Collection::stream)
+          .collect(Collectors.toList());
+      this.fileItr = fileTasks.iterator();
+      this.inputFiles = Collections.unmodifiableMap(fileTasks.stream()
+          .flatMap(fileScanTask -> Stream.concat(Stream.of(fileScanTask.file()), fileScanTask.deletes().stream()))
+          .map(file -> EncryptedFiles.encryptedInput(io.newInputFile(file.path().toString()), file.keyMetadata()))
+          .map(encryptionManager::decrypt)
+          .collect(Collectors.toMap(InputFile::location, Function.identity())));
+      this.currentIterator = CloseableIterator.empty();
+      this.expectedSchema = expectedSchema;
+      this.nameMapping = nameMapping;
+      this.caseSensitive = caseSensitive;
+      this.batchSize = batchSize;
+      this.reuseContainers = reuseContainers;
+    }
+
+    @Override
+    public boolean hasNext() {
+      try {
+        while (true) {
+          if (currentIterator.hasNext()) {
+            this.current = currentIterator.next();
+            return true;
+          } else if (fileItr.hasNext()) {
+            this.currentIterator.close();
+            this.currentTask = fileItr.next();
+            this.currentIterator = open(currentTask);
+          } else {
+            this.currentIterator.close();
+            return false;
+          }
+        }
+      } catch (IOException | RuntimeException e) {
+        if (currentTask != null && !currentTask.isDataTask()) {
+          throw new RuntimeException(
+              "Error reading file: " + getInputFile(currentTask).location() +
+                  ". Reason: the current task is not a data task, i.e. it cannot read data rows. " +

Review comment:
       Ryan, I need some clarification from you.
   (For reference, this code is inspired from https://github.com/apache/iceberg/blob/master/spark/src/main/java/org/apache/iceberg/spark/source/BaseDataReader.java#L100.) In the spark code, the same check is done to do some error logging. If this check is not correct, we can get rid of this exception (the exception is re-raised later in the catch block). Otherwise, let me know if I should correct the exception message only.




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#discussion_r638281774



##########
File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowReader.java
##########
@@ -0,0 +1,325 @@
+/*
+ * 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.arrow.vectorized;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import java.util.stream.StreamSupport;
+import org.apache.arrow.vector.NullCheckingForGet;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.arrow.vector.types.Types.MinorType;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.encryption.EncryptedFiles;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.CloseableIterator;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.types.Types;
+import org.apache.parquet.schema.MessageType;
+
+/**
+ * Vectorized reader that returns an iterator of {@link ColumnarBatch}.
+ * See {@link #open(CloseableIterable)} ()} to learn about the
+ * behavior of the iterator.
+ *
+ * <p>The following Iceberg data types are supported and have been tested:
+ * <ul>
+ *     <li>Iceberg: {@link Types.BooleanType}, Arrow: {@link MinorType#BIT}</li>
+ *     <li>Iceberg: {@link Types.IntegerType}, Arrow: {@link MinorType#INT}</li>
+ *     <li>Iceberg: {@link Types.LongType}, Arrow: {@link MinorType#BIGINT}</li>
+ *     <li>Iceberg: {@link Types.FloatType}, Arrow: {@link MinorType#FLOAT4}</li>
+ *     <li>Iceberg: {@link Types.DoubleType}, Arrow: {@link MinorType#FLOAT8}</li>
+ *     <li>Iceberg: {@link Types.StringType}, Arrow: {@link MinorType#VARCHAR}</li>
+ *     <li>Iceberg: {@link Types.TimestampType} (both with and without timezone),
+ *         Arrow: {@link MinorType#TIMEMICRO}</li>
+ *     <li>Iceberg: {@link Types.BinaryType}, Arrow: {@link MinorType#VARBINARY}</li>
+ *     <li>Iceberg: {@link Types.DateType}, Arrow: {@link MinorType#DATEDAY}</li>
+ * </ul>
+ *
+ * <p>Features that don't work in this implementation:
+ * <ul>
+ *     <li>Type promotion: In case of type promotion, the Arrow vector corresponding to
+ *     the data type in the parquet file is returned instead of the data type in the latest schema.
+ *     See https://github.com/apache/iceberg/issues/2483.</li>
+ *     <li>Columns with constant values are physically encoded as a dictionary. The Arrow vector
+ *     type is int32 instead of the type as per the schema.
+ *     See https://github.com/apache/iceberg/issues/2484.</li>
+ *     <li>Data types: {@link Types.TimeType}, {@link Types.ListType}, {@link Types.MapType},
+ *     {@link Types.StructType}, {@link Types.UUIDType}, {@link Types.FixedType} and
+ *     {@link Types.DecimalType}

Review comment:
       Spark supports decimal and fixed, right? Why not support them in this implementation?




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] mayursrivastava edited a comment on pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
mayursrivastava edited a comment on pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#issuecomment-824210180


   >./gradlew :iceberg-spark:jmh -PjmhIncludeRegex=VectorizedReadFlatParquetDataBenchmark -PjmhOutputPath=benchmark/results.txt
   
   @rymurr, is this :iceberg-spark2:jmh? Looks like this requires Java8 runtime. Am I right? I ran it on the master branch (without my changes), but it fails with the following error. Does it need a powerful machine?
   
   # JMH version: 1.21
   # VM version: JDK 1.8.0_282, OpenJDK 64-Bit Server VM, 25.282-b08
   # VM invoker: /usr/lib/jvm/java-8-openjdk-amd64/jre/bin/java
   # VM options: <none>
   # Warmup: 3 iterations, single-shot each
   # Measurement: 5 iterations, single-shot each
   # Timeout: 10 min per iteration
   # Threads: 1 thread
   # Benchmark mode: Single shot invocation time
   # Benchmark: org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadDictionaryEncodedFlatParquetDataBenchmark.readDatesIcebergVectorized5k
   
   # Run progress: 0.00% complete, ETA 00:00:00
   # Fork: 1 of 1
   # Warmup Iteration   1: (*interrupt*) <failure>
   
   java.lang.InterruptedException
           at java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireSharedInterruptibly(AbstractQueuedSynchronizer.java:998)
           at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireSharedInterruptibly(AbstractQueuedSynchronizer.java:1304)
           at scala.concurrent.impl.Promise$DefaultPromise.tryAwait(Promise.scala:206)
           at scala.concurrent.impl.Promise$DefaultPromise.ready(Promise.scala:222)
           at scala.concurrent.impl.Promise$DefaultPromise.ready(Promise.scala:157)
           at org.apache.spark.util.ThreadUtils$.awaitReady(ThreadUtils.scala:243)
           at org.apache.spark.scheduler.DAGScheduler.runJob(DAGScheduler.scala:750)
           at org.apache.spark.SparkContext.runJob(SparkContext.scala:2061)
           at org.apache.spark.sql.execution.datasources.v2.WriteToDataSourceV2Exec.doExecute(WriteToDataSourceV2Exec.scala:64)
           at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:131)
           at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:127)
           at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:155)
           at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
           at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:152)
           at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:127)
           at org.apache.spark.sql.execution.QueryExecution.toRdd$lzycompute(QueryExecution.scala:83)
           at org.apache.spark.sql.execution.QueryExecution.toRdd(QueryExecution.scala:81)
           at org.apache.spark.sql.DataFrameWriter$$anonfun$runCommand$1.apply(DataFrameWriter.scala:696)
           at org.apache.spark.sql.DataFrameWriter$$anonfun$runCommand$1.apply(DataFrameWriter.scala:696)
           at org.apache.spark.sql.execution.SQLExecution$$anonfun$withNewExecutionId$1.apply(SQLExecution.scala:80)
           at org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:127)
           at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:75)
           at org.apache.spark.sql.DataFrameWriter.runCommand(DataFrameWriter.scala:696)
           at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:280)
           at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:249)
           at org.apache.iceberg.spark.source.IcebergSourceBenchmark.appendAsFile(IcebergSourceBenchmark.java:130)
           at org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadDictionaryEncodedFlatParquetDataBenchmark.appendData(VectorizedReadDictionaryEncodedFlatParquetDataBenchmark.java:82)
           at org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadDictionaryEncodedFlatParquetDataBenchmark.setupBenchmark(VectorizedReadDictionaryEncodedFlatParquetDataBenchmark.java:56)
           at org.apache.iceberg.spark.source.parquet.vectorized.generated.VectorizedReadDictionaryEncodedFlatParquetDataBenchmark_readDatesIcebergVectorized5k_jmhTest._jmh_tryInit_f_vectorizedreaddictionaryencodedflatparquetdatabenchmark0_G(VectorizedReadDictionaryEncodedFlatParquetDataBenchmark_readDatesIcebergVectorized5k_jmhTest.java:438)
           at org.apache.iceberg.spark.source.parquet.vectorized.generated.VectorizedReadDictionaryEncodedFlatParquetDataBenchmark_readDatesIcebergVectorized5k_jmhTest.readDatesIcebergVectorized5k_SingleShotTime(VectorizedReadDictionaryEncodedFlatParquetDataBenchmark_readDatesIcebergVectorized5k_jmhTest.java:363)
           at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
           at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
           at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
           at java.lang.reflect.Method.invoke(Method.java:498)
           at org.openjdk.jmh.runner.BenchmarkHandler$BenchmarkTask.call(BenchmarkHandler.java:453)
           at org.openjdk.jmh.runner.BenchmarkHandler$BenchmarkTask.call(BenchmarkHandler.java:437)
           at java.util.concurrent.FutureTask.run(FutureTask.java:266)
           at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
           at java.util.concurrent.FutureTask.run(FutureTask.java:266)
           at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
           at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
           at java.lang.Thread.run(Thread.java:748)
   
   
   Benchmark had encountered error, and fail on error was requested
   


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] mayursrivastava commented on a change in pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
mayursrivastava commented on a change in pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#discussion_r614929142



##########
File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowReader.java
##########
@@ -0,0 +1,327 @@
+/*
+ * 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.arrow.vectorized;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import java.util.stream.StreamSupport;
+import org.apache.arrow.vector.NullCheckingForGet;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.arrow.vector.types.Types.MinorType;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.encryption.EncryptedFiles;
+import org.apache.iceberg.encryption.EncryptedInputFile;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.CloseableIterator;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.types.Types;
+import org.apache.parquet.schema.MessageType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Vectorized reader that returns an iterator of {@link ColumnarBatch}.
+ * See {@link #open(CloseableIterable)} ()} to learn about the
+ * behavior of the iterator.
+ *
+ * <p>The following Iceberg data types are supported and have been tested:
+ * <ul>
+ *     <li>Iceberg: {@link Types.BooleanType}, Arrow: {@link MinorType#BIT}</li>
+ *     <li>Iceberg: {@link Types.IntegerType}, Arrow: {@link MinorType#INT}</li>
+ *     <li>Iceberg: {@link Types.LongType}, Arrow: {@link MinorType#BIGINT}</li>
+ *     <li>Iceberg: {@link Types.FloatType}, Arrow: {@link MinorType#FLOAT4}</li>
+ *     <li>Iceberg: {@link Types.DoubleType}, Arrow: {@link MinorType#FLOAT8}</li>
+ *     <li>Iceberg: {@link Types.StringType}, Arrow: {@link MinorType#VARCHAR}</li>
+ *     <li>Iceberg: {@link Types.TimestampType} (both with and without timezone),
+ *         Arrow: {@link MinorType#TIMEMICRO}</li>
+ *     <li>Iceberg: {@link Types.BinaryType}, Arrow: {@link MinorType#VARBINARY}</li>
+ *     <li>Iceberg: {@link Types.DateType}, Arrow: {@link MinorType#DATEDAY}</li>
+ * </ul>
+ *
+ * <p>Features that don't work in this implementation:

Review comment:
       Created issues for each one of them. The javadoc is upto date, I'll add them to the javadoc.




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] mayursrivastava commented on a change in pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
mayursrivastava commented on a change in pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#discussion_r591984480



##########
File path: build.gradle
##########
@@ -239,6 +239,8 @@ project(':iceberg-data') {
     compile project(':iceberg-core')
     compileOnly project(':iceberg-parquet')
     compileOnly project(':iceberg-orc')
+    compileOnly project(':iceberg-arrow')

Review comment:
       I will move everything here to the arrow module

##########
File path: data/src/main/java/org/apache/iceberg/data/arrow/ArrowFileScanTaskReader.java
##########
@@ -0,0 +1,102 @@
+/*
+ * 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.data.arrow;
+
+import org.apache.arrow.vector.NullCheckingForGet;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.CloseableIterator;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Reads the data file and returns an iterator of {@link VectorSchemaRoot}. Only Parquet data file format is supported.
+ */
+public class ArrowFileScanTaskReader extends BaseArrowFileScanTaskReader<VectorSchemaRoot> {

Review comment:
       will do




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] mayursrivastava commented on a change in pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
mayursrivastava commented on a change in pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#discussion_r682565585



##########
File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowReader.java
##########
@@ -0,0 +1,325 @@
+/*
+ * 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.arrow.vectorized;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import java.util.stream.StreamSupport;
+import org.apache.arrow.vector.NullCheckingForGet;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.arrow.vector.types.Types.MinorType;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.encryption.EncryptedFiles;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.CloseableIterator;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.types.Types;
+import org.apache.parquet.schema.MessageType;
+
+/**
+ * Vectorized reader that returns an iterator of {@link ColumnarBatch}.
+ * See {@link #open(CloseableIterable)} ()} to learn about the
+ * behavior of the iterator.
+ *
+ * <p>The following Iceberg data types are supported and have been tested:
+ * <ul>
+ *     <li>Iceberg: {@link Types.BooleanType}, Arrow: {@link MinorType#BIT}</li>
+ *     <li>Iceberg: {@link Types.IntegerType}, Arrow: {@link MinorType#INT}</li>
+ *     <li>Iceberg: {@link Types.LongType}, Arrow: {@link MinorType#BIGINT}</li>
+ *     <li>Iceberg: {@link Types.FloatType}, Arrow: {@link MinorType#FLOAT4}</li>
+ *     <li>Iceberg: {@link Types.DoubleType}, Arrow: {@link MinorType#FLOAT8}</li>
+ *     <li>Iceberg: {@link Types.StringType}, Arrow: {@link MinorType#VARCHAR}</li>
+ *     <li>Iceberg: {@link Types.TimestampType} (both with and without timezone),
+ *         Arrow: {@link MinorType#TIMEMICRO}</li>
+ *     <li>Iceberg: {@link Types.BinaryType}, Arrow: {@link MinorType#VARBINARY}</li>
+ *     <li>Iceberg: {@link Types.DateType}, Arrow: {@link MinorType#DATEDAY}</li>
+ * </ul>
+ *
+ * <p>Features that don't work in this implementation:
+ * <ul>
+ *     <li>Type promotion: In case of type promotion, the Arrow vector corresponding to
+ *     the data type in the parquet file is returned instead of the data type in the latest schema.
+ *     See https://github.com/apache/iceberg/issues/2483.</li>
+ *     <li>Columns with constant values are physically encoded as a dictionary. The Arrow vector
+ *     type is int32 instead of the type as per the schema.
+ *     See https://github.com/apache/iceberg/issues/2484.</li>
+ *     <li>Data types: {@link Types.TimeType}, {@link Types.ListType}, {@link Types.MapType},
+ *     {@link Types.StructType}, {@link Types.UUIDType}, {@link Types.FixedType} and
+ *     {@link Types.DecimalType}
+ *     See https://github.com/apache/iceberg/issues/2485 and https://github.com/apache/iceberg/issues/2486.</li>
+ *     <li>Iceberg v2 spec is not supported.
+ *     See https://github.com/apache/iceberg/issues/2487.</li>

Review comment:
       The fixes are present in https://github.com/apache/iceberg/pull/2933.
   I've added checks similar to the Spark reader before allowing vectorized reader. 
   
   The arrow reader will check for the following conditions to be met:
   1. At least 1 column is queried.
   2. There are no delete files.
   3. Data types in columns are supported.
   
   I've also changed the reader to not process delete files because they don't work anyways with this reader. 
   

##########
File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowReader.java
##########
@@ -0,0 +1,325 @@
+/*
+ * 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.arrow.vectorized;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import java.util.stream.StreamSupport;
+import org.apache.arrow.vector.NullCheckingForGet;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.arrow.vector.types.Types.MinorType;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.encryption.EncryptedFiles;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.CloseableIterator;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.types.Types;
+import org.apache.parquet.schema.MessageType;
+
+/**
+ * Vectorized reader that returns an iterator of {@link ColumnarBatch}.
+ * See {@link #open(CloseableIterable)} ()} to learn about the
+ * behavior of the iterator.
+ *
+ * <p>The following Iceberg data types are supported and have been tested:
+ * <ul>
+ *     <li>Iceberg: {@link Types.BooleanType}, Arrow: {@link MinorType#BIT}</li>
+ *     <li>Iceberg: {@link Types.IntegerType}, Arrow: {@link MinorType#INT}</li>
+ *     <li>Iceberg: {@link Types.LongType}, Arrow: {@link MinorType#BIGINT}</li>
+ *     <li>Iceberg: {@link Types.FloatType}, Arrow: {@link MinorType#FLOAT4}</li>
+ *     <li>Iceberg: {@link Types.DoubleType}, Arrow: {@link MinorType#FLOAT8}</li>
+ *     <li>Iceberg: {@link Types.StringType}, Arrow: {@link MinorType#VARCHAR}</li>
+ *     <li>Iceberg: {@link Types.TimestampType} (both with and without timezone),
+ *         Arrow: {@link MinorType#TIMEMICRO}</li>
+ *     <li>Iceberg: {@link Types.BinaryType}, Arrow: {@link MinorType#VARBINARY}</li>
+ *     <li>Iceberg: {@link Types.DateType}, Arrow: {@link MinorType#DATEDAY}</li>
+ * </ul>
+ *
+ * <p>Features that don't work in this implementation:
+ * <ul>
+ *     <li>Type promotion: In case of type promotion, the Arrow vector corresponding to
+ *     the data type in the parquet file is returned instead of the data type in the latest schema.
+ *     See https://github.com/apache/iceberg/issues/2483.</li>
+ *     <li>Columns with constant values are physically encoded as a dictionary. The Arrow vector
+ *     type is int32 instead of the type as per the schema.
+ *     See https://github.com/apache/iceberg/issues/2484.</li>
+ *     <li>Data types: {@link Types.TimeType}, {@link Types.ListType}, {@link Types.MapType},
+ *     {@link Types.StructType}, {@link Types.UUIDType}, {@link Types.FixedType} and
+ *     {@link Types.DecimalType}

Review comment:
       yes, they will throw UnsupportedOperationException.
   I've added explicit exception in the reader code.
   The fixes are present in https://github.com/apache/iceberg/pull/2933.
   

##########
File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowReader.java
##########
@@ -0,0 +1,325 @@
+/*
+ * 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.arrow.vectorized;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import java.util.stream.StreamSupport;
+import org.apache.arrow.vector.NullCheckingForGet;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.arrow.vector.types.Types.MinorType;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.encryption.EncryptedFiles;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.CloseableIterator;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.types.Types;
+import org.apache.parquet.schema.MessageType;
+
+/**
+ * Vectorized reader that returns an iterator of {@link ColumnarBatch}.
+ * See {@link #open(CloseableIterable)} ()} to learn about the
+ * behavior of the iterator.
+ *
+ * <p>The following Iceberg data types are supported and have been tested:
+ * <ul>
+ *     <li>Iceberg: {@link Types.BooleanType}, Arrow: {@link MinorType#BIT}</li>
+ *     <li>Iceberg: {@link Types.IntegerType}, Arrow: {@link MinorType#INT}</li>
+ *     <li>Iceberg: {@link Types.LongType}, Arrow: {@link MinorType#BIGINT}</li>
+ *     <li>Iceberg: {@link Types.FloatType}, Arrow: {@link MinorType#FLOAT4}</li>
+ *     <li>Iceberg: {@link Types.DoubleType}, Arrow: {@link MinorType#FLOAT8}</li>
+ *     <li>Iceberg: {@link Types.StringType}, Arrow: {@link MinorType#VARCHAR}</li>
+ *     <li>Iceberg: {@link Types.TimestampType} (both with and without timezone),
+ *         Arrow: {@link MinorType#TIMEMICRO}</li>
+ *     <li>Iceberg: {@link Types.BinaryType}, Arrow: {@link MinorType#VARBINARY}</li>
+ *     <li>Iceberg: {@link Types.DateType}, Arrow: {@link MinorType#DATEDAY}</li>
+ * </ul>
+ *
+ * <p>Features that don't work in this implementation:
+ * <ul>
+ *     <li>Type promotion: In case of type promotion, the Arrow vector corresponding to
+ *     the data type in the parquet file is returned instead of the data type in the latest schema.
+ *     See https://github.com/apache/iceberg/issues/2483.</li>
+ *     <li>Columns with constant values are physically encoded as a dictionary. The Arrow vector
+ *     type is int32 instead of the type as per the schema.
+ *     See https://github.com/apache/iceberg/issues/2484.</li>
+ *     <li>Data types: {@link Types.TimeType}, {@link Types.ListType}, {@link Types.MapType},
+ *     {@link Types.StructType}, {@link Types.UUIDType}, {@link Types.FixedType} and
+ *     {@link Types.DecimalType}
+ *     See https://github.com/apache/iceberg/issues/2485 and https://github.com/apache/iceberg/issues/2486.</li>
+ *     <li>Iceberg v2 spec is not supported.
+ *     See https://github.com/apache/iceberg/issues/2487.</li>
+ * </ul>
+ */
+public class ArrowReader extends CloseableGroup {
+
+  private final Schema schema;
+  private final FileIO io;
+  private final EncryptionManager encryption;
+  private final int batchSize;
+  private final boolean reuseContainers;
+
+  /**
+   * Create a new instance of the reader.
+   *
+   * @param scan the table scan object.
+   * @param batchSize the maximum number of rows per Arrow batch.
+   * @param reuseContainers whether to reuse Arrow vectors when iterating through the data.
+   *                        If set to {@code false}, every {@link Iterator#next()} call creates
+   *                        new instances of Arrow vectors.
+   *                        If set to {@code true}, the Arrow vectors in the previous
+   *                        {@link Iterator#next()} may be reused for the data returned
+   *                        in the current {@link Iterator#next()}.
+   *                        This option avoids allocating memory again and again.
+   *                        Irrespective of the value of {@code reuseContainers}, the Arrow vectors
+   *                        in the previous {@link Iterator#next()} call are closed before creating
+   *                        new instances if the current {@link Iterator#next()}.
+   */
+  public ArrowReader(TableScan scan, int batchSize, boolean reuseContainers) {
+    this.schema = scan.schema();
+    this.io = scan.table().io();
+    this.encryption = scan.table().encryption();
+    this.batchSize = batchSize;
+    // start planning tasks in the background
+    this.reuseContainers = reuseContainers;
+  }
+
+  /**
+   * Returns a new iterator of {@link ColumnarBatch} objects.
+   * <p>
+   * Note that the reader owns the {@link ColumnarBatch} objects and takes care of closing them.
+   * The caller should not hold onto a {@link ColumnarBatch} or try to close them.
+   *
+   * <p>If {@code reuseContainers} is {@code false}, the Arrow vectors in the
+   * previous {@link ColumnarBatch} are closed before returning the next {@link ColumnarBatch} object.
+   * This implies that the caller should either use the {@link ColumnarBatch} or transfer the ownership of
+   * {@link ColumnarBatch} before getting the next {@link ColumnarBatch}.
+   *
+   * <p>If {@code reuseContainers} is {@code true}, the Arrow vectors in the
+   * previous {@link ColumnarBatch} may be reused for the next {@link ColumnarBatch}.
+   * This implies that the caller should either use the {@link ColumnarBatch} or deep copy the
+   * {@link ColumnarBatch} before getting the next {@link ColumnarBatch}.
+   */
+  public CloseableIterator<ColumnarBatch> open(CloseableIterable<CombinedScanTask> tasks) {
+    CloseableIterator<ColumnarBatch> itr = new VectorizedCombinedScanIterator(
+        tasks,
+        schema,
+        null,
+        io,
+        encryption,
+        true,
+        batchSize,
+        reuseContainers
+    );
+    addCloseable(itr);
+    return itr;
+  }
+
+  @Override
+  public void close() throws IOException {
+    super.close(); // close data files
+  }
+
+  /**
+   * Reads the data file and returns an iterator of {@link VectorSchemaRoot}.
+   * Only Parquet data file format is supported.
+   */
+  private static final class VectorizedCombinedScanIterator implements CloseableIterator<ColumnarBatch> {
+
+    private final List<FileScanTask> fileTasks;
+    private final Iterator<FileScanTask> fileItr;
+    private final Map<String, InputFile> inputFiles;
+    private final Schema expectedSchema;
+    private final String nameMapping;
+    private final boolean caseSensitive;
+    private final int batchSize;
+    private final boolean reuseContainers;
+    private CloseableIterator<ColumnarBatch> currentIterator;
+    private ColumnarBatch current;
+    private FileScanTask currentTask;
+
+    /**
+     * Create a new instance.
+     *
+     * @param tasks             Combined file scan tasks.
+     * @param expectedSchema    Read schema. The returned data will have this schema.
+     * @param nameMapping       Mapping from external schema names to Iceberg type IDs.
+     * @param io                File I/O.
+     * @param encryptionManager Encryption manager.
+     * @param caseSensitive     If {@code true}, column names are case sensitive.
+     *                          If {@code false}, column names are not case sensitive.
+     * @param batchSize         Batch size in number of rows. Each Arrow batch contains
+     *                          a maximum of {@code batchSize} rows.
+     * @param reuseContainers   If set to {@code false}, every {@link Iterator#next()} call creates
+     *                          new instances of Arrow vectors.
+     *                          If set to {@code true}, the Arrow vectors in the previous
+     *                          {@link Iterator#next()} may be reused for the data returned
+     *                          in the current {@link Iterator#next()}.
+     *                          This option avoids allocating memory again and again.
+     *                          Irrespective of the value of {@code reuseContainers}, the Arrow vectors
+     *                          in the previous {@link Iterator#next()} call are closed before creating
+     *                          new instances if the current {@link Iterator#next()}.
+     */
+    VectorizedCombinedScanIterator(
+        CloseableIterable<CombinedScanTask> tasks,
+        Schema expectedSchema,
+        String nameMapping,
+        FileIO io,
+        EncryptionManager encryptionManager,
+        boolean caseSensitive,
+        int batchSize,
+        boolean reuseContainers) {

Review comment:
       Fixed in https://github.com/apache/iceberg/pull/2933

##########
File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowReader.java
##########
@@ -0,0 +1,325 @@
+/*
+ * 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.arrow.vectorized;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import java.util.stream.StreamSupport;
+import org.apache.arrow.vector.NullCheckingForGet;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.arrow.vector.types.Types.MinorType;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.encryption.EncryptedFiles;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.CloseableIterator;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.types.Types;
+import org.apache.parquet.schema.MessageType;
+
+/**
+ * Vectorized reader that returns an iterator of {@link ColumnarBatch}.
+ * See {@link #open(CloseableIterable)} ()} to learn about the
+ * behavior of the iterator.
+ *
+ * <p>The following Iceberg data types are supported and have been tested:
+ * <ul>
+ *     <li>Iceberg: {@link Types.BooleanType}, Arrow: {@link MinorType#BIT}</li>
+ *     <li>Iceberg: {@link Types.IntegerType}, Arrow: {@link MinorType#INT}</li>
+ *     <li>Iceberg: {@link Types.LongType}, Arrow: {@link MinorType#BIGINT}</li>
+ *     <li>Iceberg: {@link Types.FloatType}, Arrow: {@link MinorType#FLOAT4}</li>
+ *     <li>Iceberg: {@link Types.DoubleType}, Arrow: {@link MinorType#FLOAT8}</li>
+ *     <li>Iceberg: {@link Types.StringType}, Arrow: {@link MinorType#VARCHAR}</li>
+ *     <li>Iceberg: {@link Types.TimestampType} (both with and without timezone),
+ *         Arrow: {@link MinorType#TIMEMICRO}</li>
+ *     <li>Iceberg: {@link Types.BinaryType}, Arrow: {@link MinorType#VARBINARY}</li>
+ *     <li>Iceberg: {@link Types.DateType}, Arrow: {@link MinorType#DATEDAY}</li>
+ * </ul>
+ *
+ * <p>Features that don't work in this implementation:
+ * <ul>
+ *     <li>Type promotion: In case of type promotion, the Arrow vector corresponding to
+ *     the data type in the parquet file is returned instead of the data type in the latest schema.
+ *     See https://github.com/apache/iceberg/issues/2483.</li>
+ *     <li>Columns with constant values are physically encoded as a dictionary. The Arrow vector
+ *     type is int32 instead of the type as per the schema.
+ *     See https://github.com/apache/iceberg/issues/2484.</li>
+ *     <li>Data types: {@link Types.TimeType}, {@link Types.ListType}, {@link Types.MapType},
+ *     {@link Types.StructType}, {@link Types.UUIDType}, {@link Types.FixedType} and
+ *     {@link Types.DecimalType}
+ *     See https://github.com/apache/iceberg/issues/2485 and https://github.com/apache/iceberg/issues/2486.</li>
+ *     <li>Iceberg v2 spec is not supported.
+ *     See https://github.com/apache/iceberg/issues/2487.</li>
+ * </ul>
+ */
+public class ArrowReader extends CloseableGroup {
+
+  private final Schema schema;
+  private final FileIO io;
+  private final EncryptionManager encryption;
+  private final int batchSize;
+  private final boolean reuseContainers;
+
+  /**
+   * Create a new instance of the reader.
+   *
+   * @param scan the table scan object.
+   * @param batchSize the maximum number of rows per Arrow batch.
+   * @param reuseContainers whether to reuse Arrow vectors when iterating through the data.
+   *                        If set to {@code false}, every {@link Iterator#next()} call creates
+   *                        new instances of Arrow vectors.
+   *                        If set to {@code true}, the Arrow vectors in the previous
+   *                        {@link Iterator#next()} may be reused for the data returned
+   *                        in the current {@link Iterator#next()}.
+   *                        This option avoids allocating memory again and again.
+   *                        Irrespective of the value of {@code reuseContainers}, the Arrow vectors
+   *                        in the previous {@link Iterator#next()} call are closed before creating
+   *                        new instances if the current {@link Iterator#next()}.
+   */
+  public ArrowReader(TableScan scan, int batchSize, boolean reuseContainers) {
+    this.schema = scan.schema();
+    this.io = scan.table().io();
+    this.encryption = scan.table().encryption();
+    this.batchSize = batchSize;
+    // start planning tasks in the background
+    this.reuseContainers = reuseContainers;
+  }
+
+  /**
+   * Returns a new iterator of {@link ColumnarBatch} objects.
+   * <p>
+   * Note that the reader owns the {@link ColumnarBatch} objects and takes care of closing them.
+   * The caller should not hold onto a {@link ColumnarBatch} or try to close them.
+   *
+   * <p>If {@code reuseContainers} is {@code false}, the Arrow vectors in the
+   * previous {@link ColumnarBatch} are closed before returning the next {@link ColumnarBatch} object.
+   * This implies that the caller should either use the {@link ColumnarBatch} or transfer the ownership of
+   * {@link ColumnarBatch} before getting the next {@link ColumnarBatch}.
+   *
+   * <p>If {@code reuseContainers} is {@code true}, the Arrow vectors in the
+   * previous {@link ColumnarBatch} may be reused for the next {@link ColumnarBatch}.
+   * This implies that the caller should either use the {@link ColumnarBatch} or deep copy the
+   * {@link ColumnarBatch} before getting the next {@link ColumnarBatch}.
+   */
+  public CloseableIterator<ColumnarBatch> open(CloseableIterable<CombinedScanTask> tasks) {
+    CloseableIterator<ColumnarBatch> itr = new VectorizedCombinedScanIterator(
+        tasks,
+        schema,
+        null,
+        io,
+        encryption,
+        true,
+        batchSize,
+        reuseContainers
+    );
+    addCloseable(itr);
+    return itr;
+  }
+
+  @Override
+  public void close() throws IOException {
+    super.close(); // close data files
+  }
+
+  /**
+   * Reads the data file and returns an iterator of {@link VectorSchemaRoot}.
+   * Only Parquet data file format is supported.
+   */
+  private static final class VectorizedCombinedScanIterator implements CloseableIterator<ColumnarBatch> {
+
+    private final List<FileScanTask> fileTasks;
+    private final Iterator<FileScanTask> fileItr;
+    private final Map<String, InputFile> inputFiles;
+    private final Schema expectedSchema;
+    private final String nameMapping;
+    private final boolean caseSensitive;
+    private final int batchSize;
+    private final boolean reuseContainers;
+    private CloseableIterator<ColumnarBatch> currentIterator;
+    private ColumnarBatch current;
+    private FileScanTask currentTask;
+
+    /**
+     * Create a new instance.
+     *
+     * @param tasks             Combined file scan tasks.
+     * @param expectedSchema    Read schema. The returned data will have this schema.
+     * @param nameMapping       Mapping from external schema names to Iceberg type IDs.
+     * @param io                File I/O.
+     * @param encryptionManager Encryption manager.
+     * @param caseSensitive     If {@code true}, column names are case sensitive.
+     *                          If {@code false}, column names are not case sensitive.
+     * @param batchSize         Batch size in number of rows. Each Arrow batch contains
+     *                          a maximum of {@code batchSize} rows.
+     * @param reuseContainers   If set to {@code false}, every {@link Iterator#next()} call creates
+     *                          new instances of Arrow vectors.
+     *                          If set to {@code true}, the Arrow vectors in the previous
+     *                          {@link Iterator#next()} may be reused for the data returned
+     *                          in the current {@link Iterator#next()}.
+     *                          This option avoids allocating memory again and again.
+     *                          Irrespective of the value of {@code reuseContainers}, the Arrow vectors
+     *                          in the previous {@link Iterator#next()} call are closed before creating
+     *                          new instances if the current {@link Iterator#next()}.
+     */
+    VectorizedCombinedScanIterator(
+        CloseableIterable<CombinedScanTask> tasks,
+        Schema expectedSchema,
+        String nameMapping,
+        FileIO io,
+        EncryptionManager encryptionManager,
+        boolean caseSensitive,
+        int batchSize,
+        boolean reuseContainers) {
+      this.fileTasks = StreamSupport.stream(tasks.spliterator(), false)
+          .map(CombinedScanTask::files)
+          .flatMap(Collection::stream)
+          .collect(Collectors.toList());
+      this.fileItr = fileTasks.iterator();
+      this.inputFiles = Collections.unmodifiableMap(fileTasks.stream()
+          .flatMap(fileScanTask -> Stream.concat(Stream.of(fileScanTask.file()), fileScanTask.deletes().stream()))
+          .map(file -> EncryptedFiles.encryptedInput(io.newInputFile(file.path().toString()), file.keyMetadata()))
+          .map(encryptionManager::decrypt)
+          .collect(Collectors.toMap(InputFile::location, Function.identity())));
+      this.currentIterator = CloseableIterator.empty();
+      this.expectedSchema = expectedSchema;
+      this.nameMapping = nameMapping;
+      this.caseSensitive = caseSensitive;
+      this.batchSize = batchSize;
+      this.reuseContainers = reuseContainers;
+    }
+
+    @Override
+    public boolean hasNext() {
+      try {
+        while (true) {
+          if (currentIterator.hasNext()) {
+            this.current = currentIterator.next();
+            return true;
+          } else if (fileItr.hasNext()) {
+            this.currentIterator.close();
+            this.currentTask = fileItr.next();
+            this.currentIterator = open(currentTask);
+          } else {
+            this.currentIterator.close();
+            return false;
+          }
+        }
+      } catch (IOException | RuntimeException e) {
+        if (currentTask != null && !currentTask.isDataTask()) {
+          throw new RuntimeException(
+              "Error reading file: " + getInputFile(currentTask).location() +
+                  ". Reason: the current task is not a data task, i.e. it cannot read data rows. " +

Review comment:
       Falling back to the same logic as in the BaseDataReader.
   But, the Iterator doesn't support throwing checked exceptions, so I still have to wrap the exception as a RuntimeException.
   Fixes present in https://github.com/apache/iceberg/pull/2933.
   

##########
File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowReader.java
##########
@@ -0,0 +1,325 @@
+/*
+ * 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.arrow.vectorized;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import java.util.stream.StreamSupport;
+import org.apache.arrow.vector.NullCheckingForGet;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.arrow.vector.types.Types.MinorType;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.encryption.EncryptedFiles;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.CloseableIterator;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.types.Types;
+import org.apache.parquet.schema.MessageType;
+
+/**
+ * Vectorized reader that returns an iterator of {@link ColumnarBatch}.
+ * See {@link #open(CloseableIterable)} ()} to learn about the
+ * behavior of the iterator.
+ *
+ * <p>The following Iceberg data types are supported and have been tested:
+ * <ul>
+ *     <li>Iceberg: {@link Types.BooleanType}, Arrow: {@link MinorType#BIT}</li>
+ *     <li>Iceberg: {@link Types.IntegerType}, Arrow: {@link MinorType#INT}</li>
+ *     <li>Iceberg: {@link Types.LongType}, Arrow: {@link MinorType#BIGINT}</li>
+ *     <li>Iceberg: {@link Types.FloatType}, Arrow: {@link MinorType#FLOAT4}</li>
+ *     <li>Iceberg: {@link Types.DoubleType}, Arrow: {@link MinorType#FLOAT8}</li>
+ *     <li>Iceberg: {@link Types.StringType}, Arrow: {@link MinorType#VARCHAR}</li>
+ *     <li>Iceberg: {@link Types.TimestampType} (both with and without timezone),
+ *         Arrow: {@link MinorType#TIMEMICRO}</li>
+ *     <li>Iceberg: {@link Types.BinaryType}, Arrow: {@link MinorType#VARBINARY}</li>
+ *     <li>Iceberg: {@link Types.DateType}, Arrow: {@link MinorType#DATEDAY}</li>
+ * </ul>
+ *
+ * <p>Features that don't work in this implementation:
+ * <ul>
+ *     <li>Type promotion: In case of type promotion, the Arrow vector corresponding to
+ *     the data type in the parquet file is returned instead of the data type in the latest schema.
+ *     See https://github.com/apache/iceberg/issues/2483.</li>
+ *     <li>Columns with constant values are physically encoded as a dictionary. The Arrow vector
+ *     type is int32 instead of the type as per the schema.
+ *     See https://github.com/apache/iceberg/issues/2484.</li>
+ *     <li>Data types: {@link Types.TimeType}, {@link Types.ListType}, {@link Types.MapType},
+ *     {@link Types.StructType}, {@link Types.UUIDType}, {@link Types.FixedType} and
+ *     {@link Types.DecimalType}
+ *     See https://github.com/apache/iceberg/issues/2485 and https://github.com/apache/iceberg/issues/2486.</li>
+ *     <li>Iceberg v2 spec is not supported.
+ *     See https://github.com/apache/iceberg/issues/2487.</li>
+ * </ul>
+ */
+public class ArrowReader extends CloseableGroup {
+
+  private final Schema schema;
+  private final FileIO io;
+  private final EncryptionManager encryption;
+  private final int batchSize;
+  private final boolean reuseContainers;
+
+  /**
+   * Create a new instance of the reader.
+   *
+   * @param scan the table scan object.
+   * @param batchSize the maximum number of rows per Arrow batch.
+   * @param reuseContainers whether to reuse Arrow vectors when iterating through the data.
+   *                        If set to {@code false}, every {@link Iterator#next()} call creates
+   *                        new instances of Arrow vectors.
+   *                        If set to {@code true}, the Arrow vectors in the previous
+   *                        {@link Iterator#next()} may be reused for the data returned
+   *                        in the current {@link Iterator#next()}.
+   *                        This option avoids allocating memory again and again.
+   *                        Irrespective of the value of {@code reuseContainers}, the Arrow vectors
+   *                        in the previous {@link Iterator#next()} call are closed before creating
+   *                        new instances if the current {@link Iterator#next()}.
+   */
+  public ArrowReader(TableScan scan, int batchSize, boolean reuseContainers) {
+    this.schema = scan.schema();
+    this.io = scan.table().io();
+    this.encryption = scan.table().encryption();
+    this.batchSize = batchSize;
+    // start planning tasks in the background
+    this.reuseContainers = reuseContainers;
+  }
+
+  /**
+   * Returns a new iterator of {@link ColumnarBatch} objects.
+   * <p>
+   * Note that the reader owns the {@link ColumnarBatch} objects and takes care of closing them.
+   * The caller should not hold onto a {@link ColumnarBatch} or try to close them.
+   *
+   * <p>If {@code reuseContainers} is {@code false}, the Arrow vectors in the
+   * previous {@link ColumnarBatch} are closed before returning the next {@link ColumnarBatch} object.
+   * This implies that the caller should either use the {@link ColumnarBatch} or transfer the ownership of
+   * {@link ColumnarBatch} before getting the next {@link ColumnarBatch}.
+   *
+   * <p>If {@code reuseContainers} is {@code true}, the Arrow vectors in the
+   * previous {@link ColumnarBatch} may be reused for the next {@link ColumnarBatch}.
+   * This implies that the caller should either use the {@link ColumnarBatch} or deep copy the
+   * {@link ColumnarBatch} before getting the next {@link ColumnarBatch}.
+   */
+  public CloseableIterator<ColumnarBatch> open(CloseableIterable<CombinedScanTask> tasks) {
+    CloseableIterator<ColumnarBatch> itr = new VectorizedCombinedScanIterator(
+        tasks,
+        schema,
+        null,
+        io,
+        encryption,
+        true,
+        batchSize,
+        reuseContainers
+    );
+    addCloseable(itr);
+    return itr;
+  }
+
+  @Override
+  public void close() throws IOException {
+    super.close(); // close data files
+  }
+
+  /**
+   * Reads the data file and returns an iterator of {@link VectorSchemaRoot}.
+   * Only Parquet data file format is supported.
+   */
+  private static final class VectorizedCombinedScanIterator implements CloseableIterator<ColumnarBatch> {
+
+    private final List<FileScanTask> fileTasks;
+    private final Iterator<FileScanTask> fileItr;
+    private final Map<String, InputFile> inputFiles;
+    private final Schema expectedSchema;
+    private final String nameMapping;
+    private final boolean caseSensitive;
+    private final int batchSize;
+    private final boolean reuseContainers;
+    private CloseableIterator<ColumnarBatch> currentIterator;
+    private ColumnarBatch current;
+    private FileScanTask currentTask;
+
+    /**
+     * Create a new instance.
+     *
+     * @param tasks             Combined file scan tasks.
+     * @param expectedSchema    Read schema. The returned data will have this schema.
+     * @param nameMapping       Mapping from external schema names to Iceberg type IDs.
+     * @param io                File I/O.
+     * @param encryptionManager Encryption manager.
+     * @param caseSensitive     If {@code true}, column names are case sensitive.
+     *                          If {@code false}, column names are not case sensitive.
+     * @param batchSize         Batch size in number of rows. Each Arrow batch contains
+     *                          a maximum of {@code batchSize} rows.
+     * @param reuseContainers   If set to {@code false}, every {@link Iterator#next()} call creates
+     *                          new instances of Arrow vectors.
+     *                          If set to {@code true}, the Arrow vectors in the previous
+     *                          {@link Iterator#next()} may be reused for the data returned
+     *                          in the current {@link Iterator#next()}.
+     *                          This option avoids allocating memory again and again.
+     *                          Irrespective of the value of {@code reuseContainers}, the Arrow vectors
+     *                          in the previous {@link Iterator#next()} call are closed before creating
+     *                          new instances if the current {@link Iterator#next()}.
+     */
+    VectorizedCombinedScanIterator(
+        CloseableIterable<CombinedScanTask> tasks,
+        Schema expectedSchema,
+        String nameMapping,
+        FileIO io,
+        EncryptionManager encryptionManager,
+        boolean caseSensitive,
+        int batchSize,
+        boolean reuseContainers) {
+      this.fileTasks = StreamSupport.stream(tasks.spliterator(), false)
+          .map(CombinedScanTask::files)
+          .flatMap(Collection::stream)
+          .collect(Collectors.toList());
+      this.fileItr = fileTasks.iterator();
+      this.inputFiles = Collections.unmodifiableMap(fileTasks.stream()
+          .flatMap(fileScanTask -> Stream.concat(Stream.of(fileScanTask.file()), fileScanTask.deletes().stream()))
+          .map(file -> EncryptedFiles.encryptedInput(io.newInputFile(file.path().toString()), file.keyMetadata()))
+          .map(encryptionManager::decrypt)

Review comment:
       This was already fixed in #2720

##########
File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ColumnVector.java
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.arrow.vectorized;
+
+import org.apache.arrow.vector.FieldVector;
+import org.apache.iceberg.types.Types;
+
+/**
+ * This class is inspired by Spark's {@code ColumnVector}.
+ * This class represents the column data for an Iceberg table query.
+ * It wraps an arrow {@link FieldVector} and provides simple
+ * accessors for the row values. Advanced users can access
+ * the {@link FieldVector}.
+ * <p>
+ *   Supported Iceberg data types:
+ *   <ul>
+ *     <li>{@link Types.BooleanType}</li>
+ *     <li>{@link Types.IntegerType}</li>
+ *     <li>{@link Types.LongType}</li>
+ *     <li>{@link Types.FloatType}</li>
+ *     <li>{@link Types.DoubleType}</li>
+ *     <li>{@link Types.StringType}</li>
+ *     <li>{@link Types.BinaryType}</li>
+ *     <li>{@link Types.TimestampType} (with and without timezone)</li>
+ *     <li>{@link Types.DateType}</li>
+ *   </ul>
+ */
+public class ColumnVector implements AutoCloseable {

Review comment:
       My use case is to use Arrow VectorSchemaRoot directly, but I agree with @rymurr's suggestions to wrap the arrow data structures. 
   
   I see the following benefits with the wrapper interface:
   1. Lifecycle management is better.
   2. The current parquet reader returns physical repr of the data as arrow vectors. This means that the dictionary encoded columns are returned as int32 and columns that were widened return the physical file column width (e.g. if int32 was widened to int64 and the data contains int32, the arrow vector is int32). The wrapper classes can handle dictionary encoding and type widening correctly. Note that this is also done in the Spark version. This, however, is a limitation of the current implementation and it will be better for the Arrow Reader to return arrow vectors with logical types.
   3. The wrapper interface is easier to use for most users.
   
   The cons are following:
   1. A new API is introduced that has to be learned and maintained.
   
   I also don't feel strong about the wrapper interface and if the community doesn't find it useful abstraction, I agree.




-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#discussion_r638342133



##########
File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowReader.java
##########
@@ -0,0 +1,325 @@
+/*
+ * 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.arrow.vectorized;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import java.util.stream.StreamSupport;
+import org.apache.arrow.vector.NullCheckingForGet;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.arrow.vector.types.Types.MinorType;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.encryption.EncryptedFiles;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.CloseableIterator;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.types.Types;
+import org.apache.parquet.schema.MessageType;
+
+/**
+ * Vectorized reader that returns an iterator of {@link ColumnarBatch}.
+ * See {@link #open(CloseableIterable)} ()} to learn about the
+ * behavior of the iterator.
+ *
+ * <p>The following Iceberg data types are supported and have been tested:
+ * <ul>
+ *     <li>Iceberg: {@link Types.BooleanType}, Arrow: {@link MinorType#BIT}</li>
+ *     <li>Iceberg: {@link Types.IntegerType}, Arrow: {@link MinorType#INT}</li>
+ *     <li>Iceberg: {@link Types.LongType}, Arrow: {@link MinorType#BIGINT}</li>
+ *     <li>Iceberg: {@link Types.FloatType}, Arrow: {@link MinorType#FLOAT4}</li>
+ *     <li>Iceberg: {@link Types.DoubleType}, Arrow: {@link MinorType#FLOAT8}</li>
+ *     <li>Iceberg: {@link Types.StringType}, Arrow: {@link MinorType#VARCHAR}</li>
+ *     <li>Iceberg: {@link Types.TimestampType} (both with and without timezone),
+ *         Arrow: {@link MinorType#TIMEMICRO}</li>
+ *     <li>Iceberg: {@link Types.BinaryType}, Arrow: {@link MinorType#VARBINARY}</li>
+ *     <li>Iceberg: {@link Types.DateType}, Arrow: {@link MinorType#DATEDAY}</li>
+ * </ul>
+ *
+ * <p>Features that don't work in this implementation:
+ * <ul>
+ *     <li>Type promotion: In case of type promotion, the Arrow vector corresponding to
+ *     the data type in the parquet file is returned instead of the data type in the latest schema.
+ *     See https://github.com/apache/iceberg/issues/2483.</li>
+ *     <li>Columns with constant values are physically encoded as a dictionary. The Arrow vector
+ *     type is int32 instead of the type as per the schema.
+ *     See https://github.com/apache/iceberg/issues/2484.</li>
+ *     <li>Data types: {@link Types.TimeType}, {@link Types.ListType}, {@link Types.MapType},
+ *     {@link Types.StructType}, {@link Types.UUIDType}, {@link Types.FixedType} and
+ *     {@link Types.DecimalType}

Review comment:
       What happens if any of these types are used? An `UnsupportedOperationException`?




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] mayursrivastava commented on a change in pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
mayursrivastava commented on a change in pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#discussion_r618416064



##########
File path: build.gradle
##########
@@ -735,9 +735,12 @@ project(':iceberg-arrow') {
       exclude group: 'com.google.code.findbugs', module: 'jsr305'
     }
     compile("org.apache.arrow:arrow-memory-netty") {
-      exclude group: 'io.netty', module: 'netty-common'
       exclude group: 'com.google.code.findbugs', module: 'jsr305'

Review comment:
       That works but the version needs to be added in the versions.props. If this is fine, I'll make the change.
   
   > 
   $ git diff
   diff --git a/build.gradle b/build.gradle
   index 75c69935..14f9bfae 100644
   --- a/build.gradle
   +++ b/build.gradle
   @@ -741,7 +741,7 @@ project(':iceberg-arrow') {
    
        testCompile project(path: ':iceberg-core', configuration: 'testArtifacts')
        // netty-common is needed to run ArrowReaderTest test cases.
   -    testCompile("org.apache.arrow:arrow-memory-netty")
   +    testCompile("io.netty:netty-common")
        testCompile("org.apache.hadoop:hadoop-common")
        testCompile("org.apache.hadoop:hadoop-mapreduce-client-core")
      }
   diff --git a/versions.props b/versions.props
   index dc4caddd..a147f50c 100644
   --- a/versions.props
   +++ b/versions.props
   @@ -31,3 +31,4 @@ org.apache.hive:hive-service = 2.3.8
    org.apache.tez:tez-dag = 0.8.4
    org.apache.tez:tez-mapreduce = 0.8.4
    com.adobe.testing:s3mock-junit4 = 2.1.28
   +io.netty:netty-common = 4.1.48.Final
   > 
   
   




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] mayursrivastava commented on a change in pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
mayursrivastava commented on a change in pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#discussion_r682575840



##########
File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ColumnVector.java
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.arrow.vectorized;
+
+import org.apache.arrow.vector.FieldVector;
+import org.apache.iceberg.types.Types;
+
+/**
+ * This class is inspired by Spark's {@code ColumnVector}.
+ * This class represents the column data for an Iceberg table query.
+ * It wraps an arrow {@link FieldVector} and provides simple
+ * accessors for the row values. Advanced users can access
+ * the {@link FieldVector}.
+ * <p>
+ *   Supported Iceberg data types:
+ *   <ul>
+ *     <li>{@link Types.BooleanType}</li>
+ *     <li>{@link Types.IntegerType}</li>
+ *     <li>{@link Types.LongType}</li>
+ *     <li>{@link Types.FloatType}</li>
+ *     <li>{@link Types.DoubleType}</li>
+ *     <li>{@link Types.StringType}</li>
+ *     <li>{@link Types.BinaryType}</li>
+ *     <li>{@link Types.TimestampType} (with and without timezone)</li>
+ *     <li>{@link Types.DateType}</li>
+ *   </ul>
+ */
+public class ColumnVector implements AutoCloseable {

Review comment:
       My use case is to use Arrow VectorSchemaRoot directly, but I agree with @rymurr's suggestions to wrap the arrow data structures. 
   
   I see the following benefits with the wrapper interface:
   1. Lifecycle management is better.
   2. The current parquet reader returns physical repr of the data as arrow vectors. This means that the dictionary encoded columns are returned as int32 and columns that were widened return the physical file column width (e.g. if int32 was widened to int64 and the data contains int32, the arrow vector is int32). The wrapper classes can handle dictionary encoding and type widening correctly. Note that this is also done in the Spark version. This, however, is a limitation of the current implementation and it will be better for the Arrow Reader to return arrow vectors with logical types.
   3. The wrapper interface is easier to use for most users.
   
   The cons are following:
   1. A new API is introduced that has to be learned and maintained.
   
   I also don't feel strong about the wrapper interface and if the community doesn't find it useful abstraction, I agree.




-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] mayursrivastava commented on a change in pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
mayursrivastava commented on a change in pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#discussion_r591984804



##########
File path: data/src/main/java/org/apache/iceberg/data/arrow/VectorSchemaRootReader.java
##########
@@ -0,0 +1,112 @@
+/*
+ * 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.data.arrow;
+
+import java.util.List;
+import java.util.Map;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.iceberg.arrow.vectorized.VectorHolder;
+import org.apache.iceberg.arrow.vectorized.VectorizedArrowReader;
+import org.apache.iceberg.parquet.VectorizedReader;
+import org.apache.parquet.Preconditions;
+import org.apache.parquet.column.page.PageReadStore;
+import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
+import org.apache.parquet.hadoop.metadata.ColumnPath;
+
+/**
+ * A collection of vectorized readers per column (in the expected read schema) and Arrow Vector holders. This class owns
+ * the Arrow vectors and is responsible for closing the Arrow vectors.
+ */
+public class VectorSchemaRootReader implements VectorizedReader<VectorSchemaRoot> {

Review comment:
       sure, will do




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] mayursrivastava commented on a change in pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
mayursrivastava commented on a change in pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#discussion_r609628264



##########
File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/VectorizedParquetReaders.java
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.arrow.vectorized;
+
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.parquet.schema.MessageType;
+
+/**
+ * Builds an {@link ArrowBatchReader}.
+ */
+class VectorizedParquetReaders {
+
+  private VectorizedParquetReaders() {
+  }
+
+  /**
+   * Build the {@link ArrowBatchReader} for the expected schema and file schema.
+   *
+   * @param expectedSchema         Expected schema of the data returned.
+   * @param fileSchema             Schema of the data file.
+   * @param setArrowValidityVector Indicates whether to set the validity vector in Arrow vectors.
+   */
+  public static ArrowBatchReader buildReader(

Review comment:
       sure




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] mayursrivastava commented on a change in pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
mayursrivastava commented on a change in pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#discussion_r618443600



##########
File path: build.gradle
##########
@@ -735,9 +735,12 @@ project(':iceberg-arrow') {
       exclude group: 'com.google.code.findbugs', module: 'jsr305'
     }
     compile("org.apache.arrow:arrow-memory-netty") {
-      exclude group: 'io.netty', module: 'netty-common'
       exclude group: 'com.google.code.findbugs', module: 'jsr305'

Review comment:
       Sure. Updated




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] mayursrivastava commented on a change in pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
mayursrivastava commented on a change in pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#discussion_r638325608



##########
File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowReader.java
##########
@@ -0,0 +1,325 @@
+/*
+ * 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.arrow.vectorized;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import java.util.stream.StreamSupport;
+import org.apache.arrow.vector.NullCheckingForGet;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.arrow.vector.types.Types.MinorType;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.encryption.EncryptedFiles;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.CloseableIterator;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.types.Types;
+import org.apache.parquet.schema.MessageType;
+
+/**
+ * Vectorized reader that returns an iterator of {@link ColumnarBatch}.
+ * See {@link #open(CloseableIterable)} ()} to learn about the
+ * behavior of the iterator.
+ *
+ * <p>The following Iceberg data types are supported and have been tested:
+ * <ul>
+ *     <li>Iceberg: {@link Types.BooleanType}, Arrow: {@link MinorType#BIT}</li>
+ *     <li>Iceberg: {@link Types.IntegerType}, Arrow: {@link MinorType#INT}</li>
+ *     <li>Iceberg: {@link Types.LongType}, Arrow: {@link MinorType#BIGINT}</li>
+ *     <li>Iceberg: {@link Types.FloatType}, Arrow: {@link MinorType#FLOAT4}</li>
+ *     <li>Iceberg: {@link Types.DoubleType}, Arrow: {@link MinorType#FLOAT8}</li>
+ *     <li>Iceberg: {@link Types.StringType}, Arrow: {@link MinorType#VARCHAR}</li>
+ *     <li>Iceberg: {@link Types.TimestampType} (both with and without timezone),
+ *         Arrow: {@link MinorType#TIMEMICRO}</li>
+ *     <li>Iceberg: {@link Types.BinaryType}, Arrow: {@link MinorType#VARBINARY}</li>
+ *     <li>Iceberg: {@link Types.DateType}, Arrow: {@link MinorType#DATEDAY}</li>
+ * </ul>
+ *
+ * <p>Features that don't work in this implementation:
+ * <ul>
+ *     <li>Type promotion: In case of type promotion, the Arrow vector corresponding to
+ *     the data type in the parquet file is returned instead of the data type in the latest schema.
+ *     See https://github.com/apache/iceberg/issues/2483.</li>
+ *     <li>Columns with constant values are physically encoded as a dictionary. The Arrow vector
+ *     type is int32 instead of the type as per the schema.
+ *     See https://github.com/apache/iceberg/issues/2484.</li>
+ *     <li>Data types: {@link Types.TimeType}, {@link Types.ListType}, {@link Types.MapType},
+ *     {@link Types.StructType}, {@link Types.UUIDType}, {@link Types.FixedType} and
+ *     {@link Types.DecimalType}

Review comment:
       Spark supports these types. I was not able to write test for the decimal and fixed types; this is the reason why they are not in this pr. I created this issue https://github.com/apache/iceberg/issues/2486 to take care of this. Does this sound good to you?




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr edited a comment on pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
rymurr edited a comment on pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#issuecomment-828386504


   Thanks @mayursrivastava these look semsible. I am trying to compare locally w/ the master branch to ensure there is no regression. 


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] mayursrivastava commented on pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
mayursrivastava commented on pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#issuecomment-790149134


   Thanks for looking into it @rymurr 
   
   I'm looking to integrate this with our existing Apache Arrow/Flight service and some internal services which use VectorSchemaRoot and Arrow Field Vectors directly. Using ColumnBatch or a similar class will require us to somehow get access to the internal Arrow data structures (VectorSchemaRoot or Arrow Field Vector). On the lifecycle question I agree, Arrow Field Vectors or VectorSchemaRoot have a lifecycle policy (which is currently being handled by the reader) and need some careful handling by the user. So, yes, this is for advanced users. 
   
   If we want to provide more user friendly accessors in addition to the VectorSchemaRoot or Field Vectors, we can try to move the accessor classes from Spark to the iceberg-arrow module, but we will still need to provide access to Arrow internal data structures so that they can be used by advanced users.
   


----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] mayursrivastava commented on pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
mayursrivastava commented on pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#issuecomment-792735826


   Hi @rymurr
   
   Let me know if you have any questions on the pr, I'll be happy to answer them.
   
   Thanks,
   Mayur
   


----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] mayursrivastava commented on pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
mayursrivastava commented on pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#issuecomment-804384770


   Thanks @rymurr. No worries. Have a good one.


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#discussion_r604137792



##########
File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/VectorizedParquetReaders.java
##########
@@ -0,0 +1,51 @@
+/*
+ * 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.arrow.vectorized;
+
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.parquet.schema.MessageType;
+
+/**
+ * Builds an {@link ArrowBatchReader}.
+ */
+class VectorizedParquetReaders {
+
+  private VectorizedParquetReaders() {
+  }
+
+  /**
+   * Build the {@link ArrowBatchReader} for the expected schema and file schema.
+   *
+   * @param expectedSchema         Expected schema of the data returned.
+   * @param fileSchema             Schema of the data file.
+   * @param setArrowValidityVector Indicates whether to set the validity vector in Arrow vectors.
+   */
+  public static ArrowBatchReader buildReader(

Review comment:
       could this live in the iterator? I am not sure it needs its own static class for this.

##########
File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowBatch.java
##########
@@ -0,0 +1,82 @@
+/*
+ * 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.arrow.vectorized;
+
+import java.util.Arrays;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.VectorSchemaRoot;
+
+/**
+ * This class is inspired by Spark's {@code ColumnarBatch}.
+ * This class wraps a columnar batch in the result set of an Iceberg table query.
+ */
+public class ArrowBatch implements AutoCloseable {

Review comment:
       nit: could we stick w/ `ColumnBatch` here and elsewhere. Just to hide from non-experts the Arrow-ness of the vectorized reader

##########
File path: arrow/src/test/java/org/apache/iceberg/arrow/vectorized/ArrowReaderTest.java
##########
@@ -0,0 +1,734 @@
+/*
+ * 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.arrow.vectorized;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.time.temporal.ChronoUnit;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+import org.apache.arrow.vector.BigIntVector;
+import org.apache.arrow.vector.BitVector;
+import org.apache.arrow.vector.DateDayVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.Float4Vector;
+import org.apache.arrow.vector.Float8Vector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.TimeStampMicroTZVector;
+import org.apache.arrow.vector.TimeStampMicroVector;
+import org.apache.arrow.vector.VarBinaryVector;
+import org.apache.arrow.vector.VarCharVector;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.arrow.vector.types.Types.MinorType;
+import org.apache.arrow.vector.types.pojo.ArrowType;
+import org.apache.arrow.vector.types.pojo.Field;
+import org.apache.arrow.vector.types.pojo.FieldType;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.OverwriteFiles;
+import org.apache.iceberg.PartitionKey;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.data.parquet.GenericParquetWriter;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.types.Types;
+import org.junit.Ignore;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.Files.localInput;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Test cases for {@link ArrowReader}.
+ * <p>All tests create a table with monthly partitions and write 1 year of data to the table.
+ */
+public class ArrowReaderTest {
+
+  private static final int NUM_ROWS_PER_MONTH = 20;
+  private static final ImmutableList<String> ALL_COLUMNS =
+      ImmutableList.of(
+          "timestamp",
+          "timestamp_nullable",
+          "boolean",
+          "boolean_nullable",
+          "int",
+          "int_nullable",
+          "long",
+          "long_nullable",
+          "float",
+          "float_nullable",
+          "double",
+          "double_nullable",
+          "timestamp_tz",
+          "timestamp_tz_nullable",
+          "string",
+          "string_nullable",
+          "bytes",
+          "bytes_nullable",
+          "date",
+          "date_nullable",
+          "int_promotion"
+      );
+
+  @Rule
+  public final TemporaryFolder temp = new TemporaryFolder();
+
+  private HadoopTables tables;
+
+  private String tableLocation;
+  private List<GenericRecord> rowsWritten;
+
+  /**
+   * Read all rows and columns from the table without any filter. The test asserts that the Arrow {@link
+   * VectorSchemaRoot} contains the expected schema and expected vector types. Then the test asserts that the vectors
+   * contains expected values. The test also asserts the total number of rows match the expected value.
+   */
+  @Test
+  public void testReadAll() throws Exception {
+    writeTableWithIncrementalRecords();
+    Table table = tables.load(tableLocation);
+    readAndCheckVectorSchemaRoots(table.newScan(), NUM_ROWS_PER_MONTH, 12 * NUM_ROWS_PER_MONTH, ALL_COLUMNS);
+  }
+
+  /**
+   * This test writes each partition with constant value rows. The Arrow vectors returned are mostly of type int32
+   * which is unexpected. This is happening because of dictionary encoding at the storage level.
+   * <p>
+   * Following are the expected and actual Arrow schema:
+   * <pre>
+   * Expected Arrow Schema:
+   * timestamp: Timestamp(MICROSECOND, null) not null,
+   * timestamp_nullable: Timestamp(MICROSECOND, null),
+   * boolean: Bool not null,
+   * boolean_nullable: Bool,
+   * int: Int(32, true) not null,
+   * int_nullable: Int(32, true),
+   * long: Int(64, true) not null,
+   * long_nullable: Int(64, true),
+   * float: FloatingPoint(SINGLE) not null,
+   * float_nullable: FloatingPoint(SINGLE),
+   * double: FloatingPoint(DOUBLE) not null,
+   * double_nullable: FloatingPoint(DOUBLE),
+   * timestamp_tz: Timestamp(MICROSECOND, UTC) not null,
+   * timestamp_tz_nullable: Timestamp(MICROSECOND, UTC),
+   * string: Utf8 not null,
+   * string_nullable: Utf8,
+   * bytes: Binary not null,
+   * bytes_nullable: Binary,
+   * date: Date(DAY) not null,
+   * date_nullable: Date(DAY),
+   * int_promotion: Int(32, true) not null
+   *
+   * Actual Arrow Schema:
+   * timestamp: Int(32, true) not null,
+   * timestamp_nullable: Int(32, true),
+   * boolean: Bool not null,
+   * boolean_nullable: Bool,
+   * int: Int(32, true) not null,
+   * int_nullable: Int(32, true),
+   * long: Int(32, true) not null,
+   * long_nullable: Int(32, true),
+   * float: Int(32, true) not null,
+   * float_nullable: Int(32, true),
+   * double: Int(32, true) not null,
+   * double_nullable: Int(32, true),
+   * timestamp_tz: Int(32, true) not null,
+   * timestamp_tz_nullable: Int(32, true),
+   * string: Int(32, true) not null,
+   * string_nullable: Int(32, true),
+   * bytes: Int(32, true) not null,
+   * bytes_nullable: Int(32, true),
+   * date: Date(DAY) not null,
+   * date_nullable: Date(DAY),
+   * int_promotion: Int(32, true) not null
+   * </pre>
+   * <p>
+   * TODO: fix the returned Arrow vectors to have vector types consistent with Iceberg types.
+   * <p>
+   * Read all rows and columns from the table without any filter. The test asserts that the Arrow {@link
+   * VectorSchemaRoot} contains the expected schema and expected vector types. Then the test asserts that the vectors
+   * contains expected values. The test also asserts the total number of rows match the expected value.
+   */
+  @Test
+  @Ignore
+  public void testReadAllWithConstantRecords() throws Exception {
+    writeTableWithConstantRecords();
+    Table table = tables.load(tableLocation);
+    readAndCheckVectorSchemaRoots(table.newScan(), NUM_ROWS_PER_MONTH, 12 * NUM_ROWS_PER_MONTH, ALL_COLUMNS);
+  }
+
+  /**
+   * Read all rows and columns from the table without any filter. The test uses a batch size smaller than the number of
+   * rows in a partition. The test asserts that the Arrow {@link VectorSchemaRoot} contains the expected schema and
+   * expected vector types. Then the test asserts that the vectors contains expected values. The test also asserts the
+   * total number of rows match the expected value.
+   */
+  @Test
+  public void testReadAllWithSmallerBatchSize() throws Exception {
+    writeTableWithIncrementalRecords();
+    Table table = tables.load(tableLocation);
+    TableScan scan = table.newScan();
+    readAndCheckVectorSchemaRoots(scan, 10, 12 * NUM_ROWS_PER_MONTH, ALL_COLUMNS);
+  }
+
+  /**
+   * Read selected rows and all columns from the table using a time range row filter. The test asserts that the Arrow
+   * {@link VectorSchemaRoot} contains the expected schema and expected vector types. Then the test asserts that the
+   * vectors contains expected values. The test also asserts the total number of rows match the expected value.
+   */
+  @Test
+  public void testReadRangeFilter() throws Exception {
+    writeTableWithIncrementalRecords();
+    Table table = tables.load(tableLocation);
+    LocalDateTime beginTime = LocalDateTime.of(2020, 1, 1, 0, 0, 0);
+    LocalDateTime endTime = LocalDateTime.of(2020, 2, 1, 0, 0, 0);
+    TableScan scan = table.newScan()
+        .filter(Expressions.and(
+            Expressions.greaterThanOrEqual("timestamp", timestampToMicros(beginTime)),
+            Expressions.lessThan("timestamp", timestampToMicros(endTime))));
+    readAndCheckVectorSchemaRoots(scan, NUM_ROWS_PER_MONTH, NUM_ROWS_PER_MONTH, ALL_COLUMNS);
+  }
+
+  /**
+   * Read selected rows and all columns from the table using a time range row filter.
+   * The test asserts that the result is empty.
+   */
+  @Test
+  public void testReadRangeFilterEmptyResult() throws Exception {
+    writeTableWithIncrementalRecords();
+    Table table = tables.load(tableLocation);
+    LocalDateTime beginTime = LocalDateTime.of(2021, 1, 1, 0, 0, 0);
+    LocalDateTime endTime = LocalDateTime.of(2021, 2, 1, 0, 0, 0);
+    TableScan scan = table.newScan()
+            .filter(Expressions.and(
+                    Expressions.greaterThanOrEqual("timestamp", timestampToMicros(beginTime)),
+                    Expressions.lessThan("timestamp", timestampToMicros(endTime))));
+    int numRoots = 0;
+    for (ArrowBatch batch : new VectorizedTableScanIterable(scan, NUM_ROWS_PER_MONTH, false)) {
+      numRoots++;
+    }
+    assertEquals(0, numRoots);
+  }
+
+  /**
+   * Read all rows and selected columns from the table with a column selection filter. The test asserts that the Arrow
+   * {@link VectorSchemaRoot} contains the expected schema and expected vector types. Then the test asserts that the
+   * vectors contains expected values. The test also asserts the total number of rows match the expected value.
+   */
+  @Test
+  public void testReadColumnFilter1() throws Exception {
+    writeTableWithIncrementalRecords();
+    Table table = tables.load(tableLocation);
+    TableScan scan = table.newScan()
+        .select("timestamp", "int", "string");
+    readAndCheckVectorSchemaRoots(
+        scan, NUM_ROWS_PER_MONTH, 12 * NUM_ROWS_PER_MONTH,
+        ImmutableList.of("timestamp", "int", "string"));
+  }
+
+  /**
+   * Read all rows and a single column from the table with a column selection filter. The test asserts that the Arrow
+   * {@link VectorSchemaRoot} contains the expected schema and expected vector types. Then the test asserts that the
+   * vectors contains expected values. The test also asserts the total number of rows match the expected value.
+   */
+  @Test
+  public void testReadColumnFilter2() throws Exception {
+    writeTableWithIncrementalRecords();
+    Table table = tables.load(tableLocation);
+    TableScan scan = table.newScan()
+        .select("timestamp");
+    readAndCheckVectorSchemaRoots(
+        scan, NUM_ROWS_PER_MONTH, 12 * NUM_ROWS_PER_MONTH,
+        ImmutableList.of("timestamp"));
+  }
+
+  private void readAndCheckVectorSchemaRoots(
+      TableScan scan,
+      int numRowsPerRoot,
+      int expectedTotalRows,
+      List<String> columns) {
+    Set<String> columnSet = ImmutableSet.copyOf(columns);
+    int rowIndex = 0;
+    int totalRows = 0;
+    for (ArrowBatch batch : new VectorizedTableScanIterable(scan, numRowsPerRoot, false)) {

Review comment:
       should this be in a try resource block? `VectorizedTableScanIterable` is never closed right?

##########
File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/VectorizedCombinedScanIterator.java
##########
@@ -0,0 +1,194 @@
+/*
+ * 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.arrow.vectorized;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.stream.Stream;
+import org.apache.arrow.vector.NullCheckingForGet;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.encryption.EncryptedFiles;
+import org.apache.iceberg.encryption.EncryptedInputFile;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.CloseableIterator;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Reads the data file and returns an iterator of {@link VectorSchemaRoot}.
+ * Only Parquet data file format is supported.
+ */
+class VectorizedCombinedScanIterator implements CloseableIterator<ArrowBatch> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(VectorizedCombinedScanIterator.class);
+
+  private final Iterator<FileScanTask> tasks;
+  private final Map<String, InputFile> inputFiles;
+  private final Schema expectedSchema;
+  private final String nameMapping;
+  private final boolean caseSensitive;
+  private final int batchSize;
+  private final boolean reuseContainers;
+  private CloseableIterator<ArrowBatch> currentIterator;
+  private ArrowBatch current;
+  private FileScanTask currentTask;
+
+  /**
+   * Create a new instance.
+   *
+   * @param task              Combined file scan task.
+   * @param expectedSchema    Read schema. The returned data will have this schema.
+   * @param nameMapping       Mapping from external schema names to Iceberg type IDs.
+   * @param io                File I/O.
+   * @param encryptionManager Encryption manager.
+   * @param caseSensitive     If {@code true}, column names are case sensitive.
+   *                          If {@code false}, column names are not case sensitive.
+   * @param batchSize         Batch size in number of rows. Each Arrow batch contains
+   *                          a maximum of {@code batchSize} rows.
+   * @param reuseContainers   If set to {@code false}, every {@link Iterator#next()} call creates
+   *                          new instances of Arrow vectors.
+   *                          If set to {@code true}, the Arrow vectors in the previous
+   *                          {@link Iterator#next()} may be reused for the data returned
+   *                          in the current {@link Iterator#next()}.
+   *                          This option avoids allocating memory again and again.
+   *                          Irrespective of the value of {@code reuseContainers}, the Arrow vectors
+   *                          in the previous {@link Iterator#next()} call are closed before creating
+   *                          new instances if the current {@link Iterator#next()}.
+   */
+  VectorizedCombinedScanIterator(
+          CombinedScanTask task,
+          Schema expectedSchema,
+          String nameMapping,
+          FileIO io,
+          EncryptionManager encryptionManager,
+          boolean caseSensitive,
+          int batchSize,
+          boolean reuseContainers) {
+    this.tasks = task.files().iterator();
+    Map<String, ByteBuffer> keyMetadata = Maps.newHashMap();
+    task.files().stream()
+        .flatMap(fileScanTask -> Stream.concat(Stream.of(fileScanTask.file()), fileScanTask.deletes().stream()))
+        .forEach(file -> keyMetadata.put(file.path().toString(), file.keyMetadata()));
+    Stream<EncryptedInputFile> encrypted = keyMetadata.entrySet().stream()
+        .map(entry -> EncryptedFiles.encryptedInput(io.newInputFile(entry.getKey()), entry.getValue()));
+
+    // decrypt with the batch call to avoid multiple RPCs to a key server, if possible
+    Iterable<InputFile> decryptedFiles = encryptionManager.decrypt(encrypted::iterator);
+
+    Map<String, InputFile> files = Maps.newHashMapWithExpectedSize(task.files().size());
+    decryptedFiles.forEach(decrypted -> files.putIfAbsent(decrypted.location(), decrypted));
+    this.inputFiles = Collections.unmodifiableMap(files);
+
+    this.currentIterator = CloseableIterator.empty();
+    this.expectedSchema = expectedSchema;
+    this.nameMapping = nameMapping;
+    this.caseSensitive = caseSensitive;
+    this.batchSize = batchSize;
+    this.reuseContainers = reuseContainers;
+  }
+
+  @Override
+  public boolean hasNext() {

Review comment:
       What is the difference between this class and `ArrowReader` seems to be some duplication?

##########
File path: arrow/src/test/java/org/apache/iceberg/arrow/vectorized/ArrowReaderTest.java
##########
@@ -0,0 +1,734 @@
+/*
+ * 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.arrow.vectorized;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.time.temporal.ChronoUnit;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+import org.apache.arrow.vector.BigIntVector;
+import org.apache.arrow.vector.BitVector;
+import org.apache.arrow.vector.DateDayVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.Float4Vector;
+import org.apache.arrow.vector.Float8Vector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.TimeStampMicroTZVector;
+import org.apache.arrow.vector.TimeStampMicroVector;
+import org.apache.arrow.vector.VarBinaryVector;
+import org.apache.arrow.vector.VarCharVector;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.arrow.vector.types.Types.MinorType;
+import org.apache.arrow.vector.types.pojo.ArrowType;
+import org.apache.arrow.vector.types.pojo.Field;
+import org.apache.arrow.vector.types.pojo.FieldType;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.OverwriteFiles;
+import org.apache.iceberg.PartitionKey;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.data.parquet.GenericParquetWriter;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.types.Types;
+import org.junit.Ignore;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.Files.localInput;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Test cases for {@link ArrowReader}.
+ * <p>All tests create a table with monthly partitions and write 1 year of data to the table.
+ */
+public class ArrowReaderTest {
+
+  private static final int NUM_ROWS_PER_MONTH = 20;
+  private static final ImmutableList<String> ALL_COLUMNS =
+      ImmutableList.of(
+          "timestamp",
+          "timestamp_nullable",
+          "boolean",
+          "boolean_nullable",
+          "int",
+          "int_nullable",
+          "long",
+          "long_nullable",
+          "float",
+          "float_nullable",
+          "double",
+          "double_nullable",
+          "timestamp_tz",
+          "timestamp_tz_nullable",
+          "string",
+          "string_nullable",
+          "bytes",
+          "bytes_nullable",
+          "date",
+          "date_nullable",
+          "int_promotion"
+      );
+
+  @Rule
+  public final TemporaryFolder temp = new TemporaryFolder();
+
+  private HadoopTables tables;
+
+  private String tableLocation;
+  private List<GenericRecord> rowsWritten;
+
+  /**
+   * Read all rows and columns from the table without any filter. The test asserts that the Arrow {@link
+   * VectorSchemaRoot} contains the expected schema and expected vector types. Then the test asserts that the vectors
+   * contains expected values. The test also asserts the total number of rows match the expected value.
+   */
+  @Test
+  public void testReadAll() throws Exception {
+    writeTableWithIncrementalRecords();
+    Table table = tables.load(tableLocation);
+    readAndCheckVectorSchemaRoots(table.newScan(), NUM_ROWS_PER_MONTH, 12 * NUM_ROWS_PER_MONTH, ALL_COLUMNS);
+  }
+
+  /**
+   * This test writes each partition with constant value rows. The Arrow vectors returned are mostly of type int32
+   * which is unexpected. This is happening because of dictionary encoding at the storage level.
+   * <p>
+   * Following are the expected and actual Arrow schema:
+   * <pre>
+   * Expected Arrow Schema:
+   * timestamp: Timestamp(MICROSECOND, null) not null,
+   * timestamp_nullable: Timestamp(MICROSECOND, null),
+   * boolean: Bool not null,
+   * boolean_nullable: Bool,
+   * int: Int(32, true) not null,
+   * int_nullable: Int(32, true),
+   * long: Int(64, true) not null,
+   * long_nullable: Int(64, true),
+   * float: FloatingPoint(SINGLE) not null,
+   * float_nullable: FloatingPoint(SINGLE),
+   * double: FloatingPoint(DOUBLE) not null,
+   * double_nullable: FloatingPoint(DOUBLE),
+   * timestamp_tz: Timestamp(MICROSECOND, UTC) not null,
+   * timestamp_tz_nullable: Timestamp(MICROSECOND, UTC),
+   * string: Utf8 not null,
+   * string_nullable: Utf8,
+   * bytes: Binary not null,
+   * bytes_nullable: Binary,
+   * date: Date(DAY) not null,
+   * date_nullable: Date(DAY),
+   * int_promotion: Int(32, true) not null
+   *
+   * Actual Arrow Schema:
+   * timestamp: Int(32, true) not null,
+   * timestamp_nullable: Int(32, true),
+   * boolean: Bool not null,
+   * boolean_nullable: Bool,
+   * int: Int(32, true) not null,
+   * int_nullable: Int(32, true),
+   * long: Int(32, true) not null,
+   * long_nullable: Int(32, true),
+   * float: Int(32, true) not null,
+   * float_nullable: Int(32, true),
+   * double: Int(32, true) not null,
+   * double_nullable: Int(32, true),
+   * timestamp_tz: Int(32, true) not null,
+   * timestamp_tz_nullable: Int(32, true),
+   * string: Int(32, true) not null,
+   * string_nullable: Int(32, true),
+   * bytes: Int(32, true) not null,
+   * bytes_nullable: Int(32, true),
+   * date: Date(DAY) not null,
+   * date_nullable: Date(DAY),
+   * int_promotion: Int(32, true) not null
+   * </pre>
+   * <p>
+   * TODO: fix the returned Arrow vectors to have vector types consistent with Iceberg types.
+   * <p>
+   * Read all rows and columns from the table without any filter. The test asserts that the Arrow {@link
+   * VectorSchemaRoot} contains the expected schema and expected vector types. Then the test asserts that the vectors
+   * contains expected values. The test also asserts the total number of rows match the expected value.
+   */
+  @Test
+  @Ignore
+  public void testReadAllWithConstantRecords() throws Exception {
+    writeTableWithConstantRecords();
+    Table table = tables.load(tableLocation);
+    readAndCheckVectorSchemaRoots(table.newScan(), NUM_ROWS_PER_MONTH, 12 * NUM_ROWS_PER_MONTH, ALL_COLUMNS);
+  }
+
+  /**
+   * Read all rows and columns from the table without any filter. The test uses a batch size smaller than the number of
+   * rows in a partition. The test asserts that the Arrow {@link VectorSchemaRoot} contains the expected schema and
+   * expected vector types. Then the test asserts that the vectors contains expected values. The test also asserts the
+   * total number of rows match the expected value.
+   */
+  @Test
+  public void testReadAllWithSmallerBatchSize() throws Exception {
+    writeTableWithIncrementalRecords();
+    Table table = tables.load(tableLocation);
+    TableScan scan = table.newScan();
+    readAndCheckVectorSchemaRoots(scan, 10, 12 * NUM_ROWS_PER_MONTH, ALL_COLUMNS);
+  }
+
+  /**
+   * Read selected rows and all columns from the table using a time range row filter. The test asserts that the Arrow
+   * {@link VectorSchemaRoot} contains the expected schema and expected vector types. Then the test asserts that the
+   * vectors contains expected values. The test also asserts the total number of rows match the expected value.
+   */
+  @Test
+  public void testReadRangeFilter() throws Exception {
+    writeTableWithIncrementalRecords();
+    Table table = tables.load(tableLocation);
+    LocalDateTime beginTime = LocalDateTime.of(2020, 1, 1, 0, 0, 0);
+    LocalDateTime endTime = LocalDateTime.of(2020, 2, 1, 0, 0, 0);
+    TableScan scan = table.newScan()
+        .filter(Expressions.and(
+            Expressions.greaterThanOrEqual("timestamp", timestampToMicros(beginTime)),
+            Expressions.lessThan("timestamp", timestampToMicros(endTime))));
+    readAndCheckVectorSchemaRoots(scan, NUM_ROWS_PER_MONTH, NUM_ROWS_PER_MONTH, ALL_COLUMNS);
+  }
+
+  /**
+   * Read selected rows and all columns from the table using a time range row filter.
+   * The test asserts that the result is empty.
+   */
+  @Test
+  public void testReadRangeFilterEmptyResult() throws Exception {
+    writeTableWithIncrementalRecords();
+    Table table = tables.load(tableLocation);
+    LocalDateTime beginTime = LocalDateTime.of(2021, 1, 1, 0, 0, 0);
+    LocalDateTime endTime = LocalDateTime.of(2021, 2, 1, 0, 0, 0);
+    TableScan scan = table.newScan()
+            .filter(Expressions.and(
+                    Expressions.greaterThanOrEqual("timestamp", timestampToMicros(beginTime)),
+                    Expressions.lessThan("timestamp", timestampToMicros(endTime))));
+    int numRoots = 0;
+    for (ArrowBatch batch : new VectorizedTableScanIterable(scan, NUM_ROWS_PER_MONTH, false)) {
+      numRoots++;
+    }
+    assertEquals(0, numRoots);
+  }
+
+  /**
+   * Read all rows and selected columns from the table with a column selection filter. The test asserts that the Arrow
+   * {@link VectorSchemaRoot} contains the expected schema and expected vector types. Then the test asserts that the
+   * vectors contains expected values. The test also asserts the total number of rows match the expected value.
+   */
+  @Test
+  public void testReadColumnFilter1() throws Exception {
+    writeTableWithIncrementalRecords();
+    Table table = tables.load(tableLocation);
+    TableScan scan = table.newScan()
+        .select("timestamp", "int", "string");
+    readAndCheckVectorSchemaRoots(
+        scan, NUM_ROWS_PER_MONTH, 12 * NUM_ROWS_PER_MONTH,
+        ImmutableList.of("timestamp", "int", "string"));
+  }
+
+  /**
+   * Read all rows and a single column from the table with a column selection filter. The test asserts that the Arrow
+   * {@link VectorSchemaRoot} contains the expected schema and expected vector types. Then the test asserts that the
+   * vectors contains expected values. The test also asserts the total number of rows match the expected value.
+   */
+  @Test
+  public void testReadColumnFilter2() throws Exception {
+    writeTableWithIncrementalRecords();
+    Table table = tables.load(tableLocation);
+    TableScan scan = table.newScan()
+        .select("timestamp");
+    readAndCheckVectorSchemaRoots(
+        scan, NUM_ROWS_PER_MONTH, 12 * NUM_ROWS_PER_MONTH,
+        ImmutableList.of("timestamp"));
+  }
+
+  private void readAndCheckVectorSchemaRoots(
+      TableScan scan,
+      int numRowsPerRoot,
+      int expectedTotalRows,
+      List<String> columns) {
+    Set<String> columnSet = ImmutableSet.copyOf(columns);
+    int rowIndex = 0;
+    int totalRows = 0;
+    for (ArrowBatch batch : new VectorizedTableScanIterable(scan, numRowsPerRoot, false)) {
+      VectorSchemaRoot root = batch.createVectorSchemaRootFromVectors();

Review comment:
       As this is likely to be used as a reference for anyone looking to utilize the vectorized reader it would be good to include some tests of the `ArrowBatch` as well

##########
File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowVector.java
##########
@@ -0,0 +1,513 @@
+/*
+ * 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.arrow.vectorized;
+
+import java.nio.charset.StandardCharsets;
+import java.util.stream.IntStream;
+import org.apache.arrow.vector.BigIntVector;
+import org.apache.arrow.vector.BitVector;
+import org.apache.arrow.vector.DateDayVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.Float4Vector;
+import org.apache.arrow.vector.Float8Vector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.TimeStampMicroTZVector;
+import org.apache.arrow.vector.TimeStampMicroVector;
+import org.apache.arrow.vector.ValueVector;
+import org.apache.arrow.vector.VarBinaryVector;
+import org.apache.arrow.vector.VarCharVector;
+import org.apache.arrow.vector.holders.NullableVarCharHolder;
+import org.apache.iceberg.types.Types;
+import org.apache.parquet.Preconditions;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.PrimitiveType;
+
+/**
+ * This class is inspired by Spark's {@code ColumnVector}.
+ * This class represents the column data for an Iceberg table query.
+ * It wraps an arrow {@link FieldVector} and provides simple
+ * accessors for the row values. Advanced users can access
+ * the {@link FieldVector}.
+ * <p>
+ *   Supported Iceberg data types:
+ *   <ul>
+ *     <li>{@link Types.BooleanType}</li>
+ *     <li>{@link Types.IntegerType}</li>
+ *     <li>{@link Types.LongType}</li>
+ *     <li>{@link Types.FloatType}</li>
+ *     <li>{@link Types.DoubleType}</li>
+ *     <li>{@link Types.StringType}</li>
+ *     <li>{@link Types.BinaryType}</li>
+ *     <li>{@link Types.TimestampType} (with and without timezone)</li>
+ *     <li>{@link Types.DateType}</li>
+ *   </ul>
+ */
+public class ArrowVector implements AutoCloseable {

Review comment:
       There is a fair bit of duplication between here and `org.apache.iceberg.spark.data.vectorized`. I think we should be able to collapse these and the spark classes w/ eg a parameter or so. We can then remove the spark classes and reference these.




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] mayursrivastava commented on a change in pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
mayursrivastava commented on a change in pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#discussion_r612036731



##########
File path: arrow/src/test/java/org/apache/iceberg/arrow/vectorized/ArrowReaderTest.java
##########
@@ -0,0 +1,734 @@
+/*
+ * 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.arrow.vectorized;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.time.temporal.ChronoUnit;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+import org.apache.arrow.vector.BigIntVector;
+import org.apache.arrow.vector.BitVector;
+import org.apache.arrow.vector.DateDayVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.Float4Vector;
+import org.apache.arrow.vector.Float8Vector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.TimeStampMicroTZVector;
+import org.apache.arrow.vector.TimeStampMicroVector;
+import org.apache.arrow.vector.VarBinaryVector;
+import org.apache.arrow.vector.VarCharVector;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.arrow.vector.types.Types.MinorType;
+import org.apache.arrow.vector.types.pojo.ArrowType;
+import org.apache.arrow.vector.types.pojo.Field;
+import org.apache.arrow.vector.types.pojo.FieldType;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.OverwriteFiles;
+import org.apache.iceberg.PartitionKey;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.data.parquet.GenericParquetWriter;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.types.Types;
+import org.junit.Ignore;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.Files.localInput;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Test cases for {@link ArrowReader}.
+ * <p>All tests create a table with monthly partitions and write 1 year of data to the table.
+ */
+public class ArrowReaderTest {
+
+  private static final int NUM_ROWS_PER_MONTH = 20;
+  private static final ImmutableList<String> ALL_COLUMNS =
+      ImmutableList.of(
+          "timestamp",
+          "timestamp_nullable",
+          "boolean",
+          "boolean_nullable",
+          "int",
+          "int_nullable",
+          "long",
+          "long_nullable",
+          "float",
+          "float_nullable",
+          "double",
+          "double_nullable",
+          "timestamp_tz",
+          "timestamp_tz_nullable",
+          "string",
+          "string_nullable",
+          "bytes",
+          "bytes_nullable",
+          "date",
+          "date_nullable",
+          "int_promotion"
+      );
+
+  @Rule
+  public final TemporaryFolder temp = new TemporaryFolder();
+
+  private HadoopTables tables;
+
+  private String tableLocation;
+  private List<GenericRecord> rowsWritten;
+
+  /**
+   * Read all rows and columns from the table without any filter. The test asserts that the Arrow {@link
+   * VectorSchemaRoot} contains the expected schema and expected vector types. Then the test asserts that the vectors
+   * contains expected values. The test also asserts the total number of rows match the expected value.
+   */
+  @Test
+  public void testReadAll() throws Exception {
+    writeTableWithIncrementalRecords();
+    Table table = tables.load(tableLocation);
+    readAndCheckVectorSchemaRoots(table.newScan(), NUM_ROWS_PER_MONTH, 12 * NUM_ROWS_PER_MONTH, ALL_COLUMNS);
+  }
+
+  /**
+   * This test writes each partition with constant value rows. The Arrow vectors returned are mostly of type int32
+   * which is unexpected. This is happening because of dictionary encoding at the storage level.
+   * <p>
+   * Following are the expected and actual Arrow schema:
+   * <pre>
+   * Expected Arrow Schema:
+   * timestamp: Timestamp(MICROSECOND, null) not null,
+   * timestamp_nullable: Timestamp(MICROSECOND, null),
+   * boolean: Bool not null,
+   * boolean_nullable: Bool,
+   * int: Int(32, true) not null,
+   * int_nullable: Int(32, true),
+   * long: Int(64, true) not null,
+   * long_nullable: Int(64, true),
+   * float: FloatingPoint(SINGLE) not null,
+   * float_nullable: FloatingPoint(SINGLE),
+   * double: FloatingPoint(DOUBLE) not null,
+   * double_nullable: FloatingPoint(DOUBLE),
+   * timestamp_tz: Timestamp(MICROSECOND, UTC) not null,
+   * timestamp_tz_nullable: Timestamp(MICROSECOND, UTC),
+   * string: Utf8 not null,
+   * string_nullable: Utf8,
+   * bytes: Binary not null,
+   * bytes_nullable: Binary,
+   * date: Date(DAY) not null,
+   * date_nullable: Date(DAY),
+   * int_promotion: Int(32, true) not null
+   *
+   * Actual Arrow Schema:
+   * timestamp: Int(32, true) not null,
+   * timestamp_nullable: Int(32, true),
+   * boolean: Bool not null,
+   * boolean_nullable: Bool,
+   * int: Int(32, true) not null,
+   * int_nullable: Int(32, true),
+   * long: Int(32, true) not null,
+   * long_nullable: Int(32, true),
+   * float: Int(32, true) not null,
+   * float_nullable: Int(32, true),
+   * double: Int(32, true) not null,
+   * double_nullable: Int(32, true),
+   * timestamp_tz: Int(32, true) not null,
+   * timestamp_tz_nullable: Int(32, true),
+   * string: Int(32, true) not null,
+   * string_nullable: Int(32, true),
+   * bytes: Int(32, true) not null,
+   * bytes_nullable: Int(32, true),
+   * date: Date(DAY) not null,
+   * date_nullable: Date(DAY),
+   * int_promotion: Int(32, true) not null
+   * </pre>
+   * <p>
+   * TODO: fix the returned Arrow vectors to have vector types consistent with Iceberg types.
+   * <p>
+   * Read all rows and columns from the table without any filter. The test asserts that the Arrow {@link
+   * VectorSchemaRoot} contains the expected schema and expected vector types. Then the test asserts that the vectors
+   * contains expected values. The test also asserts the total number of rows match the expected value.
+   */
+  @Test
+  @Ignore
+  public void testReadAllWithConstantRecords() throws Exception {
+    writeTableWithConstantRecords();
+    Table table = tables.load(tableLocation);
+    readAndCheckVectorSchemaRoots(table.newScan(), NUM_ROWS_PER_MONTH, 12 * NUM_ROWS_PER_MONTH, ALL_COLUMNS);
+  }
+
+  /**
+   * Read all rows and columns from the table without any filter. The test uses a batch size smaller than the number of
+   * rows in a partition. The test asserts that the Arrow {@link VectorSchemaRoot} contains the expected schema and
+   * expected vector types. Then the test asserts that the vectors contains expected values. The test also asserts the
+   * total number of rows match the expected value.
+   */
+  @Test
+  public void testReadAllWithSmallerBatchSize() throws Exception {
+    writeTableWithIncrementalRecords();
+    Table table = tables.load(tableLocation);
+    TableScan scan = table.newScan();
+    readAndCheckVectorSchemaRoots(scan, 10, 12 * NUM_ROWS_PER_MONTH, ALL_COLUMNS);
+  }
+
+  /**
+   * Read selected rows and all columns from the table using a time range row filter. The test asserts that the Arrow
+   * {@link VectorSchemaRoot} contains the expected schema and expected vector types. Then the test asserts that the
+   * vectors contains expected values. The test also asserts the total number of rows match the expected value.
+   */
+  @Test
+  public void testReadRangeFilter() throws Exception {
+    writeTableWithIncrementalRecords();
+    Table table = tables.load(tableLocation);
+    LocalDateTime beginTime = LocalDateTime.of(2020, 1, 1, 0, 0, 0);
+    LocalDateTime endTime = LocalDateTime.of(2020, 2, 1, 0, 0, 0);
+    TableScan scan = table.newScan()
+        .filter(Expressions.and(
+            Expressions.greaterThanOrEqual("timestamp", timestampToMicros(beginTime)),
+            Expressions.lessThan("timestamp", timestampToMicros(endTime))));
+    readAndCheckVectorSchemaRoots(scan, NUM_ROWS_PER_MONTH, NUM_ROWS_PER_MONTH, ALL_COLUMNS);
+  }
+
+  /**
+   * Read selected rows and all columns from the table using a time range row filter.
+   * The test asserts that the result is empty.
+   */
+  @Test
+  public void testReadRangeFilterEmptyResult() throws Exception {
+    writeTableWithIncrementalRecords();
+    Table table = tables.load(tableLocation);
+    LocalDateTime beginTime = LocalDateTime.of(2021, 1, 1, 0, 0, 0);
+    LocalDateTime endTime = LocalDateTime.of(2021, 2, 1, 0, 0, 0);
+    TableScan scan = table.newScan()
+            .filter(Expressions.and(
+                    Expressions.greaterThanOrEqual("timestamp", timestampToMicros(beginTime)),
+                    Expressions.lessThan("timestamp", timestampToMicros(endTime))));
+    int numRoots = 0;
+    for (ArrowBatch batch : new VectorizedTableScanIterable(scan, NUM_ROWS_PER_MONTH, false)) {
+      numRoots++;
+    }
+    assertEquals(0, numRoots);
+  }
+
+  /**
+   * Read all rows and selected columns from the table with a column selection filter. The test asserts that the Arrow
+   * {@link VectorSchemaRoot} contains the expected schema and expected vector types. Then the test asserts that the
+   * vectors contains expected values. The test also asserts the total number of rows match the expected value.
+   */
+  @Test
+  public void testReadColumnFilter1() throws Exception {
+    writeTableWithIncrementalRecords();
+    Table table = tables.load(tableLocation);
+    TableScan scan = table.newScan()
+        .select("timestamp", "int", "string");
+    readAndCheckVectorSchemaRoots(
+        scan, NUM_ROWS_PER_MONTH, 12 * NUM_ROWS_PER_MONTH,
+        ImmutableList.of("timestamp", "int", "string"));
+  }
+
+  /**
+   * Read all rows and a single column from the table with a column selection filter. The test asserts that the Arrow
+   * {@link VectorSchemaRoot} contains the expected schema and expected vector types. Then the test asserts that the
+   * vectors contains expected values. The test also asserts the total number of rows match the expected value.
+   */
+  @Test
+  public void testReadColumnFilter2() throws Exception {
+    writeTableWithIncrementalRecords();
+    Table table = tables.load(tableLocation);
+    TableScan scan = table.newScan()
+        .select("timestamp");
+    readAndCheckVectorSchemaRoots(
+        scan, NUM_ROWS_PER_MONTH, 12 * NUM_ROWS_PER_MONTH,
+        ImmutableList.of("timestamp"));
+  }
+
+  private void readAndCheckVectorSchemaRoots(
+      TableScan scan,
+      int numRowsPerRoot,
+      int expectedTotalRows,
+      List<String> columns) {
+    Set<String> columnSet = ImmutableSet.copyOf(columns);
+    int rowIndex = 0;
+    int totalRows = 0;
+    for (ArrowBatch batch : new VectorizedTableScanIterable(scan, numRowsPerRoot, false)) {

Review comment:
       fixed




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#discussion_r638553663



##########
File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ColumnVector.java
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.arrow.vectorized;
+
+import org.apache.arrow.vector.FieldVector;
+import org.apache.iceberg.types.Types;
+
+/**
+ * This class is inspired by Spark's {@code ColumnVector}.
+ * This class represents the column data for an Iceberg table query.
+ * It wraps an arrow {@link FieldVector} and provides simple
+ * accessors for the row values. Advanced users can access
+ * the {@link FieldVector}.
+ * <p>
+ *   Supported Iceberg data types:
+ *   <ul>
+ *     <li>{@link Types.BooleanType}</li>
+ *     <li>{@link Types.IntegerType}</li>
+ *     <li>{@link Types.LongType}</li>
+ *     <li>{@link Types.FloatType}</li>
+ *     <li>{@link Types.DoubleType}</li>
+ *     <li>{@link Types.StringType}</li>
+ *     <li>{@link Types.BinaryType}</li>
+ *     <li>{@link Types.TimestampType} (with and without timezone)</li>
+ *     <li>{@link Types.DateType}</li>
+ *   </ul>
+ */
+public class ColumnVector implements AutoCloseable {

Review comment:
       My thinking was to hide the Arrow imp from clients at the cost of an extra layer of indirection. I think the Arrow stuff restricted to the arrow module and out of the public iceberg api will allow more flexibility for future vectorised impls and doesn't couple the two projects too tightly. Happy to discuss if anyone disagrees?




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#discussion_r638343310



##########
File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowReader.java
##########
@@ -0,0 +1,325 @@
+/*
+ * 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.arrow.vectorized;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import java.util.stream.StreamSupport;
+import org.apache.arrow.vector.NullCheckingForGet;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.arrow.vector.types.Types.MinorType;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.encryption.EncryptedFiles;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.CloseableIterator;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.types.Types;
+import org.apache.parquet.schema.MessageType;
+
+/**
+ * Vectorized reader that returns an iterator of {@link ColumnarBatch}.
+ * See {@link #open(CloseableIterable)} ()} to learn about the
+ * behavior of the iterator.
+ *
+ * <p>The following Iceberg data types are supported and have been tested:
+ * <ul>
+ *     <li>Iceberg: {@link Types.BooleanType}, Arrow: {@link MinorType#BIT}</li>
+ *     <li>Iceberg: {@link Types.IntegerType}, Arrow: {@link MinorType#INT}</li>
+ *     <li>Iceberg: {@link Types.LongType}, Arrow: {@link MinorType#BIGINT}</li>
+ *     <li>Iceberg: {@link Types.FloatType}, Arrow: {@link MinorType#FLOAT4}</li>
+ *     <li>Iceberg: {@link Types.DoubleType}, Arrow: {@link MinorType#FLOAT8}</li>
+ *     <li>Iceberg: {@link Types.StringType}, Arrow: {@link MinorType#VARCHAR}</li>
+ *     <li>Iceberg: {@link Types.TimestampType} (both with and without timezone),
+ *         Arrow: {@link MinorType#TIMEMICRO}</li>
+ *     <li>Iceberg: {@link Types.BinaryType}, Arrow: {@link MinorType#VARBINARY}</li>
+ *     <li>Iceberg: {@link Types.DateType}, Arrow: {@link MinorType#DATEDAY}</li>
+ * </ul>
+ *
+ * <p>Features that don't work in this implementation:
+ * <ul>
+ *     <li>Type promotion: In case of type promotion, the Arrow vector corresponding to
+ *     the data type in the parquet file is returned instead of the data type in the latest schema.
+ *     See https://github.com/apache/iceberg/issues/2483.</li>
+ *     <li>Columns with constant values are physically encoded as a dictionary. The Arrow vector
+ *     type is int32 instead of the type as per the schema.
+ *     See https://github.com/apache/iceberg/issues/2484.</li>
+ *     <li>Data types: {@link Types.TimeType}, {@link Types.ListType}, {@link Types.MapType},
+ *     {@link Types.StructType}, {@link Types.UUIDType}, {@link Types.FixedType} and
+ *     {@link Types.DecimalType}
+ *     See https://github.com/apache/iceberg/issues/2485 and https://github.com/apache/iceberg/issues/2486.</li>
+ *     <li>Iceberg v2 spec is not supported.
+ *     See https://github.com/apache/iceberg/issues/2487.</li>
+ * </ul>
+ */
+public class ArrowReader extends CloseableGroup {
+
+  private final Schema schema;
+  private final FileIO io;
+  private final EncryptionManager encryption;
+  private final int batchSize;
+  private final boolean reuseContainers;
+
+  /**
+   * Create a new instance of the reader.
+   *
+   * @param scan the table scan object.
+   * @param batchSize the maximum number of rows per Arrow batch.
+   * @param reuseContainers whether to reuse Arrow vectors when iterating through the data.
+   *                        If set to {@code false}, every {@link Iterator#next()} call creates
+   *                        new instances of Arrow vectors.
+   *                        If set to {@code true}, the Arrow vectors in the previous
+   *                        {@link Iterator#next()} may be reused for the data returned
+   *                        in the current {@link Iterator#next()}.
+   *                        This option avoids allocating memory again and again.
+   *                        Irrespective of the value of {@code reuseContainers}, the Arrow vectors
+   *                        in the previous {@link Iterator#next()} call are closed before creating
+   *                        new instances if the current {@link Iterator#next()}.
+   */
+  public ArrowReader(TableScan scan, int batchSize, boolean reuseContainers) {
+    this.schema = scan.schema();
+    this.io = scan.table().io();
+    this.encryption = scan.table().encryption();
+    this.batchSize = batchSize;
+    // start planning tasks in the background
+    this.reuseContainers = reuseContainers;
+  }
+
+  /**
+   * Returns a new iterator of {@link ColumnarBatch} objects.
+   * <p>
+   * Note that the reader owns the {@link ColumnarBatch} objects and takes care of closing them.
+   * The caller should not hold onto a {@link ColumnarBatch} or try to close them.
+   *
+   * <p>If {@code reuseContainers} is {@code false}, the Arrow vectors in the
+   * previous {@link ColumnarBatch} are closed before returning the next {@link ColumnarBatch} object.
+   * This implies that the caller should either use the {@link ColumnarBatch} or transfer the ownership of
+   * {@link ColumnarBatch} before getting the next {@link ColumnarBatch}.
+   *
+   * <p>If {@code reuseContainers} is {@code true}, the Arrow vectors in the
+   * previous {@link ColumnarBatch} may be reused for the next {@link ColumnarBatch}.
+   * This implies that the caller should either use the {@link ColumnarBatch} or deep copy the
+   * {@link ColumnarBatch} before getting the next {@link ColumnarBatch}.
+   */
+  public CloseableIterator<ColumnarBatch> open(CloseableIterable<CombinedScanTask> tasks) {
+    CloseableIterator<ColumnarBatch> itr = new VectorizedCombinedScanIterator(
+        tasks,
+        schema,
+        null,
+        io,
+        encryption,
+        true,
+        batchSize,
+        reuseContainers
+    );
+    addCloseable(itr);
+    return itr;
+  }
+
+  @Override
+  public void close() throws IOException {
+    super.close(); // close data files
+  }
+
+  /**
+   * Reads the data file and returns an iterator of {@link VectorSchemaRoot}.
+   * Only Parquet data file format is supported.
+   */
+  private static final class VectorizedCombinedScanIterator implements CloseableIterator<ColumnarBatch> {
+
+    private final List<FileScanTask> fileTasks;
+    private final Iterator<FileScanTask> fileItr;
+    private final Map<String, InputFile> inputFiles;
+    private final Schema expectedSchema;
+    private final String nameMapping;
+    private final boolean caseSensitive;
+    private final int batchSize;
+    private final boolean reuseContainers;
+    private CloseableIterator<ColumnarBatch> currentIterator;
+    private ColumnarBatch current;
+    private FileScanTask currentTask;
+
+    /**
+     * Create a new instance.
+     *
+     * @param tasks             Combined file scan tasks.
+     * @param expectedSchema    Read schema. The returned data will have this schema.
+     * @param nameMapping       Mapping from external schema names to Iceberg type IDs.
+     * @param io                File I/O.
+     * @param encryptionManager Encryption manager.
+     * @param caseSensitive     If {@code true}, column names are case sensitive.
+     *                          If {@code false}, column names are not case sensitive.
+     * @param batchSize         Batch size in number of rows. Each Arrow batch contains
+     *                          a maximum of {@code batchSize} rows.
+     * @param reuseContainers   If set to {@code false}, every {@link Iterator#next()} call creates
+     *                          new instances of Arrow vectors.
+     *                          If set to {@code true}, the Arrow vectors in the previous
+     *                          {@link Iterator#next()} may be reused for the data returned
+     *                          in the current {@link Iterator#next()}.
+     *                          This option avoids allocating memory again and again.
+     *                          Irrespective of the value of {@code reuseContainers}, the Arrow vectors
+     *                          in the previous {@link Iterator#next()} call are closed before creating
+     *                          new instances if the current {@link Iterator#next()}.
+     */
+    VectorizedCombinedScanIterator(
+        CloseableIterable<CombinedScanTask> tasks,
+        Schema expectedSchema,
+        String nameMapping,
+        FileIO io,
+        EncryptionManager encryptionManager,
+        boolean caseSensitive,
+        int batchSize,
+        boolean reuseContainers) {

Review comment:
       We have most things checked by checkstyle, but this is one that we don't have for some reason. I need to write a style guide.




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] mayursrivastava commented on a change in pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
mayursrivastava commented on a change in pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#discussion_r610803813



##########
File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowVector.java
##########
@@ -0,0 +1,513 @@
+/*
+ * 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.arrow.vectorized;
+
+import java.nio.charset.StandardCharsets;
+import java.util.stream.IntStream;
+import org.apache.arrow.vector.BigIntVector;
+import org.apache.arrow.vector.BitVector;
+import org.apache.arrow.vector.DateDayVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.Float4Vector;
+import org.apache.arrow.vector.Float8Vector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.TimeStampMicroTZVector;
+import org.apache.arrow.vector.TimeStampMicroVector;
+import org.apache.arrow.vector.ValueVector;
+import org.apache.arrow.vector.VarBinaryVector;
+import org.apache.arrow.vector.VarCharVector;
+import org.apache.arrow.vector.holders.NullableVarCharHolder;
+import org.apache.iceberg.types.Types;
+import org.apache.parquet.Preconditions;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.PrimitiveType;
+
+/**
+ * This class is inspired by Spark's {@code ColumnVector}.
+ * This class represents the column data for an Iceberg table query.
+ * It wraps an arrow {@link FieldVector} and provides simple
+ * accessors for the row values. Advanced users can access
+ * the {@link FieldVector}.
+ * <p>
+ *   Supported Iceberg data types:
+ *   <ul>
+ *     <li>{@link Types.BooleanType}</li>
+ *     <li>{@link Types.IntegerType}</li>
+ *     <li>{@link Types.LongType}</li>
+ *     <li>{@link Types.FloatType}</li>
+ *     <li>{@link Types.DoubleType}</li>
+ *     <li>{@link Types.StringType}</li>
+ *     <li>{@link Types.BinaryType}</li>
+ *     <li>{@link Types.TimestampType} (with and without timezone)</li>
+ *     <li>{@link Types.DateType}</li>
+ *   </ul>
+ */
+public class ArrowVector implements AutoCloseable {

Review comment:
       After looking at the source code, I think we can move ArrowVectorAccessor and ArrowVectorAccessors to arrow module. These classes have most of the common conversion code. These are public classes but don't expose their constructors or any methods publicly except one: ArrowVectorAccessor.getVector() which is used by a test case - however, I'm not sure if anyone outside is using these classes. I hope this is ok.
   
   I'll change the renamed ArrowVector (to ColumnVector) in the arrow module  to use ArrowVectorAccessor. I'll do this and update the pr, let me know how it looks.
   
   Moving IcebergArrowColumnVector from spark code to here require some dependency changes because the interface/abstract class ColumnVector lives in spark-sql.




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#discussion_r639077401



##########
File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ColumnVector.java
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.arrow.vectorized;
+
+import org.apache.arrow.vector.FieldVector;
+import org.apache.iceberg.types.Types;
+
+/**
+ * This class is inspired by Spark's {@code ColumnVector}.
+ * This class represents the column data for an Iceberg table query.
+ * It wraps an arrow {@link FieldVector} and provides simple
+ * accessors for the row values. Advanced users can access
+ * the {@link FieldVector}.
+ * <p>
+ *   Supported Iceberg data types:
+ *   <ul>
+ *     <li>{@link Types.BooleanType}</li>
+ *     <li>{@link Types.IntegerType}</li>
+ *     <li>{@link Types.LongType}</li>
+ *     <li>{@link Types.FloatType}</li>
+ *     <li>{@link Types.DoubleType}</li>
+ *     <li>{@link Types.StringType}</li>
+ *     <li>{@link Types.BinaryType}</li>
+ *     <li>{@link Types.TimestampType} (with and without timezone)</li>
+ *     <li>{@link Types.DateType}</li>
+ *   </ul>
+ */
+public class ColumnVector implements AutoCloseable {

Review comment:
       My concern is that we're introducing a new API that is specific to Iceberg and will need to be supported for a long time. Not that we can't do that, but I want to make sure we're carefully considering whether we want our own columnar API.
   
   That said, I think there's an argument for doing this because we probably won't be producing Arrow batches from ORC any time soon. @aokolnychyi, what do you think?




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] mayursrivastava commented on pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
mayursrivastava commented on pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#issuecomment-824210180


   `./gradlew :iceberg-spark:jmh -PjmhIncludeRegex=VectorizedReadFlatParquetDataBenchmark
     -PjmhOutputPath=benchmark/results.txt`
   
   @rymurr, is this :iceberg-spark2:jmh? Looks like this requires Java8 runtime. Am I right? I ran it on the master branch (without my changes), but it fails with the following error. Does it need a powerful machine?
   
   `# JMH version: 1.21
   # VM version: JDK 1.8.0_282, OpenJDK 64-Bit Server VM, 25.282-b08
   # VM invoker: /usr/lib/jvm/java-8-openjdk-amd64/jre/bin/java
   # VM options: <none>
   # Warmup: 3 iterations, single-shot each
   # Measurement: 5 iterations, single-shot each
   # Timeout: 10 min per iteration
   # Threads: 1 thread
   # Benchmark mode: Single shot invocation time
   # Benchmark: org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadDictionaryEncodedFlatParquetDataBenchmark.readDatesIcebergVectorized5k
   
   # Run progress: 0.00% complete, ETA 00:00:00
   # Fork: 1 of 1
   # Warmup Iteration   1: (*interrupt*) <failure>
   
   java.lang.InterruptedException
           at java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireSharedInterruptibly(AbstractQueuedSynchronizer.java:998)
           at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireSharedInterruptibly(AbstractQueuedSynchronizer.java:1304)
           at scala.concurrent.impl.Promise$DefaultPromise.tryAwait(Promise.scala:206)
           at scala.concurrent.impl.Promise$DefaultPromise.ready(Promise.scala:222)
           at scala.concurrent.impl.Promise$DefaultPromise.ready(Promise.scala:157)
           at org.apache.spark.util.ThreadUtils$.awaitReady(ThreadUtils.scala:243)
           at org.apache.spark.scheduler.DAGScheduler.runJob(DAGScheduler.scala:750)
           at org.apache.spark.SparkContext.runJob(SparkContext.scala:2061)
           at org.apache.spark.sql.execution.datasources.v2.WriteToDataSourceV2Exec.doExecute(WriteToDataSourceV2Exec.scala:64)
           at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:131)
           at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:127)
           at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:155)
           at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
           at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:152)
           at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:127)
           at org.apache.spark.sql.execution.QueryExecution.toRdd$lzycompute(QueryExecution.scala:83)
           at org.apache.spark.sql.execution.QueryExecution.toRdd(QueryExecution.scala:81)
           at org.apache.spark.sql.DataFrameWriter$$anonfun$runCommand$1.apply(DataFrameWriter.scala:696)
           at org.apache.spark.sql.DataFrameWriter$$anonfun$runCommand$1.apply(DataFrameWriter.scala:696)
           at org.apache.spark.sql.execution.SQLExecution$$anonfun$withNewExecutionId$1.apply(SQLExecution.scala:80)
           at org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:127)
           at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:75)
           at org.apache.spark.sql.DataFrameWriter.runCommand(DataFrameWriter.scala:696)
           at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:280)
           at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:249)
           at org.apache.iceberg.spark.source.IcebergSourceBenchmark.appendAsFile(IcebergSourceBenchmark.java:130)
           at org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadDictionaryEncodedFlatParquetDataBenchmark.appendData(VectorizedReadDictionaryEncodedFlatParquetDataBenchmark.java:82)
           at org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadDictionaryEncodedFlatParquetDataBenchmark.setupBenchmark(VectorizedReadDictionaryEncodedFlatParquetDataBenchmark.java:56)
           at org.apache.iceberg.spark.source.parquet.vectorized.generated.VectorizedReadDictionaryEncodedFlatParquetDataBenchmark_readDatesIcebergVectorized5k_jmhTest._jmh_tryInit_f_vectorizedreaddictionaryencodedflatparquetdatabenchmark0_G(VectorizedReadDictionaryEncodedFlatParquetDataBenchmark_readDatesIcebergVectorized5k_jmhTest.java:438)
           at org.apache.iceberg.spark.source.parquet.vectorized.generated.VectorizedReadDictionaryEncodedFlatParquetDataBenchmark_readDatesIcebergVectorized5k_jmhTest.readDatesIcebergVectorized5k_SingleShotTime(VectorizedReadDictionaryEncodedFlatParquetDataBenchmark_readDatesIcebergVectorized5k_jmhTest.java:363)
           at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
           at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
           at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
           at java.lang.reflect.Method.invoke(Method.java:498)
           at org.openjdk.jmh.runner.BenchmarkHandler$BenchmarkTask.call(BenchmarkHandler.java:453)
           at org.openjdk.jmh.runner.BenchmarkHandler$BenchmarkTask.call(BenchmarkHandler.java:437)
           at java.util.concurrent.FutureTask.run(FutureTask.java:266)
           at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
           at java.util.concurrent.FutureTask.run(FutureTask.java:266)
           at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
           at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
           at java.lang.Thread.run(Thread.java:748)
   
   
   Benchmark had encountered error, and fail on error was requested
   `


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] mayursrivastava commented on a change in pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
mayursrivastava commented on a change in pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#discussion_r610803813



##########
File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowVector.java
##########
@@ -0,0 +1,513 @@
+/*
+ * 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.arrow.vectorized;
+
+import java.nio.charset.StandardCharsets;
+import java.util.stream.IntStream;
+import org.apache.arrow.vector.BigIntVector;
+import org.apache.arrow.vector.BitVector;
+import org.apache.arrow.vector.DateDayVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.Float4Vector;
+import org.apache.arrow.vector.Float8Vector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.TimeStampMicroTZVector;
+import org.apache.arrow.vector.TimeStampMicroVector;
+import org.apache.arrow.vector.ValueVector;
+import org.apache.arrow.vector.VarBinaryVector;
+import org.apache.arrow.vector.VarCharVector;
+import org.apache.arrow.vector.holders.NullableVarCharHolder;
+import org.apache.iceberg.types.Types;
+import org.apache.parquet.Preconditions;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.PrimitiveType;
+
+/**
+ * This class is inspired by Spark's {@code ColumnVector}.
+ * This class represents the column data for an Iceberg table query.
+ * It wraps an arrow {@link FieldVector} and provides simple
+ * accessors for the row values. Advanced users can access
+ * the {@link FieldVector}.
+ * <p>
+ *   Supported Iceberg data types:
+ *   <ul>
+ *     <li>{@link Types.BooleanType}</li>
+ *     <li>{@link Types.IntegerType}</li>
+ *     <li>{@link Types.LongType}</li>
+ *     <li>{@link Types.FloatType}</li>
+ *     <li>{@link Types.DoubleType}</li>
+ *     <li>{@link Types.StringType}</li>
+ *     <li>{@link Types.BinaryType}</li>
+ *     <li>{@link Types.TimestampType} (with and without timezone)</li>
+ *     <li>{@link Types.DateType}</li>
+ *   </ul>
+ */
+public class ArrowVector implements AutoCloseable {

Review comment:
       After looking at the source code, I think we can move parts of ArrowVectorAccessor and ArrowVectorAccessors to arrow module. These classes have most of the common conversion code. 
   
   I'll change the renamed ArrowVector (to ColumnVector) in the arrow module  to use ArrowVectorAccessor. I'll do this and update the pr, let me know how it looks.
   
   Moving IcebergArrowColumnVector from spark code to here require some dependency changes because the interface/abstract class ColumnVector lives in spark-sql.




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] mayursrivastava commented on pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
mayursrivastava commented on pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#issuecomment-821290918


   Hi @rymurr, I've updated the pr as per comments. Please take a look when you get a chance. Thanks, Mayur 


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] mayursrivastava commented on a change in pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
mayursrivastava commented on a change in pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#discussion_r638325976



##########
File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowReader.java
##########
@@ -0,0 +1,325 @@
+/*
+ * 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.arrow.vectorized;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import java.util.stream.StreamSupport;
+import org.apache.arrow.vector.NullCheckingForGet;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.arrow.vector.types.Types.MinorType;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.encryption.EncryptedFiles;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.CloseableIterator;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.types.Types;
+import org.apache.parquet.schema.MessageType;
+
+/**
+ * Vectorized reader that returns an iterator of {@link ColumnarBatch}.
+ * See {@link #open(CloseableIterable)} ()} to learn about the
+ * behavior of the iterator.
+ *
+ * <p>The following Iceberg data types are supported and have been tested:
+ * <ul>
+ *     <li>Iceberg: {@link Types.BooleanType}, Arrow: {@link MinorType#BIT}</li>
+ *     <li>Iceberg: {@link Types.IntegerType}, Arrow: {@link MinorType#INT}</li>
+ *     <li>Iceberg: {@link Types.LongType}, Arrow: {@link MinorType#BIGINT}</li>
+ *     <li>Iceberg: {@link Types.FloatType}, Arrow: {@link MinorType#FLOAT4}</li>
+ *     <li>Iceberg: {@link Types.DoubleType}, Arrow: {@link MinorType#FLOAT8}</li>
+ *     <li>Iceberg: {@link Types.StringType}, Arrow: {@link MinorType#VARCHAR}</li>
+ *     <li>Iceberg: {@link Types.TimestampType} (both with and without timezone),
+ *         Arrow: {@link MinorType#TIMEMICRO}</li>
+ *     <li>Iceberg: {@link Types.BinaryType}, Arrow: {@link MinorType#VARBINARY}</li>
+ *     <li>Iceberg: {@link Types.DateType}, Arrow: {@link MinorType#DATEDAY}</li>
+ * </ul>
+ *
+ * <p>Features that don't work in this implementation:
+ * <ul>
+ *     <li>Type promotion: In case of type promotion, the Arrow vector corresponding to
+ *     the data type in the parquet file is returned instead of the data type in the latest schema.
+ *     See https://github.com/apache/iceberg/issues/2483.</li>
+ *     <li>Columns with constant values are physically encoded as a dictionary. The Arrow vector
+ *     type is int32 instead of the type as per the schema.
+ *     See https://github.com/apache/iceberg/issues/2484.</li>
+ *     <li>Data types: {@link Types.TimeType}, {@link Types.ListType}, {@link Types.MapType},
+ *     {@link Types.StructType}, {@link Types.UUIDType}, {@link Types.FixedType} and
+ *     {@link Types.DecimalType}
+ *     See https://github.com/apache/iceberg/issues/2485 and https://github.com/apache/iceberg/issues/2486.</li>
+ *     <li>Iceberg v2 spec is not supported.
+ *     See https://github.com/apache/iceberg/issues/2487.</li>
+ * </ul>
+ */
+public class ArrowReader extends CloseableGroup {
+
+  private final Schema schema;
+  private final FileIO io;
+  private final EncryptionManager encryption;
+  private final int batchSize;
+  private final boolean reuseContainers;
+
+  /**
+   * Create a new instance of the reader.
+   *
+   * @param scan the table scan object.
+   * @param batchSize the maximum number of rows per Arrow batch.
+   * @param reuseContainers whether to reuse Arrow vectors when iterating through the data.
+   *                        If set to {@code false}, every {@link Iterator#next()} call creates
+   *                        new instances of Arrow vectors.
+   *                        If set to {@code true}, the Arrow vectors in the previous
+   *                        {@link Iterator#next()} may be reused for the data returned
+   *                        in the current {@link Iterator#next()}.
+   *                        This option avoids allocating memory again and again.
+   *                        Irrespective of the value of {@code reuseContainers}, the Arrow vectors
+   *                        in the previous {@link Iterator#next()} call are closed before creating
+   *                        new instances if the current {@link Iterator#next()}.
+   */
+  public ArrowReader(TableScan scan, int batchSize, boolean reuseContainers) {
+    this.schema = scan.schema();
+    this.io = scan.table().io();
+    this.encryption = scan.table().encryption();
+    this.batchSize = batchSize;
+    // start planning tasks in the background
+    this.reuseContainers = reuseContainers;
+  }
+
+  /**
+   * Returns a new iterator of {@link ColumnarBatch} objects.
+   * <p>
+   * Note that the reader owns the {@link ColumnarBatch} objects and takes care of closing them.
+   * The caller should not hold onto a {@link ColumnarBatch} or try to close them.
+   *
+   * <p>If {@code reuseContainers} is {@code false}, the Arrow vectors in the
+   * previous {@link ColumnarBatch} are closed before returning the next {@link ColumnarBatch} object.
+   * This implies that the caller should either use the {@link ColumnarBatch} or transfer the ownership of
+   * {@link ColumnarBatch} before getting the next {@link ColumnarBatch}.
+   *
+   * <p>If {@code reuseContainers} is {@code true}, the Arrow vectors in the
+   * previous {@link ColumnarBatch} may be reused for the next {@link ColumnarBatch}.
+   * This implies that the caller should either use the {@link ColumnarBatch} or deep copy the
+   * {@link ColumnarBatch} before getting the next {@link ColumnarBatch}.
+   */
+  public CloseableIterator<ColumnarBatch> open(CloseableIterable<CombinedScanTask> tasks) {
+    CloseableIterator<ColumnarBatch> itr = new VectorizedCombinedScanIterator(
+        tasks,
+        schema,
+        null,
+        io,
+        encryption,
+        true,
+        batchSize,
+        reuseContainers
+    );
+    addCloseable(itr);
+    return itr;
+  }
+
+  @Override
+  public void close() throws IOException {
+    super.close(); // close data files
+  }
+
+  /**
+   * Reads the data file and returns an iterator of {@link VectorSchemaRoot}.
+   * Only Parquet data file format is supported.
+   */
+  private static final class VectorizedCombinedScanIterator implements CloseableIterator<ColumnarBatch> {
+
+    private final List<FileScanTask> fileTasks;
+    private final Iterator<FileScanTask> fileItr;
+    private final Map<String, InputFile> inputFiles;
+    private final Schema expectedSchema;
+    private final String nameMapping;
+    private final boolean caseSensitive;
+    private final int batchSize;
+    private final boolean reuseContainers;
+    private CloseableIterator<ColumnarBatch> currentIterator;
+    private ColumnarBatch current;
+    private FileScanTask currentTask;
+
+    /**
+     * Create a new instance.
+     *
+     * @param tasks             Combined file scan tasks.
+     * @param expectedSchema    Read schema. The returned data will have this schema.
+     * @param nameMapping       Mapping from external schema names to Iceberg type IDs.
+     * @param io                File I/O.
+     * @param encryptionManager Encryption manager.
+     * @param caseSensitive     If {@code true}, column names are case sensitive.
+     *                          If {@code false}, column names are not case sensitive.
+     * @param batchSize         Batch size in number of rows. Each Arrow batch contains
+     *                          a maximum of {@code batchSize} rows.
+     * @param reuseContainers   If set to {@code false}, every {@link Iterator#next()} call creates
+     *                          new instances of Arrow vectors.
+     *                          If set to {@code true}, the Arrow vectors in the previous
+     *                          {@link Iterator#next()} may be reused for the data returned
+     *                          in the current {@link Iterator#next()}.
+     *                          This option avoids allocating memory again and again.
+     *                          Irrespective of the value of {@code reuseContainers}, the Arrow vectors
+     *                          in the previous {@link Iterator#next()} call are closed before creating
+     *                          new instances if the current {@link Iterator#next()}.
+     */
+    VectorizedCombinedScanIterator(
+        CloseableIterable<CombinedScanTask> tasks,
+        Schema expectedSchema,
+        String nameMapping,
+        FileIO io,
+        EncryptionManager encryptionManager,
+        boolean caseSensitive,
+        int batchSize,
+        boolean reuseContainers) {

Review comment:
       I will fix it. Is there a coding style guideline for the project? If so, it will be very helpful for the developers to follow.




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#discussion_r614783252



##########
File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowReader.java
##########
@@ -0,0 +1,327 @@
+/*
+ * 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.arrow.vectorized;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import java.util.stream.StreamSupport;
+import org.apache.arrow.vector.NullCheckingForGet;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.arrow.vector.types.Types.MinorType;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.encryption.EncryptedFiles;
+import org.apache.iceberg.encryption.EncryptedInputFile;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.CloseableIterator;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.types.Types;
+import org.apache.parquet.schema.MessageType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Vectorized reader that returns an iterator of {@link ColumnarBatch}.
+ * See {@link #open(CloseableIterable)} ()} to learn about the
+ * behavior of the iterator.
+ *
+ * <p>The following Iceberg data types are supported and have been tested:
+ * <ul>
+ *     <li>Iceberg: {@link Types.BooleanType}, Arrow: {@link MinorType#BIT}</li>
+ *     <li>Iceberg: {@link Types.IntegerType}, Arrow: {@link MinorType#INT}</li>
+ *     <li>Iceberg: {@link Types.LongType}, Arrow: {@link MinorType#BIGINT}</li>
+ *     <li>Iceberg: {@link Types.FloatType}, Arrow: {@link MinorType#FLOAT4}</li>
+ *     <li>Iceberg: {@link Types.DoubleType}, Arrow: {@link MinorType#FLOAT8}</li>
+ *     <li>Iceberg: {@link Types.StringType}, Arrow: {@link MinorType#VARCHAR}</li>
+ *     <li>Iceberg: {@link Types.TimestampType} (both with and without timezone),
+ *         Arrow: {@link MinorType#TIMEMICRO}</li>
+ *     <li>Iceberg: {@link Types.BinaryType}, Arrow: {@link MinorType#VARBINARY}</li>
+ *     <li>Iceberg: {@link Types.DateType}, Arrow: {@link MinorType#DATEDAY}</li>
+ * </ul>
+ *
+ * <p>Features that don't work in this implementation:

Review comment:
       can you raise a set of issues to document limitations and follow ups from this PR? Also is this javadoc still up to date?

##########
File path: arrow/src/test/java/org/apache/iceberg/arrow/vectorized/ArrowReaderTest.java
##########
@@ -0,0 +1,904 @@
+/*
+ * 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.arrow.vectorized;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.time.temporal.ChronoUnit;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+import org.apache.arrow.vector.BigIntVector;
+import org.apache.arrow.vector.BitVector;
+import org.apache.arrow.vector.DateDayVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.Float4Vector;
+import org.apache.arrow.vector.Float8Vector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.TimeStampMicroTZVector;
+import org.apache.arrow.vector.TimeStampMicroVector;
+import org.apache.arrow.vector.VarBinaryVector;
+import org.apache.arrow.vector.VarCharVector;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.arrow.vector.types.Types.MinorType;
+import org.apache.arrow.vector.types.pojo.ArrowType;
+import org.apache.arrow.vector.types.pojo.Field;
+import org.apache.arrow.vector.types.pojo.FieldType;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.OverwriteFiles;
+import org.apache.iceberg.PartitionKey;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.data.parquet.GenericParquetWriter;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.types.Types;
+import org.junit.Ignore;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.Files.localInput;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Test cases for {@link ArrowReader}.
+ * <p>All tests create a table with monthly partitions and write 1 year of data to the table.
+ */
+public class ArrowReaderTest {
+
+  private static final int NUM_ROWS_PER_MONTH = 20;
+  private static final ImmutableList<String> ALL_COLUMNS =
+      ImmutableList.of(
+          "timestamp",
+          "timestamp_nullable",
+          "boolean",
+          "boolean_nullable",
+          "int",
+          "int_nullable",
+          "long",
+          "long_nullable",
+          "float",
+          "float_nullable",
+          "double",
+          "double_nullable",
+          "timestamp_tz",
+          "timestamp_tz_nullable",
+          "string",
+          "string_nullable",
+          "bytes",
+          "bytes_nullable",
+          "date",
+          "date_nullable",
+          "int_promotion"
+      );
+
+  @Rule
+  public final TemporaryFolder temp = new TemporaryFolder();
+
+  private HadoopTables tables;
+
+  private String tableLocation;
+  private List<GenericRecord> rowsWritten;
+
+  /**
+   * Read all rows and columns from the table without any filter. The test asserts that the Arrow {@link
+   * VectorSchemaRoot} contains the expected schema and expected vector types. Then the test asserts that the vectors
+   * contains expected values. The test also asserts the total number of rows match the expected value.
+   */
+  @Test
+  public void testReadAll() throws Exception {
+    writeTableWithIncrementalRecords();
+    Table table = tables.load(tableLocation);
+    readAndCheckVectorSchemaRoots(table.newScan(), NUM_ROWS_PER_MONTH, 12 * NUM_ROWS_PER_MONTH, ALL_COLUMNS);
+  }
+
+  /**
+   * This test writes each partition with constant value rows. The Arrow vectors returned are mostly of type int32
+   * which is unexpected. This is happening because of dictionary encoding at the storage level.
+   * <p>
+   * Following are the expected and actual Arrow schema:
+   * <pre>
+   * Expected Arrow Schema:
+   * timestamp: Timestamp(MICROSECOND, null) not null,
+   * timestamp_nullable: Timestamp(MICROSECOND, null),
+   * boolean: Bool not null,
+   * boolean_nullable: Bool,
+   * int: Int(32, true) not null,
+   * int_nullable: Int(32, true),
+   * long: Int(64, true) not null,
+   * long_nullable: Int(64, true),
+   * float: FloatingPoint(SINGLE) not null,
+   * float_nullable: FloatingPoint(SINGLE),
+   * double: FloatingPoint(DOUBLE) not null,
+   * double_nullable: FloatingPoint(DOUBLE),
+   * timestamp_tz: Timestamp(MICROSECOND, UTC) not null,
+   * timestamp_tz_nullable: Timestamp(MICROSECOND, UTC),
+   * string: Utf8 not null,
+   * string_nullable: Utf8,
+   * bytes: Binary not null,
+   * bytes_nullable: Binary,
+   * date: Date(DAY) not null,
+   * date_nullable: Date(DAY),
+   * int_promotion: Int(32, true) not null
+   *
+   * Actual Arrow Schema:
+   * timestamp: Int(32, true) not null,
+   * timestamp_nullable: Int(32, true),
+   * boolean: Bool not null,
+   * boolean_nullable: Bool,
+   * int: Int(32, true) not null,
+   * int_nullable: Int(32, true),
+   * long: Int(32, true) not null,
+   * long_nullable: Int(32, true),
+   * float: Int(32, true) not null,
+   * float_nullable: Int(32, true),
+   * double: Int(32, true) not null,
+   * double_nullable: Int(32, true),
+   * timestamp_tz: Int(32, true) not null,
+   * timestamp_tz_nullable: Int(32, true),
+   * string: Int(32, true) not null,
+   * string_nullable: Int(32, true),
+   * bytes: Int(32, true) not null,
+   * bytes_nullable: Int(32, true),
+   * date: Date(DAY) not null,
+   * date_nullable: Date(DAY),
+   * int_promotion: Int(32, true) not null
+   * </pre>
+   * <p>
+   * TODO: fix the returned Arrow vectors to have vector types consistent with Iceberg types.
+   * <p>
+   * Read all rows and columns from the table without any filter. The test asserts that the Arrow {@link
+   * VectorSchemaRoot} contains the expected schema and expected vector types. Then the test asserts that the vectors
+   * contains expected values. The test also asserts the total number of rows match the expected value.
+   */
+  @Test
+  @Ignore
+  public void testReadAllWithConstantRecords() throws Exception {
+    writeTableWithConstantRecords();
+    Table table = tables.load(tableLocation);
+    readAndCheckVectorSchemaRoots(table.newScan(), NUM_ROWS_PER_MONTH, 12 * NUM_ROWS_PER_MONTH, ALL_COLUMNS);
+  }
+
+  /**
+   * Read all rows and columns from the table without any filter. The test uses a batch size smaller than the number of
+   * rows in a partition. The test asserts that the Arrow {@link VectorSchemaRoot} contains the expected schema and
+   * expected vector types. Then the test asserts that the vectors contains expected values. The test also asserts the
+   * total number of rows match the expected value.
+   */
+  @Test
+  public void testReadAllWithSmallerBatchSize() throws Exception {
+    writeTableWithIncrementalRecords();
+    Table table = tables.load(tableLocation);
+    TableScan scan = table.newScan();
+    readAndCheckVectorSchemaRoots(scan, 10, 12 * NUM_ROWS_PER_MONTH, ALL_COLUMNS);
+  }
+
+  /**
+   * Read selected rows and all columns from the table using a time range row filter. The test asserts that the Arrow
+   * {@link VectorSchemaRoot} contains the expected schema and expected vector types. Then the test asserts that the
+   * vectors contains expected values. The test also asserts the total number of rows match the expected value.
+   */
+  @Test
+  public void testReadRangeFilter() throws Exception {
+    writeTableWithIncrementalRecords();
+    Table table = tables.load(tableLocation);
+    LocalDateTime beginTime = LocalDateTime.of(2020, 1, 1, 0, 0, 0);
+    LocalDateTime endTime = LocalDateTime.of(2020, 2, 1, 0, 0, 0);
+    TableScan scan = table.newScan()
+        .filter(Expressions.and(
+            Expressions.greaterThanOrEqual("timestamp", timestampToMicros(beginTime)),
+            Expressions.lessThan("timestamp", timestampToMicros(endTime))));
+    readAndCheckVectorSchemaRoots(scan, NUM_ROWS_PER_MONTH, NUM_ROWS_PER_MONTH, ALL_COLUMNS);
+  }
+
+  /**
+   * Read selected rows and all columns from the table using a time range row filter.
+   * The test asserts that the result is empty.
+   */
+  @Test
+  public void testReadRangeFilterEmptyResult() throws Exception {
+    writeTableWithIncrementalRecords();
+    Table table = tables.load(tableLocation);
+    LocalDateTime beginTime = LocalDateTime.of(2021, 1, 1, 0, 0, 0);
+    LocalDateTime endTime = LocalDateTime.of(2021, 2, 1, 0, 0, 0);
+    TableScan scan = table.newScan()
+            .filter(Expressions.and(
+                    Expressions.greaterThanOrEqual("timestamp", timestampToMicros(beginTime)),
+                    Expressions.lessThan("timestamp", timestampToMicros(endTime))));
+    int numRoots = 0;
+    try (VectorizedTableScanIterable itr = new VectorizedTableScanIterable(scan, NUM_ROWS_PER_MONTH, false)) {
+      for (ColumnarBatch batch : itr) {
+        numRoots++;
+      }
+    }
+    assertEquals(0, numRoots);
+  }
+
+  /**
+   * Read all rows and selected columns from the table with a column selection filter. The test asserts that the Arrow
+   * {@link VectorSchemaRoot} contains the expected schema and expected vector types. Then the test asserts that the
+   * vectors contains expected values. The test also asserts the total number of rows match the expected value.
+   */
+  @Test
+  public void testReadColumnFilter1() throws Exception {
+    writeTableWithIncrementalRecords();
+    Table table = tables.load(tableLocation);
+    TableScan scan = table.newScan()
+        .select("timestamp", "int", "string");
+    readAndCheckVectorSchemaRoots(
+        scan, NUM_ROWS_PER_MONTH, 12 * NUM_ROWS_PER_MONTH,
+        ImmutableList.of("timestamp", "int", "string"));
+  }
+
+  /**
+   * Read all rows and a single column from the table with a column selection filter. The test asserts that the Arrow
+   * {@link VectorSchemaRoot} contains the expected schema and expected vector types. Then the test asserts that the
+   * vectors contains expected values. The test also asserts the total number of rows match the expected value.
+   */
+  @Test
+  public void testReadColumnFilter2() throws Exception {
+    writeTableWithIncrementalRecords();
+    Table table = tables.load(tableLocation);
+    TableScan scan = table.newScan()
+        .select("timestamp");
+    readAndCheckVectorSchemaRoots(
+        scan, NUM_ROWS_PER_MONTH, 12 * NUM_ROWS_PER_MONTH,
+        ImmutableList.of("timestamp"));
+  }
+
+  private void readAndCheckVectorSchemaRoots(
+      TableScan scan,
+      int numRowsPerRoot,
+      int expectedTotalRows,
+      List<String> columns) throws IOException {
+    Set<String> columnSet = ImmutableSet.copyOf(columns);
+    int rowIndex = 0;
+    int totalRows = 0;
+    try (VectorizedTableScanIterable itr = new VectorizedTableScanIterable(scan, numRowsPerRoot, false)) {
+      for (ColumnarBatch batch : itr) {
+        List<GenericRecord> expectedRows = rowsWritten.subList(rowIndex, rowIndex + numRowsPerRoot);
+        checkColumnarBatch(numRowsPerRoot, expectedRows, batch, columns);

Review comment:
       nit: can you split this up a bit and add some docs as to why the assertion is happening twice? Just a simple javadoc explaining the two use cases is good




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#discussion_r617404005



##########
File path: build.gradle
##########
@@ -735,9 +735,12 @@ project(':iceberg-arrow') {
       exclude group: 'com.google.code.findbugs', module: 'jsr305'
     }
     compile("org.apache.arrow:arrow-memory-netty") {
-      exclude group: 'io.netty', module: 'netty-common'
       exclude group: 'com.google.code.findbugs', module: 'jsr305'

Review comment:
       how come the netty module was removed. Presumably this will have knock-on effects w/ shaded jars?

##########
File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowReader.java
##########
@@ -0,0 +1,331 @@
+/*
+ * 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.arrow.vectorized;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import java.util.stream.StreamSupport;
+import org.apache.arrow.vector.NullCheckingForGet;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.arrow.vector.types.Types.MinorType;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.encryption.EncryptedFiles;
+import org.apache.iceberg.encryption.EncryptedInputFile;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.CloseableIterator;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.types.Types;
+import org.apache.parquet.schema.MessageType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Vectorized reader that returns an iterator of {@link ColumnarBatch}.
+ * See {@link #open(CloseableIterable)} ()} to learn about the
+ * behavior of the iterator.
+ *
+ * <p>The following Iceberg data types are supported and have been tested:
+ * <ul>
+ *     <li>Iceberg: {@link Types.BooleanType}, Arrow: {@link MinorType#BIT}</li>
+ *     <li>Iceberg: {@link Types.IntegerType}, Arrow: {@link MinorType#INT}</li>
+ *     <li>Iceberg: {@link Types.LongType}, Arrow: {@link MinorType#BIGINT}</li>
+ *     <li>Iceberg: {@link Types.FloatType}, Arrow: {@link MinorType#FLOAT4}</li>
+ *     <li>Iceberg: {@link Types.DoubleType}, Arrow: {@link MinorType#FLOAT8}</li>
+ *     <li>Iceberg: {@link Types.StringType}, Arrow: {@link MinorType#VARCHAR}</li>
+ *     <li>Iceberg: {@link Types.TimestampType} (both with and without timezone),
+ *         Arrow: {@link MinorType#TIMEMICRO}</li>
+ *     <li>Iceberg: {@link Types.BinaryType}, Arrow: {@link MinorType#VARBINARY}</li>
+ *     <li>Iceberg: {@link Types.DateType}, Arrow: {@link MinorType#DATEDAY}</li>
+ * </ul>
+ *
+ * <p>Features that don't work in this implementation:
+ * <ul>
+ *     <li>Type promotion: In case of type promotion, the Arrow vector corresponding to
+ *     the data type in the parquet file is returned instead of the data type in the latest schema.
+ *     See https://github.com/apache/iceberg/issues/2483.</li>
+ *     <li>Columns with constant values are physically encoded as a dictionary. The Arrow vector
+ *     type is int32 instead of the type as per the schema.
+ *     See https://github.com/apache/iceberg/issues/2484.</li>
+ *     <li>Data types: {@link Types.TimeType}, {@link Types.ListType}, {@link Types.MapType},
+ *     {@link Types.StructType}, {@link Types.UUIDType}, {@link Types.FixedType} and
+ *     {@link Types.DecimalType}
+ *     See https://github.com/apache/iceberg/issues/2485 and https://github.com/apache/iceberg/issues/2486.</li>
+ *     <li>Iceberg v2 spec is not supported.
+ *     See https://github.com/apache/iceberg/issues/2487.</li>
+ * </ul>
+ */
+public class ArrowReader extends CloseableGroup {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ArrowReader.class);
+
+  private final Schema schema;
+  private final FileIO io;
+  private final EncryptionManager encryption;
+  private final int batchSize;
+  private final boolean reuseContainers;
+
+  /**
+   * Create a new instance of the reader.
+   *
+   * @param scan the table scan object.
+   * @param batchSize the maximum number of rows per Arrow batch.
+   * @param reuseContainers whether to reuse Arrow vectors when iterating through the data.
+   *                        If set to {@code false}, every {@link Iterator#next()} call creates
+   *                        new instances of Arrow vectors.
+   *                        If set to {@code true}, the Arrow vectors in the previous
+   *                        {@link Iterator#next()} may be reused for the data returned
+   *                        in the current {@link Iterator#next()}.
+   *                        This option avoids allocating memory again and again.
+   *                        Irrespective of the value of {@code reuseContainers}, the Arrow vectors
+   *                        in the previous {@link Iterator#next()} call are closed before creating
+   *                        new instances if the current {@link Iterator#next()}.
+   */
+  public ArrowReader(TableScan scan, int batchSize, boolean reuseContainers) {
+    this.schema = scan.schema();
+    this.io = scan.table().io();
+    this.encryption = scan.table().encryption();
+    this.batchSize = batchSize;
+    // start planning tasks in the background
+    this.reuseContainers = reuseContainers;
+  }
+
+  /**
+   * Returns a new iterator of {@link ColumnarBatch} objects.
+   * <p>
+   * Note that the reader owns the {@link ColumnarBatch} objects and takes care of closing them.
+   * The caller should not hold onto a {@link ColumnarBatch} or try to close them.
+   *
+   * <p>If {@code reuseContainers} is {@code false}, the Arrow vectors in the
+   * previous {@link ColumnarBatch} are closed before returning the next {@link ColumnarBatch} object.
+   * This implies that the caller should either use the {@link ColumnarBatch} or transfer the ownership of
+   * {@link ColumnarBatch} before getting the next {@link ColumnarBatch}.
+   *
+   * <p>If {@code reuseContainers} is {@code true}, the Arrow vectors in the
+   * previous {@link ColumnarBatch} may be reused for the next {@link ColumnarBatch}.
+   * This implies that the caller should either use the {@link ColumnarBatch} or deep copy the
+   * {@link ColumnarBatch} before getting the next {@link ColumnarBatch}.
+   */
+  public CloseableIterator<ColumnarBatch> open(CloseableIterable<CombinedScanTask> tasks) {
+    CloseableIterator<ColumnarBatch> itr = new VectorizedCombinedScanIterator(
+        tasks,
+        schema,
+        null,
+        io,
+        encryption,
+        true,
+        batchSize,
+        reuseContainers
+    );
+    addCloseable(itr);
+    return itr;
+  }
+
+  @Override
+  public void close() throws IOException {
+    super.close(); // close data files
+  }
+
+  /**
+   * Reads the data file and returns an iterator of {@link VectorSchemaRoot}.
+   * Only Parquet data file format is supported.
+   */
+  private static final class VectorizedCombinedScanIterator implements CloseableIterator<ColumnarBatch> {
+
+    private final Iterator<FileScanTask> fileItr;
+    private final Map<String, InputFile> inputFiles;
+    private final Schema expectedSchema;
+    private final String nameMapping;
+    private final boolean caseSensitive;
+    private final int batchSize;
+    private final boolean reuseContainers;
+    private CloseableIterator<ColumnarBatch> currentIterator;
+    private ColumnarBatch current;
+    private FileScanTask currentTask;
+
+    /**
+     * Create a new instance.
+     *
+     * @param tasks             Combined file scan tasks.
+     * @param expectedSchema    Read schema. The returned data will have this schema.
+     * @param nameMapping       Mapping from external schema names to Iceberg type IDs.
+     * @param io                File I/O.
+     * @param encryptionManager Encryption manager.
+     * @param caseSensitive     If {@code true}, column names are case sensitive.
+     *                          If {@code false}, column names are not case sensitive.
+     * @param batchSize         Batch size in number of rows. Each Arrow batch contains
+     *                          a maximum of {@code batchSize} rows.
+     * @param reuseContainers   If set to {@code false}, every {@link Iterator#next()} call creates
+     *                          new instances of Arrow vectors.
+     *                          If set to {@code true}, the Arrow vectors in the previous
+     *                          {@link Iterator#next()} may be reused for the data returned
+     *                          in the current {@link Iterator#next()}.
+     *                          This option avoids allocating memory again and again.
+     *                          Irrespective of the value of {@code reuseContainers}, the Arrow vectors
+     *                          in the previous {@link Iterator#next()} call are closed before creating
+     *                          new instances if the current {@link Iterator#next()}.
+     */
+    VectorizedCombinedScanIterator(
+        CloseableIterable<CombinedScanTask> tasks,
+        Schema expectedSchema,
+        String nameMapping,
+        FileIO io,
+        EncryptionManager encryptionManager,
+        boolean caseSensitive,
+        int batchSize,
+        boolean reuseContainers) {
+      List<FileScanTask> fileTasks = StreamSupport.stream(tasks.spliterator(), false)
+          .map(CombinedScanTask::files)
+          .flatMap(Collection::stream)
+          .collect(Collectors.toList());
+      this.fileItr = fileTasks.iterator();
+      Map<String, ByteBuffer> keyMetadata = Maps.newHashMap();
+      fileTasks.stream()
+          .flatMap(fileScanTask -> Stream.concat(Stream.of(fileScanTask.file()), fileScanTask.deletes().stream()))
+          .forEach(file -> keyMetadata.put(file.path().toString(), file.keyMetadata()));

Review comment:
       rather than allocating this to a `Map` can this be directly transformed into a Stream? 

##########
File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowReader.java
##########
@@ -0,0 +1,331 @@
+/*
+ * 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.arrow.vectorized;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import java.util.stream.StreamSupport;
+import org.apache.arrow.vector.NullCheckingForGet;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.arrow.vector.types.Types.MinorType;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.encryption.EncryptedFiles;
+import org.apache.iceberg.encryption.EncryptedInputFile;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.CloseableIterator;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.types.Types;
+import org.apache.parquet.schema.MessageType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Vectorized reader that returns an iterator of {@link ColumnarBatch}.
+ * See {@link #open(CloseableIterable)} ()} to learn about the
+ * behavior of the iterator.
+ *
+ * <p>The following Iceberg data types are supported and have been tested:
+ * <ul>
+ *     <li>Iceberg: {@link Types.BooleanType}, Arrow: {@link MinorType#BIT}</li>
+ *     <li>Iceberg: {@link Types.IntegerType}, Arrow: {@link MinorType#INT}</li>
+ *     <li>Iceberg: {@link Types.LongType}, Arrow: {@link MinorType#BIGINT}</li>
+ *     <li>Iceberg: {@link Types.FloatType}, Arrow: {@link MinorType#FLOAT4}</li>
+ *     <li>Iceberg: {@link Types.DoubleType}, Arrow: {@link MinorType#FLOAT8}</li>
+ *     <li>Iceberg: {@link Types.StringType}, Arrow: {@link MinorType#VARCHAR}</li>
+ *     <li>Iceberg: {@link Types.TimestampType} (both with and without timezone),
+ *         Arrow: {@link MinorType#TIMEMICRO}</li>
+ *     <li>Iceberg: {@link Types.BinaryType}, Arrow: {@link MinorType#VARBINARY}</li>
+ *     <li>Iceberg: {@link Types.DateType}, Arrow: {@link MinorType#DATEDAY}</li>
+ * </ul>
+ *
+ * <p>Features that don't work in this implementation:
+ * <ul>
+ *     <li>Type promotion: In case of type promotion, the Arrow vector corresponding to
+ *     the data type in the parquet file is returned instead of the data type in the latest schema.
+ *     See https://github.com/apache/iceberg/issues/2483.</li>
+ *     <li>Columns with constant values are physically encoded as a dictionary. The Arrow vector
+ *     type is int32 instead of the type as per the schema.
+ *     See https://github.com/apache/iceberg/issues/2484.</li>
+ *     <li>Data types: {@link Types.TimeType}, {@link Types.ListType}, {@link Types.MapType},
+ *     {@link Types.StructType}, {@link Types.UUIDType}, {@link Types.FixedType} and
+ *     {@link Types.DecimalType}
+ *     See https://github.com/apache/iceberg/issues/2485 and https://github.com/apache/iceberg/issues/2486.</li>
+ *     <li>Iceberg v2 spec is not supported.
+ *     See https://github.com/apache/iceberg/issues/2487.</li>
+ * </ul>
+ */
+public class ArrowReader extends CloseableGroup {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ArrowReader.class);
+
+  private final Schema schema;
+  private final FileIO io;
+  private final EncryptionManager encryption;
+  private final int batchSize;
+  private final boolean reuseContainers;
+
+  /**
+   * Create a new instance of the reader.
+   *
+   * @param scan the table scan object.
+   * @param batchSize the maximum number of rows per Arrow batch.
+   * @param reuseContainers whether to reuse Arrow vectors when iterating through the data.
+   *                        If set to {@code false}, every {@link Iterator#next()} call creates
+   *                        new instances of Arrow vectors.
+   *                        If set to {@code true}, the Arrow vectors in the previous
+   *                        {@link Iterator#next()} may be reused for the data returned
+   *                        in the current {@link Iterator#next()}.
+   *                        This option avoids allocating memory again and again.
+   *                        Irrespective of the value of {@code reuseContainers}, the Arrow vectors
+   *                        in the previous {@link Iterator#next()} call are closed before creating
+   *                        new instances if the current {@link Iterator#next()}.
+   */
+  public ArrowReader(TableScan scan, int batchSize, boolean reuseContainers) {
+    this.schema = scan.schema();
+    this.io = scan.table().io();
+    this.encryption = scan.table().encryption();
+    this.batchSize = batchSize;
+    // start planning tasks in the background
+    this.reuseContainers = reuseContainers;
+  }
+
+  /**
+   * Returns a new iterator of {@link ColumnarBatch} objects.
+   * <p>
+   * Note that the reader owns the {@link ColumnarBatch} objects and takes care of closing them.
+   * The caller should not hold onto a {@link ColumnarBatch} or try to close them.
+   *
+   * <p>If {@code reuseContainers} is {@code false}, the Arrow vectors in the
+   * previous {@link ColumnarBatch} are closed before returning the next {@link ColumnarBatch} object.
+   * This implies that the caller should either use the {@link ColumnarBatch} or transfer the ownership of
+   * {@link ColumnarBatch} before getting the next {@link ColumnarBatch}.
+   *
+   * <p>If {@code reuseContainers} is {@code true}, the Arrow vectors in the
+   * previous {@link ColumnarBatch} may be reused for the next {@link ColumnarBatch}.
+   * This implies that the caller should either use the {@link ColumnarBatch} or deep copy the
+   * {@link ColumnarBatch} before getting the next {@link ColumnarBatch}.
+   */
+  public CloseableIterator<ColumnarBatch> open(CloseableIterable<CombinedScanTask> tasks) {
+    CloseableIterator<ColumnarBatch> itr = new VectorizedCombinedScanIterator(
+        tasks,
+        schema,
+        null,
+        io,
+        encryption,
+        true,
+        batchSize,
+        reuseContainers
+    );
+    addCloseable(itr);
+    return itr;
+  }
+
+  @Override
+  public void close() throws IOException {
+    super.close(); // close data files
+  }
+
+  /**
+   * Reads the data file and returns an iterator of {@link VectorSchemaRoot}.
+   * Only Parquet data file format is supported.
+   */
+  private static final class VectorizedCombinedScanIterator implements CloseableIterator<ColumnarBatch> {
+
+    private final Iterator<FileScanTask> fileItr;
+    private final Map<String, InputFile> inputFiles;
+    private final Schema expectedSchema;
+    private final String nameMapping;
+    private final boolean caseSensitive;
+    private final int batchSize;
+    private final boolean reuseContainers;
+    private CloseableIterator<ColumnarBatch> currentIterator;
+    private ColumnarBatch current;
+    private FileScanTask currentTask;
+
+    /**
+     * Create a new instance.
+     *
+     * @param tasks             Combined file scan tasks.
+     * @param expectedSchema    Read schema. The returned data will have this schema.
+     * @param nameMapping       Mapping from external schema names to Iceberg type IDs.
+     * @param io                File I/O.
+     * @param encryptionManager Encryption manager.
+     * @param caseSensitive     If {@code true}, column names are case sensitive.
+     *                          If {@code false}, column names are not case sensitive.
+     * @param batchSize         Batch size in number of rows. Each Arrow batch contains
+     *                          a maximum of {@code batchSize} rows.
+     * @param reuseContainers   If set to {@code false}, every {@link Iterator#next()} call creates
+     *                          new instances of Arrow vectors.
+     *                          If set to {@code true}, the Arrow vectors in the previous
+     *                          {@link Iterator#next()} may be reused for the data returned
+     *                          in the current {@link Iterator#next()}.
+     *                          This option avoids allocating memory again and again.
+     *                          Irrespective of the value of {@code reuseContainers}, the Arrow vectors
+     *                          in the previous {@link Iterator#next()} call are closed before creating
+     *                          new instances if the current {@link Iterator#next()}.
+     */
+    VectorizedCombinedScanIterator(
+        CloseableIterable<CombinedScanTask> tasks,
+        Schema expectedSchema,
+        String nameMapping,
+        FileIO io,
+        EncryptionManager encryptionManager,
+        boolean caseSensitive,
+        int batchSize,
+        boolean reuseContainers) {
+      List<FileScanTask> fileTasks = StreamSupport.stream(tasks.spliterator(), false)
+          .map(CombinedScanTask::files)
+          .flatMap(Collection::stream)
+          .collect(Collectors.toList());
+      this.fileItr = fileTasks.iterator();
+      Map<String, ByteBuffer> keyMetadata = Maps.newHashMap();
+      fileTasks.stream()
+          .flatMap(fileScanTask -> Stream.concat(Stream.of(fileScanTask.file()), fileScanTask.deletes().stream()))
+          .forEach(file -> keyMetadata.put(file.path().toString(), file.keyMetadata()));
+      Stream<EncryptedInputFile> encrypted = keyMetadata.entrySet().stream()
+          .map(entry -> EncryptedFiles.encryptedInput(io.newInputFile(entry.getKey()), entry.getValue()));
+
+      // decrypt with the batch call to avoid multiple RPCs to a key server, if possible
+      Iterable<InputFile> decryptedFiles = encryptionManager.decrypt(encrypted::iterator);
+
+      Map<String, InputFile> files = Maps.newHashMapWithExpectedSize(fileTasks.size());

Review comment:
       similarily this looks like it could just use a `Collector.toMap`

##########
File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowBatchReader.java
##########
@@ -0,0 +1,106 @@
+/*
+ * 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.arrow.vectorized;
+
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.parquet.VectorizedReader;
+import org.apache.parquet.Preconditions;
+import org.apache.parquet.column.page.PageReadStore;
+import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
+import org.apache.parquet.hadoop.metadata.ColumnPath;
+
+/**
+ * A collection of vectorized readers per column (in the expected read schema) and Arrow Vector holders. This class owns
+ * the Arrow vectors and is responsible for closing the Arrow vectors.
+ */
+class ArrowBatchReader implements VectorizedReader<ColumnarBatch> {
+
+  private final VectorizedArrowReader[] readers;
+  private final VectorHolder[] vectorHolders;
+
+  ArrowBatchReader(List<VectorizedReader<?>> readers) {
+    this.readers = readers.stream()
+        .map(VectorizedArrowReader.class::cast)
+        .toArray(VectorizedArrowReader[]::new);
+    this.vectorHolders = new VectorHolder[readers.size()];
+  }
+
+  @Override
+  public final void setRowGroupInfo(
+      PageReadStore pageStore, Map<ColumnPath, ColumnChunkMetaData> metaData, long rowPosition) {
+    for (VectorizedArrowReader reader : readers) {

Review comment:
       this implies the reader could be `null`? How/why would that happen? 

##########
File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowReader.java
##########
@@ -0,0 +1,331 @@
+/*
+ * 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.arrow.vectorized;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import java.util.stream.StreamSupport;
+import org.apache.arrow.vector.NullCheckingForGet;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.arrow.vector.types.Types.MinorType;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.encryption.EncryptedFiles;
+import org.apache.iceberg.encryption.EncryptedInputFile;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.CloseableIterator;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.types.Types;
+import org.apache.parquet.schema.MessageType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Vectorized reader that returns an iterator of {@link ColumnarBatch}.
+ * See {@link #open(CloseableIterable)} ()} to learn about the
+ * behavior of the iterator.
+ *
+ * <p>The following Iceberg data types are supported and have been tested:
+ * <ul>
+ *     <li>Iceberg: {@link Types.BooleanType}, Arrow: {@link MinorType#BIT}</li>
+ *     <li>Iceberg: {@link Types.IntegerType}, Arrow: {@link MinorType#INT}</li>
+ *     <li>Iceberg: {@link Types.LongType}, Arrow: {@link MinorType#BIGINT}</li>
+ *     <li>Iceberg: {@link Types.FloatType}, Arrow: {@link MinorType#FLOAT4}</li>
+ *     <li>Iceberg: {@link Types.DoubleType}, Arrow: {@link MinorType#FLOAT8}</li>
+ *     <li>Iceberg: {@link Types.StringType}, Arrow: {@link MinorType#VARCHAR}</li>
+ *     <li>Iceberg: {@link Types.TimestampType} (both with and without timezone),
+ *         Arrow: {@link MinorType#TIMEMICRO}</li>
+ *     <li>Iceberg: {@link Types.BinaryType}, Arrow: {@link MinorType#VARBINARY}</li>
+ *     <li>Iceberg: {@link Types.DateType}, Arrow: {@link MinorType#DATEDAY}</li>
+ * </ul>
+ *
+ * <p>Features that don't work in this implementation:
+ * <ul>
+ *     <li>Type promotion: In case of type promotion, the Arrow vector corresponding to
+ *     the data type in the parquet file is returned instead of the data type in the latest schema.
+ *     See https://github.com/apache/iceberg/issues/2483.</li>
+ *     <li>Columns with constant values are physically encoded as a dictionary. The Arrow vector
+ *     type is int32 instead of the type as per the schema.
+ *     See https://github.com/apache/iceberg/issues/2484.</li>
+ *     <li>Data types: {@link Types.TimeType}, {@link Types.ListType}, {@link Types.MapType},
+ *     {@link Types.StructType}, {@link Types.UUIDType}, {@link Types.FixedType} and
+ *     {@link Types.DecimalType}
+ *     See https://github.com/apache/iceberg/issues/2485 and https://github.com/apache/iceberg/issues/2486.</li>
+ *     <li>Iceberg v2 spec is not supported.
+ *     See https://github.com/apache/iceberg/issues/2487.</li>
+ * </ul>
+ */
+public class ArrowReader extends CloseableGroup {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ArrowReader.class);
+
+  private final Schema schema;
+  private final FileIO io;
+  private final EncryptionManager encryption;
+  private final int batchSize;
+  private final boolean reuseContainers;
+
+  /**
+   * Create a new instance of the reader.
+   *
+   * @param scan the table scan object.
+   * @param batchSize the maximum number of rows per Arrow batch.
+   * @param reuseContainers whether to reuse Arrow vectors when iterating through the data.
+   *                        If set to {@code false}, every {@link Iterator#next()} call creates
+   *                        new instances of Arrow vectors.
+   *                        If set to {@code true}, the Arrow vectors in the previous
+   *                        {@link Iterator#next()} may be reused for the data returned
+   *                        in the current {@link Iterator#next()}.
+   *                        This option avoids allocating memory again and again.
+   *                        Irrespective of the value of {@code reuseContainers}, the Arrow vectors
+   *                        in the previous {@link Iterator#next()} call are closed before creating
+   *                        new instances if the current {@link Iterator#next()}.
+   */
+  public ArrowReader(TableScan scan, int batchSize, boolean reuseContainers) {
+    this.schema = scan.schema();
+    this.io = scan.table().io();
+    this.encryption = scan.table().encryption();
+    this.batchSize = batchSize;
+    // start planning tasks in the background
+    this.reuseContainers = reuseContainers;
+  }
+
+  /**
+   * Returns a new iterator of {@link ColumnarBatch} objects.
+   * <p>
+   * Note that the reader owns the {@link ColumnarBatch} objects and takes care of closing them.
+   * The caller should not hold onto a {@link ColumnarBatch} or try to close them.
+   *
+   * <p>If {@code reuseContainers} is {@code false}, the Arrow vectors in the
+   * previous {@link ColumnarBatch} are closed before returning the next {@link ColumnarBatch} object.
+   * This implies that the caller should either use the {@link ColumnarBatch} or transfer the ownership of
+   * {@link ColumnarBatch} before getting the next {@link ColumnarBatch}.
+   *
+   * <p>If {@code reuseContainers} is {@code true}, the Arrow vectors in the
+   * previous {@link ColumnarBatch} may be reused for the next {@link ColumnarBatch}.
+   * This implies that the caller should either use the {@link ColumnarBatch} or deep copy the
+   * {@link ColumnarBatch} before getting the next {@link ColumnarBatch}.
+   */
+  public CloseableIterator<ColumnarBatch> open(CloseableIterable<CombinedScanTask> tasks) {
+    CloseableIterator<ColumnarBatch> itr = new VectorizedCombinedScanIterator(
+        tasks,
+        schema,
+        null,
+        io,
+        encryption,
+        true,
+        batchSize,
+        reuseContainers
+    );
+    addCloseable(itr);
+    return itr;
+  }
+
+  @Override
+  public void close() throws IOException {
+    super.close(); // close data files
+  }
+
+  /**
+   * Reads the data file and returns an iterator of {@link VectorSchemaRoot}.
+   * Only Parquet data file format is supported.
+   */
+  private static final class VectorizedCombinedScanIterator implements CloseableIterator<ColumnarBatch> {
+
+    private final Iterator<FileScanTask> fileItr;
+    private final Map<String, InputFile> inputFiles;
+    private final Schema expectedSchema;
+    private final String nameMapping;
+    private final boolean caseSensitive;
+    private final int batchSize;
+    private final boolean reuseContainers;
+    private CloseableIterator<ColumnarBatch> currentIterator;
+    private ColumnarBatch current;
+    private FileScanTask currentTask;
+
+    /**
+     * Create a new instance.
+     *
+     * @param tasks             Combined file scan tasks.
+     * @param expectedSchema    Read schema. The returned data will have this schema.
+     * @param nameMapping       Mapping from external schema names to Iceberg type IDs.
+     * @param io                File I/O.
+     * @param encryptionManager Encryption manager.
+     * @param caseSensitive     If {@code true}, column names are case sensitive.
+     *                          If {@code false}, column names are not case sensitive.
+     * @param batchSize         Batch size in number of rows. Each Arrow batch contains
+     *                          a maximum of {@code batchSize} rows.
+     * @param reuseContainers   If set to {@code false}, every {@link Iterator#next()} call creates
+     *                          new instances of Arrow vectors.
+     *                          If set to {@code true}, the Arrow vectors in the previous
+     *                          {@link Iterator#next()} may be reused for the data returned
+     *                          in the current {@link Iterator#next()}.
+     *                          This option avoids allocating memory again and again.
+     *                          Irrespective of the value of {@code reuseContainers}, the Arrow vectors
+     *                          in the previous {@link Iterator#next()} call are closed before creating
+     *                          new instances if the current {@link Iterator#next()}.
+     */
+    VectorizedCombinedScanIterator(
+        CloseableIterable<CombinedScanTask> tasks,
+        Schema expectedSchema,
+        String nameMapping,
+        FileIO io,
+        EncryptionManager encryptionManager,
+        boolean caseSensitive,
+        int batchSize,
+        boolean reuseContainers) {
+      List<FileScanTask> fileTasks = StreamSupport.stream(tasks.spliterator(), false)
+          .map(CombinedScanTask::files)
+          .flatMap(Collection::stream)
+          .collect(Collectors.toList());
+      this.fileItr = fileTasks.iterator();
+      Map<String, ByteBuffer> keyMetadata = Maps.newHashMap();
+      fileTasks.stream()
+          .flatMap(fileScanTask -> Stream.concat(Stream.of(fileScanTask.file()), fileScanTask.deletes().stream()))
+          .forEach(file -> keyMetadata.put(file.path().toString(), file.keyMetadata()));
+      Stream<EncryptedInputFile> encrypted = keyMetadata.entrySet().stream()
+          .map(entry -> EncryptedFiles.encryptedInput(io.newInputFile(entry.getKey()), entry.getValue()));
+
+      // decrypt with the batch call to avoid multiple RPCs to a key server, if possible
+      Iterable<InputFile> decryptedFiles = encryptionManager.decrypt(encrypted::iterator);
+
+      Map<String, InputFile> files = Maps.newHashMapWithExpectedSize(fileTasks.size());
+      decryptedFiles.forEach(decrypted -> files.putIfAbsent(decrypted.location(), decrypted));
+      this.inputFiles = Collections.unmodifiableMap(files);
+
+      this.currentIterator = CloseableIterator.empty();
+      this.expectedSchema = expectedSchema;
+      this.nameMapping = nameMapping;
+      this.caseSensitive = caseSensitive;
+      this.batchSize = batchSize;
+      this.reuseContainers = reuseContainers;
+    }
+
+    @Override
+    public boolean hasNext() {
+      try {
+        while (true) {
+          if (currentIterator.hasNext()) {
+            this.current = currentIterator.next();
+            return true;
+          } else if (fileItr.hasNext()) {
+            this.currentIterator.close();
+            this.currentTask = fileItr.next();
+            this.currentIterator = open(currentTask);
+          } else {
+            this.currentIterator.close();
+            return false;
+          }
+        }
+      } catch (IOException | RuntimeException e) {
+        if (currentTask != null && !currentTask.isDataTask()) {
+          LOG.error("Error reading file: {}", getInputFile(currentTask).location(), e);

Review comment:
       Can the error message be of the form "Cannot do x. reason. fix if exists

##########
File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowReader.java
##########
@@ -0,0 +1,331 @@
+/*
+ * 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.arrow.vectorized;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import java.util.stream.StreamSupport;
+import org.apache.arrow.vector.NullCheckingForGet;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.arrow.vector.types.Types.MinorType;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.encryption.EncryptedFiles;
+import org.apache.iceberg.encryption.EncryptedInputFile;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.CloseableIterator;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.types.Types;
+import org.apache.parquet.schema.MessageType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Vectorized reader that returns an iterator of {@link ColumnarBatch}.
+ * See {@link #open(CloseableIterable)} ()} to learn about the
+ * behavior of the iterator.
+ *
+ * <p>The following Iceberg data types are supported and have been tested:
+ * <ul>
+ *     <li>Iceberg: {@link Types.BooleanType}, Arrow: {@link MinorType#BIT}</li>
+ *     <li>Iceberg: {@link Types.IntegerType}, Arrow: {@link MinorType#INT}</li>
+ *     <li>Iceberg: {@link Types.LongType}, Arrow: {@link MinorType#BIGINT}</li>
+ *     <li>Iceberg: {@link Types.FloatType}, Arrow: {@link MinorType#FLOAT4}</li>
+ *     <li>Iceberg: {@link Types.DoubleType}, Arrow: {@link MinorType#FLOAT8}</li>
+ *     <li>Iceberg: {@link Types.StringType}, Arrow: {@link MinorType#VARCHAR}</li>
+ *     <li>Iceberg: {@link Types.TimestampType} (both with and without timezone),
+ *         Arrow: {@link MinorType#TIMEMICRO}</li>
+ *     <li>Iceberg: {@link Types.BinaryType}, Arrow: {@link MinorType#VARBINARY}</li>
+ *     <li>Iceberg: {@link Types.DateType}, Arrow: {@link MinorType#DATEDAY}</li>
+ * </ul>
+ *
+ * <p>Features that don't work in this implementation:
+ * <ul>
+ *     <li>Type promotion: In case of type promotion, the Arrow vector corresponding to
+ *     the data type in the parquet file is returned instead of the data type in the latest schema.
+ *     See https://github.com/apache/iceberg/issues/2483.</li>
+ *     <li>Columns with constant values are physically encoded as a dictionary. The Arrow vector
+ *     type is int32 instead of the type as per the schema.
+ *     See https://github.com/apache/iceberg/issues/2484.</li>
+ *     <li>Data types: {@link Types.TimeType}, {@link Types.ListType}, {@link Types.MapType},
+ *     {@link Types.StructType}, {@link Types.UUIDType}, {@link Types.FixedType} and
+ *     {@link Types.DecimalType}
+ *     See https://github.com/apache/iceberg/issues/2485 and https://github.com/apache/iceberg/issues/2486.</li>
+ *     <li>Iceberg v2 spec is not supported.
+ *     See https://github.com/apache/iceberg/issues/2487.</li>
+ * </ul>
+ */
+public class ArrowReader extends CloseableGroup {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ArrowReader.class);
+
+  private final Schema schema;
+  private final FileIO io;
+  private final EncryptionManager encryption;
+  private final int batchSize;
+  private final boolean reuseContainers;
+
+  /**
+   * Create a new instance of the reader.
+   *
+   * @param scan the table scan object.
+   * @param batchSize the maximum number of rows per Arrow batch.
+   * @param reuseContainers whether to reuse Arrow vectors when iterating through the data.
+   *                        If set to {@code false}, every {@link Iterator#next()} call creates
+   *                        new instances of Arrow vectors.
+   *                        If set to {@code true}, the Arrow vectors in the previous
+   *                        {@link Iterator#next()} may be reused for the data returned
+   *                        in the current {@link Iterator#next()}.
+   *                        This option avoids allocating memory again and again.
+   *                        Irrespective of the value of {@code reuseContainers}, the Arrow vectors
+   *                        in the previous {@link Iterator#next()} call are closed before creating
+   *                        new instances if the current {@link Iterator#next()}.
+   */
+  public ArrowReader(TableScan scan, int batchSize, boolean reuseContainers) {
+    this.schema = scan.schema();
+    this.io = scan.table().io();
+    this.encryption = scan.table().encryption();
+    this.batchSize = batchSize;
+    // start planning tasks in the background
+    this.reuseContainers = reuseContainers;
+  }
+
+  /**
+   * Returns a new iterator of {@link ColumnarBatch} objects.
+   * <p>
+   * Note that the reader owns the {@link ColumnarBatch} objects and takes care of closing them.
+   * The caller should not hold onto a {@link ColumnarBatch} or try to close them.
+   *
+   * <p>If {@code reuseContainers} is {@code false}, the Arrow vectors in the
+   * previous {@link ColumnarBatch} are closed before returning the next {@link ColumnarBatch} object.
+   * This implies that the caller should either use the {@link ColumnarBatch} or transfer the ownership of
+   * {@link ColumnarBatch} before getting the next {@link ColumnarBatch}.
+   *
+   * <p>If {@code reuseContainers} is {@code true}, the Arrow vectors in the
+   * previous {@link ColumnarBatch} may be reused for the next {@link ColumnarBatch}.
+   * This implies that the caller should either use the {@link ColumnarBatch} or deep copy the
+   * {@link ColumnarBatch} before getting the next {@link ColumnarBatch}.
+   */
+  public CloseableIterator<ColumnarBatch> open(CloseableIterable<CombinedScanTask> tasks) {
+    CloseableIterator<ColumnarBatch> itr = new VectorizedCombinedScanIterator(
+        tasks,
+        schema,
+        null,
+        io,
+        encryption,
+        true,
+        batchSize,
+        reuseContainers
+    );
+    addCloseable(itr);
+    return itr;
+  }
+
+  @Override
+  public void close() throws IOException {
+    super.close(); // close data files
+  }
+
+  /**
+   * Reads the data file and returns an iterator of {@link VectorSchemaRoot}.
+   * Only Parquet data file format is supported.
+   */
+  private static final class VectorizedCombinedScanIterator implements CloseableIterator<ColumnarBatch> {
+
+    private final Iterator<FileScanTask> fileItr;
+    private final Map<String, InputFile> inputFiles;
+    private final Schema expectedSchema;
+    private final String nameMapping;
+    private final boolean caseSensitive;
+    private final int batchSize;
+    private final boolean reuseContainers;
+    private CloseableIterator<ColumnarBatch> currentIterator;
+    private ColumnarBatch current;
+    private FileScanTask currentTask;
+
+    /**
+     * Create a new instance.
+     *
+     * @param tasks             Combined file scan tasks.
+     * @param expectedSchema    Read schema. The returned data will have this schema.
+     * @param nameMapping       Mapping from external schema names to Iceberg type IDs.
+     * @param io                File I/O.
+     * @param encryptionManager Encryption manager.
+     * @param caseSensitive     If {@code true}, column names are case sensitive.
+     *                          If {@code false}, column names are not case sensitive.
+     * @param batchSize         Batch size in number of rows. Each Arrow batch contains
+     *                          a maximum of {@code batchSize} rows.
+     * @param reuseContainers   If set to {@code false}, every {@link Iterator#next()} call creates
+     *                          new instances of Arrow vectors.
+     *                          If set to {@code true}, the Arrow vectors in the previous
+     *                          {@link Iterator#next()} may be reused for the data returned
+     *                          in the current {@link Iterator#next()}.
+     *                          This option avoids allocating memory again and again.
+     *                          Irrespective of the value of {@code reuseContainers}, the Arrow vectors
+     *                          in the previous {@link Iterator#next()} call are closed before creating
+     *                          new instances if the current {@link Iterator#next()}.
+     */
+    VectorizedCombinedScanIterator(
+        CloseableIterable<CombinedScanTask> tasks,
+        Schema expectedSchema,
+        String nameMapping,
+        FileIO io,
+        EncryptionManager encryptionManager,
+        boolean caseSensitive,
+        int batchSize,
+        boolean reuseContainers) {
+      List<FileScanTask> fileTasks = StreamSupport.stream(tasks.spliterator(), false)
+          .map(CombinedScanTask::files)
+          .flatMap(Collection::stream)
+          .collect(Collectors.toList());
+      this.fileItr = fileTasks.iterator();
+      Map<String, ByteBuffer> keyMetadata = Maps.newHashMap();
+      fileTasks.stream()
+          .flatMap(fileScanTask -> Stream.concat(Stream.of(fileScanTask.file()), fileScanTask.deletes().stream()))
+          .forEach(file -> keyMetadata.put(file.path().toString(), file.keyMetadata()));
+      Stream<EncryptedInputFile> encrypted = keyMetadata.entrySet().stream()
+          .map(entry -> EncryptedFiles.encryptedInput(io.newInputFile(entry.getKey()), entry.getValue()));
+
+      // decrypt with the batch call to avoid multiple RPCs to a key server, if possible
+      Iterable<InputFile> decryptedFiles = encryptionManager.decrypt(encrypted::iterator);
+
+      Map<String, InputFile> files = Maps.newHashMapWithExpectedSize(fileTasks.size());
+      decryptedFiles.forEach(decrypted -> files.putIfAbsent(decrypted.location(), decrypted));
+      this.inputFiles = Collections.unmodifiableMap(files);
+
+      this.currentIterator = CloseableIterator.empty();
+      this.expectedSchema = expectedSchema;
+      this.nameMapping = nameMapping;
+      this.caseSensitive = caseSensitive;
+      this.batchSize = batchSize;
+      this.reuseContainers = reuseContainers;
+    }
+
+    @Override
+    public boolean hasNext() {
+      try {
+        while (true) {
+          if (currentIterator.hasNext()) {
+            this.current = currentIterator.next();
+            return true;
+          } else if (fileItr.hasNext()) {
+            this.currentIterator.close();
+            this.currentTask = fileItr.next();
+            this.currentIterator = open(currentTask);
+          } else {
+            this.currentIterator.close();
+            return false;
+          }
+        }
+      } catch (IOException | RuntimeException e) {
+        if (currentTask != null && !currentTask.isDataTask()) {
+          LOG.error("Error reading file: {}", getInputFile(currentTask).location(), e);
+        }
+        throw new RuntimeException(e);

Review comment:
       can you add a message/more context to this exception

##########
File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowVectorAccessors.java
##########
@@ -0,0 +1,71 @@
+/*
+ * 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.arrow.vectorized;
+
+import java.nio.charset.StandardCharsets;
+import java.util.function.Supplier;
+import org.apache.arrow.vector.VarCharVector;
+import org.apache.iceberg.arrow.vectorized.GenericArrowVectorAccessorFactory.StringFactory;
+
+final class ArrowVectorAccessors {
+
+  private static final GenericArrowVectorAccessorFactory<?, String, ?, ?> factory;
+
+  static {
+    factory = new GenericArrowVectorAccessorFactory<>(
+        throwingSupplier("Decimal type is not supported"),
+        JavaStringFactory::new,
+        throwingSupplier("Struct type is not supported"),
+        throwingSupplier("List type is not supported")
+    );
+  }
+
+  private static <T> Supplier<T> throwingSupplier(String message) {
+    return () -> {
+      throw new UnsupportedOperationException(message);
+    };
+  }
+
+  private ArrowVectorAccessors() {
+    throw new UnsupportedOperationException();

Review comment:
       log message

##########
File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/GenericArrowVectorAccessorFactory.java
##########
@@ -0,0 +1,693 @@
+/*
+ * 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.arrow.vectorized;
+
+import java.lang.reflect.Array;
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.util.function.IntFunction;
+import java.util.function.Supplier;
+import java.util.stream.IntStream;
+import org.apache.arrow.vector.BigIntVector;
+import org.apache.arrow.vector.BitVector;
+import org.apache.arrow.vector.DateDayVector;
+import org.apache.arrow.vector.DecimalVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.Float4Vector;
+import org.apache.arrow.vector.Float8Vector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.TimeStampMicroTZVector;
+import org.apache.arrow.vector.TimeStampMicroVector;
+import org.apache.arrow.vector.ValueVector;
+import org.apache.arrow.vector.VarBinaryVector;
+import org.apache.arrow.vector.VarCharVector;
+import org.apache.arrow.vector.complex.ListVector;
+import org.apache.arrow.vector.complex.StructVector;
+import org.apache.arrow.vector.util.DecimalUtility;
+import org.apache.parquet.Preconditions;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.PrimitiveType;
+
+/**
+ * This class is creates typed {@link ArrowVectorAccessor} from {@link VectorHolder}.
+ * It provides a generic implementation for following Arrow types:
+ * <ul>
+ *   <li>Decimal type can be deserialized to a type that supports decimal,
+ *   e.g. BigDecimal or Spark's Decimal.</li>
+ *   <li>UTF8 String type can deserialized to a Java String or Spark's UTF8String.</li>
+ *   <li>List type: the child elements of a list can be deserialized to Spark's ColumnarArray or similar type.</li>
+ *   <li>Struct type: the child elements of a struct can be deserialized to a Spark's ArrowColumnVector
+ *   or similar type.</li>
+ * </ul>
+ * @param <DecimalT> A concrete type that can represent a decimal.
+ * @param <Utf8StringT> A concrete type that can represent a UTF8 string.
+ * @param <ArrayT> A concrete type that can represent an array value in a list vector, e.g. Spark's ColumnarArray.
+ * @param <ChildVectorT> A concrete type that can represent a child vector in a struct, e.g. Spark's ArrowColumnVector.
+ */
+public class GenericArrowVectorAccessorFactory<DecimalT, Utf8StringT, ArrayT, ChildVectorT extends AutoCloseable> {
+
+  private final Supplier<DecimalFactory<DecimalT>> decimalFactorySupplier;
+  private final Supplier<StringFactory<Utf8StringT>> stringFactorySupplier;
+  private final Supplier<StructChildFactory<ChildVectorT>> structChildFactorySupplier;
+  private final Supplier<ArrayFactory<ChildVectorT, ArrayT>> arrayFactorySupplier;
+
+  /**
+   * The constructor is parameterized using the decimal, string, struct and array factories.
+   * If a specific type is not supported, the factory supplier can raise an
+   * {@link UnsupportedOperationException}.
+   */
+  protected GenericArrowVectorAccessorFactory(
+          Supplier<DecimalFactory<DecimalT>> decimalFactorySupplier,
+          Supplier<StringFactory<Utf8StringT>> stringFactorySupplier,
+          Supplier<StructChildFactory<ChildVectorT>> structChildFactorySupplier,
+          Supplier<ArrayFactory<ChildVectorT, ArrayT>> arrayFactorySupplier) {
+    this.decimalFactorySupplier = decimalFactorySupplier;
+    this.stringFactorySupplier = stringFactorySupplier;
+    this.structChildFactorySupplier = structChildFactorySupplier;
+    this.arrayFactorySupplier = arrayFactorySupplier;
+  }
+
+  public ArrowVectorAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT> getVectorAccessor(VectorHolder holder) {
+    Dictionary dictionary = holder.dictionary();
+    boolean isVectorDictEncoded = holder.isDictionaryEncoded();
+    FieldVector vector = holder.vector();
+    if (isVectorDictEncoded) {
+      ColumnDescriptor desc = holder.descriptor();
+      PrimitiveType primitive = desc.getPrimitiveType();
+      return getDictionaryVectorAccessor(dictionary, desc, vector, primitive);
+    } else {
+      return getPlainVectorAccessor(vector);
+    }
+  }
+
+  private ArrowVectorAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT> getDictionaryVectorAccessor(
+      Dictionary dictionary,
+      ColumnDescriptor desc,
+      FieldVector vector, PrimitiveType primitive) {
+    Preconditions.checkState(vector instanceof IntVector, "Dictionary ids should be stored in IntVectors only");
+    if (primitive.getOriginalType() != null) {
+      switch (desc.getPrimitiveType().getOriginalType()) {
+        case ENUM:
+        case JSON:
+        case UTF8:
+        case BSON:
+          return new DictionaryStringAccessor<>((IntVector) vector, dictionary, stringFactorySupplier.get());
+        case INT_64:
+        case TIMESTAMP_MILLIS:
+        case TIMESTAMP_MICROS:
+          return new DictionaryLongAccessor<>((IntVector) vector, dictionary);
+        case DECIMAL:
+          switch (primitive.getPrimitiveTypeName()) {
+            case BINARY:
+            case FIXED_LEN_BYTE_ARRAY:
+              return new DictionaryDecimalBinaryAccessor<>(
+                  (IntVector) vector,
+                  dictionary,
+                  decimalFactorySupplier.get());
+            case INT64:
+              return new DictionaryDecimalLongAccessor<>(
+                  (IntVector) vector,
+                  dictionary,
+                  decimalFactorySupplier.get());
+            case INT32:
+              return new DictionaryDecimalIntAccessor<>(
+                  (IntVector) vector,
+                  dictionary,
+                  decimalFactorySupplier.get());
+            default:
+              throw new UnsupportedOperationException(
+                  "Unsupported base type for decimal: " + primitive.getPrimitiveTypeName());
+          }
+        default:
+          throw new UnsupportedOperationException(
+              "Unsupported logical type: " + primitive.getOriginalType());
+      }
+    } else {
+      switch (primitive.getPrimitiveTypeName()) {
+        case FIXED_LEN_BYTE_ARRAY:
+        case BINARY:
+          return new DictionaryBinaryAccessor<>((IntVector) vector, dictionary);
+        case FLOAT:
+          return new DictionaryFloatAccessor<>((IntVector) vector, dictionary);
+        case INT64:
+          return new DictionaryLongAccessor<>((IntVector) vector, dictionary);
+        case DOUBLE:
+          return new DictionaryDoubleAccessor<>((IntVector) vector, dictionary);
+        default:
+          throw new UnsupportedOperationException("Unsupported type: " + primitive);
+      }
+    }
+  }
+
+  @SuppressWarnings("checkstyle:CyclomaticComplexity")
+  private ArrowVectorAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT>
+      getPlainVectorAccessor(FieldVector vector) {
+    if (vector instanceof BitVector) {
+      return new BooleanAccessor<>((BitVector) vector);
+    } else if (vector instanceof IntVector) {
+      return new IntAccessor<>((IntVector) vector);
+    } else if (vector instanceof BigIntVector) {
+      return new LongAccessor<>((BigIntVector) vector);
+    } else if (vector instanceof Float4Vector) {
+      return new FloatAccessor<>((Float4Vector) vector);
+    } else if (vector instanceof Float8Vector) {
+      return new DoubleAccessor<>((Float8Vector) vector);
+    } else if (vector instanceof DecimalVector) {
+      return new DecimalAccessor<>((DecimalVector) vector, decimalFactorySupplier.get());
+    } else if (vector instanceof VarCharVector) {
+      return new StringAccessor<>((VarCharVector) vector, stringFactorySupplier.get());
+    } else if (vector instanceof VarBinaryVector) {
+      return new BinaryAccessor<>((VarBinaryVector) vector);
+    } else if (vector instanceof DateDayVector) {
+      return new DateAccessor<>((DateDayVector) vector);
+    } else if (vector instanceof TimeStampMicroTZVector) {
+      return new TimestampMicroTzAccessor<>((TimeStampMicroTZVector) vector);
+    } else if (vector instanceof TimeStampMicroVector) {
+      return new TimestampMicroAccessor<>((TimeStampMicroVector) vector);
+    } else if (vector instanceof ListVector) {
+      ListVector listVector = (ListVector) vector;
+      return new ArrayAccessor<>(listVector, arrayFactorySupplier.get());
+    } else if (vector instanceof StructVector) {
+      StructVector structVector = (StructVector) vector;
+      return new StructAccessor<>(structVector, structChildFactorySupplier.get());
+    }
+    throw new UnsupportedOperationException("Unsupported vector: " + vector.getClass());
+  }
+
+  private static class BooleanAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT extends AutoCloseable>
+          extends ArrowVectorAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT> {
+    private final BitVector vector;
+
+    BooleanAccessor(BitVector vector) {
+      super(vector);
+      this.vector = vector;
+    }
+
+    @Override
+    public final boolean getBoolean(int rowId) {
+      return vector.get(rowId) == 1;
+    }
+  }
+
+  private static class IntAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT extends AutoCloseable>
+          extends ArrowVectorAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT> {
+
+    private final IntVector vector;
+
+    IntAccessor(IntVector vector) {
+      super(vector);
+      this.vector = vector;
+    }
+
+    @Override
+    public final int getInt(int rowId) {
+      return vector.get(rowId);
+    }
+
+    @Override
+    public final long getLong(int rowId) {
+      return getInt(rowId);
+    }
+  }
+
+  private static class LongAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT extends AutoCloseable>
+          extends ArrowVectorAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT> {
+
+    private final BigIntVector vector;
+
+    LongAccessor(BigIntVector vector) {
+      super(vector);
+      this.vector = vector;
+    }
+
+    @Override
+    public final long getLong(int rowId) {
+      return vector.get(rowId);
+    }
+  }
+
+  private static class DictionaryLongAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT extends AutoCloseable>
+          extends ArrowVectorAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT> {
+    private final IntVector offsetVector;
+    private final long[] decodedDictionary;
+
+    DictionaryLongAccessor(IntVector vector, Dictionary dictionary) {
+      super(vector);
+      this.offsetVector = vector;
+      this.decodedDictionary = IntStream.rangeClosed(0, dictionary.getMaxId())
+          .mapToLong(dictionary::decodeToLong)
+          .toArray();
+    }
+
+    @Override
+    public final long getLong(int rowId) {
+      return decodedDictionary[offsetVector.get(rowId)];
+    }
+  }
+
+  private static class FloatAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT extends AutoCloseable>
+          extends ArrowVectorAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT> {
+
+    private final Float4Vector vector;
+
+    FloatAccessor(Float4Vector vector) {
+      super(vector);
+      this.vector = vector;
+    }
+
+    @Override
+    public final float getFloat(int rowId) {
+      return vector.get(rowId);
+    }
+
+    @Override
+    public final double getDouble(int rowId) {
+      return getFloat(rowId);
+    }
+  }
+
+  private static class DictionaryFloatAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT extends AutoCloseable>
+          extends ArrowVectorAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT> {
+    private final IntVector offsetVector;
+    private final float[] decodedDictionary;
+
+    DictionaryFloatAccessor(IntVector vector, Dictionary dictionary) {
+      super(vector);
+      this.offsetVector = vector;
+      this.decodedDictionary = new float[dictionary.getMaxId() + 1];
+      for (int i = 0; i <= dictionary.getMaxId(); i++) {
+        decodedDictionary[i] = dictionary.decodeToFloat(i);
+      }
+    }
+
+    @Override
+    public final float getFloat(int rowId) {
+      return decodedDictionary[offsetVector.get(rowId)];
+    }
+
+    @Override
+    public final double getDouble(int rowId) {
+      return getFloat(rowId);
+    }
+  }
+
+  private static class DoubleAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT extends AutoCloseable>
+          extends ArrowVectorAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT> {
+
+    private final Float8Vector vector;
+
+    DoubleAccessor(Float8Vector vector) {
+      super(vector);
+      this.vector = vector;
+    }
+
+    @Override
+    public final double getDouble(int rowId) {
+      return vector.get(rowId);
+    }
+  }
+
+  private static class DictionaryDoubleAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT extends AutoCloseable>
+          extends ArrowVectorAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT> {
+    private final IntVector offsetVector;
+    private final double[] decodedDictionary;
+
+    DictionaryDoubleAccessor(IntVector vector, Dictionary dictionary) {
+      super(vector);
+      this.offsetVector = vector;
+      this.decodedDictionary = IntStream.rangeClosed(0, dictionary.getMaxId())
+          .mapToDouble(dictionary::decodeToDouble)
+          .toArray();
+    }
+
+    @Override
+    public final double getDouble(int rowId) {
+      return decodedDictionary[offsetVector.get(rowId)];
+    }
+  }
+
+  private static class StringAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT extends AutoCloseable>
+          extends ArrowVectorAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT> {
+
+    private final VarCharVector vector;
+    private final StringFactory<Utf8StringT> stringFactory;
+
+    StringAccessor(VarCharVector vector, StringFactory<Utf8StringT> stringFactory) {
+      super(vector);
+      this.vector = vector;
+      this.stringFactory = stringFactory;
+    }
+
+    @Override
+    public final Utf8StringT getUTF8String(int rowId) {
+      return stringFactory.ofRow(vector, rowId);
+    }
+  }
+
+  private static class DictionaryStringAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT extends AutoCloseable>
+          extends ArrowVectorAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT> {
+    private final Utf8StringT[] decodedDictionary;
+    private final IntVector offsetVector;
+
+    DictionaryStringAccessor(IntVector vector, Dictionary dictionary, StringFactory<Utf8StringT> stringFactory) {
+      super(vector);
+      this.offsetVector = vector;
+      this.decodedDictionary = IntStream.rangeClosed(0, dictionary.getMaxId())
+          .mapToObj(dictionary::decodeToBinary)
+          .map(binary -> stringFactory.ofBytes(binary.getBytes()))
+          .toArray(genericArray(stringFactory.getGenericClass()));
+    }
+
+    @Override
+    public final Utf8StringT getUTF8String(int rowId) {
+      int offset = offsetVector.get(rowId);
+      return decodedDictionary[offset];
+    }
+  }
+
+  private static class BinaryAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT extends AutoCloseable>
+          extends ArrowVectorAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT> {
+
+    private final VarBinaryVector vector;
+
+    BinaryAccessor(VarBinaryVector vector) {
+      super(vector);
+      this.vector = vector;
+    }
+
+    @Override
+    public final byte[] getBinary(int rowId) {
+      return vector.get(rowId);
+    }
+  }
+
+  private static class DictionaryBinaryAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT extends AutoCloseable>
+          extends ArrowVectorAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT> {
+    private final IntVector offsetVector;
+    private final byte[][] decodedDictionary;
+
+    DictionaryBinaryAccessor(IntVector vector, Dictionary dictionary) {
+      super(vector);
+      this.offsetVector = vector;
+      this.decodedDictionary = IntStream.rangeClosed(0, dictionary.getMaxId())
+          .mapToObj(dictionary::decodeToBinary)
+          .map(Binary::getBytes)
+          .toArray(byte[][]::new);
+    }
+
+    @Override
+    public final byte[] getBinary(int rowId) {
+      int offset = offsetVector.get(rowId);
+      return decodedDictionary[offset];
+    }
+  }
+
+  private static class DateAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT extends AutoCloseable>
+          extends ArrowVectorAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT> {
+
+    private final DateDayVector vector;
+
+    DateAccessor(DateDayVector vector) {
+      super(vector);
+      this.vector = vector;
+    }
+
+    @Override
+    public final int getInt(int rowId) {
+      return vector.get(rowId);
+    }
+  }
+
+  private static class TimestampMicroTzAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT extends AutoCloseable>
+          extends ArrowVectorAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT> {
+
+    private final TimeStampMicroTZVector vector;
+
+    TimestampMicroTzAccessor(TimeStampMicroTZVector vector) {
+      super(vector);
+      this.vector = vector;
+    }
+
+    @Override
+    public final long getLong(int rowId) {
+      return vector.get(rowId);
+    }
+  }
+
+  private static class TimestampMicroAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT extends AutoCloseable>
+      extends ArrowVectorAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT> {
+
+    private final TimeStampMicroVector vector;
+
+    TimestampMicroAccessor(TimeStampMicroVector vector) {
+      super(vector);
+      this.vector = vector;
+    }
+
+    @Override
+    public final long getLong(int rowId) {
+      return vector.get(rowId);
+    }
+  }
+
+  private static class ArrayAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT extends AutoCloseable>
+          extends ArrowVectorAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT> {
+
+    private final ListVector vector;
+    private final ChildVectorT arrayData;
+    private final ArrayFactory<ChildVectorT, ArrayT> arrayFactory;
+
+    ArrayAccessor(ListVector vector, ArrayFactory<ChildVectorT, ArrayT> arrayFactory) {
+      super(vector);
+      this.vector = vector;
+      this.arrayFactory = arrayFactory;
+      this.arrayData = arrayFactory.ofChild(vector.getDataVector());
+    }
+
+    @Override
+    public final ArrayT getArray(int rowId) {
+      return arrayFactory.ofRow(vector, arrayData, rowId);
+    }
+  }
+
+  private static class StructAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT extends AutoCloseable>
+          extends ArrowVectorAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT> {
+    StructAccessor(StructVector structVector, StructChildFactory<ChildVectorT> structChildFactory) {
+      super(structVector, IntStream.range(0, structVector.size())
+              .mapToObj(structVector::getVectorById)
+              .map(structChildFactory::of)
+              .toArray(genericArray(structChildFactory.getGenericClass())));
+    }
+  }
+
+  private static class DecimalAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT extends AutoCloseable>
+          extends ArrowVectorAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT> {
+
+    private final DecimalVector vector;
+    private final DecimalFactory<DecimalT> decimalFactory;
+
+    DecimalAccessor(DecimalVector vector, DecimalFactory<DecimalT> decimalFactory) {
+      super(vector);
+      this.vector = vector;
+      this.decimalFactory = decimalFactory;
+    }
+
+    @Override
+    public final DecimalT getDecimal(int rowId, int precision, int scale) {
+      return decimalFactory.ofBigDecimal(
+              DecimalUtility.getBigDecimalFromArrowBuf(vector.getDataBuffer(), rowId, scale),
+              precision, scale);
+    }
+  }
+
+  @SuppressWarnings("checkstyle:VisibilityModifier")
+  private abstract static class
+      DictionaryDecimalAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT extends AutoCloseable>
+          extends ArrowVectorAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT> {
+    final DecimalT[] cache;

Review comment:
       why not private?

##########
File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowVectorAccessor.java
##########
@@ -17,79 +17,82 @@
  * under the License.
  */
 
-package org.apache.iceberg.spark.data.vectorized;
+package org.apache.iceberg.arrow.vectorized;
 
 import org.apache.arrow.vector.ValueVector;
-import org.apache.spark.sql.types.Decimal;
-import org.apache.spark.sql.vectorized.ArrowColumnVector;
-import org.apache.spark.sql.vectorized.ColumnarArray;
-import org.apache.spark.unsafe.types.UTF8String;
-
-@SuppressWarnings("checkstyle:VisibilityModifier")
-public abstract class ArrowVectorAccessor {
 
+public class ArrowVectorAccessor<DecimalT, Utf8StringT, ArrayT, ChildVectorT extends AutoCloseable> {
   private final ValueVector vector;
-  private final ArrowColumnVector[] childColumns;
+  private final ChildVectorT[] childColumns;
 
-  ArrowVectorAccessor(ValueVector vector) {
-    this.vector = vector;
-    this.childColumns = new ArrowColumnVector[0];
+  protected ArrowVectorAccessor(ValueVector vector) {
+    this(vector, null);
   }
 
-  ArrowVectorAccessor(ValueVector vector, ArrowColumnVector[] children) {
+  protected ArrowVectorAccessor(ValueVector vector, ChildVectorT[] children) {
     this.vector = vector;
     this.childColumns = children;
   }
 
-  final void close() {
-    for (ArrowColumnVector column : childColumns) {
-      // Closing an ArrowColumnVector is expected to not throw any exception
-      column.close();
+  public final void close() {

Review comment:
       how come final? Does that still make sense now that we are using thsi as a base class?




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#discussion_r618422020



##########
File path: build.gradle
##########
@@ -735,9 +735,12 @@ project(':iceberg-arrow') {
       exclude group: 'com.google.code.findbugs', module: 'jsr305'
     }
     compile("org.apache.arrow:arrow-memory-netty") {
-      exclude group: 'io.netty', module: 'netty-common'
       exclude group: 'com.google.code.findbugs', module: 'jsr305'

Review comment:
       ahhh...I get it. Ok, yeah in that case leave it as you have it, maybe with a comment as to why?




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#discussion_r614770900



##########
File path: build.gradle
##########
@@ -738,6 +738,13 @@ project(':iceberg-arrow') {
       exclude group: 'io.netty', module: 'netty-common'
       exclude group: 'com.google.code.findbugs', module: 'jsr305'
     }
+
+    testCompile("org.apache.arrow:arrow-memory-netty") {

Review comment:
       how come `arrow-memory-netty` is listed twice?




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#discussion_r638277988



##########
File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ColumnVector.java
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.arrow.vectorized;
+
+import org.apache.arrow.vector.FieldVector;
+import org.apache.iceberg.types.Types;
+
+/**
+ * This class is inspired by Spark's {@code ColumnVector}.
+ * This class represents the column data for an Iceberg table query.
+ * It wraps an arrow {@link FieldVector} and provides simple
+ * accessors for the row values. Advanced users can access
+ * the {@link FieldVector}.
+ * <p>
+ *   Supported Iceberg data types:
+ *   <ul>
+ *     <li>{@link Types.BooleanType}</li>
+ *     <li>{@link Types.IntegerType}</li>
+ *     <li>{@link Types.LongType}</li>
+ *     <li>{@link Types.FloatType}</li>
+ *     <li>{@link Types.DoubleType}</li>
+ *     <li>{@link Types.StringType}</li>
+ *     <li>{@link Types.BinaryType}</li>
+ *     <li>{@link Types.TimestampType} (with and without timezone)</li>
+ *     <li>{@link Types.DateType}</li>
+ *   </ul>
+ */
+public class ColumnVector implements AutoCloseable {

Review comment:
       @rymurr, why did we choose to add a public vector representation? Is Arrow itself insufficient?




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] mayursrivastava commented on a change in pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
mayursrivastava commented on a change in pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#discussion_r682567396



##########
File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowReader.java
##########
@@ -0,0 +1,325 @@
+/*
+ * 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.arrow.vectorized;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import java.util.stream.StreamSupport;
+import org.apache.arrow.vector.NullCheckingForGet;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.arrow.vector.types.Types.MinorType;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.encryption.EncryptedFiles;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.CloseableIterator;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.types.Types;
+import org.apache.parquet.schema.MessageType;
+
+/**
+ * Vectorized reader that returns an iterator of {@link ColumnarBatch}.
+ * See {@link #open(CloseableIterable)} ()} to learn about the
+ * behavior of the iterator.
+ *
+ * <p>The following Iceberg data types are supported and have been tested:
+ * <ul>
+ *     <li>Iceberg: {@link Types.BooleanType}, Arrow: {@link MinorType#BIT}</li>
+ *     <li>Iceberg: {@link Types.IntegerType}, Arrow: {@link MinorType#INT}</li>
+ *     <li>Iceberg: {@link Types.LongType}, Arrow: {@link MinorType#BIGINT}</li>
+ *     <li>Iceberg: {@link Types.FloatType}, Arrow: {@link MinorType#FLOAT4}</li>
+ *     <li>Iceberg: {@link Types.DoubleType}, Arrow: {@link MinorType#FLOAT8}</li>
+ *     <li>Iceberg: {@link Types.StringType}, Arrow: {@link MinorType#VARCHAR}</li>
+ *     <li>Iceberg: {@link Types.TimestampType} (both with and without timezone),
+ *         Arrow: {@link MinorType#TIMEMICRO}</li>
+ *     <li>Iceberg: {@link Types.BinaryType}, Arrow: {@link MinorType#VARBINARY}</li>
+ *     <li>Iceberg: {@link Types.DateType}, Arrow: {@link MinorType#DATEDAY}</li>
+ * </ul>
+ *
+ * <p>Features that don't work in this implementation:
+ * <ul>
+ *     <li>Type promotion: In case of type promotion, the Arrow vector corresponding to
+ *     the data type in the parquet file is returned instead of the data type in the latest schema.
+ *     See https://github.com/apache/iceberg/issues/2483.</li>
+ *     <li>Columns with constant values are physically encoded as a dictionary. The Arrow vector
+ *     type is int32 instead of the type as per the schema.
+ *     See https://github.com/apache/iceberg/issues/2484.</li>
+ *     <li>Data types: {@link Types.TimeType}, {@link Types.ListType}, {@link Types.MapType},
+ *     {@link Types.StructType}, {@link Types.UUIDType}, {@link Types.FixedType} and
+ *     {@link Types.DecimalType}

Review comment:
       yes, they will throw UnsupportedOperationException.
   I've added explicit exception in the reader code.
   The fixes are present in https://github.com/apache/iceberg/pull/2933.
   




-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] mayursrivastava commented on pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
mayursrivastava commented on pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#issuecomment-804354522


   Hi @rymurr, I wanted to check if you got a chance to take a look at the updates to this pr. I'm happy to make any more changes. Thanks, Mayur


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] mayursrivastava commented on pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
mayursrivastava commented on pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#issuecomment-829465334


   Thank you @rymurr 


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] mayursrivastava edited a comment on pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
mayursrivastava edited a comment on pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#issuecomment-824210180


   `
   ./gradlew :iceberg-spark:jmh -PjmhIncludeRegex=VectorizedReadFlatParquetDataBenchmark -PjmhOutputPath=benchmark/results.txt
   `
   
   @rymurr, is this :iceberg-spark2:jmh? Looks like this requires Java8 runtime. Am I right? I ran it on the master branch (without my changes), but it fails with the following error. Does it need a powerful machine?
   
   `
   JMH version: 1.21
   VM version: JDK 1.8.0_282, OpenJDK 64-Bit Server VM, 25.282-b08
   VM invoker: /usr/lib/jvm/java-8-openjdk-amd64/jre/bin/java
   VM options: <none>
   Warmup: 3 iterations, single-shot each
   Measurement: 5 iterations, single-shot each
   Timeout: 10 min per iteration
   Threads: 1 thread
   Benchmark mode: Single shot invocation time
   Benchmark: org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadDictionaryEncodedFlatParquetDataBenchmark.readDatesIcebergVectorized5k
   
   Run progress: 0.00% complete, ETA 00:00:00
   Fork: 1 of 1
   Warmup Iteration   1: (*interrupt*) <failure>
   
   java.lang.InterruptedException
           at java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireSharedInterruptibly(AbstractQueuedSynchronizer.java:998)
           at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireSharedInterruptibly(AbstractQueuedSynchronizer.java:1304)
           at scala.concurrent.impl.Promise$DefaultPromise.tryAwait(Promise.scala:206)
           at scala.concurrent.impl.Promise$DefaultPromise.ready(Promise.scala:222)
           at scala.concurrent.impl.Promise$DefaultPromise.ready(Promise.scala:157)
           at org.apache.spark.util.ThreadUtils$.awaitReady(ThreadUtils.scala:243)
           at org.apache.spark.scheduler.DAGScheduler.runJob(DAGScheduler.scala:750)
           at org.apache.spark.SparkContext.runJob(SparkContext.scala:2061)
           at org.apache.spark.sql.execution.datasources.v2.WriteToDataSourceV2Exec.doExecute(WriteToDataSourceV2Exec.scala:64)
           at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:131)
           at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:127)
           at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:155)
           at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
           at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:152)
           at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:127)
           at org.apache.spark.sql.execution.QueryExecution.toRdd$lzycompute(QueryExecution.scala:83)
           at org.apache.spark.sql.execution.QueryExecution.toRdd(QueryExecution.scala:81)
           at org.apache.spark.sql.DataFrameWriter$$anonfun$runCommand$1.apply(DataFrameWriter.scala:696)
           at org.apache.spark.sql.DataFrameWriter$$anonfun$runCommand$1.apply(DataFrameWriter.scala:696)
           at org.apache.spark.sql.execution.SQLExecution$$anonfun$withNewExecutionId$1.apply(SQLExecution.scala:80)
           at org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:127)
           at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:75)
           at org.apache.spark.sql.DataFrameWriter.runCommand(DataFrameWriter.scala:696)
           at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:280)
           at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:249)
           at org.apache.iceberg.spark.source.IcebergSourceBenchmark.appendAsFile(IcebergSourceBenchmark.java:130)
           at org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadDictionaryEncodedFlatParquetDataBenchmark.appendData(VectorizedReadDictionaryEncodedFlatParquetDataBenchmark.java:82)
           at org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadDictionaryEncodedFlatParquetDataBenchmark.setupBenchmark(VectorizedReadDictionaryEncodedFlatParquetDataBenchmark.java:56)
           at org.apache.iceberg.spark.source.parquet.vectorized.generated.VectorizedReadDictionaryEncodedFlatParquetDataBenchmark_readDatesIcebergVectorized5k_jmhTest._jmh_tryInit_f_vectorizedreaddictionaryencodedflatparquetdatabenchmark0_G(VectorizedReadDictionaryEncodedFlatParquetDataBenchmark_readDatesIcebergVectorized5k_jmhTest.java:438)
           at org.apache.iceberg.spark.source.parquet.vectorized.generated.VectorizedReadDictionaryEncodedFlatParquetDataBenchmark_readDatesIcebergVectorized5k_jmhTest.readDatesIcebergVectorized5k_SingleShotTime(VectorizedReadDictionaryEncodedFlatParquetDataBenchmark_readDatesIcebergVectorized5k_jmhTest.java:363)
           at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
           at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
           at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
           at java.lang.reflect.Method.invoke(Method.java:498)
           at org.openjdk.jmh.runner.BenchmarkHandler$BenchmarkTask.call(BenchmarkHandler.java:453)
           at org.openjdk.jmh.runner.BenchmarkHandler$BenchmarkTask.call(BenchmarkHandler.java:437)
           at java.util.concurrent.FutureTask.run(FutureTask.java:266)
           at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
           at java.util.concurrent.FutureTask.run(FutureTask.java:266)
           at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
           at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
           at java.lang.Thread.run(Thread.java:748)
   
   
   Benchmark had encountered error, and fail on error was requested
   `


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
rymurr commented on pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#issuecomment-828386504


   Thanks @mayursrivastava these look semsible. I am trying to compare locally w/ the master branch to spot a regression. 


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#discussion_r638341939



##########
File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowReader.java
##########
@@ -0,0 +1,325 @@
+/*
+ * 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.arrow.vectorized;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import java.util.stream.StreamSupport;
+import org.apache.arrow.vector.NullCheckingForGet;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.arrow.vector.types.Types.MinorType;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.encryption.EncryptedFiles;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.CloseableIterator;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.types.Types;
+import org.apache.parquet.schema.MessageType;
+
+/**
+ * Vectorized reader that returns an iterator of {@link ColumnarBatch}.
+ * See {@link #open(CloseableIterable)} ()} to learn about the
+ * behavior of the iterator.
+ *
+ * <p>The following Iceberg data types are supported and have been tested:
+ * <ul>
+ *     <li>Iceberg: {@link Types.BooleanType}, Arrow: {@link MinorType#BIT}</li>
+ *     <li>Iceberg: {@link Types.IntegerType}, Arrow: {@link MinorType#INT}</li>
+ *     <li>Iceberg: {@link Types.LongType}, Arrow: {@link MinorType#BIGINT}</li>
+ *     <li>Iceberg: {@link Types.FloatType}, Arrow: {@link MinorType#FLOAT4}</li>
+ *     <li>Iceberg: {@link Types.DoubleType}, Arrow: {@link MinorType#FLOAT8}</li>
+ *     <li>Iceberg: {@link Types.StringType}, Arrow: {@link MinorType#VARCHAR}</li>
+ *     <li>Iceberg: {@link Types.TimestampType} (both with and without timezone),
+ *         Arrow: {@link MinorType#TIMEMICRO}</li>
+ *     <li>Iceberg: {@link Types.BinaryType}, Arrow: {@link MinorType#VARBINARY}</li>
+ *     <li>Iceberg: {@link Types.DateType}, Arrow: {@link MinorType#DATEDAY}</li>
+ * </ul>
+ *
+ * <p>Features that don't work in this implementation:
+ * <ul>
+ *     <li>Type promotion: In case of type promotion, the Arrow vector corresponding to
+ *     the data type in the parquet file is returned instead of the data type in the latest schema.
+ *     See https://github.com/apache/iceberg/issues/2483.</li>
+ *     <li>Columns with constant values are physically encoded as a dictionary. The Arrow vector
+ *     type is int32 instead of the type as per the schema.
+ *     See https://github.com/apache/iceberg/issues/2484.</li>
+ *     <li>Data types: {@link Types.TimeType}, {@link Types.ListType}, {@link Types.MapType},
+ *     {@link Types.StructType}, {@link Types.UUIDType}, {@link Types.FixedType} and
+ *     {@link Types.DecimalType}
+ *     See https://github.com/apache/iceberg/issues/2485 and https://github.com/apache/iceberg/issues/2486.</li>
+ *     <li>Iceberg v2 spec is not supported.
+ *     See https://github.com/apache/iceberg/issues/2487.</li>

Review comment:
       Okay, we should definitely fix that so it is safe to read data. Otherwise there is a correctness issue with v2 tables.




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] mayursrivastava commented on a change in pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
mayursrivastava commented on a change in pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#discussion_r590680042



##########
File path: data/src/main/java/org/apache/iceberg/data/arrow/BaseArrowFileScanTaskReader.java
##########
@@ -0,0 +1,124 @@
+/*
+ * 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.data.arrow;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.stream.Stream;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.encryption.EncryptedFiles;
+import org.apache.iceberg.encryption.EncryptedInputFile;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.io.CloseableIterator;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Base class for vectorized Arrow reader.
+ */
+abstract class BaseArrowFileScanTaskReader<T> implements CloseableIterator<T> {

Review comment:
       Sounds good to me but this will require adding a class similar to ColumnVector (and different implementations for all supported types) as well.




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] mayursrivastava commented on a change in pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
mayursrivastava commented on a change in pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#discussion_r612382087



##########
File path: arrow/src/test/java/org/apache/iceberg/arrow/vectorized/ArrowReaderTest.java
##########
@@ -0,0 +1,734 @@
+/*
+ * 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.arrow.vectorized;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.time.temporal.ChronoUnit;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+import org.apache.arrow.vector.BigIntVector;
+import org.apache.arrow.vector.BitVector;
+import org.apache.arrow.vector.DateDayVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.Float4Vector;
+import org.apache.arrow.vector.Float8Vector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.TimeStampMicroTZVector;
+import org.apache.arrow.vector.TimeStampMicroVector;
+import org.apache.arrow.vector.VarBinaryVector;
+import org.apache.arrow.vector.VarCharVector;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.arrow.vector.types.Types.MinorType;
+import org.apache.arrow.vector.types.pojo.ArrowType;
+import org.apache.arrow.vector.types.pojo.Field;
+import org.apache.arrow.vector.types.pojo.FieldType;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.DataFiles;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.OverwriteFiles;
+import org.apache.iceberg.PartitionKey;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.StructLike;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.data.parquet.GenericParquetWriter;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.types.Types;
+import org.junit.Ignore;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.Files.localInput;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Test cases for {@link ArrowReader}.
+ * <p>All tests create a table with monthly partitions and write 1 year of data to the table.
+ */
+public class ArrowReaderTest {
+
+  private static final int NUM_ROWS_PER_MONTH = 20;
+  private static final ImmutableList<String> ALL_COLUMNS =
+      ImmutableList.of(
+          "timestamp",
+          "timestamp_nullable",
+          "boolean",
+          "boolean_nullable",
+          "int",
+          "int_nullable",
+          "long",
+          "long_nullable",
+          "float",
+          "float_nullable",
+          "double",
+          "double_nullable",
+          "timestamp_tz",
+          "timestamp_tz_nullable",
+          "string",
+          "string_nullable",
+          "bytes",
+          "bytes_nullable",
+          "date",
+          "date_nullable",
+          "int_promotion"
+      );
+
+  @Rule
+  public final TemporaryFolder temp = new TemporaryFolder();
+
+  private HadoopTables tables;
+
+  private String tableLocation;
+  private List<GenericRecord> rowsWritten;
+
+  /**
+   * Read all rows and columns from the table without any filter. The test asserts that the Arrow {@link
+   * VectorSchemaRoot} contains the expected schema and expected vector types. Then the test asserts that the vectors
+   * contains expected values. The test also asserts the total number of rows match the expected value.
+   */
+  @Test
+  public void testReadAll() throws Exception {
+    writeTableWithIncrementalRecords();
+    Table table = tables.load(tableLocation);
+    readAndCheckVectorSchemaRoots(table.newScan(), NUM_ROWS_PER_MONTH, 12 * NUM_ROWS_PER_MONTH, ALL_COLUMNS);
+  }
+
+  /**
+   * This test writes each partition with constant value rows. The Arrow vectors returned are mostly of type int32
+   * which is unexpected. This is happening because of dictionary encoding at the storage level.
+   * <p>
+   * Following are the expected and actual Arrow schema:
+   * <pre>
+   * Expected Arrow Schema:
+   * timestamp: Timestamp(MICROSECOND, null) not null,
+   * timestamp_nullable: Timestamp(MICROSECOND, null),
+   * boolean: Bool not null,
+   * boolean_nullable: Bool,
+   * int: Int(32, true) not null,
+   * int_nullable: Int(32, true),
+   * long: Int(64, true) not null,
+   * long_nullable: Int(64, true),
+   * float: FloatingPoint(SINGLE) not null,
+   * float_nullable: FloatingPoint(SINGLE),
+   * double: FloatingPoint(DOUBLE) not null,
+   * double_nullable: FloatingPoint(DOUBLE),
+   * timestamp_tz: Timestamp(MICROSECOND, UTC) not null,
+   * timestamp_tz_nullable: Timestamp(MICROSECOND, UTC),
+   * string: Utf8 not null,
+   * string_nullable: Utf8,
+   * bytes: Binary not null,
+   * bytes_nullable: Binary,
+   * date: Date(DAY) not null,
+   * date_nullable: Date(DAY),
+   * int_promotion: Int(32, true) not null
+   *
+   * Actual Arrow Schema:
+   * timestamp: Int(32, true) not null,
+   * timestamp_nullable: Int(32, true),
+   * boolean: Bool not null,
+   * boolean_nullable: Bool,
+   * int: Int(32, true) not null,
+   * int_nullable: Int(32, true),
+   * long: Int(32, true) not null,
+   * long_nullable: Int(32, true),
+   * float: Int(32, true) not null,
+   * float_nullable: Int(32, true),
+   * double: Int(32, true) not null,
+   * double_nullable: Int(32, true),
+   * timestamp_tz: Int(32, true) not null,
+   * timestamp_tz_nullable: Int(32, true),
+   * string: Int(32, true) not null,
+   * string_nullable: Int(32, true),
+   * bytes: Int(32, true) not null,
+   * bytes_nullable: Int(32, true),
+   * date: Date(DAY) not null,
+   * date_nullable: Date(DAY),
+   * int_promotion: Int(32, true) not null
+   * </pre>
+   * <p>
+   * TODO: fix the returned Arrow vectors to have vector types consistent with Iceberg types.
+   * <p>
+   * Read all rows and columns from the table without any filter. The test asserts that the Arrow {@link
+   * VectorSchemaRoot} contains the expected schema and expected vector types. Then the test asserts that the vectors
+   * contains expected values. The test also asserts the total number of rows match the expected value.
+   */
+  @Test
+  @Ignore
+  public void testReadAllWithConstantRecords() throws Exception {
+    writeTableWithConstantRecords();
+    Table table = tables.load(tableLocation);
+    readAndCheckVectorSchemaRoots(table.newScan(), NUM_ROWS_PER_MONTH, 12 * NUM_ROWS_PER_MONTH, ALL_COLUMNS);
+  }
+
+  /**
+   * Read all rows and columns from the table without any filter. The test uses a batch size smaller than the number of
+   * rows in a partition. The test asserts that the Arrow {@link VectorSchemaRoot} contains the expected schema and
+   * expected vector types. Then the test asserts that the vectors contains expected values. The test also asserts the
+   * total number of rows match the expected value.
+   */
+  @Test
+  public void testReadAllWithSmallerBatchSize() throws Exception {
+    writeTableWithIncrementalRecords();
+    Table table = tables.load(tableLocation);
+    TableScan scan = table.newScan();
+    readAndCheckVectorSchemaRoots(scan, 10, 12 * NUM_ROWS_PER_MONTH, ALL_COLUMNS);
+  }
+
+  /**
+   * Read selected rows and all columns from the table using a time range row filter. The test asserts that the Arrow
+   * {@link VectorSchemaRoot} contains the expected schema and expected vector types. Then the test asserts that the
+   * vectors contains expected values. The test also asserts the total number of rows match the expected value.
+   */
+  @Test
+  public void testReadRangeFilter() throws Exception {
+    writeTableWithIncrementalRecords();
+    Table table = tables.load(tableLocation);
+    LocalDateTime beginTime = LocalDateTime.of(2020, 1, 1, 0, 0, 0);
+    LocalDateTime endTime = LocalDateTime.of(2020, 2, 1, 0, 0, 0);
+    TableScan scan = table.newScan()
+        .filter(Expressions.and(
+            Expressions.greaterThanOrEqual("timestamp", timestampToMicros(beginTime)),
+            Expressions.lessThan("timestamp", timestampToMicros(endTime))));
+    readAndCheckVectorSchemaRoots(scan, NUM_ROWS_PER_MONTH, NUM_ROWS_PER_MONTH, ALL_COLUMNS);
+  }
+
+  /**
+   * Read selected rows and all columns from the table using a time range row filter.
+   * The test asserts that the result is empty.
+   */
+  @Test
+  public void testReadRangeFilterEmptyResult() throws Exception {
+    writeTableWithIncrementalRecords();
+    Table table = tables.load(tableLocation);
+    LocalDateTime beginTime = LocalDateTime.of(2021, 1, 1, 0, 0, 0);
+    LocalDateTime endTime = LocalDateTime.of(2021, 2, 1, 0, 0, 0);
+    TableScan scan = table.newScan()
+            .filter(Expressions.and(
+                    Expressions.greaterThanOrEqual("timestamp", timestampToMicros(beginTime)),
+                    Expressions.lessThan("timestamp", timestampToMicros(endTime))));
+    int numRoots = 0;
+    for (ArrowBatch batch : new VectorizedTableScanIterable(scan, NUM_ROWS_PER_MONTH, false)) {
+      numRoots++;
+    }
+    assertEquals(0, numRoots);
+  }
+
+  /**
+   * Read all rows and selected columns from the table with a column selection filter. The test asserts that the Arrow
+   * {@link VectorSchemaRoot} contains the expected schema and expected vector types. Then the test asserts that the
+   * vectors contains expected values. The test also asserts the total number of rows match the expected value.
+   */
+  @Test
+  public void testReadColumnFilter1() throws Exception {
+    writeTableWithIncrementalRecords();
+    Table table = tables.load(tableLocation);
+    TableScan scan = table.newScan()
+        .select("timestamp", "int", "string");
+    readAndCheckVectorSchemaRoots(
+        scan, NUM_ROWS_PER_MONTH, 12 * NUM_ROWS_PER_MONTH,
+        ImmutableList.of("timestamp", "int", "string"));
+  }
+
+  /**
+   * Read all rows and a single column from the table with a column selection filter. The test asserts that the Arrow
+   * {@link VectorSchemaRoot} contains the expected schema and expected vector types. Then the test asserts that the
+   * vectors contains expected values. The test also asserts the total number of rows match the expected value.
+   */
+  @Test
+  public void testReadColumnFilter2() throws Exception {
+    writeTableWithIncrementalRecords();
+    Table table = tables.load(tableLocation);
+    TableScan scan = table.newScan()
+        .select("timestamp");
+    readAndCheckVectorSchemaRoots(
+        scan, NUM_ROWS_PER_MONTH, 12 * NUM_ROWS_PER_MONTH,
+        ImmutableList.of("timestamp"));
+  }
+
+  private void readAndCheckVectorSchemaRoots(
+      TableScan scan,
+      int numRowsPerRoot,
+      int expectedTotalRows,
+      List<String> columns) {
+    Set<String> columnSet = ImmutableSet.copyOf(columns);
+    int rowIndex = 0;
+    int totalRows = 0;
+    for (ArrowBatch batch : new VectorizedTableScanIterable(scan, numRowsPerRoot, false)) {
+      VectorSchemaRoot root = batch.createVectorSchemaRootFromVectors();

Review comment:
       added tests for ColumnBatch

##########
File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowBatch.java
##########
@@ -0,0 +1,82 @@
+/*
+ * 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.arrow.vectorized;
+
+import java.util.Arrays;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.VectorSchemaRoot;
+
+/**
+ * This class is inspired by Spark's {@code ColumnarBatch}.
+ * This class wraps a columnar batch in the result set of an Iceberg table query.
+ */
+public class ArrowBatch implements AutoCloseable {

Review comment:
       renamed to ColumnBatch




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr merged pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
rymurr merged pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286


   


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
rymurr commented on pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#issuecomment-829471521


   no prob @mayursrivastava ! Thanks again for the contribution!
   
   I noticed that the build failed after the merge. Could you just double check that it was transient and unrelated to your change?


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] mayursrivastava commented on a change in pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
mayursrivastava commented on a change in pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#discussion_r612037892



##########
File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/VectorizedCombinedScanIterator.java
##########
@@ -0,0 +1,194 @@
+/*
+ * 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.arrow.vectorized;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.stream.Stream;
+import org.apache.arrow.vector.NullCheckingForGet;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.encryption.EncryptedFiles;
+import org.apache.iceberg.encryption.EncryptedInputFile;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.CloseableIterator;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Reads the data file and returns an iterator of {@link VectorSchemaRoot}.
+ * Only Parquet data file format is supported.
+ */
+class VectorizedCombinedScanIterator implements CloseableIterator<ArrowBatch> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(VectorizedCombinedScanIterator.class);
+
+  private final Iterator<FileScanTask> tasks;
+  private final Map<String, InputFile> inputFiles;
+  private final Schema expectedSchema;
+  private final String nameMapping;
+  private final boolean caseSensitive;
+  private final int batchSize;
+  private final boolean reuseContainers;
+  private CloseableIterator<ArrowBatch> currentIterator;
+  private ArrowBatch current;
+  private FileScanTask currentTask;
+
+  /**
+   * Create a new instance.
+   *
+   * @param task              Combined file scan task.
+   * @param expectedSchema    Read schema. The returned data will have this schema.
+   * @param nameMapping       Mapping from external schema names to Iceberg type IDs.
+   * @param io                File I/O.
+   * @param encryptionManager Encryption manager.
+   * @param caseSensitive     If {@code true}, column names are case sensitive.
+   *                          If {@code false}, column names are not case sensitive.
+   * @param batchSize         Batch size in number of rows. Each Arrow batch contains
+   *                          a maximum of {@code batchSize} rows.
+   * @param reuseContainers   If set to {@code false}, every {@link Iterator#next()} call creates
+   *                          new instances of Arrow vectors.
+   *                          If set to {@code true}, the Arrow vectors in the previous
+   *                          {@link Iterator#next()} may be reused for the data returned
+   *                          in the current {@link Iterator#next()}.
+   *                          This option avoids allocating memory again and again.
+   *                          Irrespective of the value of {@code reuseContainers}, the Arrow vectors
+   *                          in the previous {@link Iterator#next()} call are closed before creating
+   *                          new instances if the current {@link Iterator#next()}.
+   */
+  VectorizedCombinedScanIterator(
+          CombinedScanTask task,
+          Schema expectedSchema,
+          String nameMapping,
+          FileIO io,
+          EncryptionManager encryptionManager,
+          boolean caseSensitive,
+          int batchSize,
+          boolean reuseContainers) {
+    this.tasks = task.files().iterator();
+    Map<String, ByteBuffer> keyMetadata = Maps.newHashMap();
+    task.files().stream()
+        .flatMap(fileScanTask -> Stream.concat(Stream.of(fileScanTask.file()), fileScanTask.deletes().stream()))
+        .forEach(file -> keyMetadata.put(file.path().toString(), file.keyMetadata()));
+    Stream<EncryptedInputFile> encrypted = keyMetadata.entrySet().stream()
+        .map(entry -> EncryptedFiles.encryptedInput(io.newInputFile(entry.getKey()), entry.getValue()));
+
+    // decrypt with the batch call to avoid multiple RPCs to a key server, if possible
+    Iterable<InputFile> decryptedFiles = encryptionManager.decrypt(encrypted::iterator);
+
+    Map<String, InputFile> files = Maps.newHashMapWithExpectedSize(task.files().size());
+    decryptedFiles.forEach(decrypted -> files.putIfAbsent(decrypted.location(), decrypted));
+    this.inputFiles = Collections.unmodifiableMap(files);
+
+    this.currentIterator = CloseableIterator.empty();
+    this.expectedSchema = expectedSchema;
+    this.nameMapping = nameMapping;
+    this.caseSensitive = caseSensitive;
+    this.batchSize = batchSize;
+    this.reuseContainers = reuseContainers;
+  }
+
+  @Override
+  public boolean hasNext() {

Review comment:
       removed duplication (by removing ConcatReader) and moved the class to ArrowReader.




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] mayursrivastava commented on pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
mayursrivastava commented on pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#issuecomment-829474850


   @rymurr could you point me to the error? I can't find the failure. (btw, I cloned locally from master and the build was passing).


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] mayursrivastava commented on pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
mayursrivastava commented on pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#issuecomment-825992281


   Hi @rymurr, I ran the benchmark and the results are listed below.  
   
   Command:
   
   ```
   ./gradlew :iceberg-spark2:jmh -PjmhIncludeRegex=VectorizedReadFlatParquetDataBenchmark -PjmhOutputPath=benchmark/results.txt
   ```
   
   Results:
   
   ```
   # JMH version: 1.21
   # VM version: JDK 1.8.0_282, OpenJDK 64-Bit Server VM, 25.282-b08
   # VM invoker: /usr/lib/jvm/java-8-openjdk-amd64/jre/bin/java
   # VM options: <none>
   # Warmup: 3 iterations, single-shot each
   # Measurement: 5 iterations, single-shot each
   # Timeout: 10 min per iteration
   # Threads: 1 thread
   # Benchmark mode: Single shot invocation time
   # Benchmark: org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadFlatParquetDataBenchmark.readDatesIcebergVectorized5k
   
   # Run progress: 0.00% complete, ETA 00:00:00
   # Fork: 1 of 1
   # Warmup Iteration   1: 2.089 s/op
   # Warmup Iteration   2: 1.616 s/op
   # Warmup Iteration   3: 1.539 s/op
   Iteration   1: 1.561 s/op
   Iteration   2: 1.453 s/op
   Iteration   3: 1.483 s/op
   Iteration   4: 1.444 s/op
   Iteration   5: 1.482 s/op
   
   
   Result "org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadFlatParquetDataBenchmark.readDatesIcebergVectorized5k":
     N = 5
     mean =      1.485 ±(99.9%) 0.178 s/op
   
     Histogram, s/op:
       [1.400, 1.413) = 0 
       [1.413, 1.425) = 0 
       [1.425, 1.438) = 0 
       [1.438, 1.450) = 1 
       [1.450, 1.463) = 1 
       [1.463, 1.475) = 0 
       [1.475, 1.488) = 2 
       [1.488, 1.500) = 0 
       [1.500, 1.513) = 0 
       [1.513, 1.525) = 0 
       [1.525, 1.538) = 0 
       [1.538, 1.550) = 0 
       [1.550, 1.563) = 1 
       [1.563, 1.575) = 0 
       [1.575, 1.588) = 0 
   
     Percentiles, s/op:
         p(0.0000) =      1.444 s/op
        p(50.0000) =      1.482 s/op
        p(90.0000) =      1.561 s/op
        p(95.0000) =      1.561 s/op
        p(99.0000) =      1.561 s/op
        p(99.9000) =      1.561 s/op
        p(99.9900) =      1.561 s/op
        p(99.9990) =      1.561 s/op
        p(99.9999) =      1.561 s/op
       p(100.0000) =      1.561 s/op
   
   
   # JMH version: 1.21
   # VM version: JDK 1.8.0_282, OpenJDK 64-Bit Server VM, 25.282-b08
   # VM invoker: /usr/lib/jvm/java-8-openjdk-amd64/jre/bin/java
   # VM options: <none>
   # Warmup: 3 iterations, single-shot each
   # Measurement: 5 iterations, single-shot each
   # Timeout: 10 min per iteration
   # Threads: 1 thread
   # Benchmark mode: Single shot invocation time
   # Benchmark: org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadFlatParquetDataBenchmark.readDatesSparkVectorized5k
   
   # Run progress: 6.25% complete, ETA 00:52:39
   # Fork: 1 of 1
   # Warmup Iteration   1: 1.971 s/op
   # Warmup Iteration   2: 1.503 s/op
   # Warmup Iteration   3: 1.522 s/op
   Iteration   1: 1.476 s/op
   Iteration   2: 1.448 s/op
   Iteration   3: 1.490 s/op
   Iteration   4: 1.421 s/op
   Iteration   5: 1.480 s/op
   
   
   Result "org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadFlatParquetDataBenchmark.readDatesSparkVectorized5k":
     N = 5
     mean =      1.463 ±(99.9%) 0.108 s/op
   
     Histogram, s/op:
       [1.420, 1.425) = 1 
       [1.425, 1.430) = 0 
       [1.430, 1.435) = 0 
       [1.435, 1.440) = 0 
       [1.440, 1.445) = 0 
       [1.445, 1.450) = 1 
       [1.450, 1.455) = 0 
       [1.455, 1.460) = 0 
       [1.460, 1.465) = 0 
       [1.465, 1.470) = 0 
       [1.470, 1.475) = 0 
       [1.475, 1.480) = 2 
       [1.480, 1.485) = 0 
       [1.485, 1.490) = 0 
       [1.490, 1.495) = 1 
       [1.495, 1.500) = 0 
   
     Percentiles, s/op:
         p(0.0000) =      1.421 s/op
        p(50.0000) =      1.476 s/op
        p(90.0000) =      1.490 s/op
        p(95.0000) =      1.490 s/op
        p(99.0000) =      1.490 s/op
        p(99.9000) =      1.490 s/op
        p(99.9900) =      1.490 s/op
        p(99.9990) =      1.490 s/op
        p(99.9999) =      1.490 s/op
       p(100.0000) =      1.490 s/op
   
   
   # JMH version: 1.21
   # VM version: JDK 1.8.0_282, OpenJDK 64-Bit Server VM, 25.282-b08
   # VM invoker: /usr/lib/jvm/java-8-openjdk-amd64/jre/bin/java
   # VM options: <none>
   # Warmup: 3 iterations, single-shot each
   # Measurement: 5 iterations, single-shot each
   # Timeout: 10 min per iteration
   # Threads: 1 thread
   # Benchmark mode: Single shot invocation time
   # Benchmark: org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadFlatParquetDataBenchmark.readDecimalsIcebergVectorized5k
   
   # Run progress: 12.50% complete, ETA 00:50:03
   # Fork: 1 of 1
   # Warmup Iteration   1: 9.773 s/op
   # Warmup Iteration   2: 9.007 s/op
   # Warmup Iteration   3: 8.929 s/op
   Iteration   1: 8.875 s/op
   Iteration   2: 8.893 s/op
   Iteration   3: 8.946 s/op
   Iteration   4: 8.916 s/op
   Iteration   5: 8.880 s/op
   
   
   Result "org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadFlatParquetDataBenchmark.readDecimalsIcebergVectorized5k":
     N = 5
     mean =      8.902 ±(99.9%) 0.112 s/op
   
     Histogram, s/op:
       [8.870, 8.875) = 0 
       [8.875, 8.880) = 1 
       [8.880, 8.885) = 1 
       [8.885, 8.890) = 0 
       [8.890, 8.895) = 1 
       [8.895, 8.900) = 0 
       [8.900, 8.905) = 0 
       [8.905, 8.910) = 0 
       [8.910, 8.915) = 0 
       [8.915, 8.920) = 1 
       [8.920, 8.925) = 0 
       [8.925, 8.930) = 0 
       [8.930, 8.935) = 0 
       [8.935, 8.940) = 0 
       [8.940, 8.945) = 0 
       [8.945, 8.950) = 1 
   
     Percentiles, s/op:
         p(0.0000) =      8.875 s/op
        p(50.0000) =      8.893 s/op
        p(90.0000) =      8.946 s/op
        p(95.0000) =      8.946 s/op
        p(99.0000) =      8.946 s/op
        p(99.9000) =      8.946 s/op
        p(99.9900) =      8.946 s/op
        p(99.9990) =      8.946 s/op
        p(99.9999) =      8.946 s/op
       p(100.0000) =      8.946 s/op
   
   
   # JMH version: 1.21
   # VM version: JDK 1.8.0_282, OpenJDK 64-Bit Server VM, 25.282-b08
   # VM invoker: /usr/lib/jvm/java-8-openjdk-amd64/jre/bin/java
   # VM options: <none>
   # Warmup: 3 iterations, single-shot each
   # Measurement: 5 iterations, single-shot each
   # Timeout: 10 min per iteration
   # Threads: 1 thread
   # Benchmark mode: Single shot invocation time
   # Benchmark: org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadFlatParquetDataBenchmark.readDecimalsSparkVectorized5k
   
   # Run progress: 18.75% complete, ETA 00:50:50
   # Fork: 1 of 1
   # Warmup Iteration   1: 9.365 s/op
   # Warmup Iteration   2: 8.648 s/op
   # Warmup Iteration   3: 8.669 s/op
   Iteration   1: 8.648 s/op
   Iteration   2: 8.612 s/op
   Iteration   3: 8.655 s/op
   Iteration   4: 8.630 s/op
   Iteration   5: 8.647 s/op
   
   
   Result "org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadFlatParquetDataBenchmark.readDecimalsSparkVectorized5k":
     N = 5
     mean =      8.638 ±(99.9%) 0.067 s/op
   
     Histogram, s/op:
       [8.610, 8.615) = 1 
       [8.615, 8.620) = 0 
       [8.620, 8.625) = 0 
       [8.625, 8.630) = 0 
       [8.630, 8.635) = 1 
       [8.635, 8.640) = 0 
       [8.640, 8.645) = 0 
       [8.645, 8.650) = 2 
       [8.650, 8.655) = 1 
       [8.655, 8.660) = 0 
   
     Percentiles, s/op:
         p(0.0000) =      8.612 s/op
        p(50.0000) =      8.647 s/op
        p(90.0000) =      8.655 s/op
        p(95.0000) =      8.655 s/op
        p(99.0000) =      8.655 s/op
        p(99.9000) =      8.655 s/op
        p(99.9900) =      8.655 s/op
        p(99.9990) =      8.655 s/op
        p(99.9999) =      8.655 s/op
       p(100.0000) =      8.655 s/op
   
   
   # JMH version: 1.21
   # VM version: JDK 1.8.0_282, OpenJDK 64-Bit Server VM, 25.282-b08
   # VM invoker: /usr/lib/jvm/java-8-openjdk-amd64/jre/bin/java
   # VM options: <none>
   # Warmup: 3 iterations, single-shot each
   # Measurement: 5 iterations, single-shot each
   # Timeout: 10 min per iteration
   # Threads: 1 thread
   # Benchmark mode: Single shot invocation time
   # Benchmark: org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadFlatParquetDataBenchmark.readDoublesIcebergVectorized5k
   
   # Run progress: 25.00% complete, ETA 00:48:46
   # Fork: 1 of 1
   # Warmup Iteration   1: 3.312 s/op
   # Warmup Iteration   2: 2.751 s/op
   # Warmup Iteration   3: 2.661 s/op
   Iteration   1: 2.567 s/op
   Iteration   2: 2.612 s/op
   Iteration   3: 2.659 s/op
   Iteration   4: 2.594 s/op
   Iteration   5: 2.632 s/op
   
   
   Result "org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadFlatParquetDataBenchmark.readDoublesIcebergVectorized5k":
     N = 5
     mean =      2.613 ±(99.9%) 0.135 s/op
   
     Histogram, s/op:
       [2.560, 2.570) = 1 
       [2.570, 2.580) = 0 
       [2.580, 2.590) = 0 
       [2.590, 2.600) = 1 
       [2.600, 2.610) = 0 
       [2.610, 2.620) = 1 
       [2.620, 2.630) = 0 
       [2.630, 2.640) = 1 
       [2.640, 2.650) = 0 
       [2.650, 2.660) = 1 
   
     Percentiles, s/op:
         p(0.0000) =      2.567 s/op
        p(50.0000) =      2.612 s/op
        p(90.0000) =      2.659 s/op
        p(95.0000) =      2.659 s/op
        p(99.0000) =      2.659 s/op
        p(99.9000) =      2.659 s/op
        p(99.9900) =      2.659 s/op
        p(99.9990) =      2.659 s/op
        p(99.9999) =      2.659 s/op
       p(100.0000) =      2.659 s/op
   
   
   # JMH version: 1.21
   # VM version: JDK 1.8.0_282, OpenJDK 64-Bit Server VM, 25.282-b08
   # VM invoker: /usr/lib/jvm/java-8-openjdk-amd64/jre/bin/java
   # VM options: <none>
   # Warmup: 3 iterations, single-shot each
   # Measurement: 5 iterations, single-shot each
   # Timeout: 10 min per iteration
   # Threads: 1 thread
   # Benchmark mode: Single shot invocation time
   # Benchmark: org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadFlatParquetDataBenchmark.readDoublesSparkVectorized5k
   
   # Run progress: 31.25% complete, ETA 00:43:59
   # Fork: 1 of 1
   # Warmup Iteration   1: 3.136 s/op
   # Warmup Iteration   2: 2.547 s/op
   # Warmup Iteration   3: 2.515 s/op
   Iteration   1: 2.479 s/op
   Iteration   2: 2.409 s/op
   Iteration   3: 2.420 s/op
   Iteration   4: 2.394 s/op
   Iteration   5: 2.404 s/op
   
   
   Result "org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadFlatParquetDataBenchmark.readDoublesSparkVectorized5k":
     N = 5
     mean =      2.421 ±(99.9%) 0.131 s/op
   
     Histogram, s/op:
       [2.390, 2.395) = 1 
       [2.395, 2.400) = 0 
       [2.400, 2.405) = 1 
       [2.405, 2.410) = 1 
       [2.410, 2.415) = 0 
       [2.415, 2.420) = 1 
       [2.420, 2.425) = 0 
       [2.425, 2.430) = 0 
       [2.430, 2.435) = 0 
       [2.435, 2.440) = 0 
       [2.440, 2.445) = 0 
       [2.445, 2.450) = 0 
       [2.450, 2.455) = 0 
       [2.455, 2.460) = 0 
       [2.460, 2.465) = 0 
       [2.465, 2.470) = 0 
       [2.470, 2.475) = 0 
   
     Percentiles, s/op:
         p(0.0000) =      2.394 s/op
        p(50.0000) =      2.409 s/op
        p(90.0000) =      2.479 s/op
        p(95.0000) =      2.479 s/op
        p(99.0000) =      2.479 s/op
        p(99.9000) =      2.479 s/op
        p(99.9900) =      2.479 s/op
        p(99.9990) =      2.479 s/op
        p(99.9999) =      2.479 s/op
       p(100.0000) =      2.479 s/op
   
   
   # JMH version: 1.21
   # VM version: JDK 1.8.0_282, OpenJDK 64-Bit Server VM, 25.282-b08
   # VM invoker: /usr/lib/jvm/java-8-openjdk-amd64/jre/bin/java
   # VM options: <none>
   # Warmup: 3 iterations, single-shot each
   # Measurement: 5 iterations, single-shot each
   # Timeout: 10 min per iteration
   # Threads: 1 thread
   # Benchmark mode: Single shot invocation time
   # Benchmark: org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadFlatParquetDataBenchmark.readFloatsIcebergVectorized5k
   
   # Run progress: 37.50% complete, ETA 00:39:34
   # Fork: 1 of 1
   # Warmup Iteration   1: 3.092 s/op
   # Warmup Iteration   2: 2.515 s/op
   # Warmup Iteration   3: 2.491 s/op
   Iteration   1: 2.455 s/op
   Iteration   2: 2.465 s/op
   Iteration   3: 2.482 s/op
   Iteration   4: 2.434 s/op
   Iteration   5: 2.418 s/op
   
   
   Result "org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadFlatParquetDataBenchmark.readFloatsIcebergVectorized5k":
     N = 5
     mean =      2.451 ±(99.9%) 0.098 s/op
   
     Histogram, s/op:
       [2.410, 2.415) = 0 
       [2.415, 2.420) = 1 
       [2.420, 2.425) = 0 
       [2.425, 2.430) = 0 
       [2.430, 2.435) = 1 
       [2.435, 2.440) = 0 
       [2.440, 2.445) = 0 
       [2.445, 2.450) = 0 
       [2.450, 2.455) = 0 
       [2.455, 2.460) = 1 
       [2.460, 2.465) = 0 
       [2.465, 2.470) = 1 
       [2.470, 2.475) = 0 
       [2.475, 2.480) = 0 
       [2.480, 2.485) = 1 
       [2.485, 2.490) = 0 
   
     Percentiles, s/op:
         p(0.0000) =      2.418 s/op
        p(50.0000) =      2.455 s/op
        p(90.0000) =      2.482 s/op
        p(95.0000) =      2.482 s/op
        p(99.0000) =      2.482 s/op
        p(99.9000) =      2.482 s/op
        p(99.9900) =      2.482 s/op
        p(99.9990) =      2.482 s/op
        p(99.9999) =      2.482 s/op
       p(100.0000) =      2.482 s/op
   
   
   # JMH version: 1.21
   # VM version: JDK 1.8.0_282, OpenJDK 64-Bit Server VM, 25.282-b08
   # VM invoker: /usr/lib/jvm/java-8-openjdk-amd64/jre/bin/java
   # VM options: <none>
   # Warmup: 3 iterations, single-shot each
   # Measurement: 5 iterations, single-shot each
   # Timeout: 10 min per iteration
   # Threads: 1 thread
   # Benchmark mode: Single shot invocation time
   # Benchmark: org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadFlatParquetDataBenchmark.readFloatsSparkVectorized5k
   
   # Run progress: 43.75% complete, ETA 00:35:22
   # Fork: 1 of 1
   # Warmup Iteration   1: 3.106 s/op
   # Warmup Iteration   2: 2.391 s/op
   # Warmup Iteration   3: 2.364 s/op
   Iteration   1: 2.317 s/op
   Iteration   2: 2.261 s/op
   Iteration   3: 2.306 s/op
   Iteration   4: 2.272 s/op
   Iteration   5: 2.291 s/op
   
   
   Result "org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadFlatParquetDataBenchmark.readFloatsSparkVectorized5k":
     N = 5
     mean =      2.289 ±(99.9%) 0.089 s/op
   
     Histogram, s/op:
       [2.260, 2.265) = 1 
       [2.265, 2.270) = 0 
       [2.270, 2.275) = 1 
       [2.275, 2.280) = 0 
       [2.280, 2.285) = 0 
       [2.285, 2.290) = 0 
       [2.290, 2.295) = 1 
       [2.295, 2.300) = 0 
       [2.300, 2.305) = 0 
       [2.305, 2.310) = 1 
       [2.310, 2.315) = 0 
   
     Percentiles, s/op:
         p(0.0000) =      2.261 s/op
        p(50.0000) =      2.291 s/op
        p(90.0000) =      2.317 s/op
        p(95.0000) =      2.317 s/op
        p(99.0000) =      2.317 s/op
        p(99.9000) =      2.317 s/op
        p(99.9900) =      2.317 s/op
        p(99.9990) =      2.317 s/op
        p(99.9999) =      2.317 s/op
       p(100.0000) =      2.317 s/op
   
   
   # JMH version: 1.21
   # VM version: JDK 1.8.0_282, OpenJDK 64-Bit Server VM, 25.282-b08
   # VM invoker: /usr/lib/jvm/java-8-openjdk-amd64/jre/bin/java
   # VM options: <none>
   # Warmup: 3 iterations, single-shot each
   # Measurement: 5 iterations, single-shot each
   # Timeout: 10 min per iteration
   # Threads: 1 thread
   # Benchmark mode: Single shot invocation time
   # Benchmark: org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadFlatParquetDataBenchmark.readIntegersIcebergVectorized5k
   
   # Run progress: 50.00% complete, ETA 00:31:14
   # Fork: 1 of 1
   # Warmup Iteration   1: 3.161 s/op
   # Warmup Iteration   2: 2.559 s/op
   # Warmup Iteration   3: 2.539 s/op
   Iteration   1: 2.449 s/op
   Iteration   2: 2.454 s/op
   Iteration   3: 2.527 s/op
   Iteration   4: 2.474 s/op
   Iteration   5: 2.503 s/op
   
   
   Result "org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadFlatParquetDataBenchmark.readIntegersIcebergVectorized5k":
     N = 5
     mean =      2.481 ±(99.9%) 0.128 s/op
   
     Histogram, s/op:
       [2.440, 2.445) = 0 
       [2.445, 2.450) = 1 
       [2.450, 2.455) = 1 
       [2.455, 2.460) = 0 
       [2.460, 2.465) = 0 
       [2.465, 2.470) = 0 
       [2.470, 2.475) = 1 
       [2.475, 2.480) = 0 
       [2.480, 2.485) = 0 
       [2.485, 2.490) = 0 
       [2.490, 2.495) = 0 
       [2.495, 2.500) = 0 
       [2.500, 2.505) = 1 
       [2.505, 2.510) = 0 
       [2.510, 2.515) = 0 
       [2.515, 2.520) = 0 
       [2.520, 2.525) = 0 
       [2.525, 2.530) = 1 
   
     Percentiles, s/op:
         p(0.0000) =      2.449 s/op
        p(50.0000) =      2.474 s/op
        p(90.0000) =      2.527 s/op
        p(95.0000) =      2.527 s/op
        p(99.0000) =      2.527 s/op
        p(99.9000) =      2.527 s/op
        p(99.9900) =      2.527 s/op
        p(99.9990) =      2.527 s/op
        p(99.9999) =      2.527 s/op
       p(100.0000) =      2.527 s/op
   
   
   # JMH version: 1.21
   # VM version: JDK 1.8.0_282, OpenJDK 64-Bit Server VM, 25.282-b08
   # VM invoker: /usr/lib/jvm/java-8-openjdk-amd64/jre/bin/java
   # VM options: <none>
   # Warmup: 3 iterations, single-shot each
   # Measurement: 5 iterations, single-shot each
   # Timeout: 10 min per iteration
   # Threads: 1 thread
   # Benchmark mode: Single shot invocation time
   # Benchmark: org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadFlatParquetDataBenchmark.readIntegersSparkVectorized5k
   
   # Run progress: 56.25% complete, ETA 00:27:11
   # Fork: 1 of 1
   # Warmup Iteration   1: 3.138 s/op
   # Warmup Iteration   2: 2.358 s/op
   # Warmup Iteration   3: 2.344 s/op
   Iteration   1: 2.272 s/op
   Iteration   2: 2.251 s/op
   Iteration   3: 2.277 s/op
   Iteration   4: 2.233 s/op
   Iteration   5: 2.238 s/op
   
   
   Result "org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadFlatParquetDataBenchmark.readIntegersSparkVectorized5k":
     N = 5
     mean =      2.254 ±(99.9%) 0.076 s/op
   
     Histogram, s/op:
       [2.230, 2.235) = 1 
       [2.235, 2.240) = 1 
       [2.240, 2.245) = 0 
       [2.245, 2.250) = 0 
       [2.250, 2.255) = 1 
       [2.255, 2.260) = 0 
       [2.260, 2.265) = 0 
       [2.265, 2.270) = 0 
       [2.270, 2.275) = 1 
       [2.275, 2.280) = 1 
   
     Percentiles, s/op:
         p(0.0000) =      2.233 s/op
        p(50.0000) =      2.251 s/op
        p(90.0000) =      2.277 s/op
        p(95.0000) =      2.277 s/op
        p(99.0000) =      2.277 s/op
        p(99.9000) =      2.277 s/op
        p(99.9900) =      2.277 s/op
        p(99.9990) =      2.277 s/op
        p(99.9999) =      2.277 s/op
       p(100.0000) =      2.277 s/op
   
   
   # JMH version: 1.21
   # VM version: JDK 1.8.0_282, OpenJDK 64-Bit Server VM, 25.282-b08
   # VM invoker: /usr/lib/jvm/java-8-openjdk-amd64/jre/bin/java
   # VM options: <none>
   # Warmup: 3 iterations, single-shot each
   # Measurement: 5 iterations, single-shot each
   # Timeout: 10 min per iteration
   # Threads: 1 thread
   # Benchmark mode: Single shot invocation time
   # Benchmark: org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadFlatParquetDataBenchmark.readLongsIcebergVectorized5k
   
   # Run progress: 62.50% complete, ETA 00:23:12
   # Fork: 1 of 1
   # Warmup Iteration   1: 3.201 s/op
   # Warmup Iteration   2: 2.687 s/op
   # Warmup Iteration   3: 2.602 s/op
   Iteration   1: 2.505 s/op
   Iteration   2: 2.506 s/op
   Iteration   3: 2.585 s/op
   Iteration   4: 2.512 s/op
   Iteration   5: 2.565 s/op
   
   
   Result "org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadFlatParquetDataBenchmark.readLongsIcebergVectorized5k":
     N = 5
     mean =      2.535 ±(99.9%) 0.144 s/op
   
     Histogram, s/op:
       [2.500, 2.505) = 0 
       [2.505, 2.510) = 2 
       [2.510, 2.515) = 1 
       [2.515, 2.520) = 0 
       [2.520, 2.525) = 0 
       [2.525, 2.530) = 0 
       [2.530, 2.535) = 0 
       [2.535, 2.540) = 0 
       [2.540, 2.545) = 0 
       [2.545, 2.550) = 0 
       [2.550, 2.555) = 0 
       [2.555, 2.560) = 0 
       [2.560, 2.565) = 1 
       [2.565, 2.570) = 0 
       [2.570, 2.575) = 0 
       [2.575, 2.580) = 0 
       [2.580, 2.585) = 1 
   
     Percentiles, s/op:
         p(0.0000) =      2.505 s/op
        p(50.0000) =      2.512 s/op
        p(90.0000) =      2.585 s/op
        p(95.0000) =      2.585 s/op
        p(99.0000) =      2.585 s/op
        p(99.9000) =      2.585 s/op
        p(99.9900) =      2.585 s/op
        p(99.9990) =      2.585 s/op
        p(99.9999) =      2.585 s/op
       p(100.0000) =      2.585 s/op
   
   
   # JMH version: 1.21
   # VM version: JDK 1.8.0_282, OpenJDK 64-Bit Server VM, 25.282-b08
   # VM invoker: /usr/lib/jvm/java-8-openjdk-amd64/jre/bin/java
   # VM options: <none>
   # Warmup: 3 iterations, single-shot each
   # Measurement: 5 iterations, single-shot each
   # Timeout: 10 min per iteration
   # Threads: 1 thread
   # Benchmark mode: Single shot invocation time
   # Benchmark: org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadFlatParquetDataBenchmark.readLongsSparkVectorized5k
   
   # Run progress: 68.75% complete, ETA 00:19:18
   # Fork: 1 of 1
   # Warmup Iteration   1: 3.144 s/op
   # Warmup Iteration   2: 2.515 s/op
   # Warmup Iteration   3: 2.425 s/op
   Iteration   1: 2.421 s/op
   Iteration   2: 2.354 s/op
   Iteration   3: 2.362 s/op
   Iteration   4: 2.355 s/op
   Iteration   5: 2.328 s/op
   
   
   Result "org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadFlatParquetDataBenchmark.readLongsSparkVectorized5k":
     N = 5
     mean =      2.364 ±(99.9%) 0.132 s/op
   
     Histogram, s/op:
       [2.320, 2.330) = 1 
       [2.330, 2.340) = 0 
       [2.340, 2.350) = 0 
       [2.350, 2.360) = 2 
       [2.360, 2.370) = 1 
       [2.370, 2.380) = 0 
       [2.380, 2.390) = 0 
       [2.390, 2.400) = 0 
       [2.400, 2.410) = 0 
       [2.410, 2.420) = 0 
       [2.420, 2.430) = 1 
   
     Percentiles, s/op:
         p(0.0000) =      2.328 s/op
        p(50.0000) =      2.355 s/op
        p(90.0000) =      2.421 s/op
        p(95.0000) =      2.421 s/op
        p(99.0000) =      2.421 s/op
        p(99.9000) =      2.421 s/op
        p(99.9900) =      2.421 s/op
        p(99.9990) =      2.421 s/op
        p(99.9999) =      2.421 s/op
       p(100.0000) =      2.421 s/op
   
   
   # JMH version: 1.21
   # VM version: JDK 1.8.0_282, OpenJDK 64-Bit Server VM, 25.282-b08
   # VM invoker: /usr/lib/jvm/java-8-openjdk-amd64/jre/bin/java
   # VM options: <none>
   # Warmup: 3 iterations, single-shot each
   # Measurement: 5 iterations, single-shot each
   # Timeout: 10 min per iteration
   # Threads: 1 thread
   # Benchmark mode: Single shot invocation time
   # Benchmark: org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadFlatParquetDataBenchmark.readStringsIcebergVectorized5k
   
   # Run progress: 75.00% complete, ETA 00:15:25
   # Fork: 1 of 1
   # Warmup Iteration   1: 4.920 s/op
   # Warmup Iteration   2: 4.326 s/op
   # Warmup Iteration   3: 4.209 s/op
   Iteration   1: 4.156 s/op
   Iteration   2: 4.128 s/op
   Iteration   3: 4.224 s/op
   Iteration   4: 4.189 s/op
   Iteration   5: 4.182 s/op
   
   
   Result "org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadFlatParquetDataBenchmark.readStringsIcebergVectorized5k":
     N = 5
     mean =      4.176 ±(99.9%) 0.139 s/op
   
     Histogram, s/op:
       [4.120, 4.130) = 1 
       [4.130, 4.140) = 0 
       [4.140, 4.150) = 0 
       [4.150, 4.160) = 1 
       [4.160, 4.170) = 0 
       [4.170, 4.180) = 0 
       [4.180, 4.190) = 2 
       [4.190, 4.200) = 0 
       [4.200, 4.210) = 0 
       [4.210, 4.220) = 0 
       [4.220, 4.230) = 1 
   
     Percentiles, s/op:
         p(0.0000) =      4.128 s/op
        p(50.0000) =      4.182 s/op
        p(90.0000) =      4.224 s/op
        p(95.0000) =      4.224 s/op
        p(99.0000) =      4.224 s/op
        p(99.9000) =      4.224 s/op
        p(99.9900) =      4.224 s/op
        p(99.9990) =      4.224 s/op
        p(99.9999) =      4.224 s/op
       p(100.0000) =      4.224 s/op
   
   
   # JMH version: 1.21
   # VM version: JDK 1.8.0_282, OpenJDK 64-Bit Server VM, 25.282-b08
   # VM invoker: /usr/lib/jvm/java-8-openjdk-amd64/jre/bin/java
   # VM options: <none>
   # Warmup: 3 iterations, single-shot each
   # Measurement: 5 iterations, single-shot each
   # Timeout: 10 min per iteration
   # Threads: 1 thread
   # Benchmark mode: Single shot invocation time
   # Benchmark: org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadFlatParquetDataBenchmark.readStringsSparkVectorized5k
   
   # Run progress: 81.25% complete, ETA 00:11:36
   # Fork: 1 of 1
   # Warmup Iteration   1: 4.754 s/op
   # Warmup Iteration   2: 4.098 s/op
   # Warmup Iteration   3: 4.068 s/op
   Iteration   1: 4.017 s/op
   Iteration   2: 3.976 s/op
   Iteration   3: 3.989 s/op
   Iteration   4: 3.976 s/op
   Iteration   5: 3.989 s/op
   
   
   Result "org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadFlatParquetDataBenchmark.readStringsSparkVectorized5k":
     N = 5
     mean =      3.989 ±(99.9%) 0.065 s/op
   
     Histogram, s/op:
       [3.970, 3.975) = 0 
       [3.975, 3.980) = 2 
       [3.980, 3.985) = 0 
       [3.985, 3.990) = 2 
       [3.990, 3.995) = 0 
       [3.995, 4.000) = 0 
       [4.000, 4.005) = 0 
       [4.005, 4.010) = 0 
       [4.010, 4.015) = 0 
       [4.015, 4.020) = 1 
   
     Percentiles, s/op:
         p(0.0000) =      3.976 s/op
        p(50.0000) =      3.989 s/op
        p(90.0000) =      4.017 s/op
        p(95.0000) =      4.017 s/op
        p(99.0000) =      4.017 s/op
        p(99.9000) =      4.017 s/op
        p(99.9900) =      4.017 s/op
        p(99.9990) =      4.017 s/op
        p(99.9999) =      4.017 s/op
       p(100.0000) =      4.017 s/op
   
   
   # JMH version: 1.21
   # VM version: JDK 1.8.0_282, OpenJDK 64-Bit Server VM, 25.282-b08
   # VM invoker: /usr/lib/jvm/java-8-openjdk-amd64/jre/bin/java
   # VM options: <none>
   # Warmup: 3 iterations, single-shot each
   # Measurement: 5 iterations, single-shot each
   # Timeout: 10 min per iteration
   # Threads: 1 thread
   # Benchmark mode: Single shot invocation time
   # Benchmark: org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadFlatParquetDataBenchmark.readTimestampsIcebergVectorized5k
   
   # Run progress: 87.50% complete, ETA 00:07:45
   # Fork: 1 of 1
   # Warmup Iteration   1: 2.098 s/op
   # Warmup Iteration   2: 1.681 s/op
   # Warmup Iteration   3: 1.606 s/op
   Iteration   1: 1.543 s/op
   Iteration   2: 1.495 s/op
   Iteration   3: 1.554 s/op
   Iteration   4: 1.550 s/op
   Iteration   5: 1.530 s/op
   
   
   Result "org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadFlatParquetDataBenchmark.readTimestampsIcebergVectorized5k":
     N = 5
     mean =      1.534 ±(99.9%) 0.091 s/op
   
     Histogram, s/op:
       [1.490, 1.495) = 0 
       [1.495, 1.500) = 1 
       [1.500, 1.505) = 0 
       [1.505, 1.510) = 0 
       [1.510, 1.515) = 0 
       [1.515, 1.520) = 0 
       [1.520, 1.525) = 0 
       [1.525, 1.530) = 1 
       [1.530, 1.535) = 0 
       [1.535, 1.540) = 0 
       [1.540, 1.545) = 1 
       [1.545, 1.550) = 1 
       [1.550, 1.555) = 1 
       [1.555, 1.560) = 0 
   
     Percentiles, s/op:
         p(0.0000) =      1.495 s/op
        p(50.0000) =      1.543 s/op
        p(90.0000) =      1.554 s/op
        p(95.0000) =      1.554 s/op
        p(99.0000) =      1.554 s/op
        p(99.9000) =      1.554 s/op
        p(99.9900) =      1.554 s/op
        p(99.9990) =      1.554 s/op
        p(99.9999) =      1.554 s/op
       p(100.0000) =      1.554 s/op
   
   
   # JMH version: 1.21
   # VM version: JDK 1.8.0_282, OpenJDK 64-Bit Server VM, 25.282-b08
   # VM invoker: /usr/lib/jvm/java-8-openjdk-amd64/jre/bin/java
   # VM options: <none>
   # Warmup: 3 iterations, single-shot each
   # Measurement: 5 iterations, single-shot each
   # Timeout: 10 min per iteration
   # Threads: 1 thread
   # Benchmark mode: Single shot invocation time
   # Benchmark: org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadFlatParquetDataBenchmark.readTimestampsSparkVectorized5k
   
   # Run progress: 93.75% complete, ETA 00:03:51
   # Fork: 1 of 1
   # Warmup Iteration   1: 2.094 s/op
   # Warmup Iteration   2: 1.539 s/op
   # Warmup Iteration   3: 1.491 s/op
   Iteration   1: 1.493 s/op
   Iteration   2: 1.455 s/op
   Iteration   3: 1.482 s/op
   Iteration   4: 1.458 s/op
   Iteration   5: 1.416 s/op
   
   
   Result "org.apache.iceberg.spark.source.parquet.vectorized.VectorizedReadFlatParquetDataBenchmark.readTimestampsSparkVectorized5k":
     N = 5
     mean =      1.461 ±(99.9%) 0.114 s/op
   
     Histogram, s/op:
       [1.410, 1.415) = 0 
       [1.415, 1.420) = 1 
       [1.420, 1.425) = 0 
       [1.425, 1.430) = 0 
       [1.430, 1.435) = 0 
       [1.435, 1.440) = 0 
       [1.440, 1.445) = 0 
       [1.445, 1.450) = 0 
       [1.450, 1.455) = 0 
       [1.455, 1.460) = 2 
       [1.460, 1.465) = 0 
       [1.465, 1.470) = 0 
       [1.470, 1.475) = 0 
       [1.475, 1.480) = 0 
       [1.480, 1.485) = 1 
       [1.485, 1.490) = 0 
       [1.490, 1.495) = 1 
       [1.495, 1.500) = 0 
   
     Percentiles, s/op:
         p(0.0000) =      1.416 s/op
        p(50.0000) =      1.458 s/op
        p(90.0000) =      1.493 s/op
        p(95.0000) =      1.493 s/op
        p(99.0000) =      1.493 s/op
        p(99.9000) =      1.493 s/op
        p(99.9900) =      1.493 s/op
        p(99.9990) =      1.493 s/op
        p(99.9999) =      1.493 s/op
       p(100.0000) =      1.493 s/op
   
   
   # Run complete. Total time: 01:01:35
   
   REMEMBER: The numbers below are just data. To gain reusable insights, you need to follow up on
   why the numbers are the way they are. Use profilers (see -prof, -lprof), design factorial
   experiments, perform baseline and negative tests that provide experimental control, make sure
   the benchmarking environment is safe on JVM/OS/HW level, ask for reviews from the domain experts.
   Do not assume the numbers tell you what you want them to tell.
   
   Benchmark                                                                 Mode  Cnt  Score   Error  Units
   VectorizedReadFlatParquetDataBenchmark.readDatesIcebergVectorized5k         ss    5  1.485 ± 0.178   s/op
   VectorizedReadFlatParquetDataBenchmark.readDatesSparkVectorized5k           ss    5  1.463 ± 0.108   s/op
   VectorizedReadFlatParquetDataBenchmark.readDecimalsIcebergVectorized5k      ss    5  8.902 ± 0.112   s/op
   VectorizedReadFlatParquetDataBenchmark.readDecimalsSparkVectorized5k        ss    5  8.638 ± 0.067   s/op
   VectorizedReadFlatParquetDataBenchmark.readDoublesIcebergVectorized5k       ss    5  2.613 ± 0.135   s/op
   VectorizedReadFlatParquetDataBenchmark.readDoublesSparkVectorized5k         ss    5  2.421 ± 0.131   s/op
   VectorizedReadFlatParquetDataBenchmark.readFloatsIcebergVectorized5k        ss    5  2.451 ± 0.098   s/op
   VectorizedReadFlatParquetDataBenchmark.readFloatsSparkVectorized5k          ss    5  2.289 ± 0.089   s/op
   VectorizedReadFlatParquetDataBenchmark.readIntegersIcebergVectorized5k      ss    5  2.481 ± 0.128   s/op
   VectorizedReadFlatParquetDataBenchmark.readIntegersSparkVectorized5k        ss    5  2.254 ± 0.076   s/op
   VectorizedReadFlatParquetDataBenchmark.readLongsIcebergVectorized5k         ss    5  2.535 ± 0.144   s/op
   VectorizedReadFlatParquetDataBenchmark.readLongsSparkVectorized5k           ss    5  2.364 ± 0.132   s/op
   VectorizedReadFlatParquetDataBenchmark.readStringsIcebergVectorized5k       ss    5  4.176 ± 0.139   s/op
   VectorizedReadFlatParquetDataBenchmark.readStringsSparkVectorized5k         ss    5  3.989 ± 0.065   s/op
   VectorizedReadFlatParquetDataBenchmark.readTimestampsIcebergVectorized5k    ss    5  1.534 ± 0.091   s/op
   VectorizedReadFlatParquetDataBenchmark.readTimestampsSparkVectorized5k      ss    5  1.461 ± 0.114   s/op
   
   Benchmark result is saved to XXX/spark2/build/reports/jmh/results.txt
   
   ```
   


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#discussion_r611397994



##########
File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowVector.java
##########
@@ -0,0 +1,513 @@
+/*
+ * 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.arrow.vectorized;
+
+import java.nio.charset.StandardCharsets;
+import java.util.stream.IntStream;
+import org.apache.arrow.vector.BigIntVector;
+import org.apache.arrow.vector.BitVector;
+import org.apache.arrow.vector.DateDayVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.Float4Vector;
+import org.apache.arrow.vector.Float8Vector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.TimeStampMicroTZVector;
+import org.apache.arrow.vector.TimeStampMicroVector;
+import org.apache.arrow.vector.ValueVector;
+import org.apache.arrow.vector.VarBinaryVector;
+import org.apache.arrow.vector.VarCharVector;
+import org.apache.arrow.vector.holders.NullableVarCharHolder;
+import org.apache.iceberg.types.Types;
+import org.apache.parquet.Preconditions;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.PrimitiveType;
+
+/**
+ * This class is inspired by Spark's {@code ColumnVector}.
+ * This class represents the column data for an Iceberg table query.
+ * It wraps an arrow {@link FieldVector} and provides simple
+ * accessors for the row values. Advanced users can access
+ * the {@link FieldVector}.
+ * <p>
+ *   Supported Iceberg data types:
+ *   <ul>
+ *     <li>{@link Types.BooleanType}</li>
+ *     <li>{@link Types.IntegerType}</li>
+ *     <li>{@link Types.LongType}</li>
+ *     <li>{@link Types.FloatType}</li>
+ *     <li>{@link Types.DoubleType}</li>
+ *     <li>{@link Types.StringType}</li>
+ *     <li>{@link Types.BinaryType}</li>
+ *     <li>{@link Types.TimestampType} (with and without timezone)</li>
+ *     <li>{@link Types.DateType}</li>
+ *   </ul>
+ */
+public class ArrowVector implements AutoCloseable {

Review comment:
       Sounds good @mayursrivastava I concur.




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] mayursrivastava commented on a change in pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
mayursrivastava commented on a change in pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#discussion_r610803813



##########
File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowVector.java
##########
@@ -0,0 +1,513 @@
+/*
+ * 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.arrow.vectorized;
+
+import java.nio.charset.StandardCharsets;
+import java.util.stream.IntStream;
+import org.apache.arrow.vector.BigIntVector;
+import org.apache.arrow.vector.BitVector;
+import org.apache.arrow.vector.DateDayVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.Float4Vector;
+import org.apache.arrow.vector.Float8Vector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.TimeStampMicroTZVector;
+import org.apache.arrow.vector.TimeStampMicroVector;
+import org.apache.arrow.vector.ValueVector;
+import org.apache.arrow.vector.VarBinaryVector;
+import org.apache.arrow.vector.VarCharVector;
+import org.apache.arrow.vector.holders.NullableVarCharHolder;
+import org.apache.iceberg.types.Types;
+import org.apache.parquet.Preconditions;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.Dictionary;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.PrimitiveType;
+
+/**
+ * This class is inspired by Spark's {@code ColumnVector}.
+ * This class represents the column data for an Iceberg table query.
+ * It wraps an arrow {@link FieldVector} and provides simple
+ * accessors for the row values. Advanced users can access
+ * the {@link FieldVector}.
+ * <p>
+ *   Supported Iceberg data types:
+ *   <ul>
+ *     <li>{@link Types.BooleanType}</li>
+ *     <li>{@link Types.IntegerType}</li>
+ *     <li>{@link Types.LongType}</li>
+ *     <li>{@link Types.FloatType}</li>
+ *     <li>{@link Types.DoubleType}</li>
+ *     <li>{@link Types.StringType}</li>
+ *     <li>{@link Types.BinaryType}</li>
+ *     <li>{@link Types.TimestampType} (with and without timezone)</li>
+ *     <li>{@link Types.DateType}</li>
+ *   </ul>
+ */
+public class ArrowVector implements AutoCloseable {

Review comment:
       After looking at the source code, I think we can move ArrowVectorAccessor and ArrowVectorAccessors to arrow module. These classes have most of the common conversion code. I'll change the renamed ArrowVector (to ColumnVector) in the arrow module  to use ArrowVectorAccessor. I'll do this and update the pr, let me know how it looks.
   
   Moving IcebergArrowColumnVector from spark code to here require some dependency changes because the interface/abstract class ColumnVector lives in spark-sql.




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
rymurr commented on pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#issuecomment-828395638


   Awesome, thanks @mayursrivastava I will take one quick pass but I think its good to go! Thanks for your patience!


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] mayursrivastava commented on a change in pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
mayursrivastava commented on a change in pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#discussion_r591984269



##########
File path: data/src/main/java/org/apache/iceberg/data/arrow/VectorizedParquetReaders.java
##########
@@ -0,0 +1,155 @@
+/*
+ * 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.data.arrow;
+
+import java.util.List;
+import java.util.Map;
+import java.util.stream.IntStream;
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.arrow.ArrowAllocation;
+import org.apache.iceberg.arrow.vectorized.VectorizedArrowReader;
+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.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+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;
+
+/**
+ * Builds an {@link VectorSchemaRootReader}.
+ */
+public class VectorizedParquetReaders {

Review comment:
       I'll refactor the internal builder that can be reused in both arrow and spark




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] mayursrivastava commented on a change in pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
mayursrivastava commented on a change in pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#discussion_r682565585



##########
File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowReader.java
##########
@@ -0,0 +1,325 @@
+/*
+ * 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.arrow.vectorized;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import java.util.stream.StreamSupport;
+import org.apache.arrow.vector.NullCheckingForGet;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.arrow.vector.types.Types.MinorType;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.encryption.EncryptedFiles;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.io.CloseableGroup;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.CloseableIterator;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.parquet.TypeWithSchemaVisitor;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.types.Types;
+import org.apache.parquet.schema.MessageType;
+
+/**
+ * Vectorized reader that returns an iterator of {@link ColumnarBatch}.
+ * See {@link #open(CloseableIterable)} ()} to learn about the
+ * behavior of the iterator.
+ *
+ * <p>The following Iceberg data types are supported and have been tested:
+ * <ul>
+ *     <li>Iceberg: {@link Types.BooleanType}, Arrow: {@link MinorType#BIT}</li>
+ *     <li>Iceberg: {@link Types.IntegerType}, Arrow: {@link MinorType#INT}</li>
+ *     <li>Iceberg: {@link Types.LongType}, Arrow: {@link MinorType#BIGINT}</li>
+ *     <li>Iceberg: {@link Types.FloatType}, Arrow: {@link MinorType#FLOAT4}</li>
+ *     <li>Iceberg: {@link Types.DoubleType}, Arrow: {@link MinorType#FLOAT8}</li>
+ *     <li>Iceberg: {@link Types.StringType}, Arrow: {@link MinorType#VARCHAR}</li>
+ *     <li>Iceberg: {@link Types.TimestampType} (both with and without timezone),
+ *         Arrow: {@link MinorType#TIMEMICRO}</li>
+ *     <li>Iceberg: {@link Types.BinaryType}, Arrow: {@link MinorType#VARBINARY}</li>
+ *     <li>Iceberg: {@link Types.DateType}, Arrow: {@link MinorType#DATEDAY}</li>
+ * </ul>
+ *
+ * <p>Features that don't work in this implementation:
+ * <ul>
+ *     <li>Type promotion: In case of type promotion, the Arrow vector corresponding to
+ *     the data type in the parquet file is returned instead of the data type in the latest schema.
+ *     See https://github.com/apache/iceberg/issues/2483.</li>
+ *     <li>Columns with constant values are physically encoded as a dictionary. The Arrow vector
+ *     type is int32 instead of the type as per the schema.
+ *     See https://github.com/apache/iceberg/issues/2484.</li>
+ *     <li>Data types: {@link Types.TimeType}, {@link Types.ListType}, {@link Types.MapType},
+ *     {@link Types.StructType}, {@link Types.UUIDType}, {@link Types.FixedType} and
+ *     {@link Types.DecimalType}
+ *     See https://github.com/apache/iceberg/issues/2485 and https://github.com/apache/iceberg/issues/2486.</li>
+ *     <li>Iceberg v2 spec is not supported.
+ *     See https://github.com/apache/iceberg/issues/2487.</li>

Review comment:
       The fixes are present in https://github.com/apache/iceberg/pull/2933.
   I've added checks similar to the Spark reader before allowing vectorized reader. 
   
   The arrow reader will check for the following conditions to be met:
   1. At least 1 column is queried.
   2. There are no delete files.
   3. Data types in columns are supported.
   
   I've also changed the reader to not process delete files because they don't work anyways with this reader. 
   




-- 
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-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#discussion_r610715371



##########
File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowBatch.java
##########
@@ -0,0 +1,82 @@
+/*
+ * 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.arrow.vectorized;
+
+import java.util.Arrays;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.VectorSchemaRoot;
+
+/**
+ * This class is inspired by Spark's {@code ColumnarBatch}.
+ * This class wraps a columnar batch in the result set of an Iceberg table query.
+ */
+public class ArrowBatch implements AutoCloseable {

Review comment:
       Just a rename is ok. I was just aiming to keep the word `Arrow` out of the class name




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#discussion_r639541557



##########
File path: arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ColumnVector.java
##########
@@ -0,0 +1,116 @@
+/*
+ * 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.arrow.vectorized;
+
+import org.apache.arrow.vector.FieldVector;
+import org.apache.iceberg.types.Types;
+
+/**
+ * This class is inspired by Spark's {@code ColumnVector}.
+ * This class represents the column data for an Iceberg table query.
+ * It wraps an arrow {@link FieldVector} and provides simple
+ * accessors for the row values. Advanced users can access
+ * the {@link FieldVector}.
+ * <p>
+ *   Supported Iceberg data types:
+ *   <ul>
+ *     <li>{@link Types.BooleanType}</li>
+ *     <li>{@link Types.IntegerType}</li>
+ *     <li>{@link Types.LongType}</li>
+ *     <li>{@link Types.FloatType}</li>
+ *     <li>{@link Types.DoubleType}</li>
+ *     <li>{@link Types.StringType}</li>
+ *     <li>{@link Types.BinaryType}</li>
+ *     <li>{@link Types.TimestampType} (with and without timezone)</li>
+ *     <li>{@link Types.DateType}</li>
+ *   </ul>
+ */
+public class ColumnVector implements AutoCloseable {

Review comment:
       > My concern is that we're introducing a new API that is specific to Iceberg and will need to be supported for a long time.
   
   This was my concern as well. I figured that a new api was better than having to be coupled to any changes arrow may make. At least we have control over how this API will evolve. Perhaps some javadoc to that point and to clearly state the intention of this API?
   
   The other goal I had was to assist the user of this API with Arrow buffer lifetime/usage, which I have found to be finicky on occasion. I don't think Iceberg users should require knowledge of Arrow Vectors just to use the vectorised readers.
   
   Having said that I don't feel that strong and if the feeling in the community is its not a useful abstraction then I happily agree.




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] mayursrivastava commented on pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
mayursrivastava commented on pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#issuecomment-824225827


   The benchmark was failing on the master. Currently, I'm running through wsl2/ubuntu. I'll provision another linux machine to try it again. 


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rymurr commented on a change in pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
rymurr commented on a change in pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#discussion_r618358578



##########
File path: build.gradle
##########
@@ -735,9 +735,12 @@ project(':iceberg-arrow') {
       exclude group: 'com.google.code.findbugs', module: 'jsr305'
     }
     compile("org.apache.arrow:arrow-memory-netty") {
-      exclude group: 'io.netty', module: 'netty-common'
       exclude group: 'com.google.code.findbugs', module: 'jsr305'

Review comment:
       cant't we just add `testCompile("io.netty:netty-common)`? How come we have to re-add the whole dependency in test?




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] mayursrivastava commented on a change in pull request #2286: Add Arrow vectorized reader

Posted by GitBox <gi...@apache.org>.
mayursrivastava commented on a change in pull request #2286:
URL: https://github.com/apache/iceberg/pull/2286#discussion_r614914293



##########
File path: build.gradle
##########
@@ -738,6 +738,13 @@ project(':iceberg-arrow') {
       exclude group: 'io.netty', module: 'netty-common'
       exclude group: 'com.google.code.findbugs', module: 'jsr305'
     }
+
+    testCompile("org.apache.arrow:arrow-memory-netty") {

Review comment:
       I will remove testCompile instance.




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org