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

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


The following commit(s) were added to refs/heads/main by this push:
     new ac4187320 ORC-1205: `nextVector` should invoke `ensureSize` when 
reusing vectors
ac4187320 is described below

commit ac4187320a5fec416d9fb758e461b5d7638dc5b4
Author: Laszlo Bodor <[email protected]>
AuthorDate: Sat Jun 25 06:21:15 2022 -0700

    ORC-1205: `nextVector` should invoke `ensureSize` when reusing vectors
    
    ### What changes were proposed in this pull request?
    The fix simply ensures the size of some batches in ConvertTreeReader 
instances without API change.
    
    ### Why are the changes needed?
    As described on jira, if there are batches of increasing size while 
reading, if they are in different stripes, we can hit an edge case where 
nextBatch doesn't force them to ensure the size of some batches.
    
    ### How was this patch tested?
    Unit tests added for all possible ConvertTreeReader, also tested in hive 
locally.
    
    Closes #1166 from abstractdog/ORC-1205-main.
    
    Authored-by: Laszlo Bodor <[email protected]>
    Signed-off-by: Dongjoon Hyun <[email protected]>
---
 .../apache/orc/impl/ConvertTreeReaderFactory.java  |  52 ++++
 .../orc/impl/TestConvertTreeReaderFactory.java     | 284 +++++++++++++++++++--
 2 files changed, 320 insertions(+), 16 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 91bcacdca..9cc1bc89b 100644
