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/12/22 08:32:30 UTC

[01/10] hive git commit: HIVE-15335: Fast Decimal (Matt McCline, reviewed by Sergey Shelukhin, Prasanth Jayachandran, Owen O'Malley)

Repository: hive
Updated Branches:
  refs/heads/master 597ca1bdc -> 4ba713ccd


http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/storage-api/src/test/org/apache/hadoop/hive/common/type/TestHiveDecimalVersion.java
----------------------------------------------------------------------
diff --git a/storage-api/src/test/org/apache/hadoop/hive/common/type/TestHiveDecimalVersion.java b/storage-api/src/test/org/apache/hadoop/hive/common/type/TestHiveDecimalVersion.java
new file mode 100644
index 0000000..cf80b0c
--- /dev/null
+++ b/storage-api/src/test/org/apache/hadoop/hive/common/type/TestHiveDecimalVersion.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.common.type;
+
+import java.lang.annotation.Annotation;
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.lang.reflect.Field;
+import java.sql.Timestamp;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.Arrays;
+import java.util.TreeSet;
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.math.BigDecimal;
+import java.math.BigInteger;
+
+import org.apache.hadoop.hive.VersionTestBase;
+import org.junit.*;
+
+import static org.junit.Assert.*;
+
+public class TestHiveDecimalVersion extends VersionTestBase {
+
+  /*
+   * Validation:
+   * 1) Substitute class name for "ThisClass".
+   * 2) Only public fields and methods are versioned.
+   * 3) Methods compare on [non-]static, return type, name, parameter types, exceptions thrown.
+   * 4) Fields compare on [non-]static, type, name, value when static
+   */
+  @Test
+  public void testVerifyHiveDecimalPublicMethodsAndFieldsVersions() throws IllegalAccessException {
+
+    Map<Class, String> versionedClassToNameMap = new HashMap<Class, String>();
+    versionedClassToNameMap.put(HiveDecimalV1.class, "HiveDecimal");
+    versionedClassToNameMap.put(HiveDecimal.class, "HiveDecimal");
+
+    doVerifyVersions(
+        HiveDecimalV1.class, HiveDecimalVersionV1.class,
+        HiveDecimal.class, HiveDecimalVersionV2.class,
+        versionedClassToNameMap);
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/storage-api/src/test/org/apache/hadoop/hive/serde2/io/TestHiveDecimalWritable.java
----------------------------------------------------------------------
diff --git a/storage-api/src/test/org/apache/hadoop/hive/serde2/io/TestHiveDecimalWritable.java b/storage-api/src/test/org/apache/hadoop/hive/serde2/io/TestHiveDecimalWritable.java
new file mode 100644
index 0000000..b8f5472
--- /dev/null
+++ b/storage-api/src/test/org/apache/hadoop/hive/serde2/io/TestHiveDecimalWritable.java
@@ -0,0 +1,52 @@
+/**
+ * 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.serde2.io;
+
+import org.junit.*;
+import static org.junit.Assert.*;
+
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.ArrayList;
+
+import org.apache.hadoop.hive.common.type.HiveDecimal;
+
+/**
+ * Unit tests for tsting the fast allocation-free conversion
+ * between HiveDecimalWritable and Decimal128
+ */
+public class TestHiveDecimalWritable {
+
+  @Test
+  public void testHiveDecimalWritable() {
+
+    HiveDecimalWritable decWritable;
+
+    HiveDecimal nullDec = null;
+    decWritable = new HiveDecimalWritable(nullDec);
+    assertTrue(!decWritable.isSet());
+    decWritable = new HiveDecimalWritable("1");
+    assertTrue(decWritable.isSet());
+
+    // UNDONE: more!
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/storage-api/src/test/org/apache/hadoop/hive/serde2/io/TestHiveDecimalWritableVersion.java
----------------------------------------------------------------------
diff --git a/storage-api/src/test/org/apache/hadoop/hive/serde2/io/TestHiveDecimalWritableVersion.java b/storage-api/src/test/org/apache/hadoop/hive/serde2/io/TestHiveDecimalWritableVersion.java
new file mode 100644
index 0000000..8e0f396
--- /dev/null
+++ b/storage-api/src/test/org/apache/hadoop/hive/serde2/io/TestHiveDecimalWritableVersion.java
@@ -0,0 +1,71 @@
+/**
+ * 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.serde2.io;
+
+import java.lang.annotation.Annotation;
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.lang.reflect.Field;
+import java.sql.Timestamp;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.Arrays;
+import java.util.TreeSet;
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.math.BigDecimal;
+import java.math.BigInteger;
+
+import org.apache.hadoop.hive.VersionTestBase;
+import org.apache.hadoop.hive.common.type.HiveDecimal;
+import org.apache.hadoop.hive.common.type.HiveDecimalV1;
+import org.junit.*;
+
+import static org.junit.Assert.*;
+
+public class TestHiveDecimalWritableVersion extends VersionTestBase {
+
+  /*
+   * Validation:
+   * 1) Substitute class name for "ThisClass".
+   * 2) Only public fields and methods are versioned.
+   * 3) Methods compare on [non-]static, return type, name, parameter types, exceptions thrown.
+   * 4) Fields compare on [non-]static, type, name, value when static
+   */
+  @Test
+  public void testVerifyHiveDecimalWritablePublicMethodsAndFieldsVersions() throws IllegalAccessException {
+
+    Map<Class, String> versionedClassToNameMap = new HashMap<Class, String>();
+    versionedClassToNameMap.put(HiveDecimalV1.class, "HiveDecimal");
+    versionedClassToNameMap.put(HiveDecimal.class, "HiveDecimal");
+    versionedClassToNameMap.put(HiveDecimalWritableV1.class, "HiveDecimalWritable");
+    versionedClassToNameMap.put(HiveDecimalWritable.class, "HiveDecimalWritable");
+
+    doVerifyVersions(
+        HiveDecimalWritableV1.class, HiveDecimalWritableVersionV1.class,
+        HiveDecimalWritable.class, HiveDecimalWritableVersionV2.class,
+        versionedClassToNameMap);
+
+  }
+}


[03/10] hive git commit: HIVE-15335: Fast Decimal (Matt McCline, reviewed by Sergey Shelukhin, Prasanth Jayachandran, Owen O'Malley)

Posted by mm...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/storage-api/src/java/org/apache/hadoop/hive/serde2/io/HiveDecimalWritable.java
----------------------------------------------------------------------
diff --git a/storage-api/src/java/org/apache/hadoop/hive/serde2/io/HiveDecimalWritable.java b/storage-api/src/java/org/apache/hadoop/hive/serde2/io/HiveDecimalWritable.java
index 41452da..94d61b4 100644
--- a/storage-api/src/java/org/apache/hadoop/hive/serde2/io/HiveDecimalWritable.java
+++ b/storage-api/src/java/org/apache/hadoop/hive/serde2/io/HiveDecimalWritable.java
@@ -17,154 +17,992 @@
  */
 package org.apache.hadoop.hive.serde2.io;
 
+import java.util.Arrays;
 import java.io.DataInput;
 import java.io.DataOutput;
+import java.io.InputStream;
+import java.io.OutputStream;
 import java.io.IOException;
 import java.math.BigInteger;
 
 import org.apache.hadoop.hive.common.type.HiveDecimal;
-
+import org.apache.hadoop.hive.common.type.FastHiveDecimal;
+import org.apache.hadoop.hive.common.type.FastHiveDecimalImpl;
+import org.apache.hadoop.hive.common.type.HiveDecimalVersionV2;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.io.WritableUtils;
 
-public class HiveDecimalWritable implements WritableComparable<HiveDecimalWritable> {
+/**
+ * A mutable decimal.
+ * <p>
+ * ------------------------------------- Version 2 ------------------------------------------------
+ * <p>
+ * This is the 2nd major version of HiveDecimalWritable called V2.  The previous version has been
+ * renamed to HiveDecimalWritableV1 and is kept as a test and behavior reference.
+ * <p>
+ * For good performance we do not represent the decimal using a byte array containing BigInteger
+ * bytes like the previous version V1 did.  Instead V2 HiveDecimalWritable is is a private subclass
+ * of the same fast decimal class also used by HiveDecimal.  So it stores the decimal value
+ * directly.
+ * <p>
+ * Many of the methods of HiveDecimal have been added to HiveDecimalWritable in V2 so code can
+ * modify the decimal instead of calling getHiveDecimal(), doing operations on HiveDecimal, and then
+ * setting HiveDecimalWritable back to the result.
+ *  <p>
+ * Operations that modify have a prefix of "mutate" in their name.  For example mutateAdd is used
+ * instead of the immutable operation add in HiveDecimal that returns a new decimal object.
+ * <p>
+ * This should have much better performance.
+ * <p>
+ * The original V1 public methods and fields are annotated with @HiveDecimalWritableVersionV1; new
+ * public methods and fields are annotated with @HiveDecimalWritableVersionV2.
+ *
+ */
+public final class HiveDecimalWritable extends FastHiveDecimal
+    implements WritableComparable<HiveDecimalWritable> {
+
+  // Is the decimal value currently valid?
+  private boolean isSet;
 
-  private byte[] internalStorage = new byte[0];
-  private int scale;
+  /*
+   * Scratch arrays used in fastBigIntegerBytes calls for better performance.
+   */
+
+  // An optional long array of length FastHiveDecimal.FAST_SCRATCH_LONGS_LEN.
+  private long[] internalScratchLongs;
+
+  // An optional byte array of FastHiveDecimal.FAST_SCRATCH_BUFFER_LEN_BIG_INTEGER_BYTES.
+  private byte[] internalScratchBuffer;
 
+  /**
+   * Create a decimal writable with no current value (i.e. isSet() will return false).
+   */
+  @HiveDecimalWritableVersionV1
   public HiveDecimalWritable() {
+    super();
+    isSet = false;
+    internalScratchLongs = null;
+    internalScratchBuffer = null;
   }
 
-  public HiveDecimalWritable(String value) {
-    set(HiveDecimal.create(value));
+  /**
+   * Create a decimal writable with an initial value from a String.
+   * <p>
+   * If the conversion from String to decimal is successful, isSet() will return true.
+   *
+   */
+  @HiveDecimalWritableVersionV1
+  public HiveDecimalWritable(String string) {
+    super();
+    isSet = fastSetFromString(string, false);
+    if (!isSet) {
+      fastReset();
+    }
   }
 
-  public HiveDecimalWritable(byte[] bytes, int scale) {
-    set(bytes, scale);
+  /**
+   * Create a decimal writable with an initial value from BigInteger bytes and a specified scale.
+   * <p>
+   * If the conversion to decimal is successful, isSet() will return true.
+   *
+   */
+  @HiveDecimalWritableVersionV1
+  public HiveDecimalWritable(byte[] bigIntegerBytes, int scale) {
+    super();
+    setFromBigIntegerBytesAndScale(bigIntegerBytes, scale);
   }
 
+  /**
+   * Create a decimal writable with an initial value from another decimal writable.
+   * <p>
+   * If the supplied writable has a value, isSet() will return true.
+   *
+   */
+  @HiveDecimalWritableVersionV1
   public HiveDecimalWritable(HiveDecimalWritable writable) {
-    set(writable.getHiveDecimal());
+    super();
+    set(writable);
   }
 
+  /**
+   * Create a decimal writable with an initial value from a HiveDecimal.
+   * <p>
+   * Afterwards, the isSet() method will return true, unless value is null.
+   *
+   */
+  @HiveDecimalWritableVersionV1
   public HiveDecimalWritable(HiveDecimal value) {
+    super();
     set(value);
   }
 
-  public HiveDecimalWritable(long value) {
-    set((HiveDecimal.create(value)));
+  /**
+   * Create a decimal writable with an initial value from a long with scale 0.
+   * <p>
+   * Afterwards, the isSet() method will return true.
+   *
+   */
+  @HiveDecimalWritableVersionV1
+  public HiveDecimalWritable(long longValue) {
+    super();
+    setFromLong(longValue);
   }
 
+  /**
+   * Set the writable's current value to a HiveDecimal's value.
+   * <p>
+   * Afterwards, the isSet() method will return true, unless value is null.
+   *
+   */
+  @HiveDecimalWritableVersionV1
   public void set(HiveDecimal value) {
-    set(value.unscaledValue().toByteArray(), value.scale());
+    if (value == null) {
+      fastReset();
+      isSet = false;
+    } else {
+      fastSet(value);
+      isSet = true;
+    }
   }
 
+  /**
+   * Set the writable's current value to a HiveDecimal's value with a specified precision / scale
+   * enforced.
+   * <p>
+   * Afterwards, the isSet() method will return true, unless value is null or value didn't fit within
+   * maxPrecision / maxScale.
+   *
+   */
+  @HiveDecimalWritableVersionV1
   public void set(HiveDecimal value, int maxPrecision, int maxScale) {
-    set(HiveDecimal.enforcePrecisionScale(value, maxPrecision, maxScale));
+    set(value);
+    if (isSet) {
+      isSet = fastEnforcePrecisionScale(maxPrecision, maxScale);
+      if (!isSet) {
+        fastReset();
+      }
+    }
   }
 
+  /**
+   * Set the writable's current value to the value in a another decimal writable.
+   * <p>
+   * If the supplied writable has a value, isSet() will return true.
+   *
+   */
+  @HiveDecimalWritableVersionV1
   public void set(HiveDecimalWritable writable) {
-    set(writable.getHiveDecimal());
+    if (writable == null || !writable.isSet()) {
+      fastReset();
+      isSet = false;
+    } else {
+      fastSet(writable);
+      isSet = true;
+    }
+  }
+
+  /**
+   * Set a decimal writable's value from BigInteger bytes and a specified scale.
+   * <p>
+   * If the conversion to decimal is successful, isSet() will return true.
+   *
+   */
+  @HiveDecimalWritableVersionV1
+  public void set(byte[] bigIntegerBytes, int scale) {
+    setFromBigIntegerBytesAndScale(bigIntegerBytes, scale);
+  }
+
+  /**
+   * Set the writable's current value to a writable's value with a specified precision / scale
+   * enforced.
+   * <p>
+   * The isSet() method will return true, unless value is null or value didn't fit within
+   * maxPrecision / maxScale.
+   *
+   */
+  @HiveDecimalWritableVersionV2
+  public void set(HiveDecimalWritable writable, int maxPrecision, int maxScale) {
+    set(writable);
+    if (isSet) {
+      isSet = fastEnforcePrecisionScale(maxPrecision, maxScale);
+      if (!isSet) {
+        fastReset();
+      }
+    }
+  }
+
+  /**
+   * Set a decimal writable's value to a long's value with scale 0.
+   * <p>
+   * Afterwards, the isSet() method will return true since all long values fit in a decimal.
+   *
+   */
+  @HiveDecimalWritableVersionV2
+  public void setFromLong(long longValue) {
+    fastReset();
+    fastSetFromLong(longValue);
+    isSet = true;
+  }
+
+  /**
+   * Set a decimal writable's value to a doubles value.
+   * <p>
+   * Afterwards, the isSet() method will return true if the double to decimal conversion was successful.
+   *
+   */
+  @HiveDecimalWritableVersionV2
+  public void setFromDouble(double doubleValue) {
+    fastReset();
+    isSet = fastSetFromDouble(doubleValue);
+    if (!isSet) {
+      fastReset();
+    }
+  }
+
+  /**
+   * Set the writable's current value to the decimal in a UTF-8 byte slice.
+   * <p>
+   * Afterwards, the isSet() method will return true, unless byte slice could not be converted.
+   *
+   */
+  @HiveDecimalWritableVersionV2
+  public void setFromBytes(byte[] bytes, int offset, int length) {
+    fastReset();
+    isSet = fastSetFromBytes(bytes, offset, length, false);
+    if (!isSet) {
+      fastReset();
+    }
+  }
+
+  @HiveDecimalWritableVersionV2
+  public void setFromBytes(byte[] bytes, int offset, int length, boolean trimBlanks) {
+    fastReset();
+    isSet = fastSetFromBytes(bytes, offset, length, trimBlanks);
+    if (!isSet) {
+      fastReset();
+    }
+  }
+
+  /**
+   * Set the writable's current value to the decimal digits only in a UTF-8 byte slice, a sign
+   * flag, and a scale.
+   * <p>
+   * Afterwards, the isSet() method will return true, unless byte slice etc could not be converted.
+   *
+   */
+  @HiveDecimalWritableVersionV2
+  public void setFromDigitsOnlyBytesWithScale(
+      boolean isNegative, byte[] bytes, int offset, int length, int scale) {
+    fastReset();
+    isSet = fastSetFromDigitsOnlyBytesAndScale(isNegative, bytes, offset, length, scale);
+    if (!isSet) {
+      fastReset();
+    }
+  }
+
+  /**
+   * Set the writable's current value to the signed value from BigInteger bytes and a specified
+   * scale.
+   * <p>
+   * Afterwards, the isSet() method will return true, unless conversion failed.
+   *
+   */
+  @HiveDecimalWritableVersionV2
+  public void setFromBigIntegerBytesAndScale(byte[] bigIntegerBytes, int scale) {
+    fastReset();
+    isSet = fastSetFromBigIntegerBytesAndScale(bigIntegerBytes, 0, bigIntegerBytes.length, scale);
+    if (!isSet) {
+      fastReset();
+    }
+  }
+
+  @HiveDecimalWritableVersionV2
+  public void setFromBigIntegerBytesAndScale(
+      byte[] bigIntegerBytes, int offset, int length, int scale) {
+    fastReset();
+    isSet = fastSetFromBigIntegerBytesAndScale(bigIntegerBytes, offset, length, scale);
+    if (!isSet) {
+      fastReset();
+    }
   }
 
-  public void set(byte[] bytes, int scale) {
-    this.internalStorage = bytes;
-    this.scale = scale;
+  /**
+   * Set the writable's current value to the long's value at a specified
+   * scale.
+   * <p>
+   * Afterwards, the isSet() method will return true, unless conversion failed.
+   *
+   */
+  @HiveDecimalWritableVersionV2
+  public void setFromLongAndScale(long longValue, int scale) {
+    fastReset();
+    isSet = fastSetFromLongAndScale(longValue, scale);
+    if (!isSet) {
+      fastReset();
+    }
+  }
+
+  /**
+   * Does this writable have a current value?
+   * <p>
+   * A return of false means a current value wasn't set, or an operation like mutateAdd overflowed,
+   * or a set* method couldn't convert the input value, etc.
+   *
+   */
+  @HiveDecimalWritableVersionV2
+  public boolean isSet() {
+    return isSet;
   }
 
+  /**
+   * Returns a HiveDecimal for the writable's current value.
+   * <p>
+   * Returns null if the writable isn't set.
+   *
+   */
+  @HiveDecimalWritableVersionV1
   public HiveDecimal getHiveDecimal() {
-    return HiveDecimal.create(new BigInteger(internalStorage), scale);
+    if (!isSet) {
+      return null;
+    }
+    HiveDecimal result = HiveDecimal.createFromFast(this);
+    return result;
   }
 
   /**
    * Get a HiveDecimal instance from the writable and constraint it with maximum precision/scale.
-   *
+   * <p>
    * @param maxPrecision maximum precision
    * @param maxScale maximum scale
    * @return HiveDecimal instance
    */
+  @HiveDecimalWritableVersionV1
   public HiveDecimal getHiveDecimal(int maxPrecision, int maxScale) {
-     return HiveDecimal.enforcePrecisionScale(HiveDecimal.
-             create(new BigInteger(internalStorage), scale),
-         maxPrecision, maxScale);
+    if (!isSet) {
+      return null;
+    }
+    HiveDecimal dec = HiveDecimal.createFromFast(this);
+    HiveDecimal result = HiveDecimal.enforcePrecisionScale(dec, maxPrecision, maxScale);
+    return result;
   }
 
+  /**
+   * Standard Writable method that deserialize the fields of this object from a DataInput.
+   * 
+   */
+  @HiveDecimalWritableVersionV1
   @Override
   public void readFields(DataInput in) throws IOException {
-    scale = WritableUtils.readVInt(in);
+    int scale = WritableUtils.readVInt(in);
     int byteArrayLen = WritableUtils.readVInt(in);
-    if (internalStorage.length != byteArrayLen) {
-      internalStorage = new byte[byteArrayLen];
+    byte[] bytes = new byte[byteArrayLen];
+    in.readFully(bytes);
+
+    fastReset();
+    if (!fastSetFromBigIntegerBytesAndScale(bytes, 0, bytes.length, scale)) {
+      throw new IOException("Couldn't convert decimal");
     }
-    in.readFully(internalStorage);
+    isSet = true;
   }
 
+  /**
+   * Standard Writable method that serialize the fields of this object to a DataOutput.
+   *
+   */
+  @HiveDecimalWritableVersionV1
   @Override
   public void write(DataOutput out) throws IOException {
-    WritableUtils.writeVInt(out, scale);
-    WritableUtils.writeVInt(out, internalStorage.length);
-    out.write(internalStorage);
+    if (!isSet()) {
+      throw new RuntimeException("no value set");
+    }
+
+    if (internalScratchLongs == null) {
+      internalScratchLongs = new long[FastHiveDecimal.FAST_SCRATCH_LONGS_LEN];
+      internalScratchBuffer = new byte[FastHiveDecimal.FAST_SCRATCH_BUFFER_LEN_BIG_INTEGER_BYTES];
+    }
+
+    write(out, internalScratchLongs, internalScratchBuffer);
   }
 
+
+  /**
+   * A variation of the standard Writable method that serialize the fields of this object to a
+   * DataOutput with scratch buffers for good performance.
+   * <p>
+   * Allocate scratchLongs with HiveDecimal.SCRATCH_LONGS_LEN longs.
+   * And, allocate scratch buffer with HiveDecimal.SCRATCH_BUFFER_LEN_BIG_INTEGER_BYTES bytes.
+   *
+   */
+  @HiveDecimalWritableVersionV2
+  public void write(
+      DataOutput out,
+      long[] scratchLongs, byte[] scratchBuffer) throws IOException {
+    if (!isSet()) {
+      throw new RuntimeException("no value set");
+    }
+
+    WritableUtils.writeVInt(out, fastScale());
+
+    int byteLength =
+        fastBigIntegerBytes(
+            scratchLongs, scratchBuffer);
+    if (byteLength == 0) {
+      throw new RuntimeException("Couldn't convert decimal to binary");
+    }
+
+    WritableUtils.writeVInt(out, byteLength);
+    out.write(scratchBuffer, 0, byteLength);
+  }
+
+  /**
+   * See the comments for HiveDecimal.serializationUtilsRead.
+   */
+  @HiveDecimalWritableVersionV2
+  public boolean serializationUtilsRead(
+      InputStream inputStream, int scale,
+      byte[] scratchBytes)
+          throws IOException {
+    fastReset();
+    isSet =
+        fastSerializationUtilsRead(
+            inputStream,
+            scale,
+            scratchBytes);
+    return isSet;
+  }
+
+  /**
+   * See the comments for HiveDecimal.serializationUtilsWrite.
+   */
+  @HiveDecimalWritableVersionV2
+  public boolean serializationUtilsWrite(
+      OutputStream outputStream,
+      long[] scratchLongs)
+          throws IOException {
+    if (!isSet()) {
+      throw new RuntimeException("no value set");
+    }
+    return
+        fastSerializationUtilsWrite(
+            outputStream,
+            scratchLongs);
+  }
+
+  /**
+   * Returns the length of the decimal converted to bytes.
+   * Call bigIntegerBytesBuffer() to get a reference to the converted bytes.
+   *
+   */
+  @HiveDecimalWritableVersionV2
+  public int bigIntegerBytesInternalScratch() {
+
+    if (!isSet()) {
+      throw new RuntimeException("no value set");
+    }
+
+    if (internalScratchLongs == null) {
+      internalScratchLongs = new long[FastHiveDecimal.FAST_SCRATCH_LONGS_LEN];
+      internalScratchBuffer = new byte[FastHiveDecimal.FAST_SCRATCH_BUFFER_LEN_BIG_INTEGER_BYTES];
+    }
+
+    int byteLength =
+        fastBigIntegerBytes(
+            internalScratchLongs, internalScratchBuffer);
+    if (byteLength == 0) {
+      throw new RuntimeException("Couldn't convert decimal to binary");
+    }
+    return byteLength;
+  }
+
+  /**
+   * Returns the scratch array containing the result after a call to bigIntegerBytesInternalScratch.
+   *
+   */
+  @HiveDecimalWritableVersionV2
+  public byte[] bigIntegerBytesInternalScratchBuffer() {
+    return internalScratchBuffer;
+  }
+
+  /**
+  * Allocate scratchLongs with HiveDecimal.SCRATCH_LONGS_LEN longs.
+  * And, allocate scratch scratchBuffer with HiveDecimal.SCRATCH_BUFFER_LEN_BIG_INTEGER_BYTES bytes.
+  *
+  */
+  @HiveDecimalWritableVersionV2
+  public byte[] bigIntegerBytesCopy(
+      long[] scratchLongs, byte[] scratchBuffer) {
+
+    if (!isSet()) {
+      throw new RuntimeException("no value set");
+    }
+
+    int byteLength =
+        fastBigIntegerBytes(
+            scratchLongs, scratchBuffer);
+    if (byteLength == 0) {
+      throw new RuntimeException("Couldn't convert decimal to binary");
+    }
+    return Arrays.copyOf(scratchBuffer, byteLength);
+  }
+
+  @HiveDecimalWritableVersionV2
+  public int bigIntegerBytes(
+      long[] scratchLongs, byte[] scratchBuffer) {
+    if (!isSet()) {
+      throw new RuntimeException("no value set");
+    }
+    int byteLength =
+        fastBigIntegerBytes(
+            scratchLongs, scratchBuffer);
+    return byteLength;
+  }
+
+  @HiveDecimalWritableVersionV2
+  public int signum() {
+    if (!isSet()) {
+      throw new RuntimeException("no value set");
+    }
+    return fastSignum();
+  }
+
+  @HiveDecimalWritableVersionV2
+  public int precision() {
+    if (!isSet()) {
+      throw new RuntimeException("no value set");
+    }
+    return fastSqlPrecision();
+  }
+
+  @HiveDecimalWritableVersionV2
+  public int rawPrecision() {
+    if (!isSet()) {
+      throw new RuntimeException("no value set");
+    }
+    return fastRawPrecision();
+  }
+
+  @HiveDecimalWritableVersionV2
+  public int scale() {
+    if (!isSet()) {
+      throw new RuntimeException("no value set");
+    }
+    return fastScale();
+  }
+
+  @HiveDecimalWritableVersionV2
+  public boolean isByte() {
+    if (!isSet()) {
+      throw new RuntimeException("no value set");
+    }
+    return fastIsByte();
+  }
+
+  @HiveDecimalWritableVersionV2
+  public byte byteValue() {
+    if (!isSet()) {
+      throw new RuntimeException("no value set");
+    }
+    return fastByteValueClip();
+  }
+
+  @HiveDecimalWritableVersionV2
+  public boolean isShort() {
+    if (!isSet()) {
+      throw new RuntimeException("no value set");
+    }
+    return fastIsShort();
+  }
+
+  @HiveDecimalWritableVersionV2
+  public short shortValue() {
+    if (!isSet()) {
+      throw new RuntimeException("no value set");
+    }
+    return fastShortValueClip();
+  }
+
+  @HiveDecimalWritableVersionV2
+  public boolean isInt() {
+    if (!isSet()) {
+      throw new RuntimeException("no value set");
+    }
+    return fastIsInt();
+  }
+
+  @HiveDecimalWritableVersionV2
+  public int intValue() {
+    if (!isSet()) {
+      throw new RuntimeException("no value set");
+    }
+    return fastIntValueClip();
+  }
+
+  @HiveDecimalWritableVersionV2
+  public boolean isLong() {
+    if (!isSet()) {
+      throw new RuntimeException("no value set");
+    }
+    return fastIsLong();
+  }
+
+  @HiveDecimalWritableVersionV2
+  public long longValue() {
+    if (!isSet()) {
+      throw new RuntimeException("no value set");
+    }
+    return fastLongValueClip();
+  }
+
+  @HiveDecimalWritableVersionV2
+  public float floatValue() {
+    if (!isSet()) {
+      throw new RuntimeException("no value set");
+    }
+    return fastFloatValue();
+   }
+
+  @HiveDecimalWritableVersionV2
+  public double doubleValue() {
+    if (!isSet()) {
+      throw new RuntimeException("no value set");
+    }
+    return fastDoubleValue();
+  }
+
+  //-----------------------------------------------------------------------------------------------
+  // Mutate operations.
+  //-----------------------------------------------------------------------------------------------
+
+  @HiveDecimalWritableVersionV2
+  public void mutateAbs() {
+    if (!isSet) {
+      return;
+    }
+    fastAbs();
+  }
+
+  @HiveDecimalWritableVersionV2
+  public void mutateNegate() {
+    if (!isSet) {
+      return;
+    }
+    fastNegate();
+  }
+
+  @HiveDecimalWritableVersionV2
+  public void mutateAdd(HiveDecimalWritable decWritable) {
+    if (!isSet || !decWritable.isSet) {
+      isSet = false;
+      return;
+    }
+    isSet =
+        fastAdd(decWritable, this);
+  }
+
+  @HiveDecimalWritableVersionV2
+  public void mutateAdd(HiveDecimal dec) {
+    if (!isSet) {
+      return;
+    }
+    isSet =
+        fastAdd(dec, this);
+  }
+
+  @HiveDecimalWritableVersionV2
+  public void mutateSubtract(HiveDecimalWritable decWritable) {
+    if (!isSet || !decWritable.isSet) {
+      isSet = false;
+      return;
+    }
+    isSet =
+        fastSubtract(decWritable, this);
+  }
+
+  @HiveDecimalWritableVersionV2
+  public void mutateSubtract(HiveDecimal dec) {
+    if (!isSet) {
+      return;
+    }
+    isSet =
+        fastSubtract(dec, this);
+  }
+
+  @HiveDecimalWritableVersionV2
+  public void mutateMultiply(HiveDecimalWritable decWritable) {
+    if (!isSet || !decWritable.isSet) {
+      isSet = false;
+      return;
+    }
+    isSet =
+        fastMultiply(decWritable, this);
+  }
+
+  @HiveDecimalWritableVersionV2
+  public void mutateMultiply(HiveDecimal dec) {
+    if (!isSet) {
+      return;
+    }
+    isSet =
+        fastMultiply(dec, this);
+  }
+
+  @HiveDecimalWritableVersionV2
+  public void mutateDivide(HiveDecimalWritable decWritable) {
+    if (!isSet || !decWritable.isSet) {
+      isSet = false;
+      return;
+    }
+    isSet =
+        fastDivide(decWritable, this);
+  }
+
+  @HiveDecimalWritableVersionV2
+  public void mutateDivide(HiveDecimal dec) {
+    if (!isSet) {
+      return;
+    }
+    isSet =
+        fastDivide(dec, this);
+
+  }
+
+  @HiveDecimalWritableVersionV2
+  public void mutateRemainder(HiveDecimalWritable decWritable) {
+    if (!isSet || !decWritable.isSet) {
+      isSet = false;
+      return;
+    }
+    isSet =
+        fastRemainder(decWritable, this);
+  }
+
+  @HiveDecimalWritableVersionV2
+  public void mutateRemainder(HiveDecimal dec) {
+    if (!isSet) {
+      return;
+    }
+    isSet =
+        fastRemainder(dec, this);
+  }
+
+  @HiveDecimalWritableVersionV2
+  public void mutateScaleByPowerOfTen(int power) {
+    if (!isSet) {
+      return;
+    }
+    isSet = fastScaleByPowerOfTen(power, this);
+  }
+
+  @HiveDecimalWritableVersionV2
+  public void mutateFractionPortion() {
+    if (!isSet) {
+      return;
+    }
+    fastFractionPortion();
+  }
+
+  @HiveDecimalWritableVersionV2
+  public void mutateIntegerPortion() {
+    if (!isSet) {
+      return;
+    }
+    fastIntegerPortion();
+  }
+
+  //-----------------------------------------------------------------------------------------------
+  // Standard overrides methods.
+  //-----------------------------------------------------------------------------------------------
+
+  @HiveDecimalWritableVersionV1
   @Override
-  public int compareTo(HiveDecimalWritable that) {
-    return getHiveDecimal().compareTo(that.getHiveDecimal());
+  public int compareTo(HiveDecimalWritable writable) {
+    if (!isSet() || writable == null || !writable.isSet()) {
+      throw new RuntimeException("Invalid comparision operand(s)");
+    }
+    return fastCompareTo(writable);
+  }
+
+  @HiveDecimalWritableVersionV2
+  public int compareTo(HiveDecimal dec) {
+    if (!isSet() || dec == null) {
+      throw new RuntimeException("Invalid comparision operand(s)");
+    }
+    return fastCompareTo(dec);
+  }
+
+  @HiveDecimalWritableVersionV2
+  public static int compareTo(HiveDecimal dec, HiveDecimalWritable writable) {
+    if (dec == null || !writable.isSet()) {
+      throw new RuntimeException("Invalid comparision operand(s)");
+    }
+    return FastHiveDecimal.fastCompareTo(dec, writable);
   }
 
+  @HiveDecimalWritableVersionV2
+  public int toBytes(byte[] scratchBuffer) {
+    if (!isSet()) {
+      throw new RuntimeException("no value set");
+    }
+    return fastToBytes(scratchBuffer);
+  }
+
+  @HiveDecimalWritableVersionV1
   @Override
   public String toString() {
-    return getHiveDecimal().toString();
+    if (!isSet()) {
+      throw new RuntimeException("no value set");
+    }
+    return fastToString();
+  }
+
+  @HiveDecimalWritableVersionV2
+  public String toString(
+      byte[] scratchBuffer) {
+    if (!isSet()) {
+      throw new RuntimeException("no value set");
+    }
+    if (fastSerializationScale() != -1) {
+
+      // Use the serialization scale and format the string with trailing zeroes (or
+      // round the decimal) if necessary.
+      return
+          fastToFormatString(
+              fastSerializationScale(),
+              scratchBuffer);
+    } else {
+      return
+          fastToString(scratchBuffer);
+    }
+  }
+
+  @HiveDecimalWritableVersionV2
+  public String toFormatString(
+      int formatScale) {
+    if (!isSet()) {
+      throw new RuntimeException("no value set");
+    }
+    return
+        fastToFormatString(
+            formatScale);
+  }
+
+  @HiveDecimalWritableVersionV2
+  public int toFormatBytes(
+      int formatScale,
+      byte[] scratchBuffer) {
+    if (!isSet()) {
+      throw new RuntimeException("no value set");
+    }
+    return
+        fastToFormatBytes(
+            formatScale,
+            scratchBuffer);
+  }
+
+  @HiveDecimalWritableVersionV2
+  public int toDigitsOnlyBytes(
+      byte[] scratchBuffer) {
+    if (!isSet()) {
+      throw new RuntimeException("no value set");
+    }
+    return
+        fastToDigitsOnlyBytes(
+            scratchBuffer);
   }
 
+  @HiveDecimalWritableVersionV1
   @Override
   public boolean equals(Object other) {
+    if (!isSet) {
+      return false;
+    }
     if (this == other) {
       return true;
     }
     if (other == null || getClass() != other.getClass()) {
       return false;
     }
-    HiveDecimalWritable bdw = (HiveDecimalWritable) other;
+    HiveDecimalWritable otherHiveDecWritable = (HiveDecimalWritable) other;
+    if (!otherHiveDecWritable.isSet()) {
+      return false;
+    }
+    return fastEquals((FastHiveDecimal) otherHiveDecWritable);
 
-    // 'equals' and 'compareTo' are not compatible with HiveDecimals. We want
-    // compareTo which returns true iff the numbers are equal (e.g.: 3.14 is
-    // the same as 3.140). 'Equals' returns true iff equal and the same scale
-    // is set in the decimals (e.g.: 3.14 is not the same as 3.140)
-    return getHiveDecimal().compareTo(bdw.getHiveDecimal()) == 0;
   }
 
+  @HiveDecimalWritableVersionV2
+  public int newFasterHashCode() {
+    if (!isSet()) {
+      throw new RuntimeException("no value set");
+    }
+    return fastNewFasterHashCode();
+  }
+
+  @HiveDecimalWritableVersionV1
   @Override
   public int hashCode() {
-    return getHiveDecimal().hashCode();
+    if (!isSet()) {
+      throw new RuntimeException("no value set");
+    }
+    return fastHashCode();
   }
 
-  /* (non-Javadoc)
-   * In order to update a Decimal128 fast (w/o allocation) we need to expose access to the
-   * internal storage bytes and scale.
-   * @return
-   */
+  @HiveDecimalWritableVersionV1
   public byte[] getInternalStorage() {
-    return internalStorage;
+    if (!isSet()) {
+      throw new RuntimeException("no value set");
+    }
+
+    if (internalScratchLongs == null) {
+      internalScratchLongs = new long[FastHiveDecimal.FAST_SCRATCH_LONGS_LEN];
+      internalScratchBuffer = new byte[FastHiveDecimal.FAST_SCRATCH_BUFFER_LEN_BIG_INTEGER_BYTES];
+    }
+
+    return bigIntegerBytesCopy(
+        internalScratchLongs, internalScratchBuffer);
   }
 
-  /* (non-Javadoc)
-   * In order to update a Decimal128 fast (w/o allocation) we need to expose access to the
-   * internal storage bytes and scale.
-   */
+  @HiveDecimalWritableVersionV1
   public int getScale() {
-    return scale;
+    if (!isSet()) {
+      throw new RuntimeException("no value set");
+    }
+    return fastScale();
   }
 
-  public static
-  HiveDecimalWritable enforcePrecisionScale(HiveDecimalWritable writable,
-                                            int precision, int scale) {
-    if (writable == null) {
-      return null;
+  @HiveDecimalWritableVersionV2
+  public void mutateSetScale(int roundingPoint, int roundingMode) {
+    if (!isSet) {
+      return;
+    }
+    isSet = fastRound(roundingPoint, roundingMode, this);
+    if (!isSet) {
+      fastReset();
+    }
+  }
+
+  @HiveDecimalWritableVersionV2
+  public boolean mutateEnforcePrecisionScale(int precision, int scale) {
+    if (!isSet) {
+      return false;
     }
+    isSet = fastEnforcePrecisionScale(precision, scale);
+    if (!isSet) {
+      fastReset();
+    }
+    return isSet;
+  }
 
-    HiveDecimal dec =
-        HiveDecimal.enforcePrecisionScale(writable.getHiveDecimal(), precision,
-            scale);
-    return dec == null ? null : new HiveDecimalWritable(dec);
+  @HiveDecimalWritableVersionV1
+  public static HiveDecimalWritable enforcePrecisionScale(HiveDecimalWritable writable, int precision, int scale) {
+    if (!writable.isSet) {
+      return null;
+    }
+    HiveDecimalWritable result = new HiveDecimalWritable(writable);
+    result.mutateEnforcePrecisionScale(precision, scale);
+    if (!result.isSet()) {
+      return null;
+    }
+    return result;
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/storage-api/src/java/org/apache/hadoop/hive/serde2/io/HiveDecimalWritableV1.java
----------------------------------------------------------------------
diff --git a/storage-api/src/java/org/apache/hadoop/hive/serde2/io/HiveDecimalWritableV1.java b/storage-api/src/java/org/apache/hadoop/hive/serde2/io/HiveDecimalWritableV1.java
new file mode 100644
index 0000000..ec976e9
--- /dev/null
+++ b/storage-api/src/java/org/apache/hadoop/hive/serde2/io/HiveDecimalWritableV1.java
@@ -0,0 +1,191 @@
+/**
+ * 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.serde2.io;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.math.BigInteger;
+
+import org.apache.hadoop.hive.common.type.HiveDecimalV1;
+
+import org.apache.hadoop.io.WritableComparable;
+import org.apache.hadoop.io.WritableUtils;
+
+public class HiveDecimalWritableV1 implements WritableComparable<HiveDecimalWritableV1> {
+
+  private byte[] internalStorage = new byte[0];
+  private int scale;
+
+  @HiveDecimalWritableVersionV1
+  public HiveDecimalWritableV1() {
+  }
+
+  @HiveDecimalWritableVersionV1
+  public HiveDecimalWritableV1(String value) {
+    set(HiveDecimalV1.create(value));
+  }
+
+  @HiveDecimalWritableVersionV1
+  public HiveDecimalWritableV1(byte[] bytes, int scale) {
+    set(bytes, scale);
+  }
+
+  @HiveDecimalWritableVersionV1
+  public HiveDecimalWritableV1(HiveDecimalWritableV1 writable) {
+    set(writable.getHiveDecimal());
+  }
+
+  @HiveDecimalWritableVersionV1
+  public HiveDecimalWritableV1(HiveDecimalV1 value) {
+    set(value);
+  }
+
+  @HiveDecimalWritableVersionV1
+  public HiveDecimalWritableV1(long value) {
+    set((HiveDecimalV1.create(value)));
+  }
+
+  @HiveDecimalWritableVersionV1
+  public void set(HiveDecimalV1 value) {
+    set(value.unscaledValue().toByteArray(), value.scale());
+  }
+
+  @HiveDecimalWritableVersionV1
+  public void set(HiveDecimalV1 value, int maxPrecision, int maxScale) {
+    set(HiveDecimalV1.enforcePrecisionScale(value, maxPrecision, maxScale));
+  }
+
+  @HiveDecimalWritableVersionV1
+  public void set(HiveDecimalWritableV1 writable) {
+    set(writable.getHiveDecimal());
+  }
+
+  @HiveDecimalWritableVersionV1
+  public void set(byte[] bytes, int scale) {
+    this.internalStorage = bytes;
+    this.scale = scale;
+  }
+
+  @HiveDecimalWritableVersionV1
+  public HiveDecimalV1 getHiveDecimal() {
+    return HiveDecimalV1.create(new BigInteger(internalStorage), scale);
+  }
+
+  /**
+   * Get a OldHiveDecimal instance from the writable and constraint it with maximum precision/scale.
+   *
+   * @param maxPrecision maximum precision
+   * @param maxScale maximum scale
+   * @return OldHiveDecimal instance
+   */
+  @HiveDecimalWritableVersionV1
+  public HiveDecimalV1 getHiveDecimal(int maxPrecision, int maxScale) {
+     return HiveDecimalV1.enforcePrecisionScale(HiveDecimalV1.
+             create(new BigInteger(internalStorage), scale),
+         maxPrecision, maxScale);
+  }
+
+  @HiveDecimalWritableVersionV1
+  @Override
+  public void readFields(DataInput in) throws IOException {
+    scale = WritableUtils.readVInt(in);
+    int byteArrayLen = WritableUtils.readVInt(in);
+    if (internalStorage.length != byteArrayLen) {
+      internalStorage = new byte[byteArrayLen];
+    }
+    in.readFully(internalStorage);
+  }
+
+  @HiveDecimalWritableVersionV1
+  @Override
+  public void write(DataOutput out) throws IOException {
+    WritableUtils.writeVInt(out, scale);
+    WritableUtils.writeVInt(out, internalStorage.length);
+    out.write(internalStorage);
+  }
+
+  @HiveDecimalWritableVersionV1
+  @Override
+  public int compareTo(HiveDecimalWritableV1 that) {
+    return getHiveDecimal().compareTo(that.getHiveDecimal());
+  }
+
+  @HiveDecimalWritableVersionV1
+  @Override
+  public String toString() {
+    return getHiveDecimal().toString();
+  }
+
+  @HiveDecimalWritableVersionV1
+  @Override
+  public boolean equals(Object other) {
+    if (this == other) {
+      return true;
+    }
+    if (other == null || getClass() != other.getClass()) {
+      return false;
+    }
+    HiveDecimalWritableV1 bdw = (HiveDecimalWritableV1) other;
+
+    // 'equals' and 'compareTo' are not compatible with HiveDecimals. We want
+    // compareTo which returns true iff the numbers are equal (e.g.: 3.14 is
+    // the same as 3.140). 'Equals' returns true iff equal and the same scale
+    // is set in the decimals (e.g.: 3.14 is not the same as 3.140)
+    return getHiveDecimal().compareTo(bdw.getHiveDecimal()) == 0;
+  }
+
+  @HiveDecimalWritableVersionV1
+  @Override
+  public int hashCode() {
+    return getHiveDecimal().hashCode();
+  }
+
+  /* (non-Javadoc)
+   * In order to update a Decimal128 fast (w/o allocation) we need to expose access to the
+   * internal storage bytes and scale.
+   * @return
+   */
+  @HiveDecimalWritableVersionV1
+  public byte[] getInternalStorage() {
+    return internalStorage;
+  }
+
+  /* (non-Javadoc)
+   * In order to update a Decimal128 fast (w/o allocation) we need to expose access to the
+   * internal storage bytes and scale.
+   */
+  @HiveDecimalWritableVersionV1
+  public int getScale() {
+    return scale;
+  }
+
+  @HiveDecimalWritableVersionV1
+  public static
+  HiveDecimalWritableV1 enforcePrecisionScale(HiveDecimalWritableV1 writable,
+                                            int precision, int scale) {
+    if (writable == null) {
+      return null;
+    }
+
+    HiveDecimalV1 dec =
+        HiveDecimalV1.enforcePrecisionScale(writable.getHiveDecimal(), precision,
+            scale);
+    return dec == null ? null : new HiveDecimalWritableV1(dec);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/storage-api/src/java/org/apache/hadoop/hive/serde2/io/HiveDecimalWritableVersionV1.java
----------------------------------------------------------------------
diff --git a/storage-api/src/java/org/apache/hadoop/hive/serde2/io/HiveDecimalWritableVersionV1.java b/storage-api/src/java/org/apache/hadoop/hive/serde2/io/HiveDecimalWritableVersionV1.java
new file mode 100644
index 0000000..1aaa6f0
--- /dev/null
+++ b/storage-api/src/java/org/apache/hadoop/hive/serde2/io/HiveDecimalWritableVersionV1.java
@@ -0,0 +1,33 @@
+/**
+ * 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.serde2.io;
+
+import java.lang.annotation.Documented;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+
+/**
+ * Marks methods including static methods and fields as being part of version 1 HiveDecimalWritable.
+ *
+ */
+@Documented
+@Retention(RetentionPolicy.RUNTIME)
+public @interface HiveDecimalWritableVersionV1 {
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/storage-api/src/java/org/apache/hadoop/hive/serde2/io/HiveDecimalWritableVersionV2.java
----------------------------------------------------------------------
diff --git a/storage-api/src/java/org/apache/hadoop/hive/serde2/io/HiveDecimalWritableVersionV2.java b/storage-api/src/java/org/apache/hadoop/hive/serde2/io/HiveDecimalWritableVersionV2.java
new file mode 100644
index 0000000..3c2b6f9
--- /dev/null
+++ b/storage-api/src/java/org/apache/hadoop/hive/serde2/io/HiveDecimalWritableVersionV2.java
@@ -0,0 +1,33 @@
+/**
+ * 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.serde2.io;
+
+import java.lang.annotation.Documented;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+
+/**
+ * Marks methods including static methods and fields as being part of version 2 HiveDecimalWritable.
+ *
+ */
+@Documented
+@Retention(RetentionPolicy.RUNTIME)
+public @interface HiveDecimalWritableVersionV2 {
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/storage-api/src/test/org/apache/hadoop/hive/VersionTestBase.java
----------------------------------------------------------------------
diff --git a/storage-api/src/test/org/apache/hadoop/hive/VersionTestBase.java b/storage-api/src/test/org/apache/hadoop/hive/VersionTestBase.java
new file mode 100644
index 0000000..489496e
--- /dev/null
+++ b/storage-api/src/test/org/apache/hadoop/hive/VersionTestBase.java
@@ -0,0 +1,322 @@
+/**
+ * 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;
+
+import java.lang.annotation.Annotation;
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.lang.reflect.Field;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeSet;
+
+import org.junit.*;
+
+import static org.junit.Assert.*;
+
+public class VersionTestBase {
+
+  public String getParameterTypeName(Class<?> parameterType,
+      Map<Class, String> versionedClassToNameMap) {
+    if (versionedClassToNameMap.containsKey(parameterType)) {
+      return versionedClassToNameMap.get(parameterType);
+    } else {
+      return parameterType.getSimpleName();
+    }
+  }
+
+  public String getMethodKey(Method method, Map<Class, String> versionedClassToNameMap) {
+    //-------------------------------------------------
+    StringBuilder sb = new StringBuilder();
+
+    int modifiers = method.getModifiers();
+    if ((modifiers & Modifier.STATIC) != 0) {
+      sb.append("static");
+    } else {
+      sb.append("non-static");
+    }
+    sb.append(" ");
+    Class<?> returnType = method.getReturnType();
+    sb.append(getParameterTypeName(returnType, versionedClassToNameMap));
+    sb.append(" ");
+    sb.append(method.getName());
+    Class<?>[] parameterTypes = method.getParameterTypes();
+    sb.append("(");
+    boolean isFirst = true;
+    for (Class<?> parameterType : parameterTypes) {
+      if (!isFirst) {
+        sb.append(", ");
+      }
+      sb.append(getParameterTypeName(parameterType, versionedClassToNameMap));
+      isFirst = false;
+    }
+    sb.append(")");
+    Class<?>[] exceptionsThrown = method.getExceptionTypes();
+    if (exceptionsThrown.length > 0) {
+      sb.append(" throws ");
+      isFirst = true;
+      for (Class<?> exceptionThrown : exceptionsThrown) {
+        if (!isFirst) {
+          sb.append(", ");
+        }
+        sb.append(exceptionThrown.getSimpleName());
+        isFirst = false;
+      }
+    }
+
+    return sb.toString();
+    //-------------------------------------------------
+  }
+
+  public String getFieldKey(Field field, Map<Class, String> versionedClassToNameMap) throws IllegalAccessException {
+    //-------------------------------------------------
+    StringBuilder sb = new StringBuilder();
+
+    int modifiers = field.getModifiers();
+    if ((modifiers & Modifier.STATIC) != 0) {
+      sb.append("static");
+    } else {
+      sb.append("non-static");
+    }
+    sb.append(" ");
+    Class<?> fieldType = field.getType();
+    sb.append(getParameterTypeName(fieldType, versionedClassToNameMap));
+    sb.append(" ");
+    sb.append(field.getName());
+    if ((modifiers & Modifier.STATIC) != 0) {
+      sb.append(" ");
+      sb.append(field.get(null));
+    }
+
+    return sb.toString();
+    //-------------------------------------------------
+  }
+
+  public Method[] onlyPublicMethods(Method[] methods) {
+    List<Method> resultList = new ArrayList<Method>();
+    for (Method method : methods) {
+      if ((method.getModifiers() & Modifier.PUBLIC) != 0) {
+        resultList.add(method);
+      }
+    }
+    return resultList.toArray(new Method[0]);
+  }
+
+  public Field[] onlyPublicFields(Field[] fields) {
+    List<Field> resultList = new ArrayList<Field>();
+    for (Field field : fields) {
+      if ((field.getModifiers() & Modifier.PUBLIC) != 0) {
+        resultList.add(field);
+      }
+    }
+    return resultList.toArray(new Field[0]);
+  }
+
+  public TreeSet<String> getMethodKeySetForAnnotation(Method[] methods, Class annotationClass,
+      Map<Class, String> versionedClassToNameMap)
+     throws IllegalAccessException {
+    TreeSet<String> result = new TreeSet<String>();
+
+    for (Method method : methods) {
+      Annotation[] declaredAnnotations = method.getDeclaredAnnotations();
+      boolean isFound = false;
+      for (Annotation declaredAnnotation : declaredAnnotations) {
+        if (declaredAnnotation.annotationType().equals(annotationClass)) {
+          isFound = true;
+          break;
+        }
+      }
+      if (!isFound) {
+        continue;
+      }
+      result.add(getMethodKey(method, versionedClassToNameMap));
+    }
+    return result;
+  }
+
+  public TreeSet<String> getMethodKeySetExcludingAnnotations(Method[] methods,
+      List<Class> versionAnnotations, Map<Class, String> versionedClassToNameMap)
+      throws IllegalAccessException {
+     TreeSet<String> result = new TreeSet<String>();
+
+     for (Method method : methods) {
+       Annotation[] declaredAnnotations = method.getDeclaredAnnotations();
+       boolean isFound = false;
+       for (Annotation declaredAnnotation : declaredAnnotations) {
+         for (Class versionAnnotation : versionAnnotations) {
+           if (declaredAnnotation.annotationType().equals(versionAnnotation)) {
+             isFound = true;
+             break;
+           }
+         }
+         if (isFound) {
+           break;
+         }
+       }
+       if (isFound) {
+         continue;
+       }
+       String methodKey = getMethodKey(method, versionedClassToNameMap);
+       if (!methodKey.equals("non-static int compareTo(Object)")) {
+         result.add(methodKey);
+       }
+     }
+     return result;
+   }
+
+  public TreeSet<String> getFieldKeySetForAnnotation(Field[] fields,
+      Class annotationClass, Map<Class, String> versionedClassToNameMap)
+     throws IllegalAccessException {
+    TreeSet<String> result = new TreeSet<String>();
+
+    for (Field field : fields) {
+      Annotation[] declaredAnnotations = field.getDeclaredAnnotations();
+      boolean isFound = false;
+      for (Annotation declaredAnnotation : declaredAnnotations) {
+        if (declaredAnnotation.annotationType().equals(annotationClass)) {
+          isFound = true;
+          break;
+        }
+      }
+      if (!isFound) {
+        continue;
+      }
+      result.add(getFieldKey(field, versionedClassToNameMap));
+    }
+    return result;
+  }
+
+  public TreeSet<String> getFieldKeySetExcludingAnnotations(Field[] fields,
+      List<Class> versionAnnotations, Map<Class, String> versionedClassToNameMap)
+      throws IllegalAccessException {
+     TreeSet<String> result = new TreeSet<String>();
+
+     for (Field field : fields) {
+       Annotation[] declaredAnnotations = field.getDeclaredAnnotations();
+       boolean isFound = false;
+       for (Annotation declaredAnnotation : declaredAnnotations) {
+         for (Class versionAnnotation : versionAnnotations) {
+           if (declaredAnnotation.annotationType().equals(versionAnnotation)) {
+             isFound = true;
+             break;
+           }
+         }
+         if (isFound) {
+           break;
+         }
+       }
+       if (isFound) {
+         continue;
+       }
+       result.add(getFieldKey(field, versionedClassToNameMap));
+     }
+     return result;
+   }
+
+  // For now, olderClass has 1 version and newerClass 2 versions...
+  public void  doVerifyVersions(
+      Class olderClass, Class olderVersionClass,
+      Class newerClass, Class newerVersionClass,
+      Map<Class, String> versionedClassToNameMap) throws IllegalAccessException {
+
+    List<Class> olderVersionClasses = new ArrayList<Class>();
+    olderVersionClasses.add(olderVersionClass);
+
+    List<Class> newerVersionClasses = new ArrayList<Class>();
+    newerVersionClasses.add(olderVersionClass);
+    newerVersionClasses.add(newerVersionClass);
+
+    //----------------------------------------------------------------------------------------------
+    Method[] olderMethods = onlyPublicMethods(olderClass.getDeclaredMethods());
+    TreeSet<String> olderMethodSet =
+        getMethodKeySetForAnnotation(olderMethods, olderVersionClass, versionedClassToNameMap);
+
+    TreeSet<String> olderNoMethodAnnotationsSet =
+        getMethodKeySetExcludingAnnotations(olderMethods, olderVersionClasses, versionedClassToNameMap);
+
+    Field[] olderFields = onlyPublicFields(olderClass.getFields());
+    TreeSet<String> olderFieldSet =
+        getFieldKeySetForAnnotation(olderFields, olderVersionClass, versionedClassToNameMap);
+
+    TreeSet<String> olderNoFieldAnnotationsSet =
+        getFieldKeySetExcludingAnnotations(olderFields, olderVersionClasses, versionedClassToNameMap);
+    //----------------------------------------------------------------------------------------------
+
+    Method[] newerMethods = onlyPublicMethods(newerClass.getDeclaredMethods());
+    TreeSet<String> newerMethodSetV1 =
+        getMethodKeySetForAnnotation(newerMethods, olderVersionClass, versionedClassToNameMap);
+    TreeSet<String> newerMethodSetV2 =
+        getMethodKeySetForAnnotation(newerMethods, newerVersionClass, versionedClassToNameMap);
+
+    TreeSet<String> newerNoMethodAnnotationsSetV1andV2 =
+        getMethodKeySetExcludingAnnotations(newerMethods, newerVersionClasses, versionedClassToNameMap);
+
+    Field[] newerFields = onlyPublicFields(newerClass.getFields());
+    // doDisplayFields(newerFields, newerClass);
+    TreeSet<String> newerFieldSetV1 =
+        getFieldKeySetForAnnotation(newerFields, olderVersionClass, versionedClassToNameMap);
+    TreeSet<String> newerFieldSetV2 =
+        getFieldKeySetForAnnotation(newerFields, newerVersionClass, versionedClassToNameMap);
+
+    TreeSet<String> newerNoFieldAnnotationsSetV1andV2 =
+        getFieldKeySetExcludingAnnotations(newerFields, newerVersionClasses, versionedClassToNameMap);
+
+    //----------------------------------------------------------------------------------------------
+    // VALIDATION
+    //----------------------------------------------------------------------------------------------
+
+    // No version annotation?
+    if (olderNoMethodAnnotationsSet.size() != 0) {
+      Assert.assertTrue("Class " + olderClass.getSimpleName() + " has 1 or more public methods without a version V1 annotation " +
+          olderNoMethodAnnotationsSet.toString(), false);
+    }
+    if (olderNoFieldAnnotationsSet.size() != 0) {
+      Assert.assertTrue("Class " + olderClass.getSimpleName() + " has 1 or more public fields without a version V1 annotation " +
+          olderNoFieldAnnotationsSet.toString(), false);
+    }
+    if (newerNoMethodAnnotationsSetV1andV2.size() != 0) {
+      Assert.assertTrue("Class " + newerClass.getSimpleName() + " has 1 or more public methods without a version V1 or V2 annotation " +
+          newerNoMethodAnnotationsSetV1andV2.toString(), false);
+    }
+    if (newerNoFieldAnnotationsSetV1andV2.size() != 0) {
+      Assert.assertTrue("Class " + newerClass.getSimpleName() + " has 1 or more public methods without a version V1 or V2 annotation " +
+          newerNoFieldAnnotationsSetV1andV2.toString(), false);
+    }
+
+    // Do the V1 methods of older and newer match?
+    if (!olderMethodSet.equals(newerMethodSetV1)) {
+      TreeSet<String> leftCopy = new TreeSet<String>(olderMethodSet);
+      leftCopy.removeAll(newerMethodSetV1);
+      TreeSet<String> rightCopy = new TreeSet<String>(newerMethodSetV1);
+      rightCopy.removeAll(olderMethodSet);
+      Assert.assertTrue("Class " + olderClass.getSimpleName() + " and class " + newerClass.getSimpleName() + " methods are different for V1 " +
+          leftCopy.toString() + " " + rightCopy.toString(), false);
+    }
+
+    // Do the V1 fields of older and newer match?
+    if (!olderFieldSet.equals(newerFieldSetV1)) {
+      TreeSet<String> leftCopy = new TreeSet<String>(olderFieldSet);
+      leftCopy.removeAll(newerFieldSetV1);
+      TreeSet<String> rightCopy = new TreeSet<String>(newerFieldSetV1);
+      rightCopy.removeAll(olderFieldSet);
+      Assert.assertTrue("Class " + olderClass.getSimpleName() + " and class " + newerClass.getSimpleName() + " fields are different for V1 " +
+          leftCopy.toString() + " " + rightCopy.toString(), false);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/storage-api/src/test/org/apache/hadoop/hive/common/type/HiveDecimalTestBase.java
----------------------------------------------------------------------
diff --git a/storage-api/src/test/org/apache/hadoop/hive/common/type/HiveDecimalTestBase.java b/storage-api/src/test/org/apache/hadoop/hive/common/type/HiveDecimalTestBase.java
new file mode 100644
index 0000000..553c456
--- /dev/null
+++ b/storage-api/src/test/org/apache/hadoop/hive/common/type/HiveDecimalTestBase.java
@@ -0,0 +1,558 @@
+/**
+ * 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.common.type;
+
+import java.util.Random;
+import java.math.BigDecimal;
+import java.math.BigInteger;
+
+import org.apache.hadoop.hive.common.type.RandomTypeUtil;
+
+public class HiveDecimalTestBase {
+
+  public static int POUND_FACTOR = 1000;
+
+  public static enum BigDecimalFlavor {
+    NORMAL_RANGE,
+    FRACTIONS_ONLY,
+    NEGATIVE_SCALE,
+    LONG_TAIL
+  }
+
+  public static enum BigDecimalPairFlavor {
+    RANDOM,
+    NEAR,
+    INVERSE
+  }
+
+  public BigDecimal randHiveBigDecimal(Random r, String digitAlphabet, BigDecimalFlavor bigDecimalFlavor) {
+    switch (bigDecimalFlavor) {
+    case NORMAL_RANGE:
+      return randHiveBigDecimalNormalRange(r, digitAlphabet);
+    case FRACTIONS_ONLY:
+      return randHiveBigDecimalFractionsOnly(r, digitAlphabet);
+    case NEGATIVE_SCALE:
+      return randHiveBigDecimalNegativeScale(r, digitAlphabet);
+    case LONG_TAIL:
+      return randHiveBigDecimalLongTail(r, digitAlphabet);
+    default:
+      throw new RuntimeException("Unexpected big decimal flavor " + bigDecimalFlavor);
+    }
+  }
+
+  public BigDecimal[] randHiveBigDecimalPair(Random r, String digitAlphabet,
+      BigDecimalFlavor bigDecimalFlavor, BigDecimalPairFlavor bigDecimalPairFlavor) {
+    BigDecimal[] pair = new BigDecimal[2];
+    BigDecimal bigDecimal1 = randHiveBigDecimal(r, digitAlphabet, bigDecimalFlavor);
+    pair[0] = bigDecimal1;
+
+    BigDecimal bigDecimal2;
+    switch (bigDecimalPairFlavor) {
+    case RANDOM:
+      bigDecimal2 = randHiveBigDecimal(r, digitAlphabet, bigDecimalFlavor);
+      break;
+    case NEAR:
+      bigDecimal2 = randHiveBigDecimalNear(r, bigDecimal1);
+      break;
+    case INVERSE:
+      bigDecimal2 = randHiveBigDecimalNear(r, bigDecimal1);
+      break;
+    default:
+      throw new RuntimeException("Unexpected big decimal pair flavor " + bigDecimalPairFlavor);
+    }
+    pair[1] = bigDecimal2;
+    return pair;
+  }
+
+  public BigDecimal randHiveBigDecimalNormalRange(Random r, String digitAlphabet) {
+    String digits = RandomTypeUtil.getRandString(r, digitAlphabet, 1 + r.nextInt(38));
+    BigInteger bigInteger = new BigInteger(digits);
+    boolean negated = false;
+    if (r.nextBoolean()) {
+      bigInteger = bigInteger.negate();
+      negated = true;
+    }
+    int scale = 0 + r.nextInt(38 + 1);
+    return new BigDecimal(bigInteger, scale);
+  }
+
+  public BigDecimal randHiveBigDecimalNegativeScale(Random r, String digitAlphabet) {
+    String digits = RandomTypeUtil.getRandString(r, digitAlphabet, 1 + r.nextInt(38));
+    BigInteger bigInteger = new BigInteger(digits);
+    boolean negated = false;
+    if (r.nextBoolean()) {
+      bigInteger = bigInteger.negate();
+      negated = true;
+    }
+    int scale = 0 + (r.nextBoolean() ? 0 : r.nextInt(38 + 1));
+    if (r.nextBoolean()) {
+      scale = -scale;
+    }
+    return new BigDecimal(bigInteger, scale);
+  }
+
+  public BigDecimal randHiveBigDecimalLongTail(Random r, String digitAlphabet) {
+    int scale = 0 + r.nextInt(38 + 20);
+    final int maxDigits = 38 + (scale == 0 ? 0 : 20);
+    String digits = RandomTypeUtil.getRandString(r, digitAlphabet, 1 + r.nextInt(maxDigits));
+    BigInteger bigInteger = new BigInteger(digits);
+    boolean negated = false;
+    if (r.nextBoolean()) {
+      bigInteger = bigInteger.negate();
+      negated = true;
+    }
+    return new BigDecimal(bigInteger, scale);
+  }
+
+  public BigDecimal randHiveBigDecimalFractionsOnly(Random r, String digitAlphabet) {
+    int scale = 1 + r.nextInt(38 + 1);
+    String digits = RandomTypeUtil.getRandString(r, digitAlphabet, 1 + r.nextInt(scale));
+    BigInteger bigInteger = new BigInteger(digits);
+    boolean negated = false;
+    if (r.nextBoolean()) {
+      bigInteger = bigInteger.negate();
+      negated = true;
+    }
+    return new BigDecimal(bigInteger, scale);
+  }
+
+  public BigDecimal randHiveBigDecimalNear(Random r, BigDecimal bigDecimal) {
+
+    int scale = bigDecimal.scale();
+    int delta = r.nextInt(10);
+    if (r.nextBoolean()) {
+      return bigDecimal.add(new BigDecimal(BigInteger.valueOf(delta), scale));
+    } else {
+      return bigDecimal.subtract(new BigDecimal(BigInteger.valueOf(delta), scale));
+    }
+  }
+
+  public BigDecimal randHiveBigDecimalInverse(Random r, BigDecimal bigDecimal) {
+    if (bigDecimal.signum() == 0) {
+      return bigDecimal;
+    }
+    return BigDecimal.ONE.divide(bigDecimal);
+  }
+
+  public BigInteger randHiveBigInteger(Random r, String digitAlphabet) {
+    String digits = RandomTypeUtil.getRandString(r, digitAlphabet, 1 + r.nextInt(38));
+    BigInteger bigInteger = new BigInteger(digits);
+    boolean negated = false;
+    if (r.nextBoolean()) {
+      bigInteger = bigInteger.negate();
+      negated = true;
+    }
+    return bigInteger;
+  }
+
+  public boolean isTenPowerBug(String string) {
+    // // System.out.println("TEST_IS_TEN_TO_38_STRING isTenPowerBug " + string);
+    if (string.charAt(0) == '-') {
+      string = string.substring(1);
+    }
+    int index = string.indexOf('.');
+    if (index != -1) {
+      if (index == 0) {
+        string = string.substring(1);
+      } else {
+        string = string.substring(0, index) + string.substring(index + 1);
+      }
+    }
+    // // System.out.println("TEST_IS_TEN_TO_38_STRING isTenPowerBug " + string);
+    return string.equals("100000000000000000000000000000000000000");
+  }
+
+  //------------------------------------------------------------------------------------------------
+
+  public static String[] specialDecimalStrings = new String[] {
+    "0",
+    "1",
+    "-1",
+    "10",
+    "-10",
+    "100",
+    "-100",
+    "127",                                          // Byte.MAX_VALUE
+    "127.1",
+    "127.0008",
+    "127.49",
+    "127.5",
+    "127.9999999999999999999",
+    "-127",
+    "-127.1",
+    "-127.0008",
+    "-127.49",
+    "-127.5",
+    "-127.999999",
+    "128",
+    "128.1",
+    "128.0008",
+    "128.49",
+    "128.5",
+    "128.9999999999999999999",
+    "-128",                                         // Byte.MIN_VALUE
+    "-128.1",
+    "-128.0008",
+    "-128.49",
+    "-128.5",
+    "-128.999",
+    "129",
+    "129.1",
+    "-129",
+    "-129.1",
+    "1000",
+    "-1000",
+    "10000",
+    "-10000",
+    "32767",                                        // Short.MAX_VALUE
+    "32767.1",
+    "32767.0008",
+    "32767.49",
+    "32767.5",
+    "32767.99999999999",
+    "-32767",
+    "-32767.1",
+    "-32767.0008",
+    "-32767.49",
+    "-32767.5",
+    "-32767.9",
+    "32768",
+    "32768.1",
+    "32768.0008",
+    "32768.49",
+    "32768.5",
+    "32768.9999999999",
+    "-32768",                                       // Short.MIN_VALUE
+    "-32768.1",
+    "-32768.0008",
+    "-32768.49",
+    "-32768.5",
+    "-32768.9999999",
+    "32769",
+    "32769.1",
+    "-32769",
+    "-32769.1",
+    "100000",
+    "-100000",
+    "1000000",
+    "-1000000",
+    "10000000",
+    "-10000000",
+    "100000000",
+    "99999999",                                     // 10^8 - 1
+    "-99999999",
+    "-100000000",
+    "1000000000",
+    "-1000000000",
+    "2147483647",                                  // Integer.MAX_VALUE
+    "2147483647.1",
+    "2147483647.0008",
+    "2147483647.49",
+    "2147483647.5",
+    "2147483647.9999999999",
+    "-2147483647",
+    "-2147483647.1",
+    "-2147483647.0008",
+    "-2147483647.49",
+    "-2147483647.5",
+    "-2147483647.9999999999999999999",
+    "2147483648",
+    "2147483648.1",
+    "2147483648.0008",
+    "2147483648.49",
+    "2147483648.5",
+    "2147483648.9",
+    "-2147483648",                                 // Integer.MIN_VALUE
+    "-2147483648.1",
+    "-2147483648.0008",
+    "-2147483648.49",
+    "-2147483648.5",
+    "-2147483648.999",
+    "2147483649",
+    "2147483649.1",
+    "-2147483649",
+    "-2147483649.1",
+    "10000000000",
+    "-10000000000",
+    "100000000000",
+    "-100000000000",
+    "1000000000000",
+    "-1000000000000",
+    "10000000000000",
+    "-10000000000000",
+    "100000000000000",
+    "-100000000000000",
+    "999999999999999",
+    "-999999999999999",
+    "1000000000000000",                            // 10^15
+    "-1000000000000000",
+    "9999999999999999",                            // 10^16 - 1
+    "-9999999999999999",
+    "10000000000000000",                           // 10^16
+    "-10000000000000000",
+    "100000000000000000",
+    "-100000000000000000",
+    "1000000000000000000",
+    "-1000000000000000000",
+    "9223372036854775807",                         // Long.MAX_VALUE
+    "9223372036854775807.1",
+    "9223372036854775807.0008",
+    "9223372036854775807.49",
+    "9223372036854775807.5",
+    "9223372036854775807.9",
+    "-9223372036854775807",
+    "-9223372036854775807.1",
+    "-9223372036854775807.0008",
+    "-9223372036854775807.49",
+    "-9223372036854775807.5",
+    "-9223372036854775807.9999999999999999999",
+    "-9223372036854775808",
+    "-9223372036854775808.1",
+    "9223372036854775808",
+    "9223372036854775808.1",
+    "9223372036854775808.0008",
+    "9223372036854775808.49",
+    "9223372036854775808.5",
+    "9223372036854775808.9",
+    "9223372036854775809",
+    "9223372036854775809.1",
+    "-9223372036854775808",                        // Long.MIN_VALUE
+    "-9223372036854775808.1",
+    "-9223372036854775808.0008",
+    "-9223372036854775808.49",
+    "-9223372036854775808.5",
+    "-9223372036854775808.9999999",
+    "9223372036854775809",
+    "9223372036854775809.1",
+    "-9223372036854775809",
+    "-9223372036854775809.1",
+    "10000000000000000000000000000000",            // 10^31
+    "-10000000000000000000000000000000",
+    "99999999999999999999999999999999",            // 10^32 - 1
+    "-99999999999999999999999999999999", 
+    "100000000000000000000000000000000",           // 10^32
+    "-100000000000000000000000000000000",
+    "10000000000000000000000000000000000000",      // 10^37
+    "-10000000000000000000000000000000000000",
+    "99999999999999999999999999999999999999",      // 10^38 - 1
+    "-99999999999999999999999999999999999999",
+    "100000000000000000000000000000000000000",     // 10^38
+    "-100000000000000000000000000000000000000", 
+    "1000000000000000000000000000000000000000",    // 10^39
+    "-1000000000000000000000000000000000000000",
+
+    "18446744073709551616",                        // Unsigned 64 max.
+    "-18446744073709551616",
+    "340282366920938463463374607431768211455",     // 2^128 - 1
+    "-340282366920938463463374607431768211455",
+
+    "0.999999999999999",
+    "-0.999999999999999",
+    "0.0000000000000001",                          // 10^-15
+    "-0.0000000000000001",
+    "0.9999999999999999",
+    "-0.9999999999999999",
+    "0.00000000000000001",                         // 10^-16
+    "-0.00000000000000001",
+    "0.00000000000000000000000000000001",          // 10^-31
+    "-0.00000000000000000000000000000001",
+    "0.99999999999999999999999999999999",          // 10^-32 + 1
+    "-0.99999999999999999999999999999999",
+    "0.000000000000000000000000000000001",         // 10^-32
+    "-0.000000000000000000000000000000001",
+    "0.00000000000000000000000000000000000001",    // 10^-37
+    "-0.00000000000000000000000000000000000001",
+    "0.99999999999999999999999999999999999999",    // 10^-38 + 1
+    "-0.99999999999999999999999999999999999999",
+    "0.000000000000000000000000000000000000001",   // 10^-38
+    "-0.000000000000000000000000000000000000001",
+    "0.0000000000000000000000000000000000000001",  // 10^-39
+    "-0.0000000000000000000000000000000000000001",
+    "0.0000000000000000000000000000000000000005",  // 10^-39  (rounds)
+    "-0.0000000000000000000000000000000000000005",
+    "0.340282366920938463463374607431768211455",   // (2^128 - 1) * 10^-39
+    "-0.340282366920938463463374607431768211455",
+    "0.000000000000000000000000000000000000001",   // 10^-38
+    "-0.000000000000000000000000000000000000001",
+    "0.000000000000000000000000000000000000005",   // 10^-38
+    "-0.000000000000000000000000000000000000005",
+
+    "234.79",
+    "342348.343",
+    "12.25",
+    "-12.25",
+    "72057594037927935",                           // 2^56 - 1
+    "-72057594037927935",
+    "72057594037927936",                           // 2^56
+    "-72057594037927936",
+    "5192296858534827628530496329220095",          // 2^56 * 2^56 - 1
+    "-5192296858534827628530496329220095",
+    "5192296858534827628530496329220096",          // 2^56 * 2^56
+    "-5192296858534827628530496329220096",
+
+    "54216721532321902598.70",
+    "-906.62545207002374150309544832320",
+    "-0.0709351061072",
+    "1460849063411925.53",
+    "8.809130E-33",
+    "-4.0786300706013636202E-20",
+    "-3.8823936518E-1",
+    "-3.8823936518E-28",
+    "-3.8823936518E-29",
+    "598575157855521918987423259.94094",
+    "299999448432.001342152474197",
+    "1786135888657847525803324040144343378.09799306448796128931113691624",  // More than 38 digits.
+    "-1786135888657847525803324040144343378.09799306448796128931113691624",
+    "57847525803324040144343378.09799306448796128931113691624",
+    "0.999999999999999999990000",
+    "005.34000",
+    "1E-90",
+
+    "0.4",
+    "-0.4",
+    "0.5",
+    "-0.5",
+    "0.6",
+    "-0.6",
+    "1.4",
+    "-1.4",
+    "1.5",
+    "-1.5",
+    "1.6",
+    "-1.6",
+    "2.4",
+    "-2.4",
+    "2.49",
+    "-2.49",
+    "2.5",
+    "-2.5",
+    "2.51",
+    "-2.51",
+    "-2.5",
+    "2.6",
+    "-2.6",
+    "3.00001415926",
+    "0.00",
+    "-12.25",
+    "234.79"
+  };
+
+  public static BigDecimal[] specialBigDecimals = stringArrayToBigDecimals(specialDecimalStrings);
+
+  // decimal_1_1.txt
+  public static String[] decimal_1_1_txt = {
+    "0.0",
+    "0.0000",
+    ".0",
+    "0.1",
+    "0.15",
+    "0.9",
+    "0.94",
+    "0.99",
+    "0.345",
+    "1.0",
+    "1",
+    "0",
+    "00",
+    "22",
+    "1E-9",
+    "-0.0",
+    "-0.0000",
+    "-.0",
+    "-0.1",
+    "-0.15",
+    "-0.9",
+    "-0.94",
+    "-0.99",
+    "-0.345",
+    "-1.0",
+    "-1",
+    "-0",
+    "-00",
+    "-22",
+    "-1E-9"
+  };
+
+  // kv7.txt KEYS
+  public static String[] kv7_txt_keys = {
+    "-4400",
+    "1E+99",
+    "1E-99",
+    "0",
+    "100",
+    "10",
+    "1",
+    "0.1",
+    "0.01",
+    "200",
+    "20",
+    "2",
+    "0",
+    "0.2",
+    "0.02",
+    "0.3",
+    "0.33",
+    "0.333",
+    "-0.3",
+    "-0.33",
+    "-0.333",
+    "1.0",
+    "2",
+    "3.14",
+    "-1.12",
+    "-1.12",
+    "-1.122",
+    "1.12",
+    "1.122",
+    "124.00",
+    "125.2",
+    "-1255.49",
+    "3.14",
+    "3.14",
+    "3.140",
+    "0.9999999999999999999999999",
+    "-1234567890.1234567890",
+    "1234567890.1234567800"
+  };
+
+  public static String standardAlphabet = "0123456789";
+
+  public static String[] sparseAlphabets = new String[] {
+
+    "0000000000000000000000000000000000000003",
+    "0000000000000000000000000000000000000009",
+    "0000000000000000000000000000000000000001",
+    "0000000000000000000003",
+    "0000000000000000000009",
+    "0000000000000000000001",
+    "0000000000091",
+    "000000000005",
+    "9",
+    "5555555555999999999000000000000001111111",
+    "24680",
+    "1"
+  };
+
+  public static BigDecimal[] stringArrayToBigDecimals(String[] strings) {
+    BigDecimal[] result = new BigDecimal[strings.length];
+    for (int i = 0; i < strings.length; i++) {
+      result[i] = new BigDecimal(strings[i]);
+    }
+    return result;
+  }
+}
\ No newline at end of file


[06/10] hive git commit: HIVE-15335: Fast Decimal (Matt McCline, reviewed by Sergey Shelukhin, Prasanth Jayachandran, Owen O'Malley)

Posted by mm...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/storage-api/src/java/org/apache/hadoop/hive/common/type/FastHiveDecimal.java
----------------------------------------------------------------------
diff --git a/storage-api/src/java/org/apache/hadoop/hive/common/type/FastHiveDecimal.java b/storage-api/src/java/org/apache/hadoop/hive/common/type/FastHiveDecimal.java
new file mode 100644
index 0000000..a8c183a
--- /dev/null
+++ b/storage-api/src/java/org/apache/hadoop/hive/common/type/FastHiveDecimal.java
@@ -0,0 +1,741 @@
+/**
+ * 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.common.type;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.math.BigDecimal;
+import java.math.BigInteger;
+
+/**
+ *    FastHiveDecimal is a mutable fast decimal object.  It is the base class for both the
+ *    HiveDecimal and HiveDecimalWritable classes.  All fast* methods are protected so they
+ *    cannot be accessed by clients of HiveDecimal and HiveDecimalWritable.  HiveDecimal ensures
+ *    it creates new objects when the value changes since it provides immutable semantics;
+ *    HiveDecimalWritable does not create new objects since it provides mutable semantics.
+ *
+ *    The methods in this class are shells that pickup the member variables from FastHiveDecimal
+ *    parameters and pass them as individual parameters to static methods in the FastHiveDecimalImpl
+ *    class that do the real work.
+ *
+ *    NOTE: The rationale for fast decimal is in FastHiveDecimalImpl.
+ */
+public class FastHiveDecimal {
+
+  /*
+   * We use protected for the fields so the FastHiveDecimalImpl class can access them.  Other
+   * classes including HiveDecimal should not access these fields directly.
+   */
+
+  // See FastHiveDecimalImpl for more details on these fields.
+
+  // -1 when negative; 0 when decimal is zero; 1 when positive.
+  protected int fastSignum;
+
+  // Decimal longwords.
+  protected long fast2;
+  protected long fast1;
+  protected long fast0;
+
+  // The number of integer digits in the decimal.  When the integer portion is zero, this is 0.
+  protected int fastIntegerDigitCount;
+
+  // The scale of the decimal.
+  protected int fastScale;
+
+  // Used for legacy HiveDecimalV1 setScale compatibility for binary / display serialization of
+  // trailing zeroes (or rounding).
+  protected int fastSerializationScale;
+
+  protected FastHiveDecimal() {
+    fastReset();
+  }
+
+  protected FastHiveDecimal(FastHiveDecimal fastDec) {
+    this();
+    fastSignum = fastDec.fastSignum;
+    fast0 = fastDec.fast0;
+    fast1 = fastDec.fast1;
+    fast2 = fastDec.fast2;
+    fastIntegerDigitCount = fastDec.fastIntegerDigitCount;
+    fastScale = fastDec.fastScale;
+
+    // Not propagated.
+    fastSerializationScale = -1;
+  }
+
+  protected FastHiveDecimal(int fastSignum, FastHiveDecimal fastDec) {
+    this();
+    this.fastSignum = fastSignum;
+    fast0 = fastDec.fast0;
+    fast1 = fastDec.fast1;
+    fast2 = fastDec.fast2;
+    fastIntegerDigitCount = fastDec.fastIntegerDigitCount;
+    fastScale = fastDec.fastScale;
+
+    // Not propagated.
+    fastSerializationScale = -1;
+  }
+
+  protected FastHiveDecimal(
+      int fastSignum, long fast0, long fast1, long fast2,
+      int fastIntegerDigitCount, int fastScale) {
+    this();
+    this.fastSignum = fastSignum;
+    this.fast0 = fast0;
+    this.fast1 = fast1;
+    this.fast2 = fast2;
+    this.fastIntegerDigitCount = fastIntegerDigitCount;
+    this.fastScale = fastScale;
+
+    fastSerializationScale = -1;
+  }
+
+  protected FastHiveDecimal(long longValue) {
+    this();
+    FastHiveDecimalImpl.fastSetFromLong(longValue, this);
+  }
+
+  protected FastHiveDecimal(String string) {
+    this();
+    FastHiveDecimalImpl.fastSetFromString(string, false, this);
+  }
+
+  protected void fastReset() {
+    fastSignum = 0;
+    fast0 = 0;
+    fast1 = 0;
+    fast2 = 0;
+    fastIntegerDigitCount = 0;
+    fastScale = 0;
+    fastSerializationScale = -1;
+  }
+
+  protected void fastSet(FastHiveDecimal fastDec) {
+    fastSignum = fastDec.fastSignum;
+    fast0 = fastDec.fast0;
+    fast1 = fastDec.fast1;
+    fast2 = fastDec.fast2;
+    fastIntegerDigitCount = fastDec.fastIntegerDigitCount;
+    fastScale = fastDec.fastScale;
+    fastSerializationScale = fastDec.fastSerializationScale;
+  }
+
+  protected void fastSet(
+      int fastSignum, long fast0, long fast1, long fast2, int fastIntegerDigitCount, int fastScale) {
+    this.fastSignum = fastSignum;
+    this.fast0 = fast0;
+    this.fast1 = fast1;
+    this.fast2 = fast2;
+    this.fastIntegerDigitCount = fastIntegerDigitCount;
+    this.fastScale = fastScale;
+
+    // Not specified.
+    fastSerializationScale = -1;
+  }
+
+  protected void fastSetSerializationScale(int fastSerializationScale) {
+    this.fastSerializationScale = fastSerializationScale;
+  }
+
+  protected int fastSerializationScale() {
+    return fastSerializationScale;
+  }
+
+  protected static final String STRING_ENFORCE_PRECISION_OUT_OF_RANGE =
+      "Decimal precision out of allowed range [1," + HiveDecimal.MAX_PRECISION + "]";
+  protected static final String STRING_ENFORCE_SCALE_OUT_OF_RANGE =
+      "Decimal scale out of allowed range [0," + HiveDecimal.MAX_SCALE + "]";
+  protected static final String STRING_ENFORCE_SCALE_LESS_THAN_EQUAL_PRECISION =
+      "Decimal scale must be less than or equal to precision";
+
+  protected boolean fastSetFromBigDecimal(
+      BigDecimal bigDecimal, boolean allowRounding) {
+    return
+        FastHiveDecimalImpl.fastSetFromBigDecimal(
+            bigDecimal, allowRounding, this);
+  }
+
+  protected boolean fastSetFromBigInteger(
+      BigInteger bigInteger) {
+    return
+        FastHiveDecimalImpl.fastSetFromBigInteger(
+            bigInteger, this);
+  }
+
+  protected boolean fastSetFromBigIntegerAndScale(
+      BigInteger bigInteger, int scale) {
+    return
+        FastHiveDecimalImpl.fastSetFromBigInteger(
+            bigInteger, scale, this);
+  }
+
+  protected boolean fastSetFromString(String string, boolean trimBlanks) {
+    byte[] bytes = string.getBytes();
+    return
+        fastSetFromBytes(
+            bytes, 0, bytes.length, trimBlanks);
+  }
+
+  protected boolean fastSetFromBytes(byte[] bytes, int offset, int length, boolean trimBlanks) {
+    return
+        FastHiveDecimalImpl.fastSetFromBytes(
+            bytes, offset, length, trimBlanks, this);
+  }
+
+  protected boolean fastSetFromDigitsOnlyBytesAndScale(
+      boolean isNegative, byte[] bytes, int offset, int length, int scale) {
+    return
+        FastHiveDecimalImpl.fastSetFromDigitsOnlyBytesAndScale(
+            isNegative, bytes, offset, length, scale, this);
+  }
+
+  protected void fastSetFromInt(int intValue) {
+    FastHiveDecimalImpl.fastSetFromInt(intValue, this);
+  }
+
+  protected void fastSetFromLong(long longValue) {
+    FastHiveDecimalImpl.fastSetFromLong(longValue, this);
+  }
+
+  protected boolean fastSetFromLongAndScale(long longValue, int scale) {
+    return
+        FastHiveDecimalImpl.fastSetFromLongAndScale(
+            longValue, scale, this);
+  }
+
+  protected boolean fastSetFromFloat(float floatValue) {
+    return
+        FastHiveDecimalImpl.fastSetFromFloat(
+            floatValue, this);
+  }
+
+  protected boolean fastSetFromDouble(double doubleValue) {
+    return
+        FastHiveDecimalImpl.fastSetFromDouble(
+            doubleValue, this);
+  }
+
+  protected void fastFractionPortion() {
+    FastHiveDecimalImpl.fastFractionPortion(
+        fastSignum, fast0, fast1, fast2, fastIntegerDigitCount, fastScale,
+        this);
+  }
+
+  protected void fastIntegerPortion() {
+    FastHiveDecimalImpl.fastIntegerPortion(
+        fastSignum, fast0, fast1, fast2, fastIntegerDigitCount, fastScale,
+        this);
+  }
+
+  protected static final int FAST_SCRATCH_BUFFER_LEN_SERIALIZATION_UTILS_READ = 8 * 3;
+
+  protected boolean fastSerializationUtilsRead(
+      InputStream inputStream, int scale,
+      byte[] scratchBytes) throws IOException, EOFException {
+    return
+        FastHiveDecimalImpl.fastSerializationUtilsRead(
+            inputStream, scale, scratchBytes, this);
+  }
+
+  protected boolean fastSetFromBigIntegerBytesAndScale(
+      byte[] bytes, int offset, int length, int scale) {
+    return FastHiveDecimalImpl.fastSetFromBigIntegerBytesAndScale(
+        bytes, offset, length, scale, this);
+  }
+
+  protected static final int SCRATCH_LONGS_LEN_FAST_SERIALIZATION_UTILS_WRITE = 6;
+
+  protected boolean fastSerializationUtilsWrite(OutputStream outputStream,
+      long[] scratchLongs)
+          throws IOException {
+    return
+        FastHiveDecimalImpl.fastSerializationUtilsWrite(
+            outputStream,
+            fastSignum, fast0, fast1, fast2, fastIntegerDigitCount, fastScale,
+            scratchLongs);
+  }
+
+  // The fastBigIntegerBytes method returns 3 56 bit (7 byte) words and a possible sign byte.
+  // However, the fastBigIntegerBytes can take on trailing zeroes -- so make it larger.
+  protected static final int FAST_SCRATCH_BUFFER_LEN_BIG_INTEGER_BYTES = 1 + 48;
+  protected static final int FAST_SCRATCH_LONGS_LEN = 6;
+
+  protected int fastBigIntegerBytes(
+      long[] scratchLongs, byte[] buffer) {
+    return
+        FastHiveDecimalImpl.fastBigIntegerBytes(
+            fastSignum, fast0, fast1, fast2,
+            fastIntegerDigitCount, fastScale,
+            fastSerializationScale,
+            scratchLongs, buffer);
+  }
+
+  protected int fastBigIntegerBytesScaled(
+      int serializationScale,
+      long[] scratchLongs, byte[] buffer) {
+    return
+        FastHiveDecimalImpl.fastBigIntegerBytesScaled(
+            fastSignum, fast0, fast1, fast2,
+            fastIntegerDigitCount, fastScale,
+            serializationScale,
+            scratchLongs, buffer);
+  }
+
+  protected boolean fastIsByte() {
+    return
+        FastHiveDecimalImpl.fastIsByte(
+            fastSignum, fast0, fast1, fast2, fastIntegerDigitCount, fastScale);
+  }
+
+  protected byte fastByteValueClip() {
+    return
+        FastHiveDecimalImpl.fastByteValueClip(
+            fastSignum, fast0, fast1, fast2, fastIntegerDigitCount, fastScale);
+  }
+
+  protected boolean fastIsShort() {
+    return
+        FastHiveDecimalImpl.fastIsShort(
+            fastSignum, fast0, fast1, fast2, fastIntegerDigitCount, fastScale);
+  }
+
+  protected short fastShortValueClip() {
+    return
+        FastHiveDecimalImpl.fastShortValueClip(
+            fastSignum, fast0, fast1, fast2, fastIntegerDigitCount, fastScale);
+  }
+
+  protected boolean fastIsInt() {
+    return
+        FastHiveDecimalImpl.fastIsInt(
+            fastSignum, fast0, fast1, fast2, fastIntegerDigitCount, fastScale);
+  }
+
+  protected int fastIntValueClip() {
+    return
+        FastHiveDecimalImpl.fastIntValueClip(
+            fastSignum, fast0, fast1, fast2, fastIntegerDigitCount, fastScale);
+  }
+
+  protected boolean fastIsLong() {
+    return
+        FastHiveDecimalImpl.fastIsLong(
+            fastSignum, fast0, fast1, fast2, fastIntegerDigitCount, fastScale);
+  }
+
+  protected long fastLongValueClip() {
+    return
+        FastHiveDecimalImpl.fastLongValueClip(
+            fastSignum, fast0, fast1, fast2, fastIntegerDigitCount, fastScale);
+  }
+
+  protected float fastFloatValue() {
+    return
+        FastHiveDecimalImpl.fastFloatValue(
+            fastSignum, fast0, fast1, fast2, fastIntegerDigitCount, fastScale);
+  }
+
+  protected double fastDoubleValue() {
+    return
+        FastHiveDecimalImpl.fastDoubleValue(
+            fastSignum, fast0, fast1, fast2, fastIntegerDigitCount, fastScale);
+  }
+
+  protected BigInteger fastBigIntegerValue() {
+    return
+        FastHiveDecimalImpl.fastBigIntegerValue(
+            fastSignum, fast0, fast1, fast2,
+            fastIntegerDigitCount, fastScale,
+            fastSerializationScale);
+  }
+
+  protected BigDecimal fastBigDecimalValue() {
+    return
+        FastHiveDecimalImpl.fastBigDecimalValue(
+            fastSignum, fast0, fast1, fast2,
+            fastIntegerDigitCount, fastScale);
+  }
+
+  protected int fastScale() {
+    return fastScale;
+  }
+
+  protected int fastSignum() {
+    return fastSignum;
+  }
+
+  protected int fastCompareTo(FastHiveDecimal right) {
+    return
+        FastHiveDecimalImpl.fastCompareTo(
+            fastSignum, fast0, fast1, fast2,
+            fastScale,
+            right.fastSignum, right.fast0, right.fast1, right.fast2,
+            right.fastScale);
+  }
+
+  protected static int fastCompareTo(FastHiveDecimal left, FastHiveDecimal right) {
+    return
+        FastHiveDecimalImpl.fastCompareTo(
+            left.fastSignum, left.fast0, left.fast1, left.fast2,
+            left.fastScale,
+            right.fastSignum, right.fast0, right.fast1, right.fast2,
+            right.fastScale);
+  }
+
+  protected boolean fastEquals(FastHiveDecimal that) {
+    return
+        FastHiveDecimalImpl.fastEquals(
+          fastSignum, fast0, fast1, fast2,
+          fastScale,
+          that.fastSignum, that.fast0, that.fast1, that.fast2,
+          that.fastScale);
+  }
+
+  protected void fastAbs() {
+    fastSignum = 1;
+  }
+
+  protected void fastNegate() {
+    if (fastSignum == 0) {
+      return;
+    }
+    fastSignum = (fastSignum == 1 ? -1 : 1);
+  }
+
+  protected int fastNewFasterHashCode() {
+    return
+        FastHiveDecimalImpl.fastNewFasterHashCode(
+            fastSignum, fast0, fast1, fast2, fastIntegerDigitCount, fastScale);
+  }
+
+  protected int fastHashCode() {
+    return
+        FastHiveDecimalImpl.fastHashCode(
+            fastSignum, fast0, fast1, fast2, fastIntegerDigitCount, fastScale);
+  }
+
+  protected int fastIntegerDigitCount() {
+    return fastIntegerDigitCount;
+  }
+
+  protected int fastSqlPrecision() {
+    return
+        FastHiveDecimalImpl.fastSqlPrecision(
+            fastSignum, fast0, fast1, fast2,
+            fastIntegerDigitCount, fastScale);
+  }
+
+  protected int fastRawPrecision() {
+    return
+        FastHiveDecimalImpl.fastRawPrecision(
+            fastSignum, fast0, fast1, fast2);
+  }
+
+  protected boolean fastScaleByPowerOfTen(
+      int n,
+      FastHiveDecimal fastResult) {
+    return
+        FastHiveDecimalImpl.fastScaleByPowerOfTen(
+            fastSignum, fast0, fast1, fast2, fastIntegerDigitCount, fastScale,
+            n,
+            fastResult);
+  }
+
+  protected static String fastRoundingModeToString(int roundingMode) {
+    String roundingModeString;
+    switch (roundingMode) {
+    case BigDecimal.ROUND_DOWN:
+      roundingModeString = "ROUND_DOWN";
+      break;
+    case BigDecimal.ROUND_UP:
+      roundingModeString = "ROUND_UP";
+      break;
+    case BigDecimal.ROUND_FLOOR:
+      roundingModeString = "ROUND_FLOOR";
+      break;
+    case BigDecimal.ROUND_CEILING:
+      roundingModeString = "ROUND_CEILING";
+      break;
+    case BigDecimal.ROUND_HALF_UP:
+      roundingModeString = "ROUND_HALF_UP";
+      break;
+    case BigDecimal.ROUND_HALF_EVEN:
+      roundingModeString = "ROUND_HALF_EVEN";
+      break;
+    default:
+      roundingModeString = "Unknown";
+    }
+    return roundingModeString + " (" + roundingMode + ")";
+  }
+
+  protected boolean fastRound(
+      int newScale, int roundingMode,
+      FastHiveDecimal fastResult) {
+    return
+        FastHiveDecimalImpl.fastRound(
+            fastSignum, fast0, fast1, fast2,
+            fastIntegerDigitCount, fastScale,
+            newScale, roundingMode,
+            fastResult);
+  }
+
+  protected boolean isAllZeroesBelow(
+      int power) {
+    return
+        FastHiveDecimalImpl.isAllZeroesBelow(
+            fastSignum, fast0, fast1, fast2, power);
+  }
+
+  protected boolean fastEnforcePrecisionScale(
+      int maxPrecision, int maxScale) {
+    if (maxPrecision <= 0 || maxPrecision > HiveDecimal.MAX_PRECISION) {
+      return false;
+    }
+    if (maxScale < 0 || maxScale > HiveDecimal.MAX_SCALE) {
+      return false;
+    }
+    /*
+    if (!fastIsValid()) {
+      fastRaiseInvalidException();
+    }
+    */
+    FastCheckPrecisionScaleStatus status =
+        FastHiveDecimalImpl.fastCheckPrecisionScale(
+          fastSignum, fast0, fast1, fast2,
+          fastIntegerDigitCount, fastScale,
+          maxPrecision, maxScale);
+    switch (status) {
+    case NO_CHANGE:
+      return true;
+    case OVERFLOW:
+      return false;
+    case UPDATE_SCALE_DOWN:
+      {
+        if (!FastHiveDecimalImpl.fastUpdatePrecisionScale(
+          fastSignum, fast0, fast1, fast2,
+          fastIntegerDigitCount, fastScale,
+          maxPrecision, maxScale, status,
+          this)) {
+          return false;
+        }
+        /*
+        if (!fastIsValid()) {
+          fastRaiseInvalidException();
+        }
+        */
+        return true;
+      }
+    default:
+      throw new RuntimeException("Unknown fast decimal check precision and scale status " + status);
+    }
+  }
+
+  protected FastCheckPrecisionScaleStatus fastCheckPrecisionScale(
+      int maxPrecision, int maxScale) {
+    return
+        FastHiveDecimalImpl.fastCheckPrecisionScale(
+            fastSignum, fast0, fast1, fast2,
+            fastIntegerDigitCount, fastScale,
+            maxPrecision, maxScale);
+  }
+
+  protected static enum FastCheckPrecisionScaleStatus {
+    NO_CHANGE,
+    OVERFLOW,
+    UPDATE_SCALE_DOWN;
+  }
+
+  protected boolean fastUpdatePrecisionScale(
+      int maxPrecision, int maxScale, FastCheckPrecisionScaleStatus status,
+      FastHiveDecimal fastResult) {
+    return
+        FastHiveDecimalImpl.fastUpdatePrecisionScale(
+            fastSignum, fast0, fast1, fast2,
+            fastIntegerDigitCount, fastScale,
+            maxPrecision, maxScale, status,
+            fastResult);
+  }
+
+  protected boolean fastAdd(
+      FastHiveDecimal fastRight,
+      FastHiveDecimal fastResult) {
+    return
+        FastHiveDecimalImpl.fastAdd(
+            fastSignum, fast0, fast1, fast2,
+            fastIntegerDigitCount, fastScale,
+            fastRight.fastSignum, fastRight.fast0, fastRight.fast1, fastRight.fast2,
+            fastRight.fastIntegerDigitCount, fastRight.fastScale,
+            fastResult);
+  }
+
+  protected boolean fastSubtract(
+      FastHiveDecimal fastRight,
+      FastHiveDecimal fastResult) {
+    return
+        FastHiveDecimalImpl.fastSubtract(
+            fastSignum, fast0, fast1, fast2,
+            fastIntegerDigitCount, fastScale,
+            fastRight.fastSignum, fastRight.fast0, fastRight.fast1, fastRight.fast2,
+            fastRight.fastIntegerDigitCount, fastRight.fastScale,
+            fastResult);
+  }
+
+  protected boolean fastMultiply(
+      FastHiveDecimal fastRight,
+      FastHiveDecimal fastResult) {
+    return
+        FastHiveDecimalImpl.fastMultiply(
+            fastSignum, fast0, fast1, fast2,
+            fastIntegerDigitCount, fastScale,
+            fastRight.fastSignum, fastRight.fast0, fastRight.fast1, fastRight.fast2,
+            fastRight.fastIntegerDigitCount, fastRight.fastScale,
+            fastResult);
+  }
+
+  protected boolean fastRemainder(
+      FastHiveDecimal fastRight,
+      FastHiveDecimal fastResult) {
+    return
+        FastHiveDecimalImpl.fastRemainder(
+            fastSignum, fast0, fast1, fast2,
+            fastIntegerDigitCount, fastScale,
+            fastRight.fastSignum, fastRight.fast0, fastRight.fast1, fastRight.fast2,
+            fastRight.fastIntegerDigitCount, fastRight.fastScale,
+            fastResult);
+  }
+
+  protected boolean fastDivide(
+      FastHiveDecimal fastRight,
+      FastHiveDecimal fastResult) {
+    return
+        FastHiveDecimalImpl.fastDivide(
+            fastSignum, fast0, fast1, fast2,
+            fastIntegerDigitCount, fastScale,
+            fastRight.fastSignum, fastRight.fast0, fastRight.fast1, fastRight.fast2,
+            fastRight.fastIntegerDigitCount, fastRight.fastScale,
+            fastResult);
+  }
+
+  protected boolean fastPow(
+      int exponent,
+      FastHiveDecimal fastResult) {
+    return
+        FastHiveDecimalImpl.fastPow(
+            fastSignum, fast0, fast1, fast2, fastIntegerDigitCount, fastScale,
+            exponent,
+            fastResult);
+  }
+
+  protected String fastToString(
+      byte[] scratchBuffer) {
+    return
+        FastHiveDecimalImpl.fastToString(
+            fastSignum, fast0, fast1, fast2,
+            fastIntegerDigitCount, fastScale, -1,
+            scratchBuffer);
+  }
+
+  protected String fastToString() {
+    return
+        FastHiveDecimalImpl.fastToString(
+            fastSignum, fast0, fast1, fast2,
+            fastIntegerDigitCount, fastScale, -1);
+  }
+
+  protected String fastToFormatString(int formatScale) {
+    return
+        FastHiveDecimalImpl.fastToFormatString(
+            fastSignum, fast0, fast1, fast2,
+            fastIntegerDigitCount, fastScale,
+            formatScale);
+  }
+
+  protected String fastToFormatString(
+      int formatScale,
+      byte[] scratchBuffer) {
+    return
+        FastHiveDecimalImpl.fastToFormatString(
+            fastSignum, fast0, fast1, fast2,
+            fastIntegerDigitCount, fastScale,
+            formatScale,
+            scratchBuffer);
+  }
+
+  protected String fastToDigitsOnlyString() {
+    return
+        FastHiveDecimalImpl.fastToDigitsOnlyString(
+            fast0, fast1, fast2,
+            fastIntegerDigitCount);
+  }
+
+  // Sign, zero, dot, 2 * digits (to support toFormatString which can add a lot of trailing zeroes).
+  protected final static int FAST_SCRATCH_BUFFER_LEN_TO_BYTES =
+      1 + 1 + 1 + 2 * FastHiveDecimalImpl.MAX_DECIMAL_DIGITS;
+
+  protected int fastToBytes(
+      byte[] scratchBuffer) {
+    return
+        FastHiveDecimalImpl.fastToBytes(
+            fastSignum, fast0, fast1, fast2,
+            fastIntegerDigitCount, fastScale, -1,
+            scratchBuffer);
+  }
+
+  protected int fastToFormatBytes(
+      int formatScale,
+      byte[] scratchBuffer) {
+    return
+        FastHiveDecimalImpl.fastToFormatBytes(
+            fastSignum, fast0, fast1, fast2,
+            fastIntegerDigitCount, fastScale,
+            formatScale,
+            scratchBuffer);
+  }
+
+  protected int fastToDigitsOnlyBytes(
+      byte[] scratchBuffer) {
+    return
+        FastHiveDecimalImpl.fastToDigitsOnlyBytes(
+            fast0, fast1, fast2,
+            fastIntegerDigitCount,
+            scratchBuffer);
+  }
+
+  @Override
+  public String toString() {
+    return
+        FastHiveDecimalImpl.fastToString(
+            fastSignum, fast0, fast1, fast2,
+            fastIntegerDigitCount, fastScale, -1);
+  }
+
+  protected boolean fastIsValid() {
+    return FastHiveDecimalImpl.fastIsValid(this);
+  }
+
+  protected void fastRaiseInvalidException() {
+    FastHiveDecimalImpl.fastRaiseInvalidException(this);
+  }
+
+  protected void fastRaiseInvalidException(String parameters) {
+    FastHiveDecimalImpl.fastRaiseInvalidException(this, parameters);
+  }
+}
\ No newline at end of file


[04/10] hive git commit: HIVE-15335: Fast Decimal (Matt McCline, reviewed by Sergey Shelukhin, Prasanth Jayachandran, Owen O'Malley)

Posted by mm...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/storage-api/src/java/org/apache/hadoop/hive/common/type/HiveDecimal.java
----------------------------------------------------------------------
diff --git a/storage-api/src/java/org/apache/hadoop/hive/common/type/HiveDecimal.java b/storage-api/src/java/org/apache/hadoop/hive/common/type/HiveDecimal.java
index 674400c..bf954a8 100644
--- a/storage-api/src/java/org/apache/hadoop/hive/common/type/HiveDecimal.java
+++ b/storage-api/src/java/org/apache/hadoop/hive/common/type/HiveDecimal.java
@@ -17,322 +17,1485 @@
  */
 package org.apache.hadoop.hive.common.type;
 
+import java.util.Arrays;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
 import java.math.BigDecimal;
 import java.math.BigInteger;
-import java.math.RoundingMode;
 
 /**
- *
- * HiveDecimal. Simple wrapper for BigDecimal. Adds fixed max precision and non scientific string
- * representation
+ * HiveDecimal is a decimal data type with a maximum precision and scale.
+ * <p>
+ * It is the Hive DECIMAL data type.
+ * <p>
+ * The scale is the number of fractional decimal digits.  The digits after the dot.  It is limited
+ * to 38 (MAX_SCALE).
+ * <p>
+ * The precision is the integer (or whole-number) decimal digits plus fractional decimal digits.
+ * It is limited to a total of 38 digits (MAX_PRECISION).
+ * <p>
+ * Hive syntax for declaring DECIMAL has 3 forms:
+ * <p>
+ * {@code
+ *       DECIMAL                            // Use the default precision/scale.}
+ * <p>
+ * {@code
+ *       DECIMAL(precision)                 // Use the default scale.}
+ * <p>
+ * {@code
+ *       DECIMAL(precision, scale)}
+ * }
+ * <p>
+ * The declared scale must be <= precision.
+ * <p>
+ * Use DECIMAL instead of DOUBLE when exact numeric accuracy is required.  Not all decimal numbers
+ * (radix 10) are exactly representable in the binary (radix 2 based) floating point type DOUBLE and
+ * cause accuracy anomalies (i.e. wrong results).  See the Internet for more details.
+ * <p>
+ * HiveDecimal is implemented as a classic Java immutable object.  All operations on HiveDecimal
+ * that produce a different value will create a new HiveDecimal object.
+ * <p>
+ * Decimals are physically stored without any extra leading or trailing zeroes.  The scale of
+ * a decimal is the number of non-trailing zero fractional digits.
+ * <p>
+ * Math operations on decimals typically cause the scale to change as a result of the math and
+ * from trailing fractional digit elimination.
+ * <p>
+ * Typically, Hive, when it wants to make sure a result decimal fits in the column decimal's
+ * precision/scale it calls enforcePrecisionScale.  That method will scale down or trim off
+ * result fractional digits if necessary with rounding when the column has a smaller scale.
+ * And, it will also indicate overflow when the decimal has exceeded the column's maximum precision.
+ * <p>
+ * NOTE: When Hive gets ready to serialize a decimal into text or binary, it usually sometimes
+ * wants trailing fractional zeroes.  See the special notes for toFormatString and
+ * bigIntegerBytesScaled for details.
+ * <p>
+ * ------------------------------------- Version 2 ------------------------------------------------
+ * <p>
+ * This is the 2nd major version of HiveDecimal called V2.  The previous version has been
+ * renamed to HiveDecimalV1 and is kept as a test and behavior reference.
+ * <p>
+ * For good performance we do not represent the decimal using a BigDecimal object like the previous
+ * version V1 did.  Using Java objects to represent our decimal incurs too high a penalty
+ * for memory allocations and general logic.
+ * <p>
+ * The original V1 public methods and fields are annotated with @HiveDecimalVersionV1; new public
+ * methods and fields are annotated with @HiveDecimalVersionV2.
  *
  */
-public class HiveDecimal implements Comparable<HiveDecimal> {
+public final class HiveDecimal extends FastHiveDecimal implements Comparable<HiveDecimal> {
+
+  /*
+   * IMPLEMENTATION NOTE:
+   *    We implement HiveDecimal with the mutable FastHiveDecimal class.  That class uses
+   *    protected on all its methods so they will not be visible in the HiveDecimal class.
+   *
+   *    So even if one casts to FastHiveDecimal, you shouldn't be able to violate the immutability
+   *    of a HiveDecimal class.
+   */
+
+  @HiveDecimalVersionV1
   public static final int MAX_PRECISION = 38;
+  @HiveDecimalVersionV1
   public static final int MAX_SCALE = 38;
 
   /**
    * Default precision/scale when user doesn't specify in the column metadata, such as
    * decimal and decimal(8).
    */
+  @HiveDecimalVersionV1
   public static final int USER_DEFAULT_PRECISION = 10;
+  @HiveDecimalVersionV1
   public static final int USER_DEFAULT_SCALE = 0;
 
   /**
    *  Default precision/scale when system is not able to determine them, such as in case
    *  of a non-generic udf.
    */
+  @HiveDecimalVersionV1
   public static final int SYSTEM_DEFAULT_PRECISION = 38;
+  @HiveDecimalVersionV1
   public static final int SYSTEM_DEFAULT_SCALE = 18;
 
-  public static final HiveDecimal ZERO = new HiveDecimal(BigDecimal.ZERO);
-  public static final HiveDecimal ONE = new HiveDecimal(BigDecimal.ONE);
+  /**
+   * Common values.
+   */
+  @HiveDecimalVersionV1
+  public static final HiveDecimal ZERO = HiveDecimal.create(0);
+  @HiveDecimalVersionV1
+  public static final HiveDecimal ONE = HiveDecimal.create(1);
 
+  /**
+   * ROUND_FLOOR:
+   * <p>
+   *   Round towards negative infinity.
+   * <p>
+   *   The Hive function is FLOOR.
+   * <p>
+   *   Positive numbers: The round fraction is thrown away.
+   * <p>
+   *       (Example here rounds at scale 0)
+   *       Value        FLOOR
+   *        0.3           0
+   *        2             2
+   *        2.1           2
+   * <p>
+   *   Negative numbers: If there is a round fraction, throw it away and subtract 1.
+   * <p>
+   *       (Example here rounds at scale 0)
+   *       Value        FLOOR
+   *       -0.3           -1
+   *       -2             -2
+   *       -2.1           -3
+   */
+  @HiveDecimalVersionV1
   public static final int ROUND_FLOOR = BigDecimal.ROUND_FLOOR;
+
+  /**
+   * ROUND_CEILING:
+   * <p>
+   *   Round towards positive infinity.
+   * <p>
+   *   The Hive function is CEILING.
+   * <p>
+   *   Positive numbers: If there is a round fraction, throw it away and add 1
+   * <p>
+   *       (Example here rounds at scale 0)
+   *       Value        CEILING
+   *        0.3           1
+   *        2             2
+   *        2.1           3
+   * <p>
+   *   Negative numbers: The round fraction is thrown away.
+   * <p>
+   *       (Example here rounds at scale 0)
+   *       Value        CEILING
+   *       -0.3           0
+   *       -2             -2
+   *       -2.1           -2
+   */
+  @HiveDecimalVersionV1
   public static final int ROUND_CEILING = BigDecimal.ROUND_CEILING;
+
+  /**
+   * ROUND_HALF_UP:
+   * <p>
+   *   Round towards "nearest neighbor" unless both neighbors are equidistant then round up.
+   * <p>
+   *   The Hive function is ROUND.
+   * <p>
+   *   For result, throw away round fraction.  If the round fraction is >= 0.5, then add 1 when
+   *   positive and subtract 1 when negative.  So, the sign is irrelevant.
+   * <p>
+   *      (Example here rounds at scale 0)
+   *       Value        ROUND                  Value        ROUND
+   *       0.3           0                     -0.3           0
+   *       2             2                     -2            -2
+   *       2.1           2                     -2.1          -2
+   *       2.49          2                     -2.49         -2
+   *       2.5           3                     -2.5          -3
+   *
+   */
+  @HiveDecimalVersionV1
   public static final int ROUND_HALF_UP = BigDecimal.ROUND_HALF_UP;
+
+  /**
+   * ROUND_HALF_EVEN:
+   *   Round towards the "nearest neighbor" unless both neighbors are equidistant, then round
+   *   towards the even neighbor.
+   * <p>
+   *   The Hive function is BROUND.
+   * <p>
+   *   Known as Banker\u2019s Rounding.
+   * <p>
+   *   When you add values rounded with ROUND_HALF_UP you have a bias that grows as you add more
+   *   numbers.  Banker's Rounding is a way to minimize that bias.  It rounds toward the nearest
+   *   even number when the fraction is 0.5 exactly.  In table below, notice that 2.5 goes DOWN to
+   *   2 (even) but 3.5 goes UP to 4 (even), etc.
+   * <p>
+   *   So, the sign is irrelevant.
+   * <p>
+   *       (Example here rounds at scale 0)
+   *       Value        BROUND                  Value        BROUND
+   *        0.49          0                     -0.49          0
+   *        0.5           0                     -0.5           0
+   *        0.51          1                     -0.51         -1
+   *        1.5           2                     -1.5          -2
+   *        2.5           2                     -2.5          -2
+   *        2.51          3                     -2.51         -3
+   *        3.5           4                     -3.5          -4
+   *        4.5           4                     -4.5          -4
+   *        4.51          5                     -4.51         -5
+   *
+   */
+  @HiveDecimalVersionV1
   public static final int ROUND_HALF_EVEN = BigDecimal.ROUND_HALF_EVEN;
 
-  private BigDecimal bd = BigDecimal.ZERO;
+  //-----------------------------------------------------------------------------------------------
+  // Constructors are marked private; use create methods.
+  //-----------------------------------------------------------------------------------------------
+
+  private HiveDecimal() {
+    super();
+  }
+
+  private HiveDecimal(HiveDecimal dec) {
+    super(dec);
+  }
+
+  private HiveDecimal(FastHiveDecimal fastDec) {
+    super(fastDec);
+  }
+
+  private HiveDecimal(int fastSignum, FastHiveDecimal fastDec) {
+    super(fastSignum, fastDec);
+  }
+
+  private HiveDecimal(
+      int fastSignum, long fast0, long fast1, long fast2,
+      int fastIntegerDigitCount, int fastScale) {
+    super(fastSignum, fast0, fast1, fast2, fastIntegerDigitCount, fastScale);
+  }
+
+  //-----------------------------------------------------------------------------------------------
+  // Create methods.
+  //-----------------------------------------------------------------------------------------------
+
+  /**
+   * Create a HiveDecimal from a FastHiveDecimal object. Used by HiveDecimalWritable.
+   */
+  @HiveDecimalVersionV2
+  public static HiveDecimal createFromFast(FastHiveDecimal fastDec) {
+    return new HiveDecimal(fastDec);
+  }
+
+  /**
+   * Create a HiveDecimal from BigDecimal object.
+   *
+   * A BigDecimal object has a decimal scale.
+   *
+   * We will have overflow if BigDecimal's integer part exceed MAX_PRECISION digits or
+   * 99,999,999,999,999,999,999,999,999,999,999,999,999 or 10^38 - 1.
+   *
+   * When the BigDecimal value's precision exceeds MAX_PRECISION and there are fractional digits
+   * because of scale > 0, then lower digits are trimmed off with rounding to meet the
+   * MAX_PRECISION requirement.
+   *
+   * Also, BigDecimal supports negative scale -- which means multiplying the value by 10^abs(scale).
+   * And, BigDecimal allows for a non-zero scale for zero.  We normalize that so zero always has
+   * scale 0.
+   *
+   * @param bigDecimal
+   * @return  The HiveDecimal with the BigDecimal's value adjusted down to a maximum precision.
+   *          Otherwise, null is returned for overflow.
+   */
+  @HiveDecimalVersionV1
+  public static HiveDecimal create(BigDecimal bigDecimal) {
+    return create(bigDecimal, true);
+  }
+
+  /**
+   * Same as the above create method, except fractional digit rounding can be turned off.
+   * @param bigDecimal
+   * @param allowRounding  True requires all of the bigDecimal value be converted to the decimal
+   *                       without loss of precision.
+   * @return
+   */
+  @HiveDecimalVersionV1
+  public static HiveDecimal create(BigDecimal bigDecimal, boolean allowRounding) {
+    HiveDecimal result = new HiveDecimal();
+    if (!result.fastSetFromBigDecimal(
+        bigDecimal, allowRounding)) {
+      return null;
+    }
+    return result;
+  }
+
+  /**
+   * Creates a HiveDecimal from a BigInteger's value with a scale of 0.
+   *
+   * We will have overflow if BigInteger exceed MAX_PRECISION digits or
+   * 99,999,999,999,999,999,999,999,999,999,999,999,999 or 10^38 - 1.
+   *
+   * @param bigInteger
+   * @return  A HiveDecimal object with the exact BigInteger's value.
+   *          Otherwise, null is returned on overflow.
+   */
+  @HiveDecimalVersionV1
+  public static HiveDecimal create(BigInteger bigInteger) {
+    HiveDecimal result = new HiveDecimal();
+    if (!result.fastSetFromBigInteger(
+        bigInteger)) {
+      return null;
+    }
+    return result;
+  }
+
+  /**
+   * Creates a HiveDecimal from a BigInteger's value with a specified scale.
+   *
+   * We will have overflow if BigInteger exceed MAX_PRECISION digits or
+   * 99,999,999,999,999,999,999,999,999,999,999,999,999 or 10^38 - 1.
+   *
+   * The resulting decimal will have fractional digits when the specified scale is greater than 0.
+   *
+   * When the BigInteger's value's precision exceeds MAX_PRECISION and there are fractional digits
+   * because of scale > 0, then lower digits are trimmed off with rounding to meet the
+   * MAX_PRECISION requirement.
+   *
+   * @param bigInteger
+   * @param scale
+   * @return  A HiveDecimal object with the BigInteger's value adjusted for scale.
+   *          Otherwise, null is returned on overflow.
+   */
+  @HiveDecimalVersionV1
+  public static HiveDecimal create(BigInteger bigInteger, int scale) {
+    HiveDecimal result = new HiveDecimal();
+    if (!result.fastSetFromBigIntegerAndScale(
+        bigInteger, scale)) {
+      return null;
+    }
+    return result;
+  }
+
+  /**
+   * Create a HiveDecimal by parsing a whole string.
+   *
+   * We support parsing a decimal with an exponent because the previous version
+   * (i.e. OldHiveDecimal) uses the BigDecimal parser and was able to.
+   *
+   */
+  @HiveDecimalVersionV1
+  public static HiveDecimal create(String string) {
+    HiveDecimal result = new HiveDecimal();
+    if (!result.fastSetFromString(
+        string, true)) {
+      return null;
+    }
+    return result;
+  }
+
+  /**
+   * Same as the method above, except blanks before and after are tolerated.
+   * @param string
+   * @param trimBlanks  True specifies leading and trailing blanks are to be ignored.
+   * @return
+   */
+  @HiveDecimalVersionV2
+  public static HiveDecimal create(String string, boolean trimBlanks) {
+    HiveDecimal result = new HiveDecimal();
+    if (!result.fastSetFromString(
+        string, trimBlanks)) {
+      return null;
+    }
+    return result;
+  }
+
+  /**
+   * Create a HiveDecimal by parsing the characters in a whole byte array.
+   *
+   * Same rules as create(String string) above.
+   *
+   */
+  @HiveDecimalVersionV2
+  public static HiveDecimal create(byte[] bytes) {
+    HiveDecimal result = new HiveDecimal();
+    if (!result.fastSetFromBytes(
+        bytes, 0, bytes.length, false)) {
+      return null;
+    }
+    return result;
+  }
+
+  /**
+   * Same as the method above, except blanks before and after are tolerated.
+   *
+   */
+  @HiveDecimalVersionV2
+  public static HiveDecimal create(byte[] bytes, boolean trimBlanks) {
+    HiveDecimal result = new HiveDecimal();
+    if (!result.fastSetFromBytes(
+        bytes, 0, bytes.length, trimBlanks)) {
+      return null;
+    }
+    return result;
+  }
+
+  /**
+   * This method takes in digits only UTF-8 characters, a sign flag, and a scale and returns
+   * a decimal.
+   */
+  @HiveDecimalVersionV2
+  public static HiveDecimal create(boolean isNegative, byte[] bytes, int scale) {
+    HiveDecimal result = new HiveDecimal();
+    if (!result.fastSetFromDigitsOnlyBytesAndScale(
+        isNegative, bytes, 0, bytes.length, scale)) {
+      return null;
+    }
+    if (isNegative) {
+      result.fastNegate();
+    }
+    return result;
+  }
+
+  @HiveDecimalVersionV2
+  public static HiveDecimal create(
+      boolean isNegative, byte[] bytes, int offset, int length, int scale) {
+    HiveDecimal result = new HiveDecimal();
+    if (!result.fastSetFromDigitsOnlyBytesAndScale(
+        isNegative, bytes, offset, length, scale)) {
+      return null;
+    }
+    return result;
+  }
 
-  private HiveDecimal(BigDecimal bd) {
-    this.bd = bd;
+  /**
+   * Create a HiveDecimal by parsing the characters in a slice of a byte array.
+   *
+   * Same rules as create(String string) above.
+   *
+   */
+  @HiveDecimalVersionV2
+  public static HiveDecimal create(byte[] bytes, int offset, int length) {
+    HiveDecimal result = new HiveDecimal();
+    if (!result.fastSetFromBytes(
+        bytes, offset, length, false)) {
+      return null;
+    }
+    return result;
   }
 
-  public static HiveDecimal create(BigDecimal b) {
-    return create(b, true);
+  /**
+   * Same as the method above, except blanks before and after are tolerated.
+   *
+   */
+  @HiveDecimalVersionV2
+  public static HiveDecimal create(
+      byte[] bytes, int offset, int length, boolean trimBlanks) {
+    HiveDecimal result = new HiveDecimal();
+    if (!result.fastSetFromBytes(
+        bytes, offset, length, trimBlanks)) {
+      return null;
+    }
+    return result;
   }
 
-  public static HiveDecimal create(BigDecimal b, boolean allowRounding) {
-    BigDecimal bd = normalize(b, allowRounding);
-    return bd == null ? null : new HiveDecimal(bd);
+  /**
+   * Create a HiveDecimal object from an int.
+   *
+   */
+  @HiveDecimalVersionV1
+  public static HiveDecimal create(int intValue) {
+    HiveDecimal result = new HiveDecimal();
+    result.fastSetFromInt(intValue);
+    return result;
   }
 
-  public static HiveDecimal create(BigInteger unscaled, int scale) {
-    BigDecimal bd = normalize(new BigDecimal(unscaled, scale), true);
-    return bd == null ? null : new HiveDecimal(bd);
+  /**
+   * Create a HiveDecimal object from a long.
+   *
+   */
+  @HiveDecimalVersionV1
+  public static HiveDecimal create(long longValue) {
+    HiveDecimal result = new HiveDecimal();
+    result.fastSetFromLong(longValue);
+    return result;
   }
 
-  public static HiveDecimal create(String dec) {
-    BigDecimal bd;
-    try {
-      bd = new BigDecimal(dec.trim());
-    } catch (NumberFormatException ex) {
+  /**
+   * Create a HiveDecimal object from a long with a specified scale.
+   *
+   */
+  @HiveDecimalVersionV2
+  public static HiveDecimal create(long longValue, int scale) {
+    HiveDecimal result = new HiveDecimal();
+    if (!result.fastSetFromLongAndScale(
+        longValue, scale)) {
       return null;
     }
-    bd = normalize(bd, true);
-    return bd == null ? null : new HiveDecimal(bd);
+    return result;
   }
 
-  public static HiveDecimal create(BigInteger bi) {
-    BigDecimal bd = normalize(new BigDecimal(bi), true);
-    return bd == null ? null : new HiveDecimal(bd);
+  /**
+   * Create a HiveDecimal object from a float.
+   * <p>
+   * This method is equivalent to HiveDecimal.create(Float.toString(floatValue))
+   */
+  @HiveDecimalVersionV2
+  public static HiveDecimal create(float floatValue) {
+    HiveDecimal result = new HiveDecimal();
+    if (!result.fastSetFromFloat(floatValue)) {
+      return null;
+    }
+    return result;
   }
 
-  public static HiveDecimal create(int i) {
-    return new HiveDecimal(new BigDecimal(i));
+  /**
+   * Create a HiveDecimal object from a double.
+   * <p>
+   * This method is equivalent to HiveDecimal.create(Double.toString(doubleValue))
+   */
+  @HiveDecimalVersionV2
+  public static HiveDecimal create(double doubleValue) {
+    HiveDecimal result = new HiveDecimal();
+    if (!result.fastSetFromDouble(doubleValue)) {
+      return null;
+    }
+    return result;
   }
 
-  public static HiveDecimal create(long l) {
-    return new HiveDecimal(new BigDecimal(l));
+  //-----------------------------------------------------------------------------------------------
+  // Serialization methods.
+  //-----------------------------------------------------------------------------------------------
+
+  // The byte length of the scratch byte array that needs to be passed to serializationUtilsRead.
+  @HiveDecimalVersionV2
+  public static final int SCRATCH_BUFFER_LEN_SERIALIZATION_UTILS_READ =
+      FAST_SCRATCH_BUFFER_LEN_SERIALIZATION_UTILS_READ;
+
+  /**
+   * Deserialize data written in the format used by the SerializationUtils methods
+   * readBigInteger/writeBigInteger and create a decimal using the supplied scale.
+   * <p>
+   * ORC uses those SerializationUtils methods for its serialization.
+   * <p>
+   * A scratch bytes array is necessary to do the binary to decimal conversion for better
+   * performance.  Pass a SCRATCH_BUFFER_LEN_SERIALIZATION_UTILS_READ byte array for scratchBytes.
+   * <p>
+   * @return The deserialized decimal or null if the conversion failed.
+   */
+  @HiveDecimalVersionV2
+  public static HiveDecimal serializationUtilsRead(
+      InputStream inputStream, int scale,
+      byte[] scratchBytes)
+      throws IOException {
+    HiveDecimal result = new HiveDecimal();
+    if (!result.fastSerializationUtilsRead(
+        inputStream, scale,
+        scratchBytes)) {
+      return null;
+    }
+    return result;
   }
 
+  /**
+   * Convert bytes in the format used by BigInteger's toByteArray format (and accepted by its
+   * constructor) into a decimal using the specified scale.
+   * <p>
+   * Our bigIntegerBytes methods create bytes in this format, too.
+   * <p>
+   * This method is designed for high performance and does not create an actual BigInteger during
+   * binary to decimal conversion.
+   *
+   */
+  @HiveDecimalVersionV2
+  public static HiveDecimal createFromBigIntegerBytesAndScale(
+      byte[] bytes, int scale) {
+    HiveDecimal result = new HiveDecimal();
+    if (!result.fastSetFromBigIntegerBytesAndScale(
+        bytes, 0, bytes.length, scale)) {
+      return null;
+    }
+    return result;
+  }
+
+  @HiveDecimalVersionV2
+  public static HiveDecimal createFromBigIntegerBytesAndScale(
+      byte[] bytes, int offset, int length, int scale) {
+    HiveDecimal result = new HiveDecimal();
+    if (!result.fastSetFromBigIntegerBytesAndScale(
+        bytes, offset, length, scale)) {
+      return null;
+    }
+    return result;
+  }
+
+  // The length of the long array that needs to be passed to serializationUtilsWrite.
+  @HiveDecimalVersionV2
+  public static final int SCRATCH_LONGS_LEN = FAST_SCRATCH_LONGS_LEN;
+
+  /**
+   * Serialize this decimal's BigInteger equivalent unscaled value using the format that the
+   * SerializationUtils methods readBigInteger/writeBigInteger use.
+   * <p>
+   * ORC uses those SerializationUtils methods for its serialization.
+   * <p>
+   * Scratch objects necessary to do the decimal to binary conversion without actually creating a
+   * BigInteger object are passed for better performance.
+   * <p>
+   * Allocate scratchLongs with SCRATCH_LONGS_LEN longs.
+   *
+   */
+  @HiveDecimalVersionV2
+  public boolean serializationUtilsWrite(
+      OutputStream outputStream,
+      long[] scratchLongs)
+          throws IOException {
+    return
+        fastSerializationUtilsWrite(
+            outputStream,
+            scratchLongs);
+  }
+
+  // The length of the scratch byte array that needs to be passed to bigIntegerBytes, etc.
+  @HiveDecimalVersionV2
+  public static final int SCRATCH_BUFFER_LEN_BIG_INTEGER_BYTES =
+      FAST_SCRATCH_BUFFER_LEN_BIG_INTEGER_BYTES;
+
+  /**
+   * Return binary representation of this decimal's BigInteger equivalent unscaled value using
+   * the format that the BigInteger's toByteArray method returns (and the BigInteger constructor
+   * accepts).
+   * <p>
+   * Used by LazyBinary, Avro, and Parquet serialization.
+   * <p>
+   * Scratch objects necessary to do the decimal to binary conversion without actually creating a
+   * BigInteger object are passed for better performance.
+   * <p>
+   * Allocate scratchLongs with SCRATCH_LONGS_LEN longs.
+   * And, allocate buffer with SCRATCH_BUFFER_LEN_BIG_INTEGER_BYTES bytes.
+   * <p>
+   * @param scratchLongs
+   * @param buffer
+   * @return The number of bytes used for the binary result in buffer.  Otherwise, 0 if the
+   *         conversion failed.
+   */
+  @HiveDecimalVersionV2
+  public int bigIntegerBytes(
+      long[] scratchLongs, byte[] buffer) {
+    return
+        fastBigIntegerBytes(
+            scratchLongs, buffer);
+  }
+
+  @HiveDecimalVersionV2
+  public byte[] bigIntegerBytes() {
+    long[] scratchLongs = new long[SCRATCH_LONGS_LEN];
+    byte[] buffer = new byte[SCRATCH_BUFFER_LEN_BIG_INTEGER_BYTES];
+    final int byteLength =
+        fastBigIntegerBytes(
+            scratchLongs, buffer);
+    return Arrays.copyOfRange(buffer, 0, byteLength);
+  }
+
+  /**
+   * Convert decimal to BigInteger binary bytes with a serialize scale, similar to the formatScale
+   * for toFormatString.  It adds trailing zeroes the (emulated) BigInteger toByteArray result
+   * when a serializeScale is greater than current scale.  Or, rounds if scale is less than
+   * current scale.
+   * <p>
+   * Used by Avro and Parquet serialization.
+   * <p>
+   * This emulates the OldHiveDecimal setScale AND THEN OldHiveDecimal getInternalStorage() behavior.
+   *
+   */
+  @HiveDecimalVersionV2
+  public int bigIntegerBytesScaled(
+      int serializeScale,
+      long[] scratchLongs, byte[] buffer) {
+    return
+        fastBigIntegerBytesScaled(
+            serializeScale,
+            scratchLongs, buffer);
+  }
+
+  @HiveDecimalVersionV2
+  public byte[] bigIntegerBytesScaled(int serializeScale) {
+    long[] scratchLongs = new long[SCRATCH_LONGS_LEN];
+    byte[] buffer = new byte[SCRATCH_BUFFER_LEN_BIG_INTEGER_BYTES];
+    int byteLength =
+        fastBigIntegerBytesScaled(
+            serializeScale,
+            scratchLongs, buffer);
+    return Arrays.copyOfRange(buffer, 0, byteLength);
+  }
+
+  //-----------------------------------------------------------------------------------------------
+  // Convert to string/UTF-8 ASCII bytes methods.
+  //-----------------------------------------------------------------------------------------------
+
+  /**
+   * Return a string representation of the decimal.
+   * <p>
+   * It is the equivalent of calling bigDecimalValue().toPlainString -- it does not add exponent
+   * notation -- but is much faster.
+   * <p>
+   * NOTE: If setScale(int serializationScale) was used to create the decimal object, then trailing
+   * fractional digits will be added to display to the serializationScale.  Or, the display may
+   * get rounded.  See the comments for that method.
+   *
+   */
+  @HiveDecimalVersionV1
   @Override
   public String toString() {
-     return bd.toPlainString();
+    if (fastSerializationScale() != -1) {
+
+      // Use the serialization scale and format the string with trailing zeroes (or
+      // round the decimal) if necessary.
+      return
+          fastToFormatString(fastSerializationScale());
+    } else {
+      return
+          fastToString();
+    }
+  }
+
+  @HiveDecimalVersionV2
+  public String toString(
+      byte[] scratchBuffer) {
+    if (fastSerializationScale() != -1) {
+
+      // Use the serialization scale and format the string with trailing zeroes (or
+      // round the decimal) if necessary.
+      return
+          fastToFormatString(
+              fastSerializationScale(),
+              scratchBuffer);
+    } else {
+      return
+          fastToString(scratchBuffer);
+    }
   }
-  
+
+  /**
+   * Return a string representation of the decimal using the specified scale.
+   * <p>
+   * This method is designed to ALWAYS SUCCEED (unless the newScale parameter is out of range).
+   * <p>
+   * Is does the equivalent of a setScale(int newScale).  So, more than 38 digits may be returned.
+   * See that method for more details on how this can happen.
+   * <p>
+   * @param scale The number of digits after the decimal point
+   * @return The scaled decimal representation string representation.
+   */
+  @HiveDecimalVersionV1
+  public String toFormatString(int formatScale) {
+    return
+        fastToFormatString(
+            formatScale);
+  }
+
+  @HiveDecimalVersionV2
+  public String toFormatString(int formatScale, byte[] scratchBuffer) {
+    return
+        fastToFormatString(
+            formatScale,
+            scratchBuffer);
+  }
+
+  @HiveDecimalVersionV2
+  public String toDigitsOnlyString() {
+    return
+        fastToDigitsOnlyString();
+  }
+
+  // The length of the scratch buffer that needs to be passed to toBytes, toFormatBytes,
+  // toDigitsOnlyBytes.
+  @HiveDecimalVersionV2
+  public final static int SCRATCH_BUFFER_LEN_TO_BYTES = FAST_SCRATCH_BUFFER_LEN_TO_BYTES;
+
   /**
-   * Return a string representation of the number with the number of decimal digits as
-   * the given scale. Please note that this is different from toString().
-   * @param scale the number of digits after the decimal point
-   * @return the string representation of exact number of decimal digits
+   * Decimal to ASCII bytes conversion.
+   * <p>
+   * The scratch buffer will contain the result afterwards.  It should be
+   * SCRATCH_BUFFER_LEN_TO_BYTES bytes long.
+   * <p>
+   * The result is produced at the end of the scratch buffer, so the return value is the byte
+   * index of the first byte.  The byte slice is [byteIndex:SCRATCH_BUFFER_LEN_TO_BYTES-1].
+   *
    */
-  public String toFormatString(int scale) {
-    return (bd.scale() == scale ? bd :
-      bd.setScale(scale, RoundingMode.HALF_UP)).toPlainString();
+  @HiveDecimalVersionV2
+  public int toBytes(
+      byte[] scratchBuffer) {
+    return
+        fastToBytes(
+            scratchBuffer);
   }
 
-  public HiveDecimal setScale(int i) {
-    return new HiveDecimal(bd.setScale(i, RoundingMode.HALF_UP));
+  /**
+   * This is the serialization version of decimal to string conversion.
+   * <p>
+   * It adds trailing zeroes when the formatScale is greater than the current scale.  Or, it
+   * does round if the formatScale is less than the current scale.
+   * <p>
+   * Note that you can get more than 38 (MAX_PRECISION) digits in the output with this method.
+   *
+   */
+  @HiveDecimalVersionV2
+  public int toFormatBytes(
+      int formatScale,
+      byte[] scratchBuffer) {
+    return
+        fastToFormatBytes(
+            formatScale,
+            scratchBuffer);
+  }
+
+  /**
+   * Convert decimal to just the digits -- no dot.
+   * <p>
+   * Currently used by BinarySortable serialization.
+   * <p>
+   * A faster way to get just the digits than calling unscaledValue.toString().getBytes().
+   *
+   */
+  @HiveDecimalVersionV2
+  public int toDigitsOnlyBytes(
+      byte[] scratchBuffer) {
+    return
+        fastToDigitsOnlyBytes(
+            scratchBuffer);
   }
 
+  //-----------------------------------------------------------------------------------------------
+  // Comparison methods.
+  //-----------------------------------------------------------------------------------------------
+
+  @HiveDecimalVersionV1
   @Override
   public int compareTo(HiveDecimal dec) {
-    return bd.compareTo(dec.bd);
+    return fastCompareTo(dec);
+  }
+
+  /**
+   * Hash code based on (new) decimal representation.
+   * <p>
+   * Faster than hashCode().
+   * <p>
+   * Used by map join and other Hive internal purposes where performance is important.
+   * <p>
+   * IMPORTANT: See comments for hashCode(), too.
+   */
+  @HiveDecimalVersionV2
+  public int newFasterHashCode() {
+    return fastNewFasterHashCode();
   }
 
+  /**
+   * This is returns original hash code as returned by HiveDecimalV1.
+   * <p>
+   * We need this when the HiveDecimalV1 hash code has been exposed and and written or affected
+   * how data is written.
+   * <p>
+   * This method supports compatibility.
+   * <p>
+   * Examples: bucketing, Hive hash() function, and Hive statistics.
+   * <p>
+   * NOTE: It is necessary to create a BigDecimal object and use its hash code, so this method is
+   *       slow.
+   */
+  @HiveDecimalVersionV1
   @Override
   public int hashCode() {
-    return bd.hashCode();
+    return fastHashCode();
   }
 
+  /**
+   * Are two decimal content (values) equal?
+   * <p>
+   * @obj   The 2nd decimal.
+   * @return  When obj is null or not class HiveDecimal, the return is false.
+   *          Otherwise, returns true when the decimal values are exactly equal.
+   */
+  @HiveDecimalVersionV1
   @Override
   public boolean equals(Object obj) {
     if (obj == null || obj.getClass() != getClass()) {
       return false;
     }
-    return bd.equals(((HiveDecimal) obj).bd);
+    return fastEquals((HiveDecimal) obj);
   }
 
+
+  //-----------------------------------------------------------------------------------------------
+  // Attribute methods.
+  //-----------------------------------------------------------------------------------------------
+
+  /**
+   * Returns the scale of the decimal.  Range 0 .. MAX_SCALE.
+   *
+   */
+  @HiveDecimalVersionV1
   public int scale() {
-    return bd.scale();
+    return fastScale();
+  }
+
+  /**
+   * Returns the number of integer digits in the decimal.
+   * <p>
+   * When the integer portion is zero, this method returns 0.
+   *
+   */
+  @HiveDecimalVersionV2
+  public int integerDigitCount() {
+    return fastIntegerDigitCount();
   }
 
   /**
    * Returns the number of digits (integer and fractional) in the number, which is equivalent
-   * to SQL decimal precision. Note that this is different from BigDecimal.precision(),
-   * which returns the precision of the unscaled value (BigDecimal.valueOf(0.01).precision() = 1,
-   * whereas HiveDecimal.create("0.01").precision() = 2).
-   * If you want the BigDecimal precision, use HiveDecimal.bigDecimalValue().precision()
-   * @return
+   * to SQL decimal precision.
+   * <p>
+   * Note that this method is different from rawPrecision(), which returns the number of digits
+   * ignoring the scale.  Note that rawPrecision returns 0 when the value is 0.
+   *
+   *     Decimal            precision              rawPrecision
+   *        0                    1                         0
+   *        1                    1                         1
+   *       -7                    1                         1
+   *       0.1                   1                         1
+   *       0.04                  2                         1
+   *       0.00380               5                         3
+   *     104.0009                7                         7
+   * <p>
+   * If you just want the actual number of digits, use rawPrecision().
+   *
    */
+  @HiveDecimalVersionV1
   public int precision() {
-    int bdPrecision = bd.precision();
-    int bdScale = bd.scale();
+    return fastSqlPrecision();
+  }
 
-    if (bdPrecision < bdScale) {
-      // This can happen for numbers less than 0.1
-      // For 0.001234: bdPrecision=4, bdScale=6
-      // In this case, we'll set the type to have the same precision as the scale.
-      return bdScale;
-    }
-    return bdPrecision;
+  // See comments for sqlPrecision.
+  @HiveDecimalVersionV2
+  public int rawPrecision() {
+    return fastRawPrecision();
   }
 
-  /** Note - this method will corrupt the value if it doesn't fit. */
-  public int intValue() {
-    return bd.intValue();
+  /**
+   * Get the sign of the decimal.
+   * <p>
+   * @return 0 if the decimal is equal to 0, -1 if less than zero, and 1 if greater than 0
+   */
+  @HiveDecimalVersionV1
+  public int signum() {
+    return fastSignum();
   }
 
-  public double doubleValue() {
-    return bd.doubleValue();
+  //-----------------------------------------------------------------------------------------------
+  // Value conversion methods.
+  //-----------------------------------------------------------------------------------------------
+
+  /**
+   * Is the decimal value a byte? Range -128            to      127.
+   *                                    Byte.MIN_VALUE          Byte.MAX_VALUE
+   * <p>
+   * Emulates testing for no value corruption:
+   *      bigDecimalValue().setScale(0).equals(BigDecimal.valueOf(bigDecimalValue().byteValue()))
+   * <p>
+   * NOTE: Fractional digits are ignored in the test since byteValue() will
+   *       remove them (round down).
+   * <p>
+   * @return True when byteValue() will return a correct byte.
+   */
+  @HiveDecimalVersionV2
+  public boolean isByte() {
+    return fastIsByte();
   }
 
-  /** Note - this method will corrupt the value if it doesn't fit. */
-  public long longValue() {
-    return bd.longValue();
+  /**
+   * A byte variation of longValue()
+   * <p>
+   * This method will return a corrupted value unless isByte() is true.
+   */
+  @HiveDecimalVersionV1
+  public byte byteValue() {
+    return fastByteValueClip();
+  }
+
+  /**
+   * Is the decimal value a short? Range -32,768         to     32,767.
+   *                                     Short.MIN_VALUE        Short.MAX_VALUE
+   * <p>
+   * Emulates testing for no value corruption:
+   *      bigDecimalValue().setScale(0).equals(BigDecimal.valueOf(bigDecimalValue().shortValue()))
+   * <p>
+   * NOTE: Fractional digits are ignored in the test since shortValue() will
+   *       remove them (round down).
+   * <p>
+   * @return True when shortValue() will return a correct short.
+   */
+  @HiveDecimalVersionV2
+  public boolean isShort() {
+    return fastIsShort();
   }
 
-  /** Note - this method will corrupt the value if it doesn't fit. */
+  /**
+   * A short variation of longValue().
+   * <p>
+   * This method will return a corrupted value unless isShort() is true.
+   */
+  @HiveDecimalVersionV1
   public short shortValue() {
-    return bd.shortValue();
+    return fastShortValueClip();
   }
 
-  public float floatValue() {
-    return bd.floatValue();
+  /**
+   * Is the decimal value a int? Range -2,147,483,648     to   2,147,483,647.
+   *                                   Integer.MIN_VALUE       Integer.MAX_VALUE
+   * <p>
+   * Emulates testing for no value corruption:
+   *      bigDecimalValue().setScale(0).equals(BigDecimal.valueOf(bigDecimalValue().intValue()))
+   * <p>
+   * NOTE: Fractional digits are ignored in the test since intValue() will
+   *       remove them (round down).
+   * <p>
+   * @return True when intValue() will return a correct int.
+   */
+  @HiveDecimalVersionV2
+  public boolean isInt() {
+    return fastIsInt();
   }
 
-  public BigDecimal bigDecimalValue() {
-    return bd;
+  /**
+   * An int variation of longValue().
+   * <p>
+   * This method will return a corrupted value unless isInt() is true.
+   */
+  @HiveDecimalVersionV1
+  public int intValue() {
+    return fastIntValueClip();
   }
 
-  public byte byteValue() {
-    return bd.byteValue();
+  /**
+   * Is the decimal value a long? Range -9,223,372,036,854,775,808 to 9,223,372,036,854,775,807.
+   *                                    Long.MIN_VALUE                Long.MAX_VALUE
+   * <p>
+   * Emulates testing for no value corruption:
+   *      bigDecimalValue().setScale(0).equals(BigDecimal.valueOf(bigDecimalValue().longValue()))
+   * <p>
+   * NOTE: Fractional digits are ignored in the test since longValue() will
+   *       remove them (round down).
+   * <p>
+   * @return True when longValue() will return a correct long.
+   */
+  @HiveDecimalVersionV2
+  public boolean isLong() {
+    return fastIsLong();
   }
 
-  public HiveDecimal setScale(int adjustedScale, int rm) {
-    return create(bd.setScale(adjustedScale, rm));
+  /**
+   * Return the long value of a decimal.
+   * <p>
+   * This method will return a corrupted value unless isLong() is true.
+   */
+  @HiveDecimalVersionV1
+  public long longValue() {
+    return fastLongValueClip();
   }
 
-  public HiveDecimal subtract(HiveDecimal dec) {
-    return create(bd.subtract(dec.bd));
+  @HiveDecimalVersionV1
+  public long longValueExact() {
+    if (!isLong()) {
+      throw new ArithmeticException();
+    }
+    return fastLongValueClip();
   }
 
-  public HiveDecimal multiply(HiveDecimal dec) {
-    return create(bd.multiply(dec.bd), false);
+  /**
+   * Return a float representing the decimal.  Due the limitations of float, some values will not
+   * be accurate.
+   *
+   */
+  @HiveDecimalVersionV1
+  public float floatValue() {
+    return fastFloatValue();
+   }
+
+  /**
+   * Return a double representing the decimal.  Due the limitations of double, some values will not
+   * be accurate.
+   *
+   */
+  @HiveDecimalVersionV1
+  public double doubleValue() {
+    return fastDoubleValue();
   }
 
-  public BigInteger unscaledValue() {
-    return bd.unscaledValue();
+  /**
+   * Return a BigDecimal representing the decimal.  The BigDecimal class is able to accurately
+   * represent the decimal.
+   *
+   * NOTE: We are not representing our decimal as BigDecimal now as OldHiveDecimal did, so this
+   * is now slower.
+   *
+   */
+  @HiveDecimalVersionV1
+  public BigDecimal bigDecimalValue() {
+    return fastBigDecimalValue();
   }
 
-  public HiveDecimal scaleByPowerOfTen(int n) {
-    return create(bd.scaleByPowerOfTen(n));
+  /**
+   * Get a BigInteger representing the decimal's digits without a dot.
+   * <p>
+   * @return Returns a signed BigInteger.
+   */
+  @HiveDecimalVersionV1
+  public BigInteger unscaledValue() {
+    return fastBigIntegerValue();
   }
 
-  public HiveDecimal abs() {
-    return create(bd.abs());
+  /**
+   * Return a decimal with only the fractional digits.
+   * <p>
+   * Zero is returned when there are no fractional digits (i.e. scale is 0).
+   *
+   */
+  @HiveDecimalVersionV2
+  public HiveDecimal fractionPortion() {
+    HiveDecimal result = new HiveDecimal();
+    result.fastFractionPortion();
+    return result;
   }
 
-  public HiveDecimal negate() {
-    return create(bd.negate());
+  /**
+   * Return a decimal with only the integer digits.
+   * <p>
+   * Any fractional digits are removed.  E.g. 2.083 scale 3 returns as 2 scale 0.
+   *
+   */
+  @HiveDecimalVersionV2
+  public HiveDecimal integerPortion() {
+    HiveDecimal result = new HiveDecimal();
+    result.fastIntegerPortion();
+    return result;
   }
 
+  //-----------------------------------------------------------------------------------------------
+  // Math methods.
+  //-----------------------------------------------------------------------------------------------
+
+  /**
+   * Add the current decimal and another decimal and return the result.
+   *
+   */
+  @HiveDecimalVersionV1
   public HiveDecimal add(HiveDecimal dec) {
-    return create(bd.add(dec.bd));
+    HiveDecimal result = new HiveDecimal();
+    if (!fastAdd(
+        dec,
+        result)) {
+      return null;
+    }
+    return result;
   }
 
-  public HiveDecimal pow(int n) {
-    BigDecimal result = normalize(bd.pow(n), false);
-    return result == null ? null : new HiveDecimal(result);
+  /**
+   * Subtract from the current decimal another decimal and return the result.
+   *
+   */
+  @HiveDecimalVersionV1
+  public HiveDecimal subtract(HiveDecimal dec) {
+    HiveDecimal result = new HiveDecimal();
+    if (!fastSubtract(
+        dec,
+        result)) {
+      return null;
+    }
+    return result;
   }
 
-  public HiveDecimal remainder(HiveDecimal dec) {
-    return create(bd.remainder(dec.bd));
+  /**
+   * Multiply two decimals.
+   * <p>
+   * NOTE: Overflow Determination for Multiply
+   * <p>
+   *   OldDecimal.multiply performs the multiply with BigDecimal but DOES NOT ALLOW ROUNDING
+   *   (i.e. no throwing away lower fractional digits).
+   * <p>
+   *   CONSIDER: Allowing rounding.  This would eliminate cases today where we return null for
+   *             the multiplication result.
+   * <p>
+   * IMPLEMENTATION NOTE: HiveDecimalV1 code does this:
+   * <p>
+   * return create(bd.multiply(dec.bd), false);
+   */
+  @HiveDecimalVersionV1
+  public HiveDecimal multiply(HiveDecimal dec) {
+    HiveDecimal result = new HiveDecimal();
+    if (!fastMultiply(
+        dec,
+        result)) {
+      return null;
+    }
+    return result;
   }
 
-  public HiveDecimal divide(HiveDecimal dec) {
-    return create(bd.divide(dec.bd, MAX_SCALE, RoundingMode.HALF_UP), true);
+  /**
+   * Multiplies a decimal by a power of 10.
+   * <p>
+   * The decimal 19350 scale 0 will return 193.5 scale 1 when power is -2 (negative).
+   * <p>
+   * The decimal 1.000923 scale 6 will return 10009.23 scale 2 when power is 4 (positive).
+   * <p>
+   * @param power
+   * @return Returns a HiveDecimal whose value is value * 10^power.
+   */
+  @HiveDecimalVersionV1
+  public HiveDecimal scaleByPowerOfTen(int power) {
+    if (power == 0 || fastSignum() == 0) {
+      // No change for multiply by 10^0 or value 0.
+      return this;
+    }
+    HiveDecimal result = new HiveDecimal();
+    if (!fastScaleByPowerOfTen(
+        power,
+        result)) {
+      return null;
+    }
+    return result;
   }
 
   /**
-   * Get the sign of the underlying decimal.
-   * @return 0 if the decimal is equal to 0, -1 if less than zero, and 1 if greater than 0
+   * Take the absolute value of a decimal.
+   * <p>
+   * @return When the decimal is negative, returns a new HiveDecimal with the positive value.
+   *         Otherwise, returns the current 0 or positive value object;
    */
-  public int signum() {
-    return bd.signum();
+  @HiveDecimalVersionV1
+  public HiveDecimal abs() {
+    if (fastSignum() != -1) {
+      return this;
+    }
+    HiveDecimal result = new HiveDecimal(this);
+    result.fastAbs();
+    return result;
   }
 
-  private static BigDecimal trim(BigDecimal d) {
-    if (d.compareTo(BigDecimal.ZERO) == 0) {
-      // Special case for 0, because java doesn't strip zeros correctly on that number.
-      d = BigDecimal.ZERO;
-    } else {
-      d = d.stripTrailingZeros();
-      if (d.scale() < 0) {
-        // no negative scale decimals
-        d = d.setScale(0);
-      }
+  /**
+   * Reverse the sign of a decimal.
+   * <p>
+   * @return Returns a new decimal with the sign flipped.  When the value is 0, the current
+   * object is returned.
+   */
+  @HiveDecimalVersionV1
+  public HiveDecimal negate() {
+    if (fastSignum() == 0) {
+      return this;
     }
-    return d;
+    HiveDecimal result = new HiveDecimal(this);
+    result.fastNegate();
+    return result;
   }
 
-  private static BigDecimal normalize(BigDecimal bd, boolean allowRounding) {
-    if (bd == null) {
-      return null;
-    }
+  //-----------------------------------------------------------------------------------------------
+  // Rounding / setScale methods.
+  //-----------------------------------------------------------------------------------------------
 
-    bd = trim(bd);
+  /**
+   * DEPRECATED for V2.
+   * <p>
+   * Create a decimal from another decimal whose only change is it is MARKED and will display /
+   * serialize with a specified scale that will add trailing zeroes (or round) if necessary.
+   * <p>
+   * After display / serialization, the MARKED object is typically thrown away.
+   * <p>
+   * A MARKED decimal ONLY affects these 2 methods since these were the only ways setScale was
+   * used in the old code.
+   * <p>
+   *    toString
+   *    unscaleValue
+   * <p>
+   * This method has been deprecated because has poor performance by creating a throw away object.
+   * <p>
+   * For setScale(scale).toString() use toFormatString(scale) instead.
+   * For setScale(scale).unscaledValue().toByteArray() use V2 bigIntegerBytesScaled(scale) instead.
+   * <p>
+   * For better performance, use the V2 form of toFormatString that takes a scratch buffer,
+   * or even better use toFormatBytes.
+   * <p>
+   * And, use the form of bigIntegerBytesScaled that takes scratch objects for better performance.
+   *
+   */
+  @Deprecated
+  @HiveDecimalVersionV1
+  public HiveDecimal setScale(int serializationScale) {
+    HiveDecimal result = new HiveDecimal(this);
+    result.fastSetSerializationScale(serializationScale);
+    return result;
+  }
 
-    int intDigits = bd.precision() - bd.scale();
+  /**
+   * Do decimal rounding and return the result.
+   * <p>
+   * When the roundingPoint is 0 or positive, we round away lower fractional digits if the
+   * roundingPoint is less than current scale.  In this case, we will round the result using the
+   * specified rounding mode.
+   * <p>
+   * When the roundingPoint is negative, the rounding will occur within the integer digits.  Integer
+   * digits below the roundPoint will be cleared.  If the rounding occurred, a one will be added
+   * just above the roundingPoint.  Note this may cause overflow.
+   * <p>
+   * No effect when the roundingPoint equals the current scale.  The current object is returned.
+   * <p>
+   * The name setScale is taken from BigDecimal.setScale -- a better name would have been round.
+   *
+   */
+  @HiveDecimalVersionV1
+  public HiveDecimal setScale(
+      int roundingPoint, int roundingMode) {
+    if (fastScale() == roundingPoint) {
+      // No change.
+      return this;
+    }
 
-    if (intDigits > MAX_PRECISION) {
+    // Even if we are just setting the scale when newScale is greater than the current scale,
+    // we need a new object to obey our immutable behavior.
+    HiveDecimal result = new HiveDecimal();
+    if (!fastRound(
+        roundingPoint, roundingMode,
+        result)) {
       return null;
     }
+    return result;
+  }
 
-    int maxScale = Math.min(MAX_SCALE, Math.min(MAX_PRECISION - intDigits, bd.scale()));
-    if (bd.scale() > maxScale ) {
-      if (allowRounding) {
-        bd = bd.setScale(maxScale, RoundingMode.HALF_UP);
-        // Trimming is again necessary, because rounding may introduce new trailing 0's.
-        bd = trim(bd);
-      } else {
-        bd = null;
-      }
+  /**
+   * Return the result of decimal^exponent
+   * <p>
+   * CONSIDER: Currently, negative exponent is not supported.
+   * CONSIDER: Does anybody use this method?
+   *
+   */
+  @HiveDecimalVersionV1
+  public HiveDecimal pow(int exponent) {
+    HiveDecimal result = new HiveDecimal(this);
+    if (!fastPow(
+        exponent, result)) {
+      return null;
     }
-
-    return bd;
+    return result;
   }
 
-  private static BigDecimal enforcePrecisionScale(BigDecimal bd, int maxPrecision, int maxScale) {
-    if (bd == null) {
+  /**
+   * Divides this decimal by another decimal and returns a new decimal with the result.
+   *
+   */
+  @HiveDecimalVersionV1
+  public HiveDecimal divide(HiveDecimal divisor) {
+    HiveDecimal result = new HiveDecimal();
+    if (!fastDivide(
+        divisor,
+        result)) {
       return null;
     }
+    return result;
+  }
 
-    /**
-     * Specially handling the case that bd=0, and we are converting it to a type where precision=scale,
-     * such as decimal(1, 1).
-     */
-    if (bd.compareTo(BigDecimal.ZERO) == 0 && bd.scale() == 0 && maxPrecision == maxScale) {
-      return bd.setScale(maxScale);
+  /**
+   * Divides this decimal by another decimal and returns a new decimal with the remainder of the
+   * division.
+   * <p>
+   * value is (decimal % divisor)
+   * <p>
+   * The remainder is equivalent to BigDecimal:
+   *    bigDecimalValue().subtract(bigDecimalValue().divideToIntegralValue(divisor).multiply(divisor))
+   *
+   */
+  @HiveDecimalVersionV1
+  public HiveDecimal remainder(HiveDecimal divisor) {
+    HiveDecimal result = new HiveDecimal();
+    if (!fastRemainder(
+        divisor,
+        result)) {
+      return null;
     }
+    return result;
+  }
+
+  //-----------------------------------------------------------------------------------------------
+  // Precision/scale enforcement methods.
+  //-----------------------------------------------------------------------------------------------
 
-    bd = trim(bd);
+  /**
+   * Determine if a decimal fits within a specified maxPrecision and maxScale, and round
+   * off fractional digits if necessary to make the decimal fit.
+   * <p>
+   * The relationship between the enforcement maxPrecision and maxScale is restricted. The
+   * specified maxScale must be less than or equal to the maxPrecision.
+   * <p>
+   * Normally, decimals that result from creation operation, arithmetic operations, etc are
+   * "free range" up to MAX_PRECISION and MAX_SCALE.  Each operation checks if the result decimal
+   * is beyond MAX_PRECISION and MAX_SCALE.  If so the result decimal is rounded off using
+   * ROUND_HALF_UP.  If the round digit is 5 or more, one is added to the lowest remaining digit.
+   * The round digit is the digit just below the round point. Result overflow can occur if a
+   * result decimal's integer portion exceeds MAX_PRECISION.
+   * <p>
+   * This method supports enforcing to a declared Hive DECIMAL's precision/scale.
+   * E.g. DECIMAL(10,4)
+   * <p>
+   * Here are the enforcement/rounding checks of this method:
+   * <p>
+   *   1) Maximum integer digits = maxPrecision - maxScale
+   * <p>
+   *      If the decimal's integer digit count exceeds this, the decimal does not fit (overflow).
+   * <p>
+   *   2) If decimal's scale is greater than maxScale, then excess fractional digits are
+   *      rounded off.  When rounding increases the remaining decimal, it may exceed the
+   *      limits and overflow.
+   * <p>
+   * @param dec
+   * @param maxPrecision
+   * @param maxScale
+   * @return The original decimal if no adjustment is necessary.
+   *         A rounded off decimal if adjustment was necessary.
+   *         Otherwise, null if the decimal doesn't fit within maxPrecision / maxScale or rounding
+   *         caused a result that exceeds the specified limits or MAX_PRECISION integer digits.
+   */
+  @HiveDecimalVersionV1
+  public static HiveDecimal enforcePrecisionScale(
+      HiveDecimal dec, int maxPrecision, int maxScale) {
 
-    if (bd.scale() > maxScale) {
-      bd = bd.setScale(maxScale, RoundingMode.HALF_UP);
+    if (maxPrecision < 1 || maxPrecision > MAX_PRECISION) {
+      throw new IllegalArgumentException(STRING_ENFORCE_PRECISION_OUT_OF_RANGE);
     }
 
-    int maxIntDigits = maxPrecision - maxScale;
-    int intDigits = bd.precision() - bd.scale();
-    if (intDigits > maxIntDigits) {
-      return null;
+    if (maxScale < 0 || maxScale > HiveDecimal.MAX_SCALE) {
+      throw new IllegalArgumentException(STRING_ENFORCE_SCALE_OUT_OF_RANGE);
     }
 
-    return bd;
-  }
+    if (maxPrecision < maxScale) {
+      throw new IllegalArgumentException(STRING_ENFORCE_SCALE_LESS_THAN_EQUAL_PRECISION);
+    }
 
-  public static HiveDecimal enforcePrecisionScale(HiveDecimal dec, int maxPrecision, int maxScale) {
     if (dec == null) {
       return null;
     }
 
-    // Minor optimization, avoiding creating new objects.
-    if (dec.precision() - dec.scale() <= maxPrecision - maxScale &&
-        dec.scale() <= maxScale) {
+    FastCheckPrecisionScaleStatus status =
+        dec.fastCheckPrecisionScale(
+            maxPrecision, maxScale);
+    switch (status) {
+    case NO_CHANGE:
       return dec;
-    }
-
-    BigDecimal bd = enforcePrecisionScale(dec.bd, maxPrecision, maxScale);
-    if (bd == null) {
+    case OVERFLOW:
       return null;
+    case UPDATE_SCALE_DOWN:
+      {
+        HiveDecimal result = new HiveDecimal();
+        if (!dec.fastUpdatePrecisionScale(
+          maxPrecision, maxScale, status,
+          result)) {
+          return null;
+        }
+        return result;
+      }
+    default:
+      throw new RuntimeException("Unknown fast decimal check precision and scale status " + status);
     }
-
-    return HiveDecimal.create(bd);
   }
 
-  public long longValueExact() {
-    return bd.longValueExact();
+  //-----------------------------------------------------------------------------------------------
+  // Validation methods.
+  //-----------------------------------------------------------------------------------------------
+
+  /**
+   * Throws an exception if the current decimal value is invalid.
+   */
+  @HiveDecimalVersionV2
+  public void validate() {
+    if (!fastIsValid()) {
+      fastRaiseInvalidException();
+    }
   }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/storage-api/src/java/org/apache/hadoop/hive/common/type/HiveDecimalV1.java
----------------------------------------------------------------------
diff --git a/storage-api/src/java/org/apache/hadoop/hive/common/type/HiveDecimalV1.java b/storage-api/src/java/org/apache/hadoop/hive/common/type/HiveDecimalV1.java
new file mode 100644
index 0000000..f99ffee
--- /dev/null
+++ b/storage-api/src/java/org/apache/hadoop/hive/common/type/HiveDecimalV1.java
@@ -0,0 +1,386 @@
+/**
+ * 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.common.type;
+
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.math.RoundingMode;
+
+/**
+ *
+ * HiveDecimal. Simple wrapper for BigDecimal. Adds fixed max precision and non scientific string
+ * representation
+ *
+ */
+public final class HiveDecimalV1 implements Comparable<HiveDecimalV1> {
+  @HiveDecimalVersionV1
+  public static final int MAX_PRECISION = 38;
+  @HiveDecimalVersionV1
+  public static final int MAX_SCALE = 38;
+
+  /**
+   * Default precision/scale when user doesn't specify in the column metadata, such as
+   * decimal and decimal(8).
+   */
+  @HiveDecimalVersionV1
+  public static final int USER_DEFAULT_PRECISION = 10;
+  @HiveDecimalVersionV1
+  public static final int USER_DEFAULT_SCALE = 0;
+
+  /**
+   *  Default precision/scale when system is not able to determine them, such as in case
+   *  of a non-generic udf.
+   */
+  @HiveDecimalVersionV1
+  public static final int SYSTEM_DEFAULT_PRECISION = 38;
+  @HiveDecimalVersionV1
+  public static final int SYSTEM_DEFAULT_SCALE = 18;
+
+  @HiveDecimalVersionV1
+  public static final HiveDecimalV1 ZERO = new HiveDecimalV1(BigDecimal.ZERO);
+  @HiveDecimalVersionV1
+  public static final HiveDecimalV1 ONE = new HiveDecimalV1(BigDecimal.ONE);
+
+  @HiveDecimalVersionV1
+  public static final int ROUND_FLOOR = BigDecimal.ROUND_FLOOR;
+  @HiveDecimalVersionV1
+  public static final int ROUND_CEILING = BigDecimal.ROUND_CEILING;
+  @HiveDecimalVersionV1
+  public static final int ROUND_HALF_UP = BigDecimal.ROUND_HALF_UP;
+  @HiveDecimalVersionV1
+  public static final int ROUND_HALF_EVEN = BigDecimal.ROUND_HALF_EVEN;
+
+  private BigDecimal bd = BigDecimal.ZERO;
+
+  private HiveDecimalV1(BigDecimal bd) {
+    this.bd = bd;
+  }
+
+  @HiveDecimalVersionV1
+  public static HiveDecimalV1 create(BigDecimal b) {
+    return create(b, true);
+  }
+
+  @HiveDecimalVersionV1
+  public static HiveDecimalV1 create(BigDecimal b, boolean allowRounding) {
+    BigDecimal bd = normalize(b, allowRounding);
+    return bd == null ? null : new HiveDecimalV1(bd);
+  }
+
+  @HiveDecimalVersionV1
+  public static HiveDecimalV1 create(BigInteger unscaled, int scale) {
+    BigDecimal bd = normalize(new BigDecimal(unscaled, scale), true);
+    return bd == null ? null : new HiveDecimalV1(bd);
+  }
+
+  @HiveDecimalVersionV1
+  public static HiveDecimalV1 create(String dec) {
+    BigDecimal bd;
+    try {
+      bd = new BigDecimal(dec.trim());
+    } catch (NumberFormatException ex) {
+      return null;
+    }
+    bd = normalize(bd, true);
+    return bd == null ? null : new HiveDecimalV1(bd);
+  }
+
+  @HiveDecimalVersionV1
+  public static HiveDecimalV1 create(BigInteger bi) {
+    BigDecimal bd = normalize(new BigDecimal(bi), true);
+    return bd == null ? null : new HiveDecimalV1(bd);
+  }
+
+  @HiveDecimalVersionV1
+  public static HiveDecimalV1 create(int i) {
+    return new HiveDecimalV1(new BigDecimal(i));
+  }
+
+  @HiveDecimalVersionV1
+  public static HiveDecimalV1 create(long l) {
+    return new HiveDecimalV1(new BigDecimal(l));
+  }
+
+  @HiveDecimalVersionV1
+  @Override
+  public String toString() {
+     return bd.toPlainString();
+  }
+  
+  /**
+   * Return a string representation of the number with the number of decimal digits as
+   * the given scale. Please note that this is different from toString().
+   * @param scale the number of digits after the decimal point
+   * @return the string representation of exact number of decimal digits
+   */
+  @HiveDecimalVersionV1
+  public String toFormatString(int scale) {
+    return (bd.scale() == scale ? bd :
+      bd.setScale(scale, RoundingMode.HALF_UP)).toPlainString();
+  }
+
+  @HiveDecimalVersionV1
+  public HiveDecimalV1 setScale(int i) {
+    return new HiveDecimalV1(bd.setScale(i, RoundingMode.HALF_UP));
+  }
+
+  @HiveDecimalVersionV1
+  @Override
+  public int compareTo(HiveDecimalV1 dec) {
+    return bd.compareTo(dec.bd);
+  }
+
+  @HiveDecimalVersionV1
+  @Override
+  public int hashCode() {
+    return bd.hashCode();
+  }
+
+  @HiveDecimalVersionV1
+  @Override
+  public boolean equals(Object obj) {
+    if (obj == null || obj.getClass() != getClass()) {
+      return false;
+    }
+    return bd.equals(((HiveDecimalV1) obj).bd);
+  }
+
+  @HiveDecimalVersionV1
+  public int scale() {
+    return bd.scale();
+  }
+
+  /**
+   * Returns the number of digits (integer and fractional) in the number, which is equivalent
+   * to SQL decimal precision. Note that this is different from BigDecimal.precision(),
+   * which returns the precision of the unscaled value (BigDecimal.valueOf(0.01).precision() = 1,
+   * whereas HiveDecimal.create("0.01").precision() = 2).
+   * If you want the BigDecimal precision, use HiveDecimal.bigDecimalValue().precision()
+   * @return
+   */
+  @HiveDecimalVersionV1
+  public int precision() {
+    int bdPrecision = bd.precision();
+    int bdScale = bd.scale();
+
+    if (bdPrecision < bdScale) {
+      // This can happen for numbers less than 0.1
+      // For 0.001234: bdPrecision=4, bdScale=6
+      // In this case, we'll set the type to have the same precision as the scale.
+      return bdScale;
+    }
+    return bdPrecision;
+  }
+
+  /** Note - this method will corrupt the value if it doesn't fit. */
+  @HiveDecimalVersionV1
+  public int intValue() {
+    return bd.intValue();
+  }
+
+  @HiveDecimalVersionV1
+  public double doubleValue() {
+    return bd.doubleValue();
+  }
+
+  /** Note - this method will corrupt the value if it doesn't fit. */
+  @HiveDecimalVersionV1
+  public long longValue() {
+    return bd.longValue();
+  }
+
+  /** Note - this method will corrupt the value if it doesn't fit. */
+  @HiveDecimalVersionV1
+  public short shortValue() {
+    return bd.shortValue();
+  }
+
+  @HiveDecimalVersionV1
+  public float floatValue() {
+    return bd.floatValue();
+  }
+
+  @HiveDecimalVersionV1
+  public BigDecimal bigDecimalValue() {
+    return bd;
+  }
+
+  @HiveDecimalVersionV1
+  public byte byteValue() {
+    return bd.byteValue();
+  }
+
+  @HiveDecimalVersionV1
+  public HiveDecimalV1 setScale(int adjustedScale, int rm) {
+    return create(bd.setScale(adjustedScale, rm));
+  }
+
+  @HiveDecimalVersionV1
+  public HiveDecimalV1 subtract(HiveDecimalV1 dec) {
+    return create(bd.subtract(dec.bd));
+  }
+
+  @HiveDecimalVersionV1
+  public HiveDecimalV1 multiply(HiveDecimalV1 dec) {
+    return create(bd.multiply(dec.bd), false);
+  }
+
+  @HiveDecimalVersionV1
+  public BigInteger unscaledValue() {
+    return bd.unscaledValue();
+  }
+
+  @HiveDecimalVersionV1
+  public HiveDecimalV1 scaleByPowerOfTen(int n) {
+    return create(bd.scaleByPowerOfTen(n));
+  }
+
+  @HiveDecimalVersionV1
+  public HiveDecimalV1 abs() {
+    return create(bd.abs());
+  }
+
+  @HiveDecimalVersionV1
+  public HiveDecimalV1 negate() {
+    return create(bd.negate());
+  }
+
+  @HiveDecimalVersionV1
+  public HiveDecimalV1 add(HiveDecimalV1 dec) {
+    return create(bd.add(dec.bd));
+  }
+
+  @HiveDecimalVersionV1
+  public HiveDecimalV1 pow(int n) {
+    BigDecimal result = normalize(bd.pow(n), false);
+    return result == null ? null : new HiveDecimalV1(result);
+  }
+
+  @HiveDecimalVersionV1
+  public HiveDecimalV1 remainder(HiveDecimalV1 dec) {
+    return create(bd.remainder(dec.bd));
+  }
+
+  @HiveDecimalVersionV1
+  public HiveDecimalV1 divide(HiveDecimalV1 dec) {
+    return create(bd.divide(dec.bd, MAX_SCALE, RoundingMode.HALF_UP), true);
+  }
+
+  /**
+   * Get the sign of the underlying decimal.
+   * @return 0 if the decimal is equal to 0, -1 if less than zero, and 1 if greater than 0
+   */
+  @HiveDecimalVersionV1
+  public int signum() {
+    return bd.signum();
+  }
+
+  private static BigDecimal trim(BigDecimal d) {
+    if (d.compareTo(BigDecimal.ZERO) == 0) {
+      // Special case for 0, because java doesn't strip zeros correctly on that number.
+      d = BigDecimal.ZERO;
+    } else {
+      d = d.stripTrailingZeros();
+      if (d.scale() < 0) {
+        // no negative scale decimals
+        d = d.setScale(0);
+      }
+    }
+    return d;
+  }
+
+  private static BigDecimal normalize(BigDecimal bd, boolean allowRounding) {
+    if (bd == null) {
+      return null;
+    }
+
+    bd = trim(bd);
+
+    int intDigits = bd.precision() - bd.scale();
+
+    if (intDigits > MAX_PRECISION) {
+      return null;
+    }
+
+    int maxScale = Math.min(MAX_SCALE, Math.min(MAX_PRECISION - intDigits, bd.scale()));
+    if (bd.scale() > maxScale ) {
+      if (allowRounding) {
+        bd = bd.setScale(maxScale, RoundingMode.HALF_UP);
+        // Trimming is again necessary, because rounding may introduce new trailing 0's.
+        bd = trim(bd);
+      } else {
+        bd = null;
+      }
+    }
+
+    return bd;
+  }
+
+  private static BigDecimal enforcePrecisionScale(BigDecimal bd, int maxPrecision, int maxScale) {
+    if (bd == null) {
+      return null;
+    }
+
+    /**
+     * Specially handling the case that bd=0, and we are converting it to a type where precision=scale,
+     * such as decimal(1, 1).
+     */
+    if (bd.compareTo(BigDecimal.ZERO) == 0 && bd.scale() == 0 && maxPrecision == maxScale) {
+      return bd.setScale(maxScale);
+    }
+
+    bd = trim(bd);
+
+    if (bd.scale() > maxScale) {
+      bd = bd.setScale(maxScale, RoundingMode.HALF_UP);
+    }
+
+    int maxIntDigits = maxPrecision - maxScale;
+    int intDigits = bd.precision() - bd.scale();
+    if (intDigits > maxIntDigits) {
+      return null;
+    }
+
+    return bd;
+  }
+
+  @HiveDecimalVersionV1
+  public static HiveDecimalV1 enforcePrecisionScale(HiveDecimalV1 dec, int maxPrecision, int maxScale) {
+    if (dec == null) {
+      return null;
+    }
+
+    // Minor optimization, avoiding creating new objects.
+    if (dec.precision() - dec.scale() <= maxPrecision - maxScale &&
+        dec.scale() <= maxScale) {
+      return dec;
+    }
+
+    BigDecimal bd = enforcePrecisionScale(dec.bd, maxPrecision, maxScale);
+    if (bd == null) {
+      return null;
+    }
+
+    return HiveDecimalV1.create(bd);
+  }
+
+  @HiveDecimalVersionV1
+  public long longValueExact() {
+    return bd.longValueExact();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/storage-api/src/java/org/apache/hadoop/hive/common/type/HiveDecimalVersionV1.java
----------------------------------------------------------------------
diff --git a/storage-api/src/java/org/apache/hadoop/hive/common/type/HiveDecimalVersionV1.java b/storage-api/src/java/org/apache/hadoop/hive/common/type/HiveDecimalVersionV1.java
new file mode 100644
index 0000000..82b769a
--- /dev/null
+++ b/storage-api/src/java/org/apache/hadoop/hive/common/type/HiveDecimalVersionV1.java
@@ -0,0 +1,33 @@
+/**
+ * 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.common.type;
+
+import java.lang.annotation.Documented;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+
+/**
+ * Marks methods including static methods and fields as being part of version 1 HiveDecimal.
+ *
+ */
+@Documented
+@Retention(RetentionPolicy.RUNTIME)
+public @interface HiveDecimalVersionV1 {
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/storage-api/src/java/org/apache/hadoop/hive/common/type/HiveDecimalVersionV2.java
----------------------------------------------------------------------
diff --git a/storage-api/src/java/org/apache/hadoop/hive/common/type/HiveDecimalVersionV2.java b/storage-api/src/java/org/apache/hadoop/hive/common/type/HiveDecimalVersionV2.java
new file mode 100644
index 0000000..a47513e
--- /dev/null
+++ b/storage-api/src/java/org/apache/hadoop/hive/common/type/HiveDecimalVersionV2.java
@@ -0,0 +1,33 @@
+/**
+ * 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.common.type;
+
+import java.lang.annotation.Documented;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+
+/**
+ * Marks methods including static methods and fields as being part of version 2 HiveDecimal.
+ *
+ */
+@Documented
+@Retention(RetentionPolicy.RUNTIME)
+public @interface HiveDecimalVersionV2 {
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/storage-api/src/java/org/apache/hadoop/hive/common/type/RandomTypeUtil.java
----------------------------------------------------------------------
diff --git a/storage-api/src/java/org/apache/hadoop/hive/common/type/RandomTypeUtil.java b/storage-api/src/java/org/apache/hadoop/hive/common/type/RandomTypeUtil.java
index 53a7823..8d950a2 100644
--- a/storage-api/src/java/org/apache/hadoop/hive/common/type/RandomTypeUtil.java
+++ b/storage-api/src/java/org/apache/hadoop/hive/common/type/RandomTypeUtil.java
@@ -57,19 +57,7 @@ public class RandomTypeUtil {
 
   private static final String DECIMAL_CHARS = "0123456789";
 
-  public static class HiveDecimalAndPrecisionScale {
-    public HiveDecimal hiveDecimal;
-    public int precision;
-    public int scale;
-
-    HiveDecimalAndPrecisionScale(HiveDecimal hiveDecimal, int precision, int scale) {
-      this.hiveDecimal = hiveDecimal;
-      this.precision = precision;
-      this.scale = scale;
-    }
-  }
-
-  public static HiveDecimalAndPrecisionScale getRandHiveDecimal(Random r) {
+  public static HiveDecimal getRandHiveDecimal(Random r) {
     int precision;
     int scale;
     while (true) {
@@ -93,18 +81,7 @@ public class RandomTypeUtil {
         sb.append(getRandString(r, DECIMAL_CHARS, scale));
       }
 
-      HiveDecimal bd = HiveDecimal.create(sb.toString());
-      precision = bd.precision();
-      scale = bd.scale();
-      if (scale > precision) {
-        // Sometimes weird decimals are produced?
-        continue;
-      }
-
-      // For now, punt.
-      precision = HiveDecimal.SYSTEM_DEFAULT_PRECISION;
-      scale = HiveDecimal.SYSTEM_DEFAULT_SCALE;
-      return new HiveDecimalAndPrecisionScale(bd, precision, scale);
+      return HiveDecimal.create(sb.toString());
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/DecimalColumnVector.java
----------------------------------------------------------------------
diff --git a/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/DecimalColumnVector.java b/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/DecimalColumnVector.java
index 2488631..e4f8d82 100644
--- a/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/DecimalColumnVector.java
+++ b/storage-api/src/java/org/apache/hadoop/hive/ql/exec/vector/DecimalColumnVector.java
@@ -17,10 +17,12 @@
  */
 
 package org.apache.hadoop.hive.ql.exec.vector;
+
 import java.math.BigInteger;
 
 import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
+import org.apache.hadoop.hive.common.type.FastHiveDecimal;
 
 public class DecimalColumnVector extends ColumnVector {
 
@@ -45,7 +47,7 @@ public class DecimalColumnVector extends ColumnVector {
     this.scale = (short) scale;
     vector = new HiveDecimalWritable[size];
     for (int i = 0; i < size; i++) {
-      vector[i] = new HiveDecimalWritable(HiveDecimal.ZERO);
+      vector[i] = new HiveDecimalWritable(0);  // Initially zero.
     }
   }
 
@@ -71,15 +73,14 @@ public class DecimalColumnVector extends ColumnVector {
       inputElementNum = 0;
     }
     if (inputVector.noNulls || !inputVector.isNull[inputElementNum]) {
-      HiveDecimal hiveDec =
-          ((DecimalColumnVector) inputVector).vector[inputElementNum]
-              .getHiveDecimal(precision, scale);
-      if (hiveDec == null) {
+      vector[outElementNum].set(
+          ((DecimalColumnVector) inputVector).vector[inputElementNum],
+          precision, scale);
+      if (!vector[outElementNum].isSet()) {
         isNull[outElementNum] = true;
         noNulls = false;
       } else {
         isNull[outElementNum] = false;
-        vector[outElementNum].set(hiveDec);
       }
     } else {
       isNull[outElementNum] = true;
@@ -100,34 +101,28 @@ public class DecimalColumnVector extends ColumnVector {
   }
 
   public void set(int elementNum, HiveDecimalWritable writeable) {
-    if (writeable == null) {
+    vector[elementNum].set(writeable, precision, scale);
+    if (!vector[elementNum].isSet()) {
       noNulls = false;
       isNull[elementNum] = true;
     } else {
-      HiveDecimal hiveDec = writeable.getHiveDecimal(precision, scale);
-      if (hiveDec == null) {
-        noNulls = false;
-        isNull[elementNum] = true;
-      } else {
-        vector[elementNum].set(hiveDec);
-      }
+      isNull[elementNum] = false;
     }
   }
 
   public void set(int elementNum, HiveDecimal hiveDec) {
-    HiveDecimal checkedDec = HiveDecimal.enforcePrecisionScale(hiveDec, precision, scale);
-    if (checkedDec == null) {
+    vector[elementNum].set(hiveDec, precision, scale);
+    if (!vector[elementNum].isSet()) {
       noNulls = false;
       isNull[elementNum] = true;
     } else {
-      vector[elementNum].set(checkedDec);
+      isNull[elementNum] = false;
     }
   }
 
   public void setNullDataValue(int elementNum) {
     // E.g. For scale 2 the minimum is "0.01"
-    HiveDecimal minimumNonZeroValue = HiveDecimal.create(BigInteger.ONE, scale);
-    vector[elementNum].set(minimumNonZeroValue);
+    vector[elementNum].setFromLongAndScale(1L, scale);
   }
 
   @Override
@@ -144,7 +139,7 @@ public class DecimalColumnVector extends ColumnVector {
       System.arraycopy(oldArray, 0, vector, 0 , oldArray.length);
     }
     for (int i = initPos; i < vector.length; ++i) {
-      vector[i] = new HiveDecimalWritable(HiveDecimal.ZERO);
+      vector[i] = new HiveDecimalWritable(0);  // Initially zero.
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/storage-api/src/java/org/apache/hadoop/hive/ql/util/TimestampUtils.java
----------------------------------------------------------------------
diff --git a/storage-api/src/java/org/apache/hadoop/hive/ql/util/TimestampUtils.java b/storage-api/src/java/org/apache/hadoop/hive/ql/util/TimestampUtils.java
index 41db9ca..c16d67e 100644
--- a/storage-api/src/java/org/apache/hadoop/hive/ql/util/TimestampUtils.java
+++ b/storage-api/src/java/org/apache/hadoop/hive/ql/util/TimestampUtils.java
@@ -19,6 +19,8 @@
 package org.apache.hadoop.hive.ql.util;
 
 import org.apache.hadoop.hive.common.type.HiveDecimal;
+import org.apache.hadoop.hive.common.type.HiveDecimalV1;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 
 import java.math.BigDecimal;
 import java.sql.Timestamp;
@@ -68,9 +70,86 @@ public class TimestampUtils {
     }
   }
 
-  public static Timestamp decimalToTimestamp(HiveDecimal d) {
+  /**
+   * Take a HiveDecimal and return the timestamp representation where the fraction part is the
+   * nanoseconds and integer part is the number of seconds.
+   * @param dec
+   * @return
+   */
+  public static Timestamp decimalToTimestamp(HiveDecimal dec) {
+
+    HiveDecimalWritable nanosWritable = new HiveDecimalWritable(dec);
+    nanosWritable.mutateFractionPortion();               // Clip off seconds portion.
+    nanosWritable.mutateScaleByPowerOfTen(9);            // Bring nanoseconds into integer portion.
+    if (!nanosWritable.isSet() || !nanosWritable.isInt()) {
+      return null;
+    }
+    int nanos = nanosWritable.intValue();
+    if (nanos < 0) {
+      nanos += 1000000000;
+    }
+    nanosWritable.setFromLong(nanos);
+
+    HiveDecimalWritable nanoInstant = new HiveDecimalWritable(dec);
+    nanoInstant.mutateScaleByPowerOfTen(9);
+
+    nanoInstant.mutateSubtract(nanosWritable);
+    nanoInstant.mutateScaleByPowerOfTen(-9);              // Back to seconds.
+    if (!nanoInstant.isSet() || !nanoInstant.isLong()) {
+      return null;
+    }
+    long seconds = nanoInstant.longValue();
+    Timestamp t = new Timestamp(seconds * 1000);
+    t.setNanos(nanos);
+    return t;
+  }
+
+  /**
+   * Take a HiveDecimalWritable and return the timestamp representation where the fraction part
+   * is the nanoseconds and integer part is the number of seconds.
+   *
+   * This is a HiveDecimalWritable variation with supplied scratch objects.
+   * @param decdecWritable
+   * @param scratchDecWritable1
+   * @param scratchDecWritable2
+   * @return
+   */
+  public static Timestamp decimalToTimestamp(
+      HiveDecimalWritable decWritable,
+      HiveDecimalWritable scratchDecWritable1, HiveDecimalWritable scratchDecWritable2) {
+
+    HiveDecimalWritable nanosWritable = scratchDecWritable1;
+    nanosWritable.set(decWritable);
+    nanosWritable.mutateFractionPortion();               // Clip off seconds portion.
+    nanosWritable.mutateScaleByPowerOfTen(9);            // Bring nanoseconds into integer portion.
+    if (!nanosWritable.isSet() || !nanosWritable.isInt()) {
+      return null;
+    }
+    int nanos = nanosWritable.intValue();
+    if (nanos < 0) {
+      nanos += 1000000000;
+    }
+    nanosWritable.setFromLong(nanos);
+
+    HiveDecimalWritable nanoInstant = scratchDecWritable2;
+    nanoInstant.set(decWritable);
+    nanoInstant.mutateScaleByPowerOfTen(9);
+
+    nanoInstant.mutateSubtract(nanosWritable);
+    nanoInstant.mutateScaleByPowerOfTen(-9);              // Back to seconds.
+    if (!nanoInstant.isSet() || !nanoInstant.isLong()) {
+      return null;
+    }
+    long seconds = nanoInstant.longValue();
+
+    Timestamp timestamp = new Timestamp(seconds * 1000L);
+    timestamp.setNanos(nanos);
+    return timestamp;
+  }
+
+  public static Timestamp decimalToTimestamp(HiveDecimalV1 dec) {
     try {
-      BigDecimal nanoInstant = d.bigDecimalValue().multiply(BILLION_BIG_DECIMAL);
+      BigDecimal nanoInstant = dec.bigDecimalValue().multiply(BILLION_BIG_DECIMAL);
       int nanos = nanoInstant.remainder(BILLION_BIG_DECIMAL).intValue();
       if (nanos < 0) {
         nanos += 1000000000;


[09/10] hive git commit: HIVE-15335: Fast Decimal (Matt McCline, reviewed by Sergey Shelukhin, Prasanth Jayachandran, Owen O'Malley)

Posted by mm...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java
index d2f5408..f6b6447 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorizationContext.java
@@ -383,20 +383,20 @@ public class VectorizationContext {
     //Vectorized row batch for processing. The index in the row batch is
     //equal to the index in this array plus initialOutputCol.
     //Start with size 100 and double when needed.
-    private String [] outputColumnsTypes = new String[100];
+    private String [] scratchVectorTypeNames = new String[100];
 
     private final Set<Integer> usedOutputColumns = new HashSet<Integer>();
 
-    int allocateOutputColumn(String hiveTypeName) throws HiveException {
+    int allocateOutputColumn(TypeInfo typeInfo) throws HiveException {
         if (initialOutputCol < 0) {
-          // This is a test
+          // This is a test calling.
           return 0;
         }
 
-        // We need to differentiate DECIMAL columns by their precision and scale...
-        String normalizedTypeName = getNormalizedName(hiveTypeName);
-        int relativeCol = allocateOutputColumnInternal(normalizedTypeName);
-        // LOG.info("allocateOutputColumn for hiveTypeName " + hiveTypeName + " column " + (initialOutputCol + relativeCol));
+        // CONCERN: We currently differentiate DECIMAL columns by their precision and scale...,
+        // which could lead to a lot of extra unnecessary scratch columns.
+        String vectorTypeName = getScratchName(typeInfo);
+        int relativeCol = allocateOutputColumnInternal(vectorTypeName);
         return initialOutputCol + relativeCol;
       }
 
@@ -405,7 +405,7 @@ public class VectorizationContext {
 
         // Re-use an existing, available column of the same required type.
         if (usedOutputColumns.contains(i) ||
-            !(outputColumnsTypes)[i].equalsIgnoreCase(columnType)) {
+            !(scratchVectorTypeNames)[i].equalsIgnoreCase(columnType)) {
           continue;
         }
         //Use i
@@ -413,16 +413,16 @@ public class VectorizationContext {
         return i;
       }
       //Out of allocated columns
-      if (outputColCount < outputColumnsTypes.length) {
+      if (outputColCount < scratchVectorTypeNames.length) {
         int newIndex = outputColCount;
-        outputColumnsTypes[outputColCount++] = columnType;
+        scratchVectorTypeNames[outputColCount++] = columnType;
         usedOutputColumns.add(newIndex);
         return newIndex;
       } else {
         //Expand the array
-        outputColumnsTypes = Arrays.copyOf(outputColumnsTypes, 2*outputColCount);
+        scratchVectorTypeNames = Arrays.copyOf(scratchVectorTypeNames, 2*outputColCount);
         int newIndex = outputColCount;
-        outputColumnsTypes[outputColCount++] = columnType;
+        scratchVectorTypeNames[outputColCount++] = columnType;
         usedOutputColumns.add(newIndex);
         return newIndex;
       }
@@ -448,8 +448,8 @@ public class VectorizationContext {
     }
   }
 
-  public int allocateScratchColumn(String hiveTypeName) throws HiveException {
-    return ocm.allocateOutputColumn(hiveTypeName);
+  public int allocateScratchColumn(TypeInfo typeInfo) throws HiveException {
+    return ocm.allocateOutputColumn(typeInfo);
   }
 
   public int[] currentScratchColumns() {
@@ -1044,7 +1044,7 @@ public class VectorizationContext {
     }
     int outCol = -1;
     if (mode == VectorExpressionDescriptor.Mode.PROJECTION) {
-      outCol = ocm.allocateOutputColumn(typeName);
+      outCol = ocm.allocateOutputColumn(typeInfo);
     }
     if (constantValue == null) {
       return new ConstantVectorExpression(outCol, typeName, true);
@@ -1286,24 +1286,26 @@ public class VectorizationContext {
       // Additional argument is needed, which is the outputcolumn.
       Object [] newArgs = null;
       try {
-        String outType;
-
-        // Special handling for decimal because decimal types need scale and precision parameter.
-        // This special handling should be avoided by using returnType uniformly for all cases.
-        if (returnType != null) {
-          outType = getNormalizedName(returnType.getTypeName()).toLowerCase();
-          if (outType == null) {
-           throw new HiveException("No vector type for type name " + returnType);
+        String returnTypeName;
+        if (returnType == null) {
+          returnTypeName = ((VectorExpression) vclass.newInstance()).getOutputType().toLowerCase();
+          if (returnTypeName.equals("long")) {
+            returnTypeName = "bigint";
           }
+          returnType = TypeInfoUtils.getTypeInfoFromTypeString(returnTypeName);
         } else {
-          outType = ((VectorExpression) vclass.newInstance()).getOutputType();
+          returnTypeName = returnType.getTypeName();
         }
-        int outputCol = ocm.allocateOutputColumn(outType);
+
+        // Special handling for decimal because decimal types need scale and precision parameter.
+        // This special handling should be avoided by using returnType uniformly for all cases.
+        int outputCol = ocm.allocateOutputColumn(returnType);
+
         newArgs = Arrays.copyOf(args, numParams);
         newArgs[numParams-1] = outputCol;
 
         ve = (VectorExpression) ctor.newInstance(newArgs);
-        ve.setOutputType(outType);
+        ve.setOutputType(returnTypeName);
       } catch (Exception ex) {
           throw new HiveException("Could not instantiate " + vclass.getSimpleName() + " with arguments " + getNewInstanceArgumentString(newArgs) + ", exception: " +
                       StringUtils.stringifyException(ex));
@@ -1398,7 +1400,7 @@ public class VectorizationContext {
         inputColumns[i++] = ve.getOutputColumn();
       }
 
-      int outColumn = ocm.allocateOutputColumn(returnType.getTypeName());
+      int outColumn = ocm.allocateOutputColumn(returnType);
       VectorCoalesce vectorCoalesce = new VectorCoalesce(inputColumns, outColumn);
       vectorCoalesce.setOutputType(returnType.getTypeName());
       vectorCoalesce.setChildExpressions(vectorChildren);
@@ -1425,7 +1427,7 @@ public class VectorizationContext {
         inputColumns[i++] = ve.getOutputColumn();
       }
 
-      int outColumn = ocm.allocateOutputColumn(returnType.getTypeName());
+      int outColumn = ocm.allocateOutputColumn(returnType);
       VectorElt vectorElt = new VectorElt(inputColumns, outColumn);
       vectorElt.setOutputType(returnType.getTypeName());
       vectorElt.setChildExpressions(vectorChildren);
@@ -1607,7 +1609,7 @@ public class VectorizationContext {
 
     // Create a single child representing the scratch column where we will
     // generate the serialized keys of the batch.
-    int scratchBytesCol = ocm.allocateOutputColumn("string");
+    int scratchBytesCol = ocm.allocateOutputColumn(TypeInfoFactory.stringTypeInfo);
 
     Class<?> cl = (mode == VectorExpressionDescriptor.Mode.FILTER ? FilterStructColumnInList.class : StructColumnInList.class);
 
@@ -1729,6 +1731,20 @@ public class VectorizationContext {
     return (byte[]) o;
   }
 
+  private PrimitiveCategory getAnyIntegerPrimitiveCategoryFromUdfClass(Class<? extends UDF> udfClass) {
+    if (udfClass.equals(UDFToByte.class)) {
+      return PrimitiveCategory.BYTE;
+    } else if (udfClass.equals(UDFToShort.class)) {
+      return PrimitiveCategory.SHORT;
+    } else if (udfClass.equals(UDFToInteger.class)) {
+      return PrimitiveCategory.INT;
+    } else if (udfClass.equals(UDFToLong.class)) {
+      return PrimitiveCategory.LONG;
+    } else {
+      throw new RuntimeException("Unexpected any integery UDF class " + udfClass.getName());
+    }
+  }
+
   /**
    * Invoke special handling for expressions that can't be vectorized by regular
    * descriptor based lookup.
@@ -1738,7 +1754,9 @@ public class VectorizationContext {
     Class<? extends UDF> cl = udf.getUdfClass();
     VectorExpression ve = null;
     if (isCastToIntFamily(cl)) {
-      ve = getCastToLongExpression(childExpr);
+      PrimitiveCategory integerPrimitiveCategory =
+          getAnyIntegerPrimitiveCategoryFromUdfClass(cl);
+      ve = getCastToLongExpression(childExpr, integerPrimitiveCategory);
     } else if (cl.equals(UDFToBoolean.class)) {
       ve = getCastToBoolean(childExpr);
     } else if (isCastToFloatFamily(cl)) {
@@ -1838,7 +1856,8 @@ public class VectorizationContext {
     }
   }
 
-  private Long castConstantToLong(Object scalar, TypeInfo type) throws HiveException {
+  private Long castConstantToLong(Object scalar, TypeInfo type,
+      PrimitiveCategory integerPrimitiveCategory) throws HiveException {
     if (null == scalar) {
       return null;
     }
@@ -1854,7 +1873,36 @@ public class VectorizationContext {
       return ((Number) scalar).longValue();
     case DECIMAL:
       HiveDecimal decimalVal = (HiveDecimal) scalar;
-      return decimalVal.longValueExact();
+      switch (integerPrimitiveCategory) {
+      case BYTE:
+        if (!decimalVal.isByte()) {
+          // Accurate byte value cannot be obtained.
+          return null;
+        }
+        break;
+      case SHORT:
+        if (!decimalVal.isShort()) {
+          // Accurate short value cannot be obtained.
+          return null;
+        }
+        break;
+      case INT:
+        if (!decimalVal.isInt()) {
+          // Accurate int value cannot be obtained.
+          return null;
+        }
+        break;
+      case LONG:
+        if (!decimalVal.isLong()) {
+          // Accurate long value cannot be obtained.
+          return null;
+        }
+        break;
+      default:
+        throw new RuntimeException("Unexpected integer primitive type " + integerPrimitiveCategory);
+      }
+      // We only store longs in our LongColumnVector.
+      return decimalVal.longValue();
     default:
       throw new HiveException("Unsupported type "+typename+" for cast to Long");
     }
@@ -2004,7 +2052,7 @@ public class VectorizationContext {
       VectorExpression lenExpr = createVectorExpression(StringLength.class, childExpr,
           VectorExpressionDescriptor.Mode.PROJECTION, null);
 
-      int outputCol = ocm.allocateOutputColumn("Long");
+      int outputCol = ocm.allocateOutputColumn(TypeInfoFactory.longTypeInfo);
       VectorExpression lenToBoolExpr =
           new CastLongToBooleanViaLongToLong(lenExpr.getOutputColumn(), outputCol);
       lenToBoolExpr.setChildExpressions(new VectorExpression[] {lenExpr});
@@ -2014,14 +2062,14 @@ public class VectorizationContext {
     return null;
   }
 
-  private VectorExpression getCastToLongExpression(List<ExprNodeDesc> childExpr)
+  private VectorExpression getCastToLongExpression(List<ExprNodeDesc> childExpr, PrimitiveCategory integerPrimitiveCategory)
       throws HiveException {
     ExprNodeDesc child = childExpr.get(0);
     String inputType = childExpr.get(0).getTypeString();
     if (child instanceof ExprNodeConstantDesc) {
         // Return a constant vector expression
         Object constantValue = ((ExprNodeConstantDesc) child).getValue();
-        Long longValue = castConstantToLong(constantValue, child.getTypeInfo());
+        Long longValue = castConstantToLong(constantValue, child.getTypeInfo(), integerPrimitiveCategory);
         return getConstantVectorExpression(longValue, TypeInfoFactory.longTypeInfo, VectorExpressionDescriptor.Mode.PROJECTION);
     }
     // Float family, timestamp are handled via descriptor based lookup, int family needs
@@ -2196,12 +2244,10 @@ public class VectorizationContext {
     int outputCol = -1;
     String resultTypeName = expr.getTypeInfo().getTypeName();
 
-    outputCol = ocm.allocateOutputColumn(resultTypeName);
+    outputCol = ocm.allocateOutputColumn(expr.getTypeInfo());
 
     // Make vectorized operator
-    String normalizedName = getNormalizedName(resultTypeName);
-
-    VectorExpression ve = new VectorUDFAdaptor(expr, outputCol, normalizedName, argDescs);
+    VectorExpression ve = new VectorUDFAdaptor(expr, outputCol, resultTypeName, argDescs);
 
     // Set child expressions
     VectorExpression[] childVEs = null;
@@ -2395,36 +2441,15 @@ public class VectorizationContext {
     }
   }
 
-  static String getNormalizedName(String hiveTypeName) throws HiveException {
-    VectorExpressionDescriptor.ArgumentType argType = VectorExpressionDescriptor.ArgumentType.fromHiveTypeName(hiveTypeName);
-    switch (argType) {
-    case INT_FAMILY:
-      return "Long";
-    case FLOAT_FAMILY:
-      return "Double";
-    case DECIMAL:
-      //Return the decimal type as is, it includes scale and precision.
-      return hiveTypeName;
-    case STRING:
-      return "String";
-    case CHAR:
-      //Return the CHAR type as is, it includes maximum length
-      return hiveTypeName;
-    case VARCHAR:
-      //Return the VARCHAR type as is, it includes maximum length.
-      return hiveTypeName;
-    case BINARY:
-      return "Binary";
-    case DATE:
-      return "Date";
-    case TIMESTAMP:
-      return "Timestamp";
-    case INTERVAL_YEAR_MONTH:
-    case INTERVAL_DAY_TIME:
-      return hiveTypeName;
-    default:
-      throw new HiveException("Unexpected hive type name " + hiveTypeName);
+  static String getScratchName(TypeInfo typeInfo) throws HiveException {
+    // For now, leave DECIMAL precision/scale in the name so DecimalColumnVector scratch columns
+    // don't need their precision/scale adjusted...
+    if (typeInfo.getCategory() == Category.PRIMITIVE &&
+        ((PrimitiveTypeInfo) typeInfo).getPrimitiveCategory() == PrimitiveCategory.DECIMAL) {
+      return typeInfo.getTypeName();
     }
+    Type columnVectorType = VectorizationContext.getColumnVectorTypeFromTypeInfo(typeInfo);
+    return columnVectorType.name().toLowerCase();
   }
 
   static String getUndecoratedName(String hiveTypeName) throws HiveException {
@@ -2702,9 +2727,16 @@ public class VectorizationContext {
   public String[] getScratchColumnTypeNames() {
     String[] result = new String[ocm.outputColCount];
     for (int i = 0; i < ocm.outputColCount; i++) {
-      String typeName = ocm.outputColumnsTypes[i];
-      if (typeName.equalsIgnoreCase("long")) {
-        typeName = "bigint";   // Convert our synonym to a real Hive type name.
+      String vectorTypeName = ocm.scratchVectorTypeNames[i];
+      String typeName;
+      if (vectorTypeName.equalsIgnoreCase("bytes")) {
+        // Use hive type name.
+        typeName = "string";
+      } else if (vectorTypeName.equalsIgnoreCase("long")) {
+        // Use hive type name.
+        typeName = "bigint";
+      } else {
+        typeName = vectorTypeName;
       }
       result[i] =  typeName;
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToBoolean.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToBoolean.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToBoolean.java
index 9621cd3..ac52373 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToBoolean.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToBoolean.java
@@ -41,6 +41,11 @@ public class CastDecimalToBoolean extends FuncDecimalToLong {
    * Otherwise, return 1 for true.
    */
   protected void func(LongColumnVector outV, DecimalColumnVector inV,  int i) {
-    outV.vector[i] = inV.vector[i].getHiveDecimal().signum() == 0 ? 0 : 1;
+    outV.vector[i] = inV.vector[i].signum() == 0 ? 0 : 1;
+  }
+
+  @Override
+  public String getOutputType() {
+    return "Boolean";
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToChar.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToChar.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToChar.java
index aab3e70..e753a6e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToChar.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToChar.java
@@ -37,8 +37,8 @@ public class CastDecimalToChar extends CastDecimalToString implements TruncStrin
   }
 
   @Override
-  protected void assign(BytesColumnVector outV, int i, byte[] bytes, int length) {
-    StringExpr.rightTrimAndTruncate(outV, i, bytes, 0, length, maxLength);
+  protected void assign(BytesColumnVector outV, int i, byte[] bytes, int offset, int length) {
+    StringExpr.rightTrimAndTruncate(outV, i, bytes, offset, length, maxLength);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToDouble.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToDouble.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToDouble.java
index 63d878d..9cf97f4 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToDouble.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToDouble.java
@@ -34,6 +34,6 @@ public class CastDecimalToDouble extends FuncDecimalToDouble {
   }
 
   protected void func(DoubleColumnVector outV, DecimalColumnVector inV, int i) {
-    outV.vector[i] = inV.vector[i].getHiveDecimal().doubleValue();
+    outV.vector[i] = inV.vector[i].doubleValue();
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToLong.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToLong.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToLong.java
index 2ff6b79..28a2d74 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToLong.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToLong.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hive.ql.exec.vector.expressions;
 
 import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 
 /**
  * Type cast decimal to long
@@ -37,6 +38,47 @@ public class CastDecimalToLong extends FuncDecimalToLong {
 
   @Override
   protected void func(LongColumnVector outV, DecimalColumnVector inV,  int i) {
-    outV.vector[i] = inV.vector[i].getHiveDecimal().longValue(); // TODO: lossy conversion!
+    HiveDecimalWritable decWritable = inV.vector[i];
+
+    // Check based on the Hive integer type we need to test with isByte, isShort, isInt, isLong
+    // so we do not use corrupted (truncated) values for the Hive integer type.
+    boolean isInRange;
+    switch (integerPrimitiveCategory) {
+    case BYTE:
+      isInRange = decWritable.isByte();
+      break;
+    case SHORT:
+      isInRange = decWritable.isShort();
+      break;
+    case INT:
+      isInRange = decWritable.isInt();
+      break;
+    case LONG:
+      isInRange = decWritable.isLong();
+      break;
+    default:
+      throw new RuntimeException("Unexpected integer primitive category " + integerPrimitiveCategory);
+    }
+    if (!isInRange) {
+      outV.isNull[i] = true;
+      outV.noNulls = false;
+      return;
+    }
+    switch (integerPrimitiveCategory) {
+    case BYTE:
+      outV.vector[i] = decWritable.byteValue();
+      break;
+    case SHORT:
+      outV.vector[i] = decWritable.shortValue();
+      break;
+    case INT:
+      outV.vector[i] = decWritable.intValue();
+      break;
+    case LONG:
+      outV.vector[i] = decWritable.longValue();
+      break;
+    default:
+      throw new RuntimeException("Unexpected integer primitive category " + integerPrimitiveCategory);
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToString.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToString.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToString.java
index 243a807..ca58890 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToString.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToString.java
@@ -18,8 +18,10 @@
 
 package org.apache.hadoop.hive.ql.exec.vector.expressions;
 
+import org.apache.hadoop.hive.common.type.HiveDecimal;
 import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 
 /**
  * To support vectorized cast of decimal to string.
@@ -28,29 +30,28 @@ public class CastDecimalToString extends DecimalToStringUnaryUDF {
 
   private static final long serialVersionUID = 1L;
 
+  // We use a scratch buffer with the HiveDecimalWritable toBytes method so
+  // we don't incur poor performance creating a String result.
+  private byte[] scratchBuffer;
+
   public CastDecimalToString() {
     super();
   }
 
   public CastDecimalToString(int inputColumn, int outputColumn) {
     super(inputColumn, outputColumn);
+    scratchBuffer = new byte[HiveDecimal.SCRATCH_BUFFER_LEN_TO_BYTES];
   }
 
   // The assign method will be overridden for CHAR and VARCHAR.
-  protected void assign(BytesColumnVector outV, int i, byte[] bytes, int length) {
-    outV.setVal(i, bytes, 0, length);
+  protected void assign(BytesColumnVector outV, int i, byte[] bytes, int offset, int length) {
+    outV.setVal(i, bytes, offset, length);
   }
 
   @Override
   protected void func(BytesColumnVector outV, DecimalColumnVector inV, int i) {
-    String s = inV.vector[i].getHiveDecimal().toString();
-    byte[] b = null;
-    try {
-      b = s.getBytes("UTF-8");
-    } catch (Exception e) {
-      // This should never happen. If it does, there is a bug.
-      throw new RuntimeException("Internal error:  unable to convert decimal to string", e);
-    }
-    assign(outV, i, b, b.length);
+    HiveDecimalWritable decWritable = inV.vector[i];
+    final int byteIndex = decWritable.toBytes(scratchBuffer);
+    assign(outV, i, scratchBuffer, byteIndex, HiveDecimal.SCRATCH_BUFFER_LEN_TO_BYTES - byteIndex);
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToTimestamp.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToTimestamp.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToTimestamp.java
index 8963449..dfd9802 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToTimestamp.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToTimestamp.java
@@ -23,6 +23,7 @@ import java.sql.Timestamp;
 import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
 import org.apache.hadoop.hive.ql.util.TimestampUtils;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 
 /**
  * Type cast decimal to timestamp. The decimal value is interpreted
@@ -32,8 +33,13 @@ import org.apache.hadoop.hive.ql.util.TimestampUtils;
 public class CastDecimalToTimestamp extends FuncDecimalToTimestamp {
   private static final long serialVersionUID = 1L;
 
+  private HiveDecimalWritable scratchHiveDecimalWritable1;
+  private HiveDecimalWritable scratchHiveDecimalWritable2;
+
   public CastDecimalToTimestamp(int inputColumn, int outputColumn) {
     super(inputColumn, outputColumn);
+    scratchHiveDecimalWritable1 = new HiveDecimalWritable();
+    scratchHiveDecimalWritable2 = new HiveDecimalWritable();
   }
 
   public CastDecimalToTimestamp() {
@@ -41,7 +47,10 @@ public class CastDecimalToTimestamp extends FuncDecimalToTimestamp {
 
   @Override
   protected void func(TimestampColumnVector outV, DecimalColumnVector inV,  int i) {
-    Timestamp timestamp = TimestampUtils.decimalToTimestamp(inV.vector[i].getHiveDecimal());
+    Timestamp timestamp =
+        TimestampUtils.decimalToTimestamp(
+            inV.vector[i],
+            scratchHiveDecimalWritable1, scratchHiveDecimalWritable2);
     outV.set(i, timestamp);
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToVarChar.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToVarChar.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToVarChar.java
index 267b0b1..3a2c2d0 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToVarChar.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDecimalToVarChar.java
@@ -37,8 +37,8 @@ public class CastDecimalToVarChar extends CastDecimalToString implements TruncSt
   }
 
   @Override
-  protected void assign(BytesColumnVector outV, int i, byte[] bytes, int length) {
-    StringExpr.truncate(outV, i, bytes, 0, length, maxLength);
+  protected void assign(BytesColumnVector outV, int i, byte[] bytes, int offset, int length) {
+    StringExpr.truncate(outV, i, bytes, offset, length, maxLength);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDoubleToDecimal.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDoubleToDecimal.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDoubleToDecimal.java
index 6d6b588..79478b9 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDoubleToDecimal.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/CastDoubleToDecimal.java
@@ -18,9 +18,9 @@
 
 package org.apache.hadoop.hive.ql.exec.vector.expressions;
 
-import org.apache.hadoop.hive.common.type.HiveDecimal;
 import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 
 /**
  * Cast input double to a decimal. Get target value scale from output column vector.
@@ -39,7 +39,11 @@ public class CastDoubleToDecimal extends FuncDoubleToDecimal {
 
   @Override
   protected void func(DecimalColumnVector outV, DoubleColumnVector inV, int i) {
-    String s = ((Double) inV.vector[i]).toString();
-    outV.vector[i].set(HiveDecimal.create(s));
+    HiveDecimalWritable decWritable = outV.vector[i];
+    decWritable.setFromDouble(inV.vector[i]);
+    if (!decWritable.isSet()) {
+      outV.isNull[i] = true;
+      outV.noNulls = false;
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/DecimalColumnInList.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/DecimalColumnInList.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/DecimalColumnInList.java
index 0601c66..d4d8fea 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/DecimalColumnInList.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/DecimalColumnInList.java
@@ -37,7 +37,10 @@ public class DecimalColumnInList extends VectorExpression implements IDecimalInE
   private int outputColumn;
 
   // The set object containing the IN list.
-  private transient HashSet<HiveDecimal> inSet;
+  // We use a HashSet of HiveDecimalWritable objects instead of HiveDecimal objects so
+  // we can lookup DecimalColumnVector HiveDecimalWritable quickly without creating
+  // a HiveDecimal lookup object.
+  private transient HashSet<HiveDecimalWritable> inSet;
 
   public DecimalColumnInList() {
     super();
@@ -61,9 +64,9 @@ public class DecimalColumnInList extends VectorExpression implements IDecimalInE
     }
 
     if (inSet == null) {
-      inSet = new HashSet<HiveDecimal>(inListValues.length);
+      inSet = new HashSet<HiveDecimalWritable>(inListValues.length);
       for (HiveDecimal val : inListValues) {
-        inSet.add(val);
+        inSet.add(new HiveDecimalWritable(val));
       }
     }
 
@@ -88,16 +91,16 @@ public class DecimalColumnInList extends VectorExpression implements IDecimalInE
 
         // All must be selected otherwise size would be zero
         // Repeating property will not change.
-        outputVector[0] = inSet.contains(vector[0].getHiveDecimal()) ? 1 : 0;
+        outputVector[0] = inSet.contains(vector[0]) ? 1 : 0;
         outputColVector.isRepeating = true;
       } else if (batch.selectedInUse) {
         for(int j = 0; j != n; j++) {
           int i = sel[j];
-          outputVector[i] = inSet.contains(vector[i].getHiveDecimal()) ? 1 : 0;
+          outputVector[i] = inSet.contains(vector[i]) ? 1 : 0;
         }
       } else {
         for(int i = 0; i != n; i++) {
-          outputVector[i] = inSet.contains(vector[i].getHiveDecimal()) ? 1 : 0;
+          outputVector[i] = inSet.contains(vector[i]) ? 1 : 0;
         }
       }
     } else {
@@ -106,7 +109,7 @@ public class DecimalColumnInList extends VectorExpression implements IDecimalInE
         //All must be selected otherwise size would be zero
         //Repeating property will not change.
         if (!nullPos[0]) {
-          outputVector[0] = inSet.contains(vector[0].getHiveDecimal()) ? 1 : 0;
+          outputVector[0] = inSet.contains(vector[0]) ? 1 : 0;
           outNulls[0] = false;
         } else {
           outNulls[0] = true;
@@ -117,14 +120,14 @@ public class DecimalColumnInList extends VectorExpression implements IDecimalInE
           int i = sel[j];
           outNulls[i] = nullPos[i];
           if (!nullPos[i]) {
-            outputVector[i] = inSet.contains(vector[i].getHiveDecimal()) ? 1 : 0;
+            outputVector[i] = inSet.contains(vector[i]) ? 1 : 0;
           }
         }
       } else {
         System.arraycopy(nullPos, 0, outNulls, 0, n);
         for(int i = 0; i != n; i++) {
           if (!nullPos[i]) {
-            outputVector[i] = inSet.contains(vector[i].getHiveDecimal()) ? 1 : 0;
+            outputVector[i] = inSet.contains(vector[i]) ? 1 : 0;
           }
         }
       }

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/DecimalUtil.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/DecimalUtil.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/DecimalUtil.java
index a01f7a2..ba4646f 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/DecimalUtil.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/DecimalUtil.java
@@ -34,15 +34,17 @@ public class DecimalUtil {
   }
 
   public static int compare(HiveDecimal left, HiveDecimalWritable writableRight) {
-    return left.compareTo(writableRight.getHiveDecimal());
+    return HiveDecimalWritable.compareTo(left, writableRight);
   }
 
   // Addition with overflow check. Overflow produces NULL output.
   public static void addChecked(int i, HiveDecimal left, HiveDecimal right,
       DecimalColumnVector outputColVector) {
-    try {
-      outputColVector.set(i, left.add(right));
-    } catch (ArithmeticException e) {  // catch on overflow
+    HiveDecimalWritable decWritable = outputColVector.vector[i];
+    decWritable.set(left);
+    decWritable.mutateAdd(right);
+    decWritable.mutateEnforcePrecisionScale(outputColVector.precision, outputColVector.scale);
+    if (!decWritable.isSet()) {
       outputColVector.noNulls = false;
       outputColVector.isNull[i] = true;
     }
@@ -50,9 +52,11 @@ public class DecimalUtil {
 
   public static void addChecked(int i, HiveDecimalWritable left, HiveDecimalWritable right,
       DecimalColumnVector outputColVector) {
-    try {
-      outputColVector.set(i, left.getHiveDecimal().add(right.getHiveDecimal()));
-    } catch (ArithmeticException e) {  // catch on overflow
+    HiveDecimalWritable decWritable = outputColVector.vector[i];
+    decWritable.set(left);
+    decWritable.mutateAdd(right);
+    decWritable.mutateEnforcePrecisionScale(outputColVector.precision, outputColVector.scale);
+    if (!decWritable.isSet()) {
       outputColVector.noNulls = false;
       outputColVector.isNull[i] = true;
     }
@@ -60,9 +64,11 @@ public class DecimalUtil {
 
   public static void addChecked(int i, HiveDecimalWritable left, HiveDecimal right,
       DecimalColumnVector outputColVector) {
-    try {
-      outputColVector.set(i, left.getHiveDecimal().add(right));
-    } catch (ArithmeticException e) {  // catch on overflow
+    HiveDecimalWritable decWritable = outputColVector.vector[i];
+    decWritable.set(left);
+    decWritable.mutateAdd(right);
+    decWritable.mutateEnforcePrecisionScale(outputColVector.precision, outputColVector.scale);
+    if (!decWritable.isSet()) {
       outputColVector.noNulls = false;
       outputColVector.isNull[i] = true;
     }
@@ -70,9 +76,11 @@ public class DecimalUtil {
 
   public static void addChecked(int i, HiveDecimal left, HiveDecimalWritable right,
       DecimalColumnVector outputColVector) {
-    try {
-      outputColVector.set(i, left.add(right.getHiveDecimal()));
-    } catch (ArithmeticException e) {  // catch on overflow
+    HiveDecimalWritable decWritable = outputColVector.vector[i];
+    decWritable.set(left);
+    decWritable.mutateAdd(right);
+    decWritable.mutateEnforcePrecisionScale(outputColVector.precision, outputColVector.scale);
+    if (!decWritable.isSet()) {
       outputColVector.noNulls = false;
       outputColVector.isNull[i] = true;
     }
@@ -81,9 +89,11 @@ public class DecimalUtil {
   // Subtraction with overflow check. Overflow produces NULL output.
   public static void subtractChecked(int i, HiveDecimal left, HiveDecimal right,
       DecimalColumnVector outputColVector) {
-    try {
-      outputColVector.set(i, left.subtract(right));
-    } catch (ArithmeticException e) {  // catch on overflow
+    HiveDecimalWritable decWritable = outputColVector.vector[i];
+    decWritable.set(left);
+    decWritable.mutateSubtract(right);
+    decWritable.mutateEnforcePrecisionScale(outputColVector.precision, outputColVector.scale);
+    if (!decWritable.isSet()) {
       outputColVector.noNulls = false;
       outputColVector.isNull[i] = true;
     }
@@ -91,9 +101,11 @@ public class DecimalUtil {
 
   public static void subtractChecked(int i, HiveDecimalWritable left, HiveDecimalWritable right,
       DecimalColumnVector outputColVector) {
-    try {
-      outputColVector.set(i, left.getHiveDecimal().subtract(right.getHiveDecimal()));
-    } catch (ArithmeticException e) {  // catch on overflow
+    HiveDecimalWritable decWritable = outputColVector.vector[i];
+    decWritable.set(left);
+    decWritable.mutateSubtract(right);
+    decWritable.mutateEnforcePrecisionScale(outputColVector.precision, outputColVector.scale);
+    if (!decWritable.isSet()) {
       outputColVector.noNulls = false;
       outputColVector.isNull[i] = true;
     }
@@ -101,9 +113,11 @@ public class DecimalUtil {
 
   public static void subtractChecked(int i, HiveDecimalWritable left, HiveDecimal right,
       DecimalColumnVector outputColVector) {
-    try {
-      outputColVector.set(i, left.getHiveDecimal().subtract(right));
-    } catch (ArithmeticException e) {  // catch on overflow
+    HiveDecimalWritable decWritable = outputColVector.vector[i];
+    decWritable.set(left);
+    decWritable.mutateSubtract(right);
+    decWritable.mutateEnforcePrecisionScale(outputColVector.precision, outputColVector.scale);
+    if (!decWritable.isSet()) {
       outputColVector.noNulls = false;
       outputColVector.isNull[i] = true;
     }
@@ -111,9 +125,11 @@ public class DecimalUtil {
 
   public static void subtractChecked(int i, HiveDecimal left, HiveDecimalWritable right,
       DecimalColumnVector outputColVector) {
-    try {
-      outputColVector.set(i, left.subtract(right.getHiveDecimal()));
-    } catch (ArithmeticException e) {  // catch on overflow
+    HiveDecimalWritable decWritable = outputColVector.vector[i];
+    decWritable.set(left);
+    decWritable.mutateSubtract(right);
+    decWritable.mutateEnforcePrecisionScale(outputColVector.precision, outputColVector.scale);
+    if (!decWritable.isSet()) {
       outputColVector.noNulls = false;
       outputColVector.isNull[i] = true;
     }
@@ -122,9 +138,11 @@ public class DecimalUtil {
   // Multiplication with overflow check. Overflow produces NULL output.
   public static void multiplyChecked(int i, HiveDecimal left, HiveDecimal right,
       DecimalColumnVector outputColVector) {
-    try {
-      outputColVector.set(i, left.multiply(right));
-    } catch (ArithmeticException e) {  // catch on overflow
+    HiveDecimalWritable decWritable = outputColVector.vector[i];
+    decWritable.set(left);
+    decWritable.mutateMultiply(right);
+    decWritable.mutateEnforcePrecisionScale(outputColVector.precision, outputColVector.scale);
+    if (!decWritable.isSet()) {
       outputColVector.noNulls = false;
       outputColVector.isNull[i] = true;
     }
@@ -132,9 +150,11 @@ public class DecimalUtil {
 
   public static void multiplyChecked(int i, HiveDecimalWritable left, HiveDecimalWritable right,
       DecimalColumnVector outputColVector) {
-    try {
-      outputColVector.set(i, left.getHiveDecimal().multiply(right.getHiveDecimal()));
-    } catch (ArithmeticException e) {  // catch on overflow
+    HiveDecimalWritable decWritable = outputColVector.vector[i];
+    decWritable.set(left);
+    decWritable.mutateMultiply(right);
+    decWritable.mutateEnforcePrecisionScale(outputColVector.precision, outputColVector.scale);
+    if (!decWritable.isSet()) {
       outputColVector.noNulls = false;
       outputColVector.isNull[i] = true;
     }
@@ -142,9 +162,11 @@ public class DecimalUtil {
 
   public static void multiplyChecked(int i, HiveDecimalWritable left, HiveDecimal right,
       DecimalColumnVector outputColVector) {
-    try {
-      outputColVector.set(i, left.getHiveDecimal().multiply(right));
-    } catch (ArithmeticException e) {  // catch on overflow
+    HiveDecimalWritable decWritable = outputColVector.vector[i];
+    decWritable.set(left);
+    decWritable.mutateMultiply(right);
+    decWritable.mutateEnforcePrecisionScale(outputColVector.precision, outputColVector.scale);
+    if (!decWritable.isSet()) {
       outputColVector.noNulls = false;
       outputColVector.isNull[i] = true;
     }
@@ -152,9 +174,11 @@ public class DecimalUtil {
 
   public static void multiplyChecked(int i, HiveDecimal left, HiveDecimalWritable right,
       DecimalColumnVector outputColVector) {
-    try {
-      outputColVector.set(i, left.multiply(right.getHiveDecimal()));
-    } catch (ArithmeticException e) {  // catch on overflow
+    HiveDecimalWritable decWritable = outputColVector.vector[i];
+    decWritable.set(left);
+    decWritable.mutateMultiply(right);
+    decWritable.mutateEnforcePrecisionScale(outputColVector.precision, outputColVector.scale);
+    if (!decWritable.isSet()) {
       outputColVector.noNulls = false;
       outputColVector.isNull[i] = true;
     }
@@ -163,9 +187,11 @@ public class DecimalUtil {
   // Division with overflow/zero-divide check. Error produces NULL output.
   public static void divideChecked(int i, HiveDecimal left, HiveDecimal right,
       DecimalColumnVector outputColVector) {
-    try {
-      outputColVector.set(i, left.divide(right));
-    } catch (ArithmeticException e) {  // catch on error
+    HiveDecimalWritable decWritable = outputColVector.vector[i];
+    decWritable.set(left);
+    decWritable.mutateDivide(right);
+    decWritable.mutateEnforcePrecisionScale(outputColVector.precision, outputColVector.scale);
+    if (!decWritable.isSet()) {
       outputColVector.noNulls = false;
       outputColVector.isNull[i] = true;
     }
@@ -173,9 +199,11 @@ public class DecimalUtil {
 
   public static void divideChecked(int i, HiveDecimalWritable left, HiveDecimalWritable right,
       DecimalColumnVector outputColVector) {
-    try {
-      outputColVector.set(i, left.getHiveDecimal().divide(right.getHiveDecimal()));
-    } catch (ArithmeticException e) {  // catch on error
+    HiveDecimalWritable decWritable = outputColVector.vector[i];
+    decWritable.set(left);
+    decWritable.mutateDivide(right);
+    decWritable.mutateEnforcePrecisionScale(outputColVector.precision, outputColVector.scale);
+    if (!decWritable.isSet()) {
       outputColVector.noNulls = false;
       outputColVector.isNull[i] = true;
     }
@@ -183,9 +211,11 @@ public class DecimalUtil {
 
   public static void divideChecked(int i, HiveDecimalWritable left, HiveDecimal right,
       DecimalColumnVector outputColVector) {
-    try {
-      outputColVector.set(i, left.getHiveDecimal().divide(right));
-    } catch (ArithmeticException e) {  // catch on error
+    HiveDecimalWritable decWritable = outputColVector.vector[i];
+    decWritable.set(left);
+    decWritable.mutateDivide(right);
+    decWritable.mutateEnforcePrecisionScale(outputColVector.precision, outputColVector.scale);
+    if (!decWritable.isSet()) {
       outputColVector.noNulls = false;
       outputColVector.isNull[i] = true;
     }
@@ -193,9 +223,11 @@ public class DecimalUtil {
 
   public static void divideChecked(int i, HiveDecimal left, HiveDecimalWritable right,
       DecimalColumnVector outputColVector) {
-    try {
-      outputColVector.set(i, left.divide(right.getHiveDecimal()));
-    } catch (ArithmeticException e) {  // catch on error
+    HiveDecimalWritable decWritable = outputColVector.vector[i];
+    decWritable.set(left);
+    decWritable.mutateDivide(right);
+    decWritable.mutateEnforcePrecisionScale(outputColVector.precision, outputColVector.scale);
+    if (!decWritable.isSet()) {
       outputColVector.noNulls = false;
       outputColVector.isNull[i] = true;
     }
@@ -204,9 +236,11 @@ public class DecimalUtil {
   // Modulo operator with overflow/zero-divide check.
   public static void moduloChecked(int i, HiveDecimal left, HiveDecimal right,
       DecimalColumnVector outputColVector) {
-    try {
-      outputColVector.set(i, left.remainder(right));
-    } catch (ArithmeticException e) {  // catch on error
+    HiveDecimalWritable decWritable = outputColVector.vector[i];
+    decWritable.set(left);
+    decWritable.mutateRemainder(right);
+    decWritable.mutateEnforcePrecisionScale(outputColVector.precision, outputColVector.scale);
+    if (!decWritable.isSet()) {
       outputColVector.noNulls = false;
       outputColVector.isNull[i] = true;
     }
@@ -214,9 +248,11 @@ public class DecimalUtil {
 
   public static void moduloChecked(int i, HiveDecimalWritable left, HiveDecimalWritable right,
       DecimalColumnVector outputColVector) {
-    try {
-      outputColVector.set(i, left.getHiveDecimal().remainder(right.getHiveDecimal()));
-    } catch (ArithmeticException e) {  // catch on error
+    HiveDecimalWritable decWritable = outputColVector.vector[i];
+    decWritable.set(left);
+    decWritable.mutateRemainder(right);
+    decWritable.mutateEnforcePrecisionScale(outputColVector.precision, outputColVector.scale);
+    if (!decWritable.isSet()) {
       outputColVector.noNulls = false;
       outputColVector.isNull[i] = true;
     }
@@ -224,9 +260,11 @@ public class DecimalUtil {
 
   public static void moduloChecked(int i, HiveDecimalWritable left, HiveDecimal right,
       DecimalColumnVector outputColVector) {
-    try {
-      outputColVector.set(i, left.getHiveDecimal().remainder(right));
-    } catch (ArithmeticException e) {  // catch on error
+    HiveDecimalWritable decWritable = outputColVector.vector[i];
+    decWritable.set(left);
+    decWritable.mutateRemainder(right);
+    decWritable.mutateEnforcePrecisionScale(outputColVector.precision, outputColVector.scale);
+    if (!decWritable.isSet()) {
       outputColVector.noNulls = false;
       outputColVector.isNull[i] = true;
     }
@@ -234,99 +272,122 @@ public class DecimalUtil {
 
   public static void moduloChecked(int i, HiveDecimal left, HiveDecimalWritable right,
       DecimalColumnVector outputColVector) {
-    try {
-      outputColVector.set(i, left.remainder(right.getHiveDecimal()));
-    } catch (ArithmeticException e) {  // catch on error
+    HiveDecimalWritable decWritable = outputColVector.vector[i];
+    decWritable.set(left);
+    decWritable.mutateRemainder(right);
+    decWritable.mutateEnforcePrecisionScale(outputColVector.precision, outputColVector.scale);
+    if (!decWritable.isSet()) {
       outputColVector.noNulls = false;
       outputColVector.isNull[i] = true;
     }
   }
 
+  // UNDONE: Why don't these methods take decimalPlaces?
   public static void floor(int i, HiveDecimal input, DecimalColumnVector outputColVector) {
-    try {
-      outputColVector.set(i, input.setScale(0, HiveDecimal.ROUND_FLOOR));
-    } catch (ArithmeticException e) {
+    HiveDecimalWritable decWritable = outputColVector.vector[i];
+    decWritable.set(input);
+    decWritable.mutateSetScale(0, HiveDecimal.ROUND_FLOOR);
+    decWritable.mutateEnforcePrecisionScale(outputColVector.precision, outputColVector.scale);
+    if (!decWritable.isSet()) {
       outputColVector.noNulls = false;
       outputColVector.isNull[i] = true;
     }
   }
 
   public static void floor(int i, HiveDecimalWritable input, DecimalColumnVector outputColVector) {
-    try {
-      outputColVector.set(i, input.getHiveDecimal().setScale(0, HiveDecimal.ROUND_FLOOR));
-    } catch (ArithmeticException e) {
+    HiveDecimalWritable decWritable = outputColVector.vector[i];
+    decWritable.set(input);
+    decWritable.mutateSetScale(0, HiveDecimal.ROUND_FLOOR);
+    decWritable.mutateEnforcePrecisionScale(outputColVector.precision, outputColVector.scale);
+    if (!decWritable.isSet()) {
       outputColVector.noNulls = false;
       outputColVector.isNull[i] = true;
     }
   }
 
   public static void ceiling(int i, HiveDecimal input, DecimalColumnVector outputColVector) {
-    try {
-      outputColVector.set(i, input.setScale(0, HiveDecimal.ROUND_CEILING));
-    } catch (ArithmeticException e) {
+    HiveDecimalWritable decWritable = outputColVector.vector[i];
+    decWritable.set(input);
+    decWritable.mutateSetScale(0, HiveDecimal.ROUND_CEILING);
+    decWritable.mutateEnforcePrecisionScale(outputColVector.precision, outputColVector.scale);
+    if (!decWritable.isSet()) {
       outputColVector.noNulls = false;
       outputColVector.isNull[i] = true;
     }
   }
 
   public static void ceiling(int i, HiveDecimalWritable input, DecimalColumnVector outputColVector) {
-    try {
-      outputColVector.set(i, input.getHiveDecimal().setScale(0, HiveDecimal.ROUND_CEILING));
-    } catch (ArithmeticException e) {
+    HiveDecimalWritable decWritable = outputColVector.vector[i];
+    decWritable.set(input);
+    decWritable.mutateSetScale(0, HiveDecimal.ROUND_CEILING);
+    decWritable.mutateEnforcePrecisionScale(outputColVector.precision, outputColVector.scale);
+    if (!decWritable.isSet()) {
       outputColVector.noNulls = false;
       outputColVector.isNull[i] = true;
     }
   }
 
   public static void round(int i, HiveDecimal input, int decimalPlaces, DecimalColumnVector outputColVector) {
-    try {
-      outputColVector.set(i, RoundUtils.round(input, decimalPlaces));
-    } catch (ArithmeticException e) {
+    HiveDecimalWritable decWritable = outputColVector.vector[i];
+    decWritable.set(input);
+    decWritable.mutateSetScale(decimalPlaces, HiveDecimal.ROUND_HALF_UP);
+    decWritable.mutateEnforcePrecisionScale(outputColVector.precision, outputColVector.scale);
+    if (!decWritable.isSet()) {
       outputColVector.noNulls = false;
       outputColVector.isNull[i] = true;
     }
   }
 
   public static void round(int i, HiveDecimalWritable input, int decimalPlaces, DecimalColumnVector outputColVector) {
-    try {
-      outputColVector.set(i, RoundUtils.round(input.getHiveDecimal(), decimalPlaces));
-    } catch (ArithmeticException e) {
+    HiveDecimalWritable decWritable = outputColVector.vector[i];
+    decWritable.set(input);
+    decWritable.mutateSetScale(decimalPlaces, HiveDecimal.ROUND_HALF_UP);
+    decWritable.mutateEnforcePrecisionScale(outputColVector.precision, outputColVector.scale);
+    if (!decWritable.isSet()) {
       outputColVector.noNulls = false;
       outputColVector.isNull[i] = true;
     }
   }
 
   public static void round(int i, HiveDecimal input, DecimalColumnVector outputColVector) {
-    try {
-      outputColVector.set(i, RoundUtils.round(input, outputColVector.scale));
-    } catch (ArithmeticException e) {
+    HiveDecimalWritable decWritable = outputColVector.vector[i];
+    decWritable.set(input);
+    decWritable.mutateSetScale(outputColVector.scale, HiveDecimal.ROUND_HALF_UP);
+    decWritable.mutateEnforcePrecisionScale(outputColVector.precision, outputColVector.scale);
+    if (!decWritable.isSet()) {
       outputColVector.noNulls = false;
       outputColVector.isNull[i] = true;
     }
   }
 
   public static void round(int i, HiveDecimalWritable input, DecimalColumnVector outputColVector) {
-    try {
-      outputColVector.set(i, RoundUtils.round(input.getHiveDecimal(), outputColVector.scale));
-    } catch (ArithmeticException e) {
+    HiveDecimalWritable decWritable = outputColVector.vector[i];
+    decWritable.set(input);
+    decWritable.mutateSetScale(outputColVector.scale, HiveDecimal.ROUND_HALF_UP);
+    decWritable.mutateEnforcePrecisionScale(outputColVector.precision, outputColVector.scale);
+    if (!decWritable.isSet()) {
       outputColVector.noNulls = false;
       outputColVector.isNull[i] = true;
     }
   }
 
   public static void bround(int i, HiveDecimalWritable input, int decimalPlaces, DecimalColumnVector outputColVector) {
-    try {
-      outputColVector.set(i, RoundUtils.bround(input.getHiveDecimal(), decimalPlaces));
-    } catch (ArithmeticException e) {
+    HiveDecimalWritable decWritable = outputColVector.vector[i];
+    decWritable.set(input);
+    decWritable.mutateSetScale(decimalPlaces, HiveDecimal.ROUND_HALF_EVEN);
+    decWritable.mutateEnforcePrecisionScale(outputColVector.precision, outputColVector.scale);
+    if (!decWritable.isSet()) {
       outputColVector.noNulls = false;
       outputColVector.isNull[i] = true;
     }
   }
 
   public static void bround(int i, HiveDecimalWritable input, DecimalColumnVector outputColVector) {
-    try {
-      outputColVector.set(i, RoundUtils.bround(input.getHiveDecimal(), outputColVector.scale));
-    } catch (ArithmeticException e) {
+    HiveDecimalWritable decWritable = outputColVector.vector[i];
+    decWritable.set(input);
+    decWritable.mutateSetScale(outputColVector.scale, HiveDecimal.ROUND_HALF_EVEN);
+    decWritable.mutateEnforcePrecisionScale(outputColVector.precision, outputColVector.scale);
+    if (!decWritable.isSet()) {
       outputColVector.noNulls = false;
       outputColVector.isNull[i] = true;
     }
@@ -337,42 +398,30 @@ public class DecimalUtil {
   }
 
   public static void sign(int i, HiveDecimalWritable input, LongColumnVector outputColVector) {
-    outputColVector.vector[i] = input.getHiveDecimal().signum();
+    outputColVector.vector[i] = input.signum();
   }
 
   public static void abs(int i, HiveDecimal input, DecimalColumnVector outputColVector) {
-    try {
-      outputColVector.set(i, input.abs());
-    } catch (ArithmeticException e) {
-      outputColVector.noNulls = false;
-      outputColVector.isNull[i] = true;
-    }
+    HiveDecimalWritable decWritable = outputColVector.vector[i];
+    decWritable.set(input);
+    decWritable.mutateAbs();
   }
 
   public static void abs(int i, HiveDecimalWritable input, DecimalColumnVector outputColVector) {
-    try {
-      outputColVector.set(i, input.getHiveDecimal().abs());
-    } catch (ArithmeticException e) {
-      outputColVector.noNulls = false;
-      outputColVector.isNull[i] = true;
-    }
+    HiveDecimalWritable decWritable = outputColVector.vector[i];
+    decWritable.set(input);
+    decWritable.mutateAbs();
   }
 
   public static void negate(int i, HiveDecimal input, DecimalColumnVector outputColVector) {
-    try {
-      outputColVector.set(i, input.negate());
-    } catch (ArithmeticException e) {
-      outputColVector.noNulls = false;
-      outputColVector.isNull[i] = true;
-    }
+    HiveDecimalWritable decWritable = outputColVector.vector[i];
+    decWritable.set(input);
+    decWritable.mutateNegate();
   }
 
   public static void negate(int i, HiveDecimalWritable input, DecimalColumnVector outputColVector) {
-    try {
-      outputColVector.set(i, input.getHiveDecimal().negate());
-    } catch (ArithmeticException e) {
-      outputColVector.noNulls = false;
-      outputColVector.isNull[i] = true;
-    }
+    HiveDecimalWritable decWritable = outputColVector.vector[i];
+    decWritable.set(input);
+    decWritable.mutateNegate();
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FilterDecimalColumnInList.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FilterDecimalColumnInList.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FilterDecimalColumnInList.java
index a865343..79d3fe3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FilterDecimalColumnInList.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FilterDecimalColumnInList.java
@@ -35,7 +35,7 @@ public class FilterDecimalColumnInList extends VectorExpression implements IDeci
   private HiveDecimal[] inListValues;
 
   // The set object containing the IN list.
-  private transient HashSet<HiveDecimal> inSet;
+  private transient HashSet<HiveDecimalWritable> inSet;
 
   public FilterDecimalColumnInList() {
     super();
@@ -58,9 +58,9 @@ public class FilterDecimalColumnInList extends VectorExpression implements IDeci
     }
 
     if (inSet == null) {
-      inSet = new HashSet<HiveDecimal>(inListValues.length);
+      inSet = new HashSet<HiveDecimalWritable>(inListValues.length);
       for (HiveDecimal val : inListValues) {
-        inSet.add(val);
+        inSet.add(new HiveDecimalWritable(val));
       }
     }
 
@@ -81,7 +81,7 @@ public class FilterDecimalColumnInList extends VectorExpression implements IDeci
         // All must be selected otherwise size would be zero
         // Repeating property will not change.
 
-        if (!(inSet.contains(vector[0].getHiveDecimal()))) {
+        if (!(inSet.contains(vector[0]))) {
           //Entire batch is filtered out.
           batch.size = 0;
         }
@@ -89,7 +89,7 @@ public class FilterDecimalColumnInList extends VectorExpression implements IDeci
         int newSize = 0;
         for(int j = 0; j != n; j++) {
           int i = sel[j];
-          if (inSet.contains(vector[i].getHiveDecimal())) {
+          if (inSet.contains(vector[i])) {
             sel[newSize++] = i;
           }
         }
@@ -97,7 +97,7 @@ public class FilterDecimalColumnInList extends VectorExpression implements IDeci
       } else {
         int newSize = 0;
         for(int i = 0; i != n; i++) {
-          if (inSet.contains(vector[i].getHiveDecimal())) {
+          if (inSet.contains(vector[i])) {
             sel[newSize++] = i;
           }
         }
@@ -112,7 +112,7 @@ public class FilterDecimalColumnInList extends VectorExpression implements IDeci
         //All must be selected otherwise size would be zero
         //Repeating property will not change.
         if (!nullPos[0]) {
-          if (!inSet.contains(vector[0].getHiveDecimal())) {
+          if (!inSet.contains(vector[0])) {
 
             //Entire batch is filtered out.
             batch.size = 0;
@@ -125,7 +125,7 @@ public class FilterDecimalColumnInList extends VectorExpression implements IDeci
         for(int j = 0; j != n; j++) {
           int i = sel[j];
           if (!nullPos[i]) {
-           if (inSet.contains(vector[i].getHiveDecimal())) {
+           if (inSet.contains(vector[i])) {
              sel[newSize++] = i;
            }
           }
@@ -137,7 +137,7 @@ public class FilterDecimalColumnInList extends VectorExpression implements IDeci
         int newSize = 0;
         for(int i = 0; i != n; i++) {
           if (!nullPos[i]) {
-            if (inSet.contains(vector[i].getHiveDecimal())) {
+            if (inSet.contains(vector[i])) {
               sel[newSize++] = i;
             }
           }

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FilterStructColumnInList.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FilterStructColumnInList.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FilterStructColumnInList.java
index 70b393c..1e21fea 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FilterStructColumnInList.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FilterStructColumnInList.java
@@ -112,7 +112,7 @@ public class FilterStructColumnInList extends FilterStringColumnInList implement
           case DECIMAL:
             DecimalColumnVector decColVector = ((DecimalColumnVector) colVec);
             binarySortableSerializeWrite.writeHiveDecimal(
-                decColVector.vector[adjustedIndex].getHiveDecimal(), decColVector.scale);
+                decColVector.vector[adjustedIndex], decColVector.scale);
             break;
 
           default:

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncDecimalToLong.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncDecimalToLong.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncDecimalToLong.java
index 4691fe1..7f005a1 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncDecimalToLong.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/FuncDecimalToLong.java
@@ -22,6 +22,10 @@ import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorExpressionDescriptor;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector.PrimitiveCategory;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
+import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
 
 /**
  * This is a superclass for unary decimal functions and expressions returning integers that
@@ -32,6 +36,9 @@ public abstract class FuncDecimalToLong extends VectorExpression {
   int inputColumn;
   int outputColumn;
 
+  private transient boolean integerPrimitiveCategoryKnown = false;
+  protected transient PrimitiveCategory integerPrimitiveCategory;
+
   public FuncDecimalToLong(int inputColumn, int outputColumn) {
     this.inputColumn = inputColumn;
     this.outputColumn = outputColumn;
@@ -50,6 +57,13 @@ public abstract class FuncDecimalToLong extends VectorExpression {
       super.evaluateChildren(batch);
     }
 
+    if (!integerPrimitiveCategoryKnown) {
+      String typeName = getOutputType().toLowerCase();
+      TypeInfo typeInfo = TypeInfoUtils.getTypeInfoFromTypeString(typeName);
+      integerPrimitiveCategory = ((PrimitiveTypeInfo) typeInfo).getPrimitiveCategory();
+      integerPrimitiveCategoryKnown = true;
+    }
+
     DecimalColumnVector inV = (DecimalColumnVector) batch.cols[inputColumn];
     int[] sel = batch.selected;
     int n = batch.size;
@@ -117,11 +131,6 @@ public abstract class FuncDecimalToLong extends VectorExpression {
   }
 
   @Override
-  public String getOutputType() {
-    return "long";
-  }
-
-  @Override
   public VectorExpressionDescriptor.Descriptor getDescriptor() {
     VectorExpressionDescriptor.Builder b = new VectorExpressionDescriptor.Builder();
     b.setMode(VectorExpressionDescriptor.Mode.PROJECTION)

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StructColumnInList.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StructColumnInList.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StructColumnInList.java
index 769c70a..8134108 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StructColumnInList.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/StructColumnInList.java
@@ -113,7 +113,7 @@ public class StructColumnInList extends StringColumnInList implements IStructInE
           case DECIMAL:
             DecimalColumnVector decColVector = ((DecimalColumnVector) colVec);
             binarySortableSerializeWrite.writeHiveDecimal(
-                decColVector.vector[adjustedIndex].getHiveDecimal(), decColVector.scale);
+                decColVector.vector[adjustedIndex], decColVector.scale);
             break;
 
           default:

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFAvgDecimal.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFAvgDecimal.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFAvgDecimal.java
index d0ff5fa..4f6d652 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFAvgDecimal.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFAvgDecimal.java
@@ -60,45 +60,25 @@ public class VectorUDAFAvgDecimal extends VectorAggregateExpression {
       transient private long count;
       transient private boolean isNull;
 
-      // We use this to catch overflow.
-      transient private boolean isOutOfRange;
-
-      public void sumValueWithNullCheck(HiveDecimalWritable writable, short scale) {
-        if (isOutOfRange) {
-          return;
-        }
-        HiveDecimal value = writable.getHiveDecimal();
+      public void sumValueWithNullCheck(HiveDecimalWritable writable) {
         if (isNull) {
-          sum.set(value);
+          // Make a copy since we intend to mutate sum.
+          sum.set(writable);
           count = 1;
           isNull = false;
         } else {
-          HiveDecimal result;
-          try {
-            result = sum.getHiveDecimal().add(value);
-          } catch (ArithmeticException e) {  // catch on overflow
-            isOutOfRange = true;
-            return;
-          }
-          sum.set(result);
+          // Note that if sum is out of range, mutateAdd will ignore the call.
+          // At the end, sum.isSet() can be checked for null.
+          sum.mutateAdd(writable);
           count++;
         }
       }
 
-      public void sumValueNoNullCheck(HiveDecimalWritable writable, short scale) {
-        HiveDecimal value = writable.getHiveDecimal();
-        HiveDecimal result;
-        try {
-          result = sum.getHiveDecimal().add(value);
-        } catch (ArithmeticException e) {  // catch on overflow
-          isOutOfRange = true;
-          return;
-        }
-        sum.set(result);
+      public void sumValueNoNullCheck(HiveDecimalWritable writable) {
+        sum.mutateAdd(writable);
         count++;
       }
 
-
       @Override
       public int getVariableSize() {
         throw new UnsupportedOperationException();
@@ -107,9 +87,8 @@ public class VectorUDAFAvgDecimal extends VectorAggregateExpression {
       @Override
       public void reset() {
         isNull = true;
-        isOutOfRange = false;
-        sum.set(HiveDecimal.ZERO);
-        count = 0L;
+        sum.setFromLong(0L);
+        count = 0;
       }
     }
 
@@ -251,7 +230,7 @@ public class VectorUDAFAvgDecimal extends VectorAggregateExpression {
           aggregationBufferSets,
           bufferIndex,
           i);
-        myagg.sumValueWithNullCheck(value, this.sumScale);
+        myagg.sumValueWithNullCheck(value);
       }
     }
 
@@ -267,7 +246,7 @@ public class VectorUDAFAvgDecimal extends VectorAggregateExpression {
           aggregationBufferSets,
           bufferIndex,
           i);
-        myagg.sumValueWithNullCheck(values[selection[i]], this.sumScale);
+        myagg.sumValueWithNullCheck(values[selection[i]]);
       }
     }
 
@@ -281,7 +260,7 @@ public class VectorUDAFAvgDecimal extends VectorAggregateExpression {
           aggregationBufferSets,
           bufferIndex,
           i);
-        myagg.sumValueWithNullCheck(values[i], this.sumScale);
+        myagg.sumValueWithNullCheck(values[i]);
       }
     }
 
@@ -302,7 +281,7 @@ public class VectorUDAFAvgDecimal extends VectorAggregateExpression {
           aggregationBufferSets,
           bufferIndex,
           i);
-        myagg.sumValueWithNullCheck(value, this.sumScale);
+        myagg.sumValueWithNullCheck(value);
       }
 
     }
@@ -323,7 +302,7 @@ public class VectorUDAFAvgDecimal extends VectorAggregateExpression {
           aggregationBufferSets,
           bufferIndex,
           i);
-        myagg.sumValueWithNullCheck(value, this.sumScale);
+        myagg.sumValueWithNullCheck(value);
       }
     }
 
@@ -342,7 +321,7 @@ public class VectorUDAFAvgDecimal extends VectorAggregateExpression {
             aggregationBufferSets,
             bufferIndex,
             j);
-          myagg.sumValueWithNullCheck(values[i], this.sumScale);
+          myagg.sumValueWithNullCheck(values[i]);
         }
       }
    }
@@ -360,7 +339,7 @@ public class VectorUDAFAvgDecimal extends VectorAggregateExpression {
             aggregationBufferSets,
             bufferIndex,
             i);
-          myagg.sumValueWithNullCheck(values[i], this.sumScale);
+          myagg.sumValueWithNullCheck(values[i]);
         }
       }
    }
@@ -389,25 +368,12 @@ public class VectorUDAFAvgDecimal extends VectorAggregateExpression {
           if (inputVector.noNulls) {
             if (myagg.isNull) {
               myagg.isNull = false;
-              myagg.sum.set(HiveDecimal.ZERO);
+              myagg.sum.setFromLong(0L);
               myagg.count = 0;
             }
             HiveDecimal value = vector[0].getHiveDecimal();
-            HiveDecimal multiple;
-            try {
-              multiple = value.multiply(HiveDecimal.create(batchSize));
-            } catch (ArithmeticException e) {  // catch on overflow
-              myagg.isOutOfRange = true;
-              return;
-            }
-            HiveDecimal result;
-            try {
-              result = myagg.sum.getHiveDecimal().add(multiple);
-            } catch (ArithmeticException e) {  // catch on overflow
-              myagg.isOutOfRange = true;
-              return;
-            }
-            myagg.sum.set(result);
+            HiveDecimal multiple = value.multiply(HiveDecimal.create(batchSize));
+            myagg.sum.mutateAdd(multiple);
             myagg.count += batchSize;
           }
           return;
@@ -437,8 +403,7 @@ public class VectorUDAFAvgDecimal extends VectorAggregateExpression {
       for (int j=0; j< batchSize; ++j) {
         int i = selected[j];
         if (!isNull[i]) {
-          HiveDecimalWritable value = vector[i];
-          myagg.sumValueWithNullCheck(value, this.sumScale);
+          myagg.sumValueWithNullCheck(vector[i]);
         }
       }
     }
@@ -451,13 +416,12 @@ public class VectorUDAFAvgDecimal extends VectorAggregateExpression {
 
       if (myagg.isNull) {
         myagg.isNull = false;
-        myagg.sum.set(HiveDecimal.ZERO);
+        myagg.sum.setFromLong(0L);
         myagg.count = 0;
       }
 
       for (int i=0; i< batchSize; ++i) {
-        HiveDecimalWritable value = vector[selected[i]];
-        myagg.sumValueNoNullCheck(value, this.sumScale);
+        myagg.sumValueNoNullCheck(vector[selected[i]]);
       }
     }
 
@@ -469,8 +433,7 @@ public class VectorUDAFAvgDecimal extends VectorAggregateExpression {
 
       for(int i=0;i<batchSize;++i) {
         if (!isNull[i]) {
-          HiveDecimalWritable value = vector[i];
-          myagg.sumValueWithNullCheck(value, this.sumScale);
+          myagg.sumValueWithNullCheck(vector[i]);
         }
       }
     }
@@ -481,13 +444,12 @@ public class VectorUDAFAvgDecimal extends VectorAggregateExpression {
         int batchSize) {
       if (myagg.isNull) {
         myagg.isNull = false;
-        myagg.sum.set(HiveDecimal.ZERO);
+        myagg.sum.setFromLong(0L);
         myagg.count = 0;
       }
 
       for (int i=0;i<batchSize;++i) {
-        HiveDecimalWritable value = vector[i];
-        myagg.sumValueNoNullCheck(value, this.sumScale);
+        myagg.sumValueNoNullCheck(vector[i]);
       }
     }
 
@@ -506,13 +468,14 @@ public class VectorUDAFAvgDecimal extends VectorAggregateExpression {
     public Object evaluateOutput(
         AggregationBuffer agg) throws HiveException {
       Aggregation myagg = (Aggregation) agg;
-      if (myagg.isNull || myagg.isOutOfRange) {
+      // !isSet checks for overflow.
+      if (myagg.isNull || !myagg.sum.isSet()) {
         return null;
       }
       else {
         assert(0 < myagg.count);
         resultCount.set (myagg.count);
-        resultSum.set(myagg.sum.getHiveDecimal());
+        resultSum.set(myagg.sum);
         return partialResult;
       }
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFSumDecimal.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFSumDecimal.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFSumDecimal.java
index 3a5fef6..508c960 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFSumDecimal.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFSumDecimal.java
@@ -48,29 +48,16 @@ public class VectorUDAFSumDecimal extends VectorAggregateExpression {
 
       private static final long serialVersionUID = 1L;
 
-      transient private HiveDecimalWritable sum = new HiveDecimalWritable();
+      transient private final HiveDecimalWritable sum = new HiveDecimalWritable();
       transient private boolean isNull;
 
-      // We use this to catch overflow.
-      transient private boolean isOutOfRange;
-
-      public void sumValue(HiveDecimalWritable writable, short scale) {
-        if (isOutOfRange) {
-          return;
-        }
-        HiveDecimal value = writable.getHiveDecimal();
+      public void sumValue(HiveDecimalWritable writable) {
         if (isNull) {
-          sum.set(value);
+          // Make a copy since we intend to mutate sum.
+          sum.set(writable);
           isNull = false;
         } else {
-          HiveDecimal result;
-          try {
-            result = sum.getHiveDecimal().add(value);
-          } catch (ArithmeticException e) {  // catch on overflow
-            isOutOfRange = true;
-            return;
-          }
-          sum.set(result);
+          sum.mutateAdd(writable);
         }
       }
 
@@ -82,13 +69,11 @@ public class VectorUDAFSumDecimal extends VectorAggregateExpression {
       @Override
       public void reset() {
         isNull = true;
-        isOutOfRange = false;
-        sum.set(HiveDecimal.ZERO);
+        sum.setFromLong(0L);
       }
     }
 
     private VectorExpression inputExpression;
-    transient private final HiveDecimalWritable scratchDecimal;
 
     public VectorUDAFSumDecimal(VectorExpression inputExpression) {
       this();
@@ -97,7 +82,6 @@ public class VectorUDAFSumDecimal extends VectorAggregateExpression {
 
     public VectorUDAFSumDecimal() {
       super();
-      scratchDecimal = new HiveDecimalWritable();
     }
 
     private Aggregation getCurrentAggregationBuffer(
@@ -131,18 +115,18 @@ public class VectorUDAFSumDecimal extends VectorAggregateExpression {
         if (inputVector.isRepeating) {
           iterateNoNullsRepeatingWithAggregationSelection(
             aggregationBufferSets, aggregateIndex,
-            vector[0], inputVector.scale,
+            vector[0],
             batchSize);
         } else {
           if (batch.selectedInUse) {
             iterateNoNullsSelectionWithAggregationSelection(
               aggregationBufferSets, aggregateIndex,
-              vector, inputVector.scale,
+              vector,
               batch.selected, batchSize);
           } else {
             iterateNoNullsWithAggregationSelection(
               aggregationBufferSets, aggregateIndex,
-              vector, inputVector.scale,
+              vector,
               batchSize);
           }
         }
@@ -151,24 +135,24 @@ public class VectorUDAFSumDecimal extends VectorAggregateExpression {
           if (batch.selectedInUse) {
             iterateHasNullsRepeatingSelectionWithAggregationSelection(
               aggregationBufferSets, aggregateIndex,
-              vector[0], inputVector.scale,
+              vector[0],
               batchSize, batch.selected, inputVector.isNull);
           } else {
             iterateHasNullsRepeatingWithAggregationSelection(
               aggregationBufferSets, aggregateIndex,
-              vector[0], inputVector.scale,
+              vector[0],
               batchSize, inputVector.isNull);
           }
         } else {
           if (batch.selectedInUse) {
             iterateHasNullsSelectionWithAggregationSelection(
               aggregationBufferSets, aggregateIndex,
-              vector, inputVector.scale,
+              vector,
               batchSize, batch.selected, inputVector.isNull);
           } else {
             iterateHasNullsWithAggregationSelection(
               aggregationBufferSets, aggregateIndex,
-              vector,inputVector.scale,
+              vector,
               batchSize, inputVector.isNull);
           }
         }
@@ -179,7 +163,6 @@ public class VectorUDAFSumDecimal extends VectorAggregateExpression {
       VectorAggregationBufferRow[] aggregationBufferSets,
       int aggregateIndex,
       HiveDecimalWritable value,
-      short scale,
       int batchSize) {
 
       for (int i=0; i < batchSize; ++i) {
@@ -187,7 +170,7 @@ public class VectorUDAFSumDecimal extends VectorAggregateExpression {
           aggregationBufferSets,
           aggregateIndex,
           i);
-        myagg.sumValue(value, scale);
+        myagg.sumValue(value);
       }
     }
 
@@ -195,7 +178,6 @@ public class VectorUDAFSumDecimal extends VectorAggregateExpression {
       VectorAggregationBufferRow[] aggregationBufferSets,
       int aggregateIndex,
       HiveDecimalWritable[] values,
-      short scale,
       int[] selection,
       int batchSize) {
 
@@ -204,7 +186,7 @@ public class VectorUDAFSumDecimal extends VectorAggregateExpression {
           aggregationBufferSets,
           aggregateIndex,
           i);
-        myagg.sumValue(values[selection[i]], scale);
+        myagg.sumValue(values[selection[i]]);
       }
     }
 
@@ -212,14 +194,13 @@ public class VectorUDAFSumDecimal extends VectorAggregateExpression {
       VectorAggregationBufferRow[] aggregationBufferSets,
       int aggregateIndex,
       HiveDecimalWritable[] values,
-      short scale,
       int batchSize) {
       for (int i=0; i < batchSize; ++i) {
         Aggregation myagg = getCurrentAggregationBuffer(
           aggregationBufferSets,
           aggregateIndex,
           i);
-        myagg.sumValue(values[i], scale);
+        myagg.sumValue(values[i]);
       }
     }
 
@@ -227,7 +208,6 @@ public class VectorUDAFSumDecimal extends VectorAggregateExpression {
       VectorAggregationBufferRow[] aggregationBufferSets,
       int aggregateIndex,
       HiveDecimalWritable value,
-      short scale,
       int batchSize,
       int[] selection,
       boolean[] isNull) {
@@ -241,7 +221,7 @@ public class VectorUDAFSumDecimal extends VectorAggregateExpression {
           aggregationBufferSets,
           aggregateIndex,
           i);
-        myagg.sumValue(value, scale);
+        myagg.sumValue(value);
       }
 
     }
@@ -250,7 +230,6 @@ public class VectorUDAFSumDecimal extends VectorAggregateExpression {
       VectorAggregationBufferRow[] aggregationBufferSets,
       int aggregateIndex,
       HiveDecimalWritable value,
-      short scale,
       int batchSize,
       boolean[] isNull) {
 
@@ -263,7 +242,7 @@ public class VectorUDAFSumDecimal extends VectorAggregateExpression {
           aggregationBufferSets,
           aggregateIndex,
           i);
-        myagg.sumValue(value, scale);
+        myagg.sumValue(value);
       }
     }
 
@@ -271,7 +250,6 @@ public class VectorUDAFSumDecimal extends VectorAggregateExpression {
       VectorAggregationBufferRow[] aggregationBufferSets,
       int aggregateIndex,
       HiveDecimalWritable[] values,
-      short scale,
       int batchSize,
       int[] selection,
       boolean[] isNull) {
@@ -283,7 +261,7 @@ public class VectorUDAFSumDecimal extends VectorAggregateExpression {
             aggregationBufferSets,
             aggregateIndex,
             j);
-          myagg.sumValue(values[i], scale);
+          myagg.sumValue(values[i]);
         }
       }
    }
@@ -292,7 +270,6 @@ public class VectorUDAFSumDecimal extends VectorAggregateExpression {
       VectorAggregationBufferRow[] aggregationBufferSets,
       int aggregateIndex,
       HiveDecimalWritable[] values,
-      short scale,
       int batchSize,
       boolean[] isNull) {
 
@@ -302,7 +279,7 @@ public class VectorUDAFSumDecimal extends VectorAggregateExpression {
             aggregationBufferSets,
             aggregateIndex,
             i);
-          myagg.sumValue(values[i], scale);
+          myagg.sumValue(values[i]);
         }
       }
    }
@@ -324,9 +301,6 @@ public class VectorUDAFSumDecimal extends VectorAggregateExpression {
       }
 
       Aggregation myagg = (Aggregation)agg;
-      if (myagg.isOutOfRange) {
-        return;
-      }
 
       HiveDecimalWritable[] vector = inputVector.vector;
 
@@ -334,46 +308,32 @@ public class VectorUDAFSumDecimal extends VectorAggregateExpression {
         if ((inputVector.noNulls) || !inputVector.isNull[0]) {
           if (myagg.isNull) {
             myagg.isNull = false;
-            myagg.sum.set(HiveDecimal.ZERO);
+            myagg.sum.setFromLong(0L);
           }
           HiveDecimal value = vector[0].getHiveDecimal();
-          HiveDecimal multiple;
-          try {
-            multiple = value.multiply(HiveDecimal.create(batchSize));
-          } catch (ArithmeticException e) {  // catch on overflow
-            myagg.isOutOfRange = true;
-            return;
-          }
-          HiveDecimal result;
-          try {
-            result = myagg.sum.getHiveDecimal().add(multiple);
-          } catch (ArithmeticException e) {  // catch on overflow
-            myagg.isOutOfRange = true;
-            return;
-          }
-          myagg.sum.set(result);
+          HiveDecimal multiple = value.multiply(HiveDecimal.create(batchSize));
+          myagg.sum.mutateAdd(multiple);
         }
         return;
       }
 
       if (!batch.selectedInUse && inputVector.noNulls) {
-        iterateNoSelectionNoNulls(myagg, vector, inputVector.scale, batchSize);
+        iterateNoSelectionNoNulls(myagg, vector, batchSize);
       }
       else if (!batch.selectedInUse) {
-        iterateNoSelectionHasNulls(myagg, vector, inputVector.scale, batchSize, inputVector.isNull);
+        iterateNoSelectionHasNulls(myagg, vector, batchSize, inputVector.isNull);
       }
       else if (inputVector.noNulls){
-        iterateSelectionNoNulls(myagg, vector, inputVector.scale, batchSize, batch.selected);
+        iterateSelectionNoNulls(myagg, vector, batchSize, batch.selected);
       }
       else {
-        iterateSelectionHasNulls(myagg, vector, inputVector.scale, batchSize, inputVector.isNull, batch.selected);
+        iterateSelectionHasNulls(myagg, vector, batchSize, inputVector.isNull, batch.selected);
       }
     }
 
     private void iterateSelectionHasNulls(
         Aggregation myagg,
         HiveDecimalWritable[] vector,
-        short scale,
         int batchSize,
         boolean[] isNull,
         int[] selected) {
@@ -383,17 +343,9 @@ public class VectorUDAFSumDecimal extends VectorAggregateExpression {
         if (!isNull[i]) {
           if (myagg.isNull) {
             myagg.isNull = false;
-            myagg.sum.set(HiveDecimal.ZERO);
-          }
-          HiveDecimal value = vector[i].getHiveDecimal();
-          HiveDecimal result;
-          try {
-            result = myagg.sum.getHiveDecimal().add(value);
-          } catch (ArithmeticException e) {  // catch on overflow
-            myagg.isOutOfRange = true;
-            return;
+            myagg.sum.setFromLong(0L);
           }
-          myagg.sum.set(result);
+          myagg.sum.mutateAdd(vector[i]);
         }
       }
     }
@@ -401,50 +353,32 @@ public class VectorUDAFSumDecimal extends VectorAggregateExpression {
     private void iterateSelectionNoNulls(
         Aggregation myagg,
         HiveDecimalWritable[] vector,
-        short scale,
         int batchSize,
         int[] selected) {
 
       if (myagg.isNull) {
-        myagg.sum.set(HiveDecimal.ZERO);
+        myagg.sum.setFromLong(0L);
         myagg.isNull = false;
       }
 
       for (int i=0; i< batchSize; ++i) {
-        HiveDecimal value = vector[selected[i]].getHiveDecimal();
-        HiveDecimal result;
-        try {
-          result = myagg.sum.getHiveDecimal().add(value);
-        } catch (ArithmeticException e) {  // catch on overflow
-          myagg.isOutOfRange = true;
-          return;
-        }
-        myagg.sum.set(result);
+        myagg.sum.mutateAdd(vector[selected[i]]);
       }
     }
 
     private void iterateNoSelectionHasNulls(
         Aggregation myagg,
         HiveDecimalWritable[] vector,
-        short scale,
         int batchSize,
         boolean[] isNull) {
 
       for(int i=0;i<batchSize;++i) {
         if (!isNull[i]) {
           if (myagg.isNull) {
-            myagg.sum.set(HiveDecimal.ZERO);
+            myagg.sum.setFromLong(0L);
             myagg.isNull = false;
           }
-          HiveDecimal value = vector[i].getHiveDecimal();
-          HiveDecimal result;
-          try {
-            result = myagg.sum.getHiveDecimal().add(value);
-          } catch (ArithmeticException e) {  // catch on overflow
-            myagg.isOutOfRange = true;
-            return;
-          }
-          myagg.sum.set(result);
+          myagg.sum.mutateAdd(vector[i]);
         }
       }
     }
@@ -452,23 +386,14 @@ public class VectorUDAFSumDecimal extends VectorAggregateExpression {
     private void iterateNoSelectionNoNulls(
         Aggregation myagg,
         HiveDecimalWritable[] vector,
-        short scale,
         int batchSize) {
       if (myagg.isNull) {
-        myagg.sum.set(HiveDecimal.ZERO);
+        myagg.sum.setFromLong(0L);
         myagg.isNull = false;
       }
 
       for (int i=0;i<batchSize;++i) {
-        HiveDecimal value = vector[i].getHiveDecimal();
-        HiveDecimal result;
-        try {
-          result = myagg.sum.getHiveDecimal().add(value);
-        } catch (ArithmeticException e) {  // catch on overflow
-          myagg.isOutOfRange = true;
-          return;
-        }
-        myagg.sum.set(result);
+        myagg.sum.mutateAdd(vector[i]);
       }
     }
 
@@ -486,7 +411,8 @@ public class VectorUDAFSumDecimal extends VectorAggregateExpression {
     @Override
     public Object evaluateOutput(AggregationBuffer agg) throws HiveException {
       Aggregation myagg = (Aggregation) agg;
-      if (myagg.isNull || myagg.isOutOfRange) {
+      // !isSet checks for overflow.
+      if (myagg.isNull || !myagg.sum.isSet()) {
         return null;
       }
       else {

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinCommonOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinCommonOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinCommonOperator.java
index c288731..476446a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinCommonOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/mapjoin/VectorMapJoinCommonOperator.java
@@ -369,7 +369,8 @@ public abstract class VectorMapJoinCommonOperator extends MapJoinOperator implem
             // we must have a physical (scratch) column for those keys.  We cannot use the
             // projection optimization used by inner joins above.
 
-            int scratchColumn = vOutContext.allocateScratchColumn(typeName);
+            TypeInfo typeInfo = TypeInfoUtils.getTypeInfoFromTypeString(typeName);
+            int scratchColumn = vOutContext.allocateScratchColumn(typeInfo);
             projectionMapping.add(nextOutputColumn, scratchColumn, typeName);
 
             bigTableRetainedMapping.add(batchKeyColumn, scratchColumn, typeName);
@@ -385,7 +386,8 @@ public abstract class VectorMapJoinCommonOperator extends MapJoinOperator implem
           String typeName = smallTableExprs.get(i).getTypeString();
 
           // Make a new big table scratch column for the small table value.
-          int scratchColumn = vOutContext.allocateScratchColumn(typeName);
+          TypeInfo typeInfo = TypeInfoUtils.getTypeInfoFromTypeString(typeName);
+          int scratchColumn = vOutContext.allocateScratchColumn(typeInfo);
           projectionMapping.add(nextOutputColumn, scratchColumn, typeName);
 
           smallTableMapping.add(smallTableValueIndex, scratchColumn, typeName);
@@ -401,9 +403,10 @@ public abstract class VectorMapJoinCommonOperator extends MapJoinOperator implem
         int smallTableValueIndex = smallTableRetainList.get(i);
 
         // Make a new big table scratch column for the small table value.
-        String typeName = smallTableExprs.get(i).getTypeString();
-        int scratchColumn = vOutContext.allocateScratchColumn(typeName);
+        TypeInfo typeInfo = smallTableExprs.get(i).getTypeInfo();
+        int scratchColumn = vOutContext.allocateScratchColumn(typeInfo);
 
+        String typeName = smallTableExprs.get(i).getTypeString();
         projectionMapping.add(nextOutputColumn, scratchColumn, typeName);
 
         smallTableMapping.add(smallTableValueIndex, scratchColumn, typeName);

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/write/DataWritableWriter.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/write/DataWritableWriter.java b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/write/DataWritableWriter.java
index 1e26c19..6b7b50a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/write/DataWritableWriter.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/parquet/write/DataWritableWriter.java
@@ -517,7 +517,8 @@ public class DataWritableWriter {
     private Binary decimalToBinary(final HiveDecimal hiveDecimal, final DecimalTypeInfo decimalTypeInfo) {
       int prec = decimalTypeInfo.precision();
       int scale = decimalTypeInfo.scale();
-      byte[] decimalBytes = hiveDecimal.setScale(scale).unscaledValue().toByteArray();
+
+      byte[] decimalBytes = hiveDecimal.bigIntegerBytesScaled(scale);
 
       // Estimated number of bytes needed.
       int precToBytes = ParquetHiveSerDe.PRECISION_TO_BYTE_COUNT[prec - 1];


[10/10] hive git commit: HIVE-15335: Fast Decimal (Matt McCline, reviewed by Sergey Shelukhin, Prasanth Jayachandran, Owen O'Malley)

Posted by mm...@apache.org.
HIVE-15335: Fast Decimal (Matt McCline, reviewed by Sergey Shelukhin, Prasanth Jayachandran, Owen O'Malley)


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

Branch: refs/heads/master
Commit: 4ba713ccd85c3706d195aeef9476e6e6363f1c21
Parents: 597ca1b
Author: Matt McCline <mm...@hortonworks.com>
Authored: Thu Dec 22 00:32:08 2016 -0800
Committer: Matt McCline <mm...@hortonworks.com>
Committed: Thu Dec 22 00:32:08 2016 -0800

----------------------------------------------------------------------
 .../hive/common/type/HiveDecimalTestBase.java   |  561 ++
 .../hive/common/type/TestHiveDecimal.java       |  216 -
 .../TestHiveDecimalOrcSerializationUtils.java   |  378 +
 .../apache/orc/impl/ColumnStatisticsImpl.java   |   50 +-
 .../orc/impl/ConvertTreeReaderFactory.java      |   81 +-
 .../org/apache/orc/impl/TreeReaderFactory.java  |   30 +-
 .../java/org/apache/orc/impl/WriterImpl.java    |   33 +-
 .../org/apache/orc/TestColumnStatistics.java    |   13 +-
 .../UDAFTemplates/VectorUDAFMinMaxDecimal.txt   |   45 +-
 .../ql/exec/vector/VectorDeserializeRow.java    |    3 +-
 .../hive/ql/exec/vector/VectorExtractRow.java   |    3 +-
 .../ql/exec/vector/VectorHashKeyWrapper.java    |    3 +-
 .../exec/vector/VectorHashKeyWrapperBatch.java  |    2 +-
 .../hive/ql/exec/vector/VectorSerializeRow.java |    7 +-
 .../ql/exec/vector/VectorizationContext.java    |  174 +-
 .../expressions/CastDecimalToBoolean.java       |    7 +-
 .../vector/expressions/CastDecimalToChar.java   |    4 +-
 .../vector/expressions/CastDecimalToDouble.java |    2 +-
 .../vector/expressions/CastDecimalToLong.java   |   44 +-
 .../vector/expressions/CastDecimalToString.java |   23 +-
 .../expressions/CastDecimalToTimestamp.java     |   11 +-
 .../expressions/CastDecimalToVarChar.java       |    4 +-
 .../vector/expressions/CastDoubleToDecimal.java |   10 +-
 .../vector/expressions/DecimalColumnInList.java |   21 +-
 .../ql/exec/vector/expressions/DecimalUtil.java |  281 +-
 .../expressions/FilterDecimalColumnInList.java  |   18 +-
 .../expressions/FilterStructColumnInList.java   |    2 +-
 .../vector/expressions/FuncDecimalToLong.java   |   19 +-
 .../vector/expressions/StructColumnInList.java  |    2 +-
 .../aggregates/VectorUDAFAvgDecimal.java        |   95 +-
 .../aggregates/VectorUDAFSumDecimal.java        |  148 +-
 .../mapjoin/VectorMapJoinCommonOperator.java    |   11 +-
 .../ql/io/parquet/write/DataWritableWriter.java |    3 +-
 .../hive/ql/io/sarg/ConvertAstToSearchArg.java  |    6 +-
 .../org/apache/hadoop/hive/ql/udf/UDFLog.java   |    4 +-
 .../org/apache/hadoop/hive/ql/udf/UDFMath.java  |    2 +-
 .../org/apache/hadoop/hive/ql/udf/UDFSign.java  |    6 +-
 .../apache/hadoop/hive/ql/udf/UDFToBoolean.java |    4 +-
 .../apache/hadoop/hive/ql/udf/UDFToByte.java    |    4 +-
 .../apache/hadoop/hive/ql/udf/UDFToDouble.java  |    4 +-
 .../apache/hadoop/hive/ql/udf/UDFToFloat.java   |    4 +-
 .../apache/hadoop/hive/ql/udf/UDFToInteger.java |    4 +-
 .../apache/hadoop/hive/ql/udf/UDFToLong.java    |    4 +-
 .../apache/hadoop/hive/ql/udf/UDFToShort.java   |    4 +-
 .../hive/ql/udf/generic/GenericUDAFSum.java     |   44 +-
 .../hive/ql/udf/generic/GenericUDFAbs.java      |    3 +-
 .../hive/ql/udf/generic/GenericUDFBRound.java   |    7 +-
 .../hive/ql/udf/generic/GenericUDFCeil.java     |    4 +-
 .../hive/ql/udf/generic/GenericUDFFloor.java    |    4 +-
 .../ql/udf/generic/GenericUDFOPNegative.java    |    4 +-
 .../udf/generic/GenericUDFOPNumericMinus.java   |    7 +-
 .../ql/udf/generic/GenericUDFOPNumericPlus.java |    9 +-
 .../hive/ql/udf/generic/GenericUDFRound.java    |   16 +-
 .../exec/vector/TestVectorGroupByOperator.java  |   12 +-
 .../hive/ql/exec/vector/TestVectorSerDeRow.java |  226 +-
 .../ql/exec/vector/VectorRandomRowSource.java   |   26 +-
 .../vector/expressions/TestDecimalUtil.java     |    1 -
 .../vector/expressions/TestVectorTypeCasts.java |   28 +-
 .../mapjoin/fast/CheckFastRowHashMap.java       |   27 +-
 .../exec/vector/mapjoin/fast/VerifyFastRow.java |    5 +-
 .../vector/util/VectorizedRowGroupGenUtil.java  |    6 +-
 .../test/results/clientpositive/decimal_2.q.out |    4 +-
 ...ema_evol_orc_nonvec_part_all_primitive.q.out |    4 +-
 ...schema_evol_orc_vec_part_all_primitive.q.out |    4 +-
 ...ma_evol_text_nonvec_part_all_primitive.q.out |    8 +-
 ...chema_evol_text_vec_part_all_primitive.q.out |    8 +-
 ...ma_evol_text_vecrow_part_all_primitive.q.out |    8 +-
 .../clientpositive/llap/vector_decimal_2.q.out  |    4 +-
 .../llap/vector_decimal_expressions.q.out       |   20 +-
 .../vector_decimal_expressions.q.out            |   20 +-
 .../hadoop/hive/serde2/avro/AvroSerdeUtils.java |    5 +-
 .../binarysortable/BinarySortableSerDe.java     |  138 +-
 .../fast/BinarySortableDeserializeRead.java     |   35 +-
 .../fast/BinarySortableSerializeWrite.java      |   26 +-
 .../hadoop/hive/serde2/fast/SerializeWrite.java |    5 +
 .../hive/serde2/lazy/LazyHiveDecimal.java       |   72 +-
 .../lazy/fast/LazySimpleDeserializeRead.java    |   23 +-
 .../lazy/fast/LazySimpleSerializeWrite.java     |   26 +-
 .../LazyHiveDecimalObjectInspector.java         |    9 +-
 .../lazybinary/LazyBinaryHiveDecimal.java       |    6 +-
 .../hive/serde2/lazybinary/LazyBinarySerDe.java |   74 +-
 .../fast/LazyBinaryDeserializeRead.java         |   28 +-
 .../fast/LazyBinarySerializeWrite.java          |   56 +-
 .../objectinspector/ObjectInspectorUtils.java   |    2 +
 .../PrimitiveObjectInspectorUtils.java          |   58 +-
 ...tableConstantHiveDecimalObjectInspector.java |   17 +-
 .../hive/serde2/typeinfo/HiveDecimalUtils.java  |    5 +-
 .../hive/serde2/SerdeRandomRowSource.java       |   28 +-
 .../apache/hadoop/hive/serde2/VerifyFast.java   |    2 +-
 .../hive/serde2/avro/TestAvroSerializer.java    |    8 +-
 .../binarysortable/MyTestPrimitiveClass.java    |   16 +-
 .../binarysortable/TestBinarySortableFast.java  |   43 +-
 .../hive/serde2/io/TestHiveDecimalWritable.java |  250 -
 .../hive/common/type/FastHiveDecimal.java       |  741 ++
 .../hive/common/type/FastHiveDecimalImpl.java   | 9149 ++++++++++++++++++
 .../hadoop/hive/common/type/HiveDecimal.java    | 1487 ++-
 .../hadoop/hive/common/type/HiveDecimalV1.java  |  386 +
 .../hive/common/type/HiveDecimalVersionV1.java  |   33 +
 .../hive/common/type/HiveDecimalVersionV2.java  |   33 +
 .../hadoop/hive/common/type/RandomTypeUtil.java |   27 +-
 .../ql/exec/vector/DecimalColumnVector.java     |   35 +-
 .../hadoop/hive/ql/util/TimestampUtils.java     |   83 +-
 .../hive/serde2/io/HiveDecimalWritable.java     |  956 +-
 .../hive/serde2/io/HiveDecimalWritableV1.java   |  191 +
 .../serde2/io/HiveDecimalWritableVersionV1.java |   33 +
 .../serde2/io/HiveDecimalWritableVersionV2.java |   33 +
 .../org/apache/hadoop/hive/VersionTestBase.java |  322 +
 .../hive/common/type/HiveDecimalTestBase.java   |  558 ++
 .../hive/common/type/TestHiveDecimal.java       | 3404 +++++++
 .../common/type/TestHiveDecimalVersion.java     |   67 +
 .../hive/serde2/io/TestHiveDecimalWritable.java |   52 +
 .../io/TestHiveDecimalWritableVersion.java      |   71 +
 112 files changed, 19789 insertions(+), 1663 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/common/src/test/org/apache/hadoop/hive/common/type/HiveDecimalTestBase.java
----------------------------------------------------------------------
diff --git a/common/src/test/org/apache/hadoop/hive/common/type/HiveDecimalTestBase.java b/common/src/test/org/apache/hadoop/hive/common/type/HiveDecimalTestBase.java
new file mode 100644
index 0000000..dee49f9
--- /dev/null
+++ b/common/src/test/org/apache/hadoop/hive/common/type/HiveDecimalTestBase.java
@@ -0,0 +1,561 @@
+/**
+ * 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.common.type;
+
+import java.util.Random;
+import java.math.BigDecimal;
+import java.math.BigInteger;
+
+import org.apache.hadoop.hive.common.type.RandomTypeUtil;
+
+// A COPY of the one in storage-api since currently sharing test classes isn't supported in
+// our build.
+
+public class HiveDecimalTestBase {
+
+  public static int POUND_FACTOR = 1000;
+
+  public static enum BigDecimalFlavor {
+    NORMAL_RANGE,
+    FRACTIONS_ONLY,
+    NEGATIVE_SCALE,
+    LONG_TAIL
+  }
+
+  public static enum BigDecimalPairFlavor {
+    RANDOM,
+    NEAR,
+    INVERSE
+  }
+
+  public BigDecimal randHiveBigDecimal(Random r, String digitAlphabet, BigDecimalFlavor bigDecimalFlavor) {
+    switch (bigDecimalFlavor) {
+    case NORMAL_RANGE:
+      return randHiveBigDecimalNormalRange(r, digitAlphabet);
+    case FRACTIONS_ONLY:
+      return randHiveBigDecimalFractionsOnly(r, digitAlphabet);
+    case NEGATIVE_SCALE:
+      return randHiveBigDecimalNegativeScale(r, digitAlphabet);
+    case LONG_TAIL:
+      return randHiveBigDecimalLongTail(r, digitAlphabet);
+    default:
+      throw new RuntimeException("Unexpected big decimal flavor " + bigDecimalFlavor);
+    }
+  }
+
+  public BigDecimal[] randHiveBigDecimalPair(Random r, String digitAlphabet,
+      BigDecimalFlavor bigDecimalFlavor, BigDecimalPairFlavor bigDecimalPairFlavor) {
+    BigDecimal[] pair = new BigDecimal[2];
+    BigDecimal bigDecimal1 = randHiveBigDecimal(r, digitAlphabet, bigDecimalFlavor);
+    pair[0] = bigDecimal1;
+
+    BigDecimal bigDecimal2;
+    switch (bigDecimalPairFlavor) {
+    case RANDOM:
+      bigDecimal2 = randHiveBigDecimal(r, digitAlphabet, bigDecimalFlavor);
+      break;
+    case NEAR:
+      bigDecimal2 = randHiveBigDecimalNear(r, bigDecimal1);
+      break;
+    case INVERSE:
+      bigDecimal2 = randHiveBigDecimalNear(r, bigDecimal1);
+      break;
+    default:
+      throw new RuntimeException("Unexpected big decimal pair flavor " + bigDecimalPairFlavor);
+    }
+    pair[1] = bigDecimal2;
+    return pair;
+  }
+
+  public BigDecimal randHiveBigDecimalNormalRange(Random r, String digitAlphabet) {
+    String digits = RandomTypeUtil.getRandString(r, digitAlphabet, 1 + r.nextInt(38));
+    BigInteger bigInteger = new BigInteger(digits);
+    boolean negated = false;
+    if (r.nextBoolean()) {
+      bigInteger = bigInteger.negate();
+      negated = true;
+    }
+    int scale = 0 + r.nextInt(38 + 1);
+    return new BigDecimal(bigInteger, scale);
+  }
+
+  public BigDecimal randHiveBigDecimalNegativeScale(Random r, String digitAlphabet) {
+    String digits = RandomTypeUtil.getRandString(r, digitAlphabet, 1 + r.nextInt(38));
+    BigInteger bigInteger = new BigInteger(digits);
+    boolean negated = false;
+    if (r.nextBoolean()) {
+      bigInteger = bigInteger.negate();
+      negated = true;
+    }
+    int scale = 0 + (r.nextBoolean() ? 0 : r.nextInt(38 + 1));
+    if (r.nextBoolean()) {
+      scale = -scale;
+    }
+    return new BigDecimal(bigInteger, scale);
+  }
+
+  public BigDecimal randHiveBigDecimalLongTail(Random r, String digitAlphabet) {
+    int scale = 0 + r.nextInt(38 + 20);
+    final int maxDigits = 38 + (scale == 0 ? 0 : 20);
+    String digits = RandomTypeUtil.getRandString(r, digitAlphabet, 1 + r.nextInt(maxDigits));
+    BigInteger bigInteger = new BigInteger(digits);
+    boolean negated = false;
+    if (r.nextBoolean()) {
+      bigInteger = bigInteger.negate();
+      negated = true;
+    }
+    return new BigDecimal(bigInteger, scale);
+  }
+
+  public BigDecimal randHiveBigDecimalFractionsOnly(Random r, String digitAlphabet) {
+    int scale = 1 + r.nextInt(38 + 1);
+    String digits = RandomTypeUtil.getRandString(r, digitAlphabet, 1 + r.nextInt(scale));
+    BigInteger bigInteger = new BigInteger(digits);
+    boolean negated = false;
+    if (r.nextBoolean()) {
+      bigInteger = bigInteger.negate();
+      negated = true;
+    }
+    return new BigDecimal(bigInteger, scale);
+  }
+
+  public BigDecimal randHiveBigDecimalNear(Random r, BigDecimal bigDecimal) {
+
+    int scale = bigDecimal.scale();
+    int delta = r.nextInt(10);
+    if (r.nextBoolean()) {
+      return bigDecimal.add(new BigDecimal(BigInteger.valueOf(delta), scale));
+    } else {
+      return bigDecimal.subtract(new BigDecimal(BigInteger.valueOf(delta), scale));
+    }
+  }
+
+  public BigDecimal randHiveBigDecimalInverse(Random r, BigDecimal bigDecimal) {
+    if (bigDecimal.signum() == 0) {
+      return bigDecimal;
+    }
+    return BigDecimal.ONE.divide(bigDecimal);
+  }
+
+  public BigInteger randHiveBigInteger(Random r, String digitAlphabet) {
+    String digits = RandomTypeUtil.getRandString(r, digitAlphabet, 1 + r.nextInt(38));
+    BigInteger bigInteger = new BigInteger(digits);
+    boolean negated = false;
+    if (r.nextBoolean()) {
+      bigInteger = bigInteger.negate();
+      negated = true;
+    }
+    return bigInteger;
+  }
+
+  public boolean isTenPowerBug(String string) {
+    // // System.out.println("TEST_IS_TEN_TO_38_STRING isTenPowerBug " + string);
+    if (string.charAt(0) == '-') {
+      string = string.substring(1);
+    }
+    int index = string.indexOf('.');
+    if (index != -1) {
+      if (index == 0) {
+        string = string.substring(1);
+      } else {
+        string = string.substring(0, index) + string.substring(index + 1);
+      }
+    }
+    // // System.out.println("TEST_IS_TEN_TO_38_STRING isTenPowerBug " + string);
+    return string.equals("100000000000000000000000000000000000000");
+  }
+
+  //------------------------------------------------------------------------------------------------
+
+  public static String[] specialDecimalStrings = new String[] {
+    "0",
+    "1",
+    "-1",
+    "10",
+    "-10",
+    "100",
+    "-100",
+    "127",                                          // Byte.MAX_VALUE
+    "127.1",
+    "127.0008",
+    "127.49",
+    "127.5",
+    "127.9999999999999999999",
+    "-127",
+    "-127.1",
+    "-127.0008",
+    "-127.49",
+    "-127.5",
+    "-127.999999",
+    "128",
+    "128.1",
+    "128.0008",
+    "128.49",
+    "128.5",
+    "128.9999999999999999999",
+    "-128",                                         // Byte.MIN_VALUE
+    "-128.1",
+    "-128.0008",
+    "-128.49",
+    "-128.5",
+    "-128.999",
+    "129",
+    "129.1",
+    "-129",
+    "-129.1",
+    "1000",
+    "-1000",
+    "10000",
+    "-10000",
+    "32767",                                        // Short.MAX_VALUE
+    "32767.1",
+    "32767.0008",
+    "32767.49",
+    "32767.5",
+    "32767.99999999999",
+    "-32767",
+    "-32767.1",
+    "-32767.0008",
+    "-32767.49",
+    "-32767.5",
+    "-32767.9",
+    "32768",
+    "32768.1",
+    "32768.0008",
+    "32768.49",
+    "32768.5",
+    "32768.9999999999",
+    "-32768",                                       // Short.MIN_VALUE
+    "-32768.1",
+    "-32768.0008",
+    "-32768.49",
+    "-32768.5",
+    "-32768.9999999",
+    "32769",
+    "32769.1",
+    "-32769",
+    "-32769.1",
+    "100000",
+    "-100000",
+    "1000000",
+    "-1000000",
+    "10000000",
+    "-10000000",
+    "100000000",
+    "99999999",                                     // 10^8 - 1
+    "-99999999",
+    "-100000000",
+    "1000000000",
+    "-1000000000",
+    "2147483647",                                  // Integer.MAX_VALUE
+    "2147483647.1",
+    "2147483647.0008",
+    "2147483647.49",
+    "2147483647.5",
+    "2147483647.9999999999",
+    "-2147483647",
+    "-2147483647.1",
+    "-2147483647.0008",
+    "-2147483647.49",
+    "-2147483647.5",
+    "-2147483647.9999999999999999999",
+    "2147483648",
+    "2147483648.1",
+    "2147483648.0008",
+    "2147483648.49",
+    "2147483648.5",
+    "2147483648.9",
+    "-2147483648",                                 // Integer.MIN_VALUE
+    "-2147483648.1",
+    "-2147483648.0008",
+    "-2147483648.49",
+    "-2147483648.5",
+    "-2147483648.999",
+    "2147483649",
+    "2147483649.1",
+    "-2147483649",
+    "-2147483649.1",
+    "10000000000",
+    "-10000000000",
+    "100000000000",
+    "-100000000000",
+    "1000000000000",
+    "-1000000000000",
+    "10000000000000",
+    "-10000000000000",
+    "100000000000000",
+    "-100000000000000",
+    "999999999999999",
+    "-999999999999999",
+    "1000000000000000",                            // 10^15
+    "-1000000000000000",
+    "9999999999999999",                            // 10^16 - 1
+    "-9999999999999999",
+    "10000000000000000",                           // 10^16
+    "-10000000000000000",
+    "100000000000000000",
+    "-100000000000000000",
+    "1000000000000000000",
+    "-1000000000000000000",
+    "9223372036854775807",                         // Long.MAX_VALUE
+    "9223372036854775807.1",
+    "9223372036854775807.0008",
+    "9223372036854775807.49",
+    "9223372036854775807.5",
+    "9223372036854775807.9",
+    "-9223372036854775807",
+    "-9223372036854775807.1",
+    "-9223372036854775807.0008",
+    "-9223372036854775807.49",
+    "-9223372036854775807.5",
+    "-9223372036854775807.9999999999999999999",
+    "-9223372036854775808",
+    "-9223372036854775808.1",
+    "9223372036854775808",
+    "9223372036854775808.1",
+    "9223372036854775808.0008",
+    "9223372036854775808.49",
+    "9223372036854775808.5",
+    "9223372036854775808.9",
+    "9223372036854775809",
+    "9223372036854775809.1",
+    "-9223372036854775808",                        // Long.MIN_VALUE
+    "-9223372036854775808.1",
+    "-9223372036854775808.0008",
+    "-9223372036854775808.49",
+    "-9223372036854775808.5",
+    "-9223372036854775808.9999999",
+    "9223372036854775809",
+    "9223372036854775809.1",
+    "-9223372036854775809",
+    "-9223372036854775809.1",
+    "10000000000000000000000000000000",            // 10^31
+    "-10000000000000000000000000000000",
+    "99999999999999999999999999999999",            // 10^32 - 1
+    "-99999999999999999999999999999999", 
+    "100000000000000000000000000000000",           // 10^32
+    "-100000000000000000000000000000000",
+    "10000000000000000000000000000000000000",      // 10^37
+    "-10000000000000000000000000000000000000",
+    "99999999999999999999999999999999999999",      // 10^38 - 1
+    "-99999999999999999999999999999999999999",
+    "100000000000000000000000000000000000000",     // 10^38
+    "-100000000000000000000000000000000000000", 
+    "1000000000000000000000000000000000000000",    // 10^39
+    "-1000000000000000000000000000000000000000",
+
+    "18446744073709551616",                        // Unsigned 64 max.
+    "-18446744073709551616",
+    "340282366920938463463374607431768211455",     // 2^128 - 1
+    "-340282366920938463463374607431768211455",
+
+    "0.999999999999999",
+    "-0.999999999999999",
+    "0.0000000000000001",                          // 10^-15
+    "-0.0000000000000001",
+    "0.9999999999999999",
+    "-0.9999999999999999",
+    "0.00000000000000001",                         // 10^-16
+    "-0.00000000000000001",
+    "0.00000000000000000000000000000001",          // 10^-31
+    "-0.00000000000000000000000000000001",
+    "0.99999999999999999999999999999999",          // 10^-32 + 1
+    "-0.99999999999999999999999999999999",
+    "0.000000000000000000000000000000001",         // 10^-32
+    "-0.000000000000000000000000000000001",
+    "0.00000000000000000000000000000000000001",    // 10^-37
+    "-0.00000000000000000000000000000000000001",
+    "0.99999999999999999999999999999999999999",    // 10^-38 + 1
+    "-0.99999999999999999999999999999999999999",
+    "0.000000000000000000000000000000000000001",   // 10^-38
+    "-0.000000000000000000000000000000000000001",
+    "0.0000000000000000000000000000000000000001",  // 10^-39
+    "-0.0000000000000000000000000000000000000001",
+    "0.0000000000000000000000000000000000000005",  // 10^-39  (rounds)
+    "-0.0000000000000000000000000000000000000005",
+    "0.340282366920938463463374607431768211455",   // (2^128 - 1) * 10^-39
+    "-0.340282366920938463463374607431768211455",
+    "0.000000000000000000000000000000000000001",   // 10^-38
+    "-0.000000000000000000000000000000000000001",
+    "0.000000000000000000000000000000000000005",   // 10^-38
+    "-0.000000000000000000000000000000000000005",
+
+    "234.79",
+    "342348.343",
+    "12.25",
+    "-12.25",
+    "72057594037927935",                           // 2^56 - 1
+    "-72057594037927935",
+    "72057594037927936",                           // 2^56
+    "-72057594037927936",
+    "5192296858534827628530496329220095",          // 2^56 * 2^56 - 1
+    "-5192296858534827628530496329220095",
+    "5192296858534827628530496329220096",          // 2^56 * 2^56
+    "-5192296858534827628530496329220096",
+
+    "54216721532321902598.70",
+    "-906.62545207002374150309544832320",
+    "-0.0709351061072",
+    "1460849063411925.53",
+    "8.809130E-33",
+    "-4.0786300706013636202E-20",
+    "-3.8823936518E-1",
+    "-3.8823936518E-28",
+    "-3.8823936518E-29",
+    "598575157855521918987423259.94094",
+    "299999448432.001342152474197",
+    "1786135888657847525803324040144343378.09799306448796128931113691624",  // More than 38 digits.
+    "-1786135888657847525803324040144343378.09799306448796128931113691624",
+    "57847525803324040144343378.09799306448796128931113691624",
+    "0.999999999999999999990000",
+    "005.34000",
+    "1E-90",
+
+    "0.4",
+    "-0.4",
+    "0.5",
+    "-0.5",
+    "0.6",
+    "-0.6",
+    "1.4",
+    "-1.4",
+    "1.5",
+    "-1.5",
+    "1.6",
+    "-1.6",
+    "2.4",
+    "-2.4",
+    "2.49",
+    "-2.49",
+    "2.5",
+    "-2.5",
+    "2.51",
+    "-2.51",
+    "-2.5",
+    "2.6",
+    "-2.6",
+    "3.00001415926",
+    "0.00",
+    "-12.25",
+    "234.79"
+  };
+
+  public static BigDecimal[] specialBigDecimals = stringArrayToBigDecimals(specialDecimalStrings);
+
+  // decimal_1_1.txt
+  public static String[] decimal_1_1_txt = {
+    "0.0",
+    "0.0000",
+    ".0",
+    "0.1",
+    "0.15",
+    "0.9",
+    "0.94",
+    "0.99",
+    "0.345",
+    "1.0",
+    "1",
+    "0",
+    "00",
+    "22",
+    "1E-9",
+    "-0.0",
+    "-0.0000",
+    "-.0",
+    "-0.1",
+    "-0.15",
+    "-0.9",
+    "-0.94",
+    "-0.99",
+    "-0.345",
+    "-1.0",
+    "-1",
+    "-0",
+    "-00",
+    "-22",
+    "-1E-9"
+  };
+
+  // kv7.txt KEYS
+  public static String[] kv7_txt_keys = {
+    "-4400",
+    "1E+99",
+    "1E-99",
+    "0",
+    "100",
+    "10",
+    "1",
+    "0.1",
+    "0.01",
+    "200",
+    "20",
+    "2",
+    "0",
+    "0.2",
+    "0.02",
+    "0.3",
+    "0.33",
+    "0.333",
+    "-0.3",
+    "-0.33",
+    "-0.333",
+    "1.0",
+    "2",
+    "3.14",
+    "-1.12",
+    "-1.12",
+    "-1.122",
+    "1.12",
+    "1.122",
+    "124.00",
+    "125.2",
+    "-1255.49",
+    "3.14",
+    "3.14",
+    "3.140",
+    "0.9999999999999999999999999",
+    "-1234567890.1234567890",
+    "1234567890.1234567800"
+  };
+
+  public static String standardAlphabet = "0123456789";
+
+  public static String[] sparseAlphabets = new String[] {
+
+    "0000000000000000000000000000000000000003",
+    "0000000000000000000000000000000000000009",
+    "0000000000000000000000000000000000000001",
+    "0000000000000000000003",
+    "0000000000000000000009",
+    "0000000000000000000001",
+    "0000000000091",
+    "000000000005",
+    "9",
+    "5555555555999999999000000000000001111111",
+    "24680",
+    "1"
+  };
+
+  public static BigDecimal[] stringArrayToBigDecimals(String[] strings) {
+    BigDecimal[] result = new BigDecimal[strings.length];
+    for (int i = 0; i < strings.length; i++) {
+      result[i] = new BigDecimal(strings[i]);
+    }
+    return result;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/common/src/test/org/apache/hadoop/hive/common/type/TestHiveDecimal.java
----------------------------------------------------------------------
diff --git a/common/src/test/org/apache/hadoop/hive/common/type/TestHiveDecimal.java b/common/src/test/org/apache/hadoop/hive/common/type/TestHiveDecimal.java
deleted file mode 100644
index f68842c..0000000
--- a/common/src/test/org/apache/hadoop/hive/common/type/TestHiveDecimal.java
+++ /dev/null
@@ -1,216 +0,0 @@
-/**
- * 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.common.type;
-
-import java.math.BigDecimal;
-import java.math.BigInteger;
-
-import com.google.code.tempusfugit.concurrency.annotations.*;
-import com.google.code.tempusfugit.concurrency.*;
-import org.junit.*;
-import static org.junit.Assert.*;
-
-public class TestHiveDecimal {
-
-  @Rule public ConcurrentRule concurrentRule = new ConcurrentRule();
-  @Rule public RepeatingRule repeatingRule = new RepeatingRule();
-
-  @Test
-  @Concurrent(count=4)
-  @Repeating(repetition=100)
-  public void testPrecisionScaleEnforcement() {
-    String decStr = "1786135888657847525803324040144343378.09799306448796128931113691624";
-    HiveDecimal dec = HiveDecimal.create(decStr);
-    Assert.assertEquals("1786135888657847525803324040144343378.1", dec.toString());
-    Assert.assertTrue("Decimal precision should not go above maximum",
-        dec.precision() <= HiveDecimal.MAX_PRECISION);
-    Assert.assertTrue("Decimal scale should not go above maximum", dec.scale() <= HiveDecimal.MAX_SCALE);
-
-    decStr = "57847525803324040144343378.09799306448796128931113691624";
-    HiveDecimal bd = HiveDecimal.create(decStr);
-    HiveDecimal bd1 = HiveDecimal.enforcePrecisionScale(bd, 20, 5);
-    Assert.assertNull(bd1);
-    bd1 = HiveDecimal.enforcePrecisionScale(bd, 35, 5);
-    Assert.assertEquals("57847525803324040144343378.09799", bd1.toString());
-    bd1 = HiveDecimal.enforcePrecisionScale(bd, 45, 20);
-    Assert.assertNull(bd1);
-
-    dec = HiveDecimal.create(new BigDecimal(decStr), false);
-    Assert.assertNull(dec);
-
-    dec = HiveDecimal.create("-1786135888657847525803324040144343378.09799306448796128931113691624");
-    Assert.assertEquals("-1786135888657847525803324040144343378.1", dec.toString());
-
-    dec = HiveDecimal.create("005.34000");
-    Assert.assertEquals(dec.precision(), 3);
-    Assert.assertEquals(dec.scale(), 2);
-
-    dec = HiveDecimal.create("178613588865784752580332404014434337809799306448796128931113691624");
-    Assert.assertNull(dec);
-
-    // Rounding numbers that increase int digits
-    Assert.assertEquals("10",
-        HiveDecimal.enforcePrecisionScale(HiveDecimal.create("9.5"), 2, 0).toString());
-    Assert.assertNull(HiveDecimal.enforcePrecisionScale(HiveDecimal.create("9.5"), 1, 0));
-    Assert.assertEquals("9",
-        HiveDecimal.enforcePrecisionScale(HiveDecimal.create("9.4"), 1, 0).toString());
-  }
-
-  @Test
-  @Concurrent(count=4)
-  @Repeating(repetition=100)
-  public void testTrailingZeroRemovalAfterEnforcement() {
-    String decStr = "8.090000000000000000000000000000000000000123456";
-    HiveDecimal dec = HiveDecimal.create(decStr);
-    Assert.assertEquals("8.09", dec.toString());
-  }
-
-  @Test
-  @Concurrent(count=4)
-  @Repeating(repetition=100)
-  public void testMultiply() {
-    HiveDecimal dec1 = HiveDecimal.create("0.00001786135888657847525803");
-    HiveDecimal dec2 = HiveDecimal.create("3.0000123456789");
-    Assert.assertNull(dec1.multiply(dec2));
-
-    dec1 = HiveDecimal.create("178613588865784752580323232232323444.4");
-    dec2 = HiveDecimal.create("178613588865784752580302323232.3");
-    Assert.assertNull(dec1.multiply(dec2));
-
-    dec1 = HiveDecimal.create("47.324");
-    dec2 = HiveDecimal.create("9232.309");
-    Assert.assertEquals("436909.791116", dec1.multiply(dec2).toString());
-
-    dec1 = HiveDecimal.create("3.140");
-    dec2 = HiveDecimal.create("1.00");
-    Assert.assertEquals("3.14", dec1.multiply(dec2).toString());
-
-    dec1 = HiveDecimal.create("43.010");
-    dec2 = HiveDecimal.create("2");
-    Assert.assertEquals("86.02", dec1.multiply(dec2).toString());
-  }
-
-  @Test
-  @Concurrent(count=4)
-  @Repeating(repetition=100)
-  public void testPow() {
-    HiveDecimal dec = HiveDecimal.create("3.00001415926");
-    Assert.assertEquals(dec.pow(2), dec.multiply(dec));
-
-    HiveDecimal dec1 = HiveDecimal.create("0.000017861358882");
-    dec1 = dec1.pow(3);
-    Assert.assertNull(dec1);
-
-    dec1 = HiveDecimal.create("3.140");
-    Assert.assertEquals("9.8596", dec1.pow(2).toString());
-  }
-
-  @Test
-  @Concurrent(count=4)
-  @Repeating(repetition=100)
-  public void testDivide() {
-    HiveDecimal dec1 = HiveDecimal.create("3.14");
-    HiveDecimal dec2 = HiveDecimal.create("3");
-    Assert.assertNotNull(dec1.divide(dec2));
-
-    dec1 = HiveDecimal.create("15");
-    dec2 = HiveDecimal.create("5");
-    Assert.assertEquals("3", dec1.divide(dec2).toString());
-
-    dec1 = HiveDecimal.create("3.140");
-    dec2 = HiveDecimal.create("1.00");
-    Assert.assertEquals("3.14", dec1.divide(dec2).toString());
-  }
-
-  @Test
-  @Concurrent(count=4)
-  @Repeating(repetition=100)
-  public void testPlus() {
-    HiveDecimal dec1 = HiveDecimal.create("99999999999999999999999999999999999");
-    HiveDecimal dec2 = HiveDecimal.create("1");
-    Assert.assertNotNull(dec1.add(dec2));
-
-    dec1 = HiveDecimal.create("3.140");
-    dec2 = HiveDecimal.create("1.00");
-    Assert.assertEquals("4.14", dec1.add(dec2).toString());
-  }
-
-
-  @Test
-  @Concurrent(count=4)
-  @Repeating(repetition=100)
-  public void testSubtract() {
-      HiveDecimal dec1 = HiveDecimal.create("3.140");
-      HiveDecimal dec2 = HiveDecimal.create("1.00");
-      Assert.assertEquals("2.14", dec1.subtract(dec2).toString());
-  }
-
-  @Test
-  @Concurrent(count=4)
-  @Repeating(repetition=100)
-  public void testPosMod() {
-    HiveDecimal hd1 = HiveDecimal.create("-100.91");
-    HiveDecimal hd2 = HiveDecimal.create("9.8");
-    HiveDecimal dec = hd1.remainder(hd2).add(hd2).remainder(hd2);
-    Assert.assertEquals("6.89", dec.toString());
-  }
-
-  @Test
-  @Concurrent(count=4)
-  @Repeating(repetition=100)
-  public void testHashCode() {
-      Assert.assertEquals(HiveDecimal.create("9").hashCode(), HiveDecimal.create("9.00").hashCode());
-      Assert.assertEquals(HiveDecimal.create("0").hashCode(), HiveDecimal.create("0.00").hashCode());
-  }
-
-  @Test
-  @Concurrent(count=4)
-  @Repeating(repetition=100)
-  public void testException() {
-    HiveDecimal dec = HiveDecimal.create("3.1415.926");
-    Assert.assertNull(dec);
-    dec = HiveDecimal.create("3abc43");
-    Assert.assertNull(dec);
-  }
-
-  @Test
-  @Concurrent(count=4)
-  @Repeating(repetition=100)
-  public void testBinaryConversion() {
-    testBinaryConversion("0.00");
-    testBinaryConversion("-12.25");
-    testBinaryConversion("234.79");
-  }
-
-  private void testBinaryConversion(String num) {
-    HiveDecimal dec = HiveDecimal.create(num);
-    int scale = 2;
-    byte[] d = dec.setScale(2).unscaledValue().toByteArray();
-    Assert.assertEquals(dec, HiveDecimal.create(new BigInteger(d), scale));
-    int prec = 5;
-    int len =  (int)
-        Math.ceil((Math.log(Math.pow(10, prec) - 1) / Math.log(2) + 1) / 8);
-    byte[] res = new byte[len];
-    if ( dec.signum() == -1)
-      for (int i = 0; i < len; i++)
-        res[i] |= 0xFF;
-    System.arraycopy(d, 0, res, len-d.length, d.length); // Padding leading zeros.
-    Assert.assertEquals(dec, HiveDecimal.create(new BigInteger(res), scale));
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/common/src/test/org/apache/hadoop/hive/common/type/TestHiveDecimalOrcSerializationUtils.java
----------------------------------------------------------------------
diff --git a/common/src/test/org/apache/hadoop/hive/common/type/TestHiveDecimalOrcSerializationUtils.java b/common/src/test/org/apache/hadoop/hive/common/type/TestHiveDecimalOrcSerializationUtils.java
new file mode 100644
index 0000000..882ff86
--- /dev/null
+++ b/common/src/test/org/apache/hadoop/hive/common/type/TestHiveDecimalOrcSerializationUtils.java
@@ -0,0 +1,378 @@
+/**
+ * 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.common.type;
+
+import java.sql.Timestamp;
+import java.util.Random;
+import java.util.Arrays;
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.math.BigDecimal;
+import java.math.BigInteger;
+
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritableV1;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.orc.impl.SerializationUtils;
+import org.apache.hadoop.hive.common.type.RandomTypeUtil;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.StringExpr;
+import org.apache.hadoop.hive.ql.util.TimestampUtils;
+
+import com.google.code.tempusfugit.concurrency.annotations.*;
+import com.google.code.tempusfugit.concurrency.*;
+
+import org.junit.*;
+
+import static org.junit.Assert.*;
+
+public class TestHiveDecimalOrcSerializationUtils extends HiveDecimalTestBase {
+
+  //------------------------------------------------------------------------------------------------
+
+  @Test
+  @Concurrent(count=4)
+  public void testSerializationUtilsWriteRead() {
+    testSerializationUtilsWriteRead("0.00");
+    testSerializationUtilsWriteRead("1");
+    testSerializationUtilsWriteRead("234.79");
+    testSerializationUtilsWriteRead("-12.25");
+    testSerializationUtilsWriteRead("99999999999999999999999999999999");
+    testSerializationUtilsWriteRead("-99999999999999999999999999999999");
+    testSerializationUtilsWriteRead("99999999999999999999999999999999999999");
+    //                               12345678901234567890123456789012345678
+    testSerializationUtilsWriteRead("-99999999999999999999999999999999999999");
+    testSerializationUtilsWriteRead("999999999999.99999999999999999999");
+    testSerializationUtilsWriteRead("-999999.99999999999999999999999999");
+    testSerializationUtilsWriteRead("9999999999999999999999.9999999999999999");
+    testSerializationUtilsWriteRead("-9999999999999999999999999999999.9999999");
+
+    testSerializationUtilsWriteRead("4611686018427387903");  // 2^62 - 1
+    testSerializationUtilsWriteRead("-4611686018427387903");
+    testSerializationUtilsWriteRead("4611686018427387904");  // 2^62
+    testSerializationUtilsWriteRead("-4611686018427387904");
+
+    testSerializationUtilsWriteRead("42535295865117307932921825928971026431");  // 2^62*2^63 - 1
+    testSerializationUtilsWriteRead("-42535295865117307932921825928971026431");
+    testSerializationUtilsWriteRead("42535295865117307932921825928971026432");  // 2^62*2^63
+    testSerializationUtilsWriteRead("-42535295865117307932921825928971026432");
+
+    testSerializationUtilsWriteRead("54216721532321902598.70");
+    testSerializationUtilsWriteRead("-906.62545207002374150309544832320");
+  }
+
+  private void testSerializationUtilsWriteRead(String string) {
+    // System.out.println("TEST_FAST_SERIALIZATION_UTILS_WRITE_BIG_INTEGER ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ START ~~~~~~~~~~~~~~~~~");
+
+    HiveDecimal dec = HiveDecimal.create(string);
+    assertTrue(dec != null);
+    // System.out.println("TEST_FAST_SERIALIZATION_UTILS_WRITE_BIG_INTEGER dec " + dec.toString());
+
+    BigInteger bigInteger = dec.unscaledValue();
+    int scale = dec.scale();
+    // System.out.println("TEST_FAST_SERIALIZATION_UTILS_WRITE_BIG_INTEGER bigInteger " + bigInteger.toString());
+    // System.out.println("TEST_FAST_SERIALIZATION_UTILS_WRITE_BIG_INTEGER scale " + scale);
+
+    //---------------------------------------------------
+    HiveDecimalV1 oldDec = HiveDecimalV1.create(string);
+    assertTrue(oldDec != null);
+    // System.out.println("TEST_FAST_SERIALIZATION_UTILS_WRITE_BIG_INTEGER oldDec " + oldDec.toString());
+
+    BigInteger oldBigInteger = oldDec.unscaledValue();
+    int oldScale = oldDec.scale();
+    // System.out.println("TEST_FAST_SERIALIZATION_UTILS_WRITE_BIG_INTEGER oldBigInteger " + oldBigInteger.toString());
+    // System.out.println("TEST_FAST_SERIALIZATION_UTILS_WRITE_BIG_INTEGER oldScale " + oldScale);
+    //---------------------------------------------------
+
+    long[] scratchLongs = new long[HiveDecimal.SCRATCH_LONGS_LEN];
+
+    int which = 0;
+    try {
+      ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
+      if (!dec.serializationUtilsWrite(
+          outputStream, scratchLongs)) {
+        // System.out.println("TEST_FAST_SERIALIZATION_UTILS_WRITE_BIG_INTEGER serializationUtilsWrite failed");
+        fail();
+      }
+      byte[] bytes = outputStream.toByteArray();
+  
+      ByteArrayOutputStream outputStreamExpected = new ByteArrayOutputStream();
+      SerializationUtils.writeBigInteger(outputStreamExpected, bigInteger);
+      byte[] bytesExpected = outputStreamExpected.toByteArray();
+  
+      // System.out.println("TEST_FAST_SERIALIZATION_UTILS_WRITE_BIG_INTEGER check streams");
+      // System.out.println("TEST_FAST_SERIALIZATION_UTILS_WRITE_BIG_INTEGER bytes1        " + displayBytes(bytes, 0, bytes.length));
+      if (!StringExpr.equal(bytes, 0, bytes.length, bytesExpected, 0, bytesExpected.length)) {
+        // Tailing zeroes difference ok.
+        // System.out.println("TEST_FAST_SERIALIZATION_UTILS_WRITE_BIG_INTEGER streams not equal");
+        // System.out.println("TEST_FAST_SERIALIZATION_UTILS_WRITE_BIG_INTEGER bytesExpected " + displayBytes(bytesExpected, 0, bytesExpected.length));
+      }
+      // Deserialize and check...
+      which = 1;
+      ByteArrayInputStream byteArrayInputStream = new ByteArrayInputStream(bytes);
+      BigInteger deserializedBigInteger = SerializationUtils.readBigInteger(byteArrayInputStream);
+
+      which = 2;
+      ByteArrayInputStream byteArrayInputStreamExpected = new ByteArrayInputStream(bytesExpected);
+      BigInteger deserializedBigIntegerExpected = SerializationUtils.readBigInteger(byteArrayInputStreamExpected);
+      // System.out.println("TEST_FAST_SERIALIZATION_UTILS_WRITE_BIG_INTEGER deserialized equals " +
+      //    deserializedBigInteger.equals(deserializedBigIntegerExpected));
+      if (!deserializedBigInteger.equals(deserializedBigIntegerExpected)) {
+        // System.out.println("TEST_FAST_SERIALIZATION_UTILS_WRITE_BIG_INTEGER deserializedBigInteger " + deserializedBigInteger.toString() +
+        //    " deserializedBigIntegerExpected " + deserializedBigIntegerExpected.toString());
+        fail();
+      }
+
+      which = 3;
+      ByteArrayInputStream byteArrayInputStreamRead = new ByteArrayInputStream(bytes);
+      byte[] scratchBytes = new byte[HiveDecimal.SCRATCH_BUFFER_LEN_SERIALIZATION_UTILS_READ];
+      HiveDecimal readHiveDecimal =
+          HiveDecimal.serializationUtilsRead(byteArrayInputStreamRead, scale, scratchBytes);
+      assertTrue(readHiveDecimal != null);
+      // System.out.println("TEST_FAST_SERIALIZATION_UTILS_WRITE_BIG_INTEGER read readHiveDecimal " + readHiveDecimal.toString() +
+      //    " dec " + dec.toString() + " (scale parameter " + scale + ")");
+      // System.out.println("TEST_FAST_SERIALIZATION_UTILS_WRITE_BIG_INTEGER read toString equals " +
+      //    readHiveDecimal.toString().equals(dec.toString()));
+      assertEquals(readHiveDecimal.toString(), dec.toString());
+      // System.out.println("TEST_FAST_SERIALIZATION_UTILS_WRITE_BIG_INTEGER read equals " +
+      //    readHiveDecimal.equals(dec));
+      assertEquals(readHiveDecimal, dec);
+    } catch (IOException e) {
+      // System.out.println("TEST_FAST_SERIALIZATION_UTILS_WRITE_BIG_INTEGER " + e + " which " + which);
+      fail();
+    }
+    // System.out.println("TEST_FAST_SERIALIZATION_UTILS_WRITE_BIG_INTEGER ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~  END  ~~~~~~~~~~~~~~~~~");
+
+  }
+
+  //------------------------------------------------------------------------------------------------
+
+  @Test
+  public void testRandomSerializationUtilsRead()
+      throws IOException {
+    doTestRandomSerializationUtilsRead(standardAlphabet);
+  }
+
+  @Test
+  public void testRandomSerializationUtilsReadSparse()
+      throws IOException {
+    for (String digitAlphabet : sparseAlphabets) {
+      doTestRandomSerializationUtilsRead(digitAlphabet);
+    }
+  }
+
+  private void doTestRandomSerializationUtilsRead(String digitAlphabet)
+      throws IOException {
+
+    Random r = new Random(2389);
+    for (int i = 0; i < POUND_FACTOR; i++) {
+      BigInteger bigInteger = randHiveBigInteger(r, digitAlphabet);
+
+      doTestSerializationUtilsRead(r, bigInteger);
+    }
+  }
+
+  @Test
+  public void testSerializationUtilsReadSpecial()
+      throws IOException {
+    Random r = new Random(9923);
+    for (BigDecimal bigDecimal : specialBigDecimals) {
+      doTestSerializationUtilsRead(r, bigDecimal.unscaledValue());
+    }
+  }
+
+  private void doTestSerializationUtilsRead(Random r, BigInteger bigInteger)
+     throws IOException {
+
+    // System.out.println("TEST_SERIALIZATION_UTILS_READ bigInteger " + bigInteger);
+
+    HiveDecimalV1 oldDec = HiveDecimalV1.create(bigInteger);
+    if (oldDec != null && isTenPowerBug(oldDec.toString())) {
+      return;
+    }
+    HiveDecimal dec = HiveDecimal.create(bigInteger);
+    if (oldDec == null) {
+      assertTrue(dec == null);
+      return;
+    }
+    assertTrue(dec != null);
+    dec.validate();
+    // System.out.println("TEST_SERIALIZATION_UTILS_READ oldDec " + oldDec);
+    // System.out.println("TEST_SERIALIZATION_UTILS_READ dec " + dec);
+
+    Assert.assertEquals(bigInteger, oldDec.unscaledValue());
+    Assert.assertEquals(bigInteger, dec.unscaledValue());
+
+    ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
+    SerializationUtils.writeBigInteger(outputStream, bigInteger);
+    byte[] bytes = outputStream.toByteArray();
+
+    ByteArrayInputStream byteArrayInputStream = new ByteArrayInputStream(bytes);
+    BigInteger deserializedBigInteger =
+        SerializationUtils.readBigInteger(byteArrayInputStream);
+
+    // Verify SerializationUtils first.
+    Assert.assertEquals(bigInteger, deserializedBigInteger);
+
+    // Now HiveDecimal
+    byte[] scratchBytes = new byte[HiveDecimal.SCRATCH_BUFFER_LEN_SERIALIZATION_UTILS_READ];
+
+    byteArrayInputStream = new ByteArrayInputStream(bytes);
+    HiveDecimal resultDec =
+        dec.serializationUtilsRead(
+            byteArrayInputStream, dec.scale(),
+            scratchBytes);
+    assertTrue(resultDec != null);
+    resultDec.validate();
+
+    Assert.assertEquals(dec.toString(), resultDec.toString());
+
+    //----------------------------------------------------------------------------------------------
+
+    // Add scale.
+
+    int scale = 0 + r.nextInt(38 + 1);
+    BigDecimal bigDecimal = new BigDecimal(bigInteger, scale);
+
+    oldDec = HiveDecimalV1.create(bigDecimal);
+    dec = HiveDecimal.create(bigDecimal);
+    if (oldDec == null) {
+      assertTrue(dec == null);
+      return;
+    }
+    assertTrue(dec != null);
+    dec.validate();
+    // System.out.println("TEST_SERIALIZATION_UTILS_READ with scale oldDec " + oldDec);
+    // System.out.println("TEST_SERIALIZATION_UTILS_READ with scale dec " + dec);
+
+    outputStream = new ByteArrayOutputStream();
+    SerializationUtils.writeBigInteger(outputStream, dec.unscaledValue());
+    bytes = outputStream.toByteArray();
+
+    // Now HiveDecimal
+    byteArrayInputStream = new ByteArrayInputStream(bytes);
+    resultDec =
+        dec.serializationUtilsRead(
+            byteArrayInputStream, dec.scale(),
+            scratchBytes);
+    assertTrue(resultDec != null);
+    resultDec.validate();
+
+    Assert.assertEquals(dec.toString(), resultDec.toString());
+  }
+
+  //------------------------------------------------------------------------------------------------
+
+  @Test
+  public void testRandomSerializationUtilsWrite()
+      throws IOException {
+    doTestRandomSerializationUtilsWrite(standardAlphabet, false);
+  }
+
+  @Test
+  public void testRandomSerializationUtilsWriteFractionsOnly()
+      throws IOException {
+    doTestRandomSerializationUtilsWrite(standardAlphabet, true);
+  }
+
+  @Test
+  public void testRandomSerializationUtilsWriteSparse()
+      throws IOException {
+    for (String digitAlphabet : sparseAlphabets) {
+      doTestRandomSerializationUtilsWrite(digitAlphabet, false);
+    }
+  }
+
+  private void doTestRandomSerializationUtilsWrite(String digitAlphabet, boolean fractionsOnly)
+      throws IOException {
+
+    Random r = new Random(823);
+    for (int i = 0; i < POUND_FACTOR; i++) {
+      BigInteger bigInteger = randHiveBigInteger(r, digitAlphabet);
+
+      doTestSerializationUtilsWrite(r, bigInteger);
+    }
+  }
+
+  @Test
+  public void testSerializationUtilsWriteSpecial()
+      throws IOException {
+    Random r = new Random(998737);
+    for (BigDecimal bigDecimal : specialBigDecimals) {
+      doTestSerializationUtilsWrite(r, bigDecimal.unscaledValue());
+    }
+  }
+
+  private void doTestSerializationUtilsWrite(Random r, BigInteger bigInteger)
+     throws IOException {
+
+    // System.out.println("TEST_SERIALIZATION_UTILS_WRITE bigInteger " + bigInteger);
+
+    HiveDecimalV1 oldDec = HiveDecimalV1.create(bigInteger);
+    if (oldDec != null && isTenPowerBug(oldDec.toString())) {
+      return;
+    }
+    HiveDecimal dec = HiveDecimal.create(bigInteger);
+    if (oldDec == null) {
+      assertTrue(dec == null);
+      return;
+    }
+    assertTrue(dec != null);
+    dec.validate();
+    // System.out.println("TEST_SERIALIZATION_UTILS_WRITE oldDec " + oldDec);
+    // System.out.println("TEST_SERIALIZATION_UTILS_WRITE dec " + dec);
+
+    Assert.assertEquals(bigInteger, oldDec.unscaledValue());
+    Assert.assertEquals(bigInteger, dec.unscaledValue());
+
+    ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
+    SerializationUtils.writeBigInteger(outputStream, bigInteger);
+    byte[] bytes = outputStream.toByteArray();
+
+    ByteArrayInputStream byteArrayInputStream = new ByteArrayInputStream(bytes);
+    BigInteger deserializedBigInteger =
+        SerializationUtils.readBigInteger(byteArrayInputStream);
+
+    // Verify SerializationUtils first.
+    Assert.assertEquals(bigInteger, deserializedBigInteger);
+
+    ByteArrayOutputStream decOutputStream = new ByteArrayOutputStream();
+
+    long[] scratchLongs = new long[HiveDecimal.SCRATCH_LONGS_LEN];
+
+    boolean successful =
+        dec.serializationUtilsWrite(
+            decOutputStream, scratchLongs);
+    Assert.assertTrue(successful);
+    byte[] decBytes = decOutputStream.toByteArray();
+
+    if (!StringExpr.equal(bytes, 0, bytes.length, decBytes, 0, decBytes.length)) {
+      // Tailing zeroes difference ok...
+      // System.out.println("TEST_SERIALIZATION_UTILS_WRITE streams not equal");
+      // System.out.println("TEST_SERIALIZATION_UTILS_WRITE bytes " + displayBytes(bytes, 0, bytes.length));
+      // System.out.println("TEST_SERIALIZATION_UTILS_WRITE decBytes " + displayBytes(decBytes, 0, decBytes.length));
+    }
+
+    ByteArrayInputStream decByteArrayInputStream = new ByteArrayInputStream(decBytes);
+    BigInteger decDeserializedBigInteger =
+        SerializationUtils.readBigInteger(decByteArrayInputStream);
+
+    Assert.assertEquals(bigInteger, decDeserializedBigInteger);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/orc/src/java/org/apache/orc/impl/ColumnStatisticsImpl.java
----------------------------------------------------------------------
diff --git a/orc/src/java/org/apache/orc/impl/ColumnStatisticsImpl.java b/orc/src/java/org/apache/orc/impl/ColumnStatisticsImpl.java
index 745ed9a..1118c5c 100644
--- a/orc/src/java/org/apache/orc/impl/ColumnStatisticsImpl.java
+++ b/orc/src/java/org/apache/orc/impl/ColumnStatisticsImpl.java
@@ -20,6 +20,7 @@ package org.apache.orc.impl;
 import java.sql.Date;
 import java.sql.Timestamp;
 
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
 import org.apache.hadoop.hive.serde2.io.DateWritable;
 import org.apache.hadoop.io.BytesWritable;
@@ -573,9 +574,11 @@ public class ColumnStatisticsImpl implements ColumnStatistics {
 
   private static final class DecimalStatisticsImpl extends ColumnStatisticsImpl
       implements DecimalColumnStatistics {
-    private HiveDecimal minimum = null;
-    private HiveDecimal maximum = null;
-    private HiveDecimal sum = HiveDecimal.ZERO;
+
+    // These objects are mutable for better performance.
+    private HiveDecimalWritable minimum = null;
+    private HiveDecimalWritable maximum = null;
+    private HiveDecimalWritable sum = new HiveDecimalWritable(0);
 
     DecimalStatisticsImpl() {
     }
@@ -584,13 +587,13 @@ public class ColumnStatisticsImpl implements ColumnStatistics {
       super(stats);
       OrcProto.DecimalStatistics dec = stats.getDecimalStatistics();
       if (dec.hasMaximum()) {
-        maximum = HiveDecimal.create(dec.getMaximum());
+        maximum = new HiveDecimalWritable(dec.getMaximum());
       }
       if (dec.hasMinimum()) {
-        minimum = HiveDecimal.create(dec.getMinimum());
+        minimum = new HiveDecimalWritable(dec.getMinimum());
       }
       if (dec.hasSum()) {
-        sum = HiveDecimal.create(dec.getSum());
+        sum = new HiveDecimalWritable(dec.getSum());
       } else {
         sum = null;
       }
@@ -601,21 +604,21 @@ public class ColumnStatisticsImpl implements ColumnStatistics {
       super.reset();
       minimum = null;
       maximum = null;
-      sum = HiveDecimal.ZERO;
+      sum = new HiveDecimalWritable(0);
     }
 
     @Override
-    public void updateDecimal(HiveDecimal value) {
+    public void updateDecimal(HiveDecimalWritable value) {
       if (minimum == null) {
-        minimum = value;
-        maximum = value;
+        minimum = new HiveDecimalWritable(value);
+        maximum = new HiveDecimalWritable(value);
       } else if (minimum.compareTo(value) > 0) {
-        minimum = value;
+        minimum.set(value);
       } else if (maximum.compareTo(value) < 0) {
-        maximum = value;
+        maximum.set(value);
       }
       if (sum != null) {
-        sum = sum.add(value);
+        sum.mutateAdd(value);
       }
     }
 
@@ -624,20 +627,20 @@ public class ColumnStatisticsImpl implements ColumnStatistics {
       if (other instanceof DecimalStatisticsImpl) {
         DecimalStatisticsImpl dec = (DecimalStatisticsImpl) other;
         if (minimum == null) {
-          minimum = dec.minimum;
-          maximum = dec.maximum;
+          minimum = (dec.minimum != null ? new HiveDecimalWritable(dec.minimum) : null);
+          maximum = (dec.maximum != null ? new HiveDecimalWritable(dec.maximum) : null);
           sum = dec.sum;
         } else if (dec.minimum != null) {
           if (minimum.compareTo(dec.minimum) > 0) {
-            minimum = dec.minimum;
+            minimum.set(dec.minimum);
           }
           if (maximum.compareTo(dec.maximum) < 0) {
-            maximum = dec.maximum;
+            maximum.set(dec.maximum);
           }
           if (sum == null || dec.sum == null) {
             sum = null;
           } else {
-            sum = sum.add(dec.sum);
+            sum.mutateAdd(dec.sum);
           }
         }
       } else {
@@ -657,7 +660,8 @@ public class ColumnStatisticsImpl implements ColumnStatistics {
         dec.setMinimum(minimum.toString());
         dec.setMaximum(maximum.toString());
       }
-      if (sum != null) {
+      // Check isSet for overflow.
+      if (sum != null && sum.isSet()) {
         dec.setSum(sum.toString());
       }
       result.setDecimalStatistics(dec);
@@ -666,17 +670,17 @@ public class ColumnStatisticsImpl implements ColumnStatistics {
 
     @Override
     public HiveDecimal getMinimum() {
-      return minimum;
+      return minimum.getHiveDecimal();
     }
 
     @Override
     public HiveDecimal getMaximum() {
-      return maximum;
+      return maximum.getHiveDecimal();
     }
 
     @Override
     public HiveDecimal getSum() {
-      return sum;
+      return sum.getHiveDecimal();
     }
 
     @Override
@@ -987,7 +991,7 @@ public class ColumnStatisticsImpl implements ColumnStatistics {
     throw new UnsupportedOperationException("Can't update string");
   }
 
-  public void updateDecimal(HiveDecimal value) {
+  public void updateDecimal(HiveDecimalWritable value) {
     throw new UnsupportedOperationException("Can't update decimal");
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/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 5d5f991..e60075f 100644
--- a/orc/src/java/org/apache/orc/impl/ConvertTreeReaderFactory.java
+++ b/orc/src/java/org/apache/orc/impl/ConvertTreeReaderFactory.java
@@ -608,19 +608,55 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
       setConvertTreeReader(decimalTreeReader);
     }
 
-    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 {
-      HiveDecimal decimalValue = decimalColVector.vector[elementNum].getHiveDecimal();
-      if (decimalValue.compareTo(DECIMAL_MAX_LONG) > 0 ||
-          decimalValue.compareTo(DECIMAL_MIN_LONG) < 0) {
+      HiveDecimalWritable decWritable = decimalColVector.vector[elementNum];
+      long[] vector = longColVector.vector;
+      Category readerCategory = readerType.getCategory();
+
+      // Check to see if the decimal will fit in the Hive integer data type.
+      // If not, set the element to null.
+      boolean isInRange;
+      switch (readerCategory) {
+      case BOOLEAN:
+        // No data loss for boolean.
+        vector[elementNum] = decWritable.signum() == 0 ? 0 : 1;
+        return;
+      case BYTE:
+        isInRange = decWritable.isByte();
+        break;
+      case SHORT:
+        isInRange = decWritable.isShort();
+        break;
+      case INT:
+        isInRange = decWritable.isInt();
+        break;
+      case LONG:
+        isInRange = decWritable.isLong();
+        break;
+      default:
+        throw new RuntimeException("Unexpected type kind " + readerCategory.name());
+      }
+      if (!isInRange) {
         longColVector.isNull[elementNum] = true;
         longColVector.noNulls = false;
-      } else {
-        // TODO: lossy conversion!
-        downCastAnyInteger(longColVector, elementNum, decimalValue.longValue(), readerType);
+        return;
+      }
+      switch (readerCategory) {
+      case BYTE:
+        vector[elementNum] = decWritable.byteValue();
+        break;
+      case SHORT:
+        vector[elementNum] = decWritable.shortValue();
+        break;
+      case INT:
+        vector[elementNum] = decWritable.intValue();
+        break;
+      case LONG:
+        vector[elementNum] = decWritable.longValue();
+        break;
+      default:
+        throw new RuntimeException("Unexpected type kind " + readerCategory.name());
       }
     }
 
@@ -828,7 +864,7 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
     @Override
     public void setConvertVectorElement(int elementNum) throws IOException {
       doubleColVector.vector[elementNum] =
-          (float) decimalColVector.vector[elementNum].getHiveDecimal().doubleValue();
+          (float) decimalColVector.vector[elementNum].doubleValue();
     }
 
     @Override
@@ -1034,7 +1070,7 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
     @Override
     public void setConvertVectorElement(int elementNum) throws IOException {
       doubleColVector.vector[elementNum] =
-          decimalColVector.vector[elementNum].getHiveDecimal().doubleValue();
+          decimalColVector.vector[elementNum].doubleValue();
     }
 
     @Override
@@ -1371,14 +1407,8 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
     @Override
     public void setConvertVectorElement(int elementNum) throws IOException {
 
-      HiveDecimalWritable valueWritable = HiveDecimalWritable.enforcePrecisionScale(
-          fileDecimalColVector.vector[elementNum], readerPrecision, readerScale);
-      if (valueWritable != null) {
-        decimalColVector.set(elementNum, valueWritable);
-      } else {
-        decimalColVector.noNulls = false;
-        decimalColVector.isNull[elementNum] = true;
-      }
+      decimalColVector.set(elementNum, fileDecimalColVector.vector[elementNum]);
+
     }
 
     @Override
@@ -1540,6 +1570,7 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
     private final TypeDescription readerType;
     private DecimalColumnVector decimalColVector;
     private BytesColumnVector bytesColVector;
+    private byte[] scratchBuffer;
 
     StringGroupFromDecimalTreeReader(int columnId, TypeDescription fileType,
         TypeDescription readerType, boolean skipCorrupt) throws IOException {
@@ -1549,13 +1580,19 @@ public class ConvertTreeReaderFactory extends TreeReaderFactory {
       this.readerType = readerType;
       decimalTreeReader = new DecimalTreeReader(columnId, precision, scale);
       setConvertTreeReader(decimalTreeReader);
+      scratchBuffer = new byte[HiveDecimal.SCRATCH_BUFFER_LEN_TO_BYTES];
     }
 
     @Override
     public void setConvertVectorElement(int elementNum) {
-      String string = decimalColVector.vector[elementNum].getHiveDecimal().toString();
-      byte[] bytes = string.getBytes();
-      assignStringGroupVectorEntry(bytesColVector, elementNum, readerType, bytes);
+      HiveDecimalWritable decWritable = decimalColVector.vector[elementNum];
+
+      // Convert decimal into bytes instead of a String for better performance.
+      final int byteIndex = decWritable.toBytes(scratchBuffer);
+
+      assignStringGroupVectorEntry(
+          bytesColVector, elementNum, readerType,
+          scratchBuffer, byteIndex, HiveDecimal.SCRATCH_BUFFER_LEN_TO_BYTES - byteIndex);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/orc/src/java/org/apache/orc/impl/TreeReaderFactory.java
----------------------------------------------------------------------
diff --git a/orc/src/java/org/apache/orc/impl/TreeReaderFactory.java b/orc/src/java/org/apache/orc/impl/TreeReaderFactory.java
index 484209b..3ddafba 100644
--- a/orc/src/java/org/apache/orc/impl/TreeReaderFactory.java
+++ b/orc/src/java/org/apache/orc/impl/TreeReaderFactory.java
@@ -40,6 +40,7 @@ import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.UnionColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.StringExpr;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 import org.apache.orc.TypeDescription;
 import org.apache.orc.OrcProto;
 
@@ -1044,6 +1045,7 @@ public class TreeReaderFactory {
     protected InStream valueStream;
     protected IntegerReader scaleReader = null;
     private int[] scratchScaleVector;
+    private byte[] scratchBytes;
 
     private final int precision;
     private final int scale;
@@ -1060,6 +1062,7 @@ public class TreeReaderFactory {
       this.scale = scale;
       this.scratchScaleVector = new int[VectorizedRowBatch.DEFAULT_SIZE];
       this.valueStream = valueStream;
+      this.scratchBytes = new byte[HiveDecimal.SCRATCH_BUFFER_LEN_SERIALIZATION_UTILS_READ];
       if (scaleStream != null && encoding != null) {
         checkEncoding(encoding);
         this.scaleReader = createIntegerReader(encoding.getKind(), scaleStream, true, false);
@@ -1112,18 +1115,30 @@ public class TreeReaderFactory {
       // read the scales
       scaleReader.nextVector(result, scratchScaleVector, batchSize);
       // Read value entries based on isNull entries
+      // Use the fast ORC deserialization method that emulates SerializationUtils.readBigInteger
+      // provided by HiveDecimalWritable.
+      HiveDecimalWritable[] vector = result.vector;
+      HiveDecimalWritable decWritable;
       if (result.noNulls) {
         for (int r=0; r < batchSize; ++r) {
-          BigInteger bInt = SerializationUtils.readBigInteger(valueStream);
-          HiveDecimal dec = HiveDecimal.create(bInt, scratchScaleVector[r]);
-          result.set(r, dec);
+          decWritable = vector[r];
+          if (!decWritable.serializationUtilsRead(
+              valueStream, scratchScaleVector[r],
+              scratchBytes)) {
+            result.isNull[r] = true;
+            result.noNulls = false;
+          }
         }
       } else if (!result.isRepeating || !result.isNull[0]) {
         for (int r=0; r < batchSize; ++r) {
           if (!result.isNull[r]) {
-            BigInteger bInt = SerializationUtils.readBigInteger(valueStream);
-            HiveDecimal dec = HiveDecimal.create(bInt, scratchScaleVector[r]);
-            result.set(r, dec);
+            decWritable = vector[r];
+            if (!decWritable.serializationUtilsRead(
+                valueStream, scratchScaleVector[r],
+                scratchBytes)) {
+              result.isNull[r] = true;
+              result.noNulls = false;
+            }
           }
         }
       }
@@ -1132,8 +1147,9 @@ public class TreeReaderFactory {
     @Override
     void skipRows(long items) throws IOException {
       items = countNonNulls(items);
+      HiveDecimalWritable scratchDecWritable = new HiveDecimalWritable();
       for (int i = 0; i < items; i++) {
-        SerializationUtils.readBigInteger(valueStream);
+        scratchDecWritable.serializationUtilsRead(valueStream, 0, scratchBytes);
       }
       scaleReader.skip(items);
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/orc/src/java/org/apache/orc/impl/WriterImpl.java
----------------------------------------------------------------------
diff --git a/orc/src/java/org/apache/orc/impl/WriterImpl.java b/orc/src/java/org/apache/orc/impl/WriterImpl.java
index b17fb41..518a5f7 100644
--- a/orc/src/java/org/apache/orc/impl/WriterImpl.java
+++ b/orc/src/java/org/apache/orc/impl/WriterImpl.java
@@ -30,6 +30,7 @@ import java.util.Map;
 import java.util.TimeZone;
 import java.util.TreeMap;
 
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 import org.apache.hadoop.hive.ql.util.JavaDataModel;
 import org.apache.orc.BinaryColumnStatistics;
 import org.apache.orc.BloomFilterIO;
@@ -1610,6 +1611,11 @@ public class WriterImpl implements Writer, MemoryManager.Callback {
 
   private static class DecimalTreeWriter extends TreeWriter {
     private final PositionedOutputStream valueStream;
+
+    // These scratch buffers allow us to serialize decimals much faster.
+    private final long[] scratchLongs;
+    private final byte[] scratchBuffer;
+
     private final IntegerWriter scaleStream;
     private final boolean isDirectV2;
 
@@ -1620,6 +1626,8 @@ public class WriterImpl implements Writer, MemoryManager.Callback {
       super(columnId, schema, writer, nullable);
       this.isDirectV2 = isNewWriteFormat(writer);
       valueStream = writer.createStream(id, OrcProto.Stream.Kind.DATA);
+      scratchLongs = new long[HiveDecimal.SCRATCH_LONGS_LEN];
+      scratchBuffer = new byte[HiveDecimal.SCRATCH_BUFFER_LEN_TO_BYTES];
       this.scaleStream = createIntegerWriter(writer.createStream(id,
           OrcProto.Stream.Kind.SECONDARY), true, isDirectV2, writer);
       recordPosition(rowIndexPosition);
@@ -1642,27 +1650,36 @@ public class WriterImpl implements Writer, MemoryManager.Callback {
       DecimalColumnVector vec = (DecimalColumnVector) vector;
       if (vector.isRepeating) {
         if (vector.noNulls || !vector.isNull[0]) {
-          HiveDecimal value = vec.vector[0].getHiveDecimal();
+          HiveDecimalWritable value = vec.vector[0];
           indexStatistics.updateDecimal(value);
           if (createBloomFilter) {
-            bloomFilter.addString(value.toString());
+
+            // The HiveDecimalWritable toString() method with a scratch buffer for good performance
+            // when creating the String.  We need to use a String hash code and not UTF-8 byte[]
+            // hash code in order to get the right hash code.
+            bloomFilter.addString(value.toString(scratchBuffer));
           }
           for(int i=0; i < length; ++i) {
-            SerializationUtils.writeBigInteger(valueStream,
-                value.unscaledValue());
+
+            // Use the fast ORC serialization method that emulates SerializationUtils.writeBigInteger
+            // provided by HiveDecimalWritable.
+            value.serializationUtilsWrite(
+                valueStream,
+                scratchLongs);
             scaleStream.write(value.scale());
           }
         }
       } else {
         for(int i=0; i < length; ++i) {
           if (vec.noNulls || !vec.isNull[i + offset]) {
-            HiveDecimal value = vec.vector[i + offset].getHiveDecimal();
-            SerializationUtils.writeBigInteger(valueStream,
-                value.unscaledValue());
+            HiveDecimalWritable value = vec.vector[i + offset];
+            value.serializationUtilsWrite(
+                valueStream,
+                scratchLongs);
             scaleStream.write(value.scale());
             indexStatistics.updateDecimal(value);
             if (createBloomFilter) {
-              bloomFilter.addString(value.toString());
+              bloomFilter.addString(value.toString(scratchBuffer));
             }
           }
         }

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/orc/src/test/org/apache/orc/TestColumnStatistics.java
----------------------------------------------------------------------
diff --git a/orc/src/test/org/apache/orc/TestColumnStatistics.java b/orc/src/test/org/apache/orc/TestColumnStatistics.java
index 1837dbb..93d4bdb 100644
--- a/orc/src/test/org/apache/orc/TestColumnStatistics.java
+++ b/orc/src/test/org/apache/orc/TestColumnStatistics.java
@@ -30,6 +30,7 @@ import java.util.List;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
 import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
@@ -170,17 +171,17 @@ public class TestColumnStatistics {
 
     ColumnStatisticsImpl stats1 = ColumnStatisticsImpl.create(schema);
     ColumnStatisticsImpl stats2 = ColumnStatisticsImpl.create(schema);
-    stats1.updateDecimal(HiveDecimal.create(10));
-    stats1.updateDecimal(HiveDecimal.create(100));
-    stats2.updateDecimal(HiveDecimal.create(1));
-    stats2.updateDecimal(HiveDecimal.create(1000));
+    stats1.updateDecimal(new HiveDecimalWritable(10));
+    stats1.updateDecimal(new HiveDecimalWritable(100));
+    stats2.updateDecimal(new HiveDecimalWritable(1));
+    stats2.updateDecimal(new HiveDecimalWritable(1000));
     stats1.merge(stats2);
     DecimalColumnStatistics typed = (DecimalColumnStatistics) stats1;
     assertEquals(1, typed.getMinimum().longValue());
     assertEquals(1000, typed.getMaximum().longValue());
     stats1.reset();
-    stats1.updateDecimal(HiveDecimal.create(-10));
-    stats1.updateDecimal(HiveDecimal.create(10000));
+    stats1.updateDecimal(new HiveDecimalWritable(-10));
+    stats1.updateDecimal(new HiveDecimalWritable(10000));
     stats1.merge(stats2);
     assertEquals(-10, typed.getMinimum().longValue());
     assertEquals(10000, typed.getMaximum().longValue());

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFMinMaxDecimal.txt
----------------------------------------------------------------------
diff --git a/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFMinMaxDecimal.txt b/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFMinMaxDecimal.txt
index 9a48171..b532e2f 100644
--- a/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFMinMaxDecimal.txt
+++ b/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFMinMaxDecimal.txt
@@ -61,12 +61,11 @@ public class <ClassName> extends VectorAggregateExpression {
       }
 
       public void checkValue(HiveDecimalWritable writable, short scale) {
-        HiveDecimal value = writable.getHiveDecimal();
         if (isNull) {
           isNull = false;
-          this.value.set(value);
-        } else if (this.value.getHiveDecimal().compareTo(value) <OperatorSymbol> 0) {
-          this.value.set(value);
+          this.value.set(writable);
+        } else if (this.value.compareTo(writable) <OperatorSymbol> 0) {
+          this.value.set(writable);
         }
       }
 
@@ -321,8 +320,7 @@ public class <ClassName> extends VectorAggregateExpression {
           if (inputVector.noNulls &&
             (myagg.isNull || (myagg.value.compareTo(vector[0]) <OperatorSymbol> 0))) {
             myagg.isNull = false;
-            HiveDecimal value = vector[0].getHiveDecimal();
-            myagg.value.set(value);
+            myagg.value.set(vector[0]);
           }
           return;
         }
@@ -354,13 +352,13 @@ public class <ClassName> extends VectorAggregateExpression {
       for (int j=0; j< batchSize; ++j) {
         int i = selected[j];
         if (!isNull[i]) {
-          HiveDecimal value = vector[i].getHiveDecimal();
+          HiveDecimalWritable writable = vector[i];
           if (myagg.isNull) {
             myagg.isNull = false;
-            myagg.value.set(value);
+            myagg.value.set(writable);
           }
-          else if (myagg.value.getHiveDecimal().compareTo(value) <OperatorSymbol> 0) {
-            myagg.value.set(value);
+          else if (myagg.value.compareTo(writable) <OperatorSymbol> 0) {
+            myagg.value.set(writable);
           }
         }
       }
@@ -374,15 +372,14 @@ public class <ClassName> extends VectorAggregateExpression {
         int[] selected) {
 
       if (myagg.isNull) {
-        HiveDecimal value = vector[selected[0]].getHiveDecimal();
-        myagg.value.set(value);
+        myagg.value.set(vector[selected[0]]);
         myagg.isNull = false;
       }
 
       for (int i=0; i< batchSize; ++i) {
-        HiveDecimal value = vector[selected[i]].getHiveDecimal();
-        if (myagg.value.getHiveDecimal().compareTo(value) <OperatorSymbol> 0) {
-          myagg.value.set(value);
+        HiveDecimalWritable writable = vector[selected[i]];
+        if (myagg.value.compareTo(writable) <OperatorSymbol> 0) {
+          myagg.value.set(writable);
         }
       }
     }
@@ -396,13 +393,13 @@ public class <ClassName> extends VectorAggregateExpression {
 
       for(int i=0;i<batchSize;++i) {
         if (!isNull[i]) {
-          HiveDecimal value = vector[i].getHiveDecimal();
+          HiveDecimalWritable writable = vector[i];
           if (myagg.isNull) {
-            myagg.value.set(value);
+            myagg.value.set(writable);
             myagg.isNull = false;
           }
-          else if (myagg.value.getHiveDecimal().compareTo(value) <OperatorSymbol> 0) {
-            myagg.value.set(value);
+          else if (myagg.value.compareTo(writable) <OperatorSymbol> 0) {
+            myagg.value.set(writable);
           }
         }
       }
@@ -414,15 +411,14 @@ public class <ClassName> extends VectorAggregateExpression {
         short scale,
         int batchSize) {
       if (myagg.isNull) {
-        HiveDecimal value = vector[0].getHiveDecimal();
-        myagg.value.set(value);
+        myagg.value.set(vector[0]);
         myagg.isNull = false;
       }
 
       for (int i=0;i<batchSize;++i) {
-        HiveDecimal value = vector[i].getHiveDecimal();
-        if (myagg.value.getHiveDecimal().compareTo(value) <OperatorSymbol> 0) {
-          myagg.value.set(value);
+        HiveDecimalWritable writable = vector[i];
+        if (myagg.value.compareTo(writable) <OperatorSymbol> 0) {
+          myagg.value.set(writable);
         }
       }
     }
@@ -472,4 +468,3 @@ public class <ClassName> extends VectorAggregateExpression {
     this.inputExpression = inputExpression;
   }
 }
-

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorDeserializeRow.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorDeserializeRow.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorDeserializeRow.java
index d31d338..fc82cf7 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorDeserializeRow.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorDeserializeRow.java
@@ -534,8 +534,9 @@ public final class VectorDeserializeRow<T extends DeserializeRead> {
           }
           break;
         case DECIMAL:
+          // The DecimalColumnVector set method will quickly copy the deserialized decimal writable fields.
           ((DecimalColumnVector) batch.cols[projectionColumnNum]).set(
-              batchIndex, deserializeRead.currentHiveDecimalWritable.getHiveDecimal());
+              batchIndex, deserializeRead.currentHiveDecimalWritable);
           break;
         case INTERVAL_YEAR_MONTH:
           ((LongColumnVector) batch.cols[projectionColumnNum]).vector[batchIndex] =

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorExtractRow.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorExtractRow.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorExtractRow.java
index e6dc9ec..631dcb2 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorExtractRow.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorExtractRow.java
@@ -310,8 +310,9 @@ public class VectorExtractRow {
             return primitiveWritable;
           }
         case DECIMAL:
+          // The HiveDecimalWritable set method will quickly copy the deserialized decimal writable fields.
           ((HiveDecimalWritable) primitiveWritable).set(
-              ((DecimalColumnVector) batch.cols[projectionColumnNum]).vector[adjustedIndex].getHiveDecimal());
+              ((DecimalColumnVector) batch.cols[projectionColumnNum]).vector[adjustedIndex]);
           return primitiveWritable;
         case INTERVAL_YEAR_MONTH:
           ((HiveIntervalYearMonthWritable) primitiveWritable).set(

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorHashKeyWrapper.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorHashKeyWrapper.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorHashKeyWrapper.java
index 8a101a6..2bd1850 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorHashKeyWrapper.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorHashKeyWrapper.java
@@ -109,7 +109,8 @@ public class VectorHashKeyWrapper extends KeyWrapper {
         Arrays.hashCode(isNull);
 
     for (int i = 0; i < decimalValues.length; i++) {
-      hashcode ^= decimalValues[i].getHiveDecimal().hashCode();
+      // Use the new faster hash code since we are hashing memory objects.
+      hashcode ^= decimalValues[i].newFasterHashCode();
     }
 
     for (int i = 0; i < timestampValues.length; i++) {

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorHashKeyWrapperBatch.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorHashKeyWrapperBatch.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorHashKeyWrapperBatch.java
index bfd26ae..b4708b5 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorHashKeyWrapperBatch.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorHashKeyWrapperBatch.java
@@ -773,7 +773,7 @@ public class VectorHashKeyWrapperBatch extends VectorColumnSetInfo {
     } else if (klh.decimalIndex >= 0) {
       return kw.getIsDecimalNull(klh.decimalIndex)? null :
           keyOutputWriter.writeValue(
-                kw.getDecimal(klh.decimalIndex).getHiveDecimal());
+                kw.getDecimal(klh.decimalIndex));
     } else if (klh.timestampIndex >= 0) {
       return kw.getIsTimestampNull(klh.timestampIndex)? null :
           keyOutputWriter.writeValue(

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorSerializeRow.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorSerializeRow.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorSerializeRow.java
index 6af3d99..a95098a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorSerializeRow.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorSerializeRow.java
@@ -497,7 +497,10 @@ public final class VectorSerializeRow<T extends SerializeWrite> {
 
       if (colVector.isRepeating) {
         if (colVector.noNulls || !colVector.isNull[0]) {
-          serializeWrite.writeHiveDecimal(colVector.vector[0].getHiveDecimal(), colVector.scale);
+          // We serialize specifying the HiveDecimalWritable but also the desired
+          // serialization scale that will be used by text serialization for adding
+          // trailing fractional zeroes.
+          serializeWrite.writeHiveDecimal(colVector.vector[0], colVector.scale);
           return true;
         } else {
           serializeWrite.writeNull();
@@ -505,7 +508,7 @@ public final class VectorSerializeRow<T extends SerializeWrite> {
         }
       } else {
         if (colVector.noNulls || !colVector.isNull[batchIndex]) {
-          serializeWrite.writeHiveDecimal(colVector.vector[batchIndex].getHiveDecimal(), colVector.scale);
+          serializeWrite.writeHiveDecimal(colVector.vector[batchIndex], colVector.scale);
           return true;
         } else {
           serializeWrite.writeNull();


[07/10] hive git commit: HIVE-15335: Fast Decimal (Matt McCline, reviewed by Sergey Shelukhin, Prasanth Jayachandran, Owen O'Malley)

Posted by mm...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/serde/src/java/org/apache/hadoop/hive/serde2/binarysortable/BinarySortableSerDe.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/binarysortable/BinarySortableSerDe.java b/serde/src/java/org/apache/hadoop/hive/serde2/binarysortable/BinarySortableSerDe.java
index 5e119d7..0a035c6 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/binarysortable/BinarySortableSerDe.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/binarysortable/BinarySortableSerDe.java
@@ -29,6 +29,7 @@ import java.util.Map;
 import java.util.Properties;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.common.type.HiveDecimalV1;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
 import org.apache.hadoop.hive.common.type.HiveIntervalDayTime;
 import org.apache.hadoop.hive.common.type.HiveIntervalYearMonth;
@@ -797,18 +798,6 @@ public class BinarySortableSerDe extends AbstractSerDe {
         return;
       }
       case DECIMAL: {
-        // decimals are encoded in three pieces:
-        // sign: 1, 2 or 3 for smaller, equal or larger than 0 respectively
-        // factor: Number that indicates the amount of digits you have to move
-        // the decimal point left or right until the resulting number is smaller
-        // than zero but has something other than 0 as the first digit.
-        // digits: which is a string of all the digits in the decimal. If the number
-        // is negative the binary string will be inverted to get the correct ordering.
-        // Example: 0.00123
-        // Sign is 3 (bigger than 0)
-        // Factor is -2 (move decimal point 2 positions right)
-        // Digits are: 123
-
         HiveDecimalObjectInspector boi = (HiveDecimalObjectInspector) poi;
         HiveDecimal dec = boi.getPrimitiveJavaObject(o);
         serializeHiveDecimal(buffer, dec, invert);
@@ -980,22 +969,22 @@ public class BinarySortableSerDe extends AbstractSerDe {
     serializeInt(buffer, nanos, invert);
   }
 
-  public static void serializeHiveDecimal(ByteStream.Output buffer, HiveDecimal dec, boolean invert) {
+  public static void serializeOldHiveDecimal(ByteStream.Output buffer, HiveDecimalV1 oldDec, boolean invert) {
     // get the sign of the big decimal
-    int sign = dec.compareTo(HiveDecimal.ZERO);
+    int sign = oldDec.compareTo(HiveDecimalV1.ZERO);
 
     // we'll encode the absolute value (sign is separate)
-    dec = dec.abs();
+    oldDec = oldDec.abs();
 
     // get the scale factor to turn big decimal into a decimal < 1
     // This relies on the BigDecimal precision value, which as of HIVE-10270
     // is now different from HiveDecimal.precision()
-    int factor = dec.bigDecimalValue().precision() - dec.bigDecimalValue().scale();
+    int factor = oldDec.bigDecimalValue().precision() - oldDec.bigDecimalValue().scale();
     factor = sign == 1 ? factor : -factor;
 
     // convert the absolute big decimal to string
-    dec.scaleByPowerOfTen(Math.abs(dec.scale()));
-    String digits = dec.unscaledValue().toString();
+    oldDec.scaleByPowerOfTen(Math.abs(oldDec.scale()));
+    String digits = oldDec.unscaledValue().toString();
 
     // finally write out the pieces (sign, scale, digits)
     writeByte(buffer, (byte) ( sign + 1), invert);
@@ -1007,6 +996,119 @@ public class BinarySortableSerDe extends AbstractSerDe {
         digits.length(), sign == -1 ? !invert : invert);
   }
 
+  // See comments for next method.
+  public static void serializeHiveDecimal(ByteStream.Output buffer, HiveDecimal dec, boolean invert) {
+
+    byte[] scratchBuffer = new byte[HiveDecimal.SCRATCH_BUFFER_LEN_TO_BYTES];
+    serializeHiveDecimal(buffer, dec, invert, scratchBuffer);
+  }
+
+  /**
+   * Decimals are encoded in three pieces:Decimals are encoded in three pieces:
+   *
+   * Sign:   1, 2 or 3 for smaller, equal or larger than 0 respectively
+   * Factor: Number that indicates the amount of digits you have to move
+   *         the decimal point left or right until the resulting number is smaller
+   *         than zero but has something other than 0 as the first digit.
+   * Digits: which is a string of all the digits in the decimal. If the number
+   *         is negative the binary string will be inverted to get the correct ordering.
+   *
+   * UNDONE: Is this example correct?
+   *   Example: 0.00123
+   *   Sign is 3 (bigger than 0)
+   *   Factor is -2 (move decimal point 2 positions right)
+   *   Digits are: 123
+   *
+   * @param buffer
+   * @param dec
+   * @param invert
+   * @param scratchBuffer
+   */
+  public static void serializeHiveDecimal(
+    ByteStream.Output buffer, HiveDecimal dec, boolean invert,
+    byte[] scratchBuffer) {
+
+    // Get the sign of the decimal.
+    int signum = dec.signum();
+
+    // Get the 10^N power to turn digits into the desired decimal with a possible
+    // fractional part.
+    // To be compatible with the OldHiveDecimal version, zero has factor 1.
+    int factor;
+    if (signum == 0) {
+      factor = 1;
+    } else {
+      factor = dec.rawPrecision() - dec.scale();
+    }
+
+    // To make comparisons work properly, the "factor" gets the decimal's sign, too.
+    factor = signum == 1 ? factor : -factor;
+
+    // Convert just the decimal digits (no dot, sign, etc) into bytes.
+    //
+    // This is much faster than converting the BigInteger value from unscaledValue() which is no
+    // longer part of the HiveDecimal representation anymore to string, then bytes.
+    int index = dec.toDigitsOnlyBytes(scratchBuffer);
+
+    /*
+     * Finally write out the pieces (sign, power, digits)
+     */
+    writeByte(buffer, (byte) ( signum + 1), invert);
+    writeByte(buffer, (byte) ((factor >> 24) ^ 0x80), invert);
+    writeByte(buffer, (byte) ( factor >> 16), invert);
+    writeByte(buffer, (byte) ( factor >> 8), invert);
+    writeByte(buffer, (byte)   factor, invert);
+
+    // The toDigitsOnlyBytes stores digits at the end of the scratch buffer.
+    serializeBytes(
+        buffer,
+        scratchBuffer, index, scratchBuffer.length - index,
+        signum == -1 ? !invert : invert);
+  }
+
+  // A HiveDecimalWritable version.
+  public static void serializeHiveDecimal(
+      ByteStream.Output buffer, HiveDecimalWritable decWritable, boolean invert,
+      byte[] scratchBuffer) {
+
+      // Get the sign of the decimal.
+      int signum = decWritable.signum();
+
+      // Get the 10^N power to turn digits into the desired decimal with a possible
+      // fractional part.
+      // To be compatible with the OldHiveDecimal version, zero has factor 1.
+      int factor;
+      if (signum == 0) {
+        factor = 1;
+      } else {
+        factor = decWritable.rawPrecision() - decWritable.scale();
+      }
+
+      // To make comparisons work properly, the "factor" gets the decimal's sign, too.
+      factor = signum == 1 ? factor : -factor;
+
+      // Convert just the decimal digits (no dot, sign, etc) into bytes.
+      //
+      // This is much faster than converting the BigInteger value from unscaledValue() which is no
+      // longer part of the HiveDecimal representation anymore to string, then bytes.
+      int index = decWritable.toDigitsOnlyBytes(scratchBuffer);
+
+      /*
+       * Finally write out the pieces (sign, power, digits)
+       */
+      writeByte(buffer, (byte) ( signum + 1), invert);
+      writeByte(buffer, (byte) ((factor >> 24) ^ 0x80), invert);
+      writeByte(buffer, (byte) ( factor >> 16), invert);
+      writeByte(buffer, (byte) ( factor >> 8), invert);
+      writeByte(buffer, (byte)   factor, invert);
+
+      // The toDigitsOnlyBytes stores digits at the end of the scratch buffer.
+      serializeBytes(
+          buffer,
+          scratchBuffer, index, scratchBuffer.length - index,
+          signum == -1 ? !invert : invert);
+  }
+
   @Override
   public SerDeStats getSerDeStats() {
     // no support for statistics

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/serde/src/java/org/apache/hadoop/hive/serde2/binarysortable/fast/BinarySortableDeserializeRead.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/binarysortable/fast/BinarySortableDeserializeRead.java b/serde/src/java/org/apache/hadoop/hive/serde2/binarysortable/fast/BinarySortableDeserializeRead.java
index a7785b2..41087dc 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/binarysortable/fast/BinarySortableDeserializeRead.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/binarysortable/fast/BinarySortableDeserializeRead.java
@@ -21,10 +21,11 @@ package org.apache.hadoop.hive.serde2.binarysortable.fast;
 import java.io.IOException;
 import java.math.BigInteger;
 import java.util.Arrays;
+import java.nio.charset.StandardCharsets;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.apache.hadoop.hive.common.type.HiveDecimal;
+import org.apache.hadoop.hive.common.type.FastHiveDecimal;
 import org.apache.hadoop.hive.serde2.binarysortable.BinarySortableSerDe;
 import org.apache.hadoop.hive.serde2.binarysortable.InputByteBuffer;
 import org.apache.hadoop.hive.serde2.fast.DeserializeRead;
@@ -391,6 +392,7 @@ public final class BinarySortableDeserializeRead extends DeserializeRead {
           length++;
         } while (true);
 
+        // CONSIDER: Allocate a larger initial size.
         if(tempDecimalBuffer == null || tempDecimalBuffer.length < length) {
           tempDecimalBuffer = new byte[length];
         }
@@ -403,29 +405,30 @@ public final class BinarySortableDeserializeRead extends DeserializeRead {
         // read the null byte again
         inputByteBuffer.read(positive ? invert : !invert);
 
-        String digits = new String(tempDecimalBuffer, 0, length, BinarySortableSerDe.decimalCharSet);
-        BigInteger bi = new BigInteger(digits);
-        HiveDecimal bd = HiveDecimal.create(bi).scaleByPowerOfTen(factor-length);
+        String digits = new String(tempDecimalBuffer, 0, length, StandardCharsets.UTF_8);
 
-        if (!positive) {
-          bd = bd.negate();
-        }
+        // Set the value of the writable from the decimal digits that were written with no dot.
+        int scale = length - factor;
+        currentHiveDecimalWritable.setFromDigitsOnlyBytesWithScale(
+            !positive, tempDecimalBuffer, 0, length, scale);
+        boolean decimalIsNull = !currentHiveDecimalWritable.isSet();
+        if (!decimalIsNull) {
 
-        // We have a decimal.  After we enforce precision and scale, will it become a NULL?
+          // We have a decimal.  After we enforce precision and scale, will it become a NULL?
 
-        currentHiveDecimalWritable.set(bd);
+          DecimalTypeInfo decimalTypeInfo = (DecimalTypeInfo) typeInfos[fieldIndex];
 
-        DecimalTypeInfo decimalTypeInfo = (DecimalTypeInfo) typeInfos[fieldIndex];
+          int enforcePrecision = decimalTypeInfo.getPrecision();
+          int enforceScale = decimalTypeInfo.getScale();
 
-        int precision = decimalTypeInfo.getPrecision();
-        int scale = decimalTypeInfo.getScale();
+          decimalIsNull =
+              !currentHiveDecimalWritable.mutateEnforcePrecisionScale(
+                  enforcePrecision, enforceScale);
 
-        HiveDecimal decimal = currentHiveDecimalWritable.getHiveDecimal(precision, scale);
-        if (decimal == null) {
+        }
+        if (decimalIsNull) {
           return false;
         }
-        // Put value back into writable.
-        currentHiveDecimalWritable.set(decimal);
       }
       return true;
     default:

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/serde/src/java/org/apache/hadoop/hive/serde2/binarysortable/fast/BinarySortableSerializeWrite.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/binarysortable/fast/BinarySortableSerializeWrite.java b/serde/src/java/org/apache/hadoop/hive/serde2/binarysortable/fast/BinarySortableSerializeWrite.java
index 62bcaa5..a9ea7c0 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/binarysortable/fast/BinarySortableSerializeWrite.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/binarysortable/fast/BinarySortableSerializeWrite.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.hive.serde2.ByteStream.Output;
 import org.apache.hadoop.hive.serde2.binarysortable.BinarySortableSerDe;
 import org.apache.hadoop.hive.serde2.fast.SerializeWrite;
 import org.apache.hadoop.hive.serde2.io.DateWritable;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 import org.apache.hadoop.hive.serde2.io.TimestampWritable;
 import org.apache.hive.common.util.DateUtils;
 import org.slf4j.Logger;
@@ -61,6 +62,8 @@ public final class BinarySortableSerializeWrite implements SerializeWrite {
 
   private TimestampWritable tempTimestampWritable;
 
+  private byte[] decimalBytesScratch;
+
   public BinarySortableSerializeWrite(boolean[] columnSortOrderIsDesc,
           byte[] columnNullMarker, byte[] columnNotNullMarker) {
     this();
@@ -397,6 +400,9 @@ public final class BinarySortableSerializeWrite implements SerializeWrite {
 
   /*
    * DECIMAL.
+   *
+   * NOTE: The scale parameter is for text serialization (e.g. HiveDecimal.toFormatString) that
+   * creates trailing zeroes output decimals.
    */
   @Override
   public void writeHiveDecimal(HiveDecimal dec, int scale) throws IOException {
@@ -407,6 +413,24 @@ public final class BinarySortableSerializeWrite implements SerializeWrite {
     // This field is not a null.
     BinarySortableSerDe.writeByte(output, columnNotNullMarker[index], invert);
 
-    BinarySortableSerDe.serializeHiveDecimal(output, dec, invert);
+    if (decimalBytesScratch == null) {
+      decimalBytesScratch = new byte[HiveDecimal.SCRATCH_BUFFER_LEN_TO_BYTES];
+    }
+    BinarySortableSerDe.serializeHiveDecimal(output, dec, invert, decimalBytesScratch);
+  }
+
+  @Override
+  public void writeHiveDecimal(HiveDecimalWritable decWritable, int scale) throws IOException {
+    ++index;
+
+    final boolean invert = columnSortOrderIsDesc[index];
+
+    // This field is not a null.
+    BinarySortableSerDe.writeByte(output, columnNotNullMarker[index], invert);
+
+    if (decimalBytesScratch == null) {
+      decimalBytesScratch = new byte[HiveDecimal.SCRATCH_BUFFER_LEN_TO_BYTES];
+    }
+    BinarySortableSerDe.serializeHiveDecimal(output, decWritable, invert, decimalBytesScratch);
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/serde/src/java/org/apache/hadoop/hive/serde2/fast/SerializeWrite.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/fast/SerializeWrite.java b/serde/src/java/org/apache/hadoop/hive/serde2/fast/SerializeWrite.java
index fb41420..17d2385 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/fast/SerializeWrite.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/fast/SerializeWrite.java
@@ -24,6 +24,7 @@ import java.sql.Timestamp;
 
 import org.apache.hadoop.hive.common.type.HiveChar;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 import org.apache.hadoop.hive.common.type.HiveIntervalDayTime;
 import org.apache.hadoop.hive.common.type.HiveIntervalYearMonth;
 import org.apache.hadoop.hive.common.type.HiveVarchar;
@@ -147,6 +148,10 @@ public interface SerializeWrite {
 
   /*
    * DECIMAL.
+   *
+   * NOTE: The scale parameter is for text serialization (e.g. HiveDecimal.toFormatString) that
+   * creates trailing zeroes output decimals.
    */
   void writeHiveDecimal(HiveDecimal dec, int scale) throws IOException;
+  void writeHiveDecimal(HiveDecimalWritable decWritable, int scale) throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyHiveDecimal.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyHiveDecimal.java b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyHiveDecimal.java
index 4e82e9b..4d2ff22 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyHiveDecimal.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/LazyHiveDecimal.java
@@ -21,6 +21,7 @@ import java.io.IOException;
 import java.io.OutputStream;
 import java.nio.ByteBuffer;
 import java.nio.charset.CharacterCodingException;
+import java.nio.charset.StandardCharsets;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -66,31 +67,21 @@ public class LazyHiveDecimal extends LazyPrimitive<LazyHiveDecimalObjectInspecto
    */
   @Override
   public void init(ByteArrayRef bytes, int start, int length) {
-    String byteData = null;
-    try {
-      byteData = Text.decode(bytes.getData(), start, length);
-    } catch (CharacterCodingException e) {
-      isNull = true;
-      LOG.debug("Data not in the HiveDecimal data type range so converted to null.", e);
-      return;
-    }
 
-    HiveDecimal dec = HiveDecimal.create(byteData);
-    dec = enforcePrecisionScale(dec);
-    if (dec != null) {
-      data.set(dec);
-      isNull = false;
+    // Set the HiveDecimalWritable from bytes without converting to String first for
+    // better performance.
+    data.setFromBytes(bytes.getData(), start, length);
+    if (!data.isSet()) {
+      isNull = true;
     } else {
+      isNull = !data.mutateEnforcePrecisionScale(precision, scale);
+    }
+    if (isNull) {
       LOG.debug("Data not in the HiveDecimal data type range so converted to null. Given data is :"
-          + byteData);
-      isNull = true;
+          + new String(bytes.getData(), start, length, StandardCharsets.UTF_8));
     }
   }
 
-  private HiveDecimal enforcePrecisionScale(HiveDecimal dec) {
-    return HiveDecimal.enforcePrecisionScale(dec, precision, scale);
-  }
-
   @Override
   public HiveDecimalWritable getWritableObject() {
     return data;
@@ -107,8 +98,47 @@ public class LazyHiveDecimal extends LazyPrimitive<LazyHiveDecimalObjectInspecto
     if (hiveDecimal == null) {
       outputStream.write(nullBytes);
     } else {
-      ByteBuffer b = Text.encode(hiveDecimal.toFormatString(scale));
-      outputStream.write(b.array(), 0, b.limit());
+      byte[] scratchBuffer = new byte[HiveDecimal.SCRATCH_BUFFER_LEN_TO_BYTES];
+      int index = hiveDecimal.toFormatBytes(scale, scratchBuffer);
+      outputStream.write(scratchBuffer, index, scratchBuffer.length - index);
+    }
+  }
+
+  /**
+   * Writes HiveDecimal object to output stream as string
+   * @param outputStream
+   * @param hiveDecimal
+   * @throws IOException
+   */
+  public static void writeUTF8(
+      OutputStream outputStream,
+      HiveDecimal hiveDecimal, int scale,
+      byte[] scratchBuffer)
+    throws IOException {
+    if (hiveDecimal == null) {
+      outputStream.write(nullBytes);
+    } else {
+      int index = hiveDecimal.toFormatBytes(scale, scratchBuffer);
+      outputStream.write(scratchBuffer, index, scratchBuffer.length - index);
+    }
+  }
+
+  /**
+   * Writes HiveDecimalWritable object to output stream as string
+   * @param outputStream
+   * @param hiveDecimal
+   * @throws IOException
+   */
+  public static void writeUTF8(
+      OutputStream outputStream,
+      HiveDecimalWritable hiveDecimalWritable, int scale,
+      byte[] scratchBuffer)
+    throws IOException {
+    if (hiveDecimalWritable == null || !hiveDecimalWritable.isSet()) {
+      outputStream.write(nullBytes);
+    } else {
+      int index = hiveDecimalWritable.toFormatBytes(scale, scratchBuffer);
+      outputStream.write(scratchBuffer, index, scratchBuffer.length - index);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/serde/src/java/org/apache/hadoop/hive/serde2/lazy/fast/LazySimpleDeserializeRead.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/fast/LazySimpleDeserializeRead.java b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/fast/LazySimpleDeserializeRead.java
index daf2cfb..a597fd7 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/fast/LazySimpleDeserializeRead.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/fast/LazySimpleDeserializeRead.java
@@ -27,7 +27,6 @@ import java.util.Arrays;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.apache.hadoop.hive.common.type.HiveDecimal;
 import org.apache.hadoop.hive.common.type.HiveIntervalDayTime;
 import org.apache.hadoop.hive.common.type.HiveIntervalYearMonth;
 import org.apache.hadoop.hive.serde2.fast.DeserializeRead;
@@ -556,20 +555,24 @@ public final class LazySimpleDeserializeRead extends DeserializeRead {
           if (!LazyUtils.isNumberMaybe(bytes, fieldStart, fieldLength)) {
             return false;
           }
-          String byteData = new String(bytes, fieldStart, fieldLength, StandardCharsets.UTF_8);
-          HiveDecimal decimal = HiveDecimal.create(byteData);
-          DecimalTypeInfo decimalTypeInfo = (DecimalTypeInfo) typeInfos[fieldIndex];
-          int precision = decimalTypeInfo.getPrecision();
-          int scale = decimalTypeInfo.getScale();
-          decimal = HiveDecimal.enforcePrecisionScale(decimal, precision, scale);
-          if (decimal == null) {
+          // Trim blanks because OldHiveDecimal did...
+          currentHiveDecimalWritable.setFromBytes(bytes, fieldStart, fieldLength, /* trimBlanks */ true);
+          boolean decimalIsNull = !currentHiveDecimalWritable.isSet();
+          if (!decimalIsNull) {
+            DecimalTypeInfo decimalTypeInfo = (DecimalTypeInfo) typeInfos[fieldIndex];
+
+            int precision = decimalTypeInfo.getPrecision();
+            int scale = decimalTypeInfo.getScale();
+
+            decimalIsNull = !currentHiveDecimalWritable.mutateEnforcePrecisionScale(precision, scale);
+          }
+          if (decimalIsNull) {
             if (LOG.isDebugEnabled()) {
               LOG.debug("Data not in the HiveDecimal data type range so converted to null. Given data is :"
-                + byteData);
+                + new String(bytes, fieldStart, fieldLength, StandardCharsets.UTF_8));
             }
             return false;
           }
-          currentHiveDecimalWritable.set(decimal);
         }
         return true;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/serde/src/java/org/apache/hadoop/hive/serde2/lazy/fast/LazySimpleSerializeWrite.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/fast/LazySimpleSerializeWrite.java b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/fast/LazySimpleSerializeWrite.java
index 280c2b0..1401ac3 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/fast/LazySimpleSerializeWrite.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/fast/LazySimpleSerializeWrite.java
@@ -33,6 +33,7 @@ import org.apache.hadoop.hive.common.type.HiveIntervalYearMonth;
 import org.apache.hadoop.hive.common.type.HiveVarchar;
 import org.apache.hadoop.hive.serde2.ByteStream.Output;
 import org.apache.hadoop.hive.serde2.io.DateWritable;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 import org.apache.hadoop.hive.serde2.io.HiveIntervalDayTimeWritable;
 import org.apache.hadoop.hive.serde2.io.HiveIntervalYearMonthWritable;
 import org.apache.hadoop.hive.serde2.io.TimestampWritable;
@@ -76,6 +77,7 @@ public final class LazySimpleSerializeWrite implements SerializeWrite {
   private HiveIntervalYearMonthWritable hiveIntervalYearMonthWritable;
   private HiveIntervalDayTimeWritable hiveIntervalDayTimeWritable;
   private HiveIntervalDayTime hiveIntervalDayTime;
+  private byte[] decimalScratchBuffer;
 
   public LazySimpleSerializeWrite(int fieldCount,
     byte separator, LazySerDeParameters lazyParams) {
@@ -475,14 +477,34 @@ public final class LazySimpleSerializeWrite implements SerializeWrite {
 
   /*
    * DECIMAL.
+   *
+   * NOTE: The scale parameter is for text serialization (e.g. HiveDecimal.toFormatString) that
+   * creates trailing zeroes output decimals.
    */
   @Override
-  public void writeHiveDecimal(HiveDecimal v, int scale) throws IOException {
+  public void writeHiveDecimal(HiveDecimal dec, int scale) throws IOException {
+    if (index > 0) {
+      output.write(separator);
+    }
+
+    if (decimalScratchBuffer == null) {
+      decimalScratchBuffer = new byte[HiveDecimal.SCRATCH_BUFFER_LEN_TO_BYTES];
+    }
+    LazyHiveDecimal.writeUTF8(output, dec, scale, decimalScratchBuffer);
+
+    index++;
+  }
+
+  @Override
+  public void writeHiveDecimal(HiveDecimalWritable decWritable, int scale) throws IOException {
     if (index > 0) {
       output.write(separator);
     }
 
-    LazyHiveDecimal.writeUTF8(output, v, scale);
+    if (decimalScratchBuffer == null) {
+      decimalScratchBuffer = new byte[HiveDecimal.SCRATCH_BUFFER_LEN_TO_BYTES];
+    }
+    LazyHiveDecimal.writeUTF8(output, decWritable, scale, decimalScratchBuffer);
 
     index++;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/serde/src/java/org/apache/hadoop/hive/serde2/lazy/objectinspector/primitive/LazyHiveDecimalObjectInspector.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/objectinspector/primitive/LazyHiveDecimalObjectInspector.java b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/objectinspector/primitive/LazyHiveDecimalObjectInspector.java
index 55ab3e6..fe57df0 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/lazy/objectinspector/primitive/LazyHiveDecimalObjectInspector.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/lazy/objectinspector/primitive/LazyHiveDecimalObjectInspector.java
@@ -43,8 +43,13 @@ implements HiveDecimalObjectInspector {
       return null;
     }
 
-    HiveDecimal dec = ((LazyHiveDecimal)o).getWritableObject().getHiveDecimal();
-    return HiveDecimalUtils.enforcePrecisionScale(dec, (DecimalTypeInfo) typeInfo);
+    DecimalTypeInfo decimalTypeInfo = (DecimalTypeInfo) typeInfo;
+    // We do not want to modify the writable provided by the object o since it is not a copy.
+    HiveDecimalWritable decWritable = ((LazyHiveDecimal)o).getWritableObject();
+    HiveDecimalWritable result = HiveDecimalWritable.enforcePrecisionScale(
+        decWritable, decimalTypeInfo.getPrecision(), decimalTypeInfo.getScale());
+
+    return (result != null && result.isSet() ? result.getHiveDecimal() : null);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryHiveDecimal.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryHiveDecimal.java b/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryHiveDecimal.java
index f8469a7..b7bb67e 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryHiveDecimal.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinaryHiveDecimal.java
@@ -44,9 +44,7 @@ public class LazyBinaryHiveDecimal extends
 
   @Override
   public void init(ByteArrayRef bytes, int start, int length) {
-    LazyBinarySerDe.setFromBytes(bytes.getData(), start, length, data);
-    HiveDecimal dec = data.getHiveDecimal(precision, scale);
-    data = dec == null ? null : new HiveDecimalWritable(dec);
+    LazyBinarySerDe.setFromBigIntegerBytesAndScale(bytes.getData(), start, length, data);
+    data.mutateEnforcePrecisionScale(precision, scale);
   }
-
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinarySerDe.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinarySerDe.java b/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinarySerDe.java
index 54bfd2d..99abb5d 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinarySerDe.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/LazyBinarySerDe.java
@@ -36,6 +36,7 @@ import org.apache.hadoop.hive.serde2.SerDeSpec;
 import org.apache.hadoop.hive.serde2.SerDeStats;
 import org.apache.hadoop.hive.serde2.io.DateWritable;
 import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.hadoop.hive.common.type.HiveDecimal;
 import org.apache.hadoop.hive.serde2.io.HiveIntervalDayTimeWritable;
 import org.apache.hadoop.hive.serde2.io.HiveIntervalYearMonthWritable;
 import org.apache.hadoop.hive.serde2.io.TimestampWritable;
@@ -316,7 +317,7 @@ public class LazyBinarySerDe extends AbstractSerDe {
     LazyBinaryUtils.writeVInt(byteStream, date.getDays());
   }
 
-  public static void setFromBytes(byte[] bytes, int offset, int length,
+  public static void setFromBigIntegerBytesAndScale(byte[] bytes, int offset, int length,
                                   HiveDecimalWritable dec) {
     LazyBinaryUtils.VInt vInt = new LazyBinaryUtils.VInt();
     LazyBinaryUtils.readVInt(bytes, offset, vInt);
@@ -324,20 +325,69 @@ public class LazyBinarySerDe extends AbstractSerDe {
     offset += vInt.length;
     LazyBinaryUtils.readVInt(bytes, offset, vInt);
     offset += vInt.length;
-    byte[] internalStorage = dec.getInternalStorage();
-    if (internalStorage.length != vInt.value) {
-      internalStorage = new byte[vInt.value];
-    }
-    System.arraycopy(bytes, offset, internalStorage, 0, vInt.value);
-    dec.set(internalStorage, scale);
+    dec.setFromBigIntegerBytesAndScale(bytes, offset, vInt.value, scale);
   }
 
   public static void writeToByteStream(RandomAccessOutput byteStream,
-                                       HiveDecimalWritable dec) {
-    LazyBinaryUtils.writeVInt(byteStream, dec.getScale());
-    byte[] internalStorage = dec.getInternalStorage();
-    LazyBinaryUtils.writeVInt(byteStream, internalStorage.length);
-    byteStream.write(internalStorage, 0, internalStorage.length);
+                                       HiveDecimalWritable decWritable) {
+    LazyBinaryUtils.writeVInt(byteStream, decWritable.scale());
+
+    // NOTE: This writes into a scratch buffer within HiveDecimalWritable.
+    //
+    int byteLength = decWritable.bigIntegerBytesInternalScratch();
+
+    LazyBinaryUtils.writeVInt(byteStream, byteLength);
+    byteStream.write(decWritable.bigIntegerBytesInternalScratchBuffer(), 0, byteLength);
+  }
+
+  /**
+   *
+   * Allocate scratchLongs with HiveDecimal.SCRATCH_LONGS_LEN longs.
+   * And, allocate scratch buffer with HiveDecimal.SCRATCH_BUFFER_LEN_BIG_INTEGER_BYTES bytes.
+   *
+   * @param byteStream
+   * @param dec
+   * @param scratchLongs
+   * @param buffer
+   */
+  public static void writeToByteStream(
+      RandomAccessOutput byteStream,
+      HiveDecimal dec,
+      long[] scratchLongs, byte[] scratchBytes) {
+    LazyBinaryUtils.writeVInt(byteStream, dec.scale());
+
+    // Convert decimal into the scratch buffer without allocating a byte[] each time
+    // for better performance.
+    int byteLength = 
+        dec.bigIntegerBytes(
+            scratchLongs, scratchBytes);
+    if (byteLength == 0) {
+      throw new RuntimeException("Decimal to binary conversion failed");
+    }
+    LazyBinaryUtils.writeVInt(byteStream, byteLength);
+    byteStream.write(scratchBytes, 0, byteLength);
+  }
+
+  /**
+  *
+  * Allocate scratchLongs with HiveDecimal.SCRATCH_LONGS_LEN longs.
+  * And, allocate scratch buffer with HiveDecimal.SCRATCH_BUFFER_LEN_BIG_INTEGER_BYTES bytes.
+  *
+  * @param byteStream
+  * @param dec
+  * @param scratchLongs
+  * @param buffer
+  */
+  public static void writeToByteStream(
+      RandomAccessOutput byteStream,
+      HiveDecimalWritable decWritable,
+      long[] scratchLongs, byte[] scratchBytes) {
+    LazyBinaryUtils.writeVInt(byteStream, decWritable.scale());
+    int byteLength =
+        decWritable.bigIntegerBytes(
+            scratchLongs, scratchBytes);
+    LazyBinaryUtils.writeVInt(byteStream, byteLength);
+    byteStream.write(scratchBytes, 0, byteLength);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/fast/LazyBinaryDeserializeRead.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/fast/LazyBinaryDeserializeRead.java b/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/fast/LazyBinaryDeserializeRead.java
index ee945d4..e94ae99 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/fast/LazyBinaryDeserializeRead.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/fast/LazyBinaryDeserializeRead.java
@@ -24,7 +24,6 @@ import java.util.Arrays;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.apache.hadoop.hive.common.type.HiveDecimal;
 import org.apache.hadoop.hive.serde2.fast.DeserializeRead;
 import org.apache.hadoop.hive.serde2.io.TimestampWritable;
 import org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe;
@@ -313,8 +312,8 @@ public final class LazyBinaryDeserializeRead extends DeserializeRead {
             throw new EOFException();
           }
           LazyBinaryUtils.readVInt(bytes, offset, tempVInt);
-          int saveStart = offset;
           offset += tempVInt.length;
+          int readScale = tempVInt.value;
 
           // Parse the first byte of a vint/vlong to determine the number of bytes.
           if (offset + WritableUtils.decodeVIntSize(bytes[offset]) > end) {
@@ -322,7 +321,7 @@ public final class LazyBinaryDeserializeRead extends DeserializeRead {
           }
           LazyBinaryUtils.readVInt(bytes, offset, tempVInt);
           offset += tempVInt.length;
-
+          int saveStart = offset;
           offset += tempVInt.value;
           // Last item -- ok to be at end.
           if (offset > end) {
@@ -330,16 +329,23 @@ public final class LazyBinaryDeserializeRead extends DeserializeRead {
           }
           int length = offset - saveStart;
 
-          LazyBinarySerDe.setFromBytes(bytes, saveStart, length,
-              currentHiveDecimalWritable);
+          //   scale = 2, length = 6, value = -6065716379.11
+          //   \002\006\255\114\197\131\083\105
+          //           \255\114\197\131\083\105
 
-          DecimalTypeInfo decimalTypeInfo = (DecimalTypeInfo) typeInfos[fieldIndex];
+          currentHiveDecimalWritable.setFromBigIntegerBytesAndScale(
+              bytes, saveStart, length, readScale);
+          boolean decimalIsNull = !currentHiveDecimalWritable.isSet();
+          if (!decimalIsNull) {
 
-          int precision = decimalTypeInfo.getPrecision();
-          int scale = decimalTypeInfo.getScale();
+            DecimalTypeInfo decimalTypeInfo = (DecimalTypeInfo) typeInfos[fieldIndex];
 
-          HiveDecimal decimal = currentHiveDecimalWritable.getHiveDecimal(precision, scale);
-          if (decimal == null) {
+            int precision = decimalTypeInfo.getPrecision();
+            int scale = decimalTypeInfo.getScale();
+
+            decimalIsNull = !currentHiveDecimalWritable.mutateEnforcePrecisionScale(precision, scale);
+          }
+          if (decimalIsNull) {
 
             // Logically move past this field.
             fieldIndex++;
@@ -356,8 +362,6 @@ public final class LazyBinaryDeserializeRead extends DeserializeRead {
             }
             return false;
           }
-          // Put value back into writable.
-          currentHiveDecimalWritable.set(decimal);
         }
         break;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/fast/LazyBinarySerializeWrite.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/fast/LazyBinarySerializeWrite.java b/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/fast/LazyBinarySerializeWrite.java
index 91ef12d..6bc4622 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/fast/LazyBinarySerializeWrite.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/lazybinary/fast/LazyBinarySerializeWrite.java
@@ -56,11 +56,12 @@ public class LazyBinarySerializeWrite implements SerializeWrite {
   private long nullOffset;
 
   // For thread safety, we allocate private writable objects for our use only.
-  private HiveDecimalWritable hiveDecimalWritable;
   private TimestampWritable timestampWritable;
   private HiveIntervalYearMonthWritable hiveIntervalYearMonthWritable;
   private HiveIntervalDayTimeWritable hiveIntervalDayTimeWritable;
   private HiveIntervalDayTime hiveIntervalDayTime;
+  private long[] scratchLongs;
+  private byte[] scratchBuffer;
 
   public LazyBinarySerializeWrite(int fieldCount) {
     this();
@@ -675,9 +676,48 @@ public class LazyBinarySerializeWrite implements SerializeWrite {
 
   /*
    * DECIMAL.
+   *
+   * NOTE: The scale parameter is for text serialization (e.g. HiveDecimal.toFormatString) that
+   * creates trailing zeroes output decimals.
    */
   @Override
-  public void writeHiveDecimal(HiveDecimal v, int scale) throws IOException {
+  public void writeHiveDecimal(HiveDecimal dec, int scale) throws IOException {
+
+    // Every 8 fields we write a NULL byte.
+    if ((fieldIndex % 8) == 0) {
+      if (fieldIndex > 0) {
+        // Write back previous 8 field's NULL byte.
+        output.writeByte(nullOffset, nullByte);
+        nullByte = 0;
+        nullOffset = output.getLength();
+      }
+      // Allocate next NULL byte.
+      output.reserve(1);
+    }
+
+    // Set bit in NULL byte when a field is NOT NULL.
+    nullByte |= 1 << (fieldIndex % 8);
+
+    if (scratchLongs == null) {
+      scratchLongs = new long[HiveDecimal.SCRATCH_LONGS_LEN];
+      scratchBuffer = new byte[HiveDecimal.SCRATCH_BUFFER_LEN_BIG_INTEGER_BYTES];
+    }
+    LazyBinarySerDe.writeToByteStream(
+        output,
+        dec,
+        scratchLongs,
+        scratchBuffer);
+
+    fieldIndex++;
+
+    if (fieldIndex == fieldCount) {
+      // Write back the final NULL byte before the last fields.
+      output.writeByte(nullOffset, nullByte);
+    }
+  }
+
+  @Override
+  public void writeHiveDecimal(HiveDecimalWritable decWritable, int scale) throws IOException {
 
     // Every 8 fields we write a NULL byte.
     if ((fieldIndex % 8) == 0) {
@@ -694,11 +734,15 @@ public class LazyBinarySerializeWrite implements SerializeWrite {
     // Set bit in NULL byte when a field is NOT NULL.
     nullByte |= 1 << (fieldIndex % 8);
 
-    if (hiveDecimalWritable == null) {
-      hiveDecimalWritable = new HiveDecimalWritable();
+    if (scratchLongs == null) {
+      scratchLongs = new long[HiveDecimal.SCRATCH_LONGS_LEN];
+      scratchBuffer = new byte[HiveDecimal.SCRATCH_BUFFER_LEN_BIG_INTEGER_BYTES];
     }
-    hiveDecimalWritable.set(v);
-    LazyBinarySerDe.writeToByteStream(output, hiveDecimalWritable);
+    LazyBinarySerDe.writeToByteStream(
+        output,
+        decWritable,
+        scratchLongs,
+        scratchBuffer);
 
     fieldIndex++;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/ObjectInspectorUtils.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/ObjectInspectorUtils.java b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/ObjectInspectorUtils.java
index 1ac72c6..6945a67 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/ObjectInspectorUtils.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/ObjectInspectorUtils.java
@@ -698,6 +698,8 @@ public final class ObjectInspectorUtils {
             .getPrimitiveWritableObject(o);
         return intervalDayTime.hashCode();
       case DECIMAL:
+        // Since getBucketHashCode uses this, HiveDecimal return the old (much slower) but
+        // compatible hash code.
         return ((HiveDecimalObjectInspector) poi).getPrimitiveWritableObject(o).hashCode();
 
       default: {

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorUtils.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorUtils.java b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorUtils.java
index 26b19f5..9642a7e 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorUtils.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/PrimitiveObjectInspectorUtils.java
@@ -580,7 +580,24 @@ public final class PrimitiveObjectInspectorUtils {
    * NumberFormatException will be thrown if o is not a valid number.
    */
   public static byte getByte(Object o, PrimitiveObjectInspector oi) {
-    return (byte) getInt(o, oi);
+    byte result;
+    switch (oi.getPrimitiveCategory()) {
+    case DECIMAL:
+      {
+        HiveDecimal dec = ((HiveDecimalObjectInspector) oi)
+            .getPrimitiveJavaObject(o);
+        if (!dec.isByte()) {
+          throw new NumberFormatException();
+        }
+        result = dec.byteValue();
+      }
+      break;
+    default:
+      // For all other data types, use int conversion.  At some point, we should have all
+      // conversions make sure the value fits.
+      return (byte) getInt(o, oi);
+    }
+    return result;
   }
 
   /**
@@ -589,7 +606,24 @@ public final class PrimitiveObjectInspectorUtils {
    * NumberFormatException will be thrown if o is not a valid number.
    */
   public static short getShort(Object o, PrimitiveObjectInspector oi) {
-    return (short) getInt(o, oi);
+    short result;
+    switch (oi.getPrimitiveCategory()) {
+    case DECIMAL:
+      {
+        HiveDecimal dec = ((HiveDecimalObjectInspector) oi)
+            .getPrimitiveJavaObject(o);
+        if (!dec.isShort()) {
+          throw new NumberFormatException();
+        }
+        result = dec.shortValue();
+      }
+      break;
+    default:
+      // For all other data types, use int conversion.  At some point, we should have all
+      // conversions make sure the value fits.
+      return (short) getInt(o, oi);
+    }
+    return result;
   }
 
   /**
@@ -653,8 +687,14 @@ public final class PrimitiveObjectInspectorUtils {
           .getPrimitiveWritableObject(o).getSeconds());
       break;
     case DECIMAL:
-      result = ((HiveDecimalObjectInspector) oi)
-          .getPrimitiveJavaObject(o).intValue();  // TODO: lossy conversion!
+      {
+        HiveDecimal dec = ((HiveDecimalObjectInspector) oi)
+            .getPrimitiveJavaObject(o);
+        if (!dec.isInt()) {
+          throw new NumberFormatException();
+        }
+        result = dec.intValue();
+      }
       break;
     case DATE:  // unsupported conversion
     default: {
@@ -717,8 +757,14 @@ public final class PrimitiveObjectInspectorUtils {
           .getSeconds();
       break;
     case DECIMAL:
-      result = ((HiveDecimalObjectInspector) oi)
-          .getPrimitiveJavaObject(o).longValue();  // TODO: lossy conversion!
+      {
+        HiveDecimal dec = ((HiveDecimalObjectInspector) oi)
+            .getPrimitiveJavaObject(o);
+        if (!dec.isLong()) {
+          throw new NumberFormatException();
+        }
+        result = dec.longValue();
+      }
       break;
     case DATE:  // unsupported conversion
     default:

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableConstantHiveDecimalObjectInspector.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableConstantHiveDecimalObjectInspector.java b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableConstantHiveDecimalObjectInspector.java
index b87d1f8..daa51c1 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableConstantHiveDecimalObjectInspector.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/primitive/WritableConstantHiveDecimalObjectInspector.java
@@ -43,17 +43,16 @@ implements ConstantObjectInspector {
 
   @Override
   public HiveDecimalWritable getWritableConstantValue() {
+
     // We need to enforce precision/scale here.
-    // A little inefficiency here as we need to create a HiveDecimal instance from the writable and
-    // recreate a HiveDecimalWritable instance on the HiveDecimal instance. However, we don't know
-    // the precision/scale of the original writable until we get a HiveDecimal instance from it.
-    DecimalTypeInfo decTypeInfo = (DecimalTypeInfo)typeInfo;
-    HiveDecimal dec = value == null ? null :
-      value.getHiveDecimal(decTypeInfo.precision(), decTypeInfo.scale());
-    if (dec == null) {
+
+    DecimalTypeInfo decTypeInfo = (DecimalTypeInfo) typeInfo;
+    HiveDecimalWritable result = new HiveDecimalWritable(value);
+    result.mutateEnforcePrecisionScale(decTypeInfo.precision(), decTypeInfo.scale());
+    if (!result.isSet()) {
       return null;
     }
-    return new HiveDecimalWritable(dec);
+    return result;
   }
 
   @Override
@@ -61,7 +60,7 @@ implements ConstantObjectInspector {
     if (value == null) {
       return super.precision();
     }
-    return value.getHiveDecimal().precision();
+    return value.precision();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/HiveDecimalUtils.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/HiveDecimalUtils.java b/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/HiveDecimalUtils.java
index 5caaf6b..cee9c45 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/HiveDecimalUtils.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/typeinfo/HiveDecimalUtils.java
@@ -34,8 +34,9 @@ public class HiveDecimalUtils {
       return null;
     }
 
-    HiveDecimal dec = enforcePrecisionScale(writable.getHiveDecimal(), typeInfo);
-    return dec == null ? null : new HiveDecimalWritable(dec);
+    HiveDecimalWritable result = new HiveDecimalWritable(writable);
+    result.mutateEnforcePrecisionScale(typeInfo.precision(), typeInfo.scale());
+    return (result.isSet() ? result : null);
   }
 
   public static void validateParameter(int precision, int scale) {

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/serde/src/test/org/apache/hadoop/hive/serde2/SerdeRandomRowSource.java
----------------------------------------------------------------------
diff --git a/serde/src/test/org/apache/hadoop/hive/serde2/SerdeRandomRowSource.java b/serde/src/test/org/apache/hadoop/hive/serde2/SerdeRandomRowSource.java
index f08a075..301ee8b 100644
--- a/serde/src/test/org/apache/hadoop/hive/serde2/SerdeRandomRowSource.java
+++ b/serde/src/test/org/apache/hadoop/hive/serde2/SerdeRandomRowSource.java
@@ -269,27 +269,27 @@ public class SerdeRandomRowSource {
       {
         WritableHiveCharObjectInspector writableCharObjectInspector =
                 new WritableHiveCharObjectInspector( (CharTypeInfo) primitiveTypeInfo);
-        return writableCharObjectInspector.create(new HiveChar(StringUtils.EMPTY, -1));
+        return writableCharObjectInspector.create((HiveChar) object);
       }
     case VARCHAR:
       {
         WritableHiveVarcharObjectInspector writableVarcharObjectInspector =
                 new WritableHiveVarcharObjectInspector( (VarcharTypeInfo) primitiveTypeInfo);
-        return writableVarcharObjectInspector.create(new HiveVarchar(StringUtils.EMPTY, -1));
+        return writableVarcharObjectInspector.create((HiveVarchar) object);
       }
     case BINARY:
-      return PrimitiveObjectInspectorFactory.writableBinaryObjectInspector.create(ArrayUtils.EMPTY_BYTE_ARRAY);
+      return PrimitiveObjectInspectorFactory.writableBinaryObjectInspector.create((byte[]) object);
     case TIMESTAMP:
-      return ((WritableTimestampObjectInspector) objectInspector).create(new Timestamp(0));
+      return ((WritableTimestampObjectInspector) objectInspector).create((Timestamp) object);
     case INTERVAL_YEAR_MONTH:
-      return ((WritableHiveIntervalYearMonthObjectInspector) objectInspector).create(new HiveIntervalYearMonth(0));
+      return ((WritableHiveIntervalYearMonthObjectInspector) objectInspector).create((HiveIntervalYearMonth) object);
     case INTERVAL_DAY_TIME:
-      return ((WritableHiveIntervalDayTimeObjectInspector) objectInspector).create(new HiveIntervalDayTime(0, 0));
+      return ((WritableHiveIntervalDayTimeObjectInspector) objectInspector).create((HiveIntervalDayTime) object);
     case DECIMAL:
       {
         WritableHiveDecimalObjectInspector writableDecimalObjectInspector =
                 new WritableHiveDecimalObjectInspector((DecimalTypeInfo) primitiveTypeInfo);
-        return writableDecimalObjectInspector.create(HiveDecimal.ZERO);
+        return writableDecimalObjectInspector.create((HiveDecimal) object);
       }
     default:
       throw new Error("Unknown primitive category " + primitiveCategory);
@@ -331,7 +331,10 @@ public class SerdeRandomRowSource {
     case INTERVAL_DAY_TIME:
       return getRandIntervalDayTime(r);
     case DECIMAL:
-      return getRandHiveDecimal(r, (DecimalTypeInfo) primitiveTypeInfo);
+      {
+        HiveDecimal dec = getRandHiveDecimal(r, (DecimalTypeInfo) primitiveTypeInfo);
+        return dec;
+      }
     default:
       throw new Error("Unknown primitive category " + primitiveCategory);
     }
@@ -382,14 +385,9 @@ public class SerdeRandomRowSource {
         sb.append(".");
         sb.append(RandomTypeUtil.getRandString(r, DECIMAL_CHARS, scale));
       }
+      HiveDecimal dec = HiveDecimal.create(sb.toString());
 
-      HiveDecimal bd = HiveDecimal.create(sb.toString());
-      if (bd.scale() > bd.precision()) {
-        // Sometimes weird decimals are produced?
-        continue;
-      }
-
-      return bd;
+      return dec;
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/serde/src/test/org/apache/hadoop/hive/serde2/VerifyFast.java
----------------------------------------------------------------------
diff --git a/serde/src/test/org/apache/hadoop/hive/serde2/VerifyFast.java b/serde/src/test/org/apache/hadoop/hive/serde2/VerifyFast.java
index 3ac339d..19b04bb 100644
--- a/serde/src/test/org/apache/hadoop/hive/serde2/VerifyFast.java
+++ b/serde/src/test/org/apache/hadoop/hive/serde2/VerifyFast.java
@@ -68,7 +68,7 @@ public class VerifyFast {
     isNull = !deserializeRead.readNextField();
     if (isNull) {
       if (writable != null) {
-        TestCase.fail("Field reports null but object is not null");
+        TestCase.fail("Field reports null but object is not null (class " + writable.getClass().getName() + ", " + writable.toString() + ")");
       }
       return;
     } else if (writable == null) {

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/serde/src/test/org/apache/hadoop/hive/serde2/avro/TestAvroSerializer.java
----------------------------------------------------------------------
diff --git a/serde/src/test/org/apache/hadoop/hive/serde2/avro/TestAvroSerializer.java b/serde/src/test/org/apache/hadoop/hive/serde2/avro/TestAvroSerializer.java
index 45be2dd..93eafc1 100644
--- a/serde/src/test/org/apache/hadoop/hive/serde2/avro/TestAvroSerializer.java
+++ b/serde/src/test/org/apache/hadoop/hive/serde2/avro/TestAvroSerializer.java
@@ -24,6 +24,7 @@ import org.apache.avro.generic.GenericData;
 import org.apache.avro.generic.GenericRecord;
 import org.apache.avro.generic.GenericEnumSymbol;
 import org.apache.hadoop.hive.common.type.HiveDecimal;
+import org.apache.hadoop.hive.common.type.HiveDecimalV1;
 import org.apache.hadoop.hive.serde2.SerDeException;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
@@ -121,7 +122,7 @@ public class TestAvroSerializer {
 
   @Test
   public void canSerializeDecimals() throws SerDeException, IOException {
-    ByteBuffer bb = ByteBuffer.wrap(HiveDecimal.create("3.1416").unscaledValue().toByteArray());
+    ByteBuffer bb = ByteBuffer.wrap(HiveDecimal.create("3.1416").bigIntegerBytes());
     singleFieldTest("dec1", bb.rewind(),
         "{\"type\":\"bytes\", \"logicalType\":\"decimal\", \"precision\":5, \"scale\":4}");
   }
@@ -229,7 +230,10 @@ public class TestAvroSerializer {
     HiveDecimal dec = HiveDecimal.create("3.1415926");
     r = serializeAndDeserialize(field, "union1", AvroSerdeUtils.getBufferFromDecimal(dec, 4));
     HiveDecimal dec1 = AvroSerdeUtils.getHiveDecimalFromByteBuffer((ByteBuffer) r.get("union1"), 4);
-    assertEquals(dec.setScale(4), dec1);
+
+    // For now, old class.
+    HiveDecimalV1 oldDec = HiveDecimalV1.create(dec.bigDecimalValue());
+    assertEquals(oldDec.setScale(4).toString(), dec1.toString());
   }
 
   private enum enum1 {BLUE, RED , GREEN};

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/serde/src/test/org/apache/hadoop/hive/serde2/binarysortable/MyTestPrimitiveClass.java
----------------------------------------------------------------------
diff --git a/serde/src/test/org/apache/hadoop/hive/serde2/binarysortable/MyTestPrimitiveClass.java b/serde/src/test/org/apache/hadoop/hive/serde2/binarysortable/MyTestPrimitiveClass.java
index 321b574..18b2032 100644
--- a/serde/src/test/org/apache/hadoop/hive/serde2/binarysortable/MyTestPrimitiveClass.java
+++ b/serde/src/test/org/apache/hadoop/hive/serde2/binarysortable/MyTestPrimitiveClass.java
@@ -203,18 +203,10 @@ public class MyTestPrimitiveClass {
           sb.append(getRandString(r, DECIMAL_CHARS, scale));
         }
 
-        HiveDecimal bd = HiveDecimal.create(sb.toString());
-        extraTypeInfo.precision = bd.precision();
-        extraTypeInfo.scale = bd.scale();
-        if (extraTypeInfo.scale > extraTypeInfo.precision) {
-          // Sometimes weird decimals are produced?
-          continue;
-        }
-
-        // For now, punt.
-        extraTypeInfo.precision = HiveDecimal.SYSTEM_DEFAULT_PRECISION;
-        extraTypeInfo.scale = HiveDecimal.SYSTEM_DEFAULT_SCALE;
-        return bd;
+        HiveDecimal dec = HiveDecimal.create(sb.toString());
+        extraTypeInfo.precision = dec.precision();
+        extraTypeInfo.scale = dec.scale();
+        return dec;
       }
     }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/serde/src/test/org/apache/hadoop/hive/serde2/binarysortable/TestBinarySortableFast.java
----------------------------------------------------------------------
diff --git a/serde/src/test/org/apache/hadoop/hive/serde2/binarysortable/TestBinarySortableFast.java b/serde/src/test/org/apache/hadoop/hive/serde2/binarysortable/TestBinarySortableFast.java
index 1c84fe6..5f5b03a 100644
--- a/serde/src/test/org/apache/hadoop/hive/serde2/binarysortable/TestBinarySortableFast.java
+++ b/serde/src/test/org/apache/hadoop/hive/serde2/binarysortable/TestBinarySortableFast.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hive.serde2.binarysortable;
 
+import java.util.ArrayList;
 import java.io.EOFException;
 import java.util.Arrays;
 import java.util.List;
@@ -196,7 +197,11 @@ public class TestBinarySortableFast extends TestCase {
           }
         } else {
           if (!object.equals(expected)) {
-            fail("SerDe deserialized value does not match");
+            fail("SerDe deserialized value does not match (expected " +
+              expected.getClass().getName() + " " +
+              expected.toString() + ", actual " +
+              object.getClass().getName() + " " +
+              object.toString() + ")");
           }
         }
       }
@@ -233,11 +238,37 @@ public class TestBinarySortableFast extends TestCase {
           fail("Different byte array lengths: serDeOutput.length " + serDeOutput.length + ", serializeWriteExpected.length " + serializeWriteExpected.length +
                   " mismatchPos " + mismatchPos + " perFieldWriteLengths " + Arrays.toString(perFieldWriteLengthsArray[i]));
         }
+        List<Integer> differentPositions = new ArrayList();
         for (int b = 0; b < serDeOutput.length; b++) {
           if (serDeOutput[b] != serializeWriteExpected[b]) {
-            fail("SerializeWrite and SerDe serialization does not match at position " + b);
+            differentPositions.add(b);
           }
         }
+        if (differentPositions.size() > 0) {
+          List<String> serializeWriteExpectedFields = new ArrayList<String>();
+          List<String> serDeFields = new ArrayList<String>();
+          int f = 0;
+          int lastBegin = 0;
+          for (int b = 0; b < serDeOutput.length; b++) {
+            int writeLength = perFieldWriteLengthsArray[i][f];
+            if (b + 1 == writeLength) {
+              serializeWriteExpectedFields.add(
+                  displayBytes(serializeWriteExpected, lastBegin, writeLength - lastBegin));
+              serDeFields.add(
+                  displayBytes(serDeOutput, lastBegin, writeLength - lastBegin));
+              f++;
+              lastBegin = b + 1;
+            }
+          }
+          fail("SerializeWrite and SerDe serialization does not match at positions " + differentPositions.toString() +
+              "\n(SerializeWrite: " +
+                  serializeWriteExpectedFields.toString() +
+              "\nSerDe: " +
+                  serDeFields.toString() +
+              "\nperFieldWriteLengths " + Arrays.toString(perFieldWriteLengthsArray[i]) +
+              "\nprimitiveTypeInfos " + Arrays.toString(primitiveTypeInfos) +
+              "\nrow " + Arrays.toString(row));
+        }
       }
       serdeBytes[i] = bytesWritable;
     }
@@ -426,4 +457,12 @@ public class TestBinarySortableFast extends TestCase {
       throw e;
     }
   }
+
+  private static String displayBytes(byte[] bytes, int start, int length) {
+    StringBuilder sb = new StringBuilder();
+    for (int i = start; i < start + length; i++) {
+      sb.append(String.format("\\%03d", (int) (bytes[i] & 0xff)));
+    }
+    return sb.toString();
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/serde/src/test/org/apache/hadoop/hive/serde2/io/TestHiveDecimalWritable.java
----------------------------------------------------------------------
diff --git a/serde/src/test/org/apache/hadoop/hive/serde2/io/TestHiveDecimalWritable.java b/serde/src/test/org/apache/hadoop/hive/serde2/io/TestHiveDecimalWritable.java
deleted file mode 100644
index 3b12514..0000000
--- a/serde/src/test/org/apache/hadoop/hive/serde2/io/TestHiveDecimalWritable.java
+++ /dev/null
@@ -1,250 +0,0 @@
-/**
- * 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.serde2.io;
-
-import com.google.code.tempusfugit.concurrency.annotations.*;
-import com.google.code.tempusfugit.concurrency.*;
-import org.junit.*;
-import static org.junit.Assert.*;
-
-import java.math.BigDecimal;
-import java.math.BigInteger;
-import java.nio.ByteBuffer;
-import java.nio.ByteOrder;
-import java.util.ArrayList;
-
-import org.apache.hadoop.hive.common.type.Decimal128;
-import org.apache.hadoop.hive.common.type.HiveDecimal;
-import org.apache.hive.common.util.Decimal128FastBuffer;
-
-/**
- * Unit tests for tsting the fast allocation-free conversion
- * between HiveDecimalWritable and Decimal128
- */
-public class TestHiveDecimalWritable {
-
-    @Rule public ConcurrentRule concurrentRule = new ConcurrentRule();
-    @Rule public RepeatingRule repeatingRule = new RepeatingRule();
-
-    @Before
-    public void setUp() throws Exception {
-    }
-
-    private void doTestFastStreamForHiveDecimal(String valueString) {
-      Decimal128FastBuffer scratch = new Decimal128FastBuffer();
-      BigDecimal value = new BigDecimal(valueString);
-      Decimal128 dec = new Decimal128();
-      dec.update(value);
-
-      HiveDecimalWritable witness = new HiveDecimalWritable();
-      witness.set(HiveDecimal.create(value));
-
-      int bufferUsed = dec.fastSerializeForHiveDecimal(scratch);
-      HiveDecimalWritable hdw = new HiveDecimalWritable();
-      hdw.set(scratch.getBytes(bufferUsed), dec.getScale());
-
-      HiveDecimal hd = hdw.getHiveDecimal();
-
-      BigDecimal readValue = hd.bigDecimalValue();
-
-      assertEquals(value, readValue);
-
-      // Now test fastUpdate from the same serialized HiveDecimal
-      Decimal128 decRead = new Decimal128().fastUpdateFromInternalStorage(
-              witness.getInternalStorage(), (short) witness.getScale());
-
-      assertEquals(dec, decRead);
-
-      // Test fastUpdate from it's own (not fully compacted) serialized output
-      Decimal128 decReadSelf = new Decimal128().fastUpdateFromInternalStorage(
-              hdw.getInternalStorage(), (short) hdw.getScale());
-      assertEquals(dec, decReadSelf);
-    }
-
-    @Test
-    @Concurrent(count=4)
-    @Repeating(repetition=100)
-    public void testFastStreamForHiveDecimal() {
-
-      doTestFastStreamForHiveDecimal("0");
-      doTestFastStreamForHiveDecimal("-0");
-      doTestFastStreamForHiveDecimal("1");
-      doTestFastStreamForHiveDecimal("-1");
-      doTestFastStreamForHiveDecimal("2");
-      doTestFastStreamForHiveDecimal("-2");
-      doTestFastStreamForHiveDecimal("127");
-      doTestFastStreamForHiveDecimal("-127");
-      doTestFastStreamForHiveDecimal("128");
-      doTestFastStreamForHiveDecimal("-128");
-      doTestFastStreamForHiveDecimal("255");
-      doTestFastStreamForHiveDecimal("-255");
-      doTestFastStreamForHiveDecimal("256");
-      doTestFastStreamForHiveDecimal("-256");
-      doTestFastStreamForHiveDecimal("65535");
-      doTestFastStreamForHiveDecimal("-65535");
-      doTestFastStreamForHiveDecimal("65536");
-      doTestFastStreamForHiveDecimal("-65536");
-
-      doTestFastStreamForHiveDecimal("10");
-      doTestFastStreamForHiveDecimal("1000");
-      doTestFastStreamForHiveDecimal("1000000");
-      doTestFastStreamForHiveDecimal("1000000000");
-      doTestFastStreamForHiveDecimal("1000000000000");
-      doTestFastStreamForHiveDecimal("1000000000000000");
-      doTestFastStreamForHiveDecimal("1000000000000000000");
-      doTestFastStreamForHiveDecimal("1000000000000000000000");
-      doTestFastStreamForHiveDecimal("1000000000000000000000000");
-      doTestFastStreamForHiveDecimal("1000000000000000000000000000");
-      doTestFastStreamForHiveDecimal("1000000000000000000000000000000");
-
-      doTestFastStreamForHiveDecimal("-10");
-      doTestFastStreamForHiveDecimal("-1000");
-      doTestFastStreamForHiveDecimal("-1000000");
-      doTestFastStreamForHiveDecimal("-1000000000");
-      doTestFastStreamForHiveDecimal("-1000000000000");
-      doTestFastStreamForHiveDecimal("-1000000000000000000");
-      doTestFastStreamForHiveDecimal("-1000000000000000000000");
-      doTestFastStreamForHiveDecimal("-1000000000000000000000000");
-      doTestFastStreamForHiveDecimal("-1000000000000000000000000000");
-      doTestFastStreamForHiveDecimal("-1000000000000000000000000000000");
-
-
-      doTestFastStreamForHiveDecimal("0.01");
-      doTestFastStreamForHiveDecimal("-0.01");
-      doTestFastStreamForHiveDecimal("0.02");
-      doTestFastStreamForHiveDecimal("-0.02");
-      doTestFastStreamForHiveDecimal("0.0127");
-      doTestFastStreamForHiveDecimal("-0.0127");
-      doTestFastStreamForHiveDecimal("0.0128");
-      doTestFastStreamForHiveDecimal("-0.0128");
-      doTestFastStreamForHiveDecimal("0.0255");
-      doTestFastStreamForHiveDecimal("-0.0255");
-      doTestFastStreamForHiveDecimal("0.0256");
-      doTestFastStreamForHiveDecimal("-0.0256");
-      doTestFastStreamForHiveDecimal("0.065535");
-      doTestFastStreamForHiveDecimal("-0.065535");
-      doTestFastStreamForHiveDecimal("0.065536");
-      doTestFastStreamForHiveDecimal("-0.065536");
-
-      doTestFastStreamForHiveDecimal("0.101");
-      doTestFastStreamForHiveDecimal("0.10001");
-      doTestFastStreamForHiveDecimal("0.10000001");
-      doTestFastStreamForHiveDecimal("0.10000000001");
-      doTestFastStreamForHiveDecimal("0.10000000000001");
-      doTestFastStreamForHiveDecimal("0.10000000000000001");
-      doTestFastStreamForHiveDecimal("0.10000000000000000001");
-      doTestFastStreamForHiveDecimal("0.10000000000000000000001");
-      doTestFastStreamForHiveDecimal("0.10000000000000000000000001");
-      doTestFastStreamForHiveDecimal("0.10000000000000000000000000001");
-      doTestFastStreamForHiveDecimal("0.10000000000000000000000000000001");
-
-      doTestFastStreamForHiveDecimal("-0.101");
-      doTestFastStreamForHiveDecimal("-0.10001");
-      doTestFastStreamForHiveDecimal("-0.10000001");
-      doTestFastStreamForHiveDecimal("-0.10000000001");
-      doTestFastStreamForHiveDecimal("-0.10000000000001");
-      doTestFastStreamForHiveDecimal("-0.10000000000000000001");
-      doTestFastStreamForHiveDecimal("-0.10000000000000000000001");
-      doTestFastStreamForHiveDecimal("-0.10000000000000000000000001");
-      doTestFastStreamForHiveDecimal("-0.10000000000000000000000000001");
-      doTestFastStreamForHiveDecimal("-0.10000000000000000000000000000001");
-
-      doTestFastStreamForHiveDecimal(Integer.toString(Integer.MAX_VALUE));
-      doTestFastStreamForHiveDecimal(Integer.toString(Integer.MIN_VALUE));
-      doTestFastStreamForHiveDecimal(Long.toString(Long.MAX_VALUE));
-      doTestFastStreamForHiveDecimal(Long.toString(Long.MIN_VALUE));
-      doTestFastStreamForHiveDecimal(Decimal128.MAX_VALUE.toFormalString());
-      doTestFastStreamForHiveDecimal(Decimal128.MIN_VALUE.toFormalString());
-
-            // Test known serialization tricky values
-      int[] values = new int[] {
-              0x80,
-              0x8000,
-              0x800000,
-              0x80000000,
-              0x81,
-                    0x8001,
-                    0x800001,
-                    0x80000001,
-              0x7f,
-              0x7fff,
-              0x7fffff,
-              0x7fffffff,
-              0xff,
-              0xffff,
-              0xffffff,
-              0xffffffff};
-
-
-      for(int value: values) {
-          for (int i = 0; i < 4; ++i) {
-              int[] pos = new int[] {1, 0, 0, 0, 0};
-              int[] neg = new int[] {0xff, 0, 0, 0, 0};
-
-              pos[i+1] = neg[i+1] = value;
-
-              doTestDecimalWithBoundsCheck(new Decimal128().update32(pos, 0));
-              doTestDecimalWithBoundsCheck(new Decimal128().update32(neg, 0));
-              doTestDecimalWithBoundsCheck(new Decimal128().update64(pos, 0));
-              doTestDecimalWithBoundsCheck(new Decimal128().update64(neg, 0));
-              doTestDecimalWithBoundsCheck(new Decimal128().update96(pos, 0));
-              doTestDecimalWithBoundsCheck(new Decimal128().update96(neg, 0));
-              doTestDecimalWithBoundsCheck(new Decimal128().update128(pos, 0));
-              doTestDecimalWithBoundsCheck(new Decimal128().update128(neg, 0));
-          }
-      }
-    }
-
-    void doTestDecimalWithBoundsCheck(Decimal128 value) {
-       if ((value.compareTo(Decimal128.MAX_VALUE)) > 0 ||
-           (value.compareTo(Decimal128.MIN_VALUE)) < 0) {
-             // Ignore this one, out of bounds and HiveDecimal will NPE
-             return;
-       }
-       doTestFastStreamForHiveDecimal(value.toFormalString());
-    }
-
-    @Test
-    @Concurrent(count=4)
-    @Repeating(repetition=100)
-    public void testHive6594() {
-      Decimal128FastBuffer scratch = new Decimal128FastBuffer();
-      String[] vs = new String[] {
-          "-4033.445769230769",
-          "6984454.211097692"};
-
-      Decimal128 d = new Decimal128(0L, (short) 14);
-      for (String s:vs) {
-        Decimal128 p = new Decimal128(s, (short) 14);
-        d.addDestructive(p, (short) (short) 14);
-      }
-
-      int bufferUsed = d.fastSerializeForHiveDecimal(scratch);
-      HiveDecimalWritable hdw = new HiveDecimalWritable();
-      hdw.set(scratch.getBytes(bufferUsed), d.getScale());
-
-      HiveDecimal hd = hdw.getHiveDecimal();
-
-      BigDecimal readValue = hd.bigDecimalValue();
-
-      assertEquals(d.toBigDecimal().stripTrailingZeros(),
-          readValue.stripTrailingZeros());
-    }
-}
-


[05/10] hive git commit: HIVE-15335: Fast Decimal (Matt McCline, reviewed by Sergey Shelukhin, Prasanth Jayachandran, Owen O'Malley)

Posted by mm...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/storage-api/src/java/org/apache/hadoop/hive/common/type/FastHiveDecimalImpl.java
----------------------------------------------------------------------
diff --git a/storage-api/src/java/org/apache/hadoop/hive/common/type/FastHiveDecimalImpl.java b/storage-api/src/java/org/apache/hadoop/hive/common/type/FastHiveDecimalImpl.java
new file mode 100644
index 0000000..a4fed5d
--- /dev/null
+++ b/storage-api/src/java/org/apache/hadoop/hive/common/type/FastHiveDecimalImpl.java
@@ -0,0 +1,9149 @@
+/**
+ * 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.common.type;
+
+import java.util.Arrays;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.math.RoundingMode;
+
+import org.apache.commons.lang.StringUtils;
+
+/**
+ *    This class is a companion to the FastHiveDecimal class that separates the essential of code
+ *    out of FastHiveDecimal into static methods in this class so that they can be used directly
+ *    by vectorization to implement decimals by storing the fast0, fast1, and fast2 longs and
+ *    the fastSignum, fastScale, etc ints in the DecimalColumnVector class.
+ */
+public class FastHiveDecimalImpl extends FastHiveDecimal {
+
+  /**
+   * Representation of fast decimals.
+   *
+   * We use 3 long words to store the 38 digits of fast decimals and and 3 integers for sign,
+   * integer digit count, and scale.
+   *
+   * The lower and middle long words store 16 decimal digits each; the high long word has
+   * 6 decimal digits; total 38 decimal digits.
+   *
+   * We do not try and represent fast decimal value as an unsigned 128 bit binary number in 2 longs.
+   * There are several important reasons for this.
+   *
+   * The effort to represent an unsigned 128 integer in 2 Java signed longs is very difficult,
+   * error prone, hard to debug, and not worth the effort.
+   *
+   * The focus here is on reusing memory (i.e. with HiveDecimalWritable) as often as possible.
+   * Reusing memory is good for grouping of fast decimal objects and related objects in CPU cache
+   * lines for fast memory access and eliminating the cost of allocating temporary objects and
+   * reducing the global cost of garbage collection.
+   *
+   * In other words, we are focused on avoiding the poor performance of Java general immutable
+   * objects.
+   *
+   * Reducing memory size or being concerned about the memory size of using 3 longs vs. 2 longs
+   * for 128 unsigned bits is not the focus here.
+   *
+   * Besides focusing on reusing memory, storing a limited number (16) decimal digits in the longs
+   * rather than compacting the value into all binary bits of 2 longs has a surprising benefit.
+   *
+   * One big part of implementing decimals turns out to be manipulating decimal digits.
+   *
+   * For example, rounding a decimal involves trimming off lower digits or clearing lower digits.
+   * Since radix 10 digits cannot be masked with binary masks, we use division and multiplication
+   * using powers of 10.  We can easily manipulate the decimal digits in a long word using simple
+   * integer multiplication / division without doing emulated 128 binary bit multiplication /
+   * division (e.g. the defunct Decimal128 class).
+   *
+   * For example, say we want to scale (round) down the fraction digits of a decimal.
+   *
+   *      final long divideFactor = powerOfTenTable[scaleDown];
+   *      final long multiplyFactor = powerOfTenTable[LONGWORD_DECIMAL_DIGITS - scaleDown];
+   *
+   *      result0 =
+   *          fast0 / divideFactor
+   *        + ((fast1 % divideFactor) * multiplyFactor);
+   *      result1 =
+   *          fast1 / divideFactor
+   *        + ((fast2 % divideFactor) * multiplyFactor);
+   *      result2 =
+   *          fast2 / divideFactor;
+   *
+   * It also turns out to do addition and subtraction of decimals with different scales can involve
+   * overlap using more than 3 long words.  Manipulating extra words is a natural extension of
+   * the existing techniques.
+   *
+   * Why is the decimal digits representation easier to debug?  You can see the decimal digits in
+   * the 3 long words and do not have to convert binary words to decimal to see the value.
+   *
+   * 16 decimal digits for a long was choose so that an int can have 1/2 or 8 decimal digits during
+   * multiplication of int half words so intermediate multiplication results do not overflow a long.
+   * And, so addition overflow is well below the sign bit of a long.
+   */
+
+  // Code Sections:
+  //   Initialize (fastSetFrom*).
+  //   Take Integer or Fractional Portion.
+  //   Binary to Decimal Conversion.
+  //   Decimal to Binary Conversion.r
+  //   Emulate SerializationUtils Deserialization used by ORC.
+  //   Emulate SerializationUtils Serialization used by ORC.
+  //   Emulate BigInteger Deserialization used by LazyBinary and others.
+  //   Emulate BigInteger Serialization used by LazyBinary and others.
+  //   Decimal to Integer Conversion.
+  //   Decimal to Non-Integer Conversion.
+  //   Decimal Comparison.
+  //   Decimal Rounding.
+  //   Decimal Scale Up/Down.
+  //   Decimal Precision / Trailing Zeroes.
+  //   Decimal Addition / Subtraction.
+  //   Decimal Multiply.
+  //   Decimal Division / Remainder.
+  //   Decimal String Formatting.
+  //   Decimal Validation.
+  //   Decimal Debugging.
+
+  private static final long[] powerOfTenTable = {
+    1L,                   // 0
+    10L,
+    100L,
+    1000L,
+    10000L,
+    100000L,
+    1000000L,
+    10000000L,
+    100000000L,           // 8
+    1000000000L,
+    10000000000L,
+    100000000000L,
+    1000000000000L,
+    10000000000000L,
+    100000000000000L,
+    1000000000000000L,
+    10000000000000000L    // 16
+  };
+
+  public static final int MAX_DECIMAL_DIGITS = 38;
+
+  /**
+   * Int: 8 decimal digits.  An even number and 1/2 of MAX_LONGWORD_DECIMAL.
+   */
+  private static final int INTWORD_DECIMAL_DIGITS = 8;
+  private static final int MAX_INTWORD_DECIMAL = (int) powerOfTenTable[INTWORD_DECIMAL_DIGITS] - 1;
+  private static final int MULTIPLER_INTWORD_DECIMAL = (int) powerOfTenTable[INTWORD_DECIMAL_DIGITS];
+
+  /**
+   * Long: 16 decimal digits.  An even number and twice MAX_INTWORD_DECIMAL.
+   */
+  private static final int LONGWORD_DECIMAL_DIGITS = 16;
+  private static final long MAX_LONGWORD_DECIMAL = powerOfTenTable[LONGWORD_DECIMAL_DIGITS] - 1;
+  private static final long MULTIPLER_LONGWORD_DECIMAL = powerOfTenTable[LONGWORD_DECIMAL_DIGITS];
+
+  private static final int TWO_X_LONGWORD_DECIMAL_DIGITS = 2 * LONGWORD_DECIMAL_DIGITS;
+  private static final int THREE_X_LONGWORD_DECIMAL_DIGITS = 3 * LONGWORD_DECIMAL_DIGITS;
+  private static final int FOUR_X_LONGWORD_DECIMAL_DIGITS = 4 * LONGWORD_DECIMAL_DIGITS;
+
+  // 38 decimal maximum - 32 digits in 2 lower longs (6 digits here).
+  private static final int HIGHWORD_DECIMAL_DIGITS = MAX_DECIMAL_DIGITS - TWO_X_LONGWORD_DECIMAL_DIGITS;
+  private static final long MAX_HIGHWORD_DECIMAL =
+      powerOfTenTable[HIGHWORD_DECIMAL_DIGITS] - 1;
+
+  private static long HIGHWORD_DIVIDE_FACTOR = powerOfTenTable[LONGWORD_DECIMAL_DIGITS - HIGHWORD_DECIMAL_DIGITS];
+  private static long HIGHWORD_MULTIPLY_FACTOR = powerOfTenTable[HIGHWORD_DECIMAL_DIGITS];
+
+  // 38 * 2 or 76 full decimal maximum - (64 + 8) digits in 4 lower longs (4 digits here).
+  private static final long FULL_MAX_HIGHWORD_DECIMAL =
+      powerOfTenTable[MAX_DECIMAL_DIGITS * 2 - (FOUR_X_LONGWORD_DECIMAL_DIGITS + INTWORD_DECIMAL_DIGITS)] - 1;
+
+  /**
+   * BigInteger constants.
+   */
+
+  private static final BigInteger BIG_INTEGER_TWO = BigInteger.valueOf(2);
+  private static final BigInteger BIG_INTEGER_FIVE = BigInteger.valueOf(5);
+  private static final BigInteger BIG_INTEGER_TEN = BigInteger.valueOf(10);
+
+  public static final BigInteger BIG_INTEGER_MAX_DECIMAL =
+      BIG_INTEGER_TEN.pow(MAX_DECIMAL_DIGITS).subtract(BigInteger.ONE);
+
+  private static final BigInteger BIG_INTEGER_MAX_LONGWORD_DECIMAL =
+      BigInteger.valueOf(MAX_LONGWORD_DECIMAL);
+
+  private static final BigInteger BIG_INTEGER_LONGWORD_MULTIPLIER =
+      BigInteger.ONE.add(BIG_INTEGER_MAX_LONGWORD_DECIMAL);
+  private static final BigInteger BIG_INTEGER_LONGWORD_MULTIPLIER_2X =
+      BIG_INTEGER_LONGWORD_MULTIPLIER.multiply(BIG_INTEGER_LONGWORD_MULTIPLIER);
+  private static final BigInteger BIG_INTEGER_LONGWORD_MULTIPLIER_3X =
+      BIG_INTEGER_LONGWORD_MULTIPLIER_2X.multiply(BIG_INTEGER_LONGWORD_MULTIPLIER);
+  private static final BigInteger BIG_INTEGER_LONGWORD_MULTIPLIER_4X =
+      BIG_INTEGER_LONGWORD_MULTIPLIER_3X.multiply(BIG_INTEGER_LONGWORD_MULTIPLIER);
+
+  private static final BigInteger BIG_INTEGER_MAX_HIGHWORD_DECIMAL =
+      BigInteger.valueOf(MAX_HIGHWORD_DECIMAL);
+  private static final BigInteger BIG_INTEGER_HIGHWORD_MULTIPLIER =
+      BigInteger.ONE.add(BIG_INTEGER_MAX_HIGHWORD_DECIMAL);
+
+  // UTF-8 byte constants used by string/UTF-8 bytes to decimal and decimal to String/UTF-8 byte
+  // conversion.
+
+  // There is only one blank in UTF-8.
+  private final static byte BYTE_BLANK = (byte) ' ';
+
+  private final static byte BYTE_DIGIT_ZERO = (byte) '0';
+  private final static byte BYTE_DIGIT_NINE = (byte) '9';
+
+  // Decimal point.
+  private final static byte BYTE_DOT = (byte) '.';
+
+  // Sign.
+  private final static byte BYTE_MINUS = (byte) '-';
+  private final static byte BYTE_PLUS = (byte) '+';
+
+  // Exponent E or e.
+  private final static byte BYTE_EXPONENT_LOWER = (byte) 'e';
+  private final static byte BYTE_EXPONENT_UPPER = (byte) 'E';
+
+  //************************************************************************************************
+  // Initialize (fastSetFrom*).
+
+  /*
+   * All of the fastSetFrom* methods require the caller to pass a fastResult parameter has been
+   * reset for better performance.
+   */
+
+  private static void doRaiseSetFromBytesInvalid(
+      byte[] bytes, int offset, int length,
+      FastHiveDecimal fastResult) {
+    final int end = offset + length;
+    throw new RuntimeException(
+        "Invalid fast decimal \"" +
+            new String(bytes, offset, end) + "\"" +
+        " fastSignum " + fastResult.fastSignum + " fast0 " + fastResult.fast0 + " fast1 " + fastResult.fast1 + " fast2 " + fastResult.fast2 +
+            " fastIntegerDigitCount " + fastResult.fastIntegerDigitCount +" fastScale " + fastResult.fastScale +
+        " stack trace: " + getStackTraceAsSingleLine(Thread.currentThread().getStackTrace()));
+  }
+
+  /**
+   * Scan a byte array slice for a decimal number in UTF-8 bytes.
+   *
+   * Syntax:
+   *   [+|-][integerPortion][.[fractionalDigits]][{E|e}[+|-]exponent]
+   *                                                  // Where at least one integer or fractional
+   *                                                  // digit is required...
+   *
+   * We handle too many fractional digits by doing rounding ROUND_HALF_UP.
+   *
+   * NOTE: The fastSetFromBytes method requires the caller to pass a fastResult parameter has been
+   * reset for better performance.
+   *
+   * @param fastResult  True if the byte array slice was successfully converted to a decimal.
+   * @return
+   */
+  public static boolean fastSetFromBytes(byte[] bytes, int offset, int length, boolean trimBlanks,
+      FastHiveDecimal fastResult) {
+
+    final int bytesLength = bytes.length;
+
+    if (offset < 0 || offset >= bytesLength) {
+      return false;
+    }
+    final int end = offset + length;
+    if (end <= offset || end > bytesLength) {
+      return false;
+    }
+
+    // We start here with at least one byte.
+    int index = offset;
+
+    if (trimBlanks) {
+      while (bytes[index] == BYTE_BLANK) {
+        if (++index >= end) {
+          return false;
+        }
+      }
+    }
+
+    // Started with a few ideas from BigDecimal(char[] in, int offset, int len) constructor...
+    // But soon became very fast decimal specific.
+
+    boolean isNegative = false;
+    if (bytes[index] == BYTE_MINUS) {
+      isNegative = true;
+      if (++index >= end) {
+        return false;
+      }
+    } else if (bytes[index] == BYTE_PLUS) {
+      if (++index >= end) {
+        return false;
+      }
+    }
+
+    int precision = 0;
+
+    // We fill starting with highest digit in highest longword (HIGHWORD_DECIMAL_DIGITS) and
+    // move down.  At end will will shift everything down if necessary.
+
+    int longWordIndex = 0;   // Where 0 is the highest longword; 1 is middle longword, etc.
+
+    int digitNum = HIGHWORD_DECIMAL_DIGITS;
+    long multiplier = powerOfTenTable[HIGHWORD_DECIMAL_DIGITS - 1];
+
+    int digitValue = 0;
+    long longWord = 0;
+
+    long fast0 = 0;
+    long fast1 = 0;
+    long fast2 = 0;
+
+    byte work;
+
+    // Parse integer portion.
+
+    boolean haveInteger = false;
+    while (true) {
+      work = bytes[index];
+      if (work < BYTE_DIGIT_ZERO || work > BYTE_DIGIT_NINE) {
+        break;
+      }
+      haveInteger = true;
+      if (precision == 0 && work == BYTE_DIGIT_ZERO) {
+        // Ignore leading zeroes.
+        if (++index >= end) {
+          break;
+        }
+        continue;
+      }
+      digitValue = work - BYTE_DIGIT_ZERO;
+      if (digitNum == 0) {
+
+        // Integer parsing move to next lower longword.
+
+        // Save previous longword.
+        if (longWordIndex == 0) {
+          fast2 = longWord;
+        } else if (longWordIndex == 1) {
+          fast1 = longWord;
+        } else if (longWordIndex == 2) {
+
+          // We have filled HiveDecimal.MAX_PRECISION digits and have no more room in our limit precision
+          // fast decimal.
+          return false;
+        }
+        longWordIndex++;
+
+        // The middle and lowest longwords highest digit number is LONGWORD_DECIMAL_DIGITS.
+        digitNum = LONGWORD_DECIMAL_DIGITS;
+        multiplier = powerOfTenTable[LONGWORD_DECIMAL_DIGITS - 1];
+        longWord = 0;
+      }
+      longWord += digitValue * multiplier;
+      multiplier /= 10;
+      digitNum--;
+      precision++;
+      if (++index >= end) {
+        break;
+      }
+    }
+
+    // At this point we may have parsed an integer.
+
+    // Try to eat a dot now since it could be the end.  We remember if we saw a dot so we can
+    // do error checking later and detect just a dot.
+    boolean sawDot = false;
+    if (index < end && bytes[index] == BYTE_DOT) {
+      sawDot = true;
+      index++;
+    }
+
+    // Try to eat trailing blank padding.
+    if (trimBlanks && index < end && bytes[index] == BYTE_BLANK) {
+      index++;
+      while (index < end && bytes[index] == BYTE_BLANK) {
+        index++;
+      }
+      if (index < end) {
+        // Junk after trailing blank padding.
+        return false;
+      }
+      // Otherwise, fall through and process the what we saw before possible trailing blanks.
+    }
+
+    // Any more input?
+    if (index >= end) {
+
+      // We hit the end after getting optional integer and optional dot and optional blank padding.
+
+      if (!haveInteger) {
+        return false;
+      }
+
+      if (precision == 0) {
+
+        // We just had leading zeroes (and possibly a dot and trailing blanks).
+        // Value is 0.
+        return true;
+      }
+      // Save last longword.
+      if (longWordIndex == 0) {
+        fast2 = longWord;
+      } else if (longWordIndex == 1) {
+        fast1 = longWord;
+      } else {
+        fast0 = longWord;
+      }
+      fastResult.fastSignum = (isNegative ? -1 : 1);
+      fastResult.fastIntegerDigitCount = precision;
+      fastResult.fastScale = 0;
+      final int scaleDown = HiveDecimal.MAX_PRECISION - precision;
+      if (scaleDown > 0) {
+        doFastScaleDown(fast0, fast1, fast2, scaleDown, fastResult);
+      } else {
+        fastResult.fast0 = fast0;
+        fastResult.fast1 = fast1;
+        fastResult.fast2 = fast2;
+      }
+      return true;
+    }
+
+    // We have more input but did we start with something valid?
+    if (!haveInteger && !sawDot) {
+
+      // Must have one of those at this point.
+      return false;
+    }
+
+    int integerDigitCount = precision;
+
+    int nonTrailingZeroScale = 0;
+    boolean roundingNecessary = false;
+    if (sawDot) {
+
+      // Parse fraction portion.
+
+      while (true) {
+        work = bytes[index];
+        if (work < BYTE_DIGIT_ZERO || work > BYTE_DIGIT_NINE) {
+          if (!haveInteger) {
+
+            // Naked dot.
+            return false;
+          }
+          break;
+        }
+        digitValue = work - BYTE_DIGIT_ZERO;
+        if (digitNum == 0) {
+
+          // Fraction digit parsing move to next lower longword.
+
+          // Save previous longword.
+          if (longWordIndex == 0) {
+            fast2 = longWord;
+          } else if (longWordIndex == 1) {
+            fast1 = longWord;
+          } else if (longWordIndex == 2) {
+
+            // We have filled HiveDecimal.MAX_PRECISION digits and have no more room in our limit precision
+            // fast decimal.  However, since we are processing fractional digits, we do rounding.
+            // away.
+            if (digitValue >= 5) {
+              roundingNecessary = true;
+            }
+
+            // Scan through any remaining digits...
+            while (++index < end) {
+              work = bytes[index];
+              if (work < BYTE_DIGIT_ZERO || work > BYTE_DIGIT_NINE) {
+                break;
+              }
+            }
+            break;
+          }
+          longWordIndex++;
+          digitNum = LONGWORD_DECIMAL_DIGITS;
+          multiplier = powerOfTenTable[digitNum - 1];
+          longWord = 0;
+        }
+        longWord += digitValue * multiplier;
+        multiplier /= 10;
+        digitNum--;
+        precision++;
+        if (digitValue != 0) {
+          nonTrailingZeroScale = precision - integerDigitCount;
+        }
+        if (++index >= end) {
+          break;
+        }
+      }
+    }
+
+    boolean haveExponent = false;
+    if (index < end &&
+        (bytes[index] == BYTE_EXPONENT_UPPER || bytes[index] == BYTE_EXPONENT_LOWER)) {
+      haveExponent = true;
+      index++;
+      if (index >= end) {
+        // More required.
+        return false;
+      }
+    }
+
+    // At this point we have a number.  Save it in fastResult.  Round it.  If we have an exponent,
+    // we will do a power 10 operation on fastResult.
+
+    // Save last longword.
+    if (longWordIndex == 0) {
+      fast2 = longWord;
+    } else if (longWordIndex == 1) {
+      fast1 = longWord;
+    } else {
+      fast0 = longWord;
+    }
+
+    int trailingZeroesScale = precision - integerDigitCount;
+    if (integerDigitCount == 0 && nonTrailingZeroScale == 0) {
+      // Zero(es).
+    } else {
+      fastResult.fastSignum = (isNegative ? -1 : 1);
+      fastResult.fastIntegerDigitCount = integerDigitCount;
+      fastResult.fastScale = nonTrailingZeroScale;
+      final int trailingZeroCount = trailingZeroesScale - fastResult.fastScale;
+      final int scaleDown = HiveDecimal.MAX_PRECISION - precision + trailingZeroCount;
+      if (scaleDown > 0) {
+        doFastScaleDown(fast0, fast1, fast2, scaleDown, fastResult);
+      } else {
+        fastResult.fast0 = fast0;
+        fastResult.fast1 = fast1;
+        fastResult.fast2 = fast2;
+      }
+    }
+
+    if (roundingNecessary) {
+
+      if (fastResult.fastSignum == 0) {
+        fastResult.fastSignum = (isNegative ? -1 : 1);
+        fastResult.fast0 = 1;
+        fastResult.fastIntegerDigitCount = 0;
+        fastResult.fastScale = HiveDecimal.MAX_SCALE;
+      } else {
+        if (!fastAdd(
+          fastResult.fastSignum, fastResult.fast0, fastResult.fast1, fastResult.fast2,
+          fastResult.fastIntegerDigitCount, fastResult.fastScale,
+          fastResult.fastSignum, 1, 0, 0, 0, trailingZeroesScale,
+          fastResult)) {
+          return false;
+        }
+      }
+    }
+
+    if (!haveExponent) {
+
+      // Try to eat trailing blank padding.
+      if (trimBlanks && index < end && bytes[index] == BYTE_BLANK) {
+        index++;
+        while (index < end && bytes[index] == BYTE_BLANK) {
+          index++;
+        }
+      }
+      if (index < end) {
+        // Junk after trailing blank padding.
+        return false;
+      }
+      return true;
+    }
+
+    // At this point, we have seen the exponent letter E or e and have decimal information as:
+    //     isNegative, precision, integerDigitCount, nonTrailingZeroScale, and
+    //     fast0, fast1, fast2.
+    //
+    // After we determine the exponent, we will do appropriate scaling and fill in fastResult.
+
+    boolean isExponentNegative = false;
+    if (bytes[index] == BYTE_MINUS) {
+      isExponentNegative = true;
+      if (++index >= end) {
+        return false;
+      }
+    } else if (bytes[index] == BYTE_PLUS) {
+      if (++index >= end) {
+        return false;
+      }
+    }
+
+    long exponent = 0;
+    multiplier = 1;
+    while (true) {
+      work = bytes[index];
+      if (work < BYTE_DIGIT_ZERO || work > BYTE_DIGIT_NINE) {
+        break;
+      }
+      if (multiplier > 10) {
+        // Power of ten way beyond our precision/scale...
+        return false;
+      }
+      digitValue = work - BYTE_DIGIT_ZERO;
+      if (digitValue != 0 || exponent != 0) {
+        exponent = exponent * 10 + digitValue;
+        multiplier *= 10;
+      }
+      if (++index >= end) {
+        break;
+      }
+    }
+    if (isExponentNegative) {
+      exponent = -exponent;
+    }
+
+    // Try to eat trailing blank padding.
+    if (trimBlanks && index < end && bytes[index] == BYTE_BLANK) {
+      index++;
+      while (index < end && bytes[index] == BYTE_BLANK) {
+        index++;
+      }
+    }
+    if (index < end) {
+      // Junk after exponent.
+      return false;
+    }
+
+
+    if (integerDigitCount == 0 && nonTrailingZeroScale == 0) {
+      // Zero(es).
+      return true;
+    }
+
+    if (exponent == 0) {
+
+      // No effect since 10^0 = 1.
+
+    } else {
+
+      // We for these input with exponents, we have at this point an intermediate decimal,
+      // an exponent power, and a result:
+      //
+      //                     intermediate
+      //   input               decimal      exponent        result
+      // 701E+1            701 scale 0        +1            7010 scale 0
+      // 3E+4              3 scale 0          +4               3 scale 0
+      // 3.223E+9          3.223 scale 3      +9      3223000000 scale 0
+      // 0.009E+10         0.009 scale 4      +10       90000000 scale 0
+      // 0.3221E-2         0.3221 scale 4     -2               0.003221 scale 6
+      // 0.00223E-20       0.00223 scale 5    -20              0.0000000000000000000000223 scale 25
+      //
+
+      if (!fastScaleByPowerOfTen(
+          fastResult,
+          (int) exponent,
+          fastResult)) {
+        return false;
+      }
+    }
+
+    final int trailingZeroCount =
+        fastTrailingDecimalZeroCount(
+            fastResult.fast0, fastResult.fast1, fastResult.fast2,
+            fastResult.fastIntegerDigitCount, fastResult.fastScale);
+    if (trailingZeroCount > 0) {
+      doFastScaleDown(
+          fastResult,
+          trailingZeroCount,
+          fastResult);
+      fastResult.fastScale -= trailingZeroCount;
+    }
+
+    return true;
+  }
+
+  /**
+   * Scans a byte array slice for UNSIGNED RAW DIGITS ONLY in UTF-8 (ASCII) characters
+   * and forms a decimal from the digits and a sign and scale.
+   *
+   * Designed for BinarySortable serialization format that separates the sign and scale
+   * from the raw digits.
+   *
+   * NOTE: The fastSetFromDigitsOnlyBytesAndScale method requires the caller to pass a fastResult
+   * parameter has been reset for better performance.
+   *
+   * @return True if the sign, digits, and scale were successfully converted to a decimal.
+   */
+  public static boolean fastSetFromDigitsOnlyBytesAndScale(
+      boolean isNegative, byte[] bytes, int offset, int length, int scale,
+      FastHiveDecimal fastResult) {
+
+    final int bytesLength = bytes.length;
+
+    if (offset < 0 || offset >= bytesLength) {
+      return false;
+    }
+    final int end = offset + length;
+    if (end <= offset || end > bytesLength) {
+      return false;
+    }
+
+    // We start here with at least one byte.
+    int index = offset;
+
+    // A stripped down version of fastSetFromBytes.
+
+    int precision = 0;
+
+    // We fill starting with highest digit in highest longword (HIGHWORD_DECIMAL_DIGITS) and
+    // move down.  At end will will shift everything down if necessary.
+
+    int longWordIndex = 0;   // Where 0 is the highest longword; 1 is middle longword, etc.
+
+    int digitNum = HIGHWORD_DECIMAL_DIGITS;
+    long multiplier = powerOfTenTable[HIGHWORD_DECIMAL_DIGITS - 1];
+
+    int digitValue;
+    long longWord = 0;
+
+    long fast0 = 0;
+    long fast1 = 0;
+    long fast2 = 0;
+
+    byte work;
+
+    // Parse digits.
+
+    boolean haveInteger = false;
+    while (true) {
+      work = bytes[index];
+      if (work < BYTE_DIGIT_ZERO || work > BYTE_DIGIT_NINE) {
+        if (!haveInteger) {
+          return false;
+        }
+        break;
+      }
+      haveInteger = true;
+      if (precision == 0 && work == BYTE_DIGIT_ZERO) {
+        // Ignore leading zeroes.
+        if (++index >= end) {
+          break;
+        }
+        continue;
+      }
+      digitValue = work - BYTE_DIGIT_ZERO;
+      if (digitNum == 0) {
+
+        // Integer parsing move to next lower longword.
+
+        // Save previous longword.
+        if (longWordIndex == 0) {
+          fast2 = longWord;
+        } else if (longWordIndex == 1) {
+          fast1 = longWord;
+        } else if (longWordIndex == 2) {
+
+          // We have filled HiveDecimal.MAX_PRECISION digits and have no more room in our limit precision
+          // fast decimal.
+          return false;
+        }
+        longWordIndex++;
+
+        // The middle and lowest longwords highest digit number is LONGWORD_DECIMAL_DIGITS.
+        digitNum = LONGWORD_DECIMAL_DIGITS;
+        multiplier = powerOfTenTable[LONGWORD_DECIMAL_DIGITS - 1];
+        longWord = 0;
+      }
+      longWord += digitValue * multiplier;
+      multiplier /= 10;
+      digitNum--;
+      precision++;
+      if (++index >= end) {
+        break;
+      }
+    }
+
+    // Just an digits?
+    if (index < end) {
+      return false;
+    }
+
+    if (precision == 0) {
+      // We just had leading zeroes.
+      // Value is 0.
+      return true;
+    }
+
+    // Save last longword.
+    if (longWordIndex == 0) {
+      fast2 = longWord;
+    } else if (longWordIndex == 1) {
+      fast1 = longWord;
+    } else {
+      fast0 = longWord;
+    }
+    fastResult.fastSignum = (isNegative ? -1 : 1);
+    fastResult.fastIntegerDigitCount = Math.max(0, precision - scale);
+    fastResult.fastScale = scale;
+    final int scaleDown = HiveDecimal.MAX_PRECISION - precision;
+    if (scaleDown > 0) {
+      doFastScaleDown(fast0, fast1, fast2, scaleDown, fastResult);
+    } else {
+      fastResult.fast0 = fast0;
+      fastResult.fast1 = fast1;
+      fastResult.fast2 = fast2;
+    }
+    return true;
+
+  }
+
+  /**
+   * Scale down a BigInteger by a power of 10 and round off if necessary using ROUND_HALF_UP.
+   * @return The scaled and rounded BigInteger.
+   */
+  private static BigInteger doBigIntegerScaleDown(BigInteger unscaledValue, int scaleDown) {
+    BigInteger[] quotientAndRemainder = unscaledValue.divideAndRemainder(BigInteger.TEN.pow(scaleDown));
+    BigInteger quotient = quotientAndRemainder[0];
+    BigInteger round = quotientAndRemainder[1].divide(BigInteger.TEN.pow(scaleDown - 1));
+    if (round.compareTo(BIG_INTEGER_FIVE) >= 0) {
+      quotient = quotient.add(BigInteger.ONE);
+    }
+    return quotient;
+  }
+
+  /**
+   * Create a fast decimal from a BigDecimal.
+   *
+   * NOTE: The fastSetFromBigDecimal method requires the caller to pass a fastResult
+   * parameter has been reset for better performance.
+   *
+   * @return True if the BigDecimal could be converted to our decimal representation.
+   */
+  public static boolean fastSetFromBigDecimal(
+      BigDecimal bigDecimal, boolean allowRounding, FastHiveDecimal fastResult) {
+
+    // We trim the trailing zero fraction digits so we don't cause unnecessary precision
+    // overflow later.
+    if (bigDecimal.signum() == 0) {
+      if (bigDecimal.scale() != 0) {
+
+        // For some strange reason BigDecimal 0 can have a scale.  We do not support that.
+        bigDecimal = BigDecimal.ZERO;
+      }
+    } else {
+      BigDecimal bigDecimalStripped = bigDecimal.stripTrailingZeros();
+      int stripTrailingZerosScale = bigDecimalStripped.scale();
+      // System.out.println("FAST_SET_FROM_BIG_DECIMAL bigDecimal " + bigDecimal);
+      // System.out.println("FAST_SET_FROM_BIG_DECIMAL bigDecimalStripped " + bigDecimalStripped);
+      // System.out.println("FAST_SET_FROM_BIG_DECIMAL stripTrailingZerosScale " + stripTrailingZerosScale);
+      if (stripTrailingZerosScale < 0) {
+
+        // The trailing zeroes extend into the integer part -- we only want to eliminate the
+        // fractional zero digits.
+
+        bigDecimal = bigDecimal.setScale(0);
+      } else {
+
+        // Ok, use result with some or all fractional digits stripped.
+
+        bigDecimal = bigDecimalStripped;
+      }
+    }
+    // System.out.println("FAST_SET_FROM_BIG_DECIMAL adjusted for zeroes/scale " + bigDecimal + " scale " + bigDecimal.scale());
+
+    BigInteger unscaledValue = bigDecimal.unscaledValue();
+    // System.out.println("FAST_SET_FROM_BIG_DECIMAL unscaledValue " + unscaledValue + " length " + unscaledValue.toString().length());
+
+    final int scale = bigDecimal.scale();
+    if (!allowRounding) {
+      if (scale < 0 || scale > HiveDecimal.MAX_SCALE) {
+        return false;
+      }
+      // The digits must fit without rounding.
+      if (!fastSetFromBigInteger(unscaledValue, fastResult)) {
+        return false;
+      }
+      if (fastResult.fastSignum != 0) {
+        fastResult.fastIntegerDigitCount = Math.max(0, fastResult.fastIntegerDigitCount - scale);
+        fastResult.fastScale = scale;
+      }
+      return true;
+    }
+    // This method will scale down and round to fit, if necessary.
+    if (!fastSetFromBigInteger(unscaledValue, scale, fastResult)) {
+      return false;
+    }
+    return true;
+  }
+
+  /**
+   * Scan a String for a decimal number in UTF-8 characters.
+   *
+   * NOTE: The fastSetFromString method requires the caller to pass a fastResult
+   * parameter has been reset for better performance.
+   *
+   * @return True if the String was successfully converted to a decimal.
+   */
+  public static boolean fastSetFromString(
+      String string, boolean trimBlanks, FastHiveDecimal result) {
+    byte[] bytes = string.getBytes();
+    return fastSetFromBytes(bytes, 0, bytes.length, trimBlanks, result);
+  }
+
+  /**
+   * Creates a scale 0 fast decimal from an int.
+   *
+   * NOTE: The fastSetFromString method requires the caller to pass a fastResult
+   * parameter has been reset for better performance.
+   *
+   */
+  public static void fastSetFromInt(int intValue, FastHiveDecimal fastResult) {
+    if (intValue == 0) {
+      // Zero special case.
+      return;
+    }
+    if (intValue > 0) {
+      fastResult.fastSignum = 1;
+    } else {
+      fastResult.fastSignum = -1;
+      intValue = Math.abs(intValue);
+    }
+    // 10 digit int is all in lowest 16 decimal digit longword.
+    // Since we are creating with scale 0, no fraction digits to zero trim.
+    fastResult.fast0 = intValue & 0xFFFFFFFFL;
+    fastResult.fastIntegerDigitCount =
+        fastLongWordPrecision(fastResult.fast0);
+  }
+
+  /**
+   * Creates a scale 0 fast decimal from a long.
+   *
+   * NOTE: The fastSetFromLong method requires the caller to pass a fastResult
+   * parameter has been reset for better performance.
+   *
+   */
+  public static void fastSetFromLong(
+      long longValue, FastHiveDecimal fastResult) {
+    if (longValue == 0) {
+      // Zero special case.
+      return;
+    }
+    // Handle minimum integer case that doesn't have abs().
+    if (longValue == Long.MIN_VALUE) {
+      // Split -9,223,372,036,854,775,808 into 16 digit middle and lowest longwords by hand.
+      fastResult.fastSignum = -1;
+      fastResult.fast1 = 922L;
+      fastResult.fast0 = 3372036854775808L;
+      fastResult.fastIntegerDigitCount = 19;
+    } else {
+      if (longValue > 0) {
+        fastResult.fastSignum = 1;
+      } else {
+        fastResult.fastSignum = -1;
+        longValue = Math.abs(longValue);
+      }
+      // Split into 16 digit middle and lowest longwords remainder / division.
+      fastResult.fast1 = longValue / MULTIPLER_LONGWORD_DECIMAL;
+      fastResult.fast0 = longValue % MULTIPLER_LONGWORD_DECIMAL;
+      if (fastResult.fast1 != 0) {
+        fastResult.fastIntegerDigitCount =
+            LONGWORD_DECIMAL_DIGITS + fastLongWordPrecision(fastResult.fast1);
+      } else {
+        fastResult.fastIntegerDigitCount =
+            fastLongWordPrecision(fastResult.fast0);
+      }
+    }
+    return;
+  }
+
+  /**
+   * Creates a fast decimal from a long with a specified scale.
+   *
+   * NOTE: The fastSetFromLongAndScale method requires the caller to pass a fastResult
+   * parameter has been reset for better performance.
+   *
+   */
+  public static boolean fastSetFromLongAndScale(
+      long longValue, int scale, FastHiveDecimal fastResult) {
+
+    if (scale < 0 || scale > HiveDecimal.MAX_SCALE) {
+      return false;
+    }
+
+    fastSetFromLong(longValue, fastResult);
+    if (scale == 0) {
+      return true;
+    }
+
+    if (!fastScaleByPowerOfTen(
+        fastResult,
+        -scale,
+        fastResult)) {
+      return false;
+    }
+    return true;
+  }
+
+  /**
+   * Creates fast decimal from a float.
+   *
+   * NOTE: The fastSetFromFloat method requires the caller to pass a fastResult
+   * parameter has been reset for better performance.
+   *
+   */
+  public static boolean fastSetFromFloat(
+      float floatValue, FastHiveDecimal fastResult) {
+
+    String floatString = Float.toString(floatValue);
+    return fastSetFromString(floatString, false, fastResult);
+
+  }
+
+  /**
+   * Creates fast decimal from a double.
+   *
+   * NOTE: The fastSetFromDouble method requires the caller to pass a fastResult
+   * parameter has been reset for better performance.
+   *
+   */
+  public static boolean fastSetFromDouble(
+      double doubleValue, FastHiveDecimal fastResult) {
+
+    String doubleString = Double.toString(doubleValue);
+    return fastSetFromString(doubleString, false, fastResult);
+
+  }
+
+  /**
+   * Creates a fast decimal from a BigInteger with scale 0.
+   *
+   * For efficiency, we assume that fastResult is fastReset.  This method does not set the
+   * fastScale field.
+   *
+   * NOTE: The fastSetFromBigInteger method requires the caller to pass a fastResult
+   * parameter has been reset for better performance.
+   *
+   * @return Return true if the BigInteger value fit within HiveDecimal.MAX_PRECISION.  Otherwise,
+   *         false for overflow.
+   */
+  public static boolean fastSetFromBigInteger(
+      BigInteger bigInteger, FastHiveDecimal fastResult) {
+
+    final int signum = bigInteger.signum();
+    if (signum == 0) {
+      // Zero special case.
+      return true;
+    }
+    fastResult.fastSignum = signum;
+    if (signum == -1) {
+      bigInteger = bigInteger.negate();
+    }
+    if (bigInteger.compareTo(BIG_INTEGER_LONGWORD_MULTIPLIER) < 0) {
+
+      // Fits in one longword.
+      fastResult.fast0 = bigInteger.longValue();
+      if (fastResult.fast0 == 0) {
+        fastResult.fastSignum = 0;
+      } else {
+        fastResult.fastIntegerDigitCount = fastLongWordPrecision(fastResult.fast0);
+      }
+      return true;
+    }
+    BigInteger[] quotientAndRemainder =
+        bigInteger.divideAndRemainder(BIG_INTEGER_LONGWORD_MULTIPLIER);
+    fastResult.fast0 = quotientAndRemainder[1].longValue();
+    BigInteger quotient = quotientAndRemainder[0];
+    if (quotient.compareTo(BIG_INTEGER_LONGWORD_MULTIPLIER) < 0) {
+
+      // Fits in two longwords.
+      fastResult.fast1 = quotient.longValue();
+      if (fastResult.fast0 == 0 && fastResult.fast1 == 0) {
+        // The special case zero logic at the beginning should have caught this.
+        throw new RuntimeException("Unexpected");
+      } else {
+        fastResult.fastIntegerDigitCount =
+            LONGWORD_DECIMAL_DIGITS + fastLongWordPrecision(fastResult.fast1);
+      }
+      return true;
+    }
+
+    // Uses all 3 decimal longs.
+    quotientAndRemainder =
+        quotient.divideAndRemainder(BIG_INTEGER_LONGWORD_MULTIPLIER);
+    fastResult.fast1 = quotientAndRemainder[1].longValue();
+    quotient = quotientAndRemainder[0];
+    if (quotient.compareTo(BIG_INTEGER_HIGHWORD_MULTIPLIER) >= 0) {
+      // Overflow.
+      return false;
+    }
+    fastResult.fast2 = quotient.longValue();
+    if (fastResult.fast0 == 0 && fastResult.fast1 == 0 && fastResult.fast2 == 0) {
+      fastResult.fastSignum = 0;
+    } else {
+      fastResult.fastIntegerDigitCount =
+          TWO_X_LONGWORD_DECIMAL_DIGITS + fastHighWordPrecision(fastResult.fast2);
+    }
+    return true;
+  }
+
+  /**
+   * Creates a fast decimal from a BigInteger with a specified scale.
+   *
+   * NOTE: The fastSetFromBigInteger method requires the caller to pass a fastResult
+   * parameter has been reset for better performance.
+   *
+   * @return True if the BigInteger and scale were successfully converted to a decimal.
+   */
+  public static boolean fastSetFromBigInteger(
+      BigInteger bigInteger, int scale, FastHiveDecimal fastResult) {
+
+    if (scale < 0) {
+
+      // Multiply by 10^(-scale) to normalize.  We do not use negative scale in our representation.
+      //
+      // Example:
+      //    4.172529E+20 has a negative scale -20 since scale is number of digits below the dot.
+      //    417252900000000000000 normalized as scale 0.
+      //
+      bigInteger = bigInteger.multiply(BIG_INTEGER_TEN.pow(-scale));
+      scale = 0;
+    }
+
+    int signum = bigInteger.signum();
+    if (signum == 0) {
+      // Zero.
+      return true;
+    } else if (signum == -1) {
+      // Normalize to positive.
+      bigInteger = bigInteger.negate();
+    }
+
+    // A slow way to get the number of decimal digits.
+    int precision = bigInteger.toString().length();
+
+    // System.out.println("FAST_SET_FROM_BIG_INTEGER adjusted bigInteger " + bigInteger + " precision " + precision);
+
+    int integerDigitCount = precision - scale;
+    // System.out.println("FAST_SET_FROM_BIG_INTEGER integerDigitCount " + integerDigitCount + " scale " + scale);
+    int maxScale;
+    if (integerDigitCount >= 0) {
+      if (integerDigitCount > HiveDecimal.MAX_PRECISION) {
+        return false;
+      }
+      maxScale = HiveDecimal.MAX_SCALE - integerDigitCount;
+    } else {
+      maxScale = HiveDecimal.MAX_SCALE;
+    }
+    // System.out.println("FAST_SET_FROM_BIG_INTEGER maxScale " + maxScale);
+
+    if (scale > maxScale) {
+
+      // A larger scale is ok -- we will knock off lower digits and round.
+
+      final int trimAwayCount = scale - maxScale;
+      // System.out.println("FAST_SET_FROM_BIG_INTEGER trimAwayCount " + trimAwayCount);
+      if (trimAwayCount > 1) {
+        // First, throw away digits below round digit.
+        BigInteger bigIntegerThrowAwayBelowRoundDigitDivisor = BIG_INTEGER_TEN.pow(trimAwayCount - 1);
+        bigInteger = bigInteger.divide(bigIntegerThrowAwayBelowRoundDigitDivisor);
+      }
+      // System.out.println("FAST_SET_FROM_BIG_INTEGER with round digit bigInteger " + bigInteger + " length " + bigInteger.toString().length());
+
+      BigInteger[] quotientAndRemainder = bigInteger.divideAndRemainder(BIG_INTEGER_TEN);
+      // System.out.println("FAST_SET_FROM_BIG_INTEGER quotientAndRemainder " + Arrays.toString(quotientAndRemainder));
+
+      BigInteger quotient = quotientAndRemainder[0];
+      if (quotientAndRemainder[1].intValue() >= 5) {
+        if (quotient.equals(BIG_INTEGER_MAX_DECIMAL)) {
+
+          // 38 9's digits.
+          // System.out.println("FAST_SET_FROM_BIG_INTEGER quotient is BIG_INTEGER_MAX_DECIMAL");
+
+          if (maxScale == 0) {
+            // No room above for rounding.
+            return false;
+          }
+
+          // System.out.println("FAST_SET_FROM_BIG_INTEGER reached here... scale " + scale + " maxScale " + maxScale);
+          // Rounding results in 10^N.
+          bigInteger = BIG_INTEGER_TEN.pow(integerDigitCount);
+          maxScale = 0;
+        } else {
+
+          // Round up.
+          bigInteger = quotient.add(BigInteger.ONE);
+        }
+      } else {
+
+        // No rounding.
+        bigInteger = quotient;
+      }
+      scale = maxScale;
+    }
+    if (!fastSetFromBigInteger(bigInteger, fastResult)) {
+      return false;
+    }
+
+    if (fastResult.fast0 == 0 && fastResult.fast1 == 0 && fastResult.fast2 == 0) {
+      fastResult.fastSignum = 0;
+    } else {
+      fastResult.fastSignum = signum;
+      fastResult.fastIntegerDigitCount = Math.max(0, fastResult.fastIntegerDigitCount - scale);
+      fastResult.fastScale = scale;
+
+      final int trailingZeroCount =
+          fastTrailingDecimalZeroCount(
+              fastResult.fast0, fastResult.fast1, fastResult.fast2,
+              fastResult.fastIntegerDigitCount, scale);
+      if (trailingZeroCount > 0) {
+        doFastScaleDown(
+            fastResult,
+            trailingZeroCount,
+            fastResult);
+        fastResult.fastScale -= trailingZeroCount;
+      }
+    }
+
+    return true;
+  }
+
+  //************************************************************************************************
+  // Take Integer or Fractional Portion.
+
+  /**
+   * Creates fast decimal from the fraction portion of a fast decimal.
+   *
+   * NOTE: The fastFractionPortion method requires the caller to pass a fastResult
+   * parameter has been reset for better performance.
+   */
+  public static void fastFractionPortion(
+      int fastSignum, long fast0, long fast1, long fast2,
+      int fastIntegerDigitCount, int fastScale,
+      FastHiveDecimal fastResult) {
+
+    if (fastSignum == 0 || fastScale == 0) {
+      fastResult.fastReset();
+      return;
+    }
+
+    // Clear integer portion; keep fraction.
+
+    // Adjust all longs using power 10 division/remainder.
+    long result0;
+    long result1;
+    long result2;
+    if (fastScale < LONGWORD_DECIMAL_DIGITS) {
+
+      // Part of lowest word survives.
+
+      final long clearFactor = powerOfTenTable[fastScale];
+
+      result0 = fast0 % clearFactor;
+      result1 = 0;
+      result2 = 0;
+
+    } else if (fastScale < TWO_X_LONGWORD_DECIMAL_DIGITS) {
+
+      // Throw away lowest word.
+
+      final int adjustedScaleDown = fastScale - LONGWORD_DECIMAL_DIGITS;
+
+      final long clearFactor = powerOfTenTable[adjustedScaleDown];
+
+      result0 = fast0;
+      result1 = fast1 % clearFactor;
+      result2 = 0;
+
+    } else {
+
+      // Throw away middle and lowest words.
+
+      final int adjustedScaleDown = fastScale - 2*LONGWORD_DECIMAL_DIGITS;
+
+      final long clearFactor = powerOfTenTable[adjustedScaleDown];
+
+      result0 = fast0;
+      result1 = fast1;
+      result2 = fast2 % clearFactor;
+
+    }
+    if (result0 == 0 && result1 == 0 && result2 == 0) {
+      fastResult.fastReset();
+    } else {
+      fastResult.fastSet(fastSignum, result0, result1, result2, /* fastIntegerDigitCount */ 0, fastScale);
+    }
+  }
+
+  /**
+   * Creates fast decimal from the integer portion.
+   *
+   * NOTE: The fastFractionPortion method requires the caller to pass a fastResult
+   * parameter has been reset for better performance.
+   */
+  public static void fastIntegerPortion(
+      int fastSignum, long fast0, long fast1, long fast2,
+      int fastIntegerDigitCount, int fastScale,
+      FastHiveDecimal fastResult) {
+
+    if (fastSignum == 0) {
+      fastResult.fastReset();
+      return;
+    }
+    if (fastScale == 0) {
+      fastResult.fastSet(fastSignum, fast0, fast1, fast2, fastIntegerDigitCount, fastScale);
+    }
+
+    // Scale down no rounding to clear fraction.
+    fastResult.fastSignum = fastSignum;
+    doFastScaleDown(
+        fast0, fast1, fast2,
+        fastScale,
+        fastResult);
+    fastResult.fastIntegerDigitCount = fastIntegerDigitCount;
+    fastResult.fastScale = 0;
+  }
+
+  //************************************************************************************************
+  // Binary to Decimal Conversion.
+
+  /**
+   * Convert 3 binary words of N bits each to a fast decimal (scale 0).
+   *
+   * The 3 binary words highWord, middleWord, and lowerWord form a large binary value:
+   *
+   *    highWord * 2^(M+L) + middleWord * 2^L + lowerWord.
+   *
+   * Where L is the number of bits in the lower word; M is the number of bits in the middle word.
+   * We let L and M be different to support the SerializationUtil serialization where the lower
+   * word is 62 bits and the remaining words are 63 bits...
+   *
+   * The fast decimal middleWordMultiplier is 2^L.
+   * The fast decimal highWordMultiplier is 2^(M+L).
+   *
+   * @return True if the conversion of the 3 binary words to decimal was successful.
+   */
+  public static boolean doBinaryToDecimalConversion(
+      long lowerWord, long middleWord, long highWord,
+      FastHiveDecimal middleWordMultiplier,
+      FastHiveDecimal highWordMultiplier,
+      FastHiveDecimal fastResult) {
+
+    /*
+     * Challenge: How to do the math to get this raw binary back to our decimal form.
+     *
+     * Briefly, for the middle and upper binary words, convert the middle/upper word into a decimal
+     * long words and then multiply those by the binary word's power of 2.
+     *
+     * And, add the multiply results into the result decimal longwords.
+     *
+     */
+    long result0 =
+        lowerWord % MULTIPLER_LONGWORD_DECIMAL;
+    long result1 =
+        lowerWord / MULTIPLER_LONGWORD_DECIMAL;
+    long result2 = 0;
+
+    if (middleWord != 0 || highWord != 0) {
+
+      if (highWord == 0) {
+  
+        // Form result from lower and middle words.
+
+        if (!fastMultiply5x5HalfWords(
+            middleWord % MULTIPLER_LONGWORD_DECIMAL,
+            middleWord / MULTIPLER_LONGWORD_DECIMAL,
+            0,
+            middleWordMultiplier.fast0, middleWordMultiplier.fast1, middleWordMultiplier.fast2,
+            fastResult)) {
+          return false;
+        }
+
+        final long calc0 =
+            result0
+          + fastResult.fast0;
+        result0 =
+            calc0 % MULTIPLER_LONGWORD_DECIMAL;
+        final long calc1 =
+            calc0 / MULTIPLER_LONGWORD_DECIMAL
+          + result1
+          + fastResult.fast1;
+        result1 =
+            calc1 % MULTIPLER_LONGWORD_DECIMAL;
+        result2 =
+            calc1 / MULTIPLER_LONGWORD_DECIMAL
+          + fastResult.fast2;
+
+      } else if (middleWord == 0) {
+
+        // Form result from lower and high words.
+
+        if (!fastMultiply5x5HalfWords(
+            highWord % MULTIPLER_LONGWORD_DECIMAL,
+            highWord / MULTIPLER_LONGWORD_DECIMAL,
+            0,
+            highWordMultiplier.fast0, highWordMultiplier.fast1, highWordMultiplier.fast2,
+            fastResult)) {
+          return false;
+        }
+
+        final long calc0 =
+            result0
+          + fastResult.fast0;
+        result0 =
+            calc0 % MULTIPLER_LONGWORD_DECIMAL;
+        final long calc1 =
+            calc0 / MULTIPLER_LONGWORD_DECIMAL
+          + result1
+          + fastResult.fast1;
+        result1 =
+            calc1 % MULTIPLER_LONGWORD_DECIMAL;
+        result2 =
+            calc1 / MULTIPLER_LONGWORD_DECIMAL
+          + fastResult.fast2;
+
+      } else {
+
+        // Form result from lower, middle, and middle words.
+
+        if (!fastMultiply5x5HalfWords(
+            middleWord % MULTIPLER_LONGWORD_DECIMAL,
+            middleWord / MULTIPLER_LONGWORD_DECIMAL,
+            0,
+            middleWordMultiplier.fast0, middleWordMultiplier.fast1, middleWordMultiplier.fast2,
+            fastResult)) {
+          return false;
+        }
+
+        long middleResult0 = fastResult.fast0;
+        long middleResult1 = fastResult.fast1;
+        long middleResult2 = fastResult.fast2;
+
+        if (!fastMultiply5x5HalfWords(
+            highWord % MULTIPLER_LONGWORD_DECIMAL,
+            highWord / MULTIPLER_LONGWORD_DECIMAL,
+            0,
+            highWordMultiplier.fast0, highWordMultiplier.fast1, highWordMultiplier.fast2,
+            fastResult)) {
+          return false;
+        }
+
+        long calc0 =
+            result0
+          + middleResult0
+          + fastResult.fast0;
+        result0 =
+            calc0 % MULTIPLER_LONGWORD_DECIMAL;
+        long calc1 =
+            calc0 / MULTIPLER_LONGWORD_DECIMAL
+          + result1
+          + middleResult1
+          + fastResult.fast1;
+        result1 =
+            calc1 % MULTIPLER_LONGWORD_DECIMAL;
+        result2 =
+            calc1 / MULTIPLER_LONGWORD_DECIMAL
+          + middleResult2
+          + fastResult.fast2;
+      }
+    }
+
+    // Let caller set negative sign if necessary.
+    if (result2 != 0) {
+      fastResult.fastIntegerDigitCount = TWO_X_LONGWORD_DECIMAL_DIGITS + fastHighWordPrecision(result2);
+      fastResult.fastSignum = 1;
+    } else if (result1 != 0) {
+      fastResult.fastIntegerDigitCount = LONGWORD_DECIMAL_DIGITS + fastHighWordPrecision(result1);
+      fastResult.fastSignum = 1;
+    } else if (result0 != 0) {
+      fastResult.fastIntegerDigitCount = fastHighWordPrecision(result0);
+      fastResult.fastSignum = 1;
+    } else {
+      fastResult.fastIntegerDigitCount = 0;
+      fastResult.fastSignum = 0;
+    }
+
+    fastResult.fast0 = result0;
+    fastResult.fast1 = result1;
+    fastResult.fast2 = result2;
+
+    return true;
+  }
+
+  //************************************************************************************************
+  // Decimal to Binary Conversion.
+
+  /**
+   * A helper method that produces a single binary word remainder from a fast decimal (and
+   * quotient).
+   *
+   * The fast decimal is longwords of 16 digits each and we need binary words of 2^N.  Since
+   * we are in decimal form, we have do work to get to convert to binary form.
+   *
+   * We effectively need to produce on big binary value (i.e. greater than 64 bits since
+   * HiveDecimal needs 128 bits of binary which Java does not provide primitive support for)
+   * from the decimal long words and get the lower N binary bit remainder.
+   *
+   * We could try and do decimal division by 2^N to get the (integer) quotient, multiply the
+   * quotient by 2^N decimal, and finally do a decimal subtract that from the original decimal.
+   * The resulting decimal can be used to easily get the binary remainder.
+   *
+   * However, currently, we do not have fast decimal division.
+   *
+   * The "trick" we do here is to remember from your Algebra in school than multiplication and
+   * division are inverses of each other.
+   *
+   * So instead of doing decimal division by 2^N we multiply by the inverse: 2^-N.
+   *
+   * We produce 1 binary word (remainder) and a decimal quotient for the higher portion.
+   *
+   * So, the parameters are:
+   *
+   *   The input decimal (dividendFast0, dividendFast1, and dividendFast2) that will produce a
+   *   single binary word remainder and decimal quotient.
+   *
+   *   The fast decimal inverse of 2^N = 2^-N (fastInverseConst).
+   *
+   *   Where in the inverse multiplication result (quotientIntegerWordNum and
+   *   quotientIntegerDigitNum) to find the quotient integer decimal portion.
+   *
+   *   The fast decimal multiplier for converting the quotient integer to the larger number to
+   *   subtract from the input decimal to get the remainder.
+   *
+   *   And, the scratch longs where to store the result remainder word (index 3) and result quotient
+   *   decimal longwords (indices 0 .. 2).
+   *
+   * @return True if the results were produced without overflow.
+   */
+  public static boolean doDecimalToBinaryDivisionRemainder(
+      long dividendFast0, long dividendFast1, long dividendFast2,
+      FastHiveDecimal fastInverseConst,
+      int quotientIntegerWordNum,
+      int quotientIntegerDigitNum,
+      FastHiveDecimal fastMultiplierConst,
+      long[] scratchLongs) {
+
+    // Multiply by inverse (2^-N) to do the 2^N division.
+    if (!fastMultiply5x6HalfWords(
+        dividendFast0, dividendFast1, dividendFast2,
+        fastInverseConst.fast0, fastInverseConst.fast1, fastInverseConst.fast2,
+        scratchLongs)) {
+      // Overflow.
+      return false;
+    }
+
+    final long divideFactor = powerOfTenTable[quotientIntegerDigitNum];
+    final long multiplyFactor = powerOfTenTable[LONGWORD_DECIMAL_DIGITS - quotientIntegerDigitNum];
+
+    // Extract the integer portion to get the quotient.
+    long quotientFast0 =
+        scratchLongs[quotientIntegerWordNum] / divideFactor
+      + ((scratchLongs[quotientIntegerWordNum + 1] % divideFactor) * multiplyFactor);
+    long quotientFast1 =
+        scratchLongs[quotientIntegerWordNum + 1] / divideFactor
+      + ((scratchLongs[quotientIntegerWordNum + 2] % divideFactor) * multiplyFactor);
+    long quotientFast2 =
+        scratchLongs[quotientIntegerWordNum + 2] / divideFactor;
+
+    // Multiply the integer quotient back out so we can subtract it from the original to get
+    // the remainder.
+    if (!fastMultiply5x6HalfWords(
+        quotientFast0, quotientFast1, quotientFast2,
+        fastMultiplierConst.fast0, fastMultiplierConst.fast1, fastMultiplierConst.fast2,
+        scratchLongs)) {
+      return false;
+    }
+
+    long quotientMultiplied0 = scratchLongs[0];
+    long quotientMultiplied1 = scratchLongs[1];
+    long quotientMultiplied2 = scratchLongs[2];
+
+    if (!doSubtractSameScaleNoUnderflow(
+        dividendFast0, dividendFast1, dividendFast2,
+        quotientMultiplied0, quotientMultiplied1, quotientMultiplied2,
+        scratchLongs)) {
+      // Underflow.
+      return false;
+    }
+
+    long remainderBinaryWord =
+        scratchLongs[1] * MULTIPLER_LONGWORD_DECIMAL
+      + scratchLongs[0];
+
+    // Pack the output into the scratch longs.
+    scratchLongs[0] = quotientFast0;
+    scratchLongs[1] = quotientFast1;
+    scratchLongs[2] = quotientFast2;
+
+    scratchLongs[3] = remainderBinaryWord;
+
+    return true;
+  }
+
+  /**
+   * Convert a fast decimal into 3 binary words of N bits each.
+   * 
+   * The 3 binary words will form a large binary value that is the unsigned unscaled decimal value:
+   *
+   *    highWord * 2^(M+L) + middleWord * 2^L + lowerWord.
+   *
+   * Where L is the number of bits in the lower word; M is the number of bits in the middle word.
+   * We let L and M be different to support the SerializationUtil serialization where the lower
+   * word is 62 bits and the remaining words are 63 bits...
+   *
+   * The fast decimal is longwords of 16 digits each and we need binary words of 2^N.  Since
+   * we are in decimal form, we have do work to get to convert to binary form.
+   *
+   * See the comments for doDecimalToBinaryDivisionRemainder for details on the parameters.
+   *
+   * The lowerWord is produced by calling doDecimalToBinaryDivisionRemainder.  The quotient from
+   * that is passed to doDecimalToBinaryDivisionRemainder to produce the middleWord.  The final
+   * quotient is used to produce the highWord.
+   *
+   * @return True if the 3 binary words were produced without overflow.  Overflow is not expected.
+   */
+  private static boolean doDecimalToBinaryConversion(
+      long fast0, long fast1, long fast2,
+      FastHiveDecimal fastInverseConst,
+      int quotientIntegerWordNum,
+      int quotientIntegerDigitNum,
+      FastHiveDecimal fastMultiplierConst,
+      long[] scratchLongs) {
+
+    long lowerBinaryWord;
+    long middleBinaryWord = 0;
+    long highBinaryWord = 0;
+
+    if (fastCompareTo(
+            1,
+            fast0, fast1, fast2, 0,
+            1,
+            fastMultiplierConst.fast0, fastMultiplierConst.fast1, fastMultiplierConst.fast2, 0) < 0) {
+
+      // Optimize: whole decimal fits in one binary word.
+
+      lowerBinaryWord =
+          fast1 * MULTIPLER_LONGWORD_DECIMAL
+        + fast0;
+
+    } else {
+
+      // Do division/remainder to get lower binary word; quotient will either be middle decimal
+      // or be both high and middle decimal that requires another division/remainder.
+
+      if (!doDecimalToBinaryDivisionRemainder(
+          fast0, fast1, fast2,
+          fastInverseConst,
+          quotientIntegerWordNum,
+          quotientIntegerDigitNum,
+          fastMultiplierConst,
+          scratchLongs)) {
+        // Overflow.
+        return false;
+      }
+
+      // Unpack the output.
+      long quotientFast0 = scratchLongs[0];
+      long quotientFast1 = scratchLongs[1];
+      long quotientFast2 = scratchLongs[2];
+
+      lowerBinaryWord = scratchLongs[3];
+
+      if (fastCompareTo(
+          1,
+          quotientFast0, quotientFast1, quotientFast2, 0,
+          1,
+          fastMultiplierConst.fast0, fastMultiplierConst.fast1, fastMultiplierConst.fast2, 0) < 0) {
+
+        // Optimize: whole decimal fits in two binary words.
+
+        middleBinaryWord =
+            quotientFast1 * MULTIPLER_LONGWORD_DECIMAL
+          + quotientFast0;
+
+      } else {
+        if (!doDecimalToBinaryDivisionRemainder(
+            quotientFast0, quotientFast1, quotientFast2,
+            fastInverseConst,
+            quotientIntegerWordNum,
+            quotientIntegerDigitNum,
+            fastMultiplierConst,
+            scratchLongs)) {
+          // Overflow.
+          return false;
+        }
+
+        highBinaryWord =
+            scratchLongs[1] * MULTIPLER_LONGWORD_DECIMAL
+          + scratchLongs[0];
+
+        middleBinaryWord = scratchLongs[3];
+
+      }
+    }
+
+    scratchLongs[0] = lowerBinaryWord;
+    scratchLongs[1] = middleBinaryWord;
+    scratchLongs[2] = highBinaryWord;
+
+    return true;
+  }
+
+  //************************************************************************************************
+  // Emulate SerializationUtils Deserialization used by ORC.
+
+  /*
+   * fastSerializationUtilsRead lower word is 62 bits (the lower bit is used as the sign and is
+   * removed).  So, we need a multiplier 2^62
+   *
+   *    2^62 =
+   *      4611686018427387904 or
+   *      4,611,686,018,427,387,904 or
+   *      461,1686018427387904 (16 digit comma'd)
+   */
+  private static FastHiveDecimal FAST_HIVE_DECIMAL_TWO_POWER_62 =
+      new FastHiveDecimal(1, 1686018427387904L, 461L, 0, 19, 0);
+
+  /*
+   * fastSerializationUtilsRead middle word is 63 bits. So, we need a multiplier 2^63 
+   *
+   *    2^63 =
+   *      9223372036854775808 (Long.MAX_VALUE) or
+   *      9,223,372,036,854,775,808 or
+   *      922,3372036854775808 (16 digit comma'd)
+   */
+  private static FastHiveDecimal FAST_HIVE_DECIMAL_TWO_POWER_63 =
+      new FastHiveDecimal(1, 3372036854775808L, 922L, 0, 19, 0);
+
+  /*
+   * fastSerializationUtilsRead high word multiplier:
+   *
+   *    Multiply by 2^(62 + 63)                      -- 38 digits or 3 decimal words.
+   *
+   *    (2^62)*(2^63) =
+   *      42535295865117307932921825928971026432 or
+   *     (12345678901234567890123456789012345678)
+   *     (         1         2         3        )
+   *      42,535,295,865,117,307,932,921,825,928,971,026,432 or
+   *      425352,9586511730793292,1825928971026432  (16 digit comma'd)
+   */
+  private static FastHiveDecimal FAST_HIVE_DECIMAL_TWO_POWER_125 =
+      new FastHiveDecimal(1, 1825928971026432L, 9586511730793292L, 425352L, 38, 0);
+
+  /*
+   * Inverse of 2^63 = 2^-63.  Please see comments for doDecimalToBinaryDivisionRemainder.
+   *
+   * Multiply by 1/2^63 = 1.08420217248550443400745280086994171142578125e-19 to divide by 2^63.
+   * As 16 digit comma'd 1084202172485,5044340074528008,6994171142578125
+   *
+   * Scale down: 63 = 44 fraction digits + 19 (negative exponent or number of zeros after dot).
+   *
+   * 3*16 (48) + 15 --> 63 down shift.
+   */
+  private static FastHiveDecimal FAST_HIVE_DECIMAL_TWO_POWER_63_INVERSE =
+      new FastHiveDecimal(1, 6994171142578125L, 5044340074528008L, 1084202172485L, 45, 0);
+
+  /*
+   * Where in the inverse multiplication result to find the quotient integer decimal portion.
+   *
+   * Please see comments for doDecimalToBinaryDivisionRemainder.
+   */
+  private static final int SERIALIZATION_UTILS_WRITE_QUOTIENT_INTEGER_WORD_NUM = 3;
+  private static final int SERIALIZATION_UTILS_WRITE_QUOTIENT_INTEGER_DIGIT_NUM = 15;
+
+  /**
+   * Deserialize data written in the format used by the SerializationUtils methods
+   * readBigInteger/writeBigInteger and create a decimal using the supplied scale.
+   *
+   * ORC uses those SerializationUtils methods for its serialization.
+   *
+   * A scratch bytes array is necessary to do the binary to decimal conversion for better
+   * performance.  Pass a FAST_SCRATCH_BUFFER_LEN_SERIALIZATION_UTILS_READ byte array for
+   * scratchBytes.
+   *
+   * @return The deserialized decimal or null if the conversion failed.
+   */
+  public static boolean fastSerializationUtilsRead(InputStream inputStream, int scale,
+      byte[] scratchBytes,
+      FastHiveDecimal fastResult) throws IOException, EOFException {
+
+    // Following a suggestion from Gopal, quickly read in the bytes from the stream.
+    // CONSIDER: Have ORC read the whole input stream into a big byte array with one call to
+    // the read(byte[] b, int off, int len) method and then let this method read from the big
+    // byte array.
+    int readCount = 0;
+    int input;
+    do {
+      input = inputStream.read();
+      if (input == -1) {
+        throw new EOFException("Reading BigInteger past EOF from " + inputStream);
+      }
+      scratchBytes[readCount++] = (byte) input;
+    } while (input >= 0x80);
+
+    /*
+     * Determine the 3 binary words like what SerializationUtils.readBigInteger does.
+     */
+
+    long lowerWord63 = 0;
+    long middleWord63 = 0;
+    long highWord63 = 0;
+
+    long work = 0;
+    int offset = 0;
+    int readIndex = 0;
+    long b;
+    do {
+      b = scratchBytes[readIndex++];
+      work |= (0x7f & b) << (offset % 63);
+      offset += 7;
+      // if we've read 63 bits, roll them into the result
+      if (offset == 63) {
+        lowerWord63 = work;
+        work = 0;
+      } else if (offset % 63 == 0) {
+        if (offset == 126) {
+          middleWord63 = work;
+        } else if (offset == 189) {
+          highWord63 = work;
+        } else {
+          throw new EOFException("Reading more than 3 words of BigInteger");
+        }
+        work = 0;
+      }
+    } while (readIndex < readCount);
+
+    if (work != 0) {
+      if (offset < 63) {
+        lowerWord63 = work;
+      } else if (offset < 126) {
+        middleWord63 = work;
+      } else if (offset < 189) {
+        highWord63 =work;
+      } else {
+        throw new EOFException("Reading more than 3 words of BigInteger");
+      }
+    }
+
+    // Grab sign bit and shift it away.
+    boolean isNegative = ((lowerWord63 & 0x1) != 0);
+    lowerWord63 >>= 1;
+
+    /*
+     * Use common binary to decimal conversion method we share with fastSetFromBigIntegerBytes.
+     */
+    if (!doBinaryToDecimalConversion(
+            lowerWord63, middleWord63, highWord63,
+            FAST_HIVE_DECIMAL_TWO_POWER_62,
+            FAST_HIVE_DECIMAL_TWO_POWER_125,    // 2^(62 + 63)
+            fastResult)) {
+      return false;
+    }
+
+    if (isNegative) {
+
+      // Adjust negative result, again doing what SerializationUtils.readBigInteger does.
+      if (!doAddSameScaleSameSign(
+          /* resultSignum */ 1,
+          fastResult.fast0, fastResult.fast1, fastResult.fast2,
+          1, 0, 0,
+          fastResult)) {
+        return false;
+      }
+    }
+
+    if (fastResult.fast0 == 0 && fastResult.fast1 == 0 && fastResult.fast2 == 0) {
+      fastResult.fastSignum = 0;
+    } else {
+      fastResult.fastSignum = (isNegative ? -1 : 1);
+      final int rawPrecision = fastRawPrecision(fastResult);
+      fastResult.fastIntegerDigitCount = Math.max(0, rawPrecision - scale);
+      fastResult.fastScale = scale;
+
+      /*
+       * Just in case we deserialize a decimal with trailing zeroes...
+       */
+      final int resultTrailingZeroCount =
+          fastTrailingDecimalZeroCount(
+              fastResult.fast0, fastResult.fast1, fastResult.fast2,
+              fastResult.fastIntegerDigitCount, fastResult.fastScale);
+      if (resultTrailingZeroCount > 0) {
+        doFastScaleDown(
+            fastResult,
+            resultTrailingZeroCount,
+            fastResult);
+
+        fastResult.fastScale -= resultTrailingZeroCount;
+      }
+    }
+
+    return true;
+  }
+
+  //************************************************************************************************
+  // Emulate SerializationUtils Serialization used by ORC.
+
+  /**
+   * Write the value of this decimal just like SerializationUtils.writeBigInteger.  It header
+   * comments are:
+   *
+   *     Write the arbitrarily sized signed BigInteger in vint format.
+   *
+   *     Signed integers are encoded using the low bit as the sign bit using zigzag
+   *     encoding.
+   *
+   *     Each byte uses the low 7 bits for data and the high bit for stop/continue.
+   *
+   *     Bytes are stored LSB first.
+   *
+   * NOTE:
+   *    SerializationUtils.writeBigInteger sometimes pads the result with extra zeroes due to
+   *    BigInteger.bitLength -- we do not emulate that.  SerializationUtils.readBigInteger will
+   *    produce the same result for both.
+   *
+   * @return True if the decimal was successfully serialized into the output stream.
+   */
+  public static boolean fastSerializationUtilsWrite(OutputStream outputStream,
+      int fastSignum, long fast0, long fast1, long fast2,
+      int fastIntegerDigitCount, int fastScale,
+      long[] scratchLongs)
+          throws IOException {
+
+    boolean isNegative = (fastSignum == -1);
+
+    /*
+     * The sign is encoded as the least significant bit.
+     *
+     * We need to adjust our decimal before conversion to binary.
+     *
+     * Positive:
+     *   Multiply by 2.
+     *
+     * Negative:
+     *   Logic in SerializationUtils.writeBigInteger does a negate on the BigInteger. We
+     *   do not have to since FastHiveDecimal stores the numbers unsigned in fast0, fast1,
+     *   and fast2.  We do need to subtract one though.
+     *
+     *   And then multiply by 2 and add in the 1 sign bit.
+     *
+     *   CONSIDER: This could be combined.
+     */
+    long adjust0;
+    long adjust1;
+    long adjust2;
+
+    if (isNegative) {
+
+      // Subtract 1.
+      long r0 = fast0 - 1;
+      long r1;
+      if (r0 < 0) {
+        adjust0 = r0 + MULTIPLER_LONGWORD_DECIMAL;
+        r1 = fast1 - 1;
+      } else {
+        adjust0 = r0;
+        r1 = fast1;
+      }
+      if (r1 < 0) {
+        adjust1 = r1 + MULTIPLER_LONGWORD_DECIMAL;
+        adjust2 = fast2 - 1;
+      } else {
+        adjust1 = r1;
+        adjust2 = fast2;
+      }
+      if (adjust2 < 0) {
+        return false;
+      }
+
+      // Now multiply by 2 and add 1 sign bit.
+      r0 = adjust0 * 2 + 1;
+      adjust0 =
+          r0 % MULTIPLER_LONGWORD_DECIMAL;
+      r1 =
+          adjust1 * 2
+        + r0 / MULTIPLER_LONGWORD_DECIMAL;
+      adjust1 =
+          r1 % MULTIPLER_LONGWORD_DECIMAL;
+      adjust2 =
+          adjust2 * 2
+        + r1 / MULTIPLER_LONGWORD_DECIMAL;
+
+    } else {
+
+      // Multiply by 2 to make room for 0 sign bit.
+      long r0 = fast0 * 2;
+      adjust0 =
+          r0 % MULTIPLER_LONGWORD_DECIMAL;
+      final long r1 =
+          fast1 * 2
+        + r0 / MULTIPLER_LONGWORD_DECIMAL;
+      adjust1 =
+          r1 % MULTIPLER_LONGWORD_DECIMAL;
+      adjust2 =
+          fast2 * 2
+        + r1 / MULTIPLER_LONGWORD_DECIMAL;
+
+    }
+
+    /*
+     * Use common decimal to binary conversion method we share with fastBigIntegerBytes.
+     */
+    if (!doDecimalToBinaryConversion(
+        adjust0, adjust1, adjust2,
+        FAST_HIVE_DECIMAL_TWO_POWER_63_INVERSE,
+        SERIALIZATION_UTILS_WRITE_QUOTIENT_INTEGER_WORD_NUM,
+        SERIALIZATION_UTILS_WRITE_QUOTIENT_INTEGER_DIGIT_NUM,
+        FAST_HIVE_DECIMAL_TWO_POWER_63,
+        scratchLongs)) {
+      // Overflow.
+      return false;
+    }
+
+    long lowerWord63 = scratchLongs[0];
+    long middleWord63 = scratchLongs[1];
+    long highWord63 = scratchLongs[2];
+
+    int wordCount;
+    if (highWord63 != 0) {
+      wordCount = 3;
+    } else if (middleWord63 != 0) {
+      wordCount = 2;
+    } else {
+      wordCount = 1;
+    }
+
+    // Write out the first 63 bits worth of data.
+    long lowBits = lowerWord63;
+    for(int i=0; i < 9; ++i) {
+      // If this is the last byte, leave the high bit off
+      if (wordCount == 1 && (lowBits & ~0x7f) == 0) {
+        outputStream.write((byte) lowBits);
+        return true;
+      } else {
+        outputStream.write((byte) (0x80 | (lowBits & 0x7f)));
+        lowBits >>>= 7;
+      }
+    }
+    if (wordCount <= 1) {
+      throw new RuntimeException("Expecting write word count > 1");
+    }
+
+    lowBits = middleWord63;
+    for(int i=0; i < 9; ++i) {
+      // If this is the last byte, leave the high bit off
+      if (wordCount == 2 && (lowBits & ~0x7f) == 0) {
+        outputStream.write((byte) lowBits);
+        return true;
+      } else {
+        outputStream.write((byte) (0x80 | (lowBits & 0x7f)));
+        lowBits >>>= 7;
+      }
+    }
+
+    lowBits = highWord63;
+    for(int i=0; i < 9; ++i) {
+      // If this is the last byte, leave the high bit off
+      if ((lowBits & ~0x7f) == 0) {
+        outputStream.write((byte) lowBits);
+        return true;
+      } else {
+        outputStream.write((byte) (0x80 | (lowBits & 0x7f)));
+        lowBits >>>= 7;
+      }
+    }
+
+    // Should not get here.
+    throw new RuntimeException("Unexpected");
+  }
+
+  //************************************************************************************************
+  // Emulate BigInteger deserialization used by LazyBinary and others.
+
+  /*
+   * fastSetFromBigIntegerBytes word size we choose is 56 bits to stay below the 64 bit sign bit:
+   * So, we need a multiplier 2^56
+   *
+   *    2^56 =
+   *      72057594037927936 or
+   *      72,057,594,037,927,936 or
+   *      7,2057594037927936  (16 digit comma'd)
+   */
+  private static FastHiveDecimal FAST_HIVE_DECIMAL_TWO_POWER_56 =
+      new FastHiveDecimal(1, 2057594037927936L, 7L, 0, 17, 0);
+
+  /*
+   * fastSetFromBigIntegerBytes high word multiplier is 2^(56 + 56)
+   *
+   *    (2^56)*(2^56) =
+   *      5192296858534827628530496329220096 or
+   *     (1234567890123456789012345678901234)
+   *     (         1         2         3    )
+   *      5,192,296,858,534,827,628,530,496,329,220,096 or
+   *      51,9229685853482762,8530496329220096  (16 digit comma'd)
+   */
+  private static FastHiveDecimal FAST_HIVE_DECIMAL_TWO_POWER_112 =
+      new FastHiveDecimal(1, 8530496329220096L, 9229685853482762L, 51L, 34, 0);
+
+  // Multiply by 1/2^56 or 1.387778780781445675529539585113525390625e-17 to divide by 2^56.
+  // As 16 digit comma'd 13877787,8078144567552953,9585113525390625
+  //
+  // Scale down: 56 = 39 fraction digits + 17 (negative exponent or number of zeros after dot).
+  //
+  // 3*16 (48) + 8 --> 56 down shift.
+  //
+  private static FastHiveDecimal FAST_HIVE_DECIMAL_TWO_POWER_56_INVERSE =
+      new FastHiveDecimal(1, 9585113525390625L, 8078144567552953L, 13877787L, 40, 0);
+
+  /*
+   * Where in the inverse multiplication result to find the quotient integer decimal portion.
+   *
+   * Please see comments for doDecimalToBinaryDivisionRemainder.
+   */
+  private static final int BIG_INTEGER_BYTES_QUOTIENT_INTEGER_WORD_NUM = 3;
+  private static final int BIG_INTEGER_BYTES_QUOTIENT_INTEGER_DIGIT_NUM = 8;
+
+  private static int INITIAL_SHIFT = 48;   // 56 bits minus 1 byte.
+
+  // Long masks and values.
+  private static long LONG_56_BIT_MASK = 0xFFFFFFFFFFFFFFL;
+  private static long LONG_TWO_TO_56_POWER = LONG_56_BIT_MASK + 1L;
+  private static long LONG_BYTE_MASK = 0xFFL;
+  private static long LONG_BYTE_HIGH_BIT_MASK = 0x80L;
+
+  // Byte values.
+  private static byte BYTE_ALL_BITS = (byte) 0xFF;
+
+  /**
+   * Convert bytes in the format used by BigInteger's toByteArray format (and accepted by its
+   * constructor) into a decimal using the specified scale.
+   *
+   * Our bigIntegerBytes methods create bytes in this format, too.
+   *
+   * This method is designed for high performance and does not create an actual BigInteger during
+   * binary to decimal conversion.
+   *
+   * @return
+   */
+  public static boolean fastSetFromBigIntegerBytesAndScale(
+      byte[] bytes, int offset, int length, int scale,
+      FastHiveDecimal fastResult) {
+
+    final int bytesLength = bytes.length;
+
+    if (offset < 0 || offset >= bytesLength) {
+      return false;
+    }
+    final int end = offset + length;
+    if (end <= offset || end > bytesLength) {
+      return false;
+    }
+
+    final int startOffset = offset;
+
+    // Roughly based on BigInteger code.
+
+    boolean isNegative = (bytes[offset] < 0);
+    if (isNegative) {
+
+      // Find first non-sign (0xff) byte of input.
+      while (offset < end) {
+        if (bytes[offset] != -1) {
+          break;
+        }
+        offset++;
+      }
+      if (offset > end) {
+        return false;
+      }
+    } else {
+
+      // Strip leading zeroes -- although there shouldn't be any for a decimal.
+
+      while (offset < end && bytes[offset] == 0) {
+        offset++;
+      }
+      if (offset >= end) {
+        // Zero.
+        return true;
+      }
+    }
+
+    long lowerWord56 = 0;
+    long middleWord56 = 0;
+    long highWord56 = 0;
+
+    int reverseIndex = end;
+
+    long work;
+    int shift;
+
+    final int lowestCount = Math.min(reverseIndex - offset, 7);
+    shift = 0;
+    for (int i = 0; i < lowestCount; i++) {
+      work = bytes[--reverseIndex] & 0xFF;
+      lowerWord56 |= work << shift;
+      shift += 8;
+    }
+
+    if (reverseIndex <= offset) {
+      if (isNegative) {
+        lowerWord56 = ~lowerWord56 & ((1L << shift) - 1);
+      }
+    } else {
+
+      // Go on to middle word.
+
+      final int middleCount = Math.min(reverseIndex - offset, 7);
+      shift = 0;
+      for (int i = 0; i < middleCount; i++) {
+        work = bytes[--reverseIndex] & 0xFF;
+        middleWord56 |= work << shift;
+        shift += 8;
+      }
+      if (reverseIndex <= offset) {
+        if (isNegative) {
+          lowerWord56 = ~lowerWord56 & LONG_56_BIT_MASK;
+          middleWord56 = ~middleWord56 & ((1L << shift) - 1);
+        }
+      } else {
+
+        // Go on to high word.
+
+        final int highCount = Math.min(reverseIndex - offset, 7);
+        shift = 0;
+        for (int i = 0; i < highCount; i++) {
+          work = bytes[--reverseIndex] & 0xFF;
+          highWord56 |= work << shift;
+          shift += 8;
+        }
+        if (isNegative) {
+          // We only need to apply negation to all 3 words when there are 3 words, etc.
+          lowerWord56 = ~lowerWord56 & LONG_56_BIT_MASK;
+          middleWord56 = ~middleWord56 & LONG_56_BIT_MASK;
+          highWord56 = ~highWord56 & ((1L << shift) - 1);
+        }
+      }
+    }
+
+    if (!doBinaryToDecimalConversion(
+          lowerWord56, middleWord56, highWord56,
+          FAST_HIVE_DECIMAL_TWO_POWER_56,
+          FAST_HIVE_DECIMAL_TWO_POWER_112,    // 2^(56 + 56)
+          fastResult)) {
+      // Overflow.  Use slower alternate.
+      return doAlternateSetFromBigIntegerBytesAndScale(
+          bytes, startOffset, length, scale,
+          fastResult);
+    }
+
+    // System.out.println("fastSetFromBigIntegerBytesAndScale fast0 " + fastResult.fast0 + " fast1 " + fastResult.fast1 + " fast2 " + fastResult.fast2);
+    if (isNegative) {
+      if (!doAddSameScaleSameSign(
+          /* resultSignum */ 1,
+          fastResult.fast0, fastResult.fast1, fastResult.fast2,
+          1, 0, 0,
+          fastResult)) {
+        // Overflow.  Use slower alternate.
+        return doAlternateSetFromBigIntegerBytesAndScale(
+            bytes, startOffset, length, scale,
+            fastResult);
+      }
+    }
+
+    if (fastResult.fast0 == 0 && fastResult.fast1 == 0 && fastResult.fast2 == 0) {
+      fastResult.fastSignum = 0;
+    } else {
+      fastResult.fastSignum = (isNegative ? -1 : 1);
+      fastResult.fastScale = scale;
+      final int rawPrecision = fastRawPrecision(fastResult);
+      fastResult.fastIntegerDigitCount = Math.max(0, rawPrecision - scale);
+
+      /*
+       * Just in case we deserialize a decimal with trailing zeroes...
+       */
+      final int resultTrailingZeroCount =
+          fastTrailingDecimalZeroCount(
+              fastResult.fast0, fastResult.fast1, fastResult.fast2,
+              fastResult.fastIntegerDigitCount, fastResult.fastScale);
+      if (resultTrailingZeroCount > 0) {
+        doFastScaleDown(
+            fastResult,
+            resultTrailingZeroCount,
+            fastResult);
+
+        fastResult.fastScale -= resultTrailingZeroCount;
+      }
+    }
+
+    return true;
+  }
+
+  /**
+   * When fastSetFromBigIntegerBytesAndScale can handle the input because it is too large,
+   * we fall back to this.
+   */
+  private static boolean doAlternateSetFromBigIntegerBytesAndScale(
+      byte[] bytes, int offset, int length, int scale,
+      FastHiveDecimal fastResult) {
+
+    byte[] byteArray = Arrays.copyOfRange(bytes, offset, offset + length);
+
+    BigInteger bigInteger = new BigInteger(byteArray);
+    // System.out.println("doAlternateSetFromBigIntegerBytesAndScale bigInteger " + bigInteger);
+    BigDecimal bigDecimal = new BigDecimal(bigInteger, scale);
+    // System.out.println("doAlternateSetFromBigIntegerBytesAndScale bigDecimal " + bigDecimal);
+    fastResult.fastReset();
+    return fastSetFromBigDecimal(bigDecimal, true, fastResult);
+  }
+
+  //************************************************************************************************
+  // Emulate BigInteger serialization used by LazyBinary, Avro, Parquet, and possibly others.
+
+  public static int fastBigIntegerBytes(
+      final int fastSignum, long fast0, long fast1, long fast2,
+      int fastIntegerDigitCount, int fastScale,
+      int fastSerializeScale,
+      long[] scratchLongs, byte[] buffer) {
+    if (fastSerializeScale != -1) {
+      return
+          fastBigIntegerBytesScaled(
+              fastSignum, fast0, fast1, fast2,
+              fastIntegerDigitCount, fastScale,
+              fastSerializeScale,
+              scratchLongs, buffer);
+    } else {
+      return
+          fastBigIntegerBytesUnscaled(
+              fastSignum, fast0, fast1, fast2,
+              scratchLongs, buffer);
+    }
+  }
+
+  /**
+   * Return binary representation of this decimal's BigInteger equivalent unscaled value using
+   * the format that the BigInteger's toByteArray method returns (and the BigInteger constructor
+   * accepts).
+   *
+   * Used by LazyBinary, Avro, and Parquet serialization.
+   *
+   * Scratch objects necessary to do the decimal to binary conversion without actually creating a
+   * BigInteger object are passed for better performance.
+   *
+   * Allocate scratchLongs with SCRATCH_LONGS_LEN longs.
+   * And, allocate buffer with SCRATCH_BUFFER_LEN_BIG_INTEGER_BYTES bytes.
+   * @return The number of bytes used for the binary result in buffer.  Otherwise, 0 if the
+   *         conversion failed.
+   */
+  public static int fastBigIntegerBytesUnscaled(
+      final int fastSignum, long fast0, long fast1, long fast2,
+      long[] scratchLongs, byte[] buffer) {
+
+    /*
+     * Algorithm:
+     * 1) Convert decimal to three 56-bit words (three is enough for the decimal since we
+     *    represent the decimal with trailing zeroes trimmed).
+     * 2) Skip leading zeroes in the words.
+     * 3) Once we find real data (i.e. a non-zero byte), add a sign byte to buffer if necessary.
+     * 4) Add bytes from the (rest of) 56-bit words.
+     * 5) Return byte count.
+     */
+
+    if (fastSignum == 0) {
+      buffer[0] = 0;
+      return 1;
+    }
+
+    boolean isNegative = (fastSignum == -1);
+
+    /*
+     * Use common conversion method we share with fastSerializationUtilsWrite.
+     */
+    if (!doDecimalToBinaryConversion(
+        fast0, fast1, fast2,
+        FAST_HIVE_DECIMAL_TWO_POWER_56_INVERSE,
+        BIG_INTEGER_BYTES_QUOTIENT_INTEGER_WORD_NUM,
+        BIG_INTEGER_BYTES_QUOTIENT_INTEGER_DIGIT_NUM,
+        FAST_HIVE_DECIMAL_TWO_POWER_56,
+        scratchLongs)) {
+      // Overflow.  This is not expected.
+      return 0;
+    }
+
+    int byteIndex = 0;
+
+    long word0 = scratchLongs[0];
+    long word1 = scratchLongs[1];
+    long word2 = scratchLongs[2];
+
+    if (!isNegative) {
+
+      // Positive number.
+
+      long longWork = 0;
+
+      int shift = INITIAL_SHIFT;
+
+      if (word2 != 0L) {
+
+        // Skip leading zeroes in word2.
+
+        while (true) {
+          longWork = (word2 >> shift) & LONG_BYTE_MASK;
+          if (longWork != 0) {
+            break;
+          }
+          if (shift == 0) {
+            throw new RuntimeException("Unexpected #1");
+          }
+          shift -= Byte.SIZE;
+        }
+
+        // Now that we have found real data, emit sign byte if necessary.
+        if ((longWork & LONG_BYTE_HIGH_BIT_MASK) != 0) {
+          // Add sign byte since high bit is on.
+          buffer[byteIndex++] = (byte) 0;
+        }
+
+        // Emit the rest of word2
+        while (true) {
+          buffer[byteIndex++] = (byte) longWork;
+          if (shift == 0) {
+            break;
+          }
+          shift -= Byte.SIZE;
+          longWork = (word2 >> shift) & LONG_BYTE_MASK;
+        }
+
+        shift = INITIAL_SHIFT;
+      }
+
+      if (byteIndex == 0 && word1 == 0L) {
+
+        // Skip word1, also.
+
+      } else {
+
+        if (byteIndex == 0) {
+
+          // Skip leading zeroes in word1.
+
+          while (true) {
+            longWork = (word1 >> shift) & LONG_BYTE_MASK;
+            if (longWork != 0) {
+              break;
+            }
+            if (shift == 0) {
+              throw new RuntimeException("Unexpected #2");
+            }
+            shift -= Byte.SIZE;
+          }
+
+          // Now that we have found real data, emit sign byte if necessary.
+          if ((longWork & LONG_BYTE_HIGH_BIT_MASK) != 0) {
+            // Add sign byte since high bit is on.
+            buffer[byteIndex++] = (byte) 0;
+          }
+
+        } else {
+          longWork = (word1 >> shift) & LONG_BYTE_MASK;
+        }
+
+        // Emit the rest of word1
+
+        while (true) {
+          buffer[byteIndex++] = (byte) longWork;
+          if (shift == 0) {
+            break;
+          }
+          shift -= Byte.SIZE;
+          longWork = (word1 >> shift) & LONG_BYTE_MASK;
+        }
+
+        shift = INITIAL_SHIFT;
+      }
+
+      if (byteIndex == 0) {
+
+        // Skip leading zeroes in word0.
+
+        while (true) {
+          longWork = (word0 >> shift) & LONG_BYTE_MASK;
+          if (longWork != 0) {
+            break;
+          }
+          if (shift == 0) {
+
+            // All zeroes -- we should have handled this earlier.
+            throw new RuntimeException("Unexpected #3");
+          }
+          shift -= Byte.SIZE;
+        }
+
+        // Now that we have found real data, emit sign byte if necessary.
+        if ((longWork & LONG_BYTE_HIGH_BIT_MASK) != 0) {
+          // Add sign byte since high bit is on.
+          buffer[byteIndex++] = (byte) 0;
+        }
+
+      } else {
+        longWork = (word0 >> shift) & LONG_BYTE_MASK;
+      }
+
+      // Emit the rest of word0.
+      while (true) {
+        buffer[byteIndex++] = (byte) longWork;
+        if (shift == 0) {
+          break;
+        }
+        shift -= Byte.SIZE;
+        longWork = (word0 >> shift) & LONG_BYTE_MASK;
+      }
+
+    } else {
+
+      // Negative number.
+
+      // Subtract 1 for two's compliment adjustment.
+      word0--;
+      if (word0 < 0) {
+        word0 += LONG_TWO_TO_56_POWER;
+        word1--;
+        if (word1 < 0) {
+          word1 += LONG_TWO_TO_56_POWER;
+          word2--;
+          if (word2 < 0) {
+            // Underflow.
+            return 0;
+          }
+        }
+      }
+
+      long longWork = 0;
+
+      int shift = INITIAL_SHIFT;
+
+      if (word2 != 0L) {
+
+        // Skip leading zeroes in word2.
+
+        while (true) {
+          longWork = (word2 >> shift) & LONG_BYTE_MASK;
+          if (longWork != 0) {
+            break;
+          }
+          if (shift == 0) {
+            throw new RuntimeException("Unexpected #1");
+          }
+          shift -= Byte.SIZE;
+        }
+
+        // Now that we have found real data, emit sign byte if necessary and do negative fixup.
+
+        longWork = (~longWork & LONG_BYTE_MASK);
+        if (((longWork) & LONG_BYTE_HIGH_BIT_MASK) == 0) {
+          // Add sign byte since high bit is off.
+          buffer[byteIndex++] = BYTE_ALL_BITS;
+        }
+
+        // Invert words.
+        word2 = ~word2;
+        word1 = ~word1;
+        word0 = ~word0;
+
+        // Emit the rest of word2
+        while (true) {
+          buffer[byteIndex++] = (byte) longWork;
+          if (shift == 0) {
+            break;
+          }
+          shift -= Byte.SIZE;
+          longWork = (word2 >> shift) & LONG_BYTE_MASK;
+        }
+
+        shift = INITIAL_SHIFT;
+      }
+
+      if (byteIndex == 0 && word1 == 0L) {
+
+        // Skip word1, also.
+
+      } else {
+
+        if (byteIndex == 0) {
+
+          // Skip leading zeroes in word1.
+
+          while (true) {
+            longWork = (word1 >> shift) & LONG_BYTE_MASK;
+            if (longWork != 0) {
+              break;
+            }
+            if (shift == 0) {
+              throw new RuntimeException("Unexpected #2");
+            }
+            shift -= Byte.SIZE;
+          }
+
+          // Now that we have found real data, emit sign byte if necessary and do negative fixup.
+
+          longWork = (~longWork & LONG_BYTE_MASK);
+          if ((longWork & LONG_BYTE_HIGH_BIT_MASK) == 0) {
+            // Add sign byte since high bit is off.
+            buffer[byteIndex++] = BYTE_ALL_BITS;
+          }
+
+          // Invert words.
+          word1 = ~word1;
+          word0 = ~word0;
+
+        } else {
+          longWork = (word1 >> shift) & LONG_BYTE_MASK;
+        }
+
+        // Emit the rest of word1
+
+        while (true) {
+          buffer[byteIndex++] = (byte) longWork;
+          if (shift == 0) {
+            break;
+          }
+          shift -= Byte.SIZE;
+          longWork = (word1 >> shift) & LONG_BYTE_MASK;
+        }
+
+        shift = INITIAL_SHIFT;
+      }
+
+      if (byteIndex == 0) {
+
+        // Skip leading zeroes in word0.
+
+        while (true) {
+          longWork = (word0 >> shift) & LONG_BYTE_MASK;
+          if (longWork != 0) {
+            break;
+          }
+          if (shift == 0) {
+
+            // All zeroes.
+
+            // -1 special case.  Unsigned magnitude 1 - two's compliment adjustment 1 = 0.
+            buffer[0] = BYTE_ALL_BITS;
+            return 1;
+          }
+          shift -= Byte.SIZE;
+        }
+
+        // Now that we have found real data, emit sign byte if necessary and do negative fixup.
+
+        longWork = (~longWork & LONG_BYTE_MASK);
+        if ((longWork & LONG_BYTE_HIGH_BIT_MASK) == 0) {
+          // Add sign byte since high bit is off.
+       

<TRUNCATED>

[02/10] hive git commit: HIVE-15335: Fast Decimal (Matt McCline, reviewed by Sergey Shelukhin, Prasanth Jayachandran, Owen O'Malley)

Posted by mm...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/storage-api/src/test/org/apache/hadoop/hive/common/type/TestHiveDecimal.java
----------------------------------------------------------------------
diff --git a/storage-api/src/test/org/apache/hadoop/hive/common/type/TestHiveDecimal.java b/storage-api/src/test/org/apache/hadoop/hive/common/type/TestHiveDecimal.java
new file mode 100644
index 0000000..f8a36e5
--- /dev/null
+++ b/storage-api/src/test/org/apache/hadoop/hive/common/type/TestHiveDecimal.java
@@ -0,0 +1,3404 @@
+/**
+ * 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.common.type;
+
+import java.sql.Timestamp;
+import java.util.Random;
+import java.util.Arrays;
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.math.BigDecimal;
+import java.math.BigInteger;
+
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritableV1;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
+import org.apache.hadoop.hive.common.type.RandomTypeUtil;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.StringExpr;
+import org.apache.hadoop.hive.ql.util.TimestampUtils;
+
+import org.junit.*;
+
+import static org.junit.Assert.*;
+
+public class TestHiveDecimal extends HiveDecimalTestBase {
+
+  @Test
+  public void testInvalidStringInput() {
+
+    HiveDecimalV1 oldDec;
+    HiveDecimalV1 resultOldDec;
+    HiveDecimal dec;
+    HiveDecimal resultDec;
+
+    //---------------------------------------------------
+    oldDec = HiveDecimalV1.create("-");
+    Assert.assertTrue(oldDec == null);
+    //---------------------------------------------------
+    dec = HiveDecimal.create("-");
+    Assert.assertTrue(dec == null);
+
+    //---------------------------------------------------
+    oldDec = HiveDecimalV1.create("+");
+    Assert.assertTrue(oldDec == null);
+    //---------------------------------------------------
+    dec = HiveDecimal.create("+");
+    Assert.assertTrue(dec == null);
+
+    // Naked dot.
+    //---------------------------------------------------
+    oldDec = HiveDecimalV1.create(".");
+    Assert.assertTrue(oldDec == null);
+    //---------------------------------------------------
+    dec = HiveDecimal.create(".");
+    Assert.assertTrue(dec == null);
+
+    //---------------------------------------------------
+    oldDec = HiveDecimalV1.create("-.");
+    Assert.assertTrue(oldDec == null);
+    //---------------------------------------------------
+    dec = HiveDecimal.create("-.");
+    Assert.assertTrue(dec == null);
+
+    //---------------------------------------------------
+    oldDec = HiveDecimalV1.create("+.");
+    Assert.assertTrue(oldDec == null);
+    //---------------------------------------------------
+    dec = HiveDecimal.create("+.");
+    Assert.assertTrue(dec == null);
+
+    // Naked E/e.
+    //---------------------------------------------------
+    oldDec = HiveDecimalV1.create("E");
+    Assert.assertTrue(oldDec == null);
+    //---------------------------------------------------
+    dec = HiveDecimal.create("E");
+    Assert.assertTrue(dec == null);
+
+    //---------------------------------------------------
+    oldDec = HiveDecimalV1.create(".E");
+    Assert.assertTrue(oldDec == null);
+    //---------------------------------------------------
+    dec = HiveDecimal.create(".E");
+    Assert.assertTrue(dec == null);
+
+    //---------------------------------------------------
+    oldDec = HiveDecimalV1.create("-E");
+    Assert.assertTrue(oldDec == null);
+    //---------------------------------------------------
+    dec = HiveDecimal.create("-E");
+    Assert.assertTrue(dec == null);
+
+    //---------------------------------------------------
+    oldDec = HiveDecimalV1.create("+E");
+    Assert.assertTrue(oldDec == null);
+    //---------------------------------------------------
+    dec = HiveDecimal.create("+E");
+    Assert.assertTrue(dec == null);
+ 
+    //---------------------------------------------------
+    oldDec = HiveDecimalV1.create("e");
+    Assert.assertTrue(oldDec == null);
+    //---------------------------------------------------
+    dec = HiveDecimal.create("e");
+    Assert.assertTrue(dec == null);
+
+    //---------------------------------------------------
+    oldDec = HiveDecimalV1.create(".e");
+    Assert.assertTrue(oldDec == null);
+    //---------------------------------------------------
+    dec = HiveDecimal.create(".e");
+    Assert.assertTrue(dec == null);
+
+    //---------------------------------------------------
+    oldDec = HiveDecimalV1.create("-e");
+    Assert.assertTrue(oldDec == null);
+    //---------------------------------------------------
+    dec = HiveDecimal.create("-e");
+    Assert.assertTrue(dec == null);
+
+    //---------------------------------------------------
+    oldDec = HiveDecimalV1.create("+e");
+    Assert.assertTrue(oldDec == null);
+    //---------------------------------------------------
+    dec = HiveDecimal.create("+e");
+    Assert.assertTrue(dec == null);
+
+    //---------------------------------------------------
+    oldDec = HiveDecimalV1.create("error");
+    Assert.assertTrue(oldDec == null);
+    //---------------------------------------------------
+    dec = HiveDecimal.create("error");
+    Assert.assertTrue(dec == null);
+
+    //---------------------------------------------------
+    oldDec = HiveDecimalV1.create("0x0");
+    Assert.assertTrue(oldDec == null);
+    //---------------------------------------------------
+    dec = HiveDecimal.create("0x0");
+    Assert.assertTrue(dec == null);
+
+    //---------------------------------------------------
+    oldDec = HiveDecimalV1.create("0e");
+    Assert.assertTrue(oldDec == null);
+    //---------------------------------------------------
+    dec = HiveDecimal.create("0e");
+    Assert.assertTrue(dec == null);
+
+    //---------------------------------------------------
+    oldDec = HiveDecimalV1.create("7e");
+    Assert.assertTrue(oldDec == null);
+    //---------------------------------------------------
+    dec = HiveDecimal.create("7e");
+    Assert.assertTrue(dec == null);
+
+    //---------------------------------------------------
+    oldDec = HiveDecimalV1.create("233e-");
+    Assert.assertTrue(oldDec == null);
+    //---------------------------------------------------
+    dec = HiveDecimal.create("233e-");
+    Assert.assertTrue(dec == null);
+
+    //---------------------------------------------------
+    oldDec = HiveDecimalV1.create("32e+");
+    Assert.assertTrue(oldDec == null);
+    //---------------------------------------------------
+    dec = HiveDecimal.create("32e+");
+    Assert.assertTrue(dec == null);
+
+    //---------------------------------------------------
+    oldDec = HiveDecimalV1.create(".0e");
+    Assert.assertTrue(oldDec == null);
+    //---------------------------------------------------
+    dec = HiveDecimal.create(".0e");
+    Assert.assertTrue(dec == null);
+
+    //---------------------------------------------------
+    oldDec = HiveDecimalV1.create(".4e");
+    Assert.assertTrue(oldDec == null);
+    //---------------------------------------------------
+    dec = HiveDecimal.create(".4e");
+    Assert.assertTrue(dec == null);
+  }
+
+  @Test
+  public void testVariousCases() {
+
+    HiveDecimalV1 oldDec;
+    HiveDecimalV1 resultOldDec;
+    HiveDecimal dec;
+    HiveDecimal resultDec;
+
+    BigDecimal bigDecimal = new BigDecimal("-99999999999999999999999999999999999999.99999999999999999");
+
+    //---------------------------------------------------
+    oldDec = HiveDecimalV1.create(bigDecimal);
+    Assert.assertEquals("-100000000000000000000000000000000000000", oldDec.toString());
+    //---------------------------------------------------
+    dec = HiveDecimal.create(bigDecimal);
+    Assert.assertTrue(dec == null);
+
+    // One less integer digit...
+    bigDecimal = new BigDecimal("-9999999999999999999999999999999999999.99999999999999999");
+
+    //---------------------------------------------------
+    oldDec = HiveDecimalV1.create(bigDecimal);
+    Assert.assertEquals("-10000000000000000000000000000000000000", oldDec.toString());
+    //---------------------------------------------------
+    dec = HiveDecimal.create(bigDecimal);
+    Assert.assertEquals("-10000000000000000000000000000000000000", dec.toString());
+
+    //---------------------------------------------------
+    oldDec = HiveDecimalV1.create("101");
+    resultOldDec = HiveDecimalV1.enforcePrecisionScale(oldDec, 10, 0);
+    Assert.assertEquals("101", resultOldDec.toString());
+    //---------------------------------------------------
+    dec = HiveDecimal.create("101");
+    resultDec = HiveDecimal.enforcePrecisionScale(dec, 10, 0);
+    Assert.assertEquals("101", resultDec.toString());
+
+    //---------------------------------------------------
+    oldDec = HiveDecimalV1.create("1");
+    resultOldDec = oldDec.scaleByPowerOfTen(-99);
+    Assert.assertEquals("0", resultOldDec.toString());
+    //---------------------------------------------------
+    dec = HiveDecimal.create("1");
+    resultDec = dec.scaleByPowerOfTen(-99);
+    Assert.assertEquals("0", resultDec.toString());
+  }
+
+  @Test
+  public void testCreateFromBigIntegerRounding() {
+
+    BigInteger bigInt;
+    HiveDecimalV1 oldDec;
+    HiveDecimal dec;
+
+    // 1786135888657847525803324040144343378.09799306448796128931113691624
+    bigInt = new BigInteger(
+                        "178613588865784752580332404014434337809799306448796128931113691624");
+    Assert.assertEquals("178613588865784752580332404014434337809799306448796128931113691624", bigInt.toString());
+    //                   12345678901234567890123456789012345678
+    //                            1         2         3
+    //                                                        12345678901234567890123456789
+    dec = HiveDecimal.create(bigInt, 29);
+    Assert.assertEquals("1786135888657847525803324040144343378.1", dec.toString());
+
+    // 8.090000000000000000000000000000000000000123456
+    bigInt = new BigInteger(
+                        "8090000000000000000000000000000000000000123456");
+    //                   123456789012345678901234567890123456789012345
+    //                             1         2         3         4
+    Assert.assertEquals("8090000000000000000000000000000000000000123456", bigInt.toString());
+    //---------------------------------------------------
+    oldDec = HiveDecimalV1.create(bigInt, 45);
+    Assert.assertEquals("8.09", oldDec.toString());
+    //---------------------------------------------------
+    dec = HiveDecimal.create(bigInt, 45);
+    Assert.assertEquals("8.09", dec.toString());
+
+    // 99999999.99999999999999999999999999999949999
+    // MAX_DECIMAL 9's WITH NO ROUND (longer than 38 digits)
+    bigInt = new BigInteger(
+                        "9999999999999999999999999999999999999949999");
+    //                   12345678901234567890123456789012345678
+    //                             1         2         3
+    //                   99999999.99999999999999999999999999999949999
+    Assert.assertEquals("9999999999999999999999999999999999999949999", bigInt.toString());
+    //---------------------------------------------------
+    oldDec = HiveDecimalV1.create(bigInt, 35);
+    Assert.assertEquals("99999999.999999999999999999999999999999", oldDec.toString());
+    //---------------------------------------------------
+    // Without the round, this conversion fails.
+    dec = HiveDecimal.create(bigInt, 35);
+    Assert.assertEquals("99999999.999999999999999999999999999999", dec.toString());
+
+    // MAX_DECIMAL 9's WITH ROUND.
+    bigInt = new BigInteger(
+                        "9999999999999999999999999999999999999979999");
+    //                   12346678.901234667890123466789012346678
+    //                             1         2         3
+    Assert.assertEquals("9999999999999999999999999999999999999979999", bigInt.toString());
+    //---------------------------------------------------
+    oldDec = HiveDecimalV1.create(bigInt, 35);
+    Assert.assertEquals("100000000", oldDec.toString());
+    //---------------------------------------------------
+    dec = HiveDecimal.create(bigInt, 35);
+    Assert.assertEquals("100000000", dec.toString());
+  }
+
+  @Test
+  public void testCreateFromBigDecimal() {
+
+    BigDecimal bigDec;
+    HiveDecimalV1 oldDec;
+    HiveDecimal dec;
+
+    bigDec = new BigDecimal("0");
+    Assert.assertEquals("0", bigDec.toString());
+    dec = HiveDecimal.create(bigDec);
+    Assert.assertEquals("0", dec.toString());
+
+    bigDec = new BigDecimal("1");
+    Assert.assertEquals("1", bigDec.toString());
+    dec = HiveDecimal.create(bigDec);
+    Assert.assertEquals("1", dec.toString());
+
+    bigDec = new BigDecimal("0.999");
+    Assert.assertEquals("0.999", bigDec.toString());
+    dec = HiveDecimal.create(bigDec);
+    Assert.assertEquals("0.999", dec.toString());
+
+    // HiveDecimal suppresses trailing zeroes.
+    bigDec = new BigDecimal("0.9990");
+    Assert.assertEquals("0.9990", bigDec.toString());
+    dec = HiveDecimal.create(bigDec);
+    Assert.assertEquals("0.999", dec.toString());
+  }
+
+  @Test
+  public void testCreateFromBigDecimalRounding() {
+
+    BigDecimal bigDec;
+    HiveDecimalV1 oldDec;
+    HiveDecimal dec;
+
+    bigDec = new BigDecimal(
+                        "1786135888657847525803324040144343378.09799306448796128931113691624");
+    Assert.assertEquals("1786135888657847525803324040144343378.09799306448796128931113691624", bigDec.toString());
+    //                   1234567890123456789012345678901234567.8
+    //                            1         2         3
+    // Without the round, this conversion fails.
+    dec = HiveDecimal.create(bigDec, false);
+    Assert.assertTrue(dec == null);
+    dec = HiveDecimal.create(bigDec, true);
+    Assert.assertEquals("1786135888657847525803324040144343378.1", dec.toString());
+
+    bigDec = new BigDecimal(
+                        "8.090000000000000000000000000000000000000123456");
+    //                   1.23456789012345678901234567890123456789012345
+    //                             1         2         3         4
+    Assert.assertEquals("8.090000000000000000000000000000000000000123456", bigDec.toString());
+    //---------------------------------------------------
+    HiveDecimalV1 oldDec4 = HiveDecimalV1.create(bigDec, false);
+    Assert.assertTrue(oldDec4 == null);
+    oldDec4 = HiveDecimalV1.create(bigDec, true);
+    Assert.assertEquals("8.09", oldDec4.toString());
+    //---------------------------------------------------
+    // Without the round, this conversion fails.
+    dec = HiveDecimal.create(bigDec, false);
+    Assert.assertTrue(dec == null);
+    dec = HiveDecimal.create(bigDec, true);
+    Assert.assertEquals("8.09", dec.toString());
+
+    // MAX_DECIMAL 9's WITH NO ROUND (longer than 38 digits)
+    bigDec = new BigDecimal(
+                        "99999999.99999999999999999999999999999949999");
+    //                   12345678.901234567890123456789012345678
+    //                             1         2         3
+    Assert.assertEquals("99999999.99999999999999999999999999999949999", bigDec.toString());
+    //---------------------------------------------------
+    HiveDecimalV1 oldDec5 = HiveDecimalV1.create(bigDec, false);
+    Assert.assertTrue(oldDec5 == null);
+    oldDec5 = HiveDecimalV1.create(bigDec, true);
+    Assert.assertEquals("99999999.999999999999999999999999999999", oldDec5.toString());
+    //---------------------------------------------------
+    dec = HiveDecimal.create(bigDec, false);
+    Assert.assertTrue(dec == null);
+    dec = HiveDecimal.create(bigDec, true);
+    Assert.assertEquals("99999999.999999999999999999999999999999", dec.toString());
+
+    // MAX_DECIMAL 9's WITH ROUND.
+    bigDec = new BigDecimal(
+                        "99999999.99999999999999999999999999999979999");
+    //                   12346678.901234667890123466789012346678
+    //                             1         2         3
+    Assert.assertEquals("99999999.99999999999999999999999999999979999", bigDec.toString());
+    //---------------------------------------------------
+    HiveDecimalV1 oldDec6 = HiveDecimalV1.create(bigDec, false);
+    Assert.assertTrue(oldDec6 == null);
+    oldDec6 = HiveDecimalV1.create(bigDec, true);
+    Assert.assertEquals("100000000", oldDec6.toString());
+    //---------------------------------------------------
+    dec = HiveDecimal.create(bigDec, false);
+    Assert.assertTrue(dec == null);
+    dec = HiveDecimal.create(bigDec, true);
+    Assert.assertEquals("100000000", dec.toString());
+  }
+
+
+  @Test
+  public void testPrecisionScaleEnforcement() {
+
+    HiveDecimalV1 oldDec;
+    HiveDecimalV1 oldResultDec;
+
+    HiveDecimal dec;
+    HiveDecimal resultDec;
+
+    //---------------------------------------------------
+    oldDec = HiveDecimalV1.create("0.02538461538461538461538461538461538462");
+    Assert.assertEquals("0.02538461538461538461538",
+        HiveDecimalV1.enforcePrecisionScale(oldDec, 38, 23).toString());
+    //---------------------------------------------------
+    dec = HiveDecimal.create("0.02538461538461538461538461538461538462");
+    Assert.assertEquals("0.02538461538461538461538",
+        HiveDecimal.enforcePrecisionScale(dec, 38, 23).toString());
+
+    //---------------------------------------------------
+    oldDec = HiveDecimalV1.create("005.34000");
+    Assert.assertEquals(oldDec.precision(), 3);  // 1 integer digit; 2 fraction digits.
+    Assert.assertEquals(oldDec.scale(), 2);      // Trailing zeroes are suppressed.
+    //---------------------------------------------------
+    dec = HiveDecimal.create("005.34000");
+    Assert.assertEquals(dec.precision(), 3);  // 1 integer digit; 2 fraction digits.
+    Assert.assertEquals(dec.scale(), 2);      // Trailing zeroes are suppressed.
+
+    dec = HiveDecimal.create("178613588865784752580332404014434337809799306448796128931113691624");
+    Assert.assertNull(dec);
+
+    // Rounding numbers that increase int digits
+    //---------------------------------------------------
+    oldDec = HiveDecimalV1.create("9.5");
+    Assert.assertEquals("10",
+        HiveDecimalV1.enforcePrecisionScale(oldDec, 2, 0).toString());
+    Assert.assertNull(
+        HiveDecimalV1.enforcePrecisionScale(oldDec, 1, 0));
+    oldDec = HiveDecimalV1.create("9.4");
+    Assert.assertEquals("9",
+        HiveDecimalV1.enforcePrecisionScale(oldDec, 1, 0).toString());
+    //---------------------------------------------------
+    dec = HiveDecimal.create("9.5");
+    Assert.assertEquals("10",
+        HiveDecimal.enforcePrecisionScale(dec, 2, 0).toString());
+    Assert.assertNull(
+        HiveDecimal.enforcePrecisionScale(dec, 1, 0));
+    dec = HiveDecimal.create("9.4");
+    Assert.assertEquals("9",
+        HiveDecimal.enforcePrecisionScale(dec, 1, 0).toString());
+  }
+
+  @Test
+  public void testPrecisionScaleEnforcementEdgeCond() {
+
+    // Since HiveDecimal now uses FastHiveDecimal which stores 16 decimal digits per long,
+    // lets test edge conditions here.
+
+    HiveDecimal fifteenFractionalNinesDec = HiveDecimal.create("0.999999999999999");
+    Assert.assertNotNull(fifteenFractionalNinesDec);
+    Assert.assertEquals("0.999999999999999",
+        HiveDecimal.enforcePrecisionScale(fifteenFractionalNinesDec, 15, 15).toString());
+
+    HiveDecimal sixteenFractionalNines = HiveDecimal.create("0.9999999999999999");
+    Assert.assertNotNull(sixteenFractionalNines);
+    Assert.assertEquals("0.9999999999999999",
+        HiveDecimal.enforcePrecisionScale(sixteenFractionalNines, 16, 16).toString());
+
+    HiveDecimal seventeenFractionalNines = HiveDecimal.create("0.99999999999999999");
+    Assert.assertNotNull(seventeenFractionalNines);
+    Assert.assertEquals("0.99999999999999999",
+        HiveDecimal.enforcePrecisionScale(seventeenFractionalNines, 17, 17).toString());
+
+  }
+
+  @Test
+  public void testTrailingZeroRemovalAfterEnforcement() {
+    String decStr = "8.090000000000000000000000000000000000000123456";
+    //                 123456789012345678901234567890123456789012345
+    //                          1         2         3         4
+    HiveDecimal dec = HiveDecimal.create(decStr);
+    Assert.assertEquals("8.09", dec.toString());
+  }
+
+  @Test
+  public void testMultiply() {
+
+    // This multiply produces more than 38 digits --> overflow.
+    //---------------------------------------------------
+    HiveDecimalV1 oldDec1 = HiveDecimalV1.create("0.00001786135888657847525803");
+    HiveDecimalV1 oldDec2 = HiveDecimalV1.create("3.0000123456789");
+    HiveDecimalV1 oldResult = oldDec1.multiply(oldDec2);
+    Assert.assertTrue(oldResult == null);
+    //---------------------------------------------------
+    HiveDecimal dec1 = HiveDecimal.create("0.00001786135888657847525803");
+    HiveDecimal dec2 = HiveDecimal.create("3.0000123456789");
+    HiveDecimal result = dec1.multiply(dec2);
+    Assert.assertTrue(result == null);
+
+    dec1 = HiveDecimal.create("178613588865784752580323232232323444.4");
+    dec2 = HiveDecimal.create("178613588865784752580302323232.3");
+    Assert.assertNull(dec1.multiply(dec2));   // i.e. Overflow.
+
+    dec1 = HiveDecimal.create("47.324");
+    dec2 = HiveDecimal.create("9232.309");
+    Assert.assertEquals("436909.791116", dec1.multiply(dec2).toString());
+
+    dec1 = HiveDecimal.create("3.140");
+    dec2 = HiveDecimal.create("1.00");
+    Assert.assertEquals("3.14", dec1.multiply(dec2).toString());
+
+    dec1 = HiveDecimal.create("43.010");
+    dec2 = HiveDecimal.create("2");
+    Assert.assertEquals("86.02", dec1.multiply(dec2).toString());
+  }
+
+  @Test
+  public void testMultiply2() {
+    // 0.09765625BD * 0.09765625BD * 0.0125BD * 578992BD
+    HiveDecimal dec1 = HiveDecimal.create("0.09765625");
+    HiveDecimal dec2 = HiveDecimal.create("0.09765625");
+    HiveDecimal dec3 = HiveDecimal.create("0.0125");
+    HiveDecimal dec4 = HiveDecimal.create("578992");
+    HiveDecimal result1 = dec1.multiply(dec2);
+    Assert.assertNotNull(result1);
+    HiveDecimal result2 = result1.multiply(dec3);
+    Assert.assertNotNull(result2);
+    HiveDecimal result = result2.multiply(dec4);
+    Assert.assertNotNull(result);
+    Assert.assertEquals("69.0212249755859375", result.toString());
+  }
+
+  @Test
+  public void testPow() {
+
+    HiveDecimal dec;
+
+    dec = HiveDecimal.create("3.00001415926");
+    HiveDecimal decPow2 = dec.pow(2);
+    HiveDecimal decMultiplyTwice = dec.multiply(dec);
+    Assert.assertEquals(decPow2, decMultiplyTwice);
+
+    dec = HiveDecimal.create("0.000017861358882");
+    dec = dec.pow(3);
+    Assert.assertNull(dec);
+
+    dec = HiveDecimal.create("3.140");
+    Assert.assertEquals("9.8596", dec.pow(2).toString());
+  }
+
+  @Test
+  public void testScaleByPowerOfTen() {
+
+    HiveDecimalV1 oldDec;
+    HiveDecimal dec;
+    HiveDecimalV1 oldResultDec;
+    HiveDecimal resultDec;
+
+    //**********************************************************************************************
+
+    //---------------------------------------------------
+    oldDec = HiveDecimalV1.create(
+        "1");
+    Assert.assertEquals(0, oldDec.scale());
+    oldResultDec = oldDec.scaleByPowerOfTen(2);
+    Assert.assertEquals(
+        "100", oldResultDec.toString());
+    //---------------------------------------------------
+    dec = HiveDecimal.create(
+        "1");
+    Assert.assertEquals(0, dec.scale());
+    // resultDec = dec.scaleByPowerOfTen(2);
+    // Assert.assertEquals(
+    //     "100", resultDec.toString());
+
+    //---------------------------------------------------
+    oldDec = HiveDecimalV1.create(
+        "0.00000000000000000000000000000000000001");
+    Assert.assertEquals(38, oldDec.scale());
+    oldResultDec = oldDec.scaleByPowerOfTen(2);
+    Assert.assertEquals(
+        "0.000000000000000000000000000000000001", oldResultDec.toString());
+    //---------------------------------------------------
+    dec = HiveDecimal.create(
+        "0.00000000000000000000000000000000000001");
+    Assert.assertEquals(38, dec.scale());
+    resultDec = dec.scaleByPowerOfTen(2);
+    Assert.assertEquals(
+        "0.000000000000000000000000000000000001", resultDec.toString());
+
+    //---------------------------------------------------
+    oldDec = HiveDecimalV1.create(
+        "0.00000000000000000000000000000000000001");
+    Assert.assertEquals(38, oldDec.scale());
+    oldResultDec = oldDec.scaleByPowerOfTen(38);
+    Assert.assertEquals(
+        "1", oldResultDec.toString());
+    //---------------------------------------------------
+    dec = HiveDecimal.create(
+        "0.00000000000000000000000000000000000001");
+    Assert.assertEquals(38, dec.scale());
+    resultDec = dec.scaleByPowerOfTen(38);
+    Assert.assertEquals(
+        "1", resultDec.toString());
+
+    //---------------------------------------------------
+    oldDec = HiveDecimalV1.create(
+        "0.00000000000000000000000000000000000001");
+    Assert.assertEquals(38, oldDec.scale());
+    oldResultDec = oldDec.scaleByPowerOfTen(2 * 38 - 1);
+    Assert.assertEquals(
+        "10000000000000000000000000000000000000", oldResultDec.toString());
+    Assert.assertEquals(0, oldResultDec.scale());
+    //---------------------------------------------------
+    dec = HiveDecimal.create(
+        "0.00000000000000000000000000000000000001");
+    Assert.assertEquals(38, dec.scale());
+    resultDec = dec.scaleByPowerOfTen(2 * 38 - 1);
+    Assert.assertEquals(
+        "10000000000000000000000000000000000000", resultDec.toString());
+    Assert.assertEquals(0, resultDec.scale());
+
+    //---------------------------------------------------
+    oldDec = HiveDecimalV1.create(
+        "0.00000000000000000000000000000000000001");
+    Assert.assertEquals(38, oldDec.scale());
+    oldResultDec = oldDec.scaleByPowerOfTen(2 * 38);
+    Assert.assertTrue(oldResultDec == null);
+    //---------------------------------------------------
+    dec = HiveDecimal.create(
+        "0.00000000000000000000000000000000000001");
+    Assert.assertEquals(38, dec.scale());
+    resultDec = dec.scaleByPowerOfTen(2 * 38);
+    Assert.assertTrue(resultDec == null);
+
+
+    //---------------------------------------------------
+    oldDec = HiveDecimalV1.create(
+        "0.00000000000000000000000000000000000022");
+    Assert.assertEquals(38, oldDec.scale());
+    oldResultDec = oldDec.scaleByPowerOfTen(38);
+    Assert.assertEquals(
+        "22", oldResultDec.toString());
+    Assert.assertEquals(0, oldResultDec.scale());
+    //---------------------------------------------------
+    dec = HiveDecimal.create(
+        "0.00000000000000000000000000000000000022");
+    Assert.assertEquals(38, dec.scale());
+    resultDec = dec.scaleByPowerOfTen(38);
+    Assert.assertEquals(
+        "22", resultDec.toString());
+    Assert.assertEquals(0, resultDec.scale());
+
+    //---------------------------------------------------
+    oldDec = HiveDecimalV1.create("3.00001415926");
+    Assert.assertEquals(11, oldDec.scale());
+    oldResultDec = oldDec.scaleByPowerOfTen(2);
+    Assert.assertEquals("300.001415926", oldResultDec.toString());
+    Assert.assertEquals(9, oldResultDec.scale());
+    oldResultDec = oldDec.scaleByPowerOfTen(5);
+    Assert.assertEquals("300001.415926", oldResultDec.toString());
+    Assert.assertEquals(6, oldResultDec.scale());
+    oldResultDec = oldDec.scaleByPowerOfTen(18);
+    Assert.assertEquals("3000014159260000000", oldResultDec.toString());
+    Assert.assertEquals(0, oldResultDec.scale());
+    oldResultDec = oldDec.scaleByPowerOfTen(35);
+    Assert.assertEquals("300001415926000000000000000000000000", oldResultDec.toString());
+    Assert.assertEquals(0, oldResultDec.scale());
+    oldResultDec = oldDec.scaleByPowerOfTen(37);
+    Assert.assertEquals("30000141592600000000000000000000000000", oldResultDec.toString());
+    Assert.assertEquals(0, oldResultDec.scale());
+    //---------------------------------------------------
+    dec = HiveDecimal.create("3.00001415926");
+    Assert.assertEquals(11, dec.scale());
+    Assert.assertEquals(1, dec.integerDigitCount());
+    resultDec = dec.scaleByPowerOfTen(2);
+    Assert.assertEquals("300.001415926", resultDec.toString());
+    Assert.assertEquals(9, resultDec.scale());
+    Assert.assertEquals(3, resultDec.integerDigitCount());
+    resultDec = dec.scaleByPowerOfTen(5);
+    Assert.assertEquals("300001.415926", resultDec.toString());
+    Assert.assertEquals(6, resultDec.scale());
+    Assert.assertEquals(6, resultDec.integerDigitCount());
+    resultDec = dec.scaleByPowerOfTen(18);
+    Assert.assertEquals("3000014159260000000", resultDec.toString());
+    Assert.assertEquals(0, resultDec.scale());
+    Assert.assertEquals(19, resultDec.integerDigitCount());
+    resultDec = dec.scaleByPowerOfTen(35);
+    Assert.assertEquals("300001415926000000000000000000000000", resultDec.toString());
+    Assert.assertEquals(0, resultDec.scale());
+    Assert.assertEquals(36, resultDec.integerDigitCount());
+    resultDec = dec.scaleByPowerOfTen(37);
+    Assert.assertEquals("30000141592600000000000000000000000000", resultDec.toString());
+    Assert.assertEquals(0, resultDec.scale());
+    Assert.assertEquals(38, resultDec.integerDigitCount());
+  }
+
+  @Test
+  public void testSingleWordDivision() {
+ 
+    HiveDecimalV1 oldDec1;
+    HiveDecimalV1 oldDec2;
+    HiveDecimalV1 oldResultDec;
+
+    HiveDecimal dec1;
+    HiveDecimal dec2;
+    HiveDecimal resultDec;
+
+    //---------------------------------------------------
+    oldDec1 = HiveDecimalV1.create("839293");
+    oldDec2 = HiveDecimalV1.create("8");
+    oldResultDec = oldDec1.divide(oldDec2);
+    Assert.assertEquals("104911.625", oldResultDec.toString());
+    //---------------------------------------------------
+    dec1 = HiveDecimal.create("839293");
+    dec2 = HiveDecimal.create("8");
+    resultDec = dec1.divide(dec2);
+    Assert.assertEquals("104911.625", resultDec.toString());  // UNDONE
+
+    //---------------------------------------------------
+    oldDec1 = HiveDecimalV1.create("1");
+    oldDec2 = HiveDecimalV1.create("3");
+    oldResultDec = oldDec1.divide(oldDec2);
+    Assert.assertEquals("0.33333333333333333333333333333333333333", oldResultDec.toString());
+    //---------------------------------------------------
+    dec1 = HiveDecimal.create("1");
+    dec2 = HiveDecimal.create("3");
+    resultDec = dec1.divide(dec2);
+    Assert.assertEquals("0.33333333333333333333333333333333333333", resultDec.toString());  // UNDONE
+
+    //---------------------------------------------------
+    oldDec1 = HiveDecimalV1.create("1");
+    oldDec2 = HiveDecimalV1.create("9");
+    oldResultDec = oldDec1.divide(oldDec2);
+    Assert.assertEquals("0.11111111111111111111111111111111111111", oldResultDec.toString());
+    //---------------------------------------------------
+    dec1 = HiveDecimal.create("1");
+    dec2 = HiveDecimal.create("9");
+    resultDec = dec1.divide(dec2);
+    Assert.assertEquals("0.11111111111111111111111111111111111111", resultDec.toString());  // UNDONE
+
+    //---------------------------------------------------
+    oldDec1 = HiveDecimalV1.create("22");
+    oldDec2 = HiveDecimalV1.create("7");
+    oldResultDec = oldDec1.divide(oldDec2);
+    Assert.assertEquals("3.1428571428571428571428571428571428571", oldResultDec.toString());
+    //---------------------------------------------------
+    dec1 = HiveDecimal.create("22");
+    dec2 = HiveDecimal.create("7");
+    resultDec = dec1.divide(dec2);
+    Assert.assertEquals("3.1428571428571428571428571428571428571", resultDec.toString());  // UNDONE
+
+    //---------------------------------------------------
+    oldDec1 = HiveDecimalV1.create("1");
+    oldDec2 = HiveDecimalV1.create("81");
+    oldResultDec = oldDec1.divide(oldDec2);
+    Assert.assertEquals("0.01234567901234567901234567901234567901", oldResultDec.toString());
+    //---------------------------------------------------
+    dec1 = HiveDecimal.create("1");
+    dec2 = HiveDecimal.create("81");
+    resultDec = dec1.divide(dec2);
+    Assert.assertEquals("0.01234567901234567901234567901234567901", resultDec.toString());  // UNDONE
+
+    //---------------------------------------------------
+    oldDec1 = HiveDecimalV1.create("425");
+    oldDec2 = HiveDecimalV1.create("1000000000000000");
+    oldResultDec = oldDec1.divide(oldDec2);
+    Assert.assertEquals("0.000000000000425", oldResultDec.toString());
+    //---------------------------------------------------
+    dec1 = HiveDecimal.create("425");
+    dec2 = HiveDecimal.create("1000000000000000");
+    resultDec = dec1.divide(dec2);
+    Assert.assertEquals("0.000000000000425", resultDec.toString());  // UNDONE
+
+    //---------------------------------------------------
+    oldDec1 = HiveDecimalV1.create("0.000000000088");
+    oldDec2 = HiveDecimalV1.create("1000000000000000");
+    oldResultDec = oldDec1.divide(oldDec2);
+    Assert.assertEquals("0.000000000000000000000000088", oldResultDec.toString());
+    Assert.assertEquals(27, oldResultDec.scale());
+    //---------------------------------------------------
+    dec1 = HiveDecimal.create("0.000000000088");
+    dec2 = HiveDecimal.create("1000000000000000");
+    resultDec = dec1.divide(dec2);
+    Assert.assertEquals("0.000000000000000000000000088", resultDec.toString());  // UNDONE
+    Assert.assertEquals(27, resultDec.scale());
+   }
+
+  @Test
+  public void testDivide() {
+    HiveDecimal dec1 = HiveDecimal.create("3.14");
+    HiveDecimal dec2 = HiveDecimal.create("3");
+    Assert.assertNotNull(dec1.divide(dec2));
+
+    dec1 = HiveDecimal.create("15");
+    dec2 = HiveDecimal.create("5");
+    Assert.assertEquals("3", dec1.divide(dec2).toString());
+
+    dec1 = HiveDecimal.create("3.140");
+    dec2 = HiveDecimal.create("1.00");
+    Assert.assertEquals("3.14", dec1.divide(dec2).toString());
+  }
+
+  @Test
+  public void testPlus() {
+
+    HiveDecimalV1 oldDec;
+    HiveDecimalV1 oldDec2;
+    HiveDecimalV1 oldResultDec;
+
+    HiveDecimal dec;
+    HiveDecimal dec1;
+    HiveDecimal dec2;
+    HiveDecimal resultDec;
+
+    String decStr;
+    String decStr2;
+
+    dec1 = HiveDecimal.create("3.140");
+    dec1.validate();
+    dec2 = HiveDecimal.create("1.00");
+    dec2.validate();
+    resultDec = dec1.add(dec2);
+    resultDec.validate();
+    Assert.assertEquals("4.14", resultDec.toString());
+
+    decStr = "3.140";
+    decStr2 = "1.00";
+    //---------------------------------------------------
+    oldDec = HiveDecimalV1.create(decStr);
+    oldDec2 = HiveDecimalV1.create(decStr2);
+    oldResultDec = oldDec.add(oldDec2);
+    Assert.assertEquals("4.14", oldResultDec.toString());
+    //---------------------------------------------------
+    dec = HiveDecimal.create(decStr);
+    assertTrue(dec != null);
+    dec.validate();
+    dec2 = HiveDecimal.create(decStr2);
+    assertTrue(dec2 != null);
+    dec2.validate();
+    resultDec = dec.add(dec2);
+    assertTrue(resultDec != null);
+    resultDec.validate();
+    Assert.assertEquals("4.14", resultDec.toString());
+    Assert.assertEquals(1, resultDec.integerDigitCount());
+
+    decStr = "3.140";
+    decStr2 = "1.00000008733";
+    //---------------------------------------------------
+    oldDec = HiveDecimalV1.create(decStr);
+    oldDec2 = HiveDecimalV1.create(decStr2);
+    oldResultDec = oldDec.add(oldDec2);
+    Assert.assertEquals("4.14000008733", oldResultDec.toString());
+    //---------------------------------------------------
+    dec = HiveDecimal.create(decStr);
+    assertTrue(dec != null);
+    dec.validate();
+    dec2 = HiveDecimal.create(decStr2);
+    assertTrue(dec2 != null);
+    dec2.validate();
+    resultDec = dec.add(dec2);
+    assertTrue(resultDec != null);
+    resultDec.validate();
+    Assert.assertEquals("4.14000008733", resultDec.toString());
+    Assert.assertEquals(1, resultDec.integerDigitCount());
+
+    decStr = "3.140";
+    decStr2 = "1.00000000000000000008733";
+    //---------------------------------------------------
+    oldDec = HiveDecimalV1.create(decStr);
+    oldDec2 = HiveDecimalV1.create(decStr2);
+    oldResultDec = oldDec.add(oldDec2);
+    Assert.assertEquals("4.14000000000000000008733", oldResultDec.toString());
+    //---------------------------------------------------
+    dec = HiveDecimal.create(decStr);
+    assertTrue(dec != null);
+    dec.validate();
+    dec2 = HiveDecimal.create(decStr2);
+    assertTrue(dec2 != null);
+    dec2.validate();
+    resultDec = dec.add(dec2);
+    assertTrue(resultDec != null);
+    resultDec.validate();
+    Assert.assertEquals("4.14000000000000000008733", resultDec.toString());
+    Assert.assertEquals(1, resultDec.integerDigitCount());
+
+    decStr = "30000000000.140";
+    decStr2 = "1.00000000000000000008733";
+    //---------------------------------------------------
+    oldDec = HiveDecimalV1.create(decStr);
+    oldDec2 = HiveDecimalV1.create(decStr2);
+    oldResultDec = oldDec.add(oldDec2);
+    Assert.assertEquals("30000000001.14000000000000000008733", oldResultDec.toString());
+    //---------------------------------------------------
+    dec = HiveDecimal.create(decStr);
+    assertTrue(dec != null);
+    dec.validate();
+    dec2 = HiveDecimal.create(decStr2);
+    assertTrue(dec2 != null);
+    dec2.validate();
+    resultDec = dec.add(dec2);
+    assertTrue(resultDec != null);
+    resultDec.validate();
+    Assert.assertEquals("30000000001.14000000000000000008733", resultDec.toString());
+    // Assert.assertEquals(1, resultDec.integerDigitCount());
+
+    decStr = "300000000000000.140";
+    decStr2 = "1.00000000000000000008733";
+    //---------------------------------------------------
+    oldDec = HiveDecimalV1.create(decStr);
+    oldDec2 = HiveDecimalV1.create(decStr2);
+    oldResultDec = oldDec.add(oldDec2);
+    Assert.assertEquals("300000000000001.14000000000000000008733", oldResultDec.toString());
+    //---------------------------------------------------
+    dec = HiveDecimal.create(decStr);
+    assertTrue(dec != null);
+    dec.validate();
+    dec2 = HiveDecimal.create(decStr2);
+    assertTrue(dec2 != null);
+    dec2.validate();
+    resultDec = dec.add(dec2);
+    assertTrue(resultDec != null);
+    resultDec.validate();
+    Assert.assertEquals("300000000000001.14000000000000000008733", resultDec.toString());
+    // Assert.assertEquals(1, resultDec.integerDigitCount());
+
+    // Edge case?
+    decStr = "3000000000000000.140";
+    decStr2 = "1.00000000000000000008733";
+    //---------------------------------------------------
+    oldDec = HiveDecimalV1.create(decStr);
+    oldDec2 = HiveDecimalV1.create(decStr2);
+    oldResultDec = oldDec.add(oldDec2);
+    Assert.assertEquals("3000000000000001.1400000000000000000873", oldResultDec.toString());
+    //---------------------------------------------------
+    dec = HiveDecimal.create(decStr);
+    assertTrue(dec != null);
+    dec.validate();
+    dec2 = HiveDecimal.create(decStr2);
+    assertTrue(dec2 != null);
+    dec2.validate();
+    resultDec = dec.add(dec2);
+    assertTrue(resultDec != null);
+    resultDec.validate();
+    Assert.assertEquals("3000000000000001.1400000000000000000873", resultDec.toString());
+    // Assert.assertEquals(1, resultDec.integerDigitCount());
+
+    decStr = "300000000000000000000000.14";
+    decStr2 = "0.0000055555555550008733";
+    //---------------------------------------------------
+    oldDec = HiveDecimalV1.create(decStr);
+    oldDec2 = HiveDecimalV1.create(decStr2);
+    oldResultDec = oldDec.add(oldDec2);
+    Assert.assertEquals("300000000000000000000000.14000555555556", oldResultDec.toString());
+    //---------------------------------------------------
+    dec = HiveDecimal.create(decStr);
+    assertTrue(dec != null);
+    dec.validate();
+    dec2 = HiveDecimal.create(decStr2);
+    assertTrue(dec2 != null);
+    dec2.validate();
+    resultDec = dec.add(dec2);
+    assertTrue(resultDec != null);
+    resultDec.validate();
+    Assert.assertEquals("300000000000000000000000.14000555555556", resultDec.toString());
+    // Assert.assertEquals(1, resultDec.integerDigitCount());
+
+    decStr = "300000000000000000000000.14";
+    decStr2 = "0.000005555555555000873355";
+    //---------------------------------------------------
+    oldDec = HiveDecimalV1.create(decStr);
+    oldDec2 = HiveDecimalV1.create(decStr2);
+    oldResultDec = oldDec.add(oldDec2);
+    Assert.assertEquals("300000000000000000000000.14000555555556", oldResultDec.toString());
+    //---------------------------------------------------
+    dec = HiveDecimal.create(decStr);
+    assertTrue(dec != null);
+    dec.validate();
+    dec2 = HiveDecimal.create(decStr2);
+    assertTrue(dec2 != null);
+    dec2.validate();
+    resultDec = dec.add(dec2);
+    assertTrue(resultDec != null);
+    resultDec.validate();
+    Assert.assertEquals("300000000000000000000000.14000555555556", resultDec.toString());
+    // Assert.assertEquals(1, resultDec.integerDigitCount());
+
+
+
+    // Example from HiveDecimal.add header comments.
+    decStr = "598575157855521918987423259.94094";
+    decStr2 = "0.0000000000006711991169422033";
+    //---------------------------------------------------
+    oldDec = HiveDecimalV1.create(decStr);
+    oldDec2 = HiveDecimalV1.create(decStr2);
+    oldResultDec = oldDec.add(oldDec2);
+    Assert.assertEquals("598575157855521918987423259.94094", oldResultDec.toString());
+    //---------------------------------------------------
+    dec = HiveDecimal.create(decStr);
+    assertTrue(dec != null);
+    dec.validate();
+    dec2 = HiveDecimal.create(decStr2);
+    assertTrue(dec2 != null);
+    dec2.validate();
+    resultDec = dec.add(dec2);
+    assertTrue(resultDec != null);
+    resultDec.validate();
+    Assert.assertEquals("598575157855521918987423259.94094", resultDec.toString());
+    Assert.assertEquals(27, resultDec.integerDigitCount());
+
+    decStr = "598575157855521918987423259.94094";
+    decStr2 = "0.5555555555556711991169422033";
+    //---------------------------------------------------
+    oldDec = HiveDecimalV1.create(decStr);
+    oldDec2 = HiveDecimalV1.create(decStr2);
+    oldResultDec = oldDec.add(oldDec2);
+    Assert.assertEquals("598575157855521918987423260.49649555556", oldResultDec.toString());
+    //---------------------------------------------------
+    dec = HiveDecimal.create(decStr);
+    assertTrue(dec != null);
+    dec.validate();
+    dec2 = HiveDecimal.create(decStr2);
+    assertTrue(dec2 != null);
+    dec2.validate();
+    resultDec = dec.add(dec2);
+    assertTrue(resultDec != null);
+    resultDec.validate();
+    Assert.assertEquals("598575157855521918987423260.49649555556", resultDec.toString());
+    Assert.assertEquals(27, resultDec.integerDigitCount());
+
+    decStr = "199999999.99995";
+    decStr2 = "100000000.00005";
+    //---------------------------------------------------
+    oldDec = HiveDecimalV1.create(decStr);
+    oldDec2 = HiveDecimalV1.create(decStr2);
+    oldResultDec = oldDec.add(oldDec2);
+    Assert.assertEquals("300000000", oldResultDec.toString());
+    //---------------------------------------------------
+    dec = HiveDecimal.create(decStr);
+    assertTrue(dec != null);
+    dec.validate();
+    dec2 = HiveDecimal.create(decStr2);
+    assertTrue(dec2 != null);
+    dec2.validate();
+    resultDec = dec.add(dec2);
+    assertTrue(resultDec != null);
+    resultDec.validate();
+    Assert.assertEquals("300000000", resultDec.toString());
+    Assert.assertEquals(9, resultDec.integerDigitCount());
+
+    dec1 = HiveDecimal.create("99999999999999999999999999999999999999");
+    dec1.validate();
+    Assert.assertEquals(38, dec1.integerDigitCount());
+    dec2 = HiveDecimal.create("1");
+    dec2.validate();
+    Assert.assertNull(dec1.add(dec2));
+  }
+
+  @Test
+  public void testAdd() {
+
+    HiveDecimalV1 oldDec;
+    HiveDecimalV1 oldDec2;
+    HiveDecimalV1 oldResultDec;
+
+    HiveDecimal dec;
+    HiveDecimal dec2;
+    HiveDecimal resultDec;
+
+    // Use the example from HIVE-13423 where the integer digits of the result exceed the
+    // enforced precision/scale.
+    //---------------------------------------------------
+    oldDec = HiveDecimalV1.create("98765432109876543210.12345");
+    oldResultDec = oldDec.add(oldDec);
+    Assert.assertEquals("197530864219753086420.2469", oldResultDec.toString());
+    oldResultDec = HiveDecimalV1.enforcePrecisionScale(oldResultDec, 38, 18);
+    Assert.assertTrue(oldResultDec == null);
+    //---------------------------------------------------
+    dec = HiveDecimal.create("98765432109876543210.12345");
+    assertTrue(dec != null);
+    dec.validate();
+    resultDec = dec.add(dec);
+    assertTrue(resultDec != null);
+    resultDec.validate();
+    Assert.assertEquals("197530864219753086420.2469", resultDec.toString());
+    // Assert.assertEquals(21, resultDec.integerDigitCount());
+    resultDec = HiveDecimal.enforcePrecisionScale(resultDec, 38, 18);
+    Assert.assertTrue(resultDec == null);
+ 
+    // Make sure zero trimming doesn't extend into the integer digits.
+    //---------------------------------------------------
+    oldDec = HiveDecimalV1.create("199999999.99995");
+    oldDec2 = HiveDecimalV1.create("100000000.00005");
+    oldResultDec = oldDec.add(oldDec2);
+    Assert.assertEquals("300000000", oldResultDec.toString());
+    //---------------------------------------------------
+    dec = HiveDecimal.create("199999999.99995");
+    assertTrue(dec != null);
+    dec.validate();
+    dec2 = HiveDecimal.create("100000000.00005");
+    assertTrue(dec2 != null);
+    dec2.validate();
+    resultDec = dec.add(dec2);
+    assertTrue(resultDec != null);
+    resultDec.validate();
+    Assert.assertEquals("300000000", resultDec.toString());
+    Assert.assertEquals(9, resultDec.integerDigitCount());
+   }
+
+  @Test
+  public void testMinus() {
+
+    HiveDecimalV1 oldDec;
+    HiveDecimalV1 oldDec2;
+    HiveDecimalV1 oldResultDec;
+
+    HiveDecimal dec;
+    HiveDecimal dec1;
+    HiveDecimal dec2;
+    HiveDecimal resultDec;
+
+    String decStr;
+    String decStr2;
+
+    dec1 = HiveDecimal.create("3.140");
+    dec1.validate();
+    dec2 = HiveDecimal.create("1.00");
+    dec2.validate();
+    resultDec = dec1.add(dec2);
+    resultDec.validate();
+    Assert.assertEquals("4.14", resultDec.toString());
+
+    decStr = "3.140";
+    decStr2 = "1.00";
+    //---------------------------------------------------
+    oldDec = HiveDecimalV1.create(decStr);
+    oldDec2 = HiveDecimalV1.create(decStr2);
+    oldResultDec = oldDec.subtract(oldDec2);
+    Assert.assertEquals("2.14", oldResultDec.toString());
+    //---------------------------------------------------
+    dec = HiveDecimal.create(decStr);
+    assertTrue(dec != null);
+    dec.validate();
+    dec2 = HiveDecimal.create(decStr2);
+    assertTrue(dec2 != null);
+    dec2.validate();
+    resultDec = dec.subtract(dec2);
+    assertTrue(resultDec != null);
+    resultDec.validate();
+    Assert.assertEquals("2.14", resultDec.toString());
+    // Assert.assertEquals(1, resultDec.integerDigitCount());
+
+    decStr = "3.140";
+    decStr2 = "1.00000008733";
+    //---------------------------------------------------
+    oldDec = HiveDecimalV1.create(decStr);
+    oldDec2 = HiveDecimalV1.create(decStr2);
+    oldResultDec = oldDec.subtract(oldDec2);
+    Assert.assertEquals("2.13999991267", oldResultDec.toString());
+    //---------------------------------------------------
+    dec = HiveDecimal.create(decStr);
+    assertTrue(dec != null);
+    dec.validate();
+    dec2 = HiveDecimal.create(decStr2);
+    assertTrue(dec2 != null);
+    dec2.validate();
+    resultDec = dec.subtract(dec2);
+    assertTrue(resultDec != null);
+    resultDec.validate();
+    Assert.assertEquals("2.13999991267", resultDec.toString());
+    // Assert.assertEquals(1, resultDec.integerDigitCount());
+
+    decStr = "3.140";
+    decStr2 = "1.00000000000000000008733";
+    //---------------------------------------------------
+    oldDec = HiveDecimalV1.create(decStr);
+    oldDec2 = HiveDecimalV1.create(decStr2);
+    oldResultDec = oldDec.subtract(oldDec2);
+    Assert.assertEquals("2.13999999999999999991267", oldResultDec.toString());
+    //---------------------------------------------------
+    dec = HiveDecimal.create(decStr);
+    assertTrue(dec != null);
+    dec.validate();
+    dec2 = HiveDecimal.create(decStr2);
+    assertTrue(dec2 != null);
+    dec2.validate();
+    resultDec = dec.subtract(dec2);
+    assertTrue(resultDec != null);
+    resultDec.validate();
+    Assert.assertEquals("2.13999999999999999991267", resultDec.toString());
+    // Assert.assertEquals(1, resultDec.integerDigitCount());
+
+    decStr = "30000000000.140";
+    decStr2 = "1.00000000000000000008733";
+    //---------------------------------------------------
+    oldDec = HiveDecimalV1.create(decStr);
+    oldDec2 = HiveDecimalV1.create(decStr2);
+    oldResultDec = oldDec.subtract(oldDec2);
+    Assert.assertEquals("29999999999.13999999999999999991267", oldResultDec.toString());
+    //---------------------------------------------------
+    dec = HiveDecimal.create(decStr);
+    assertTrue(dec != null);
+    dec.validate();
+    dec2 = HiveDecimal.create(decStr2);
+    assertTrue(dec2 != null);
+    dec2.validate();
+    resultDec = dec.subtract(dec2);
+    assertTrue(resultDec != null);
+    resultDec.validate();
+    Assert.assertEquals("29999999999.13999999999999999991267", resultDec.toString());
+    // Assert.assertEquals(1, resultDec.integerDigitCount());
+
+    decStr = "300000000000000.140";
+    decStr2 = "1.00000000000000000008733";
+    //---------------------------------------------------
+    oldDec = HiveDecimalV1.create(decStr);
+    oldDec2 = HiveDecimalV1.create(decStr2);
+    oldResultDec = oldDec.subtract(oldDec2);
+    Assert.assertEquals("299999999999999.13999999999999999991267", oldResultDec.toString());
+    //---------------------------------------------------
+    dec = HiveDecimal.create(decStr);
+    assertTrue(dec != null);
+    dec.validate();
+    dec2 = HiveDecimal.create(decStr2);
+    assertTrue(dec2 != null);
+    dec2.validate();
+    resultDec = dec.subtract(dec2);
+    assertTrue(resultDec != null);
+    resultDec.validate();
+    Assert.assertEquals("299999999999999.13999999999999999991267", resultDec.toString());
+    // Assert.assertEquals(1, resultDec.integerDigitCount());
+
+    // Edge case?
+    decStr = "3000000000000000.140";
+    decStr2 = "1.00000000000000000008733";
+    //---------------------------------------------------
+    oldDec = HiveDecimalV1.create(decStr);
+    oldDec2 = HiveDecimalV1.create(decStr2);
+    oldResultDec = oldDec.subtract(oldDec2);
+    Assert.assertEquals("2999999999999999.1399999999999999999127", oldResultDec.toString());
+    //---------------------------------------------------
+    dec = HiveDecimal.create(decStr);
+    assertTrue(dec != null);
+    dec.validate();
+    dec2 = HiveDecimal.create(decStr2);
+    assertTrue(dec2 != null);
+    dec2.validate();
+    resultDec = dec.subtract(dec2);
+    assertTrue(resultDec != null);
+    resultDec.validate();
+    Assert.assertEquals("2999999999999999.1399999999999999999127", resultDec.toString());
+    // Assert.assertEquals(1, resultDec.integerDigitCount());
+
+    decStr = "300000000000000000000000.14";
+    decStr2 = "0.0000055555555550008733";
+    //---------------------------------------------------
+    oldDec = HiveDecimalV1.create(decStr);
+    oldDec2 = HiveDecimalV1.create(decStr2);
+    oldResultDec = oldDec.subtract(oldDec2);
+    Assert.assertEquals("300000000000000000000000.13999444444444", oldResultDec.toString());
+    //---------------------------------------------------
+    dec = HiveDecimal.create(decStr);
+    assertTrue(dec != null);
+    dec.validate();
+    dec2 = HiveDecimal.create(decStr2);
+    assertTrue(dec2 != null);
+    dec2.validate();
+    resultDec = dec.subtract(dec2);
+    assertTrue(resultDec != null);
+    resultDec.validate();
+    Assert.assertEquals("300000000000000000000000.13999444444444", resultDec.toString());
+    // Assert.assertEquals(1, resultDec.integerDigitCount());
+
+    decStr = "300000000000000000000000.14";
+    decStr2 = "0.000005555555555000873355";
+    //---------------------------------------------------
+    oldDec = HiveDecimalV1.create(decStr);
+    oldDec2 = HiveDecimalV1.create(decStr2);
+    oldResultDec = oldDec.subtract(oldDec2);
+    Assert.assertEquals("300000000000000000000000.13999444444444", oldResultDec.toString());
+    //---------------------------------------------------
+    dec = HiveDecimal.create(decStr);
+    assertTrue(dec != null);
+    dec.validate();
+    dec2 = HiveDecimal.create(decStr2);
+    assertTrue(dec2 != null);
+    dec2.validate();
+    resultDec = dec.subtract(dec2);
+    assertTrue(resultDec != null);
+    resultDec.validate();
+    Assert.assertEquals("300000000000000000000000.13999444444444", resultDec.toString());
+    // Assert.assertEquals(1, resultDec.integerDigitCount());
+
+    // Example from HiveDecimal.subtract header comments.
+    decStr = "598575157855521918987423259.94094";
+    decStr2 = "0.0000000000006711991169422033";
+    //---------------------------------------------------
+    oldDec = HiveDecimalV1.create(decStr);
+    oldDec2 = HiveDecimalV1.create(decStr2);
+    oldResultDec = oldDec.subtract(oldDec2);
+    Assert.assertEquals("598575157855521918987423259.94094", oldResultDec.toString());
+    //---------------------------------------------------
+    dec = HiveDecimal.create(decStr);
+    assertTrue(dec != null);
+    dec.validate();
+    dec2 = HiveDecimal.create(decStr2);
+    assertTrue(dec2 != null);
+    dec2.validate();
+    resultDec = dec.subtract(dec2);
+    assertTrue(resultDec != null);
+    resultDec.validate();
+    Assert.assertEquals("598575157855521918987423259.94094", resultDec.toString());
+    // Assert.assertEquals(1, resultDec.integerDigitCount());
+
+    decStr = "598575157855521918987423259.94094";
+    decStr2 = "0.5555555555556711991169422033";
+    //---------------------------------------------------
+    oldDec = HiveDecimalV1.create(decStr);
+    oldDec2 = HiveDecimalV1.create(decStr2);
+    oldResultDec = oldDec.subtract(oldDec2);
+    Assert.assertEquals("598575157855521918987423259.38538444444", oldResultDec.toString());
+    //---------------------------------------------------
+    dec = HiveDecimal.create(decStr);
+    assertTrue(dec != null);
+    dec.validate();
+    dec2 = HiveDecimal.create(decStr2);
+    assertTrue(dec2 != null);
+    dec2.validate();
+    resultDec = dec.subtract(dec2);
+    assertTrue(resultDec != null);
+    resultDec.validate();
+    Assert.assertEquals("598575157855521918987423259.38538444444", resultDec.toString());
+    // Assert.assertEquals(1, resultDec.integerDigitCount());
+  }
+
+  @Test
+  public void testSubtract() {
+    HiveDecimal dec1 = HiveDecimal.create("3.140");
+    assertTrue(dec1 != null);
+    dec1.validate();
+    HiveDecimal dec2 = HiveDecimal.create("1.00");
+    assertTrue(dec2 != null);
+    dec2.validate();
+    HiveDecimal result = dec1.subtract(dec2);
+    assertTrue(result != null);
+    result.validate();
+    Assert.assertEquals("2.14", result.toString());
+
+    dec1 = HiveDecimal.create("0.00001786135888657847525803");
+    assertTrue(dec1 != null);
+    dec1.validate();
+    dec2 = HiveDecimal.create("3.0000123456789");
+    assertTrue(dec2 != null);
+    dec2.validate();
+    result = dec1.subtract(dec2);
+    assertTrue(result != null);
+    result.validate();
+    Assert.assertEquals("-2.99999448432001342152474197", result.toString());
+  }
+
+  @Test
+  public void testPosMod() {
+    HiveDecimal hd1 = HiveDecimal.create("-100.91");
+    assertTrue(hd1 != null);
+    hd1.validate();
+    HiveDecimal hd2 = HiveDecimal.create("9.8");
+    assertTrue(hd2 != null);
+    hd2.validate();
+    HiveDecimal dec = hd1.remainder(hd2).add(hd2).remainder(hd2);
+    assertTrue(dec != null);
+    dec.validate();
+    Assert.assertEquals("6.89", dec.toString());
+  }
+
+  @Test
+  public void testHashCode() {
+      Assert.assertEquals(HiveDecimal.create("9").hashCode(), HiveDecimal.create("9.00").hashCode());
+      Assert.assertEquals(HiveDecimal.create("0").hashCode(), HiveDecimal.create("0.00").hashCode());
+  }
+
+  @Test
+  public void testException() {
+    HiveDecimal dec = HiveDecimal.create("3.1415.926");
+    Assert.assertNull(dec);
+    dec = HiveDecimal.create("3abc43");
+    Assert.assertNull(dec);
+  }
+
+  @Test
+  public void testBinaryConversion() {
+    Random r = new Random(2399);
+    for (String decString : specialDecimalStrings) {
+      doTestBinaryConversion(decString, r);
+    }
+  }
+
+  private void doTestBinaryConversion(String num, Random r) {
+    int scale = r.nextInt(HiveDecimal.MAX_SCALE);
+    HiveDecimal dec = HiveDecimal.create(num);
+    if (dec == null) {
+      return;
+    }
+    byte[] d = dec.setScale(scale).unscaledValue().toByteArray();
+    HiveDecimal roundedDec = dec.setScale(scale, HiveDecimal.ROUND_HALF_UP);
+    Assert.assertEquals(roundedDec, HiveDecimal.create(new BigInteger(d), scale));
+  }
+
+//------------------------------------------------------------------------------------------------
+
+  @Test
+  public void testDecimalsWithOneOne() {
+    doTestDecimalsWithPrecisionScale(decimal_1_1_txt, 1, 1);
+  }
+
+  @Test
+  public void testDecimalsWithKv7Keys() {
+    doTestDecimalsWithPrecisionScale(kv7_txt_keys, 38, 18);
+  }
+
+  public void doTestDecimalsWithPrecisionScale(String[] decStrings, int precision, int scale) {
+
+    HiveDecimalV1 oldSum = HiveDecimalV1.create(0);
+    HiveDecimalWritable sum = new HiveDecimalWritable(0);
+
+    for (int i = 0; i < decStrings.length; i++) {
+
+      String string = decStrings[i];
+
+      HiveDecimalV1 oldDec = HiveDecimalV1.create(string);
+
+      HiveDecimalV1 resultOldDec;
+      if (oldDec == null) {
+        resultOldDec = null;
+      } else {
+        resultOldDec = HiveDecimalV1.enforcePrecisionScale(oldDec, precision, scale);
+      }
+
+      HiveDecimal dec = HiveDecimal.create(string);
+
+      if (oldDec == null) {
+        Assert.assertTrue(dec == null);
+        continue;
+      }
+      HiveDecimal resultDec = HiveDecimal.enforcePrecisionScale(dec, precision, scale);
+      if (resultOldDec == null) {
+        Assert.assertTrue(resultDec == null);
+        continue;
+      }
+
+      Assert.assertEquals(resultOldDec.toString(), resultDec.toString());
+      Assert.assertEquals(resultOldDec.toFormatString(scale), resultDec.toFormatString(scale));
+
+      oldSum = oldSum.add(resultOldDec);
+      sum.mutateAdd(resultDec);
+    }
+
+    Assert.assertEquals(oldSum.toString(), sum.toString());
+  }
+
+//------------------------------------------------------------------------------------------------
+
+  @Test
+  public void testDecimalsWithOneOneWritable() {
+    doTestDecimalsWithPrecisionScaleWritable(decimal_1_1_txt, 1, 1);
+  }
+
+  @Test
+  public void testDecimalsWithKv7KeysWritable() {
+    doTestDecimalsWithPrecisionScaleWritable(kv7_txt_keys, 38, 18);
+  }
+
+  public void doTestDecimalsWithPrecisionScaleWritable(String[] decStrings, int precision, int scale) {
+
+    HiveDecimalV1 oldSum = HiveDecimalV1.create(0);
+    HiveDecimalWritable sum = new HiveDecimalWritable(0);
+
+    for (int i = 0; i < decStrings.length; i++) {
+      String string = decStrings[i];
+
+      HiveDecimalV1 oldDec = HiveDecimalV1.create(string);
+      HiveDecimalV1 resultOldDec;
+      if (oldDec == null) {
+        resultOldDec = null;
+      } else {
+        resultOldDec = HiveDecimalV1.enforcePrecisionScale(oldDec, precision, scale);
+      }
+
+      HiveDecimalWritable decWritable = new HiveDecimalWritable(string);
+      if (oldDec == null) {
+        Assert.assertTrue(!decWritable.isSet());
+        continue;
+      }
+      decWritable.mutateEnforcePrecisionScale(precision, scale);;
+      if (resultOldDec == null) {
+        Assert.assertTrue(!decWritable.isSet());
+        continue;
+      }
+
+      Assert.assertEquals(resultOldDec.toString(), decWritable.toString());
+      Assert.assertEquals(resultOldDec.toFormatString(scale), decWritable.toFormatString(scale));
+
+      oldSum = oldSum.add(resultOldDec);
+      sum.mutateAdd(decWritable);
+    }
+
+    Assert.assertEquals(oldSum.toString(), sum.toString());
+  }
+
+  //------------------------------------------------------------------------------------------------
+
+  @Test
+  public void testSort() {
+    doTestSort(decimal_1_1_txt);
+  }
+
+  @Test
+  public void testSortSpecial() {
+    doTestSort(specialDecimalStrings);
+  }
+
+  @Test
+  public void testSortRandom() {
+    Random r = new Random(14434);
+    for (BigDecimalFlavor bigDecimalFlavor : BigDecimalFlavor.values()) {
+      doTestSortRandom(r, standardAlphabet, bigDecimalFlavor);
+    }
+    for (BigDecimalFlavor bigDecimalFlavor : BigDecimalFlavor.values()) {
+      for (String sparseAlphabet : sparseAlphabets) {
+        doTestSortRandom(r, sparseAlphabet, bigDecimalFlavor);
+      }
+    }
+  }
+
+  public void doTestSortRandom(Random r, String digitAlphabet, BigDecimalFlavor bigDecimalFlavor) {
+    String[] randomStrings = new String[POUND_FACTOR];
+
+    for (int i = 0; i < POUND_FACTOR; i++) {
+      BigDecimal bigDecimal = randHiveBigDecimal(r, digitAlphabet, bigDecimalFlavor);
+
+      randomStrings[i] = bigDecimal.toString();
+    }
+
+    doTestSort(randomStrings);
+  }
+
+  public void doTestSort(String[] decStrings) {
+
+    HiveDecimalV1[] oldDecSortArray = new HiveDecimalV1[decStrings.length];
+    HiveDecimal[] decSortArray = new HiveDecimal[decStrings.length];
+
+    int count = 0;
+    for (int i = 0; i < decStrings.length; i++) {
+      String string = decStrings[i];
+
+      HiveDecimalV1 oldDec = HiveDecimalV1.create(string);
+      if (oldDec == null) {
+        continue;
+      }
+      if (isTenPowerBug(oldDec.toString())) {
+        continue;
+      }
+      oldDecSortArray[count] = oldDec;
+
+      HiveDecimal dec = HiveDecimal.create(string);
+      if (dec == null) {
+        Assert.fail();
+      }
+      decSortArray[count] = dec;
+      count++;
+    }
+
+    oldDecSortArray = Arrays.copyOf(oldDecSortArray,  count);
+    decSortArray = Arrays.copyOf(decSortArray, count);
+
+    Arrays.sort(oldDecSortArray);
+    Arrays.sort(decSortArray);
+
+    for (int i = 0; i < count; i++) {
+      String oldDecString = oldDecSortArray[i].toString();
+      String decString = decSortArray[i].toString();
+
+      if (!oldDecString.equals(decString)) {
+        Assert.fail();
+      }
+    }
+  }
+
+  //------------------------------------------------------------------------------------------------
+
+  @Test
+  public void testRandomCreateFromBigDecimal() {
+    Random r = new Random(14434);
+    for (BigDecimalFlavor bigDecimalFlavor : BigDecimalFlavor.values()) {
+      doTestRandomCreateFromBigDecimal(r, standardAlphabet, bigDecimalFlavor);
+    }
+    for (BigDecimalFlavor bigDecimalFlavor : BigDecimalFlavor.values()) {
+      for (String sparseAlphabet : sparseAlphabets) {
+        doTestRandomCreateFromBigDecimal(r, sparseAlphabet, bigDecimalFlavor);
+      }
+    }
+  }
+
+  private void doTestRandomCreateFromBigDecimal(Random r, String digitAlphabet, BigDecimalFlavor bigDecimalFlavor) {
+
+    for (int i = 0; i < POUND_FACTOR; i++) {
+      BigDecimal bigDecimal = randHiveBigDecimal(r, digitAlphabet, bigDecimalFlavor);
+
+      doTestCreateFromBigDecimal(bigDecimal);
+    }
+  }
+
+  @Test
+  public void testCreateFromBigDecimalSpecial() {
+    for (BigDecimal bigDecimal : specialBigDecimals) {
+      doTestCreateFromBigDecimal(bigDecimal);
+    }
+  }
+
+  private void doTestCreateFromBigDecimal(BigDecimal bigDecimal) {
+
+    HiveDecimalV1 oldDec = HiveDecimalV1.create(bigDecimal);
+    if (oldDec != null && isTenPowerBug(oldDec.toString())) {
+      return;
+    }
+    HiveDecimal dec = HiveDecimal.create(bigDecimal);
+    if (oldDec == null) {
+      assertTrue(dec == null);
+      return;
+    }
+    assertTrue(dec != null);
+    dec.validate();
+
+    Assert.assertEquals(oldDec.toString(), dec.toString());
+  }
+
+  //------------------------------------------------------------------------------------------------
+
+  @Test
+  public void testRandomCreateFromBigDecimalNoRound() {
+    Random r = new Random(14434);
+    for (BigDecimalFlavor bigDecimalFlavor : BigDecimalFlavor.values()) {
+      doTestRandomCreateFromBigDecimalNoRound(r, standardAlphabet, bigDecimalFlavor);
+    }
+    for (BigDecimalFlavor bigDecimalFlavor : BigDecimalFlavor.values()) {
+      for (String sparseAlphabet : sparseAlphabets) {
+        doTestRandomCreateFromBigDecimalNoRound(r, sparseAlphabet, bigDecimalFlavor);
+      }
+    }
+  }
+
+  private void doTestRandomCreateFromBigDecimalNoRound(Random r, String digitAlphabet, BigDecimalFlavor bigDecimalFlavor) {
+
+    for (int i = 0; i < POUND_FACTOR; i++) {
+      BigDecimal bigDecimal = randHiveBigDecimal(r, digitAlphabet, bigDecimalFlavor);
+
+      doTestCreateFromBigDecimalNoRound(bigDecimal);
+    }
+  }
+
+  @Test
+  public void testCreateFromBigDecimalNoRoundSpecial() {
+    for (BigDecimal bigDecimal : specialBigDecimals) {
+      doTestCreateFromBigDecimalNoRound(bigDecimal);
+    }
+  }
+
+  private void doTestCreateFromBigDecimalNoRound(BigDecimal bigDecimal) {
+
+    HiveDecimalV1 oldDec = HiveDecimalV1.create(bigDecimal, /* allowRounding */ false);
+    HiveDecimal dec = HiveDecimal.create(bigDecimal, /* allowRounding */ false);
+    if (oldDec == null) {
+      assertTrue(dec == null);
+      return;
+    }
+    if (dec == null) {
+      Assert.fail();
+    }
+    dec.validate();
+
+    Assert.assertEquals(oldDec.toString(), dec.toString());
+
+  }
+
+  //------------------------------------------------------------------------------------------------
+
+  @Test
+  public void testCreateFromBigDecimalNegativeScaleSpecial() {
+    Random r = new Random(223965);
+    for (BigDecimal bigDecimal : specialBigDecimals) {
+      int negativeScale = -(0 + r.nextInt(38 + 1));
+      bigDecimal = bigDecimal.setScale(negativeScale, BigDecimal.ROUND_HALF_UP);
+      doTestCreateFromBigDecimalNegativeScale(bigDecimal);
+    }
+  }
+
+  private void doTestCreateFromBigDecimalNegativeScale(BigDecimal bigDecimal) {
+
+    HiveDecimalV1 oldDec = HiveDecimalV1.create(bigDecimal);
+    HiveDecimal dec = HiveDecimal.create(bigDecimal);
+    if (oldDec == null) {
+      assertTrue(dec == null);
+      return;
+    }
+    assertTrue(dec != null);
+    dec.validate();
+
+    Assert.assertEquals(oldDec.toString(), dec.toString());
+  }
+
+  //------------------------------------------------------------------------------------------------
+
+  @Test
+  public void testRandomCreateFromBigInteger() {
+    doTestRandomCreateFromBigInteger(standardAlphabet);
+  }
+
+  @Test
+  public void testRandomCreateFromBigIntegerSparse() {
+    for (String digitAlphabet : sparseAlphabets) {
+      doTestRandomCreateFromBigInteger(digitAlphabet);
+    }
+  }
+
+  private void doTestRandomCreateFromBigInteger(String digitAlphabet) {
+
+    Random r = new Random(11241);
+    for (int i = 0; i < POUND_FACTOR; i++) {
+      BigInteger bigInteger = randHiveBigInteger(r, digitAlphabet);
+
+      doTestCreateFromBigInteger(bigInteger);
+    }
+  }
+
+  @Test
+  public void testCreateFromBigIntegerSpecial() {
+    for (BigDecimal bigDecimal : specialBigDecimals) {
+      doTestCreateFromBigInteger(bigDecimal.unscaledValue());
+    }
+  }
+
+  private void doTestCreateFromBigInteger(BigInteger bigInteger) {
+
+    HiveDecimalV1 oldDec = HiveDecimalV1.create(bigInteger);
+    HiveDecimal dec = HiveDecimal.create(bigInteger);
+    if (oldDec == null) {
+      assertTrue(dec == null);
+      return;
+    }
+    assertTrue(dec != null);
+    dec.validate();
+
+    Assert.assertEquals(oldDec.toString(), dec.toString());
+
+  }
+
+  //------------------------------------------------------------------------------------------------
+
+  @Test
+  public void testRandomCreateFromBigIntegerScale() {
+    doTestRandomCreateFromBigIntegerScale(standardAlphabet, false);
+  }
+
+  @Test
+  public void testRandomCreateFromBigIntegerScaleFractionsOnly() {
+    doTestRandomCreateFromBigIntegerScale(standardAlphabet, true);
+  }
+
+  @Test
+  public void testRandomCreateFromBigIntegerScaleSparse() {
+    for (String digitAlphabet : sparseAlphabets) {
+      doTestRandomCreateFromBigIntegerScale(digitAlphabet, false);
+    }
+  }
+
+  private void doTestRandomCreateFromBigIntegerScale(String digitAlphabet, boolean fractionsOnly) {
+
+    Random r = new Random(4448);
+    for (int i = 0; i < POUND_FACTOR; i++) {
+      BigInteger bigInteger = randHiveBigInteger(r, digitAlphabet);
+
+      int scale;
+      if (fractionsOnly) {
+        scale = 1 + r.nextInt(38);
+      } else {
+        scale = 0 + r.nextInt(38 + 1);
+      }
+
+      doTestCreateFromBigIntegerScale(bigInteger, scale);
+    }
+  }
+
+  @Test
+  public void testCreateFromBigIntegerScaleSpecial() {
+    for (BigDecimal bigDecimal : specialBigDecimals) {
+      doTestCreateFromBigIntegerScale(bigDecimal.unscaledValue(), bigDecimal.scale());
+    }
+  }
+
+  private void doTestCreateFromBigIntegerScale(BigInteger bigInteger, int scale) {
+
+    HiveDecimalV1 oldDec = HiveDecimalV1.create(bigInteger, scale);
+    HiveDecimal dec = HiveDecimal.create(bigInteger, scale);
+    if (oldDec == null) {
+      assertTrue(dec == null);
+      return;
+    }
+    assertTrue(dec != null);
+    dec.validate();
+
+    Assert.assertEquals(oldDec.toString(), dec.toString());
+
+  }
+
+  //------------------------------------------------------------------------------------------------
+
+  @Test
+  public void testRandomSetFromDouble() {
+    Random r = new Random(14434);
+    for (BigDecimalFlavor bigDecimalFlavor : BigDecimalFlavor.values()) {
+      doTestRandomSetFromDouble(r, standardAlphabet, bigDecimalFlavor);
+    }
+    for (BigDecimalFlavor bigDecimalFlavor : BigDecimalFlavor.values()) {
+      for (String sparseAlphabet : sparseAlphabets) {
+        doTestRandomSetFromDouble(r, sparseAlphabet, bigDecimalFlavor);
+      }
+    }
+  }
+
+  private void doTestRandomSetFromDouble(Random r, String digitAlphabet, BigDecimalFlavor bigDecimalFlavor) {
+
+    for (int i = 0; i < POUND_FACTOR; i++) {
+      BigDecimal bigDecimal = randHiveBigDecimal(r, digitAlphabet, bigDecimalFlavor);
+
+      doTestSetFromDouble(bigDecimal.doubleValue());
+    }
+  }
+
+  private void doTestRandomSetFromDouble() {
+
+    Random r = new Random(94762);
+    for (int i = 0; i < POUND_FACTOR; i++) {
+      double randomDouble = r.nextDouble();
+ 
+      doTestSetFromDouble(randomDouble);
+    }
+  }
+
+  @Test
+  public void testSetFromDoubleSpecial() {
+
+    for (String specialString : specialDecimalStrings) {
+      double specialDouble = Double.valueOf(specialString);
+      doTestSetFromDouble(specialDouble);
+    }
+  }
+
+  private void doTestSetFromDouble(double doubleValue) {
+
+    HiveDecimalV1 oldDec = HiveDecimalV1.create(Double.toString(doubleValue));
+    if (oldDec == null) {
+      return;
+    }
+    HiveDecimal dec = HiveDecimal.create(doubleValue);
+    if (dec == null) {
+      Assert.fail();
+    }
+    dec.validate();
+    if (!oldDec.toString().equals(dec.toString())) {
+      BigDecimal bigDecimal = new BigDecimal(dec.toString());
+      for (int i = 16; i < 18;i++) {
+        BigDecimal trial = bigDecimal.setScale(i, HiveDecimal.ROUND_HALF_UP);
+      }
+    }
+  }
+
+  //------------------------------------------------------------------------------------------------
+
+  @Test
+  public void testRandomCreateFromString() {
+    Random r = new Random(1221);
+    for (BigDecimalFlavor bigDecimalFlavor : BigDecimalFlavor.values()) {
+      doTestRandomCreateFromString(r, standardAlphabet, bigDecimalFlavor);
+    }
+    for (BigDecimalFlavor bigDecimalFlavor : BigDecimalFlavor.values()) {
+      for (String sparseAlphabet : sparseAlphabets) {
+        doTestRandomCreateFromString(r, sparseAlphabet, bigDecimalFlavor);
+      }
+    }
+  }
+
+  private void doTestRandomCreateFromString(Random r, String digitAlphabet, BigDecimalFlavor bigDecimalFlavor) {
+
+    for (int i = 0; i < POUND_FACTOR; i++) {
+      BigDecimal bigDecimal = randHiveBigDecimal(r, digitAlphabet, bigDecimalFlavor);
+
+      doTestCreateFromString(bigDecimal);
+    }
+  }
+
+  @Test
+  public void testCreateFromStringSpecial() {
+    for (BigDecimal bigDecimal : specialBigDecimals) {
+      doTestCreateFromString(bigDecimal);
+    }
+  }
+
+  private void doTestCreateFromString(BigDecimal bigDecimal) {
+
+    String decString = bigDecimal.toPlainString();
+
+    HiveDecimalV1 oldDec = HiveDecimalV1.create(decString);
+    if (oldDec != null && isTenPowerBug(oldDec.toString())) {
+      return;
+    }
+    HiveDecimal dec = HiveDecimal.create(decString);
+    if (oldDec == null) {
+      assertTrue(dec == null);
+      return;
+    }
+    assertTrue(dec != null);
+    dec.validate();
+
+    Assert.assertEquals(oldDec.toString(), dec.toString());
+  }
+
+  //------------------------------------------------------------------------------------------------
+
+  @Test
+  public void testRandomCreateFromStringPadded() {
+    Random r = new Random(9774);
+    for (BigDecimalFlavor bigDecimalFlavor : BigDecimalFlavor.values()) {
+      doTestRandomCreateFromStringPadded(r, standardAlphabet, bigDecimalFlavor);
+    }
+    for (BigDecimalFlavor bigDecimalFlavor : BigDecimalFlavor.values()) {
+      for (String sparseAlphabet : sparseAlphabets) {
+        doTestRandomCreateFromStringPadded(r, sparseAlphabet, bigDecimalFlavor);
+      }
+    }
+  }
+
+  private void doTestRandomCreateFromStringPadded(Random r, String digitAlphabet, BigDecimalFlavor bigDecimalFlavor) {
+
+    for (int i = 0; i < POUND_FACTOR; i++) {
+      BigDecimal bigDecimal = randHiveBigDecimal(r, digitAlphabet, bigDecimalFlavor);
+
+      doTestCreateFromStringPadded(bigDecimal);
+    }
+  }
+
+  @Test
+  public void testCreateFromStringPaddedSpecial() {
+    for (BigDecimal bigDecimal : specialBigDecimals) {
+      doTestCreateFromStringPadded(bigDecimal);
+    }
+  }
+
+  private void doTestCreateFromStringPadded(BigDecimal bigDecimal) {
+
+    String decString = bigDecimal.toPlainString();
+    String decString1 = " " + decString;
+    String decString2 = decString + " ";
+    String decString3 = " " + decString + " ";
+    String decString4 = "  " + decString;
+    String decString5 = decString + "  ";
+    String decString6 = "  " + decString + "  ";
+
+    HiveDecimalV1 oldDec;
+    HiveDecimal dec;
+
+    oldDec = HiveDecimalV1.create(decString);
+    if (oldDec != null && isTenPowerBug(oldDec.toString())) {
+      return;
+    }
+
+    dec = HiveDecimal.create(decString1, true);
+    if (oldDec == null) {
+      assertTrue(dec == null);
+    } else {
+      assertTrue(dec != null);
+      dec.validate();
+
+      Assert.assertEquals(oldDec.toString(), dec.toString());
+    }
+
+    dec = HiveDecimal.create(decString2, true);
+    if (oldDec == null) {
+      assertTrue(dec == null);
+    } else {
+      assertTrue(dec != null);
+      dec.validate();
+
+      Assert.assertEquals(oldDec.toString(), dec.toString());
+    }
+
+    dec = HiveDecimal.create(decString3, true);
+    if (oldDec == null) {
+      assertTrue(dec == null);
+    } else {
+      assertTrue(dec != null);
+      dec.validate();
+
+      Assert.assertEquals(oldDec.toString(), dec.toString());
+    }
+
+    dec = HiveDecimal.create(decString4, true);
+    if (oldDec == null) {
+      assertTrue(dec == null);
+    } else {
+      assertTrue(dec != null);
+      dec.validate();
+
+      Assert.assertEquals(oldDec.toString(), dec.toString());
+    }
+
+    dec = HiveDecimal.create(decString5, true);
+    if (oldDec == null) {
+      assertTrue(dec == null);
+    } else {
+      assertTrue(dec != null);
+      dec.validate();
+
+      Assert.assertEquals(oldDec.toString(), dec.toString());
+    }
+
+    dec = HiveDecimal.create(decString6, true);
+    if (oldDec == null) {
+      assertTrue(dec == null);
+    } else {
+      assertTrue(dec != null);
+      dec.validate();
+
+      Assert.assertEquals(oldDec.toString(), dec.toString());
+    }
+  }
+
+  //------------------------------------------------------------------------------------------------
+
+  @Test
+  public void testRandomCreateFromStringExponent() {
+    Random r = new Random(297111);
+    for (BigDecimalFlavor bigDecimalFlavor : BigDecimalFlavor.values()) {
+      doTestRandomCreateFromStringPadded(r, standardAlphabet, bigDecimalFlavor);
+    }
+    for (BigDecimalFlavor bigDecimalFlavor : BigDecimalFlavor.values()) {
+      for (String sparseAlphabet : sparseAlphabets) {
+        doTestRandomCreateFromStringPadded(r, sparseAlphabet, bigDecimalFlavor);
+      }
+    }
+  }
+
+  private void doTestRandomCreateFromStringExponent(Random r, String digitAlphabet, BigDecimalFlavor bigDecimalFlavor) {
+
+    for (int i = 0; i < POUND_FACTOR; i++) {
+      BigDecimal bigDecimal = randHiveBigDecimal(r, digitAlphabet, bigDecimalFlavor);
+
+      doTestCreateFromStringExponent(bigDecimal);
+    }
+  }
+
+  @Test
+  public void testCreateFromStringExponentSpecial() {
+    for (BigDecimal bigDecimal : specialBigDecimals) {
+      doTestCreateFromStringExponent(bigDecimal);
+    }
+  }
+
+  private void doTestCreateFromStringExponent(BigDecimal bigDecimal) {
+
+    // Use toString which will have exponents instead of toPlainString.
+    String decString = bigDecimal.toString();
+
+    HiveDecimalV1 oldDec = HiveDecimalV1.create(decString);
+    HiveDecimal dec = HiveDecimal.create(decString);
+    if (oldDec == null) {
+      assertTrue(dec == null);
+      return;
+    }
+    assertTrue(dec != null);
+    dec.validate();
+
+    Assert.assertEquals(oldDec.toString(), dec.toString());
+  }
+
+  //------------------------------------------------------------------------------------------------
+
+  @Test
+  public void testRandomLongValue() {
+    Random r = new Random(73293);
+    for (BigDecimalFlavor bigDecimalFlavor : BigDecimalFlavor.values()) {
+      doTestRandomLongValue(r, standardAlphabet, bigDecimalFlavor);
+    }
+    for (BigDecimalFlavor bigDecimalFlavor : BigDecimalFlavor.values()) {
+      for (String sparseAlphabet : sparseAlphabets) {
+        doTestRandomLongValue(r, sparseAlphabet, bigDecimalFlavor);
+      }
+    }
+  }
+
+  private void doTestRandomLongValue(Random r, String digitAlphabet, BigDecimalFlavor bigDecimalFlavor) {
+
+    for (int i = 0; i < POUND_FACTOR; i++) {
+      BigDecimal bigDecimal = randHiveBigDecimal(r, digitAlphabet, bigDecimalFlavor);
+
+      doTestLongValue(bigDecimal);
+    }
+  }
+
+  @Test
+  public void testLongValueSpecial() {
+    for (BigDecimal bigDecimal : specialBigDecimals) {
+      doTestLongValue(bigDecimal);
+    }
+  }
+
+  private void doTestLongValue(BigDecimal bigDecimal) {
+
+    HiveDecimalV1 oldDec = HiveDecimalV1.create(bigDecimal);
+    if (oldDec != null && isTenPowerBug(oldDec.toString())) {
+      return;
+    }
+    HiveDecimal dec = HiveDecimal.create(bigDecimal);
+    if (oldDec == null) {
+      assertTrue(dec == null);
+      return;
+    }
+    assertTrue(dec != null);
+    dec.validate();
+
+    BigDecimal bigDecimalOldDec = oldDec.bigDecimalValue();
+    BigDecimal bigDecimalDec = dec.bigDecimalValue();
+    Assert.assertEquals(bigDecimalOldDec, bigDecimalDec);
+
+    BigDecimal bigDecimalFloor = bigDecimalDec.setScale(0, BigDecimal.ROUND_DOWN);
+    long longValueBigDecimalFloor = bigDecimalFloor.longValue();
+    boolean isLongExpected =
+        bigDecimalFloor.equals(bigDecimalDec.valueOf(longValueBigDecimalFloor));
+
+    boolean decIsLong = dec.isLong();
+    long oldDecLong = oldDec.longValue();
+    long decLong = dec.longValue();
+    if (isLongExpected != decIsLong) {
+      Assert.fail();
+    }
+
+    if (decIsLong) {
+      if (oldDecLong != decLong) {
+        Assert.fail();
+      }
+    }
+  }
+
+  //------------------------------------------------------------------------------------------------
+
+  @Test
+  public void testRandomIntValue() {
+    Random r = new Random(98333);
+    for (BigDecimalFlavor bigDecimalFlavor : BigDecimalFlavor.values()) {
+      doTestRandomIntValue(r, standardAlphabet, bigDecimalFlavor);
+    }
+    for (BigDecimalFlavor bigDecimalFlavor : BigDecimalFlavor.values()) {
+      for (String sparseAlphabet : sparseAlphabets) {
+        doTestRandomIntValue(r, sparseAlphabet, bigDecimalFlavor);
+      }
+    }
+  }
+
+  private void doTestRandomIntValue(Random r, String digitAlphabet, BigDecimalFlavor bigDecimalFlavor) {
+
+    for (int i = 0; i < POUND_FACTOR; i++) {
+      BigDecimal bigDecimal = randHiveBigDecimal(r, digitAlphabet, bigDecimalFlavor);
+
+      doTestIntValue(bigDecimal);
+    }
+  }
+
+  @Test
+  public void testIntValueSpecial() {
+    for (BigDecimal bigDecimal : specialBigDecimals) {
+      doTestIntValue(bigDecimal);
+    }
+  }
+
+  private void doTestIntValue(BigDecimal bigDecimal) {
+
+    HiveDecimalV1 oldDec = HiveDecimalV1.create(bigDecimal);
+    if (oldDec != null && isTenPowerBug(oldDec.toString())) {
+      return;
+    }
+    HiveDecimal dec = HiveDecimal.create(bigDecimal);
+    if (oldDec == null) {
+      assertTrue(dec == null);
+      return;
+    }
+    assertTrue(dec != null);
+    dec.validate();
+
+    BigDecimal bigDecimalOldDec = oldDec.bigDecimalValue();
+    BigDecimal bigDecimalDec = dec.bigDecimalValue();
+    Assert.assertEquals(bigDecimalOldDec, bigDecimalDec);
+
+    BigDecimal bigDecimalFloor = bigDecimalDec.setScale(0, BigDecimal.ROUND_DOWN);
+    int intValueBigDecimalFloor = bigDecimalFloor.intValue();
+    boolean isIntExpected =
+        bigDecimalFloor.equals(bigDecimalDec.valueOf(intValueBigDecimalFloor));
+
+    boolean decIsInt = dec.isInt();
+    int oldDecInt = oldDec.intValue();
+    int decInt = dec.intValue();
+    if (isIntExpected != decIsInt) {
+      Assert.fail();
+    }
+
+    if (decIsInt) {
+      if (oldDecInt != decInt) {
+        Assert.fail();
+      }
+    }
+  }
+
+  //------------------------------------------------------------------------------------------------
+
+  @Test
+  public void testRandomShortValue() {
+    Random r = new Random(15);
+    for (BigDecimalFlavor bigDecimalFlavor : BigDecimalFlavor.values()) {
+      doTestRandomShortValue(r, standardAlphabet, bigDecimalFlavor);
+    }
+    for (BigDecimalFlavor bigDecimalFlavor : BigDecimalFlavor.values()) {
+      for (String sparseAlphabet : sparseAlphabets) {
+        doTestRandomShortValue(r, sparseAlphabet, bigDecimalFlavor);
+      }
+    }
+  }
+
+  private void doTestRandomShortValue(Random r, String digitAlphabet, BigDecimalFlavor bigDecimalFlavor) {
+    for (int i = 0; i < POUND_FACTOR; i++) {
+      BigDecimal bigDecimal = randHiveBigDecimal(r, digitAlphabet, bigDecimalFlavor);
+
+      doTestShortValue(bigDecimal);
+    }
+  }
+
+  @Test
+  public void testShortValueSpecial() {
+    for (BigDecimal bigDecimal : specialBigDecimals) {
+      doTestShortValue(bigDecimal);
+    }
+  }
+
+  private void doTestShortValue(BigDecimal bigDecimal) {
+
+    HiveDecimalV1 oldDec = HiveDecimalV1.create(bigDecimal);
+    if (oldDec != null && isTenPowerBug(oldDec.toString())) {
+      return;
+    }
+    HiveDecimal dec = HiveDecimal.create(bigDecimal);
+    if (oldDec == null) {
+      assertTrue(dec == null);
+      return;
+    }
+    assertTrue(dec != null);
+    dec.validate();
+
+    BigDecimal bigDecimalOldDec = oldDec.bigDecimalValue();
+    BigDecimal bigDecimalDec = dec.bigDecimalValue();
+    Assert.assertEquals(bigDecimalOldDec, bigDecimalDec);
+
+    BigDecimal bigDecimalFloor = bigDecimalDec.setScale(0, BigDecimal.ROUND_DOWN);
+    short shortValueBigDecimalFloor = bigDecimalFloor.shortValue();
+    boolean isShortExpected =
+        bigDecimalFloor.equals(bigDecimalDec.valueOf(shortValueBigDecimalFloor));
+
+    boolean decIsShort = dec.isShort();
+    short oldDecShort = oldDec.shortValue();
+    short decShort = dec.shortValue();
+    if (isShortExpected != decIsShort) {
+      Assert.fail();
+    }
+
+    if (decIsShort) {
+      if (oldDecShort != decShort) {
+        Assert.fail();
+      }
+    }
+  }
+
+  //------------------------------------------------------------------------------------------------
+
+  @Test
+  public void testRandomByteValue() {
+    Random r = new Random(9292);
+    for (BigDecimalFlavor bigDecimalFlavor : BigDecimalFlavor.values()) {
+      doTestRandomByteValue(r, standardAlphabet, bigDecimalFlavor);
+    }
+    for (BigDecimalFlavor bigDecimalFlavor : BigDecimalFlavor.values()) {
+      for (String sparseAlphabet : sparseAlphabets) {
+        doTestRandomByteValue(r, sparseAlphabet, bigDecimalFlavor);
+      }
+    }
+  }
+
+  private void doTestRandomByteValue(Random r, String digitAlphabet, BigDecimalFlavor bigDecimalFlavor) {
+
+    for (int i = 0; i < POUND_FACTOR; i++) {
+      BigDecimal bigDecimal = randHiveBigDecimal(r, digitAlphabet, bigDecimalFlavor);
+
+      doTestByteValue(bigDecimal);
+    }
+  }
+
+  @Test
+  public void testByteValueSpecial() {
+    for (BigDecimal bigDecimal : specialBigDecimals) {
+      doTestByteValue(bigDecimal);
+    }
+  }
+
+  private void doTestByteValue(BigDecimal bigDecimal) {
+
+    HiveDecimalV1 oldDec = HiveDecimalV1.create(bigDecimal);
+    if (oldDec != null && isTenPowerBug(oldDec.toString())) {
+      return;
+    }
+    HiveDecimal dec = HiveDecimal.create(bigDecimal);
+    if (oldDec == null) {
+      assertTrue(dec == null);
+      return;
+    }
+    assertTrue(dec != null);
+    dec.validate();
+
+    BigDecimal bigDecimalOldDec = oldDec.bigDecimalValue();
+    BigDecimal bigDecimalDec = dec.bigDecimalValue();
+    Assert.assertEquals(bigDecimalOldDec, bigDecimalDec);
+
+    BigDecimal bigDecimalFloor = bigDecimalDec.setScale(0, BigDecimal.ROUND_DOWN);
+    byte byteValueBigDecimalFloor = bigDecimalFloor.byteValue();
+    boolean isByteExpected =
+        bigDecimalFloor.equals(bigDecimalDec.valueOf(byteValueBigDecimalFloor));
+
+    boolean decIsByte = dec.isByte();
+    byte oldDecByte = oldDec.byteValue();
+    byte decByte = dec.byteValue();
+    if (isByteExpected != decIsByte) {
+      Assert.fail();
+    }
+
+    if (decIsByte) {
+      if (oldDecByte != decByte) {
+        Assert.fail();
+      }
+    }
+  }
+
+  //------------------------------------------------------------------------------------------------
+
+  @Test
+  public void testRandomTimestamp() {
+    Random r = new Random(5476);
+    for (BigDecimalFlavor bigDecimalFlavor : BigDecimalFlavor.values()) {
+      doTestRandomTimestamp(r, standardAlphabet, bigDecimalFlavor);
+    }
+    for (BigDecimalFlavor bigDecimalFlavor : BigDecimalFlavor.values()) {
+      for (String sparseAlphabet : sparseAlphabets) {
+        doTestRandomTimestamp(r, sparseAlphabet, bigDecimalFlavor);
+      }
+    }
+  }
+
+  private void doTestRandomTimestamp(Random r, String digitAlphabet, BigDecimalFlavor bigDecimalFlavor) {
+    for (int i = 0; i < POUND_FACTOR; i++) {
+      BigDecimal bigDecimal = randHiveBigDecimal(r, digitAlphabet, bigDecimalFlavor);
+
+      doTestTimestamp(bigDecimal);
+    }
+  }
+
+  @Test
+  public void testTimestampSpecial() {
+    for (BigDecimal bigDecimal : specialBigDecimals) {
+      doTestTimestamp(bigDecimal);
+    }
+  }
+
+  private void doTestTimestamp(BigDecimal bigDecimal) {
+
+    HiveDecimalV1 oldDec = HiveDecimalV1.create(bigDecimal);
+    if (oldDec != null && isTenPowerBug(oldDec.toString())) {
+      return;
+    }
+    HiveDecimal dec = HiveDecimal.create(bigDecimal);
+    if (oldDec == null) {
+      assertTrue(dec == null);
+      return;
+    }
+    assertTrue(dec != null);
+   

<TRUNCATED>

[08/10] hive git commit: HIVE-15335: Fast Decimal (Matt McCline, reviewed by Sergey Shelukhin, Prasanth Jayachandran, Owen O'Malley)

Posted by mm...@apache.org.
http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/ql/src/java/org/apache/hadoop/hive/ql/io/sarg/ConvertAstToSearchArg.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/sarg/ConvertAstToSearchArg.java b/ql/src/java/org/apache/hadoop/hive/ql/io/sarg/ConvertAstToSearchArg.java
index 9013084..9d900e4 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/sarg/ConvertAstToSearchArg.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/sarg/ConvertAstToSearchArg.java
@@ -141,7 +141,11 @@ public class ConvertAstToSearchArg {
     switch (type) {
       case LONG:
         if (lit instanceof HiveDecimal) {
-          return ((HiveDecimal)lit).longValueExact();
+          HiveDecimal dec = (HiveDecimal) lit;
+          if (!dec.isLong()) {
+            throw new ArithmeticException("Overflow");
+          }
+          return dec.longValue();
         }
         return ((Number) lit).longValue();
       case STRING:

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFLog.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFLog.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFLog.java
index 31e2878..d214a96 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFLog.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFLog.java
@@ -59,8 +59,8 @@ public class UDFLog extends UDFMath {
       return null;
     }
 
-    double base = baseWritable.getHiveDecimal().bigDecimalValue().doubleValue();
-    double d = writable.getHiveDecimal().bigDecimalValue().doubleValue();
+    double base = baseWritable.doubleValue();
+    double d = writable.doubleValue();
     return log(base, d);
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFMath.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFMath.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFMath.java
index 8087df1..378b3d7 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFMath.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFMath.java
@@ -54,7 +54,7 @@ public abstract class UDFMath extends UDF {
       return null;
     }
 
-    double d = writable.getHiveDecimal().bigDecimalValue().doubleValue();
+    double d = writable.doubleValue();
     doubleWritable.set(d);
     return doEvaluate(doubleWritable);
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFSign.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFSign.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFSign.java
index 67d62d9..449848a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFSign.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFSign.java
@@ -79,12 +79,12 @@ public class UDFSign extends UDF {
    *
    * @return -1, 0, or 1 representing the sign of the input decimal
    */
-  public IntWritable evaluate(HiveDecimalWritable dec) {
-    if (dec == null || dec.getHiveDecimal() == null) {
+  public IntWritable evaluate(HiveDecimalWritable decWritable) {
+    if (decWritable == null || !decWritable.isSet()) {
       return null;
     }
 
-    intWritable.set(dec.getHiveDecimal().signum());
+    intWritable.set(decWritable.signum());
     return intWritable;
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToBoolean.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToBoolean.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToBoolean.java
index 17b892c..0cc0c9e 100755
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToBoolean.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToBoolean.java
@@ -191,10 +191,10 @@ public class UDFToBoolean extends UDF {
   }
 
   public BooleanWritable evaluate(HiveDecimalWritable i) {
-    if (i == null) {
+    if (i == null || !i.isSet()) {
       return null;
     } else {
-      booleanWritable.set(HiveDecimal.ZERO.compareTo(i.getHiveDecimal()) != 0);
+      booleanWritable.set(i.compareTo(HiveDecimal.ZERO) != 0);
       return booleanWritable;
     }
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToByte.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToByte.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToByte.java
index efae82d..df2b42f 100755
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToByte.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToByte.java
@@ -192,10 +192,10 @@ public class UDFToByte extends UDF {
   }
 
   public ByteWritable evaluate(HiveDecimalWritable i) {
-    if (i == null) {
+    if (i == null || !i.isSet() || !i.isByte()) {
       return null;
     } else {
-      byteWritable.set(i.getHiveDecimal().byteValue());
+      byteWritable.set(i.byteValue());
       return byteWritable;
     }
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToDouble.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToDouble.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToDouble.java
index 9cbc114..5fcae42 100755
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToDouble.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToDouble.java
@@ -195,10 +195,10 @@ public class UDFToDouble extends UDF {
   }
 
   public DoubleWritable evaluate(HiveDecimalWritable i) {
-    if (i == null) {
+    if (i == null || !i.isSet()) {
       return null;
     } else {
-      doubleWritable.set(i.getHiveDecimal().doubleValue());
+      doubleWritable.set(i.doubleValue());
       return doubleWritable;
     }
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToFloat.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToFloat.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToFloat.java
index 5808c90..c8e32f4 100755
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToFloat.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToFloat.java
@@ -196,10 +196,10 @@ public class UDFToFloat extends UDF {
   }
 
   public FloatWritable evaluate(HiveDecimalWritable i) {
-    if (i == null) {
+    if (i == null || !i.isSet()) {
       return null;
     } else {
-      floatWritable.set(i.getHiveDecimal().floatValue());
+      floatWritable.set(i.floatValue());
       return floatWritable;
     }
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToInteger.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToInteger.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToInteger.java
index a7551cb..42972c7 100755
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToInteger.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToInteger.java
@@ -201,10 +201,10 @@ public class UDFToInteger extends UDF {
   }
 
   public IntWritable evaluate(HiveDecimalWritable i) {
-    if (i == null) {
+    if (i == null || !i.isSet() || !i.isInt()) {
       return null;
     } else {
-      intWritable.set(i.getHiveDecimal().intValue());  // TODO: lossy conversion!
+      intWritable.set(i.intValue());
       return intWritable;
     }
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToLong.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToLong.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToLong.java
index c961d14..847b535 100755
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToLong.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToLong.java
@@ -204,10 +204,10 @@ public class UDFToLong extends UDF {
   }
 
   public LongWritable evaluate(HiveDecimalWritable i) {
-    if (i == null) {
+    if (i == null || !i.isSet() || !i.isLong()) {
       return null;
     } else {
-      longWritable.set(i.getHiveDecimal().longValue()); // TODO: lossy conversion!
+      longWritable.set(i.longValue());
       return longWritable;
     }
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToShort.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToShort.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToShort.java
index 570408a..0ae0c13 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToShort.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/UDFToShort.java
@@ -194,10 +194,10 @@ public class UDFToShort extends UDF {
   }
 
   public ShortWritable evaluate(HiveDecimalWritable i) {
-    if (i == null) {
+    if (i == null || !i.isShort() || !i.isShort()) {
       return null;
     } else {
-      shortWritable.set(i.getHiveDecimal().shortValue());   // TODO: lossy conversion!
+      shortWritable.set(i.shortValue());
       return shortWritable;
     }
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFSum.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFSum.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFSum.java
index 38269f4..e2cd213 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFSum.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFSum.java
@@ -206,7 +206,7 @@ public class GenericUDAFSum extends AbstractGenericUDAFResolver {
     public ObjectInspector init(Mode m, ObjectInspector[] parameters) throws HiveException {
       assert (parameters.length == 1);
       super.init(m, parameters);
-      result = new HiveDecimalWritable(HiveDecimal.ZERO);
+      result = new HiveDecimalWritable(0);
       inputOI = (PrimitiveObjectInspector) parameters[0];
       // The output precision is 10 greater than the input which should cover at least
       // 10b rows. The scale is the same as the input.
@@ -226,21 +226,21 @@ public class GenericUDAFSum extends AbstractGenericUDAFResolver {
 
     /** class for storing decimal sum value. */
     @AggregationType(estimable = false) // hard to know exactly for decimals
-    static class SumHiveDecimalAgg extends SumAgg<HiveDecimal> {
+    static class SumHiveDecimalWritableAgg extends SumAgg<HiveDecimalWritable> {
     }
 
     @Override
     public AggregationBuffer getNewAggregationBuffer() throws HiveException {
-      SumHiveDecimalAgg agg = new SumHiveDecimalAgg();
+      SumHiveDecimalWritableAgg agg = new SumHiveDecimalWritableAgg();
       reset(agg);
       return agg;
     }
 
     @Override
     public void reset(AggregationBuffer agg) throws HiveException {
-      SumAgg<HiveDecimal> bdAgg = (SumAgg<HiveDecimal>) agg;
+      SumAgg<HiveDecimalWritable> bdAgg = (SumAgg<HiveDecimalWritable>) agg;
       bdAgg.empty = true;
-      bdAgg.sum = HiveDecimal.ZERO;
+      bdAgg.sum = new HiveDecimalWritable(0);
       bdAgg.uniqueObjects = new HashSet<ObjectInspectorObject>();
     }
 
@@ -250,9 +250,9 @@ public class GenericUDAFSum extends AbstractGenericUDAFResolver {
     public void iterate(AggregationBuffer agg, Object[] parameters) throws HiveException {
       assert (parameters.length == 1);
       try {
-        if (isEligibleValue((SumHiveDecimalAgg) agg, parameters[0])) {
-          ((SumHiveDecimalAgg)agg).empty = false;
-          ((SumHiveDecimalAgg)agg).sum = ((SumHiveDecimalAgg)agg).sum.add(
+        if (isEligibleValue((SumHiveDecimalWritableAgg) agg, parameters[0])) {
+          ((SumHiveDecimalWritableAgg)agg).empty = false;
+          ((SumHiveDecimalWritableAgg)agg).sum.mutateAdd(
               PrimitiveObjectInspectorUtils.getHiveDecimal(parameters[0], inputOI));
         }
       } catch (NumberFormatException e) {
@@ -270,8 +270,8 @@ public class GenericUDAFSum extends AbstractGenericUDAFResolver {
     @Override
     public void merge(AggregationBuffer agg, Object partial) throws HiveException {
       if (partial != null) {
-        SumHiveDecimalAgg myagg = (SumHiveDecimalAgg) agg;
-        if (myagg.sum == null) {
+        SumHiveDecimalWritableAgg myagg = (SumHiveDecimalWritableAgg) agg;
+        if (myagg.sum == null || !myagg.sum.isSet()) {
           return;
         }
 
@@ -279,22 +279,22 @@ public class GenericUDAFSum extends AbstractGenericUDAFResolver {
         if (isWindowingDistinct()) {
           throw new HiveException("Distinct windowing UDAF doesn't support merge and terminatePartial");
         } else {
-          myagg.sum = myagg.sum.add(PrimitiveObjectInspectorUtils.getHiveDecimal(partial, inputOI));
+          myagg.sum.mutateAdd(PrimitiveObjectInspectorUtils.getHiveDecimal(partial, inputOI));
         }
       }
     }
 
     @Override
     public Object terminate(AggregationBuffer agg) throws HiveException {
-      SumHiveDecimalAgg myagg = (SumHiveDecimalAgg) agg;
-      if (myagg.empty || myagg.sum == null) {
+      SumHiveDecimalWritableAgg myagg = (SumHiveDecimalWritableAgg) agg;
+      if (myagg.empty || myagg.sum == null || !myagg.sum.isSet()) {
         return null;
       }
-      if (myagg.sum != null) {
-        if (HiveDecimalUtils.enforcePrecisionScale(myagg.sum, (DecimalTypeInfo)outputOI.getTypeInfo()) == null) {
-          LOG.warn("The sum of a column with data type HiveDecimal is out of range");
-          return null;
-        }
+      DecimalTypeInfo decimalTypeInfo = (DecimalTypeInfo)outputOI.getTypeInfo();
+      myagg.sum.mutateEnforcePrecisionScale(decimalTypeInfo.getPrecision(), decimalTypeInfo.getScale());
+      if (!myagg.sum.isSet()) {
+        LOG.warn("The sum of a column with data type HiveDecimal is out of range");
+        return null;
       }
 
       result.set(myagg.sum);
@@ -315,8 +315,8 @@ public class GenericUDAFSum extends AbstractGenericUDAFResolver {
         protected HiveDecimalWritable getNextResult(
             org.apache.hadoop.hive.ql.udf.generic.GenericUDAFStreamingEvaluator.SumAvgEnhancer<HiveDecimalWritable, HiveDecimal>.SumAvgStreamingState ss)
             throws HiveException {
-          SumHiveDecimalAgg myagg = (SumHiveDecimalAgg) ss.wrappedBuf;
-          HiveDecimal r = myagg.empty ? null : myagg.sum;
+          SumHiveDecimalWritableAgg myagg = (SumHiveDecimalWritableAgg) ss.wrappedBuf;
+          HiveDecimal r = myagg.empty ? null : myagg.sum.getHiveDecimal();
           HiveDecimal d = ss.retrieveNextIntermediateValue();
           if (d != null ) {
             r = r == null ? null : r.subtract(d);
@@ -329,8 +329,8 @@ public class GenericUDAFSum extends AbstractGenericUDAFResolver {
         protected HiveDecimal getCurrentIntermediateResult(
             org.apache.hadoop.hive.ql.udf.generic.GenericUDAFStreamingEvaluator.SumAvgEnhancer<HiveDecimalWritable, HiveDecimal>.SumAvgStreamingState ss)
             throws HiveException {
-          SumHiveDecimalAgg myagg = (SumHiveDecimalAgg) ss.wrappedBuf;
-          return myagg.empty ? null : myagg.sum;
+          SumHiveDecimalWritableAgg myagg = (SumHiveDecimalWritableAgg) ss.wrappedBuf;
+          return myagg.empty ? null : myagg.sum.getHiveDecimal();
         }
 
       };

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFAbs.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFAbs.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFAbs.java
index a8e2786..bf1fed0 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFAbs.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFAbs.java
@@ -141,7 +141,8 @@ public class GenericUDFAbs extends GenericUDF {
       HiveDecimalWritable val = decimalOI.getPrimitiveWritableObject(valObject);
 
       if (val != null) {
-        resultDecimal.set(val.getHiveDecimal().abs());
+        resultDecimal.set(val);
+        resultDecimal.mutateAbs();
         val = resultDecimal;
       }
       return val;

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFBRound.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFBRound.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFBRound.java
index 4a59eb3..cb901cf 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFBRound.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFBRound.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.hive.ql.exec.vector.expressions.FuncBRoundWithNumDigits
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.FuncBRoundDecimalToDecimal;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.gen.FuncBRoundDoubleToDouble;
 import org.apache.hadoop.hive.serde2.io.DoubleWritable;
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 
 @Description(name = "bround",
 value = "_FUNC_(x[, d]) - round x to d decimal places using HALF_EVEN rounding mode.",
@@ -37,8 +38,10 @@ extended = "Banker's rounding. The value is rounded to the nearest even number.
 public class GenericUDFBRound extends GenericUDFRound {
 
   @Override
-  protected HiveDecimal round(HiveDecimal input, int scale) {
-    return RoundUtils.bround(input, scale);
+  protected HiveDecimalWritable round(HiveDecimalWritable inputDecWritable, int scale) {
+    HiveDecimalWritable result = new HiveDecimalWritable(inputDecWritable);
+    result.mutateSetScale(scale, HiveDecimal.ROUND_HALF_EVEN);
+    return result;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFCeil.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFCeil.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFCeil.java
index 95ec32e..bc0bf5e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFCeil.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFCeil.java
@@ -50,8 +50,8 @@ public final class GenericUDFCeil extends GenericUDFFloorCeilBase {
 
   @Override
   protected HiveDecimalWritable evaluate(HiveDecimalWritable input) {
-    HiveDecimal bd = input.getHiveDecimal();
-    decimalWritable.set(bd.setScale(0, HiveDecimal.ROUND_CEILING));
+    decimalWritable.set(input);
+    decimalWritable.mutateSetScale(0, HiveDecimal.ROUND_CEILING);
     return decimalWritable;
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFFloor.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFFloor.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFFloor.java
index 8ad15e9..a8cb9cc 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFFloor.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFFloor.java
@@ -50,8 +50,8 @@ public final class GenericUDFFloor extends GenericUDFFloorCeilBase {
 
   @Override
   protected HiveDecimalWritable evaluate(HiveDecimalWritable input) {
-    HiveDecimal bd = input.getHiveDecimal();
-    decimalWritable.set(bd.setScale(0, HiveDecimal.ROUND_FLOOR));
+    decimalWritable.set(input);
+    decimalWritable.mutateSetScale(0, HiveDecimal.ROUND_FLOOR);
     return decimalWritable;
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPNegative.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPNegative.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPNegative.java
index de964d6..45a3504 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPNegative.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPNegative.java
@@ -82,8 +82,8 @@ public class GenericUDFOPNegative extends GenericUDFBaseUnary {
       doubleWritable.set(-(((DoubleWritable)input).get()));
       return doubleWritable;
     case DECIMAL:
-      HiveDecimal dec = ((HiveDecimalWritable)input).getHiveDecimal();
-      decimalWritable.set(dec.negate());
+      decimalWritable.set((HiveDecimalWritable)input);
+      decimalWritable.mutateNegate();
       return decimalWritable;
     case INTERVAL_YEAR_MONTH:
       HiveIntervalYearMonth intervalYearMonth =

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPNumericMinus.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPNumericMinus.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPNumericMinus.java
index 28f7907..268a722 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPNumericMinus.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPNumericMinus.java
@@ -78,11 +78,8 @@ public class GenericUDFOPNumericMinus extends GenericUDFBaseNumeric {
 
   @Override
   protected HiveDecimalWritable evaluate(HiveDecimal left, HiveDecimal right) {
-    HiveDecimal dec = left.subtract(right);
-    if (dec == null) {
-      return null;
-    }
-    decimalWritable.set(dec);
+    decimalWritable.set(left);
+    decimalWritable.mutateSubtract(right);
     return decimalWritable;
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPNumericPlus.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPNumericPlus.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPNumericPlus.java
index b2b76f0..f9f5ff7 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPNumericPlus.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFOPNumericPlus.java
@@ -87,13 +87,8 @@ public class GenericUDFOPNumericPlus extends GenericUDFBaseNumeric {
 
   @Override
   protected HiveDecimalWritable evaluate(HiveDecimal left, HiveDecimal right) {
-    HiveDecimal dec = left.add(right);
-
-    if (dec == null) {
-      return null;
-    }
-
-    decimalWritable.set(dec);
+    decimalWritable.set(left);
+    decimalWritable.mutateAdd(right);
     return decimalWritable;
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFRound.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFRound.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFRound.java
index ae81fe3..e8b0d15 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFRound.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFRound.java
@@ -201,12 +201,12 @@ public class GenericUDFRound extends GenericUDF {
     case VOID:
       return null;
     case DECIMAL:
-      HiveDecimalWritable decimalWritable = (HiveDecimalWritable) inputOI.getPrimitiveWritableObject(input);
-      HiveDecimal dec = round(decimalWritable.getHiveDecimal(), scale);
-      if (dec == null) {
-        return null;
+      {
+        // The getPrimitiveWritableObject method returns a new writable.
+        HiveDecimalWritable decimalWritable = (HiveDecimalWritable) inputOI.getPrimitiveWritableObject(input);
+        // Call the different round flavor.
+        return round(decimalWritable, scale);
       }
-      return new HiveDecimalWritable(dec);
     case BYTE:
       ByteWritable byteWritable = (ByteWritable)inputOI.getPrimitiveWritableObject(input);
       if (scale >= 0) {
@@ -255,8 +255,10 @@ public class GenericUDFRound extends GenericUDF {
     }
   }
 
-  protected HiveDecimal round(HiveDecimal input, int scale) {
-    return RoundUtils.round(input, scale);
+  protected HiveDecimalWritable round(HiveDecimalWritable inputDecWritable, int scale) {
+    HiveDecimalWritable result = new HiveDecimalWritable(inputDecWritable);
+    result.mutateSetScale(scale, HiveDecimal.ROUND_HALF_UP);
+    return result;
   }
 
   protected long round(long input, int scale) {

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorGroupByOperator.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorGroupByOperator.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorGroupByOperator.java
index f5b5d9d..0ddebf8 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorGroupByOperator.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorGroupByOperator.java
@@ -732,12 +732,12 @@ public class TestVectorGroupByOperator {
         "sum",
         4,
         Arrays.asList(new Object[]{
-                HiveDecimal.create("1234.2401").setScale(scale),
-                HiveDecimal.create("1868.52").setScale(scale),
-                HiveDecimal.ZERO.setScale(scale),
-                HiveDecimal.create("456.84").setScale(scale),
-                HiveDecimal.create("121.89").setScale(scale)}),
-       HiveDecimal.create("3681.4901").setScale( scale));
+                HiveDecimal.create("1234.2401"),
+                HiveDecimal.create("1868.52"),
+                HiveDecimal.ZERO,
+                HiveDecimal.create("456.84"),
+                HiveDecimal.create("121.89")}),
+       HiveDecimal.create("3681.4901"));
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorSerDeRow.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorSerDeRow.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorSerDeRow.java
index 8ffff9d..b29bb8b 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorSerDeRow.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/TestVectorSerDeRow.java
@@ -97,7 +97,9 @@ public class TestVectorSerDeRow extends TestCase {
       PrimitiveCategory primitiveCategory = primitiveCategories[i];
       PrimitiveTypeInfo primitiveTypeInfo = source.primitiveTypeInfos()[i];
       if (!deserializeRead.readNextField()) {
-        throw new HiveException("Unexpected NULL");
+        throw new HiveException("Unexpected NULL when reading primitiveCategory " + primitiveCategory +
+            " expected (" + expected.getClass().getName() + ", " + expected.toString() + ") " +
+            " deserializeRead " + deserializeRead.getClass().getName());
       }
       switch (primitiveCategory) {
       case BOOLEAN:
@@ -307,7 +309,7 @@ public class TestVectorSerDeRow extends TestCase {
     }
   }
 
-  void testVectorSerializeRow(int caseNum, Random r, SerializationType serializationType)
+  void testVectorSerializeRow(Random r, SerializationType serializationType)
       throws HiveException, IOException, SerDeException {
 
     String[] emptyScratchTypeNames = new String[0];
@@ -383,7 +385,9 @@ public class TestVectorSerDeRow extends TestCase {
         Object rowObj = row[c];
         Object expectedObj = expectedRow[c];
         if (rowObj == null) {
-          fail("Unexpected NULL from extractRow");
+          fail("Unexpected NULL from extractRow.  Expected class " +
+              expectedObj.getClass().getName() + " value " + expectedObj.toString() +
+              " batch index " + i + " firstRandomRowIndex " + firstRandomRowIndex);
         }
         if (!rowObj.equals(expectedObj)) {
           fail("Row " + (firstRandomRowIndex + i) + " and column " + c + " mismatch (" + primitiveTypeInfos[c].getPrimitiveCategory() + " actual value " + rowObj + " and expected value " + expectedObj + ")");
@@ -541,7 +545,7 @@ public class TestVectorSerDeRow extends TestCase {
     return new LazySerDeParameters(conf, tbl, LazySimpleSerDe.class.getName());
   }
 
-  void testVectorDeserializeRow(int caseNum, Random r, SerializationType serializationType,
+  void testVectorDeserializeRow(Random r, SerializationType serializationType,
       boolean alternate1, boolean alternate2,
       boolean useExternalBuffer)
           throws HiveException, IOException, SerDeException {
@@ -689,113 +693,111 @@ public class TestVectorSerDeRow extends TestCase {
     }
   }
 
-  public void testVectorSerDeRow() throws Throwable {
-
-    Random r = new Random(5678);
-
-    int c = 0;
-
-    /*
-     * SERIALIZE tests.
-     */
-      testVectorSerializeRow(c++, r, SerializationType.BINARY_SORTABLE);
-
-      testVectorSerializeRow(c++, r, SerializationType.LAZY_BINARY);
-
-      testVectorSerializeRow(c++, r, SerializationType.LAZY_SIMPLE);
-
-    /*
-     * DESERIALIZE tests.
-     */
-
-    // BINARY_SORTABLE
-
-      testVectorDeserializeRow(c++, r,
-          SerializationType.BINARY_SORTABLE,
-          /* alternate1 = useColumnSortOrderIsDesc */ false,
-          /* alternate2 = useBinarySortableCharsNeedingEscape */ false,
-          /* useExternalBuffer */ false);
-
-      testVectorDeserializeRow(c++, r,
-          SerializationType.BINARY_SORTABLE,
-          /* alternate1 = useColumnSortOrderIsDesc */ true,
-          /* alternate2 = useBinarySortableCharsNeedingEscape */ false,
-          /* useExternalBuffer */ false);
-
-      testVectorDeserializeRow(c++, r,
-          SerializationType.BINARY_SORTABLE,
-          /* alternate1 = useColumnSortOrderIsDesc */ false,
-          /* alternate2 = useBinarySortableCharsNeedingEscape */ false,
-          /* useExternalBuffer */ true);
-
-      testVectorDeserializeRow(c++, r,
-          SerializationType.BINARY_SORTABLE,
-          /* alternate1 = useColumnSortOrderIsDesc */ true,
-          /* alternate2 = useBinarySortableCharsNeedingEscape */ false,
-          /* useExternalBuffer */ true);
-
-      testVectorDeserializeRow(c++, r,
-          SerializationType.BINARY_SORTABLE,
-          /* alternate1 = useColumnSortOrderIsDesc */ false,
-          /* alternate2 = useBinarySortableCharsNeedingEscape */ true,
-          /* useExternalBuffer */ false);
-
-      testVectorDeserializeRow(c++, r,
-          SerializationType.BINARY_SORTABLE,
-          /* alternate1 = useColumnSortOrderIsDesc */ true,
-          /* alternate2 = useBinarySortableCharsNeedingEscape */ true,
-          /* useExternalBuffer */ false);
-
-      testVectorDeserializeRow(c++, r,
-          SerializationType.BINARY_SORTABLE,
-          /* alternate1 = useColumnSortOrderIsDesc */ false,
-          /* alternate2 = useBinarySortableCharsNeedingEscape */ true,
-          /* useExternalBuffer */ true);
-
-      testVectorDeserializeRow(c++, r,
-          SerializationType.BINARY_SORTABLE,
-          /* alternate1 = useColumnSortOrderIsDesc */ true,
-          /* alternate2 = useBinarySortableCharsNeedingEscape */ true,
-          /* useExternalBuffer */ true);
-
-    // LAZY_BINARY
-
-      testVectorDeserializeRow(c++, r,
-          SerializationType.LAZY_BINARY,
-          /* alternate1 = unused */ false,
-          /* alternate2 = unused */ false,
-          /* useExternalBuffer */ false);
-
-      testVectorDeserializeRow(c++, r,
-          SerializationType.LAZY_BINARY,
-          /* alternate1 = unused */ false,
-          /* alternate2 = unused */ false,
-          /* useExternalBuffer */ true);
-
-    // LAZY_SIMPLE
-
-      testVectorDeserializeRow(c++, r,
-          SerializationType.LAZY_SIMPLE,
-          /* alternate1 = useLazySimpleEscapes */ false,
-          /* alternate2 = unused */ false,
-          /* useExternalBuffer */ false);
-
-      testVectorDeserializeRow(c++, r,
-          SerializationType.LAZY_SIMPLE,
-          /* alternate1 = useLazySimpleEscapes */ false,
-          /* alternate2 = unused */ false,
-          /* useExternalBuffer */ true);
-
-      testVectorDeserializeRow(c++, r,
-          SerializationType.LAZY_SIMPLE,
-          /* alternate1 = useLazySimpleEscapes */ true,
-          /* alternate2 = unused */ false,
-          /* useExternalBuffer */ false);
-
-      testVectorDeserializeRow(c++, r,
-          SerializationType.LAZY_SIMPLE,
-          /* alternate1 = useLazySimpleEscapes */ true,
-          /* alternate2 = unused */ false,
-          /* useExternalBuffer */ true);
+  public void testVectorBinarySortableSerializeRow() throws Throwable {
+    Random r = new Random(8732);
+    testVectorSerializeRow(r, SerializationType.BINARY_SORTABLE);
+  }
+
+  public void testVectorLazyBinarySerializeRow() throws Throwable {
+    Random r = new Random(8732);
+    testVectorSerializeRow(r, SerializationType.LAZY_BINARY);
+  }
+
+  public void testVectorLazySimpleSerializeRow() throws Throwable {
+    Random r = new Random(8732);
+    testVectorSerializeRow(r, SerializationType.LAZY_SIMPLE);
+  }
+ 
+  public void testVectorBinarySortableDeserializeRow() throws Throwable {
+    Random r = new Random(8732);
+    testVectorDeserializeRow(r,
+        SerializationType.BINARY_SORTABLE,
+        /* alternate1 = useColumnSortOrderIsDesc */ false,
+        /* alternate2 = useBinarySortableCharsNeedingEscape */ false,
+        /* useExternalBuffer */ false);
+
+    testVectorDeserializeRow(r,
+        SerializationType.BINARY_SORTABLE,
+        /* alternate1 = useColumnSortOrderIsDesc */ true,
+        /* alternate2 = useBinarySortableCharsNeedingEscape */ false,
+        /* useExternalBuffer */ false);
+
+    testVectorDeserializeRow(r,
+        SerializationType.BINARY_SORTABLE,
+        /* alternate1 = useColumnSortOrderIsDesc */ false,
+        /* alternate2 = useBinarySortableCharsNeedingEscape */ false,
+        /* useExternalBuffer */ true);
+
+    testVectorDeserializeRow(r,
+        SerializationType.BINARY_SORTABLE,
+        /* alternate1 = useColumnSortOrderIsDesc */ true,
+        /* alternate2 = useBinarySortableCharsNeedingEscape */ false,
+        /* useExternalBuffer */ true);
+
+    testVectorDeserializeRow(r,
+        SerializationType.BINARY_SORTABLE,
+        /* alternate1 = useColumnSortOrderIsDesc */ false,
+        /* alternate2 = useBinarySortableCharsNeedingEscape */ true,
+        /* useExternalBuffer */ false);
+
+    testVectorDeserializeRow(r,
+        SerializationType.BINARY_SORTABLE,
+        /* alternate1 = useColumnSortOrderIsDesc */ true,
+        /* alternate2 = useBinarySortableCharsNeedingEscape */ true,
+        /* useExternalBuffer */ false);
+
+    testVectorDeserializeRow(r,
+        SerializationType.BINARY_SORTABLE,
+        /* alternate1 = useColumnSortOrderIsDesc */ false,
+        /* alternate2 = useBinarySortableCharsNeedingEscape */ true,
+        /* useExternalBuffer */ true);
+
+    testVectorDeserializeRow(r,
+        SerializationType.BINARY_SORTABLE,
+        /* alternate1 = useColumnSortOrderIsDesc */ true,
+        /* alternate2 = useBinarySortableCharsNeedingEscape */ true,
+        /* useExternalBuffer */ true);
+  }
+
+  public void testVectorLazyBinaryDeserializeRow() throws Throwable {
+    Random r = new Random(8732);
+    testVectorDeserializeRow(r,
+        SerializationType.LAZY_BINARY,
+        /* alternate1 = unused */ false,
+        /* alternate2 = unused */ false,
+        /* useExternalBuffer */ false);
+
+    testVectorDeserializeRow(r,
+        SerializationType.LAZY_BINARY,
+        /* alternate1 = unused */ false,
+        /* alternate2 = unused */ false,
+        /* useExternalBuffer */ true);
+  }
+
+  public void testVectorLazySimpleDeserializeRow() throws Throwable {
+    Random r = new Random(8732);
+    testVectorDeserializeRow(r,
+        SerializationType.LAZY_SIMPLE,
+        /* alternate1 = useLazySimpleEscapes */ false,
+        /* alternate2 = unused */ false,
+        /* useExternalBuffer */ false);
+
+    testVectorDeserializeRow(r,
+        SerializationType.LAZY_SIMPLE,
+        /* alternate1 = useLazySimpleEscapes */ false,
+        /* alternate2 = unused */ false,
+        /* useExternalBuffer */ true);
+
+    testVectorDeserializeRow(r,
+        SerializationType.LAZY_SIMPLE,
+        /* alternate1 = useLazySimpleEscapes */ true,
+        /* alternate2 = unused */ false,
+        /* useExternalBuffer */ false);
+
+    testVectorDeserializeRow(r,
+        SerializationType.LAZY_SIMPLE,
+        /* alternate1 = useLazySimpleEscapes */ true,
+        /* alternate2 = unused */ false,
+        /* useExternalBuffer */ true);
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/VectorRandomRowSource.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/VectorRandomRowSource.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/VectorRandomRowSource.java
index 57bf60d..cbde615 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/VectorRandomRowSource.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/VectorRandomRowSource.java
@@ -25,6 +25,7 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Random;
 
+import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable;
 import org.apache.commons.lang.ArrayUtils;
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.hive.common.type.HiveChar;
@@ -336,27 +337,28 @@ public class VectorRandomRowSource {
       {
         WritableHiveCharObjectInspector writableCharObjectInspector =
                 new WritableHiveCharObjectInspector( (CharTypeInfo) primitiveTypeInfo);
-        return writableCharObjectInspector.create(new HiveChar(StringUtils.EMPTY, -1));
+        return writableCharObjectInspector.create((HiveChar) object);
       }
     case VARCHAR:
       {
         WritableHiveVarcharObjectInspector writableVarcharObjectInspector =
                 new WritableHiveVarcharObjectInspector( (VarcharTypeInfo) primitiveTypeInfo);
-        return writableVarcharObjectInspector.create(new HiveVarchar(StringUtils.EMPTY, -1));
+        return writableVarcharObjectInspector.create((HiveVarchar) object);
       }
     case BINARY:
-      return PrimitiveObjectInspectorFactory.writableBinaryObjectInspector.create(ArrayUtils.EMPTY_BYTE_ARRAY);
+      return PrimitiveObjectInspectorFactory.writableBinaryObjectInspector.create((byte[]) object);
     case TIMESTAMP:
-      return ((WritableTimestampObjectInspector) objectInspector).create(new Timestamp(0));
+      return ((WritableTimestampObjectInspector) objectInspector).create((Timestamp) object);
     case INTERVAL_YEAR_MONTH:
-      return ((WritableHiveIntervalYearMonthObjectInspector) objectInspector).create(new HiveIntervalYearMonth(0));
+      return ((WritableHiveIntervalYearMonthObjectInspector) objectInspector).create((HiveIntervalYearMonth) object);
     case INTERVAL_DAY_TIME:
-      return ((WritableHiveIntervalDayTimeObjectInspector) objectInspector).create(new HiveIntervalDayTime(0, 0));
+      return ((WritableHiveIntervalDayTimeObjectInspector) objectInspector).create((HiveIntervalDayTime) object);
     case DECIMAL:
       {
         WritableHiveDecimalObjectInspector writableDecimalObjectInspector =
                 new WritableHiveDecimalObjectInspector((DecimalTypeInfo) primitiveTypeInfo);
-        return writableDecimalObjectInspector.create(HiveDecimal.ZERO);
+        HiveDecimalWritable result = (HiveDecimalWritable) writableDecimalObjectInspector.create((HiveDecimal) object);
+        return result;
       }
     default:
       throw new Error("Unknown primitive category " + primitiveCategory);
@@ -420,7 +422,7 @@ public class VectorRandomRowSource {
           case CHAR:
             return new HiveChar(result, ((CharTypeInfo) primitiveTypeInfo).getLength());
           case VARCHAR:
-            return new HiveChar(result, ((VarcharTypeInfo) primitiveTypeInfo).getLength());
+            return new HiveVarchar(result, ((VarcharTypeInfo) primitiveTypeInfo).getLength());
           default:
             throw new Error("Unknown primitive category " + primitiveCategory);
           }
@@ -497,13 +499,9 @@ public class VectorRandomRowSource {
         sb.append(RandomTypeUtil.getRandString(r, DECIMAL_CHARS, scale));
       }
 
-      HiveDecimal bd = HiveDecimal.create(sb.toString());
-      if (bd.scale() > bd.precision()) {
-        // Sometimes weird decimals are produced?
-        continue;
-      }
+      HiveDecimal dec = HiveDecimal.create(sb.toString());
 
-      return bd;
+      return dec;
     }
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestDecimalUtil.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestDecimalUtil.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestDecimalUtil.java
index da9ebca..9f4bd9d 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestDecimalUtil.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestDecimalUtil.java
@@ -229,7 +229,6 @@ public class TestDecimalUtil {
 
     HiveDecimal d3 = HiveDecimal.create("0.00000");
     Assert.assertEquals(0, d3.scale());
-    d3.setScale(5);
     DecimalUtil.sign(0, d3, lcv);
     Assert.assertEquals(0, lcv.vector[0]);
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorTypeCasts.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorTypeCasts.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorTypeCasts.java
index e7a044e..c7d9fae 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorTypeCasts.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/expressions/TestVectorTypeCasts.java
@@ -204,6 +204,10 @@ public class TestVectorTypeCasts {
     // test basic case
     VectorizedRowBatch b = getBatchDecimalLong();
     VectorExpression expr = new CastDecimalToLong(0, 1);
+
+    // With the integer type range checking, we need to know the Hive data type.
+    expr.setOutputType("bigint");
+
     expr.evaluate(b);
     LongColumnVector r = (LongColumnVector) b.cols[1];
     assertEquals(1, r.vector[0]);
@@ -265,9 +269,9 @@ public class TestVectorTypeCasts {
 
     b.size = 3;
 
-    dv.vector[0].set(HiveDecimal.create("1.1").setScale(scale));
-    dv.vector[1].set(HiveDecimal.create("-2.2").setScale(scale));
-    dv.vector[2].set(HiveDecimal.create("9999999999999999.00").setScale(scale));
+    dv.vector[0].set(HiveDecimal.create("1.1"));
+    dv.vector[1].set(HiveDecimal.create("-2.2"));
+    dv.vector[2].set(HiveDecimal.create("9999999999999999.00"));
 
     return b;
   }
@@ -325,9 +329,9 @@ public class TestVectorTypeCasts {
 
     b.size = 3;
 
-    dv.vector[0].set(HiveDecimal.create("1.1").setScale(scale));
-    dv.vector[1].set(HiveDecimal.create("-2.2").setScale(scale));
-    dv.vector[2].set(HiveDecimal.create("9999999999999999.00").setScale(scale));
+    dv.vector[0].set(HiveDecimal.create("1.1"));
+    dv.vector[1].set(HiveDecimal.create("-2.2"));
+    dv.vector[2].set(HiveDecimal.create("9999999999999999.00"));
 
     return b;
   }
@@ -366,9 +370,9 @@ public class TestVectorTypeCasts {
 
     b.size = 3;
 
-    dv.vector[0].set(HiveDecimal.create("1.1").setScale(scale));
-    dv.vector[1].set(HiveDecimal.create("-2.2").setScale(scale));
-    dv.vector[2].set(HiveDecimal.create("9999999999999999.00").setScale(scale));
+    dv.vector[0].set(HiveDecimal.create("1.1"));
+    dv.vector[1].set(HiveDecimal.create("-2.2"));
+    dv.vector[2].set(HiveDecimal.create("9999999999999999.00"));
 
     return b;
   }
@@ -399,9 +403,9 @@ public class TestVectorTypeCasts {
 
     b.size = 3;
 
-    dv.vector[0].set(HiveDecimal.create("1.111111111").setScale(scale));
-    dv.vector[1].set(HiveDecimal.create("-2.222222222").setScale(scale));
-    dv.vector[2].set(HiveDecimal.create("31536000.999999999").setScale(scale));
+    dv.vector[0].set(HiveDecimal.create("1.111111111"));
+    dv.vector[1].set(HiveDecimal.create("-2.222222222"));
+    dv.vector[2].set(HiveDecimal.create("31536000.999999999"));
 
     return b;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/CheckFastRowHashMap.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/CheckFastRowHashMap.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/CheckFastRowHashMap.java
index bc7a658..638ccc5 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/CheckFastRowHashMap.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/CheckFastRowHashMap.java
@@ -166,7 +166,9 @@ public class CheckFastRowHashMap extends CheckFastHashTable {
         }
       } else {
         if (thrown) {
-          TestCase.fail("Not expecting an exception to be thrown for the non-clipped case...");
+          TestCase.fail("Not expecting an exception to be thrown for the non-clipped case... " +
+              " exception message " + debugExceptionMessage +
+              " stack trace " + getStackTraceAsSingleLine(debugStackTrace));
         }
         TestCase.assertTrue(lazyBinaryDeserializeRead.isEndOfInputReached());
       }
@@ -382,4 +384,27 @@ public class CheckFastRowHashMap extends CheckFastHashTable {
       }
     }
   }
+
+  static int STACK_LENGTH_LIMIT = 20;
+  public static String getStackTraceAsSingleLine(StackTraceElement[] stackTrace) {
+    StringBuilder sb = new StringBuilder();
+    sb.append("Stack trace: ");
+    int length = stackTrace.length;
+    boolean isTruncated = false;
+    if (length > STACK_LENGTH_LIMIT) {
+      length = STACK_LENGTH_LIMIT;
+      isTruncated = true;
+    }
+    for (int i = 0; i < length; i++) {
+      if (i > 0) {
+        sb.append(", ");
+      }
+      sb.append(stackTrace[i]);
+    }
+    if (isTruncated) {
+      sb.append(", ...");
+    }
+
+    return sb.toString();
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VerifyFastRow.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VerifyFastRow.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VerifyFastRow.java
index 239db73..91b3ead 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VerifyFastRow.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/mapjoin/fast/VerifyFastRow.java
@@ -67,7 +67,10 @@ public class VerifyFastRow {
     isNull = !deserializeRead.readNextField();
     if (isNull) {
       if (writable != null) {
-        TestCase.fail("Field reports null but object is not null");
+        TestCase.fail(
+            deserializeRead.getClass().getName() +
+            " field reports null but object is not null " +
+            "(class " + writable.getClass().getName() + ", " + writable.toString() + ")");
       }
       return;
     } else if (writable == null) {

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/util/VectorizedRowGroupGenUtil.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/util/VectorizedRowGroupGenUtil.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/util/VectorizedRowGroupGenUtil.java
index 84717b1..1f50282 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/util/VectorizedRowGroupGenUtil.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/vector/util/VectorizedRowGroupGenUtil.java
@@ -151,7 +151,7 @@ public class VectorizedRowGroupGenUtil {
 
     HiveDecimalWritable repeatingValue = new HiveDecimalWritable();
     do{
-      repeatingValue.set(HiveDecimal.create(((Double) rand.nextDouble()).toString()).setScale((short)typeInfo.scale()));
+      repeatingValue.set(HiveDecimal.create(((Double) rand.nextDouble()).toString()).setScale((short)typeInfo.scale(), HiveDecimal.ROUND_HALF_UP));
     }while(repeatingValue.getHiveDecimal().doubleValue() == 0);
 
     int nullFrequency = generateNullFrequency(rand);
@@ -159,14 +159,14 @@ public class VectorizedRowGroupGenUtil {
     for(int i = 0; i < size; i++) {
       if(nulls && (repeating || i % nullFrequency == 0)) {
         dcv.isNull[i] = true;
-        dcv.vector[i] = null;//Decimal128.ONE;
+        dcv.vector[i] = null;
 
       }else {
         dcv.isNull[i] = false;
         if (repeating) {
           dcv.vector[i].set(repeatingValue);
         } else {
-          dcv.vector[i].set(HiveDecimal.create(((Double) rand.nextDouble()).toString()).setScale((short) typeInfo.scale()));
+          dcv.vector[i].set(HiveDecimal.create(((Double) rand.nextDouble()).toString()).setScale((short) typeInfo.scale(), HiveDecimal.ROUND_HALF_UP));
         }
 
         if(dcv.vector[i].getHiveDecimal().doubleValue() == 0) {

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/ql/src/test/results/clientpositive/decimal_2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/decimal_2.q.out b/ql/src/test/results/clientpositive/decimal_2.q.out
index 934590c..f3168f6 100644
--- a/ql/src/test/results/clientpositive/decimal_2.q.out
+++ b/ql/src/test/results/clientpositive/decimal_2.q.out
@@ -129,7 +129,7 @@ POSTHOOK: query: select cast(t as tinyint) from decimal_2
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@decimal_2
 #### A masked pattern was here ####
-13
+NULL
 PREHOOK: query: select cast(t as smallint) from decimal_2
 PREHOOK: type: QUERY
 PREHOOK: Input: default@decimal_2
@@ -138,7 +138,7 @@ POSTHOOK: query: select cast(t as smallint) from decimal_2
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@decimal_2
 #### A masked pattern was here ####
--3827
+NULL
 PREHOOK: query: select cast(t as int) from decimal_2
 PREHOOK: type: QUERY
 PREHOOK: Input: default@decimal_2

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/ql/src/test/results/clientpositive/llap/schema_evol_orc_nonvec_part_all_primitive.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/schema_evol_orc_nonvec_part_all_primitive.q.out b/ql/src/test/results/clientpositive/llap/schema_evol_orc_nonvec_part_all_primitive.q.out
index 7831114..e0834bc 100644
--- a/ql/src/test/results/clientpositive/llap/schema_evol_orc_nonvec_part_all_primitive.q.out
+++ b/ql/src/test/results/clientpositive/llap/schema_evol_orc_nonvec_part_all_primitive.q.out
@@ -309,10 +309,10 @@ POSTHOOK: Input: default@part_change_various_various_boolean_to_bigint
 POSTHOOK: Input: default@part_change_various_various_boolean_to_bigint@part=1
 #### A masked pattern was here ####
 insert_num	part	c1	c2	c3	c4	c5	c6	c7	c8	c9	c10	c11	c12	c13	c14	c15	c16	c17	c18	c19	c20	c21	c22	c23	c24	c25	c26	c27	c28	c29	c30	c31	c32	c33	c34	c35	c36	c37	c38	c39	c40	c41	c42	c43	c44	c45	c46	c47	c48	c49	c50	c51	c52	c53	b
-101	1	true	NULL	true	NULL	NULL	NULL	NULL	NULL	true	1	NULL	NULL	NULL	NULL	NULL	NULL	-128	-128	-128	NULL	1	-128	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	1	-128	NULL	NULL	NULL	NULL	NULL	-2147483648	-2147483648	-2147483648	NULL	1	-128	NULL	-2147483648	NULL	NULL	NULL	NULL	NULL	NULL	134416490068	original
+101	1	true	NULL	true	NULL	NULL	NULL	true	NULL	true	1	NULL	NULL	NULL	NULL	NULL	NULL	-128	-128	-128	NULL	1	-128	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	1	-128	NULL	NULL	NULL	NULL	NULL	-2147483648	-2147483648	-2147483648	NULL	1	-128	NULL	-2147483648	NULL	NULL	NULL	NULL	NULL	NULL	134416490068	original
 101	1	true	true	true	true	true	true	true	true	true	-128	-128	-128	-128	-128	-128	-128	-128	-128	-128	-128	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	-2147483648	-2147483648	-2147483648	-2147483648	-2147483648	-2147483648	-2147483648	-2147483648	-2147483648	-2147483648	-2147483648	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	new
 102	1	false	false	false	false	false	false	false	false	false	127	127	127	127	127	127	127	127	127	127	127	32767	32767	32767	32767	32767	32767	32767	32767	32767	32767	32767	2147483647	2147483647	2147483647	2147483647	2147483647	2147483647	2147483647	2147483647	2147483647	2147483647	2147483647	9223372036854775807	9223372036854775807	9223372036854775807	9223372036854775807	9223372036854775807	9223372036854775807	9223372036854775807	9223372036854775807	9223372036854775807	9223372036854775807	9223372036854775807	new
-102	1	true	true	true	true	NULL	NULL	NULL	NULL	true	0	NULL	NULL	NULL	NULL	NULL	NULL	127	127	127	NULL	0	127	NULL	NULL	NULL	NULL	NULL	32767	32767	32767	NULL	0	127	32767	NULL	NULL	NULL	NULL	2147483647	2147483647	2147483647	NULL	0	127	32767	2147483647	NULL	NULL	NULL	9223372036854775807	9223372036854775807	9223372036854775807	126117945050	original
+102	1	true	true	true	true	NULL	NULL	true	NULL	true	0	NULL	NULL	NULL	NULL	NULL	NULL	127	127	127	NULL	0	127	NULL	NULL	NULL	NULL	NULL	32767	32767	32767	NULL	0	127	32767	NULL	NULL	NULL	NULL	2147483647	2147483647	2147483647	NULL	0	127	32767	2147483647	NULL	NULL	NULL	9223372036854775807	9223372036854775807	9223372036854775807	126117945050	original
 103	1	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	new
 103	1	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	original
 104	1	true	true	true	true	true	true	true	NULL	true	1	NULL	NULL	NULL	-100	30	NULL	23	23	23	NULL	1	23	NULL	NULL	-100	30	NULL	834	834	834	NULL	1	23	834	NULL	-100	30	66475	203332	203332	203332	270912854	1	23	834	203332	-100	30	66475	888888857923222	888888857923222	888888857923222	270912854	original

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/ql/src/test/results/clientpositive/llap/schema_evol_orc_vec_part_all_primitive.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/schema_evol_orc_vec_part_all_primitive.q.out b/ql/src/test/results/clientpositive/llap/schema_evol_orc_vec_part_all_primitive.q.out
index d240f82..e1496bb 100644
--- a/ql/src/test/results/clientpositive/llap/schema_evol_orc_vec_part_all_primitive.q.out
+++ b/ql/src/test/results/clientpositive/llap/schema_evol_orc_vec_part_all_primitive.q.out
@@ -309,10 +309,10 @@ POSTHOOK: Input: default@part_change_various_various_boolean_to_bigint
 POSTHOOK: Input: default@part_change_various_various_boolean_to_bigint@part=1
 #### A masked pattern was here ####
 insert_num	part	c1	c2	c3	c4	c5	c6	c7	c8	c9	c10	c11	c12	c13	c14	c15	c16	c17	c18	c19	c20	c21	c22	c23	c24	c25	c26	c27	c28	c29	c30	c31	c32	c33	c34	c35	c36	c37	c38	c39	c40	c41	c42	c43	c44	c45	c46	c47	c48	c49	c50	c51	c52	c53	b
-101	1	true	NULL	true	NULL	NULL	NULL	NULL	NULL	true	1	NULL	NULL	NULL	NULL	NULL	NULL	-128	-128	-128	NULL	1	-128	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	1	-128	NULL	NULL	NULL	NULL	NULL	-2147483648	-2147483648	-2147483648	NULL	1	-128	NULL	-2147483648	NULL	NULL	NULL	NULL	NULL	NULL	134416490068	original
+101	1	true	NULL	true	NULL	NULL	NULL	true	NULL	true	1	NULL	NULL	NULL	NULL	NULL	NULL	-128	-128	-128	NULL	1	-128	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	1	-128	NULL	NULL	NULL	NULL	NULL	-2147483648	-2147483648	-2147483648	NULL	1	-128	NULL	-2147483648	NULL	NULL	NULL	NULL	NULL	NULL	134416490068	original
 101	1	true	true	true	true	true	true	true	true	true	-128	-128	-128	-128	-128	-128	-128	-128	-128	-128	-128	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	-2147483648	-2147483648	-2147483648	-2147483648	-2147483648	-2147483648	-2147483648	-2147483648	-2147483648	-2147483648	-2147483648	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	new
 102	1	false	false	false	false	false	false	false	false	false	127	127	127	127	127	127	127	127	127	127	127	32767	32767	32767	32767	32767	32767	32767	32767	32767	32767	32767	2147483647	2147483647	2147483647	2147483647	2147483647	2147483647	2147483647	2147483647	2147483647	2147483647	2147483647	9223372036854775807	9223372036854775807	9223372036854775807	9223372036854775807	9223372036854775807	9223372036854775807	9223372036854775807	9223372036854775807	9223372036854775807	9223372036854775807	9223372036854775807	new
-102	1	true	true	true	true	NULL	NULL	NULL	NULL	true	0	NULL	NULL	NULL	NULL	NULL	NULL	127	127	127	NULL	0	127	NULL	NULL	NULL	NULL	NULL	32767	32767	32767	NULL	0	127	32767	NULL	NULL	NULL	NULL	2147483647	2147483647	2147483647	NULL	0	127	32767	2147483647	NULL	NULL	NULL	9223372036854775807	9223372036854775807	9223372036854775807	126117945050	original
+102	1	true	true	true	true	NULL	NULL	true	NULL	true	0	NULL	NULL	NULL	NULL	NULL	NULL	127	127	127	NULL	0	127	NULL	NULL	NULL	NULL	NULL	32767	32767	32767	NULL	0	127	32767	NULL	NULL	NULL	NULL	2147483647	2147483647	2147483647	NULL	0	127	32767	2147483647	NULL	NULL	NULL	9223372036854775807	9223372036854775807	9223372036854775807	126117945050	original
 103	1	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	new
 103	1	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	original
 104	1	true	true	true	true	true	true	true	NULL	true	1	NULL	NULL	NULL	-100	30	NULL	23	23	23	NULL	1	23	NULL	NULL	-100	30	NULL	834	834	834	NULL	1	23	834	NULL	-100	30	66475	203332	203332	203332	270912854	1	23	834	203332	-100	30	66475	888888857923222	888888857923222	888888857923222	270912854	original

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/ql/src/test/results/clientpositive/llap/schema_evol_text_nonvec_part_all_primitive.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/schema_evol_text_nonvec_part_all_primitive.q.out b/ql/src/test/results/clientpositive/llap/schema_evol_text_nonvec_part_all_primitive.q.out
index b07986e..2a2e48a 100644
--- a/ql/src/test/results/clientpositive/llap/schema_evol_text_nonvec_part_all_primitive.q.out
+++ b/ql/src/test/results/clientpositive/llap/schema_evol_text_nonvec_part_all_primitive.q.out
@@ -310,15 +310,15 @@ POSTHOOK: Input: default@part_change_various_various_boolean_to_bigint@part=1
 #### A masked pattern was here ####
 insert_num	part	c1	c2	c3	c4	c5	c6	c7	c8	c9	c10	c11	c12	c13	c14	c15	c16	c17	c18	c19	c20	c21	c22	c23	c24	c25	c26	c27	c28	c29	c30	c31	c32	c33	c34	c35	c36	c37	c38	c39	c40	c41	c42	c43	c44	c45	c46	c47	c48	c49	c50	c51	c52	c53	b
 101	1	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	NULL	-128	-128	-128	-128	-128	-128	-128	-128	-128	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	-2147483648	-2147483648	-2147483648	-2147483648	-2147483648	-2147483648	-2147483648	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	new
-101	1	true	NULL	true	NULL	true	true	true	true	true	1	NULL	0	NULL	-1	-1	-1	-128	-128	-128	84	1	-128	0	NULL	-1	-1	-1	NULL	NULL	NULL	-8620	1	-128	NULL	NULL	2147483647	2147483647	1661992959	-2147483648	-2147483648	-2147483648	1272503892	1	-128	NULL	-2147483648	9223372036854775807	9223372036854775807	7766279631452241919	NULL	NULL	NULL	134416490068	original
+101	1	true	NULL	true	NULL	true	true	true	true	true	1	NULL	0	NULL	-1	-1	NULL	-128	-128	-128	84	1	-128	0	NULL	-1	-1	NULL	NULL	NULL	NULL	-8620	1	-128	NULL	NULL	2147483647	2147483647	NULL	-2147483648	-2147483648	-2147483648	1272503892	1	-128	NULL	-2147483648	9223372036854775807	9223372036854775807	NULL	NULL	NULL	NULL	134416490068	original
 102	1	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	NULL	127	127	127	127	127	127	127	127	127	NULL	NULL	NULL	32767	32767	32767	32767	32767	32767	32767	32767	NULL	NULL	NULL	NULL	2147483647	2147483647	2147483647	2147483647	2147483647	2147483647	2147483647	NULL	NULL	NULL	NULL	NULL	9223372036854775807	9223372036854775807	9223372036854775807	9223372036854775807	9223372036854775807	9223372036854775807	NULL	new
-102	1	true	true	true	true	true	true	true	true	true	0	-1	-1	-1	0	0	1	127	127	127	-38	0	127	-1	-1	0	0	1	32767	32767	32767	7898	0	127	32767	-1	-2147483648	-2147483648	-1661992959	2147483647	2147483647	2147483647	1563893466	0	127	32767	2147483647	-9223372036854775808	-9223372036854775808	-7766279631452241919	9223372036854775807	9223372036854775807	9223372036854775807	126117945050	original
+102	1	true	true	true	true	true	true	true	true	true	0	-1	-1	-1	0	0	NULL	127	127	127	-38	0	127	-1	-1	0	0	NULL	32767	32767	32767	7898	0	127	32767	-1	-2147483648	-2147483648	NULL	2147483647	2147483647	2147483647	1563893466	0	127	32767	2147483647	-9223372036854775808	-9223372036854775808	NULL	9223372036854775807	9223372036854775807	9223372036854775807	126117945050	original
 103	1	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	new
 103	1	NULL	NULL	NULL	NULL	NULL	NULL	NULL	false	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	original
 104	1	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	NULL	23	23	23	23	23	23	23	23	23	NULL	NULL	NULL	834	834	834	834	834	834	834	834	NULL	NULL	NULL	NULL	203332	203332	203332	203332	203332	203332	203332	NULL	NULL	NULL	NULL	NULL	888888847499264	888888857923222	888888857923222	888888857923222	888888857923222	888888857923222	NULL	new
-104	1	true	true	true	true	true	true	true	true	true	1	66	68	-106	-100	30	-85	23	23	23	86	1	23	6724	3734	-100	30	939	834	834	834	-12970	1	23	834	-1868624234	-100	30	66475	203332	203332	203332	270912854	1	23	834	203332	-100	30	66475	888888857923222	888888857923222	888888857923222	270912854	original
+104	1	true	true	true	true	true	true	true	true	true	1	66	68	-106	-100	30	NULL	23	23	23	86	1	23	6724	3734	-100	30	NULL	834	834	834	-12970	1	23	834	-1868624234	-100	30	66475	203332	203332	203332	270912854	1	23	834	203332	-100	30	66475	888888857923222	888888857923222	888888857923222	270912854	original
 105	1	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	NULL	-99	-99	-99	-99	-99	-99	-99	-99	-99	NULL	NULL	NULL	-28300	-28300	-28300	-28300	-28300	-28300	-28300	-28300	NULL	NULL	NULL	NULL	-999992	-999992	-999992	-999992	-999992	-999992	-999992	NULL	NULL	NULL	NULL	NULL	-222282153984	-222282153733	-222282153733	-222282153733	-222282153733	-222282153733	NULL	new
-105	1	true	true	true	true	NULL	true	true	true	true	0	116	-56	-5	NULL	34	36	-99	-99	-99	-41	0	-99	-16952	-32517	NULL	-19422	9764	-28300	-28300	-28300	-16681	0	-99	-28300	1056145659	NULL	46114	9250340	-999992	-999992	-999992	663207639	0	-99	-28300	-999992	NULL	46114	9250340	-222282153733	-222282153733	-222282153733	663207639	original
+105	1	true	true	true	true	NULL	true	true	true	true	0	116	-56	-5	NULL	34	NULL	-99	-99	-99	-41	0	-99	-16952	-32517	NULL	-19422	NULL	-28300	-28300	-28300	-16681	0	-99	-28300	1056145659	NULL	46114	9250340	-999992	-999992	-999992	663207639	0	-99	-28300	-999992	NULL	46114	9250340	-222282153733	-222282153733	-222282153733	663207639	original
 PREHOOK: query: drop table part_change_various_various_boolean_to_bigint
 PREHOOK: type: DROPTABLE
 PREHOOK: Input: default@part_change_various_various_boolean_to_bigint

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/ql/src/test/results/clientpositive/llap/schema_evol_text_vec_part_all_primitive.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/schema_evol_text_vec_part_all_primitive.q.out b/ql/src/test/results/clientpositive/llap/schema_evol_text_vec_part_all_primitive.q.out
index fb38687..46da4f0 100644
--- a/ql/src/test/results/clientpositive/llap/schema_evol_text_vec_part_all_primitive.q.out
+++ b/ql/src/test/results/clientpositive/llap/schema_evol_text_vec_part_all_primitive.q.out
@@ -314,15 +314,15 @@ POSTHOOK: Input: default@part_change_various_various_boolean_to_bigint@part=1
 #### A masked pattern was here ####
 insert_num	part	c1	c2	c3	c4	c5	c6	c7	c8	c9	c10	c11	c12	c13	c14	c15	c16	c17	c18	c19	c20	c21	c22	c23	c24	c25	c26	c27	c28	c29	c30	c31	c32	c33	c34	c35	c36	c37	c38	c39	c40	c41	c42	c43	c44	c45	c46	c47	c48	c49	c50	c51	c52	c53	b
 101	1	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	-128	-128	-128	-128	-128	-128	-128	-128	-128	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	-2147483648	-2147483648	-2147483648	-2147483648	-2147483648	-2147483648	-2147483648	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	new
-101	1	true	NULL	true	NULL	true	true	true	true	true	NULL	NULL	0	NULL	-1	-1	-1	-128	-128	-128	84	NULL	-128	0	NULL	-1	-1	-1	NULL	NULL	NULL	-8620	NULL	-128	NULL	NULL	2147483647	2147483647	1661992959	-2147483648	-2147483648	-2147483648	1272503892	NULL	-128	NULL	-2147483648	9223372036854775807	9223372036854775807	7766279631452241919	NULL	NULL	NULL	134416490068	original
+101	1	true	NULL	true	NULL	true	true	true	true	true	NULL	NULL	0	NULL	-1	-1	NULL	-128	-128	-128	84	NULL	-128	0	NULL	-1	-1	NULL	NULL	NULL	NULL	-8620	NULL	-128	NULL	NULL	2147483647	2147483647	NULL	-2147483648	-2147483648	-2147483648	1272503892	NULL	-128	NULL	-2147483648	9223372036854775807	9223372036854775807	NULL	NULL	NULL	NULL	134416490068	original
 102	1	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	NULL	127	127	127	127	127	127	127	127	127	NULL	NULL	NULL	32767	32767	32767	32767	32767	32767	32767	32767	NULL	NULL	NULL	NULL	2147483647	2147483647	2147483647	2147483647	2147483647	2147483647	2147483647	NULL	NULL	NULL	NULL	NULL	9223372036854775807	9223372036854775807	9223372036854775807	9223372036854775807	9223372036854775807	9223372036854775807	NULL	new
-102	1	true	true	true	true	true	true	true	true	true	0	-1	-1	-1	0	0	1	127	127	127	-38	0	127	-1	-1	0	0	1	32767	32767	32767	7898	0	127	32767	-1	-2147483648	-2147483648	-1661992959	2147483647	2147483647	2147483647	1563893466	0	127	32767	2147483647	-9223372036854775808	-9223372036854775808	-7766279631452241919	9223372036854775807	9223372036854775807	9223372036854775807	126117945050	original
+102	1	true	true	true	true	true	true	true	true	true	0	-1	-1	-1	0	0	NULL	127	127	127	-38	0	127	-1	-1	0	0	NULL	32767	32767	32767	7898	0	127	32767	-1	-2147483648	-2147483648	NULL	2147483647	2147483647	2147483647	1563893466	0	127	32767	2147483647	-9223372036854775808	-9223372036854775808	NULL	9223372036854775807	9223372036854775807	9223372036854775807	126117945050	original
 103	1	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	new
 103	1	NULL	NULL	NULL	NULL	NULL	NULL	NULL	false	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	original
 104	1	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	23	23	23	23	23	23	23	23	23	NULL	NULL	NULL	834	834	834	834	834	834	834	834	NULL	NULL	NULL	NULL	203332	203332	203332	203332	203332	203332	203332	NULL	NULL	NULL	NULL	NULL	888888847499264	888888857923222	888888857923222	888888857923222	888888857923222	888888857923222	NULL	new
-104	1	true	true	true	true	true	true	true	true	true	NULL	66	68	-106	-100	30	-85	23	23	23	86	NULL	23	6724	3734	-100	30	939	834	834	834	-12970	NULL	23	834	-1868624234	-100	30	66475	203332	203332	203332	270912854	NULL	23	834	203332	-100	30	66475	888888857923222	888888857923222	888888857923222	270912854	original
+104	1	true	true	true	true	true	true	true	true	true	NULL	66	68	-106	-100	30	NULL	23	23	23	86	NULL	23	6724	3734	-100	30	NULL	834	834	834	-12970	NULL	23	834	-1868624234	-100	30	66475	203332	203332	203332	270912854	NULL	23	834	203332	-100	30	66475	888888857923222	888888857923222	888888857923222	270912854	original
 105	1	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	NULL	-99	-99	-99	-99	-99	-99	-99	-99	-99	NULL	NULL	NULL	-28300	-28300	-28300	-28300	-28300	-28300	-28300	-28300	NULL	NULL	NULL	NULL	-999992	-999992	-999992	-999992	-999992	-999992	-999992	NULL	NULL	NULL	NULL	NULL	-222282153984	-222282153733	-222282153733	-222282153733	-222282153733	-222282153733	NULL	new
-105	1	true	true	true	true	NULL	true	true	true	true	0	116	-56	-5	NULL	34	36	-99	-99	-99	-41	0	-99	-16952	-32517	NULL	-19422	9764	-28300	-28300	-28300	-16681	0	-99	-28300	1056145659	NULL	46114	9250340	-999992	-999992	-999992	663207639	0	-99	-28300	-999992	NULL	46114	9250340	-222282153733	-222282153733	-222282153733	663207639	original
+105	1	true	true	true	true	NULL	true	true	true	true	0	116	-56	-5	NULL	34	NULL	-99	-99	-99	-41	0	-99	-16952	-32517	NULL	-19422	NULL	-28300	-28300	-28300	-16681	0	-99	-28300	1056145659	NULL	46114	9250340	-999992	-999992	-999992	663207639	0	-99	-28300	-999992	NULL	46114	9250340	-222282153733	-222282153733	-222282153733	663207639	original
 PREHOOK: query: drop table part_change_various_various_boolean_to_bigint
 PREHOOK: type: DROPTABLE
 PREHOOK: Input: default@part_change_various_various_boolean_to_bigint

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/ql/src/test/results/clientpositive/llap/schema_evol_text_vecrow_part_all_primitive.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/schema_evol_text_vecrow_part_all_primitive.q.out b/ql/src/test/results/clientpositive/llap/schema_evol_text_vecrow_part_all_primitive.q.out
index 85116e7..41a61a8 100644
--- a/ql/src/test/results/clientpositive/llap/schema_evol_text_vecrow_part_all_primitive.q.out
+++ b/ql/src/test/results/clientpositive/llap/schema_evol_text_vecrow_part_all_primitive.q.out
@@ -314,15 +314,15 @@ POSTHOOK: Input: default@part_change_various_various_boolean_to_bigint@part=1
 #### A masked pattern was here ####
 insert_num	part	c1	c2	c3	c4	c5	c6	c7	c8	c9	c10	c11	c12	c13	c14	c15	c16	c17	c18	c19	c20	c21	c22	c23	c24	c25	c26	c27	c28	c29	c30	c31	c32	c33	c34	c35	c36	c37	c38	c39	c40	c41	c42	c43	c44	c45	c46	c47	c48	c49	c50	c51	c52	c53	b
 101	1	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	NULL	-128	-128	-128	-128	-128	-128	-128	-128	-128	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	-2147483648	-2147483648	-2147483648	-2147483648	-2147483648	-2147483648	-2147483648	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	new
-101	1	true	NULL	true	NULL	true	true	true	true	true	1	NULL	0	NULL	-1	-1	-1	-128	-128	-128	84	1	-128	0	NULL	-1	-1	-1	NULL	NULL	NULL	-8620	1	-128	NULL	NULL	2147483647	2147483647	1661992959	-2147483648	-2147483648	-2147483648	1272503892	1	-128	NULL	-2147483648	9223372036854775807	9223372036854775807	7766279631452241919	NULL	NULL	NULL	134416490068	original
+101	1	true	NULL	true	NULL	true	true	true	true	true	1	NULL	0	NULL	-1	-1	NULL	-128	-128	-128	84	1	-128	0	NULL	-1	-1	NULL	NULL	NULL	NULL	-8620	1	-128	NULL	NULL	2147483647	2147483647	NULL	-2147483648	-2147483648	-2147483648	1272503892	1	-128	NULL	-2147483648	9223372036854775807	9223372036854775807	NULL	NULL	NULL	NULL	134416490068	original
 102	1	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	NULL	127	127	127	127	127	127	127	127	127	NULL	NULL	NULL	32767	32767	32767	32767	32767	32767	32767	32767	NULL	NULL	NULL	NULL	2147483647	2147483647	2147483647	2147483647	2147483647	2147483647	2147483647	NULL	NULL	NULL	NULL	NULL	9223372036854775807	9223372036854775807	9223372036854775807	9223372036854775807	9223372036854775807	9223372036854775807	NULL	new
-102	1	true	true	true	true	true	true	true	true	true	0	-1	-1	-1	0	0	1	127	127	127	-38	0	127	-1	-1	0	0	1	32767	32767	32767	7898	0	127	32767	-1	-2147483648	-2147483648	-1661992959	2147483647	2147483647	2147483647	1563893466	0	127	32767	2147483647	-9223372036854775808	-9223372036854775808	-7766279631452241919	9223372036854775807	9223372036854775807	9223372036854775807	126117945050	original
+102	1	true	true	true	true	true	true	true	true	true	0	-1	-1	-1	0	0	NULL	127	127	127	-38	0	127	-1	-1	0	0	NULL	32767	32767	32767	7898	0	127	32767	-1	-2147483648	-2147483648	NULL	2147483647	2147483647	2147483647	1563893466	0	127	32767	2147483647	-9223372036854775808	-9223372036854775808	NULL	9223372036854775807	9223372036854775807	9223372036854775807	126117945050	original
 103	1	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	new
 103	1	NULL	NULL	NULL	NULL	NULL	NULL	NULL	false	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	NULL	original
 104	1	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	NULL	23	23	23	23	23	23	23	23	23	NULL	NULL	NULL	834	834	834	834	834	834	834	834	NULL	NULL	NULL	NULL	203332	203332	203332	203332	203332	203332	203332	NULL	NULL	NULL	NULL	NULL	888888847499264	888888857923222	888888857923222	888888857923222	888888857923222	888888857923222	NULL	new
-104	1	true	true	true	true	true	true	true	true	true	1	66	68	-106	-100	30	-85	23	23	23	86	1	23	6724	3734	-100	30	939	834	834	834	-12970	1	23	834	-1868624234	-100	30	66475	203332	203332	203332	270912854	1	23	834	203332	-100	30	66475	888888857923222	888888857923222	888888857923222	270912854	original
+104	1	true	true	true	true	true	true	true	true	true	1	66	68	-106	-100	30	NULL	23	23	23	86	1	23	6724	3734	-100	30	NULL	834	834	834	-12970	1	23	834	-1868624234	-100	30	66475	203332	203332	203332	270912854	1	23	834	203332	-100	30	66475	888888857923222	888888857923222	888888857923222	270912854	original
 105	1	NULL	NULL	NULL	NULL	NULL	NULL	NULL	true	NULL	NULL	-99	-99	-99	-99	-99	-99	-99	-99	-99	NULL	NULL	NULL	-28300	-28300	-28300	-28300	-28300	-28300	-28300	-28300	NULL	NULL	NULL	NULL	-999992	-999992	-999992	-999992	-999992	-999992	-999992	NULL	NULL	NULL	NULL	NULL	-222282153984	-222282153733	-222282153733	-222282153733	-222282153733	-222282153733	NULL	new
-105	1	true	true	true	true	NULL	true	true	true	true	0	116	-56	-5	NULL	34	36	-99	-99	-99	-41	0	-99	-16952	-32517	NULL	-19422	9764	-28300	-28300	-28300	-16681	0	-99	-28300	1056145659	NULL	46114	9250340	-999992	-999992	-999992	663207639	0	-99	-28300	-999992	NULL	46114	9250340	-222282153733	-222282153733	-222282153733	663207639	original
+105	1	true	true	true	true	NULL	true	true	true	true	0	116	-56	-5	NULL	34	NULL	-99	-99	-99	-41	0	-99	-16952	-32517	NULL	-19422	NULL	-28300	-28300	-28300	-16681	0	-99	-28300	1056145659	NULL	46114	9250340	-999992	-999992	-999992	663207639	0	-99	-28300	-999992	NULL	46114	9250340	-222282153733	-222282153733	-222282153733	663207639	original
 PREHOOK: query: drop table part_change_various_various_boolean_to_bigint
 PREHOOK: type: DROPTABLE
 PREHOOK: Input: default@part_change_various_various_boolean_to_bigint

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/ql/src/test/results/clientpositive/llap/vector_decimal_2.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/vector_decimal_2.q.out b/ql/src/test/results/clientpositive/llap/vector_decimal_2.q.out
index ef1e561..db5e183 100644
--- a/ql/src/test/results/clientpositive/llap/vector_decimal_2.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_decimal_2.q.out
@@ -661,7 +661,7 @@ POSTHOOK: query: select cast(t as tinyint) from decimal_2 order by t
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@decimal_2
 #### A masked pattern was here ####
-13
+NULL
 PREHOOK: query: explain
 select cast(t as smallint) from decimal_2 order by t
 PREHOOK: type: QUERY
@@ -724,7 +724,7 @@ POSTHOOK: query: select cast(t as smallint) from decimal_2 order by t
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@decimal_2
 #### A masked pattern was here ####
--3827
+NULL
 PREHOOK: query: explain
 select cast(t as int) from decimal_2 order by t
 PREHOOK: type: QUERY

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/ql/src/test/results/clientpositive/llap/vector_decimal_expressions.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/llap/vector_decimal_expressions.q.out b/ql/src/test/results/clientpositive/llap/vector_decimal_expressions.q.out
index cf264e8..b37e30b 100644
--- a/ql/src/test/results/clientpositive/llap/vector_decimal_expressions.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_decimal_expressions.q.out
@@ -90,13 +90,13 @@ LIMIT 10
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@decimal_test
 #### A masked pattern was here ####
-1836.44199584197700	-1166.02723492725400	0.8372697814834	245972.55810810255804469	5.6189189189	835	1000	-24	835	true	1000.823076923077	835.6189	1000.823076923077	1969-12-31 16:13:55.618918918
-1856.13222453224620	-1178.52931392929240	0.8372449787014	251275.44324324968747899	4.5783783784	844	1011	-13	844	true	1011.5538461538462	844.57837	1011.5538461538462	1969-12-31 16:14:04.578378378
-1858.75758835761550	-1180.19625779623100	0.8372417113669	251986.76756757564861519	5.7729729730	845	1012	-12	845	true	1012.9846153846155	845.77295	1012.9846153846155	1969-12-31 16:14:05.772972973
-1862.69563409566930	-1182.69667359663860	0.8372368276345	253055.63918919969667286	7.5648648649	847	1015	-9	847	true	1015.1307692307693	847.5649	1015.1307692307693	1969-12-31 16:14:07.564864864
-1883.69854469852330	-1196.03222453224660	0.8372111259286	258794.49324323677116559	7.1216216216	857	1026	2	857	true	1026.5769230769233	857.12164	1026.5769230769233	1969-12-31 16:14:17.121621621
-1886.32390852389240	-1197.69916839918480	0.8372079534582	259516.37432431944456816	8.3162162162	858	1028	4	858	true	1028.0076923076924	858.3162	1028.0076923076924	1969-12-31 16:14:18.316216216
-1887.63659043657700	-1198.53264033265400	0.8372063705322	259877.69189188782259834	8.9135135135	858	1028	4	858	true	1028.723076923077	858.9135	1028.723076923077	1969-12-31 16:14:18.913513513
-1895.51268191268460	-1203.53347193346920	0.8371969190171	262050.87567567649292835	2.4972972973	862	1033	9	862	true	1033.0153846153846	862.4973	1033.0153846153846	1969-12-31 16:14:22.497297297
-1909.95218295221550	-1212.70166320163100	0.8371797936946	266058.54729730725574014	9.0675675676	869	1040	16	869	true	1040.8846153846155	869.06757	1040.8846153846155	1969-12-31 16:14:29.067567567
-1913.89022869026920	-1215.20207900203840	0.8371751679996	267156.82702703945592392	0.8594594595	870	1043	19	870	true	1043.0307692307692	870.85944	1043.0307692307692	1969-12-31 16:14:30.859459459
+1836.44199584197700	-1166.02723492725400	0.8372697814834	245972.55810810255804469	5.6189189189	835	1000	NULL	835	true	1000.823076923077	835.6189	1000.823076923077	1969-12-31 16:13:55.618918918
+1856.13222453224620	-1178.52931392929240	0.8372449787014	251275.44324324968747899	4.5783783784	844	1011	NULL	844	true	1011.5538461538462	844.57837	1011.5538461538462	1969-12-31 16:14:04.578378378
+1858.75758835761550	-1180.19625779623100	0.8372417113669	251986.76756757564861519	5.7729729730	845	1012	NULL	845	true	1012.9846153846155	845.77295	1012.9846153846155	1969-12-31 16:14:05.772972973
+1862.69563409566930	-1182.69667359663860	0.8372368276345	253055.63918919969667286	7.5648648649	847	1015	NULL	847	true	1015.1307692307693	847.5649	1015.1307692307693	1969-12-31 16:14:07.564864864
+1883.69854469852330	-1196.03222453224660	0.8372111259286	258794.49324323677116559	7.1216216216	857	1026	NULL	857	true	1026.5769230769233	857.12164	1026.5769230769233	1969-12-31 16:14:17.121621621
+1886.32390852389240	-1197.69916839918480	0.8372079534582	259516.37432431944456816	8.3162162162	858	1028	NULL	858	true	1028.0076923076924	858.3162	1028.0076923076924	1969-12-31 16:14:18.316216216
+1887.63659043657700	-1198.53264033265400	0.8372063705322	259877.69189188782259834	8.9135135135	858	1028	NULL	858	true	1028.723076923077	858.9135	1028.723076923077	1969-12-31 16:14:18.913513513
+1895.51268191268460	-1203.53347193346920	0.8371969190171	262050.87567567649292835	2.4972972973	862	1033	NULL	862	true	1033.0153846153846	862.4973	1033.0153846153846	1969-12-31 16:14:22.497297297
+1909.95218295221550	-1212.70166320163100	0.8371797936946	266058.54729730725574014	9.0675675676	869	1040	NULL	869	true	1040.8846153846155	869.06757	1040.8846153846155	1969-12-31 16:14:29.067567567
+1913.89022869026920	-1215.20207900203840	0.8371751679996	267156.82702703945592392	0.8594594595	870	1043	NULL	870	true	1043.0307692307692	870.85944	1043.0307692307692	1969-12-31 16:14:30.859459459

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/ql/src/test/results/clientpositive/vector_decimal_expressions.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/vector_decimal_expressions.q.out b/ql/src/test/results/clientpositive/vector_decimal_expressions.q.out
index 895116d..316dabc 100644
--- a/ql/src/test/results/clientpositive/vector_decimal_expressions.q.out
+++ b/ql/src/test/results/clientpositive/vector_decimal_expressions.q.out
@@ -81,13 +81,13 @@ LIMIT 10
 POSTHOOK: type: QUERY
 POSTHOOK: Input: default@decimal_test
 #### A masked pattern was here ####
-1836.44199584197700	-1166.02723492725400	0.8372697814834	245972.55810810255804469	5.6189189189	835	1000	-24	835	true	1000.823076923077	835.6189	1000.823076923077	1969-12-31 16:13:55.618918918
-1856.13222453224620	-1178.52931392929240	0.8372449787014	251275.44324324968747899	4.5783783784	844	1011	-13	844	true	1011.5538461538462	844.57837	1011.5538461538462	1969-12-31 16:14:04.578378378
-1858.75758835761550	-1180.19625779623100	0.8372417113669	251986.76756757564861519	5.7729729730	845	1012	-12	845	true	1012.9846153846155	845.77295	1012.9846153846155	1969-12-31 16:14:05.772972973
-1862.69563409566930	-1182.69667359663860	0.8372368276345	253055.63918919969667286	7.5648648649	847	1015	-9	847	true	1015.1307692307693	847.5649	1015.1307692307693	1969-12-31 16:14:07.564864864
-1883.69854469852330	-1196.03222453224660	0.8372111259286	258794.49324323677116559	7.1216216216	857	1026	2	857	true	1026.5769230769233	857.12164	1026.5769230769233	1969-12-31 16:14:17.121621621
-1886.32390852389240	-1197.69916839918480	0.8372079534582	259516.37432431944456816	8.3162162162	858	1028	4	858	true	1028.0076923076924	858.3162	1028.0076923076924	1969-12-31 16:14:18.316216216
-1887.63659043657700	-1198.53264033265400	0.8372063705322	259877.69189188782259834	8.9135135135	858	1028	4	858	true	1028.723076923077	858.9135	1028.723076923077	1969-12-31 16:14:18.913513513
-1895.51268191268460	-1203.53347193346920	0.8371969190171	262050.87567567649292835	2.4972972973	862	1033	9	862	true	1033.0153846153846	862.4973	1033.0153846153846	1969-12-31 16:14:22.497297297
-1909.95218295221550	-1212.70166320163100	0.8371797936946	266058.54729730725574014	9.0675675676	869	1040	16	869	true	1040.8846153846155	869.06757	1040.8846153846155	1969-12-31 16:14:29.067567567
-1913.89022869026920	-1215.20207900203840	0.8371751679996	267156.82702703945592392	0.8594594595	870	1043	19	870	true	1043.0307692307692	870.85944	1043.0307692307692	1969-12-31 16:14:30.859459459
+1836.44199584197700	-1166.02723492725400	0.8372697814834	245972.55810810255804469	5.6189189189	835	1000	NULL	835	true	1000.823076923077	835.6189	1000.823076923077	1969-12-31 16:13:55.618918918
+1856.13222453224620	-1178.52931392929240	0.8372449787014	251275.44324324968747899	4.5783783784	844	1011	NULL	844	true	1011.5538461538462	844.57837	1011.5538461538462	1969-12-31 16:14:04.578378378
+1858.75758835761550	-1180.19625779623100	0.8372417113669	251986.76756757564861519	5.7729729730	845	1012	NULL	845	true	1012.9846153846155	845.77295	1012.9846153846155	1969-12-31 16:14:05.772972973
+1862.69563409566930	-1182.69667359663860	0.8372368276345	253055.63918919969667286	7.5648648649	847	1015	NULL	847	true	1015.1307692307693	847.5649	1015.1307692307693	1969-12-31 16:14:07.564864864
+1883.69854469852330	-1196.03222453224660	0.8372111259286	258794.49324323677116559	7.1216216216	857	1026	NULL	857	true	1026.5769230769233	857.12164	1026.5769230769233	1969-12-31 16:14:17.121621621
+1886.32390852389240	-1197.69916839918480	0.8372079534582	259516.37432431944456816	8.3162162162	858	1028	NULL	858	true	1028.0076923076924	858.3162	1028.0076923076924	1969-12-31 16:14:18.316216216
+1887.63659043657700	-1198.53264033265400	0.8372063705322	259877.69189188782259834	8.9135135135	858	1028	NULL	858	true	1028.723076923077	858.9135	1028.723076923077	1969-12-31 16:14:18.913513513
+1895.51268191268460	-1203.53347193346920	0.8371969190171	262050.87567567649292835	2.4972972973	862	1033	NULL	862	true	1033.0153846153846	862.4973	1033.0153846153846	1969-12-31 16:14:22.497297297
+1909.95218295221550	-1212.70166320163100	0.8371797936946	266058.54729730725574014	9.0675675676	869	1040	NULL	869	true	1040.8846153846155	869.06757	1040.8846153846155	1969-12-31 16:14:29.067567567
+1913.89022869026920	-1215.20207900203840	0.8371751679996	267156.82702703945592392	0.8594594595	870	1043	NULL	870	true	1043.0307692307692	870.85944	1043.0307692307692	1969-12-31 16:14:30.859459459

http://git-wip-us.apache.org/repos/asf/hive/blob/4ba713cc/serde/src/java/org/apache/hadoop/hive/serde2/avro/AvroSerdeUtils.java
----------------------------------------------------------------------
diff --git a/serde/src/java/org/apache/hadoop/hive/serde2/avro/AvroSerdeUtils.java b/serde/src/java/org/apache/hadoop/hive/serde2/avro/AvroSerdeUtils.java
index 3a539b2..b6b23c2 100644
--- a/serde/src/java/org/apache/hadoop/hive/serde2/avro/AvroSerdeUtils.java
+++ b/serde/src/java/org/apache/hadoop/hive/serde2/avro/AvroSerdeUtils.java
@@ -241,8 +241,9 @@ public class AvroSerdeUtils {
       return null;
     }
 
-    dec = dec.setScale(scale);
-    return AvroSerdeUtils.getBufferFromBytes(dec.unscaledValue().toByteArray());
+    // NOTE: Previously, we did OldHiveDecimal.setScale(scale), called OldHiveDecimal
+    //       unscaledValue().toByteArray().
+    return AvroSerdeUtils.getBufferFromBytes(dec.bigIntegerBytesScaled(scale));
   }
 
   public static byte[] getBytesFromByteBuffer(ByteBuffer byteBuffer) {