You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by om...@apache.org on 2013/08/12 17:03:31 UTC

svn commit: r1513155 [3/3] - in /hive/trunk: common/src/java/org/apache/hadoop/hive/conf/ ql/src/gen/protobuf/gen-java/org/apache/hadoop/hive/ql/io/orc/ ql/src/java/org/apache/hadoop/hive/ql/io/orc/ ql/src/protobuf/org/apache/hadoop/hive/ql/io/orc/ ql/...

Added: hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestNewIntegerEncoding.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestNewIntegerEncoding.java?rev=1513155&view=auto
==============================================================================
--- hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestNewIntegerEncoding.java (added)
+++ hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestNewIntegerEncoding.java Mon Aug 12 15:03:30 2013
@@ -0,0 +1,824 @@
+/**
+ * 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.hadoop.hive.ql.io.orc;
+
+import static junit.framework.Assert.assertEquals;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.nio.MappedByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.charset.Charset;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.LongWritable;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+
+import com.google.common.collect.Lists;
+import com.google.common.primitives.Longs;
+
+public class TestNewIntegerEncoding {
+
+  public static class Row {
+    Integer int1;
+    Long long1;
+
+    public Row(int val, long l) {
+      this.int1 = val;
+      this.long1 = l;
+    }
+  }
+
+  public List<Long> fetchData(String path) throws IOException {
+    List<Long> input = new ArrayList<Long>();
+    FileInputStream stream = new FileInputStream(new File(path));
+    try {
+      FileChannel fc = stream.getChannel();
+      MappedByteBuffer bb = fc.map(FileChannel.MapMode.READ_ONLY, 0, fc.size());
+      /* Instead of using default, pass in a decoder. */
+      String[] lines = Charset.defaultCharset().decode(bb).toString()
+          .split("\n");
+      for(String line : lines) {
+        long val = 0;
+        try {
+          val = Long.parseLong(line);
+        } catch (NumberFormatException e) {
+          // for now lets ignore (assign 0)
+        }
+        input.add(val);
+      }
+    } finally {
+      stream.close();
+    }
+    return input;
+  }
+
+  Path workDir = new Path(System.getProperty("test.tmp.dir", "target"
+      + File.separator + "test" + File.separator + "tmp"));
+
+  Configuration conf;
+  FileSystem fs;
+  Path testFilePath;
+  String resDir = "ql/src/test/resources";
+
+  @Rule
+  public TestName testCaseName = new TestName();
+
+  @Before
+  public void openFileSystem() throws Exception {
+    conf = new Configuration();
+    fs = FileSystem.getLocal(conf);
+    testFilePath = new Path(workDir, "TestOrcFile."
+        + testCaseName.getMethodName() + ".orc");
+    fs.delete(testFilePath, false);
+  }
+
+  @Test
+  public void testBasicRow() throws Exception {
+    ObjectInspector inspector;
+    synchronized (TestOrcFile.class) {
+      inspector = ObjectInspectorFactory.getReflectionObjectInspector(
+          Row.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
+    }
+
+    Writer writer = OrcFile.createWriter(fs, testFilePath, conf, inspector,
+        100000, CompressionKind.NONE, 10000, 10000);
+    writer.addRow(new Row(111, 1111L));
+    writer.addRow(new Row(111, 1111L));
+    writer.addRow(new Row(111, 1111L));
+    writer.close();
+
+    Reader reader = OrcFile.createReader(fs, testFilePath);
+    RecordReader rows = reader.rows(null);
+    while (rows.hasNext()) {
+      Object row = rows.next(null);
+      assertEquals(new IntWritable(111), ((OrcStruct) row).getFieldValue(0));
+      assertEquals(new LongWritable(1111), ((OrcStruct) row).getFieldValue(1));
+    }
+  }
+
+  @Test
+  public void testBasicOld() throws Exception {
+    ObjectInspector inspector;
+    synchronized (TestOrcFile.class) {
+      inspector = ObjectInspectorFactory.getReflectionObjectInspector(
+          Long.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
+    }
+
+    long[] inp = new long[] { 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 2, 3, 4, 5, 6,
+        7, 8, 9, 10, 1, 1, 1, 1, 1, 1, 10, 9, 7, 6, 5, 4, 3, 2, 1, 1, 1, 1, 1,
+        2, 5, 1, 3, 7, 1, 9, 2, 6, 3, 7, 1, 9, 2, 6, 3, 7, 1, 9, 2, 6, 3, 7, 1,
+        9, 2, 6, 3, 7, 1, 9, 2, 6, 2000, 2, 1, 1, 1, 1, 1, 3, 7, 1, 9, 2, 6, 1,
+        1, 1, 1, 1 };
+    List<Long> input = Lists.newArrayList(Longs.asList(inp));
+    conf.set("hive.exec.orc.write.format", "0.11");
+    Writer writer = OrcFile.createWriter(fs, testFilePath, conf, inspector,
+        100000, CompressionKind.NONE, 10000, 10000);
+    for(Long l : input) {
+      writer.addRow(l);
+    }
+    writer.close();
+
+    Reader reader = OrcFile.createReader(fs, testFilePath);
+    RecordReader rows = reader.rows(null);
+    int idx = 0;
+    while (rows.hasNext()) {
+      Object row = rows.next(null);
+      assertEquals(input.get(idx++).longValue(), ((LongWritable) row).get());
+    }
+  }
+
+  @Test
+  public void testBasicNew() throws Exception {
+    ObjectInspector inspector;
+    synchronized (TestOrcFile.class) {
+      inspector = ObjectInspectorFactory.getReflectionObjectInspector(
+          Long.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
+    }
+
+    long[] inp = new long[] { 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 1, 2, 3, 4, 5, 6,
+        7, 8, 9, 10, 1, 1, 1, 1, 1, 1, 10, 9, 7, 6, 5, 4, 3, 2, 1, 1, 1, 1, 1,
+        2, 5, 1, 3, 7, 1, 9, 2, 6, 3, 7, 1, 9, 2, 6, 3, 7, 1, 9, 2, 6, 3, 7, 1,
+        9, 2, 6, 3, 7, 1, 9, 2, 6, 2000, 2, 1, 1, 1, 1, 1, 3, 7, 1, 9, 2, 6, 1,
+        1, 1, 1, 1 };
+    List<Long> input = Lists.newArrayList(Longs.asList(inp));
+
+    Writer writer = OrcFile.createWriter(fs, testFilePath, conf, inspector,
+        100000, CompressionKind.NONE, 10000, 10000);
+    for(Long l : input) {
+      writer.addRow(l);
+    }
+    writer.close();
+
+    Reader reader = OrcFile.createReader(fs, testFilePath);
+    RecordReader rows = reader.rows(null);
+    int idx = 0;
+    while (rows.hasNext()) {
+      Object row = rows.next(null);
+      assertEquals(input.get(idx++).longValue(), ((LongWritable) row).get());
+    }
+  }
+  
+  @Test
+  public void testBasicDelta1() throws Exception {
+    ObjectInspector inspector;
+    synchronized (TestOrcFile.class) {
+      inspector = ObjectInspectorFactory.getReflectionObjectInspector(
+          Long.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
+    }
+
+    long[] inp = new long[] { -500, -400, -350, -325, -310 };
+    List<Long> input = Lists.newArrayList(Longs.asList(inp));
+
+    Writer writer = OrcFile.createWriter(fs, testFilePath, conf, inspector,
+        100000, CompressionKind.NONE, 10000, 10000);
+    for(Long l : input) {
+      writer.addRow(l);
+    }
+    writer.close();
+
+    Reader reader = OrcFile.createReader(fs, testFilePath);
+    RecordReader rows = reader.rows(null);
+    int idx = 0;
+    while (rows.hasNext()) {
+      Object row = rows.next(null);
+      assertEquals(input.get(idx++).longValue(), ((LongWritable) row).get());
+    }
+  }
+
+  @Test
+  public void testBasicDelta2() throws Exception {
+    ObjectInspector inspector;
+    synchronized (TestOrcFile.class) {
+      inspector = ObjectInspectorFactory.getReflectionObjectInspector(
+          Long.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
+    }
+
+    long[] inp = new long[] { -500, -600, -650, -675, -710 };
+    List<Long> input = Lists.newArrayList(Longs.asList(inp));
+
+    Writer writer = OrcFile.createWriter(fs, testFilePath, conf, inspector,
+        100000, CompressionKind.NONE, 10000, 10000);
+    for(Long l : input) {
+      writer.addRow(l);
+    }
+    writer.close();
+
+    Reader reader = OrcFile.createReader(fs, testFilePath);
+    RecordReader rows = reader.rows(null);
+    int idx = 0;
+    while (rows.hasNext()) {
+      Object row = rows.next(null);
+      assertEquals(input.get(idx++).longValue(), ((LongWritable) row).get());
+    }
+  }
+
+  @Test
+  public void testBasicDelta3() throws Exception {
+    ObjectInspector inspector;
+    synchronized (TestOrcFile.class) {
+      inspector = ObjectInspectorFactory.getReflectionObjectInspector(
+          Long.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
+    }
+
+    long[] inp = new long[] { 500, 400, 350, 325, 310 };
+    List<Long> input = Lists.newArrayList(Longs.asList(inp));
+
+    Writer writer = OrcFile.createWriter(fs, testFilePath, conf, inspector,
+        100000, CompressionKind.NONE, 10000, 10000);
+    for(Long l : input) {
+      writer.addRow(l);
+    }
+    writer.close();
+
+    Reader reader = OrcFile.createReader(fs, testFilePath);
+    RecordReader rows = reader.rows(null);
+    int idx = 0;
+    while (rows.hasNext()) {
+      Object row = rows.next(null);
+      assertEquals(input.get(idx++).longValue(), ((LongWritable) row).get());
+    }
+  }
+
+  @Test
+  public void testBasicDelta4() throws Exception {
+    ObjectInspector inspector;
+    synchronized (TestOrcFile.class) {
+      inspector = ObjectInspectorFactory.getReflectionObjectInspector(
+          Long.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
+    }
+
+    long[] inp = new long[] { 500, 600, 650, 675, 710 };
+    List<Long> input = Lists.newArrayList(Longs.asList(inp));
+
+    Writer writer = OrcFile.createWriter(fs, testFilePath, conf, inspector,
+        100000, CompressionKind.NONE, 10000, 10000);
+    for(Long l : input) {
+      writer.addRow(l);
+    }
+    writer.close();
+
+    Reader reader = OrcFile.createReader(fs, testFilePath);
+    RecordReader rows = reader.rows(null);
+    int idx = 0;
+    while (rows.hasNext()) {
+      Object row = rows.next(null);
+      assertEquals(input.get(idx++).longValue(), ((LongWritable) row).get());
+    }
+  }
+
+  @Test
+  public void testIntegerMin() throws Exception {
+    ObjectInspector inspector;
+    synchronized (TestOrcFile.class) {
+      inspector = ObjectInspectorFactory.getReflectionObjectInspector(
+          Long.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
+    }
+
+    List<Long> input = Lists.newArrayList();
+    input.add((long) Integer.MIN_VALUE);
+
+    Writer writer = OrcFile.createWriter(fs, testFilePath, conf, inspector,
+        100000, CompressionKind.ZLIB, 10000, 10000);
+    for(Long l : input) {
+      writer.addRow(l);
+    }
+    writer.close();
+
+    Reader reader = OrcFile.createReader(fs, testFilePath);
+    RecordReader rows = reader.rows(null);
+    int idx = 0;
+    while (rows.hasNext()) {
+      Object row = rows.next(null);
+      assertEquals(input.get(idx++).longValue(), ((LongWritable) row).get());
+    }
+  }
+
+  @Test
+  public void testIntegerMax() throws Exception {
+    ObjectInspector inspector;
+    synchronized (TestOrcFile.class) {
+      inspector = ObjectInspectorFactory.getReflectionObjectInspector(
+          Long.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
+    }
+
+    List<Long> input = Lists.newArrayList();
+    input.add((long) Integer.MAX_VALUE);
+
+    Writer writer = OrcFile.createWriter(fs, testFilePath, conf, inspector,
+        100000, CompressionKind.NONE, 10000, 10000);
+    for(Long l : input) {
+      writer.addRow(l);
+    }
+    writer.close();
+
+    Reader reader = OrcFile.createReader(fs, testFilePath);
+    RecordReader rows = reader.rows(null);
+    int idx = 0;
+    while (rows.hasNext()) {
+      Object row = rows.next(null);
+      assertEquals(input.get(idx++).longValue(), ((LongWritable) row).get());
+    }
+  }
+
+  @Test
+  public void testLongMin() throws Exception {
+    ObjectInspector inspector;
+    synchronized (TestOrcFile.class) {
+      inspector = ObjectInspectorFactory.getReflectionObjectInspector(
+          Long.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
+    }
+
+    List<Long> input = Lists.newArrayList();
+    input.add(Long.MIN_VALUE);
+
+    Writer writer = OrcFile.createWriter(fs, testFilePath, conf, inspector,
+        100000, CompressionKind.NONE, 10000, 10000);
+    for(Long l : input) {
+      writer.addRow(l);
+    }
+    writer.close();
+
+    Reader reader = OrcFile.createReader(fs, testFilePath);
+    RecordReader rows = reader.rows(null);
+    int idx = 0;
+    while (rows.hasNext()) {
+      Object row = rows.next(null);
+      assertEquals(input.get(idx++).longValue(), ((LongWritable) row).get());
+    }
+  }
+
+  @Test
+  public void testLongMax() throws Exception {
+    ObjectInspector inspector;
+    synchronized (TestOrcFile.class) {
+      inspector = ObjectInspectorFactory.getReflectionObjectInspector(
+          Long.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
+    }
+
+    List<Long> input = Lists.newArrayList();
+    input.add(Long.MAX_VALUE);
+
+    Writer writer = OrcFile.createWriter(fs, testFilePath, conf, inspector,
+        100000, CompressionKind.NONE, 10000, 10000);
+    for(Long l : input) {
+      writer.addRow(l);
+    }
+    writer.close();
+
+    Reader reader = OrcFile.createReader(fs, testFilePath);
+    RecordReader rows = reader.rows(null);
+    int idx = 0;
+    while (rows.hasNext()) {
+      Object row = rows.next(null);
+      assertEquals(input.get(idx++).longValue(), ((LongWritable) row).get());
+    }
+  }
+
+  @Test
+  public void testRandomInt() throws Exception {
+    ObjectInspector inspector;
+    synchronized (TestOrcFile.class) {
+      inspector = ObjectInspectorFactory.getReflectionObjectInspector(
+          Long.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
+    }
+
+    List<Long> input = Lists.newArrayList();
+    Random rand = new Random();
+    for(int i = 0; i < 100000; i++) {
+      input.add((long) rand.nextInt());
+    }
+
+    Writer writer = OrcFile.createWriter(fs, testFilePath, conf, inspector,
+        100000, CompressionKind.NONE, 10000, 10000);
+    for(Long l : input) {
+      writer.addRow(l);
+    }
+    writer.close();
+
+    Reader reader = OrcFile.createReader(fs, testFilePath);
+    RecordReader rows = reader.rows(null);
+    int idx = 0;
+    while (rows.hasNext()) {
+      Object row = rows.next(null);
+      assertEquals(input.get(idx++).longValue(), ((LongWritable) row).get());
+    }
+  }
+
+  @Test
+  public void testRandomLong() throws Exception {
+    ObjectInspector inspector;
+    synchronized (TestOrcFile.class) {
+      inspector = ObjectInspectorFactory.getReflectionObjectInspector(
+          Long.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
+    }
+
+    List<Long> input = Lists.newArrayList();
+    Random rand = new Random();
+    for(int i = 0; i < 100000; i++) {
+      input.add(rand.nextLong());
+    }
+
+    Writer writer = OrcFile.createWriter(fs, testFilePath, conf, inspector,
+        100000, CompressionKind.NONE, 10000, 10000);
+    for(Long l : input) {
+      writer.addRow(l);
+    }
+    writer.close();
+
+    Reader reader = OrcFile.createReader(fs, testFilePath);
+    RecordReader rows = reader.rows(null);
+    int idx = 0;
+    while (rows.hasNext()) {
+      Object row = rows.next(null);
+      assertEquals(input.get(idx++).longValue(), ((LongWritable) row).get());
+    }
+  }
+
+  @Test
+  public void testPatchedBaseNegativeMin() throws Exception {
+    ObjectInspector inspector;
+    synchronized (TestOrcFile.class) {
+      inspector = ObjectInspectorFactory.getReflectionObjectInspector(
+          Long.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
+    }
+
+    long[] inp = new long[] { 20, 2, 3, 2, 1, 3, 17, 71, 35, 2, 1, 139, 2, 2,
+        3, 1783, 475, 2, 1, 1, 3, 1, 3, 2, 32, 1, 2, 3, 1, 8, 30, 1, 3, 414, 1,
+        1, 135, 3, 3, 1, 414, 2, 1, 2, 2, 594, 2, 5, 6, 4, 11, 1, 2, 2, 1, 1,
+        52, 4, 1, 2, 7, 1, 17, 334, 1, 2, 1, 2, 2, 6, 1, 266, 1, 2, 217, 2, 6,
+        2, 13, 2, 2, 1, 2, 3, 5, 1, 2, 1, 7244, 11813, 1, 33, 2, -13, 1, 2, 3,
+        13, 1, 92, 3, 13, 5, 14, 9, 141, 12, 6, 15, 25, 1, 1, 1, 46, 2, 1, 1,
+        141, 3, 1, 1, 1, 1, 2, 1, 4, 34, 5, 78, 8, 1, 2, 2, 1, 9, 10, 2, 1, 4,
+        13, 1, 5, 4, 4, 19, 5, 1, 1, 1, 68, 33, 399, 1, 1885, 25, 5, 2, 4, 1,
+        1, 2, 16, 1, 2966, 3, 1, 1, 25501, 1, 1, 1, 66, 1, 3, 8, 131, 14, 5, 1,
+        2, 2, 1, 1, 8, 1, 1, 2, 1, 5, 9, 2, 3, 112, 13, 2, 2, 1, 5, 10, 3, 1,
+        1, 13, 2, 3, 4, 1, 3, 1, 1, 2, 1, 1, 2, 4, 2, 207, 1, 1, 2, 4, 3, 3, 2,
+        2, 16 };
+    List<Long> input = Lists.newArrayList(Longs.asList(inp));
+
+    Writer writer = OrcFile.createWriter(fs, testFilePath, conf, inspector,
+        100000, CompressionKind.NONE, 10000, 10000);
+    for(Long l : input) {
+      writer.addRow(l);
+    }
+    writer.close();
+
+    Reader reader = OrcFile.createReader(fs, testFilePath);
+    RecordReader rows = reader.rows(null);
+    int idx = 0;
+    while (rows.hasNext()) {
+      Object row = rows.next(null);
+      assertEquals(input.get(idx++).longValue(), ((LongWritable) row).get());
+    }
+  }
+
+  @Test
+  public void testPatchedBaseNegativeMin2() throws Exception {
+    ObjectInspector inspector;
+    synchronized (TestOrcFile.class) {
+      inspector = ObjectInspectorFactory.getReflectionObjectInspector(
+          Long.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
+    }
+
+    long[] inp = new long[] { 20, 2, 3, 2, 1, 3, 17, 71, 35, 2, 1, 139, 2, 2,
+        3, 1783, 475, 2, 1, 1, 3, 1, 3, 2, 32, 1, 2, 3, 1, 8, 30, 1, 3, 414, 1,
+        1, 135, 3, 3, 1, 414, 2, 1, 2, 2, 594, 2, 5, 6, 4, 11, 1, 2, 2, 1, 1,
+        52, 4, 1, 2, 7, 1, 17, 334, 1, 2, 1, 2, 2, 6, 1, 266, 1, 2, 217, 2, 6,
+        2, 13, 2, 2, 1, 2, 3, 5, 1, 2, 1, 7244, 11813, 1, 33, 2, -1, 1, 2, 3,
+        13, 1, 92, 3, 13, 5, 14, 9, 141, 12, 6, 15, 25, 1, 1, 1, 46, 2, 1, 1,
+        141, 3, 1, 1, 1, 1, 2, 1, 4, 34, 5, 78, 8, 1, 2, 2, 1, 9, 10, 2, 1, 4,
+        13, 1, 5, 4, 4, 19, 5, 1, 1, 1, 68, 33, 399, 1, 1885, 25, 5, 2, 4, 1,
+        1, 2, 16, 1, 2966, 3, 1, 1, 25501, 1, 1, 1, 66, 1, 3, 8, 131, 14, 5, 1,
+        2, 2, 1, 1, 8, 1, 1, 2, 1, 5, 9, 2, 3, 112, 13, 2, 2, 1, 5, 10, 3, 1,
+        1, 13, 2, 3, 4, 1, 3, 1, 1, 2, 1, 1, 2, 4, 2, 207, 1, 1, 2, 4, 3, 3, 2,
+        2, 16 };
+    List<Long> input = Lists.newArrayList(Longs.asList(inp));
+
+    Writer writer = OrcFile.createWriter(fs, testFilePath, conf, inspector,
+        100000, CompressionKind.NONE, 10000, 10000);
+    for(Long l : input) {
+      writer.addRow(l);
+    }
+    writer.close();
+
+    Reader reader = OrcFile.createReader(fs, testFilePath);
+    RecordReader rows = reader.rows(null);
+    int idx = 0;
+    while (rows.hasNext()) {
+      Object row = rows.next(null);
+      assertEquals(input.get(idx++).longValue(), ((LongWritable) row).get());
+    }
+  }
+
+  @Test
+  public void testPatchedBaseNegativeMin3() throws Exception {
+    ObjectInspector inspector;
+    synchronized (TestOrcFile.class) {
+      inspector = ObjectInspectorFactory.getReflectionObjectInspector(
+          Long.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
+    }
+
+    long[] inp = new long[] { 20, 2, 3, 2, 1, 3, 17, 71, 35, 2, 1, 139, 2, 2,
+        3, 1783, 475, 2, 1, 1, 3, 1, 3, 2, 32, 1, 2, 3, 1, 8, 30, 1, 3, 414, 1,
+        1, 135, 3, 3, 1, 414, 2, 1, 2, 2, 594, 2, 5, 6, 4, 11, 1, 2, 2, 1, 1,
+        52, 4, 1, 2, 7, 1, 17, 334, 1, 2, 1, 2, 2, 6, 1, 266, 1, 2, 217, 2, 6,
+        2, 13, 2, 2, 1, 2, 3, 5, 1, 2, 1, 7244, 11813, 1, 33, 2, 0, 1, 2, 3,
+        13, 1, 92, 3, 13, 5, 14, 9, 141, 12, 6, 15, 25, 1, 1, 1, 46, 2, 1, 1,
+        141, 3, 1, 1, 1, 1, 2, 1, 4, 34, 5, 78, 8, 1, 2, 2, 1, 9, 10, 2, 1, 4,
+        13, 1, 5, 4, 4, 19, 5, 1, 1, 1, 68, 33, 399, 1, 1885, 25, 5, 2, 4, 1,
+        1, 2, 16, 1, 2966, 3, 1, 1, 25501, 1, 1, 1, 66, 1, 3, 8, 131, 14, 5, 1,
+        2, 2, 1, 1, 8, 1, 1, 2, 1, 5, 9, 2, 3, 112, 13, 2, 2, 1, 5, 10, 3, 1,
+        1, 13, 2, 3, 4, 1, 3, 1, 1, 2, 1, 1, 2, 4, 2, 207, 1, 1, 2, 4, 3, 3, 2,
+        2, 16 };
+    List<Long> input = Lists.newArrayList(Longs.asList(inp));
+
+    Writer writer = OrcFile.createWriter(fs, testFilePath, conf, inspector,
+        100000, CompressionKind.NONE, 10000, 10000);
+    for(Long l : input) {
+      writer.addRow(l);
+    }
+    writer.close();
+
+    Reader reader = OrcFile.createReader(fs, testFilePath);
+    RecordReader rows = reader.rows(null);
+    int idx = 0;
+    while (rows.hasNext()) {
+      Object row = rows.next(null);
+      assertEquals(input.get(idx++).longValue(), ((LongWritable) row).get());
+    }
+  }
+
+  @Test
+  public void testPatchedBaseNegativeMin4() throws Exception {
+    ObjectInspector inspector;
+    synchronized (TestOrcFile.class) {
+      inspector = ObjectInspectorFactory.getReflectionObjectInspector(
+          Long.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
+    }
+
+    long[] inp = new long[] { 13, 13, 11, 8, 13, 10, 10, 11, 11, 14, 11, 7, 13,
+        12, 12, 11, 15, 12, 12, 9, 8, 10, 13, 11, 8, 6, 5, 6, 11, 7, 15, 10, 7,
+        6, 8, 7, 9, 9, 11, 33, 11, 3, 7, 4, 6, 10, 14, 12, 5, 14, 7, 6 };
+    List<Long> input = Lists.newArrayList(Longs.asList(inp));
+
+    Writer writer = OrcFile.createWriter(fs, testFilePath, conf, inspector,
+        100000, CompressionKind.NONE, 10000, 10000);
+    for(Long l : input) {
+      writer.addRow(l);
+    }
+    writer.close();
+
+    Reader reader = OrcFile.createReader(fs, testFilePath);
+    RecordReader rows = reader.rows(null);
+    int idx = 0;
+    while (rows.hasNext()) {
+      Object row = rows.next(null);
+      assertEquals(input.get(idx++).longValue(), ((LongWritable) row).get());
+    }
+  }
+
+  @Test
+  public void testPatchedBaseAt0() throws Exception {
+    ObjectInspector inspector;
+    synchronized (TestOrcFile.class) {
+      inspector = ObjectInspectorFactory.getReflectionObjectInspector(
+          Long.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
+    }
+
+    List<Long> input = Lists.newArrayList();
+    Random rand = new Random();
+    for(int i = 0; i < 5120; i++) {
+      input.add((long) rand.nextInt(100));
+    }
+    input.set(0, 20000L);
+
+    Writer writer = OrcFile.createWriter(fs, testFilePath, conf, inspector,
+        100000, CompressionKind.NONE, 10000, 10000);
+    for(Long l : input) {
+      writer.addRow(l);
+    }
+    writer.close();
+
+    Reader reader = OrcFile.createReader(fs, testFilePath);
+    RecordReader rows = reader.rows(null);
+    int idx = 0;
+    while (rows.hasNext()) {
+      Object row = rows.next(null);
+      assertEquals(input.get(idx++).longValue(), ((LongWritable) row).get());
+    }
+  }
+
+  @Test
+  public void testPatchedBaseAt1() throws Exception {
+    ObjectInspector inspector;
+    synchronized (TestOrcFile.class) {
+      inspector = ObjectInspectorFactory.getReflectionObjectInspector(
+          Long.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
+    }
+
+    List<Long> input = Lists.newArrayList();
+    Random rand = new Random();
+    for(int i = 0; i < 5120; i++) {
+      input.add((long) rand.nextInt(100));
+    }
+    input.set(1, 20000L);
+
+    Writer writer = OrcFile.createWriter(fs, testFilePath, conf, inspector,
+        100000, CompressionKind.NONE, 10000, 10000);
+    for(Long l : input) {
+      writer.addRow(l);
+    }
+    writer.close();
+
+    Reader reader = OrcFile.createReader(fs, testFilePath);
+    RecordReader rows = reader.rows(null);
+    int idx = 0;
+    while (rows.hasNext()) {
+      Object row = rows.next(null);
+      assertEquals(input.get(idx++).longValue(), ((LongWritable) row).get());
+    }
+  }
+
+  @Test
+  public void testPatchedBaseAt255() throws Exception {
+    ObjectInspector inspector;
+    synchronized (TestOrcFile.class) {
+      inspector = ObjectInspectorFactory.getReflectionObjectInspector(
+          Long.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
+    }
+
+    List<Long> input = Lists.newArrayList();
+    Random rand = new Random();
+    for(int i = 0; i < 5120; i++) {
+      input.add((long) rand.nextInt(100));
+    }
+    input.set(255, 20000L);
+
+    Writer writer = OrcFile.createWriter(fs, testFilePath, conf, inspector,
+        100000, CompressionKind.ZLIB, 10000, 10000);
+    for(Long l : input) {
+      writer.addRow(l);
+    }
+    writer.close();
+
+    Reader reader = OrcFile.createReader(fs, testFilePath);
+    RecordReader rows = reader.rows(null);
+    int idx = 0;
+    while (rows.hasNext()) {
+      Object row = rows.next(null);
+      assertEquals(input.get(idx++).longValue(), ((LongWritable) row).get());
+    }
+  }
+
+  @Test
+  public void testPatchedBaseAt256() throws Exception {
+    ObjectInspector inspector;
+    synchronized (TestOrcFile.class) {
+      inspector = ObjectInspectorFactory.getReflectionObjectInspector(
+          Long.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
+    }
+
+    List<Long> input = Lists.newArrayList();
+    Random rand = new Random();
+    for(int i = 0; i < 5120; i++) {
+      input.add((long) rand.nextInt(100));
+    }
+    input.set(256, 20000L);
+
+    Writer writer = OrcFile.createWriter(fs, testFilePath, conf, inspector,
+        100000, CompressionKind.ZLIB, 10000, 10000);
+    for(Long l : input) {
+      writer.addRow(l);
+    }
+    writer.close();
+
+    Reader reader = OrcFile.createReader(fs, testFilePath);
+    RecordReader rows = reader.rows(null);
+    int idx = 0;
+    while (rows.hasNext()) {
+      Object row = rows.next(null);
+      assertEquals(input.get(idx++).longValue(), ((LongWritable) row).get());
+    }
+  }
+
+  @Test
+  public void testPatchedBase510() throws Exception {
+    ObjectInspector inspector;
+    synchronized (TestOrcFile.class) {
+      inspector = ObjectInspectorFactory.getReflectionObjectInspector(
+          Long.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
+    }
+
+    List<Long> input = Lists.newArrayList();
+    Random rand = new Random();
+    for(int i = 0; i < 5120; i++) {
+      input.add((long) rand.nextInt(100));
+    }
+    input.set(510, 20000L);
+
+    Writer writer = OrcFile.createWriter(fs, testFilePath, conf, inspector,
+        100000, CompressionKind.ZLIB, 10000, 10000);
+    for(Long l : input) {
+      writer.addRow(l);
+    }
+    writer.close();
+
+    Reader reader = OrcFile.createReader(fs, testFilePath);
+    RecordReader rows = reader.rows(null);
+    int idx = 0;
+    while (rows.hasNext()) {
+      Object row = rows.next(null);
+      assertEquals(input.get(idx++).longValue(), ((LongWritable) row).get());
+    }
+  }
+
+  @Test
+  public void testPatchedBase511() throws Exception {
+    ObjectInspector inspector;
+    synchronized (TestOrcFile.class) {
+      inspector = ObjectInspectorFactory.getReflectionObjectInspector(
+          Long.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
+    }
+
+    List<Long> input = Lists.newArrayList();
+    Random rand = new Random();
+    for(int i = 0; i < 5120; i++) {
+      input.add((long) rand.nextInt(100));
+    }
+    input.set(511, 20000L);
+
+    Writer writer = OrcFile.createWriter(fs, testFilePath, conf, inspector,
+        100000, CompressionKind.ZLIB, 10000, 10000);
+    for(Long l : input) {
+      writer.addRow(l);
+    }
+    writer.close();
+
+    Reader reader = OrcFile.createReader(fs, testFilePath);
+    RecordReader rows = reader.rows(null);
+    int idx = 0;
+    while (rows.hasNext()) {
+      Object row = rows.next(null);
+      assertEquals(input.get(idx++).longValue(), ((LongWritable) row).get());
+    }
+  }
+
+  @Test
+  public void testSeek() throws Exception {
+    ObjectInspector inspector;
+    synchronized (TestOrcFile.class) {
+      inspector = ObjectInspectorFactory.getReflectionObjectInspector(
+          Long.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
+    }
+
+    List<Long> input = Lists.newArrayList();
+    Random rand = new Random();
+    for(int i = 0; i < 100000; i++) {
+      input.add((long) rand.nextInt());
+    }
+    conf.set("hive.exec.orc.write.format", "0.11");
+    Writer writer = OrcFile.createWriter(fs, testFilePath, conf, inspector,
+        100000, CompressionKind.NONE, 10000, 10000);
+    for(Long l : input) {
+      writer.addRow(l);
+    }
+    writer.close();
+
+    Reader reader = OrcFile.createReader(fs, testFilePath);
+    RecordReader rows = reader.rows(null);
+    int idx = 55555;
+    rows.seekToRow(idx);
+    while (rows.hasNext()) {
+      Object row = rows.next(null);
+      assertEquals(input.get(idx++).longValue(), ((LongWritable) row).get());
+    }
+  }
+}

Modified: hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcNullOptimization.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcNullOptimization.java?rev=1513155&r1=1513154&r2=1513155&view=diff
==============================================================================
--- hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcNullOptimization.java (original)
+++ hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcNullOptimization.java Mon Aug 12 15:03:30 2013
@@ -1,3 +1,20 @@
+/**
+ * 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.hadoop.hive.ql.io.orc;
 
 import static junit.framework.Assert.assertEquals;

Modified: hive/trunk/ql/src/test/resources/orc-file-dump-dictionary-threshold.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/resources/orc-file-dump-dictionary-threshold.out?rev=1513155&r1=1513154&r2=1513155&view=diff
==============================================================================
--- hive/trunk/ql/src/test/resources/orc-file-dump-dictionary-threshold.out (original)
+++ hive/trunk/ql/src/test/resources/orc-file-dump-dictionary-threshold.out Mon Aug 12 15:03:30 2013
@@ -11,68 +11,68 @@ Statistics:
   Column 3: count: 21000 min: Darkness,-230 max: worst-54-290-346-648-908-996-1038-1080-1560-1584-1620-1744-1770-1798-1852-1966-2162-2244-2286-2296-2534-2660-3114-3676-3788-4068-4150-4706-4744-5350-5420-5582-5696-5726-6006-6020-6024-6098-6184-6568-6636-6802-6994-7004-7318-7498-7758-7780-7798-7920-7952-7960-7988-8232-8256-8390-8416-8478-8620-8840-8984-9038-9128-9236-9248-9344-9594-9650-9714-9928-9938-10178-10368-10414-10502-10732-10876-11008-11158-11410-11722-11836-11964-12054-12096-12126-12136-12202-12246-12298-12616-12774-12782-12790-12802-12976-13216-13246-13502-13766-14454-14974-15004-15124-15252-15294-15356-15530-15610-16316-16936-17024-17122-17214-17310-17528-17682-17742-17870-17878-18010-18410-18524-18788-19204-19254-19518-19596-19786-19874-19904-20390-20752-20936
 
 Stripes:
-  Stripe: offset: 3 data: 107035 rows: 4000 tail: 65 index: 217
+  Stripe: offset: 3 data: 102311 rows: 4000 tail: 68 index: 217
     Stream: column 0 section ROW_INDEX start: 3 length 10
     Stream: column 1 section ROW_INDEX start: 13 length 36
     Stream: column 2 section ROW_INDEX start: 49 length 39
     Stream: column 3 section ROW_INDEX start: 88 length 132
-    Stream: column 1 section DATA start: 220 length 18043
-    Stream: column 2 section DATA start: 18263 length 34740
-    Stream: column 3 section DATA start: 53003 length 50887
-    Stream: column 3 section LENGTH start: 103890 length 3365
+    Stream: column 1 section DATA start: 220 length 16022
+    Stream: column 2 section DATA start: 16242 length 32028
+    Stream: column 3 section DATA start: 48270 length 50887
+    Stream: column 3 section LENGTH start: 99157 length 3374
     Encoding column 0: DIRECT
-    Encoding column 1: DIRECT
-    Encoding column 2: DIRECT
-    Encoding column 3: DIRECT
-  Stripe: offset: 107320 data: 289727 rows: 5000 tail: 65 index: 349
-    Stream: column 0 section ROW_INDEX start: 107320 length 10
-    Stream: column 1 section ROW_INDEX start: 107330 length 36
-    Stream: column 2 section ROW_INDEX start: 107366 length 39
-    Stream: column 3 section ROW_INDEX start: 107405 length 264
-    Stream: column 1 section DATA start: 107669 length 22581
-    Stream: column 2 section DATA start: 130250 length 43426
-    Stream: column 3 section DATA start: 173676 length 219588
-    Stream: column 3 section LENGTH start: 393264 length 4132
+    Encoding column 1: DIRECT_V2
+    Encoding column 2: DIRECT_V2
+    Encoding column 3: DIRECT_V2
+  Stripe: offset: 102599 data: 284999 rows: 5000 tail: 68 index: 349
+    Stream: column 0 section ROW_INDEX start: 102599 length 10
+    Stream: column 1 section ROW_INDEX start: 102609 length 36
+    Stream: column 2 section ROW_INDEX start: 102645 length 39
+    Stream: column 3 section ROW_INDEX start: 102684 length 264
+    Stream: column 1 section DATA start: 102948 length 20029
+    Stream: column 2 section DATA start: 122977 length 40035
+    Stream: column 3 section DATA start: 163012 length 219588
+    Stream: column 3 section LENGTH start: 382600 length 5347
     Encoding column 0: DIRECT
-    Encoding column 1: DIRECT
-    Encoding column 2: DIRECT
-    Encoding column 3: DIRECT
-  Stripe: offset: 397461 data: 496162 rows: 5000 tail: 66 index: 536
-    Stream: column 0 section ROW_INDEX start: 397461 length 10
-    Stream: column 1 section ROW_INDEX start: 397471 length 36
-    Stream: column 2 section ROW_INDEX start: 397507 length 39
-    Stream: column 3 section ROW_INDEX start: 397546 length 451
-    Stream: column 1 section DATA start: 397997 length 22605
-    Stream: column 2 section DATA start: 420602 length 43444
-    Stream: column 3 section DATA start: 464046 length 425862
-    Stream: column 3 section LENGTH start: 889908 length 4251
+    Encoding column 1: DIRECT_V2
+    Encoding column 2: DIRECT_V2
+    Encoding column 3: DIRECT_V2
+  Stripe: offset: 388015 data: 491655 rows: 5000 tail: 69 index: 536
+    Stream: column 0 section ROW_INDEX start: 388015 length 10
+    Stream: column 1 section ROW_INDEX start: 388025 length 36
+    Stream: column 2 section ROW_INDEX start: 388061 length 39
+    Stream: column 3 section ROW_INDEX start: 388100 length 451
+    Stream: column 1 section DATA start: 388551 length 20029
+    Stream: column 2 section DATA start: 408580 length 40035
+    Stream: column 3 section DATA start: 448615 length 425862
+    Stream: column 3 section LENGTH start: 874477 length 5729
     Encoding column 0: DIRECT
-    Encoding column 1: DIRECT
-    Encoding column 2: DIRECT
-    Encoding column 3: DIRECT
-  Stripe: offset: 894225 data: 711982 rows: 5000 tail: 65 index: 677
-    Stream: column 0 section ROW_INDEX start: 894225 length 10
-    Stream: column 1 section ROW_INDEX start: 894235 length 36
-    Stream: column 2 section ROW_INDEX start: 894271 length 39
-    Stream: column 3 section ROW_INDEX start: 894310 length 592
-    Stream: column 1 section DATA start: 894902 length 22591
-    Stream: column 2 section DATA start: 917493 length 43414
-    Stream: column 3 section DATA start: 960907 length 641580
-    Stream: column 3 section LENGTH start: 1602487 length 4397
+    Encoding column 1: DIRECT_V2
+    Encoding column 2: DIRECT_V2
+    Encoding column 3: DIRECT_V2
+  Stripe: offset: 880275 data: 707368 rows: 5000 tail: 68 index: 677
+    Stream: column 0 section ROW_INDEX start: 880275 length 10
+    Stream: column 1 section ROW_INDEX start: 880285 length 36
+    Stream: column 2 section ROW_INDEX start: 880321 length 39
+    Stream: column 3 section ROW_INDEX start: 880360 length 592
+    Stream: column 1 section DATA start: 880952 length 20029
+    Stream: column 2 section DATA start: 900981 length 40035
+    Stream: column 3 section DATA start: 941016 length 641580
+    Stream: column 3 section LENGTH start: 1582596 length 5724
     Encoding column 0: DIRECT
-    Encoding column 1: DIRECT
-    Encoding column 2: DIRECT
-    Encoding column 3: DIRECT
-  Stripe: offset: 1606949 data: 350645 rows: 2000 tail: 66 index: 786
-    Stream: column 0 section ROW_INDEX start: 1606949 length 10
-    Stream: column 1 section ROW_INDEX start: 1606959 length 36
-    Stream: column 2 section ROW_INDEX start: 1606995 length 39
-    Stream: column 3 section ROW_INDEX start: 1607034 length 701
-    Stream: column 1 section DATA start: 1607735 length 9027
-    Stream: column 2 section DATA start: 1616762 length 17375
-    Stream: column 3 section DATA start: 1634137 length 322259
-    Stream: column 3 section LENGTH start: 1956396 length 1984
+    Encoding column 1: DIRECT_V2
+    Encoding column 2: DIRECT_V2
+    Encoding column 3: DIRECT_V2
+  Stripe: offset: 1588388 data: 348697 rows: 2000 tail: 67 index: 786
+    Stream: column 0 section ROW_INDEX start: 1588388 length 10
+    Stream: column 1 section ROW_INDEX start: 1588398 length 36
+    Stream: column 2 section ROW_INDEX start: 1588434 length 39
+    Stream: column 3 section ROW_INDEX start: 1588473 length 701
+    Stream: column 1 section DATA start: 1589174 length 8011
+    Stream: column 2 section DATA start: 1597185 length 16014
+    Stream: column 3 section DATA start: 1613199 length 322259
+    Stream: column 3 section LENGTH start: 1935458 length 2413
     Encoding column 0: DIRECT
-    Encoding column 1: DIRECT
-    Encoding column 2: DIRECT
-    Encoding column 3: DIRECT
+    Encoding column 1: DIRECT_V2
+    Encoding column 2: DIRECT_V2
+    Encoding column 3: DIRECT_V2
\ No newline at end of file

Modified: hive/trunk/ql/src/test/resources/orc-file-dump.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/resources/orc-file-dump.out?rev=1513155&r1=1513154&r2=1513155&view=diff
==============================================================================
--- hive/trunk/ql/src/test/resources/orc-file-dump.out (original)
+++ hive/trunk/ql/src/test/resources/orc-file-dump.out Mon Aug 12 15:03:30 2013
@@ -11,73 +11,73 @@ Statistics:
   Column 3: count: 21000 min: Darkness, max: worst
 
 Stripes:
-  Stripe: offset: 3 data: 69605 rows: 5000 tail: 72 index: 119
+  Stripe: offset: 3 data: 63766 rows: 5000 tail: 74 index: 119
     Stream: column 0 section ROW_INDEX start: 3 length 10
     Stream: column 1 section ROW_INDEX start: 13 length 35
     Stream: column 2 section ROW_INDEX start: 48 length 39
     Stream: column 3 section ROW_INDEX start: 87 length 35
-    Stream: column 1 section DATA start: 122 length 22605
-    Stream: column 2 section DATA start: 22727 length 43426
-    Stream: column 3 section DATA start: 66153 length 3403
-    Stream: column 3 section LENGTH start: 69556 length 38
-    Stream: column 3 section DICTIONARY_DATA start: 69594 length 133
+    Stream: column 1 section DATA start: 122 length 20029
+    Stream: column 2 section DATA start: 20151 length 40035
+    Stream: column 3 section DATA start: 60186 length 3544
+    Stream: column 3 section LENGTH start: 63730 length 25
+    Stream: column 3 section DICTIONARY_DATA start: 63755 length 133
     Encoding column 0: DIRECT
-    Encoding column 1: DIRECT
-    Encoding column 2: DIRECT
-    Encoding column 3: DICTIONARY[35]
-  Stripe: offset: 69799 data: 69584 rows: 5000 tail: 73 index: 118
-    Stream: column 0 section ROW_INDEX start: 69799 length 10
-    Stream: column 1 section ROW_INDEX start: 69809 length 34
-    Stream: column 2 section ROW_INDEX start: 69843 length 39
-    Stream: column 3 section ROW_INDEX start: 69882 length 35
-    Stream: column 1 section DATA start: 69917 length 22597
-    Stream: column 2 section DATA start: 92514 length 43439
-    Stream: column 3 section DATA start: 135953 length 3377
-    Stream: column 3 section LENGTH start: 139330 length 38
-    Stream: column 3 section DICTIONARY_DATA start: 139368 length 133
+    Encoding column 1: DIRECT_V2
+    Encoding column 2: DIRECT_V2
+    Encoding column 3: DICTIONARY_V2
+  Stripe: offset: 63962 data: 63755 rows: 5000 tail: 76 index: 118
+    Stream: column 0 section ROW_INDEX start: 63962 length 10
+    Stream: column 1 section ROW_INDEX start: 63972 length 34
+    Stream: column 2 section ROW_INDEX start: 64006 length 39
+    Stream: column 3 section ROW_INDEX start: 64045 length 35
+    Stream: column 1 section DATA start: 64080 length 20029
+    Stream: column 2 section DATA start: 84109 length 40035
+    Stream: column 3 section DATA start: 124144 length 3533
+    Stream: column 3 section LENGTH start: 127677 length 25
+    Stream: column 3 section DICTIONARY_DATA start: 127702 length 133
     Encoding column 0: DIRECT
-    Encoding column 1: DIRECT
-    Encoding column 2: DIRECT
-    Encoding column 3: DICTIONARY[35]
-  Stripe: offset: 139574 data: 69570 rows: 5000 tail: 73 index: 120
-    Stream: column 0 section ROW_INDEX start: 139574 length 10
-    Stream: column 1 section ROW_INDEX start: 139584 length 36
-    Stream: column 2 section ROW_INDEX start: 139620 length 39
-    Stream: column 3 section ROW_INDEX start: 139659 length 35
-    Stream: column 1 section DATA start: 139694 length 22594
-    Stream: column 2 section DATA start: 162288 length 43415
-    Stream: column 3 section DATA start: 205703 length 3390
-    Stream: column 3 section LENGTH start: 209093 length 38
-    Stream: column 3 section DICTIONARY_DATA start: 209131 length 133
+    Encoding column 1: DIRECT_V2
+    Encoding column 2: DIRECT_V2
+    Encoding column 3: DICTIONARY_V2
+  Stripe: offset: 127911 data: 63766 rows: 5000 tail: 76 index: 120
+    Stream: column 0 section ROW_INDEX start: 127911 length 10
+    Stream: column 1 section ROW_INDEX start: 127921 length 36
+    Stream: column 2 section ROW_INDEX start: 127957 length 39
+    Stream: column 3 section ROW_INDEX start: 127996 length 35
+    Stream: column 1 section DATA start: 128031 length 20029
+    Stream: column 2 section DATA start: 148060 length 40035
+    Stream: column 3 section DATA start: 188095 length 3544
+    Stream: column 3 section LENGTH start: 191639 length 25
+    Stream: column 3 section DICTIONARY_DATA start: 191664 length 133
     Encoding column 0: DIRECT
-    Encoding column 1: DIRECT
-    Encoding column 2: DIRECT
-    Encoding column 3: DICTIONARY[35]
-  Stripe: offset: 209337 data: 69551 rows: 5000 tail: 72 index: 119
-    Stream: column 0 section ROW_INDEX start: 209337 length 10
-    Stream: column 1 section ROW_INDEX start: 209347 length 35
-    Stream: column 2 section ROW_INDEX start: 209382 length 39
-    Stream: column 3 section ROW_INDEX start: 209421 length 35
-    Stream: column 1 section DATA start: 209456 length 22575
-    Stream: column 2 section DATA start: 232031 length 43426
-    Stream: column 3 section DATA start: 275457 length 3379
-    Stream: column 3 section LENGTH start: 278836 length 38
-    Stream: column 3 section DICTIONARY_DATA start: 278874 length 133
+    Encoding column 1: DIRECT_V2
+    Encoding column 2: DIRECT_V2
+    Encoding column 3: DICTIONARY_V2
+  Stripe: offset: 191873 data: 63796 rows: 5000 tail: 74 index: 119
+    Stream: column 0 section ROW_INDEX start: 191873 length 10
+    Stream: column 1 section ROW_INDEX start: 191883 length 35
+    Stream: column 2 section ROW_INDEX start: 191918 length 39
+    Stream: column 3 section ROW_INDEX start: 191957 length 35
+    Stream: column 1 section DATA start: 191992 length 20029
+    Stream: column 2 section DATA start: 212021 length 40035
+    Stream: column 3 section DATA start: 252056 length 3574
+    Stream: column 3 section LENGTH start: 255630 length 25
+    Stream: column 3 section DICTIONARY_DATA start: 255655 length 133
     Encoding column 0: DIRECT
-    Encoding column 1: DIRECT
-    Encoding column 2: DIRECT
-    Encoding column 3: DICTIONARY[35]
-  Stripe: offset: 279079 data: 14096 rows: 1000 tail: 68 index: 120
-    Stream: column 0 section ROW_INDEX start: 279079 length 10
-    Stream: column 1 section ROW_INDEX start: 279089 length 36
-    Stream: column 2 section ROW_INDEX start: 279125 length 39
-    Stream: column 3 section ROW_INDEX start: 279164 length 35
-    Stream: column 1 section DATA start: 279199 length 4529
-    Stream: column 2 section DATA start: 283728 length 8690
-    Stream: column 3 section DATA start: 292418 length 706
-    Stream: column 3 section LENGTH start: 293124 length 38
-    Stream: column 3 section DICTIONARY_DATA start: 293162 length 133
+    Encoding column 1: DIRECT_V2
+    Encoding column 2: DIRECT_V2
+    Encoding column 3: DICTIONARY_V2
+  Stripe: offset: 255862 data: 12940 rows: 1000 tail: 71 index: 120
+    Stream: column 0 section ROW_INDEX start: 255862 length 10
+    Stream: column 1 section ROW_INDEX start: 255872 length 36
+    Stream: column 2 section ROW_INDEX start: 255908 length 39
+    Stream: column 3 section ROW_INDEX start: 255947 length 35
+    Stream: column 1 section DATA start: 255982 length 4007
+    Stream: column 2 section DATA start: 259989 length 8007
+    Stream: column 3 section DATA start: 267996 length 768
+    Stream: column 3 section LENGTH start: 268764 length 25
+    Stream: column 3 section DICTIONARY_DATA start: 268789 length 133
     Encoding column 0: DIRECT
-    Encoding column 1: DIRECT
-    Encoding column 2: DIRECT
-    Encoding column 3: DICTIONARY[35]
+    Encoding column 1: DIRECT_V2
+    Encoding column 2: DIRECT_V2
+    Encoding column 3: DICTIONARY_V2
\ No newline at end of file