You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by sm...@apache.org on 2016/08/19 21:29:08 UTC

arrow git commit: ARROW-265: Pad negative decimal values with1

Repository: arrow
Updated Branches:
  refs/heads/master fd2e52491 -> 282fcacc8


ARROW-265: Pad negative decimal values with1


Project: http://git-wip-us.apache.org/repos/asf/arrow/repo
Commit: http://git-wip-us.apache.org/repos/asf/arrow/commit/282fcacc
Tree: http://git-wip-us.apache.org/repos/asf/arrow/tree/282fcacc
Diff: http://git-wip-us.apache.org/repos/asf/arrow/diff/282fcacc

Branch: refs/heads/master
Commit: 282fcacc86c9232c9dc1b1030e9fc9299bbc3f8d
Parents: fd2e524
Author: Steven Phillips <st...@dremio.com>
Authored: Fri Aug 19 14:28:05 2016 -0700
Committer: Steven Phillips <st...@dremio.com>
Committed: Fri Aug 19 14:28:05 2016 -0700

----------------------------------------------------------------------
 .../codegen/templates/FixedValueVectors.java    |   8 +-
 .../codegen/templates/HolderReaderImpl.java     |   5 +-
 .../arrow/vector/util/DecimalUtility.java       | 579 +------------------
 .../apache/arrow/vector/TestDecimalVector.java  |   7 +-
 4 files changed, 27 insertions(+), 572 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/arrow/blob/282fcacc/java/vector/src/main/codegen/templates/FixedValueVectors.java
----------------------------------------------------------------------
diff --git a/java/vector/src/main/codegen/templates/FixedValueVectors.java b/java/vector/src/main/codegen/templates/FixedValueVectors.java
index fe2b5c5..37946f6 100644
--- a/java/vector/src/main/codegen/templates/FixedValueVectors.java
+++ b/java/vector/src/main/codegen/templates/FixedValueVectors.java
@@ -16,6 +16,8 @@
  * limitations under the License.
  */
 
+import org.apache.arrow.vector.util.DecimalUtility;
+
 import java.lang.Override;
 
 <@pp.dropOutputFile />
@@ -444,11 +446,7 @@ public final class ${minor.class}Vector extends BaseDataValueVector implements F
 
     @Override
     public ${friendlyType} getObject(int index) {
-      byte[] bytes = new byte[${type.width}];
-      int start = ${type.width} * index;
-      data.getBytes(start, bytes, 0, ${type.width});
-      ${friendlyType} value = new BigDecimal(new BigInteger(bytes), scale);
-      return value;
+      return org.apache.arrow.vector.util.DecimalUtility.getBigDecimalFromArrowBuf(data, index, scale);
     }
 
     <#else>

