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 2015/10/01 13:12:37 UTC

[3/4] hive git commit: HIVE-4243. Fix column names in ORC metadata.

http://git-wip-us.apache.org/repos/asf/hive/blob/7b1ed3d3/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestInputOutputFormat.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestInputOutputFormat.java b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestInputOutputFormat.java
index f451fce..69cb6ff 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestInputOutputFormat.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestInputOutputFormat.java
@@ -1106,6 +1106,8 @@ public class TestInputOutputFormat {
   @SuppressWarnings("unchecked,deprecation")
   public void testInOutFormat() throws Exception {
     Properties properties = new Properties();
+    properties.setProperty("columns", "x,y");
+    properties.setProperty("columns.types", "int:int");
     StructObjectInspector inspector;
     synchronized (TestOrcFile.class) {
       inspector = (StructObjectInspector)
@@ -1122,8 +1124,6 @@ public class TestInputOutputFormat {
     writer.write(serde.serialize(new MyRow(3,2), inspector));
     writer.close(true);
     serde = new OrcSerde();
-    properties.setProperty("columns", "x,y");
-    properties.setProperty("columns.types", "int:int");
     SerDeUtils.initializeSerDe(serde, conf, properties, null);
     assertEquals(OrcSerde.OrcSerdeRow.class, serde.getSerializedClass());
     inspector = (StructObjectInspector) serde.getObjectInspector();
@@ -1295,13 +1295,13 @@ public class TestInputOutputFormat {
   @SuppressWarnings("deprecation")
   public void testEmptyFile() throws Exception {
     Properties properties = new Properties();
+    properties.setProperty("columns", "x,y");
+    properties.setProperty("columns.types", "int:int");
     HiveOutputFormat<?, ?> outFormat = new OrcOutputFormat();
     org.apache.hadoop.hive.ql.exec.FileSinkOperator.RecordWriter writer =
         outFormat.getHiveRecordWriter(conf, testFilePath, MyRow.class, true,
             properties, Reporter.NULL);
     writer.close(true);
-    properties.setProperty("columns", "x,y");
-    properties.setProperty("columns.types", "int:int");
     SerDe serde = new OrcSerde();
     SerDeUtils.initializeSerDe(serde, conf, properties, null);
     InputFormat<?,?> in = new OrcInputFormat();
@@ -1352,6 +1352,8 @@ public class TestInputOutputFormat {
   @SuppressWarnings("unchecked,deprecation")
   public void testDefaultTypes() throws Exception {
     Properties properties = new Properties();
+    properties.setProperty("columns", "str,str2");
+    properties.setProperty("columns.types", "string:string");
     StructObjectInspector inspector;
     synchronized (TestOrcFile.class) {
       inspector = (StructObjectInspector)
@@ -1371,7 +1373,6 @@ public class TestInputOutputFormat {
     writer.write(serde.serialize(new StringRow("miles"), inspector));
     writer.close(true);
     serde = new OrcSerde();
-    properties.setProperty("columns", "str,str2");
     SerDeUtils.initializeSerDe(serde, conf, properties, null);
     inspector = (StructObjectInspector) serde.getObjectInspector();
     assertEquals("struct<str:string,str2:string>", inspector.getTypeName());
@@ -1892,6 +1893,8 @@ public class TestInputOutputFormat {
   @SuppressWarnings("unchecked,deprecation")
   public void testSplitElimination() throws Exception {
     Properties properties = new Properties();
+    properties.setProperty("columns", "z,r");
+    properties.setProperty("columns.types", "int:struct<x:int,y:int>");
     StructObjectInspector inspector;
     synchronized (TestOrcFile.class) {
       inspector = (StructObjectInspector)
@@ -1920,8 +1923,6 @@ public class TestInputOutputFormat {
             .build();
     conf.set("sarg.pushdown", toKryo(sarg));
     conf.set("hive.io.file.readcolumn.names", "z,r");
-    properties.setProperty("columns", "z,r");
-    properties.setProperty("columns.types", "int:struct<x:int,y:int>");
     SerDeUtils.initializeSerDe(serde, conf, properties, null);
     inspector = (StructObjectInspector) serde.getObjectInspector();
     InputFormat<?,?> in = new OrcInputFormat();

http://git-wip-us.apache.org/repos/asf/hive/blob/7b1ed3d3/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcFile.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcFile.java b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcFile.java
index 0bb8401..06e3362 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcFile.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcFile.java
@@ -519,9 +519,9 @@ public class TestOrcFile {
       Object row = rows.next(null);
       assertEquals(tslist.get(idx++).getNanos(), ((TimestampWritable) row).getNanos());
     }
-    assertEquals(1, OrcUtils.getFlattenedColumnsCount(inspector));
+    assertEquals(0, writer.getSchema().getMaximumId());
     boolean[] expected = new boolean[] {false};
-    boolean[] included = OrcUtils.includeColumns("", "ts", inspector);
+    boolean[] included = OrcUtils.includeColumns("", writer.getSchema());
     assertEquals(true, Arrays.equals(expected, included));
   }
 
@@ -546,17 +546,18 @@ public class TestOrcFile {
     Reader reader = OrcFile.createReader(testFilePath,
         OrcFile.readerOptions(conf).filesystem(fs));
 
-    assertEquals(3, OrcUtils.getFlattenedColumnsCount(inspector));
+    TypeDescription schema = writer.getSchema();
+    assertEquals(2, schema.getMaximumId());
     boolean[] expected = new boolean[] {false, false, true};
-    boolean[] included = OrcUtils.includeColumns("string1", "bytes1,string1", inspector);
+    boolean[] included = OrcUtils.includeColumns("string1", schema);
     assertEquals(true, Arrays.equals(expected, included));
 
     expected = new boolean[] {false, false, false};
-    included = OrcUtils.includeColumns("", "bytes1,string1", inspector);
+    included = OrcUtils.includeColumns("", schema);
     assertEquals(true, Arrays.equals(expected, included));
 
     expected = new boolean[] {false, false, false};
-    included = OrcUtils.includeColumns(null, "bytes1,string1", inspector);
+    included = OrcUtils.includeColumns(null, schema);
     assertEquals(true, Arrays.equals(expected, included));
 
     // check the stats
@@ -656,9 +657,10 @@ public class TestOrcFile {
     Reader reader = OrcFile.createReader(testFilePath,
         OrcFile.readerOptions(conf).filesystem(fs));
 
-    assertEquals(3, OrcUtils.getFlattenedColumnsCount(inspector));
+    TypeDescription schema = writer.getSchema();
+    assertEquals(2, schema.getMaximumId());
     boolean[] expected = new boolean[] {false, true, false};
-    boolean[] included = OrcUtils.includeColumns("int1", "int1,string1", inspector);
+    boolean[] included = OrcUtils.includeColumns("int1", schema);
     assertEquals(true, Arrays.equals(expected, included));
 
     Metadata metadata = reader.getMetadata();
@@ -742,14 +744,14 @@ public class TestOrcFile {
     Reader reader = OrcFile.createReader(testFilePath,
         OrcFile.readerOptions(conf).filesystem(fs));
 
-    assertEquals(24, OrcUtils.getFlattenedColumnsCount(inspector));
+    TypeDescription schema = writer.getSchema();
+    assertEquals(23, schema.getMaximumId());
     boolean[] expected = new boolean[] {false, false, false, false, false,
         false, false, false, false, false,
         false, false, false, false, false,
         false, false, false, false, false,
         false, false, false, false};
-    boolean[] included = OrcUtils.includeColumns("",
-        "boolean1,byte1,short1,int1,long1,float1,double1,bytes1,string1,middle,list,map", inspector);
+    boolean[] included = OrcUtils.includeColumns("", schema);
     assertEquals(true, Arrays.equals(expected, included));
 
     expected = new boolean[] {false, true, false, false, false,
@@ -757,8 +759,7 @@ public class TestOrcFile {
         true, true, true, true, true,
         false, false, false, false, true,
         true, true, true, true};
-    included = OrcUtils.includeColumns("boolean1,string1,middle,map",
-        "boolean1,byte1,short1,int1,long1,float1,double1,bytes1,string1,middle,list,map", inspector);
+    included = OrcUtils.includeColumns("boolean1,string1,middle,map", schema);
     assertEquals(true, Arrays.equals(expected, included));
 
     expected = new boolean[] {false, true, false, false, false,
@@ -766,8 +767,7 @@ public class TestOrcFile {
         true, true, true, true, true,
         false, false, false, false, true,
         true, true, true, true};
-    included = OrcUtils.includeColumns("boolean1,string1,middle,map",
-        "boolean1,byte1,short1,int1,long1,float1,double1,bytes1,string1,middle,list,map", inspector);
+    included = OrcUtils.includeColumns("boolean1,string1,middle,map", schema);
     assertEquals(true, Arrays.equals(expected, included));
 
     expected = new boolean[] {false, true, true, true, true,
@@ -777,7 +777,7 @@ public class TestOrcFile {
         true, true, true, true};
     included = OrcUtils.includeColumns(
         "boolean1,byte1,short1,int1,long1,float1,double1,bytes1,string1,middle,list,map",
-        "boolean1,byte1,short1,int1,long1,float1,double1,bytes1,string1,middle,list,map", inspector);
+        schema);
     assertEquals(true, Arrays.equals(expected, included));
 
     Metadata metadata = reader.getMetadata();
@@ -1312,17 +1312,18 @@ public class TestOrcFile {
     Reader reader = OrcFile.createReader(testFilePath,
         OrcFile.readerOptions(conf).filesystem(fs));
 
-    assertEquals(6, OrcUtils.getFlattenedColumnsCount(inspector));
+    TypeDescription schema = writer.getSchema();
+    assertEquals(5, schema.getMaximumId());
     boolean[] expected = new boolean[] {false, false, false, false, false, false};
-    boolean[] included = OrcUtils.includeColumns("", "time,union,decimal", inspector);
+    boolean[] included = OrcUtils.includeColumns("", schema);
     assertEquals(true, Arrays.equals(expected, included));
 
     expected = new boolean[] {false, true, false, false, false, true};
-    included = OrcUtils.includeColumns("time,decimal", "time,union,decimal", inspector);
+    included = OrcUtils.includeColumns("time,decimal", schema);
     assertEquals(true, Arrays.equals(expected, included));
 
     expected = new boolean[] {false, false, true, true, true, false};
-    included = OrcUtils.includeColumns("union", "time,union,decimal", inspector);
+    included = OrcUtils.includeColumns("union", schema);
     assertEquals(true, Arrays.equals(expected, included));
 
     assertEquals(false, reader.getMetadataKeys().iterator().hasNext());

http://git-wip-us.apache.org/repos/asf/hive/blob/7b1ed3d3/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcRawRecordMerger.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcRawRecordMerger.java b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcRawRecordMerger.java
index 39f71f1..60af40a 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcRawRecordMerger.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcRawRecordMerger.java
@@ -896,7 +896,7 @@ public class TestOrcRawRecordMerger {
    */
   @Test
   public void testRecordReaderNewBaseAndDelta() throws Exception {
-    final int BUCKET = 10;
+    final int BUCKET = 11;
     Configuration conf = new Configuration();
     OrcOutputFormat of = new OrcOutputFormat();
     FileSystem fs = FileSystem.getLocal(conf);

http://git-wip-us.apache.org/repos/asf/hive/blob/7b1ed3d3/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcWideTable.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcWideTable.java b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcWideTable.java
index a3d3ec5..f838cbc 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcWideTable.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestOrcWideTable.java
@@ -38,241 +38,45 @@ import org.junit.rules.TestName;
 
 public class TestOrcWideTable {
 
-  private static final int MEMORY_FOR_ORC = 512 * 1024 * 1024;
-  Path workDir = new Path(System.getProperty("test.tmp.dir", "target" + File.separator + "test"
-      + File.separator + "tmp"));
-
-  Configuration conf;
-  FileSystem fs;
-  Path testFilePath;
-  float memoryPercent;
-
-  @Rule
-  public TestName testCaseName = new TestName();
-
-  @Before
-  public void openFileSystem() throws Exception {
-    conf = new Configuration();
-    fs = FileSystem.getLocal(conf);
-    testFilePath = new Path(workDir, "TestOrcFile." + testCaseName.getMethodName() + ".orc");
-    fs.delete(testFilePath, false);
-    // make sure constant memory is available for ORC always
-    memoryPercent = (float) MEMORY_FOR_ORC / (float) ManagementFactory.getMemoryMXBean().
-        getHeapMemoryUsage().getMax();
-    conf.setFloat(HiveConf.ConfVars.HIVE_ORC_FILE_MEMORY_POOL.varname, memoryPercent);
-  }
-
   @Test
   public void testBufferSizeFor1Col() throws IOException {
-    ObjectInspector inspector;
-    synchronized (TestOrcFile.class) {
-      inspector = ObjectInspectorFactory.getReflectionObjectInspector(Long.class,
-          ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    }
-    int bufferSize = 128 * 1024;
-    Writer writer = OrcFile.createWriter(
-        testFilePath,
-        OrcFile.writerOptions(conf).inspector(inspector).stripeSize(100000)
-            .compress(CompressionKind.NONE).bufferSize(bufferSize));
-    final int newBufferSize;
-    if (writer instanceof WriterImpl) {
-      WriterImpl orcWriter = (WriterImpl) writer;
-      newBufferSize = orcWriter.getEstimatedBufferSize(bufferSize);
-      assertEquals(bufferSize, newBufferSize);
-    }
+    assertEquals(128 * 1024, WriterImpl.getEstimatedBufferSize(512 * 1024 * 1024,
+        false, 1, 128*1024));
   }
 
   @Test
   public void testBufferSizeFor1000Col() throws IOException {
-    ObjectInspector inspector;
-    synchronized (TestOrcFile.class) {
-      inspector = ObjectInspectorFactory.getReflectionObjectInspector(Long.class,
-          ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    }
-    int bufferSize = 128 * 1024;
-    String columns = getRandomColumnNames(1000);
-    // just for testing. manually write the column names
-    conf.set(IOConstants.COLUMNS, columns);
-    Writer writer = OrcFile.createWriter(
-        testFilePath,
-        OrcFile.writerOptions(conf).inspector(inspector).stripeSize(100000)
-            .compress(CompressionKind.NONE).bufferSize(bufferSize));
-    final int newBufferSize;
-    if (writer instanceof WriterImpl) {
-      WriterImpl orcWriter = (WriterImpl) writer;
-      newBufferSize = orcWriter.getEstimatedBufferSize(bufferSize);
-      assertEquals(bufferSize, newBufferSize);
-    }
+    assertEquals(128 * 1024, WriterImpl.getEstimatedBufferSize(512 * 1024 * 1024,
+        false, 1000, 128*1024));
   }
 
   @Test
   public void testBufferSizeFor2000Col() throws IOException {
-    ObjectInspector inspector;
-    synchronized (TestOrcFile.class) {
-      inspector = ObjectInspectorFactory.getReflectionObjectInspector(Long.class,
-          ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    }
-    int bufferSize = 256 * 1024;
-    String columns = getRandomColumnNames(2000);
-    // just for testing. manually write the column names
-    conf.set(IOConstants.COLUMNS, columns);
-    Writer writer = OrcFile.createWriter(
-        testFilePath,
-        OrcFile.writerOptions(conf).inspector(inspector).stripeSize(100000)
-            .compress(CompressionKind.ZLIB).bufferSize(bufferSize));
-    final int newBufferSize;
-    if (writer instanceof WriterImpl) {
-      WriterImpl orcWriter = (WriterImpl) writer;
-      newBufferSize = orcWriter.getEstimatedBufferSize(bufferSize);
-      assertEquals(32 * 1024, newBufferSize);
-    }
+    assertEquals(32 * 1024, WriterImpl.getEstimatedBufferSize(512 * 1024 * 1024,
+        true, 2000, 256*1024));
   }
 
   @Test
   public void testBufferSizeFor2000ColNoCompression() throws IOException {
-    ObjectInspector inspector;
-    synchronized (TestOrcFile.class) {
-      inspector = ObjectInspectorFactory.getReflectionObjectInspector(Long.class,
-          ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    }
-    int bufferSize = 256 * 1024;
-    String columns = getRandomColumnNames(2000);
-    // just for testing. manually write the column names
-    conf.set(IOConstants.COLUMNS, columns);
-    Writer writer = OrcFile.createWriter(
-        testFilePath,
-        OrcFile.writerOptions(conf).inspector(inspector).stripeSize(100000)
-            .compress(CompressionKind.NONE).bufferSize(bufferSize));
-    final int newBufferSize;
-    if (writer instanceof WriterImpl) {
-      WriterImpl orcWriter = (WriterImpl) writer;
-      newBufferSize = orcWriter.getEstimatedBufferSize(bufferSize);
-      assertEquals(64 * 1024, newBufferSize);
-    }
+    assertEquals(64 * 1024, WriterImpl.getEstimatedBufferSize(512 * 1024 * 1024,
+        false, 2000, 256*1024));
   }
 
   @Test
   public void testBufferSizeFor4000Col() throws IOException {
-    ObjectInspector inspector;
-    synchronized (TestOrcFile.class) {
-      inspector = ObjectInspectorFactory.getReflectionObjectInspector(Long.class,
-          ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    }
-    int bufferSize = 256 * 1024;
-    String columns = getRandomColumnNames(4000);
-    // just for testing. manually write the column names
-    conf.set(IOConstants.COLUMNS, columns);
-    Writer writer = OrcFile.createWriter(
-        testFilePath,
-        OrcFile.writerOptions(conf).inspector(inspector).stripeSize(100000)
-            .compress(CompressionKind.ZLIB).bufferSize(bufferSize));
-    final int newBufferSize;
-    if (writer instanceof WriterImpl) {
-      WriterImpl orcWriter = (WriterImpl) writer;
-      newBufferSize = orcWriter.getEstimatedBufferSize(bufferSize);
-      assertEquals(16 * 1024, newBufferSize);
-    }
+    assertEquals(16 * 1024, WriterImpl.getEstimatedBufferSize(512 * 1024 * 1024,
+        true, 4000, 256*1024));
   }
 
   @Test
   public void testBufferSizeFor4000ColNoCompression() throws IOException {
-    ObjectInspector inspector;
-    synchronized (TestOrcFile.class) {
-      inspector = ObjectInspectorFactory.getReflectionObjectInspector(Long.class,
-          ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    }
-    int bufferSize = 256 * 1024;
-    String columns = getRandomColumnNames(4000);
-    // just for testing. manually write the column names
-    conf.set(IOConstants.COLUMNS, columns);
-    Writer writer = OrcFile.createWriter(
-        testFilePath,
-        OrcFile.writerOptions(conf).inspector(inspector).stripeSize(100000)
-            .compress(CompressionKind.NONE).bufferSize(bufferSize));
-    final int newBufferSize;
-    if (writer instanceof WriterImpl) {
-      WriterImpl orcWriter = (WriterImpl) writer;
-      newBufferSize = orcWriter.getEstimatedBufferSize(bufferSize);
-      assertEquals(32 * 1024, newBufferSize);
-    }
+    assertEquals(32 * 1024, WriterImpl.getEstimatedBufferSize(512 * 1024 * 1024,
+        false, 4000, 256*1024));
   }
 
   @Test
   public void testBufferSizeFor25000Col() throws IOException {
-    ObjectInspector inspector;
-    synchronized (TestOrcFile.class) {
-      inspector = ObjectInspectorFactory.getReflectionObjectInspector(Long.class,
-          ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    }
-    int bufferSize = 256 * 1024;
-    String columns = getRandomColumnNames(25000);
-    // just for testing. manually write the column names
-    conf.set(IOConstants.COLUMNS, columns);
-    Writer writer = OrcFile.createWriter(
-        testFilePath,
-        OrcFile.writerOptions(conf).inspector(inspector).stripeSize(100000)
-            .compress(CompressionKind.NONE).bufferSize(bufferSize));
-    final int newBufferSize;
-    if (writer instanceof WriterImpl) {
-      WriterImpl orcWriter = (WriterImpl) writer;
-      newBufferSize = orcWriter.getEstimatedBufferSize(bufferSize);
-      // 4K is the minimum buffer size
-      assertEquals(4 * 1024, newBufferSize);
-    }
-  }
-
-  @Test
-  public void testBufferSizeManualOverride1() throws IOException {
-    ObjectInspector inspector;
-    synchronized (TestOrcFile.class) {
-      inspector = ObjectInspectorFactory.getReflectionObjectInspector(Long.class,
-          ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    }
-    int bufferSize = 1024;
-    String columns = getRandomColumnNames(2000);
-    // just for testing. manually write the column names
-    conf.set(IOConstants.COLUMNS, columns);
-    Writer writer = OrcFile.createWriter(
-        testFilePath,
-        OrcFile.writerOptions(conf).inspector(inspector).stripeSize(100000)
-            .compress(CompressionKind.NONE).bufferSize(bufferSize));
-    final int newBufferSize;
-    if (writer instanceof WriterImpl) {
-      WriterImpl orcWriter = (WriterImpl) writer;
-      newBufferSize = orcWriter.getEstimatedBufferSize(bufferSize);
-      assertEquals(bufferSize, newBufferSize);
-    }
-  }
-
-  @Test
-  public void testBufferSizeManualOverride2() throws IOException {
-    ObjectInspector inspector;
-    synchronized (TestOrcFile.class) {
-      inspector = ObjectInspectorFactory.getReflectionObjectInspector(Long.class,
-          ObjectInspectorFactory.ObjectInspectorOptions.JAVA);
-    }
-    int bufferSize = 2 * 1024;
-    String columns = getRandomColumnNames(4000);
-    // just for testing. manually write the column names
-    conf.set(IOConstants.COLUMNS, columns);
-    Writer writer = OrcFile.createWriter(
-        testFilePath,
-        OrcFile.writerOptions(conf).inspector(inspector).stripeSize(100000)
-            .compress(CompressionKind.NONE).bufferSize(bufferSize));
-    final int newBufferSize;
-    if (writer instanceof WriterImpl) {
-      WriterImpl orcWriter = (WriterImpl) writer;
-      newBufferSize = orcWriter.getEstimatedBufferSize(bufferSize);
-      assertEquals(bufferSize, newBufferSize);
-    }
-  }
-
-  private String getRandomColumnNames(int n) {
-    StringBuilder sb = new StringBuilder();
-    for (int i = 0; i < n - 1; i++) {
-      sb.append("col").append(i).append(",");
-    }
-    sb.append("col").append(n - 1);
-    return sb.toString();
+    assertEquals(4 * 1024, WriterImpl.getEstimatedBufferSize(512 * 1024 * 1024,
+        false, 25000, 256*1024));
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/7b1ed3d3/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestTypeDescription.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestTypeDescription.java b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestTypeDescription.java
new file mode 100644
index 0000000..c3095f7
--- /dev/null
+++ b/ql/src/test/org/apache/hadoop/hive/ql/io/orc/TestTypeDescription.java
@@ -0,0 +1,67 @@
+/**
+ * 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 org.junit.Assert.assertEquals;
+
+import org.junit.Test;
+
+public class TestTypeDescription {
+
+  @Test
+  public void testJson() {
+    TypeDescription bin = TypeDescription.createBinary();
+    assertEquals("{\"category\": \"binary\", \"id\": 0, \"max\": 0}",
+        bin.toJson());
+    assertEquals("binary", bin.toString());
+    TypeDescription struct = TypeDescription.createStruct()
+        .addField("f1", TypeDescription.createInt())
+        .addField("f2", TypeDescription.createString())
+        .addField("f3", TypeDescription.createDecimal());
+    assertEquals("struct<f1:int,f2:string,f3:decimal(38,10)>",
+        struct.toString());
+    assertEquals("{\"category\": \"struct\", \"id\": 0, \"max\": 3, \"fields\": [\n"
+            + "  \"f1\": {\"category\": \"int\", \"id\": 1, \"max\": 1},\n"
+            + "  \"f2\": {\"category\": \"string\", \"id\": 2, \"max\": 2},\n"
+            + "  \"f3\": {\"category\": \"decimal\", \"id\": 3, \"max\": 3, \"precision\": 38, \"scale\": 10}]}",
+        struct.toJson());
+    struct = TypeDescription.createStruct()
+        .addField("f1", TypeDescription.createUnion()
+            .addUnionChild(TypeDescription.createByte())
+            .addUnionChild(TypeDescription.createDecimal()
+                .withPrecision(20).withScale(10)))
+        .addField("f2", TypeDescription.createStruct()
+            .addField("f3", TypeDescription.createDate())
+            .addField("f4", TypeDescription.createDouble())
+            .addField("f5", TypeDescription.createBoolean()))
+        .addField("f6", TypeDescription.createChar().withMaxLength(100));
+    assertEquals("struct<f1:union<tinyint,decimal(20,10)>,f2:struct<f3:date,f4:double,f5:boolean>,f6:char(100)>",
+        struct.toString());
+    assertEquals(
+        "{\"category\": \"struct\", \"id\": 0, \"max\": 8, \"fields\": [\n" +
+            "  \"f1\": {\"category\": \"union\", \"id\": 1, \"max\": 3, \"children\": [\n" +
+            "    {\"category\": \"tinyint\", \"id\": 2, \"max\": 2},\n" +
+            "    {\"category\": \"decimal\", \"id\": 3, \"max\": 3, \"precision\": 20, \"scale\": 10}]},\n" +
+            "  \"f2\": {\"category\": \"struct\", \"id\": 4, \"max\": 7, \"fields\": [\n" +
+            "    \"f3\": {\"category\": \"date\", \"id\": 5, \"max\": 5},\n" +
+            "    \"f4\": {\"category\": \"double\", \"id\": 6, \"max\": 6},\n" +
+            "    \"f5\": {\"category\": \"boolean\", \"id\": 7, \"max\": 7}]},\n" +
+            "  \"f6\": {\"category\": \"char\", \"id\": 8, \"max\": 8, \"length\": 100}]}",
+        struct.toJson());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/7b1ed3d3/ql/src/test/resources/orc-file-dump-bloomfilter.out
----------------------------------------------------------------------
diff --git a/ql/src/test/resources/orc-file-dump-bloomfilter.out b/ql/src/test/resources/orc-file-dump-bloomfilter.out
index add163c..19a2f65 100644
--- a/ql/src/test/resources/orc-file-dump-bloomfilter.out
+++ b/ql/src/test/resources/orc-file-dump-bloomfilter.out
@@ -1,5 +1,5 @@
 Structure for TestFileDump.testDump.orc
-File Version: 0.12 with HIVE_8732
+File Version: 0.12 with HIVE_4243
 Rows: 21000
 Compression: ZLIB
 Compression size: 10000

http://git-wip-us.apache.org/repos/asf/hive/blob/7b1ed3d3/ql/src/test/resources/orc-file-dump-bloomfilter2.out
----------------------------------------------------------------------
diff --git a/ql/src/test/resources/orc-file-dump-bloomfilter2.out b/ql/src/test/resources/orc-file-dump-bloomfilter2.out
index 06b65ce..a37408c 100644
--- a/ql/src/test/resources/orc-file-dump-bloomfilter2.out
+++ b/ql/src/test/resources/orc-file-dump-bloomfilter2.out
@@ -1,5 +1,5 @@
 Structure for TestFileDump.testDump.orc
-File Version: 0.12 with HIVE_8732
+File Version: 0.12 with HIVE_4243
 Rows: 21000
 Compression: ZLIB
 Compression size: 10000

http://git-wip-us.apache.org/repos/asf/hive/blob/7b1ed3d3/ql/src/test/resources/orc-file-dump-dictionary-threshold.out
----------------------------------------------------------------------
diff --git a/ql/src/test/resources/orc-file-dump-dictionary-threshold.out b/ql/src/test/resources/orc-file-dump-dictionary-threshold.out
index 13e316e..73f9f05 100644
--- a/ql/src/test/resources/orc-file-dump-dictionary-threshold.out
+++ b/ql/src/test/resources/orc-file-dump-dictionary-threshold.out
@@ -1,5 +1,5 @@
 Structure for TestFileDump.testDump.orc
-File Version: 0.12 with HIVE_8732
+File Version: 0.12 with HIVE_4243
 Rows: 21000
 Compression: ZLIB
 Compression size: 10000

http://git-wip-us.apache.org/repos/asf/hive/blob/7b1ed3d3/ql/src/test/resources/orc-file-dump.json
----------------------------------------------------------------------
diff --git a/ql/src/test/resources/orc-file-dump.json b/ql/src/test/resources/orc-file-dump.json
index fe31d5e..14cf962 100644
--- a/ql/src/test/resources/orc-file-dump.json
+++ b/ql/src/test/resources/orc-file-dump.json
@@ -1,7 +1,7 @@
 {
   "fileName": "TestFileDump.testDump.orc",
   "fileVersion": "0.12",
-  "writerVersion": "HIVE_8732",
+  "writerVersion": "HIVE_4243",
   "numberOfRows": 21000,
   "compression": "ZLIB",
   "compressionBufferSize": 10000,

http://git-wip-us.apache.org/repos/asf/hive/blob/7b1ed3d3/ql/src/test/resources/orc-file-dump.out
----------------------------------------------------------------------
diff --git a/ql/src/test/resources/orc-file-dump.out b/ql/src/test/resources/orc-file-dump.out
index 2f5962b..28935ba 100644
--- a/ql/src/test/resources/orc-file-dump.out
+++ b/ql/src/test/resources/orc-file-dump.out
@@ -1,5 +1,5 @@
 Structure for TestFileDump.testDump.orc
-File Version: 0.12 with HIVE_8732
+File Version: 0.12 with HIVE_4243
 Rows: 21000
 Compression: ZLIB
 Compression size: 10000

http://git-wip-us.apache.org/repos/asf/hive/blob/7b1ed3d3/ql/src/test/resources/orc-file-has-null.out
----------------------------------------------------------------------
diff --git a/ql/src/test/resources/orc-file-has-null.out b/ql/src/test/resources/orc-file-has-null.out
index fdc3862..9c4e83c 100644
--- a/ql/src/test/resources/orc-file-has-null.out
+++ b/ql/src/test/resources/orc-file-has-null.out
@@ -1,5 +1,5 @@
 Structure for TestOrcFile.testHasNull.orc
-File Version: 0.12 with HIVE_8732
+File Version: 0.12 with HIVE_4243
 Rows: 20000
 Compression: ZLIB
 Compression size: 10000

http://git-wip-us.apache.org/repos/asf/hive/blob/7b1ed3d3/ql/src/test/results/clientpositive/annotate_stats_part.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/annotate_stats_part.q.out b/ql/src/test/results/clientpositive/annotate_stats_part.q.out
index 2bec917..cf523cb 100644
--- a/ql/src/test/results/clientpositive/annotate_stats_part.q.out
+++ b/ql/src/test/results/clientpositive/annotate_stats_part.q.out
@@ -98,7 +98,7 @@ STAGE PLANS:
       Processor Tree:
         TableScan
           alias: loc_orc
-          Statistics: Num rows: 6 Data size: 767 Basic stats: COMPLETE Column stats: PARTIAL
+          Statistics: Num rows: 6 Data size: 780 Basic stats: COMPLETE Column stats: PARTIAL
           Select Operator
             expressions: state (type: string), locid (type: int), zip (type: bigint), year (type: string)
             outputColumnNames: _col0, _col1, _col2, _col3
@@ -133,11 +133,11 @@ STAGE PLANS:
       Processor Tree:
         TableScan
           alias: loc_orc
-          Statistics: Num rows: 3 Data size: 342 Basic stats: COMPLETE Column stats: NONE
+          Statistics: Num rows: 3 Data size: 348 Basic stats: COMPLETE Column stats: NONE
           Select Operator
             expressions: state (type: string), locid (type: int), zip (type: bigint), '__HIVE_DEFAULT_PARTITION__' (type: string)
             outputColumnNames: _col0, _col1, _col2, _col3
-            Statistics: Num rows: 3 Data size: 342 Basic stats: COMPLETE Column stats: NONE
+            Statistics: Num rows: 3 Data size: 348 Basic stats: COMPLETE Column stats: NONE
             ListSink
 
 PREHOOK: query: -- basicStatState: PARTIAL colStatState: NONE

http://git-wip-us.apache.org/repos/asf/hive/blob/7b1ed3d3/ql/src/test/results/clientpositive/annotate_stats_table.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/annotate_stats_table.q.out b/ql/src/test/results/clientpositive/annotate_stats_table.q.out
index 87e2fa6..ebc6c5b 100644
--- a/ql/src/test/results/clientpositive/annotate_stats_table.q.out
+++ b/ql/src/test/results/clientpositive/annotate_stats_table.q.out
@@ -89,11 +89,11 @@ STAGE PLANS:
       Processor Tree:
         TableScan
           alias: emp_orc
-          Statistics: Num rows: 3 Data size: 384 Basic stats: COMPLETE Column stats: NONE
+          Statistics: Num rows: 3 Data size: 394 Basic stats: COMPLETE Column stats: NONE
           Select Operator
             expressions: lastname (type: string), deptid (type: int)
             outputColumnNames: _col0, _col1
-            Statistics: Num rows: 3 Data size: 384 Basic stats: COMPLETE Column stats: NONE
+            Statistics: Num rows: 3 Data size: 394 Basic stats: COMPLETE Column stats: NONE
             ListSink
 
 PREHOOK: query: -- table level analyze statistics

http://git-wip-us.apache.org/repos/asf/hive/blob/7b1ed3d3/ql/src/test/results/clientpositive/dynpart_sort_opt_vectorization.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/dynpart_sort_opt_vectorization.q.out b/ql/src/test/results/clientpositive/dynpart_sort_opt_vectorization.q.out
index 6f79d69..2f12b8d 100644
--- a/ql/src/test/results/clientpositive/dynpart_sort_opt_vectorization.q.out
+++ b/ql/src/test/results/clientpositive/dynpart_sort_opt_vectorization.q.out
@@ -894,7 +894,7 @@ Partition Parameters:
 	numFiles            	2                   
 	numRows             	32                  
 	rawDataSize         	640                 
-	totalSize           	1400                
+	totalSize           	1392                
 #### A masked pattern was here ####
 	 	 
 # Storage Information	 	 
@@ -936,7 +936,7 @@ Partition Parameters:
 	numFiles            	2                   
 	numRows             	6                   
 	rawDataSize         	120                 
-	totalSize           	1102                
+	totalSize           	1096                
 #### A masked pattern was here ####
 	 	 
 # Storage Information	 	 
@@ -978,7 +978,7 @@ Partition Parameters:
 	numFiles            	2                   
 	numRows             	14                  
 	rawDataSize         	280                 
-	totalSize           	1216                
+	totalSize           	1210                
 #### A masked pattern was here ####
 	 	 
 # Storage Information	 	 
@@ -1020,7 +1020,7 @@ Partition Parameters:
 	numFiles            	2                   
 	numRows             	6                   
 	rawDataSize         	120                 
-	totalSize           	1102                
+	totalSize           	1096                
 #### A masked pattern was here ####
 	 	 
 # Storage Information	 	 
@@ -1061,7 +1061,7 @@ Partition Parameters:
 	numFiles            	8                   
 	numRows             	32                  
 	rawDataSize         	640                 
-	totalSize           	4548                
+	totalSize           	4524                
 #### A masked pattern was here ####
 	 	 
 # Storage Information	 	 
@@ -1102,7 +1102,7 @@ Partition Parameters:
 	numFiles            	8                   
 	numRows             	6                   
 	rawDataSize         	120                 
-	totalSize           	2212                
+	totalSize           	2400                
 #### A masked pattern was here ####
 	 	 
 # Storage Information	 	 
@@ -1143,7 +1143,7 @@ Partition Parameters:
 	numFiles            	8                   
 	numRows             	32                  
 	rawDataSize         	640                 
-	totalSize           	4534                
+	totalSize           	4510                
 #### A masked pattern was here ####
 	 	 
 # Storage Information	 	 
@@ -1184,7 +1184,7 @@ Partition Parameters:
 	numFiles            	8                   
 	numRows             	6                   
 	rawDataSize         	120                 
-	totalSize           	2212                
+	totalSize           	2400                
 #### A masked pattern was here ####
 	 	 
 # Storage Information	 	 

http://git-wip-us.apache.org/repos/asf/hive/blob/7b1ed3d3/ql/src/test/results/clientpositive/dynpart_sort_optimization2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/dynpart_sort_optimization2.q.out b/ql/src/test/results/clientpositive/dynpart_sort_optimization2.q.out
index cb0eb58..24ac550 100644
--- a/ql/src/test/results/clientpositive/dynpart_sort_optimization2.q.out
+++ b/ql/src/test/results/clientpositive/dynpart_sort_optimization2.q.out
@@ -1157,7 +1157,7 @@ Partition Parameters:
 	numFiles            	1                   
 	numRows             	11                  
 	rawDataSize         	88                  
-	totalSize           	433                 
+	totalSize           	454                 
 #### A masked pattern was here ####
 	 	 
 # Storage Information	 	 
@@ -1217,7 +1217,7 @@ Partition Parameters:
 	numFiles            	1                   
 	numRows             	13                  
 	rawDataSize         	104                 
-	totalSize           	456                 
+	totalSize           	477                 
 #### A masked pattern was here ####
 	 	 
 # Storage Information	 	 
@@ -1374,7 +1374,7 @@ Partition Parameters:
 	numFiles            	1                   
 	numRows             	11                  
 	rawDataSize         	88                  
-	totalSize           	433                 
+	totalSize           	454                 
 #### A masked pattern was here ####
 	 	 
 # Storage Information	 	 
@@ -1434,7 +1434,7 @@ Partition Parameters:
 	numFiles            	1                   
 	numRows             	13                  
 	rawDataSize         	104                 
-	totalSize           	456                 
+	totalSize           	477                 
 #### A masked pattern was here ####
 	 	 
 # Storage Information	 	 

http://git-wip-us.apache.org/repos/asf/hive/blob/7b1ed3d3/ql/src/test/results/clientpositive/extrapolate_part_stats_full.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/extrapolate_part_stats_full.q.out b/ql/src/test/results/clientpositive/extrapolate_part_stats_full.q.out
index 0f6b15d..f87a539 100644
--- a/ql/src/test/results/clientpositive/extrapolate_part_stats_full.q.out
+++ b/ql/src/test/results/clientpositive/extrapolate_part_stats_full.q.out
@@ -136,7 +136,7 @@ STAGE PLANS:
               serialization.ddl struct loc_orc_1d { string state, i32 locid, i32 zip}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              totalSize 362
+              totalSize 369
 #### A masked pattern was here ####
             serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
           
@@ -179,7 +179,7 @@ STAGE PLANS:
               serialization.ddl struct loc_orc_1d { string state, i32 locid, i32 zip}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              totalSize 382
+              totalSize 389
 #### A masked pattern was here ####
             serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
           
@@ -269,7 +269,7 @@ STAGE PLANS:
               serialization.ddl struct loc_orc_1d { string state, i32 locid, i32 zip}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              totalSize 362
+              totalSize 369
 #### A masked pattern was here ####
             serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
           
@@ -312,7 +312,7 @@ STAGE PLANS:
               serialization.ddl struct loc_orc_1d { string state, i32 locid, i32 zip}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              totalSize 382
+              totalSize 389
 #### A masked pattern was here ####
             serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
           
@@ -469,7 +469,7 @@ STAGE PLANS:
               serialization.ddl struct loc_orc_2d { string state, i32 locid}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              totalSize 277
+              totalSize 281
 #### A masked pattern was here ####
             serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
           
@@ -513,7 +513,7 @@ STAGE PLANS:
               serialization.ddl struct loc_orc_2d { string state, i32 locid}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              totalSize 279
+              totalSize 283
 #### A masked pattern was here ####
             serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
           
@@ -557,7 +557,7 @@ STAGE PLANS:
               serialization.ddl struct loc_orc_2d { string state, i32 locid}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              totalSize 261
+              totalSize 265
 #### A masked pattern was here ####
             serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
           
@@ -601,7 +601,7 @@ STAGE PLANS:
               serialization.ddl struct loc_orc_2d { string state, i32 locid}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              totalSize 261
+              totalSize 265
 #### A masked pattern was here ####
             serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
           
@@ -688,7 +688,7 @@ STAGE PLANS:
               serialization.ddl struct loc_orc_2d { string state, i32 locid}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              totalSize 277
+              totalSize 281
 #### A masked pattern was here ####
             serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
           
@@ -732,7 +732,7 @@ STAGE PLANS:
               serialization.ddl struct loc_orc_2d { string state, i32 locid}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              totalSize 279
+              totalSize 283
 #### A masked pattern was here ####
             serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
           
@@ -776,7 +776,7 @@ STAGE PLANS:
               serialization.ddl struct loc_orc_2d { string state, i32 locid}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              totalSize 261
+              totalSize 265
 #### A masked pattern was here ####
             serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
           
@@ -820,7 +820,7 @@ STAGE PLANS:
               serialization.ddl struct loc_orc_2d { string state, i32 locid}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              totalSize 261
+              totalSize 265
 #### A masked pattern was here ####
             serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
           

http://git-wip-us.apache.org/repos/asf/hive/blob/7b1ed3d3/ql/src/test/results/clientpositive/extrapolate_part_stats_partial.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/extrapolate_part_stats_partial.q.out b/ql/src/test/results/clientpositive/extrapolate_part_stats_partial.q.out
index 1fdeb90..5903cd1 100644
--- a/ql/src/test/results/clientpositive/extrapolate_part_stats_partial.q.out
+++ b/ql/src/test/results/clientpositive/extrapolate_part_stats_partial.q.out
@@ -153,7 +153,7 @@ STAGE PLANS:
               serialization.ddl struct loc_orc_1d { string state, i32 locid, i32 zip}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              totalSize 362
+              totalSize 369
 #### A masked pattern was here ####
             serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
           
@@ -196,7 +196,7 @@ STAGE PLANS:
               serialization.ddl struct loc_orc_1d { string state, i32 locid, i32 zip}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              totalSize 383
+              totalSize 390
 #### A masked pattern was here ####
             serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
           
@@ -239,7 +239,7 @@ STAGE PLANS:
               serialization.ddl struct loc_orc_1d { string state, i32 locid, i32 zip}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              totalSize 404
+              totalSize 410
 #### A masked pattern was here ####
             serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
           
@@ -282,7 +282,7 @@ STAGE PLANS:
               serialization.ddl struct loc_orc_1d { string state, i32 locid, i32 zip}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              totalSize 412
+              totalSize 419
 #### A masked pattern was here ####
             serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
           
@@ -372,7 +372,7 @@ STAGE PLANS:
               serialization.ddl struct loc_orc_1d { string state, i32 locid, i32 zip}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              totalSize 362
+              totalSize 369
 #### A masked pattern was here ####
             serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
           
@@ -415,7 +415,7 @@ STAGE PLANS:
               serialization.ddl struct loc_orc_1d { string state, i32 locid, i32 zip}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              totalSize 383
+              totalSize 390
 #### A masked pattern was here ####
             serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
           
@@ -458,7 +458,7 @@ STAGE PLANS:
               serialization.ddl struct loc_orc_1d { string state, i32 locid, i32 zip}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              totalSize 404
+              totalSize 410
 #### A masked pattern was here ####
             serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
           
@@ -501,7 +501,7 @@ STAGE PLANS:
               serialization.ddl struct loc_orc_1d { string state, i32 locid, i32 zip}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              totalSize 412
+              totalSize 419
 #### A masked pattern was here ####
             serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
           
@@ -604,7 +604,7 @@ STAGE PLANS:
               serialization.ddl struct loc_orc_1d { string state, i32 locid, i32 zip}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              totalSize 362
+              totalSize 369
 #### A masked pattern was here ####
             serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
           
@@ -647,7 +647,7 @@ STAGE PLANS:
               serialization.ddl struct loc_orc_1d { string state, i32 locid, i32 zip}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              totalSize 383
+              totalSize 390
 #### A masked pattern was here ####
             serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
           
@@ -690,7 +690,7 @@ STAGE PLANS:
               serialization.ddl struct loc_orc_1d { string state, i32 locid, i32 zip}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              totalSize 404
+              totalSize 410
 #### A masked pattern was here ####
             serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
           
@@ -733,7 +733,7 @@ STAGE PLANS:
               serialization.ddl struct loc_orc_1d { string state, i32 locid, i32 zip}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              totalSize 412
+              totalSize 419
 #### A masked pattern was here ####
             serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
           
@@ -819,7 +819,7 @@ STAGE PLANS:
               serialization.ddl struct loc_orc_1d { string state, i32 locid, i32 zip}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              totalSize 362
+              totalSize 369
 #### A masked pattern was here ####
             serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
           
@@ -862,7 +862,7 @@ STAGE PLANS:
               serialization.ddl struct loc_orc_1d { string state, i32 locid, i32 zip}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              totalSize 383
+              totalSize 390
 #### A masked pattern was here ####
             serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
           
@@ -905,7 +905,7 @@ STAGE PLANS:
               serialization.ddl struct loc_orc_1d { string state, i32 locid, i32 zip}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              totalSize 404
+              totalSize 410
 #### A masked pattern was here ####
             serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
           
@@ -948,7 +948,7 @@ STAGE PLANS:
               serialization.ddl struct loc_orc_1d { string state, i32 locid, i32 zip}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              totalSize 412
+              totalSize 419
 #### A masked pattern was here ####
             serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
           
@@ -1106,7 +1106,7 @@ STAGE PLANS:
               serialization.ddl struct loc_orc_2d { string state, i32 locid}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              totalSize 281
+              totalSize 286
 #### A masked pattern was here ####
             serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
           
@@ -1150,7 +1150,7 @@ STAGE PLANS:
               serialization.ddl struct loc_orc_2d { string state, i32 locid}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              totalSize 291
+              totalSize 295
 #### A masked pattern was here ####
             serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
           
@@ -1194,7 +1194,7 @@ STAGE PLANS:
               serialization.ddl struct loc_orc_2d { string state, i32 locid}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              totalSize 298
+              totalSize 302
 #### A masked pattern was here ####
             serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
           
@@ -1238,7 +1238,7 @@ STAGE PLANS:
               serialization.ddl struct loc_orc_2d { string state, i32 locid}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              totalSize 277
+              totalSize 281
 #### A masked pattern was here ####
             serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
           
@@ -1282,7 +1282,7 @@ STAGE PLANS:
               serialization.ddl struct loc_orc_2d { string state, i32 locid}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              totalSize 272
+              totalSize 276
 #### A masked pattern was here ####
             serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
           
@@ -1326,7 +1326,7 @@ STAGE PLANS:
               serialization.ddl struct loc_orc_2d { string state, i32 locid}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              totalSize 283
+              totalSize 288
 #### A masked pattern was here ####
             serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
           
@@ -1370,7 +1370,7 @@ STAGE PLANS:
               serialization.ddl struct loc_orc_2d { string state, i32 locid}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              totalSize 292
+              totalSize 297
 #### A masked pattern was here ####
             serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
           
@@ -1414,7 +1414,7 @@ STAGE PLANS:
               serialization.ddl struct loc_orc_2d { string state, i32 locid}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              totalSize 261
+              totalSize 265
 #### A masked pattern was here ####
             serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
           
@@ -1458,7 +1458,7 @@ STAGE PLANS:
               serialization.ddl struct loc_orc_2d { string state, i32 locid}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              totalSize 261
+              totalSize 265
 #### A masked pattern was here ####
             serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
           
@@ -1502,7 +1502,7 @@ STAGE PLANS:
               serialization.ddl struct loc_orc_2d { string state, i32 locid}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              totalSize 294
+              totalSize 298
 #### A masked pattern was here ####
             serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
           
@@ -1546,7 +1546,7 @@ STAGE PLANS:
               serialization.ddl struct loc_orc_2d { string state, i32 locid}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              totalSize 291
+              totalSize 295
 #### A masked pattern was here ####
             serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
           
@@ -1633,7 +1633,7 @@ STAGE PLANS:
               serialization.ddl struct loc_orc_2d { string state, i32 locid}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              totalSize 281
+              totalSize 286
 #### A masked pattern was here ####
             serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
           
@@ -1677,7 +1677,7 @@ STAGE PLANS:
               serialization.ddl struct loc_orc_2d { string state, i32 locid}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              totalSize 291
+              totalSize 295
 #### A masked pattern was here ####
             serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
           
@@ -1721,7 +1721,7 @@ STAGE PLANS:
               serialization.ddl struct loc_orc_2d { string state, i32 locid}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              totalSize 298
+              totalSize 302
 #### A masked pattern was here ####
             serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
           
@@ -1765,7 +1765,7 @@ STAGE PLANS:
               serialization.ddl struct loc_orc_2d { string state, i32 locid}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              totalSize 277
+              totalSize 281
 #### A masked pattern was here ####
             serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
           
@@ -1809,7 +1809,7 @@ STAGE PLANS:
               serialization.ddl struct loc_orc_2d { string state, i32 locid}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              totalSize 272
+              totalSize 276
 #### A masked pattern was here ####
             serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
           
@@ -1853,7 +1853,7 @@ STAGE PLANS:
               serialization.ddl struct loc_orc_2d { string state, i32 locid}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              totalSize 283
+              totalSize 288
 #### A masked pattern was here ####
             serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
           
@@ -1897,7 +1897,7 @@ STAGE PLANS:
               serialization.ddl struct loc_orc_2d { string state, i32 locid}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              totalSize 292
+              totalSize 297
 #### A masked pattern was here ####
             serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
           
@@ -1941,7 +1941,7 @@ STAGE PLANS:
               serialization.ddl struct loc_orc_2d { string state, i32 locid}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              totalSize 261
+              totalSize 265
 #### A masked pattern was here ####
             serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
           
@@ -1985,7 +1985,7 @@ STAGE PLANS:
               serialization.ddl struct loc_orc_2d { string state, i32 locid}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              totalSize 261
+              totalSize 265
 #### A masked pattern was here ####
             serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
           
@@ -2029,7 +2029,7 @@ STAGE PLANS:
               serialization.ddl struct loc_orc_2d { string state, i32 locid}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              totalSize 294
+              totalSize 298
 #### A masked pattern was here ####
             serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
           
@@ -2073,7 +2073,7 @@ STAGE PLANS:
               serialization.ddl struct loc_orc_2d { string state, i32 locid}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              totalSize 291
+              totalSize 295
 #### A masked pattern was here ####
             serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
           

http://git-wip-us.apache.org/repos/asf/hive/blob/7b1ed3d3/ql/src/test/results/clientpositive/extrapolate_part_stats_partial_ndv.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/extrapolate_part_stats_partial_ndv.q.out b/ql/src/test/results/clientpositive/extrapolate_part_stats_partial_ndv.q.out
index 81ac963..2ea1e6e 100644
--- a/ql/src/test/results/clientpositive/extrapolate_part_stats_partial_ndv.q.out
+++ b/ql/src/test/results/clientpositive/extrapolate_part_stats_partial_ndv.q.out
@@ -230,7 +230,7 @@ STAGE PLANS:
               serialization.ddl struct loc_orc_1d { string state, double locid, decimal(10,0) cnt, i32 zip}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              totalSize 521
+              totalSize 531
 #### A masked pattern was here ####
             serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
           
@@ -273,7 +273,7 @@ STAGE PLANS:
               serialization.ddl struct loc_orc_1d { string state, double locid, decimal(10,0) cnt, i32 zip}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              totalSize 553
+              totalSize 562
 #### A masked pattern was here ####
             serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
           
@@ -316,7 +316,7 @@ STAGE PLANS:
               serialization.ddl struct loc_orc_1d { string state, double locid, decimal(10,0) cnt, i32 zip}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              totalSize 571
+              totalSize 580
 #### A masked pattern was here ####
             serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
           
@@ -359,7 +359,7 @@ STAGE PLANS:
               serialization.ddl struct loc_orc_1d { string state, double locid, decimal(10,0) cnt, i32 zip}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              totalSize 593
+              totalSize 602
 #### A masked pattern was here ####
             serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
           
@@ -543,7 +543,7 @@ STAGE PLANS:
               serialization.ddl struct loc_orc_1d { string state, double locid, decimal(10,0) cnt, i32 zip}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              totalSize 521
+              totalSize 531
 #### A masked pattern was here ####
             serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
           
@@ -586,7 +586,7 @@ STAGE PLANS:
               serialization.ddl struct loc_orc_1d { string state, double locid, decimal(10,0) cnt, i32 zip}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              totalSize 553
+              totalSize 562
 #### A masked pattern was here ####
             serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
           
@@ -629,7 +629,7 @@ STAGE PLANS:
               serialization.ddl struct loc_orc_1d { string state, double locid, decimal(10,0) cnt, i32 zip}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              totalSize 571
+              totalSize 580
 #### A masked pattern was here ####
             serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
           
@@ -672,7 +672,7 @@ STAGE PLANS:
               serialization.ddl struct loc_orc_1d { string state, double locid, decimal(10,0) cnt, i32 zip}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              totalSize 593
+              totalSize 602
 #### A masked pattern was here ####
             serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
           
@@ -910,7 +910,7 @@ STAGE PLANS:
               serialization.ddl struct loc_orc_2d { string state, i32 locid, decimal(10,0) cnt}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              totalSize 379
+              totalSize 386
 #### A masked pattern was here ####
             serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
           
@@ -954,7 +954,7 @@ STAGE PLANS:
               serialization.ddl struct loc_orc_2d { string state, i32 locid, decimal(10,0) cnt}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              totalSize 402
+              totalSize 409
 #### A masked pattern was here ####
             serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
           
@@ -998,7 +998,7 @@ STAGE PLANS:
               serialization.ddl struct loc_orc_2d { string state, i32 locid, decimal(10,0) cnt}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              totalSize 417
+              totalSize 423
 #### A masked pattern was here ####
             serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
           
@@ -1042,7 +1042,7 @@ STAGE PLANS:
               serialization.ddl struct loc_orc_2d { string state, i32 locid, decimal(10,0) cnt}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              totalSize 376
+              totalSize 383
 #### A masked pattern was here ####
             serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
           
@@ -1086,7 +1086,7 @@ STAGE PLANS:
               serialization.ddl struct loc_orc_2d { string state, i32 locid, decimal(10,0) cnt}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              totalSize 386
+              totalSize 394
 #### A masked pattern was here ####
             serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
           
@@ -1130,7 +1130,7 @@ STAGE PLANS:
               serialization.ddl struct loc_orc_2d { string state, i32 locid, decimal(10,0) cnt}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              totalSize 380
+              totalSize 387
 #### A masked pattern was here ####
             serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
           
@@ -1174,7 +1174,7 @@ STAGE PLANS:
               serialization.ddl struct loc_orc_2d { string state, i32 locid, decimal(10,0) cnt}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              totalSize 403
+              totalSize 409
 #### A masked pattern was here ####
             serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
           
@@ -1218,7 +1218,7 @@ STAGE PLANS:
               serialization.ddl struct loc_orc_2d { string state, i32 locid, decimal(10,0) cnt}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              totalSize 359
+              totalSize 366
 #### A masked pattern was here ####
             serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
           
@@ -1262,7 +1262,7 @@ STAGE PLANS:
               serialization.ddl struct loc_orc_2d { string state, i32 locid, decimal(10,0) cnt}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              totalSize 354
+              totalSize 361
 #### A masked pattern was here ####
             serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
           
@@ -1306,7 +1306,7 @@ STAGE PLANS:
               serialization.ddl struct loc_orc_2d { string state, i32 locid, decimal(10,0) cnt}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              totalSize 405
+              totalSize 412
 #### A masked pattern was here ####
             serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
           
@@ -1350,7 +1350,7 @@ STAGE PLANS:
               serialization.ddl struct loc_orc_2d { string state, i32 locid, decimal(10,0) cnt}
               serialization.format 1
               serialization.lib org.apache.hadoop.hive.ql.io.orc.OrcSerde
-              totalSize 405
+              totalSize 412
 #### A masked pattern was here ####
             serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
           

http://git-wip-us.apache.org/repos/asf/hive/blob/7b1ed3d3/ql/src/test/results/clientpositive/orc_analyze.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/orc_analyze.q.out b/ql/src/test/results/clientpositive/orc_analyze.q.out
index 6eb9a93..bc46852 100644
--- a/ql/src/test/results/clientpositive/orc_analyze.q.out
+++ b/ql/src/test/results/clientpositive/orc_analyze.q.out
@@ -106,7 +106,7 @@ Table Parameters:
 	numFiles            	1                   
 	numRows             	100                 
 	rawDataSize         	52600               
-	totalSize           	3174                
+	totalSize           	3202                
 #### A masked pattern was here ####
 	 	 
 # Storage Information	 	 
@@ -154,7 +154,7 @@ Table Parameters:
 	numFiles            	1                   
 	numRows             	100                 
 	rawDataSize         	52600               
-	totalSize           	3174                
+	totalSize           	3202                
 #### A masked pattern was here ####
 	 	 
 # Storage Information	 	 
@@ -202,7 +202,7 @@ Table Parameters:
 	numFiles            	1                   
 	numRows             	100                 
 	rawDataSize         	52600               
-	totalSize           	3174                
+	totalSize           	3202                
 #### A masked pattern was here ####
 	 	 
 # Storage Information	 	 
@@ -291,7 +291,7 @@ Table Parameters:
 	numFiles            	1                   
 	numRows             	100                 
 	rawDataSize         	52600               
-	totalSize           	3174                
+	totalSize           	3202                
 #### A masked pattern was here ####
 	 	 
 # Storage Information	 	 
@@ -405,7 +405,7 @@ Partition Parameters:
 	numFiles            	1                   
 	numRows             	50                  
 	rawDataSize         	21950               
-	totalSize           	2073                
+	totalSize           	2102                
 #### A masked pattern was here ####
 	 	 
 # Storage Information	 	 
@@ -448,7 +448,7 @@ Partition Parameters:
 	numFiles            	1                   
 	numRows             	50                  
 	rawDataSize         	22050               
-	totalSize           	2088                
+	totalSize           	2118                
 #### A masked pattern was here ####
 	 	 
 # Storage Information	 	 
@@ -503,7 +503,7 @@ Partition Parameters:
 	numFiles            	1                   
 	numRows             	50                  
 	rawDataSize         	21950               
-	totalSize           	2073                
+	totalSize           	2102                
 #### A masked pattern was here ####
 	 	 
 # Storage Information	 	 
@@ -546,7 +546,7 @@ Partition Parameters:
 	numFiles            	1                   
 	numRows             	50                  
 	rawDataSize         	22050               
-	totalSize           	2088                
+	totalSize           	2118                
 #### A masked pattern was here ####
 	 	 
 # Storage Information	 	 
@@ -601,7 +601,7 @@ Partition Parameters:
 	numFiles            	1                   
 	numRows             	50                  
 	rawDataSize         	21950               
-	totalSize           	2073                
+	totalSize           	2102                
 #### A masked pattern was here ####
 	 	 
 # Storage Information	 	 
@@ -644,7 +644,7 @@ Partition Parameters:
 	numFiles            	1                   
 	numRows             	50                  
 	rawDataSize         	22050               
-	totalSize           	2088                
+	totalSize           	2118                
 #### A masked pattern was here ####
 	 	 
 # Storage Information	 	 
@@ -744,7 +744,7 @@ Partition Parameters:
 	numFiles            	1                   
 	numRows             	50                  
 	rawDataSize         	21950               
-	totalSize           	2073                
+	totalSize           	2102                
 #### A masked pattern was here ####
 	 	 
 # Storage Information	 	 
@@ -787,7 +787,7 @@ Partition Parameters:
 	numFiles            	1                   
 	numRows             	50                  
 	rawDataSize         	22050               
-	totalSize           	2088                
+	totalSize           	2118                
 #### A masked pattern was here ####
 	 	 
 # Storage Information	 	 
@@ -907,7 +907,7 @@ Partition Parameters:
 	numFiles            	1                   
 	numRows             	50                  
 	rawDataSize         	21950               
-	totalSize           	2073                
+	totalSize           	2102                
 #### A masked pattern was here ####
 	 	 
 # Storage Information	 	 
@@ -950,7 +950,7 @@ Partition Parameters:
 	numFiles            	1                   
 	numRows             	50                  
 	rawDataSize         	22050               
-	totalSize           	2088                
+	totalSize           	2118                
 #### A masked pattern was here ####
 	 	 
 # Storage Information	 	 
@@ -1005,7 +1005,7 @@ Partition Parameters:
 	numFiles            	1                   
 	numRows             	50                  
 	rawDataSize         	21950               
-	totalSize           	2073                
+	totalSize           	2102                
 #### A masked pattern was here ####
 	 	 
 # Storage Information	 	 
@@ -1048,7 +1048,7 @@ Partition Parameters:
 	numFiles            	1                   
 	numRows             	50                  
 	rawDataSize         	22050               
-	totalSize           	2088                
+	totalSize           	2118                
 #### A masked pattern was here ####
 	 	 
 # Storage Information	 	 
@@ -1103,7 +1103,7 @@ Partition Parameters:
 	numFiles            	1                   
 	numRows             	50                  
 	rawDataSize         	21950               
-	totalSize           	2073                
+	totalSize           	2102                
 #### A masked pattern was here ####
 	 	 
 # Storage Information	 	 
@@ -1146,7 +1146,7 @@ Partition Parameters:
 	numFiles            	1                   
 	numRows             	50                  
 	rawDataSize         	22050               
-	totalSize           	2088                
+	totalSize           	2118                
 #### A masked pattern was here ####
 	 	 
 # Storage Information	 	 
@@ -1252,7 +1252,7 @@ Partition Parameters:
 	numFiles            	1                   
 	numRows             	50                  
 	rawDataSize         	21950               
-	totalSize           	2073                
+	totalSize           	2102                
 #### A masked pattern was here ####
 	 	 
 # Storage Information	 	 
@@ -1295,7 +1295,7 @@ Partition Parameters:
 	numFiles            	1                   
 	numRows             	50                  
 	rawDataSize         	22050               
-	totalSize           	2088                
+	totalSize           	2118                
 #### A masked pattern was here ####
 	 	 
 # Storage Information	 	 
@@ -1460,7 +1460,7 @@ Partition Parameters:
 	numFiles            	1                   
 	numRows             	50                  
 	rawDataSize         	21950               
-	totalSize           	2073                
+	totalSize           	2102                
 #### A masked pattern was here ####
 	 	 
 # Storage Information	 	 
@@ -1560,7 +1560,7 @@ Partition Parameters:
 	numFiles            	1                   
 	numRows             	50                  
 	rawDataSize         	21950               
-	totalSize           	2073                
+	totalSize           	2102                
 #### A masked pattern was here ####
 	 	 
 # Storage Information	 	 
@@ -1660,7 +1660,7 @@ Partition Parameters:
 	numFiles            	1                   
 	numRows             	50                  
 	rawDataSize         	21950               
-	totalSize           	2073                
+	totalSize           	2102                
 #### A masked pattern was here ####
 	 	 
 # Storage Information	 	 

http://git-wip-us.apache.org/repos/asf/hive/blob/7b1ed3d3/ql/src/test/results/clientpositive/orc_file_dump.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/orc_file_dump.q.out b/ql/src/test/results/clientpositive/orc_file_dump.q.out
index 67aa189..c494d47 100644
--- a/ql/src/test/results/clientpositive/orc_file_dump.q.out
+++ b/ql/src/test/results/clientpositive/orc_file_dump.q.out
@@ -93,11 +93,11 @@ PREHOOK: Input: default@orc_ppd
 #### A masked pattern was here ####
 -- BEGIN ORC FILE DUMP --
 #### A masked pattern was here ####
-File Version: 0.12 with HIVE_8732
+File Version: 0.12 with HIVE_4243
 Rows: 1049
 Compression: ZLIB
 Compression size: 262144
-Type: struct<_col0:tinyint,_col1:smallint,_col2:int,_col3:bigint,_col4:float,_col5:double,_col6:boolean,_col7:string,_col8:timestamp,_col9:decimal(4,2),_col10:binary>
+Type: struct<t:tinyint,si:smallint,i:int,b:bigint,f:float,d:double,bo:boolean,s:string,ts:timestamp,dec:decimal(4,2),bin:binary>
 
 Stripe Statistics:
   Stripe 1:
@@ -192,7 +192,7 @@ Stripes:
       Entry 1: numHashFunctions: 4 bitCount: 6272 popCount: 168 loadFactor: 0.0268 expectedFpp: 5.147697E-7
       Stripe level merge: numHashFunctions: 4 bitCount: 6272 popCount: 492 loadFactor: 0.0784 expectedFpp: 3.7864847E-5
 
-File length: 33456 bytes
+File length: 33458 bytes
 Padding length: 0 bytes
 Padding ratio: 0%
 -- END ORC FILE DUMP --
@@ -211,11 +211,11 @@ PREHOOK: Input: default@orc_ppd
 #### A masked pattern was here ####
 -- BEGIN ORC FILE DUMP --
 #### A masked pattern was here ####
-File Version: 0.12 with HIVE_8732
+File Version: 0.12 with HIVE_4243
 Rows: 1049
 Compression: ZLIB
 Compression size: 262144
-Type: struct<_col0:tinyint,_col1:smallint,_col2:int,_col3:bigint,_col4:float,_col5:double,_col6:boolean,_col7:string,_col8:timestamp,_col9:decimal(4,2),_col10:binary>
+Type: struct<t:tinyint,si:smallint,i:int,b:bigint,f:float,d:double,bo:boolean,s:string,ts:timestamp,dec:decimal(4,2),bin:binary>
 
 Stripe Statistics:
   Stripe 1:
@@ -310,7 +310,7 @@ Stripes:
       Entry 1: numHashFunctions: 7 bitCount: 9600 popCount: 285 loadFactor: 0.0297 expectedFpp: 2.0324289E-11
       Stripe level merge: numHashFunctions: 7 bitCount: 9600 popCount: 849 loadFactor: 0.0884 expectedFpp: 4.231118E-8
 
-File length: 38610 bytes
+File length: 38613 bytes
 Padding length: 0 bytes
 Padding ratio: 0%
 -- END ORC FILE DUMP --
@@ -341,11 +341,11 @@ PREHOOK: Input: default@orc_ppd_part@ds=2015/hr=10
 #### A masked pattern was here ####
 -- BEGIN ORC FILE DUMP --
 #### A masked pattern was here ####
-File Version: 0.12 with HIVE_8732
+File Version: 0.12 with HIVE_4243
 Rows: 1049
 Compression: ZLIB
 Compression size: 262144
-Type: struct<_col0:tinyint,_col1:smallint,_col2:int,_col3:bigint,_col4:float,_col5:double,_col6:boolean,_col7:string,_col8:timestamp,_col9:decimal(4,2),_col10:binary>
+Type: struct<t:tinyint,si:smallint,i:int,b:bigint,f:float,d:double,bo:boolean,s:string,ts:timestamp,dec:decimal(4,2),bin:binary>
 
 Stripe Statistics:
   Stripe 1:
@@ -440,7 +440,7 @@ Stripes:
       Entry 1: numHashFunctions: 4 bitCount: 6272 popCount: 168 loadFactor: 0.0268 expectedFpp: 5.147697E-7
       Stripe level merge: numHashFunctions: 4 bitCount: 6272 popCount: 492 loadFactor: 0.0784 expectedFpp: 3.7864847E-5
 
-File length: 33456 bytes
+File length: 33458 bytes
 Padding length: 0 bytes
 Padding ratio: 0%
 -- END ORC FILE DUMP --

http://git-wip-us.apache.org/repos/asf/hive/blob/7b1ed3d3/ql/src/test/results/clientpositive/orc_int_type_promotion.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/orc_int_type_promotion.q.out b/ql/src/test/results/clientpositive/orc_int_type_promotion.q.out
index d26dff2..03e2f7f 100644
--- a/ql/src/test/results/clientpositive/orc_int_type_promotion.q.out
+++ b/ql/src/test/results/clientpositive/orc_int_type_promotion.q.out
@@ -220,14 +220,14 @@ STAGE PLANS:
       Map Operator Tree:
           TableScan
             alias: alltypes_orc
-            Statistics: Num rows: 88 Data size: 1772 Basic stats: COMPLETE Column stats: NONE
+            Statistics: Num rows: 88 Data size: 1766 Basic stats: COMPLETE Column stats: NONE
             Select Operator
               expressions: ti (type: tinyint), si (type: smallint), i (type: int), bi (type: bigint)
               outputColumnNames: _col0, _col1, _col2, _col3
-              Statistics: Num rows: 88 Data size: 1772 Basic stats: COMPLETE Column stats: NONE
+              Statistics: Num rows: 88 Data size: 1766 Basic stats: COMPLETE Column stats: NONE
               File Output Operator
                 compressed: false
-                Statistics: Num rows: 88 Data size: 1772 Basic stats: COMPLETE Column stats: NONE
+                Statistics: Num rows: 88 Data size: 1766 Basic stats: COMPLETE Column stats: NONE
                 table:
                     input format: org.apache.hadoop.mapred.TextInputFormat
                     output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat