You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by "pavibhai (via GitHub)" <gi...@apache.org> on 2023/03/24 16:02:37 UTC

[GitHub] [iceberg] pavibhai opened a new pull request, #7197: [ORC] - Support selected vector with ORC reader on the row and batch reader

pavibhai opened a new pull request, #7197:
URL: https://github.com/apache/iceberg/pull/7197

   ## What?
   Currently Iceberg does not support the use of the selected vector when reading ORC Files. This requires reads on ORC to be run in compatibility mode by not setting `orc.filter.use.selected` in the presence of filter processing that is triggered via `orc.sarg.to.filter`.
   
   Filter processing was introduced as part of [ORC-744](https://issues.apache.org/jira/browse/ORC-744) where ORC has the ability to filter out records and indicate this status in partially filtered batches using the selected vector in VectorizedRowBatch.
   
   This PR uses the selected vector to determine valid rows when applicable.
   
   ## Why?
   ORC can only operate in compatibility mode by not setting . Enabling this will further hasten the processing of rows by ignoring rows that are already filtered out in the batch.
   
   ## Tested?
   New Unit tests have been added to verify the behavior.


-- 
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] aokolnychyi commented on pull request #7197: [ORC][Spark] - Support selected vector with ORC reader on the row and batch reader

Posted by "aokolnychyi (via GitHub)" <gi...@apache.org>.
aokolnychyi commented on PR #7197:
URL: https://github.com/apache/iceberg/pull/7197#issuecomment-1509444906

   Thanks, @pavibhai!


-- 
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] pavibhai commented on a diff in pull request #7197: [ORC][Spark] - Support selected vector with ORC reader on the row and batch reader

Posted by "pavibhai (via GitHub)" <gi...@apache.org>.
pavibhai commented on code in PR #7197:
URL: https://github.com/apache/iceberg/pull/7197#discussion_r1152007471


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkOrcReaders.java:
##########
@@ -209,7 +229,8 @@ private int numNullsHelper() {
     }
 
     protected int getRowIndex(int rowId) {
-      return vector.isRepeating ? 0 : rowId;
+      int row = isSelectedInUse ? selected[rowId] : rowId;

Review Comment:
   No, this is not a possibility. Similar to other vectors the selected vector is initialized at the start to max batch size and all of the indices in this are excepted to be within that size.



-- 
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] dpaani commented on a diff in pull request #7197: [ORC][Spark] - Support selected vector with ORC reader on the row and batch reader

