You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@orc.apache.org by pr...@apache.org on 2020/02/13 16:52:16 UTC

[orc] branch master updated: ORC-598: Unable to read ORC file with struct and array.length > 1024 (#479)

This is an automated email from the ASF dual-hosted git repository.

prasanthj pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/orc.git


The following commit(s) were added to refs/heads/master by this push:
     new 4f48184  ORC-598: Unable to read ORC file with struct and array.length > 1024 (#479)
4f48184 is described below

commit 4f48184f07802498ebfecc9820de1ca167d1fadb
Author: kasakrisz <33...@users.noreply.github.com>
AuthorDate: Thu Feb 13 17:52:08 2020 +0100

    ORC-598: Unable to read ORC file with struct and array.length > 1024 (#479)
    
    * ORC-598: Unable to read ORC file with struct and array.length > 1024
    
    * ORC-598: Unable to read ORC file with struct and array.length > 1024 - UT
---
 .../apache/orc/impl/ConvertTreeReaderFactory.java  |  10 +--
 .../orc/impl/TestConvertTreeReaderFactory.java     |  94 +++++++++++++++++++++
 java/core/src/test/resources/bigarray.orc          | Bin 0 -> 2337 bytes
 3 files changed, 99 insertions(+), 5 deletions(-)

diff --git a/java/core/src/java/org/apache/orc/impl/ConvertTreeReaderFactory.java b/java/core/src/java/org/apache/orc/impl/ConvertTreeReaderFactory.java
index 81a9b62..ead8f65 100644
--- a/java/core/src/java/org/apache/orc/impl/ConvertTreeReaderFactory.java
+++ b/java/core/src/java/org/apache/orc/impl/ConvertTreeReaderFactory.java
@@ -538,7 +538,7 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
                            final int batchSize) throws IOException {
       if (decimalColVector == null) {
         // Allocate column vector for file; cast column vector for reader.
-        decimalColVector = new DecimalColumnVector(precision, scale);
+        decimalColVector = new DecimalColumnVector(batchSize, precision, scale);
         longColVector = (LongColumnVector) previousVector;
       }
       // Read present/isNull stream
@@ -685,7 +685,7 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
                            final int batchSize) throws IOException {
       if (decimalColVector == null) {
         // Allocate column vector for file; cast column vector for reader.
-        decimalColVector = new DecimalColumnVector(precision, scale);
+        decimalColVector = new DecimalColumnVector(batchSize, precision, scale);
         doubleColVector = (DoubleColumnVector) previousVector;
       }
       // Read present/isNull stream
@@ -988,7 +988,7 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
                            final int batchSize) throws IOException {
       if (fileDecimalColVector == null) {
         // Allocate column vector for file; cast column vector for reader.
-        fileDecimalColVector = new DecimalColumnVector(filePrecision, fileScale);
+        fileDecimalColVector = new DecimalColumnVector(batchSize, filePrecision, fileScale);
         decimalColVector = previousVector;
       }
       // Read present/isNull stream
@@ -1129,7 +1129,7 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
                            final int batchSize) throws IOException {
       if (decimalColVector == null) {
         // Allocate column vector for file; cast column vector for reader.
-        decimalColVector = new DecimalColumnVector(precision, scale);
+        decimalColVector = new DecimalColumnVector(batchSize, precision, scale);
         bytesColVector = (BytesColumnVector) previousVector;
       }
       // Read present/isNull stream
@@ -1534,7 +1534,7 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
                            final int batchSize) throws IOException {
       if (decimalColVector == null) {
         // Allocate column vector for file; cast column vector for reader.
-        decimalColVector = new DecimalColumnVector(precision, scale);
+        decimalColVector = new DecimalColumnVector(batchSize, precision, scale);
         timestampColVector = (TimestampColumnVector) previousVector;
       }
       timestampColVector.changeCalendar(fileUsedProlepticGregorian, false);
diff --git a/java/core/src/test/org/apache/orc/impl/TestConvertTreeReaderFactory.java b/java/core/src/test/org/apache/orc/impl/TestConvertTreeReaderFactory.java
new file mode 100644
index 0000000..baf5e10
--- /dev/null
+++ b/java/core/src/test/org/apache/orc/impl/TestConvertTreeReaderFactory.java
@@ -0,0 +1,94 @@
+/*
+ * 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.orc.impl;
+
+import static org.junit.Assert.*;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.Decimal64ColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.ListColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.orc.OrcFile;
+import org.apache.orc.Reader;
+import org.apache.orc.RecordReader;
+import org.apache.orc.TestVectorOrcFile;
+import org.apache.orc.TypeDescription;
+import org.junit.Test;
+
+public class TestConvertTreeReaderFactory {
+
+  @Test
+  public void testArraySizeBiggerThan1024AndConvertToDecimal() throws Exception {
+    Decimal64ColumnVector columnVector = testArraySizeBiggerThan1024("decimal(6,1)", Decimal64ColumnVector.class);
+    assertEquals(columnVector.vector.length, 1025);
+  }
+
+  public <TExpectedColumn extends ColumnVector> TExpectedColumn testArraySizeBiggerThan1024(
+          String typeString, Class<TExpectedColumn> expectedColumnType) throws Exception {
+    Reader.Options options = new Reader.Options();
+    TypeDescription schema = TypeDescription.fromString("struct<col1:array<"+ typeString +">>");
+    options.schema(schema);
+    String expected = options.toString();
+
+    Configuration conf = new Configuration();
+    Path path = new Path(TestVectorOrcFile.getFileFromClasspath("bigarray.orc"));
+
+    Reader reader = OrcFile.createReader(path, OrcFile.readerOptions(conf));
+    RecordReader rows = reader.rows(options);
+    VectorizedRowBatch batch = schema.createRowBatchV2();
+    while (rows.nextBatch(batch)) {
+      assertTrue(batch.size > 0);
+    }
+
+    assertEquals(expected, options.toString());
+    assertEquals(batch.cols.length, 1);
+    assertTrue(batch.cols[0] instanceof ListColumnVector);
+    assertEquals(((ListColumnVector) batch.cols[0]).child.getClass(), expectedColumnType);
+    return (TExpectedColumn) ((ListColumnVector) batch.cols[0]).child;
+  }
+
+  @Test
+  public void testArraySizeBiggerThan1024AndConvertToVarchar() throws Exception {
+    BytesColumnVector columnVector = testArraySizeBiggerThan1024("varchar(10)", BytesColumnVector.class);
+    assertEquals(columnVector.vector.length, 1025);
+  }
+
+  @Test
+  public void testArraySizeBiggerThan1024AndConvertToDouble() throws Exception {
+    DoubleColumnVector columnVector = testArraySizeBiggerThan1024("double", DoubleColumnVector.class);
+    assertEquals(columnVector.vector.length, 1025);
+  }
+
+  @Test
+  public void testArraySizeBiggerThan1024AndConvertToInteger() throws Exception {
+    LongColumnVector columnVector = testArraySizeBiggerThan1024("int", LongColumnVector.class);
+    assertEquals(columnVector.vector.length, 1025);
+  }
+
+  @Test
+  public void testArraySizeBiggerThan1024AndConvertToTimestamp() throws Exception {
+    TimestampColumnVector columnVector = testArraySizeBiggerThan1024("timestamp", TimestampColumnVector.class);
+    assertEquals(columnVector.time.length, 1025);
+  }
+}
\ No newline at end of file
diff --git a/java/core/src/test/resources/bigarray.orc b/java/core/src/test/resources/bigarray.orc
new file mode 100644
index 0000000..002565b
Binary files /dev/null and b/java/core/src/test/resources/bigarray.orc differ