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 2016/05/20 21:22:54 UTC

[16/27] hive git commit: HIVE-11417. Move the ReaderImpl and RowReaderImpl to the ORC module, by making shims for the row by row reader. (omalley reviewed by prasanth_j)

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/orc/src/test/org/apache/orc/tools/TestFileDump.java
----------------------------------------------------------------------
diff --git a/orc/src/test/org/apache/orc/tools/TestFileDump.java b/orc/src/test/org/apache/orc/tools/TestFileDump.java
new file mode 100644
index 0000000..ce3381e
--- /dev/null
+++ b/orc/src/test/org/apache/orc/tools/TestFileDump.java
@@ -0,0 +1,486 @@
+/**
+ * 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.tools;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
+import java.io.BufferedReader;
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.FileReader;
+import java.io.PrintStream;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+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.common.type.HiveDecimal;
+import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector;
+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.MapColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.StructColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.serde2.io.DateWritable;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.orc.CompressionKind;
+import org.apache.orc.OrcConf;
+import org.apache.orc.OrcFile;
+import org.apache.orc.TypeDescription;
+import org.apache.orc.Writer;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestFileDump {
+
+  Path workDir = new Path(System.getProperty("test.tmp.dir"));
+  Configuration conf;
+  FileSystem fs;
+  Path testFilePath;
+
+  @Before
+  public void openFileSystem () throws Exception {
+    conf = new Configuration();
+    fs = FileSystem.getLocal(conf);
+    fs.setWorkingDirectory(workDir);
+    testFilePath = new Path("TestFileDump.testDump.orc");
+    fs.delete(testFilePath, false);
+  }
+
+  static TypeDescription getMyRecordType() {
+    return TypeDescription.createStruct()
+        .addField("i", TypeDescription.createInt())
+        .addField("l", TypeDescription.createLong())
+        .addField("s", TypeDescription.createString());
+  }
+
+  static void appendMyRecord(VectorizedRowBatch batch,
+                             int i,
+                             long l,
+                             String str) {
+    ((LongColumnVector) batch.cols[0]).vector[batch.size] = i;
+    ((LongColumnVector) batch.cols[1]).vector[batch.size] = l;
+    if (str == null) {
+      batch.cols[2].noNulls = false;
+      batch.cols[2].isNull[batch.size] = true;
+    } else {
+      ((BytesColumnVector) batch.cols[2]).setVal(batch.size,
+          str.getBytes());
+    }
+    batch.size += 1;
+  }
+
+  static TypeDescription getAllTypesType() {
+    return TypeDescription.createStruct()
+        .addField("b", TypeDescription.createBoolean())
+        .addField("bt", TypeDescription.createByte())
+        .addField("s", TypeDescription.createShort())
+        .addField("i", TypeDescription.createInt())
+        .addField("l", TypeDescription.createLong())
+        .addField("f", TypeDescription.createFloat())
+        .addField("d", TypeDescription.createDouble())
+        .addField("de", TypeDescription.createDecimal())
+        .addField("t", TypeDescription.createTimestamp())
+        .addField("dt", TypeDescription.createDate())
+        .addField("str", TypeDescription.createString())
+        .addField("c", TypeDescription.createChar().withMaxLength(5))
+        .addField("vc", TypeDescription.createVarchar().withMaxLength(10))
+        .addField("m", TypeDescription.createMap(
+            TypeDescription.createString(),
+            TypeDescription.createString()))
+        .addField("a", TypeDescription.createList(TypeDescription.createInt()))
+        .addField("st", TypeDescription.createStruct()
+                .addField("i", TypeDescription.createInt())
+                .addField("s", TypeDescription.createString()));
+  }
+
+  static void appendAllTypes(VectorizedRowBatch batch,
+                             boolean b,
+                             byte bt,
+                             short s,
+                             int i,
+                             long l,
+                             float f,
+                             double d,
+                             HiveDecimalWritable de,
+                             Timestamp t,
+                             DateWritable dt,
+                             String str,
+                             String c,
+                             String vc,
+                             Map<String, String> m,
+                             List<Integer> a,
+                             int sti,
+                             String sts) {
+    int row = batch.size++;
+    ((LongColumnVector) batch.cols[0]).vector[row] = b ? 1 : 0;
+    ((LongColumnVector) batch.cols[1]).vector[row] = bt;
+    ((LongColumnVector) batch.cols[2]).vector[row] = s;
+    ((LongColumnVector) batch.cols[3]).vector[row] = i;
+    ((LongColumnVector) batch.cols[4]).vector[row] = l;
+    ((DoubleColumnVector) batch.cols[5]).vector[row] = f;
+    ((DoubleColumnVector) batch.cols[6]).vector[row] = d;
+    ((DecimalColumnVector) batch.cols[7]).vector[row].set(de);
+    ((TimestampColumnVector) batch.cols[8]).set(row, t);
+    ((LongColumnVector) batch.cols[9]).vector[row] = dt.getDays();
+    ((BytesColumnVector) batch.cols[10]).setVal(row, str.getBytes());
+    ((BytesColumnVector) batch.cols[11]).setVal(row, c.getBytes());
+    ((BytesColumnVector) batch.cols[12]).setVal(row, vc.getBytes());
+    MapColumnVector map = (MapColumnVector) batch.cols[13];
+    int offset = map.childCount;
+    map.offsets[row] = offset;
+    map.lengths[row] = m.size();
+    map.childCount += map.lengths[row];
+    for(Map.Entry<String, String> entry: m.entrySet()) {
+      ((BytesColumnVector) map.keys).setVal(offset, entry.getKey().getBytes());
+      ((BytesColumnVector) map.values).setVal(offset++,
+          entry.getValue().getBytes());
+    }
+    ListColumnVector list = (ListColumnVector) batch.cols[14];
+    offset = list.childCount;
+    list.offsets[row] = offset;
+    list.lengths[row] = a.size();
+    list.childCount += list.lengths[row];
+    for(int e=0; e < a.size(); ++e) {
+      ((LongColumnVector) list.child).vector[offset + e] = a.get(e);
+    }
+    StructColumnVector struct = (StructColumnVector) batch.cols[15];
+    ((LongColumnVector) struct.fields[0]).vector[row] = sti;
+    ((BytesColumnVector) struct.fields[1]).setVal(row, sts.getBytes());
+  }
+
+  public static void checkOutput(String expected,
+                                 String actual) throws Exception {
+    BufferedReader eStream =
+        new BufferedReader(new FileReader
+            (TestJsonFileDump.getFileFromClasspath(expected)));
+    BufferedReader aStream =
+        new BufferedReader(new FileReader(actual));
+    String expectedLine = eStream.readLine().trim();
+    while (expectedLine != null) {
+      String actualLine = aStream.readLine().trim();
+      System.out.println("actual:   " + actualLine);
+      System.out.println("expected: " + expectedLine);
+      Assert.assertEquals(expectedLine, actualLine);
+      expectedLine = eStream.readLine();
+      expectedLine = expectedLine == null ? null : expectedLine.trim();
+    }
+    Assert.assertNull(eStream.readLine());
+    Assert.assertNull(aStream.readLine());
+    eStream.close();
+    aStream.close();
+  }
+
+  @Test
+  public void testDump() throws Exception {
+    TypeDescription schema = getMyRecordType();
+    conf.set(OrcConf.ENCODING_STRATEGY.getAttribute(), "COMPRESSION");
+    Writer writer = OrcFile.createWriter(testFilePath,
+        OrcFile.writerOptions(conf)
+            .fileSystem(fs)
+            .setSchema(schema)
+            .compress(CompressionKind.ZLIB)
+            .stripeSize(100000)
+            .rowIndexStride(1000));
+    Random r1 = new Random(1);
+    String[] words = new String[]{"It", "was", "the", "best", "of", "times,",
+        "it", "was", "the", "worst", "of", "times,", "it", "was", "the", "age",
+        "of", "wisdom,", "it", "was", "the", "age", "of", "foolishness,", "it",
+        "was", "the", "epoch", "of", "belief,", "it", "was", "the", "epoch",
+        "of", "incredulity,", "it", "was", "the", "season", "of", "Light,",
+        "it", "was", "the", "season", "of", "Darkness,", "it", "was", "the",
+        "spring", "of", "hope,", "it", "was", "the", "winter", "of", "despair,",
+        "we", "had", "everything", "before", "us,", "we", "had", "nothing",
+        "before", "us,", "we", "were", "all", "going", "direct", "to",
+        "Heaven,", "we", "were", "all", "going", "direct", "the", "other",
+        "way"};
+    VectorizedRowBatch batch = schema.createRowBatch(1000);
+    for(int i=0; i < 21000; ++i) {
+      appendMyRecord(batch, r1.nextInt(), r1.nextLong(),
+          words[r1.nextInt(words.length)]);
+      if (batch.size == batch.getMaxSize()) {
+        writer.addRowBatch(batch);
+        batch.reset();
+      }
+    }
+    if (batch.size > 0) {
+      writer.addRowBatch(batch);
+    }
+    writer.close();
+    PrintStream origOut = System.out;
+    String outputFilename = "orc-file-dump.out";
+    FileOutputStream myOut = new FileOutputStream(workDir + File.separator + outputFilename);
+
+    // replace stdout and run command
+    System.setOut(new PrintStream(myOut));
+    FileDump.main(new String[]{testFilePath.toString(), "--rowindex=1,2,3"});
+    System.out.flush();
+    System.setOut(origOut);
+
+
+    checkOutput(outputFilename, workDir + File.separator + outputFilename);
+  }
+
+  @Test
+  public void testDataDump() throws Exception {
+    TypeDescription schema = getAllTypesType();
+    Writer writer = OrcFile.createWriter(testFilePath,
+        OrcFile.writerOptions(conf)
+            .fileSystem(fs)
+            .setSchema(schema)
+            .stripeSize(100000)
+            .compress(CompressionKind.NONE)
+            .bufferSize(10000)
+            .rowIndexStride(1000));
+    VectorizedRowBatch batch = schema.createRowBatch(1000);
+    Map<String, String> m = new HashMap<String, String>(2);
+    m.put("k1", "v1");
+    appendAllTypes(batch,
+        true,
+        (byte) 10,
+        (short) 100,
+        1000,
+        10000L,
+        4.0f,
+        20.0,
+        new HiveDecimalWritable("4.2222"),
+        new Timestamp(1416967764000L),
+        new DateWritable(new Date(1416967764000L)),
+        "string",
+        "hello",
+       "hello",
+        m,
+        Arrays.asList(100, 200),
+        10, "foo");
+    m.clear();
+    m.put("k3", "v3");
+    appendAllTypes(
+        batch,
+        false,
+        (byte)20,
+        (short)200,
+        2000,
+        20000L,
+        8.0f,
+        40.0,
+        new HiveDecimalWritable("2.2222"),
+        new Timestamp(1416967364000L),
+        new DateWritable(new Date(1411967764000L)),
+        "abcd",
+        "world",
+        "world",
+        m,
+        Arrays.asList(200, 300),
+        20, "bar");
+    writer.addRowBatch(batch);
+
+    writer.close();
+    PrintStream origOut = System.out;
+    ByteArrayOutputStream myOut = new ByteArrayOutputStream();
+
+    // replace stdout and run command
+    System.setOut(new PrintStream(myOut));
+    FileDump.main(new String[]{testFilePath.toString(), "-d"});
+    System.out.flush();
+    System.setOut(origOut);
+    String[] lines = myOut.toString().split("\n");
+    Assert.assertEquals("{\"b\":true,\"bt\":10,\"s\":100,\"i\":1000,\"l\":10000,\"f\":4,\"d\":20,\"de\":\"4.2222\",\"t\":\"2014-11-25 18:09:24.0\",\"dt\":\"2014-11-25\",\"str\":\"string\",\"c\":\"hello\",\"vc\":\"hello\",\"m\":[{\"_key\":\"k1\",\"_value\":\"v1\"}],\"a\":[100,200],\"st\":{\"i\":10,\"s\":\"foo\"}}", lines[0]);
+    Assert.assertEquals("{\"b\":false,\"bt\":20,\"s\":200,\"i\":2000,\"l\":20000,\"f\":8,\"d\":40,\"de\":\"2.2222\",\"t\":\"2014-11-25 18:02:44.0\",\"dt\":\"2014-09-28\",\"str\":\"abcd\",\"c\":\"world\",\"vc\":\"world\",\"m\":[{\"_key\":\"k3\",\"_value\":\"v3\"}],\"a\":[200,300],\"st\":{\"i\":20,\"s\":\"bar\"}}", lines[1]);
+  }
+  
+  // Test that if the fraction of rows that have distinct strings is greater than the configured
+  // threshold dictionary encoding is turned off.  If dictionary encoding is turned off the length
+  // of the dictionary stream for the column will be 0 in the ORC file dump.
+  @Test
+  public void testDictionaryThreshold() throws Exception {
+    TypeDescription schema = getMyRecordType();
+    Configuration conf = new Configuration();
+    conf.set(OrcConf.ENCODING_STRATEGY.getAttribute(), "COMPRESSION");
+    conf.setFloat(OrcConf.DICTIONARY_KEY_SIZE_THRESHOLD.getAttribute(), 0.49f);
+    Writer writer = OrcFile.createWriter(testFilePath,
+        OrcFile.writerOptions(conf)
+            .fileSystem(fs)
+            .setSchema(schema)
+            .stripeSize(100000)
+            .compress(CompressionKind.ZLIB)
+            .rowIndexStride(1000)
+            .bufferSize(10000));
+    VectorizedRowBatch batch = schema.createRowBatch(1000);
+    Random r1 = new Random(1);
+    String[] words = new String[]{"It", "was", "the", "best", "of", "times,",
+        "it", "was", "the", "worst", "of", "times,", "it", "was", "the", "age",
+        "of", "wisdom,", "it", "was", "the", "age", "of", "foolishness,", "it",
+        "was", "the", "epoch", "of", "belief,", "it", "was", "the", "epoch",
+        "of", "incredulity,", "it", "was", "the", "season", "of", "Light,",
+        "it", "was", "the", "season", "of", "Darkness,", "it", "was", "the",
+        "spring", "of", "hope,", "it", "was", "the", "winter", "of", "despair,",
+        "we", "had", "everything", "before", "us,", "we", "had", "nothing",
+        "before", "us,", "we", "were", "all", "going", "direct", "to",
+        "Heaven,", "we", "were", "all", "going", "direct", "the", "other",
+        "way"};
+    int nextInt = 0;
+    for(int i=0; i < 21000; ++i) {
+      // Write out the same string twice, this guarantees the fraction of rows with
+      // distinct strings is 0.5
+      if (i % 2 == 0) {
+        nextInt = r1.nextInt(words.length);
+        // Append the value of i to the word, this guarantees when an index or word is repeated
+        // the actual string is unique.
+        words[nextInt] += "-" + i;
+      }
+      appendMyRecord(batch, r1.nextInt(), r1.nextLong(), words[nextInt]);
+      if (batch.size == batch.getMaxSize()) {
+        writer.addRowBatch(batch);
+        batch.reset();
+      }
+    }
+    if (batch.size != 0) {
+      writer.addRowBatch(batch);
+    }
+    writer.close();
+    PrintStream origOut = System.out;
+    String outputFilename = "orc-file-dump-dictionary-threshold.out";
+    FileOutputStream myOut = new FileOutputStream(workDir + File.separator + outputFilename);
+
+    // replace stdout and run command
+    System.setOut(new PrintStream(myOut));
+    FileDump.main(new String[]{testFilePath.toString(), "--rowindex=1,2,3"});
+    System.out.flush();
+    System.setOut(origOut);
+
+    checkOutput(outputFilename, workDir + File.separator + outputFilename);
+  }
+
+  @Test
+  public void testBloomFilter() throws Exception {
+    TypeDescription schema = getMyRecordType();
+    conf.set(OrcConf.ENCODING_STRATEGY.getAttribute(), "COMPRESSION");
+    OrcFile.WriterOptions options = OrcFile.writerOptions(conf)
+        .fileSystem(fs)
+        .setSchema(schema)
+        .stripeSize(100000)
+        .compress(CompressionKind.ZLIB)
+        .bufferSize(10000)
+        .rowIndexStride(1000)
+        .bloomFilterColumns("S");
+    Writer writer = OrcFile.createWriter(testFilePath, options);
+    Random r1 = new Random(1);
+    String[] words = new String[]{"It", "was", "the", "best", "of", "times,",
+        "it", "was", "the", "worst", "of", "times,", "it", "was", "the", "age",
+        "of", "wisdom,", "it", "was", "the", "age", "of", "foolishness,", "it",
+        "was", "the", "epoch", "of", "belief,", "it", "was", "the", "epoch",
+        "of", "incredulity,", "it", "was", "the", "season", "of", "Light,",
+        "it", "was", "the", "season", "of", "Darkness,", "it", "was", "the",
+        "spring", "of", "hope,", "it", "was", "the", "winter", "of", "despair,",
+        "we", "had", "everything", "before", "us,", "we", "had", "nothing",
+        "before", "us,", "we", "were", "all", "going", "direct", "to",
+        "Heaven,", "we", "were", "all", "going", "direct", "the", "other",
+        "way"};
+    VectorizedRowBatch batch = schema.createRowBatch(1000);
+    for(int i=0; i < 21000; ++i) {
+      appendMyRecord(batch, r1.nextInt(), r1.nextLong(),
+          words[r1.nextInt(words.length)]);
+      if (batch.size == batch.getMaxSize()) {
+        writer.addRowBatch(batch);
+        batch.reset();
+      }
+    }
+    if (batch.size > 0) {
+      writer.addRowBatch(batch);
+    }
+    writer.close();
+    PrintStream origOut = System.out;
+    String outputFilename = "orc-file-dump-bloomfilter.out";
+    FileOutputStream myOut = new FileOutputStream(workDir + File.separator + outputFilename);
+
+    // replace stdout and run command
+    System.setOut(new PrintStream(myOut));
+    FileDump.main(new String[]{testFilePath.toString(), "--rowindex=3"});
+    System.out.flush();
+    System.setOut(origOut);
+
+
+    checkOutput(outputFilename, workDir + File.separator + outputFilename);
+  }
+
+  @Test
+  public void testBloomFilter2() throws Exception {
+    TypeDescription schema = getMyRecordType();
+    conf.set(OrcConf.ENCODING_STRATEGY.getAttribute(), "COMPRESSION");
+    OrcFile.WriterOptions options = OrcFile.writerOptions(conf)
+        .fileSystem(fs)
+        .setSchema(schema)
+        .stripeSize(100000)
+        .compress(CompressionKind.ZLIB)
+        .bufferSize(10000)
+        .rowIndexStride(1000)
+        .bloomFilterColumns("l")
+        .bloomFilterFpp(0.01);
+    VectorizedRowBatch batch = schema.createRowBatch(1000);
+    Writer writer = OrcFile.createWriter(testFilePath, options);
+    Random r1 = new Random(1);
+    String[] words = new String[]{"It", "was", "the", "best", "of", "times,",
+        "it", "was", "the", "worst", "of", "times,", "it", "was", "the", "age",
+        "of", "wisdom,", "it", "was", "the", "age", "of", "foolishness,", "it",
+        "was", "the", "epoch", "of", "belief,", "it", "was", "the", "epoch",
+        "of", "incredulity,", "it", "was", "the", "season", "of", "Light,",
+        "it", "was", "the", "season", "of", "Darkness,", "it", "was", "the",
+        "spring", "of", "hope,", "it", "was", "the", "winter", "of", "despair,",
+        "we", "had", "everything", "before", "us,", "we", "had", "nothing",
+        "before", "us,", "we", "were", "all", "going", "direct", "to",
+        "Heaven,", "we", "were", "all", "going", "direct", "the", "other",
+        "way"};
+    for(int i=0; i < 21000; ++i) {
+      appendMyRecord(batch, r1.nextInt(), r1.nextLong(),
+          words[r1.nextInt(words.length)]);
+      if (batch.size == batch.getMaxSize()) {
+        writer.addRowBatch(batch);
+        batch.reset();
+      }
+    }
+    if (batch.size > 0) {
+      writer.addRowBatch(batch);
+    }
+    writer.close();
+    PrintStream origOut = System.out;
+    String outputFilename = "orc-file-dump-bloomfilter2.out";
+    FileOutputStream myOut = new FileOutputStream(workDir + File.separator + outputFilename);
+
+    // replace stdout and run command
+    System.setOut(new PrintStream(myOut));
+    FileDump.main(new String[]{testFilePath.toString(), "--rowindex=2"});
+    System.out.flush();
+    System.setOut(origOut);
+
+
+    checkOutput(outputFilename, workDir + File.separator + outputFilename);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/orc/src/test/org/apache/orc/tools/TestJsonFileDump.java
----------------------------------------------------------------------
diff --git a/orc/src/test/org/apache/orc/tools/TestJsonFileDump.java b/orc/src/test/org/apache/orc/tools/TestJsonFileDump.java
new file mode 100644
index 0000000..a514824
--- /dev/null
+++ b/orc/src/test/org/apache/orc/tools/TestJsonFileDump.java
@@ -0,0 +1,150 @@
+/**
+ * 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.tools;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.FileReader;
+import java.io.PrintStream;
+import java.net.URL;
+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.ql.exec.vector.BytesColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.orc.CompressionKind;
+import org.apache.orc.OrcConf;
+import org.apache.orc.OrcFile;
+import org.apache.orc.TypeDescription;
+import org.apache.orc.Writer;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestJsonFileDump {
+  public static String getFileFromClasspath(String name) {
+    URL url = ClassLoader.getSystemResource(name);
+    if (url == null) {
+      throw new IllegalArgumentException("Could not find " + name);
+    }
+    return url.getPath();
+  }
+
+  Path workDir = new Path(System.getProperty("test.tmp.dir"));
+  Configuration conf;
+  FileSystem fs;
+  Path testFilePath;
+
+  @Before
+  public void openFileSystem () throws Exception {
+    conf = new Configuration();
+    fs = FileSystem.getLocal(conf);
+    fs.setWorkingDirectory(workDir);
+    testFilePath = new Path("TestFileDump.testDump.orc");
+    fs.delete(testFilePath, false);
+  }
+
+  static void checkOutput(String expected,
+                                  String actual) throws Exception {
+    BufferedReader eStream =
+        new BufferedReader(new FileReader(getFileFromClasspath(expected)));
+    BufferedReader aStream =
+        new BufferedReader(new FileReader(actual));
+    String expectedLine = eStream.readLine();
+    while (expectedLine != null) {
+      String actualLine = aStream.readLine();
+      System.out.println("actual:   " + actualLine);
+      System.out.println("expected: " + expectedLine);
+      assertEquals(expectedLine, actualLine);
+      expectedLine = eStream.readLine();
+    }
+    assertNull(eStream.readLine());
+    assertNull(aStream.readLine());
+  }
+
+  @Test
+  public void testJsonDump() throws Exception {
+    TypeDescription schema = TypeDescription.createStruct()
+        .addField("i", TypeDescription.createInt())
+        .addField("l", TypeDescription.createLong())
+        .addField("s", TypeDescription.createString());
+    conf.set(OrcConf.ENCODING_STRATEGY.getAttribute(), "COMPRESSION");
+    OrcFile.WriterOptions options = OrcFile.writerOptions(conf)
+        .fileSystem(fs)
+        .setSchema(schema)
+        .stripeSize(100000)
+        .compress(CompressionKind.ZLIB)
+        .bufferSize(10000)
+        .rowIndexStride(1000)
+        .bloomFilterColumns("s");
+    Writer writer = OrcFile.createWriter(testFilePath, options);
+    Random r1 = new Random(1);
+    String[] words = new String[]{"It", "was", "the", "best", "of", "times,",
+        "it", "was", "the", "worst", "of", "times,", "it", "was", "the", "age",
+        "of", "wisdom,", "it", "was", "the", "age", "of", "foolishness,", "it",
+        "was", "the", "epoch", "of", "belief,", "it", "was", "the", "epoch",
+        "of", "incredulity,", "it", "was", "the", "season", "of", "Light,",
+        "it", "was", "the", "season", "of", "Darkness,", "it", "was", "the",
+        "spring", "of", "hope,", "it", "was", "the", "winter", "of", "despair,",
+        "we", "had", "everything", "before", "us,", "we", "had", "nothing",
+        "before", "us,", "we", "were", "all", "going", "direct", "to",
+        "Heaven,", "we", "were", "all", "going", "direct", "the", "other",
+        "way"};
+    VectorizedRowBatch batch = schema.createRowBatch(1000);
+    for(int i=0; i < 21000; ++i) {
+      ((LongColumnVector) batch.cols[0]).vector[batch.size] = r1.nextInt();
+      ((LongColumnVector) batch.cols[1]).vector[batch.size] = r1.nextLong();
+      if (i % 100 == 0) {
+        batch.cols[2].noNulls = false;
+        batch.cols[2].isNull[batch.size] = true;
+      } else {
+        ((BytesColumnVector) batch.cols[2]).setVal(batch.size,
+            words[r1.nextInt(words.length)].getBytes());
+      }
+      batch.size += 1;
+      if (batch.size == batch.getMaxSize()) {
+        writer.addRowBatch(batch);
+        batch.reset();
+      }
+    }
+    if (batch.size > 0) {
+      writer.addRowBatch(batch);
+    }
+
+    writer.close();
+    PrintStream origOut = System.out;
+    String outputFilename = "orc-file-dump.json";
+    FileOutputStream myOut = new FileOutputStream(workDir + File.separator + outputFilename);
+
+    // replace stdout and run command
+    System.setOut(new PrintStream(myOut));
+    FileDump.main(new String[]{testFilePath.toString(), "-j", "-p", "--rowindex=3"});
+    System.out.flush();
+    System.setOut(origOut);
+
+
+    checkOutput(outputFilename, workDir + File.separator + outputFilename);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/orc/src/test/resources/orc-file-11-format.orc
----------------------------------------------------------------------
diff --git a/orc/src/test/resources/orc-file-11-format.orc b/orc/src/test/resources/orc-file-11-format.orc
new file mode 100644
index 0000000..41653c8
Binary files /dev/null and b/orc/src/test/resources/orc-file-11-format.orc differ

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/orc/src/test/resources/orc-file-dump-bloomfilter.out
----------------------------------------------------------------------
diff --git a/orc/src/test/resources/orc-file-dump-bloomfilter.out b/orc/src/test/resources/orc-file-dump-bloomfilter.out
new file mode 100644
index 0000000..18fd2fb
--- /dev/null
+++ b/orc/src/test/resources/orc-file-dump-bloomfilter.out
@@ -0,0 +1,179 @@
+Structure for TestFileDump.testDump.orc
+File Version: 0.12 with HIVE_13083
+Rows: 21000
+Compression: ZLIB
+Compression size: 4096
+Type: struct<i:int,l:bigint,s:string>
+
+Stripe Statistics:
+  Stripe 1:
+    Column 0: count: 5000 hasNull: false
+    Column 1: count: 5000 hasNull: false min: -2146021688 max: 2147223299 sum: 515792826
+    Column 2: count: 5000 hasNull: false min: -9218592812243954469 max: 9221614132680747961
+    Column 3: count: 5000 hasNull: false min: Darkness, max: worst sum: 19280
+  Stripe 2:
+    Column 0: count: 5000 hasNull: false
+    Column 1: count: 5000 hasNull: false min: -2146733128 max: 2147001622 sum: 7673427
+    Column 2: count: 5000 hasNull: false min: -9220818777591257749 max: 9222259462014003839
+    Column 3: count: 5000 hasNull: false min: Darkness, max: worst sum: 19504
+  Stripe 3:
+    Column 0: count: 5000 hasNull: false
+    Column 1: count: 5000 hasNull: false min: -2146993718 max: 2147378179 sum: 132660742551
+    Column 2: count: 5000 hasNull: false min: -9218342074710552826 max: 9222303228623055266
+    Column 3: count: 5000 hasNull: false min: Darkness, max: worst sum: 19641
+  Stripe 4:
+    Column 0: count: 5000 hasNull: false
+    Column 1: count: 5000 hasNull: false min: -2146658006 max: 2145520931 sum: 8533549236
+    Column 2: count: 5000 hasNull: false min: -9222758097219661129 max: 9221043130193737406
+    Column 3: count: 5000 hasNull: false min: Darkness, max: worst sum: 19470
+  Stripe 5:
+    Column 0: count: 1000 hasNull: false
+    Column 1: count: 1000 hasNull: false min: -2146245500 max: 2146378640 sum: 51299706363
+    Column 2: count: 1000 hasNull: false min: -9208193203370316142 max: 9218567213558056476
+    Column 3: count: 1000 hasNull: false min: Darkness, max: worst sum: 3866
+
+File Statistics:
+  Column 0: count: 21000 hasNull: false
+  Column 1: count: 21000 hasNull: false min: -2146993718 max: 2147378179 sum: 193017464403
+  Column 2: count: 21000 hasNull: false min: -9222758097219661129 max: 9222303228623055266
+  Column 3: count: 21000 hasNull: false min: Darkness, max: worst sum: 81761
+
+Stripes:
+  Stripe: offset: 3 data: 63786 rows: 5000 tail: 86 index: 951
+    Stream: column 0 section ROW_INDEX start: 3 length 17
+    Stream: column 1 section ROW_INDEX start: 20 length 166
+    Stream: column 2 section ROW_INDEX start: 186 length 169
+    Stream: column 3 section ROW_INDEX start: 355 length 87
+    Stream: column 3 section BLOOM_FILTER start: 442 length 512
+    Stream: column 1 section DATA start: 954 length 20035
+    Stream: column 2 section DATA start: 20989 length 40050
+    Stream: column 3 section DATA start: 61039 length 3543
+    Stream: column 3 section LENGTH start: 64582 length 25
+    Stream: column 3 section DICTIONARY_DATA start: 64607 length 133
+    Encoding column 0: DIRECT
+    Encoding column 1: DIRECT_V2
+    Encoding column 2: DIRECT_V2
+    Encoding column 3: DICTIONARY_V2[35]
+    Row group indices for column 3:
+      Entry 0: count: 1000 hasNull: false min: Darkness, max: worst sum: 3862 positions: 0,0,0
+      Entry 1: count: 1000 hasNull: false min: Darkness, max: worst sum: 3884 positions: 0,659,149
+      Entry 2: count: 1000 hasNull: false min: Darkness, max: worst sum: 3893 positions: 0,1531,3
+      Entry 3: count: 1000 hasNull: false min: Darkness, max: worst sum: 3798 positions: 0,2281,32
+      Entry 4: count: 1000 hasNull: false min: Darkness, max: worst sum: 3843 positions: 0,3033,45
+    Bloom filters for column 3:
+      Entry 0: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
+      Entry 1: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
+      Entry 2: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
+      Entry 3: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
+      Entry 4: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
+      Stripe level merge: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
+  Stripe: offset: 64826 data: 63775 rows: 5000 tail: 86 index: 944
+    Stream: column 0 section ROW_INDEX start: 64826 length 17
+    Stream: column 1 section ROW_INDEX start: 64843 length 164
+    Stream: column 2 section ROW_INDEX start: 65007 length 168
+    Stream: column 3 section ROW_INDEX start: 65175 length 83
+    Stream: column 3 section BLOOM_FILTER start: 65258 length 512
+    Stream: column 1 section DATA start: 65770 length 20035
+    Stream: column 2 section DATA start: 85805 length 40050
+    Stream: column 3 section DATA start: 125855 length 3532
+    Stream: column 3 section LENGTH start: 129387 length 25
+    Stream: column 3 section DICTIONARY_DATA start: 129412 length 133
+    Encoding column 0: DIRECT
+    Encoding column 1: DIRECT_V2
+    Encoding column 2: DIRECT_V2
+    Encoding column 3: DICTIONARY_V2[35]
+    Row group indices for column 3:
+      Entry 0: count: 1000 hasNull: false min: Darkness, max: worst sum: 3923 positions: 0,0,0
+      Entry 1: count: 1000 hasNull: false min: Darkness, max: worst sum: 3869 positions: 0,761,12
+      Entry 2: count: 1000 hasNull: false min: Darkness, max: worst sum: 3817 positions: 0,1472,70
+      Entry 3: count: 1000 hasNull: false min: Darkness, max: worst sum: 3931 positions: 0,2250,43
+      Entry 4: count: 1000 hasNull: false min: Darkness, max: worst sum: 3964 positions: 0,2978,88
+    Bloom filters for column 3:
+      Entry 0: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
+      Entry 1: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
+      Entry 2: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
+      Entry 3: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
+      Entry 4: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
+      Stripe level merge: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
+  Stripe: offset: 129631 data: 63787 rows: 5000 tail: 86 index: 950
+    Stream: column 0 section ROW_INDEX start: 129631 length 17
+    Stream: column 1 section ROW_INDEX start: 129648 length 163
+    Stream: column 2 section ROW_INDEX start: 129811 length 168
+    Stream: column 3 section ROW_INDEX start: 129979 length 90
+    Stream: column 3 section BLOOM_FILTER start: 130069 length 512
+    Stream: column 1 section DATA start: 130581 length 20035
+    Stream: column 2 section DATA start: 150616 length 40050
+    Stream: column 3 section DATA start: 190666 length 3544
+    Stream: column 3 section LENGTH start: 194210 length 25
+    Stream: column 3 section DICTIONARY_DATA start: 194235 length 133
+    Encoding column 0: DIRECT
+    Encoding column 1: DIRECT_V2
+    Encoding column 2: DIRECT_V2
+    Encoding column 3: DICTIONARY_V2[35]
+    Row group indices for column 3:
+      Entry 0: count: 1000 hasNull: false min: Darkness, max: worst sum: 3817 positions: 0,0,0
+      Entry 1: count: 1000 hasNull: false min: Darkness, max: worst sum: 4008 positions: 0,634,174
+      Entry 2: count: 1000 hasNull: false min: Darkness, max: worst sum: 3999 positions: 0,1469,69
+      Entry 3: count: 1000 hasNull: false min: Darkness, max: worst sum: 3817 positions: 0,2133,194
+      Entry 4: count: 1000 hasNull: false min: Darkness, max: worst sum: 4000 positions: 0,3005,43
+    Bloom filters for column 3:
+      Entry 0: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
+      Entry 1: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
+      Entry 2: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
+      Entry 3: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
+      Entry 4: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
+      Stripe level merge: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
+  Stripe: offset: 194454 data: 63817 rows: 5000 tail: 86 index: 952
+    Stream: column 0 section ROW_INDEX start: 194454 length 17
+    Stream: column 1 section ROW_INDEX start: 194471 length 165
+    Stream: column 2 section ROW_INDEX start: 194636 length 167
+    Stream: column 3 section ROW_INDEX start: 194803 length 91
+    Stream: column 3 section BLOOM_FILTER start: 194894 length 512
+    Stream: column 1 section DATA start: 195406 length 20035
+    Stream: column 2 section DATA start: 215441 length 40050
+    Stream: column 3 section DATA start: 255491 length 3574
+    Stream: column 3 section LENGTH start: 259065 length 25
+    Stream: column 3 section DICTIONARY_DATA start: 259090 length 133
+    Encoding column 0: DIRECT
+    Encoding column 1: DIRECT_V2
+    Encoding column 2: DIRECT_V2
+    Encoding column 3: DICTIONARY_V2[35]
+    Row group indices for column 3:
+      Entry 0: count: 1000 hasNull: false min: Darkness, max: worst sum: 3901 positions: 0,0,0
+      Entry 1: count: 1000 hasNull: false min: Darkness, max: worst sum: 3900 positions: 0,431,431
+      Entry 2: count: 1000 hasNull: false min: Darkness, max: worst sum: 3909 positions: 0,1485,52
+      Entry 3: count: 1000 hasNull: false min: Darkness, max: worst sum: 3947 positions: 0,2196,104
+      Entry 4: count: 1000 hasNull: false min: Darkness, max: worst sum: 3813 positions: 0,2934,131
+    Bloom filters for column 3:
+      Entry 0: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
+      Entry 1: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
+      Entry 2: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
+      Entry 3: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
+      Entry 4: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
+      Stripe level merge: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
+  Stripe: offset: 259309 data: 12943 rows: 1000 tail: 78 index: 432
+    Stream: column 0 section ROW_INDEX start: 259309 length 12
+    Stream: column 1 section ROW_INDEX start: 259321 length 38
+    Stream: column 2 section ROW_INDEX start: 259359 length 41
+    Stream: column 3 section ROW_INDEX start: 259400 length 40
+    Stream: column 3 section BLOOM_FILTER start: 259440 length 301
+    Stream: column 1 section DATA start: 259741 length 4007
+    Stream: column 2 section DATA start: 263748 length 8010
+    Stream: column 3 section DATA start: 271758 length 768
+    Stream: column 3 section LENGTH start: 272526 length 25
+    Stream: column 3 section DICTIONARY_DATA start: 272551 length 133
+    Encoding column 0: DIRECT
+    Encoding column 1: DIRECT_V2
+    Encoding column 2: DIRECT_V2
+    Encoding column 3: DICTIONARY_V2[35]
+    Row group indices for column 3:
+      Entry 0: count: 1000 hasNull: false min: Darkness, max: worst sum: 3866 positions: 0,0,0
+    Bloom filters for column 3:
+      Entry 0: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
+      Stripe level merge: numHashFunctions: 4 bitCount: 6272 popCount: 138 loadFactor: 0.022 expectedFpp: 2.343647E-7
+
+File length: 273307 bytes
+Padding length: 0 bytes
+Padding ratio: 0%
+________________________________________________________________________________________________________________________
+

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/orc/src/test/resources/orc-file-dump-bloomfilter2.out
----------------------------------------------------------------------
diff --git a/orc/src/test/resources/orc-file-dump-bloomfilter2.out b/orc/src/test/resources/orc-file-dump-bloomfilter2.out
new file mode 100644
index 0000000..fa5cc2d
--- /dev/null
+++ b/orc/src/test/resources/orc-file-dump-bloomfilter2.out
@@ -0,0 +1,179 @@
+Structure for TestFileDump.testDump.orc
+File Version: 0.12 with HIVE_13083
+Rows: 21000
+Compression: ZLIB
+Compression size: 4096
+Type: struct<i:int,l:bigint,s:string>
+
+Stripe Statistics:
+  Stripe 1:
+    Column 0: count: 5000 hasNull: false
+    Column 1: count: 5000 hasNull: false min: -2146021688 max: 2147223299 sum: 515792826
+    Column 2: count: 5000 hasNull: false min: -9218592812243954469 max: 9221614132680747961
+    Column 3: count: 5000 hasNull: false min: Darkness, max: worst sum: 19280
+  Stripe 2:
+    Column 0: count: 5000 hasNull: false
+    Column 1: count: 5000 hasNull: false min: -2146733128 max: 2147001622 sum: 7673427
+    Column 2: count: 5000 hasNull: false min: -9220818777591257749 max: 9222259462014003839
+    Column 3: count: 5000 hasNull: false min: Darkness, max: worst sum: 19504
+  Stripe 3:
+    Column 0: count: 5000 hasNull: false
+    Column 1: count: 5000 hasNull: false min: -2146993718 max: 2147378179 sum: 132660742551
+    Column 2: count: 5000 hasNull: false min: -9218342074710552826 max: 9222303228623055266
+    Column 3: count: 5000 hasNull: false min: Darkness, max: worst sum: 19641
+  Stripe 4:
+    Column 0: count: 5000 hasNull: false
+    Column 1: count: 5000 hasNull: false min: -2146658006 max: 2145520931 sum: 8533549236
+    Column 2: count: 5000 hasNull: false min: -9222758097219661129 max: 9221043130193737406
+    Column 3: count: 5000 hasNull: false min: Darkness, max: worst sum: 19470
+  Stripe 5:
+    Column 0: count: 1000 hasNull: false
+    Column 1: count: 1000 hasNull: false min: -2146245500 max: 2146378640 sum: 51299706363
+    Column 2: count: 1000 hasNull: false min: -9208193203370316142 max: 9218567213558056476
+    Column 3: count: 1000 hasNull: false min: Darkness, max: worst sum: 3866
+
+File Statistics:
+  Column 0: count: 21000 hasNull: false
+  Column 1: count: 21000 hasNull: false min: -2146993718 max: 2147378179 sum: 193017464403
+  Column 2: count: 21000 hasNull: false min: -9222758097219661129 max: 9222303228623055266
+  Column 3: count: 21000 hasNull: false min: Darkness, max: worst sum: 81761
+
+Stripes:
+  Stripe: offset: 3 data: 63786 rows: 5000 tail: 85 index: 6974
+    Stream: column 0 section ROW_INDEX start: 3 length 17
+    Stream: column 1 section ROW_INDEX start: 20 length 166
+    Stream: column 2 section ROW_INDEX start: 186 length 169
+    Stream: column 2 section BLOOM_FILTER start: 355 length 6535
+    Stream: column 3 section ROW_INDEX start: 6890 length 87
+    Stream: column 1 section DATA start: 6977 length 20035
+    Stream: column 2 section DATA start: 27012 length 40050
+    Stream: column 3 section DATA start: 67062 length 3543
+    Stream: column 3 section LENGTH start: 70605 length 25
+    Stream: column 3 section DICTIONARY_DATA start: 70630 length 133
+    Encoding column 0: DIRECT
+    Encoding column 1: DIRECT_V2
+    Encoding column 2: DIRECT_V2
+    Encoding column 3: DICTIONARY_V2[35]
+    Row group indices for column 2:
+      Entry 0: count: 1000 hasNull: false min: -9200577545527640566 max: 9175500305011173751 positions: 0,0,0
+      Entry 1: count: 1000 hasNull: false min: -9203618157670445774 max: 9208123824411178101 positions: 4099,2,488
+      Entry 2: count: 1000 hasNull: false min: -9218592812243954469 max: 9221351515892923972 positions: 12297,6,464
+      Entry 3: count: 1000 hasNull: false min: -9206585617947511272 max: 9167703224425685487 positions: 20495,10,440
+      Entry 4: count: 1000 hasNull: false min: -9206645795733282496 max: 9221614132680747961 positions: 28693,14,416
+    Bloom filters for column 2:
+      Entry 0: numHashFunctions: 7 bitCount: 9600 popCount: 4931 loadFactor: 0.5136 expectedFpp: 0.009432924
+      Entry 1: numHashFunctions: 7 bitCount: 9600 popCount: 4956 loadFactor: 0.5163 expectedFpp: 0.009772834
+      Entry 2: numHashFunctions: 7 bitCount: 9600 popCount: 4971 loadFactor: 0.5178 expectedFpp: 0.009981772
+      Entry 3: numHashFunctions: 7 bitCount: 9600 popCount: 4971 loadFactor: 0.5178 expectedFpp: 0.009981772
+      Entry 4: numHashFunctions: 7 bitCount: 9600 popCount: 4949 loadFactor: 0.5155 expectedFpp: 0.009676614
+      Stripe level merge: numHashFunctions: 7 bitCount: 9600 popCount: 9347 loadFactor: 0.9736 expectedFpp: 0.829482
+  Stripe: offset: 70848 data: 63775 rows: 5000 tail: 85 index: 6965
+    Stream: column 0 section ROW_INDEX start: 70848 length 17
+    Stream: column 1 section ROW_INDEX start: 70865 length 164
+    Stream: column 2 section ROW_INDEX start: 71029 length 168
+    Stream: column 2 section BLOOM_FILTER start: 71197 length 6533
+    Stream: column 3 section ROW_INDEX start: 77730 length 83
+    Stream: column 1 section DATA start: 77813 length 20035
+    Stream: column 2 section DATA start: 97848 length 40050
+    Stream: column 3 section DATA start: 137898 length 3532
+    Stream: column 3 section LENGTH start: 141430 length 25
+    Stream: column 3 section DICTIONARY_DATA start: 141455 length 133
+    Encoding column 0: DIRECT
+    Encoding column 1: DIRECT_V2
+    Encoding column 2: DIRECT_V2
+    Encoding column 3: DICTIONARY_V2[35]
+    Row group indices for column 2:
+      Entry 0: count: 1000 hasNull: false min: -9218450653857701562 max: 9189819526332228512 positions: 0,0,0
+      Entry 1: count: 1000 hasNull: false min: -9220818777591257749 max: 9178821722829648113 positions: 4099,2,488
+      Entry 2: count: 1000 hasNull: false min: -9220031433030423388 max: 9210838931786956852 positions: 12297,6,464
+      Entry 3: count: 1000 hasNull: false min: -9208195729739635607 max: 9222259462014003839 positions: 20495,10,440
+      Entry 4: count: 1000 hasNull: false min: -9174271499932339698 max: 9212277876771676916 positions: 28693,14,416
+    Bloom filters for column 2:
+      Entry 0: numHashFunctions: 7 bitCount: 9600 popCount: 4971 loadFactor: 0.5178 expectedFpp: 0.009981772
+      Entry 1: numHashFunctions: 7 bitCount: 9600 popCount: 4988 loadFactor: 0.5196 expectedFpp: 0.010223193
+      Entry 2: numHashFunctions: 7 bitCount: 9600 popCount: 5002 loadFactor: 0.521 expectedFpp: 0.01042575
+      Entry 3: numHashFunctions: 7 bitCount: 9600 popCount: 4962 loadFactor: 0.5169 expectedFpp: 0.009855959
+      Entry 4: numHashFunctions: 7 bitCount: 9600 popCount: 4966 loadFactor: 0.5173 expectedFpp: 0.009911705
+      Stripe level merge: numHashFunctions: 7 bitCount: 9600 popCount: 9344 loadFactor: 0.9733 expectedFpp: 0.8276205
+  Stripe: offset: 141673 data: 63787 rows: 5000 tail: 85 index: 6971
+    Stream: column 0 section ROW_INDEX start: 141673 length 17
+    Stream: column 1 section ROW_INDEX start: 141690 length 163
+    Stream: column 2 section ROW_INDEX start: 141853 length 168
+    Stream: column 2 section BLOOM_FILTER start: 142021 length 6533
+    Stream: column 3 section ROW_INDEX start: 148554 length 90
+    Stream: column 1 section DATA start: 148644 length 20035
+    Stream: column 2 section DATA start: 168679 length 40050
+    Stream: column 3 section DATA start: 208729 length 3544
+    Stream: column 3 section LENGTH start: 212273 length 25
+    Stream: column 3 section DICTIONARY_DATA start: 212298 length 133
+    Encoding column 0: DIRECT
+    Encoding column 1: DIRECT_V2
+    Encoding column 2: DIRECT_V2
+    Encoding column 3: DICTIONARY_V2[35]
+    Row group indices for column 2:
+      Entry 0: count: 1000 hasNull: false min: -9211978436552246208 max: 9179058898902097152 positions: 0,0,0
+      Entry 1: count: 1000 hasNull: false min: -9195645160817780503 max: 9189147759444307708 positions: 4099,2,488
+      Entry 2: count: 1000 hasNull: false min: -9202888157616520823 max: 9193561362676960747 positions: 12297,6,464
+      Entry 3: count: 1000 hasNull: false min: -9216318198067839390 max: 9221286760675829363 positions: 20495,10,440
+      Entry 4: count: 1000 hasNull: false min: -9218342074710552826 max: 9222303228623055266 positions: 28693,14,416
+    Bloom filters for column 2:
+      Entry 0: numHashFunctions: 7 bitCount: 9600 popCount: 4967 loadFactor: 0.5174 expectedFpp: 0.009925688
+      Entry 1: numHashFunctions: 7 bitCount: 9600 popCount: 5002 loadFactor: 0.521 expectedFpp: 0.01042575
+      Entry 2: numHashFunctions: 7 bitCount: 9600 popCount: 4964 loadFactor: 0.5171 expectedFpp: 0.009883798
+      Entry 3: numHashFunctions: 7 bitCount: 9600 popCount: 4943 loadFactor: 0.5149 expectedFpp: 0.009594797
+      Entry 4: numHashFunctions: 7 bitCount: 9600 popCount: 4930 loadFactor: 0.5135 expectedFpp: 0.009419539
+      Stripe level merge: numHashFunctions: 7 bitCount: 9600 popCount: 9333 loadFactor: 0.9722 expectedFpp: 0.82082444
+  Stripe: offset: 212516 data: 63817 rows: 5000 tail: 85 index: 6964
+    Stream: column 0 section ROW_INDEX start: 212516 length 17
+    Stream: column 1 section ROW_INDEX start: 212533 length 165
+    Stream: column 2 section ROW_INDEX start: 212698 length 167
+    Stream: column 2 section BLOOM_FILTER start: 212865 length 6524
+    Stream: column 3 section ROW_INDEX start: 219389 length 91
+    Stream: column 1 section DATA start: 219480 length 20035
+    Stream: column 2 section DATA start: 239515 length 40050
+    Stream: column 3 section DATA start: 279565 length 3574
+    Stream: column 3 section LENGTH start: 283139 length 25
+    Stream: column 3 section DICTIONARY_DATA start: 283164 length 133
+    Encoding column 0: DIRECT
+    Encoding column 1: DIRECT_V2
+    Encoding column 2: DIRECT_V2
+    Encoding column 3: DICTIONARY_V2[35]
+    Row group indices for column 2:
+      Entry 0: count: 1000 hasNull: false min: -9222731174895935707 max: 9214167447015056056 positions: 0,0,0
+      Entry 1: count: 1000 hasNull: false min: -9222758097219661129 max: 9221043130193737406 positions: 4099,2,488
+      Entry 2: count: 1000 hasNull: false min: -9174483776261243438 max: 9208134757538374043 positions: 12297,6,464
+      Entry 3: count: 1000 hasNull: false min: -9174329712613510612 max: 9197412874152820822 positions: 20495,10,440
+      Entry 4: count: 1000 hasNull: false min: -9221162005892422758 max: 9220625004936875965 positions: 28693,14,416
+    Bloom filters for column 2:
+      Entry 0: numHashFunctions: 7 bitCount: 9600 popCount: 4951 loadFactor: 0.5157 expectedFpp: 0.009704026
+      Entry 1: numHashFunctions: 7 bitCount: 9600 popCount: 4969 loadFactor: 0.5176 expectedFpp: 0.009953696
+      Entry 2: numHashFunctions: 7 bitCount: 9600 popCount: 4994 loadFactor: 0.5202 expectedFpp: 0.010309587
+      Entry 3: numHashFunctions: 7 bitCount: 9600 popCount: 4941 loadFactor: 0.5147 expectedFpp: 0.009567649
+      Entry 4: numHashFunctions: 7 bitCount: 9600 popCount: 4993 loadFactor: 0.5201 expectedFpp: 0.010295142
+      Stripe level merge: numHashFunctions: 7 bitCount: 9600 popCount: 9353 loadFactor: 0.9743 expectedFpp: 0.8332165
+  Stripe: offset: 283382 data: 12943 rows: 1000 tail: 78 index: 1468
+    Stream: column 0 section ROW_INDEX start: 283382 length 12
+    Stream: column 1 section ROW_INDEX start: 283394 length 38
+    Stream: column 2 section ROW_INDEX start: 283432 length 41
+    Stream: column 2 section BLOOM_FILTER start: 283473 length 1337
+    Stream: column 3 section ROW_INDEX start: 284810 length 40
+    Stream: column 1 section DATA start: 284850 length 4007
+    Stream: column 2 section DATA start: 288857 length 8010
+    Stream: column 3 section DATA start: 296867 length 768
+    Stream: column 3 section LENGTH start: 297635 length 25
+    Stream: column 3 section DICTIONARY_DATA start: 297660 length 133
+    Encoding column 0: DIRECT
+    Encoding column 1: DIRECT_V2
+    Encoding column 2: DIRECT_V2
+    Encoding column 3: DICTIONARY_V2[35]
+    Row group indices for column 2:
+      Entry 0: count: 1000 hasNull: false min: -9208193203370316142 max: 9218567213558056476 positions: 0,0,0
+    Bloom filters for column 2:
+      Entry 0: numHashFunctions: 7 bitCount: 9600 popCount: 4948 loadFactor: 0.5154 expectedFpp: 0.00966294
+      Stripe level merge: numHashFunctions: 7 bitCount: 9600 popCount: 4948 loadFactor: 0.5154 expectedFpp: 0.00966294
+
+File length: 298416 bytes
+Padding length: 0 bytes
+Padding ratio: 0%
+________________________________________________________________________________________________________________________
+

http://git-wip-us.apache.org/repos/asf/hive/blob/ffb79509/orc/src/test/resources/orc-file-dump-dictionary-threshold.out
----------------------------------------------------------------------
diff --git a/orc/src/test/resources/orc-file-dump-dictionary-threshold.out b/orc/src/test/resources/orc-file-dump-dictionary-threshold.out
new file mode 100644
index 0000000..17a964b
--- /dev/null
+++ b/orc/src/test/resources/orc-file-dump-dictionary-threshold.out
@@ -0,0 +1,190 @@
+Structure for TestFileDump.testDump.orc
+File Version: 0.12 with HIVE_13083
+Rows: 21000
+Compression: ZLIB
+Compression size: 4096
+Type: struct<i:int,l:bigint,s:string>
+
+Stripe Statistics:
+  Stripe 1:
+    Column 0: count: 5000 hasNull: false
+    Column 1: count: 5000 hasNull: false min: -2147115959 max: 2145911404 sum: 159677169195
+    Column 2: count: 5000 hasNull: false min: -9216505819108477308 max: 9217851628057711416
+    Column 3: count: 5000 hasNull: false 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 sum: 381254
+  Stripe 2:
+    Column 0: count: 5000 hasNull: false
+    Column 1: count: 5000 hasNull: false min: -2147390285 max: 2147224606 sum: -14961457759
+    Column 2: count: 5000 hasNull: false min: -9222178666167296739 max: 9221301751385928177
+    Column 3: count: 5000 hasNull: false min: Darkness,-230-368-488-586-862-930-1686-2044-2636-2652-2872-3108-3162-3192-3404-3442-3508-3542-3550-3712-3980-4146-4204-4336-4390-4418-4424-4490-4512-4650-4768-4924-4950-5210 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 sum: 1117994
+  Stripe 3:
+    Column 0: count: 5000 hasNull: false
+    Column 1: count: 5000 hasNull: false min: -2145842720 max: 2146718321 sum: 141092475520
+    Column 2: count: 5000 hasNull: false min: -9221963099397084326 max: 9222722740629726770
+    Column 3: count: 5000 hasNull: false min: Darkness,-230-368-488-586-862-930-1686-2044-2636-2652-2872-3108-3162-3192-3404-3442-3508-3542-3550-3712-3980-4146-4204-4336-4390-4418-4424-4490-4512-4650-4768-4924-4950-5210-5524-5630-5678-5710-5758-5952-6238-6252-6300-6366-6668-6712-6926-6942-7100-7194-7802-8030-8452-8608-8640-8862-8868-9134-9234-9412-9602-9608-9642-9678-9740-9780-10426 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 sum: 1925226
+  Stripe 4:
+    Column 0: count: 5000 hasNull: false
+    Column 1: count: 5000 hasNull: false min: -2145378214 max: 2147453086 sum: -153680004530
+    Column 2: count: 5000 hasNull: false min: -9222731174895935707 max: 9222919052987871506
+    Column 3: count: 5000 hasNull: false min: Darkness,-230-368-488-586-862-930-1686-2044-2636-2652-2872-3108-3162-3192-3404-3442-3508-3542-3550-3712-3980-4146-4204-4336-4390-4418-4424-4490-4512-4650-4768-4924-4950-5210-5524-5630-5678-5710-5758-5952-6238-6252-6300-6366-6668-6712-6926-6942-7100-7194-7802-8030-8452-8608-8640-8862-8868-9134-9234-9412-9602-9608-9642-9678-9740-9780-10426-10510-10514-10706-10814-10870-10942-11028-11244-11326-11462-11496-11656-11830-12022-12178-12418-12832-13304-13448-13590-13618-13908-14188-14246-14340-14364-14394-14762-14850-14964-15048 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 sum: 2815002
+  Stripe 5:
+    Column 0: count: 1000 hasNull: false
+    Column 1: count: 1000 hasNull: false min: -2143595397 max: 2136858458 sum: -22999664100
+    Column 2: count: 1000 hasNull: false min: -9212379634781416464 max: 9197412874152820822
+    Column 3: count: 1000 hasNull: false min: Darkness,-230-368-488-586-862-930-1686-2044-2636-2652-2872-3108-3162-3192-3404-3442-3508-3542-3550-3712-3980-4146-4204-4336-4390-4418-4424-4490-4512-4650-4768-4924-4950-5210-5524-5630-5678-5710-5758-5952-6238-6252-6300-6366-6668-6712-6926-6942-7100-7194-7802-8030-8452-8608-8640-8862-8868-9134-9234-9412-9602-9608-9642-9678-9740-9780-10426-10510-10514-10706-10814-10870-10942-11028-11244-11326-11462-11496-11656-11830-12022-12178-12418-12832-13304-13448-13590-13618-13908-14188-14246-14340-14364-14394-14762-14850-14964-15048-15494-15674-15726-16006-16056-16180-16304-16332-16452-16598-16730-16810-16994-17210-17268-17786-17962-18214-18444-18446-18724-18912-18952-19164-19348-19400-19546-19776-19896-20084 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-7
 798-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 sum: 670762
+
+File Statistics:
+  Column 0: count: 21000 hasNull: false
+  Column 1: count: 21000 hasNull: false min: -2147390285 max: 2147453086 sum: 109128518326
+  Column 2: count: 21000 hasNull: false min: -9222731174895935707 max: 9222919052987871506
+  Column 3: count: 21000 hasNull: false 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 sum: 6910238
+
+Stripes:
+  Stripe: offset: 3 data: 163602 rows: 5000 tail: 68 index: 720
+    Stream: column 0 section ROW_INDEX start: 3 length 17
+    Stream: column 1 section ROW_INDEX start: 20 length 166
+    Stream: column 2 section ROW_INDEX start: 186 length 171
+    Stream: column 3 section ROW_INDEX start: 357 length 366
+    Stream: column 1 section DATA start: 723 length 20035
+    Stream: column 2 section DATA start: 20758 length 40050
+    Stream: column 3 section DATA start: 60808 length 99226
+    Stream: column 3 section LENGTH start: 160034 length 4291
+    Encoding column 0: DIRECT
+    Encoding column 1: DIRECT_V2
+    Encoding column 2: DIRECT_V2
+    Encoding column 3: DIRECT_V2
+    Row group indices for column 1:
+      Entry 0: count: 1000 hasNull: false min: -2132329551 max: 2145911404 sum: 61941331718 positions: 0,0,0
+      Entry 1: count: 1000 hasNull: false min: -2138433136 max: 2145210552 sum: 14574030042 positions: 0,2050,488
+      Entry 2: count: 1000 hasNull: false min: -2147115959 max: 2137805337 sum: -2032493169 positions: 4099,2054,464
+      Entry 3: count: 1000 hasNull: false min: -2137828953 max: 2145877119 sum: -3167202608 positions: 8198,2058,440
+      Entry 4: count: 1000 hasNull: false min: -2146452517 max: 2142394906 sum: 88361503212 positions: 12297,2062,416
+    Row group indices for column 2:
+      Entry 0: count: 1000 hasNull: false min: -9206837518492372266 max: 9169230975203934579 positions: 0,0,0
+      Entry 1: count: 1000 hasNull: false min: -9188878639954124284 max: 9213664245516510068 positions: 4099,2,488
+      Entry 2: count: 1000 hasNull: false min: -9211329013123260308 max: 9217851628057711416 positions: 12297,6,464
+      Entry 3: count: 1000 hasNull: false min: -9185745718227889962 max: 9181722705210917931 positions: 20495,10,440
+      Entry 4: count: 1000 hasNull: false min: -9216505819108477308 max: 9196474183833079923 positions: 28693,14,416
+    Row group indices for column 3:
+      Entry 0: count: 1000 hasNull: false min: Darkness,-230 max: worst-54-290-346-648-908-996 sum: 18442 positions: 0,0,0,0,0
+      Entry 1: count: 1000 hasNull: false min: Darkness,-230-368-488-586-862-930-1686 max: worst-54-290-346-648-908-996-1038-1080-1560-1584-1620-1744-1770-1798-1852-1966 sum: 46338 positions: 4767,2058,0,695,18
+      Entry 2: count: 1000 hasNull: false min: Darkness,-230-368-488-586-862-930-1686-2044 max: worst-54-290-346-648-908-996-1038-1080-1560-1584-1620-1744-1770-1798-1852-1966-2162-2244-2286-2296-2534-2660 sum: 75448 positions: 16464,3340,0,1554,14
+      Entry 3: count: 1000 hasNull: false min: Darkness,-230-368-488-586-862-930-1686-2044-2636-2652-2872-3108 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 sum: 104868 positions: 36532,964,0,2372,90
+      Entry 4: count: 1000 hasNull: false min: Darkness,-230-368-488-586-862-930-1686-2044-2636-2652-2872-3108-3162-3192-3404-3442-3508-3542-3550-3712-3980-4146 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 sum: 136158 positions: 63067,3432,0,3354,108
+  Stripe: offset: 164393 data: 368335 rows: 5000 tail: 69 index: 956
+    Stream: column 0 section ROW_INDEX start: 164393 length 17
+    Stream: column 1 section ROW_INDEX start: 164410 length 157
+    Stream: column 2 section ROW_INDEX start: 164567 length 166
+    Stream: column 3 section ROW_INDEX start: 164733 length 616
+    Stream: column 1 section DATA start: 165349 length 20035
+    Stream: column 2 section DATA start: 185384 length 40050
+    Stream: column 3 section DATA start: 225434 length 302715
+    Stream: column 3 section LENGTH start: 528149 length 5535
+    Encoding column 0: DIRECT
+    Encoding column 1: DIRECT_V2
+    Encoding column 2: DIRECT_V2
+    Encoding column 3: DIRECT_V2
+    Row group indices for column 1:
+      Entry 0: count: 1000 hasNull: false min: -2146021688 max: 2146838901 sum: -50979197646 positions: 0,0,0
+      Entry 1: count: 1000 hasNull: false min: -2143569489 max: 2141223179 sum: 22810066834 positions: 0,2050,488
+      Entry 2: count: 1000 hasNull: false min: -2140649392 max: 2146301701 sum: -31694882346 positions: 4099,2054,464
+      Entry 3: count: 1000 hasNull: false min: -2147390285 max: 2146299933 sum: 79371934221 positions: 8198,2058,440
+      Entry 4: count: 1000 hasNull: false min: -2145928262 max: 2147224606 sum: -34469378822 positions: 12297,2062,416
+    Row group indices for column 2:
+      Entry 0: count: 1000 hasNull: false min: -9222178666167296739 max: 9191250610515369723 positions: 0,0,0
+      Entry 1: count: 1000 hasNull: false min: -9220148577547102875 max: 9213945522531717278 positions: 4099,2,488
+      Entry 2: count: 1000 hasNull: false min: -9220818777591257749 max: 9221301751385928177 positions: 12297,6,464
+      Entry 3: count: 1000 hasNull: false min: -9220031433030423388 max: 9207856144487414148 positions: 20495,10,440
+      Entry 4: count: 1000 hasNull: false min: -9201438531577205959 max: 9212462124593119846 positions: 28693,14,416
+    Row group indices for column 3:
+      Entry 0: count: 1000 hasNull: false min: Darkness,-230-368-488-586-862-930-1686-2044-2636-2652-2872-3108-3162-3192-3404-3442-3508-3542-3550-3712-3980-4146-4204-4336-4390-4418-4424-4490-4512-4650-4768-4924-4950-5210 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 sum: 166320 positions: 0,0,0,0,0
+      Entry 1: count: 1000 hasNull: false min: Darkness,-230-368-488-586-862-930-1686-2044-2636-2652-2872-3108-3162-3192-3404-3442-3508-3542-3550-3712-3980-4146-4204-4336-4390-4418-4424-4490-4512-4650-4768-4924-4950-5210-5524-5630-5678-5710-5758-5952-6238 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 sum: 193436 positions: 43833,2480,0,967,90
+      Entry 2: count: 1000 hasNull: false min: Darkness,-230-368-488-586-862-930-1686-2044-2636-2652-2872-3108-3162-3192-3404-3442-3508-3542-3550-3712-3980-4146-4204-4336-4390-4418-4424-4490-4512-4650-4768-4924-4950-5210-5524-5630-5678-5710-5758-5952-6238-6252-6300-6366-6668-6712-6926-6942-7100 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 sum: 224740 positions: 94117,3404,0,1945,222
+      Entry 3: count: 1000 hasNull: false min: Darkness,-230-368-488-586-862-930-1686-2044-2636-2652-2872-3108-3162-3192-3404-3442-3508-3542-3550-3712-3980-4146-4204-4336-4390-4418-4424-4490-4512-4650-4768-4924-4950-5210-5524-5630-5678-5710-5758-5952-6238-6252-6300-6366-6668-6712-6926-6942-7100-7194-7802-8030 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 sum: 252094 positions: 155111,2864,0,3268,48
+      Entry 4: count: 1000 hasNull: false min: Darkness,-230-368-488-586-862-930-1686-2044-2636-2652-2872-3108-3162-3192-3404-3442-3508-3542-3550-3712-3980-4146-4204-4336-4390-4418-4424-4490-4512-4650-4768-4924-4950-5210-5524-5630-5678-5710-5758-5952-6238-6252-6300-6366-6668-6712-6926-6942-7100-7194-7802-8030-8452-8608-8640-8862-8868-9134 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 sum: 281404 positions: 224570,1006,0,4064,342
+  Stripe: offset: 533753 data: 606074 rows: 5000 tail: 69 index: 1427
+    Stream: column 0 section ROW_INDEX start: 533753 length 17
+    Stream: column 1 section ROW_INDEX start: 533770 length 167
+    Stream: column 2 section ROW_INDEX start: 533937 length 168
+    Stream: column 3 section ROW_INDEX start: 534105 length 1075
+    Stream: column 1 section DATA start: 535180 length 20035
+    Stream: column 2 section DATA start: 555215 length 40050
+    Stream: column 3 section DATA start: 595265 length 540210
+    Stream: column 3 section LENGTH start: 1135475 length 5779
+    Encoding column 0: DIRECT
+    Encoding column 1: DIRECT_V2
+    Encoding column 2: DIRECT_V2
+    Encoding column 3: DIRECT_V2
+    Row group indices for column 1:
+      Entry 0: count: 1000 hasNull: false min: -2138229212 max: 2144818981 sum: -22823642812 positions: 0,0,0
+      Entry 1: count: 1000 hasNull: false min: -2145842720 max: 2144179881 sum: -12562754334 positions: 0,2050,488
+      Entry 2: count: 1000 hasNull: false min: -2143045885 max: 2146718321 sum: 82993638644 positions: 4099,2054,464
+      Entry 3: count: 1000 hasNull: false min: -2144745617 max: 2146570474 sum: 25138722367 positions: 8198,2058,440
+      Entry 4: count: 1000 hasNull: false min: -2140127150 max: 2135081620 sum: 68346511655 positions: 12297,2062,416
+    Row group indices for column 2:
+      Entry 0: count: 1000 hasNull: false min: -9204340807292138409 max: 9208698732685326961 positions: 0,0,0
+      Entry 1: count: 1000 hasNull: false min: -9221963099397084326 max: 9222722740629726770 positions: 4099,2,488
+      Entry 2: count: 1000 hasNull: false min: -9210480084701091299 max: 9207767402467343058 positions: 12297,6,464
+      Entry 3: count: 1000 hasNull: false min: -9195038026813631215 max: 9199201928563274421 positions: 20495,10,440
+      Entry 4: count: 1000 hasNull: false min: -9215483580266514322 max: 9220102792864959501 positions: 28693,14,416
+    Row group indices for column 3:
+      Entry 0: count: 1000 hasNull: false min: Darkness,-230-368-488-586-862-930-1686-2044-2636-2652-2872-3108-3162-3192-3404-3442-3508-3542-3550-3712-3980-4146-4204-4336-4390-4418-4424-4490-4512-4650-4768-4924-4950-5210-5524-5630-5678-5710-5758-5952-6238-6252-6300-6366-6668-6712-6926-6942-7100-7194-7802-8030-8452-8608-8640-8862-8868-9134-9234-9412-9602-9608-9642-9678-9740-9780-10426 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 sum: 313880 positions: 0,0,0,0,0
+      Entry 1: count: 1000 hasNull: false min: Darkness,-230-368-488-586-862-930-1686-2044-2636-2652-2872-3108-3162-3192-3404-3442-3508-3542-3550-3712-3980-4146-4204-4336-4390-4418-4424-4490-4512-4650-4768-4924-4950-5210-5524-5630-5678-5710-5758-5952-6238-6252-6300-6366-6668-6712-6926-6942-7100-7194-7802-8030-8452-8608-8640-8862-8868-9134-9234-9412-9602-9608-9642-9678-9740-9780-10426-10510-10514-10706-10814-10870-10942-11028 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 sum: 349542 positions: 87800,2584,0,1097,28
+      Entry 2: count: 1000 hasNull: false min: Darkness,-230-368-488-586-862-930-1686-2044-2636-2652-2872-3108-3162-3192-3404-3442-3508-3542-3550-3712-3980-4146-4204-4336-4390-4418-4424-4490-4512-4650-4768-4924-4950-5210-5524-5630-5678-5710-5758-5952-6238-6252-6300-6366-6668-6712-6926-6942-7100-7194-7802-8030-8452-8608-8640-8862-8868-9134-9234-9412-9602-9608-9642-9678-9740-9780-10426-10510-10514-10706-10814-10870-10942-11028-11244-11326-11462-11496-11656-11830-12022 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 sum: 386538 posit
 ions: 185635,3966,0,2077,162
+      Entry 3: count: 1000 hasNull: false min: Darkness,-230-368-488-586-862-930-1686-2044-2636-2652-2872-3108-3162-3192-3404-3442-3508-3542-3550-3712-3980-4146-4204-4336-4390-4418-4424-4490-4512-4650-4768-4924-4950-5210-5524-5630-5678-5710-5758-5952-6238-6252-6300-6366-6668-6712-6926-6942-7100-7194-7802-8030-8452-8608-8640-8862-8868-9134-9234-9412-9602-9608-9642-9678-9740-9780-10426-10510-10514-10706-10814-10870-10942-11028-11244-11326-11462-11496-11656-11830-12022-12178-12418-12832-13304 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 sum: 421660 positions: 295550,1384,0,3369,16
+      Entry 4: count: 1000 hasNull: false min: Darkness,-230-368-488-586-862-930-1686-2044-2636-2652-2872-3108-3162-3192-3404-3442-3508-3542-3550-3712-3980-4146-4204-4336-4390-4418-4424-4490-4512-4650-4768-4924-4950-5210-5524-5630-5678-5710-5758-5952-6238-6252-6300-6366-6668-6712-6926-6942-7100-7194-7802-8030-8452-8608-8640-8862-8868-9134-9234-9412-9602-9608-9642-9678-9740-9780-10426-10510-10514-10706-10814-10870-10942-11028-11244-11326-11462-11496-11656-11830-12022-12178-12418-12832-13304-13448-13590-13618-13908-14188 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 sum: 453606 positions: 412768,1156,0,4041,470
+  Stripe: offset: 1141323 data: 864001 rows: 5000 tail: 69 index: 1975
+    Stream: column 0 section ROW_INDEX start: 1141323 length 17
+    Stream: column 1 section ROW_INDEX start: 1141340 length 156
+    Stream: column 2 section ROW_INDEX start: 1141496 length 168
+    Stream: column 3 section ROW_INDEX start: 1141664 length 1634
+    Stream: column 1 section DATA start: 1143298 length 20035
+    Stream: column 2 section DATA start: 1163333 length 40050
+    Stream: column 3 section DATA start: 1203383 length 798014
+    Stream: column 3 section LENGTH start: 2001397 length 5902
+    Encoding column 0: DIRECT
+    Encoding column 1: DIRECT_V2
+    Encoding column 2: DIRECT_V2
+    Encoding column 3: DIRECT_V2
+    Row group indices for column 1:
+      Entry 0: count: 1000 hasNull: false min: -2145319330 max: 2146998132 sum: -50856753363 positions: 0,0,0
+      Entry 1: count: 1000 hasNull: false min: -2134288866 max: 2147453086 sum: -17911019023 positions: 0,2050,488
+      Entry 2: count: 1000 hasNull: false min: -2139010804 max: 2144727593 sum: -24993151857 positions: 4099,2054,464
+      Entry 3: count: 1000 hasNull: false min: -2145378214 max: 2144098933 sum: -18055164052 positions: 8198,2058,440
+      Entry 4: count: 1000 hasNull: false min: -2140494429 max: 2144595861 sum: -41863916235 positions: 12297,2062,416
+    Row group indices for column 2:
+      Entry 0: count: 1000 hasNull: false min: -9172774601303513941 max: 9212917101275642143 positions: 0,0,0
+      Entry 1: count: 1000 hasNull: false min: -9218164880949195469 max: 9222919052987871506 positions: 4099,2,488
+      Entry 2: count: 1000 hasNull: false min: -9222731174895935707 max: 9214167447015056056 positions: 12297,6,464
+      Entry 3: count: 1000 hasNull: false min: -9196276654247395117 max: 9210639275226058005 positions: 20495,10,440
+      Entry 4: count: 1000 hasNull: false min: -9197393848859294562 max: 9208134757538374043 positions: 28693,14,416
+    Row group indices for column 3:
+      Entry 0: count: 1000 hasNull: false min: Darkness,-230-368-488-586-862-930-1686-2044-2636-2652-2872-3108-3162-3192-3404-3442-3508-3542-3550-3712-3980-4146-4204-4336-4390-4418-4424-4490-4512-4650-4768-4924-4950-5210-5524-5630-5678-5710-5758-5952-6238-6252-6300-6366-6668-6712-6926-6942-7100-7194-7802-8030-8452-8608-8640-8862-8868-9134-9234-9412-9602-9608-9642-9678-9740-9780-10426-10510-10514-10706-10814-10870-10942-11028-11244-11326-11462-11496-11656-11830-12022-12178-12418-12832-13304-13448-13590-13618-13908-14188-14246-14340-14364-14394-14762-14850-14964-15048 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 sum: 492916 positions: 0,0,0,0,0
+      Entry 1: count: 1000 hasNull: false min: Darkness,-230-368-488-586-862-930-1686-2044-2636-2652-2872-3108-3162-3192-3404-3442-3508-3542-3550-3712-3980-4146-4204-4336-4390-4418-4424-4490-4512-4650-4768-4924-4950-5210-5524-5630-5678-5710-5758-5952-6238-6252-6300-6366-6668-6712-6926-6942-7100-7194-7802-8030-8452-8608-8640-8862-8868-9134-9234-9412-9602-9608-9642-9678-9740-9780-10426-10510-10514-10706-10814-10870-10942-11028-11244-11326-11462-11496-11656-11830-12022-12178-12418-12832-13304-13448-13590-13618-13908-14188-14246-14340-14364-14394-14762-14850-14964-15048-15494-15674-15726-16006 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 sum: 527290 positions: 139298,1396,0,1077,140
+      Entry 2: count: 1000 hasNull: false min: Darkness,-230-368-488-586-862-930-1686-2044-2636-2652-2872-3108-3162-3192-3404-3442-3508-3542-3550-3712-3980-4146-4204-4336-4390-4418-4424-4490-4512-4650-4768-4924-4950-5210-5524-5630-5678-5710-5758-5952-6238-6252-6300-6366-6668-6712-6926-6942-7100-7194-7802-8030-8452-8608-8640-8862-8868-9134-9234-9412-9602-9608-9642-9678-9740-9780-10426-10510-10514-10706-10814-10870-10942-11028-11244-11326-11462-11496-11656-11830-12022-12178-12418-12832-13304-13448-13590-13618-13908-14188-14246-14340-14364-14394-14762-14850-14964-15048-15494-15674-15726-16006-16056-16180-16304-16332-16452-16598-16730-16810-16994-17210 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-9
 650-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 sum: 568274 positions: 286457,302,0,1926,462
+      Entry 3: count: 1000 hasNull: false min: Darkness,-230-368-488-586-862-930-1686-2044-2636-2652-2872-3108-3162-3192-3404-3442-3508-3542-3550-3712-3980-4146-4204-4336-4390-4418-4424-4490-4512-4650-4768-4924-4950-5210-5524-5630-5678-5710-5758-5952-6238-6252-6300-6366-6668-6712-6926-6942-7100-7194-7802-8030-8452-8608-8640-8862-8868-9134-9234-9412-9602-9608-9642-9678-9740-9780-10426-10510-10514-10706-10814-10870-10942-11028-11244-11326-11462-11496-11656-11830-12022-12178-12418-12832-13304-13448-13590-13618-13908-14188-14246-14340-14364-14394-14762-14850-14964-15048-15494-15674-15726-16006-16056-16180-16304-16332-16452-16598-16730-16810-16994-17210-17268-17786-17962-18214 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-91
 28-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 sum: 594578 positions: 447943,3328,0,3444,250
+      Entry 4: count: 1000 hasNull: false min: Darkness,-230-368-488-586-862-930-1686-2044-2636-2652-2872-3108-3162-3192-3404-3442-3508-3542-3550-3712-3980-4146-4204-4336-4390-4418-4424-4490-4512-4650-4768-4924-4950-5210-5524-5630-5678-5710-5758-5952-6238-6252-6300-6366-6668-6712-6926-6942-7100-7194-7802-8030-8452-8608-8640-8862-8868-9134-9234-9412-9602-9608-9642-9678-9740-9780-10426-10510-10514-10706-10814-10870-10942-11028-11244-11326-11462-11496-11656-11830-12022-12178-12418-12832-13304-13448-13590-13618-13908-14188-14246-14340-14364-14394-14762-14850-14964-15048-15494-15674-15726-16006-16056-16180-16304-16332-16452-16598-16730-16810-16994-17210-17268-17786-17962-18214-18444-18446-18724-18912-18952-19164 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-8
 390-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 sum: 631944 positions: 616471,3986,3778,547,292
+  Stripe: offset: 2007368 data: 207295 rows: 1000 tail: 67 index: 841
+    Stream: column 0 section ROW_INDEX start: 2007368 length 12
+    Stream: column 1 section ROW_INDEX start: 2007380 length 38
+    Stream: column 2 section ROW_INDEX start: 2007418 length 41
+    Stream: column 3 section ROW_INDEX start: 2007459 length 750
+    Stream: column 1 section DATA start: 2008209 length 4007
+    Stream: column 2 section DATA start: 2012216 length 8010
+    Stream: column 3 section DATA start: 2020226 length 194018
+    Stream: column 3 section LENGTH start: 2214244 length 1260
+    Encoding column 0: DIRECT
+    Encoding column 1: DIRECT_V2
+    Encoding column 2: DIRECT_V2
+    Encoding column 3: DIRECT_V2
+    Row group indices for column 1:
+      Entry 0: count: 1000 hasNull: false min: -2143595397 max: 2136858458 sum: -22999664100 positions: 0,0,0
+    Row group indices for column 2:
+      Entry 0: count: 1000 hasNull: false min: -9212379634781416464 max: 9197412874152820822 positions: 0,0,0
+    Row group indices for column 3:
+      Entry 0: count: 1000 hasNull: false min: Darkness,-230-368-488-586-862-930-1686-2044-2636-2652-2872-3108-3162-3192-3404-3442-3508-3542-3550-3712-3980-4146-4204-4336-4390-4418-4424-4490-4512-4650-4768-4924-4950-5210-5524-5630-5678-5710-5758-5952-6238-6252-6300-6366-6668-6712-6926-6942-7100-7194-7802-8030-8452-8608-8640-8862-8868-9134-9234-9412-9602-9608-9642-9678-9740-9780-10426-10510-10514-10706-10814-10870-10942-11028-11244-11326-11462-11496-11656-11830-12022-12178-12418-12832-13304-13448-13590-13618-13908-14188-14246-14340-14364-14394-14762-14850-14964-15048-15494-15674-15726-16006-16056-16180-16304-16332-16452-16598-16730-16810-16994-17210-17268-17786-17962-18214-18444-18446-18724-18912-18952-19164-19348-19400-19546-19776-19896-20084 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 sum: 670762 positions: 0,0,0,0,0
+
+File length: 2217685 bytes
+Padding length: 0 bytes
+Padding ratio: 0%
+________________________________________________________________________________________________________________________
+