Posted by "dpaani (via GitHub)" <gi...@apache.org>.
dpaani commented on code in PR #7197:
URL: https://github.com/apache/iceberg/pull/7197#discussion_r1150878213


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/data/TestVectorizedOrcDataReader.java:
##########
@@ -0,0 +1,166 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.spark.data;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.data.orc.GenericOrcWriter;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.DataWriter;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.orc.ORC;
+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.Iterators;
+import org.apache.iceberg.spark.data.vectorized.VectorizedSparkOrcReaders;
+import org.apache.iceberg.types.Types;
+import org.apache.orc.OrcConf;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.vectorized.ColumnarBatch;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+
+public class TestVectorizedOrcDataReader {
+  @TempDir public static Path temp;
+
+  private static final Schema SCHEMA =
+      new Schema(
+          Types.NestedField.required(1, "id", Types.LongType.get()),
+          Types.NestedField.optional(2, "data", Types.StringType.get()),
+          Types.NestedField.optional(3, "binary", Types.BinaryType.get()),
+          Types.NestedField.required(
+              4, "array", Types.ListType.ofOptional(5, Types.IntegerType.get())));
+  private static OutputFile outputFile;
+
+  @BeforeAll
+  public static void createDataFile() throws IOException {
+    GenericRecord bufferRecord = GenericRecord.create(SCHEMA);
+
+    ImmutableList.Builder<Record> builder = ImmutableList.builder();
+    builder.add(
+        bufferRecord.copy(
+            ImmutableMap.of("id", 1L, "data", "a", "array", Collections.singletonList(1))));
+    builder.add(
+        bufferRecord.copy(ImmutableMap.of("id", 2L, "data", "b", "array", Arrays.asList(2, 3))));
+    builder.add(
+        bufferRecord.copy(ImmutableMap.of("id", 3L, "data", "c", "array", Arrays.asList(3, 4, 5))));
+    builder.add(
+        bufferRecord.copy(
+            ImmutableMap.of("id", 4L, "data", "d", "array", Arrays.asList(4, 5, 6, 7))));
+    builder.add(
+        bufferRecord.copy(
+            ImmutableMap.of("id", 5L, "data", "e", "array", Arrays.asList(5, 6, 7, 8, 9))));
+
+    outputFile = Files.localOutput(File.createTempFile("test", ".orc", temp.toFile()));
+
+    try (DataWriter<Record> dataWriter =
+        ORC.writeData(outputFile)
+            .schema(SCHEMA)
+            .createWriterFunc(GenericOrcWriter::buildWriter)
+            .overwrite()
+            .withSpec(PartitionSpec.unpartitioned())
+            .build()) {
+      for (Record record : builder.build()) {
+        dataWriter.write(record);
+      }
+    }
+  }
+
+  private Iterator<InternalRow> batchesToRows(Iterator<ColumnarBatch> batches) {
+    return Iterators.concat(Iterators.transform(batches, ColumnarBatch::rowIterator));
+  }
+
+  private void validateAllRows(Iterator<InternalRow> rows) {
+    long rowCount = 0;
+    long expId = 1;
+    char expChar = 'a';
+    while (rows.hasNext()) {
+      InternalRow row = rows.next();
+      Assertions.assertEquals(expId, row.getLong(0));
+      Assertions.assertEquals(expChar, row.getString(1).charAt(0));
+      Assertions.assertTrue(row.isNullAt(2));
+      expId += 1;
+      expChar += 1;
+      rowCount += 1;
+    }
+    Assertions.assertEquals(5, rowCount);
+  }
+
+  @Test
+  public void testVectorizedReader() throws IOException {
+    try (CloseableIterable<ColumnarBatch> reader =
+        ORC.read(outputFile.toInputFile())
+            .project(SCHEMA)
+            .createBatchedReaderFunc(
+                readOrcSchema ->
+                    VectorizedSparkOrcReaders.buildReader(SCHEMA, readOrcSchema, ImmutableMap.of()))
+            .build()) {
+      validateAllRows(batchesToRows(reader.iterator()));
+    }
+  }
+
+  @Test
+  public void testRowReaderWithFilter() throws IOException {
+    try (CloseableIterable<ColumnarBatch> reader =
+        ORC.read(outputFile.toInputFile())
+            .project(SCHEMA)
+            .createBatchedReaderFunc(
+                readOrcSchema ->
+                    VectorizedSparkOrcReaders.buildReader(SCHEMA, readOrcSchema, ImmutableMap.of()))
+            .filter(Expressions.equal("id", 3L))
+            .config(OrcConf.ALLOW_SARG_TO_FILTER.getAttribute(), String.valueOf(true))
+            .build()) {
+      validateAllRows(batchesToRows(reader.iterator()));
+    }
+  }
+
+  @Test
+  public void testRowReaderWithFilterWithSelected() throws IOException {
+    try (CloseableIterable<ColumnarBatch> reader =
+        ORC.read(outputFile.toInputFile())
+            .project(SCHEMA)
+            .createBatchedReaderFunc(
+                readOrcSchema ->
+                    VectorizedSparkOrcReaders.buildReader(SCHEMA, readOrcSchema, ImmutableMap.of()))
+            .filter(Expressions.equal("id", 3L))
+            .config(OrcConf.ALLOW_SARG_TO_FILTER.getAttribute(), String.valueOf(true))
+            .config(OrcConf.READER_USE_SELECTED.getAttribute(), String.valueOf(true))
+            .build()) {
+      Iterator<InternalRow> rows = batchesToRows(reader.iterator());
+      Assertions.assertTrue(rows.hasNext());
+      InternalRow row = rows.next();
+      Assertions.assertEquals(3L, row.getLong(0));
+      Assertions.assertEquals("c", row.getString(1));
+      Assertions.assertArrayEquals(new int[] {3, 4, 5}, row.getArray(3).toIntArray());

Review Comment:
   hi @pavibhai there was a recommendation to use assertj for assertions - https://github.com/apache/iceberg/blob/master/CONTRIBUTING.md#assertj
   



-- 
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] pavibhai commented on a diff in pull request #7197: [ORC][Spark] - Support selected vector with ORC reader on the row and batch reader

Posted by "pavibhai (via GitHub)" <gi...@apache.org>.
pavibhai commented on code in PR #7197:
URL: https://github.com/apache/iceberg/pull/7197#discussion_r1152541943


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/data/TestVectorizedOrcDataReader.java:
##########
@@ -0,0 +1,166 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.spark.data;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.data.orc.GenericOrcWriter;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.DataWriter;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.orc.ORC;
+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.Iterators;
+import org.apache.iceberg.spark.data.vectorized.VectorizedSparkOrcReaders;
+import org.apache.iceberg.types.Types;
+import org.apache.orc.OrcConf;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.vectorized.ColumnarBatch;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+
+public class TestVectorizedOrcDataReader {
+  @TempDir public static Path temp;
+
+  private static final Schema SCHEMA =
+      new Schema(
+          Types.NestedField.required(1, "id", Types.LongType.get()),
+          Types.NestedField.optional(2, "data", Types.StringType.get()),
+          Types.NestedField.optional(3, "binary", Types.BinaryType.get()),
+          Types.NestedField.required(
+              4, "array", Types.ListType.ofOptional(5, Types.IntegerType.get())));
+  private static OutputFile outputFile;
+
+  @BeforeAll
+  public static void createDataFile() throws IOException {
+    GenericRecord bufferRecord = GenericRecord.create(SCHEMA);
+
+    ImmutableList.Builder<Record> builder = ImmutableList.builder();
+    builder.add(
+        bufferRecord.copy(
+            ImmutableMap.of("id", 1L, "data", "a", "array", Collections.singletonList(1))));
+    builder.add(
+        bufferRecord.copy(ImmutableMap.of("id", 2L, "data", "b", "array", Arrays.asList(2, 3))));
+    builder.add(
+        bufferRecord.copy(ImmutableMap.of("id", 3L, "data", "c", "array", Arrays.asList(3, 4, 5))));
+    builder.add(
+        bufferRecord.copy(
+            ImmutableMap.of("id", 4L, "data", "d", "array", Arrays.asList(4, 5, 6, 7))));
+    builder.add(
+        bufferRecord.copy(
+            ImmutableMap.of("id", 5L, "data", "e", "array", Arrays.asList(5, 6, 7, 8, 9))));
+
+    outputFile = Files.localOutput(File.createTempFile("test", ".orc", temp.toFile()));
+
+    try (DataWriter<Record> dataWriter =
+        ORC.writeData(outputFile)
+            .schema(SCHEMA)
+            .createWriterFunc(GenericOrcWriter::buildWriter)
+            .overwrite()
+            .withSpec(PartitionSpec.unpartitioned())
+            .build()) {
+      for (Record record : builder.build()) {
+        dataWriter.write(record);
+      }
+    }
+  }
+
+  private Iterator<InternalRow> batchesToRows(Iterator<ColumnarBatch> batches) {
+    return Iterators.concat(Iterators.transform(batches, ColumnarBatch::rowIterator));
+  }
+
+  private void validateAllRows(Iterator<InternalRow> rows) {
+    long rowCount = 0;
+    long expId = 1;
+    char expChar = 'a';
+    while (rows.hasNext()) {
+      InternalRow row = rows.next();
+      Assertions.assertEquals(expId, row.getLong(0));
+      Assertions.assertEquals(expChar, row.getString(1).charAt(0));
+      Assertions.assertTrue(row.isNullAt(2));
+      expId += 1;
+      expChar += 1;
+      rowCount += 1;
+    }
+    Assertions.assertEquals(5, rowCount);
+  }
+
+  @Test
+  public void testVectorizedReader() throws IOException {
+    try (CloseableIterable<ColumnarBatch> reader =
+        ORC.read(outputFile.toInputFile())
+            .project(SCHEMA)
+            .createBatchedReaderFunc(
+                readOrcSchema ->
+                    VectorizedSparkOrcReaders.buildReader(SCHEMA, readOrcSchema, ImmutableMap.of()))
+            .build()) {
+      validateAllRows(batchesToRows(reader.iterator()));
+    }
+  }
+
+  @Test
+  public void testRowReaderWithFilter() throws IOException {
+    try (CloseableIterable<ColumnarBatch> reader =
+        ORC.read(outputFile.toInputFile())
+            .project(SCHEMA)
+            .createBatchedReaderFunc(
+                readOrcSchema ->
+                    VectorizedSparkOrcReaders.buildReader(SCHEMA, readOrcSchema, ImmutableMap.of()))
+            .filter(Expressions.equal("id", 3L))
+            .config(OrcConf.ALLOW_SARG_TO_FILTER.getAttribute(), String.valueOf(true))
+            .build()) {
+      validateAllRows(batchesToRows(reader.iterator()));
+    }
+  }
+
+  @Test
+  public void testRowReaderWithFilterWithSelected() throws IOException {
+    try (CloseableIterable<ColumnarBatch> reader =
+        ORC.read(outputFile.toInputFile())
+            .project(SCHEMA)
+            .createBatchedReaderFunc(
+                readOrcSchema ->
+                    VectorizedSparkOrcReaders.buildReader(SCHEMA, readOrcSchema, ImmutableMap.of()))
+            .filter(Expressions.equal("id", 3L))
+            .config(OrcConf.ALLOW_SARG_TO_FILTER.getAttribute(), String.valueOf(true))
+            .config(OrcConf.READER_USE_SELECTED.getAttribute(), String.valueOf(true))
+            .build()) {
+      Iterator<InternalRow> rows = batchesToRows(reader.iterator());
+      Assertions.assertTrue(rows.hasNext());
+      InternalRow row = rows.next();
+      Assertions.assertEquals(3L, row.getLong(0));
+      Assertions.assertEquals("c", row.getString(1));
+      Assertions.assertArrayEquals(new int[] {3, 4, 5}, row.getArray(3).toIntArray());

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.

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] pavibhai commented on a diff in pull request #7197: [ORC][Spark] - Support selected vector with ORC reader on the row and batch reader

Posted by "pavibhai (via GitHub)" <gi...@apache.org>.
pavibhai commented on code in PR #7197:
URL: https://github.com/apache/iceberg/pull/7197#discussion_r1152006408


##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/data/TestVectorizedOrcDataReader.java:
##########
@@ -0,0 +1,166 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.spark.data;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Path;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.data.orc.GenericOrcWriter;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.DataWriter;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.orc.ORC;
+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.Iterators;
+import org.apache.iceberg.spark.data.vectorized.VectorizedSparkOrcReaders;
+import org.apache.iceberg.types.Types;
+import org.apache.orc.OrcConf;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.vectorized.ColumnarBatch;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+
+public class TestVectorizedOrcDataReader {
+  @TempDir public static Path temp;
+
+  private static final Schema SCHEMA =
+      new Schema(
+          Types.NestedField.required(1, "id", Types.LongType.get()),
+          Types.NestedField.optional(2, "data", Types.StringType.get()),
+          Types.NestedField.optional(3, "binary", Types.BinaryType.get()),
+          Types.NestedField.required(
+              4, "array", Types.ListType.ofOptional(5, Types.IntegerType.get())));
+  private static OutputFile outputFile;
+
+  @BeforeAll
+  public static void createDataFile() throws IOException {
+    GenericRecord bufferRecord = GenericRecord.create(SCHEMA);
+
+    ImmutableList.Builder<Record> builder = ImmutableList.builder();
+    builder.add(
+        bufferRecord.copy(
+            ImmutableMap.of("id", 1L, "data", "a", "array", Collections.singletonList(1))));
+    builder.add(
+        bufferRecord.copy(ImmutableMap.of("id", 2L, "data", "b", "array", Arrays.asList(2, 3))));
+    builder.add(
+        bufferRecord.copy(ImmutableMap.of("id", 3L, "data", "c", "array", Arrays.asList(3, 4, 5))));
+    builder.add(
+        bufferRecord.copy(
+            ImmutableMap.of("id", 4L, "data", "d", "array", Arrays.asList(4, 5, 6, 7))));
+    builder.add(
+        bufferRecord.copy(
+            ImmutableMap.of("id", 5L, "data", "e", "array", Arrays.asList(5, 6, 7, 8, 9))));
+
+    outputFile = Files.localOutput(File.createTempFile("test", ".orc", temp.toFile()));
+
+    try (DataWriter<Record> dataWriter =
+        ORC.writeData(outputFile)
+            .schema(SCHEMA)
+            .createWriterFunc(GenericOrcWriter::buildWriter)
+            .overwrite()
+            .withSpec(PartitionSpec.unpartitioned())
+            .build()) {
+      for (Record record : builder.build()) {
+        dataWriter.write(record);
+      }
+    }
+  }
+
+  private Iterator<InternalRow> batchesToRows(Iterator<ColumnarBatch> batches) {
+    return Iterators.concat(Iterators.transform(batches, ColumnarBatch::rowIterator));
+  }
+
+  private void validateAllRows(Iterator<InternalRow> rows) {
+    long rowCount = 0;
+    long expId = 1;
+    char expChar = 'a';
+    while (rows.hasNext()) {
+      InternalRow row = rows.next();
+      Assertions.assertEquals(expId, row.getLong(0));
+      Assertions.assertEquals(expChar, row.getString(1).charAt(0));
+      Assertions.assertTrue(row.isNullAt(2));
+      expId += 1;
+      expChar += 1;
+      rowCount += 1;
+    }
+    Assertions.assertEquals(5, rowCount);
+  }
+
+  @Test
+  public void testVectorizedReader() throws IOException {
+    try (CloseableIterable<ColumnarBatch> reader =
+        ORC.read(outputFile.toInputFile())
+            .project(SCHEMA)
+            .createBatchedReaderFunc(
+                readOrcSchema ->
+                    VectorizedSparkOrcReaders.buildReader(SCHEMA, readOrcSchema, ImmutableMap.of()))
+            .build()) {
+      validateAllRows(batchesToRows(reader.iterator()));
+    }
+  }
+
+  @Test
+  public void testRowReaderWithFilter() throws IOException {
+    try (CloseableIterable<ColumnarBatch> reader =
+        ORC.read(outputFile.toInputFile())
+            .project(SCHEMA)
+            .createBatchedReaderFunc(
+                readOrcSchema ->
+                    VectorizedSparkOrcReaders.buildReader(SCHEMA, readOrcSchema, ImmutableMap.of()))
+            .filter(Expressions.equal("id", 3L))
+            .config(OrcConf.ALLOW_SARG_TO_FILTER.getAttribute(), String.valueOf(true))
+            .build()) {
+      validateAllRows(batchesToRows(reader.iterator()));
+    }
+  }
+
+  @Test
+  public void testRowReaderWithFilterWithSelected() throws IOException {
+    try (CloseableIterable<ColumnarBatch> reader =
+        ORC.read(outputFile.toInputFile())
+            .project(SCHEMA)
+            .createBatchedReaderFunc(
+                readOrcSchema ->
+                    VectorizedSparkOrcReaders.buildReader(SCHEMA, readOrcSchema, ImmutableMap.of()))
+            .filter(Expressions.equal("id", 3L))
+            .config(OrcConf.ALLOW_SARG_TO_FILTER.getAttribute(), String.valueOf(true))
+            .config(OrcConf.READER_USE_SELECTED.getAttribute(), String.valueOf(true))
+            .build()) {
+      Iterator<InternalRow> rows = batchesToRows(reader.iterator());
+      Assertions.assertTrue(rows.hasNext());
+      InternalRow row = rows.next();
+      Assertions.assertEquals(3L, row.getLong(0));
+      Assertions.assertEquals("c", row.getString(1));
+      Assertions.assertArrayEquals(new int[] {3, 4, 5}, row.getArray(3).toIntArray());

Review Comment:
   Thanks for calling that out



-- 
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] dpaani commented on a diff in pull request #7197: [ORC][Spark] - Support selected vector with ORC reader on the row and batch reader

Posted by "dpaani (via GitHub)" <gi...@apache.org>.
dpaani commented on code in PR #7197:
URL: https://github.com/apache/iceberg/pull/7197#discussion_r1150879414


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkOrcReaders.java:
##########
@@ -209,7 +229,8 @@ private int numNullsHelper() {
     }
 
     protected int getRowIndex(int rowId) {
-      return vector.isRepeating ? 0 : rowId;
+      int row = isSelectedInUse ? selected[rowId] : rowId;

Review Comment:
   will there be a possibility of isSelectedInUse=true and selected is empty?



-- 
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] pavibhai commented on a diff in pull request #7197: [ORC][Spark] - Support selected vector with ORC reader on the row and batch reader

Posted by "pavibhai (via GitHub)" <gi...@apache.org>.
pavibhai commented on code in PR #7197:
URL: https://github.com/apache/iceberg/pull/7197#discussion_r1152007471


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkOrcReaders.java:
##########
@@ -209,7 +229,8 @@ private int numNullsHelper() {
     }
 
     protected int getRowIndex(int rowId) {
-      return vector.isRepeating ? 0 : rowId;
+      int row = isSelectedInUse ? selected[rowId] : rowId;

Review Comment:
   No, this is not a possibility. Similar to other vectors the selected vector is initialized at the start to max batch size and all of the indices in this are excepted to be within the batch size.



-- 
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] aokolnychyi merged pull request #7197: [ORC][Spark] - Support selected vector with ORC reader on the row and batch reader

Posted by "aokolnychyi (via GitHub)" <gi...@apache.org>.
aokolnychyi merged PR #7197:
URL: https://github.com/apache/iceberg/pull/7197


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