http://git-wip-us.apache.org/repos/asf/arrow/blob/282fcacc/java/vector/src/main/codegen/templates/HolderReaderImpl.java
----------------------------------------------------------------------
diff --git a/java/vector/src/main/codegen/templates/HolderReaderImpl.java b/java/vector/src/main/codegen/templates/HolderReaderImpl.java
index 1ed9287..d66577b 100644
--- a/java/vector/src/main/codegen/templates/HolderReaderImpl.java
+++ b/java/vector/src/main/codegen/templates/HolderReaderImpl.java
@@ -156,9 +156,11 @@ public class ${holderMode}${name}HolderReaderImpl extends AbstractFieldReader {
 </#if>
 
 <#if type.major == "VarLen">
+      <#if minor.class != "Decimal">
       int length = holder.end - holder.start;
       byte[] value = new byte [length];
       holder.buffer.getBytes(holder.start, value, 0, length);
+      </#if>
 
 <#if minor.class == "VarBinary">
       return value;
@@ -169,8 +171,7 @@ public class ${holderMode}${name}HolderReaderImpl extends AbstractFieldReader {
       text.set(value);
       return text;
 <#elseif minor.class == "Decimal" >
-        return new BigDecimal(new BigInteger(value), holder.scale);
-
+      return org.apache.arrow.vector.util.DecimalUtility.getBigDecimalFromArrowBuf(holder.buffer, holder.start, holder.scale);
 </#if>
 
 <#elseif minor.class == "Interval">

http://git-wip-us.apache.org/repos/asf/arrow/blob/282fcacc/java/vector/src/main/java/org/apache/arrow/vector/util/DecimalUtility.java
----------------------------------------------------------------------
diff --git a/java/vector/src/main/java/org/apache/arrow/vector/util/DecimalUtility.java b/java/vector/src/main/java/org/apache/arrow/vector/util/DecimalUtility.java
index 4eb0d9f..e171e87 100644
--- a/java/vector/src/main/java/org/apache/arrow/vector/util/DecimalUtility.java
+++ b/java/vector/src/main/java/org/apache/arrow/vector/util/DecimalUtility.java
@@ -66,6 +66,8 @@ public class DecimalUtility {
           100000000000000000l,
           1000000000000000000l};
 
+  public static final int DECIMAL_BYTE_LENGTH = 16;
+
   /*
    * Simple function that returns the static precomputed
    * power of ten, instead of using Math.pow
@@ -100,14 +102,6 @@ public class DecimalUtility {
     }
   }
 
-  /* Given the number of actual digits this function returns the
-   * number of indexes it will occupy in the array of integers
-   * which are stored in base 1 billion
-   */
-  public static int roundUp(int ndigits) {
-    return (ndigits + MAX_DIGITS - 1)/MAX_DIGITS;
-  }
-
   /* Returns a string representation of the given integer
    * If the length of the given integer is less than the
    * passed length, this function will prepend zeroes to the string
@@ -141,572 +135,33 @@ public class DecimalUtility {
     return str;
   }
 
-  public static BigDecimal getBigDecimalFromIntermediate(ByteBuf data, int startIndex, int nDecimalDigits, int scale) {
-
-    // In the intermediate representation we don't pad the scale with zeroes, so set truncate = false
-    return getBigDecimalFromArrowBuf(data, startIndex, nDecimalDigits, scale, false);
-  }
-
-  public static BigDecimal getBigDecimalFromSparse(ArrowBuf data, int startIndex, int nDecimalDigits, int scale) {
-
-    // In the sparse representation we pad the scale with zeroes for ease of arithmetic, need to truncate
-    return getBigDecimalFromArrowBuf(data, startIndex, nDecimalDigits, scale, true);
-  }
-
-  public static BigDecimal getBigDecimalFromArrowBuf(ArrowBuf bytebuf, int start, int length, int scale) {
-    byte[] value = new byte[length];
-    bytebuf.getBytes(start, value, 0, length);
+  public static BigDecimal getBigDecimalFromArrowBuf(ArrowBuf bytebuf, int index, int scale) {
+    byte[] value = new byte[DECIMAL_BYTE_LENGTH];
+    final int startIndex = index * DECIMAL_BYTE_LENGTH;
+    bytebuf.getBytes(startIndex, value, 0, DECIMAL_BYTE_LENGTH);
     BigInteger unscaledValue = new BigInteger(value);
     return new BigDecimal(unscaledValue, scale);
   }
 
-  public static BigDecimal getBigDecimalFromByteBuffer(ByteBuffer bytebuf, int start, int length, int scale) {
-    byte[] value = new byte[length];
+  public static BigDecimal getBigDecimalFromByteBuffer(ByteBuffer bytebuf, int start, int scale) {
+    byte[] value = new byte[DECIMAL_BYTE_LENGTH];
     bytebuf.get(value);
     BigInteger unscaledValue = new BigInteger(value);
     return new BigDecimal(unscaledValue, scale);
   }
 
-  public static void writeBigDecimalToArrowBuf(ArrowBuf bytebuf, int startIndex, BigDecimal value) {
-    byte[] bytes = value.unscaledValue().toByteArray();
-    if (bytes.length > 16) {
+  public static void writeBigDecimalToArrowBuf(BigDecimal value, ArrowBuf bytebuf, int index) {
+    final byte[] bytes = value.unscaledValue().toByteArray();
+    final int startIndex = index * DECIMAL_BYTE_LENGTH;
+    if (bytes.length > DECIMAL_BYTE_LENGTH) {
       throw new UnsupportedOperationException("Decimal size greater than 16 bytes");
     }
-    bytebuf.setBytes(startIndex + 16 - bytes.length, bytes, 0, bytes.length);
-  }
-
-  /* Create a BigDecimal object using the data in the ArrowBuf.
-   * This function assumes that data is provided in a non-dense format
-   * It works on both sparse and intermediate representations.
-   */
-  public static BigDecimal getBigDecimalFromArrowBuf(ByteBuf data, int startIndex, int nDecimalDigits, int scale,
-                                                     boolean truncateScale) {
-
-    // For sparse decimal type we have padded zeroes at the end, strip them while converting to BigDecimal.
-    int actualDigits;
-
-    // Initialize the BigDecimal, first digit in the ArrowBuf has the sign so mask it out
-    BigInteger decimalDigits = BigInteger.valueOf((data.getInt(startIndex)) & 0x7FFFFFFF);
-
-    BigInteger base = BigInteger.valueOf(DIGITS_BASE);
-
-    for (int i = 1; i < nDecimalDigits; i++) {
-
-      BigInteger temp = BigInteger.valueOf(data.getInt(startIndex + (i * INTEGER_SIZE)));
-      decimalDigits = decimalDigits.multiply(base);
-      decimalDigits = decimalDigits.add(temp);
-    }
-
-    // Truncate any additional padding we might have added
-    if (truncateScale == true && scale > 0 && (actualDigits = scale % MAX_DIGITS) != 0) {
-      BigInteger truncate = BigInteger.valueOf((int)Math.pow(10, (MAX_DIGITS - actualDigits)));
-      decimalDigits = decimalDigits.divide(truncate);
-    }
-
-    // set the sign
-    if ((data.getInt(startIndex) & 0x80000000) != 0) {
-      decimalDigits = decimalDigits.negate();
+    final int padLength = DECIMAL_BYTE_LENGTH - bytes.length;
+    final int padValue = value.signum() == -1 ? 0xFF : 0;
+    for (int i = 0; i < padLength; i++) {
+      bytebuf.setByte(startIndex + i, padValue);
     }
-
-    BigDecimal decimal = new BigDecimal(decimalDigits, scale);
-
-    return decimal;
-  }
-
-  /* This function returns a BigDecimal object from the dense decimal representation.
-   * First step is to convert the dense representation into an intermediate representation
-   * and then invoke getBigDecimalFromArrowBuf() to get the BigDecimal object
-   */
-  public static BigDecimal getBigDecimalFromDense(ArrowBuf data, int startIndex, int nDecimalDigits, int scale, int maxPrecision, int width) {
-
-        /* This method converts the dense representation to
-         * an intermediate representation. The intermediate
-         * representation has one more integer than the dense
-         * representation.
-         */
-    byte[] intermediateBytes = new byte[((nDecimalDigits + 1) * INTEGER_SIZE)];
-
-    // Start storing from the least significant byte of the first integer
-    int intermediateIndex = 3;
-
-    int[] mask = {0x03, 0x0F, 0x3F, 0xFF};
-    int[] reverseMask = {0xFC, 0xF0, 0xC0, 0x00};
-
-    int maskIndex;
-    int shiftOrder;
-    byte shiftBits;
-
-    // TODO: Some of the logic here is common with casting from Dense to Sparse types, factor out common code
-    if (maxPrecision == 38) {
-      maskIndex = 0;
-      shiftOrder = 6;
-      shiftBits = 0x00;
-      intermediateBytes[intermediateIndex++] = (byte) (data.getByte(startIndex) & 0x7F);
-    } else if (maxPrecision == 28) {
-      maskIndex = 1;
-      shiftOrder = 4;
-      shiftBits = (byte) ((data.getByte(startIndex) & 0x03) << shiftOrder);
-      intermediateBytes[intermediateIndex++] = (byte) (((data.getByte(startIndex) & 0x3C) & 0xFF) >>> 2);
-    } else {
-      throw new UnsupportedOperationException("Dense types with max precision 38 and 28 are only supported");
-    }
-
-    int inputIndex = 1;
-    boolean sign = false;
-
-    if ((data.getByte(startIndex) & 0x80) != 0) {
-      sign = true;
-    }
-
-    while (inputIndex < width) {
-
-      intermediateBytes[intermediateIndex] = (byte) ((shiftBits) | (((data.getByte(startIndex + inputIndex) & reverseMask[maskIndex]) & 0xFF) >>> (8 - shiftOrder)));
-
-      shiftBits = (byte) ((data.getByte(startIndex + inputIndex) & mask[maskIndex]) << shiftOrder);
-
-      inputIndex++;
-      intermediateIndex++;
-
-      if (((inputIndex - 1) % INTEGER_SIZE) == 0) {
-        shiftBits = (byte) ((shiftBits & 0xFF) >>> 2);
-        maskIndex++;
-        shiftOrder -= 2;
-      }
-
-    }
-        /* copy the last byte */
-    intermediateBytes[intermediateIndex] = shiftBits;
-
-    if (sign == true) {
-      intermediateBytes[0] = (byte) (intermediateBytes[0] | 0x80);
-    }
-
-    final ByteBuf intermediate = UnpooledByteBufAllocator.DEFAULT.buffer(intermediateBytes.length);
-    try {
-      intermediate.setBytes(0, intermediateBytes);
-
-      BigDecimal ret = getBigDecimalFromIntermediate(intermediate, 0, nDecimalDigits + 1, scale);
-      return ret;
-    } finally {
-      intermediate.release();
-    }
-
-  }
-
- public static void getSparseFromBigDecimal(BigDecimal input, ByteBuf data, int startIndex, int scale, int precision,
-                                             int nDecimalDigits) {
-
-    // Initialize the buffer
-    for (int i = 0; i < nDecimalDigits; i++) {
-      data.setInt(startIndex + (i * INTEGER_SIZE), 0);
-    }
-
-    boolean sign = false;
-
-    if (input.signum() == -1) {
-      // negative input
-      sign = true;
-      input = input.abs();
-    }
-
-    // Truncate the input as per the scale provided
-    input = input.setScale(scale, BigDecimal.ROUND_HALF_UP);
-
-    // Separate out the integer part
-    BigDecimal integerPart = input.setScale(0, BigDecimal.ROUND_DOWN);
-
-    int destIndex = nDecimalDigits - roundUp(scale) - 1;
-
-    // we use base 1 billion integer digits for out integernal representation
-    BigDecimal base = new BigDecimal(DIGITS_BASE);
-
-    while (integerPart.compareTo(BigDecimal.ZERO) == 1) {
-      // store the modulo as the integer value
-      data.setInt(startIndex + (destIndex * INTEGER_SIZE), (integerPart.remainder(base)).intValue());
-      destIndex--;
-      // Divide by base 1 billion
-      integerPart = (integerPart.divide(base)).setScale(0, BigDecimal.ROUND_DOWN);
-    }
-
-        /* Sparse representation contains padding of additional zeroes
-         * so each digit contains MAX_DIGITS for ease of arithmetic
-         */
-    int actualDigits;
-    if ((actualDigits = (scale % MAX_DIGITS)) != 0) {
-      // Pad additional zeroes
-      scale = scale + (MAX_DIGITS - actualDigits);
-      input = input.setScale(scale, BigDecimal.ROUND_DOWN);
-    }
-
-    //separate out the fractional part
-    BigDecimal fractionalPart = input.remainder(BigDecimal.ONE).movePointRight(scale);
-
-    destIndex = nDecimalDigits - 1;
-
-    while (scale > 0) {
-      // Get next set of MAX_DIGITS (9) store it in the ArrowBuf
-      fractionalPart = fractionalPart.movePointLeft(MAX_DIGITS);
-      BigDecimal temp = fractionalPart.remainder(BigDecimal.ONE);
-
-      data.setInt(startIndex + (destIndex * INTEGER_SIZE), (temp.unscaledValue().intValue()));
-      destIndex--;
-
-      fractionalPart = fractionalPart.setScale(0, BigDecimal.ROUND_DOWN);
-      scale -= MAX_DIGITS;
-    }
-
-    // Set the negative sign
-    if (sign == true) {
-      data.setInt(startIndex, data.getInt(startIndex) | 0x80000000);
-    }
-
-  }
-
-
-  public static long getDecimal18FromBigDecimal(BigDecimal input, int scale, int precision) {
-    // Truncate or pad to set the input to the correct scale
-    input = input.setScale(scale, BigDecimal.ROUND_HALF_UP);
-
-    return (input.unscaledValue().longValue());
-  }
-
-  public static BigDecimal getBigDecimalFromPrimitiveTypes(int input, int scale, int precision) {
-    return BigDecimal.valueOf(input, scale);
-  }
-
-  public static BigDecimal getBigDecimalFromPrimitiveTypes(long input, int scale, int precision) {
-    return BigDecimal.valueOf(input, scale);
-  }
-
-
-  public static int compareDenseBytes(ArrowBuf left, int leftStart, boolean leftSign, ArrowBuf right, int rightStart, boolean rightSign, int width) {
-
-    int invert = 1;
-
-      /* If signs are different then simply look at the
-       * sign of the two inputs and determine which is greater
-       */
-    if (leftSign != rightSign) {
-
-      return((leftSign == true) ? -1 : 1);
-    } else if(leftSign == true) {
-        /* Both inputs are negative, at the end we will
-         * have to invert the comparison
-         */
-      invert = -1;
-    }
-
-    int cmp = 0;
-
-    for (int i = 0; i < width; i++) {
-      byte leftByte  = left.getByte(leftStart + i);
-      byte rightByte = right.getByte(rightStart + i);
-      // Unsigned byte comparison
-      if ((leftByte & 0xFF) > (rightByte & 0xFF)) {
-        cmp = 1;
-        break;
-      } else if ((leftByte & 0xFF) < (rightByte & 0xFF)) {
-        cmp = -1;
-        break;
-      }
-    }
-    cmp *= invert; // invert the comparison if both were negative values
-
-    return cmp;
-  }
-
-  public static int getIntegerFromSparseBuffer(ArrowBuf buffer, int start, int index) {
-    int value = buffer.getInt(start + (index * 4));
-
-    if (index == 0) {
-        /* the first byte contains sign bit, return value without it */
-      value = (value & 0x7FFFFFFF);
-    }
-    return value;
-  }
-
-  public static void setInteger(ArrowBuf buffer, int start, int index, int value) {
-    buffer.setInt(start + (index * 4), value);
-  }
-
-  public static int compareSparseBytes(ArrowBuf left, int leftStart, boolean leftSign, int leftScale, int leftPrecision, ArrowBuf right, int rightStart, boolean rightSign, int rightPrecision, int rightScale, int width, int nDecimalDigits, boolean absCompare) {
-
-    int invert = 1;
-
-    if (absCompare == false) {
-      if (leftSign != rightSign) {
-        return (leftSign == true) ? -1 : 1;
-      }
-
-      // Both values are negative invert the outcome of the comparison
-      if (leftSign == true) {
-        invert = -1;
-      }
-    }
-
-    int cmp = compareSparseBytesInner(left, leftStart, leftSign, leftScale, leftPrecision, right, rightStart, rightSign, rightPrecision, rightScale, width, nDecimalDigits);
-    return cmp * invert;
-  }
-  public static int compareSparseBytesInner(ArrowBuf left, int leftStart, boolean leftSign, int leftScale, int leftPrecision, ArrowBuf right, int rightStart, boolean rightSign, int rightPrecision, int rightScale, int width, int nDecimalDigits) {
-      /* compute the number of integer digits in each decimal */
-    int leftInt  = leftPrecision - leftScale;
-    int rightInt = rightPrecision - rightScale;
-
-      /* compute the number of indexes required for storing integer digits */
-    int leftIntRoundedUp = org.apache.arrow.vector.util.DecimalUtility.roundUp(leftInt);
-    int rightIntRoundedUp = org.apache.arrow.vector.util.DecimalUtility.roundUp(rightInt);
-
-      /* compute number of indexes required for storing scale */
-    int leftScaleRoundedUp = org.apache.arrow.vector.util.DecimalUtility.roundUp(leftScale);
-    int rightScaleRoundedUp = org.apache.arrow.vector.util.DecimalUtility.roundUp(rightScale);
-
-      /* compute index of the most significant integer digits */
-    int leftIndex1 = nDecimalDigits - leftScaleRoundedUp - leftIntRoundedUp;
-    int rightIndex1 = nDecimalDigits - rightScaleRoundedUp - rightIntRoundedUp;
-
-    int leftStopIndex = nDecimalDigits - leftScaleRoundedUp;
-    int rightStopIndex = nDecimalDigits - rightScaleRoundedUp;
-
-      /* Discard the zeroes in the integer part */
-    while (leftIndex1 < leftStopIndex) {
-      if (getIntegerFromSparseBuffer(left, leftStart, leftIndex1) != 0) {
-        break;
-      }
-
-        /* Digit in this location is zero, decrement the actual number
-         * of integer digits
-         */
-      leftIntRoundedUp--;
-      leftIndex1++;
-    }
-
-      /* If we reached the stop index then the number of integers is zero */
-    if (leftIndex1 == leftStopIndex) {
-      leftIntRoundedUp = 0;
-    }
-
-    while (rightIndex1 < rightStopIndex) {
-      if (getIntegerFromSparseBuffer(right, rightStart, rightIndex1) != 0) {
-        break;
-      }
-
-        /* Digit in this location is zero, decrement the actual number
-         * of integer digits
-         */
-      rightIntRoundedUp--;
-      rightIndex1++;
-    }
-
-    if (rightIndex1 == rightStopIndex) {
-      rightIntRoundedUp = 0;
-    }
-
-      /* We have the accurate number of non-zero integer digits,
-       * if the number of integer digits are different then we can determine
-       * which decimal is larger and needn't go down to comparing individual values
-       */
-    if (leftIntRoundedUp > rightIntRoundedUp) {
-      return 1;
-    }
-    else if (rightIntRoundedUp > leftIntRoundedUp) {
-      return -1;
-    }
-
-      /* The number of integer digits are the same, set the each index
-       * to the first non-zero integer and compare each digit
-       */
-    leftIndex1 = nDecimalDigits - leftScaleRoundedUp - leftIntRoundedUp;
-    rightIndex1 = nDecimalDigits - rightScaleRoundedUp - rightIntRoundedUp;
-
-    while (leftIndex1 < leftStopIndex && rightIndex1 < rightStopIndex) {
-      if (getIntegerFromSparseBuffer(left, leftStart, leftIndex1) > getIntegerFromSparseBuffer(right, rightStart, rightIndex1)) {
-        return 1;
-      }
-      else if (getIntegerFromSparseBuffer(right, rightStart, rightIndex1) > getIntegerFromSparseBuffer(left, leftStart, leftIndex1)) {
-        return -1;
-      }
-
-      leftIndex1++;
-      rightIndex1++;
-    }
-
-      /* The integer part of both the decimal's are equal, now compare
-       * each individual fractional part. Set the index to be at the
-       * beginning of the fractional part
-       */
-    leftIndex1 = leftStopIndex;
-    rightIndex1 = rightStopIndex;
-
-      /* Stop indexes will be the end of the array */
-    leftStopIndex = nDecimalDigits;
-    rightStopIndex = nDecimalDigits;
-
-      /* compare the two fractional parts of the decimal */
-    while (leftIndex1 < leftStopIndex && rightIndex1 < rightStopIndex) {
-      if (getIntegerFromSparseBuffer(left, leftStart, leftIndex1) > getIntegerFromSparseBuffer(right, rightStart, rightIndex1)) {
-        return 1;
-      }
-      else if (getIntegerFromSparseBuffer(right, rightStart, rightIndex1) > getIntegerFromSparseBuffer(left, leftStart, leftIndex1)) {
-        return -1;
-      }
-
-      leftIndex1++;
-      rightIndex1++;
-    }
-
-      /* Till now the fractional part of the decimals are equal, check
-       * if one of the decimal has fractional part that is remaining
-       * and is non-zero
-       */
-    while (leftIndex1 < leftStopIndex) {
-      if (getIntegerFromSparseBuffer(left, leftStart, leftIndex1) != 0) {
-        return 1;
-      }
-      leftIndex1++;
-    }
-
-    while(rightIndex1 < rightStopIndex) {
-      if (getIntegerFromSparseBuffer(right, rightStart, rightIndex1) != 0) {
-        return -1;
-      }
-      rightIndex1++;
-    }
-
-      /* Both decimal values are equal */
-    return 0;
-  }
-
-  public static BigDecimal getBigDecimalFromByteArray(byte[] bytes, int start, int length, int scale) {
-    byte[] value = Arrays.copyOfRange(bytes, start, start + length);
-    BigInteger unscaledValue = new BigInteger(value);
-    return new BigDecimal(unscaledValue, scale);
-  }
-
-  public static void roundDecimal(ArrowBuf result, int start, int nDecimalDigits, int desiredScale, int currentScale) {
-    int newScaleRoundedUp  = org.apache.arrow.vector.util.DecimalUtility.roundUp(desiredScale);
-    int origScaleRoundedUp = org.apache.arrow.vector.util.DecimalUtility.roundUp(currentScale);
-
-    if (desiredScale < currentScale) {
-
-      boolean roundUp = false;
-
-      //Extract the first digit to be truncated to check if we need to round up
-      int truncatedScaleIndex = desiredScale + 1;
-      if (truncatedScaleIndex <= currentScale) {
-        int extractDigitIndex = nDecimalDigits - origScaleRoundedUp -1;
-        extractDigitIndex += org.apache.arrow.vector.util.DecimalUtility.roundUp(truncatedScaleIndex);
-        int extractDigit = getIntegerFromSparseBuffer(result, start, extractDigitIndex);
-        int temp = org.apache.arrow.vector.util.DecimalUtility.MAX_DIGITS - (truncatedScaleIndex % org.apache.arrow.vector.util.DecimalUtility.MAX_DIGITS);
-        if (temp != 0) {
-          extractDigit = extractDigit / (int) (Math.pow(10, temp));
-        }
-        if ((extractDigit % 10)  > 4) {
-          roundUp = true;
-        }
-      }
-
-      // Get the source index beyond which we will truncate
-      int srcIntIndex = nDecimalDigits - origScaleRoundedUp - 1;
-      int srcIndex = srcIntIndex + newScaleRoundedUp;
-
-      // Truncate the remaining fractional part, move the integer part
-      int destIndex = nDecimalDigits - 1;
-      if (srcIndex != destIndex) {
-        while (srcIndex >= 0) {
-          setInteger(result, start, destIndex--, getIntegerFromSparseBuffer(result, start, srcIndex--));
-        }
-
-        // Set the remaining portion of the decimal to be zeroes
-        while (destIndex >= 0) {
-          setInteger(result, start, destIndex--, 0);
-        }
-        srcIndex = nDecimalDigits - 1;
-      }
-
-      // We truncated the decimal digit. Now we need to truncate within the base 1 billion fractional digit
-      int truncateFactor = org.apache.arrow.vector.util.DecimalUtility.MAX_DIGITS - (desiredScale % org.apache.arrow.vector.util.DecimalUtility.MAX_DIGITS);
-      if (truncateFactor != org.apache.arrow.vector.util.DecimalUtility.MAX_DIGITS) {
-        truncateFactor = (int) Math.pow(10, truncateFactor);
-        int fractionalDigits = getIntegerFromSparseBuffer(result, start, nDecimalDigits - 1);
-        fractionalDigits /= truncateFactor;
-        setInteger(result, start, nDecimalDigits - 1, fractionalDigits * truncateFactor);
-      }
-
-      // Finally round up the digit if needed
-      if (roundUp == true) {
-        srcIndex = nDecimalDigits - 1;
-        int carry;
-        if (truncateFactor != org.apache.arrow.vector.util.DecimalUtility.MAX_DIGITS) {
-          carry = truncateFactor;
-        } else {
-          carry = 1;
-        }
-
-        while (srcIndex >= 0) {
-          int value = getIntegerFromSparseBuffer(result, start, srcIndex);
-          value += carry;
-
-          if (value >= org.apache.arrow.vector.util.DecimalUtility.DIGITS_BASE) {
-            setInteger(result, start, srcIndex--, value % org.apache.arrow.vector.util.DecimalUtility.DIGITS_BASE);
-            carry = value / org.apache.arrow.vector.util.DecimalUtility.DIGITS_BASE;
-          } else {
-            setInteger(result, start, srcIndex--, value);
-            carry = 0;
-            break;
-          }
-        }
-      }
-    } else if (desiredScale > currentScale) {
-      // Add fractional digits to the decimal
-
-      // Check if we need to shift the decimal digits to the left
-      if (newScaleRoundedUp > origScaleRoundedUp) {
-        int srcIndex  = 0;
-        int destIndex = newScaleRoundedUp - origScaleRoundedUp;
-
-        // Check while extending scale, we are not overwriting integer part
-        while (srcIndex < destIndex) {
-          if (getIntegerFromSparseBuffer(result, start, srcIndex++) != 0) {
-            throw new RuntimeException("Truncate resulting in loss of integer part, reduce scale specified");
-          }
-        }
-
-        srcIndex = 0;
-        while (destIndex < nDecimalDigits) {
-          setInteger(result, start, srcIndex++, getIntegerFromSparseBuffer(result, start, destIndex++));
-        }
-
-        // Clear the remaining part
-        while (srcIndex < nDecimalDigits) {
-          setInteger(result, start, srcIndex++, 0);
-        }
-      }
-    }
-  }
-
-  public static int getFirstFractionalDigit(int decimal, int scale) {
-    if (scale == 0) {
-      return 0;
-    }
-    int temp = (int) adjustScaleDivide(decimal, scale - 1);
-    return Math.abs(temp % 10);
-  }
-
-  public static int getFirstFractionalDigit(long decimal, int scale) {
-    if (scale == 0) {
-      return 0;
-    }
-    long temp = adjustScaleDivide(decimal, scale - 1);
-    return (int) (Math.abs(temp % 10));
-  }
-
-  public static int getFirstFractionalDigit(ArrowBuf data, int scale, int start, int nDecimalDigits) {
-    if (scale == 0) {
-      return 0;
-    }
-
-    int index = nDecimalDigits - roundUp(scale);
-    return (int) (adjustScaleDivide(data.getInt(start + (index * INTEGER_SIZE)), MAX_DIGITS - 1));
+    bytebuf.setBytes(startIndex + padLength, bytes, 0, bytes.length);
   }
 }
 

http://git-wip-us.apache.org/repos/asf/arrow/blob/282fcacc/java/vector/src/test/java/org/apache/arrow/vector/TestDecimalVector.java
----------------------------------------------------------------------
diff --git a/java/vector/src/test/java/org/apache/arrow/vector/TestDecimalVector.java b/java/vector/src/test/java/org/apache/arrow/vector/TestDecimalVector.java
index 7ab7db3..cca35e4 100644
--- a/java/vector/src/test/java/org/apache/arrow/vector/TestDecimalVector.java
+++ b/java/vector/src/test/java/org/apache/arrow/vector/TestDecimalVector.java
@@ -33,9 +33,10 @@ public class TestDecimalVector {
   private static long[] intValues;
 
   static {
-    intValues = new long[30];
-    for (int i = 0; i < intValues.length; i++) {
+    intValues = new long[60];
+    for (int i = 0; i < intValues.length / 2; i++) {
       intValues[i] = 1 << i + 1;
+      intValues[2 * i] = -1 * (1 << i + 1);
     }
   }
   private int scale = 3;
@@ -50,7 +51,7 @@ public class TestDecimalVector {
       BigDecimal decimal = new BigDecimal(BigInteger.valueOf(intValues[i]), scale);
       values[i] = decimal;
       decimalVector.getMutator().setIndexDefined(i);
-      DecimalUtility.writeBigDecimalToArrowBuf(decimalVector.getBuffer(), i * 16, decimal);
+      DecimalUtility.writeBigDecimalToArrowBuf(decimal, decimalVector.getBuffer(), i);
     }
 
     decimalVector.getMutator().setValueCount(intValues.length);