--- a/java/core/src/java/org/apache/orc/impl/ConvertTreeReaderFactory.java
+++ b/java/core/src/java/org/apache/orc/impl/ConvertTreeReaderFactory.java
@@ -464,6 +464,8 @@ public class ConvertTreeReaderFactory extends 
TreeReaderFactory {
         // Allocate column vector for file; cast column vector for reader.
         doubleColVector = new DoubleColumnVector(batchSize);
         longColVector = (LongColumnVector) previousVector;
+      } else {
+        doubleColVector.ensureSize(batchSize, false);
       }
       // Read present/isNull stream
       fromReader.nextVector(doubleColVector, isNull, batchSize, filterContext, 
readPhase);
@@ -550,6 +552,8 @@ public class ConvertTreeReaderFactory extends 
TreeReaderFactory {
         // Allocate column vector for file; cast column vector for reader.
         decimalColVector = new DecimalColumnVector(batchSize, precision, 
scale);
         longColVector = (LongColumnVector) previousVector;
+      } else {
+        decimalColVector.ensureSize(batchSize, false);
       }
       // Read present/isNull stream
       fromReader.nextVector(decimalColVector, isNull, batchSize, 
filterContext, readPhase);
@@ -591,6 +595,8 @@ public class ConvertTreeReaderFactory extends 
TreeReaderFactory {
         // Allocate column vector for file; cast column vector for reader.
         bytesColVector = new BytesColumnVector(batchSize);
         longColVector = (LongColumnVector) previousVector;
+      } else {
+        bytesColVector.ensureSize(batchSize, false);
       }
       // Read present/isNull stream
       fromReader.nextVector(bytesColVector, isNull, batchSize, filterContext, 
readPhase);
@@ -628,6 +634,8 @@ public class ConvertTreeReaderFactory extends 
TreeReaderFactory {
         // Allocate column vector for file; cast column vector for reader.
         timestampColVector = new TimestampColumnVector(batchSize);
         longColVector = (LongColumnVector) previousVector;
+      } else {
+        timestampColVector.ensureSize(batchSize, false);
       }
       // Read present/isNull stream
       fromReader.nextVector(timestampColVector, isNull, batchSize, 
filterContext, readPhase);
@@ -668,6 +676,8 @@ public class ConvertTreeReaderFactory extends 
TreeReaderFactory {
         // Allocate column vector for file; cast column vector for reader.
         longColVector = new LongColumnVector(batchSize);
         doubleColVector = (DoubleColumnVector) previousVector;
+      } else {
+        longColVector.ensureSize(batchSize, false);
       }
       // Read present/isNull stream
       fromReader.nextVector(longColVector, isNull, batchSize, filterContext, 
readPhase);
@@ -706,6 +716,8 @@ public class ConvertTreeReaderFactory extends 
TreeReaderFactory {
         // Allocate column vector for file; cast column vector for reader.
         decimalColVector = new DecimalColumnVector(batchSize, precision, 
scale);
         doubleColVector = (DoubleColumnVector) previousVector;
+      } else {
+        decimalColVector.ensureSize(batchSize, false);
       }
       // Read present/isNull stream
       fromReader.nextVector(decimalColVector, isNull, batchSize, 
filterContext, readPhase);
@@ -745,6 +757,8 @@ public class ConvertTreeReaderFactory extends 
TreeReaderFactory {
         // Allocate column vector for file; cast column vector for reader.
         bytesColVector = new BytesColumnVector(batchSize);
         doubleColVector = (DoubleColumnVector) previousVector;
+      } else {
+        bytesColVector.ensureSize(batchSize, false);
       }
       // Read present/isNull stream
       fromReader.nextVector(bytesColVector, isNull, batchSize, filterContext, 
readPhase);
@@ -783,6 +797,8 @@ public class ConvertTreeReaderFactory extends 
TreeReaderFactory {
         // Allocate column vector for file; cast column vector for reader.
         timestampColVector = new TimestampColumnVector(batchSize);
         doubleColVector = (DoubleColumnVector) previousVector;
+      } else {
+        timestampColVector.ensureSize(batchSize, false);
       }
       // Read present/isNull stream
       fromReader.nextVector(timestampColVector, isNull, batchSize, 
filterContext, readPhase);
@@ -848,6 +864,8 @@ public class ConvertTreeReaderFactory extends 
TreeReaderFactory {
         // Allocate column vector for file; cast column vector for reader.
         longColVector = new LongColumnVector(batchSize);
         decimalColVector = previousVector;
+      } else {
+        longColVector.ensureSize(batchSize, false);
       }
       // Read present/isNull stream
       fromReader.nextVector(longColVector, isNull, batchSize, filterContext, 
readPhase);
@@ -896,6 +914,8 @@ public class ConvertTreeReaderFactory extends 
TreeReaderFactory {
         // Allocate column vector for file; cast column vector for reader.
         doubleColVector = new DoubleColumnVector(batchSize);
         decimalColVector = previousVector;
+      } else {
+        doubleColVector.ensureSize(batchSize, false);
       }
       // Read present/isNull stream
       fromReader.nextVector(doubleColVector, isNull, batchSize, filterContext, 
readPhase);
@@ -940,6 +960,8 @@ public class ConvertTreeReaderFactory extends 
TreeReaderFactory {
         // Allocate column vector for file; cast column vector for reader.
         bytesColVector = new BytesColumnVector(batchSize);
         decimalColVector = previousVector;
+      } else {
+        bytesColVector.ensureSize(batchSize, false);
       }
       // Read present/isNull stream
       fromReader.nextVector(bytesColVector, isNull, batchSize, filterContext, 
readPhase);
@@ -989,6 +1011,8 @@ public class ConvertTreeReaderFactory extends 
TreeReaderFactory {
         // Allocate column vector for file; cast column vector for reader.
         timestampColVector = new TimestampColumnVector(batchSize);
         decimalColVector = previousVector;
+      } else {
+        timestampColVector.ensureSize(batchSize, false);
       }
       // Read present/isNull stream
       fromReader.nextVector(timestampColVector, isNull, batchSize, 
filterContext, readPhase);
@@ -1035,6 +1059,8 @@ public class ConvertTreeReaderFactory extends 
TreeReaderFactory {
         // Allocate column vector for file; cast column vector for reader.
         fileDecimalColVector = new DecimalColumnVector(batchSize, 
filePrecision, fileScale);
         decimalColVector = previousVector;
+      } else {
+        fileDecimalColVector.ensureSize(batchSize, false);
       }
       // Read present/isNull stream
       fromReader.nextVector(fileDecimalColVector, isNull, batchSize, 
filterContext, readPhase);
@@ -1071,6 +1097,8 @@ public class ConvertTreeReaderFactory extends 
TreeReaderFactory {
         // Allocate column vector for file; cast column vector for reader.
         longColVector = new LongColumnVector(batchSize);
         bytesColVector = (BytesColumnVector) previousVector;
+      } else {
+        longColVector.ensureSize(batchSize, false);
       }
       // Read present/isNull stream
       fromReader.nextVector(longColVector, isNull, batchSize, filterContext, 
readPhase);
@@ -1133,6 +1161,8 @@ public class ConvertTreeReaderFactory extends 
TreeReaderFactory {
         // Allocate column vector for file; cast column vector for reader.
         doubleColVector = new DoubleColumnVector(batchSize);
         bytesColVector = (BytesColumnVector) previousVector;
+      } else {
+        doubleColVector.ensureSize(batchSize, false);
       }
       // Read present/isNull stream
       fromReader.nextVector(doubleColVector, isNull, batchSize, filterContext, 
readPhase);
@@ -1183,6 +1213,8 @@ public class ConvertTreeReaderFactory extends 
TreeReaderFactory {
         // Allocate column vector for file; cast column vector for reader.
         decimalColVector = new DecimalColumnVector(batchSize, precision, 
scale);
         bytesColVector = (BytesColumnVector) previousVector;
+      } else {
+        decimalColVector.ensureSize(batchSize, false);
       }
       // Read present/isNull stream
       fromReader.nextVector(decimalColVector, isNull, batchSize, 
filterContext, readPhase);
@@ -1310,6 +1342,8 @@ public class ConvertTreeReaderFactory extends 
TreeReaderFactory {
         // Allocate column vector for file; cast column vector for reader.
         timestampColVector = new TimestampColumnVector(batchSize);
         bytesColVector = (BytesColumnVector) previousVector;
+      } else {
+        timestampColVector.ensureSize(batchSize, false);
       }
       // Read present/isNull stream
       fromReader.nextVector(timestampColVector, isNull, batchSize, 
filterContext, readPhase);
@@ -1349,6 +1383,8 @@ public class ConvertTreeReaderFactory extends 
TreeReaderFactory {
         // Allocate column vector for file; cast column vector for reader.
         longColVector = new DateColumnVector(batchSize);
         bytesColVector = (BytesColumnVector) previousVector;
+      } else {
+        longColVector.ensureSize(batchSize, false);
       }
       // Read present/isNull stream
       fromReader.nextVector(longColVector, isNull, batchSize, filterContext, 
readPhase);
@@ -1439,6 +1475,8 @@ public class ConvertTreeReaderFactory extends 
TreeReaderFactory {
         // Allocate column vector for file; cast column vector for reader.
         inBytesColVector = new BytesColumnVector(batchSize);
         outBytesColVector = (BytesColumnVector) previousVector;
+      } else {
+        inBytesColVector.ensureSize(batchSize, false);
       }
       // Read present/isNull stream
       fromReader.nextVector(inBytesColVector, isNull, batchSize, 
filterContext, readPhase);
@@ -1484,6 +1522,8 @@ public class ConvertTreeReaderFactory extends 
TreeReaderFactory {
         // Allocate column vector for file; cast column vector for reader.
         longColVector = new LongColumnVector(batchSize);
         timestampColVector = (TimestampColumnVector) previousVector;
+      } else {
+        longColVector.ensureSize(batchSize, false);
       }
       timestampColVector.changeCalendar(fileUsedProlepticGregorian, false);
       // Read present/isNull stream
