[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user asfgit closed the pull request at: https://github.com/apache/spark/pull/20023 --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user gatorsmile commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r162246975 --- Diff: docs/sql-programming-guide.md --- @@ -1795,6 +1795,11 @@ options. - Since Spark 2.3, when all inputs are binary, SQL `elt()` returns an output as binary. Otherwise, it returns as a string. Until Spark 2.3, it always returns as a string despite of input types. To keep the old behavior, set `spark.sql.function.eltOutputAsString` to `true`. + - Since Spark 2.3, by default arithmetic operations between decimals return a rounded value if an exact representation is not possible. This is compliant to SQL standards and Hive's behavior introduced in HIVE-15331. This involves the following changes --- End diff -- Hive's version is 2.2. We have to emphasize it. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user gatorsmile commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r162246729 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala --- @@ -1048,6 +1048,16 @@ object SQLConf { .booleanConf .createWithDefault(true) + val DECIMAL_OPERATIONS_ALLOW_PREC_LOSS = +buildConf("spark.sql.decimalOperations.allowPrecisionLoss") + .internal() + .doc("When true (default), establishing the result type of an arithmetic operation " + +"happens according to Hive behavior and SQL ANSI 2011 specification, ie. rounding the " + +"decimal part of the result if an exact representation is not possible. Otherwise, NULL " + +"is returned in those cases, as previously.") --- End diff -- Yeah. This is better. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user gatorsmile commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r162246614 --- Diff: docs/sql-programming-guide.md --- @@ -1795,6 +1795,11 @@ options. - Since Spark 2.3, when all inputs are binary, SQL `elt()` returns an output as binary. Otherwise, it returns as a string. Until Spark 2.3, it always returns as a string despite of input types. To keep the old behavior, set `spark.sql.function.eltOutputAsString` to `true`. + - Since Spark 2.3, by default arithmetic operations between decimals return a rounded value if an exact representation is not possible. This is compliant to SQL standards and Hive's behavior introduced in HIVE-15331. This involves the following changes +- The rules to determine the result type of an arithmetic operation have been updated. In particular, if the precision / scale needed are out of the range of available values, the scale is reduced up to 6, in order to prevent the truncation of the integer part of the decimals. +- Literal values used in SQL operations are converted to DECIMAL with the exact precision and scale needed by them. +- The configuration `spark.sql.decimalOperations.allowPrecisionLoss` has been introduced. It defaults to `true`, which means the new behavior described here; if set to `false`, Spark will use the previous rules and behavior. --- End diff -- At least, we need to say, NULL will be returned in this case. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user gatorsmile commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r162246280 --- Diff: docs/sql-programming-guide.md --- @@ -1795,6 +1795,11 @@ options. - Since Spark 2.3, when all inputs are binary, SQL `elt()` returns an output as binary. Otherwise, it returns as a string. Until Spark 2.3, it always returns as a string despite of input types. To keep the old behavior, set `spark.sql.function.eltOutputAsString` to `true`. + - Since Spark 2.3, by default arithmetic operations between decimals return a rounded value if an exact representation is not possible. This is compliant to SQL standards and Hive's behavior introduced in HIVE-15331. This involves the following changes +- The rules to determine the result type of an arithmetic operation have been updated. In particular, if the precision / scale needed are out of the range of available values, the scale is reduced up to 6, in order to prevent the truncation of the integer part of the decimals. +- Literal values used in SQL operations are converted to DECIMAL with the exact precision and scale needed by them. +- The configuration `spark.sql.decimalOperations.allowPrecisionLoss` has been introduced. It defaults to `true`, which means the new behavior described here; if set to `false`, Spark will use the previous rules and behavior. --- End diff -- Also need to explain what is the previous behavior. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user gatorsmile commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r162246356 --- Diff: docs/sql-programming-guide.md --- @@ -1795,6 +1795,11 @@ options. - Since Spark 2.3, when all inputs are binary, SQL `elt()` returns an output as binary. Otherwise, it returns as a string. Until Spark 2.3, it always returns as a string despite of input types. To keep the old behavior, set `spark.sql.function.eltOutputAsString` to `true`. + - Since Spark 2.3, by default arithmetic operations between decimals return a rounded value if an exact representation is not possible. This is compliant to SQL standards and Hive's behavior introduced in HIVE-15331. This involves the following changes +- The rules to determine the result type of an arithmetic operation have been updated. In particular, if the precision / scale needed are out of the range of available values, the scale is reduced up to 6, in order to prevent the truncation of the integer part of the decimals. --- End diff -- We need to explicitly document which arithmetic operations are affected. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user cloud-fan commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r162064252 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala --- @@ -136,10 +137,52 @@ object DecimalType extends AbstractDataType { case DoubleType => DoubleDecimal } + private[sql] def forLiteral(literal: Literal): DecimalType = literal.value match { --- End diff -- but we also have `fromDecimal`... --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user mgaido91 commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r162053815 --- Diff: sql/core/src/test/resources/sql-tests/inputs/decimals.sql --- @@ -0,0 +1,53 @@ +-- +-- 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. +-- + +-- tests for decimals handling in operations --- End diff -- I see. I'll merge this file into the other then. Thanks. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user mgaido91 commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r162052996 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala --- @@ -136,10 +137,52 @@ object DecimalType extends AbstractDataType { case DoubleType => DoubleDecimal } + private[sql] def forLiteral(literal: Literal): DecimalType = literal.value match { --- End diff -- since we have `forType` I used `forLiteral` to be coherent on the naming --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user cloud-fan commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r162052779 --- Diff: sql/core/src/test/resources/sql-tests/inputs/decimals.sql --- @@ -0,0 +1,53 @@ +-- +-- 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. +-- + +-- tests for decimals handling in operations --- End diff -- that file is under `.../typeCoercion/native/`, which is meant for the default behavior(native mode of type coercion). If we introduce a sql2001 mode, we will put a same file under `.../typeCoercion/sql2001/` --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user mgaido91 commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r162051884 --- Diff: sql/core/src/test/resources/sql-tests/inputs/decimals.sql --- @@ -0,0 +1,53 @@ +-- +-- 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. +-- + +-- tests for decimals handling in operations --- End diff -- because that file was meant for the `typeCoercion` modes (eg. if we introduce a sql2011 mode which throws exception instead of returning NULL), while this is more generic about arithmetic operations' behavior. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user mgaido91 commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r162051255 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecision.scala --- @@ -242,18 +278,36 @@ object DecimalPrecision extends TypeCoercionRule { private val nondecimalAndDecimal: PartialFunction[Expression, Expression] = { // Promote integers inside a binary expression with fixed-precision decimals to decimals, // and fixed-precision decimals in an expression with floats / doubles to doubles -case b @ BinaryOperator(left, right) if left.dataType != right.dataType => - (left.dataType, right.dataType) match { -case (t: IntegralType, DecimalType.Fixed(p, s)) => - b.makeCopy(Array(Cast(left, DecimalType.forType(t)), right)) -case (DecimalType.Fixed(p, s), t: IntegralType) => - b.makeCopy(Array(left, Cast(right, DecimalType.forType(t -case (t, DecimalType.Fixed(p, s)) if isFloat(t) => - b.makeCopy(Array(left, Cast(right, DoubleType))) -case (DecimalType.Fixed(p, s), t) if isFloat(t) => - b.makeCopy(Array(Cast(left, DoubleType), right)) -case _ => - b +case b@BinaryOperator(left, right) if left.dataType != right.dataType => + (left, right) match { +// Promote literal integers inside a binary expression with fixed-precision decimals to --- End diff -- when float and double are involved, the decimal is converted to double --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user cloud-fan commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r162051075 --- Diff: sql/core/src/test/resources/sql-tests/inputs/decimals.sql --- @@ -0,0 +1,53 @@ +-- +-- 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. +-- + +-- tests for decimals handling in operations --- End diff -- Why create a new test file instead of adding more cases in `decimalArithmeticOperations.sql`? --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user cloud-fan commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r162050678 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala --- @@ -136,10 +137,52 @@ object DecimalType extends AbstractDataType { case DoubleType => DoubleDecimal } + private[sql] def forLiteral(literal: Literal): DecimalType = literal.value match { --- End diff -- `fromLiteral`? --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user cloud-fan commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r162050062 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecision.scala --- @@ -242,18 +278,36 @@ object DecimalPrecision extends TypeCoercionRule { private val nondecimalAndDecimal: PartialFunction[Expression, Expression] = { // Promote integers inside a binary expression with fixed-precision decimals to decimals, // and fixed-precision decimals in an expression with floats / doubles to doubles -case b @ BinaryOperator(left, right) if left.dataType != right.dataType => - (left.dataType, right.dataType) match { -case (t: IntegralType, DecimalType.Fixed(p, s)) => - b.makeCopy(Array(Cast(left, DecimalType.forType(t)), right)) -case (DecimalType.Fixed(p, s), t: IntegralType) => - b.makeCopy(Array(left, Cast(right, DecimalType.forType(t -case (t, DecimalType.Fixed(p, s)) if isFloat(t) => - b.makeCopy(Array(left, Cast(right, DoubleType))) -case (DecimalType.Fixed(p, s), t) if isFloat(t) => - b.makeCopy(Array(Cast(left, DoubleType), right)) -case _ => - b +case b@BinaryOperator(left, right) if left.dataType != right.dataType => + (left, right) match { +// Promote literal integers inside a binary expression with fixed-precision decimals to --- End diff -- We only need to deal with integers? how about float and double? --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user cloud-fan commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r162049618 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecision.scala --- @@ -242,18 +278,36 @@ object DecimalPrecision extends TypeCoercionRule { private val nondecimalAndDecimal: PartialFunction[Expression, Expression] = { // Promote integers inside a binary expression with fixed-precision decimals to decimals, // and fixed-precision decimals in an expression with floats / doubles to doubles -case b @ BinaryOperator(left, right) if left.dataType != right.dataType => - (left.dataType, right.dataType) match { -case (t: IntegralType, DecimalType.Fixed(p, s)) => - b.makeCopy(Array(Cast(left, DecimalType.forType(t)), right)) -case (DecimalType.Fixed(p, s), t: IntegralType) => - b.makeCopy(Array(left, Cast(right, DecimalType.forType(t -case (t, DecimalType.Fixed(p, s)) if isFloat(t) => - b.makeCopy(Array(left, Cast(right, DoubleType))) -case (DecimalType.Fixed(p, s), t) if isFloat(t) => - b.makeCopy(Array(Cast(left, DoubleType), right)) -case _ => - b +case b@BinaryOperator(left, right) if left.dataType != right.dataType => --- End diff -- nit: `b @ BinaryOperator ` --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user cloud-fan commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r162049277 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecision.scala --- @@ -93,41 +97,76 @@ object DecimalPrecision extends TypeCoercionRule { case e: BinaryArithmetic if e.left.isInstanceOf[PromotePrecision] => e case Add(e1 @ DecimalType.Expression(p1, s1), e2 @ DecimalType.Expression(p2, s2)) => - val dt = DecimalType.bounded(max(s1, s2) + max(p1 - s1, p2 - s2) + 1, max(s1, s2)) - CheckOverflow(Add(promotePrecision(e1, dt), promotePrecision(e2, dt)), dt) + val resultScale = max(s1, s2) + val resultType = if (SQLConf.get.decimalOperationsAllowPrecisionLoss) { +DecimalType.adjustPrecisionScale(max(p1 - s1, p2 - s2) + resultScale + 1, --- End diff -- I'm ok to do the adjustment for all operations, which is same as Hive. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user cloud-fan commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r162048685 --- Diff: docs/sql-programming-guide.md --- @@ -1795,6 +1795,11 @@ options. - Since Spark 2.3, when all inputs are binary, SQL `elt()` returns an output as binary. Otherwise, it returns as a string. Until Spark 2.3, it always returns as a string despite of input types. To keep the old behavior, set `spark.sql.function.eltOutputAsString` to `true`. + - Since Spark 2.3, by default arithmetic operations return a rounded value if an exact representation is not possible. This is compliant to SQL standards and Hive's behavior introduced in HIVE-15331. This involves the following changes --- End diff -- `arithmetic operations between decimals` --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user mgaido91 commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r161829582 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala --- @@ -136,10 +137,52 @@ object DecimalType extends AbstractDataType { case DoubleType => DoubleDecimal } + private[sql] def forLiteral(literal: Literal): DecimalType = literal.value match { +case v: Short => fromBigDecimal(BigDecimal(v)) +case v: Int => fromBigDecimal(BigDecimal(v)) +case v: Long => fromBigDecimal(BigDecimal(v)) +case _ => forType(literal.dataType) --- End diff -- this problem was present before this PR. Should we fix it here? Is this fix needed? I guess that if it would have been a problem, it would already have been reported. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user gatorsmile commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r161828134 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala --- @@ -136,10 +137,52 @@ object DecimalType extends AbstractDataType { case DoubleType => DoubleDecimal } + private[sql] def forLiteral(literal: Literal): DecimalType = literal.value match { +case v: Short => fromBigDecimal(BigDecimal(v)) +case v: Int => fromBigDecimal(BigDecimal(v)) +case v: Long => fromBigDecimal(BigDecimal(v)) +case _ => forType(literal.dataType) --- End diff -- ``` private[sql] def forType(dataType: DataType): DecimalType = dataType match { case ByteType => ByteDecimal case ShortType => ShortDecimal case IntegerType => IntDecimal case LongType => LongDecimal case FloatType => FloatDecimal case DoubleType => DoubleDecimal } ``` This list is incomplete. Is that possible, the input literal is `Literal(null, NullType)` --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user mgaido91 commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r161826940 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecision.scala --- @@ -243,17 +279,43 @@ object DecimalPrecision extends TypeCoercionRule { // Promote integers inside a binary expression with fixed-precision decimals to decimals, // and fixed-precision decimals in an expression with floats / doubles to doubles case b @ BinaryOperator(left, right) if left.dataType != right.dataType => - (left.dataType, right.dataType) match { -case (t: IntegralType, DecimalType.Fixed(p, s)) => - b.makeCopy(Array(Cast(left, DecimalType.forType(t)), right)) -case (DecimalType.Fixed(p, s), t: IntegralType) => - b.makeCopy(Array(left, Cast(right, DecimalType.forType(t -case (t, DecimalType.Fixed(p, s)) if isFloat(t) => - b.makeCopy(Array(left, Cast(right, DoubleType))) -case (DecimalType.Fixed(p, s), t) if isFloat(t) => - b.makeCopy(Array(Cast(left, DoubleType), right)) -case _ => - b - } + nondecimalLiteralAndDecimal(b).lift((left, right)).getOrElse( +nondecimalNonliteralAndDecimal(b).applyOrElse((left.dataType, right.dataType), + (_: (DataType, DataType)) => b)) } + + /** + * Type coercion for BinaryOperator in which one side is a non-decimal literal numeric, and the + * other side is a decimal. + */ + private def nondecimalLiteralAndDecimal( + b: BinaryOperator): PartialFunction[(Expression, Expression), Expression] = { +// Promote literal integers inside a binary expression with fixed-precision decimals to +// decimals. The precision and scale are the ones needed by the integer value. +case (l: Literal, r) if r.dataType.isInstanceOf[DecimalType] + && l.dataType.isInstanceOf[IntegralType] => + b.makeCopy(Array(Cast(l, DecimalType.forLiteral(l)), r)) --- End diff -- yes, Hive is doing so. That is the reason why I introduced the change (without it, we would have had test failures in spark hive). I will add this in the comment. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user mgaido91 commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r161826504 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala --- @@ -58,7 +58,7 @@ object Literal { case s: Short => Literal(s, ShortType) case s: String => Literal(UTF8String.fromString(s), StringType) case b: Boolean => Literal(b, BooleanType) -case d: BigDecimal => Literal(Decimal(d), DecimalType(Math.max(d.precision, d.scale), d.scale)) +case d: BigDecimal => Literal(Decimal(d), DecimalType.fromBigDecimal(d)) --- End diff -- no, it's not. It's just code re-usage. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user gatorsmile commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r161825917 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala --- @@ -58,7 +58,7 @@ object Literal { case s: Short => Literal(s, ShortType) case s: String => Literal(UTF8String.fromString(s), StringType) case b: Boolean => Literal(b, BooleanType) -case d: BigDecimal => Literal(Decimal(d), DecimalType(Math.max(d.precision, d.scale), d.scale)) +case d: BigDecimal => Literal(Decimal(d), DecimalType.fromBigDecimal(d)) --- End diff -- This is another behavior change. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user gatorsmile commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r161825714 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecision.scala --- @@ -243,17 +279,43 @@ object DecimalPrecision extends TypeCoercionRule { // Promote integers inside a binary expression with fixed-precision decimals to decimals, // and fixed-precision decimals in an expression with floats / doubles to doubles case b @ BinaryOperator(left, right) if left.dataType != right.dataType => - (left.dataType, right.dataType) match { -case (t: IntegralType, DecimalType.Fixed(p, s)) => - b.makeCopy(Array(Cast(left, DecimalType.forType(t)), right)) -case (DecimalType.Fixed(p, s), t: IntegralType) => - b.makeCopy(Array(left, Cast(right, DecimalType.forType(t -case (t, DecimalType.Fixed(p, s)) if isFloat(t) => - b.makeCopy(Array(left, Cast(right, DoubleType))) -case (DecimalType.Fixed(p, s), t) if isFloat(t) => - b.makeCopy(Array(Cast(left, DoubleType), right)) -case _ => - b - } + nondecimalLiteralAndDecimal(b).lift((left, right)).getOrElse( +nondecimalNonliteralAndDecimal(b).applyOrElse((left.dataType, right.dataType), + (_: (DataType, DataType)) => b)) } + + /** + * Type coercion for BinaryOperator in which one side is a non-decimal literal numeric, and the + * other side is a decimal. + */ + private def nondecimalLiteralAndDecimal( + b: BinaryOperator): PartialFunction[(Expression, Expression), Expression] = { +// Promote literal integers inside a binary expression with fixed-precision decimals to +// decimals. The precision and scale are the ones needed by the integer value. +case (l: Literal, r) if r.dataType.isInstanceOf[DecimalType] + && l.dataType.isInstanceOf[IntegralType] => + b.makeCopy(Array(Cast(l, DecimalType.forLiteral(l)), r)) --- End diff -- Hive is also doing this? --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user gatorsmile commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r161824857 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecision.scala --- @@ -243,17 +279,43 @@ object DecimalPrecision extends TypeCoercionRule { // Promote integers inside a binary expression with fixed-precision decimals to decimals, // and fixed-precision decimals in an expression with floats / doubles to doubles case b @ BinaryOperator(left, right) if left.dataType != right.dataType => - (left.dataType, right.dataType) match { -case (t: IntegralType, DecimalType.Fixed(p, s)) => - b.makeCopy(Array(Cast(left, DecimalType.forType(t)), right)) -case (DecimalType.Fixed(p, s), t: IntegralType) => - b.makeCopy(Array(left, Cast(right, DecimalType.forType(t -case (t, DecimalType.Fixed(p, s)) if isFloat(t) => - b.makeCopy(Array(left, Cast(right, DoubleType))) -case (DecimalType.Fixed(p, s), t) if isFloat(t) => - b.makeCopy(Array(Cast(left, DoubleType), right)) -case _ => - b - } + nondecimalLiteralAndDecimal(b).lift((left, right)).getOrElse( +nondecimalNonliteralAndDecimal(b).applyOrElse((left.dataType, right.dataType), + (_: (DataType, DataType)) => b)) } + + /** + * Type coercion for BinaryOperator in which one side is a non-decimal literal numeric, and the + * other side is a decimal. + */ + private def nondecimalLiteralAndDecimal( + b: BinaryOperator): PartialFunction[(Expression, Expression), Expression] = { +// Promote literal integers inside a binary expression with fixed-precision decimals to +// decimals. The precision and scale are the ones needed by the integer value. +case (l: Literal, r) if r.dataType.isInstanceOf[DecimalType] + && l.dataType.isInstanceOf[IntegralType] => + b.makeCopy(Array(Cast(l, DecimalType.forLiteral(l)), r)) --- End diff -- Add this example as the code comment? --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user gatorsmile commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r161820321 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecision.scala --- @@ -93,41 +97,76 @@ object DecimalPrecision extends TypeCoercionRule { case e: BinaryArithmetic if e.left.isInstanceOf[PromotePrecision] => e case Add(e1 @ DecimalType.Expression(p1, s1), e2 @ DecimalType.Expression(p2, s2)) => - val dt = DecimalType.bounded(max(s1, s2) + max(p1 - s1, p2 - s2) + 1, max(s1, s2)) - CheckOverflow(Add(promotePrecision(e1, dt), promotePrecision(e2, dt)), dt) + val resultScale = max(s1, s2) + val resultType = if (SQLConf.get.decimalOperationsAllowPrecisionLoss) { +DecimalType.adjustPrecisionScale(max(p1 - s1, p2 - s2) + resultScale + 1, --- End diff -- We need to make a decision. You know, we try our best to keep our rule unchanged. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user gatorsmile commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r161819811 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala --- @@ -136,10 +137,52 @@ object DecimalType extends AbstractDataType { case DoubleType => DoubleDecimal } + private[sql] def forLiteral(literal: Literal): DecimalType = literal.value match { +case v: Short => fromBigDecimal(BigDecimal(v)) +case v: Int => fromBigDecimal(BigDecimal(v)) +case v: Long => fromBigDecimal(BigDecimal(v)) +case _ => forType(literal.dataType) + } + + private[sql] def fromBigDecimal(d: BigDecimal): DecimalType = { +DecimalType(Math.max(d.precision, d.scale), d.scale) + } + private[sql] def bounded(precision: Int, scale: Int): DecimalType = { DecimalType(min(precision, MAX_PRECISION), min(scale, MAX_SCALE)) } + /** + * Scale adjustment implementation is based on Hive's one, which is itself inspired to + * SQLServer's one. In particular, when a result precision is greater than + * {@link #MAX_PRECISION}, the corresponding scale is reduced to prevent the integral part of a + * result from being truncated. + * + * This method is used only when `spark.sql.decimalOperations.allowPrecisionLoss` is set to true. + * + * @param precision + * @param scale + * @return + */ + private[sql] def adjustPrecisionScale(precision: Int, scale: Int): DecimalType = { --- End diff -- Yeah, this part is consistent. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user cloud-fan commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r161818562 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecision.scala --- @@ -243,17 +279,43 @@ object DecimalPrecision extends TypeCoercionRule { // Promote integers inside a binary expression with fixed-precision decimals to decimals, // and fixed-precision decimals in an expression with floats / doubles to doubles case b @ BinaryOperator(left, right) if left.dataType != right.dataType => - (left.dataType, right.dataType) match { -case (t: IntegralType, DecimalType.Fixed(p, s)) => --- End diff -- we can do ``` (left, right) match { case (l: Literal, r) => ... case (DecimalType.Expression(p, s), r @ IntergralType()) => ... } ``` --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user mgaido91 commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r161815644 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecision.scala --- @@ -243,17 +279,43 @@ object DecimalPrecision extends TypeCoercionRule { // Promote integers inside a binary expression with fixed-precision decimals to decimals, // and fixed-precision decimals in an expression with floats / doubles to doubles case b @ BinaryOperator(left, right) if left.dataType != right.dataType => - (left.dataType, right.dataType) match { -case (t: IntegralType, DecimalType.Fixed(p, s)) => --- End diff -- unfortunately this is not really feasible since we match on different thigs: here we match on `left.dataType` and `right.dataType`, while for literals we match on `left` and `right` --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user cloud-fan commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r161769022 --- Diff: sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala --- @@ -1518,21 +1518,40 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } test("decimal precision with multiply/division") { --- End diff -- can we just move these tests to the new `decimal.sql`? --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user cloud-fan commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r161768479 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala --- @@ -136,10 +137,52 @@ object DecimalType extends AbstractDataType { case DoubleType => DoubleDecimal } + private[sql] def forLiteral(literal: Literal): DecimalType = literal.value match { +case v: Short => fromBigDecimal(BigDecimal(v)) +case v: Int => fromBigDecimal(BigDecimal(v)) +case v: Long => fromBigDecimal(BigDecimal(v)) +case _ => forType(literal.dataType) + } + + private[sql] def fromBigDecimal(d: BigDecimal): DecimalType = { +DecimalType(Math.max(d.precision, d.scale), d.scale) + } + private[sql] def bounded(precision: Int, scale: Int): DecimalType = { DecimalType(min(precision, MAX_PRECISION), min(scale, MAX_SCALE)) } + /** + * Scale adjustment implementation is based on Hive's one, which is itself inspired to + * SQLServer's one. In particular, when a result precision is greater than + * {@link #MAX_PRECISION}, the corresponding scale is reduced to prevent the integral part of a + * result from being truncated. + * + * This method is used only when `spark.sql.decimalOperations.allowPrecisionLoss` is set to true. + * + * @param precision + * @param scale + * @return + */ + private[sql] def adjustPrecisionScale(precision: Int, scale: Int): DecimalType = { +// Assumptions: +// precision >= scale +// scale >= 0 +if (precision <= MAX_PRECISION) { + // Adjustment only needed when we exceed max precision + DecimalType(precision, scale) +} else { + // Precision/scale exceed maximum precision. Result must be adjusted to MAX_PRECISION. + val intDigits = precision - scale + // If original scale less than MINIMUM_ADJUSTED_SCALE, use original scale value; otherwise + // preserve at least MINIMUM_ADJUSTED_SCALE fractional digits + val minScaleValue = Math.min(scale, MINIMUM_ADJUSTED_SCALE) + val adjustedScale = Math.max(MAX_PRECISION - intDigits, minScaleValue) --- End diff -- This line needs some comments. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user cloud-fan commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r161767764 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala --- @@ -136,10 +137,52 @@ object DecimalType extends AbstractDataType { case DoubleType => DoubleDecimal } + private[sql] def forLiteral(literal: Literal): DecimalType = literal.value match { +case v: Short => fromBigDecimal(BigDecimal(v)) +case v: Int => fromBigDecimal(BigDecimal(v)) +case v: Long => fromBigDecimal(BigDecimal(v)) +case _ => forType(literal.dataType) + } + + private[sql] def fromBigDecimal(d: BigDecimal): DecimalType = { +DecimalType(Math.max(d.precision, d.scale), d.scale) + } + private[sql] def bounded(precision: Int, scale: Int): DecimalType = { DecimalType(min(precision, MAX_PRECISION), min(scale, MAX_SCALE)) } + /** + * Scale adjustment implementation is based on Hive's one, which is itself inspired to + * SQLServer's one. In particular, when a result precision is greater than + * {@link #MAX_PRECISION}, the corresponding scale is reduced to prevent the integral part of a + * result from being truncated. + * + * This method is used only when `spark.sql.decimalOperations.allowPrecisionLoss` is set to true. + * + * @param precision + * @param scale + * @return + */ + private[sql] def adjustPrecisionScale(precision: Int, scale: Int): DecimalType = { +// Assumptions: +// precision >= scale +// scale >= 0 --- End diff -- use `assert` for assumptions, not comments. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user cloud-fan commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r161767614 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala --- @@ -136,10 +137,52 @@ object DecimalType extends AbstractDataType { case DoubleType => DoubleDecimal } + private[sql] def forLiteral(literal: Literal): DecimalType = literal.value match { +case v: Short => fromBigDecimal(BigDecimal(v)) +case v: Int => fromBigDecimal(BigDecimal(v)) +case v: Long => fromBigDecimal(BigDecimal(v)) +case _ => forType(literal.dataType) + } + + private[sql] def fromBigDecimal(d: BigDecimal): DecimalType = { +DecimalType(Math.max(d.precision, d.scale), d.scale) + } + private[sql] def bounded(precision: Int, scale: Int): DecimalType = { DecimalType(min(precision, MAX_PRECISION), min(scale, MAX_SCALE)) } + /** + * Scale adjustment implementation is based on Hive's one, which is itself inspired to + * SQLServer's one. In particular, when a result precision is greater than + * {@link #MAX_PRECISION}, the corresponding scale is reduced to prevent the integral part of a + * result from being truncated. + * + * This method is used only when `spark.sql.decimalOperations.allowPrecisionLoss` is set to true. + * + * @param precision + * @param scale + * @return --- End diff -- remove the above 3 lines --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user cloud-fan commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r161767060 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala --- @@ -1048,6 +1048,16 @@ object SQLConf { .booleanConf .createWithDefault(true) + val DECIMAL_OPERATIONS_ALLOW_PREC_LOSS = +buildConf("spark.sql.decimalOperations.allowPrecisionLoss") + .internal() + .doc("When true, establishing the result type of an arithmetic operation happens " + +"according to Hive behavior and SQL ANSI 2011 specification, ie. rounding the decimal " + +"part of the result if an exact representation is not possible. Otherwise, NULL is" + +"returned in those cases, as previously (default).") + .booleanConf + .createWithDefault(false) --- End diff -- we should make it true by default as it's a more reasonable behavior and follows Hive/SQL standard. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user cloud-fan commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r161766711 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecision.scala --- @@ -243,17 +279,43 @@ object DecimalPrecision extends TypeCoercionRule { // Promote integers inside a binary expression with fixed-precision decimals to decimals, // and fixed-precision decimals in an expression with floats / doubles to doubles case b @ BinaryOperator(left, right) if left.dataType != right.dataType => - (left.dataType, right.dataType) match { -case (t: IntegralType, DecimalType.Fixed(p, s)) => --- End diff -- nit: I feel it's more readable to just put the new cases for literal before these 4 cases. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user cloud-fan commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r161766225 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecision.scala --- @@ -243,17 +279,43 @@ object DecimalPrecision extends TypeCoercionRule { // Promote integers inside a binary expression with fixed-precision decimals to decimals, // and fixed-precision decimals in an expression with floats / doubles to doubles case b @ BinaryOperator(left, right) if left.dataType != right.dataType => - (left.dataType, right.dataType) match { -case (t: IntegralType, DecimalType.Fixed(p, s)) => - b.makeCopy(Array(Cast(left, DecimalType.forType(t)), right)) -case (DecimalType.Fixed(p, s), t: IntegralType) => - b.makeCopy(Array(left, Cast(right, DecimalType.forType(t -case (t, DecimalType.Fixed(p, s)) if isFloat(t) => - b.makeCopy(Array(left, Cast(right, DoubleType))) -case (DecimalType.Fixed(p, s), t) if isFloat(t) => - b.makeCopy(Array(Cast(left, DoubleType), right)) -case _ => - b - } + nondecimalLiteralAndDecimal(b).lift((left, right)).getOrElse( +nondecimalNonliteralAndDecimal(b).applyOrElse((left.dataType, right.dataType), + (_: (DataType, DataType)) => b)) } + + /** + * Type coercion for BinaryOperator in which one side is a non-decimal literal numeric, and the + * other side is a decimal. + */ + private def nondecimalLiteralAndDecimal( + b: BinaryOperator): PartialFunction[(Expression, Expression), Expression] = { +// Promote literal integers inside a binary expression with fixed-precision decimals to +// decimals. The precision and scale are the ones needed by the integer value. +case (l: Literal, r) if r.dataType.isInstanceOf[DecimalType] + && l.dataType.isInstanceOf[IntegralType] => + b.makeCopy(Array(Cast(l, DecimalType.forLiteral(l)), r)) --- End diff -- makes sense --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user cloud-fan commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r161765831 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala --- @@ -136,10 +137,52 @@ object DecimalType extends AbstractDataType { case DoubleType => DoubleDecimal } + private[sql] def forLiteral(literal: Literal): DecimalType = literal.value match { +case v: Short => fromBigDecimal(BigDecimal(v)) +case v: Int => fromBigDecimal(BigDecimal(v)) +case v: Long => fromBigDecimal(BigDecimal(v)) +case _ => forType(literal.dataType) + } + + private[sql] def fromBigDecimal(d: BigDecimal): DecimalType = { +DecimalType(Math.max(d.precision, d.scale), d.scale) + } + private[sql] def bounded(precision: Int, scale: Int): DecimalType = { DecimalType(min(precision, MAX_PRECISION), min(scale, MAX_SCALE)) } + /** + * Scale adjustment implementation is based on Hive's one, which is itself inspired to + * SQLServer's one. In particular, when a result precision is greater than + * {@link #MAX_PRECISION}, the corresponding scale is reduced to prevent the integral part of a + * result from being truncated. + * + * This method is used only when `spark.sql.decimalOperations.allowPrecisionLoss` is set to true. + * + * @param precision + * @param scale + * @return + */ + private[sql] def adjustPrecisionScale(precision: Int, scale: Int): DecimalType = { --- End diff -- ah i see, makes sense --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user mgaido91 commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r161716242 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecision.scala --- @@ -243,17 +279,43 @@ object DecimalPrecision extends TypeCoercionRule { // Promote integers inside a binary expression with fixed-precision decimals to decimals, // and fixed-precision decimals in an expression with floats / doubles to doubles case b @ BinaryOperator(left, right) if left.dataType != right.dataType => - (left.dataType, right.dataType) match { -case (t: IntegralType, DecimalType.Fixed(p, s)) => - b.makeCopy(Array(Cast(left, DecimalType.forType(t)), right)) -case (DecimalType.Fixed(p, s), t: IntegralType) => - b.makeCopy(Array(left, Cast(right, DecimalType.forType(t -case (t, DecimalType.Fixed(p, s)) if isFloat(t) => - b.makeCopy(Array(left, Cast(right, DoubleType))) -case (DecimalType.Fixed(p, s), t) if isFloat(t) => - b.makeCopy(Array(Cast(left, DoubleType), right)) -case _ => - b - } + nondecimalLiteralAndDecimal(b).lift((left, right)).getOrElse( +nondecimalNonliteralAndDecimal(b).applyOrElse((left.dataType, right.dataType), + (_: (DataType, DataType)) => b)) } + + /** + * Type coercion for BinaryOperator in which one side is a non-decimal literal numeric, and the + * other side is a decimal. + */ + private def nondecimalLiteralAndDecimal( + b: BinaryOperator): PartialFunction[(Expression, Expression), Expression] = { +// Promote literal integers inside a binary expression with fixed-precision decimals to +// decimals. The precision and scale are the ones needed by the integer value. +case (l: Literal, r) if r.dataType.isInstanceOf[DecimalType] + && l.dataType.isInstanceOf[IntegralType] => + b.makeCopy(Array(Cast(l, DecimalType.forLiteral(l)), r)) --- End diff -- if we don't do this we have many test failure in spark-hive, because spark-hive do so. Moreover, requiring more precision is not OK, since it leads to a useless loss of precision. Think of this example: you multiply a column which is DECIMAL(38, 18) by `2`. If you don't do this, `2` is considered a DECIMAL(10, 0). According to the rules, the result should be DECIMAL(38 + 10 + 1, 18), which is out of range: then according to the rules it becomes DECIMAL(38, 7), leading to potentially loosing 11 digits of the fractional part. With this change, instead, the result would be DECIMAL(38 + 1 + 1, 18), which becomes DECIMAL(38, 16), safely having a much lower precision loss. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user mgaido91 commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r161714618 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala --- @@ -136,10 +137,52 @@ object DecimalType extends AbstractDataType { case DoubleType => DoubleDecimal } + private[sql] def forLiteral(literal: Literal): DecimalType = literal.value match { +case v: Short => fromBigDecimal(BigDecimal(v)) +case v: Int => fromBigDecimal(BigDecimal(v)) +case v: Long => fromBigDecimal(BigDecimal(v)) +case _ => forType(literal.dataType) + } + + private[sql] def fromBigDecimal(d: BigDecimal): DecimalType = { +DecimalType(Math.max(d.precision, d.scale), d.scale) + } + private[sql] def bounded(precision: Int, scale: Int): DecimalType = { DecimalType(min(precision, MAX_PRECISION), min(scale, MAX_SCALE)) } + /** + * Scale adjustment implementation is based on Hive's one, which is itself inspired to + * SQLServer's one. In particular, when a result precision is greater than + * {@link #MAX_PRECISION}, the corresponding scale is reduced to prevent the integral part of a + * result from being truncated. + * + * This method is used only when `spark.sql.decimalOperations.allowPrecisionLoss` is set to true. + * + * @param precision + * @param scale + * @return + */ + private[sql] def adjustPrecisionScale(precision: Int, scale: Int): DecimalType = { --- End diff -- @cloud-fan yes, but you have to keep in mind that we are doing so only when precision is > 38. With some simple math (given `intDigits = precision - scale`), SQL server is `min(scale, scale + 38 - precision)`. Since we perform this operation only when precision is greater than 38, the second member is always the minimum. Which means that in such a case, SQL server behaves like us, ie. it takes always `38 - intDigits`. When precision is < than 38, instead we return the input precision and scale, as SQL server does. We are just using the precision instead of the intDigits for the if. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user cloud-fan commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r161656633 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala --- @@ -136,10 +137,52 @@ object DecimalType extends AbstractDataType { case DoubleType => DoubleDecimal } + private[sql] def forLiteral(literal: Literal): DecimalType = literal.value match { +case v: Short => fromBigDecimal(BigDecimal(v)) +case v: Int => fromBigDecimal(BigDecimal(v)) +case v: Long => fromBigDecimal(BigDecimal(v)) +case _ => forType(literal.dataType) + } + + private[sql] def fromBigDecimal(d: BigDecimal): DecimalType = { +DecimalType(Math.max(d.precision, d.scale), d.scale) + } + private[sql] def bounded(precision: Int, scale: Int): DecimalType = { DecimalType(min(precision, MAX_PRECISION), min(scale, MAX_SCALE)) } + /** + * Scale adjustment implementation is based on Hive's one, which is itself inspired to + * SQLServer's one. In particular, when a result precision is greater than + * {@link #MAX_PRECISION}, the corresponding scale is reduced to prevent the integral part of a + * result from being truncated. + * + * This method is used only when `spark.sql.decimalOperations.allowPrecisionLoss` is set to true. + * + * @param precision + * @param scale + * @return + */ + private[sql] def adjustPrecisionScale(precision: Int, scale: Int): DecimalType = { --- End diff -- So the rule in document is ``` val resultPrecision = 38 if (intDigits < 32) { // This means scale > 6, as iniDigits = precision - scale and precision > 38 val maxScale = 38 - intDigits val resultScale = min(scale, maxScale) } else { if (scale < 6) { // can't round as scale is already small val resultScale = scale } else { val resltScale = 6 } } ``` I think this is a little different from the current rule ``` val minScaleValue = Math.min(scale, 6) val resultScale = max(38 - intDigits, minScaleValue) ``` Think aboout the case `iniDigits < 32`, SQL server is `min(scale, 38 - intDigits)`, we are `38 - intDigits` --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user cloud-fan commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r161655115 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecision.scala --- @@ -243,17 +279,43 @@ object DecimalPrecision extends TypeCoercionRule { // Promote integers inside a binary expression with fixed-precision decimals to decimals, // and fixed-precision decimals in an expression with floats / doubles to doubles case b @ BinaryOperator(left, right) if left.dataType != right.dataType => - (left.dataType, right.dataType) match { -case (t: IntegralType, DecimalType.Fixed(p, s)) => - b.makeCopy(Array(Cast(left, DecimalType.forType(t)), right)) -case (DecimalType.Fixed(p, s), t: IntegralType) => - b.makeCopy(Array(left, Cast(right, DecimalType.forType(t -case (t, DecimalType.Fixed(p, s)) if isFloat(t) => - b.makeCopy(Array(left, Cast(right, DoubleType))) -case (DecimalType.Fixed(p, s), t) if isFloat(t) => - b.makeCopy(Array(Cast(left, DoubleType), right)) -case _ => - b - } + nondecimalLiteralAndDecimal(b).lift((left, right)).getOrElse( +nondecimalNonliteralAndDecimal(b).applyOrElse((left.dataType, right.dataType), + (_: (DataType, DataType)) => b)) } + + /** + * Type coercion for BinaryOperator in which one side is a non-decimal literal numeric, and the + * other side is a decimal. + */ + private def nondecimalLiteralAndDecimal( + b: BinaryOperator): PartialFunction[(Expression, Expression), Expression] = { +// Promote literal integers inside a binary expression with fixed-precision decimals to +// decimals. The precision and scale are the ones needed by the integer value. +case (l: Literal, r) if r.dataType.isInstanceOf[DecimalType] + && l.dataType.isInstanceOf[IntegralType] => + b.makeCopy(Array(Cast(l, DecimalType.forLiteral(l)), r)) --- End diff -- What if we don't do this? Requiring more precision seems OK as now we allow precision lose. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user mgaido91 commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r161615248 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecision.scala --- @@ -93,41 +97,76 @@ object DecimalPrecision extends TypeCoercionRule { case e: BinaryArithmetic if e.left.isInstanceOf[PromotePrecision] => e case Add(e1 @ DecimalType.Expression(p1, s1), e2 @ DecimalType.Expression(p2, s2)) => - val dt = DecimalType.bounded(max(s1, s2) + max(p1 - s1, p2 - s2) + 1, max(s1, s2)) - CheckOverflow(Add(promotePrecision(e1, dt), promotePrecision(e2, dt)), dt) + val resultScale = max(s1, s2) + val resultType = if (SQLConf.get.decimalOperationsAllowPrecisionLoss) { +DecimalType.adjustPrecisionScale(max(p1 - s1, p2 - s2) + resultScale + 1, --- End diff -- yes, that may be a difference indeed. But I think it is a minor one, since 99% of the cases the precision is exceeded only in multiplications and divisions. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user mgaido91 commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r161614951 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala --- @@ -136,10 +137,52 @@ object DecimalType extends AbstractDataType { case DoubleType => DoubleDecimal } + private[sql] def forLiteral(literal: Literal): DecimalType = literal.value match { +case v: Short => fromBigDecimal(BigDecimal(v)) +case v: Int => fromBigDecimal(BigDecimal(v)) +case v: Long => fromBigDecimal(BigDecimal(v)) +case _ => forType(literal.dataType) + } + + private[sql] def fromBigDecimal(d: BigDecimal): DecimalType = { +DecimalType(Math.max(d.precision, d.scale), d.scale) + } + private[sql] def bounded(precision: Int, scale: Int): DecimalType = { DecimalType(min(precision, MAX_PRECISION), min(scale, MAX_SCALE)) } + /** + * Scale adjustment implementation is based on Hive's one, which is itself inspired to + * SQLServer's one. In particular, when a result precision is greater than + * {@link #MAX_PRECISION}, the corresponding scale is reduced to prevent the integral part of a + * result from being truncated. + * + * This method is used only when `spark.sql.decimalOperations.allowPrecisionLoss` is set to true. + * + * @param precision + * @param scale + * @return + */ + private[sql] def adjustPrecisionScale(precision: Int, scale: Int): DecimalType = { --- End diff -- sorry, but I think this is exactly the same which is described there. The implementation might seem doing different things but actually the result will be the same. They both take the min between 6 and the desired scale if the precision is not enough to represent the whole scale. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user gatorsmile commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r161593806 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala --- @@ -136,10 +137,52 @@ object DecimalType extends AbstractDataType { case DoubleType => DoubleDecimal } + private[sql] def forLiteral(literal: Literal): DecimalType = literal.value match { +case v: Short => fromBigDecimal(BigDecimal(v)) +case v: Int => fromBigDecimal(BigDecimal(v)) +case v: Long => fromBigDecimal(BigDecimal(v)) +case _ => forType(literal.dataType) + } + + private[sql] def fromBigDecimal(d: BigDecimal): DecimalType = { +DecimalType(Math.max(d.precision, d.scale), d.scale) + } + private[sql] def bounded(precision: Int, scale: Int): DecimalType = { DecimalType(min(precision, MAX_PRECISION), min(scale, MAX_SCALE)) } + /** + * Scale adjustment implementation is based on Hive's one, which is itself inspired to + * SQLServer's one. In particular, when a result precision is greater than + * {@link #MAX_PRECISION}, the corresponding scale is reduced to prevent the integral part of a + * result from being truncated. + * + * This method is used only when `spark.sql.decimalOperations.allowPrecisionLoss` is set to true. + * + * @param precision + * @param scale + * @return + */ + private[sql] def adjustPrecisionScale(precision: Int, scale: Int): DecimalType = { --- End diff -- The logics in this adjustment function is also different from the MS SQL Server docs. > In multiplication and division operations we need precision - scale places to store the integral part of the result. The scale might be reduced using the following rules: > The resulting scale is reduced to min(scale, 38 – (precision-scale)) if the integral part is less than 32, because it cannot be greater than 38 – (precision-scale). Result might be rounded in this case. > The scale will not be changed if it is less than 6 and if the integral part is greater than 32. In this case, overflow error might be raised if it cannot fit into decimal(38, scale) > The scale will be set to 6 if it is greater than 6 and if the integral part is greater than 32. In this case, both integral part and scale would be reduced and resulting type is decimal(38,6). Result might be rounded to 6 decimal places or overflow error will be thrown if integral part cannot fit into 32 digits. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user gatorsmile commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r161593215 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecision.scala --- @@ -93,41 +97,76 @@ object DecimalPrecision extends TypeCoercionRule { case e: BinaryArithmetic if e.left.isInstanceOf[PromotePrecision] => e case Add(e1 @ DecimalType.Expression(p1, s1), e2 @ DecimalType.Expression(p2, s2)) => - val dt = DecimalType.bounded(max(s1, s2) + max(p1 - s1, p2 - s2) + 1, max(s1, s2)) - CheckOverflow(Add(promotePrecision(e1, dt), promotePrecision(e2, dt)), dt) + val resultScale = max(s1, s2) + val resultType = if (SQLConf.get.decimalOperationsAllowPrecisionLoss) { +DecimalType.adjustPrecisionScale(max(p1 - s1, p2 - s2) + resultScale + 1, --- End diff -- This is an example. `adjustPrecisionScale` is also be applied for all the operations. However, the doc shows the adjustment is only applicable to multiplication and division. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user mgaido91 commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r161507743 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecision.scala --- @@ -42,8 +43,10 @@ import org.apache.spark.sql.types._ * e1 / e2 p1 - s1 + s2 + max(6, s1 + p2 + 1) max(6, s1 + p2 + 1) * e1 % e2 min(p1-s1, p2-s2) + max(s1, s2) max(s1, s2) * e1 union e2 max(s1, s2) + max(p1-s1, p2-s2) max(s1, s2) - * sum(e1) p1 + 10 s1 - * avg(e1) p1 + 4 s1 + 4 + * + * When `spark.sql.decimalOperations.allowTruncat` is set to true, if the precision / scale needed + * are out of the range of available values, the scale is reduced up to 6, in order to prevent the --- End diff -- Hive has it, but in the documentation it is not explained. And in the comments it just references the blog I referenced, but then removed according to @gatorsmile's comment. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user cloud-fan commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r161503031 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecision.scala --- @@ -42,8 +43,10 @@ import org.apache.spark.sql.types._ * e1 / e2 p1 - s1 + s2 + max(6, s1 + p2 + 1) max(6, s1 + p2 + 1) * e1 % e2 min(p1-s1, p2-s2) + max(s1, s2) max(s1, s2) * e1 union e2 max(s1, s2) + max(p1-s1, p2-s2) max(s1, s2) - * sum(e1) p1 + 10 s1 - * avg(e1) p1 + 4 s1 + 4 + * + * When `spark.sql.decimalOperations.allowTruncat` is set to true, if the precision / scale needed + * are out of the range of available values, the scale is reduced up to 6, in order to prevent the --- End diff -- Actually we already referred a commercial RDBMS in L33... --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user cloud-fan commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r161502866 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecision.scala --- @@ -42,8 +43,10 @@ import org.apache.spark.sql.types._ * e1 / e2 p1 - s1 + s2 + max(6, s1 + p2 + 1) max(6, s1 + p2 + 1) * e1 % e2 min(p1-s1, p2-s2) + max(s1, s2) max(s1, s2) * e1 union e2 max(s1, s2) + max(p1-s1, p2-s2) max(s1, s2) - * sum(e1) p1 + 10 s1 - * avg(e1) p1 + 4 s1 + 4 + * + * When `spark.sql.decimalOperations.allowTruncat` is set to true, if the precision / scale needed + * are out of the range of available values, the scale is reduced up to 6, in order to prevent the --- End diff -- Did any open source RDBMS have this rule? --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user mgaido91 commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r161502564 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecision.scala --- @@ -42,8 +43,10 @@ import org.apache.spark.sql.types._ * e1 / e2 p1 - s1 + s2 + max(6, s1 + p2 + 1) max(6, s1 + p2 + 1) * e1 % e2 min(p1-s1, p2-s2) + max(s1, s2) max(s1, s2) * e1 union e2 max(s1, s2) + max(p1-s1, p2-s2) max(s1, s2) - * sum(e1) p1 + 10 s1 - * avg(e1) p1 + 4 s1 + 4 + * + * When `spark.sql.decimalOperations.allowTruncat` is set to true, if the precision / scale needed + * are out of the range of available values, the scale is reduced up to 6, in order to prevent the --- End diff -- I did, but @gatorsmile told me to remove it: https://github.com/apache/spark/pull/20023#discussion_r159117817. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user cloud-fan commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r161501303 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecision.scala --- @@ -42,8 +43,10 @@ import org.apache.spark.sql.types._ * e1 / e2 p1 - s1 + s2 + max(6, s1 + p2 + 1) max(6, s1 + p2 + 1) * e1 % e2 min(p1-s1, p2-s2) + max(s1, s2) max(s1, s2) * e1 union e2 max(s1, s2) + max(p1-s1, p2-s2) max(s1, s2) - * sum(e1) p1 + 10 s1 - * avg(e1) p1 + 4 s1 + 4 + * + * When `spark.sql.decimalOperations.allowTruncat` is set to true, if the precision / scale needed + * are out of the range of available values, the scale is reduced up to 6, in order to prevent the --- End diff -- sorry, typo, "put some reference"... We can put a link to a document of a mainstream RDBMS and say this rule follows xxx... --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user mgaido91 commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r161494664 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecision.scala --- @@ -42,8 +43,10 @@ import org.apache.spark.sql.types._ * e1 / e2 p1 - s1 + s2 + max(6, s1 + p2 + 1) max(6, s1 + p2 + 1) * e1 % e2 min(p1-s1, p2-s2) + max(s1, s2) max(s1, s2) * e1 union e2 max(s1, s2) + max(p1-s1, p2-s2) max(s1, s2) - * sum(e1) p1 + 10 s1 - * avg(e1) p1 + 4 s1 + 4 + * + * When `spark.sql.decimalOperations.allowTruncat` is set to true, if the precision / scale needed + * are out of the range of available values, the scale is reduced up to 6, in order to prevent the --- End diff -- no, we would return the a type with scale 6 and in `CheckOverflow` all the numbers which don't fit will be translated to `NULL`, since this is current Spark behavior in such cases. May I kindly ask you to elaborate a but more what you mean by "push some reference"? Thanks. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user cloud-fan commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r161451478 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecision.scala --- @@ -93,41 +97,76 @@ object DecimalPrecision extends TypeCoercionRule { case e: BinaryArithmetic if e.left.isInstanceOf[PromotePrecision] => e case Add(e1 @ DecimalType.Expression(p1, s1), e2 @ DecimalType.Expression(p2, s2)) => - val dt = DecimalType.bounded(max(s1, s2) + max(p1 - s1, p2 - s2) + 1, max(s1, s2)) - CheckOverflow(Add(promotePrecision(e1, dt), promotePrecision(e2, dt)), dt) + val resultType = if (SQLConf.get.decimalOperationsAllowTruncat) { +val resultScale = max(s1, s2) --- End diff -- we can put this before the `if` --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user cloud-fan commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r161451267 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecision.scala --- @@ -42,8 +43,10 @@ import org.apache.spark.sql.types._ * e1 / e2 p1 - s1 + s2 + max(6, s1 + p2 + 1) max(6, s1 + p2 + 1) * e1 % e2 min(p1-s1, p2-s2) + max(s1, s2) max(s1, s2) * e1 union e2 max(s1, s2) + max(p1-s1, p2-s2) max(s1, s2) - * sum(e1) p1 + 10 s1 - * avg(e1) p1 + 4 s1 + 4 + * + * When `spark.sql.decimalOperations.allowTruncat` is set to true, if the precision / scale needed + * are out of the range of available values, the scale is reduced up to 6, in order to prevent the --- End diff -- if scale needs to be less than 6, we would fail the analysis, right? --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user cloud-fan commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r161450453 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecision.scala --- @@ -42,8 +43,10 @@ import org.apache.spark.sql.types._ * e1 / e2 p1 - s1 + s2 + max(6, s1 + p2 + 1) max(6, s1 + p2 + 1) * e1 % e2 min(p1-s1, p2-s2) + max(s1, s2) max(s1, s2) * e1 union e2 max(s1, s2) + max(p1-s1, p2-s2) max(s1, s2) - * sum(e1) p1 + 10 s1 - * avg(e1) p1 + 4 s1 + 4 + * + * When `spark.sql.decimalOperations.allowTruncat` is set to true, if the precision / scale needed + * are out of the range of available values, the scale is reduced up to 6, in order to prevent the --- End diff -- it's better to push some reference about where we get this rule. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user cloud-fan commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r161449017 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala --- @@ -1048,6 +1048,16 @@ object SQLConf { .booleanConf .createWithDefault(true) + val DECIMAL_OPERATIONS_ALLOW_TRUNCAT = +buildConf("spark.sql.decimalOperations.allowTruncat") --- End diff -- Sorry that was my typo... `allowPrecisionLoss` SGTM --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user mgaido91 commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r161309046 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala --- @@ -1048,6 +1048,16 @@ object SQLConf { .booleanConf .createWithDefault(true) + val DECIMAL_OPERATIONS_ALLOW_TRUNCAT = +buildConf("spark.sql.decimalOperations.allowTruncat") --- End diff -- thanks. This is what @cloud-fan suggested. Actually we are not truncating, but rounding. Personally, I'd prefer `allowPrecisionLoss`, but I am fine with any value. WDYT @cloud-fan @gatorsmile? --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user dongjoon-hyun commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r161297231 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala --- @@ -1048,6 +1048,16 @@ object SQLConf { .booleanConf .createWithDefault(true) + val DECIMAL_OPERATIONS_ALLOW_TRUNCAT = +buildConf("spark.sql.decimalOperations.allowTruncat") --- End diff -- `allowTruncat` -> `allowTruncate`? --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user dongjoon-hyun commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r161297154 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala --- @@ -1048,6 +1048,16 @@ object SQLConf { .booleanConf .createWithDefault(true) + val DECIMAL_OPERATIONS_ALLOW_TRUNCAT = --- End diff -- type? `TRUNCAT` -> `TRUNCATE`? --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user mgaido91 commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r160394589 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala --- @@ -117,6 +117,7 @@ object DecimalType extends AbstractDataType { val MAX_SCALE = 38 val SYSTEM_DEFAULT: DecimalType = DecimalType(MAX_PRECISION, 18) val USER_DEFAULT: DecimalType = DecimalType(10, 0) + val MINIMUM_ADJUSTED_SCALE = 6 --- End diff -- ok, I'll go with that, thanks @cloud-fan. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user cloud-fan commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r160376186 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala --- @@ -117,6 +117,7 @@ object DecimalType extends AbstractDataType { val MAX_SCALE = 38 val SYSTEM_DEFAULT: DecimalType = DecimalType(MAX_PRECISION, 18) val USER_DEFAULT: DecimalType = DecimalType(10, 0) + val MINIMUM_ADJUSTED_SCALE = 6 --- End diff -- We should make it an internal conf and remove it after some releases. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user cloud-fan commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r160376096 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala --- @@ -117,6 +117,7 @@ object DecimalType extends AbstractDataType { val MAX_SCALE = 38 val SYSTEM_DEFAULT: DecimalType = DecimalType(MAX_PRECISION, 18) val USER_DEFAULT: DecimalType = DecimalType(10, 0) + val MINIMUM_ADJUSTED_SCALE = 6 --- End diff -- how about `spark.sql.decimalOperations.allowTruncat`? Let's leave the mode stuff to the type coercion mode. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user mgaido91 commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r160361785 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala --- @@ -117,6 +117,7 @@ object DecimalType extends AbstractDataType { val MAX_SCALE = 38 val SYSTEM_DEFAULT: DecimalType = DecimalType(MAX_PRECISION, 18) val USER_DEFAULT: DecimalType = DecimalType(10, 0) + val MINIMUM_ADJUSTED_SCALE = 6 --- End diff -- @gatorsmile what about `spark.sql.decimalOperations.mode` which defaults to `native` and accepts also `hive` (and in future also `sql2011` for throwing exception instead of returning NULL)? --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user mgaido91 commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r160147366 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala --- @@ -117,6 +117,7 @@ object DecimalType extends AbstractDataType { val MAX_SCALE = 38 val SYSTEM_DEFAULT: DecimalType = DecimalType(MAX_PRECISION, 18) val USER_DEFAULT: DecimalType = DecimalType(10, 0) + val MINIMUM_ADJUSTED_SCALE = 6 --- End diff -- Yes, I followed Hive's implementation which works like this and applies this 6 digits minimum to all operations. This means that SQLServer allows to round more digits than us in those cases, ie. we ensure at least 6 digits for the scale, while SQLServer doesn't. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user gatorsmile commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r159122042 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala --- @@ -117,6 +117,7 @@ object DecimalType extends AbstractDataType { val MAX_SCALE = 38 val SYSTEM_DEFAULT: DecimalType = DecimalType(MAX_PRECISION, 18) val USER_DEFAULT: DecimalType = DecimalType(10, 0) + val MINIMUM_ADJUSTED_SCALE = 6 --- End diff -- Before naming a conf, I need to understand the rule you are following. https://docs.microsoft.com/en-us/sql/t-sql/data-types/precision-scale-and-length-transact-sql The SQL Server only applies `MINIMUM_ADJUSTED_SCALE` for multiplication and division. However, in your impl, you are using it for all the BinaryArithmetic operators? --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user gatorsmile commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r159117817 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala --- @@ -136,10 +137,54 @@ object DecimalType extends AbstractDataType { case DoubleType => DoubleDecimal } + private[sql] def forLiteral(literal: Literal): DecimalType = literal.value match { +case v: Short => fromBigDecimal(BigDecimal(v)) +case v: Int => fromBigDecimal(BigDecimal(v)) +case v: Long => fromBigDecimal(BigDecimal(v)) +case _ => forType(literal.dataType) + } + + private[sql] def fromBigDecimal(d: BigDecimal): DecimalType = { +DecimalType(Math.max(d.precision, d.scale), d.scale) + } + private[sql] def bounded(precision: Int, scale: Int): DecimalType = { DecimalType(min(precision, MAX_PRECISION), min(scale, MAX_SCALE)) } + // scalastyle:off line.size.limit + /** + * Decimal implementation is based on Hive's one, which is itself inspired to SQLServer's one. + * In particular, when a result precision is greater than {@link #MAX_PRECISION}, the + * corresponding scale is reduced to prevent the integral part of a result from being truncated. + * + * For further reference, please see + * https://blogs.msdn.microsoft.com/sqlprogrammability/2006/03/29/multiplication-and-division-with-numerics/. --- End diff -- Please remove the web link to the commercial products. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user mgaido91 commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r158242030 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala --- @@ -136,10 +137,54 @@ object DecimalType extends AbstractDataType { case DoubleType => DoubleDecimal } + private[sql] def forLiteral(literal: Literal): DecimalType = literal.value match { +case v: Short => fromBigDecimal(BigDecimal(v)) +case v: Int => fromBigDecimal(BigDecimal(v)) +case v: Long => fromBigDecimal(BigDecimal(v)) +case _ => forType(literal.dataType) + } + + private[sql] def fromBigDecimal(d: BigDecimal): DecimalType = { +DecimalType(Math.max(d.precision, d.scale), d.scale) + } + private[sql] def bounded(precision: Int, scale: Int): DecimalType = { DecimalType(min(precision, MAX_PRECISION), min(scale, MAX_SCALE)) } + // scalastyle:off line.size.limit + /** + * Decimal implementation is based on Hive's one, which is itself inspired to SQLServer's one. + * In particular, when a result precision is greater than {@link #MAX_PRECISION}, the + * corresponding scale is reduced to prevent the integral part of a result from being truncated. + * + * For further reference, please see + * https://blogs.msdn.microsoft.com/sqlprogrammability/2006/03/29/multiplication-and-division-with-numerics/. + * + * @param precision + * @param scale + * @return + */ + // scalastyle:on line.size.limit + private[sql] def adjustPrecisionScale(precision: Int, scale: Int): DecimalType = { +// Assumptions: +// precision >= scale +// scale >= 0 +if (precision <= MAX_PRECISION) { + // Adjustment only needed when we exceed max precision + DecimalType(precision, scale) +} else { + // Precision/scale exceed maximum precision. Result must be adjusted to MAX_PRECISION. + val intDigits = precision - scale + // If original scale less than MINIMUM_ADJUSTED_SCALE, use original scale value; otherwise + // preserve at least MINIMUM_ADJUSTED_SCALE fractional digits + val minScaleValue = Math.min(scale, MINIMUM_ADJUSTED_SCALE) --- End diff -- Yes, sorry, my answer was very poor, I will rephrase. `scale` contains the scale which we need to represent the values without any precision loss. What we are doing here is saying that the lower bound for the scale is either the scale that we need to correctly represent the value or the `MINIMUM_ADJUSTED_SCALE`. After this, in the line below we state that the scale we will use is the max between the number of digits of the precision we don't need on the left of the dot and this `minScaleValue`: ie. even though in some cases we might need a scale higher than `MINIMUM_ADJUSTED_SCALE`, but the number of digits needed on the left on the dot would force us to have a scale lower than `MINIMUM_ADJUSTED_SCALE`, we enforce that we will maintain at least `MINIMUM_ADJUSTED_SCALE`. We can't let the scale be lower that this threshold, even though it would be needed to enforce that we don't loose digits on the left of the dot. Please refer also to the blog post I linked in the comment above fo r further (hopefully better) explanation. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user viirya commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r158238981 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala --- @@ -136,10 +137,54 @@ object DecimalType extends AbstractDataType { case DoubleType => DoubleDecimal } + private[sql] def forLiteral(literal: Literal): DecimalType = literal.value match { +case v: Short => fromBigDecimal(BigDecimal(v)) +case v: Int => fromBigDecimal(BigDecimal(v)) +case v: Long => fromBigDecimal(BigDecimal(v)) +case _ => forType(literal.dataType) + } + + private[sql] def fromBigDecimal(d: BigDecimal): DecimalType = { +DecimalType(Math.max(d.precision, d.scale), d.scale) + } + private[sql] def bounded(precision: Int, scale: Int): DecimalType = { DecimalType(min(precision, MAX_PRECISION), min(scale, MAX_SCALE)) } + // scalastyle:off line.size.limit + /** + * Decimal implementation is based on Hive's one, which is itself inspired to SQLServer's one. + * In particular, when a result precision is greater than {@link #MAX_PRECISION}, the + * corresponding scale is reduced to prevent the integral part of a result from being truncated. + * + * For further reference, please see + * https://blogs.msdn.microsoft.com/sqlprogrammability/2006/03/29/multiplication-and-division-with-numerics/. + * + * @param precision + * @param scale + * @return + */ + // scalastyle:on line.size.limit + private[sql] def adjustPrecisionScale(precision: Int, scale: Int): DecimalType = { +// Assumptions: +// precision >= scale +// scale >= 0 +if (precision <= MAX_PRECISION) { + // Adjustment only needed when we exceed max precision + DecimalType(precision, scale) +} else { + // Precision/scale exceed maximum precision. Result must be adjusted to MAX_PRECISION. + val intDigits = precision - scale + // If original scale less than MINIMUM_ADJUSTED_SCALE, use original scale value; otherwise + // preserve at least MINIMUM_ADJUSTED_SCALE fractional digits + val minScaleValue = Math.min(scale, MINIMUM_ADJUSTED_SCALE) --- End diff -- > We can't have a scale lower that that... Don't you get a scale lower than `MINIMUM_ADJUSTED_SCALE` by `Math.min(scale, MINIMUM_ADJUSTED_SCALE)`? --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user mgaido91 commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r158226546 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala --- @@ -136,10 +137,54 @@ object DecimalType extends AbstractDataType { case DoubleType => DoubleDecimal } + private[sql] def forLiteral(literal: Literal): DecimalType = literal.value match { +case v: Short => fromBigDecimal(BigDecimal(v)) +case v: Int => fromBigDecimal(BigDecimal(v)) +case v: Long => fromBigDecimal(BigDecimal(v)) +case _ => forType(literal.dataType) + } + + private[sql] def fromBigDecimal(d: BigDecimal): DecimalType = { +DecimalType(Math.max(d.precision, d.scale), d.scale) + } + private[sql] def bounded(precision: Int, scale: Int): DecimalType = { DecimalType(min(precision, MAX_PRECISION), min(scale, MAX_SCALE)) } + // scalastyle:off line.size.limit + /** + * Decimal implementation is based on Hive's one, which is itself inspired to SQLServer's one. + * In particular, when a result precision is greater than {@link #MAX_PRECISION}, the + * corresponding scale is reduced to prevent the integral part of a result from being truncated. + * + * For further reference, please see + * https://blogs.msdn.microsoft.com/sqlprogrammability/2006/03/29/multiplication-and-division-with-numerics/. + * + * @param precision + * @param scale + * @return + */ + // scalastyle:on line.size.limit + private[sql] def adjustPrecisionScale(precision: Int, scale: Int): DecimalType = { +// Assumptions: +// precision >= scale +// scale >= 0 +if (precision <= MAX_PRECISION) { + // Adjustment only needed when we exceed max precision + DecimalType(precision, scale) +} else { + // Precision/scale exceed maximum precision. Result must be adjusted to MAX_PRECISION. + val intDigits = precision - scale + // If original scale less than MINIMUM_ADJUSTED_SCALE, use original scale value; otherwise + // preserve at least MINIMUM_ADJUSTED_SCALE fractional digits + val minScaleValue = Math.min(scale, MINIMUM_ADJUSTED_SCALE) + val adjustedScale = Math.max(MAX_PRECISION - intDigits, minScaleValue) --- End diff -- It is `max` because we take either the scale which would prevent a loss of "space" for `intDigits`, ie. the part on the left of the dot, or the `minScaleValue`, which is the scale we are ensuring to provide at least. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user mgaido91 commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r158225832 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala --- @@ -136,10 +137,54 @@ object DecimalType extends AbstractDataType { case DoubleType => DoubleDecimal } + private[sql] def forLiteral(literal: Literal): DecimalType = literal.value match { +case v: Short => fromBigDecimal(BigDecimal(v)) +case v: Int => fromBigDecimal(BigDecimal(v)) +case v: Long => fromBigDecimal(BigDecimal(v)) +case _ => forType(literal.dataType) + } + + private[sql] def fromBigDecimal(d: BigDecimal): DecimalType = { +DecimalType(Math.max(d.precision, d.scale), d.scale) + } + private[sql] def bounded(precision: Int, scale: Int): DecimalType = { DecimalType(min(precision, MAX_PRECISION), min(scale, MAX_SCALE)) } + // scalastyle:off line.size.limit + /** + * Decimal implementation is based on Hive's one, which is itself inspired to SQLServer's one. + * In particular, when a result precision is greater than {@link #MAX_PRECISION}, the + * corresponding scale is reduced to prevent the integral part of a result from being truncated. + * + * For further reference, please see + * https://blogs.msdn.microsoft.com/sqlprogrammability/2006/03/29/multiplication-and-division-with-numerics/. + * + * @param precision + * @param scale + * @return + */ + // scalastyle:on line.size.limit + private[sql] def adjustPrecisionScale(precision: Int, scale: Int): DecimalType = { +// Assumptions: +// precision >= scale +// scale >= 0 +if (precision <= MAX_PRECISION) { + // Adjustment only needed when we exceed max precision + DecimalType(precision, scale) +} else { + // Precision/scale exceed maximum precision. Result must be adjusted to MAX_PRECISION. + val intDigits = precision - scale + // If original scale less than MINIMUM_ADJUSTED_SCALE, use original scale value; otherwise + // preserve at least MINIMUM_ADJUSTED_SCALE fractional digits + val minScaleValue = Math.min(scale, MINIMUM_ADJUSTED_SCALE) --- End diff -- It is the `MINIMUM_ADJUSTED_SCALE`. We can't have a scale lower that that, even though we would need not to loose precision. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user mgaido91 commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r158225632 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala --- @@ -136,10 +137,54 @@ object DecimalType extends AbstractDataType { case DoubleType => DoubleDecimal } + private[sql] def forLiteral(literal: Literal): DecimalType = literal.value match { +case v: Short => fromBigDecimal(BigDecimal(v)) +case v: Int => fromBigDecimal(BigDecimal(v)) +case v: Long => fromBigDecimal(BigDecimal(v)) +case _ => forType(literal.dataType) + } + + private[sql] def fromBigDecimal(d: BigDecimal): DecimalType = { +DecimalType(Math.max(d.precision, d.scale), d.scale) + } + private[sql] def bounded(precision: Int, scale: Int): DecimalType = { DecimalType(min(precision, MAX_PRECISION), min(scale, MAX_SCALE)) } + // scalastyle:off line.size.limit + /** + * Decimal implementation is based on Hive's one, which is itself inspired to SQLServer's one. + * In particular, when a result precision is greater than {@link #MAX_PRECISION}, the + * corresponding scale is reduced to prevent the integral part of a result from being truncated. + * + * For further reference, please see + * https://blogs.msdn.microsoft.com/sqlprogrammability/2006/03/29/multiplication-and-division-with-numerics/. + * + * @param precision + * @param scale + * @return + */ + // scalastyle:on line.size.limit + private[sql] def adjustPrecisionScale(precision: Int, scale: Int): DecimalType = { +// Assumptions: +// precision >= scale +// scale >= 0 +if (precision <= MAX_PRECISION) { + // Adjustment only needed when we exceed max precision + DecimalType(precision, scale) --- End diff -- this is prevented outside this function. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user mgaido91 commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r158225505 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala --- @@ -136,10 +137,54 @@ object DecimalType extends AbstractDataType { case DoubleType => DoubleDecimal } + private[sql] def forLiteral(literal: Literal): DecimalType = literal.value match { +case v: Short => fromBigDecimal(BigDecimal(v)) --- End diff -- No, please see my comments above. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user mgaido91 commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r158225412 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala --- @@ -136,10 +137,54 @@ object DecimalType extends AbstractDataType { case DoubleType => DoubleDecimal } + private[sql] def forLiteral(literal: Literal): DecimalType = literal.value match { --- End diff -- yes, please see my comment above for an example. Thanks. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user mgaido91 commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r158225279 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecision.scala --- @@ -243,17 +248,43 @@ object DecimalPrecision extends TypeCoercionRule { // Promote integers inside a binary expression with fixed-precision decimals to decimals, // and fixed-precision decimals in an expression with floats / doubles to doubles case b @ BinaryOperator(left, right) if left.dataType != right.dataType => - (left.dataType, right.dataType) match { -case (t: IntegralType, DecimalType.Fixed(p, s)) => - b.makeCopy(Array(Cast(left, DecimalType.forType(t)), right)) -case (DecimalType.Fixed(p, s), t: IntegralType) => - b.makeCopy(Array(left, Cast(right, DecimalType.forType(t -case (t, DecimalType.Fixed(p, s)) if isFloat(t) => - b.makeCopy(Array(left, Cast(right, DoubleType))) -case (DecimalType.Fixed(p, s), t) if isFloat(t) => - b.makeCopy(Array(Cast(left, DoubleType), right)) -case _ => - b - } + nondecimalLiteralAndDecimal(b).lift((left, right)).getOrElse( +nondecimalNonliteralAndDecimal(b).applyOrElse((left.dataType, right.dataType), + (_: (DataType, DataType)) => b)) } + + /** + * Type coercion for BinaryOperator in which one side is a non-decimal literal numeric, and the + * other side is a decimal. + */ + private def nondecimalLiteralAndDecimal( --- End diff -- Yes, it is. If we don't introduce this, we have a failure in Hive compatibility tests, because Hive use the exact precision and scale needed by the literals, while we, before this change, were using conservative values for each type. For instance, if we have a `select 123.12345*3`, before this change `3` would have been interpreted as `Decimal(10, 0)`, which is the type for integers. After the change, `3` would become `Decimal(1, 0)`, as Hive does. This prevents from needing more precision that what is actually needed. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user mgaido91 commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r158224551 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala --- @@ -136,10 +137,54 @@ object DecimalType extends AbstractDataType { case DoubleType => DoubleDecimal } + private[sql] def forLiteral(literal: Literal): DecimalType = literal.value match { +case v: Short => fromBigDecimal(BigDecimal(v)) +case v: Int => fromBigDecimal(BigDecimal(v)) +case v: Long => fromBigDecimal(BigDecimal(v)) +case _ => forType(literal.dataType) + } + + private[sql] def fromBigDecimal(d: BigDecimal): DecimalType = { +DecimalType(Math.max(d.precision, d.scale), d.scale) + } + private[sql] def bounded(precision: Int, scale: Int): DecimalType = { DecimalType(min(precision, MAX_PRECISION), min(scale, MAX_SCALE)) } + // scalastyle:off line.size.limit + /** + * Decimal implementation is based on Hive's one, which is itself inspired to SQLServer's one. + * In particular, when a result precision is greater than {@link #MAX_PRECISION}, the + * corresponding scale is reduced to prevent the integral part of a result from being truncated. + * + * For further reference, please see + * https://blogs.msdn.microsoft.com/sqlprogrammability/2006/03/29/multiplication-and-division-with-numerics/. + * + * @param precision + * @param scale + * @return + */ + // scalastyle:on line.size.limit + private[sql] def adjustPrecisionScale(precision: Int, scale: Int): DecimalType = { +// Assumptions: +// precision >= scale +// scale >= 0 --- End diff -- I can add it even though it is not needed... there is no way we can violate those constraints. If you believe it is better to use assert, I will do that. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user viirya commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r158205151 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecision.scala --- @@ -243,17 +248,43 @@ object DecimalPrecision extends TypeCoercionRule { // Promote integers inside a binary expression with fixed-precision decimals to decimals, // and fixed-precision decimals in an expression with floats / doubles to doubles case b @ BinaryOperator(left, right) if left.dataType != right.dataType => - (left.dataType, right.dataType) match { -case (t: IntegralType, DecimalType.Fixed(p, s)) => - b.makeCopy(Array(Cast(left, DecimalType.forType(t)), right)) -case (DecimalType.Fixed(p, s), t: IntegralType) => - b.makeCopy(Array(left, Cast(right, DecimalType.forType(t -case (t, DecimalType.Fixed(p, s)) if isFloat(t) => - b.makeCopy(Array(left, Cast(right, DoubleType))) -case (DecimalType.Fixed(p, s), t) if isFloat(t) => - b.makeCopy(Array(Cast(left, DoubleType), right)) -case _ => - b - } + nondecimalLiteralAndDecimal(b).lift((left, right)).getOrElse( +nondecimalNonliteralAndDecimal(b).applyOrElse((left.dataType, right.dataType), + (_: (DataType, DataType)) => b)) } + + /** + * Type coercion for BinaryOperator in which one side is a non-decimal literal numeric, and the + * other side is a decimal. + */ + private def nondecimalLiteralAndDecimal( --- End diff -- Is this rule newly introduced? --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user viirya commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r158206693 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala --- @@ -136,10 +137,54 @@ object DecimalType extends AbstractDataType { case DoubleType => DoubleDecimal } + private[sql] def forLiteral(literal: Literal): DecimalType = literal.value match { +case v: Short => fromBigDecimal(BigDecimal(v)) +case v: Int => fromBigDecimal(BigDecimal(v)) +case v: Long => fromBigDecimal(BigDecimal(v)) +case _ => forType(literal.dataType) + } + + private[sql] def fromBigDecimal(d: BigDecimal): DecimalType = { +DecimalType(Math.max(d.precision, d.scale), d.scale) + } + private[sql] def bounded(precision: Int, scale: Int): DecimalType = { DecimalType(min(precision, MAX_PRECISION), min(scale, MAX_SCALE)) } + // scalastyle:off line.size.limit + /** + * Decimal implementation is based on Hive's one, which is itself inspired to SQLServer's one. + * In particular, when a result precision is greater than {@link #MAX_PRECISION}, the + * corresponding scale is reduced to prevent the integral part of a result from being truncated. + * + * For further reference, please see + * https://blogs.msdn.microsoft.com/sqlprogrammability/2006/03/29/multiplication-and-division-with-numerics/. + * + * @param precision + * @param scale + * @return + */ + // scalastyle:on line.size.limit + private[sql] def adjustPrecisionScale(precision: Int, scale: Int): DecimalType = { +// Assumptions: +// precision >= scale +// scale >= 0 +if (precision <= MAX_PRECISION) { + // Adjustment only needed when we exceed max precision + DecimalType(precision, scale) +} else { + // Precision/scale exceed maximum precision. Result must be adjusted to MAX_PRECISION. + val intDigits = precision - scale + // If original scale less than MINIMUM_ADJUSTED_SCALE, use original scale value; otherwise + // preserve at least MINIMUM_ADJUSTED_SCALE fractional digits + val minScaleValue = Math.min(scale, MINIMUM_ADJUSTED_SCALE) + val adjustedScale = Math.max(MAX_PRECISION - intDigits, minScaleValue) --- End diff -- Sounds like `Math.min`? --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user viirya commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r158207539 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala --- @@ -136,10 +137,54 @@ object DecimalType extends AbstractDataType { case DoubleType => DoubleDecimal } + private[sql] def forLiteral(literal: Literal): DecimalType = literal.value match { +case v: Short => fromBigDecimal(BigDecimal(v)) +case v: Int => fromBigDecimal(BigDecimal(v)) +case v: Long => fromBigDecimal(BigDecimal(v)) +case _ => forType(literal.dataType) + } + + private[sql] def fromBigDecimal(d: BigDecimal): DecimalType = { +DecimalType(Math.max(d.precision, d.scale), d.scale) + } + private[sql] def bounded(precision: Int, scale: Int): DecimalType = { DecimalType(min(precision, MAX_PRECISION), min(scale, MAX_SCALE)) } + // scalastyle:off line.size.limit + /** + * Decimal implementation is based on Hive's one, which is itself inspired to SQLServer's one. + * In particular, when a result precision is greater than {@link #MAX_PRECISION}, the + * corresponding scale is reduced to prevent the integral part of a result from being truncated. + * + * For further reference, please see + * https://blogs.msdn.microsoft.com/sqlprogrammability/2006/03/29/multiplication-and-division-with-numerics/. --- End diff -- Not sure if this blog link can be available for long time. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user viirya commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r158205829 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala --- @@ -136,10 +137,54 @@ object DecimalType extends AbstractDataType { case DoubleType => DoubleDecimal } + private[sql] def forLiteral(literal: Literal): DecimalType = literal.value match { +case v: Short => fromBigDecimal(BigDecimal(v)) +case v: Int => fromBigDecimal(BigDecimal(v)) +case v: Long => fromBigDecimal(BigDecimal(v)) +case _ => forType(literal.dataType) + } + + private[sql] def fromBigDecimal(d: BigDecimal): DecimalType = { +DecimalType(Math.max(d.precision, d.scale), d.scale) + } + private[sql] def bounded(precision: Int, scale: Int): DecimalType = { DecimalType(min(precision, MAX_PRECISION), min(scale, MAX_SCALE)) } + // scalastyle:off line.size.limit + /** + * Decimal implementation is based on Hive's one, which is itself inspired to SQLServer's one. + * In particular, when a result precision is greater than {@link #MAX_PRECISION}, the + * corresponding scale is reduced to prevent the integral part of a result from being truncated. + * + * For further reference, please see + * https://blogs.msdn.microsoft.com/sqlprogrammability/2006/03/29/multiplication-and-division-with-numerics/. + * + * @param precision + * @param scale + * @return + */ + // scalastyle:on line.size.limit + private[sql] def adjustPrecisionScale(precision: Int, scale: Int): DecimalType = { +// Assumptions: +// precision >= scale +// scale >= 0 +if (precision <= MAX_PRECISION) { + // Adjustment only needed when we exceed max precision + DecimalType(precision, scale) --- End diff -- Shouldn't we also prevent `scale` > `MAX_SCALE`? --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user viirya commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r158205620 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala --- @@ -136,10 +137,54 @@ object DecimalType extends AbstractDataType { case DoubleType => DoubleDecimal } + private[sql] def forLiteral(literal: Literal): DecimalType = literal.value match { --- End diff -- Is this different than `forType` if applied on `Literal.dataType`? --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user viirya commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r158206388 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala --- @@ -136,10 +137,54 @@ object DecimalType extends AbstractDataType { case DoubleType => DoubleDecimal } + private[sql] def forLiteral(literal: Literal): DecimalType = literal.value match { +case v: Short => fromBigDecimal(BigDecimal(v)) +case v: Int => fromBigDecimal(BigDecimal(v)) +case v: Long => fromBigDecimal(BigDecimal(v)) +case _ => forType(literal.dataType) + } + + private[sql] def fromBigDecimal(d: BigDecimal): DecimalType = { +DecimalType(Math.max(d.precision, d.scale), d.scale) + } + private[sql] def bounded(precision: Int, scale: Int): DecimalType = { DecimalType(min(precision, MAX_PRECISION), min(scale, MAX_SCALE)) } + // scalastyle:off line.size.limit + /** + * Decimal implementation is based on Hive's one, which is itself inspired to SQLServer's one. + * In particular, when a result precision is greater than {@link #MAX_PRECISION}, the + * corresponding scale is reduced to prevent the integral part of a result from being truncated. + * + * For further reference, please see + * https://blogs.msdn.microsoft.com/sqlprogrammability/2006/03/29/multiplication-and-division-with-numerics/. + * + * @param precision + * @param scale + * @return + */ + // scalastyle:on line.size.limit + private[sql] def adjustPrecisionScale(precision: Int, scale: Int): DecimalType = { +// Assumptions: +// precision >= scale +// scale >= 0 +if (precision <= MAX_PRECISION) { + // Adjustment only needed when we exceed max precision + DecimalType(precision, scale) +} else { + // Precision/scale exceed maximum precision. Result must be adjusted to MAX_PRECISION. + val intDigits = precision - scale + // If original scale less than MINIMUM_ADJUSTED_SCALE, use original scale value; otherwise + // preserve at least MINIMUM_ADJUSTED_SCALE fractional digits + val minScaleValue = Math.min(scale, MINIMUM_ADJUSTED_SCALE) --- End diff -- Sounds like `MAXIMUM_ADJUSTED_SCALE` instead of `MINIMUM_ADJUSTED_SCALE`. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user viirya commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r158205387 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala --- @@ -136,10 +137,54 @@ object DecimalType extends AbstractDataType { case DoubleType => DoubleDecimal } + private[sql] def forLiteral(literal: Literal): DecimalType = literal.value match { +case v: Short => fromBigDecimal(BigDecimal(v)) --- End diff -- Can't we just use `ShortDecimal`, `IntDecimal`...? --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user viirya commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r158195374 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala --- @@ -136,10 +137,54 @@ object DecimalType extends AbstractDataType { case DoubleType => DoubleDecimal } + private[sql] def forLiteral(literal: Literal): DecimalType = literal.value match { +case v: Short => fromBigDecimal(BigDecimal(v)) +case v: Int => fromBigDecimal(BigDecimal(v)) +case v: Long => fromBigDecimal(BigDecimal(v)) +case _ => forType(literal.dataType) + } + + private[sql] def fromBigDecimal(d: BigDecimal): DecimalType = { +DecimalType(Math.max(d.precision, d.scale), d.scale) + } + private[sql] def bounded(precision: Int, scale: Int): DecimalType = { DecimalType(min(precision, MAX_PRECISION), min(scale, MAX_SCALE)) } + // scalastyle:off line.size.limit + /** + * Decimal implementation is based on Hive's one, which is itself inspired to SQLServer's one. + * In particular, when a result precision is greater than {@link #MAX_PRECISION}, the + * corresponding scale is reduced to prevent the integral part of a result from being truncated. + * + * For further reference, please see + * https://blogs.msdn.microsoft.com/sqlprogrammability/2006/03/29/multiplication-and-division-with-numerics/. + * + * @param precision + * @param scale + * @return + */ + // scalastyle:on line.size.limit + private[sql] def adjustPrecisionScale(precision: Int, scale: Int): DecimalType = { +// Assumptions: +// precision >= scale +// scale >= 0 --- End diff -- Use `assert` to make sure assumptions? --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user mgaido91 commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r158157137 --- Diff: sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala --- @@ -1526,15 +1526,15 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { checkAnswer(sql("select 10.30 * 3.00"), Row(BigDecimal("30.9000", new MathContext(38 checkAnswer(sql("select 10.30 * 3.000"), - Row(null)) --- End diff -- The third case is never checked in the current codebase, ie. when we go out of the representable range of values. I haven't added a test for it, because I was waiting for feedbacks by the community about how to handle the 3rd case and I focused this PR only on points 1 and 2. But I can add a test case for it and eventually change it in a future PR to address the 3rd point in the e-mail. Thanks. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user dongjoon-hyun commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r158155039 --- Diff: sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala --- @@ -1526,15 +1526,15 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { checkAnswer(sql("select 10.30 * 3.00"), Row(BigDecimal("30.9000", new MathContext(38 checkAnswer(sql("select 10.30 * 3.000"), - Row(null)) --- End diff -- Two cases (2 and 3) were mentioned in the email. If this is the only `NULL`-return test case from previous behavior, can we have another test case? ``` Currently, Spark behaves like follows: 1. It follows some rules taken from intial Hive implementation; 2. it returns NULL; 3. it returns NULL. ``` --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user dongjoon-hyun commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r158154378 --- Diff: sql/core/src/test/resources/sql-tests/inputs/decimals.sql --- @@ -0,0 +1,16 @@ +-- tests for decimals handling in operations +-- Spark draws its inspiration byt Hive implementation --- End diff -- The hyperlinks in the PR cames from Microsoft, and the first purpose is SQL compliant. Can we remove this line? --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
Github user dongjoon-hyun commented on a diff in the pull request: https://github.com/apache/spark/pull/20023#discussion_r158154036 --- Diff: sql/core/src/test/resources/sql-tests/inputs/decimals.sql --- @@ -0,0 +1,16 @@ +-- tests for decimals handling in operations +-- Spark draws its inspiration byt Hive implementation +create table decimals_test(id int, a decimal(38,18), b decimal(38,18)) using parquet; + +insert into decimals_test values(1, 100.0, 999.0); +insert into decimals_test values(2, 12345.123, 12345.123); +insert into decimals_test values(3, 0.1234567891011, 1234.1); +insert into decimals_test values(4, 123456789123456789.0, 1.123456789123456789); --- End diff -- nit. How about making into one SQL statement? ```sql insert into decimals_test values (1, 100.0, 999.0), (2, 12345.123, 12345.123), (3, 0.1234567891011, 1234.1), (4, 123456789123456789.0, 1.123456789123456789) ``` --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #20023: [SPARK-22036][SQL] Decimal multiplication with hi...
GitHub user mgaido91 opened a pull request: https://github.com/apache/spark/pull/20023 [SPARK-22036][SQL] Decimal multiplication with high precision/scale often returns NULL ## What changes were proposed in this pull request? When there is an operation between Decimals and the result is a number which is not representable exactly with the result's precision and scale, Spark is returning `NULL`. This was done to reflect Hive's behavior, but it is against SQL ANSI 2011, which states that "If the result cannot be represented exactly in the result type, then whether it is rounded or truncated is implementation-defined". Moreover, Hive now changed its behavior in order to respect the standard, thanks to HIVE-15331. Therefore, the PR propose to: - update the rules to determine the result precision and scale according to the new Hive's ones introduces in HIVE-15331, which reflect SQLServer behavior; - round the result of the operations, when it is not representable exactly with the result's precision and scale, instead of returning `NULL` ## How was this patch tested? modified and added UTs. Comparisons with results of Hive and SQLServer. You can merge this pull request into a Git repository by running: $ git pull https://github.com/mgaido91/spark SPARK-22036 Alternatively you can review and apply these changes as the patch at: https://github.com/apache/spark/pull/20023.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #20023 commit 3037d4aa6afc4d7630d86d29b8dd7d7d724cc990 Author: Marco GaidoDate: 2017-12-17T21:45:06Z [SPARK-22036][SQL] Decimal multiplication with high precision/scale often returns NULL --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org