You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by mm...@apache.org on 2016/07/28 18:53:42 UTC

[18/25] hive git commit: HIVE-14000: (ORC) Changing a numeric type column of a partitioned table to lower type set values to something other than 'NULL' (Matt McCline, reviewed by Sergey Shelukhin)

HIVE-14000: (ORC) Changing a numeric type column of a partitioned table to lower type set values to something other than 'NULL' (Matt McCline, reviewed by Sergey Shelukhin)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/66a2ded1
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/66a2ded1
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/66a2ded1

Branch: refs/heads/branch-2.1
Commit: 66a2ded1813e56bd8a477770f092871a3bff993b
Parents: 440fb78
Author: Matt McCline <mm...@hortonworks.com>
Authored: Mon Jun 20 04:57:33 2016 -0700
Committer: Matt McCline <mm...@hortonworks.com>
Committed: Thu Jul 28 11:43:11 2016 -0700

----------------------------------------------------------------------
 .../orc/impl/ConvertTreeReaderFactory.java      | 258 ++++----
 ...evol_orc_nonvec_mapwork_part_all_primitive.q | 137 ++--
 ...ma_evol_orc_vec_mapwork_part_all_primitive.q | 137 ++--
 ...vol_text_nonvec_mapwork_part_all_primitive.q | 137 ++--
 ...a_evol_text_vec_mapwork_part_all_primitive.q | 137 ++--
 ...vol_text_vecrow_mapwork_part_all_primitive.q | 137 ++--
 ..._orc_nonvec_mapwork_part_all_primitive.q.out | 630 ++++++++++---------
 ...vol_orc_vec_mapwork_part_all_primitive.q.out | 630 ++++++++++---------
 ...text_nonvec_mapwork_part_all_primitive.q.out | 626 +++++++++---------
 ...ol_text_vec_mapwork_part_all_primitive.q.out | 626 +++++++++---------
 ...text_vecrow_mapwork_part_all_primitive.q.out | 626 +++++++++---------
 ..._orc_nonvec_mapwork_part_all_primitive.q.out | 578 +++++++++--------
 ...vol_orc_vec_mapwork_part_all_primitive.q.out | 578 +++++++++--------
 ...text_nonvec_mapwork_part_all_primitive.q.out | 574 +++++++++--------
 ...ol_text_vec_mapwork_part_all_primitive.q.out | 574 +++++++++--------
 ...text_vecrow_mapwork_part_all_primitive.q.out | 574 +++++++++--------
 .../hadoop/hive/ql/util/TimestampUtils.java     |  70 ++-
 17 files changed, 3848 insertions(+), 3181 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/66a2ded1/orc/src/java/org/apache/orc/impl/ConvertTreeReaderFactory.java
----------------------------------------------------------------------
diff --git a/orc/src/java/org/apache/orc/impl/ConvertTreeReaderFactory.java b/orc/src/java/org/apache/orc/impl/ConvertTreeReaderFactory.java
index 5b830c3..9e5f5cc 100644
--- a/orc/src/java/org/apache/orc/impl/ConvertTreeReaderFactory.java
+++ b/orc/src/java/org/apache/orc/impl/ConvertTreeReaderFactory.java
@@ -35,8 +35,6 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.StringExpr;
 import org.apache.hadoop.hive.ql.util.TimestampUtils;
 import org.apache.hadoop.hive.serde2.io.DateWritable;
 import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
-import org.apache.hadoop.io.BytesWritable;
-import org.apache.hadoop.io.FloatWritable;
 import org.apache.orc.OrcProto;
 import org.apache.orc.TypeDescription;
 import org.apache.orc.TypeDescription.Category;
@@ -263,6 +261,22 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
       return string;
     }
 
+    private static final double MIN_LONG_AS_DOUBLE = -0x1p63;
+    /*
+     * We cannot store Long.MAX_VALUE as a double without losing precision. Instead, we store
+     * Long.MAX_VALUE + 1 == -Long.MIN_VALUE, and then offset all comparisons by 1.
+     */
+    private static final double MAX_LONG_AS_DOUBLE_PLUS_ONE = 0x1p63;
+
+    public boolean doubleCanFitInLong(double doubleValue) {
+
+      // Borrowed from Guava DoubleMath.roundToLong except do not want dependency on Guava and we
+      // don't want to catch an exception.
+
+      return ((MIN_LONG_AS_DOUBLE - doubleValue < 1.0) &&
+              (doubleValue < MAX_LONG_AS_DOUBLE_PLUS_ONE));
+    }
+
     @Override
     void checkEncoding(OrcProto.ColumnEncoding encoding) throws IOException {
       // Pass-thru.
@@ -336,20 +350,44 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
       }
     }
 
-    public long downCastAnyInteger(long input, TypeDescription readerType) {
-      switch (readerType.getCategory()) {
+    public void downCastAnyInteger(LongColumnVector longColVector, int elementNum,
+        TypeDescription readerType) {
+      downCastAnyInteger(longColVector, elementNum, longColVector.vector[elementNum], readerType);
+    }
+
+    public void downCastAnyInteger(LongColumnVector longColVector, int elementNum, long inputLong,
+        TypeDescription readerType) {
+      long[] vector = longColVector.vector;
+      long outputLong;
+      Category readerCategory = readerType.getCategory();
+      switch (readerCategory) {
       case BOOLEAN:
-        return input == 0 ? 0 : 1;
+        // No data loss for boolean.
+        vector[elementNum] = inputLong == 0 ? 0 : 1;
+        return;
       case BYTE:
-        return (byte) input;
+        outputLong = (byte) inputLong;
+        break;
       case SHORT:
-        return (short) input;
+        outputLong = (short) inputLong;
+        break;
       case INT:
-        return (int) input;
+        outputLong = (int) inputLong;
+        break;
       case LONG:
-        return input;
+        // No data loss for long.
+        vector[elementNum] = inputLong;
+        return;
       default:
-        throw new RuntimeException("Unexpected type kind " + readerType.getCategory().name());
+        throw new RuntimeException("Unexpected type kind " + readerCategory.name());
+      }
+
+      if (outputLong != inputLong) {
+        // Data loss.
+        longColVector.isNull[elementNum] = true;
+        longColVector.noNulls = false;
+      } else {
+        vector[elementNum] = outputLong;
       }
     }
 
@@ -439,25 +477,22 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
       anyIntegerAsLongTreeReader.nextVector(previousVector, isNull, batchSize);
       LongColumnVector resultColVector = (LongColumnVector) previousVector;
       if (downCastNeeded) {
-        long[] resultVector = resultColVector.vector;
         if (resultColVector.isRepeating) {
           if (resultColVector.noNulls || !resultColVector.isNull[0]) {
-            resultVector[0] = downCastAnyInteger(resultVector[0], readerType);
+            downCastAnyInteger(resultColVector, 0, readerType);
           } else {
-            resultColVector.noNulls = false;
-            resultColVector.isNull[0] = true;
+            // Result remains null.
           }
         } else if (resultColVector.noNulls){
           for (int i = 0; i < batchSize; i++) {
-            resultVector[i] = downCastAnyInteger(resultVector[i], readerType);
+            downCastAnyInteger(resultColVector, i, readerType);
           }
         } else {
           for (int i = 0; i < batchSize; i++) {
             if (!resultColVector.isNull[i]) {
-              resultVector[i] = downCastAnyInteger(resultVector[i], readerType);
+              downCastAnyInteger(resultColVector, i, readerType);
             } else {
-              resultColVector.noNulls = false;
-              resultColVector.isNull[i] = true;
+              // Result remains null.
             }
           }
         }
@@ -470,7 +505,6 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
     private FloatTreeReader floatTreeReader;
 
     private final TypeDescription readerType;
-    private FloatWritable floatResult;
     private DoubleColumnVector doubleColVector;
     private LongColumnVector longColVector;
 
@@ -480,15 +514,19 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
       this.readerType = readerType;
       floatTreeReader = new FloatTreeReader(columnId);
       setConvertTreeReader(floatTreeReader);
-      floatResult = new FloatWritable();
     }
 
     @Override
     public void setConvertVectorElement(int elementNum) throws IOException {
-      float floatValue = (float) doubleColVector.vector[elementNum];
-      longColVector.vector[elementNum] =
-          downCastAnyInteger(
-              (long) floatValue, readerType);
+      double doubleValue = doubleColVector.vector[elementNum];
+      if (!doubleCanFitInLong(doubleValue)) {
+        longColVector.isNull[elementNum] = true;
+        longColVector.noNulls = false;
+      } else {
+        // UNDONE: Does the overflow check above using double really work here for float?
+        float floatValue = (float) doubleValue;
+        downCastAnyInteger(longColVector, elementNum, (long) floatValue, readerType);
+      }
     }
 
     @Override
@@ -525,9 +563,13 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
 
     @Override
     public void setConvertVectorElement(int elementNum) throws IOException {
-      longColVector.vector[elementNum] =
-          downCastAnyInteger(
-              (long) doubleColVector.vector[elementNum], readerType);
+      double doubleValue = doubleColVector.vector[elementNum];
+      if (!doubleCanFitInLong(doubleValue)) {
+        longColVector.isNull[elementNum] = true;
+        longColVector.noNulls = false;
+      } else {
+        downCastAnyInteger(longColVector, elementNum, (long) doubleValue, readerType);
+      }
     }
 
     @Override
@@ -553,7 +595,6 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
     private final int precision;
     private final int scale;
     private final TypeDescription readerType;
-    private HiveDecimalWritable hiveDecimalResult;
     private DecimalColumnVector decimalColVector;
     private LongColumnVector longColVector;
 
@@ -565,15 +606,21 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
       this.readerType = readerType;
       decimalTreeReader = new DecimalTreeReader(columnId, precision, scale);
       setConvertTreeReader(decimalTreeReader);
-      hiveDecimalResult = new HiveDecimalWritable();
     }
 
+    private static HiveDecimal DECIMAL_MAX_LONG = HiveDecimal.create(Long.MAX_VALUE);
+    private static HiveDecimal DECIMAL_MIN_LONG = HiveDecimal.create(Long.MIN_VALUE);
+
     @Override
     public void setConvertVectorElement(int elementNum) throws IOException {
-      longColVector.vector[elementNum] =
-          downCastAnyInteger(
-              decimalColVector.vector[elementNum].getHiveDecimal().longValue(),
-              readerType);
+      HiveDecimal decimalValue = decimalColVector.vector[elementNum].getHiveDecimal();
+      if (decimalValue.compareTo(DECIMAL_MAX_LONG) > 0 ||
+          decimalValue.compareTo(DECIMAL_MIN_LONG) < 0) {
+        longColVector.isNull[elementNum] = true;
+        longColVector.noNulls = false;
+      } else {
+        downCastAnyInteger(longColVector, elementNum, decimalValue.longValue(), readerType);
+      }
     }
 
     @Override
@@ -596,7 +643,6 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
 
     private TreeReader stringGroupTreeReader;
 
-    private final TypeDescription fileType;
     private final TypeDescription readerType;
     private BytesColumnVector bytesColVector;
     private LongColumnVector longColVector;
@@ -604,7 +650,6 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
     AnyIntegerFromStringGroupTreeReader(int columnId, TypeDescription fileType,
         TypeDescription readerType) throws IOException {
       super(columnId);
-      this.fileType = fileType;
       this.readerType = readerType;
       stringGroupTreeReader = getStringGroupTreeReader(columnId, fileType);
       setConvertTreeReader(stringGroupTreeReader);
@@ -615,8 +660,7 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
       String string = stringFromBytesColumnVectorEntry(bytesColVector, elementNum);
       long longValue = parseLongFromString(string);
       if (!getIsParseError()) {
-        longColVector.vector[elementNum] =
-            downCastAnyInteger(longValue, readerType);
+        downCastAnyInteger(longColVector, elementNum, longValue, readerType);
       } else {
         longColVector.noNulls = false;
         longColVector.isNull[elementNum] = true;
@@ -660,8 +704,7 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
       // Use TimestampWritable's getSeconds.
       long longValue = TimestampUtils.millisToSeconds(
           timestampColVector.asScratchTimestamp(elementNum).getTime());
-      longColVector.vector[elementNum] =
-          downCastAnyInteger(longValue, readerType);
+      downCastAnyInteger(longColVector, elementNum, longValue, readerType);
     }
 
     @Override
@@ -745,8 +788,7 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
         if (resultColVector.noNulls || !resultColVector.isNull[0]) {
           resultVector[0] = (float) resultVector[0];
         } else {
-          resultColVector.noNulls = false;
-          resultColVector.isNull[0] = true;
+          // Remains null.
         }
       } else if (resultColVector.noNulls){
         for (int i = 0; i < batchSize; i++) {
@@ -757,8 +799,7 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
           if (!resultColVector.isNull[i]) {
             resultVector[i] = (float) resultVector[i];
           } else {
-            resultColVector.noNulls = false;
-            resultColVector.isNull[i] = true;
+            // Remains null.
           }
         }
       }
@@ -771,8 +812,6 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
 
     private final int precision;
     private final int scale;
-    private final TypeDescription readerType;
-    private HiveDecimalWritable hiveDecimalResult;
     private DecimalColumnVector decimalColVector;
     private DoubleColumnVector doubleColVector;
 
@@ -781,10 +820,8 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
       super(columnId);
       this.precision = fileType.getPrecision();
       this.scale = fileType.getScale();
-      this.readerType = readerType;
       decimalTreeReader = new DecimalTreeReader(columnId, precision, scale);
       setConvertTreeReader(decimalTreeReader);
-      hiveDecimalResult = new HiveDecimalWritable();
     }
 
     @Override