@@ -1546,6 +1586,8 @@ public class ConvertTreeReaderFactory extends 
TreeReaderFactory {
         // Allocate column vector for file; cast column vector for reader.
         doubleColVector = new DoubleColumnVector(batchSize);
         timestampColVector = (TimestampColumnVector) previousVector;
+      } else {
+        doubleColVector.ensureSize(batchSize, false);
       }
       timestampColVector.changeCalendar(fileUsedProlepticGregorian, false);
       // Read present/isNull stream
@@ -1608,6 +1650,8 @@ public class ConvertTreeReaderFactory extends 
TreeReaderFactory {
         // Allocate column vector for file; cast column vector for reader.
         decimalColVector = new DecimalColumnVector(batchSize, precision, 
scale);
         timestampColVector = (TimestampColumnVector) previousVector;
+      } else {
+        decimalColVector.ensureSize(batchSize, false);
       }
       timestampColVector.changeCalendar(fileUsedProlepticGregorian, false);
       // Read present/isNull stream
@@ -1667,6 +1711,8 @@ public class ConvertTreeReaderFactory extends 
TreeReaderFactory {
         // Allocate column vector for file; cast column vector for reader.
         bytesColVector = new BytesColumnVector(batchSize);
         timestampColVector = (TimestampColumnVector) previousVector;
+      } else {
+        bytesColVector.ensureSize(batchSize, false);
       }
       // Read present/isNull stream
       fromReader.nextVector(bytesColVector, isNull, batchSize, filterContext, 
readPhase);
@@ -1711,6 +1757,8 @@ public class ConvertTreeReaderFactory extends 
TreeReaderFactory {
         // Allocate column vector for file; cast column vector for reader.
         longColVector = new DateColumnVector(batchSize);
         timestampColVector = (TimestampColumnVector) previousVector;
+      } else {
+        longColVector.ensureSize(batchSize, false);
       }
       // Read present/isNull stream
       fromReader.nextVector(longColVector, isNull, batchSize, filterContext, 
readPhase);
@@ -1764,6 +1812,8 @@ public class ConvertTreeReaderFactory extends 
TreeReaderFactory {
                                                    " proleptic Gregorian 
dates.");
           }
         }
+      } else {
+        bytesColVector.ensureSize(batchSize, false);
       }
       // Read present/isNull stream
       fromReader.nextVector(bytesColVector, isNull, batchSize, filterContext, 
readPhase);
@@ -1812,6 +1862,8 @@ public class ConvertTreeReaderFactory extends 
TreeReaderFactory {
           throw new IllegalArgumentException("Can't use LongColumnVector with" 
+
                                                  " proleptic Gregorian 
dates.");
         }
+      } else {
+        timestampColVector.ensureSize(batchSize, false);
       }
       // Read present/isNull stream
       fromReader.nextVector(timestampColVector, isNull, batchSize, 
filterContext, readPhase);
diff --git 
a/java/core/src/test/org/apache/orc/impl/TestConvertTreeReaderFactory.java 
b/java/core/src/test/org/apache/orc/impl/TestConvertTreeReaderFactory.java
index 1644210c8..49c3bf243 100644
--- a/java/core/src/test/org/apache/orc/impl/TestConvertTreeReaderFactory.java
+++ b/java/core/src/test/org/apache/orc/impl/TestConvertTreeReaderFactory.java
@@ -32,6 +32,7 @@ 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.OrcFile.WriterOptions;
 import org.apache.orc.Reader;
 import org.apache.orc.RecordReader;
 import org.apache.orc.TestProlepticConversions;
@@ -64,6 +65,8 @@ public class TestConvertTreeReaderFactory {
   private FileSystem fs;
   private Path testFilePath;
   private int LARGE_BATCH_SIZE;
+  private static final int INCREASING_BATCH_SIZE_FIRST = 30;
+  private static final int INCREASING_BATCH_SIZE_SECOND = 50;
 
   @BeforeEach
   public void setupPath(TestInfo testInfo) throws Exception {
@@ -94,22 +97,7 @@ public class TestConvertTreeReaderFactory {
     TExpectedColumnVector dcv = (TExpectedColumnVector) (listCol).child;
     batch.size = 1;
     for (int row = 0; row < LARGE_BATCH_SIZE; ++row) {
-      if (dcv instanceof DecimalColumnVector) {
-        ((DecimalColumnVector) dcv).set(row, HiveDecimal.create(row * 2 + 1));
-      } else if (dcv instanceof DoubleColumnVector) {
-        ((DoubleColumnVector) dcv).vector[row] = row * 2 + 1;
-      } else if (dcv instanceof BytesColumnVector) {
-        ((BytesColumnVector) dcv).setVal(row, ((row * 2 + 1) + 
"").getBytes(StandardCharsets.UTF_8));
-      } else if (dcv instanceof LongColumnVector) {
-        ((LongColumnVector) dcv).vector[row] = row * 2 + 1;
-      } else if (dcv instanceof TimestampColumnVector) {
-        ((TimestampColumnVector) dcv).set(row, Timestamp.valueOf((1900 + row) 
+ "-04-01 12:34:56.9"));
-      } else if (dcv instanceof DateColumnVector) {
-        String date = String.format("%04d-01-23", row * 2 + 1);
-        ((DateColumnVector) dcv).vector[row] = 
TimeUnit.MILLISECONDS.toDays(dateFormat.parse(date).getTime());
-      } else {
-        throw new IllegalStateException("Writing File with a large array of "+ 
expectedColumnType + " is not supported!");
-      }
+      setElementInVector(expectedColumnType, dateFormat, dcv, row);
     }
 
     listCol.childCount = 1;
@@ -122,6 +110,65 @@ public class TestConvertTreeReaderFactory {
     return (TExpectedColumnVector) ((ListColumnVector) batch.cols[0]).child;
   }
 
+  public <TExpectedColumnVector extends ColumnVector> TExpectedColumnVector 
createORCFileWithBatchesOfIncreasingSizeInDifferentStripes(
+      TypeDescription schema, Class<TExpectedColumnVector> typeClass, boolean 
useDecimal64)
+      throws IOException, ParseException {
+    conf = new Configuration();
+    fs = FileSystem.getLocal(conf);
+    fs.setWorkingDirectory(workDir);
+    WriterOptions options = OrcFile.writerOptions(conf);
+    Writer w = OrcFile.createWriter(testFilePath, options.setSchema(schema));
+
+    SimpleDateFormat dateFormat = 
TestProlepticConversions.createParser("yyyy-MM-dd", new GregorianCalendar());
+    VectorizedRowBatch batch = schema.createRowBatch(
+        useDecimal64 ? TypeDescription.RowBatchVersion.USE_DECIMAL64 : 
TypeDescription.RowBatchVersion.ORIGINAL,
+        INCREASING_BATCH_SIZE_FIRST);
+
+    TExpectedColumnVector columnVector = (TExpectedColumnVector) batch.cols[0];
+    batch.size = INCREASING_BATCH_SIZE_FIRST;
+    for (int row = 0; row < INCREASING_BATCH_SIZE_FIRST; ++row) {
+      setElementInVector(typeClass, dateFormat, columnVector, row);
+    }
+
+    w.addRowBatch(batch);
+    w.writeIntermediateFooter(); //forcing a new stripe
+
+    batch = schema.createRowBatch(
+        useDecimal64 ? TypeDescription.RowBatchVersion.USE_DECIMAL64 : 
TypeDescription.RowBatchVersion.ORIGINAL,
+        INCREASING_BATCH_SIZE_SECOND);
+
+    columnVector = (TExpectedColumnVector) batch.cols[0];
+    batch.size = INCREASING_BATCH_SIZE_SECOND;
+    for (int row = 0; row < INCREASING_BATCH_SIZE_SECOND; ++row) {
+      setElementInVector(typeClass, dateFormat, columnVector, row);
+    }
+
+    w.addRowBatch(batch);
+    w.close();
+    return (TExpectedColumnVector) batch.cols[0];
+  }
+
+  private void setElementInVector(
+      Class<?> expectedColumnType, SimpleDateFormat dateFormat, ColumnVector 
dcv, int row)
+      throws ParseException {
+    if (dcv instanceof DecimalColumnVector) {
+      ((DecimalColumnVector) dcv).set(row, HiveDecimal.create(row * 2 + 1));
+    } else if (dcv instanceof DoubleColumnVector) {
+      ((DoubleColumnVector) dcv).vector[row] = row * 2 + 1;
+    } else if (dcv instanceof BytesColumnVector) {
+      ((BytesColumnVector) dcv).setVal(row, ((row * 2 + 1) + 
"").getBytes(StandardCharsets.UTF_8));
+    } else if (dcv instanceof LongColumnVector) {
+      ((LongColumnVector) dcv).vector[row] = row * 2 + 1;
+    } else if (dcv instanceof TimestampColumnVector) {
+      ((TimestampColumnVector) dcv).set(row, Timestamp.valueOf((1900 + row) + 
"-04-01 12:34:56.9"));
+    } else if (dcv instanceof DateColumnVector) {
+      String date = String.format("%04d-01-23", row * 2 + 1);
+      ((DateColumnVector) dcv).vector[row] = 
TimeUnit.MILLISECONDS.toDays(dateFormat.parse(date).getTime());
+    } else {
+      throw new IllegalStateException("Writing File with a large array of "+ 
expectedColumnType + " is not supported!");
+    }
+  }
+
   public <TExpectedColumnVector extends ColumnVector> TExpectedColumnVector 
readORCFileWithLargeArray(
       String typeString, Class<TExpectedColumnVector> expectedColumnType) 
throws Exception {
     Reader.Options options = new Reader.Options();
@@ -145,6 +192,31 @@ public class TestConvertTreeReaderFactory {
     return (TExpectedColumnVector) ((ListColumnVector) batch.cols[0]).child;
   }
 
+  public void readORCFileIncreasingBatchSize(String typeString, Class<?> 
expectedColumnType) throws Exception {
+    Reader.Options options = new Reader.Options();
+    TypeDescription schema = TypeDescription.fromString("struct<col1:" + 
typeString + ">");
+    options.schema(schema);
+    String expected = options.toString();
+
+    Configuration conf = new Configuration();
+
+    Reader reader = OrcFile.createReader(testFilePath, 
OrcFile.readerOptions(conf));
+    RecordReader rows = reader.rows(options);
+    VectorizedRowBatch batch = schema.createRowBatchV2();
+
+    rows.nextBatch(batch);
+    assertEquals(INCREASING_BATCH_SIZE_FIRST , batch.size);
+    assertEquals(expected, options.toString());
+    assertEquals(batch.cols.length, 1);
+    assertEquals(batch.cols[0].getClass(), expectedColumnType);
+
+    rows.nextBatch(batch);
+    assertEquals(INCREASING_BATCH_SIZE_SECOND , batch.size);
+    assertEquals(expected, options.toString());
+    assertEquals(batch.cols.length, 1);
+    assertEquals(batch.cols[0].getClass(), expectedColumnType);
+  }
+
   public void testConvertToDecimal() throws Exception {
     Decimal64ColumnVector columnVector =
         readORCFileWithLargeArray("decimal(6,1)", Decimal64ColumnVector.class);
@@ -346,6 +418,65 @@ public class TestConvertTreeReaderFactory {
       // check ConvertTreeReaderFactory.createDateConvertTreeReader
       testConvertToVarchar();
       testConvertToTimestamp();
+    } finally {
+      fs.delete(testFilePath, false);
+    }
+  }
+
+  @Test
+  public void testDecimalVectorIncreasingSizeInDifferentStripes() throws 
Exception {
+    String typeStr = "decimal(6,1)";
+    Class typeClass = DecimalColumnVector.class;
+
+    TypeDescription schema = TypeDescription.fromString("struct<col1:" + 
typeStr + ">");
+    createORCFileWithBatchesOfIncreasingSizeInDifferentStripes(schema, 
typeClass, typeClass.equals(Decimal64ColumnVector.class));
+    try {
+      testConvertToIntegerIncreasingSize();
+      testConvertToDoubleIncreasingSize();
+      testConvertToVarcharIncreasingSize();
+      testConvertToTimestampIncreasingSize();
+      testConvertToDecimalIncreasingSize();
+    } finally {
+      fs.delete(testFilePath, false);
+    }
+  }
+
+  @Test
+  public void testDecimal64VectorIncreasingSizeInDifferentStripes() throws 
Exception {
+    String typeStr = "decimal(6,1)";
+    Class typeClass = Decimal64ColumnVector.class;
+
+    TypeDescription schema = TypeDescription.fromString("struct<col1:" + 
typeStr + ">");
+    createORCFileWithBatchesOfIncreasingSizeInDifferentStripes(schema, 
typeClass,
+        typeClass.equals(Decimal64ColumnVector.class));
+    try {
+      testConvertToIntegerIncreasingSize();
+      testConvertToDoubleIncreasingSize();
+      testConvertToVarcharIncreasingSize();
+      testConvertToTimestampIncreasingSize();
+      testConvertToDecimalIncreasingSize();
+    } finally {
+      // Make sure we delete file across tests
+      fs.delete(testFilePath, false);
+    }
+  }
+
+  @Test
+  public void testStringVectorIncreasingSizeInDifferentStripes() throws 
Exception {
+    String typeStr = "varchar(10)";
+    Class typeClass = BytesColumnVector.class;
+
+    TypeDescription schema = TypeDescription.fromString("struct<col1:" + 
typeStr + ">");
+    createORCFileWithBatchesOfIncreasingSizeInDifferentStripes(schema, 
typeClass,
+        typeClass.equals(Decimal64ColumnVector.class));
+    try {
+      testConvertToIntegerIncreasingSize();
+      testConvertToDoubleIncreasingSize();
+      testConvertToDecimalIncreasingSize();
+      testConvertToVarcharIncreasingSize();
+      testConvertToBinaryIncreasingSize();
+      testConvertToTimestampIncreasingSize();
+      testConvertToDateIncreasingSize();
     } finally {
       // Make sure we delete file across tests
       fs.delete(testFilePath, false);
@@ -387,4 +518,125 @@ public class TestConvertTreeReaderFactory {
     assertEquals("totalLength:-1 is a negative number.",
             batchSizeOneException.getMessage());
   }
+
+  public void testBinaryVectorIncreasingSizeInDifferentStripes() throws 
Exception {
+    String typeStr = "binary";
+    Class typeClass = BytesColumnVector.class;
+
+    TypeDescription schema = TypeDescription.fromString("struct<col1:" + 
typeStr + ">");
+    createORCFileWithBatchesOfIncreasingSizeInDifferentStripes(schema, 
typeClass,
+        typeClass.equals(Decimal64ColumnVector.class));
+    try {
+      testConvertToVarcharIncreasingSize();
+    } finally {
+      fs.delete(testFilePath, false);
+    }
+  }
+
+  @Test
+  public void testDoubleVectorIncreasingSizeInDifferentStripes() throws 
Exception {
+    String typeStr = "double";
+    Class typeClass = DoubleColumnVector.class;
+
+    TypeDescription schema = TypeDescription.fromString("struct<col1:" + 
typeStr + ">");
+    createORCFileWithBatchesOfIncreasingSizeInDifferentStripes(schema, 
typeClass,
+        typeClass.equals(Decimal64ColumnVector.class));
+    try {
+      testConvertToDoubleIncreasingSize();
+      testConvertToIntegerIncreasingSize();
+      testConvertToFloatIncreasingSize();
+      testConvertToDecimalIncreasingSize();
+      testConvertToVarcharIncreasingSize();
+      testConvertToTimestampIncreasingSize();
+    } finally {
+      fs.delete(testFilePath, false);
+    }
+  }
+
+  @Test
+  public void testIntVectorIncreasingSizeInDifferentStripes() throws Exception 
{
+    String typeStr = "int";
+    Class typeClass = LongColumnVector.class;
+
+    TypeDescription schema = TypeDescription.fromString("struct<col1:" + 
typeStr + ">");
+    createORCFileWithBatchesOfIncreasingSizeInDifferentStripes(schema, 
typeClass,
+        typeClass.equals(Decimal64ColumnVector.class));
+    try {
+      testConvertToIntegerIncreasingSize();
+      testConvertToDoubleIncreasingSize();
+      testConvertToDecimalIncreasingSize();
+      testConvertToVarcharIncreasingSize();
+      testConvertToTimestampIncreasingSize();
+    } finally {
+      fs.delete(testFilePath, false);
+    }
+  }
+
+  @Test
+  public void testTimestampVectorIncreasingSizeInDifferentStripes() throws 
Exception {
+    String typeStr = "timestamp";
+    Class typeClass = TimestampColumnVector.class;
+
+    TypeDescription schema = TypeDescription.fromString("struct<col1:" + 
typeStr + ">");
+    createORCFileWithBatchesOfIncreasingSizeInDifferentStripes(schema, 
typeClass,
+        typeClass.equals(Decimal64ColumnVector.class));
+    try {
+      testConvertToIntegerIncreasingSize();
+      testConvertToDoubleIncreasingSize();
+      testConvertToDecimalIncreasingSize();
+      testConvertToVarcharIncreasingSize();
+      testConvertToTimestampIncreasingSize();
+      testConvertToDateIncreasingSize();
+    } finally {
+      fs.delete(testFilePath, false);
+    }
+  }
+
+  @Test
+  public void testDateVectorIncreasingSizeInDifferentStripes() throws 
Exception {
+    String typeStr = "date";
+    Class typeClass = DateColumnVector.class;
+
+    TypeDescription schema = TypeDescription.fromString("struct<col1:" + 
typeStr + ">");
+    createORCFileWithBatchesOfIncreasingSizeInDifferentStripes(schema, 
typeClass,
+        typeClass.equals(Decimal64ColumnVector.class));
+    try {
+      testConvertToVarcharIncreasingSize();
+      testConvertToTimestampIncreasingSize();
+    } finally {
+      fs.delete(testFilePath, false);
+    }
+  }
+
+  private void testConvertToDoubleIncreasingSize() throws Exception {
+    readORCFileIncreasingBatchSize("double", DoubleColumnVector.class);
+  }
+
+  private void testConvertToIntegerIncreasingSize() throws Exception {
+    readORCFileIncreasingBatchSize("int", LongColumnVector.class);
+  }
+
+  private void testConvertToFloatIncreasingSize() throws Exception {
+    readORCFileIncreasingBatchSize("float", DoubleColumnVector.class);
+  }
+
+  public void testConvertToDecimalIncreasingSize() throws Exception {
+    readORCFileIncreasingBatchSize("decimal(6,1)", 
Decimal64ColumnVector.class);
+  }
+
+  private void testConvertToVarcharIncreasingSize() throws Exception {
+    readORCFileIncreasingBatchSize("varchar(10)", BytesColumnVector.class);
+  }
+
+  private void testConvertToTimestampIncreasingSize() throws Exception {
+    readORCFileIncreasingBatchSize("timestamp", TimestampColumnVector.class);
+  }
+
+  private void testConvertToDateIncreasingSize() throws Exception {
+    readORCFileIncreasingBatchSize("date", DateColumnVector.class);
+  }
+
+  private void testConvertToBinaryIncreasingSize() throws Exception {
+    readORCFileIncreasingBatchSize("binary", BytesColumnVector.class);
+  }
 }

Reply via email to