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:32 UTC

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

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