You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ha...@apache.org on 2013/10/01 16:44:01 UTC

svn commit: r1528108 [2/2] - in /hive/trunk/ql/src: gen/protobuf/gen-java/org/apache/hadoop/hive/ql/io/orc/ java/org/apache/hadoop/hive/ql/io/orc/ java/org/apache/hadoop/hive/ql/util/ protobuf/org/apache/hadoop/hive/ql/io/orc/ test/org/apache/hadoop/hi...

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/orc/WriterImpl.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/orc/WriterImpl.java?rev=1528108&r1=1528107&r2=1528108&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/orc/WriterImpl.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/orc/WriterImpl.java Tue Oct  1 14:44:00 2013
@@ -27,19 +27,17 @@ import java.util.List;
 import java.util.Map;
 import java.util.TreeMap;
 
-
-import com.google.protobuf.ByteString;
-import com.google.protobuf.CodedOutputStream;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
+import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.io.orc.OrcProto.RowIndexEntry;
 import org.apache.hadoop.hive.ql.io.orc.OrcProto.Type;
+import org.apache.hadoop.hive.ql.util.JavaDataModel;
 import org.apache.hadoop.hive.serde2.io.DateWritable;
 import org.apache.hadoop.hive.serde2.objectinspector.ListObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.MapObjectInspector;
@@ -66,6 +64,9 @@ import org.apache.hadoop.hive.serde2.typ
 import org.apache.hadoop.io.BytesWritable;
 import org.apache.hadoop.io.Text;
 
+import com.google.protobuf.ByteString;
+import com.google.protobuf.CodedOutputStream;
+
 /**
  * An ORC file writer. The file is divided into stripes, which is the natural
  * unit of work when reading. Each stripe is buffered in memory until the
@@ -111,6 +112,7 @@ class WriterImpl implements Writer, Memo
   private int columnCount;
   private long rowCount = 0;
   private long rowsInStripe = 0;
+  private long rawDataSize = 0;
   private int rowsInIndex = 0;
   private final List<OrcProto.StripeInformation> stripes =
     new ArrayList<OrcProto.StripeInformation>();
@@ -1085,6 +1087,7 @@ class WriterImpl implements Writer, Memo
             ((BinaryObjectInspector) inspector).getPrimitiveWritableObject(obj);
         stream.write(val.getBytes(), 0, val.getLength());
         length.write(val.getLength());
+        indexStatistics.updateBinary(val);
       }
     }
 
@@ -1760,6 +1763,74 @@ class WriterImpl implements Writer, Memo
     }
   }
 
+  private long computeRawDataSize() {
+    long result = 0;
+    for (TreeWriter child : treeWriter.getChildrenWriters()) {
+      result += getRawDataSizeFromInspectors(child, child.inspector);
+    }
+    return result;
+  }
+
+  private long getRawDataSizeFromInspectors(TreeWriter child, ObjectInspector oi) {
+    long total = 0;
+    switch (oi.getCategory()) {
+    case PRIMITIVE:
+      total += getRawDataSizeFromPrimitives(child, oi);
+      break;
+    case LIST:
+    case MAP:
+    case UNION:
+    case STRUCT:
+      for (TreeWriter tw : child.childrenWriters) {
+        total += getRawDataSizeFromInspectors(tw, tw.inspector);
+      }
+      break;
+    default:
+      LOG.debug("Unknown object inspector category.");
+      break;
+    }
+    return total;
+  }
+
+  private long getRawDataSizeFromPrimitives(TreeWriter child, ObjectInspector oi) {
+    long result = 0;
+    long numVals = child.fileStatistics.getNumberOfValues();
+    switch (((PrimitiveObjectInspector) oi).getPrimitiveCategory()) {
+    case BOOLEAN:
+    case BYTE:
+    case SHORT:
+    case INT:
+    case FLOAT:
+      return numVals * JavaDataModel.get().primitive1();
+    case LONG:
+    case DOUBLE:
+      return numVals * JavaDataModel.get().primitive2();
+    case STRING:
+      // ORC strings are converted to java Strings. so use JavaDataModel to
+      // compute the overall size of strings
+      child = (StringTreeWriter) child;
+      StringColumnStatistics scs = (StringColumnStatistics) child.fileStatistics;
+      numVals = numVals == 0 ? 1 : numVals;
+      int avgStringLen = (int) (scs.getSum() / numVals);
+      return numVals * JavaDataModel.get().lengthForStringOfLength(avgStringLen);
+    case DECIMAL:
+      return numVals * JavaDataModel.get().lengthOfDecimal();
+    case DATE:
+      return numVals * JavaDataModel.get().lengthOfDate();
+    case BINARY:
+      // get total length of binary blob
+      BinaryColumnStatistics bcs = (BinaryColumnStatistics) child.fileStatistics;
+      return bcs.getSum();
+    case TIMESTAMP:
+      return numVals * JavaDataModel.get().lengthOfTimestamp();
+    default:
+      LOG.debug("Unknown primitive category.");
+      break;
+    }
+
+    return result;
+  }
+
   private OrcProto.CompressionKind writeCompressionKind(CompressionKind kind) {
     switch (kind) {
       case NONE: return OrcProto.CompressionKind.NONE;
@@ -1786,6 +1857,8 @@ class WriterImpl implements Writer, Memo
     builder.setHeaderLength(headerLength);
     builder.setNumberOfRows(rowCount);
     builder.setRowIndexStride(rowIndexStride);
+    // populate raw data size
+    rawDataSize = computeRawDataSize();
     // serialize the types
     writeTypes(builder, treeWriter);
     // add the stripe information
@@ -1872,13 +1945,21 @@ class WriterImpl implements Writer, Memo
     }
   }
 
+  /**
+   * Raw data size will be compute when writing the file footer. Hence raw data
+   * size value will be available only after closing the writer.
+   */
   @Override
   public long getRawDataSize() {
-    return 0;
+    return rawDataSize;
   }
 