@@ -813,14 +850,12 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
 
     private TreeReader stringGroupTreeReader;
 
-    private final TypeDescription fileType;
     private BytesColumnVector bytesColVector;
     private DoubleColumnVector doubleColVector;
 
     FloatFromStringGroupTreeReader(int columnId, TypeDescription fileType)
         throws IOException {
       super(columnId);
-      this.fileType = fileType;
       stringGroupTreeReader = getStringGroupTreeReader(columnId, fileType);
       setConvertTreeReader(stringGroupTreeReader);
     }
@@ -858,14 +893,11 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
 
     private TimestampTreeReader timestampTreeReader;
 
-    private final TypeDescription readerType;
     private TimestampColumnVector timestampColVector;
     private DoubleColumnVector doubleColVector;
 
-    FloatFromTimestampTreeReader(int columnId, TypeDescription readerType,
-        boolean skipCorrupt) throws IOException {
+    FloatFromTimestampTreeReader(int columnId, boolean skipCorrupt) throws IOException {
       super(columnId);
-      this.readerType = readerType;
       timestampTreeReader = new TimestampTreeReader(columnId, skipCorrupt);
       setConvertTreeReader(timestampTreeReader);
     }
@@ -940,13 +972,10 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
 
     private FloatTreeReader floatTreeReader;
 
-    private FloatWritable floatResult;
-
     DoubleFromFloatTreeReader(int columnId) throws IOException {
       super(columnId);
       floatTreeReader = new FloatTreeReader(columnId);
       setConvertTreeReader(floatTreeReader);
-      floatResult = new FloatWritable();
     }
 
     @Override
@@ -964,20 +993,15 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
 
     private final int precision;
     private final int scale;
-    private final TypeDescription readerType;
-    private HiveDecimalWritable hiveDecimalResult;
     private DecimalColumnVector decimalColVector;
     private DoubleColumnVector doubleColVector;
 
-    DoubleFromDecimalTreeReader(int columnId, TypeDescription fileType,
-        TypeDescription readerType) throws IOException {
+    DoubleFromDecimalTreeReader(int columnId, TypeDescription fileType) throws IOException {
       super(columnId);
       this.precision = fileType.getPrecision();
       this.scale = fileType.getScale();
-      this.readerType = readerType;
       decimalTreeReader = new DecimalTreeReader(columnId, precision, scale);
       setConvertTreeReader(decimalTreeReader);
-      hiveDecimalResult = new HiveDecimalWritable();
     }
 
     @Override
@@ -1006,14 +1030,12 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
 
     private TreeReader stringGroupTreeReader;
 
-    private final TypeDescription fileType;
     private BytesColumnVector bytesColVector;
     private DoubleColumnVector doubleColVector;
 
     DoubleFromStringGroupTreeReader(int columnId, TypeDescription fileType)
         throws IOException {
       super(columnId);
-      this.fileType = fileType;
       stringGroupTreeReader = getStringGroupTreeReader(columnId, fileType);
       setConvertTreeReader(stringGroupTreeReader);
     }
@@ -1050,14 +1072,11 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
 
     private TimestampTreeReader timestampTreeReader;
 
-    private final TypeDescription readerType;
     private TimestampColumnVector timestampColVector;
     private DoubleColumnVector doubleColVector;
 
-    DoubleFromTimestampTreeReader(int columnId, TypeDescription readerType,
-        boolean skipCorrupt) throws IOException {
+    DoubleFromTimestampTreeReader(int columnId, boolean skipCorrupt) throws IOException {
       super(columnId);
-      this.readerType = readerType;
       timestampTreeReader = new TimestampTreeReader(columnId, skipCorrupt);
       setConvertTreeReader(timestampTreeReader);
     }
@@ -1088,16 +1107,12 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
 
     private AnyIntegerTreeReader anyIntegerAsLongTreeReader;
 
-    private int precision;
-    private int scale;
     private LongColumnVector longColVector;
     private DecimalColumnVector decimalColVector;
 
-    DecimalFromAnyIntegerTreeReader(int columnId, TypeDescription fileType,
-        TypeDescription readerType, boolean skipCorrupt) throws IOException {
+    DecimalFromAnyIntegerTreeReader(int columnId, TypeDescription fileType, boolean skipCorrupt)
+        throws IOException {
       super(columnId);
-      this.precision = readerType.getPrecision();
-      this.scale = readerType.getScale();
       anyIntegerAsLongTreeReader =
           new AnyIntegerTreeReader(columnId, fileType, skipCorrupt);
       setConvertTreeReader(anyIntegerAsLongTreeReader);
@@ -1106,8 +1121,8 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
     @Override
     public void setConvertVectorElement(int elementNum) {
       long longValue = longColVector.vector[elementNum];
-      HiveDecimalWritable hiveDecimalWritable =
-          new HiveDecimalWritable(longValue);
+      HiveDecimalWritable hiveDecimalWritable = new HiveDecimalWritable(longValue);
+      // The DecimalColumnVector will enforce precision and scale and set the entry to null when out of bounds.
       decimalColVector.set(elementNum, hiveDecimalWritable);
     }
 
@@ -1131,30 +1146,25 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
 
     private FloatTreeReader floatTreeReader;
 
-    private int precision;
-    private int scale;
-    private FloatWritable floatResult;
     private DoubleColumnVector doubleColVector;
     private DecimalColumnVector decimalColVector;
 
     DecimalFromFloatTreeReader(int columnId, TypeDescription readerType)
         throws IOException {
       super(columnId);
-      this.precision = readerType.getPrecision();
-      this.scale = readerType.getScale();
       floatTreeReader = new FloatTreeReader(columnId);
       setConvertTreeReader(floatTreeReader);
-      floatResult = new FloatWritable();
     }
 
     @Override
     public void setConvertVectorElement(int elementNum) throws IOException {
       float floatValue = (float) doubleColVector.vector[elementNum];
       if (!Float.isNaN(floatValue)) {
-        HiveDecimal value =
+        HiveDecimal decimalValue =
             HiveDecimal.create(Float.toString(floatValue));
-        if (value != null) {
-          decimalColVector.set(elementNum, value);
+        if (decimalValue != null) {
+          // The DecimalColumnVector will enforce precision and scale and set the entry to null when out of bounds.
+          decimalColVector.set(elementNum, decimalValue);
         } else {
           decimalColVector.noNulls = false;
           decimalColVector.isNull[elementNum] = true;
@@ -1227,14 +1237,12 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
 
     private TreeReader stringGroupTreeReader;
 
-    private final TypeDescription fileType;
     private BytesColumnVector bytesColVector;
     private DecimalColumnVector decimalColVector;
 
     DecimalFromStringGroupTreeReader(int columnId, TypeDescription fileType,
         TypeDescription readerType) throws IOException {
       super(columnId);
-      this.fileType = fileType;
       stringGroupTreeReader = getStringGroupTreeReader(columnId, fileType);
       setConvertTreeReader(stringGroupTreeReader);
     }
@@ -1244,6 +1252,7 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
       String string = stringFromBytesColumnVectorEntry(bytesColVector, elementNum);
       HiveDecimal value = parseDecimalFromString(string);
       if (value != null) {
+        // The DecimalColumnVector will enforce precision and scale and set the entry to null when out of bounds.
         decimalColVector.set(elementNum, value);
       } else {
         decimalColVector.noNulls = false;
@@ -1271,18 +1280,11 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
 
     private TimestampTreeReader timestampTreeReader;
 
-    private final TypeDescription readerType;
     private TimestampColumnVector timestampColVector;
-    private int precision;
-    private int scale;
     private DecimalColumnVector decimalColVector;
 
-    DecimalFromTimestampTreeReader(int columnId, TypeDescription readerType,
-        boolean skipCorrupt) throws IOException {
+    DecimalFromTimestampTreeReader(int columnId, boolean skipCorrupt) throws IOException {
       super(columnId);
-      this.readerType = readerType;
-      this.precision = readerType.getPrecision();
-      this.scale = readerType.getScale();
       timestampTreeReader = new TimestampTreeReader(columnId, skipCorrupt);
       setConvertTreeReader(timestampTreeReader);
     }
@@ -1293,6 +1295,7 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
           timestampColVector.asScratchTimestamp(elementNum));
       HiveDecimal value = HiveDecimal.create(Double.toString(doubleValue));
       if (value != null) {
+        // The DecimalColumnVector will enforce precision and scale and set the entry to null when out of bounds.
         decimalColVector.set(elementNum, value);
       } else {
         decimalColVector.noNulls = false;
@@ -1320,8 +1323,6 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
 
     private DecimalTreeReader decimalTreeReader;
 
-    private final TypeDescription fileType;
-    private final TypeDescription readerType;
     private DecimalColumnVector fileDecimalColVector;
     private int filePrecision;
     private int fileScale;
@@ -1332,10 +1333,8 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
     DecimalFromDecimalTreeReader(int columnId, TypeDescription fileType, TypeDescription readerType)
         throws IOException {
       super(columnId);
-      this.fileType = fileType;
       filePrecision = fileType.getPrecision();
       fileScale = fileType.getScale();
-      this.readerType = readerType;
       readerPrecision = readerType.getPrecision();
       readerScale = readerType.getScale();
       decimalTreeReader = new DecimalTreeReader(columnId, filePrecision, fileScale);
@@ -1375,7 +1374,6 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
 
     private AnyIntegerTreeReader anyIntegerAsLongTreeReader;
 
-    private final TypeDescription fileType;
     private final TypeDescription readerType;
     private LongColumnVector longColVector;
     private BytesColumnVector bytesColVector;
@@ -1383,7 +1381,6 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
     StringGroupFromAnyIntegerTreeReader(int columnId, TypeDescription fileType,
         TypeDescription readerType, boolean skipCorrupt) throws IOException {
       super(columnId);
-      this.fileType = fileType;
       this.readerType = readerType;
       anyIntegerAsLongTreeReader =
           new AnyIntegerTreeReader(columnId, fileType, skipCorrupt);
@@ -1419,7 +1416,6 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
     private FloatTreeReader floatTreeReader;
 
     private final TypeDescription readerType;
-    private FloatWritable floatResult;
     private DoubleColumnVector doubleColVector;
     private BytesColumnVector bytesColVector;
 
@@ -1430,7 +1426,6 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
       this.readerType = readerType;
       floatTreeReader = new FloatTreeReader(columnId);
       setConvertTreeReader(floatTreeReader);
-      floatResult = new FloatWritable();
     }
 
     @Override
@@ -1599,7 +1594,6 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
     private final TypeDescription readerType;
     private LongColumnVector longColVector;
     private BytesColumnVector bytesColVector;
-    private DateWritable dateWritableResult;
     private Date date;
 
     StringGroupFromDateTreeReader(int columnId, TypeDescription readerType,
@@ -1608,7 +1602,6 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
       this.readerType = readerType;
       dateTreeReader = new DateTreeReader(columnId);
       setConvertTreeReader(dateTreeReader);
-      dateWritableResult = new DateWritable();
       date = new Date(0);
     }
 
@@ -1640,13 +1633,11 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
 
     private TreeReader stringGroupTreeReader;
 
-    private final TypeDescription fileType;
     private final TypeDescription readerType;
 
     StringGroupFromStringGroupTreeReader(int columnId, TypeDescription fileType,
         TypeDescription readerType) throws IOException {
       super(columnId);
-      this.fileType = fileType;
       this.readerType = readerType;
       stringGroupTreeReader = getStringGroupTreeReader(columnId, fileType);
       setConvertTreeReader(stringGroupTreeReader);
@@ -1664,8 +1655,7 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
         if (resultColVector.noNulls || !resultColVector.isNull[0]) {
           convertStringGroupVectorElement(resultColVector, 0, readerType);
         } else {
-          resultColVector.noNulls = false;
-          resultColVector.isNull[0] = true;
+          // Remains null.
         }
       } else if (resultColVector.noNulls){
         for (int i = 0; i < batchSize; i++) {
@@ -1676,8 +1666,7 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
           if (!resultColVector.isNull[i]) {
             convertStringGroupVectorElement(resultColVector, i, readerType);
           } else {
-            resultColVector.noNulls = false;
-            resultColVector.isNull[i] = true;
+            // Remains null.
           }
         }
       }
@@ -1689,7 +1678,6 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
     private BinaryTreeReader binaryTreeReader;
 
     private final TypeDescription readerType;
-    private BytesWritable binaryWritableResult;
     private BytesColumnVector inBytesColVector;
     private BytesColumnVector outBytesColVector;
 
@@ -1699,7 +1687,6 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
       this.readerType = readerType;
       binaryTreeReader = new BinaryTreeReader(columnId);
       setConvertTreeReader(binaryTreeReader);
-      binaryWritableResult = new BytesWritable();
     }
 
     @Override
@@ -1780,7 +1767,6 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
 
     private FloatTreeReader floatTreeReader;
 
-    private FloatWritable floatResult;
     private DoubleColumnVector doubleColVector;
     private TimestampColumnVector timestampColVector;
 
@@ -1789,14 +1775,14 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
       super(columnId);
       floatTreeReader = new FloatTreeReader(columnId);
       setConvertTreeReader(floatTreeReader);
-      floatResult = new FloatWritable();
     }
 
     @Override
     public void setConvertVectorElement(int elementNum) {
       float floatValue = (float) doubleColVector.vector[elementNum];
-      timestampColVector.set(elementNum,
-          TimestampUtils.doubleToTimestamp(floatValue));
+      Timestamp timestampValue = TimestampUtils.doubleToTimestamp(floatValue);
+      // The TimestampColumnVector will set the entry to null when a null timestamp is passed in.
+      timestampColVector.set(elementNum, timestampValue);
     }
 
     @Override
@@ -1832,8 +1818,9 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
     @Override
     public void setConvertVectorElement(int elementNum) {
       double doubleValue = doubleColVector.vector[elementNum];
-      timestampColVector.set(elementNum,
-          TimestampUtils.doubleToTimestamp(doubleValue));
+      Timestamp timestampValue = TimestampUtils.doubleToTimestamp(doubleValue);
+      // The TimestampColumnVector will set the entry to null when a null timestamp is passed in.
+      timestampColVector.set(elementNum, timestampValue);
     }
 
     @Override
@@ -1858,7 +1845,6 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
 
     private final int precision;
     private final int scale;
-    private HiveDecimalWritable hiveDecimalResult;
     private DecimalColumnVector decimalColVector;
     private TimestampColumnVector timestampColVector;
 
@@ -1869,14 +1855,14 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
       this.scale = fileType.getScale();
       decimalTreeReader = new DecimalTreeReader(columnId, precision, scale);
       setConvertTreeReader(decimalTreeReader);
-      hiveDecimalResult = new HiveDecimalWritable();
     }
 
     @Override
     public void setConvertVectorElement(int elementNum) {
       Timestamp timestampValue =
-          TimestampUtils.decimalToTimestamp(
-              decimalColVector.vector[elementNum].getHiveDecimal());
+            TimestampUtils.decimalToTimestamp(
+                decimalColVector.vector[elementNum].getHiveDecimal());
+      // The TimestampColumnVector will set the entry to null when a null timestamp is passed in.
       timestampColVector.set(elementNum, timestampValue);
     }
 
@@ -1900,14 +1886,12 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
 
     private TreeReader stringGroupTreeReader;
 
-    private final TypeDescription fileType;
     private BytesColumnVector bytesColVector;
     private TimestampColumnVector timestampColVector;
 
     TimestampFromStringGroupTreeReader(int columnId, TypeDescription fileType)
         throws IOException {
       super(columnId);
-      this.fileType = fileType;
       stringGroupTreeReader = getStringGroupTreeReader(columnId, fileType);
       setConvertTreeReader(stringGroupTreeReader);
     }
@@ -1945,7 +1929,6 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
 
     private DateTreeReader dateTreeReader;
 
-    private DateWritable doubleResult;
     private LongColumnVector longColVector;
     private TimestampColumnVector timestampColVector;
 
@@ -1954,7 +1937,6 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
       super(columnId);
       dateTreeReader = new DateTreeReader(columnId);
       setConvertTreeReader(dateTreeReader);
-      doubleResult = new DateWritable();
     }
 
     @Override
@@ -1984,14 +1966,12 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
 
     private TreeReader stringGroupTreeReader;
 
-    private final TypeDescription fileType;
     private BytesColumnVector bytesColVector;
     private LongColumnVector longColVector;
 
     DateFromStringGroupTreeReader(int columnId, TypeDescription fileType)
         throws IOException {
       super(columnId);
-      this.fileType = fileType;
       stringGroupTreeReader = getStringGroupTreeReader(columnId, fileType);
       setConvertTreeReader(stringGroupTreeReader);
     }
@@ -2029,14 +2009,11 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
 
     private TimestampTreeReader timestampTreeReader;
 
-    private final TypeDescription readerType;
     private TimestampColumnVector timestampColVector;
     private LongColumnVector longColVector;
 
-    DateFromTimestampTreeReader(int columnId, TypeDescription readerType,
-        boolean skipCorrupt) throws IOException {
+    DateFromTimestampTreeReader(int columnId, boolean skipCorrupt) throws IOException {
       super(columnId);
-      this.readerType = readerType;
       timestampTreeReader = new TimestampTreeReader(columnId, skipCorrupt);
       setConvertTreeReader(timestampTreeReader);
     }
@@ -2069,12 +2046,9 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
 
     private TreeReader stringGroupTreeReader;
 
-    private final TypeDescription fileType;
-
     BinaryFromStringGroupTreeReader(int columnId, TypeDescription fileType)
         throws IOException {
       super(columnId);
-      this.fileType = fileType;
       stringGroupTreeReader = getStringGroupTreeReader(columnId, fileType);
       setConvertTreeReader(stringGroupTreeReader);
     }
@@ -2119,7 +2093,7 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
           skipCorrupt);
 
     case DECIMAL:
-      return new DecimalFromAnyIntegerTreeReader(columnId, fileType, readerType, skipCorrupt);
+      return new DecimalFromAnyIntegerTreeReader(columnId, fileType, skipCorrupt);
 
     case STRING:
     case CHAR:
@@ -2263,7 +2237,7 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
       return new FloatFromDecimalTreeReader(columnId, fileType, readerType);
 
     case DOUBLE:
-      return new DoubleFromDecimalTreeReader(columnId, fileType, readerType);
+      return new DoubleFromDecimalTreeReader(columnId, fileType);
 
     case STRING:
     case CHAR:
@@ -2477,13 +2451,13 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
       return new AnyIntegerFromTimestampTreeReader(columnId, readerType, skipCorrupt);
 
     case FLOAT:
-      return new FloatFromTimestampTreeReader(columnId, readerType, skipCorrupt);
+      return new FloatFromTimestampTreeReader(columnId, skipCorrupt);
 
     case DOUBLE:
-      return new DoubleFromTimestampTreeReader(columnId, readerType, skipCorrupt);
+      return new DoubleFromTimestampTreeReader(columnId, skipCorrupt);
 
     case DECIMAL:
-      return new DecimalFromTimestampTreeReader(columnId, readerType, skipCorrupt);
+      return new DecimalFromTimestampTreeReader(columnId, skipCorrupt);
 
     case STRING:
     case CHAR:
@@ -2495,7 +2469,7 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
           readerType.getCategory() + " to self needed");
 
     case DATE:
-      return new DateFromTimestampTreeReader(columnId, readerType, skipCorrupt);
+      return new DateFromTimestampTreeReader(columnId, skipCorrupt);
 
     // Not currently supported conversion(s):
     case BINARY:

http://git-wip-us.apache.org/repos/asf/hive/blob/66a2ded1/ql/src/test/queries/clientpositive/schema_evol_orc_nonvec_mapwork_part_all_primitive.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/schema_evol_orc_nonvec_mapwork_part_all_primitive.q b/ql/src/test/queries/clientpositive/schema_evol_orc_nonvec_mapwork_part_all_primitive.q
index f8aebc8..899b4bb 100644
--- a/ql/src/test/queries/clientpositive/schema_evol_orc_nonvec_mapwork_part_all_primitive.q
+++ b/ql/src/test/queries/clientpositive/schema_evol_orc_nonvec_mapwork_part_all_primitive.q
@@ -52,14 +52,16 @@ drop table part_change_various_various_boolean;
 
 --
 -- SUBSECTION: ALTER TABLE CHANGE COLUMNS for Various --> Various: (BOOLEAN, SHORT, INT, LONG, FLOAT, DOUBLE, DECIMAL, STRING, CHAR, VARCHAR, TIMESTAMP) --> BYTE
+-- -128 and a maximum value of 127
 --
 CREATE TABLE part_change_various_various_tinyint(insert_num int, c1 BOOLEAN, c2 SMALLINT, c3 INT, c4 BIGINT, c5 FLOAT, c6 DOUBLE, c7 DECIMAL(38,18), c8 STRING, c9 CHAR(25), c10 VARCHAR(25), c11 TIMESTAMP, b STRING) PARTITIONED BY(part INT);
 
 insert into table part_change_various_various_tinyint partition(part=1)
-    values(1, true, 2000, 72909, 3244222, -29.0764, 470614135, 470614135, '-2999', '-2999', '-2999', '0004-09-22 18:26:29.519542222', 'original'),
-          (2, 0, 1000, 483777, -23866739993, -3651.672121, 46114.284799488, 46114.284799488, '', '', '', '2007-02-09 05:17:29.368756876', 'original'),
-          (3, false, 72909, 3244222, -93222, 30.774, -66475.561431, -66475.561431, '1', '1', '1', '6229-06-28 02:54:28.970117179', 'original'),
-          (4, 1, 90000, 754072151, 3289094, 46114.284799488 ,9250340.75, 9250340.75, '5299', '5299', '5299', '2002-05-10 05:29:48.990818073', 'original');
+    values(1, true, 2000, 72909, 3244222, -29.0764, 470614135, 470614135, '129', '-128', '-2999', '0004-09-22 18:26:29.519542222', 'original'),
+          (2, 0, -128, -48, -20, -9223372036854775808.0, -9223372036854775808.0, 9223372036854775807.0, '128', '-99', '40', '2007-02-09 05:17:29.368756876', 'original'),
+          (3, -1, -129, 100, 499, -9223372036854775809.0, -9223372036854775809.0, 9223372036854775808.0, '128', '-99', '40', '2007-02-09 05:17:29.368756876', 'original'),
+          (4, false, -72, -127, 127, 30.774, 127.561431, -106.561431, '90.284799488', '90.284799488', '1', '6229-06-28 02:54:28.970117179', 'original'),
+          (5, 1, -90, 75, -38, 109.284799488 ,-128.75, 98.75, '120.4', '33.333', '0.45', '2002-05-10 05:29:48.990818073', 'original');
 
 select insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,b from part_change_various_various_tinyint order by insert_num;
 
@@ -67,10 +69,10 @@ select insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,b from part_change_var
 alter table part_change_various_various_tinyint replace columns (insert_num int, c1 TINYINT, c2 TINYINT, c3 TINYINT, c4 TINYINT, c5 TINYINT, c6 TINYINT, c7 TINYINT, c8 TINYINT, c9 TINYINT, c10 TINYINT, c11 TINYINT, b STRING);
 
 insert into table part_change_various_various_tinyint partition(part=2)
-    values (5, 23, 71, 127, 1, 131, -60, 68, -230, -182, 40, 93, 'new');
+    values (6, 23, 71, 127, 1, 131, -60, 68, -230, -182, 40, 93, 'new');
 
 insert into table part_change_various_various_tinyint partition(part=1)
-    values (6, -248, 85, -126, -167, 91, 113, -28, -63, 0, 8, 237, 'new');
+    values (7, -120, 85, -126, -167, 91, 113, -28, -63, 0, 8, 237, 'new');
 
 explain
 select insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,b from part_change_various_various_tinyint order by insert_num;
@@ -83,14 +85,16 @@ drop table part_change_various_various_tinyint;
 
 --
 -- SUBSECTION: ALTER TABLE CHANGE COLUMNS for Various --> Various: (BOOLEAN, TINYINT, INT, LONG, FLOAT, DOUBLE, DECIMAL, STRING, CHAR, VARCHAR, TIMESTAMP) --> SMALLINT
+-- -32768 and a maximum value of 32767 
 --
 CREATE TABLE part_change_various_various_smallint(insert_num int, c1 BOOLEAN, c2 TINYINT, c3 INT, c4 BIGINT, c5 FLOAT, c6 DOUBLE, c7 DECIMAL(38,18), c8 STRING, c9 CHAR(25), c10 VARCHAR(25), c11 TIMESTAMP, b STRING) PARTITIONED BY(part INT);
 
 insert into table part_change_various_various_smallint partition(part=1)
     values(1, -2999, 200, 72909, 3244222, -29.0764, 470614135, 470614135, '-2999', '-2999', '-2999', '0004-09-22 18:26:29.519542222', 'original'),
-          (2, 0, 100, 483777, -23866739993, -3651.672121, 46114.284799488, 46114.284799488, '', '', '', '2007-02-09 05:17:29.368756876', 'original'),
-          (3, false, 72, 3244222, -93222, 30.774, -66475.561431, -66475.561431, '1', '1', '1', '6229-06-28 02:54:28.970117179', 'original'),
-          (4, 1, -90, 754072151, 3289094, 46114.284799488 ,9250340.75, 9250340.75, '5299', '5299', '5299', '2002-05-10 05:29:48.990818073', 'original');
+          (2, 0, 100, -32768 , 32767, -3651.672121, -9223372036854775808.0, 9223372036854775807.0, '9000', '32767', '-32768', '2007-02-09 05:17:29.368756876', 'original'),
+          (3, 0, -127, -40000 , 32768, -3651.672121, -9223372036854775809.0, 9223372036854775808.0, '9000', '32767', '-32768', '2007-02-09 05:17:29.368756876', 'original'),
+          (4, false, 72, 32422, -9322, 30.774, -6675.561431, -6675.561431, '1', '1', '1', '6229-06-28 02:54:28.970117179', 'original'),
+          (5, 1, -90, 7151, 3094, 30000.284799488 ,-9000.75, 0.75, '5299', '5299', '5299', '2002-05-10 05:29:48.990818073', 'original');
 
 select insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,b from part_change_various_various_smallint order by insert_num;
 
@@ -98,10 +102,10 @@ select insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,b from part_change_var
 alter table part_change_various_various_smallint replace columns (insert_num int, c1 SMALLINT, c2 SMALLINT, c3 SMALLINT, c4 SMALLINT, c5 SMALLINT, c6 SMALLINT, c7 SMALLINT, c8 SMALLINT, c9 SMALLINT, c10 SMALLINT, c11 SMALLINT, b STRING);
 
 insert into table part_change_various_various_smallint partition(part=2)
-    values (5, -30486, 15230, 3117, 1, -117, -7131, 20227, -24858, -28771, 46114, 72909, 'new');
+    values (6, -30486, 15230, 3117, 1, -117, -7131, 20227, -24858, -28771, 46114, 72909, 'new');
 
 insert into table part_change_various_various_smallint partition(part=1)
-    values (6, -10542, -1805, -4844, 15507, 91, 22385, -28, -12268, 0, 66475, 774, 'new');
+    values (7, -10542, -1805, -4844, 15507, 91, 22385, -28, -12268, 0, 66475, 774, 'new');
 
 explain
 select insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,b from part_change_various_various_smallint order by insert_num;
@@ -113,14 +117,16 @@ drop table part_change_various_various_smallint;
 
 --
 -- SUBSECTION: ALTER TABLE CHANGE COLUMNS for Various --> Various: (BOOLEAN, TINYINT, SMALLINT, LONG, FLOAT, DOUBLE, DECIMAL, STRING, CHAR, VARCHAR, TIMESTAMP) --> INT
+-- \u20132147483648 to 2147483647
 --
 CREATE TABLE part_change_various_various_int(insert_num int, c1 BOOLEAN, c2 TINYINT, c3 SMALLINT, c4 BIGINT, c5 FLOAT, c6 DOUBLE, c7 DECIMAL(38,18), c8 STRING, c9 CHAR(25), c10 VARCHAR(25), c11 TIMESTAMP, b STRING) PARTITIONED BY(part INT);
 
 insert into table part_change_various_various_int partition(part=1)
     values(1, -2999, 200, 72909, 3244222, -29.0764, 470614135, 470614135, '-2999', '-2999', '-2999', '0004-09-22 18:26:29.519542222', 'original'),
-          (2, 0, 100, 483777, -23866739993, -3651.672121, 46114.284799488, 46114.284799488, '', '', '', '2007-02-09 05:17:29.368756876', 'original'),
-          (3, false, 72, 3244222, -93222, 30.774, -66475.561431, -66475.561431, '1', '1', '1', '6229-06-28 02:54:28.970117179', 'original'),
-          (4, 1, -90, 754072151, 3289094, 46114.284799488 ,9250340.75, 9250340.75, '5299', '5299', '5299', '2002-05-10 05:29:48.990818073', 'original');
+          (2, 0, 100, 2147483647, -23866739993, -3651.672121, -9223372036854775808.0, 9223372036854775807.0, '', '', '', '2007-02-09 05:17:29.368756876', 'original'),
+          (3, 0, 100, 2147483648, -23866739993, -3651.672121, -9223372036854775809.0, 9223372036854775808.0, '', '', '', '2007-02-09 05:17:29.368756876', 'original'),
+          (4, false, 72, 3244222, -93222, 30.774, -66475.561431, -66475.561431, '1', '1', '1', '6229-06-28 02:54:28.970117179', 'original'),
+          (5, 1, -90, 754072151, 3289094, 46114.284799488 ,9250340.75, 9250340.75, '5299', '5299', '5299', '2002-05-10 05:29:48.990818073', 'original');
 
 select insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,b from part_change_various_various_int order by insert_num;
 
@@ -144,14 +150,16 @@ drop table part_change_various_various_int;
 
 --
 -- SUBSECTION: ALTER TABLE CHANGE COLUMNS for Various --> Various: (BOOLEAN, TINYINT, SMALLINT, INT, FLOAT, DOUBLE, DECIMAL, STRING, CHAR, VARCHAR, TIMESTAMP) --> BIGINT
+-- -9223372036854775808 to 9223372036854775807
 --
 CREATE TABLE part_change_various_various_bigint(insert_num int, c1 BOOLEAN, c2 TINYINT, c3 SMALLINT, c4 INT, c5 FLOAT, c6 DOUBLE, c7 DECIMAL(38,18), c8 STRING, c9 CHAR(25), c10 VARCHAR(25), c11 TIMESTAMP, b STRING) PARTITIONED BY(part INT);
 
 insert into table part_change_various_various_bigint partition(part=1)
     values(1, -2999, 200, 72909, 3244222, -29.0764, 470614135, 470614135, '-2999', '-2999', '-2999', '0004-09-22 18:26:29.519542222', 'original'),
-          (2, 0, 100, 483777, -23866739993, -3651.672121, 46114.284799488, 46114.284799488, '', '', '', '2007-02-09 05:17:29.368756876', 'original'),
-          (3, false, 72, 3244222, -93222, 30.774, -66475.561431, -66475.561431, '1', '1', '1', '6229-06-28 02:54:28.970117179', 'original'),
-          (4, 1, -90, 754072151, 3289094, 46114.284799488 ,9250340.75, 9250340.75, '1998287.3541', '1998287.3541', '1998287.3541', '2002-05-10 05:29:48.990818073', 'original');
+          (2, 0, 100, 32767, -23372036854775, -3651.672121, -9223372036854775808.0, 9223372036854775807.0, '', '', '', '2007-02-09 05:17:29.368756876', 'original'),
+          (3, 0, 100, -32768, 23372036854775, -3651.672121, -9223372036854775809.0, 9223372036854775808.0, '', '', '', '2007-02-09 05:17:29.368756876', 'original'),
+          (4, false, 72, 3244222, -93222, 30.774, -66475.561431, -66475.561431, '1', '1', '1', '6229-06-28 02:54:28.970117179', 'original'),
+          (5, 1, -90, 754072151, 3289094, 46114.284799488 ,9250340.75, 9250340.75, '1998287.3541', '1998287.3541', '1998287.3541', '2002-05-10 05:29:48.990818073', 'original');
 
 select insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,b from part_change_various_various_bigint order by insert_num;
 
@@ -159,10 +167,10 @@ select insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,b from part_change_var
 alter table part_change_various_various_bigint replace columns (insert_num int, c1 BIGINT, c2 BIGINT, c3 BIGINT, c4 BIGINT, c5 BIGINT, c6 BIGINT, c7 BIGINT, c8 BIGINT, c9 BIGINT, c10 BIGINT, c11 BIGINT, b STRING);
 
 insert into table part_change_various_various_bigint partition(part=2)
-    values (5, 5573199346255528403, 71, 151775655, 1, 131, -60, 6275638713485623898, -230, -695025, 519542222, -29.0764, 'new');
+    values (6, 5573199346255528403, 71, 151775655, 1, 131, -60, 6275638713485623898, -230, -695025, 519542222, -29.0764, 'new');
 
 insert into table part_change_various_various_bigint partition(part=1)
-    values (6, -164341325, 9043162437544575070, -126, -6566204574741299000, 91, 113, -28, -63, 0, 3244222, -90, 'new');
+    values (7, -164341325, 9043162437544575070, -126, -6566204574741299000, 91, 113, -28, -63, 0, 3244222, -90, 'new');
 
 explain
 select insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,b from part_change_various_various_bigint order by insert_num;
@@ -180,9 +188,10 @@ CREATE TABLE part_change_various_various_float(insert_num int, c1 BOOLEAN, c2 TI
 
 insert into table part_change_various_various_float partition(part=1)
     values(1, -2999, 200, 72909, 3244222, -29.0764, 470614135, 470614135, '-2999', '-2999', '-2999', '0004-09-22 18:26:29.519542222', 'original'),
-          (2, 0, 100, 483777, -23866739993, -3651.672121, 46114.284799488, 46114.284799488, '', '', '', '2007-02-09 05:17:29.368756876', 'original'),
-          (3, false, 72, 3244222, -93222, 30.774, -66475.561431, -66475.561431, '2402.3', '2402.3', '2402.3', '6229-06-28 02:54:28.970117179', 'original'),
-          (4, 1, -90, 754072151, 3289094, 46114.284799488 ,9250340.75, 9250340.75, '5299', '5299', '5299', '2002-05-10 05:29:48.990818073', 'original');
+          (2, 0, 100, 32767, -23372036854775, -3651.672121, -9223372036854775808.0, 9223372036854775807.0, '', '', '', '2007-02-09 05:17:29.368756876', 'original'),
+          (3, 0, 100, -32768, 23372036854775, -3651.672121, -9223372036854775809.0, 9223372036854775808.0, '', '', '', '2007-02-09 05:17:29.368756876', 'original'),
+          (4, false, 72, 3244222, -93222, 30.774, -66475.561431, -66475.561431, '2402.3', '2402.3', '2402.3', '6229-06-28 02:54:28.970117179', 'original'),
+          (5, 1, -90, 754072151, 3289094, 46114.284799488 ,9250340.75, 9250340.75, '5299', '5299', '5299', '2002-05-10 05:29:48.990818073', 'original');
 
 select insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,b from part_change_various_various_float order by insert_num;
 
@@ -190,10 +199,10 @@ select insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,b from part_change_var
 alter table part_change_various_various_float replace columns (insert_num int, c1 FLOAT, c2 FLOAT, c3 FLOAT, c4 FLOAT, c5 FLOAT, c6 FLOAT, c7 FLOAT, c8 FLOAT, c9 FLOAT, c10 FLOAT, c11 FLOAT, b STRING);
 
 insert into table part_change_various_various_float partition(part=2)
-    values (5, 953967041., 62.0791539559013466, 718.78, 1, 203.199548118, -60, 6275638713485623898, -230, -695025, -3651.67212, 46114.28, 'new');
+    values (6, 953967041., 62.0791539559013466, 718.78, 1, 203.199548118, -60, 6275638713485623898, -230, -695025, -3651.67212, 46114.28, 'new');
 
 insert into table part_change_various_various_float partition(part=1)
-    values (6, -1255178165.77663, 9043162437544575070.974, -4314.7918, -1240033819, 91, 1698.95, -100.3597812, -63, 0, -93222.200, 29.076, 'new');
+    values (7, -1255178165.77663, 9043162437544575070.974, -4314.7918, -1240033819, 91, 1698.95, -100.3597812, -63, 0, -93222.200, 29.076, 'new');
 
 explain
 select insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,b from part_change_various_various_float order by insert_num;
@@ -211,9 +220,10 @@ CREATE TABLE part_change_various_various_double(insert_num int, c1 BOOLEAN, c2 T
 
 insert into table part_change_various_various_double partition(part=1)
     values(1, -2999, 200, 72909, 3244222, -29.0764, 470614135, 470614135, '-2999', '-2999', '-2999', '0004-09-22 18:26:29.519542222', 'original'),
-          (2, 0, 100, 483777, -23866739993, -3651.672121, 46114.284799488, 46114.284799488, '', '', '', '2007-02-09 05:17:29.368756876', 'original'),
-          (3, false, 72, 3244222, -93222, 30.774, -66475.561431, -66475.561431, '1', '1', '1', '6229-06-28 02:54:28.970117179', 'original'),
-          (4, 1, -90, 754072151, 3289094, 46114.284799488 ,9250340.75, 9250340.75, '5299', '5299', '5299', '2002-05-10 05:29:48.990818073', 'original');
+          (2, 0, 100, 32767, -23372036854775, -3651.672121, -9223372036854775808.0, 9223372036854775807.0, '', '', '', '2007-02-09 05:17:29.368756876', 'original'),
+          (3, 0, 100, -32768, 23372036854775, -3651.672121, -9223372036854775809.0, 9223372036854775808.0, '', '', '', '2007-02-09 05:17:29.368756876', 'original'),
+          (4, false, 72, 3244222, -93222, 30.774, -66475.561431, -66475.561431, '1', '1', '1', '6229-06-28 02:54:28.970117179', 'original'),
+          (5, 1, -90, 754072151, 3289094, 46114.284799488 ,9250340.75, 9250340.75, '5299', '5299', '5299', '2002-05-10 05:29:48.990818073', 'original');
 
 select insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,b from part_change_various_various_double order by insert_num;
 
@@ -221,10 +231,10 @@ select insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,b from part_change_var
 alter table part_change_various_various_double replace columns (insert_num int, c1 DOUBLE, c2 DOUBLE, c3 DOUBLE, c4 DOUBLE, c5 DOUBLE, c6 DOUBLE, c7 DOUBLE, c8 DOUBLE, c9 DOUBLE, c10 DOUBLE, c11 DOUBLE, b STRING);
 
 insert into table part_change_various_various_double partition(part=2)
-    values (5, 953967041., 62.0791539559013466, 718.78, 1, 203.199548118, -60, 6275638713485623898, -230, -695025, 0.00007011717, 4.28479948, 'new');
+    values (6, 953967041., 62.0791539559013466, 718.78, 1, 203.199548118, -60, 6275638713485623898, -230, -695025, 0.00007011717, 4.28479948, 'new');
 
 insert into table part_change_various_various_double partition(part=1)
-    values (6, -1255178165.77663, 9043162437544575070.974, -4314.7918, -1240033819, 91, 1698.95, -100.3597812, -63, 0, -66475.0000008, -284799488.1, 'new');
+    values (7, -1255178165.77663, 9043162437544575070.974, -4314.7918, -1240033819, 91, 1698.95, -100.3597812, -63, 0, -66475.0000008, -284799488.1, 'new');
 
 explain
 select insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,b from part_change_various_various_double order by insert_num;
@@ -242,9 +252,10 @@ CREATE TABLE part_change_various_various_decimal(insert_num int, c1 BOOLEAN, c2
 
 insert into table part_change_various_various_decimal partition(part=1)
     values(1, -2999, 200, 72909, 3244222, -29.0764, 470614135, 470614135, '--1551801.09502', '--1551801.09502', '--1551801.09502', '0004-09-22 18:26:29.519542222', 'original'),
-          (2, 0, 100, 483777, -23866739993, -3651.672121, 46114.284799488, 46114.284799488, '', '', '', '2007-02-09 05:17:29.368756876', 'original'),
-          (3, false, 72, 3244222, -93222, 30.774, -66475.561431, -66475.561431, '1', '1', '1', '6229-06-28 02:54:28.970117179', 'original'),
-          (4, 1, -90, 754072151, 3289094, 46114.284799488 ,9250340.75, 9250340.75, '2402.3', '2402.3', '2402.3', '2002-05-10 05:29:48.990818073', 'original');
+          (2, 0, 100, 32767, -23372036854775, -3651.672121, -9223372036854775808.0, 9223372036854775807.0, '', '', '', '2007-02-09 05:17:29.368756876', 'original'),
+          (3, 0, 100, -32768, 23372036854775, -3651.672121, -9223372036854775809.0, 9223372036854775808.0, '', '', '', '2007-02-09 05:17:29.368756876', 'original'),
+          (4, false, 72, 3244222, -93222, 30.774, -66475.561431, -66475.561431, '1', '1', '1', '6229-06-28 02:54:28.970117179', 'original'),
+          (5, 1, -90, 754072151, 3289094, 46114.284799488 ,9250340.75, 9250340.75, '2402.3', '2402.3', '2402.3', '2002-05-10 05:29:48.990818073', 'original');
 
 select insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,b from part_change_various_various_decimal order by insert_num;
 
@@ -252,10 +263,10 @@ select insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,b from part_change_var
 alter table part_change_various_various_decimal replace columns (insert_num int, c1 DECIMAL(38,18), c2 DECIMAL(38,18), c3 DECIMAL(38,18), c4 DECIMAL(38,18), c5 DECIMAL(38,18), c6 DECIMAL(38,18), c7 DECIMAL(38,18), c8 DECIMAL(38,18), c9 DECIMAL(38,18), c10 DECIMAL(38,18), c11 DECIMAL(38,18), b STRING);
 
 insert into table part_change_various_various_decimal partition(part=2)
-    values (5, 953967041., 62.0791539559013466, 718.78, 1, 203.199548118, -60, 6275638713485623898, -230, -695025, 0.00007011717, 4.28479948, 'new');
+    values (6, 953967041., 62.0791539559013466, 718.78, 1, 203.199548118, -60, 6275638713485623898, -230, -695025, 0.00007011717, 4.28479948, 'new');
 
 insert into table part_change_various_various_decimal partition(part=1)
-    values (6,-1255178165.77663, 9043162437544575070.974, -4314.7918, -1240033819, 91, 1698.95, -100.3597812, -63, 0, -66475.0000008, -284799488.1, 'new');
+    values (7,-1255178165.77663, 9043162437544575070.974, -4314.7918, -1240033819, 91, 1698.95, -100.3597812, -63, 0, -66475.0000008, -284799488.1, 'new');
 
 explain
 select insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,b from part_change_various_various_decimal order by insert_num;
@@ -273,9 +284,10 @@ CREATE TABLE part_change_various_various_string(insert_num int, c1 BOOLEAN, c2 T
 
 insert into table part_change_various_various_string partition(part=1)
     values(1, true,  200,  72909,      3244222, -99999999999,     -29.0764,      470614135,        470614135,         'dynamic reptile  ', 'dynamic reptile  ',  '0004-09-22 18:26:29.519542222', '2007-02-09', 'binary', 'original'),
-          (2, 0,     100,  483777,      14,     -23866739993,     -3651.672121,  46114.284799488,  46114.284799488,   '  baffling    ', '  baffling    ',        '2007-02-09 05:17:29.368756876', '0004-09-22', 'binary', 'original'),
-          (3, false, 72,   3244222,    -93222,   30.774,       -   66475.561431, -66475.561431,    0.561431,          '1', '1',                                  '6229-06-28 02:54:28.970117179', '5966-07-09', 'binary', 'original'),
-          (4, 1,    -90,   754072151,   3289094, 46114.284799488,  9250340.75,    9250340.75,      9250340.75,        'junkyard', 'junkyard',                    '2002-05-10 05:29:48.990818073', '1815-05-06', 'binary', 'original');
+          (2, 0, 100, 32767, -23372036854775, -3651.672121, -9223372036854775808.0, 9223372036854775807.0, '', '', '', '2007-02-09 05:17:29.368756876', 'original'),
+          (3, 0, 100, -32768, 23372036854775, -3651.672121, -9223372036854775809.0, 9223372036854775808.0, '', '', '', '2007-02-09 05:17:29.368756876', 'original'),
+          (4, false, 72,   3244222,    -93222,   30.774,       -   66475.561431, -66475.561431,    0.561431,          '1', '1',                                  '6229-06-28 02:54:28.970117179', '5966-07-09', 'binary', 'original'),
+          (5, 1,    -90,   754072151,   3289094, 46114.284799488,  9250340.75,    9250340.75,      9250340.75,        'junkyard', 'junkyard',                    '2002-05-10 05:29:48.990818073', '1815-05-06', 'binary', 'original');
 
 select insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,c13,b from part_change_various_various_string order by insert_num;
 
@@ -283,10 +295,10 @@ select insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,c13,b from part_ch
 alter table part_change_various_various_string replace columns (insert_num int, c1 STRING, c2 STRING, c3 STRING, c4 STRING, c5 STRING, c6 STRING, c7 STRING, c8 STRING, c9 STRING, c10 STRING, c11 STRING, c12 STRING, c13 STRING, b STRING);
 
 insert into table part_change_various_various_string partition(part=2)
-    values (5, 'true', '400',  '44388',       -'100',    '953967041.',       '62.079153',     '718.78',         '1',                'verdict', 'verdict', 'timestamp', 'date', 'binary', 'new');
+    values (6, 'true', '400',  '44388',       -'100',    '953967041.',       '62.079153',     '718.78',         '1',                'verdict', 'verdict', 'timestamp', 'date', 'binary', 'new');
 
 insert into table part_change_various_various_string partition(part=1)
-    values (6,-'false', '-67', '833',          '63993', ' 1255178165.77663', '905070.974', '-4314.7918',        -'1240033819',      'trial',   'trial',  '2016-03-07 03:02:22.0', '2016-03-07', 'binary', 'new');
+    values (7,-'false', '-67', '833',          '63993', ' 1255178165.77663', '905070.974', '-4314.7918',        -'1240033819',      'trial',   'trial',  '2016-03-07 03:02:22.0', '2016-03-07', 'binary', 'new');
 
 explain
 select insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,c13,b from part_change_various_various_string order by insert_num;
@@ -304,9 +316,10 @@ CREATE TABLE part_change_various_various_char(insert_num int, c1 BOOLEAN, c2 TIN
 
 insert into table part_change_various_various_char partition(part=1)
     values(1, true,  200,  72909,      3244222, -99999999999,     -29.0764,      470614135,        470614135,         'dynamic reptile  ', 'dynamic reptile  ',  '0004-09-22 18:26:29.519542222', '2007-02-09', 'binary', 'original'),
-          (2, 0,     100,  483777,      14,     -23866739993,     -3651.672121,  46114.284799488,  46114.284799488,   '  baffling    ', '  baffling    ',        '2007-02-09 05:17:29.368756876', '0004-09-22', 'binary', 'original'),
-          (3, false, 72,   3244222,    -93222,   30.774,       -   66475.561431, -66475.561431,    0.561431,         '1', '1',                                  '6229-06-28 02:54:28.970117179', '5966-07-09', 'binary', 'original'),
-          (4, 1,    -90,   754072151,   3289094, 46114.284799488,  9250340.75,    9250340.75,      9250340.75,        'junkyard', 'junkyard',                    '2002-05-10 05:29:48.990818073', '1815-05-06', 'binary', 'original');
+          (2, 0, 100, 32767, -23372036854775, -3651.672121, -9223372036854775808.0, 9223372036854775807.0, '', '', '', '2007-02-09 05:17:29.368756876', 'original'),
+          (3, 0, 100, -32768, 23372036854775, -3651.672121, -9223372036854775809.0, 9223372036854775808.0, '', '', '', '2007-02-09 05:17:29.368756876', 'original'),
+          (4, false, 72,   3244222,    -93222,   30.774,       -   66475.561431, -66475.561431,    0.561431,         '1', '1',                                  '6229-06-28 02:54:28.970117179', '5966-07-09', 'binary', 'original'),
+          (5, 1,    -90,   754072151,   3289094, 46114.284799488,  9250340.75,    9250340.75,      9250340.75,        'junkyard', 'junkyard',                    '2002-05-10 05:29:48.990818073', '1815-05-06', 'binary', 'original');
 
 select insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,c13,b from part_change_various_various_char order by insert_num;
 
@@ -314,10 +327,10 @@ select insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,c13,b from part_ch
 alter table part_change_various_various_char replace columns (insert_num int, c1 CHAR(25), c2 CHAR(25), c3 CHAR(25), c4 CHAR(25), c5 CHAR(25), c6 CHAR(25), c7 CHAR(25), c8 CHAR(25), c9 CHAR(25), c10 CHAR(25), c11 CHAR(25), c12 CHAR(25), c13 CHAR(25), b STRING);
 
 insert into table part_change_various_various_char partition(part=2)
-    values (5, 'true', '400',  '44388',       -'100',    '953967041.',       '62.079153',     '718.78',         '1',                'verdict', 'verdict', 'timestamp', 'date', 'binary', 'new');
+    values (6, 'true', '400',  '44388',       -'100',    '953967041.',       '62.079153',     '718.78',         '1',                'verdict', 'verdict', 'timestamp', 'date', 'binary', 'new');
 
 insert into table part_change_various_various_char partition(part=1)
-    values (6,-'false', '-67', '833',          '63993', ' 1255178165.77663', '905070.974', '-4314.7918',        -'1240033819',      'trial',   'trial',  '2016-03-07 03:02:22.0', '2016-03-07', 'binary', 'new');
+    values (7,-'false', '-67', '833',          '63993', ' 1255178165.77663', '905070.974', '-4314.7918',        -'1240033819',      'trial',   'trial',  '2016-03-07 03:02:22.0', '2016-03-07', 'binary', 'new');
 
 explain
 select insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,c13,b from part_change_various_various_char order by insert_num;
@@ -335,9 +348,10 @@ CREATE TABLE part_change_various_various_char_trunc(insert_num int, c1 BOOLEAN,
 
 insert into table part_change_various_various_char_trunc partition(part=1)
     values(1, true,  200,  72909,      3244222, -99999999999,     -29.0764,      470614135,        470614135,         'dynamic reptile  ', 'dynamic reptile  ',  '0004-09-22 18:26:29.519542222', '2007-02-09', 'binary', 'original'),
-          (2, 0,     100,  483777,      14,     -23866739993,     -3651.672121,  46114.284799488,  46114.284799488,   '  baffling    ', '  baffling    ',        '2007-02-09 05:17:29.368756876', '0004-09-22', 'binary', 'original'),
-          (3, false, 72,   3244222,    -93222,   30.774,       -   66475.561431, -66475.561431,    0.561431,         '1', '1',                                  '6229-06-28 02:54:28.970117179', '5966-07-09', 'binary', 'original'),
-          (4, 1,    -90,   754072151,   3289094, 46114.284799488,  9250340.75,    9250340.75,      9250340.75,        'junkyard', 'junkyard',                    '2002-05-10 05:29:48.990818073', '1815-05-06', 'binary', 'original');
+          (2, 0, 100, 32767, -23372036854775, -3651.672121, -9223372036854775808.0, 9223372036854775807.0, '', '', '', '2007-02-09 05:17:29.368756876', 'original'),
+          (3, 0, 100, -32768, 23372036854775, -3651.672121, -9223372036854775809.0, 9223372036854775808.0, '', '', '', '2007-02-09 05:17:29.368756876', 'original'),
+          (4, false, 72,   3244222,    -93222,   30.774,       -   66475.561431, -66475.561431,    0.561431,         '1', '1',                                  '6229-06-28 02:54:28.970117179', '5966-07-09', 'binary', 'original'),
+          (5, 1,    -90,   754072151,   3289094, 46114.284799488,  9250340.75,    9250340.75,      9250340.75,        'junkyard', 'junkyard',                    '2002-05-10 05:29:48.990818073', '1815-05-06', 'binary', 'original');
 
 select insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,c13,b from part_change_various_various_char_trunc order by insert_num;
 
@@ -345,10 +359,10 @@ select insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,c13,b from part_ch
 alter table part_change_various_various_char_trunc replace columns (insert_num int, c1 CHAR(8), c2 CHAR(8), c3 CHAR(8), c4 CHAR(8), c5 CHAR(8), c6 CHAR(8), c7 CHAR(8), c8 CHAR(8), c9 CHAR(8), c10 CHAR(8), c11 CHAR(8), c12 CHAR(8), c13 CHAR(8), b STRING);
 
 insert into table part_change_various_various_char_trunc partition(part=2)
-    values (5, 'true', '400',  '44388',       -'100',    '953967041.',       '62.079153',     '718.78',         '1',                'verdict', 'verdict', 'timestamp', 'date', 'binary', 'new');
+    values (6, 'true', '400',  '44388',       -'100',    '953967041.',       '62.079153',     '718.78',         '1',                'verdict', 'verdict', 'timestamp', 'date', 'binary', 'new');
 
 insert into table part_change_various_various_char_trunc partition(part=1)
-    values (6,-'false', '-67', '833',          '63993', ' 1255178165.77663', '905070.974', '-4314.7918',        -'1240033819',      'trial',   'trial',  '2016-03-07 03:02:22.0', '2016-03-07', 'binary', 'new');
+    values (7,-'false', '-67', '833',          '63993', ' 1255178165.77663', '905070.974', '-4314.7918',        -'1240033819',      'trial',   'trial',  '2016-03-07 03:02:22.0', '2016-03-07', 'binary', 'new');
 
 explain
 select insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,c13,b from part_change_various_various_char_trunc order by insert_num;
@@ -366,9 +380,10 @@ CREATE TABLE part_change_various_various_varchar(insert_num int, c1 BOOLEAN, c2
 
 insert into table part_change_various_various_varchar partition(part=1)
     values(1, true,  200,  72909,      3244222, -99999999999,     -29.0764,      470614135,        470614135,         'dynamic reptile  ', 'dynamic reptile  ',  '0004-09-22 18:26:29.519542222', '2007-02-09', 'binary', 'original'),
-          (2, 0,     100,  483777,      14,     -23866739993,     -3651.672121,  46114.284799488,  46114.284799488,   '  baffling    ', '  baffling    ',        '2007-02-09 05:17:29.368756876', '0004-09-22', 'binary', 'original'),
-          (3, false, 72,   3244222,    -93222,   30.774,       -   66475.561431, -66475.561431,    0.561431,         '1', '1',                                  '6229-06-28 02:54:28.970117179', '5966-07-09', 'binary', 'original'),
-          (4, 1,    -90,   754072151,   3289094, 46114.284799488,  9250340.75,    9250340.75,      9250340.75,        'junkyard', 'junkyard',                    '2002-05-10 05:29:48.990818073', '1815-05-06', 'binary', 'original');
+          (2, 0, 100, 32767, -23372036854775, -9223372036854775808.0, -9223372036854775808.0, 9223372036854775807.0, '', '', '', '2007-02-09 05:17:29.368756876', 'original'),
+          (3, 0, 100, -32768, 23372036854775, -9223372036854775809.0, -9223372036854775809.0, 9223372036854775808.0, '', '', '', '2007-02-09 05:17:29.368756876', 'original'),
+          (4, false, 72,   3244222,    -93222,   30.774,       -   66475.561431, -66475.561431,    0.561431,         '1', '1',                                  '6229-06-28 02:54:28.970117179', '5966-07-09', 'binary', 'original'),
+          (5, 1,    -90,   754072151,   3289094, 46114.284799488,  9250340.75,    9250340.75,      9250340.75,        'junkyard', 'junkyard',                    '2002-05-10 05:29:48.990818073', '1815-05-06', 'binary', 'original');
 
 select insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,c13,b from part_change_various_various_varchar order by insert_num;
 
@@ -376,10 +391,10 @@ select insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,c13,b from part_ch
 alter table part_change_various_various_varchar replace columns (insert_num int, c1 VARCHAR(25), c2 VARCHAR(25), c3 VARCHAR(25), c4 VARCHAR(25), c5 VARCHAR(25), c6 VARCHAR(25), c7 VARCHAR(25), c8 VARCHAR(25), c9 VARCHAR(25), c10 VARCHAR(25), c11 VARCHAR(25), c12 VARCHAR(25), c13 VARCHAR(25), b STRING);
 
 insert into table part_change_various_various_varchar partition(part=2)
-    values (5, 'true', '400',  '44388',       -'100',    '953967041.',       '62.079153',     '718.78',         '1',                'verdict', 'verdict', 'timestamp', 'date', 'binary', 'new');
+    values (6, 'true', '400',  '44388',       -'100',    '953967041.',       '62.079153',     '718.78',         '1',                'verdict', 'verdict', 'timestamp', 'date', 'binary', 'new');
 
 insert into table part_change_various_various_varchar partition(part=1)
-    values (6,-'false', '-67', '833',          '63993', ' 1255178165.77663', '905070.974', '-4314.7918',        -'1240033819',      'trial',   'trial',  '2016-03-07 03:02:22.0', '2016-03-07', 'binary', 'new');
+    values (7,-'false', '-67', '833',          '63993', ' 1255178165.77663', '905070.974', '-4314.7918',        -'1240033819',      'trial',   'trial',  '2016-03-07 03:02:22.0', '2016-03-07', 'binary', 'new');
 
 explain
 select insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,c13,b from part_change_various_various_varchar order by insert_num;
@@ -397,7 +412,8 @@ CREATE TABLE part_change_various_various_varchar_trunc(insert_num int, c1 BOOLEA
 
 insert into table part_change_various_various_varchar_trunc partition(part=1)
     values(1, true,  200,  72909,      3244222, -99999999999,     -29.0764,      470614135,        470614135,         'dynamic reptile  ', 'dynamic reptile  ',  '0004-09-22 18:26:29.519542222', '2007-02-09', 'binary', 'original'),
-          (2, 0,     100,  483777,      14,     -23866739993,     -3651.672121,  46114.284799488,  46114.284799488,   '  baffling    ', '  baffling    ',        '2007-02-09 05:17:29.368756876', '0004-09-22', 'binary', 'original'),
+          (2, 0, 100, 32767, -23372036854775, -9223372036854775808.0, -9223372036854775808.0, 9223372036854775807.0, '', '', '', '2007-02-09 05:17:29.368756876', 'original'),
+          (3, 0, 100, -32768, 23372036854775, -9223372036854775809.0, -9223372036854775809.0, 9223372036854775808.0, '', '', '', '2007-02-09 05:17:29.368756876', 'original'),
           (3, false, 72,   3244222,    -93222,   30.774,       -   66475.561431, -66475.561431,    0.561431,         '1', '1',                                  '6229-06-28 02:54:28.970117179', '5966-07-09', 'binary', 'original'),
           (4, 1,    -90,   754072151,   3289094, 46114.284799488,  9250340.75,    9250340.75,      9250340.75,        'junkyard', 'junkyard',                    '2002-05-10 05:29:48.990818073', '1815-05-06', 'binary', 'original');
 
@@ -428,9 +444,10 @@ CREATE TABLE part_change_various_various_timestamp(insert_num int, c1 BOOLEAN, c
 
 insert into table part_change_various_various_timestamp partition(part=1)
     values(1, true,  200,  72909,      3244222, -99999999999,     -29.0764,      470614135,        470614135,         '0004-09-22 18:26:29.519542222', '0004-09-22 18:26:29.519542222',  '0004-09-22 18:26:29.519542222', '2007-02-09', 'original'),
-          (2, 0,     100,  483777,      14,     -23866739993,     -3651.672121,  46114.284799488,  46114.284799488,   '2007-02-09 05:17:29.368756876', '2007-02-09 05:17:29.368756876',  '2007-02-09 05:17:29.368756876', '0004-09-22', 'original'),
-          (3, false, 72,   3244222,    -93222,   30.774,       -   66475.561431, -66475.561431,    0.561431,          '6229-06-28 02:54:28.970117179', '6229-06-28 02:54:28.970117179',  '6229-06-28 02:54:28.970117179', '5966-07-09', 'original'),
-          (4, 1,    -90,   754072151,   3289094, 46114.284799488,  9250340.75,    9250340.75,      9250340.75,        '2002-05-10 05:29:48.990818073', '2002-05-10 05:29:48.990818073',  '2002-05-10 05:29:48.990818073', '1815-05-06', 'original');
+          (2, 0, 100, 32767, -23372036854775, -9223372036854775808.0, -9223372036854775808.0, 9223372036854775807.0, '', '', '', '2007-02-09 05:17:29.368756876', 'original'),
+          (3, 0, 100, -32768, 23372036854775, -9223372036854775809.0, -9223372036854775809.0, 9223372036854775808.0, '', '', '', '2007-02-09 05:17:29.368756876', 'original'),
+          (4, false, 72,   3244222,    -93222,   30.774,       -   66475.561431, -66475.561431,    0.561431,          '6229-06-28 02:54:28.970117179', '6229-06-28 02:54:28.970117179',  '6229-06-28 02:54:28.970117179', '5966-07-09', 'original'),
+          (5, 1,    -90,   754072151,   3289094, 46114.284799488,  9250340.75,    9250340.75,      9250340.75,        '2002-05-10 05:29:48.990818073', '2002-05-10 05:29:48.990818073',  '2002-05-10 05:29:48.990818073', '1815-05-06', 'original');
 
 select insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,b from part_change_various_various_timestamp order by insert_num;
 
@@ -438,10 +455,10 @@ select insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,b from part_change
 alter table part_change_various_various_timestamp replace columns (insert_num int, c1 TIMESTAMP, c2 TIMESTAMP, c3 TIMESTAMP, c4 TIMESTAMP, c5 TIMESTAMP, c6 TIMESTAMP, c7 TIMESTAMP, c8 TIMESTAMP, c9 TIMESTAMP, c10 TIMESTAMP, c11 TIMESTAMP, c12 TIMESTAMP, b STRING);
 
 insert into table part_change_various_various_timestamp partition(part=2)
-    values (5, 'true', '400',  '44388',       -'100',    '953967041.',       '62.079153',     '718.78',         '1',                'timestamp', 'timestamp', 'timestamp', 'date', 'new');
+    values (6, 'true', '400',  '44388',       -'100',    '953967041.',       '62.079153',     '718.78',         '1',                'timestamp', 'timestamp', 'timestamp', 'date', 'new');
 
 insert into table part_change_various_various_timestamp partition(part=1)
-    values (6,-'false', '-67', '833',          '63993', ' 1255178165.77663', '905070.974', '-4314.7918',        -'1240033819',      '2016-03-07 03:02:22.0',   '2016-03-07 03:02:22.0',  '2016-03-07 03:02:22.0', '2016-03-07', 'new');
+    values (7,-'false', '-67', '833',          '63993', ' 1255178165.77663', '905070.974', '-4314.7918',        -'1240033819',      '2016-03-07 03:02:22.0',   '2016-03-07 03:02:22.0',  '2016-03-07 03:02:22.0', '2016-03-07', 'new');
 
 explain
 select insert_num,part,c1,c2,c3,c4,c5,c6,c7,c8,c9,c10,c11,c12,b from part_change_various_various_timestamp order by insert_num;