diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index 616b9e0e65b7..36daa6189d1e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -190,7 +190,7 @@ case class Cast(child: Expression, dataType: DataType) } private[this] def decimalToTimestamp(d: Decimal): Long = { - (d.toBigDecimal * 1000000L).longValue() + d.toJavaBigDecimal.multiply(java.math.BigDecimal.valueOf(1000000L)).longValue() } private[this] def doubleToTimestamp(d: Double): Any = { if (d.isNaN || d.isInfinite) null else (d * 1000000L).toLong @@ -554,10 +554,10 @@ case class Cast(child: Expression, dataType: DataType) (c, evPrim, evNull) => s""" Decimal tmpDecimal = Decimal.apply( - scala.math.BigDecimal.valueOf(${timestampToDoubleCode(c)})); + java.math.BigDecimal.valueOf(${timestampToDoubleCode(c)})); ${changePrecision("tmpDecimal", target, evPrim, evNull)} """ - case DecimalType() => + case dt: DecimalType => (c, evPrim, evNull) => s""" Decimal tmpDecimal = $c.clone(); @@ -574,7 +574,7 @@ case class Cast(child: Expression, dataType: DataType) (c, evPrim, evNull) => s""" try { - Decimal tmpDecimal = Decimal.apply(scala.math.BigDecimal.valueOf((double) $c)); + Decimal tmpDecimal = Decimal.apply(java.math.BigDecimal.valueOf((double) $c)); ${changePrecision("tmpDecimal", target, evPrim, evNull)} } catch (java.lang.NumberFormatException e) { $evNull = true; @@ -634,7 +634,7 @@ case class Cast(child: Expression, dataType: DataType) } private[this] def decimalToTimestampCode(d: String): String = - s"($d.toBigDecimal().bigDecimal().multiply(new java.math.BigDecimal(1000000L))).longValue()" + s"($d.toJavaBigDecimal().multiply(new java.math.BigDecimal(1000000L))).longValue()" private[this] def longToTimeStampCode(l: String): String = s"$l * 1000L" private[this] def timestampToIntegerCode(ts: String): String = s"java.lang.Math.floor((double) $ts / 1000000L)" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala index 98464edf4d39..b4af1aa05e1f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala @@ -450,7 +450,7 @@ case class Pmod(left: Expression, right: Expression) extends BinaryArithmetic { val decimalAdd = "$plus" s""" ${ctx.javaType(dataType)} r = $eval1.remainder($eval2); - if (r.compare(new org.apache.spark.sql.types.Decimal().set(0)) < 0) { + if (r.compare(Decimal.ZERO()) < 0) { ${ev.primitive} = (r.$decimalAdd($eval2)).remainder($eval2); } else { ${ev.primitive} = r; diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala index d95805c24521..1b89e3f4dbb5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala @@ -17,9 +17,10 @@ package org.apache.spark.sql.types -import java.math.{RoundingMode, MathContext} +import java.math.{MathContext, RoundingMode, BigDecimal => JavaBigDecimal} import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.unsafe.PlatformDependent /** * A mutable implementation of BigDecimal that can hold a Long if values are small enough. @@ -32,28 +33,18 @@ import org.apache.spark.annotation.DeveloperApi final class Decimal extends Ordered[Decimal] with Serializable { import org.apache.spark.sql.types.Decimal._ - private var decimalVal: BigDecimal = null - private var longVal: Long = 0L + private var decimalVal: JavaBigDecimal = BIG_DEC_ZERO private var _precision: Int = 1 - private var _scale: Int = 0 def precision: Int = _precision - def scale: Int = _scale + def scale: Int = decimalVal.scale() /** * Set this Decimal to the given Long. Will have precision 20 and scale 0. */ def set(longVal: Long): Decimal = { - if (longVal <= -POW_10(MAX_LONG_DIGITS) || longVal >= POW_10(MAX_LONG_DIGITS)) { - // We can't represent this compactly as a long without risking overflow - this.decimalVal = BigDecimal(longVal) - this.longVal = 0L - } else { - this.decimalVal = null - this.longVal = longVal - } - this._precision = 20 - this._scale = 0 + decimalVal = JavaBigDecimal.valueOf(longVal) + _precision = 20 this } @@ -61,68 +52,52 @@ final class Decimal extends Ordered[Decimal] with Serializable { * Set this Decimal to the given Int. Will have precision 10 and scale 0. */ def set(intVal: Int): Decimal = { - this.decimalVal = null - this.longVal = intVal - this._precision = 10 - this._scale = 0 + decimalVal = JavaBigDecimal.valueOf(intVal) + _precision = 10 this } /** * Set this Decimal to the given unscaled Long, with a given precision and scale. + * + * Note: this is used in serialization, caller will make sure that it will not overflow */ def set(unscaled: Long, precision: Int, scale: Int): Decimal = { - if (setOrNull(unscaled, precision, scale) == null) { - throw new IllegalArgumentException("Unscaled value too large for precision") - } + decimalVal = JavaBigDecimal.valueOf(unscaled, scale) + _precision = precision this } /** - * Set this Decimal to the given unscaled Long, with a given precision and scale, - * and return it, or return null if it cannot be set due to overflow. + * Set this Decimal to the given BigDecimal value, with a given precision and scale. */ - def setOrNull(unscaled: Long, precision: Int, scale: Int): Decimal = { - if (unscaled <= -POW_10(MAX_LONG_DIGITS) || unscaled >= POW_10(MAX_LONG_DIGITS)) { - // We can't represent this compactly as a long without risking overflow - if (precision < 19) { - return null // Requested precision is too low to represent this value - } - this.decimalVal = BigDecimal(unscaled) - this.longVal = 0L - } else { - val p = POW_10(math.min(precision, MAX_LONG_DIGITS)) - if (unscaled <= -p || unscaled >= p) { - return null // Requested precision is too low to represent this value - } - this.decimalVal = null - this.longVal = unscaled - } - this._precision = precision - this._scale = scale - this + def set(decimal: BigDecimal, precision: Int, scale: Int): Decimal = { + set(decimal.underlying(), precision, scale) } /** - * Set this Decimal to the given BigDecimal value, with a given precision and scale. + * Set this Decimal to the given BigDecimal value, inheriting its precision and scale. */ - def set(decimal: BigDecimal, precision: Int, scale: Int): Decimal = { - this.decimalVal = decimal.setScale(scale, ROUNDING_MODE) + def set(decimal: BigDecimal): Decimal = { + set(decimal.underlying()) + } + + /** + * Set this Decimal to the given java.math.BigDecimal value, with a given precision and scale. + */ + private[sql] def set(decimal: JavaBigDecimal, precision: Int, scale: Int): Decimal = { + decimalVal = decimal.setScale(scale, ROUNDING_MODE) require(decimalVal.precision <= precision, "Overflowed precision") - this.longVal = 0L - this._precision = precision - this._scale = scale + _precision = precision this } /** - * Set this Decimal to the given BigDecimal value, inheriting its precision and scale. + * Set this Decimal to the given java.math.BigDecimal value, inheriting its precision and scale. */ - def set(decimal: BigDecimal): Decimal = { + private[sql] def set(decimal: JavaBigDecimal): Decimal = { this.decimalVal = decimal - this.longVal = 0L this._precision = decimal.precision - this._scale = decimal.scale this } @@ -131,52 +106,36 @@ final class Decimal extends Ordered[Decimal] with Serializable { */ def set(decimal: Decimal): Decimal = { this.decimalVal = decimal.decimalVal - this.longVal = decimal.longVal this._precision = decimal._precision - this._scale = decimal._scale this } - def toBigDecimal: BigDecimal = { - if (decimalVal.ne(null)) { - decimalVal - } else { - BigDecimal(longVal, _scale) - } - } + def toBigDecimal: BigDecimal = BigDecimal(toJavaBigDecimal) - def toJavaBigDecimal: java.math.BigDecimal = toBigDecimal.underlying() + private[sql] def toJavaBigDecimal: JavaBigDecimal = decimalVal def toUnscaledLong: Long = { - if (decimalVal.ne(null)) { - decimalVal.underlying().unscaledValue().longValue() + val unscaled = PlatformDependent.UNSAFE.getLong(decimalVal, + PlatformDependent.BIG_DECIMAL_INTCOMPACT_OFFSET) + if (unscaled != Long.MinValue) { + unscaled } else { - longVal + decimalVal.unscaledValue().longValue() } } - override def toString: String = toBigDecimal.toString() + override def toString: String = decimalVal.toString() @DeveloperApi def toDebugString: String = { - if (decimalVal.ne(null)) { - s"Decimal(expanded,$decimalVal,$precision,$scale})" - } else { - s"Decimal(compact,$longVal,$precision,$scale})" - } + s"Decimal($decimalVal,${_precision})" } - def toDouble: Double = toBigDecimal.doubleValue() + def toDouble: Double = toJavaBigDecimal.doubleValue() - def toFloat: Float = toBigDecimal.floatValue() + def toFloat: Float = toJavaBigDecimal.floatValue() - def toLong: Long = { - if (decimalVal.eq(null)) { - longVal / POW_10(_scale) - } else { - decimalVal.longValue() - } - } + def toLong: Long = decimalVal.longValue() def toInt: Int = toLong.toInt @@ -191,65 +150,23 @@ final class Decimal extends Ordered[Decimal] with Serializable { */ def changePrecision(precision: Int, scale: Int): Boolean = { // fast path for UnsafeProjection - if (precision == this.precision && scale == this.scale) { + if (precision == _precision && scale == decimalVal.scale()) { return true } - // First, update our longVal if we can, or transfer over to using a BigDecimal - if (decimalVal.eq(null)) { - if (scale < _scale) { - // Easier case: we just need to divide our scale down - val diff = _scale - scale - val droppedDigits = longVal % POW_10(diff) - longVal /= POW_10(diff) - if (math.abs(droppedDigits) * 2 >= POW_10(diff)) { - longVal += (if (longVal < 0) -1L else 1L) - } - } else if (scale > _scale) { - // We might be able to multiply longVal by a power of 10 and not overflow, but if not, - // switch to using a BigDecimal - val diff = scale - _scale - val p = POW_10(math.max(MAX_LONG_DIGITS - diff, 0)) - if (diff <= MAX_LONG_DIGITS && longVal > -p && longVal < p) { - // Multiplying longVal by POW_10(diff) will still keep it below MAX_LONG_DIGITS - longVal *= POW_10(diff) - } else { - // Give up on using Longs; switch to BigDecimal, which we'll modify below - decimalVal = BigDecimal(longVal, _scale) - } - } - // In both cases, we will check whether our precision is okay below - } - if (decimalVal.ne(null)) { - // We get here if either we started with a BigDecimal, or we switched to one because we would - // have overflowed our Long; in either case we must rescale decimalVal to the new scale. - val newVal = decimalVal.setScale(scale, ROUNDING_MODE) - if (newVal.precision > precision) { - return false - } - decimalVal = newVal - } else { - // We're still using Longs, but we should check whether we match the new precision - val p = POW_10(math.min(precision, MAX_LONG_DIGITS)) - if (longVal <= -p || longVal >= p) { - // Note that we shouldn't have been able to fix this by switching to BigDecimal - return false - } + val newVal = decimalVal.setScale(scale, ROUNDING_MODE) + if (newVal.precision > precision) { + return false } - + decimalVal = newVal _precision = precision - _scale = scale true } override def clone(): Decimal = new Decimal().set(this) override def compare(other: Decimal): Int = { - if (decimalVal.eq(null) && other.decimalVal.eq(null) && _scale == other._scale) { - if (longVal < other.longVal) -1 else if (longVal == other.longVal) 0 else 1 - } else { - toBigDecimal.compare(other.toBigDecimal) - } + toJavaBigDecimal.compareTo(other.toJavaBigDecimal) } override def equals(other: Any): Boolean = other match { @@ -261,22 +178,16 @@ final class Decimal extends Ordered[Decimal] with Serializable { override def hashCode(): Int = toBigDecimal.hashCode() - def isZero: Boolean = if (decimalVal.ne(null)) decimalVal == BIG_DEC_ZERO else longVal == 0 + def isZero: Boolean = { + decimalVal.compareTo(BIG_DEC_ZERO) == 0 + } def + (that: Decimal): Decimal = { - if (decimalVal.eq(null) && that.decimalVal.eq(null) && scale == that.scale) { - Decimal(longVal + that.longVal, Math.max(precision, that.precision), scale) - } else { - Decimal(toBigDecimal + that.toBigDecimal, precision, scale) - } + Decimal(toJavaBigDecimal.add(that.toJavaBigDecimal, MATH_CONTEXT), precision, scale) } def - (that: Decimal): Decimal = { - if (decimalVal.eq(null) && that.decimalVal.eq(null) && scale == that.scale) { - Decimal(longVal - that.longVal, Math.max(precision, that.precision), scale) - } else { - Decimal(toBigDecimal - that.toBigDecimal, precision, scale) - } + Decimal(toJavaBigDecimal.subtract(that.toJavaBigDecimal, MATH_CONTEXT), precision, scale) } // HiveTypeCoercion will take care of the precision, scale of result @@ -293,30 +204,23 @@ final class Decimal extends Ordered[Decimal] with Serializable { def remainder(that: Decimal): Decimal = this % that def unary_- : Decimal = { - if (decimalVal.ne(null)) { - Decimal(-decimalVal, precision, scale) - } else { - Decimal(-longVal, precision, scale) - } + Decimal(decimalVal.negate(), precision, scale) } def abs: Decimal = if (this.compare(Decimal.ZERO) < 0) this.unary_- else this } object Decimal { - private val ROUNDING_MODE = BigDecimal.RoundingMode.HALF_UP - /** Maximum number of decimal digits a Long can represent */ val MAX_LONG_DIGITS = 18 - private val POW_10 = Array.tabulate[Long](MAX_LONG_DIGITS + 1)(i => math.pow(10, i).toLong) - - private val BIG_DEC_ZERO = BigDecimal(0) + val ZERO = Decimal(0) + val ONE = Decimal(1) - private val MATH_CONTEXT = new MathContext(DecimalType.MAX_PRECISION, RoundingMode.HALF_UP) - - private[sql] val ZERO = Decimal(0) - private[sql] val ONE = Decimal(1) + private val ROUNDING_MODE = RoundingMode.HALF_UP + private val MATH_CONTEXT = new MathContext(DecimalType.MAX_PRECISION, ROUNDING_MODE) + private val POW_10 = Array.tabulate[Long](MAX_LONG_DIGITS + 1)(i => math.pow(10, i).toLong) + private val BIG_DEC_ZERO: JavaBigDecimal = JavaBigDecimal.valueOf(0) def apply(value: Double): Decimal = new Decimal().set(value) @@ -326,12 +230,12 @@ object Decimal { def apply(value: BigDecimal): Decimal = new Decimal().set(value) - def apply(value: java.math.BigDecimal): Decimal = new Decimal().set(value) + def apply(value: JavaBigDecimal): Decimal = new Decimal().set(value) def apply(value: BigDecimal, precision: Int, scale: Int): Decimal = new Decimal().set(value, precision, scale) - def apply(value: java.math.BigDecimal, precision: Int, scale: Int): Decimal = + def apply(value: JavaBigDecimal, precision: Int, scale: Int): Decimal = new Decimal().set(value, precision, scale) def apply(unscaled: Long, precision: Int, scale: Int): Decimal = diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MathFunctionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MathFunctionsSuite.scala index 033792eee6c0..7bf8fdc5aac8 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MathFunctionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MathFunctionsSuite.scala @@ -20,9 +20,9 @@ package org.apache.spark.sql.catalyst.expressions import com.google.common.math.LongMath import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.catalyst.dsl.expressions._ -import org.apache.spark.sql.catalyst.expressions.codegen.{GenerateProjection, GenerateMutableProjection} import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.expressions.codegen.GenerateMutableProjection import org.apache.spark.sql.catalyst.optimizer.DefaultOptimizer import org.apache.spark.sql.catalyst.plans.logical.{OneRowRelation, Project} import org.apache.spark.sql.types._ @@ -30,7 +30,7 @@ import org.apache.spark.sql.types._ class MathFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { - import IntegralLiteralTestUtils._ + import org.apache.spark.sql.catalyst.expressions.IntegralLiteralTestUtils._ /** * Used for testing leaf math expressions. diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/decimal/DecimalSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/decimal/DecimalSuite.scala index 6921d15958a5..911787af9b57 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/decimal/DecimalSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/decimal/DecimalSuite.scala @@ -46,11 +46,8 @@ class DecimalSuite extends SparkFunSuite with PrivateMethodTester { checkDecimal(Decimal(1e17.toLong, 18, 0), 1e17.toLong.toString, 18, 0) checkDecimal(Decimal(Long.MaxValue), Long.MaxValue.toString, 20, 0) checkDecimal(Decimal(Long.MinValue), Long.MinValue.toString, 20, 0) - intercept[IllegalArgumentException](Decimal(170L, 2, 1)) - intercept[IllegalArgumentException](Decimal(170L, 2, 0)) intercept[IllegalArgumentException](Decimal(BigDecimal("10.030"), 2, 1)) intercept[IllegalArgumentException](Decimal(BigDecimal("-9.95"), 2, 1)) - intercept[IllegalArgumentException](Decimal(1e17.toLong, 17, 0)) } test("creating decimals with negative scale") { @@ -88,36 +85,19 @@ class DecimalSuite extends SparkFunSuite with PrivateMethodTester { checkValues(Decimal(Double.MinValue), Double.MinValue, 0L) } - // Accessor for the BigDecimal value of a Decimal, which will be null if it's using Longs - private val decimalVal = PrivateMethod[BigDecimal]('decimalVal) - - /** Check whether a decimal is represented compactly (passing whether we expect it to be) */ - private def checkCompact(d: Decimal, expected: Boolean): Unit = { - val isCompact = d.invokePrivate(decimalVal()).eq(null) - assert(isCompact == expected, s"$d ${if (expected) "was not" else "was"} compact") - } - - test("small decimals represented as unscaled long") { - checkCompact(new Decimal(), true) - checkCompact(Decimal(BigDecimal(10.03)), false) - checkCompact(Decimal(BigDecimal(1e20)), false) - checkCompact(Decimal(17L), true) - checkCompact(Decimal(17), true) - checkCompact(Decimal(17L, 2, 1), true) - checkCompact(Decimal(170L, 4, 2), true) - checkCompact(Decimal(17L, 24, 1), true) - checkCompact(Decimal(1e16.toLong), true) - checkCompact(Decimal(1e17.toLong), true) - checkCompact(Decimal(1e18.toLong - 1), true) - checkCompact(Decimal(- 1e18.toLong + 1), true) - checkCompact(Decimal(1e18.toLong - 1, 30, 10), true) - checkCompact(Decimal(- 1e18.toLong + 1, 30, 10), true) - checkCompact(Decimal(1e18.toLong), false) - checkCompact(Decimal(-1e18.toLong), false) - checkCompact(Decimal(1e18.toLong, 30, 10), false) - checkCompact(Decimal(-1e18.toLong, 30, 10), false) - checkCompact(Decimal(Long.MaxValue), false) - checkCompact(Decimal(Long.MinValue), false) + test("change precision and scale") { + assert(true === Decimal(5).changePrecision(1, 0)) + assert(false === Decimal(15).changePrecision(1, 0)) + assert(true === Decimal(5).changePrecision(2, 1)) + assert(false === Decimal(5).changePrecision(2, 2)) + assert(true === Decimal(0).changePrecision(1, 0)) + assert(true === Decimal(BigDecimal("10.5")).changePrecision(3, 0)) + assert(true === Decimal(BigDecimal("10.5")).changePrecision(3, 1)) + assert(false === Decimal(BigDecimal("10.5")).changePrecision(3, 2)) + assert(true === Decimal(BigDecimal("10.5")).changePrecision(4, 0)) + assert(true === Decimal(BigDecimal("10.5")).changePrecision(4, 1)) + assert(true === Decimal(BigDecimal("10.5")).changePrecision(4, 2)) + assert(false === Decimal(BigDecimal("10.5")).changePrecision(4, 3)) } test("hash code") { @@ -132,10 +112,6 @@ class DecimalSuite extends SparkFunSuite with PrivateMethodTester { } test("equals") { - // The decimals on the left are stored compactly, while the ones on the right aren't - checkCompact(Decimal(123), true) - checkCompact(Decimal(BigDecimal(123)), false) - checkCompact(Decimal("123"), false) assert(Decimal(123) === Decimal(BigDecimal(123))) assert(Decimal(123) === Decimal(BigDecimal("123.00"))) assert(Decimal(-123) === Decimal(BigDecimal(-123))) @@ -187,7 +163,7 @@ class DecimalSuite extends SparkFunSuite with PrivateMethodTester { assert(b.toDouble === 0.125) } - test("set/setOrNull") { + test("set") { assert(new Decimal().set(10L, 10, 0).toUnscaledLong === 10L) assert(new Decimal().set(100L, 10, 0).toUnscaledLong === 100L) assert(Decimal(Long.MaxValue, 100, 0).toUnscaledLong === Long.MaxValue) diff --git a/unsafe/src/main/java/org/apache/spark/unsafe/PlatformDependent.java b/unsafe/src/main/java/org/apache/spark/unsafe/PlatformDependent.java index b2de2a2590f0..e193436d9a67 100644 --- a/unsafe/src/main/java/org/apache/spark/unsafe/PlatformDependent.java +++ b/unsafe/src/main/java/org/apache/spark/unsafe/PlatformDependent.java @@ -18,6 +18,7 @@ package org.apache.spark.unsafe; import java.lang.reflect.Field; +import java.math.BigDecimal; import java.math.BigInteger; import sun.misc.Unsafe; @@ -119,6 +120,7 @@ public static void freeMemory(long address) { // Support for resetting final fields while deserializing public static final long BIG_INTEGER_SIGNUM_OFFSET; public static final long BIG_INTEGER_MAG_OFFSET; + public static final long BIG_DECIMAL_INTCOMPACT_OFFSET; /** * Limits the number of bytes to copy per {@link Unsafe#copyMemory(long, long, long)} to @@ -145,21 +147,27 @@ public static void freeMemory(long address) { long signumOffset = 0; long magOffset = 0; + long intCompactOffset = 0; try { signumOffset = _UNSAFE.objectFieldOffset(BigInteger.class.getDeclaredField("signum")); magOffset = _UNSAFE.objectFieldOffset(BigInteger.class.getDeclaredField("mag")); + intCompactOffset = _UNSAFE.objectFieldOffset( + BigDecimal.class.getDeclaredField("intCompact")); } catch (Exception ex) { // should not happen } BIG_INTEGER_SIGNUM_OFFSET = signumOffset; BIG_INTEGER_MAG_OFFSET = magOffset; + BIG_DECIMAL_INTCOMPACT_OFFSET = intCompactOffset; } else { + // should not happen BYTE_ARRAY_OFFSET = 0; INT_ARRAY_OFFSET = 0; LONG_ARRAY_OFFSET = 0; DOUBLE_ARRAY_OFFSET = 0; BIG_INTEGER_SIGNUM_OFFSET = 0; BIG_INTEGER_MAG_OFFSET = 0; + BIG_DECIMAL_INTCOMPACT_OFFSET = 0; } }