+  /**
+   * Row count gets updated when flushing the stripes. To get accurate row
+   * count call this method after writer is closed.
+   */
   @Override
   public long getNumberOfRows() {
-    return 0;
+    return rowCount;
   }
 }

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/util/JavaDataModel.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/util/JavaDataModel.java?rev=1528108&r1=1528107&r2=1528108&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/util/JavaDataModel.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/util/JavaDataModel.java Tue Oct  1 14:44:00 2013
@@ -157,7 +157,7 @@ public enum JavaDataModel {
 
   // ascii string
   public int lengthFor(String string) {
-    return object() + primitive1() * 3 + array() + string.length();
+    return lengthForStringOfLength(string.length());
   }
 
   public int lengthFor(NumericHistogram histogram) {
@@ -267,4 +267,33 @@ public enum JavaDataModel {
   public int lengthForBooleanArrayOfSize(int length) {
     return lengthForPrimitiveArrayOfSize(PRIMITIVE_BYTE, length);
   }
+
+  public int lengthOfDecimal() {
+    // object overhead + 8 bytes for intCompact + 4 bytes for precision
+    // + 4 bytes for scale + size of BigInteger
+    return object() + 2 * primitive2() + lengthOfBigInteger();
+  }
+
+  private int lengthOfBigInteger() {
+    // object overhead + 4 bytes for bitCount + 4 bytes for bitLength
+    // + 4 bytes for firstNonzeroByteNum + 4 bytes for firstNonzeroIntNum +
+    // + 4 bytes for lowestSetBit + 5 bytes for size of magnitude (since max precision
+    // is only 38 for HiveDecimal) + 7 bytes of padding (since java memory allocations
+    // are 8 byte aligned)
+    return object() + 4 * primitive2();
+  }
+
+  public int lengthOfTimestamp() {
+    // object overhead + 4 bytes for int (nanos) + 4 bytes of padding
+    return object() + primitive2();
+  }
+
+  public int lengthOfDate() {
+    // object overhead + 8 bytes for long (fastTime) + 16 bytes for cdate
+    return object() + 3 * primitive2();
+  }
+
+  public int lengthForStringOfLength(int strLen) {
+    return object() + primitive1() * 3 + array() + strLen;
+  }
 }

Modified: hive/trunk/ql/src/protobuf/org/apache/hadoop/hive/ql/io/orc/orc_proto.proto
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/protobuf/org/apache/hadoop/hive/ql/io/orc/orc_proto.proto?rev=1528108&r1=1528107&r2=1528108&view=diff
==============================================================================
--- hive/trunk/ql/src/protobuf/org/apache/hadoop/hive/ql/io/orc/orc_proto.proto (original)
+++ hive/trunk/ql/src/protobuf/org/apache/hadoop/hive/ql/io/orc/orc_proto.proto Tue Oct  1 14:44:00 2013
@@ -15,6 +15,8 @@ message DoubleStatistics {
 message StringStatistics {
   optional string minimum = 1;
   optional string maximum = 2;
+  // sum will store the total length of all strings in a stripe
+  optional sint64 sum = 3;
 }
 
 message BucketStatistics {
@@ -33,6 +35,11 @@ message DateStatistics {
   optional sint32 maximum = 2;
 }
 
+message BinaryStatistics {
+  // sum will store the total binary blob length in a stripe
+  optional sint64 sum = 1;
+}
+
 message ColumnStatistics {
   optional uint64 numberOfValues = 1;
   optional IntegerStatistics intStatistics = 2;
@@ -41,6 +48,7 @@ message ColumnStatistics {
   optional BucketStatistics bucketStatistics = 5;
   optional DecimalStatistics decimalStatistics = 6;
   optional DateStatistics dateStatistics = 7;
+  optional BinaryStatistics binaryStatistics = 8;
 }
 
 message RowIndexEntry {

Modified: hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcFile.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcFile.java?rev=1528108&r1=1528107&r2=1528108&view=diff
==============================================================================
--- hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcFile.java (original)
+++ hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcFile.java Tue Oct  1 14:44:00 2013
@@ -18,12 +18,28 @@
 
 package org.apache.hadoop.hive.ql.io.orc;
 
+import static junit.framework.Assert.assertEquals;
+import static junit.framework.Assert.assertNotNull;
+import static junit.framework.Assert.assertNull;
+import static junit.framework.Assert.assertTrue;
+
+import java.io.File;
+import java.io.IOException;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.sql.Timestamp;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+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.io.sarg.SearchArgument;
-import org.apache.hadoop.hive.ql.io.sarg.TestSearchArgumentImpl;
 import org.apache.hadoop.hive.serde2.io.ByteWritable;
 import org.apache.hadoop.hive.serde2.io.DoubleWritable;
 import org.apache.hadoop.hive.serde2.io.ShortWritable;
@@ -55,26 +71,25 @@ import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TestName;
 
-import java.io.File;
-import java.io.IOException;
-import java.math.BigInteger;
-import java.nio.ByteBuffer;
-import java.sql.Timestamp;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Random;
-
-import static junit.framework.Assert.*;
-import static junit.framework.Assert.assertEquals;
-
 /**
  * Tests for the top level reader/streamFactory of ORC files.
  */
 public class TestOrcFile {
 
+  public static class SimpleStruct {
+    BytesWritable bytes1;
+    Text string1;
+
+    SimpleStruct(BytesWritable b1, String s1) {
+      this.bytes1 = b1;
+      if(s1 == null) {
+        this.string1 = null;
+      } else {
+        this.string1 = new Text(s1);
+      }
+    }
+  }
+
   public static class InnerStruct {
     int int1;
     Text string1 = new Text();
@@ -132,48 +147,6 @@ public class TestOrcFile {
     }
   }
 
-  public static class AllTypesRow {
-    Boolean boolean1;
-    Byte byte1;
-    Short short1;
-    Integer int1;
-    Long long1;
-    Float float1;
-    Double double1;
-    BytesWritable bytes1;
-    Text string1;
-    MiddleStruct middle;
-    List<InnerStruct> list = new ArrayList<InnerStruct>();
-    Map<Text, InnerStruct> map = new HashMap<Text, InnerStruct>();
-    Timestamp ts;
-    HiveDecimal decimal1;
-
-    AllTypesRow(Boolean b1, Byte b2, Short s1, Integer i1, Long l1, Float f1,
-           Double d1,
-           BytesWritable b3, String s2, MiddleStruct m1,
-           List<InnerStruct> l2, Map<Text, InnerStruct> m2,
-           Timestamp ts1, HiveDecimal decimal) {
-      this.boolean1 = b1;
-      this.byte1 = b2;
-      this.short1 = s1;
-      this.int1 = i1;
-      this.long1 = l1;
-      this.float1 = f1;
-      this.double1 = d1;
-      this.bytes1 = b3;
-      if (s2 == null) {
-        this.string1 = null;
-      } else {
-        this.string1 = new Text(s2);
-      }
-      this.middle = m1;
-      this.list = l2;
-      this.map = m2;
-      this.ts = ts1;
-      this.decimal1 = decimal;
-    }
-  }
-
   private static InnerStruct inner(int i, String s) {
     return new InnerStruct(i, s);
   }
@@ -231,39 +204,6 @@ public class TestOrcFile {
   }
 
   @Test
-  public void testWriteFormat_0_11() throws Exception {
-    ObjectInspector inspector;
-    synchronized (TestOrcFile.class) {
-      inspector = ObjectInspectorFactory
-          .getReflectionObjectInspector(AllTypesRow.class,
-              ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    }
-    conf.set("hive.exec.orc.write.format", "0.11");
-    Writer writer = OrcFile.createWriter(fs, testFilePath, conf, inspector,
-        100000, CompressionKind.NONE, 10000, 10000);
-    for(int i = 0; i < 7500; i++) {
-      if (i % 2 == 0) {
-        writer.addRow(new AllTypesRow(false, (byte) 1, (short) 1024, 65536,
-            Long.MAX_VALUE, (float) 1.0, -15.0, bytes(0, 1, 2, 3, 4), "hi",
-            new MiddleStruct(inner(1, "bye"), inner(2, "sigh")), list(
-                inner(3, "good"), inner(4, "bad")), map(), Timestamp
-                .valueOf("2000-03-12 15:00:00"), new HiveDecimal(
-                "12345678.6547456")));
-      } else {
-        writer.addRow(new AllTypesRow(true, (byte) 100, (short) 2048, 65536,
-            Long.MAX_VALUE, (float) 2.0, -5.0, bytes(), "bye",
-            new MiddleStruct(inner(1, "bye"), inner(2, "sigh")), list(
-                inner(100000000, "cat"), inner(-100000, "in"),
-                inner(1234, "hat")),
-            map(inner(5, "chani"), inner(1, "mauddib")), Timestamp
-                .valueOf("2000-03-12 15:00:01"), new HiveDecimal(
-                "12345678.6547457")));
-      }
-    }
-    writer.close();
-  }
-
-  @Test
   public void testReadFormat_0_11() throws Exception {
     Path resourceDir = new Path(System.getProperty("test.build.resources", "ql"
         + File.separator + "src" + File.separator + "test" + File.separator
@@ -319,7 +259,7 @@ public class TestOrcFile {
     assertEquals("count: 7500 min: -15.0 max: -5.0 sum: -75000.0",
         stats[7].toString());
 
-    assertEquals("count: 7500 min: bye max: hi", stats[9].toString());
+    assertEquals("count: 7500 min: bye max: hi sum: 0", stats[9].toString());
 
     // check the inspectors
     StructObjectInspector readerInspector = (StructObjectInspector) reader
@@ -515,6 +455,93 @@ public class TestOrcFile {
   }
 
   @Test
+  public void testStringAndBinaryStatistics() throws Exception {
+
+    ObjectInspector inspector;
+    synchronized (TestOrcFile.class) {
+      inspector = ObjectInspectorFactory.getReflectionObjectInspector
+          (SimpleStruct.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
+    }
+    Writer writer = OrcFile.createWriter(testFilePath,
+                                         OrcFile.writerOptions(conf)
+                                         .inspector(inspector)
+                                         .stripeSize(100000)
+                                         .bufferSize(10000));
+    writer.addRow(new SimpleStruct(bytes(0,1,2,3,4), "foo"));
+    writer.addRow(new SimpleStruct(bytes(0,1,2,3), "bar"));
+    writer.addRow(new SimpleStruct(bytes(0,1,2,3,4,5), null));
+    writer.addRow(new SimpleStruct(null, "hi"));
+    writer.close();
+    Reader reader = OrcFile.createReader(fs, testFilePath);
+
+    // check the stats
+    ColumnStatistics[] stats = reader.getStatistics();
+    assertEquals(4, stats[0].getNumberOfValues());
+    assertEquals("count: 4", stats[0].toString());
+
+    assertEquals(3, stats[1].getNumberOfValues());
+    assertEquals(15, ((BinaryColumnStatistics) stats[1]).getSum());
+    assertEquals("count: 3 sum: 15", stats[1].toString());
+
+    assertEquals(3, stats[2].getNumberOfValues());
+    assertEquals("bar", ((StringColumnStatistics) stats[2]).getMinimum());
+    assertEquals("hi", ((StringColumnStatistics) stats[2]).getMaximum());
+    assertEquals(8, ((StringColumnStatistics) stats[2]).getSum());
+    assertEquals("count: 3 min: bar max: hi sum: 8",
+        stats[2].toString());
+
+    // check the inspectors
+    StructObjectInspector readerInspector =
+        (StructObjectInspector) reader.getObjectInspector();
+    assertEquals(ObjectInspector.Category.STRUCT,
+        readerInspector.getCategory());
+    assertEquals("struct<bytes1:binary,string1:string>",
+        readerInspector.getTypeName());
+    List<? extends StructField> fields =
+        readerInspector.getAllStructFieldRefs();
+    BinaryObjectInspector bi = (BinaryObjectInspector) readerInspector.
+        getStructFieldRef("bytes1").getFieldObjectInspector();
+    StringObjectInspector st = (StringObjectInspector) readerInspector.
+        getStructFieldRef("string1").getFieldObjectInspector();
+    RecordReader rows = reader.rows(null);
+    Object row = rows.next(null);
+    assertNotNull(row);
+    // check the contents of the first row
+    assertEquals(bytes(0,1,2,3,4), bi.getPrimitiveWritableObject(
+        readerInspector.getStructFieldData(row, fields.get(0))));
+    assertEquals("foo", st.getPrimitiveJavaObject(readerInspector.
+        getStructFieldData(row, fields.get(1))));
+
+    // check the contents of second row
+    assertEquals(true, rows.hasNext());
+    row = rows.next(row);
+    assertEquals(bytes(0,1,2,3), bi.getPrimitiveWritableObject(
+        readerInspector.getStructFieldData(row, fields.get(0))));
+    assertEquals("bar", st.getPrimitiveJavaObject(readerInspector.
+        getStructFieldData(row, fields.get(1))));
+
+    // check the contents of second row
+    assertEquals(true, rows.hasNext());
+    row = rows.next(row);
+    assertEquals(bytes(0,1,2,3,4,5), bi.getPrimitiveWritableObject(
+        readerInspector.getStructFieldData(row, fields.get(0))));
+    assertNull(st.getPrimitiveJavaObject(readerInspector.
+        getStructFieldData(row, fields.get(1))));
+
+    // check the contents of second row
+    assertEquals(true, rows.hasNext());
+    row = rows.next(row);
+    assertNull(bi.getPrimitiveWritableObject(
+        readerInspector.getStructFieldData(row, fields.get(0))));
+    assertEquals("hi", st.getPrimitiveJavaObject(readerInspector.
+        getStructFieldData(row, fields.get(1))));
+
+    // handle the close up
+    assertEquals(false, rows.hasNext());
+    rows.close();
+  }
+
+  @Test
   public void test1() throws Exception {
     ObjectInspector inspector;
     synchronized (TestOrcFile.class) {
@@ -567,7 +594,7 @@ public class TestOrcFile {
     assertEquals("count: 2 min: -15.0 max: -5.0 sum: -20.0",
         stats[7].toString());
 
-    assertEquals("count: 2 min: bye max: hi", stats[9].toString());
+    assertEquals("count: 2 min: bye max: hi sum: 5", stats[9].toString());
 
     // check the inspectors
     StructObjectInspector readerInspector =

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=1528108&r1=1528107&r2=1528108&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 Tue Oct  1 14:44:00 2013
@@ -125,7 +125,7 @@ public class TestOrcNullOptimization {
     assertEquals("a", ((StringColumnStatistics) stats[2]).getMinimum());
     assertEquals(19998,
                  ((StringColumnStatistics) stats[2]).getNumberOfValues());
-    assertEquals("count: 19998 min: a max: a",
+    assertEquals("count: 19998 min: a max: a sum: 19998",
         stats[2].toString());
 
     // check the inspectors
@@ -229,7 +229,7 @@ public class TestOrcNullOptimization {
     assertEquals("a", ((StringColumnStatistics) stats[2]).getMinimum());
     assertEquals(20000,
                  ((StringColumnStatistics) stats[2]).getNumberOfValues());
-    assertEquals("count: 20000 min: a max: b",
+    assertEquals("count: 20000 min: a max: b sum: 20000",
         stats[2].toString());
 
     // check the inspectors
@@ -329,7 +329,7 @@ public class TestOrcNullOptimization {
     assertEquals("h", ((StringColumnStatistics) stats[2]).getMaximum());
     assertEquals("a", ((StringColumnStatistics) stats[2]).getMinimum());
     assertEquals(7, ((StringColumnStatistics) stats[2]).getNumberOfValues());
-    assertEquals("count: 7 min: a max: h",
+    assertEquals("count: 7 min: a max: h sum: 7",
         stats[2].toString());
 
     // check the inspectors

Added: hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcSerDeStats.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcSerDeStats.java?rev=1528108&view=auto
==============================================================================
--- hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcSerDeStats.java (added)
+++ hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcSerDeStats.java Tue Oct  1 14:44:00 2013
@@ -0,0 +1,643 @@
+/**
+ * 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 static junit.framework.Assert.assertNotNull;
+import static junit.framework.Assert.assertNull;
+
+import java.io.File;
+import java.sql.Timestamp;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+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.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
+import org.apache.hadoop.hive.serde2.objectinspector.StructField;
+import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.BinaryObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.StringObjectInspector;
+import org.apache.hadoop.io.BytesWritable;
+import org.apache.hadoop.io.Text;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+
+import com.google.common.collect.Lists;
+
+public class TestOrcSerDeStats {
+
+  public static class ListStruct {
+    List<String> list1;
+
+    public ListStruct(List<String> l1) {
+      this.list1 = l1;
+    }
+  }
+
+  public static class MapStruct {
+    Map<String, Double> map1;
+
+    public MapStruct(Map<String, Double> m1) {
+      this.map1 = m1;
+    }
+  }
+
+  public static class SimpleStruct {
+    BytesWritable bytes1;
+    Text string1;
+
+    SimpleStruct(BytesWritable b1, String s1) {
+      this.bytes1 = b1;
+      if (s1 == null) {
+        this.string1 = null;
+      } else {
+        this.string1 = new Text(s1);
+      }
+    }
+  }
+
+  public static class InnerStruct {
+    int int1;
+    Text string1 = new Text();
+
+    InnerStruct(int int1, String string1) {
+      this.int1 = int1;
+      this.string1.set(string1);
+    }
+  }
+
+  public static class MiddleStruct {
+    List<InnerStruct> list = new ArrayList<InnerStruct>();
+
+    MiddleStruct(InnerStruct... items) {
+      list.clear();
+      for (InnerStruct item : items) {
+        list.add(item);
+      }
+    }
+  }
+
+  public static class BigRow {
+    Boolean boolean1;
+    Byte byte1;
+    Short short1;
+    Integer int1;
+    Long long1;
+    Float float1;
+    Double double1;
+    BytesWritable bytes1;
+    Text string1;
+    List<InnerStruct> list = new ArrayList<InnerStruct>();
+    Map<Text, InnerStruct> map = new HashMap<Text, InnerStruct>();
+    Timestamp ts;
+    HiveDecimal decimal1;
+    MiddleStruct middle;
+
+    BigRow(Boolean b1, Byte b2, Short s1, Integer i1, Long l1, Float f1,
+        Double d1,
+        BytesWritable b3, String s2, MiddleStruct m1,
+        List<InnerStruct> l2, Map<Text, InnerStruct> m2, Timestamp ts1,
+        HiveDecimal dec1) {
+      this.boolean1 = b1;
+      this.byte1 = b2;
+      this.short1 = s1;
+      this.int1 = i1;
+      this.long1 = l1;
+      this.float1 = f1;
+      this.double1 = d1;
+      this.bytes1 = b3;
+      if (s2 == null) {
+        this.string1 = null;
+      } else {
+        this.string1 = new Text(s2);
+      }
+      this.middle = m1;
+      this.list = l2;
+      this.map = m2;
+      this.ts = ts1;
+      this.decimal1 = dec1;
+    }
+  }
+
+  private static InnerStruct inner(int i, String s) {
+    return new InnerStruct(i, s);
+  }
+
+  private static Map<Text, InnerStruct> map(InnerStruct... items) {
+    Map<Text, InnerStruct> result = new HashMap<Text, InnerStruct>();
+    for (InnerStruct i : items) {
+      result.put(new Text(i.string1), i);
+    }
+    return result;
+  }
+
+  private static List<InnerStruct> list(InnerStruct... items) {
+    List<InnerStruct> result = new ArrayList<InnerStruct>();
+    for (InnerStruct s : items) {
+      result.add(s);
+    }
+    return result;
+  }
+
+  private static BytesWritable bytes(int... items) {
+    BytesWritable result = new BytesWritable();
+    result.setSize(items.length);
+    for (int i = 0; i < items.length; ++i) {
+      result.getBytes()[i] = (byte) items[i];
+    }
+    return result;
+  }
+
+  Path workDir = new Path(System.getProperty("test.tmp.dir",
+      "target" + File.separator + "test" + File.separator + "tmp"));
+
+  Configuration conf;
+  FileSystem fs;
+  Path testFilePath;
+
+  @Rule
+  public TestName testCaseName = new TestName();
+
+  @Before
+  public void openFileSystem() throws Exception {
+    conf = new Configuration();
+    fs = FileSystem.getLocal(conf);
+    testFilePath = new Path(workDir, "TestOrcSerDeStats." +
+        testCaseName.getMethodName() + ".orc");
+    fs.delete(testFilePath, false);
+  }
+
+  @Test
+  public void testStringAndBinaryStatistics() throws Exception {
+
+    ObjectInspector inspector;
+    synchronized (TestOrcSerDeStats.class) {
+      inspector = ObjectInspectorFactory.getReflectionObjectInspector
+          (SimpleStruct.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
+    }
+    Writer writer = OrcFile.createWriter(testFilePath,
+        OrcFile.writerOptions(conf)
+            .inspector(inspector)
+            .stripeSize(100000)
+            .bufferSize(10000));
+    writer.addRow(new SimpleStruct(bytes(0, 1, 2, 3, 4), "foo"));
+    writer.addRow(new SimpleStruct(bytes(0, 1, 2, 3), "bar"));
+    writer.addRow(new SimpleStruct(bytes(0, 1, 2, 3, 4, 5), null));
+    writer.addRow(new SimpleStruct(null, "hi"));
+    writer.close();
+    assertEquals(4, writer.getNumberOfRows());
+    assertEquals(273, writer.getRawDataSize());
+    Reader reader = OrcFile.createReader(fs, testFilePath);
+    assertEquals(4, reader.getNumberOfRows());
+    assertEquals(273, reader.getRawDataSize());
+    assertEquals(15, reader.getRawDataSizeOfColumns(Lists.newArrayList("bytes1")));
+    assertEquals(258, reader.getRawDataSizeOfColumns(Lists.newArrayList("string1")));
+    assertEquals(273, reader.getRawDataSizeOfColumns(Lists.newArrayList("bytes1", "string1")));
+
+    // check the stats
+    ColumnStatistics[] stats = reader.getStatistics();
+    assertEquals(4, stats[0].getNumberOfValues());
+    assertEquals("count: 4", stats[0].toString());
+
+    assertEquals(3, stats[1].getNumberOfValues());
+    assertEquals(15, ((BinaryColumnStatistics) stats[1]).getSum());
+    assertEquals("count: 3 sum: 15", stats[1].toString());
+
+    assertEquals(3, stats[2].getNumberOfValues());
+    assertEquals("bar", ((StringColumnStatistics) stats[2]).getMinimum());
+    assertEquals("hi", ((StringColumnStatistics) stats[2]).getMaximum());
+    assertEquals(8, ((StringColumnStatistics) stats[2]).getSum());
+    assertEquals("count: 3 min: bar max: hi sum: 8",
+        stats[2].toString());
+
+    // check the inspectors
+    StructObjectInspector readerInspector =
+        (StructObjectInspector) reader.getObjectInspector();
+    assertEquals(ObjectInspector.Category.STRUCT,
+        readerInspector.getCategory());
+    assertEquals("struct<bytes1:binary,string1:string>",
+        readerInspector.getTypeName());
+    List<? extends StructField> fields =
+        readerInspector.getAllStructFieldRefs();
+    BinaryObjectInspector bi = (BinaryObjectInspector) readerInspector.
+        getStructFieldRef("bytes1").getFieldObjectInspector();
+    StringObjectInspector st = (StringObjectInspector) readerInspector.
+        getStructFieldRef("string1").getFieldObjectInspector();
+    RecordReader rows = reader.rows(null);
+    Object row = rows.next(null);
+    assertNotNull(row);
+    // check the contents of the first row
+    assertEquals(bytes(0, 1, 2, 3, 4), bi.getPrimitiveWritableObject(
+        readerInspector.getStructFieldData(row, fields.get(0))));
+    assertEquals("foo", st.getPrimitiveJavaObject(readerInspector.
+        getStructFieldData(row, fields.get(1))));
+
+    // check the contents of second row
+    assertEquals(true, rows.hasNext());
+    row = rows.next(row);
+    assertEquals(bytes(0, 1, 2, 3), bi.getPrimitiveWritableObject(
+        readerInspector.getStructFieldData(row, fields.get(0))));
+    assertEquals("bar", st.getPrimitiveJavaObject(readerInspector.
+        getStructFieldData(row, fields.get(1))));
+
+    // check the contents of second row
+    assertEquals(true, rows.hasNext());
+    row = rows.next(row);
+    assertEquals(bytes(0, 1, 2, 3, 4, 5), bi.getPrimitiveWritableObject(
+        readerInspector.getStructFieldData(row, fields.get(0))));
+    assertNull(st.getPrimitiveJavaObject(readerInspector.
+        getStructFieldData(row, fields.get(1))));
+
+    // check the contents of second row
+    assertEquals(true, rows.hasNext());
+    row = rows.next(row);
+    assertNull(bi.getPrimitiveWritableObject(
+        readerInspector.getStructFieldData(row, fields.get(0))));
+    assertEquals("hi", st.getPrimitiveJavaObject(readerInspector.
+        getStructFieldData(row, fields.get(1))));
+
+    // handle the close up
+    assertEquals(false, rows.hasNext());
+    rows.close();
+  }
+
+
+  @Test
+  public void testOrcSerDeStatsList() throws Exception {
+    ObjectInspector inspector;
+    synchronized (TestOrcSerDeStats.class) {
+      inspector = ObjectInspectorFactory.getReflectionObjectInspector
+          (ListStruct.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
+    }
+    Writer writer = OrcFile.createWriter(testFilePath,
+        OrcFile.writerOptions(conf)
+            .inspector(inspector)
+            .stripeSize(10000)
+            .bufferSize(10000));
+    for (int row = 0; row < 5000; row++) {
+      List<String> test = new ArrayList<String>();
+      for (int i = 0; i < 1000; i++) {
+        test.add("hi");
+      }
+      writer.addRow(new ListStruct(test));
+    }
+    writer.close();
+    assertEquals(5000, writer.getNumberOfRows());
+    assertEquals(430000000, writer.getRawDataSize());
+
+    Reader reader = OrcFile.createReader(fs, testFilePath);
+    // stats from reader
+    assertEquals(5000, reader.getNumberOfRows());
+    assertEquals(430000000, reader.getRawDataSize());
+    assertEquals(430000000, reader.getRawDataSizeOfColumns(Lists.newArrayList("list1")));
+  }
+
+  @Test
+  public void testOrcSerDeStatsMap() throws Exception {
+    ObjectInspector inspector;
+    synchronized (TestOrcSerDeStats.class) {
+      inspector = ObjectInspectorFactory.getReflectionObjectInspector
+          (MapStruct.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
+    }
+    Writer writer = OrcFile.createWriter(testFilePath,
+        OrcFile.writerOptions(conf)
+            .inspector(inspector)
+            .stripeSize(10000)
+            .bufferSize(10000));
+    for (int row = 0; row < 1000; row++) {
+      Map<String, Double> test = new HashMap<String, Double>();
+      for (int i = 0; i < 10; i++) {
+        test.put("hi" + i, 2.0);
+      }
+      writer.addRow(new MapStruct(test));
+    }
+    writer.close();
+    // stats from writer
+    assertEquals(1000, writer.getNumberOfRows());
+    assertEquals(950000, writer.getRawDataSize());
+
+    Reader reader = OrcFile.createReader(fs, testFilePath);
+    // stats from reader
+    assertEquals(1000, reader.getNumberOfRows());
+    assertEquals(950000, reader.getRawDataSize());
+    assertEquals(950000, reader.getRawDataSizeOfColumns(Lists.newArrayList("map1")));
+  }
+
+  @Test
+  public void testOrcSerDeStatsSimpleWithNulls() throws Exception {
+    ObjectInspector inspector;
+    synchronized (TestOrcSerDeStats.class) {
+      inspector = ObjectInspectorFactory.getReflectionObjectInspector
+          (SimpleStruct.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
+    }
+    Writer writer = OrcFile.createWriter(testFilePath,
+        OrcFile.writerOptions(conf)
+            .inspector(inspector)
+            .stripeSize(10000)
+            .bufferSize(10000));
+    for (int row = 0; row < 1000; row++) {
+      if (row % 2 == 0) {
+        writer.addRow(new SimpleStruct(new BytesWritable(new byte[] {1, 2, 3}), "hi"));
+      } else {
+        writer.addRow(null);
+      }
+    }
+    writer.close();
+    // stats from writer
+    assertEquals(1000, writer.getNumberOfRows());
+    assertEquals(44500, writer.getRawDataSize());
+
+    Reader reader = OrcFile.createReader(fs, testFilePath);
+    // stats from reader
+    assertEquals(1000, reader.getNumberOfRows());
+    assertEquals(44500, reader.getRawDataSize());
+    assertEquals(1500, reader.getRawDataSizeOfColumns(Lists.newArrayList("bytes1")));
+    assertEquals(43000, reader.getRawDataSizeOfColumns(Lists.newArrayList("string1")));
+    assertEquals(44500, reader.getRawDataSizeOfColumns(Lists.newArrayList("bytes1", "string1")));
+  }
+
+  @Test
+  public void testOrcSerDeStatsComplex() throws Exception {
+    ObjectInspector inspector;
+    synchronized (TestOrcSerDeStats.class) {
+      inspector = ObjectInspectorFactory.getReflectionObjectInspector
+          (BigRow.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
+    }
+    Writer writer = OrcFile.createWriter(testFilePath,
+        OrcFile.writerOptions(conf)
+            .inspector(inspector)
+            .stripeSize(100000)
+            .bufferSize(10000));
+    // 1 + 2 + 4 + 8 + 4 + 8 + 5 + 2 + 4 + 3 + 4 + 4 + 4 + 4 + 4 + 3 = 64
+    writer.addRow(new BigRow(false, (byte) 1, (short) 1024, 65536,
+        Long.MAX_VALUE, (float) 1.0, -15.0, bytes(0, 1, 2, 3, 4), "hi",
+        new MiddleStruct(inner(1, "bye"), inner(2, "sigh")),
+        list(inner(3, "good"), inner(4, "bad")),
+        map(), Timestamp.valueOf("2000-03-12 15:00:00"), new HiveDecimal(
+            "12345678.6547456")));
+    // 1 + 2 + 4 + 8 + 4 + 8 + 3 + 4 + 3 + 4 + 4 + 4 + 3 + 4 + 2 + 4 + 3 + 5 + 4 + 5 + 7 + 4 + 7 =
+    // 97
+    writer.addRow(new BigRow(true, (byte) 100, (short) 2048, 65536,
+        Long.MAX_VALUE, (float) 2.0, -5.0, bytes(), "bye",
+        new MiddleStruct(inner(1, "bye"), inner(2, "sigh")),
+        list(inner(100000000, "cat"), inner(-100000, "in"), inner(1234, "hat")),
+        map(inner(5, "chani"), inner(1, "mauddib")), Timestamp.valueOf("2000-03-11 15:00:00"),
+        new HiveDecimal("12345678.6547452")));
+    writer.close();
+    long rowCount = writer.getNumberOfRows();
+    long rawDataSize = writer.getRawDataSize();
+    assertEquals(2, rowCount);
+    assertEquals(1740, rawDataSize);
+    Reader reader = OrcFile.createReader(fs, testFilePath);
+
+    assertEquals(2, reader.getNumberOfRows());
+    assertEquals(1740, reader.getRawDataSize());
+    assertEquals(8, reader.getRawDataSizeOfColumns(Lists.newArrayList("boolean1")));
+    assertEquals(8, reader.getRawDataSizeOfColumns(Lists.newArrayList("byte1")));
+    assertEquals(8, reader.getRawDataSizeOfColumns(Lists.newArrayList("short1")));
+    assertEquals(8, reader.getRawDataSizeOfColumns(Lists.newArrayList("int1")));
+    assertEquals(16, reader.getRawDataSizeOfColumns(Lists.newArrayList("long1")));
+    assertEquals(8, reader.getRawDataSizeOfColumns(Lists.newArrayList("float1")));
+    assertEquals(16, reader.getRawDataSizeOfColumns(Lists.newArrayList("double1")));
+    assertEquals(5, reader.getRawDataSizeOfColumns(Lists.newArrayList("bytes1")));
+    assertEquals(172, reader.getRawDataSizeOfColumns(Lists.newArrayList("string1")));
+    assertEquals(455, reader.getRawDataSizeOfColumns(Lists.newArrayList("list")));
+    assertEquals(368, reader.getRawDataSizeOfColumns(Lists.newArrayList("map")));
+    assertEquals(364, reader.getRawDataSizeOfColumns(Lists.newArrayList("middle")));
+    assertEquals(80, reader.getRawDataSizeOfColumns(Lists.newArrayList("ts")));
+    assertEquals(224, reader.getRawDataSizeOfColumns(Lists.newArrayList("decimal1")));
+    assertEquals(88, reader.getRawDataSizeOfColumns(Lists.newArrayList("ts", "int1")));
+    assertEquals(1195,
+        reader.getRawDataSizeOfColumns(Lists.newArrayList("middle", "list", "map", "float1")));
+    assertEquals(185,
+        reader.getRawDataSizeOfColumns(Lists.newArrayList("bytes1", "byte1", "string1")));
+    assertEquals(rawDataSize, reader.getRawDataSizeOfColumns(Lists.newArrayList("boolean1",
+        "byte1", "short1", "int1", "long1", "float1", "double1", "bytes1", "string1", "list",
+        "map", "middle", "ts", "decimal1")));
+
+
+    // check the stats
+    ColumnStatistics[] stats = reader.getStatistics();
+    assertEquals(2, stats[1].getNumberOfValues());
+    assertEquals(1, ((BooleanColumnStatistics) stats[1]).getFalseCount());
+    assertEquals(1, ((BooleanColumnStatistics) stats[1]).getTrueCount());
+    assertEquals("count: 2 true: 1", stats[1].toString());
+
+    assertEquals(2048, ((IntegerColumnStatistics) stats[3]).getMaximum());
+    assertEquals(1024, ((IntegerColumnStatistics) stats[3]).getMinimum());
+    assertEquals(true, ((IntegerColumnStatistics) stats[3]).isSumDefined());
+    assertEquals(3072, ((IntegerColumnStatistics) stats[3]).getSum());
+    assertEquals("count: 2 min: 1024 max: 2048 sum: 3072",
+        stats[3].toString());
+
+    assertEquals(Long.MAX_VALUE,
+        ((IntegerColumnStatistics) stats[5]).getMaximum());
+    assertEquals(Long.MAX_VALUE,
+        ((IntegerColumnStatistics) stats[5]).getMinimum());
+    assertEquals(false, ((IntegerColumnStatistics) stats[5]).isSumDefined());
+    assertEquals("count: 2 min: 9223372036854775807 max: 9223372036854775807",
+        stats[5].toString());
+
+    assertEquals(-15.0, ((DoubleColumnStatistics) stats[7]).getMinimum());
+    assertEquals(-5.0, ((DoubleColumnStatistics) stats[7]).getMaximum());
+    assertEquals(-20.0, ((DoubleColumnStatistics) stats[7]).getSum(), 0.00001);
+    assertEquals("count: 2 min: -15.0 max: -5.0 sum: -20.0",
+        stats[7].toString());
+
+    assertEquals("count: 2 min: bye max: hi sum: 5", stats[9].toString());
+  }
+
+  @Test
+  public void testOrcSerDeStatsComplexOldFormat() throws Exception {
+    ObjectInspector inspector;
+    synchronized (TestOrcSerDeStats.class) {
+      inspector = ObjectInspectorFactory.getReflectionObjectInspector
+          (BigRow.class, ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
+    }
+
+    Writer writer = OrcFile.createWriter(testFilePath,
+        OrcFile.writerOptions(conf)
+            .inspector(inspector)
+            .stripeSize(100000)
+            .version(OrcFile.Version.V_0_11)
+            .bufferSize(10000));
+    // 1 + 2 + 4 + 8 + 4 + 8 + 5 + 2 + 4 + 3 + 4 + 4 + 4 + 4 + 4 + 3 = 64
+    writer.addRow(new BigRow(false, (byte) 1, (short) 1024, 65536,
+        Long.MAX_VALUE, (float) 1.0, -15.0, bytes(0, 1, 2, 3, 4), "hi",
+        new MiddleStruct(inner(1, "bye"), inner(2, "sigh")),
+        list(inner(3, "good"), inner(4, "bad")),
+        map(), Timestamp.valueOf("2000-03-12 15:00:00"), new HiveDecimal(
+            "12345678.6547456")));
+    // 1 + 2 + 4 + 8 + 4 + 8 + 3 + 4 + 3 + 4 + 4 + 4 + 3 + 4 + 2 + 4 + 3 + 5 + 4 + 5 + 7 + 4 + 7 =
+    // 97
+    writer.addRow(new BigRow(true, (byte) 100, (short) 2048, 65536,
+        Long.MAX_VALUE, (float) 2.0, -5.0, bytes(), "bye",
+        new MiddleStruct(inner(1, "bye"), inner(2, "sigh")),
+        list(inner(100000000, "cat"), inner(-100000, "in"), inner(1234, "hat")),
+        map(inner(5, "chani"), inner(1, "mauddib")), Timestamp.valueOf("2000-03-11 15:00:00"),
+        new HiveDecimal("12345678.6547452")));
+    writer.close();
+    long rowCount = writer.getNumberOfRows();
+    long rawDataSize = writer.getRawDataSize();
+    assertEquals(2, rowCount);
+    assertEquals(1740, rawDataSize);
+    Reader reader = OrcFile.createReader(fs, testFilePath);
+
+    assertEquals(2, reader.getNumberOfRows());
+    assertEquals(1740, reader.getRawDataSize());
+    assertEquals(8, reader.getRawDataSizeOfColumns(Lists.newArrayList("boolean1")));
+    assertEquals(8, reader.getRawDataSizeOfColumns(Lists.newArrayList("byte1")));
+    assertEquals(8, reader.getRawDataSizeOfColumns(Lists.newArrayList("short1")));
+    assertEquals(8, reader.getRawDataSizeOfColumns(Lists.newArrayList("int1")));
+    assertEquals(16, reader.getRawDataSizeOfColumns(Lists.newArrayList("long1")));
+    assertEquals(8, reader.getRawDataSizeOfColumns(Lists.newArrayList("float1")));
+    assertEquals(16, reader.getRawDataSizeOfColumns(Lists.newArrayList("double1")));
+    assertEquals(5, reader.getRawDataSizeOfColumns(Lists.newArrayList("bytes1")));
+    assertEquals(172, reader.getRawDataSizeOfColumns(Lists.newArrayList("string1")));
+    assertEquals(455, reader.getRawDataSizeOfColumns(Lists.newArrayList("list")));
+    assertEquals(368, reader.getRawDataSizeOfColumns(Lists.newArrayList("map")));
+    assertEquals(364, reader.getRawDataSizeOfColumns(Lists.newArrayList("middle")));
+    assertEquals(80, reader.getRawDataSizeOfColumns(Lists.newArrayList("ts")));
+    assertEquals(224, reader.getRawDataSizeOfColumns(Lists.newArrayList("decimal1")));
+    assertEquals(88, reader.getRawDataSizeOfColumns(Lists.newArrayList("ts", "int1")));
+    assertEquals(1195,
+        reader.getRawDataSizeOfColumns(Lists.newArrayList("middle", "list", "map", "float1")));
+    assertEquals(185,
+        reader.getRawDataSizeOfColumns(Lists.newArrayList("bytes1", "byte1", "string1")));
+    assertEquals(rawDataSize, reader.getRawDataSizeOfColumns(Lists.newArrayList("boolean1",
+        "byte1", "short1", "int1", "long1", "float1", "double1", "bytes1", "string1", "list",
+        "map", "middle", "ts", "decimal1")));
+
+    // check the stats
+    ColumnStatistics[] stats = reader.getStatistics();
+    assertEquals(2, stats[1].getNumberOfValues());
+    assertEquals(1, ((BooleanColumnStatistics) stats[1]).getFalseCount());
+    assertEquals(1, ((BooleanColumnStatistics) stats[1]).getTrueCount());
+    assertEquals("count: 2 true: 1", stats[1].toString());
+
+    assertEquals(2048, ((IntegerColumnStatistics) stats[3]).getMaximum());
+    assertEquals(1024, ((IntegerColumnStatistics) stats[3]).getMinimum());
+    assertEquals(true, ((IntegerColumnStatistics) stats[3]).isSumDefined());
+    assertEquals(3072, ((IntegerColumnStatistics) stats[3]).getSum());
+    assertEquals("count: 2 min: 1024 max: 2048 sum: 3072",
+        stats[3].toString());
+
+    assertEquals(Long.MAX_VALUE,
+        ((IntegerColumnStatistics) stats[5]).getMaximum());
+    assertEquals(Long.MAX_VALUE,
+        ((IntegerColumnStatistics) stats[5]).getMinimum());
+    assertEquals(false, ((IntegerColumnStatistics) stats[5]).isSumDefined());
+    assertEquals("count: 2 min: 9223372036854775807 max: 9223372036854775807",
+        stats[5].toString());
+
+    assertEquals(-15.0, ((DoubleColumnStatistics) stats[7]).getMinimum());
+    assertEquals(-5.0, ((DoubleColumnStatistics) stats[7]).getMaximum());
+    assertEquals(-20.0, ((DoubleColumnStatistics) stats[7]).getSum(), 0.00001);
+    assertEquals("count: 2 min: -15.0 max: -5.0 sum: -20.0",
+        stats[7].toString());
+
+    assertEquals(5, ((BinaryColumnStatistics) stats[8]).getSum());
+    assertEquals("count: 2 sum: 5", stats[8].toString());
+
+    assertEquals("bye", ((StringColumnStatistics) stats[9]).getMinimum());
+    assertEquals("hi", ((StringColumnStatistics) stats[9]).getMaximum());
+    assertEquals(5, ((StringColumnStatistics) stats[9]).getSum());
+    assertEquals("count: 2 min: bye max: hi sum: 5", stats[9].toString());
+  }
+
+  @Test(expected = ClassCastException.class)
+  public void testSerdeStatsOldFormat() throws Exception {
+    Path resourceDir = new Path(System.getProperty("test.build.resources", "ql"
+        + File.separator + "src" + File.separator + "test" + File.separator
+        + "resources"));
+    Path oldFilePath = new Path(resourceDir, "orc-file-11-format.orc");
+    Reader reader = OrcFile.createReader(fs, oldFilePath);
+
+    int stripeCount = 0;
+    int rowCount = 0;
+    long currentOffset = -1;
+    for (StripeInformation stripe : reader.getStripes()) {
+      stripeCount += 1;
+      rowCount += stripe.getNumberOfRows();
+      if (currentOffset < 0) {
+        currentOffset = stripe.getOffset() + stripe.getIndexLength()
+            + stripe.getDataLength() + stripe.getFooterLength();
+      } else {
+        assertEquals(currentOffset, stripe.getOffset());
+        currentOffset += stripe.getIndexLength() + stripe.getDataLength()
+            + stripe.getFooterLength();
+      }
+    }
+    assertEquals(reader.getNumberOfRows(), rowCount);
+    assertEquals(6300000, reader.getRawDataSize());
+    assertEquals(2, stripeCount);
+
+    // check the stats
+    ColumnStatistics[] stats = reader.getStatistics();
+    assertEquals(7500, stats[1].getNumberOfValues());
+    assertEquals(3750, ((BooleanColumnStatistics) stats[1]).getFalseCount());
+    assertEquals(3750, ((BooleanColumnStatistics) stats[1]).getTrueCount());
+    assertEquals("count: 7500 true: 3750", stats[1].toString());
+
+    assertEquals(2048, ((IntegerColumnStatistics) stats[3]).getMaximum());
+    assertEquals(1024, ((IntegerColumnStatistics) stats[3]).getMinimum());
+    assertEquals(true, ((IntegerColumnStatistics) stats[3]).isSumDefined());
+    assertEquals(11520000, ((IntegerColumnStatistics) stats[3]).getSum());
+    assertEquals("count: 7500 min: 1024 max: 2048 sum: 11520000",
+        stats[3].toString());
+
+    assertEquals(Long.MAX_VALUE,
+        ((IntegerColumnStatistics) stats[5]).getMaximum());
+    assertEquals(Long.MAX_VALUE,
+        ((IntegerColumnStatistics) stats[5]).getMinimum());
+    assertEquals(false, ((IntegerColumnStatistics) stats[5]).isSumDefined());
+    assertEquals(
+        "count: 7500 min: 9223372036854775807 max: 9223372036854775807",
+        stats[5].toString());
+
+    assertEquals(-15.0, ((DoubleColumnStatistics) stats[7]).getMinimum());
+    assertEquals(-5.0, ((DoubleColumnStatistics) stats[7]).getMaximum());
+    assertEquals(-75000.0, ((DoubleColumnStatistics) stats[7]).getSum(),
+        0.00001);
+    assertEquals("count: 7500 min: -15.0 max: -5.0 sum: -75000.0",
+        stats[7].toString());
+
+    assertEquals("bye", ((StringColumnStatistics) stats[9]).getMinimum());
+    assertEquals("hi", ((StringColumnStatistics) stats[9]).getMaximum());
+    assertEquals(0, ((StringColumnStatistics) stats[9]).getSum());
+    assertEquals("count: 7500 min: bye max: hi sum: 0", stats[9].toString());
+
+    // old orc format will not have binary statistics. toString() will show only
+    // the general column statistics
+    assertEquals("count: 7500", stats[8].toString());
+    // since old orc format doesn't support binary statistics,
+    // this should throw ClassCastException
+    assertEquals(5, ((BinaryColumnStatistics) stats[8]).getSum());
+
+  }
+
+}

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=1528108&r1=1528107&r2=1528108&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 Tue Oct  1 14:44:00 2013
@@ -8,71 +8,71 @@ Statistics:
   Column 0: count: 21000
   Column 1: count: 21000 min: -2147390285 max: 2147453086 sum: 109128518326
   Column 2: count: 21000 min: -9222731174895935707 max: 9222919052987871506
-  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
+  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 sum: 6910238
 
 Stripes:
-  Stripe: offset: 3 data: 102311 rows: 4000 tail: 68 index: 217
+  Stripe: offset: 3 data: 102311 rows: 4000 tail: 68 index: 224
     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 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
+    Stream: column 3 section ROW_INDEX start: 88 length 139
+    Stream: column 1 section DATA start: 227 length 16022
+    Stream: column 2 section DATA start: 16249 length 32028
+    Stream: column 3 section DATA start: 48277 length 50887
+    Stream: column 3 section LENGTH start: 99164 length 3374
     Encoding column 0: DIRECT
     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
+  Stripe: offset: 102606 data: 284999 rows: 5000 tail: 68 index: 356
+    Stream: column 0 section ROW_INDEX start: 102606 length 10
+    Stream: column 1 section ROW_INDEX start: 102616 length 36
+    Stream: column 2 section ROW_INDEX start: 102652 length 39
+    Stream: column 3 section ROW_INDEX start: 102691 length 271
+    Stream: column 1 section DATA start: 102962 length 20029
+    Stream: column 2 section DATA start: 122991 length 40035
+    Stream: column 3 section DATA start: 163026 length 219588
+    Stream: column 3 section LENGTH start: 382614 length 5347
     Encoding column 0: DIRECT
     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
+  Stripe: offset: 388029 data: 491655 rows: 5000 tail: 69 index: 544
+    Stream: column 0 section ROW_INDEX start: 388029 length 10
+    Stream: column 1 section ROW_INDEX start: 388039 length 36
+    Stream: column 2 section ROW_INDEX start: 388075 length 39
+    Stream: column 3 section ROW_INDEX start: 388114 length 459
+    Stream: column 1 section DATA start: 388573 length 20029
+    Stream: column 2 section DATA start: 408602 length 40035
+    Stream: column 3 section DATA start: 448637 length 425862
+    Stream: column 3 section LENGTH start: 874499 length 5729
     Encoding column 0: DIRECT
     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
+  Stripe: offset: 880297 data: 707368 rows: 5000 tail: 68 index: 691
+    Stream: column 0 section ROW_INDEX start: 880297 length 10
+    Stream: column 1 section ROW_INDEX start: 880307 length 36
+    Stream: column 2 section ROW_INDEX start: 880343 length 39
+    Stream: column 3 section ROW_INDEX start: 880382 length 606
+    Stream: column 1 section DATA start: 880988 length 20029
+    Stream: column 2 section DATA start: 901017 length 40035
+    Stream: column 3 section DATA start: 941052 length 641580
+    Stream: column 3 section LENGTH start: 1582632 length 5724
     Encoding column 0: DIRECT
     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
+  Stripe: offset: 1588424 data: 348697 rows: 2000 tail: 67 index: 797
+    Stream: column 0 section ROW_INDEX start: 1588424 length 10
+    Stream: column 1 section ROW_INDEX start: 1588434 length 36
+    Stream: column 2 section ROW_INDEX start: 1588470 length 39
+    Stream: column 3 section ROW_INDEX start: 1588509 length 712
+    Stream: column 1 section DATA start: 1589221 length 8011
+    Stream: column 2 section DATA start: 1597232 length 16014
+    Stream: column 3 section DATA start: 1613246 length 322259
+    Stream: column 3 section LENGTH start: 1935505 length 2413
     Encoding column 0: DIRECT
     Encoding column 1: DIRECT_V2
     Encoding column 2: DIRECT_V2
-    Encoding column 3: DIRECT_V2
\ No newline at end of file
+    Encoding column 3: DIRECT_V2

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=1528108&r1=1528107&r2=1528108&view=diff
==============================================================================
--- hive/trunk/ql/src/test/resources/orc-file-dump.out (original)
+++ hive/trunk/ql/src/test/resources/orc-file-dump.out Tue Oct  1 14:44:00 2013
@@ -8,75 +8,75 @@ Statistics:
   Column 0: count: 21000
   Column 1: count: 21000 min: -2146993718 max: 2147378179 sum: 193017464403
   Column 2: count: 21000 min: -9222758097219661129 max: 9222303228623055266
-  Column 3: count: 21000 min: Darkness, max: worst
+  Column 3: count: 21000 min: Darkness, max: worst sum: 81761
 
 Stripes:
-  Stripe: offset: 3 data: 63766 rows: 5000 tail: 74 index: 119
+  Stripe: offset: 3 data: 63766 rows: 5000 tail: 74 index: 123
     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 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
+    Stream: column 3 section ROW_INDEX start: 87 length 39
+    Stream: column 1 section DATA start: 126 length 20029
+    Stream: column 2 section DATA start: 20155 length 40035
+    Stream: column 3 section DATA start: 60190 length 3544
+    Stream: column 3 section LENGTH start: 63734 length 25
+    Stream: column 3 section DICTIONARY_DATA start: 63759 length 133
     Encoding column 0: DIRECT
     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
+  Stripe: offset: 63966 data: 63755 rows: 5000 tail: 74 index: 122
+    Stream: column 0 section ROW_INDEX start: 63966 length 10
+    Stream: column 1 section ROW_INDEX start: 63976 length 34
+    Stream: column 2 section ROW_INDEX start: 64010 length 39
+    Stream: column 3 section ROW_INDEX start: 64049 length 39
+    Stream: column 1 section DATA start: 64088 length 20029
+    Stream: column 2 section DATA start: 84117 length 40035
+    Stream: column 3 section DATA start: 124152 length 3533
+    Stream: column 3 section LENGTH start: 127685 length 25
+    Stream: column 3 section DICTIONARY_DATA start: 127710 length 133
     Encoding column 0: DIRECT
     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
+  Stripe: offset: 127917 data: 63766 rows: 5000 tail: 74 index: 124
+    Stream: column 0 section ROW_INDEX start: 127917 length 10
+    Stream: column 1 section ROW_INDEX start: 127927 length 36
+    Stream: column 2 section ROW_INDEX start: 127963 length 39
+    Stream: column 3 section ROW_INDEX start: 128002 length 39
+    Stream: column 1 section DATA start: 128041 length 20029
+    Stream: column 2 section DATA start: 148070 length 40035
+    Stream: column 3 section DATA start: 188105 length 3544
+    Stream: column 3 section LENGTH start: 191649 length 25
+    Stream: column 3 section DICTIONARY_DATA start: 191674 length 133
     Encoding column 0: DIRECT
     Encoding column 1: DIRECT_V2
     Encoding column 2: DIRECT_V2
     Encoding column 3: DICTIONARY_V2
-  Stripe: offset: 200000 data: 63796 rows: 5000 tail: 74 index: 119
+  Stripe: offset: 200000 data: 63796 rows: 5000 tail: 74 index: 123
     Stream: column 0 section ROW_INDEX start: 200000 length 10
     Stream: column 1 section ROW_INDEX start: 200010 length 35
     Stream: column 2 section ROW_INDEX start: 200045 length 39
-    Stream: column 3 section ROW_INDEX start: 200084 length 35
-    Stream: column 1 section DATA start: 200119 length 20029
-    Stream: column 2 section DATA start: 220148 length 40035
-    Stream: column 3 section DATA start: 260183 length 3574
-    Stream: column 3 section LENGTH start: 263757 length 25
-    Stream: column 3 section DICTIONARY_DATA start: 263782 length 133
+    Stream: column 3 section ROW_INDEX start: 200084 length 39
+    Stream: column 1 section DATA start: 200123 length 20029
+    Stream: column 2 section DATA start: 220152 length 40035
+    Stream: column 3 section DATA start: 260187 length 3574
+    Stream: column 3 section LENGTH start: 263761 length 25
+    Stream: column 3 section DICTIONARY_DATA start: 263786 length 133
     Encoding column 0: DIRECT
     Encoding column 1: DIRECT_V2
     Encoding column 2: DIRECT_V2
     Encoding column 3: DICTIONARY_V2
-  Stripe: offset: 263989 data: 12940 rows: 1000 tail: 71 index: 120
-    Stream: column 0 section ROW_INDEX start: 263989 length 10
-    Stream: column 1 section ROW_INDEX start: 263999 length 36
-    Stream: column 2 section ROW_INDEX start: 264035 length 39
-    Stream: column 3 section ROW_INDEX start: 264074 length 35
-    Stream: column 1 section DATA start: 264109 length 4007
-    Stream: column 2 section DATA start: 268116 length 8007
-    Stream: column 3 section DATA start: 276123 length 768
-    Stream: column 3 section LENGTH start: 276891 length 25
-    Stream: column 3 section DICTIONARY_DATA start: 276916 length 133
+  Stripe: offset: 263993 data: 12940 rows: 1000 tail: 71 index: 123
+    Stream: column 0 section ROW_INDEX start: 263993 length 10
+    Stream: column 1 section ROW_INDEX start: 264003 length 36
+    Stream: column 2 section ROW_INDEX start: 264039 length 39
+    Stream: column 3 section ROW_INDEX start: 264078 length 38
+    Stream: column 1 section DATA start: 264116 length 4007
+    Stream: column 2 section DATA start: 268123 length 8007
+    Stream: column 3 section DATA start: 276130 length 768
+    Stream: column 3 section LENGTH start: 276898 length 25
+    Stream: column 3 section DICTIONARY_DATA start: 276923 length 133
     Encoding column 0: DIRECT
     Encoding column 1: DIRECT_V2
     Encoding column 2: DIRECT_V2