You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by we...@apache.org on 2022/06/13 06:02:50 UTC

[spark] branch master updated: [SPARK-39316][SQL] Merge PromotePrecision and CheckOverflow into decimal binary arithmetic

This is an automated email from the ASF dual-hosted git repository.

wenchen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
     new 301a1396380 [SPARK-39316][SQL] Merge PromotePrecision and CheckOverflow into decimal binary arithmetic
301a1396380 is described below

commit 301a13963808d1ad44be5cacf0a20f65b853d5a2
Author: ulysses-you <ul...@gmail.com>
AuthorDate: Mon Jun 13 14:02:12 2022 +0800

    [SPARK-39316][SQL] Merge PromotePrecision and CheckOverflow into decimal binary arithmetic
    
    ### What changes were proposed in this pull request?
    
    The main change:
    - Add a new method `resultDecimalType` in `BinaryArithmetic`
    - Add a new expression `DecimalAddNoOverflowCheck` for the internal decimal add, e.g. `Sum`/`Average`, the different with `Add` is:
      - `DecimalAddNoOverflowCheck` does not check overflow
      - `DecimalAddNoOverflowCheck` make `dataType` as its input parameter
    - Merge the decimal precision code of `DecimalPrecision` into each arithmetic data type, so every arithmetic should report the accurate decimal type. And we can remove the unused expression `PromotePrecision` and related code
    - Merge `CheckOverflow` iinto arithmetic eval and code-gen code path, so every arithmetic can handle the overflow case during runtime
    
    Merge `PromotePrecision` into `dataType`, for example, `Add`:
    ```scala
    override def resultDecimalType(p1: Int, s1: Int, p2: Int, s2: Int): DecimalType = {
      val resultScale = max(s1, s2)
      if (allowPrecisionLoss) {
        DecimalType.adjustPrecisionScale(max(p1 - s1, p2 - s2) + resultScale + 1,
          resultScale)
      } else {
        DecimalType.bounded(max(p1 - s1, p2 - s2) + resultScale + 1, resultScale)
      }
    }
    ```
    
    Merge `CheckOverflow`, for example, `Add` eval:
    ```scala
    dataType match {
      case decimalType: DecimalType =>
        val value = numeric.plus(input1, input2)
        checkOverflow(value.asInstanceOf[Decimal], decimalType)
      ...
    }
    ```
    
    Note that, `CheckOverflow` is still useful after this pr, e.g. `RowEncoder`. We can do further in a separate pr.
    
    ### Why are the changes needed?
    
    Fix the bug of `TypeCoercion`, for example:
    ```sql
    SELECT CAST(1 AS DECIMAL(28, 2))
    UNION ALL
    SELECT CAST(1 AS DECIMAL(18, 2)) / CAST(1 AS DECIMAL(18, 2));
    ```
    
    Relax the decimal precision at runtime, so we do not need redundant Cast
    
    ### Does this PR introduce _any_ user-facing change?
    
    yes, bug fix
    
    ### How was this patch tested?
    
    Pass exists test and add some bug fix test in `decimalArithmeticOperations.sql`
    
    Closes #36698 from ulysses-you/decimal.
    
    Lead-authored-by: ulysses-you <ul...@gmail.com>
    Co-authored-by: Wenchen Fan <cl...@gmail.com>
    Co-authored-by: Wenchen Fan <we...@databricks.com>
    Signed-off-by: Wenchen Fan <we...@databricks.com>
---
 .../sql/catalyst/analysis/DecimalPrecision.scala   | 130 +-----
 .../catalyst/analysis/StreamingJoinHelper.scala    |   2 -
 .../catalyst/expressions/aggregate/Average.scala   |  23 +-
 .../sql/catalyst/expressions/aggregate/Sum.scala   |  17 +-
 .../sql/catalyst/expressions/arithmetic.scala      | 278 +++++++++++--
 .../catalyst/expressions/decimalExpressions.scala  |  49 ++-
 .../scala/org/apache/spark/sql/types/Decimal.scala |   9 +-
 .../expressions/ArithmeticExpressionSuite.scala    |   9 +-
 .../catalyst/expressions/CanonicalizeSuite.scala   |   7 -
 .../expressions/DecimalExpressionSuite.scala       |   8 -
 .../SubexpressionEliminationSuite.scala            |  28 +-
 .../org/apache/spark/sql/types/DecimalSuite.scala  |   7 +
 .../inputs/decimalArithmeticOperations.sql         |  30 ++
 .../results/decimalArithmeticOperations.sql.out    |  79 +++-
 .../sql-tests/results/postgreSQL/numeric.sql.out   |   8 +-
 .../approved-plans-modified/q53.sf100/explain.txt  |   2 +-
 .../approved-plans-modified/q53/explain.txt        |   2 +-
 .../approved-plans-modified/q59.sf100/explain.txt  |   2 +-
 .../approved-plans-modified/q59/explain.txt        |   2 +-
 .../approved-plans-modified/q63.sf100/explain.txt  |   2 +-
 .../approved-plans-modified/q63/explain.txt        |   2 +-
 .../approved-plans-modified/q65.sf100/explain.txt  | 119 +++---
 .../q65.sf100/simplified.txt                       |  39 +-
 .../approved-plans-modified/q65/explain.txt        |  71 ++--
 .../approved-plans-modified/q65/simplified.txt     |  39 +-
 .../approved-plans-modified/q89.sf100/explain.txt  |   4 +-
 .../approved-plans-modified/q89/explain.txt        |   4 +-
 .../approved-plans-modified/q98.sf100/explain.txt  |   2 +-
 .../approved-plans-modified/q98/explain.txt        |   2 +-
 .../approved-plans-v1_4/q1.sf100/explain.txt       |   2 +-
 .../approved-plans-v1_4/q1/explain.txt             |   2 +-
 .../approved-plans-v1_4/q11.sf100/explain.txt      |  34 +-
 .../approved-plans-v1_4/q11.sf100/simplified.txt   |   8 +-
 .../approved-plans-v1_4/q11/explain.txt            |  34 +-
 .../approved-plans-v1_4/q11/simplified.txt         |   8 +-
 .../approved-plans-v1_4/q12.sf100/explain.txt      |   2 +-
 .../approved-plans-v1_4/q12/explain.txt            |   2 +-
 .../approved-plans-v1_4/q14a.sf100/explain.txt     |  32 +-
 .../approved-plans-v1_4/q14a.sf100/simplified.txt  |   8 +-
 .../approved-plans-v1_4/q14a/explain.txt           |  32 +-
 .../approved-plans-v1_4/q14a/simplified.txt        |   8 +-
 .../approved-plans-v1_4/q14b.sf100/explain.txt     |  24 +-
 .../approved-plans-v1_4/q14b.sf100/simplified.txt  |   6 +-
 .../approved-plans-v1_4/q14b/explain.txt           |  24 +-
 .../approved-plans-v1_4/q14b/simplified.txt        |   6 +-
 .../approved-plans-v1_4/q2.sf100/explain.txt       |   2 +-
 .../approved-plans-v1_4/q2/explain.txt             |   2 +-
 .../approved-plans-v1_4/q20.sf100/explain.txt      |   2 +-
 .../approved-plans-v1_4/q20/explain.txt            |   2 +-
 .../approved-plans-v1_4/q23a.sf100/explain.txt     | 164 ++++----
 .../approved-plans-v1_4/q23a.sf100/simplified.txt  |   6 +-
 .../approved-plans-v1_4/q23a/explain.txt           | 152 +++----
 .../approved-plans-v1_4/q23a/simplified.txt        |   6 +-
 .../approved-plans-v1_4/q23b.sf100/explain.txt     | 220 +++++-----
 .../approved-plans-v1_4/q23b.sf100/simplified.txt  |  14 +-
 .../approved-plans-v1_4/q23b/explain.txt           | 186 ++++-----
 .../approved-plans-v1_4/q23b/simplified.txt        |  12 +-
 .../approved-plans-v1_4/q24a.sf100/explain.txt     |   2 +-
 .../approved-plans-v1_4/q24a/explain.txt           |   2 +-
 .../approved-plans-v1_4/q24b.sf100/explain.txt     |   2 +-
 .../approved-plans-v1_4/q24b/explain.txt           |   2 +-
 .../approved-plans-v1_4/q30.sf100/explain.txt      |   2 +-
 .../approved-plans-v1_4/q30/explain.txt            |   2 +-
 .../approved-plans-v1_4/q31.sf100/explain.txt      |   4 +-
 .../approved-plans-v1_4/q31/explain.txt            |   6 +-
 .../approved-plans-v1_4/q32.sf100/explain.txt      |   2 +-
 .../approved-plans-v1_4/q32/explain.txt            |   2 +-
 .../approved-plans-v1_4/q36.sf100/explain.txt      |   2 +-
 .../approved-plans-v1_4/q36/explain.txt            |   2 +-
 .../approved-plans-v1_4/q4.sf100/explain.txt       |  52 +--
 .../approved-plans-v1_4/q4.sf100/simplified.txt    |  12 +-
 .../approved-plans-v1_4/q4/explain.txt             |  52 +--
 .../approved-plans-v1_4/q4/simplified.txt          |  12 +-
 .../approved-plans-v1_4/q40.sf100/explain.txt      |   8 +-
 .../approved-plans-v1_4/q40.sf100/simplified.txt   |   2 +-
 .../approved-plans-v1_4/q40/explain.txt            |   8 +-
 .../approved-plans-v1_4/q40/simplified.txt         |   2 +-
 .../approved-plans-v1_4/q44.sf100/explain.txt      |   2 +-
 .../approved-plans-v1_4/q44/explain.txt            |   2 +-
 .../approved-plans-v1_4/q47.sf100/explain.txt      |   4 +-
 .../approved-plans-v1_4/q47/explain.txt            |   4 +-
 .../approved-plans-v1_4/q49.sf100/explain.txt      |   6 +-
 .../approved-plans-v1_4/q49/explain.txt            |   6 +-
 .../approved-plans-v1_4/q5.sf100/explain.txt       |   6 +-
 .../approved-plans-v1_4/q5/explain.txt             |   6 +-
 .../approved-plans-v1_4/q53.sf100/explain.txt      |   2 +-
 .../approved-plans-v1_4/q53/explain.txt            |   2 +-
 .../approved-plans-v1_4/q54.sf100/explain.txt      |   2 +-
 .../approved-plans-v1_4/q54/explain.txt            |   2 +-
 .../approved-plans-v1_4/q57.sf100/explain.txt      |   4 +-
 .../approved-plans-v1_4/q57/explain.txt            |   4 +-
 .../approved-plans-v1_4/q58.sf100/explain.txt      |   6 +-
 .../approved-plans-v1_4/q58/explain.txt            |   6 +-
 .../approved-plans-v1_4/q59.sf100/explain.txt      |   2 +-
 .../approved-plans-v1_4/q59/explain.txt            |   2 +-
 .../approved-plans-v1_4/q61.sf100/explain.txt      |   2 +-
 .../approved-plans-v1_4/q61/explain.txt            |   2 +-
 .../approved-plans-v1_4/q63.sf100/explain.txt      |   2 +-
 .../approved-plans-v1_4/q63/explain.txt            |   2 +-
 .../approved-plans-v1_4/q65.sf100/explain.txt      | 131 +++---
 .../approved-plans-v1_4/q65.sf100/simplified.txt   |  39 +-
 .../approved-plans-v1_4/q65/explain.txt            |  71 ++--
 .../approved-plans-v1_4/q65/simplified.txt         |  39 +-
 .../approved-plans-v1_4/q66.sf100/explain.txt      |  24 +-
 .../approved-plans-v1_4/q66.sf100/simplified.txt   |   6 +-
 .../approved-plans-v1_4/q66/explain.txt            |  24 +-
 .../approved-plans-v1_4/q66/simplified.txt         |   6 +-
 .../approved-plans-v1_4/q67.sf100/explain.txt      |   8 +-
 .../approved-plans-v1_4/q67.sf100/simplified.txt   |   2 +-
 .../approved-plans-v1_4/q67/explain.txt            |   8 +-
 .../approved-plans-v1_4/q67/simplified.txt         |   2 +-
 .../approved-plans-v1_4/q77.sf100/explain.txt      |   6 +-
 .../approved-plans-v1_4/q77/explain.txt            |   6 +-
 .../approved-plans-v1_4/q80.sf100/explain.txt      |  24 +-
 .../approved-plans-v1_4/q80.sf100/simplified.txt   |   6 +-
 .../approved-plans-v1_4/q80/explain.txt            |  24 +-
 .../approved-plans-v1_4/q80/simplified.txt         |   6 +-
 .../approved-plans-v1_4/q81.sf100/explain.txt      |   2 +-
 .../approved-plans-v1_4/q81/explain.txt            |   2 +-
 .../approved-plans-v1_4/q83.sf100/explain.txt      |   2 +-
 .../approved-plans-v1_4/q83/explain.txt            |   2 +-
 .../approved-plans-v1_4/q89.sf100/explain.txt      |   4 +-
 .../approved-plans-v1_4/q89/explain.txt            |   4 +-
 .../approved-plans-v1_4/q90.sf100/explain.txt      |   2 +-
 .../approved-plans-v1_4/q90/explain.txt            |   2 +-
 .../approved-plans-v1_4/q92.sf100/explain.txt      |   2 +-
 .../approved-plans-v1_4/q92/explain.txt            |   2 +-
 .../approved-plans-v1_4/q93.sf100/explain.txt      |   2 +-
 .../approved-plans-v1_4/q93/explain.txt            |   2 +-
 .../approved-plans-v1_4/q98.sf100/explain.txt      |   2 +-
 .../approved-plans-v1_4/q98/explain.txt            |   2 +-
 .../approved-plans-v2_7/q11.sf100/explain.txt      |  34 +-
 .../approved-plans-v2_7/q11.sf100/simplified.txt   |   8 +-
 .../approved-plans-v2_7/q11/explain.txt            |  34 +-
 .../approved-plans-v2_7/q11/simplified.txt         |   8 +-
 .../approved-plans-v2_7/q12.sf100/explain.txt      |   2 +-
 .../approved-plans-v2_7/q12/explain.txt            |   2 +-
 .../approved-plans-v2_7/q14.sf100/explain.txt      |  24 +-
 .../approved-plans-v2_7/q14.sf100/simplified.txt   |   6 +-
 .../approved-plans-v2_7/q14/explain.txt            |  24 +-
 .../approved-plans-v2_7/q14/simplified.txt         |   6 +-
 .../approved-plans-v2_7/q14a.sf100/explain.txt     |  32 +-
 .../approved-plans-v2_7/q14a.sf100/simplified.txt  |   8 +-
 .../approved-plans-v2_7/q14a/explain.txt           |  32 +-
 .../approved-plans-v2_7/q14a/simplified.txt        |   8 +-
 .../approved-plans-v2_7/q20.sf100/explain.txt      |   2 +-
 .../approved-plans-v2_7/q20/explain.txt            |   2 +-
 .../approved-plans-v2_7/q24.sf100/explain.txt      |   2 +-
 .../approved-plans-v2_7/q24/explain.txt            |   2 +-
 .../approved-plans-v2_7/q36a.sf100/explain.txt     |   6 +-
 .../approved-plans-v2_7/q36a/explain.txt           |   6 +-
 .../approved-plans-v2_7/q47.sf100/explain.txt      |   4 +-
 .../approved-plans-v2_7/q47/explain.txt            |   4 +-
 .../approved-plans-v2_7/q49.sf100/explain.txt      |   6 +-
 .../approved-plans-v2_7/q49/explain.txt            |   6 +-
 .../approved-plans-v2_7/q57.sf100/explain.txt      |   4 +-
 .../approved-plans-v2_7/q57/explain.txt            |   4 +-
 .../approved-plans-v2_7/q5a.sf100/explain.txt      |   6 +-
 .../approved-plans-v2_7/q5a/explain.txt            |   6 +-
 .../approved-plans-v2_7/q6.sf100/explain.txt       | 194 ++++-----
 .../approved-plans-v2_7/q6.sf100/simplified.txt    |  52 +--
 .../approved-plans-v2_7/q6/explain.txt             |  30 +-
 .../approved-plans-v2_7/q6/simplified.txt          |  18 +-
 .../approved-plans-v2_7/q64.sf100/explain.txt      |  10 +-
 .../approved-plans-v2_7/q64.sf100/simplified.txt   |   2 +-
 .../approved-plans-v2_7/q64/explain.txt            |  18 +-
 .../approved-plans-v2_7/q64/simplified.txt         |   4 +-
 .../approved-plans-v2_7/q67a.sf100/explain.txt     |  56 +--
 .../approved-plans-v2_7/q67a.sf100/simplified.txt  |  18 +-
 .../approved-plans-v2_7/q67a/explain.txt           |  56 +--
 .../approved-plans-v2_7/q67a/simplified.txt        |  18 +-
 .../approved-plans-v2_7/q74.sf100/explain.txt      |   2 +-
 .../approved-plans-v2_7/q74/explain.txt            |   2 +-
 .../approved-plans-v2_7/q75.sf100/explain.txt      | 442 +++++++++++----------
 .../approved-plans-v2_7/q75.sf100/simplified.txt   | 436 ++++++++++----------
 .../approved-plans-v2_7/q75/explain.txt            | 442 +++++++++++----------
 .../approved-plans-v2_7/q75/simplified.txt         | 436 ++++++++++----------
 .../approved-plans-v2_7/q77a.sf100/explain.txt     |   6 +-
 .../approved-plans-v2_7/q77a/explain.txt           |   6 +-
 .../approved-plans-v2_7/q78.sf100/explain.txt      |   2 +-
 .../approved-plans-v2_7/q78/explain.txt            |   2 +-
 .../approved-plans-v2_7/q80a.sf100/explain.txt     |  66 +--
 .../approved-plans-v2_7/q80a.sf100/simplified.txt  |  18 +-
 .../approved-plans-v2_7/q80a/explain.txt           |  24 +-
 .../approved-plans-v2_7/q80a/simplified.txt        |   6 +-
 .../approved-plans-v2_7/q98.sf100/explain.txt      |   2 +-
 .../approved-plans-v2_7/q98/explain.txt            |   2 +-
 .../resources/tpch-plan-stability/q1/explain.txt   |   8 +-
 .../tpch-plan-stability/q1/simplified.txt          |   2 +-
 .../resources/tpch-plan-stability/q10/explain.txt  |   8 +-
 .../tpch-plan-stability/q10/simplified.txt         |   2 +-
 .../resources/tpch-plan-stability/q11/explain.txt  |  16 +-
 .../tpch-plan-stability/q11/simplified.txt         |   4 +-
 .../resources/tpch-plan-stability/q14/explain.txt  |   8 +-
 .../tpch-plan-stability/q14/simplified.txt         |   2 +-
 .../resources/tpch-plan-stability/q15/explain.txt  |  16 +-
 .../tpch-plan-stability/q15/simplified.txt         |   4 +-
 .../resources/tpch-plan-stability/q17/explain.txt  |   4 +-
 .../resources/tpch-plan-stability/q19/explain.txt  |   8 +-
 .../tpch-plan-stability/q19/simplified.txt         |   2 +-
 .../resources/tpch-plan-stability/q20/explain.txt  |   2 +-
 .../resources/tpch-plan-stability/q3/explain.txt   |   8 +-
 .../tpch-plan-stability/q3/simplified.txt          |   2 +-
 .../resources/tpch-plan-stability/q5/explain.txt   |   8 +-
 .../tpch-plan-stability/q5/simplified.txt          |   2 +-
 .../resources/tpch-plan-stability/q6/explain.txt   |   8 +-
 .../tpch-plan-stability/q6/simplified.txt          |   2 +-
 .../resources/tpch-plan-stability/q7/explain.txt   |   2 +-
 .../resources/tpch-plan-stability/q8/explain.txt   |   4 +-
 .../resources/tpch-plan-stability/q9/explain.txt   |   2 +-
 .../apache/spark/sql/execution/PlannerSuite.scala  |   4 +-
 211 files changed, 2867 insertions(+), 2613 deletions(-)

diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecision.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecision.scala
index b112641144c..46fbf071f43 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecision.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecision.scala
@@ -34,17 +34,8 @@ import org.apache.spark.sql.types._
  *
  *   Operation    Result Precision                        Result Scale
  *   ------------------------------------------------------------------------
- *   e1 + e2      max(s1, s2) + max(p1-s1, p2-s2) + 1     max(s1, s2)
- *   e1 - e2      max(s1, s2) + max(p1-s1, p2-s2) + 1     max(s1, s2)
- *   e1 * e2      p1 + p2 + 1                             s1 + s2
- *   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)
  *
- * When `spark.sql.decimalOperations.allowPrecisionLoss` 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 truncation of the integer part of the decimals.
- *
  * To implement the rules for fixed-precision types, we introduce casts to turn them to unlimited
  * precision, do the math on unlimited-precision numbers, then introduce casts back to the
  * required fixed precision. This allows us to do all rounding and overflow handling in the
@@ -60,7 +51,7 @@ import org.apache.spark.sql.types._
  */
 // scalastyle:on
 object DecimalPrecision extends TypeCoercionRule {
-  import scala.math.{max, min}
+  import scala.math.max
 
   private def isFloat(t: DataType): Boolean = t == FloatType || t == DoubleType
 
@@ -75,132 +66,17 @@ object DecimalPrecision extends TypeCoercionRule {
     DecimalType.bounded(range + scale, scale)
   }
 
-  private def promotePrecision(e: Expression, dataType: DataType): Expression = {
-    PromotePrecision(Cast(e, dataType))
-  }
-
   override def transform: PartialFunction[Expression, Expression] = {
     decimalAndDecimal()
       .orElse(integralAndDecimalLiteral)
       .orElse(nondecimalAndDecimal(conf.literalPickMinimumPrecision))
   }
 
-  private[catalyst] def decimalAndDecimal(): PartialFunction[Expression, Expression] = {
-    decimalAndDecimal(conf.decimalOperationsAllowPrecisionLoss, !conf.ansiEnabled)
-  }
-
-  /** Decimal precision promotion for +, -, *, /, %, pmod, and binary comparison. */
-  private[catalyst] def decimalAndDecimal(allowPrecisionLoss: Boolean, nullOnOverflow: Boolean)
-    : PartialFunction[Expression, Expression] = {
+  /** Decimal precision promotion for  binary comparison. */
+  private def decimalAndDecimal(): PartialFunction[Expression, Expression] = {
     // Skip nodes whose children have not been resolved yet
     case e if !e.childrenResolved => e
 
-    // Skip nodes who is already promoted
-    case e: BinaryArithmetic if e.left.isInstanceOf[PromotePrecision] => e
-
-    case a @ Add(e1 @ DecimalType.Expression(p1, s1), e2 @ DecimalType.Expression(p2, s2), _) =>
-      val resultScale = max(s1, s2)
-      val resultType = if (allowPrecisionLoss) {
-        DecimalType.adjustPrecisionScale(max(p1 - s1, p2 - s2) + resultScale + 1,
-          resultScale)
-      } else {
-        DecimalType.bounded(max(p1 - s1, p2 - s2) + resultScale + 1, resultScale)
-      }
-      CheckOverflow(
-        a.copy(left = promotePrecision(e1, resultType), right = promotePrecision(e2, resultType)),
-        resultType, nullOnOverflow)
-
-    case s @ Subtract(e1 @ DecimalType.Expression(p1, s1),
-        e2 @ DecimalType.Expression(p2, s2), _) =>
-      val resultScale = max(s1, s2)
-      val resultType = if (allowPrecisionLoss) {
-        DecimalType.adjustPrecisionScale(max(p1 - s1, p2 - s2) + resultScale + 1,
-          resultScale)
-      } else {
-        DecimalType.bounded(max(p1 - s1, p2 - s2) + resultScale + 1, resultScale)
-      }
-      CheckOverflow(
-        s.copy(left = promotePrecision(e1, resultType), right = promotePrecision(e2, resultType)),
-        resultType, nullOnOverflow)
-
-    case m @ Multiply(
-        e1 @ DecimalType.Expression(p1, s1), e2 @ DecimalType.Expression(p2, s2), _) =>
-      val resultType = if (allowPrecisionLoss) {
-        DecimalType.adjustPrecisionScale(p1 + p2 + 1, s1 + s2)
-      } else {
-        DecimalType.bounded(p1 + p2 + 1, s1 + s2)
-      }
-      val widerType = widerDecimalType(p1, s1, p2, s2)
-      CheckOverflow(
-        m.copy(left = promotePrecision(e1, widerType), right = promotePrecision(e2, widerType)),
-        resultType, nullOnOverflow)
-
-    case d @ Divide(e1 @ DecimalType.Expression(p1, s1), e2 @ DecimalType.Expression(p2, s2), _) =>
-      val resultType = if (allowPrecisionLoss) {
-        // Precision: p1 - s1 + s2 + max(6, s1 + p2 + 1)
-        // Scale: max(6, s1 + p2 + 1)
-        val intDig = p1 - s1 + s2
-        val scale = max(DecimalType.MINIMUM_ADJUSTED_SCALE, s1 + p2 + 1)
-        val prec = intDig + scale
-        DecimalType.adjustPrecisionScale(prec, scale)
-      } else {
-        var intDig = min(DecimalType.MAX_SCALE, p1 - s1 + s2)
-        var decDig = min(DecimalType.MAX_SCALE, max(6, s1 + p2 + 1))
-        val diff = (intDig + decDig) - DecimalType.MAX_SCALE
-        if (diff > 0) {
-          decDig -= diff / 2 + 1
-          intDig = DecimalType.MAX_SCALE - decDig
-        }
-        DecimalType.bounded(intDig + decDig, decDig)
-      }
-      val widerType = widerDecimalType(p1, s1, p2, s2)
-      CheckOverflow(
-        d.copy(left = promotePrecision(e1, widerType), right = promotePrecision(e2, widerType)),
-        resultType, nullOnOverflow)
-
-    case r @ Remainder(
-        e1 @ DecimalType.Expression(p1, s1), e2 @ DecimalType.Expression(p2, s2), _) =>
-      val resultType = if (allowPrecisionLoss) {
-        DecimalType.adjustPrecisionScale(min(p1 - s1, p2 - s2) + max(s1, s2), max(s1, s2))
-      } else {
-        DecimalType.bounded(min(p1 - s1, p2 - s2) + max(s1, s2), max(s1, s2))
-      }
-      // resultType may have lower precision, so we cast them into wider type first.
-      val widerType = widerDecimalType(p1, s1, p2, s2)
-      CheckOverflow(
-        r.copy(left = promotePrecision(e1, widerType), right = promotePrecision(e2, widerType)),
-        resultType, nullOnOverflow)
-
-    case p @ Pmod(e1 @ DecimalType.Expression(p1, s1), e2 @ DecimalType.Expression(p2, s2), _) =>
-      val resultType = if (allowPrecisionLoss) {
-        DecimalType.adjustPrecisionScale(min(p1 - s1, p2 - s2) + max(s1, s2), max(s1, s2))
-      } else {
-        DecimalType.bounded(min(p1 - s1, p2 - s2) + max(s1, s2), max(s1, s2))
-      }
-      // resultType may have lower precision, so we cast them into wider type first.
-      val widerType = widerDecimalType(p1, s1, p2, s2)
-      CheckOverflow(
-        p.copy(left = promotePrecision(e1, widerType), right = promotePrecision(e2, widerType)),
-        resultType, nullOnOverflow)
-
-    case expr @ IntegralDivide(
-        e1 @ DecimalType.Expression(p1, s1), e2 @ DecimalType.Expression(p2, s2), _) =>
-      val widerType = widerDecimalType(p1, s1, p2, s2)
-      val promotedExpr = expr.copy(
-        left = promotePrecision(e1, widerType),
-        right = promotePrecision(e2, widerType))
-      if (expr.dataType.isInstanceOf[DecimalType]) {
-        // This follows division rule
-        val intDig = p1 - s1 + s2
-        // No precision loss can happen as the result scale is 0.
-        // Overflow can happen only in the promote precision of the operands, but if none of them
-        // overflows in that phase, no overflow can happen, but CheckOverflow is needed in order
-        // to return a decimal with the proper scale and precision
-        CheckOverflow(promotedExpr, DecimalType.bounded(intDig, 0), nullOnOverflow)
-      } else {
-        promotedExpr
-      }
-
     case b @ BinaryComparison(e1 @ DecimalType.Expression(p1, s1),
     e2 @ DecimalType.Expression(p2, s2)) if p1 != p2 || s1 != s2 =>
       val resultType = widerDecimalType(p1, s1, p2, s2)
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/StreamingJoinHelper.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/StreamingJoinHelper.scala
index 737d30a41d3..380172c1a13 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/StreamingJoinHelper.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/StreamingJoinHelper.scala
@@ -237,8 +237,6 @@ object StreamingJoinHelper extends PredicateHelper with Logging {
           collect(child, !negate)
         case CheckOverflow(child, _, _) =>
           collect(child, negate)
-        case PromotePrecision(child) =>
-          collect(child, negate)
         case Cast(child, dataType, _, _) =>
           dataType match {
             case _: NumericType | _: TimestampType => collect(child, negate)
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Average.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Average.scala
index 343e27d863b..864ec7055f3 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Average.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Average.scala
@@ -17,7 +17,7 @@
 
 package org.apache.spark.sql.catalyst.expressions.aggregate
 
-import org.apache.spark.sql.catalyst.analysis.{DecimalPrecision, FunctionRegistry, TypeCheckResult}
+import org.apache.spark.sql.catalyst.analysis.{FunctionRegistry, TypeCheckResult}
 import org.apache.spark.sql.catalyst.dsl.expressions._
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.trees.TreePattern.{AVERAGE, TreePattern}
@@ -67,6 +67,11 @@ abstract class AverageBase
   lazy val sum = AttributeReference("sum", sumDataType)()
   lazy val count = AttributeReference("count", LongType)()
 
+  protected def add(left: Expression, right: Expression): Expression = left.dataType match {
+    case _: DecimalType => DecimalAddNoOverflowCheck(left, right, left.dataType)
+    case _ => Add(left, right, useAnsiAdd)
+  }
+
   override lazy val aggBufferAttributes = sum :: count :: Nil
 
   override lazy val initialValues = Seq(
@@ -75,7 +80,7 @@ abstract class AverageBase
   )
 
   protected def getMergeExpressions = Seq(
-    /* sum = */ Add(sum.left, sum.right, useAnsiAdd),
+    /* sum = */ add(sum.left, sum.right),
     /* count = */ count.left + count.right
   )
 
@@ -83,10 +88,9 @@ abstract class AverageBase
   // We can't directly use `/` as it throws an exception under ansi mode.
   protected def getEvaluateExpression(queryContext: String) = child.dataType match {
     case _: DecimalType =>
-      DecimalPrecision.decimalAndDecimal()(
-        Divide(
-          CheckOverflowInSum(sum, sumDataType.asInstanceOf[DecimalType], !useAnsiAdd, queryContext),
-          count.cast(DecimalType.LongDecimal), failOnError = false)).cast(resultType)
+      Divide(
+        CheckOverflowInSum(sum, sumDataType.asInstanceOf[DecimalType], !useAnsiAdd, queryContext),
+        count.cast(DecimalType.LongDecimal), failOnError = false).cast(resultType)
     case _: YearMonthIntervalType =>
       If(EqualTo(count, Literal(0L)),
         Literal(null, YearMonthIntervalType()), DivideYMInterval(sum, count))
@@ -99,10 +103,9 @@ abstract class AverageBase
 
   protected def getUpdateExpressions: Seq[Expression] = Seq(
     /* sum = */
-    Add(
+    add(
       sum,
-      coalesce(child.cast(sumDataType), Literal.default(sumDataType)),
-      failOnError = useAnsiAdd),
+      coalesce(child.cast(sumDataType), Literal.default(sumDataType))),
     /* count = */ If(child.isNull, count, count + 1L)
   )
 
@@ -190,7 +193,7 @@ case class TryAverage(child: Expression) extends AverageBase {
           Literal.create(null, resultType),
           // If both the buffer and the input do not overflow, just add them, as they can't be
           // null.
-          TryEval(Add(KnownNotNull(sum.left), KnownNotNull(sum.right), useAnsiAdd))),
+          TryEval(add(KnownNotNull(sum.left), KnownNotNull(sum.right)))),
           expressions(1))
     } else {
       expressions
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Sum.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Sum.scala
index fa43565d807..1c4297d735b 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Sum.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Sum.scala
@@ -63,6 +63,11 @@ abstract class SumBase(child: Expression) extends DeclarativeAggregate
 
   private lazy val zero = Literal.default(resultType)
 
+  private def add(left: Expression, right: Expression): Expression = left.dataType match {
+    case _: DecimalType => DecimalAddNoOverflowCheck(left, right, left.dataType)
+    case _ => Add(left, right, useAnsiAdd)
+  }
+
   override lazy val aggBufferAttributes = if (shouldTrackIsEmpty) {
     sum :: isEmpty :: Nil
   } else {
@@ -82,9 +87,9 @@ abstract class SumBase(child: Expression) extends DeclarativeAggregate
     // null if overflow happens under non-ansi mode.
     val sumExpr = if (child.nullable) {
       If(child.isNull, sum,
-        Add(sum, KnownNotNull(child).cast(resultType), failOnError = useAnsiAdd))
+        add(sum, KnownNotNull(child).cast(resultType)))
     } else {
-      Add(sum, child.cast(resultType), failOnError = useAnsiAdd)
+      add(sum, child.cast(resultType))
     }
     // The buffer becomes non-empty after seeing the first not-null input.
     val isEmptyExpr = if (child.nullable) {
@@ -99,10 +104,10 @@ abstract class SumBase(child: Expression) extends DeclarativeAggregate
     // in case the input is nullable. The `sum` can only be null if there is no value, as
     // non-decimal type can produce overflowed value under non-ansi mode.
     if (child.nullable) {
-      Seq(coalesce(Add(coalesce(sum, zero), child.cast(resultType), failOnError = useAnsiAdd),
+      Seq(coalesce(add(coalesce(sum, zero), child.cast(resultType)),
         sum))
     } else {
-      Seq(Add(coalesce(sum, zero), child.cast(resultType), failOnError = useAnsiAdd))
+      Seq(add(coalesce(sum, zero), child.cast(resultType)))
     }
   }
 
@@ -128,11 +133,11 @@ abstract class SumBase(child: Expression) extends DeclarativeAggregate
         // If both the buffer and the input do not overflow, just add them, as they can't be
         // null. See the comments inside `updateExpressions`: `sum` can only be null if
         // overflow happens.
-        Add(KnownNotNull(sum.left), KnownNotNull(sum.right), useAnsiAdd)),
+        add(KnownNotNull(sum.left), KnownNotNull(sum.right))),
       isEmpty.left && isEmpty.right)
   } else {
     Seq(coalesce(
-      Add(coalesce(sum.left, zero), sum.right, failOnError = useAnsiAdd),
+      add(coalesce(sum.left, zero), sum.right),
       sum.left))
   }
 
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala
index 153187f9e30..97411c05d5f 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala
@@ -17,6 +17,8 @@
 
 package org.apache.spark.sql.catalyst.expressions
 
+import scala.math.{max, min}
+
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.analysis.{FunctionRegistry, TypeCheckResult, TypeCoercion}
 import org.apache.spark.sql.catalyst.expressions.codegen._
@@ -208,12 +210,46 @@ case class Abs(child: Expression, failOnError: Boolean = SQLConf.get.ansiEnabled
   override protected def withNewChildInternal(newChild: Expression): Abs = copy(child = newChild)
 }
 
-abstract class BinaryArithmetic extends BinaryOperator with NullIntolerant
-    with SupportQueryContext {
+abstract class BinaryArithmetic extends BinaryOperator
+  with NullIntolerant with SupportQueryContext {
 
   protected val failOnError: Boolean
 
-  override def dataType: DataType = left.dataType
+  override def checkInputDataTypes(): TypeCheckResult = (left.dataType, right.dataType) match {
+    case (l: DecimalType, r: DecimalType) if inputType.acceptsType(l) && inputType.acceptsType(r) =>
+      // We allow decimal type inputs with different precision and scale, and use special formulas
+      // to calculate the result precision and scale.
+      TypeCheckResult.TypeCheckSuccess
+    case _ => super.checkInputDataTypes()
+  }
+
+  override def dataType: DataType = (left.dataType, right.dataType) match {
+    case (DecimalType.Fixed(p1, s1), DecimalType.Fixed(p2, s2)) =>
+      resultDecimalType(p1, s1, p2, s2)
+    case _ => left.dataType
+  }
+
+  // When `spark.sql.decimalOperations.allowPrecisionLoss` 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 truncation of the integer part of the decimals.
+  protected def allowPrecisionLoss: Boolean = SQLConf.get.decimalOperationsAllowPrecisionLoss
+
+  protected def resultDecimalType(p1: Int, s1: Int, p2: Int, s2: Int): DecimalType = {
+    throw new IllegalStateException(
+      s"${getClass.getSimpleName} must override `resultDecimalType`.")
+  }
+
+  override def nullable: Boolean = super.nullable || {
+    if (left.dataType.isInstanceOf[DecimalType]) {
+      // For decimal arithmetic, we may return null even if both inputs are not null, if overflow
+      // happens and this `failOnError` flag is false.
+      !failOnError
+    } else {
+      // For non-decimal arithmetic, the calculation always return non-null result when inputs are
+      // not null. If overflow happens, we return either the overflowed value or fail.
+      false
+    }
+  }
 
   final override val nodePatterns: Seq[TreePattern] = Seq(BINARY_ARITHMETIC)
 
@@ -227,6 +263,10 @@ abstract class BinaryArithmetic extends BinaryOperator with NullIntolerant
     }
   }
 
+  protected def checkDecimalOverflow(value: Decimal, precision: Int, scale: Int): Decimal = {
+    value.toPrecision(precision, scale, Decimal.ROUND_HALF_UP, !failOnError, queryContext)
+  }
+
   /** Name of the function for this expression on a [[Decimal]] type. */
   def decimalMethod: String =
     throw QueryExecutionErrors.notOverrideExpectedMethodsError("BinaryArithmetics",
@@ -243,9 +283,24 @@ abstract class BinaryArithmetic extends BinaryOperator with NullIntolerant
   def exactMathMethod: Option[String] = None
 
   override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = dataType match {
-    case _: DecimalType =>
-      // Overflow is handled in the CheckOverflow operator
-      defineCodeGen(ctx, ev, (eval1, eval2) => s"$eval1.$decimalMethod($eval2)")
+    case DecimalType.Fixed(precision, scale) =>
+      val errorContextCode = if (failOnError) {
+        ctx.addReferenceObj("errCtx", queryContext)
+      } else {
+        "\"\""
+      }
+      val updateIsNull = if (failOnError) {
+        ""
+      } else {
+        s"${ev.isNull} = ${ev.value} == null;"
+      }
+      nullSafeCodeGen(ctx, ev, (eval1, eval2) => {
+        s"""
+           |${ev.value} = $eval1.$decimalMethod($eval2).toPrecision(
+           |  $precision, $scale, Decimal.ROUND_HALF_UP(), ${!failOnError}, $errorContextCode);
+           |$updateIsNull
+       """.stripMargin
+      })
     case CalendarIntervalType =>
       val iu = IntervalUtils.getClass.getCanonicalName.stripSuffix("$")
       defineCodeGen(ctx, ev, (eval1, eval2) => s"$iu.$calendarIntervalMethod($eval1, $eval2)")
@@ -323,11 +378,30 @@ case class Add(
 
   override def decimalMethod: String = "$plus"
 
+  // scalastyle:off
+  // The formula follows Hive which is based on the SQL standard and MS SQL:
+  // https://cwiki.apache.org/confluence/download/attachments/27362075/Hive_Decimal_Precision_Scale_Support.pdf
+  // https://msdn.microsoft.com/en-us/library/ms190476.aspx
+  // Result Precision: max(s1, s2) + max(p1-s1, p2-s2) + 1
+  // Result Scale:     max(s1, s2)
+  // scalastyle:on
+  override def resultDecimalType(p1: Int, s1: Int, p2: Int, s2: Int): DecimalType = {
+    val resultScale = max(s1, s2)
+    val resultPrecision = max(p1 - s1, p2 - s2) + resultScale + 1
+    if (allowPrecisionLoss) {
+      DecimalType.adjustPrecisionScale(resultPrecision, resultScale)
+    } else {
+      DecimalType.bounded(resultPrecision, resultScale)
+    }
+  }
+
   override def calendarIntervalMethod: String = if (failOnError) "addExact" else "add"
 
   private lazy val numeric = TypeUtils.getNumeric(dataType, failOnError)
 
   protected override def nullSafeEval(input1: Any, input2: Any): Any = dataType match {
+    case DecimalType.Fixed(precision, scale) =>
+      checkDecimalOverflow(numeric.plus(input1, input2).asInstanceOf[Decimal], precision, scale)
     case CalendarIntervalType if failOnError =>
       IntervalUtils.addExact(
         input1.asInstanceOf[CalendarInterval], input2.asInstanceOf[CalendarInterval])
@@ -373,11 +447,30 @@ case class Subtract(
 
   override def decimalMethod: String = "$minus"
 
+  // scalastyle:off
+  // The formula follows Hive which is based on the SQL standard and MS SQL:
+  // https://cwiki.apache.org/confluence/download/attachments/27362075/Hive_Decimal_Precision_Scale_Support.pdf
+  // https://msdn.microsoft.com/en-us/library/ms190476.aspx
+  // Result Precision: max(s1, s2) + max(p1-s1, p2-s2) + 1
+  // Result Scale:     max(s1, s2)
+  // scalastyle:on
+  override def resultDecimalType(p1: Int, s1: Int, p2: Int, s2: Int): DecimalType = {
+    val resultScale = max(s1, s2)
+    val resultPrecision = max(p1 - s1, p2 - s2) + resultScale + 1
+    if (allowPrecisionLoss) {
+      DecimalType.adjustPrecisionScale(resultPrecision, resultScale)
+    } else {
+      DecimalType.bounded(resultPrecision, resultScale)
+    }
+  }
+
   override def calendarIntervalMethod: String = if (failOnError) "subtractExact" else "subtract"
 
   private lazy val numeric = TypeUtils.getNumeric(dataType, failOnError)
 
   protected override def nullSafeEval(input1: Any, input2: Any): Any = dataType match {
+    case DecimalType.Fixed(precision, scale) =>
+      checkDecimalOverflow(numeric.minus(input1, input2).asInstanceOf[Decimal], precision, scale)
     case CalendarIntervalType if failOnError =>
       IntervalUtils.subtractExact(
         input1.asInstanceOf[CalendarInterval], input2.asInstanceOf[CalendarInterval])
@@ -422,9 +515,28 @@ case class Multiply(
   override def symbol: String = "*"
   override def decimalMethod: String = "$times"
 
+  // scalastyle:off
+  // The formula follows Hive which is based on the SQL standard and MS SQL:
+  // https://cwiki.apache.org/confluence/download/attachments/27362075/Hive_Decimal_Precision_Scale_Support.pdf
+  // https://msdn.microsoft.com/en-us/library/ms190476.aspx
+  // Result Precision: p1 + p2 + 1
+  // Result Scale:     s1 + s2
+  // scalastyle:on
+  override def resultDecimalType(p1: Int, s1: Int, p2: Int, s2: Int): DecimalType = {
+    val resultScale = s1 + s2
+    val resultPrecision = p1 + p2 + 1
+    if (allowPrecisionLoss) {
+      DecimalType.adjustPrecisionScale(resultPrecision, resultScale)
+    } else {
+      DecimalType.bounded(resultPrecision, resultScale)
+    }
+  }
+
   private lazy val numeric = TypeUtils.getNumeric(dataType, failOnError)
 
   protected override def nullSafeEval(input1: Any, input2: Any): Any = dataType match {
+    case DecimalType.Fixed(precision, scale) =>
+      checkDecimalOverflow(numeric.times(input1, input2).asInstanceOf[Decimal], precision, scale)
     case _: IntegerType if failOnError =>
       MathUtils.multiplyExact(input1.asInstanceOf[Int], input2.asInstanceOf[Int], queryContext)
     case _: LongType if failOnError =>
@@ -490,12 +602,25 @@ trait DivModLike extends BinaryArithmetic {
       s"${eval2.value} == 0"
     }
     val javaType = CodeGenerator.javaType(dataType)
-    val operation = if (operandsDataType.isInstanceOf[DecimalType]) {
-      decimalToDataTypeCodeGen(s"${eval1.value}.$decimalMethod(${eval2.value})")
+    val errorContext = if (failOnError) {
+      ctx.addReferenceObj("errCtx", queryContext)
     } else {
-      s"($javaType)(${eval1.value} $symbol ${eval2.value})"
+      "\"\""
+    }
+    val operation = super.dataType match {
+      case DecimalType.Fixed(precision, scale) =>
+        val decimalValue = ctx.freshName("decimalValue")
+        s"""
+           |Decimal $decimalValue = ${eval1.value}.$decimalMethod(${eval2.value}).toPrecision(
+           |  $precision, $scale, Decimal.ROUND_HALF_UP(), ${!failOnError}, $errorContext);
+           |if ($decimalValue != null) {
+           |  ${ev.value} = ${decimalToDataTypeCodeGen(s"$decimalValue")};
+           |} else {
+           |  ${ev.isNull} = true;
+           |}
+           |""".stripMargin
+      case _ => s"${ev.value} = ($javaType)(${eval1.value} $symbol ${eval2.value});"
     }
-    lazy val errorContext = ctx.addReferenceObj("errCtx", queryContext)
     val checkIntegralDivideOverflow = if (checkDivideOverflow) {
       s"""
         |if (${eval1.value} == ${Long.MinValue}L && ${eval2.value} == -1)
@@ -521,7 +646,7 @@ trait DivModLike extends BinaryArithmetic {
         } else {
           ${eval1.code}
           $checkIntegralDivideOverflow
-          ${ev.value} = $operation;
+          $operation
         }""")
     } else {
       val nullOnErrorCondition = if (failOnError) "" else s" || $isZero"
@@ -543,7 +668,7 @@ trait DivModLike extends BinaryArithmetic {
           } else {
             $failOnErrorBranch
             $checkIntegralDivideOverflow
-            ${ev.value} = $operation;
+            $operation
           }
         }""")
     }
@@ -575,7 +700,36 @@ case class Divide(
   override def symbol: String = "/"
   override def decimalMethod: String = "$div"
 
+  // scalastyle:off
+  // The formula follows Hive which is based on the SQL standard and MS SQL:
+  // https://cwiki.apache.org/confluence/download/attachments/27362075/Hive_Decimal_Precision_Scale_Support.pdf
+  // https://msdn.microsoft.com/en-us/library/ms190476.aspx
+  // Result Precision: p1 - s1 + s2 + max(6, s1 + p2 + 1)
+  // Result Scale:     max(6, s1 + p2 + 1)
+  // scalastyle:on
+  override def resultDecimalType(p1: Int, s1: Int, p2: Int, s2: Int): DecimalType = {
+    if (allowPrecisionLoss) {
+      val intDig = p1 - s1 + s2
+      val scale = max(DecimalType.MINIMUM_ADJUSTED_SCALE, s1 + p2 + 1)
+      val prec = intDig + scale
+      DecimalType.adjustPrecisionScale(prec, scale)
+    } else {
+      var intDig = min(DecimalType.MAX_SCALE, p1 - s1 + s2)
+      var decDig = min(DecimalType.MAX_SCALE, max(6, s1 + p2 + 1))
+      val diff = (intDig + decDig) - DecimalType.MAX_SCALE
+      if (diff > 0) {
+        decDig -= diff / 2 + 1
+        intDig = DecimalType.MAX_SCALE - decDig
+      }
+      DecimalType.bounded(intDig + decDig, decDig)
+    }
+  }
+
   private lazy val div: (Any, Any) => Any = dataType match {
+    case d @ DecimalType.Fixed(precision, scale) => (l, r) => {
+      val value = d.fractional.asInstanceOf[Fractional[Any]].div(l, r)
+      checkDecimalOverflow(value.asInstanceOf[Decimal], precision, scale)
+    }
     case ft: FractionalType => ft.fractional.asInstanceOf[Fractional[Any]].div
   }
 
@@ -618,6 +772,14 @@ case class IntegralDivide(
   override def symbol: String = "/"
   override def decimalMethod: String = "quot"
   override def decimalToDataTypeCodeGen(decimalResult: String): String = s"$decimalResult.toLong()"
+
+  override def resultDecimalType(p1: Int, s1: Int, p2: Int, s2: Int): DecimalType = {
+    // This follows division rule
+    val intDig = p1 - s1 + s2
+    // No precision loss can happen as the result scale is 0.
+    DecimalType.bounded(intDig, 0)
+  }
+
   override def sqlOperator: String = "div"
 
   private lazy val div: (Any, Any) => Any = {
@@ -632,7 +794,11 @@ case class IntegralDivide(
         LongType.integral.asInstanceOf[Integral[Any]]
     }
     (x, y) => {
-      val res = integral.quot(x, y)
+      val res = super.dataType match {
+        case DecimalType.Fixed(precision, scale) =>
+          checkDecimalOverflow(integral.quot(x, y).asInstanceOf[Decimal], precision, scale)
+        case _ => integral.quot(x, y)
+      }
       if (res == null) {
         null
       } else {
@@ -670,6 +836,24 @@ case class Remainder(
 
   override def symbol: String = "%"
   override def decimalMethod: String = "remainder"
+
+  // scalastyle:off
+  // The formula follows Hive which is based on the SQL standard and MS SQL:
+  // https://cwiki.apache.org/confluence/download/attachments/27362075/Hive_Decimal_Precision_Scale_Support.pdf
+  // https://msdn.microsoft.com/en-us/library/ms190476.aspx
+  // Result Precision: min(p1-s1, p2-s2) + max(s1, s2)
+  // Result Scale:     max(s1, s2)
+  // scalastyle:on
+  override def resultDecimalType(p1: Int, s1: Int, p2: Int, s2: Int): DecimalType = {
+    val resultScale = max(s1, s2)
+    val resultPrecision = min(p1 - s1, p2 - s2) + resultScale
+    if (allowPrecisionLoss) {
+      DecimalType.adjustPrecisionScale(resultPrecision, resultScale)
+    } else {
+      DecimalType.bounded(resultPrecision, resultScale)
+    }
+  }
+
   override def toString: String = {
     getTagValue(FunctionRegistry.FUNC_ALIAS).getOrElse(sqlOperator) match {
       case operator if operator == sqlOperator => s"($left $sqlOperator $right)"
@@ -694,9 +878,11 @@ case class Remainder(
     case i: IntegralType =>
       val integral = i.integral.asInstanceOf[Integral[Any]]
       (left, right) => integral.rem(left, right)
-    case i: FractionalType => // should only be DecimalType for now
-      val integral = i.asIntegral.asInstanceOf[Integral[Any]]
-      (left, right) => integral.rem(left, right)
+
+    case d @ DecimalType.Fixed(precision, scale) =>
+      val integral = d.asIntegral.asInstanceOf[Integral[Any]]
+      (left, right) =>
+        checkDecimalOverflow(integral.rem(left, right).asInstanceOf[Decimal], precision, scale)
   }
 
   override def evalOperation(left: Any, right: Any): Any = mod(left, right)
@@ -734,11 +920,35 @@ case class Pmod(
 
   override def nullable: Boolean = true
 
+  override def decimalMethod: String = "remainder"
+
+  // This follows Remainder rule
+  override def resultDecimalType(p1: Int, s1: Int, p2: Int, s2: Int): DecimalType = {
+    val resultScale = max(s1, s2)
+    val resultPrecision = min(p1 - s1, p2 - s2) + resultScale
+    if (allowPrecisionLoss) {
+      DecimalType.adjustPrecisionScale(resultPrecision, resultScale)
+    } else {
+      DecimalType.bounded(resultPrecision, resultScale)
+    }
+  }
+
   private lazy val isZero: Any => Boolean = right.dataType match {
     case _: DecimalType => x => x.asInstanceOf[Decimal].isZero
     case _ => x => x == 0
   }
 
+  private lazy val pmodFunc: (Any, Any) => Any = dataType match {
+    case _: IntegerType => (l, r) => pmod(l.asInstanceOf[Int], r.asInstanceOf[Int])
+    case _: LongType => (l, r) => pmod(l.asInstanceOf[Long], r.asInstanceOf[Long])
+    case _: ShortType => (l, r) => pmod(l.asInstanceOf[Short], r.asInstanceOf[Short])
+    case _: ByteType => (l, r) => pmod(l.asInstanceOf[Byte], r.asInstanceOf[Byte])
+    case _: FloatType => (l, r) => pmod(l.asInstanceOf[Float], r.asInstanceOf[Float])
+    case _: DoubleType => (l, r) => pmod(l.asInstanceOf[Double], r.asInstanceOf[Double])
+    case DecimalType.Fixed(precision, scale) => (l, r) => checkDecimalOverflow(
+      pmod(l.asInstanceOf[Decimal], r.asInstanceOf[Decimal]), precision, scale)
+  }
+
   final override def eval(input: InternalRow): Any = {
     // evaluate right first as we have a chance to skip left if right is 0
     val input2 = right.eval(input)
@@ -753,15 +963,7 @@ case class Pmod(
           // when we reach here, failOnError must bet true.
           throw QueryExecutionErrors.divideByZeroError(queryContext)
         }
-        input1 match {
-          case i: Integer => pmod(i, input2.asInstanceOf[java.lang.Integer])
-          case l: Long => pmod(l, input2.asInstanceOf[java.lang.Long])
-          case s: Short => pmod(s, input2.asInstanceOf[java.lang.Short])
-          case b: Byte => pmod(b, input2.asInstanceOf[java.lang.Byte])
-          case f: Float => pmod(f, input2.asInstanceOf[java.lang.Float])
-          case d: Double => pmod(d, input2.asInstanceOf[java.lang.Double])
-          case d: Decimal => pmod(d, input2.asInstanceOf[Decimal])
-        }
+        pmodFunc(input1, input2)
       }
     }
   }
@@ -776,18 +978,26 @@ case class Pmod(
     }
     val remainder = ctx.freshName("remainder")
     val javaType = CodeGenerator.javaType(dataType)
-    lazy val errorContext = ctx.addReferenceObj("errCtx", queryContext)
+    val errorContext = if (failOnError) {
+      ctx.addReferenceObj("errCtx", queryContext)
+    } else {
+      "\"\""
+    }
     val result = dataType match {
-      case DecimalType.Fixed(_, _) =>
+      case DecimalType.Fixed(precision, scale) =>
         val decimalAdd = "$plus"
         s"""
-          $javaType $remainder = ${eval1.value}.remainder(${eval2.value});
-          if ($remainder.compare(new org.apache.spark.sql.types.Decimal().set(0)) < 0) {
-            ${ev.value}=($remainder.$decimalAdd(${eval2.value})).remainder(${eval2.value});
-          } else {
-            ${ev.value}=$remainder;
-          }
-        """
+           |$javaType $remainder = ${eval1.value}.$decimalMethod(${eval2.value});
+           |if ($remainder.compare(new org.apache.spark.sql.types.Decimal().set(0)) < 0) {
+           |  ${ev.value}=($remainder.$decimalAdd(${eval2.value})).$decimalMethod(${eval2.value});
+           |} else {
+           |  ${ev.value}=$remainder;
+           |}
+           |${ev.value} = ${ev.value}.toPrecision(
+           |  $precision, $scale, Decimal.ROUND_HALF_UP(), ${!failOnError}, $errorContext);
+           |${ev.isNull} = ${ev.value} == null;
+           |""".stripMargin
+
       // byte and short are casted into int when add, minus, times or divide
       case ByteType | ShortType =>
         s"""
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalExpressions.scala
index 7d25df5ae9c..7335763c253 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalExpressions.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalExpressions.scala
@@ -20,6 +20,7 @@ package org.apache.spark.sql.catalyst.expressions
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode}
 import org.apache.spark.sql.catalyst.expressions.codegen.Block._
+import org.apache.spark.sql.catalyst.util.TypeUtils
 import org.apache.spark.sql.errors.QueryExecutionErrors
 import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.types._
@@ -103,23 +104,6 @@ object MakeDecimal {
   }
 }
 
-/**
- * An expression used to wrap the children when promote the precision of DecimalType to avoid
- * promote multiple times.
- */
-case class PromotePrecision(child: Expression) extends UnaryExpression {
-  override def dataType: DataType = child.dataType
-  override def eval(input: InternalRow): Any = child.eval(input)
-  override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode =
-    child.genCode(ctx)
-  override def prettyName: String = "promote_precision"
-  override def sql: String = child.sql
-  override lazy val preCanonicalized: Expression = child.preCanonicalized
-
-  override protected def withNewChildInternal(newChild: Expression): Expression =
-    copy(child = newChild)
-}
-
 /**
  * Rounds the decimal to given scale and check whether the decimal can fit in provided precision
  * or not. If not, if `nullOnOverflow` is `true`, it returns `null`; otherwise an
@@ -232,3 +216,34 @@ case class CheckOverflowInSum(
   override protected def withNewChildInternal(newChild: Expression): CheckOverflowInSum =
     copy(child = newChild)
 }
+
+/**
+ * An add expression for decimal values which is only used internally by Sum/Avg.
+ *
+ * Nota that, this expression does not check overflow which is different with `Add`. When
+ * aggregating values, Spark writes the aggregation buffer values to `UnsafeRow` via
+ * `UnsafeRowWriter`, which already checks decimal overflow, so we don't need to do it again in the
+ * add expression used by Sum/Avg.
+ */
+case class DecimalAddNoOverflowCheck(
+    left: Expression,
+    right: Expression,
+    override val dataType: DataType) extends BinaryOperator {
+  require(dataType.isInstanceOf[DecimalType])
+
+  override def inputType: AbstractDataType = DecimalType
+  override def symbol: String = "+"
+  private def decimalMethod: String = "$plus"
+
+  private lazy val numeric = TypeUtils.getNumeric(dataType)
+
+  override protected def nullSafeEval(input1: Any, input2: Any): Any =
+    numeric.plus(input1, input2)
+
+  override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode =
+    defineCodeGen(ctx, ev, (eval1, eval2) => s"$eval1.$decimalMethod($eval2)")
+
+  override protected def withNewChildrenInternal(
+      newLeft: Expression, newRight: Expression): DecimalAddNoOverflowCheck =
+    copy(left = newLeft, right = newRight)
+}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala
index 43203e4f397..759a5dce967 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala
@@ -481,9 +481,14 @@ final class Decimal extends Ordered[Decimal] with Serializable {
 
   def isZero: Boolean = if (decimalVal.ne(null)) decimalVal == BIG_DEC_ZERO else longVal == 0
 
+  // We should follow DecimalPrecision promote if use longVal for add and subtract:
+  // Operation    Result Precision                        Result Scale
+  // ------------------------------------------------------------------------
+  // e1 + e2      max(s1, s2) + max(p1-s1, p2-s2) + 1     max(s1, s2)
+  // e1 - e2      max(s1, s2) + max(p1-s1, p2-s2) + 1     max(s1, s2)
   def + (that: Decimal): Decimal = {
     if (decimalVal.eq(null) && that.decimalVal.eq(null) && scale == that.scale) {
-      Decimal(longVal + that.longVal, Math.max(precision, that.precision), scale)
+      Decimal(longVal + that.longVal, Math.max(precision, that.precision) + 1, scale)
     } else {
       Decimal(toBigDecimal.bigDecimal.add(that.toBigDecimal.bigDecimal))
     }
@@ -491,7 +496,7 @@ final class Decimal extends Ordered[Decimal] with Serializable {
 
   def - (that: Decimal): Decimal = {
     if (decimalVal.eq(null) && that.decimalVal.eq(null) && scale == that.scale) {
-      Decimal(longVal - that.longVal, Math.max(precision, that.precision), scale)
+      Decimal(longVal - that.longVal, Math.max(precision, that.precision) + 1, scale)
     } else {
       Decimal(toBigDecimal.bigDecimal.subtract(that.toBigDecimal.bigDecimal))
     }
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala
index 32ab47f43d3..2bfa072a13a 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala
@@ -23,7 +23,6 @@ import java.time.temporal.ChronoUnit
 
 import org.apache.spark.{SparkArithmeticException, SparkFunSuite}
 import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.catalyst.analysis.DecimalPrecision
 import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.TypeCheckFailure
 import org.apache.spark.sql.catalyst.dsl.expressions._
 import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext
@@ -610,13 +609,13 @@ class ArithmeticExpressionSuite extends SparkFunSuite with ExpressionEvalHelper
         IntegralDivide(Literal(Decimal(0.2)), Literal(Decimal(0.0))), "Division by zero")
     }
     // overflows long and so returns a wrong result
-    checkEvaluation(DecimalPrecision.decimalAndDecimal.apply(IntegralDivide(
-      Literal(Decimal("99999999999999999999999999999999999")), Literal(Decimal(0.001)))),
+    checkEvaluation(IntegralDivide(
+      Literal(Decimal("99999999999999999999999999999999999")), Literal(Decimal(0.001))),
       687399551400672280L)
     // overflow during promote precision
     withSQLConf(SQLConf.ANSI_ENABLED.key -> "false") {
-      checkEvaluation(DecimalPrecision.decimalAndDecimal.apply(IntegralDivide(
-        Literal(Decimal("99999999999999999999999999999999999999")), Literal(Decimal(0.00001)))),
+      checkEvaluation(IntegralDivide(
+        Literal(Decimal("99999999999999999999999999999999999999")), Literal(Decimal(0.00001))),
         null)
     }
   }
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CanonicalizeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CanonicalizeSuite.scala
index 92c0b67d19c..d814236c3fe 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CanonicalizeSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CanonicalizeSuite.scala
@@ -171,13 +171,6 @@ class CanonicalizeSuite extends SparkFunSuite {
     }
   }
 
-  test("SPARK-35742: Expression.semanticEquals should be symmetrical") {
-    val attr = AttributeReference("col", IntegerType)()
-    val expr = PromotePrecision(attr)
-    assert(expr.semanticEquals(attr))
-    assert(attr.semanticEquals(expr))
-  }
-
   test("SPARK-38030: Canonicalization should not remove nullability of AttributeReference" +
     " dataType") {
     val structType = StructType(Seq(StructField("name", StringType, nullable = false)))
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DecimalExpressionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DecimalExpressionSuite.scala
index 1a8cd63aed0..637510d81b0 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DecimalExpressionSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DecimalExpressionSuite.scala
@@ -53,14 +53,6 @@ class DecimalExpressionSuite extends SparkFunSuite with ExpressionEvalHelper {
     }
   }
 
-  test("PromotePrecision") {
-    val d1 = Decimal("10.1")
-    checkEvaluation(PromotePrecision(Literal(d1)), d1)
-    val d2 = Decimal(101, 3, 1)
-    checkEvaluation(PromotePrecision(Literal(d2)), d2)
-    checkEvaluation(PromotePrecision(Literal.create(null, DecimalType(2, 1))), null)
-  }
-
   test("CheckOverflow") {
     val d1 = Decimal("10.1")
     checkEvaluation(CheckOverflow(Literal(d1), DecimalType(4, 0), true), Decimal("10"))
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/SubexpressionEliminationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/SubexpressionEliminationSuite.scala
index 2375d3ed35f..08fd53b3534 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/SubexpressionEliminationSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/SubexpressionEliminationSuite.scala
@@ -17,10 +17,11 @@
 package org.apache.spark.sql.catalyst.expressions
 
 import org.apache.spark.{SparkFunSuite, TaskContext, TaskContextImpl}
+import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.expressions.codegen._
 import org.apache.spark.sql.catalyst.plans.logical.LocalRelation
 import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.types.{BinaryType, DataType, Decimal, IntegerType}
+import org.apache.spark.sql.types.{BinaryType, DataType, IntegerType}
 
 class SubexpressionEliminationSuite extends SparkFunSuite with ExpressionEvalHelper {
   test("Semantic equals and hash") {
@@ -341,7 +342,7 @@ class SubexpressionEliminationSuite extends SparkFunSuite with ExpressionEvalHel
 
   test("SPARK-36073: Transparently canonicalized expressions are not necessary subexpressions") {
     val add = Add(Literal(1), Literal(2))
-    val transparent = PromotePrecision(add)
+    val transparent = ProxyExpression(add)
 
     val equivalence = new EquivalentExpressions
     equivalence.addExprTree(transparent)
@@ -435,19 +436,6 @@ class SubexpressionEliminationSuite extends SparkFunSuite with ExpressionEvalHel
     }
   }
 
-  test("SPARK-35886: PromotePrecision should not overwrite genCode") {
-    val p = PromotePrecision(Literal(Decimal("10.1")))
-
-    val ctx = new CodegenContext()
-    val subExprs = ctx.subexpressionEliminationForWholeStageCodegen(Seq(p, p))
-    val code = ctx.withSubExprEliminationExprs(subExprs.states) {
-      Seq(p.genCode(ctx))
-    }.head
-    // Decimal `Literal` will add the value by `addReferenceObj`.
-    // So if `p` is replaced by subexpression, the literal will be reused.
-    assert(code.value.toString == "((Decimal) references[0] /* literal */)")
-  }
-
   test("SPARK-39040: Respect NaNvl in EquivalentExpressions for expression elimination") {
     val add = Add(Literal(1), Literal(0))
     val n1 = NaNvl(Literal(1.0d), Add(add, add))
@@ -469,3 +457,13 @@ case class CodegenFallbackExpression(child: Expression)
   override protected def withNewChildInternal(newChild: Expression): CodegenFallbackExpression =
     copy(child = newChild)
 }
+
+case class ProxyExpression(child: Expression) extends UnaryExpression {
+  override lazy val preCanonicalized: Expression = child.preCanonicalized
+  override def dataType: DataType = child.dataType
+  override def eval(input: InternalRow): Any = child.eval(input)
+  override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode =
+    child.genCode(ctx)
+  override protected def withNewChildInternal(newChild: Expression): Expression =
+    copy(child = newChild)
+}
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DecimalSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DecimalSuite.scala
index 6f70dc51b95..95e1f832821 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DecimalSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DecimalSuite.scala
@@ -194,6 +194,13 @@ class DecimalSuite extends SparkFunSuite with PrivateMethodTester with SQLHelper
     assert(Decimal(100) % Decimal(0) === null)
   }
 
+  test("longVal arithmetic") {
+    assert(Decimal(10, 2, 0) + Decimal(10, 2, 0) === Decimal(20, 3, 0))
+    assert(Decimal(10, 2, 0) + Decimal(90, 2, 0) === Decimal(100, 3, 0))
+    assert(Decimal(10, 2, 0) - Decimal(-10, 2, 0) === Decimal(20, 3, 0))
+    assert(Decimal(10, 2, 0) - Decimal(-90, 2, 0) === Decimal(100, 3, 0))
+  }
+
   // regression test for SPARK-8359
   test("accurate precision after multiplication") {
     val decimal = (Decimal(Long.MaxValue, 38, 0) * Decimal(Long.MaxValue, 38, 0)).toJavaBigDecimal
diff --git a/sql/core/src/test/resources/sql-tests/inputs/decimalArithmeticOperations.sql b/sql/core/src/test/resources/sql-tests/inputs/decimalArithmeticOperations.sql
index 70bb9123994..0775b978033 100644
--- a/sql/core/src/test/resources/sql-tests/inputs/decimalArithmeticOperations.sql
+++ b/sql/core/src/test/resources/sql-tests/inputs/decimalArithmeticOperations.sql
@@ -54,6 +54,36 @@ select 12345678912345678912345678912.1234567 + 9999999999999999999999999999999.1
 select 123456789123456789.1234567890 * 1.123456789123456789;
 select 12345678912345.123456789123 / 0.000000012345678;
 
+-- union decimal type
+
+SELECT CAST(20 AS DECIMAL(4, 1))
+UNION ALL
+SELECT CAST(10 AS DECIMAL(3, 1)) + CAST(90 AS DECIMAL(3, 1));
+
+SELECT CAST(20 AS DECIMAL(4, 1))
+UNION ALL
+SELECT CAST(10 AS DECIMAL(3, 1)) - CAST(-90 AS DECIMAL(3, 1));
+
+SELECT CAST(20 AS DECIMAL(4, 1))
+UNION ALL
+SELECT CAST(10 AS DECIMAL(3, 1)) * CAST(10 AS DECIMAL(3, 1));
+
+SELECT CAST(20 AS DECIMAL(4, 1))
+UNION ALL
+SELECT CAST(10 AS DECIMAL(3, 1)) / CAST(10 AS DECIMAL(3, 1));
+
+SELECT CAST(20 AS DECIMAL(4, 1))
+UNION ALL
+SELECT CAST(10 AS DECIMAL(10, 2)) % CAST(3 AS DECIMAL(5, 1));
+
+SELECT CAST(20 AS DECIMAL(4, 1))
+UNION ALL
+SELECT pmod(CAST(10 AS DECIMAL(10, 2)), CAST(3 AS DECIMAL(5, 1)));
+
+SELECT CAST(20 AS DECIMAL(4, 1))
+UNION ALL
+SELECT CAST(10 AS DECIMAL(10, 2)) div CAST(3 AS DECIMAL(5, 1));
+
 -- return NULL instead of rounding, according to old Spark versions' behavior
 set spark.sql.decimalOperations.allowPrecisionLoss=false;
 
diff --git a/sql/core/src/test/resources/sql-tests/results/decimalArithmeticOperations.sql.out b/sql/core/src/test/resources/sql-tests/results/decimalArithmeticOperations.sql.out
index f58950e7071..c0ab65f41bc 100644
--- a/sql/core/src/test/resources/sql-tests/results/decimalArithmeticOperations.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/decimalArithmeticOperations.sql.out
@@ -1,5 +1,5 @@
 -- Automatically generated by SQLQueryTestSuite
--- Number of queries: 48
+-- Number of queries: 55
 
 
 -- !query
@@ -185,6 +185,83 @@ struct<(12345678912345.123456789123 / 1.2345678E-8):decimal(38,9)>
 1000000073899961059796.725866332
 
 
+-- !query
+SELECT CAST(20 AS DECIMAL(4, 1))
+UNION ALL
+SELECT CAST(10 AS DECIMAL(3, 1)) + CAST(90 AS DECIMAL(3, 1))
+-- !query schema
+struct<CAST(20 AS DECIMAL(4,1)):decimal(4,1)>
+-- !query output
+100.0
+20.0
+
+
+-- !query
+SELECT CAST(20 AS DECIMAL(4, 1))
+UNION ALL
+SELECT CAST(10 AS DECIMAL(3, 1)) - CAST(-90 AS DECIMAL(3, 1))
+-- !query schema
+struct<CAST(20 AS DECIMAL(4,1)):decimal(4,1)>
+-- !query output
+100.0
+20.0
+
+
+-- !query
+SELECT CAST(20 AS DECIMAL(4, 1))
+UNION ALL
+SELECT CAST(10 AS DECIMAL(3, 1)) * CAST(10 AS DECIMAL(3, 1))
+-- !query schema
+struct<CAST(20 AS DECIMAL(4,1)):decimal(7,2)>
+-- !query output
+100.00
+20.00
+
+
+-- !query
+SELECT CAST(20 AS DECIMAL(4, 1))
+UNION ALL
+SELECT CAST(10 AS DECIMAL(3, 1)) / CAST(10 AS DECIMAL(3, 1))
+-- !query schema
+struct<CAST(20 AS DECIMAL(4,1)):decimal(9,6)>
+-- !query output
+1.000000
+20.000000
+
+
+-- !query
+SELECT CAST(20 AS DECIMAL(4, 1))
+UNION ALL
+SELECT CAST(10 AS DECIMAL(10, 2)) % CAST(3 AS DECIMAL(5, 1))
+-- !query schema
+struct<CAST(20 AS DECIMAL(4,1)):decimal(6,2)>
+-- !query output
+1.00
+20.00
+
+
+-- !query
+SELECT CAST(20 AS DECIMAL(4, 1))
+UNION ALL
+SELECT pmod(CAST(10 AS DECIMAL(10, 2)), CAST(3 AS DECIMAL(5, 1)))
+-- !query schema
+struct<CAST(20 AS DECIMAL(4,1)):decimal(6,2)>
+-- !query output
+1.00
+20.00
+
+
+-- !query
+SELECT CAST(20 AS DECIMAL(4, 1))
+UNION ALL
+SELECT CAST(10 AS DECIMAL(10, 2)) div CAST(3 AS DECIMAL(5, 1))
+-- !query schema
+struct<CAST(20 AS DECIMAL(4,1)):decimal(21,1)>
+-- !query output
+20.0
+3.0
+
+
 -- !query
 set spark.sql.decimalOperations.allowPrecisionLoss=false
 -- !query schema
diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/numeric.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/numeric.sql.out
index e2a2d9359a9..c075775ca50 100644
--- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/numeric.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/numeric.sql.out
@@ -4836,7 +4836,7 @@ struct<(CAST(999999999999999999999 AS DECIMAL(38,0)) div 1000000000000000000000)
 -- !query
 select mod(cast(999999999999999999999 as decimal(38, 0)),1000000000000000000000)
 -- !query schema
-struct<(CAST(999999999999999999999 AS DECIMAL(38,0)) % 1000000000000000000000):decimal(22,0)>
+struct<mod(CAST(999999999999999999999 AS DECIMAL(38,0)), 1000000000000000000000):decimal(22,0)>
 -- !query output
 999999999999999999999
 
@@ -4852,7 +4852,7 @@ struct<(CAST(-9999999999999999999999 AS DECIMAL(38,0)) div 100000000000000000000
 -- !query
 select mod(cast(-9999999999999999999999 as decimal(38, 0)),1000000000000000000000)
 -- !query schema
-struct<(CAST(-9999999999999999999999 AS DECIMAL(38,0)) % 1000000000000000000000):decimal(22,0)>
+struct<mod(CAST(-9999999999999999999999 AS DECIMAL(38,0)), 1000000000000000000000):decimal(22,0)>
 -- !query output
 -999999999999999999999
 
@@ -4860,7 +4860,7 @@ struct<(CAST(-9999999999999999999999 AS DECIMAL(38,0)) % 1000000000000000000000)
 -- !query
 select div(cast(-9999999999999999999999 as decimal(38, 0)),1000000000000000000000)*1000000000000000000000 + mod(cast(-9999999999999999999999 as decimal(38, 0)),1000000000000000000000)
 -- !query schema
-struct<(((CAST(-9999999999999999999999 AS DECIMAL(38,0)) div 1000000000000000000000) * 1000000000000000000000) + (CAST(-9999999999999999999999 AS DECIMAL(38,0)) % 1000000000000000000000)):decimal(38,0)>
+struct<(((CAST(-9999999999999999999999 AS DECIMAL(38,0)) div 1000000000000000000000) * 1000000000000000000000) + mod(CAST(-9999999999999999999999 AS DECIMAL(38,0)), 1000000000000000000000)):decimal(38,0)>
 -- !query output
 -9999999999999999999999
 
@@ -4868,7 +4868,7 @@ struct<(((CAST(-9999999999999999999999 AS DECIMAL(38,0)) div 1000000000000000000
 -- !query
 select mod (70.0,70)
 -- !query schema
-struct<(70.0 % 70):decimal(3,1)>
+struct<mod(70.0, 70):decimal(3,1)>
 -- !query output
 0.0
 
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53.sf100/explain.txt
index 321d0ec4277..48d800e9a65 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53.sf100/explain.txt
@@ -146,7 +146,7 @@ Arguments: [avg(_w0#22) windowspecdefinition(i_manufact_id#5, specifiedwindowfra
 
 (26) Filter [codegen id : 7]
 Input [4]: [i_manufact_id#5, sum_sales#21, _w0#22, avg_quarterly_sales#23]
-Condition : (isnotnull(avg_quarterly_sales#23) AND ((avg_quarterly_sales#23 > 0.000000) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_quarterly_sales#23 as decimal(22,6)))), DecimalType(22,6)))) / promote_precision(cast(avg_quarterly_sales#23 as decimal(22,6)))), DecimalType(38,16)) > 0.1000000000000000)))
+Condition : ((isnotnull(avg_quarterly_sales#23) AND isnotnull(sum_sales#21)) AND ((avg_quarterly_sales#23 > 0.000000) AND ((abs((sum_sales#21 - avg_quarterly_sales#23)) / avg_quarterly_sales#23) > 0.1000000000000000)))
 
 (27) Project [codegen id : 7]
 Output [3]: [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53/explain.txt
index c91dd8a11b5..1e318c7ccda 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q53/explain.txt
@@ -146,7 +146,7 @@ Arguments: [avg(_w0#22) windowspecdefinition(i_manufact_id#5, specifiedwindowfra
 
 (26) Filter [codegen id : 7]
 Input [4]: [i_manufact_id#5, sum_sales#21, _w0#22, avg_quarterly_sales#23]
-Condition : (isnotnull(avg_quarterly_sales#23) AND ((avg_quarterly_sales#23 > 0.000000) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_quarterly_sales#23 as decimal(22,6)))), DecimalType(22,6)))) / promote_precision(cast(avg_quarterly_sales#23 as decimal(22,6)))), DecimalType(38,16)) > 0.1000000000000000)))
+Condition : ((isnotnull(avg_quarterly_sales#23) AND isnotnull(sum_sales#21)) AND ((avg_quarterly_sales#23 > 0.000000) AND ((abs((sum_sales#21 - avg_quarterly_sales#23)) / avg_quarterly_sales#23) > 0.1000000000000000)))
 
 (27) Project [codegen id : 7]
 Output [3]: [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59.sf100/explain.txt
index 1637f847f41..ce34acbd9b9 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59.sf100/explain.txt
@@ -301,7 +301,7 @@ Right keys [2]: [s_store_id2#71, (d_week_seq2#70 - 52)]
 Join condition: None
 
 (53) Project [codegen id : 10]
-Output [10]: [s_store_name1#42, s_store_id1#44, d_week_seq1#43, CheckOverflow((promote_precision(sun_sales1#45) / promote_precision(sun_sales2#72)), DecimalType(37,20)) AS (sun_sales1 / sun_sales2)#78, CheckOverflow((promote_precision(mon_sales1#46) / promote_precision(mon_sales2#73)), DecimalType(37,20)) AS (mon_sales1 / mon_sales2)#79, CheckOverflow((promote_precision(tue_sales1#47) / promote_precision(tue_sales1#47)), DecimalType(37,20)) AS (tue_sales1 / tue_sales1)#80, CheckOverflow( [...]
+Output [10]: [s_store_name1#42, s_store_id1#44, d_week_seq1#43, (sun_sales1#45 / sun_sales2#72) AS (sun_sales1 / sun_sales2)#78, (mon_sales1#46 / mon_sales2#73) AS (mon_sales1 / mon_sales2)#79, (tue_sales1#47 / tue_sales1#47) AS (tue_sales1 / tue_sales1)#80, (wed_sales1#48 / wed_sales2#74) AS (wed_sales1 / wed_sales2)#81, (thu_sales1#49 / thu_sales2#75) AS (thu_sales1 / thu_sales2)#82, (fri_sales1#50 / fri_sales2#76) AS (fri_sales1 / fri_sales2)#83, (sat_sales1#51 / sat_sales2#77) AS (sa [...]
 Input [18]: [s_store_name1#42, d_week_seq1#43, s_store_id1#44, sun_sales1#45, mon_sales1#46, tue_sales1#47, wed_sales1#48, thu_sales1#49, fri_sales1#50, sat_sales1#51, d_week_seq2#70, s_store_id2#71, sun_sales2#72, mon_sales2#73, wed_sales2#74, thu_sales2#75, fri_sales2#76, sat_sales2#77]
 
 (54) TakeOrderedAndProject
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59/explain.txt
index c2e49a39aa6..d8944347404 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q59/explain.txt
@@ -283,7 +283,7 @@ Right keys [2]: [s_store_id2#67, (d_week_seq2#66 - 52)]
 Join condition: None
 
 (50) Project [codegen id : 10]
-Output [10]: [s_store_name1#40, s_store_id1#42, d_week_seq1#41, CheckOverflow((promote_precision(sun_sales1#43) / promote_precision(sun_sales2#68)), DecimalType(37,20)) AS (sun_sales1 / sun_sales2)#74, CheckOverflow((promote_precision(mon_sales1#44) / promote_precision(mon_sales2#69)), DecimalType(37,20)) AS (mon_sales1 / mon_sales2)#75, CheckOverflow((promote_precision(tue_sales1#45) / promote_precision(tue_sales1#45)), DecimalType(37,20)) AS (tue_sales1 / tue_sales1)#76, CheckOverflow( [...]
+Output [10]: [s_store_name1#40, s_store_id1#42, d_week_seq1#41, (sun_sales1#43 / sun_sales2#68) AS (sun_sales1 / sun_sales2)#74, (mon_sales1#44 / mon_sales2#69) AS (mon_sales1 / mon_sales2)#75, (tue_sales1#45 / tue_sales1#45) AS (tue_sales1 / tue_sales1)#76, (wed_sales1#46 / wed_sales2#70) AS (wed_sales1 / wed_sales2)#77, (thu_sales1#47 / thu_sales2#71) AS (thu_sales1 / thu_sales2)#78, (fri_sales1#48 / fri_sales2#72) AS (fri_sales1 / fri_sales2)#79, (sat_sales1#49 / sat_sales2#73) AS (sa [...]
 Input [18]: [s_store_name1#40, d_week_seq1#41, s_store_id1#42, sun_sales1#43, mon_sales1#44, tue_sales1#45, wed_sales1#46, thu_sales1#47, fri_sales1#48, sat_sales1#49, d_week_seq2#66, s_store_id2#67, sun_sales2#68, mon_sales2#69, wed_sales2#70, thu_sales2#71, fri_sales2#72, sat_sales2#73]
 
 (51) TakeOrderedAndProject
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63.sf100/explain.txt
index 8a225052b4f..4455d2b1681 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63.sf100/explain.txt
@@ -146,7 +146,7 @@ Arguments: [avg(_w0#22) windowspecdefinition(i_manager_id#5, specifiedwindowfram
 
 (26) Filter [codegen id : 7]
 Input [4]: [i_manager_id#5, sum_sales#21, _w0#22, avg_monthly_sales#23]
-Condition : (isnotnull(avg_monthly_sales#23) AND ((avg_monthly_sales#23 > 0.000000) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#23 as decimal(22,6)))), DecimalType(22,6)))) / promote_precision(cast(avg_monthly_sales#23 as decimal(22,6)))), DecimalType(38,16)) > 0.1000000000000000)))
+Condition : ((isnotnull(avg_monthly_sales#23) AND isnotnull(sum_sales#21)) AND ((avg_monthly_sales#23 > 0.000000) AND ((abs((sum_sales#21 - avg_monthly_sales#23)) / avg_monthly_sales#23) > 0.1000000000000000)))
 
 (27) Project [codegen id : 7]
 Output [3]: [i_manager_id#5, sum_sales#21, avg_monthly_sales#23]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63/explain.txt
index bdb9612bfa2..131b0442849 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q63/explain.txt
@@ -146,7 +146,7 @@ Arguments: [avg(_w0#22) windowspecdefinition(i_manager_id#5, specifiedwindowfram
 
 (26) Filter [codegen id : 7]
 Input [4]: [i_manager_id#5, sum_sales#21, _w0#22, avg_monthly_sales#23]
-Condition : (isnotnull(avg_monthly_sales#23) AND ((avg_monthly_sales#23 > 0.000000) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#23 as decimal(22,6)))), DecimalType(22,6)))) / promote_precision(cast(avg_monthly_sales#23 as decimal(22,6)))), DecimalType(38,16)) > 0.1000000000000000)))
+Condition : ((isnotnull(avg_monthly_sales#23) AND isnotnull(sum_sales#21)) AND ((avg_monthly_sales#23 > 0.000000) AND ((abs((sum_sales#21 - avg_monthly_sales#23)) / avg_monthly_sales#23) > 0.1000000000000000)))
 
 (27) Project [codegen id : 7]
 Output [3]: [i_manager_id#5, sum_sales#21, avg_monthly_sales#23]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65.sf100/explain.txt
index 1be6b422f01..bca0f9234de 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65.sf100/explain.txt
@@ -1,13 +1,13 @@
 == Physical Plan ==
-TakeOrderedAndProject (38)
-+- * Project (37)
-   +- * BroadcastHashJoin Inner BuildLeft (36)
-      :- BroadcastExchange (32)
-      :  +- * Project (31)
-      :     +- * BroadcastHashJoin Inner BuildLeft (30)
-      :        :- BroadcastExchange (26)
-      :        :  +- * Project (25)
-      :        :     +- * BroadcastHashJoin Inner BuildRight (24)
+TakeOrderedAndProject (39)
++- * Project (38)
+   +- * BroadcastHashJoin Inner BuildLeft (37)
+      :- BroadcastExchange (33)
+      :  +- * Project (32)
+      :     +- * BroadcastHashJoin Inner BuildLeft (31)
+      :        :- BroadcastExchange (27)
+      :        :  +- * Project (26)
+      :        :     +- * BroadcastHashJoin Inner BuildRight (25)
       :        :        :- * Filter (10)
       :        :        :  +- * HashAggregate (9)
       :        :        :     +- Exchange (8)
@@ -18,25 +18,26 @@ TakeOrderedAndProject (38)
       :        :        :                 :  +- * ColumnarToRow (2)
       :        :        :                 :     +- Scan parquet default.store_sales (1)
       :        :        :                 +- ReusedExchange (4)
-      :        :        +- BroadcastExchange (23)
-      :        :           +- * HashAggregate (22)
-      :        :              +- Exchange (21)
-      :        :                 +- * HashAggregate (20)
-      :        :                    +- * HashAggregate (19)
-      :        :                       +- Exchange (18)
-      :        :                          +- * HashAggregate (17)
-      :        :                             +- * Project (16)
-      :        :                                +- * BroadcastHashJoin Inner BuildRight (15)
-      :        :                                   :- * Filter (13)
-      :        :                                   :  +- * ColumnarToRow (12)
-      :        :                                   :     +- Scan parquet default.store_sales (11)
-      :        :                                   +- ReusedExchange (14)
-      :        +- * Filter (29)
-      :           +- * ColumnarToRow (28)
-      :              +- Scan parquet default.store (27)
-      +- * Filter (35)
-         +- * ColumnarToRow (34)
-            +- Scan parquet default.item (33)
+      :        :        +- BroadcastExchange (24)
+      :        :           +- * Filter (23)
+      :        :              +- * HashAggregate (22)
+      :        :                 +- Exchange (21)
+      :        :                    +- * HashAggregate (20)
+      :        :                       +- * HashAggregate (19)
+      :        :                          +- Exchange (18)
+      :        :                             +- * HashAggregate (17)
+      :        :                                +- * Project (16)
+      :        :                                   +- * BroadcastHashJoin Inner BuildRight (15)
+      :        :                                      :- * Filter (13)
+      :        :                                      :  +- * ColumnarToRow (12)
+      :        :                                      :     +- Scan parquet default.store_sales (11)
+      :        :                                      +- ReusedExchange (14)
+      :        +- * Filter (30)
+      :           +- * ColumnarToRow (29)
+      :              +- Scan parquet default.store (28)
+      +- * Filter (36)
+         +- * ColumnarToRow (35)
+            +- Scan parquet default.item (34)
 
 
 (1) Scan parquet default.store_sales
@@ -54,7 +55,7 @@ Input [4]: [ss_item_sk#1, ss_store_sk#2, ss_sales_price#3, ss_sold_date_sk#4]
 Input [4]: [ss_item_sk#1, ss_store_sk#2, ss_sales_price#3, ss_sold_date_sk#4]
 Condition : (isnotnull(ss_store_sk#2) AND isnotnull(ss_item_sk#1))
 
-(4) ReusedExchange [Reuses operator id: 43]
+(4) ReusedExchange [Reuses operator id: 44]
 Output [1]: [d_date_sk#6]
 
 (5) BroadcastHashJoin [codegen id : 2]
@@ -103,7 +104,7 @@ Input [4]: [ss_item_sk#11, ss_store_sk#12, ss_sales_price#13, ss_sold_date_sk#14
 Input [4]: [ss_item_sk#11, ss_store_sk#12, ss_sales_price#13, ss_sold_date_sk#14]
 Condition : isnotnull(ss_store_sk#12)
 
-(14) ReusedExchange [Reuses operator id: 43]
+(14) ReusedExchange [Reuses operator id: 44]
 Output [1]: [d_date_sk#15]
 
 (15) BroadcastHashJoin [codegen id : 4]
@@ -151,106 +152,110 @@ Functions [1]: [avg(revenue#19)]
 Aggregate Attributes [1]: [avg(revenue#19)#24]
 Results [2]: [ss_store_sk#12, avg(revenue#19)#24 AS ave#25]
 
-(23) BroadcastExchange
+(23) Filter [codegen id : 6]
+Input [2]: [ss_store_sk#12, ave#25]
+Condition : isnotnull(ave#25)
+
+(24) BroadcastExchange
 Input [2]: [ss_store_sk#12, ave#25]
 Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4]
 
-(24) BroadcastHashJoin [codegen id : 7]
+(25) BroadcastHashJoin [codegen id : 7]
 Left keys [1]: [ss_store_sk#2]
 Right keys [1]: [ss_store_sk#12]
-Join condition: (cast(revenue#10 as decimal(23,7)) <= CheckOverflow((0.100000 * promote_precision(ave#25)), DecimalType(23,7)))
+Join condition: (cast(revenue#10 as decimal(23,7)) <= (0.1 * ave#25))
 
-(25) Project [codegen id : 7]
+(26) Project [codegen id : 7]
 Output [3]: [ss_store_sk#2, ss_item_sk#1, revenue#10]
 Input [5]: [ss_store_sk#2, ss_item_sk#1, revenue#10, ss_store_sk#12, ave#25]
 
-(26) BroadcastExchange
+(27) BroadcastExchange
 Input [3]: [ss_store_sk#2, ss_item_sk#1, revenue#10]
 Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=5]
 
-(27) Scan parquet default.store
+(28) Scan parquet default.store
 Output [2]: [s_store_sk#26, s_store_name#27]
 Batched: true
 Location [not included in comparison]/{warehouse_dir}/store]
 PushedFilters: [IsNotNull(s_store_sk)]
 ReadSchema: struct<s_store_sk:int,s_store_name:string>
 
-(28) ColumnarToRow
+(29) ColumnarToRow
 Input [2]: [s_store_sk#26, s_store_name#27]
 
-(29) Filter
+(30) Filter
 Input [2]: [s_store_sk#26, s_store_name#27]
 Condition : isnotnull(s_store_sk#26)
 
-(30) BroadcastHashJoin [codegen id : 8]
+(31) BroadcastHashJoin [codegen id : 8]
 Left keys [1]: [ss_store_sk#2]
 Right keys [1]: [s_store_sk#26]
 Join condition: None
 
-(31) Project [codegen id : 8]
+(32) Project [codegen id : 8]
 Output [3]: [ss_item_sk#1, revenue#10, s_store_name#27]
 Input [5]: [ss_store_sk#2, ss_item_sk#1, revenue#10, s_store_sk#26, s_store_name#27]
 
-(32) BroadcastExchange
+(33) BroadcastExchange
 Input [3]: [ss_item_sk#1, revenue#10, s_store_name#27]
 Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6]
 
-(33) Scan parquet default.item
+(34) Scan parquet default.item
 Output [5]: [i_item_sk#28, i_item_desc#29, i_current_price#30, i_wholesale_cost#31, i_brand#32]
 Batched: true
 Location [not included in comparison]/{warehouse_dir}/item]
 PushedFilters: [IsNotNull(i_item_sk)]
 ReadSchema: struct<i_item_sk:int,i_item_desc:string,i_current_price:decimal(7,2),i_wholesale_cost:decimal(7,2),i_brand:string>
 
-(34) ColumnarToRow
+(35) ColumnarToRow
 Input [5]: [i_item_sk#28, i_item_desc#29, i_current_price#30, i_wholesale_cost#31, i_brand#32]
 
-(35) Filter
+(36) Filter
 Input [5]: [i_item_sk#28, i_item_desc#29, i_current_price#30, i_wholesale_cost#31, i_brand#32]
 Condition : isnotnull(i_item_sk#28)
 
-(36) BroadcastHashJoin [codegen id : 9]
+(37) BroadcastHashJoin [codegen id : 9]
 Left keys [1]: [ss_item_sk#1]
 Right keys [1]: [i_item_sk#28]
 Join condition: None
 
-(37) Project [codegen id : 9]
+(38) Project [codegen id : 9]
 Output [6]: [s_store_name#27, i_item_desc#29, revenue#10, i_current_price#30, i_wholesale_cost#31, i_brand#32]
 Input [8]: [ss_item_sk#1, revenue#10, s_store_name#27, i_item_sk#28, i_item_desc#29, i_current_price#30, i_wholesale_cost#31, i_brand#32]
 
-(38) TakeOrderedAndProject
+(39) TakeOrderedAndProject
 Input [6]: [s_store_name#27, i_item_desc#29, revenue#10, i_current_price#30, i_wholesale_cost#31, i_brand#32]
 Arguments: 100, [s_store_name#27 ASC NULLS FIRST, i_item_desc#29 ASC NULLS FIRST], [s_store_name#27, i_item_desc#29, revenue#10, i_current_price#30, i_wholesale_cost#31, i_brand#32]
 
 ===== Subqueries =====
 
 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5
-BroadcastExchange (43)
-+- * Project (42)
-   +- * Filter (41)
-      +- * ColumnarToRow (40)
-         +- Scan parquet default.date_dim (39)
+BroadcastExchange (44)
++- * Project (43)
+   +- * Filter (42)
+      +- * ColumnarToRow (41)
+         +- Scan parquet default.date_dim (40)
 
 
-(39) Scan parquet default.date_dim
+(40) Scan parquet default.date_dim
 Output [2]: [d_date_sk#6, d_month_seq#33]
 Batched: true
 Location [not included in comparison]/{warehouse_dir}/date_dim]
 PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), GreaterThanOrEqual(d_date_sk,2451911), LessThanOrEqual(d_date_sk,2452275), IsNotNull(d_date_sk)]
 ReadSchema: struct<d_date_sk:int,d_month_seq:int>
 
-(40) ColumnarToRow [codegen id : 1]
+(41) ColumnarToRow [codegen id : 1]
 Input [2]: [d_date_sk#6, d_month_seq#33]
 
-(41) Filter [codegen id : 1]
+(42) Filter [codegen id : 1]
 Input [2]: [d_date_sk#6, d_month_seq#33]
 Condition : (((((isnotnull(d_month_seq#33) AND (d_month_seq#33 >= 1212)) AND (d_month_seq#33 <= 1223)) AND (d_date_sk#6 >= 2451911)) AND (d_date_sk#6 <= 2452275)) AND isnotnull(d_date_sk#6))
 
-(42) Project [codegen id : 1]
+(43) Project [codegen id : 1]
 Output [1]: [d_date_sk#6]
 Input [2]: [d_date_sk#6, d_month_seq#33]
 
-(43) BroadcastExchange
+(44) BroadcastExchange
 Input [1]: [d_date_sk#6]
 Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7]
 
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65.sf100/simplified.txt
index fd2cf0d33f5..e4041a6480e 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65.sf100/simplified.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65.sf100/simplified.txt
@@ -37,25 +37,26 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes
                             InputAdapter
                               BroadcastExchange #5
                                 WholeStageCodegen (6)
-                                  HashAggregate [ss_store_sk,sum,count] [avg(revenue),ave,sum,count]
-                                    InputAdapter
-                                      Exchange [ss_store_sk] #6
-                                        WholeStageCodegen (5)
-                                          HashAggregate [ss_store_sk,revenue] [sum,count,sum,count]
-                                            HashAggregate [ss_store_sk,ss_item_sk,sum] [sum(UnscaledValue(ss_sales_price)),revenue,sum]
-                                              InputAdapter
-                                                Exchange [ss_store_sk,ss_item_sk] #7
-                                                  WholeStageCodegen (4)
-                                                    HashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] [sum,sum]
-                                                      Project [ss_item_sk,ss_store_sk,ss_sales_price]
-                                                        BroadcastHashJoin [ss_sold_date_sk,d_date_sk]
-                                                          Filter [ss_store_sk]
-                                                            ColumnarToRow
-                                                              InputAdapter
-                                                                Scan parquet default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk]
-                                                                  ReusedSubquery [d_date_sk] #1
-                                                          InputAdapter
-                                                            ReusedExchange [d_date_sk] #4
+                                  Filter [ave]
+                                    HashAggregate [ss_store_sk,sum,count] [avg(revenue),ave,sum,count]
+                                      InputAdapter
+                                        Exchange [ss_store_sk] #6
+                                          WholeStageCodegen (5)
+                                            HashAggregate [ss_store_sk,revenue] [sum,count,sum,count]
+                                              HashAggregate [ss_store_sk,ss_item_sk,sum] [sum(UnscaledValue(ss_sales_price)),revenue,sum]
+                                                InputAdapter
+                                                  Exchange [ss_store_sk,ss_item_sk] #7
+                                                    WholeStageCodegen (4)
+                                                      HashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] [sum,sum]
+                                                        Project [ss_item_sk,ss_store_sk,ss_sales_price]
+                                                          BroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+                                                            Filter [ss_store_sk]
+                                                              ColumnarToRow
+                                                                InputAdapter
+                                                                  Scan parquet default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk]
+                                                                    ReusedSubquery [d_date_sk] #1
+                                                            InputAdapter
+                                                              ReusedExchange [d_date_sk] #4
                   Filter [s_store_sk]
                     ColumnarToRow
                       InputAdapter
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65/explain.txt
index f6dfdf7342f..f132dcc38df 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65/explain.txt
@@ -1,7 +1,7 @@
 == Physical Plan ==
-TakeOrderedAndProject (38)
-+- * Project (37)
-   +- * BroadcastHashJoin Inner BuildRight (36)
+TakeOrderedAndProject (39)
++- * Project (38)
+   +- * BroadcastHashJoin Inner BuildRight (37)
       :- * Project (22)
       :  +- * BroadcastHashJoin Inner BuildRight (21)
       :     :- * Project (16)
@@ -24,19 +24,20 @@ TakeOrderedAndProject (38)
       :        +- * Filter (19)
       :           +- * ColumnarToRow (18)
       :              +- Scan parquet default.item (17)
-      +- BroadcastExchange (35)
-         +- * HashAggregate (34)
-            +- Exchange (33)
-               +- * HashAggregate (32)
-                  +- * HashAggregate (31)
-                     +- Exchange (30)
-                        +- * HashAggregate (29)
-                           +- * Project (28)
-                              +- * BroadcastHashJoin Inner BuildRight (27)
-                                 :- * Filter (25)
-                                 :  +- * ColumnarToRow (24)
-                                 :     +- Scan parquet default.store_sales (23)
-                                 +- ReusedExchange (26)
+      +- BroadcastExchange (36)
+         +- * Filter (35)
+            +- * HashAggregate (34)
+               +- Exchange (33)
+                  +- * HashAggregate (32)
+                     +- * HashAggregate (31)
+                        +- Exchange (30)
+                           +- * HashAggregate (29)
+                              +- * Project (28)
+                                 +- * BroadcastHashJoin Inner BuildRight (27)
+                                    :- * Filter (25)
+                                    :  +- * ColumnarToRow (24)
+                                    :     +- Scan parquet default.store_sales (23)
+                                    +- ReusedExchange (26)
 
 
 (1) Scan parquet default.store
@@ -68,7 +69,7 @@ Input [4]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6]
 Input [4]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6]
 Condition : (isnotnull(ss_store_sk#4) AND isnotnull(ss_item_sk#3))
 
-(7) ReusedExchange [Reuses operator id: 43]
+(7) ReusedExchange [Reuses operator id: 44]
 Output [1]: [d_date_sk#8]
 
 (8) BroadcastHashJoin [codegen id : 2]
@@ -157,7 +158,7 @@ Input [4]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20, ss_sold_date_sk#21
 Input [4]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20, ss_sold_date_sk#21]
 Condition : isnotnull(ss_store_sk#19)
 
-(26) ReusedExchange [Reuses operator id: 43]
+(26) ReusedExchange [Reuses operator id: 44]
 Output [1]: [d_date_sk#22]
 
 (27) BroadcastHashJoin [codegen id : 6]
@@ -205,52 +206,56 @@ Functions [1]: [avg(revenue#26)]
 Aggregate Attributes [1]: [avg(revenue#26)#31]
 Results [2]: [ss_store_sk#19, avg(revenue#26)#31 AS ave#32]
 
-(35) BroadcastExchange
+(35) Filter [codegen id : 8]
+Input [2]: [ss_store_sk#19, ave#32]
+Condition : isnotnull(ave#32)
+
+(36) BroadcastExchange
 Input [2]: [ss_store_sk#19, ave#32]
 Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6]
 
-(36) BroadcastHashJoin [codegen id : 9]
+(37) BroadcastHashJoin [codegen id : 9]
 Left keys [1]: [ss_store_sk#4]
 Right keys [1]: [ss_store_sk#19]
-Join condition: (cast(revenue#12 as decimal(23,7)) <= CheckOverflow((0.100000 * promote_precision(ave#32)), DecimalType(23,7)))
+Join condition: (cast(revenue#12 as decimal(23,7)) <= (0.1 * ave#32))
 
-(37) Project [codegen id : 9]
+(38) Project [codegen id : 9]
 Output [6]: [s_store_name#2, i_item_desc#14, revenue#12, i_current_price#15, i_wholesale_cost#16, i_brand#17]
 Input [9]: [s_store_name#2, ss_store_sk#4, revenue#12, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#17, ss_store_sk#19, ave#32]
 
-(38) TakeOrderedAndProject
+(39) TakeOrderedAndProject
 Input [6]: [s_store_name#2, i_item_desc#14, revenue#12, i_current_price#15, i_wholesale_cost#16, i_brand#17]
 Arguments: 100, [s_store_name#2 ASC NULLS FIRST, i_item_desc#14 ASC NULLS FIRST], [s_store_name#2, i_item_desc#14, revenue#12, i_current_price#15, i_wholesale_cost#16, i_brand#17]
 
 ===== Subqueries =====
 
 Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7
-BroadcastExchange (43)
-+- * Project (42)
-   +- * Filter (41)
-      +- * ColumnarToRow (40)
-         +- Scan parquet default.date_dim (39)
+BroadcastExchange (44)
++- * Project (43)
+   +- * Filter (42)
+      +- * ColumnarToRow (41)
+         +- Scan parquet default.date_dim (40)
 
 
-(39) Scan parquet default.date_dim
+(40) Scan parquet default.date_dim
 Output [2]: [d_date_sk#8, d_month_seq#33]
 Batched: true
 Location [not included in comparison]/{warehouse_dir}/date_dim]
 PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), GreaterThanOrEqual(d_date_sk,2451911), LessThanOrEqual(d_date_sk,2452275), IsNotNull(d_date_sk)]
 ReadSchema: struct<d_date_sk:int,d_month_seq:int>
 
-(40) ColumnarToRow [codegen id : 1]
+(41) ColumnarToRow [codegen id : 1]
 Input [2]: [d_date_sk#8, d_month_seq#33]
 
-(41) Filter [codegen id : 1]
+(42) Filter [codegen id : 1]
 Input [2]: [d_date_sk#8, d_month_seq#33]
 Condition : (((((isnotnull(d_month_seq#33) AND (d_month_seq#33 >= 1212)) AND (d_month_seq#33 <= 1223)) AND (d_date_sk#8 >= 2451911)) AND (d_date_sk#8 <= 2452275)) AND isnotnull(d_date_sk#8))
 
-(42) Project [codegen id : 1]
+(43) Project [codegen id : 1]
 Output [1]: [d_date_sk#8]
 Input [2]: [d_date_sk#8, d_month_seq#33]
 
-(43) BroadcastExchange
+(44) BroadcastExchange
 Input [1]: [d_date_sk#8]
 Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7]
 
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65/simplified.txt
index 4b519f37a58..5e9a714f60a 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65/simplified.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q65/simplified.txt
@@ -45,22 +45,23 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes
         InputAdapter
           BroadcastExchange #5
             WholeStageCodegen (8)
-              HashAggregate [ss_store_sk,sum,count] [avg(revenue),ave,sum,count]
-                InputAdapter
-                  Exchange [ss_store_sk] #6
-                    WholeStageCodegen (7)
-                      HashAggregate [ss_store_sk,revenue] [sum,count,sum,count]
-                        HashAggregate [ss_store_sk,ss_item_sk,sum] [sum(UnscaledValue(ss_sales_price)),revenue,sum]
-                          InputAdapter
-                            Exchange [ss_store_sk,ss_item_sk] #7
-                              WholeStageCodegen (6)
-                                HashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] [sum,sum]
-                                  Project [ss_item_sk,ss_store_sk,ss_sales_price]
-                                    BroadcastHashJoin [ss_sold_date_sk,d_date_sk]
-                                      Filter [ss_store_sk]
-                                        ColumnarToRow
-                                          InputAdapter
-                                            Scan parquet default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk]
-                                              ReusedSubquery [d_date_sk] #1
-                                      InputAdapter
-                                        ReusedExchange [d_date_sk] #3
+              Filter [ave]
+                HashAggregate [ss_store_sk,sum,count] [avg(revenue),ave,sum,count]
+                  InputAdapter
+                    Exchange [ss_store_sk] #6
+                      WholeStageCodegen (7)
+                        HashAggregate [ss_store_sk,revenue] [sum,count,sum,count]
+                          HashAggregate [ss_store_sk,ss_item_sk,sum] [sum(UnscaledValue(ss_sales_price)),revenue,sum]
+                            InputAdapter
+                              Exchange [ss_store_sk,ss_item_sk] #7
+                                WholeStageCodegen (6)
+                                  HashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] [sum,sum]
+                                    Project [ss_item_sk,ss_store_sk,ss_sales_price]
+                                      BroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+                                        Filter [ss_store_sk]
+                                          ColumnarToRow
+                                            InputAdapter
+                                              Scan parquet default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk]
+                                                ReusedSubquery [d_date_sk] #1
+                                        InputAdapter
+                                          ReusedExchange [d_date_sk] #3
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89.sf100/explain.txt
index a2cfd6b6680..c217c513e41 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89.sf100/explain.txt
@@ -141,7 +141,7 @@ Arguments: [avg(_w0#19) windowspecdefinition(i_category#14, i_brand#12, s_store_
 
 (25) Filter [codegen id : 7]
 Input [9]: [i_category#14, i_class#13, i_brand#12, s_store_name#9, s_company_name#10, d_moy#7, sum_sales#18, _w0#19, avg_monthly_sales#20]
-Condition : (isnotnull(avg_monthly_sales#20) AND (NOT (avg_monthly_sales#20 = 0.000000) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#18 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#20 as decimal(22,6)))), DecimalType(22,6)))) / promote_precision(cast(avg_monthly_sales#20 as decimal(22,6)))), DecimalType(38,16)) > 0.1000000000000000)))
+Condition : ((isnotnull(avg_monthly_sales#20) AND isnotnull(sum_sales#18)) AND (NOT (avg_monthly_sales#20 = 0.000000) AND ((abs((sum_sales#18 - avg_monthly_sales#20)) / avg_monthly_sales#20) > 0.1000000000000000)))
 
 (26) Project [codegen id : 7]
 Output [8]: [i_category#14, i_class#13, i_brand#12, s_store_name#9, s_company_name#10, d_moy#7, sum_sales#18, avg_monthly_sales#20]
@@ -149,7 +149,7 @@ Input [9]: [i_category#14, i_class#13, i_brand#12, s_store_name#9, s_company_nam
 
 (27) TakeOrderedAndProject
 Input [8]: [i_category#14, i_class#13, i_brand#12, s_store_name#9, s_company_name#10, d_moy#7, sum_sales#18, avg_monthly_sales#20]
-Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#18 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#20 as decimal(22,6)))), DecimalType(22,6)) ASC NULLS FIRST, s_store_name#9 ASC NULLS FIRST], [i_category#14, i_class#13, i_brand#12, s_store_name#9, s_company_name#10, d_moy#7, sum_sales#18, avg_monthly_sales#20]
+Arguments: 100, [(sum_sales#18 - avg_monthly_sales#20) ASC NULLS FIRST, s_store_name#9 ASC NULLS FIRST], [i_category#14, i_class#13, i_brand#12, s_store_name#9, s_company_name#10, d_moy#7, sum_sales#18, avg_monthly_sales#20]
 
 ===== Subqueries =====
 
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89/explain.txt
index 4ee1a5b7c29..4257a700d53 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q89/explain.txt
@@ -141,7 +141,7 @@ Arguments: [avg(_w0#19) windowspecdefinition(i_category#4, i_brand#2, s_store_na
 
 (25) Filter [codegen id : 7]
 Input [9]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, _w0#19, avg_monthly_sales#20]
-Condition : (isnotnull(avg_monthly_sales#20) AND (NOT (avg_monthly_sales#20 = 0.000000) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#18 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#20 as decimal(22,6)))), DecimalType(22,6)))) / promote_precision(cast(avg_monthly_sales#20 as decimal(22,6)))), DecimalType(38,16)) > 0.1000000000000000)))
+Condition : ((isnotnull(avg_monthly_sales#20) AND isnotnull(sum_sales#18)) AND (NOT (avg_monthly_sales#20 = 0.000000) AND ((abs((sum_sales#18 - avg_monthly_sales#20)) / avg_monthly_sales#20) > 0.1000000000000000)))
 
 (26) Project [codegen id : 7]
 Output [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, avg_monthly_sales#20]
@@ -149,7 +149,7 @@ Input [9]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#
 
 (27) TakeOrderedAndProject
 Input [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, avg_monthly_sales#20]
-Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#18 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#20 as decimal(22,6)))), DecimalType(22,6)) ASC NULLS FIRST, s_store_name#13 ASC NULLS FIRST], [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, avg_monthly_sales#20]
+Arguments: 100, [(sum_sales#18 - avg_monthly_sales#20) ASC NULLS FIRST, s_store_name#13 ASC NULLS FIRST], [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, avg_monthly_sales#20]
 
 ===== Subqueries =====
 
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q98.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q98.sf100/explain.txt
index a0080476fc0..41270b0cb65 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q98.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q98.sf100/explain.txt
@@ -123,7 +123,7 @@ Input [8]: [i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrev
 Arguments: [sum(_w1#17) windowspecdefinition(i_class#10, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#18], [i_class#10]
 
 (22) Project [codegen id : 9]
-Output [7]: [i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrevenue#15, CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(_w0#16) * 100.00), DecimalType(21,2)) as decimal(27,2))) / promote_precision(_we0#18)), DecimalType(38,17)) AS revenueratio#19, i_item_id#7]
+Output [7]: [i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrevenue#15, ((_w0#16 * 100) / _we0#18) AS revenueratio#19, i_item_id#7]
 Input [9]: [i_item_desc#8, i_category#11, i_class#10, i_current_price#9, itemrevenue#15, _w0#16, _w1#17, i_item_id#7, _we0#18]
 
 (23) Exchange
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q98/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q98/explain.txt
index a87e71a75e1..2bd1d813c19 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q98/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-modified/q98/explain.txt
@@ -108,7 +108,7 @@ Input [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemreve
 Arguments: [sum(_w1#17) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#18], [i_class#9]
 
 (19) Project [codegen id : 6]
-Output [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(_w0#16) * 100.00), DecimalType(21,2)) as decimal(27,2))) / promote_precision(_we0#18)), DecimalType(38,17)) AS revenueratio#19, i_item_id#6]
+Output [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, ((_w0#16 * 100) / _we0#18) AS revenueratio#19, i_item_id#6]
 Input [9]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, _w1#17, i_item_id#6, _we0#18]
 
 (20) Exchange
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.sf100/explain.txt
index abc24d2519c..c5cfaf40161 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1.sf100/explain.txt
@@ -154,7 +154,7 @@ Input [3]: [ctr_store_sk#11, sum#17, count#18]
 Keys [1]: [ctr_store_sk#11]
 Functions [1]: [avg(ctr_total_return#12)]
 Aggregate Attributes [1]: [avg(ctr_total_return#12)#19]
-Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#12)#19) * 1.200000), DecimalType(24,7)) AS (avg(ctr_total_return) * 1.2)#20, ctr_store_sk#11 AS ctr_store_sk#11#21]
+Results [2]: [(avg(ctr_total_return#12)#19 * 1.2) AS (avg(ctr_total_return) * 1.2)#20, ctr_store_sk#11 AS ctr_store_sk#11#21]
 
 (23) Filter [codegen id : 6]
 Input [2]: [(avg(ctr_total_return) * 1.2)#20, ctr_store_sk#11#21]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/explain.txt
index 4d620e81d99..d92e15acdb8 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q1/explain.txt
@@ -151,7 +151,7 @@ Input [3]: [ctr_store_sk#11, sum#17, count#18]
 Keys [1]: [ctr_store_sk#11]
 Functions [1]: [avg(ctr_total_return#12)]
 Aggregate Attributes [1]: [avg(ctr_total_return#12)#19]
-Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#12)#19) * 1.200000), DecimalType(24,7)) AS (avg(ctr_total_return) * 1.2)#20, ctr_store_sk#11 AS ctr_store_sk#11#21]
+Results [2]: [(avg(ctr_total_return#12)#19 * 1.2) AS (avg(ctr_total_return) * 1.2)#20, ctr_store_sk#11 AS ctr_store_sk#11#21]
 
 (23) Filter [codegen id : 6]
 Input [2]: [(avg(ctr_total_return) * 1.2)#20, ctr_store_sk#11#21]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.sf100/explain.txt
index 796c9fc63a0..a1c8e74c0fd 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.sf100/explain.txt
@@ -150,7 +150,7 @@ Input [12]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, d_yea
 (16) HashAggregate [codegen id : 6]
 Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#13, c_login#14, c_email_address#15, ss_ext_discount_amt#2, ss_ext_list_price#3, d_year#7]
 Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#7, c_preferred_cust_flag#12, c_birth_country#13, c_login#14, c_email_address#15]
-Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#3 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#2 as decimal(8,2)))), DecimalType(8,2))))]
+Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#3 - ss_ext_discount_amt#2)))]
 Aggregate Attributes [1]: [sum#16]
 Results [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#7, c_preferred_cust_flag#12, c_birth_country#13, c_login#14, c_email_address#15, sum#17]
 
@@ -161,9 +161,9 @@ Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, d_
 (18) HashAggregate [codegen id : 7]
 Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#7, c_preferred_cust_flag#12, c_birth_country#13, c_login#14, c_email_address#15, sum#17]
 Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#7, c_preferred_cust_flag#12, c_birth_country#13, c_login#14, c_email_address#15]
-Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#3 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#2 as decimal(8,2)))), DecimalType(8,2))))]
-Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#3 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#2 as decimal(8,2)))), DecimalType(8,2))))#18]
-Results [2]: [c_customer_id#9 AS customer_id#19, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#3 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#2 as decimal(8,2)))), DecimalType(8,2))))#18,18,2) AS year_total#20]
+Functions [1]: [sum(UnscaledValue((ss_ext_list_price#3 - ss_ext_discount_amt#2)))]
+Aggregate Attributes [1]: [sum(UnscaledValue((ss_ext_list_price#3 - ss_ext_discount_amt#2)))#18]
+Results [2]: [c_customer_id#9 AS customer_id#19, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#3 - ss_ext_discount_amt#2)))#18,18,2) AS year_total#20]
 
 (19) Filter [codegen id : 7]
 Input [2]: [customer_id#19, year_total#20]
@@ -231,7 +231,7 @@ Input [12]: [ss_customer_sk#21, ss_ext_discount_amt#22, ss_ext_list_price#23, d_
 (34) HashAggregate [codegen id : 14]
 Input [10]: [c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35, ss_ext_discount_amt#22, ss_ext_list_price#23, d_year#27]
 Keys [8]: [c_customer_id#29, c_first_name#30, c_last_name#31, d_year#27, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35]
-Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#23 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#22 as decimal(8,2)))), DecimalType(8,2))))]
+Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#23 - ss_ext_discount_amt#22)))]
 Aggregate Attributes [1]: [sum#36]
 Results [9]: [c_customer_id#29, c_first_name#30, c_last_name#31, d_year#27, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35, sum#37]
 
@@ -242,9 +242,9 @@ Arguments: hashpartitioning(c_customer_id#29, c_first_name#30, c_last_name#31, d
 (36) HashAggregate [codegen id : 15]
 Input [9]: [c_customer_id#29, c_first_name#30, c_last_name#31, d_year#27, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35, sum#37]
 Keys [8]: [c_customer_id#29, c_first_name#30, c_last_name#31, d_year#27, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35]
-Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#23 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#22 as decimal(8,2)))), DecimalType(8,2))))]
-Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#23 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#22 as decimal(8,2)))), DecimalType(8,2))))#18]
-Results [3]: [c_customer_id#29 AS customer_id#38, c_preferred_cust_flag#32 AS customer_preferred_cust_flag#39, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#23 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#22 as decimal(8,2)))), DecimalType(8,2))))#18,18,2) AS year_total#40]
+Functions [1]: [sum(UnscaledValue((ss_ext_list_price#23 - ss_ext_discount_amt#22)))]
+Aggregate Attributes [1]: [sum(UnscaledValue((ss_ext_list_price#23 - ss_ext_discount_amt#22)))#18]
+Results [3]: [c_customer_id#29 AS customer_id#38, c_preferred_cust_flag#32 AS customer_preferred_cust_flag#39, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#23 - ss_ext_discount_amt#22)))#18,18,2) AS year_total#40]
 
 (37) Exchange
 Input [3]: [customer_id#38, customer_preferred_cust_flag#39, year_total#40]
@@ -317,7 +317,7 @@ Input [12]: [ws_bill_customer_sk#41, ws_ext_discount_amt#42, ws_ext_list_price#4
 (53) HashAggregate [codegen id : 23]
 Input [10]: [c_customer_id#48, c_first_name#49, c_last_name#50, c_preferred_cust_flag#51, c_birth_country#52, c_login#53, c_email_address#54, ws_ext_discount_amt#42, ws_ext_list_price#43, d_year#46]
 Keys [8]: [c_customer_id#48, c_first_name#49, c_last_name#50, c_preferred_cust_flag#51, c_birth_country#52, c_login#53, c_email_address#54, d_year#46]
-Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#43 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#42 as decimal(8,2)))), DecimalType(8,2))))]
+Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#43 - ws_ext_discount_amt#42)))]
 Aggregate Attributes [1]: [sum#55]
 Results [9]: [c_customer_id#48, c_first_name#49, c_last_name#50, c_preferred_cust_flag#51, c_birth_country#52, c_login#53, c_email_address#54, d_year#46, sum#56]
 
@@ -328,9 +328,9 @@ Arguments: hashpartitioning(c_customer_id#48, c_first_name#49, c_last_name#50, c
 (55) HashAggregate [codegen id : 24]
 Input [9]: [c_customer_id#48, c_first_name#49, c_last_name#50, c_preferred_cust_flag#51, c_birth_country#52, c_login#53, c_email_address#54, d_year#46, sum#56]
 Keys [8]: [c_customer_id#48, c_first_name#49, c_last_name#50, c_preferred_cust_flag#51, c_birth_country#52, c_login#53, c_email_address#54, d_year#46]
-Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#43 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#42 as decimal(8,2)))), DecimalType(8,2))))]
-Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#43 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#42 as decimal(8,2)))), DecimalType(8,2))))#57]
-Results [2]: [c_customer_id#48 AS customer_id#58, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#43 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#42 as decimal(8,2)))), DecimalType(8,2))))#57,18,2) AS year_total#59]
+Functions [1]: [sum(UnscaledValue((ws_ext_list_price#43 - ws_ext_discount_amt#42)))]
+Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#43 - ws_ext_discount_amt#42)))#57]
+Results [2]: [c_customer_id#48 AS customer_id#58, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#43 - ws_ext_discount_amt#42)))#57,18,2) AS year_total#59]
 
 (56) Filter [codegen id : 24]
 Input [2]: [customer_id#58, year_total#59]
@@ -407,7 +407,7 @@ Input [12]: [ws_bill_customer_sk#60, ws_ext_discount_amt#61, ws_ext_list_price#6
 (73) HashAggregate [codegen id : 32]
 Input [10]: [c_customer_id#67, c_first_name#68, c_last_name#69, c_preferred_cust_flag#70, c_birth_country#71, c_login#72, c_email_address#73, ws_ext_discount_amt#61, ws_ext_list_price#62, d_year#65]
 Keys [8]: [c_customer_id#67, c_first_name#68, c_last_name#69, c_preferred_cust_flag#70, c_birth_country#71, c_login#72, c_email_address#73, d_year#65]
-Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#62 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#61 as decimal(8,2)))), DecimalType(8,2))))]
+Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#62 - ws_ext_discount_amt#61)))]
 Aggregate Attributes [1]: [sum#74]
 Results [9]: [c_customer_id#67, c_first_name#68, c_last_name#69, c_preferred_cust_flag#70, c_birth_country#71, c_login#72, c_email_address#73, d_year#65, sum#75]
 
@@ -418,9 +418,9 @@ Arguments: hashpartitioning(c_customer_id#67, c_first_name#68, c_last_name#69, c
 (75) HashAggregate [codegen id : 33]
 Input [9]: [c_customer_id#67, c_first_name#68, c_last_name#69, c_preferred_cust_flag#70, c_birth_country#71, c_login#72, c_email_address#73, d_year#65, sum#75]
 Keys [8]: [c_customer_id#67, c_first_name#68, c_last_name#69, c_preferred_cust_flag#70, c_birth_country#71, c_login#72, c_email_address#73, d_year#65]
-Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#62 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#61 as decimal(8,2)))), DecimalType(8,2))))]
-Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#62 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#61 as decimal(8,2)))), DecimalType(8,2))))#57]
-Results [2]: [c_customer_id#67 AS customer_id#76, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#62 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#61 as decimal(8,2)))), DecimalType(8,2))))#57,18,2) AS year_total#77]
+Functions [1]: [sum(UnscaledValue((ws_ext_list_price#62 - ws_ext_discount_amt#61)))]
+Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#62 - ws_ext_discount_amt#61)))#57]
+Results [2]: [c_customer_id#67 AS customer_id#76, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#62 - ws_ext_discount_amt#61)))#57,18,2) AS year_total#77]
 
 (76) Exchange
 Input [2]: [customer_id#76, year_total#77]
@@ -433,7 +433,7 @@ Arguments: [customer_id#76 ASC NULLS FIRST], false, 0
 (78) SortMergeJoin [codegen id : 35]
 Left keys [1]: [customer_id#19]
 Right keys [1]: [customer_id#76]
-Join condition: (CASE WHEN (year_total#59 > 0.00) THEN CheckOverflow((promote_precision(year_total#77) / promote_precision(year_total#59)), DecimalType(38,20)) END > CASE WHEN (year_total#20 > 0.00) THEN CheckOverflow((promote_precision(year_total#40) / promote_precision(year_total#20)), DecimalType(38,20)) END)
+Join condition: (CASE WHEN (year_total#59 > 0.00) THEN (year_total#77 / year_total#59) END > CASE WHEN (year_total#20 > 0.00) THEN (year_total#40 / year_total#20) END)
 
 (79) Project [codegen id : 35]
 Output [1]: [customer_preferred_cust_flag#39]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.sf100/simplified.txt
index ff149df17d8..22bf9a445ba 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.sf100/simplified.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11.sf100/simplified.txt
@@ -17,7 +17,7 @@ TakeOrderedAndProject [customer_preferred_cust_flag]
                                 Exchange [customer_id] #1
                                   WholeStageCodegen (7)
                                     Filter [year_total]
-                                      HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt as decimal(8,2)))), DecimalType(8,2)))),customer_id,year_total,sum]
+                                      HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt))),customer_id,year_total,sum]
                                         InputAdapter
                                           Exchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #2
                                             WholeStageCodegen (6)
@@ -61,7 +61,7 @@ TakeOrderedAndProject [customer_preferred_cust_flag]
                               InputAdapter
                                 Exchange [customer_id] #6
                                   WholeStageCodegen (15)
-                                    HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt as decimal(8,2)))), DecimalType(8,2)))),customer_id,customer_preferred_cust_flag,year_total,sum]
+                                    HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt))),customer_id,customer_preferred_cust_flag,year_total,sum]
                                       InputAdapter
                                         Exchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #7
                                           WholeStageCodegen (14)
@@ -101,7 +101,7 @@ TakeOrderedAndProject [customer_preferred_cust_flag]
                         Exchange [customer_id] #10
                           WholeStageCodegen (24)
                             Filter [year_total]
-                              HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt as decimal(8,2)))), DecimalType(8,2)))),customer_id,year_total,sum]
+                              HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum]
                                 InputAdapter
                                   Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #11
                                     WholeStageCodegen (23)
@@ -134,7 +134,7 @@ TakeOrderedAndProject [customer_preferred_cust_flag]
               InputAdapter
                 Exchange [customer_id] #13
                   WholeStageCodegen (33)
-                    HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt as decimal(8,2)))), DecimalType(8,2)))),customer_id,year_total,sum]
+                    HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum]
                       InputAdapter
                         Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #14
                           WholeStageCodegen (32)
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/explain.txt
index 9fc721d0039..0cced43b878 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/explain.txt
@@ -130,7 +130,7 @@ Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_fl
 (13) HashAggregate [codegen id : 3]
 Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, d_year#15]
 Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8]
-Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#11 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#10 as decimal(8,2)))), DecimalType(8,2))))]
+Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#11 - ss_ext_discount_amt#10)))]
 Aggregate Attributes [1]: [sum#16]
 Results [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#17]
 
@@ -141,9 +141,9 @@ Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_ye
 (15) HashAggregate [codegen id : 16]
 Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#17]
 Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8]
-Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#11 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#10 as decimal(8,2)))), DecimalType(8,2))))]
-Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#11 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#10 as decimal(8,2)))), DecimalType(8,2))))#18]
-Results [2]: [c_customer_id#2 AS customer_id#19, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#11 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#10 as decimal(8,2)))), DecimalType(8,2))))#18,18,2) AS year_total#20]
+Functions [1]: [sum(UnscaledValue((ss_ext_list_price#11 - ss_ext_discount_amt#10)))]
+Aggregate Attributes [1]: [sum(UnscaledValue((ss_ext_list_price#11 - ss_ext_discount_amt#10)))#18]
+Results [2]: [c_customer_id#2 AS customer_id#19, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#11 - ss_ext_discount_amt#10)))#18,18,2) AS year_total#20]
 
 (16) Filter [codegen id : 16]
 Input [2]: [customer_id#19, year_total#20]
@@ -206,7 +206,7 @@ Input [12]: [c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust
 (29) HashAggregate [codegen id : 6]
 Input [10]: [c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, ss_ext_discount_amt#30, ss_ext_list_price#31, d_year#35]
 Keys [8]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28]
-Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#31 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#30 as decimal(8,2)))), DecimalType(8,2))))]
+Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#31 - ss_ext_discount_amt#30)))]
 Aggregate Attributes [1]: [sum#36]
 Results [9]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, sum#37]
 
@@ -217,9 +217,9 @@ Arguments: hashpartitioning(c_customer_id#22, c_first_name#23, c_last_name#24, d
 (31) HashAggregate [codegen id : 7]
 Input [9]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, sum#37]
 Keys [8]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28]
-Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#31 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#30 as decimal(8,2)))), DecimalType(8,2))))]
-Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#31 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#30 as decimal(8,2)))), DecimalType(8,2))))#18]
-Results [3]: [c_customer_id#22 AS customer_id#38, c_preferred_cust_flag#25 AS customer_preferred_cust_flag#39, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#31 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#30 as decimal(8,2)))), DecimalType(8,2))))#18,18,2) AS year_total#40]
+Functions [1]: [sum(UnscaledValue((ss_ext_list_price#31 - ss_ext_discount_amt#30)))]
+Aggregate Attributes [1]: [sum(UnscaledValue((ss_ext_list_price#31 - ss_ext_discount_amt#30)))#18]
+Results [3]: [c_customer_id#22 AS customer_id#38, c_preferred_cust_flag#25 AS customer_preferred_cust_flag#39, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#31 - ss_ext_discount_amt#30)))#18,18,2) AS year_total#40]
 
 (32) BroadcastExchange
 Input [3]: [customer_id#38, customer_preferred_cust_flag#39, year_total#40]
@@ -291,7 +291,7 @@ Input [12]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust
 (47) HashAggregate [codegen id : 10]
 Input [10]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, ws_ext_discount_amt#50, ws_ext_list_price#51, d_year#54]
 Keys [8]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#54]
-Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#51 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#50 as decimal(8,2)))), DecimalType(8,2))))]
+Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#51 - ws_ext_discount_amt#50)))]
 Aggregate Attributes [1]: [sum#55]
 Results [9]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#54, sum#56]
 
@@ -302,9 +302,9 @@ Arguments: hashpartitioning(c_customer_id#42, c_first_name#43, c_last_name#44, c
 (49) HashAggregate [codegen id : 11]
 Input [9]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#54, sum#56]
 Keys [8]: [c_customer_id#42, c_first_name#43, c_last_name#44, c_preferred_cust_flag#45, c_birth_country#46, c_login#47, c_email_address#48, d_year#54]
-Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#51 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#50 as decimal(8,2)))), DecimalType(8,2))))]
-Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#51 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#50 as decimal(8,2)))), DecimalType(8,2))))#57]
-Results [2]: [c_customer_id#42 AS customer_id#58, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#51 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#50 as decimal(8,2)))), DecimalType(8,2))))#57,18,2) AS year_total#59]
+Functions [1]: [sum(UnscaledValue((ws_ext_list_price#51 - ws_ext_discount_amt#50)))]
+Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#51 - ws_ext_discount_amt#50)))#57]
+Results [2]: [c_customer_id#42 AS customer_id#58, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#51 - ws_ext_discount_amt#50)))#57,18,2) AS year_total#59]
 
 (50) Filter [codegen id : 11]
 Input [2]: [customer_id#58, year_total#59]
@@ -380,7 +380,7 @@ Input [12]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust
 (66) HashAggregate [codegen id : 14]
 Input [10]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67, ws_ext_discount_amt#69, ws_ext_list_price#70, d_year#73]
 Keys [8]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67, d_year#73]
-Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#70 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#69 as decimal(8,2)))), DecimalType(8,2))))]
+Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#70 - ws_ext_discount_amt#69)))]
 Aggregate Attributes [1]: [sum#74]
 Results [9]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67, d_year#73, sum#75]
 
@@ -391,9 +391,9 @@ Arguments: hashpartitioning(c_customer_id#61, c_first_name#62, c_last_name#63, c
 (68) HashAggregate [codegen id : 15]
 Input [9]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67, d_year#73, sum#75]
 Keys [8]: [c_customer_id#61, c_first_name#62, c_last_name#63, c_preferred_cust_flag#64, c_birth_country#65, c_login#66, c_email_address#67, d_year#73]
-Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#70 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#69 as decimal(8,2)))), DecimalType(8,2))))]
-Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#70 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#69 as decimal(8,2)))), DecimalType(8,2))))#57]
-Results [2]: [c_customer_id#61 AS customer_id#76, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#70 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#69 as decimal(8,2)))), DecimalType(8,2))))#57,18,2) AS year_total#77]
+Functions [1]: [sum(UnscaledValue((ws_ext_list_price#70 - ws_ext_discount_amt#69)))]
+Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#70 - ws_ext_discount_amt#69)))#57]
+Results [2]: [c_customer_id#61 AS customer_id#76, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#70 - ws_ext_discount_amt#69)))#57,18,2) AS year_total#77]
 
 (69) BroadcastExchange
 Input [2]: [customer_id#76, year_total#77]
@@ -402,7 +402,7 @@ Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [pla
 (70) BroadcastHashJoin [codegen id : 16]
 Left keys [1]: [customer_id#19]
 Right keys [1]: [customer_id#76]
-Join condition: (CASE WHEN (year_total#59 > 0.00) THEN CheckOverflow((promote_precision(year_total#77) / promote_precision(year_total#59)), DecimalType(38,20)) END > CASE WHEN (year_total#20 > 0.00) THEN CheckOverflow((promote_precision(year_total#40) / promote_precision(year_total#20)), DecimalType(38,20)) END)
+Join condition: (CASE WHEN (year_total#59 > 0.00) THEN (year_total#77 / year_total#59) END > CASE WHEN (year_total#20 > 0.00) THEN (year_total#40 / year_total#20) END)
 
 (71) Project [codegen id : 16]
 Output [1]: [customer_preferred_cust_flag#39]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/simplified.txt
index 6e80ebc5a03..3415b60d923 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/simplified.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q11/simplified.txt
@@ -7,7 +7,7 @@ TakeOrderedAndProject [customer_preferred_cust_flag]
             Project [customer_id,year_total,customer_preferred_cust_flag,year_total]
               BroadcastHashJoin [customer_id,customer_id]
                 Filter [year_total]
-                  HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt as decimal(8,2)))), DecimalType(8,2)))),customer_id,year_total,sum]
+                  HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt))),customer_id,year_total,sum]
                     InputAdapter
                       Exchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #1
                         WholeStageCodegen (3)
@@ -39,7 +39,7 @@ TakeOrderedAndProject [customer_preferred_cust_flag]
                 InputAdapter
                   BroadcastExchange #4
                     WholeStageCodegen (7)
-                      HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt as decimal(8,2)))), DecimalType(8,2)))),customer_id,customer_preferred_cust_flag,year_total,sum]
+                      HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt))),customer_id,customer_preferred_cust_flag,year_total,sum]
                         InputAdapter
                           Exchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #5
                             WholeStageCodegen (6)
@@ -72,7 +72,7 @@ TakeOrderedAndProject [customer_preferred_cust_flag]
               BroadcastExchange #8
                 WholeStageCodegen (11)
                   Filter [year_total]
-                    HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt as decimal(8,2)))), DecimalType(8,2)))),customer_id,year_total,sum]
+                    HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum]
                       InputAdapter
                         Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #9
                           WholeStageCodegen (10)
@@ -98,7 +98,7 @@ TakeOrderedAndProject [customer_preferred_cust_flag]
         InputAdapter
           BroadcastExchange #11
             WholeStageCodegen (15)
-              HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt as decimal(8,2)))), DecimalType(8,2)))),customer_id,year_total,sum]
+              HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum]
                 InputAdapter
                   Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #12
                     WholeStageCodegen (14)
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.sf100/explain.txt
index 71275fad793..9c9304d48e1 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12.sf100/explain.txt
@@ -121,7 +121,7 @@ Input [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemreve
 Arguments: [sum(_w1#17) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#18], [i_class#9]
 
 (22) Project [codegen id : 9]
-Output [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(_w0#16) * 100.00), DecimalType(21,2)) as decimal(27,2))) / promote_precision(_we0#18)), DecimalType(38,17)) AS revenueratio#19, i_item_id#6]
+Output [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, ((_w0#16 * 100) / _we0#18) AS revenueratio#19, i_item_id#6]
 Input [9]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, _w1#17, i_item_id#6, _we0#18]
 
 (23) TakeOrderedAndProject
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/explain.txt
index a82f017df7c..fce0efc831d 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q12/explain.txt
@@ -106,7 +106,7 @@ Input [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemreve
 Arguments: [sum(_w1#17) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#18], [i_class#9]
 
 (19) Project [codegen id : 6]
-Output [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(_w0#16) * 100.00), DecimalType(21,2)) as decimal(27,2))) / promote_precision(_we0#18)), DecimalType(38,17)) AS revenueratio#19, i_item_id#6]
+Output [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, ((_w0#16 * 100) / _we0#18) AS revenueratio#19, i_item_id#6]
 Input [9]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, _w1#17, i_item_id#6, _we0#18]
 
 (20) TakeOrderedAndProject
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt
index f5b3161c7e7..6ae82107c05 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/explain.txt
@@ -456,7 +456,7 @@ Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#37, i_brand_
 (75) HashAggregate [codegen id : 43]
 Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#38, i_class_id#39, i_category_id#40]
 Keys [3]: [i_brand_id#38, i_class_id#39, i_category_id#40]
-Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(ss_quantity#2 as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2))), partial_count(1)]
+Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)]
 Aggregate Attributes [3]: [sum#41, isEmpty#42, count#43]
 Results [6]: [i_brand_id#38, i_class_id#39, i_category_id#40, sum#44, isEmpty#45, count#46]
 
@@ -467,9 +467,9 @@ Arguments: hashpartitioning(i_brand_id#38, i_class_id#39, i_category_id#40, 5),
 (77) HashAggregate [codegen id : 44]
 Input [6]: [i_brand_id#38, i_class_id#39, i_category_id#40, sum#44, isEmpty#45, count#46]
 Keys [3]: [i_brand_id#38, i_class_id#39, i_category_id#40]
-Functions [2]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#2 as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2))), count(1)]
-Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#2 as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2)))#47, count(1)#48]
-Results [5]: [i_brand_id#38, i_class_id#39, i_category_id#40, sum(CheckOverflow((promote_precision(cast(ss_quantity#2 as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2)))#47 AS sales#49, count(1)#48 AS number_sales#50]
+Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)]
+Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47, count(1)#48]
+Results [5]: [i_brand_id#38, i_class_id#39, i_category_id#40, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47 AS sales#49, count(1)#48 AS number_sales#50]
 
 (78) Filter [codegen id : 44]
 Input [5]: [i_brand_id#38, i_class_id#39, i_category_id#40, sales#49, number_sales#50]
@@ -541,7 +541,7 @@ Input [7]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, i_item_sk#59, i_bra
 (94) HashAggregate [codegen id : 87]
 Input [5]: [cs_quantity#55, cs_list_price#56, i_brand_id#60, i_class_id#61, i_category_id#62]
 Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62]
-Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cs_quantity#55 as decimal(12,2))) * promote_precision(cast(cs_list_price#56 as decimal(12,2)))), DecimalType(18,2))), partial_count(1)]
+Functions [2]: [partial_sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56)), partial_count(1)]
 Aggregate Attributes [3]: [sum#63, isEmpty#64, count#65]
 Results [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#66, isEmpty#67, count#68]
 
@@ -552,9 +552,9 @@ Arguments: hashpartitioning(i_brand_id#60, i_class_id#61, i_category_id#62, 5),
 (96) HashAggregate [codegen id : 88]
 Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#66, isEmpty#67, count#68]
 Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62]
-Functions [2]: [sum(CheckOverflow((promote_precision(cast(cs_quantity#55 as decimal(12,2))) * promote_precision(cast(cs_list_price#56 as decimal(12,2)))), DecimalType(18,2))), count(1)]
-Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cs_quantity#55 as decimal(12,2))) * promote_precision(cast(cs_list_price#56 as decimal(12,2)))), DecimalType(18,2)))#69, count(1)#70]
-Results [5]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum(CheckOverflow((promote_precision(cast(cs_quantity#55 as decimal(12,2))) * promote_precision(cast(cs_list_price#56 as decimal(12,2)))), DecimalType(18,2)))#69 AS sales#71, count(1)#70 AS number_sales#72]
+Functions [2]: [sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56)), count(1)]
+Aggregate Attributes [2]: [sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56))#69, count(1)#70]
+Results [5]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56))#69 AS sales#71, count(1)#70 AS number_sales#72]
 
 (97) Filter [codegen id : 88]
 Input [5]: [i_brand_id#60, i_class_id#61, i_category_id#62, sales#71, number_sales#72]
@@ -626,7 +626,7 @@ Input [7]: [ws_item_sk#74, ws_quantity#75, ws_list_price#76, i_item_sk#79, i_bra
 (113) HashAggregate [codegen id : 131]
 Input [5]: [ws_quantity#75, ws_list_price#76, i_brand_id#80, i_class_id#81, i_category_id#82]
 Keys [3]: [i_brand_id#80, i_class_id#81, i_category_id#82]
-Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(ws_quantity#75 as decimal(12,2))) * promote_precision(cast(ws_list_price#76 as decimal(12,2)))), DecimalType(18,2))), partial_count(1)]
+Functions [2]: [partial_sum((cast(ws_quantity#75 as decimal(10,0)) * ws_list_price#76)), partial_count(1)]
 Aggregate Attributes [3]: [sum#83, isEmpty#84, count#85]
 Results [6]: [i_brand_id#80, i_class_id#81, i_category_id#82, sum#86, isEmpty#87, count#88]
 
@@ -637,9 +637,9 @@ Arguments: hashpartitioning(i_brand_id#80, i_class_id#81, i_category_id#82, 5),
 (115) HashAggregate [codegen id : 132]
 Input [6]: [i_brand_id#80, i_class_id#81, i_category_id#82, sum#86, isEmpty#87, count#88]
 Keys [3]: [i_brand_id#80, i_class_id#81, i_category_id#82]
-Functions [2]: [sum(CheckOverflow((promote_precision(cast(ws_quantity#75 as decimal(12,2))) * promote_precision(cast(ws_list_price#76 as decimal(12,2)))), DecimalType(18,2))), count(1)]
-Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(ws_quantity#75 as decimal(12,2))) * promote_precision(cast(ws_list_price#76 as decimal(12,2)))), DecimalType(18,2)))#89, count(1)#90]
-Results [5]: [i_brand_id#80, i_class_id#81, i_category_id#82, sum(CheckOverflow((promote_precision(cast(ws_quantity#75 as decimal(12,2))) * promote_precision(cast(ws_list_price#76 as decimal(12,2)))), DecimalType(18,2)))#89 AS sales#91, count(1)#90 AS number_sales#92]
+Functions [2]: [sum((cast(ws_quantity#75 as decimal(10,0)) * ws_list_price#76)), count(1)]
+Aggregate Attributes [2]: [sum((cast(ws_quantity#75 as decimal(10,0)) * ws_list_price#76))#89, count(1)#90]
+Results [5]: [i_brand_id#80, i_class_id#81, i_category_id#82, sum((cast(ws_quantity#75 as decimal(10,0)) * ws_list_price#76))#89 AS sales#91, count(1)#90 AS number_sales#92]
 
 (116) Filter [codegen id : 132]
 Input [5]: [i_brand_id#80, i_class_id#81, i_category_id#82, sales#91, number_sales#92]
@@ -772,7 +772,7 @@ Input [4]: [ws_quantity#121, ws_list_price#122, ws_sold_date_sk#123, d_date_sk#1
 (140) HashAggregate [codegen id : 7]
 Input [2]: [quantity#113, list_price#114]
 Keys: []
-Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(quantity#113 as decimal(12,2))) * promote_precision(cast(list_price#114 as decimal(12,2)))), DecimalType(18,2)))]
+Functions [1]: [partial_avg((cast(quantity#113 as decimal(10,0)) * list_price#114))]
 Aggregate Attributes [2]: [sum#127, count#128]
 Results [2]: [sum#129, count#130]
 
@@ -783,9 +783,9 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=19]
 (142) HashAggregate [codegen id : 8]
 Input [2]: [sum#129, count#130]
 Keys: []
-Functions [1]: [avg(CheckOverflow((promote_precision(cast(quantity#113 as decimal(12,2))) * promote_precision(cast(list_price#114 as decimal(12,2)))), DecimalType(18,2)))]
-Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(quantity#113 as decimal(12,2))) * promote_precision(cast(list_price#114 as decimal(12,2)))), DecimalType(18,2)))#131]
-Results [1]: [avg(CheckOverflow((promote_precision(cast(quantity#113 as decimal(12,2))) * promote_precision(cast(list_price#114 as decimal(12,2)))), DecimalType(18,2)))#131 AS average_sales#132]
+Functions [1]: [avg((cast(quantity#113 as decimal(10,0)) * list_price#114))]
+Aggregate Attributes [1]: [avg((cast(quantity#113 as decimal(10,0)) * list_price#114))#131]
+Results [1]: [avg((cast(quantity#113 as decimal(10,0)) * list_price#114))#131 AS average_sales#132]
 
 Subquery:2 Hosting operator id = 124 Hosting Expression = ss_sold_date_sk#111 IN dynamicpruning#12
 
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/simplified.txt
index f445a370581..37b526fbb65 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/simplified.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a.sf100/simplified.txt
@@ -13,7 +13,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su
                         Filter [sales]
                           Subquery #3
                             WholeStageCodegen (8)
-                              HashAggregate [sum,count] [avg(CheckOverflow((promote_precision(cast(quantity as decimal(12,2))) * promote_precision(cast(list_price as decimal(12,2)))), DecimalType(18,2))),average_sales,sum,count]
+                              HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count]
                                 InputAdapter
                                   Exchange #17
                                     WholeStageCodegen (7)
@@ -47,7 +47,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su
                                                         ReusedSubquery [d_date_sk] #2
                                                   InputAdapter
                                                     ReusedExchange [d_date_sk] #9
-                          HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(ss_quantity as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2))),count(1),sales,number_sales,sum,isEmpty,count]
+                          HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),sales,number_sales,sum,isEmpty,count]
                             InputAdapter
                               Exchange [i_brand_id,i_class_id,i_category_id] #2
                                 WholeStageCodegen (43)
@@ -202,7 +202,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su
                       Project [sales,number_sales,i_brand_id,i_class_id,i_category_id]
                         Filter [sales]
                           ReusedSubquery [average_sales] #3
-                          HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cs_quantity as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2))),count(1),sales,number_sales,sum,isEmpty,count]
+                          HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1),sales,number_sales,sum,isEmpty,count]
                             InputAdapter
                               Exchange [i_brand_id,i_class_id,i_category_id] #18
                                 WholeStageCodegen (87)
@@ -236,7 +236,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su
                       Project [sales,number_sales,i_brand_id,i_class_id,i_category_id]
                         Filter [sales]
                           ReusedSubquery [average_sales] #3
-                          HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(ws_quantity as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2))),count(1),sales,number_sales,sum,isEmpty,count]
+                          HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1),sales,number_sales,sum,isEmpty,count]
                             InputAdapter
                               Exchange [i_brand_id,i_class_id,i_category_id] #20
                                 WholeStageCodegen (131)
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt
index fd17c5d762e..a1af8944f60 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/explain.txt
@@ -390,7 +390,7 @@ Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#37, i_
 (63) HashAggregate [codegen id : 25]
 Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#37, i_class_id#38, i_category_id#39]
 Keys [3]: [i_brand_id#37, i_class_id#38, i_category_id#39]
-Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(ss_quantity#2 as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2))), partial_count(1)]
+Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)]
 Aggregate Attributes [3]: [sum#41, isEmpty#42, count#43]
 Results [6]: [i_brand_id#37, i_class_id#38, i_category_id#39, sum#44, isEmpty#45, count#46]
 
@@ -401,9 +401,9 @@ Arguments: hashpartitioning(i_brand_id#37, i_class_id#38, i_category_id#39, 5),
 (65) HashAggregate [codegen id : 26]
 Input [6]: [i_brand_id#37, i_class_id#38, i_category_id#39, sum#44, isEmpty#45, count#46]
 Keys [3]: [i_brand_id#37, i_class_id#38, i_category_id#39]
-Functions [2]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#2 as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2))), count(1)]
-Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#2 as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2)))#47, count(1)#48]
-Results [5]: [i_brand_id#37, i_class_id#38, i_category_id#39, sum(CheckOverflow((promote_precision(cast(ss_quantity#2 as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2)))#47 AS sales#49, count(1)#48 AS number_sales#50]
+Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)]
+Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47, count(1)#48]
+Results [5]: [i_brand_id#37, i_class_id#38, i_category_id#39, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47 AS sales#49, count(1)#48 AS number_sales#50]
 
 (66) Filter [codegen id : 26]
 Input [5]: [i_brand_id#37, i_class_id#38, i_category_id#39, sales#49, number_sales#50]
@@ -463,7 +463,7 @@ Input [7]: [cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_brand_id#59,
 (79) HashAggregate [codegen id : 51]
 Input [5]: [cs_quantity#55, cs_list_price#56, i_brand_id#59, i_class_id#60, i_category_id#61]
 Keys [3]: [i_brand_id#59, i_class_id#60, i_category_id#61]
-Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cs_quantity#55 as decimal(12,2))) * promote_precision(cast(cs_list_price#56 as decimal(12,2)))), DecimalType(18,2))), partial_count(1)]
+Functions [2]: [partial_sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56)), partial_count(1)]
 Aggregate Attributes [3]: [sum#63, isEmpty#64, count#65]
 Results [6]: [i_brand_id#59, i_class_id#60, i_category_id#61, sum#66, isEmpty#67, count#68]
 
@@ -474,9 +474,9 @@ Arguments: hashpartitioning(i_brand_id#59, i_class_id#60, i_category_id#61, 5),
 (81) HashAggregate [codegen id : 52]
 Input [6]: [i_brand_id#59, i_class_id#60, i_category_id#61, sum#66, isEmpty#67, count#68]
 Keys [3]: [i_brand_id#59, i_class_id#60, i_category_id#61]
-Functions [2]: [sum(CheckOverflow((promote_precision(cast(cs_quantity#55 as decimal(12,2))) * promote_precision(cast(cs_list_price#56 as decimal(12,2)))), DecimalType(18,2))), count(1)]
-Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cs_quantity#55 as decimal(12,2))) * promote_precision(cast(cs_list_price#56 as decimal(12,2)))), DecimalType(18,2)))#69, count(1)#70]
-Results [5]: [i_brand_id#59, i_class_id#60, i_category_id#61, sum(CheckOverflow((promote_precision(cast(cs_quantity#55 as decimal(12,2))) * promote_precision(cast(cs_list_price#56 as decimal(12,2)))), DecimalType(18,2)))#69 AS sales#71, count(1)#70 AS number_sales#72]
+Functions [2]: [sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56)), count(1)]
+Aggregate Attributes [2]: [sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56))#69, count(1)#70]
+Results [5]: [i_brand_id#59, i_class_id#60, i_category_id#61, sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56))#69 AS sales#71, count(1)#70 AS number_sales#72]
 
 (82) Filter [codegen id : 52]
 Input [5]: [i_brand_id#59, i_class_id#60, i_category_id#61, sales#71, number_sales#72]
@@ -536,7 +536,7 @@ Input [7]: [ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77, i_brand_id#79,
 (95) HashAggregate [codegen id : 77]
 Input [5]: [ws_quantity#75, ws_list_price#76, i_brand_id#79, i_class_id#80, i_category_id#81]
 Keys [3]: [i_brand_id#79, i_class_id#80, i_category_id#81]
-Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(ws_quantity#75 as decimal(12,2))) * promote_precision(cast(ws_list_price#76 as decimal(12,2)))), DecimalType(18,2))), partial_count(1)]
+Functions [2]: [partial_sum((cast(ws_quantity#75 as decimal(10,0)) * ws_list_price#76)), partial_count(1)]
 Aggregate Attributes [3]: [sum#83, isEmpty#84, count#85]
 Results [6]: [i_brand_id#79, i_class_id#80, i_category_id#81, sum#86, isEmpty#87, count#88]
 
@@ -547,9 +547,9 @@ Arguments: hashpartitioning(i_brand_id#79, i_class_id#80, i_category_id#81, 5),
 (97) HashAggregate [codegen id : 78]
 Input [6]: [i_brand_id#79, i_class_id#80, i_category_id#81, sum#86, isEmpty#87, count#88]
 Keys [3]: [i_brand_id#79, i_class_id#80, i_category_id#81]
-Functions [2]: [sum(CheckOverflow((promote_precision(cast(ws_quantity#75 as decimal(12,2))) * promote_precision(cast(ws_list_price#76 as decimal(12,2)))), DecimalType(18,2))), count(1)]
-Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(ws_quantity#75 as decimal(12,2))) * promote_precision(cast(ws_list_price#76 as decimal(12,2)))), DecimalType(18,2)))#89, count(1)#90]
-Results [5]: [i_brand_id#79, i_class_id#80, i_category_id#81, sum(CheckOverflow((promote_precision(cast(ws_quantity#75 as decimal(12,2))) * promote_precision(cast(ws_list_price#76 as decimal(12,2)))), DecimalType(18,2)))#89 AS sales#91, count(1)#90 AS number_sales#92]
+Functions [2]: [sum((cast(ws_quantity#75 as decimal(10,0)) * ws_list_price#76)), count(1)]
+Aggregate Attributes [2]: [sum((cast(ws_quantity#75 as decimal(10,0)) * ws_list_price#76))#89, count(1)#90]
+Results [5]: [i_brand_id#79, i_class_id#80, i_category_id#81, sum((cast(ws_quantity#75 as decimal(10,0)) * ws_list_price#76))#89 AS sales#91, count(1)#90 AS number_sales#92]
 
 (98) Filter [codegen id : 78]
 Input [5]: [i_brand_id#79, i_class_id#80, i_category_id#81, sales#91, number_sales#92]
@@ -682,7 +682,7 @@ Input [4]: [ws_quantity#121, ws_list_price#122, ws_sold_date_sk#123, d_date_sk#1
 (122) HashAggregate [codegen id : 7]
 Input [2]: [quantity#113, list_price#114]
 Keys: []
-Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(quantity#113 as decimal(12,2))) * promote_precision(cast(list_price#114 as decimal(12,2)))), DecimalType(18,2)))]
+Functions [1]: [partial_avg((cast(quantity#113 as decimal(10,0)) * list_price#114))]
 Aggregate Attributes [2]: [sum#127, count#128]
 Results [2]: [sum#129, count#130]
 
@@ -693,9 +693,9 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=13]
 (124) HashAggregate [codegen id : 8]
 Input [2]: [sum#129, count#130]
 Keys: []
-Functions [1]: [avg(CheckOverflow((promote_precision(cast(quantity#113 as decimal(12,2))) * promote_precision(cast(list_price#114 as decimal(12,2)))), DecimalType(18,2)))]
-Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(quantity#113 as decimal(12,2))) * promote_precision(cast(list_price#114 as decimal(12,2)))), DecimalType(18,2)))#131]
-Results [1]: [avg(CheckOverflow((promote_precision(cast(quantity#113 as decimal(12,2))) * promote_precision(cast(list_price#114 as decimal(12,2)))), DecimalType(18,2)))#131 AS average_sales#132]
+Functions [1]: [avg((cast(quantity#113 as decimal(10,0)) * list_price#114))]
+Aggregate Attributes [1]: [avg((cast(quantity#113 as decimal(10,0)) * list_price#114))#131]
+Results [1]: [avg((cast(quantity#113 as decimal(10,0)) * list_price#114))#131 AS average_sales#132]
 
 Subquery:2 Hosting operator id = 106 Hosting Expression = ss_sold_date_sk#111 IN dynamicpruning#12
 
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt
index b8125b2af8e..07e4254a6a9 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14a/simplified.txt
@@ -13,7 +13,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su
                         Filter [sales]
                           Subquery #3
                             WholeStageCodegen (8)
-                              HashAggregate [sum,count] [avg(CheckOverflow((promote_precision(cast(quantity as decimal(12,2))) * promote_precision(cast(list_price as decimal(12,2)))), DecimalType(18,2))),average_sales,sum,count]
+                              HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count]
                                 InputAdapter
                                   Exchange #13
                                     WholeStageCodegen (7)
@@ -47,7 +47,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su
                                                         ReusedSubquery [d_date_sk] #2
                                                   InputAdapter
                                                     ReusedExchange [d_date_sk] #7
-                          HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(ss_quantity as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2))),count(1),sales,number_sales,sum,isEmpty,count]
+                          HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),sales,number_sales,sum,isEmpty,count]
                             InputAdapter
                               Exchange [i_brand_id,i_class_id,i_category_id] #2
                                 WholeStageCodegen (25)
@@ -166,7 +166,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su
                       Project [sales,number_sales,i_brand_id,i_class_id,i_category_id]
                         Filter [sales]
                           ReusedSubquery [average_sales] #3
-                          HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cs_quantity as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2))),count(1),sales,number_sales,sum,isEmpty,count]
+                          HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1),sales,number_sales,sum,isEmpty,count]
                             InputAdapter
                               Exchange [i_brand_id,i_class_id,i_category_id] #14
                                 WholeStageCodegen (51)
@@ -191,7 +191,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum(sales),su
                       Project [sales,number_sales,i_brand_id,i_class_id,i_category_id]
                         Filter [sales]
                           ReusedSubquery [average_sales] #3
-                          HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(ws_quantity as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2))),count(1),sales,number_sales,sum,isEmpty,count]
+                          HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1),sales,number_sales,sum,isEmpty,count]
                             InputAdapter
                               Exchange [i_brand_id,i_class_id,i_category_id] #15
                                 WholeStageCodegen (77)
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt
index 20e5edb3038..1a98aac9b4c 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/explain.txt
@@ -432,7 +432,7 @@ Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#37, i_brand_
 (75) HashAggregate [codegen id : 43]
 Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#38, i_class_id#39, i_category_id#40]
 Keys [3]: [i_brand_id#38, i_class_id#39, i_category_id#40]
-Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(ss_quantity#2 as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2))), partial_count(1)]
+Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)]
 Aggregate Attributes [3]: [sum#41, isEmpty#42, count#43]
 Results [6]: [i_brand_id#38, i_class_id#39, i_category_id#40, sum#44, isEmpty#45, count#46]
 
@@ -443,9 +443,9 @@ Arguments: hashpartitioning(i_brand_id#38, i_class_id#39, i_category_id#40, 5),
 (77) HashAggregate [codegen id : 88]
 Input [6]: [i_brand_id#38, i_class_id#39, i_category_id#40, sum#44, isEmpty#45, count#46]
 Keys [3]: [i_brand_id#38, i_class_id#39, i_category_id#40]
-Functions [2]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#2 as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2))), count(1)]
-Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#2 as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2)))#47, count(1)#48]
-Results [6]: [store AS channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum(CheckOverflow((promote_precision(cast(ss_quantity#2 as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2)))#47 AS sales#50, count(1)#48 AS number_sales#51]
+Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)]
+Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47, count(1)#48]
+Results [6]: [store AS channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47 AS sales#50, count(1)#48 AS number_sales#51]
 
 (78) Filter [codegen id : 88]
 Input [6]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sales#50, number_sales#51]
@@ -513,7 +513,7 @@ Input [7]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, i_item_sk#60, i_bra
 (93) HashAggregate [codegen id : 86]
 Input [5]: [ss_quantity#55, ss_list_price#56, i_brand_id#61, i_class_id#62, i_category_id#63]
 Keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63]
-Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(ss_quantity#55 as decimal(12,2))) * promote_precision(cast(ss_list_price#56 as decimal(12,2)))), DecimalType(18,2))), partial_count(1)]
+Functions [2]: [partial_sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56)), partial_count(1)]
 Aggregate Attributes [3]: [sum#64, isEmpty#65, count#66]
 Results [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#67, isEmpty#68, count#69]
 
@@ -524,9 +524,9 @@ Arguments: hashpartitioning(i_brand_id#61, i_class_id#62, i_category_id#63, 5),
 (95) HashAggregate [codegen id : 87]
 Input [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#67, isEmpty#68, count#69]
 Keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63]
-Functions [2]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#55 as decimal(12,2))) * promote_precision(cast(ss_list_price#56 as decimal(12,2)))), DecimalType(18,2))), count(1)]
-Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#55 as decimal(12,2))) * promote_precision(cast(ss_list_price#56 as decimal(12,2)))), DecimalType(18,2)))#70, count(1)#71]
-Results [6]: [store AS channel#72, i_brand_id#61, i_class_id#62, i_category_id#63, sum(CheckOverflow((promote_precision(cast(ss_quantity#55 as decimal(12,2))) * promote_precision(cast(ss_list_price#56 as decimal(12,2)))), DecimalType(18,2)))#70 AS sales#73, count(1)#71 AS number_sales#74]
+Functions [2]: [sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56)), count(1)]
+Aggregate Attributes [2]: [sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56))#70, count(1)#71]
+Results [6]: [store AS channel#72, i_brand_id#61, i_class_id#62, i_category_id#63, sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56))#70 AS sales#73, count(1)#71 AS number_sales#74]
 
 (96) Filter [codegen id : 87]
 Input [6]: [channel#72, i_brand_id#61, i_class_id#62, i_category_id#63, sales#73, number_sales#74]
@@ -640,7 +640,7 @@ Input [4]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#89, d_date_sk#90]
 (116) HashAggregate [codegen id : 7]
 Input [2]: [quantity#79, list_price#80]
 Keys: []
-Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(quantity#79 as decimal(12,2))) * promote_precision(cast(list_price#80 as decimal(12,2)))), DecimalType(18,2)))]
+Functions [1]: [partial_avg((cast(quantity#79 as decimal(10,0)) * list_price#80))]
 Aggregate Attributes [2]: [sum#93, count#94]
 Results [2]: [sum#95, count#96]
 
@@ -651,9 +651,9 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=17]
 (118) HashAggregate [codegen id : 8]
 Input [2]: [sum#95, count#96]
 Keys: []
-Functions [1]: [avg(CheckOverflow((promote_precision(cast(quantity#79 as decimal(12,2))) * promote_precision(cast(list_price#80 as decimal(12,2)))), DecimalType(18,2)))]
-Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(quantity#79 as decimal(12,2))) * promote_precision(cast(list_price#80 as decimal(12,2)))), DecimalType(18,2)))#97]
-Results [1]: [avg(CheckOverflow((promote_precision(cast(quantity#79 as decimal(12,2))) * promote_precision(cast(list_price#80 as decimal(12,2)))), DecimalType(18,2)))#97 AS average_sales#98]
+Functions [1]: [avg((cast(quantity#79 as decimal(10,0)) * list_price#80))]
+Aggregate Attributes [1]: [avg((cast(quantity#79 as decimal(10,0)) * list_price#80))#97]
+Results [1]: [avg((cast(quantity#79 as decimal(10,0)) * list_price#80))#97 AS average_sales#98]
 
 Subquery:2 Hosting operator id = 100 Hosting Expression = ss_sold_date_sk#77 IN dynamicpruning#12
 
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/simplified.txt
index 82e338515f4..c6d31d04f75 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/simplified.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b.sf100/simplified.txt
@@ -4,7 +4,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
       Filter [sales]
         Subquery #4
           WholeStageCodegen (8)
-            HashAggregate [sum,count] [avg(CheckOverflow((promote_precision(cast(quantity as decimal(12,2))) * promote_precision(cast(list_price as decimal(12,2)))), DecimalType(18,2))),average_sales,sum,count]
+            HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count]
               InputAdapter
                 Exchange #16
                   WholeStageCodegen (7)
@@ -38,7 +38,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
                                       ReusedSubquery [d_date_sk] #3
                                 InputAdapter
                                   ReusedExchange [d_date_sk] #8
-        HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(ss_quantity as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2))),count(1),channel,sales,number_sales,sum,isEmpty,count]
+        HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count]
           InputAdapter
             Exchange [i_brand_id,i_class_id,i_category_id] #1
               WholeStageCodegen (43)
@@ -201,7 +201,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
           WholeStageCodegen (87)
             Filter [sales]
               ReusedSubquery [average_sales] #4
-              HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(ss_quantity as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2))),count(1),channel,sales,number_sales,sum,isEmpty,count]
+              HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count]
                 InputAdapter
                   Exchange [i_brand_id,i_class_id,i_category_id] #18
                     WholeStageCodegen (86)
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt
index 1972d6840d1..0db8ff99fca 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/explain.txt
@@ -369,7 +369,7 @@ Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#37, i_
 (63) HashAggregate [codegen id : 25]
 Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#37, i_class_id#38, i_category_id#39]
 Keys [3]: [i_brand_id#37, i_class_id#38, i_category_id#39]
-Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(ss_quantity#2 as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2))), partial_count(1)]
+Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)]
 Aggregate Attributes [3]: [sum#41, isEmpty#42, count#43]
 Results [6]: [i_brand_id#37, i_class_id#38, i_category_id#39, sum#44, isEmpty#45, count#46]
 
@@ -380,9 +380,9 @@ Arguments: hashpartitioning(i_brand_id#37, i_class_id#38, i_category_id#39, 5),
 (65) HashAggregate [codegen id : 52]
 Input [6]: [i_brand_id#37, i_class_id#38, i_category_id#39, sum#44, isEmpty#45, count#46]
 Keys [3]: [i_brand_id#37, i_class_id#38, i_category_id#39]
-Functions [2]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#2 as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2))), count(1)]
-Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#2 as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2)))#47, count(1)#48]
-Results [6]: [store AS channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum(CheckOverflow((promote_precision(cast(ss_quantity#2 as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2)))#47 AS sales#50, count(1)#48 AS number_sales#51]
+Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)]
+Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47, count(1)#48]
+Results [6]: [store AS channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47 AS sales#50, count(1)#48 AS number_sales#51]
 
 (66) Filter [codegen id : 52]
 Input [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sales#50, number_sales#51]
@@ -438,7 +438,7 @@ Input [7]: [ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57, i_brand_id#60,
 (78) HashAggregate [codegen id : 50]
 Input [5]: [ss_quantity#55, ss_list_price#56, i_brand_id#60, i_class_id#61, i_category_id#62]
 Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62]
-Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(ss_quantity#55 as decimal(12,2))) * promote_precision(cast(ss_list_price#56 as decimal(12,2)))), DecimalType(18,2))), partial_count(1)]
+Functions [2]: [partial_sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56)), partial_count(1)]
 Aggregate Attributes [3]: [sum#64, isEmpty#65, count#66]
 Results [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#67, isEmpty#68, count#69]
 
@@ -449,9 +449,9 @@ Arguments: hashpartitioning(i_brand_id#60, i_class_id#61, i_category_id#62, 5),
 (80) HashAggregate [codegen id : 51]
 Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#67, isEmpty#68, count#69]
 Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62]
-Functions [2]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#55 as decimal(12,2))) * promote_precision(cast(ss_list_price#56 as decimal(12,2)))), DecimalType(18,2))), count(1)]
-Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#55 as decimal(12,2))) * promote_precision(cast(ss_list_price#56 as decimal(12,2)))), DecimalType(18,2)))#70, count(1)#71]
-Results [6]: [store AS channel#72, i_brand_id#60, i_class_id#61, i_category_id#62, sum(CheckOverflow((promote_precision(cast(ss_quantity#55 as decimal(12,2))) * promote_precision(cast(ss_list_price#56 as decimal(12,2)))), DecimalType(18,2)))#70 AS sales#73, count(1)#71 AS number_sales#74]
+Functions [2]: [sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56)), count(1)]
+Aggregate Attributes [2]: [sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56))#70, count(1)#71]
+Results [6]: [store AS channel#72, i_brand_id#60, i_class_id#61, i_category_id#62, sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56))#70 AS sales#73, count(1)#71 AS number_sales#74]
 
 (81) Filter [codegen id : 51]
 Input [6]: [channel#72, i_brand_id#60, i_class_id#61, i_category_id#62, sales#73, number_sales#74]
@@ -565,7 +565,7 @@ Input [4]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#89, d_date_sk#90]
 (101) HashAggregate [codegen id : 7]
 Input [2]: [quantity#79, list_price#80]
 Keys: []
-Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(quantity#79 as decimal(12,2))) * promote_precision(cast(list_price#80 as decimal(12,2)))), DecimalType(18,2)))]
+Functions [1]: [partial_avg((cast(quantity#79 as decimal(10,0)) * list_price#80))]
 Aggregate Attributes [2]: [sum#93, count#94]
 Results [2]: [sum#95, count#96]
 
@@ -576,9 +576,9 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=12]
 (103) HashAggregate [codegen id : 8]
 Input [2]: [sum#95, count#96]
 Keys: []
-Functions [1]: [avg(CheckOverflow((promote_precision(cast(quantity#79 as decimal(12,2))) * promote_precision(cast(list_price#80 as decimal(12,2)))), DecimalType(18,2)))]
-Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(quantity#79 as decimal(12,2))) * promote_precision(cast(list_price#80 as decimal(12,2)))), DecimalType(18,2)))#97]
-Results [1]: [avg(CheckOverflow((promote_precision(cast(quantity#79 as decimal(12,2))) * promote_precision(cast(list_price#80 as decimal(12,2)))), DecimalType(18,2)))#97 AS average_sales#98]
+Functions [1]: [avg((cast(quantity#79 as decimal(10,0)) * list_price#80))]
+Aggregate Attributes [1]: [avg((cast(quantity#79 as decimal(10,0)) * list_price#80))#97]
+Results [1]: [avg((cast(quantity#79 as decimal(10,0)) * list_price#80))#97 AS average_sales#98]
 
 Subquery:2 Hosting operator id = 85 Hosting Expression = ss_sold_date_sk#77 IN dynamicpruning#12
 
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt
index 259178d0e43..13104f1ba31 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q14b/simplified.txt
@@ -4,7 +4,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
       Filter [sales]
         Subquery #4
           WholeStageCodegen (8)
-            HashAggregate [sum,count] [avg(CheckOverflow((promote_precision(cast(quantity as decimal(12,2))) * promote_precision(cast(list_price as decimal(12,2)))), DecimalType(18,2))),average_sales,sum,count]
+            HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count]
               InputAdapter
                 Exchange #12
                   WholeStageCodegen (7)
@@ -38,7 +38,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
                                       ReusedSubquery [d_date_sk] #3
                                 InputAdapter
                                   ReusedExchange [d_date_sk] #6
-        HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(ss_quantity as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2))),count(1),channel,sales,number_sales,sum,isEmpty,count]
+        HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count]
           InputAdapter
             Exchange [i_brand_id,i_class_id,i_category_id] #1
               WholeStageCodegen (25)
@@ -165,7 +165,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
           WholeStageCodegen (51)
             Filter [sales]
               ReusedSubquery [average_sales] #4
-              HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(ss_quantity as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2))),count(1),channel,sales,number_sales,sum,isEmpty,count]
+              HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count]
                 InputAdapter
                   Exchange [i_brand_id,i_class_id,i_category_id] #14
                     WholeStageCodegen (50)
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.sf100/explain.txt
index 5e2e350d0f2..97ee1f55c5a 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2.sf100/explain.txt
@@ -274,7 +274,7 @@ Right keys [1]: [(d_week_seq2#70 - 53)]
 Join condition: None
 
 (49) Project [codegen id : 12]
-Output [8]: [d_week_seq1#44, round(CheckOverflow((promote_precision(sun_sales1#45) / promote_precision(sun_sales2#71)), DecimalType(37,20)), 2) AS round((sun_sales1 / sun_sales2), 2)#78, round(CheckOverflow((promote_precision(mon_sales1#46) / promote_precision(mon_sales2#72)), DecimalType(37,20)), 2) AS round((mon_sales1 / mon_sales2), 2)#79, round(CheckOverflow((promote_precision(tue_sales1#47) / promote_precision(tue_sales2#73)), DecimalType(37,20)), 2) AS round((tue_sales1 / tue_sales [...]
+Output [8]: [d_week_seq1#44, round((sun_sales1#45 / sun_sales2#71), 2) AS round((sun_sales1 / sun_sales2), 2)#78, round((mon_sales1#46 / mon_sales2#72), 2) AS round((mon_sales1 / mon_sales2), 2)#79, round((tue_sales1#47 / tue_sales2#73), 2) AS round((tue_sales1 / tue_sales2), 2)#80, round((wed_sales1#48 / wed_sales2#74), 2) AS round((wed_sales1 / wed_sales2), 2)#81, round((thu_sales1#49 / thu_sales2#75), 2) AS round((thu_sales1 / thu_sales2), 2)#82, round((fri_sales1#50 / fri_sales2#76), [...]
 Input [16]: [d_week_seq1#44, sun_sales1#45, mon_sales1#46, tue_sales1#47, wed_sales1#48, thu_sales1#49, fri_sales1#50, sat_sales1#51, d_week_seq2#70, sun_sales2#71, mon_sales2#72, tue_sales2#73, wed_sales2#74, thu_sales2#75, fri_sales2#76, sat_sales2#77]
 
 (50) Exchange
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/explain.txt
index 031dc924069..d0c34b39c77 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q2/explain.txt
@@ -195,7 +195,7 @@ Right keys [1]: [(d_week_seq2#59 - 53)]
 Join condition: None
 
 (35) Project [codegen id : 12]
-Output [8]: [d_week_seq1#42, round(CheckOverflow((promote_precision(sun_sales1#43) / promote_precision(sun_sales2#60)), DecimalType(37,20)), 2) AS round((sun_sales1 / sun_sales2), 2)#67, round(CheckOverflow((promote_precision(mon_sales1#44) / promote_precision(mon_sales2#61)), DecimalType(37,20)), 2) AS round((mon_sales1 / mon_sales2), 2)#68, round(CheckOverflow((promote_precision(tue_sales1#45) / promote_precision(tue_sales2#62)), DecimalType(37,20)), 2) AS round((tue_sales1 / tue_sales [...]
+Output [8]: [d_week_seq1#42, round((sun_sales1#43 / sun_sales2#60), 2) AS round((sun_sales1 / sun_sales2), 2)#67, round((mon_sales1#44 / mon_sales2#61), 2) AS round((mon_sales1 / mon_sales2), 2)#68, round((tue_sales1#45 / tue_sales2#62), 2) AS round((tue_sales1 / tue_sales2), 2)#69, round((wed_sales1#46 / wed_sales2#63), 2) AS round((wed_sales1 / wed_sales2), 2)#70, round((thu_sales1#47 / thu_sales2#64), 2) AS round((thu_sales1 / thu_sales2), 2)#71, round((fri_sales1#48 / fri_sales2#65), [...]
 Input [16]: [d_week_seq1#42, sun_sales1#43, mon_sales1#44, tue_sales1#45, wed_sales1#46, thu_sales1#47, fri_sales1#48, sat_sales1#49, d_week_seq2#59, sun_sales2#60, mon_sales2#61, tue_sales2#62, wed_sales2#63, thu_sales2#64, fri_sales2#65, sat_sales2#66]
 
 (36) Exchange
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.sf100/explain.txt
index 99d27cdf9cb..3f91994b17c 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20.sf100/explain.txt
@@ -121,7 +121,7 @@ Input [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemreve
 Arguments: [sum(_w1#17) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#18], [i_class#9]
 
 (22) Project [codegen id : 9]
-Output [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(_w0#16) * 100.00), DecimalType(21,2)) as decimal(27,2))) / promote_precision(_we0#18)), DecimalType(38,17)) AS revenueratio#19, i_item_id#6]
+Output [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, ((_w0#16 * 100) / _we0#18) AS revenueratio#19, i_item_id#6]
 Input [9]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, _w1#17, i_item_id#6, _we0#18]
 
 (23) TakeOrderedAndProject
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/explain.txt
index 775fba161d8..7d9b5a2847b 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q20/explain.txt
@@ -106,7 +106,7 @@ Input [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemreve
 Arguments: [sum(_w1#17) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#18], [i_class#9]
 
 (19) Project [codegen id : 6]
-Output [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(_w0#16) * 100.00), DecimalType(21,2)) as decimal(27,2))) / promote_precision(_we0#18)), DecimalType(38,17)) AS revenueratio#19, i_item_id#6]
+Output [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, ((_w0#16 * 100) / _we0#18) AS revenueratio#19, i_item_id#6]
 Input [9]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, _w1#17, i_item_id#6, _we0#18]
 
 (20) TakeOrderedAndProject
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/explain.txt
index 84fdda1e313..5e39b962f98 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/explain.txt
@@ -278,20 +278,20 @@ Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, c_customer_sk#
 (42) HashAggregate [codegen id : 15]
 Input [3]: [ss_quantity#21, ss_sales_price#22, c_customer_sk#24]
 Keys [1]: [c_customer_sk#24]
-Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(ss_quantity#21 as decimal(12,2))) * promote_precision(cast(ss_sales_price#22 as decimal(12,2)))), DecimalType(18,2)))]
+Functions [1]: [partial_sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))]
 Aggregate Attributes [2]: [sum#25, isEmpty#26]
 Results [3]: [c_customer_sk#24, sum#27, isEmpty#28]
 
 (43) HashAggregate [codegen id : 15]
 Input [3]: [c_customer_sk#24, sum#27, isEmpty#28]
 Keys [1]: [c_customer_sk#24]
-Functions [1]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#21 as decimal(12,2))) * promote_precision(cast(ss_sales_price#22 as decimal(12,2)))), DecimalType(18,2)))]
-Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#21 as decimal(12,2))) * promote_precision(cast(ss_sales_price#22 as decimal(12,2)))), DecimalType(18,2)))#29]
-Results [2]: [c_customer_sk#24, sum(CheckOverflow((promote_precision(cast(ss_quantity#21 as decimal(12,2))) * promote_precision(cast(ss_sales_price#22 as decimal(12,2)))), DecimalType(18,2)))#29 AS ssales#30]
+Functions [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))]
+Aggregate Attributes [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29]
+Results [2]: [c_customer_sk#24, sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29 AS ssales#30]
 
 (44) Filter [codegen id : 15]
 Input [2]: [c_customer_sk#24, ssales#30]
-Condition : (isnotnull(ssales#30) AND (cast(ssales#30 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#31, [id=#32] as decimal(32,6)))), DecimalType(38,8))))
+Condition : (isnotnull(ssales#30) AND (cast(ssales#30 as decimal(38,8)) > (0.500000 * Subquery scalar-subquery#31, [id=#32])))
 
 (45) Project [codegen id : 15]
 Output [1]: [c_customer_sk#24]
@@ -319,7 +319,7 @@ Right keys [1]: [d_date_sk#33]
 Join condition: None
 
 (51) Project [codegen id : 17]
-Output [1]: [CheckOverflow((promote_precision(cast(cs_quantity#3 as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2)) AS sales#34]
+Output [1]: [(cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4) AS sales#34]
 Input [4]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#33]
 
 (52) Scan parquet default.web_sales
@@ -432,20 +432,20 @@ Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, c_customer_sk#
 (77) HashAggregate [codegen id : 32]
 Input [3]: [ss_quantity#21, ss_sales_price#22, c_customer_sk#24]
 Keys [1]: [c_customer_sk#24]
-Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(ss_quantity#21 as decimal(12,2))) * promote_precision(cast(ss_sales_price#22 as decimal(12,2)))), DecimalType(18,2)))]
-Aggregate Attributes [2]: [sum#40, isEmpty#41]
-Results [3]: [c_customer_sk#24, sum#42, isEmpty#43]
+Functions [1]: [partial_sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))]
+Aggregate Attributes [2]: [sum#25, isEmpty#26]
+Results [3]: [c_customer_sk#24, sum#27, isEmpty#28]
 
 (78) HashAggregate [codegen id : 32]
-Input [3]: [c_customer_sk#24, sum#42, isEmpty#43]
+Input [3]: [c_customer_sk#24, sum#27, isEmpty#28]
 Keys [1]: [c_customer_sk#24]
-Functions [1]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#21 as decimal(12,2))) * promote_precision(cast(ss_sales_price#22 as decimal(12,2)))), DecimalType(18,2)))]
-Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#21 as decimal(12,2))) * promote_precision(cast(ss_sales_price#22 as decimal(12,2)))), DecimalType(18,2)))#29]
-Results [2]: [c_customer_sk#24, sum(CheckOverflow((promote_precision(cast(ss_quantity#21 as decimal(12,2))) * promote_precision(cast(ss_sales_price#22 as decimal(12,2)))), DecimalType(18,2)))#29 AS ssales#30]
+Functions [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))]
+Aggregate Attributes [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29]
+Results [2]: [c_customer_sk#24, sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29 AS ssales#30]
 
 (79) Filter [codegen id : 32]
 Input [2]: [c_customer_sk#24, ssales#30]
-Condition : (isnotnull(ssales#30) AND (cast(ssales#30 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#31, [id=#32] as decimal(32,6)))), DecimalType(38,8))))
+Condition : (isnotnull(ssales#30) AND (cast(ssales#30 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#31, [id=#32])))
 
 (80) Project [codegen id : 32]
 Output [1]: [c_customer_sk#24]
@@ -465,16 +465,16 @@ Output [3]: [ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39]
 Input [4]: [ws_bill_customer_sk#36, ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39]
 
 (84) ReusedExchange [Reuses operator id: 95]
-Output [1]: [d_date_sk#44]
+Output [1]: [d_date_sk#40]
 
 (85) BroadcastHashJoin [codegen id : 34]
 Left keys [1]: [ws_sold_date_sk#39]
-Right keys [1]: [d_date_sk#44]
+Right keys [1]: [d_date_sk#40]
 Join condition: None
 
 (86) Project [codegen id : 34]
-Output [1]: [CheckOverflow((promote_precision(cast(ws_quantity#37 as decimal(12,2))) * promote_precision(cast(ws_list_price#38 as decimal(12,2)))), DecimalType(18,2)) AS sales#45]
-Input [4]: [ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39, d_date_sk#44]
+Output [1]: [(cast(ws_quantity#37 as decimal(10,0)) * ws_list_price#38) AS sales#41]
+Input [4]: [ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39, d_date_sk#40]
 
 (87) Union
 
@@ -482,19 +482,19 @@ Input [4]: [ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39, d_date_sk#44]
 Input [1]: [sales#34]
 Keys: []
 Functions [1]: [partial_sum(sales#34)]
-Aggregate Attributes [2]: [sum#46, isEmpty#47]
-Results [2]: [sum#48, isEmpty#49]
+Aggregate Attributes [2]: [sum#42, isEmpty#43]
+Results [2]: [sum#44, isEmpty#45]
 
 (89) Exchange
-Input [2]: [sum#48, isEmpty#49]
+Input [2]: [sum#44, isEmpty#45]
 Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=9]
 
 (90) HashAggregate [codegen id : 36]
-Input [2]: [sum#48, isEmpty#49]
+Input [2]: [sum#44, isEmpty#45]
 Keys: []
 Functions [1]: [sum(sales#34)]
-Aggregate Attributes [1]: [sum(sales#34)#50]
-Results [1]: [sum(sales#34)#50 AS sum(sales)#51]
+Aggregate Attributes [1]: [sum(sales#34)#46]
+Results [1]: [sum(sales#34)#46 AS sum(sales)#47]
 
 ===== Subqueries =====
 
@@ -507,22 +507,22 @@ BroadcastExchange (95)
 
 
 (91) Scan parquet default.date_dim
-Output [3]: [d_date_sk#33, d_year#52, d_moy#53]
+Output [3]: [d_date_sk#33, d_year#48, d_moy#49]
 Batched: true
 Location [not included in comparison]/{warehouse_dir}/date_dim]
 PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)]
 ReadSchema: struct<d_date_sk:int,d_year:int,d_moy:int>
 
 (92) ColumnarToRow [codegen id : 1]
-Input [3]: [d_date_sk#33, d_year#52, d_moy#53]
+Input [3]: [d_date_sk#33, d_year#48, d_moy#49]
 
 (93) Filter [codegen id : 1]
-Input [3]: [d_date_sk#33, d_year#52, d_moy#53]
-Condition : ((((isnotnull(d_year#52) AND isnotnull(d_moy#53)) AND (d_year#52 = 2000)) AND (d_moy#53 = 2)) AND isnotnull(d_date_sk#33))
+Input [3]: [d_date_sk#33, d_year#48, d_moy#49]
+Condition : ((((isnotnull(d_year#48) AND isnotnull(d_moy#49)) AND (d_year#48 = 2000)) AND (d_moy#49 = 2)) AND isnotnull(d_date_sk#33))
 
 (94) Project [codegen id : 1]
 Output [1]: [d_date_sk#33]
-Input [3]: [d_date_sk#33, d_year#52, d_moy#53]
+Input [3]: [d_date_sk#33, d_year#48, d_moy#49]
 
 (95) BroadcastExchange
 Input [1]: [d_date_sk#33]
@@ -537,22 +537,22 @@ BroadcastExchange (100)
 
 
 (96) Scan parquet default.date_dim
-Output [3]: [d_date_sk#10, d_date#11, d_year#54]
+Output [3]: [d_date_sk#10, d_date#11, d_year#50]
 Batched: true
 Location [not included in comparison]/{warehouse_dir}/date_dim]
 PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)]
 ReadSchema: struct<d_date_sk:int,d_date:date,d_year:int>
 
 (97) ColumnarToRow [codegen id : 1]
-Input [3]: [d_date_sk#10, d_date#11, d_year#54]
+Input [3]: [d_date_sk#10, d_date#11, d_year#50]
 
 (98) Filter [codegen id : 1]
-Input [3]: [d_date_sk#10, d_date#11, d_year#54]
-Condition : (d_year#54 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10))
+Input [3]: [d_date_sk#10, d_date#11, d_year#50]
+Condition : (d_year#50 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10))
 
 (99) Project [codegen id : 1]
 Output [2]: [d_date_sk#10, d_date#11]
-Input [3]: [d_date_sk#10, d_date#11, d_year#54]
+Input [3]: [d_date_sk#10, d_date#11, d_year#50]
 
 (100) BroadcastExchange
 Input [2]: [d_date_sk#10, d_date#11]
@@ -579,89 +579,89 @@ Subquery:3 Hosting operator id = 44 Hosting Expression = Subquery scalar-subquer
 
 
 (101) Scan parquet default.store_sales
-Output [4]: [ss_customer_sk#55, ss_quantity#56, ss_sales_price#57, ss_sold_date_sk#58]
+Output [4]: [ss_customer_sk#51, ss_quantity#52, ss_sales_price#53, ss_sold_date_sk#54]
 Batched: true
 Location: InMemoryFileIndex []
-PartitionFilters: [isnotnull(ss_sold_date_sk#58), dynamicpruningexpression(ss_sold_date_sk#58 IN dynamicpruning#59)]
+PartitionFilters: [isnotnull(ss_sold_date_sk#54), dynamicpruningexpression(ss_sold_date_sk#54 IN dynamicpruning#55)]
 PushedFilters: [IsNotNull(ss_customer_sk)]
 ReadSchema: struct<ss_customer_sk:int,ss_quantity:int,ss_sales_price:decimal(7,2)>
 
 (102) ColumnarToRow [codegen id : 2]
-Input [4]: [ss_customer_sk#55, ss_quantity#56, ss_sales_price#57, ss_sold_date_sk#58]
+Input [4]: [ss_customer_sk#51, ss_quantity#52, ss_sales_price#53, ss_sold_date_sk#54]
 
 (103) Filter [codegen id : 2]
-Input [4]: [ss_customer_sk#55, ss_quantity#56, ss_sales_price#57, ss_sold_date_sk#58]
-Condition : isnotnull(ss_customer_sk#55)
+Input [4]: [ss_customer_sk#51, ss_quantity#52, ss_sales_price#53, ss_sold_date_sk#54]
+Condition : isnotnull(ss_customer_sk#51)
 
 (104) ReusedExchange [Reuses operator id: 122]
-Output [1]: [d_date_sk#60]
+Output [1]: [d_date_sk#56]
 
 (105) BroadcastHashJoin [codegen id : 2]
-Left keys [1]: [ss_sold_date_sk#58]
-Right keys [1]: [d_date_sk#60]
+Left keys [1]: [ss_sold_date_sk#54]
+Right keys [1]: [d_date_sk#56]
 Join condition: None
 
 (106) Project [codegen id : 2]
-Output [3]: [ss_customer_sk#55, ss_quantity#56, ss_sales_price#57]
-Input [5]: [ss_customer_sk#55, ss_quantity#56, ss_sales_price#57, ss_sold_date_sk#58, d_date_sk#60]
+Output [3]: [ss_customer_sk#51, ss_quantity#52, ss_sales_price#53]
+Input [5]: [ss_customer_sk#51, ss_quantity#52, ss_sales_price#53, ss_sold_date_sk#54, d_date_sk#56]
 
 (107) Exchange
-Input [3]: [ss_customer_sk#55, ss_quantity#56, ss_sales_price#57]
-Arguments: hashpartitioning(ss_customer_sk#55, 5), ENSURE_REQUIREMENTS, [plan_id=12]
+Input [3]: [ss_customer_sk#51, ss_quantity#52, ss_sales_price#53]
+Arguments: hashpartitioning(ss_customer_sk#51, 5), ENSURE_REQUIREMENTS, [plan_id=12]
 
 (108) Sort [codegen id : 3]
-Input [3]: [ss_customer_sk#55, ss_quantity#56, ss_sales_price#57]
-Arguments: [ss_customer_sk#55 ASC NULLS FIRST], false, 0
+Input [3]: [ss_customer_sk#51, ss_quantity#52, ss_sales_price#53]
+Arguments: [ss_customer_sk#51 ASC NULLS FIRST], false, 0
 
 (109) ReusedExchange [Reuses operator id: 38]
-Output [1]: [c_customer_sk#61]
+Output [1]: [c_customer_sk#57]
 
 (110) Sort [codegen id : 5]
-Input [1]: [c_customer_sk#61]
-Arguments: [c_customer_sk#61 ASC NULLS FIRST], false, 0
+Input [1]: [c_customer_sk#57]
+Arguments: [c_customer_sk#57 ASC NULLS FIRST], false, 0
 
 (111) SortMergeJoin [codegen id : 6]
-Left keys [1]: [ss_customer_sk#55]
-Right keys [1]: [c_customer_sk#61]
+Left keys [1]: [ss_customer_sk#51]
+Right keys [1]: [c_customer_sk#57]
 Join condition: None
 
 (112) Project [codegen id : 6]
-Output [3]: [ss_quantity#56, ss_sales_price#57, c_customer_sk#61]
-Input [4]: [ss_customer_sk#55, ss_quantity#56, ss_sales_price#57, c_customer_sk#61]
+Output [3]: [ss_quantity#52, ss_sales_price#53, c_customer_sk#57]
+Input [4]: [ss_customer_sk#51, ss_quantity#52, ss_sales_price#53, c_customer_sk#57]
 
 (113) HashAggregate [codegen id : 6]
-Input [3]: [ss_quantity#56, ss_sales_price#57, c_customer_sk#61]
-Keys [1]: [c_customer_sk#61]
-Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(ss_quantity#56 as decimal(12,2))) * promote_precision(cast(ss_sales_price#57 as decimal(12,2)))), DecimalType(18,2)))]
-Aggregate Attributes [2]: [sum#62, isEmpty#63]
-Results [3]: [c_customer_sk#61, sum#64, isEmpty#65]
+Input [3]: [ss_quantity#52, ss_sales_price#53, c_customer_sk#57]
+Keys [1]: [c_customer_sk#57]
+Functions [1]: [partial_sum((cast(ss_quantity#52 as decimal(10,0)) * ss_sales_price#53))]
+Aggregate Attributes [2]: [sum#58, isEmpty#59]
+Results [3]: [c_customer_sk#57, sum#60, isEmpty#61]
 
 (114) HashAggregate [codegen id : 6]
-Input [3]: [c_customer_sk#61, sum#64, isEmpty#65]
-Keys [1]: [c_customer_sk#61]
-Functions [1]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#56 as decimal(12,2))) * promote_precision(cast(ss_sales_price#57 as decimal(12,2)))), DecimalType(18,2)))]
-Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#56 as decimal(12,2))) * promote_precision(cast(ss_sales_price#57 as decimal(12,2)))), DecimalType(18,2)))#66]
-Results [1]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#56 as decimal(12,2))) * promote_precision(cast(ss_sales_price#57 as decimal(12,2)))), DecimalType(18,2)))#66 AS csales#67]
+Input [3]: [c_customer_sk#57, sum#60, isEmpty#61]
+Keys [1]: [c_customer_sk#57]
+Functions [1]: [sum((cast(ss_quantity#52 as decimal(10,0)) * ss_sales_price#53))]
+Aggregate Attributes [1]: [sum((cast(ss_quantity#52 as decimal(10,0)) * ss_sales_price#53))#62]
+Results [1]: [sum((cast(ss_quantity#52 as decimal(10,0)) * ss_sales_price#53))#62 AS csales#63]
 
 (115) HashAggregate [codegen id : 6]
-Input [1]: [csales#67]
+Input [1]: [csales#63]
 Keys: []
-Functions [1]: [partial_max(csales#67)]
-Aggregate Attributes [1]: [max#68]
-Results [1]: [max#69]
+Functions [1]: [partial_max(csales#63)]
+Aggregate Attributes [1]: [max#64]
+Results [1]: [max#65]
 
 (116) Exchange
-Input [1]: [max#69]
+Input [1]: [max#65]
 Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=13]
 
 (117) HashAggregate [codegen id : 7]
-Input [1]: [max#69]
+Input [1]: [max#65]
 Keys: []
-Functions [1]: [max(csales#67)]
-Aggregate Attributes [1]: [max(csales#67)#70]
-Results [1]: [max(csales#67)#70 AS tpcds_cmax#71]
+Functions [1]: [max(csales#63)]
+Aggregate Attributes [1]: [max(csales#63)#66]
+Results [1]: [max(csales#63)#66 AS tpcds_cmax#67]
 
-Subquery:4 Hosting operator id = 101 Hosting Expression = ss_sold_date_sk#58 IN dynamicpruning#59
+Subquery:4 Hosting operator id = 101 Hosting Expression = ss_sold_date_sk#54 IN dynamicpruning#55
 BroadcastExchange (122)
 +- * Project (121)
    +- * Filter (120)
@@ -670,25 +670,25 @@ BroadcastExchange (122)
 
 
 (118) Scan parquet default.date_dim
-Output [2]: [d_date_sk#60, d_year#72]
+Output [2]: [d_date_sk#56, d_year#68]
 Batched: true
 Location [not included in comparison]/{warehouse_dir}/date_dim]
 PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)]
 ReadSchema: struct<d_date_sk:int,d_year:int>
 
 (119) ColumnarToRow [codegen id : 1]
-Input [2]: [d_date_sk#60, d_year#72]
+Input [2]: [d_date_sk#56, d_year#68]
 
 (120) Filter [codegen id : 1]
-Input [2]: [d_date_sk#60, d_year#72]
-Condition : (d_year#72 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#60))
+Input [2]: [d_date_sk#56, d_year#68]
+Condition : (d_year#68 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#56))
 
 (121) Project [codegen id : 1]
-Output [1]: [d_date_sk#60]
-Input [2]: [d_date_sk#60, d_year#72]
+Output [1]: [d_date_sk#56]
+Input [2]: [d_date_sk#56, d_year#68]
 
 (122) BroadcastExchange
-Input [1]: [d_date_sk#60]
+Input [1]: [d_date_sk#56]
 Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=14]
 
 Subquery:5 Hosting operator id = 52 Hosting Expression = ws_sold_date_sk#39 IN dynamicpruning#6
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/simplified.txt
index 0683b263ea2..dc5f560b1ec 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/simplified.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a.sf100/simplified.txt
@@ -89,7 +89,7 @@ WholeStageCodegen (36)
                                             Exchange #10
                                               WholeStageCodegen (6)
                                                 HashAggregate [csales] [max,max]
-                                                  HashAggregate [c_customer_sk,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(ss_quantity as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2))),csales,sum,isEmpty]
+                                                  HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),csales,sum,isEmpty]
                                                     HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty]
                                                       Project [ss_quantity,ss_sales_price,c_customer_sk]
                                                         SortMergeJoin [ss_customer_sk,c_customer_sk]
@@ -120,7 +120,7 @@ WholeStageCodegen (36)
                                                               Sort [c_customer_sk]
                                                                 InputAdapter
                                                                   ReusedExchange [c_customer_sk] #9
-                                    HashAggregate [c_customer_sk,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(ss_quantity as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2))),ssales,sum,isEmpty]
+                                    HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty]
                                       HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty]
                                         Project [ss_quantity,ss_sales_price,c_customer_sk]
                                           SortMergeJoin [ss_customer_sk,c_customer_sk]
@@ -195,7 +195,7 @@ WholeStageCodegen (36)
                                 Project [c_customer_sk]
                                   Filter [ssales]
                                     ReusedSubquery [tpcds_cmax] #3
-                                    HashAggregate [c_customer_sk,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(ss_quantity as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2))),ssales,sum,isEmpty]
+                                    HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty]
                                       HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty]
                                         Project [ss_quantity,ss_sales_price,c_customer_sk]
                                           SortMergeJoin [ss_customer_sk,c_customer_sk]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/explain.txt
index 8e372bc4263..1de77707158 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/explain.txt
@@ -226,7 +226,7 @@ Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, c_customer_sk#
 (35) HashAggregate [codegen id : 8]
 Input [3]: [ss_quantity#21, ss_sales_price#22, c_customer_sk#24]
 Keys [1]: [c_customer_sk#24]
-Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(ss_quantity#21 as decimal(12,2))) * promote_precision(cast(ss_sales_price#22 as decimal(12,2)))), DecimalType(18,2)))]
+Functions [1]: [partial_sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))]
 Aggregate Attributes [2]: [sum#25, isEmpty#26]
 Results [3]: [c_customer_sk#24, sum#27, isEmpty#28]
 
@@ -237,13 +237,13 @@ Arguments: hashpartitioning(c_customer_sk#24, 5), ENSURE_REQUIREMENTS, [plan_id=
 (37) HashAggregate [codegen id : 9]
 Input [3]: [c_customer_sk#24, sum#27, isEmpty#28]
 Keys [1]: [c_customer_sk#24]
-Functions [1]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#21 as decimal(12,2))) * promote_precision(cast(ss_sales_price#22 as decimal(12,2)))), DecimalType(18,2)))]
-Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#21 as decimal(12,2))) * promote_precision(cast(ss_sales_price#22 as decimal(12,2)))), DecimalType(18,2)))#29]
-Results [2]: [c_customer_sk#24, sum(CheckOverflow((promote_precision(cast(ss_quantity#21 as decimal(12,2))) * promote_precision(cast(ss_sales_price#22 as decimal(12,2)))), DecimalType(18,2)))#29 AS ssales#30]
+Functions [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))]
+Aggregate Attributes [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29]
+Results [2]: [c_customer_sk#24, sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29 AS ssales#30]
 
 (38) Filter [codegen id : 9]
 Input [2]: [c_customer_sk#24, ssales#30]
-Condition : (isnotnull(ssales#30) AND (cast(ssales#30 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#31, [id=#32] as decimal(32,6)))), DecimalType(38,8))))
+Condition : (isnotnull(ssales#30) AND (cast(ssales#30 as decimal(38,8)) > (0.500000 * Subquery scalar-subquery#31, [id=#32])))
 
 (39) Project [codegen id : 9]
 Output [1]: [c_customer_sk#24]
@@ -271,7 +271,7 @@ Right keys [1]: [d_date_sk#33]
 Join condition: None
 
 (45) Project [codegen id : 11]
-Output [1]: [CheckOverflow((promote_precision(cast(cs_quantity#3 as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2)) AS sales#34]
+Output [1]: [(cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4) AS sales#34]
 Input [4]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, d_date_sk#33]
 
 (46) Scan parquet default.web_sales
@@ -305,18 +305,18 @@ Input [4]: [ws_bill_customer_sk#36, ws_quantity#37, ws_list_price#38, ws_sold_da
 Arguments: [ws_bill_customer_sk#36 ASC NULLS FIRST], false, 0
 
 (53) ReusedExchange [Reuses operator id: 36]
-Output [3]: [c_customer_sk#24, sum#40, isEmpty#41]
+Output [3]: [c_customer_sk#24, sum#27, isEmpty#28]
 
 (54) HashAggregate [codegen id : 20]
-Input [3]: [c_customer_sk#24, sum#40, isEmpty#41]
+Input [3]: [c_customer_sk#24, sum#27, isEmpty#28]
 Keys [1]: [c_customer_sk#24]
-Functions [1]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#21 as decimal(12,2))) * promote_precision(cast(ss_sales_price#22 as decimal(12,2)))), DecimalType(18,2)))]
-Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#21 as decimal(12,2))) * promote_precision(cast(ss_sales_price#22 as decimal(12,2)))), DecimalType(18,2)))#29]
-Results [2]: [c_customer_sk#24, sum(CheckOverflow((promote_precision(cast(ss_quantity#21 as decimal(12,2))) * promote_precision(cast(ss_sales_price#22 as decimal(12,2)))), DecimalType(18,2)))#29 AS ssales#30]
+Functions [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))]
+Aggregate Attributes [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29]
+Results [2]: [c_customer_sk#24, sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29 AS ssales#30]
 
 (55) Filter [codegen id : 20]
 Input [2]: [c_customer_sk#24, ssales#30]
-Condition : (isnotnull(ssales#30) AND (cast(ssales#30 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#31, [id=#32] as decimal(32,6)))), DecimalType(38,8))))
+Condition : (isnotnull(ssales#30) AND (cast(ssales#30 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#31, [id=#32])))
 
 (56) Project [codegen id : 20]
 Output [1]: [c_customer_sk#24]
@@ -336,16 +336,16 @@ Output [3]: [ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39]
 Input [4]: [ws_bill_customer_sk#36, ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39]
 
 (60) ReusedExchange [Reuses operator id: 71]
-Output [1]: [d_date_sk#42]
+Output [1]: [d_date_sk#40]
 
 (61) BroadcastHashJoin [codegen id : 22]
 Left keys [1]: [ws_sold_date_sk#39]
-Right keys [1]: [d_date_sk#42]
+Right keys [1]: [d_date_sk#40]
 Join condition: None
 
 (62) Project [codegen id : 22]
-Output [1]: [CheckOverflow((promote_precision(cast(ws_quantity#37 as decimal(12,2))) * promote_precision(cast(ws_list_price#38 as decimal(12,2)))), DecimalType(18,2)) AS sales#43]
-Input [4]: [ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39, d_date_sk#42]
+Output [1]: [(cast(ws_quantity#37 as decimal(10,0)) * ws_list_price#38) AS sales#41]
+Input [4]: [ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39, d_date_sk#40]
 
 (63) Union
 
@@ -353,19 +353,19 @@ Input [4]: [ws_quantity#37, ws_list_price#38, ws_sold_date_sk#39, d_date_sk#42]
 Input [1]: [sales#34]
 Keys: []
 Functions [1]: [partial_sum(sales#34)]
-Aggregate Attributes [2]: [sum#44, isEmpty#45]
-Results [2]: [sum#46, isEmpty#47]
+Aggregate Attributes [2]: [sum#42, isEmpty#43]
+Results [2]: [sum#44, isEmpty#45]
 
 (65) Exchange
-Input [2]: [sum#46, isEmpty#47]
+Input [2]: [sum#44, isEmpty#45]
 Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=8]
 
 (66) HashAggregate [codegen id : 24]
-Input [2]: [sum#46, isEmpty#47]
+Input [2]: [sum#44, isEmpty#45]
 Keys: []
 Functions [1]: [sum(sales#34)]
-Aggregate Attributes [1]: [sum(sales#34)#48]
-Results [1]: [sum(sales#34)#48 AS sum(sales)#49]
+Aggregate Attributes [1]: [sum(sales#34)#46]
+Results [1]: [sum(sales#34)#46 AS sum(sales)#47]
 
 ===== Subqueries =====
 
@@ -378,22 +378,22 @@ BroadcastExchange (71)
 
 
 (67) Scan parquet default.date_dim
-Output [3]: [d_date_sk#33, d_year#50, d_moy#51]
+Output [3]: [d_date_sk#33, d_year#48, d_moy#49]
 Batched: true
 Location [not included in comparison]/{warehouse_dir}/date_dim]
 PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)]
 ReadSchema: struct<d_date_sk:int,d_year:int,d_moy:int>
 
 (68) ColumnarToRow [codegen id : 1]
-Input [3]: [d_date_sk#33, d_year#50, d_moy#51]
+Input [3]: [d_date_sk#33, d_year#48, d_moy#49]
 
 (69) Filter [codegen id : 1]
-Input [3]: [d_date_sk#33, d_year#50, d_moy#51]
-Condition : ((((isnotnull(d_year#50) AND isnotnull(d_moy#51)) AND (d_year#50 = 2000)) AND (d_moy#51 = 2)) AND isnotnull(d_date_sk#33))
+Input [3]: [d_date_sk#33, d_year#48, d_moy#49]
+Condition : ((((isnotnull(d_year#48) AND isnotnull(d_moy#49)) AND (d_year#48 = 2000)) AND (d_moy#49 = 2)) AND isnotnull(d_date_sk#33))
 
 (70) Project [codegen id : 1]
 Output [1]: [d_date_sk#33]
-Input [3]: [d_date_sk#33, d_year#50, d_moy#51]
+Input [3]: [d_date_sk#33, d_year#48, d_moy#49]
 
 (71) BroadcastExchange
 Input [1]: [d_date_sk#33]
@@ -408,22 +408,22 @@ BroadcastExchange (76)
 
 
 (72) Scan parquet default.date_dim
-Output [3]: [d_date_sk#10, d_date#11, d_year#52]
+Output [3]: [d_date_sk#10, d_date#11, d_year#50]
 Batched: true
 Location [not included in comparison]/{warehouse_dir}/date_dim]
 PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)]
 ReadSchema: struct<d_date_sk:int,d_date:date,d_year:int>
 
 (73) ColumnarToRow [codegen id : 1]
-Input [3]: [d_date_sk#10, d_date#11, d_year#52]
+Input [3]: [d_date_sk#10, d_date#11, d_year#50]
 
 (74) Filter [codegen id : 1]
-Input [3]: [d_date_sk#10, d_date#11, d_year#52]
-Condition : (d_year#52 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10))
+Input [3]: [d_date_sk#10, d_date#11, d_year#50]
+Condition : (d_year#50 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10))
 
 (75) Project [codegen id : 1]
 Output [2]: [d_date_sk#10, d_date#11]
-Input [3]: [d_date_sk#10, d_date#11, d_year#52]
+Input [3]: [d_date_sk#10, d_date#11, d_year#50]
 
 (76) BroadcastExchange
 Input [2]: [d_date_sk#10, d_date#11]
@@ -448,81 +448,81 @@ Subquery:3 Hosting operator id = 38 Hosting Expression = Subquery scalar-subquer
 
 
 (77) Scan parquet default.store_sales
-Output [4]: [ss_customer_sk#53, ss_quantity#54, ss_sales_price#55, ss_sold_date_sk#56]
+Output [4]: [ss_customer_sk#51, ss_quantity#52, ss_sales_price#53, ss_sold_date_sk#54]
 Batched: true
 Location: InMemoryFileIndex []
-PartitionFilters: [isnotnull(ss_sold_date_sk#56), dynamicpruningexpression(ss_sold_date_sk#56 IN dynamicpruning#57)]
+PartitionFilters: [isnotnull(ss_sold_date_sk#54), dynamicpruningexpression(ss_sold_date_sk#54 IN dynamicpruning#55)]
 PushedFilters: [IsNotNull(ss_customer_sk)]
 ReadSchema: struct<ss_customer_sk:int,ss_quantity:int,ss_sales_price:decimal(7,2)>
 
 (78) ColumnarToRow [codegen id : 3]
-Input [4]: [ss_customer_sk#53, ss_quantity#54, ss_sales_price#55, ss_sold_date_sk#56]
+Input [4]: [ss_customer_sk#51, ss_quantity#52, ss_sales_price#53, ss_sold_date_sk#54]
 
 (79) Filter [codegen id : 3]
-Input [4]: [ss_customer_sk#53, ss_quantity#54, ss_sales_price#55, ss_sold_date_sk#56]
-Condition : isnotnull(ss_customer_sk#53)
+Input [4]: [ss_customer_sk#51, ss_quantity#52, ss_sales_price#53, ss_sold_date_sk#54]
+Condition : isnotnull(ss_customer_sk#51)
 
 (80) ReusedExchange [Reuses operator id: 32]
-Output [1]: [c_customer_sk#58]
+Output [1]: [c_customer_sk#56]
 
 (81) BroadcastHashJoin [codegen id : 3]
-Left keys [1]: [ss_customer_sk#53]
-Right keys [1]: [c_customer_sk#58]
+Left keys [1]: [ss_customer_sk#51]
+Right keys [1]: [c_customer_sk#56]
 Join condition: None
 
 (82) Project [codegen id : 3]
-Output [4]: [ss_quantity#54, ss_sales_price#55, ss_sold_date_sk#56, c_customer_sk#58]
-Input [5]: [ss_customer_sk#53, ss_quantity#54, ss_sales_price#55, ss_sold_date_sk#56, c_customer_sk#58]
+Output [4]: [ss_quantity#52, ss_sales_price#53, ss_sold_date_sk#54, c_customer_sk#56]
+Input [5]: [ss_customer_sk#51, ss_quantity#52, ss_sales_price#53, ss_sold_date_sk#54, c_customer_sk#56]
 
 (83) ReusedExchange [Reuses operator id: 96]
-Output [1]: [d_date_sk#59]
+Output [1]: [d_date_sk#57]
 
 (84) BroadcastHashJoin [codegen id : 3]
-Left keys [1]: [ss_sold_date_sk#56]
-Right keys [1]: [d_date_sk#59]
+Left keys [1]: [ss_sold_date_sk#54]
+Right keys [1]: [d_date_sk#57]
 Join condition: None
 
 (85) Project [codegen id : 3]
-Output [3]: [ss_quantity#54, ss_sales_price#55, c_customer_sk#58]
-Input [5]: [ss_quantity#54, ss_sales_price#55, ss_sold_date_sk#56, c_customer_sk#58, d_date_sk#59]
+Output [3]: [ss_quantity#52, ss_sales_price#53, c_customer_sk#56]
+Input [5]: [ss_quantity#52, ss_sales_price#53, ss_sold_date_sk#54, c_customer_sk#56, d_date_sk#57]
 
 (86) HashAggregate [codegen id : 3]
-Input [3]: [ss_quantity#54, ss_sales_price#55, c_customer_sk#58]
-Keys [1]: [c_customer_sk#58]
-Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(ss_quantity#54 as decimal(12,2))) * promote_precision(cast(ss_sales_price#55 as decimal(12,2)))), DecimalType(18,2)))]
-Aggregate Attributes [2]: [sum#60, isEmpty#61]
-Results [3]: [c_customer_sk#58, sum#62, isEmpty#63]
+Input [3]: [ss_quantity#52, ss_sales_price#53, c_customer_sk#56]
+Keys [1]: [c_customer_sk#56]
+Functions [1]: [partial_sum((cast(ss_quantity#52 as decimal(10,0)) * ss_sales_price#53))]
+Aggregate Attributes [2]: [sum#58, isEmpty#59]
+Results [3]: [c_customer_sk#56, sum#60, isEmpty#61]
 
 (87) Exchange
-Input [3]: [c_customer_sk#58, sum#62, isEmpty#63]
-Arguments: hashpartitioning(c_customer_sk#58, 5), ENSURE_REQUIREMENTS, [plan_id=11]
+Input [3]: [c_customer_sk#56, sum#60, isEmpty#61]
+Arguments: hashpartitioning(c_customer_sk#56, 5), ENSURE_REQUIREMENTS, [plan_id=11]
 
 (88) HashAggregate [codegen id : 4]
-Input [3]: [c_customer_sk#58, sum#62, isEmpty#63]
-Keys [1]: [c_customer_sk#58]
-Functions [1]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#54 as decimal(12,2))) * promote_precision(cast(ss_sales_price#55 as decimal(12,2)))), DecimalType(18,2)))]
-Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#54 as decimal(12,2))) * promote_precision(cast(ss_sales_price#55 as decimal(12,2)))), DecimalType(18,2)))#64]
-Results [1]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#54 as decimal(12,2))) * promote_precision(cast(ss_sales_price#55 as decimal(12,2)))), DecimalType(18,2)))#64 AS csales#65]
+Input [3]: [c_customer_sk#56, sum#60, isEmpty#61]
+Keys [1]: [c_customer_sk#56]
+Functions [1]: [sum((cast(ss_quantity#52 as decimal(10,0)) * ss_sales_price#53))]
+Aggregate Attributes [1]: [sum((cast(ss_quantity#52 as decimal(10,0)) * ss_sales_price#53))#62]
+Results [1]: [sum((cast(ss_quantity#52 as decimal(10,0)) * ss_sales_price#53))#62 AS csales#63]
 
 (89) HashAggregate [codegen id : 4]
-Input [1]: [csales#65]
+Input [1]: [csales#63]
 Keys: []
-Functions [1]: [partial_max(csales#65)]
-Aggregate Attributes [1]: [max#66]
-Results [1]: [max#67]
+Functions [1]: [partial_max(csales#63)]
+Aggregate Attributes [1]: [max#64]
+Results [1]: [max#65]
 
 (90) Exchange
-Input [1]: [max#67]
+Input [1]: [max#65]
 Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=12]
 
 (91) HashAggregate [codegen id : 5]
-Input [1]: [max#67]
+Input [1]: [max#65]
 Keys: []
-Functions [1]: [max(csales#65)]
-Aggregate Attributes [1]: [max(csales#65)#68]
-Results [1]: [max(csales#65)#68 AS tpcds_cmax#69]
+Functions [1]: [max(csales#63)]
+Aggregate Attributes [1]: [max(csales#63)#66]
+Results [1]: [max(csales#63)#66 AS tpcds_cmax#67]
 
-Subquery:4 Hosting operator id = 77 Hosting Expression = ss_sold_date_sk#56 IN dynamicpruning#57
+Subquery:4 Hosting operator id = 77 Hosting Expression = ss_sold_date_sk#54 IN dynamicpruning#55
 BroadcastExchange (96)
 +- * Project (95)
    +- * Filter (94)
@@ -531,25 +531,25 @@ BroadcastExchange (96)
 
 
 (92) Scan parquet default.date_dim
-Output [2]: [d_date_sk#59, d_year#70]
+Output [2]: [d_date_sk#57, d_year#68]
 Batched: true
 Location [not included in comparison]/{warehouse_dir}/date_dim]
 PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)]
 ReadSchema: struct<d_date_sk:int,d_year:int>
 
 (93) ColumnarToRow [codegen id : 1]
-Input [2]: [d_date_sk#59, d_year#70]
+Input [2]: [d_date_sk#57, d_year#68]
 
 (94) Filter [codegen id : 1]
-Input [2]: [d_date_sk#59, d_year#70]
-Condition : (d_year#70 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#59))
+Input [2]: [d_date_sk#57, d_year#68]
+Condition : (d_year#68 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#57))
 
 (95) Project [codegen id : 1]
-Output [1]: [d_date_sk#59]
-Input [2]: [d_date_sk#59, d_year#70]
+Output [1]: [d_date_sk#57]
+Input [2]: [d_date_sk#57, d_year#68]
 
 (96) BroadcastExchange
-Input [1]: [d_date_sk#59]
+Input [1]: [d_date_sk#57]
 Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=13]
 
 Subquery:5 Hosting operator id = 46 Hosting Expression = ws_sold_date_sk#39 IN dynamicpruning#6
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt
index d38e147d305..255e0e0cc11 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23a/simplified.txt
@@ -77,7 +77,7 @@ WholeStageCodegen (24)
                                             Exchange #10
                                               WholeStageCodegen (4)
                                                 HashAggregate [csales] [max,max]
-                                                  HashAggregate [c_customer_sk,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(ss_quantity as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2))),csales,sum,isEmpty]
+                                                  HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),csales,sum,isEmpty]
                                                     InputAdapter
                                                       Exchange [c_customer_sk] #11
                                                         WholeStageCodegen (3)
@@ -102,7 +102,7 @@ WholeStageCodegen (24)
                                                                       ReusedExchange [c_customer_sk] #9
                                                                 InputAdapter
                                                                   ReusedExchange [d_date_sk] #12
-                                    HashAggregate [c_customer_sk,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(ss_quantity as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2))),ssales,sum,isEmpty]
+                                    HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty]
                                       InputAdapter
                                         Exchange [c_customer_sk] #8
                                           WholeStageCodegen (8)
@@ -148,7 +148,7 @@ WholeStageCodegen (24)
                                 Project [c_customer_sk]
                                   Filter [ssales]
                                     ReusedSubquery [tpcds_cmax] #3
-                                    HashAggregate [c_customer_sk,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(ss_quantity as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2))),ssales,sum,isEmpty]
+                                    HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty]
                                       InputAdapter
                                         ReusedExchange [c_customer_sk,sum,isEmpty] #8
                       InputAdapter
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/explain.txt
index 49fdf208384..ffb21bf5d86 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/explain.txt
@@ -322,20 +322,20 @@ Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, c_customer_sk#
 (43) HashAggregate [codegen id : 15]
 Input [3]: [ss_quantity#21, ss_sales_price#22, c_customer_sk#24]
 Keys [1]: [c_customer_sk#24]
-Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(ss_quantity#21 as decimal(12,2))) * promote_precision(cast(ss_sales_price#22 as decimal(12,2)))), DecimalType(18,2)))]
+Functions [1]: [partial_sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))]
 Aggregate Attributes [2]: [sum#25, isEmpty#26]
 Results [3]: [c_customer_sk#24, sum#27, isEmpty#28]
 
 (44) HashAggregate [codegen id : 15]
 Input [3]: [c_customer_sk#24, sum#27, isEmpty#28]
 Keys [1]: [c_customer_sk#24]
-Functions [1]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#21 as decimal(12,2))) * promote_precision(cast(ss_sales_price#22 as decimal(12,2)))), DecimalType(18,2)))]
-Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#21 as decimal(12,2))) * promote_precision(cast(ss_sales_price#22 as decimal(12,2)))), DecimalType(18,2)))#29]
-Results [2]: [c_customer_sk#24, sum(CheckOverflow((promote_precision(cast(ss_quantity#21 as decimal(12,2))) * promote_precision(cast(ss_sales_price#22 as decimal(12,2)))), DecimalType(18,2)))#29 AS ssales#30]
+Functions [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))]
+Aggregate Attributes [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29]
+Results [2]: [c_customer_sk#24, sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29 AS ssales#30]
 
 (45) Filter [codegen id : 15]
 Input [2]: [c_customer_sk#24, ssales#30]
-Condition : (isnotnull(ssales#30) AND (cast(ssales#30 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#31, [id=#32] as decimal(32,6)))), DecimalType(38,8))))
+Condition : (isnotnull(ssales#30) AND (cast(ssales#30 as decimal(38,8)) > (0.500000 * Subquery scalar-subquery#31, [id=#32])))
 
 (46) Project [codegen id : 15]
 Output [1]: [c_customer_sk#24]
@@ -410,20 +410,20 @@ Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, c_customer_sk#
 (63) HashAggregate [codegen id : 24]
 Input [3]: [ss_quantity#21, ss_sales_price#22, c_customer_sk#24]
 Keys [1]: [c_customer_sk#24]
-Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(ss_quantity#21 as decimal(12,2))) * promote_precision(cast(ss_sales_price#22 as decimal(12,2)))), DecimalType(18,2)))]
+Functions [1]: [partial_sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))]
 Aggregate Attributes [2]: [sum#25, isEmpty#26]
 Results [3]: [c_customer_sk#24, sum#27, isEmpty#28]
 
 (64) HashAggregate [codegen id : 24]
 Input [3]: [c_customer_sk#24, sum#27, isEmpty#28]
 Keys [1]: [c_customer_sk#24]
-Functions [1]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#21 as decimal(12,2))) * promote_precision(cast(ss_sales_price#22 as decimal(12,2)))), DecimalType(18,2)))]
-Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#21 as decimal(12,2))) * promote_precision(cast(ss_sales_price#22 as decimal(12,2)))), DecimalType(18,2)))#29]
-Results [2]: [c_customer_sk#24, sum(CheckOverflow((promote_precision(cast(ss_quantity#21 as decimal(12,2))) * promote_precision(cast(ss_sales_price#22 as decimal(12,2)))), DecimalType(18,2)))#29 AS ssales#30]
+Functions [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))]
+Aggregate Attributes [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29]
+Results [2]: [c_customer_sk#24, sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29 AS ssales#30]
 
 (65) Filter [codegen id : 24]
 Input [2]: [c_customer_sk#24, ssales#30]
-Condition : (isnotnull(ssales#30) AND (cast(ssales#30 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#31, [id=#32] as decimal(32,6)))), DecimalType(38,8))))
+Condition : (isnotnull(ssales#30) AND (cast(ssales#30 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#31, [id=#32])))
 
 (66) Project [codegen id : 24]
 Output [1]: [c_customer_sk#24]
@@ -450,7 +450,7 @@ Input [6]: [cs_bill_customer_sk#1, cs_quantity#3, cs_list_price#4, c_customer_sk
 (71) HashAggregate [codegen id : 26]
 Input [4]: [cs_quantity#3, cs_list_price#4, c_first_name#35, c_last_name#36]
 Keys [2]: [c_last_name#36, c_first_name#35]
-Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cs_quantity#3 as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2)))]
+Functions [1]: [partial_sum((cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4))]
 Aggregate Attributes [2]: [sum#37, isEmpty#38]
 Results [4]: [c_last_name#36, c_first_name#35, sum#39, isEmpty#40]
 
@@ -461,9 +461,9 @@ Arguments: hashpartitioning(c_last_name#36, c_first_name#35, 5), ENSURE_REQUIREM
 (73) HashAggregate [codegen id : 27]
 Input [4]: [c_last_name#36, c_first_name#35, sum#39, isEmpty#40]
 Keys [2]: [c_last_name#36, c_first_name#35]
-Functions [1]: [sum(CheckOverflow((promote_precision(cast(cs_quantity#3 as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2)))]
-Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cs_quantity#3 as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2)))#41]
-Results [3]: [c_last_name#36, c_first_name#35, sum(CheckOverflow((promote_precision(cast(cs_quantity#3 as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2)))#41 AS sales#42]
+Functions [1]: [sum((cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4))]
+Aggregate Attributes [1]: [sum((cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4))#41]
+Results [3]: [c_last_name#36, c_first_name#35, sum((cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4))#41 AS sales#42]
 
 (74) Scan parquet default.web_sales
 Output [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47]
@@ -580,20 +580,20 @@ Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, c_customer_sk#
 (100) HashAggregate [codegen id : 42]
 Input [3]: [ss_quantity#21, ss_sales_price#22, c_customer_sk#24]
 Keys [1]: [c_customer_sk#24]
-Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(ss_quantity#21 as decimal(12,2))) * promote_precision(cast(ss_sales_price#22 as decimal(12,2)))), DecimalType(18,2)))]
-Aggregate Attributes [2]: [sum#48, isEmpty#49]
-Results [3]: [c_customer_sk#24, sum#50, isEmpty#51]
+Functions [1]: [partial_sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))]
+Aggregate Attributes [2]: [sum#25, isEmpty#26]
+Results [3]: [c_customer_sk#24, sum#27, isEmpty#28]
 
 (101) HashAggregate [codegen id : 42]
-Input [3]: [c_customer_sk#24, sum#50, isEmpty#51]
+Input [3]: [c_customer_sk#24, sum#27, isEmpty#28]
 Keys [1]: [c_customer_sk#24]
-Functions [1]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#21 as decimal(12,2))) * promote_precision(cast(ss_sales_price#22 as decimal(12,2)))), DecimalType(18,2)))]
-Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#21 as decimal(12,2))) * promote_precision(cast(ss_sales_price#22 as decimal(12,2)))), DecimalType(18,2)))#29]
-Results [2]: [c_customer_sk#24, sum(CheckOverflow((promote_precision(cast(ss_quantity#21 as decimal(12,2))) * promote_precision(cast(ss_sales_price#22 as decimal(12,2)))), DecimalType(18,2)))#29 AS ssales#30]
+Functions [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))]
+Aggregate Attributes [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29]
+Results [2]: [c_customer_sk#24, sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29 AS ssales#30]
 
 (102) Filter [codegen id : 42]
 Input [2]: [c_customer_sk#24, ssales#30]
-Condition : (isnotnull(ssales#30) AND (cast(ssales#30 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#31, [id=#32] as decimal(32,6)))), DecimalType(38,8))))
+Condition : (isnotnull(ssales#30) AND (cast(ssales#30 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#31, [id=#32])))
 
 (103) Project [codegen id : 42]
 Output [1]: [c_customer_sk#24]
@@ -609,23 +609,23 @@ Right keys [1]: [c_customer_sk#24]
 Join condition: None
 
 (106) ReusedExchange [Reuses operator id: 134]
-Output [1]: [d_date_sk#52]
+Output [1]: [d_date_sk#48]
 
 (107) BroadcastHashJoin [codegen id : 44]
 Left keys [1]: [ws_sold_date_sk#47]
-Right keys [1]: [d_date_sk#52]
+Right keys [1]: [d_date_sk#48]
 Join condition: None
 
 (108) Project [codegen id : 44]
 Output [3]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46]
-Input [5]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47, d_date_sk#52]
+Input [5]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47, d_date_sk#48]
 
 (109) ReusedExchange [Reuses operator id: 55]
-Output [3]: [c_customer_sk#53, c_first_name#54, c_last_name#55]
+Output [3]: [c_customer_sk#49, c_first_name#50, c_last_name#51]
 
 (110) Sort [codegen id : 46]
-Input [3]: [c_customer_sk#53, c_first_name#54, c_last_name#55]
-Arguments: [c_customer_sk#53 ASC NULLS FIRST], false, 0
+Input [3]: [c_customer_sk#49, c_first_name#50, c_last_name#51]
+Arguments: [c_customer_sk#49 ASC NULLS FIRST], false, 0
 
 (111) ReusedExchange [Reuses operator id: 34]
 Output [3]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22]
@@ -653,20 +653,20 @@ Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, c_customer_sk#
 (117) HashAggregate [codegen id : 51]
 Input [3]: [ss_quantity#21, ss_sales_price#22, c_customer_sk#24]
 Keys [1]: [c_customer_sk#24]
-Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(ss_quantity#21 as decimal(12,2))) * promote_precision(cast(ss_sales_price#22 as decimal(12,2)))), DecimalType(18,2)))]
-Aggregate Attributes [2]: [sum#48, isEmpty#49]
-Results [3]: [c_customer_sk#24, sum#50, isEmpty#51]
+Functions [1]: [partial_sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))]
+Aggregate Attributes [2]: [sum#25, isEmpty#26]
+Results [3]: [c_customer_sk#24, sum#27, isEmpty#28]
 
 (118) HashAggregate [codegen id : 51]
-Input [3]: [c_customer_sk#24, sum#50, isEmpty#51]
+Input [3]: [c_customer_sk#24, sum#27, isEmpty#28]
 Keys [1]: [c_customer_sk#24]
-Functions [1]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#21 as decimal(12,2))) * promote_precision(cast(ss_sales_price#22 as decimal(12,2)))), DecimalType(18,2)))]
-Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#21 as decimal(12,2))) * promote_precision(cast(ss_sales_price#22 as decimal(12,2)))), DecimalType(18,2)))#29]
-Results [2]: [c_customer_sk#24, sum(CheckOverflow((promote_precision(cast(ss_quantity#21 as decimal(12,2))) * promote_precision(cast(ss_sales_price#22 as decimal(12,2)))), DecimalType(18,2)))#29 AS ssales#30]
+Functions [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))]
+Aggregate Attributes [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29]
+Results [2]: [c_customer_sk#24, sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29 AS ssales#30]
 
 (119) Filter [codegen id : 51]
 Input [2]: [c_customer_sk#24, ssales#30]
-Condition : (isnotnull(ssales#30) AND (cast(ssales#30 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#31, [id=#32] as decimal(32,6)))), DecimalType(38,8))))
+Condition : (isnotnull(ssales#30) AND (cast(ssales#30 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#31, [id=#32])))
 
 (120) Project [codegen id : 51]
 Output [1]: [c_customer_sk#24]
@@ -677,36 +677,36 @@ Input [1]: [c_customer_sk#24]
 Arguments: [c_customer_sk#24 ASC NULLS FIRST], false, 0
 
 (122) SortMergeJoin [codegen id : 52]
-Left keys [1]: [c_customer_sk#53]
+Left keys [1]: [c_customer_sk#49]
 Right keys [1]: [c_customer_sk#24]
 Join condition: None
 
 (123) SortMergeJoin [codegen id : 53]
 Left keys [1]: [ws_bill_customer_sk#44]
-Right keys [1]: [c_customer_sk#53]
+Right keys [1]: [c_customer_sk#49]
 Join condition: None
 
 (124) Project [codegen id : 53]
-Output [4]: [ws_quantity#45, ws_list_price#46, c_first_name#54, c_last_name#55]
-Input [6]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, c_customer_sk#53, c_first_name#54, c_last_name#55]
+Output [4]: [ws_quantity#45, ws_list_price#46, c_first_name#50, c_last_name#51]
+Input [6]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, c_customer_sk#49, c_first_name#50, c_last_name#51]
 
 (125) HashAggregate [codegen id : 53]
-Input [4]: [ws_quantity#45, ws_list_price#46, c_first_name#54, c_last_name#55]
-Keys [2]: [c_last_name#55, c_first_name#54]
-Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(ws_quantity#45 as decimal(12,2))) * promote_precision(cast(ws_list_price#46 as decimal(12,2)))), DecimalType(18,2)))]
-Aggregate Attributes [2]: [sum#56, isEmpty#57]
-Results [4]: [c_last_name#55, c_first_name#54, sum#58, isEmpty#59]
+Input [4]: [ws_quantity#45, ws_list_price#46, c_first_name#50, c_last_name#51]
+Keys [2]: [c_last_name#51, c_first_name#50]
+Functions [1]: [partial_sum((cast(ws_quantity#45 as decimal(10,0)) * ws_list_price#46))]
+Aggregate Attributes [2]: [sum#52, isEmpty#53]
+Results [4]: [c_last_name#51, c_first_name#50, sum#54, isEmpty#55]
 
 (126) Exchange
-Input [4]: [c_last_name#55, c_first_name#54, sum#58, isEmpty#59]
-Arguments: hashpartitioning(c_last_name#55, c_first_name#54, 5), ENSURE_REQUIREMENTS, [plan_id=11]
+Input [4]: [c_last_name#51, c_first_name#50, sum#54, isEmpty#55]
+Arguments: hashpartitioning(c_last_name#51, c_first_name#50, 5), ENSURE_REQUIREMENTS, [plan_id=11]
 
 (127) HashAggregate [codegen id : 54]
-Input [4]: [c_last_name#55, c_first_name#54, sum#58, isEmpty#59]
-Keys [2]: [c_last_name#55, c_first_name#54]
-Functions [1]: [sum(CheckOverflow((promote_precision(cast(ws_quantity#45 as decimal(12,2))) * promote_precision(cast(ws_list_price#46 as decimal(12,2)))), DecimalType(18,2)))]
-Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(ws_quantity#45 as decimal(12,2))) * promote_precision(cast(ws_list_price#46 as decimal(12,2)))), DecimalType(18,2)))#60]
-Results [3]: [c_last_name#55, c_first_name#54, sum(CheckOverflow((promote_precision(cast(ws_quantity#45 as decimal(12,2))) * promote_precision(cast(ws_list_price#46 as decimal(12,2)))), DecimalType(18,2)))#60 AS sales#61]
+Input [4]: [c_last_name#51, c_first_name#50, sum#54, isEmpty#55]
+Keys [2]: [c_last_name#51, c_first_name#50]
+Functions [1]: [sum((cast(ws_quantity#45 as decimal(10,0)) * ws_list_price#46))]
+Aggregate Attributes [1]: [sum((cast(ws_quantity#45 as decimal(10,0)) * ws_list_price#46))#56]
+Results [3]: [c_last_name#51, c_first_name#50, sum((cast(ws_quantity#45 as decimal(10,0)) * ws_list_price#46))#56 AS sales#57]
 
 (128) Union
 
@@ -725,22 +725,22 @@ BroadcastExchange (134)
 
 
 (130) Scan parquet default.date_dim
-Output [3]: [d_date_sk#33, d_year#62, d_moy#63]
+Output [3]: [d_date_sk#33, d_year#58, d_moy#59]
 Batched: true
 Location [not included in comparison]/{warehouse_dir}/date_dim]
 PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)]
 ReadSchema: struct<d_date_sk:int,d_year:int,d_moy:int>
 
 (131) ColumnarToRow [codegen id : 1]
-Input [3]: [d_date_sk#33, d_year#62, d_moy#63]
+Input [3]: [d_date_sk#33, d_year#58, d_moy#59]
 
 (132) Filter [codegen id : 1]
-Input [3]: [d_date_sk#33, d_year#62, d_moy#63]
-Condition : ((((isnotnull(d_year#62) AND isnotnull(d_moy#63)) AND (d_year#62 = 2000)) AND (d_moy#63 = 2)) AND isnotnull(d_date_sk#33))
+Input [3]: [d_date_sk#33, d_year#58, d_moy#59]
+Condition : ((((isnotnull(d_year#58) AND isnotnull(d_moy#59)) AND (d_year#58 = 2000)) AND (d_moy#59 = 2)) AND isnotnull(d_date_sk#33))
 
 (133) Project [codegen id : 1]
 Output [1]: [d_date_sk#33]
-Input [3]: [d_date_sk#33, d_year#62, d_moy#63]
+Input [3]: [d_date_sk#33, d_year#58, d_moy#59]
 
 (134) BroadcastExchange
 Input [1]: [d_date_sk#33]
@@ -755,22 +755,22 @@ BroadcastExchange (139)
 
 
 (135) Scan parquet default.date_dim
-Output [3]: [d_date_sk#10, d_date#11, d_year#64]
+Output [3]: [d_date_sk#10, d_date#11, d_year#60]
 Batched: true
 Location [not included in comparison]/{warehouse_dir}/date_dim]
 PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)]
 ReadSchema: struct<d_date_sk:int,d_date:date,d_year:int>
 
 (136) ColumnarToRow [codegen id : 1]
-Input [3]: [d_date_sk#10, d_date#11, d_year#64]
+Input [3]: [d_date_sk#10, d_date#11, d_year#60]
 
 (137) Filter [codegen id : 1]
-Input [3]: [d_date_sk#10, d_date#11, d_year#64]
-Condition : (d_year#64 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10))
+Input [3]: [d_date_sk#10, d_date#11, d_year#60]
+Condition : (d_year#60 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10))
 
 (138) Project [codegen id : 1]
 Output [2]: [d_date_sk#10, d_date#11]
-Input [3]: [d_date_sk#10, d_date#11, d_year#64]
+Input [3]: [d_date_sk#10, d_date#11, d_year#60]
 
 (139) BroadcastExchange
 Input [2]: [d_date_sk#10, d_date#11]
@@ -797,89 +797,89 @@ Subquery:3 Hosting operator id = 45 Hosting Expression = Subquery scalar-subquer
 
 
 (140) Scan parquet default.store_sales
-Output [4]: [ss_customer_sk#65, ss_quantity#66, ss_sales_price#67, ss_sold_date_sk#68]
+Output [4]: [ss_customer_sk#61, ss_quantity#62, ss_sales_price#63, ss_sold_date_sk#64]
 Batched: true
 Location: InMemoryFileIndex []
-PartitionFilters: [isnotnull(ss_sold_date_sk#68), dynamicpruningexpression(ss_sold_date_sk#68 IN dynamicpruning#69)]
+PartitionFilters: [isnotnull(ss_sold_date_sk#64), dynamicpruningexpression(ss_sold_date_sk#64 IN dynamicpruning#65)]
 PushedFilters: [IsNotNull(ss_customer_sk)]
 ReadSchema: struct<ss_customer_sk:int,ss_quantity:int,ss_sales_price:decimal(7,2)>
 
 (141) ColumnarToRow [codegen id : 2]
-Input [4]: [ss_customer_sk#65, ss_quantity#66, ss_sales_price#67, ss_sold_date_sk#68]
+Input [4]: [ss_customer_sk#61, ss_quantity#62, ss_sales_price#63, ss_sold_date_sk#64]
 
 (142) Filter [codegen id : 2]
-Input [4]: [ss_customer_sk#65, ss_quantity#66, ss_sales_price#67, ss_sold_date_sk#68]
-Condition : isnotnull(ss_customer_sk#65)
+Input [4]: [ss_customer_sk#61, ss_quantity#62, ss_sales_price#63, ss_sold_date_sk#64]
+Condition : isnotnull(ss_customer_sk#61)
 
 (143) ReusedExchange [Reuses operator id: 161]
-Output [1]: [d_date_sk#70]
+Output [1]: [d_date_sk#66]
 
 (144) BroadcastHashJoin [codegen id : 2]
-Left keys [1]: [ss_sold_date_sk#68]
-Right keys [1]: [d_date_sk#70]
+Left keys [1]: [ss_sold_date_sk#64]
+Right keys [1]: [d_date_sk#66]
 Join condition: None
 
 (145) Project [codegen id : 2]
-Output [3]: [ss_customer_sk#65, ss_quantity#66, ss_sales_price#67]
-Input [5]: [ss_customer_sk#65, ss_quantity#66, ss_sales_price#67, ss_sold_date_sk#68, d_date_sk#70]
+Output [3]: [ss_customer_sk#61, ss_quantity#62, ss_sales_price#63]
+Input [5]: [ss_customer_sk#61, ss_quantity#62, ss_sales_price#63, ss_sold_date_sk#64, d_date_sk#66]
 
 (146) Exchange
-Input [3]: [ss_customer_sk#65, ss_quantity#66, ss_sales_price#67]
-Arguments: hashpartitioning(ss_customer_sk#65, 5), ENSURE_REQUIREMENTS, [plan_id=14]
+Input [3]: [ss_customer_sk#61, ss_quantity#62, ss_sales_price#63]
+Arguments: hashpartitioning(ss_customer_sk#61, 5), ENSURE_REQUIREMENTS, [plan_id=14]
 
 (147) Sort [codegen id : 3]
-Input [3]: [ss_customer_sk#65, ss_quantity#66, ss_sales_price#67]
-Arguments: [ss_customer_sk#65 ASC NULLS FIRST], false, 0
+Input [3]: [ss_customer_sk#61, ss_quantity#62, ss_sales_price#63]
+Arguments: [ss_customer_sk#61 ASC NULLS FIRST], false, 0
 
 (148) ReusedExchange [Reuses operator id: 39]
-Output [1]: [c_customer_sk#71]
+Output [1]: [c_customer_sk#67]
 
 (149) Sort [codegen id : 5]
-Input [1]: [c_customer_sk#71]
-Arguments: [c_customer_sk#71 ASC NULLS FIRST], false, 0
+Input [1]: [c_customer_sk#67]
+Arguments: [c_customer_sk#67 ASC NULLS FIRST], false, 0
 
 (150) SortMergeJoin [codegen id : 6]
-Left keys [1]: [ss_customer_sk#65]
-Right keys [1]: [c_customer_sk#71]
+Left keys [1]: [ss_customer_sk#61]
+Right keys [1]: [c_customer_sk#67]
 Join condition: None
 
 (151) Project [codegen id : 6]
-Output [3]: [ss_quantity#66, ss_sales_price#67, c_customer_sk#71]
-Input [4]: [ss_customer_sk#65, ss_quantity#66, ss_sales_price#67, c_customer_sk#71]
+Output [3]: [ss_quantity#62, ss_sales_price#63, c_customer_sk#67]
+Input [4]: [ss_customer_sk#61, ss_quantity#62, ss_sales_price#63, c_customer_sk#67]
 
 (152) HashAggregate [codegen id : 6]
-Input [3]: [ss_quantity#66, ss_sales_price#67, c_customer_sk#71]
-Keys [1]: [c_customer_sk#71]
-Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(ss_quantity#66 as decimal(12,2))) * promote_precision(cast(ss_sales_price#67 as decimal(12,2)))), DecimalType(18,2)))]
-Aggregate Attributes [2]: [sum#72, isEmpty#73]
-Results [3]: [c_customer_sk#71, sum#74, isEmpty#75]
+Input [3]: [ss_quantity#62, ss_sales_price#63, c_customer_sk#67]
+Keys [1]: [c_customer_sk#67]
+Functions [1]: [partial_sum((cast(ss_quantity#62 as decimal(10,0)) * ss_sales_price#63))]
+Aggregate Attributes [2]: [sum#68, isEmpty#69]
+Results [3]: [c_customer_sk#67, sum#70, isEmpty#71]
 
 (153) HashAggregate [codegen id : 6]
-Input [3]: [c_customer_sk#71, sum#74, isEmpty#75]
-Keys [1]: [c_customer_sk#71]
-Functions [1]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#66 as decimal(12,2))) * promote_precision(cast(ss_sales_price#67 as decimal(12,2)))), DecimalType(18,2)))]
-Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#66 as decimal(12,2))) * promote_precision(cast(ss_sales_price#67 as decimal(12,2)))), DecimalType(18,2)))#76]
-Results [1]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#66 as decimal(12,2))) * promote_precision(cast(ss_sales_price#67 as decimal(12,2)))), DecimalType(18,2)))#76 AS csales#77]
+Input [3]: [c_customer_sk#67, sum#70, isEmpty#71]
+Keys [1]: [c_customer_sk#67]
+Functions [1]: [sum((cast(ss_quantity#62 as decimal(10,0)) * ss_sales_price#63))]
+Aggregate Attributes [1]: [sum((cast(ss_quantity#62 as decimal(10,0)) * ss_sales_price#63))#72]
+Results [1]: [sum((cast(ss_quantity#62 as decimal(10,0)) * ss_sales_price#63))#72 AS csales#73]
 
 (154) HashAggregate [codegen id : 6]
-Input [1]: [csales#77]
+Input [1]: [csales#73]
 Keys: []
-Functions [1]: [partial_max(csales#77)]
-Aggregate Attributes [1]: [max#78]
-Results [1]: [max#79]
+Functions [1]: [partial_max(csales#73)]
+Aggregate Attributes [1]: [max#74]
+Results [1]: [max#75]
 
 (155) Exchange
-Input [1]: [max#79]
+Input [1]: [max#75]
 Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=15]
 
 (156) HashAggregate [codegen id : 7]
-Input [1]: [max#79]
+Input [1]: [max#75]
 Keys: []
-Functions [1]: [max(csales#77)]
-Aggregate Attributes [1]: [max(csales#77)#80]
-Results [1]: [max(csales#77)#80 AS tpcds_cmax#81]
+Functions [1]: [max(csales#73)]
+Aggregate Attributes [1]: [max(csales#73)#76]
+Results [1]: [max(csales#73)#76 AS tpcds_cmax#77]
 
-Subquery:4 Hosting operator id = 140 Hosting Expression = ss_sold_date_sk#68 IN dynamicpruning#69
+Subquery:4 Hosting operator id = 140 Hosting Expression = ss_sold_date_sk#64 IN dynamicpruning#65
 BroadcastExchange (161)
 +- * Project (160)
    +- * Filter (159)
@@ -888,25 +888,25 @@ BroadcastExchange (161)
 
 
 (157) Scan parquet default.date_dim
-Output [2]: [d_date_sk#70, d_year#82]
+Output [2]: [d_date_sk#66, d_year#78]
 Batched: true
 Location [not included in comparison]/{warehouse_dir}/date_dim]
 PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)]
 ReadSchema: struct<d_date_sk:int,d_year:int>
 
 (158) ColumnarToRow [codegen id : 1]
-Input [2]: [d_date_sk#70, d_year#82]
+Input [2]: [d_date_sk#66, d_year#78]
 
 (159) Filter [codegen id : 1]
-Input [2]: [d_date_sk#70, d_year#82]
-Condition : (d_year#82 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#70))
+Input [2]: [d_date_sk#66, d_year#78]
+Condition : (d_year#78 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#66))
 
 (160) Project [codegen id : 1]
-Output [1]: [d_date_sk#70]
-Input [2]: [d_date_sk#70, d_year#82]
+Output [1]: [d_date_sk#66]
+Input [2]: [d_date_sk#66, d_year#78]
 
 (161) BroadcastExchange
-Input [1]: [d_date_sk#70]
+Input [1]: [d_date_sk#66]
 Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=16]
 
 Subquery:5 Hosting operator id = 65 Hosting Expression = ReusedSubquery Subquery scalar-subquery#31, [id=#32]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/simplified.txt
index 6561fbeddef..762faed7f05 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/simplified.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b.sf100/simplified.txt
@@ -1,7 +1,7 @@
 TakeOrderedAndProject [c_last_name,c_first_name,sales]
   Union
     WholeStageCodegen (27)
-      HashAggregate [c_last_name,c_first_name,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(cs_quantity as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2))),sales,sum,isEmpty]
+      HashAggregate [c_last_name,c_first_name,sum,isEmpty] [sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),sales,sum,isEmpty]
         InputAdapter
           Exchange [c_last_name,c_first_name] #1
             WholeStageCodegen (26)
@@ -92,7 +92,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales]
                                                 Exchange #10
                                                   WholeStageCodegen (6)
                                                     HashAggregate [csales] [max,max]
-                                                      HashAggregate [c_customer_sk,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(ss_quantity as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2))),csales,sum,isEmpty]
+                                                      HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),csales,sum,isEmpty]
                                                         HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty]
                                                           Project [ss_quantity,ss_sales_price,c_customer_sk]
                                                             SortMergeJoin [ss_customer_sk,c_customer_sk]
@@ -123,7 +123,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales]
                                                                   Sort [c_customer_sk]
                                                                     InputAdapter
                                                                       ReusedExchange [c_customer_sk] #9
-                                        HashAggregate [c_customer_sk,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(ss_quantity as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2))),ssales,sum,isEmpty]
+                                        HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty]
                                           HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty]
                                             Project [ss_quantity,ss_sales_price,c_customer_sk]
                                               SortMergeJoin [ss_customer_sk,c_customer_sk]
@@ -169,7 +169,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales]
                                 Project [c_customer_sk]
                                   Filter [ssales]
                                     ReusedSubquery [tpcds_cmax] #3
-                                    HashAggregate [c_customer_sk,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(ss_quantity as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2))),ssales,sum,isEmpty]
+                                    HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty]
                                       HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty]
                                         Project [ss_quantity,ss_sales_price,c_customer_sk]
                                           SortMergeJoin [ss_customer_sk,c_customer_sk]
@@ -184,7 +184,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales]
                                                   InputAdapter
                                                     ReusedExchange [c_customer_sk] #9
     WholeStageCodegen (54)
-      HashAggregate [c_last_name,c_first_name,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(ws_quantity as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2))),sales,sum,isEmpty]
+      HashAggregate [c_last_name,c_first_name,sum,isEmpty] [sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),sales,sum,isEmpty]
         InputAdapter
           Exchange [c_last_name,c_first_name] #14
             WholeStageCodegen (53)
@@ -240,7 +240,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales]
                                     Project [c_customer_sk]
                                       Filter [ssales]
                                         ReusedSubquery [tpcds_cmax] #3
-                                        HashAggregate [c_customer_sk,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(ss_quantity as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2))),ssales,sum,isEmpty]
+                                        HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty]
                                           HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty]
                                             Project [ss_quantity,ss_sales_price,c_customer_sk]
                                               SortMergeJoin [ss_customer_sk,c_customer_sk]
@@ -270,7 +270,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales]
                                 Project [c_customer_sk]
                                   Filter [ssales]
                                     ReusedSubquery [tpcds_cmax] #3
-                                    HashAggregate [c_customer_sk,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(ss_quantity as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2))),ssales,sum,isEmpty]
+                                    HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty]
                                       HashAggregate [c_customer_sk,ss_quantity,ss_sales_price] [sum,isEmpty,sum,isEmpty]
                                         Project [ss_quantity,ss_sales_price,c_customer_sk]
                                           SortMergeJoin [ss_customer_sk,c_customer_sk]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt
index a688a3d70f2..ff7be0885d0 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/explain.txt
@@ -252,7 +252,7 @@ Input [4]: [ss_customer_sk#20, ss_quantity#21, ss_sales_price#22, c_customer_sk#
 (36) HashAggregate [codegen id : 8]
 Input [3]: [ss_quantity#21, ss_sales_price#22, c_customer_sk#24]
 Keys [1]: [c_customer_sk#24]
-Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(ss_quantity#21 as decimal(12,2))) * promote_precision(cast(ss_sales_price#22 as decimal(12,2)))), DecimalType(18,2)))]
+Functions [1]: [partial_sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))]
 Aggregate Attributes [2]: [sum#25, isEmpty#26]
 Results [3]: [c_customer_sk#24, sum#27, isEmpty#28]
 
@@ -263,13 +263,13 @@ Arguments: hashpartitioning(c_customer_sk#24, 5), ENSURE_REQUIREMENTS, [plan_id=
 (38) HashAggregate [codegen id : 9]
 Input [3]: [c_customer_sk#24, sum#27, isEmpty#28]
 Keys [1]: [c_customer_sk#24]
-Functions [1]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#21 as decimal(12,2))) * promote_precision(cast(ss_sales_price#22 as decimal(12,2)))), DecimalType(18,2)))]
-Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#21 as decimal(12,2))) * promote_precision(cast(ss_sales_price#22 as decimal(12,2)))), DecimalType(18,2)))#29]
-Results [2]: [c_customer_sk#24, sum(CheckOverflow((promote_precision(cast(ss_quantity#21 as decimal(12,2))) * promote_precision(cast(ss_sales_price#22 as decimal(12,2)))), DecimalType(18,2)))#29 AS ssales#30]
+Functions [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))]
+Aggregate Attributes [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29]
+Results [2]: [c_customer_sk#24, sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29 AS ssales#30]
 
 (39) Filter [codegen id : 9]
 Input [2]: [c_customer_sk#24, ssales#30]
-Condition : (isnotnull(ssales#30) AND (cast(ssales#30 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(Subquery scalar-subquery#31, [id=#32] as decimal(32,6)))), DecimalType(38,8))))
+Condition : (isnotnull(ssales#30) AND (cast(ssales#30 as decimal(38,8)) > (0.500000 * Subquery scalar-subquery#31, [id=#32])))
 
 (40) Project [codegen id : 9]
 Output [1]: [c_customer_sk#24]
@@ -312,13 +312,13 @@ Output [3]: [c_customer_sk#24, sum#27, isEmpty#28]
 (49) HashAggregate [codegen id : 14]
 Input [3]: [c_customer_sk#24, sum#27, isEmpty#28]
 Keys [1]: [c_customer_sk#24]
-Functions [1]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#21 as decimal(12,2))) * promote_precision(cast(ss_sales_price#22 as decimal(12,2)))), DecimalType(18,2)))]
-Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#21 as decimal(12,2))) * promote_precision(cast(ss_sales_price#22 as decimal(12,2)))), DecimalType(18,2)))#29]
-Results [2]: [c_customer_sk#24, sum(CheckOverflow((promote_precision(cast(ss_quantity#21 as decimal(12,2))) * promote_precision(cast(ss_sales_price#22 as decimal(12,2)))), DecimalType(18,2)))#29 AS ssales#30]
+Functions [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))]
+Aggregate Attributes [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29]
+Results [2]: [c_customer_sk#24, sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29 AS ssales#30]
 
 (50) Filter [codegen id : 14]
 Input [2]: [c_customer_sk#24, ssales#30]
-Condition : (isnotnull(ssales#30) AND (cast(ssales#30 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#31, [id=#32] as decimal(32,6)))), DecimalType(38,8))))
+Condition : (isnotnull(ssales#30) AND (cast(ssales#30 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#31, [id=#32])))
 
 (51) Project [codegen id : 14]
 Output [1]: [c_customer_sk#24]
@@ -361,7 +361,7 @@ Input [6]: [cs_quantity#3, cs_list_price#4, cs_sold_date_sk#5, c_first_name#34,
 (60) HashAggregate [codegen id : 17]
 Input [4]: [cs_quantity#3, cs_list_price#4, c_first_name#34, c_last_name#35]
 Keys [2]: [c_last_name#35, c_first_name#34]
-Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(cs_quantity#3 as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2)))]
+Functions [1]: [partial_sum((cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4))]
 Aggregate Attributes [2]: [sum#37, isEmpty#38]
 Results [4]: [c_last_name#35, c_first_name#34, sum#39, isEmpty#40]
 
@@ -372,9 +372,9 @@ Arguments: hashpartitioning(c_last_name#35, c_first_name#34, 5), ENSURE_REQUIREM
 (62) HashAggregate [codegen id : 18]
 Input [4]: [c_last_name#35, c_first_name#34, sum#39, isEmpty#40]
 Keys [2]: [c_last_name#35, c_first_name#34]
-Functions [1]: [sum(CheckOverflow((promote_precision(cast(cs_quantity#3 as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2)))]
-Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(cs_quantity#3 as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2)))#41]
-Results [3]: [c_last_name#35, c_first_name#34, sum(CheckOverflow((promote_precision(cast(cs_quantity#3 as decimal(12,2))) * promote_precision(cast(cs_list_price#4 as decimal(12,2)))), DecimalType(18,2)))#41 AS sales#42]
+Functions [1]: [sum((cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4))]
+Aggregate Attributes [1]: [sum((cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4))#41]
+Results [3]: [c_last_name#35, c_first_name#34, sum((cast(cs_quantity#3 as decimal(10,0)) * cs_list_price#4))#41 AS sales#42]
 
 (63) Scan parquet default.web_sales
 Output [5]: [ws_item_sk#43, ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47]
@@ -412,18 +412,18 @@ Input [4]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_da
 Arguments: [ws_bill_customer_sk#44 ASC NULLS FIRST], false, 0
 
 (71) ReusedExchange [Reuses operator id: 37]
-Output [3]: [c_customer_sk#24, sum#48, isEmpty#49]
+Output [3]: [c_customer_sk#24, sum#27, isEmpty#28]
 
 (72) HashAggregate [codegen id : 27]
-Input [3]: [c_customer_sk#24, sum#48, isEmpty#49]
+Input [3]: [c_customer_sk#24, sum#27, isEmpty#28]
 Keys [1]: [c_customer_sk#24]
-Functions [1]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#21 as decimal(12,2))) * promote_precision(cast(ss_sales_price#22 as decimal(12,2)))), DecimalType(18,2)))]
-Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#21 as decimal(12,2))) * promote_precision(cast(ss_sales_price#22 as decimal(12,2)))), DecimalType(18,2)))#29]
-Results [2]: [c_customer_sk#24, sum(CheckOverflow((promote_precision(cast(ss_quantity#21 as decimal(12,2))) * promote_precision(cast(ss_sales_price#22 as decimal(12,2)))), DecimalType(18,2)))#29 AS ssales#30]
+Functions [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))]
+Aggregate Attributes [1]: [sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29]
+Results [2]: [c_customer_sk#24, sum((cast(ss_quantity#21 as decimal(10,0)) * ss_sales_price#22))#29 AS ssales#30]
 
 (73) Filter [codegen id : 27]
 Input [2]: [c_customer_sk#24, ssales#30]
-Condition : (isnotnull(ssales#30) AND (cast(ssales#30 as decimal(38,8)) > CheckOverflow((0.500000 * promote_precision(cast(ReusedSubquery Subquery scalar-subquery#31, [id=#32] as decimal(32,6)))), DecimalType(38,8))))
+Condition : (isnotnull(ssales#30) AND (cast(ssales#30 as decimal(38,8)) > (0.500000 * ReusedSubquery Subquery scalar-subquery#31, [id=#32])))
 
 (74) Project [codegen id : 27]
 Output [1]: [c_customer_sk#24]
@@ -439,46 +439,46 @@ Right keys [1]: [c_customer_sk#24]
 Join condition: None
 
 (77) ReusedExchange [Reuses operator id: 54]
-Output [3]: [c_customer_sk#50, c_first_name#51, c_last_name#52]
+Output [3]: [c_customer_sk#48, c_first_name#49, c_last_name#50]
 
 (78) BroadcastHashJoin [codegen id : 35]
 Left keys [1]: [ws_bill_customer_sk#44]
-Right keys [1]: [c_customer_sk#50]
+Right keys [1]: [c_customer_sk#48]
 Join condition: None
 
 (79) Project [codegen id : 35]
-Output [5]: [ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47, c_first_name#51, c_last_name#52]
-Input [7]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47, c_customer_sk#50, c_first_name#51, c_last_name#52]
+Output [5]: [ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47, c_first_name#49, c_last_name#50]
+Input [7]: [ws_bill_customer_sk#44, ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47, c_customer_sk#48, c_first_name#49, c_last_name#50]
 
 (80) ReusedExchange [Reuses operator id: 92]
-Output [1]: [d_date_sk#53]
+Output [1]: [d_date_sk#51]
 
 (81) BroadcastHashJoin [codegen id : 35]
 Left keys [1]: [ws_sold_date_sk#47]
-Right keys [1]: [d_date_sk#53]
+Right keys [1]: [d_date_sk#51]
 Join condition: None
 
 (82) Project [codegen id : 35]
-Output [4]: [ws_quantity#45, ws_list_price#46, c_first_name#51, c_last_name#52]
-Input [6]: [ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47, c_first_name#51, c_last_name#52, d_date_sk#53]
+Output [4]: [ws_quantity#45, ws_list_price#46, c_first_name#49, c_last_name#50]
+Input [6]: [ws_quantity#45, ws_list_price#46, ws_sold_date_sk#47, c_first_name#49, c_last_name#50, d_date_sk#51]
 
 (83) HashAggregate [codegen id : 35]
-Input [4]: [ws_quantity#45, ws_list_price#46, c_first_name#51, c_last_name#52]
-Keys [2]: [c_last_name#52, c_first_name#51]
-Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(ws_quantity#45 as decimal(12,2))) * promote_precision(cast(ws_list_price#46 as decimal(12,2)))), DecimalType(18,2)))]
-Aggregate Attributes [2]: [sum#54, isEmpty#55]
-Results [4]: [c_last_name#52, c_first_name#51, sum#56, isEmpty#57]
+Input [4]: [ws_quantity#45, ws_list_price#46, c_first_name#49, c_last_name#50]
+Keys [2]: [c_last_name#50, c_first_name#49]
+Functions [1]: [partial_sum((cast(ws_quantity#45 as decimal(10,0)) * ws_list_price#46))]
+Aggregate Attributes [2]: [sum#52, isEmpty#53]
+Results [4]: [c_last_name#50, c_first_name#49, sum#54, isEmpty#55]
 
 (84) Exchange
-Input [4]: [c_last_name#52, c_first_name#51, sum#56, isEmpty#57]
-Arguments: hashpartitioning(c_last_name#52, c_first_name#51, 5), ENSURE_REQUIREMENTS, [plan_id=11]
+Input [4]: [c_last_name#50, c_first_name#49, sum#54, isEmpty#55]
+Arguments: hashpartitioning(c_last_name#50, c_first_name#49, 5), ENSURE_REQUIREMENTS, [plan_id=11]
 
 (85) HashAggregate [codegen id : 36]
-Input [4]: [c_last_name#52, c_first_name#51, sum#56, isEmpty#57]
-Keys [2]: [c_last_name#52, c_first_name#51]
-Functions [1]: [sum(CheckOverflow((promote_precision(cast(ws_quantity#45 as decimal(12,2))) * promote_precision(cast(ws_list_price#46 as decimal(12,2)))), DecimalType(18,2)))]
-Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(ws_quantity#45 as decimal(12,2))) * promote_precision(cast(ws_list_price#46 as decimal(12,2)))), DecimalType(18,2)))#58]
-Results [3]: [c_last_name#52, c_first_name#51, sum(CheckOverflow((promote_precision(cast(ws_quantity#45 as decimal(12,2))) * promote_precision(cast(ws_list_price#46 as decimal(12,2)))), DecimalType(18,2)))#58 AS sales#59]
+Input [4]: [c_last_name#50, c_first_name#49, sum#54, isEmpty#55]
+Keys [2]: [c_last_name#50, c_first_name#49]
+Functions [1]: [sum((cast(ws_quantity#45 as decimal(10,0)) * ws_list_price#46))]
+Aggregate Attributes [1]: [sum((cast(ws_quantity#45 as decimal(10,0)) * ws_list_price#46))#56]
+Results [3]: [c_last_name#50, c_first_name#49, sum((cast(ws_quantity#45 as decimal(10,0)) * ws_list_price#46))#56 AS sales#57]
 
 (86) Union
 
@@ -497,22 +497,22 @@ BroadcastExchange (92)
 
 
 (88) Scan parquet default.date_dim
-Output [3]: [d_date_sk#36, d_year#60, d_moy#61]
+Output [3]: [d_date_sk#36, d_year#58, d_moy#59]
 Batched: true
 Location [not included in comparison]/{warehouse_dir}/date_dim]
 PushedFilters: [IsNotNull(d_year), IsNotNull(d_moy), EqualTo(d_year,2000), EqualTo(d_moy,2), IsNotNull(d_date_sk)]
 ReadSchema: struct<d_date_sk:int,d_year:int,d_moy:int>
 
 (89) ColumnarToRow [codegen id : 1]
-Input [3]: [d_date_sk#36, d_year#60, d_moy#61]
+Input [3]: [d_date_sk#36, d_year#58, d_moy#59]
 
 (90) Filter [codegen id : 1]
-Input [3]: [d_date_sk#36, d_year#60, d_moy#61]
-Condition : ((((isnotnull(d_year#60) AND isnotnull(d_moy#61)) AND (d_year#60 = 2000)) AND (d_moy#61 = 2)) AND isnotnull(d_date_sk#36))
+Input [3]: [d_date_sk#36, d_year#58, d_moy#59]
+Condition : ((((isnotnull(d_year#58) AND isnotnull(d_moy#59)) AND (d_year#58 = 2000)) AND (d_moy#59 = 2)) AND isnotnull(d_date_sk#36))
 
 (91) Project [codegen id : 1]
 Output [1]: [d_date_sk#36]
-Input [3]: [d_date_sk#36, d_year#60, d_moy#61]
+Input [3]: [d_date_sk#36, d_year#58, d_moy#59]
 
 (92) BroadcastExchange
 Input [1]: [d_date_sk#36]
@@ -527,22 +527,22 @@ BroadcastExchange (97)
 
 
 (93) Scan parquet default.date_dim
-Output [3]: [d_date_sk#10, d_date#11, d_year#62]
+Output [3]: [d_date_sk#10, d_date#11, d_year#60]
 Batched: true
 Location [not included in comparison]/{warehouse_dir}/date_dim]
 PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)]
 ReadSchema: struct<d_date_sk:int,d_date:date,d_year:int>
 
 (94) ColumnarToRow [codegen id : 1]
-Input [3]: [d_date_sk#10, d_date#11, d_year#62]
+Input [3]: [d_date_sk#10, d_date#11, d_year#60]
 
 (95) Filter [codegen id : 1]
-Input [3]: [d_date_sk#10, d_date#11, d_year#62]
-Condition : (d_year#62 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10))
+Input [3]: [d_date_sk#10, d_date#11, d_year#60]
+Condition : (d_year#60 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#10))
 
 (96) Project [codegen id : 1]
 Output [2]: [d_date_sk#10, d_date#11]
-Input [3]: [d_date_sk#10, d_date#11, d_year#62]
+Input [3]: [d_date_sk#10, d_date#11, d_year#60]
 
 (97) BroadcastExchange
 Input [2]: [d_date_sk#10, d_date#11]
@@ -567,81 +567,81 @@ Subquery:3 Hosting operator id = 39 Hosting Expression = Subquery scalar-subquer
 
 
 (98) Scan parquet default.store_sales
-Output [4]: [ss_customer_sk#63, ss_quantity#64, ss_sales_price#65, ss_sold_date_sk#66]
+Output [4]: [ss_customer_sk#61, ss_quantity#62, ss_sales_price#63, ss_sold_date_sk#64]
 Batched: true
 Location: InMemoryFileIndex []
-PartitionFilters: [isnotnull(ss_sold_date_sk#66), dynamicpruningexpression(ss_sold_date_sk#66 IN dynamicpruning#67)]
+PartitionFilters: [isnotnull(ss_sold_date_sk#64), dynamicpruningexpression(ss_sold_date_sk#64 IN dynamicpruning#65)]
 PushedFilters: [IsNotNull(ss_customer_sk)]
 ReadSchema: struct<ss_customer_sk:int,ss_quantity:int,ss_sales_price:decimal(7,2)>
 
 (99) ColumnarToRow [codegen id : 3]
-Input [4]: [ss_customer_sk#63, ss_quantity#64, ss_sales_price#65, ss_sold_date_sk#66]
+Input [4]: [ss_customer_sk#61, ss_quantity#62, ss_sales_price#63, ss_sold_date_sk#64]
 
 (100) Filter [codegen id : 3]
-Input [4]: [ss_customer_sk#63, ss_quantity#64, ss_sales_price#65, ss_sold_date_sk#66]
-Condition : isnotnull(ss_customer_sk#63)
+Input [4]: [ss_customer_sk#61, ss_quantity#62, ss_sales_price#63, ss_sold_date_sk#64]
+Condition : isnotnull(ss_customer_sk#61)
 
 (101) ReusedExchange [Reuses operator id: 33]
-Output [1]: [c_customer_sk#68]
+Output [1]: [c_customer_sk#66]
 
 (102) BroadcastHashJoin [codegen id : 3]
-Left keys [1]: [ss_customer_sk#63]
-Right keys [1]: [c_customer_sk#68]
+Left keys [1]: [ss_customer_sk#61]
+Right keys [1]: [c_customer_sk#66]
 Join condition: None
 
 (103) Project [codegen id : 3]
-Output [4]: [ss_quantity#64, ss_sales_price#65, ss_sold_date_sk#66, c_customer_sk#68]
-Input [5]: [ss_customer_sk#63, ss_quantity#64, ss_sales_price#65, ss_sold_date_sk#66, c_customer_sk#68]
+Output [4]: [ss_quantity#62, ss_sales_price#63, ss_sold_date_sk#64, c_customer_sk#66]
+Input [5]: [ss_customer_sk#61, ss_quantity#62, ss_sales_price#63, ss_sold_date_sk#64, c_customer_sk#66]
 
 (104) ReusedExchange [Reuses operator id: 117]
-Output [1]: [d_date_sk#69]
+Output [1]: [d_date_sk#67]
 
 (105) BroadcastHashJoin [codegen id : 3]
-Left keys [1]: [ss_sold_date_sk#66]
-Right keys [1]: [d_date_sk#69]
+Left keys [1]: [ss_sold_date_sk#64]
+Right keys [1]: [d_date_sk#67]
 Join condition: None
 
 (106) Project [codegen id : 3]
-Output [3]: [ss_quantity#64, ss_sales_price#65, c_customer_sk#68]
-Input [5]: [ss_quantity#64, ss_sales_price#65, ss_sold_date_sk#66, c_customer_sk#68, d_date_sk#69]
+Output [3]: [ss_quantity#62, ss_sales_price#63, c_customer_sk#66]
+Input [5]: [ss_quantity#62, ss_sales_price#63, ss_sold_date_sk#64, c_customer_sk#66, d_date_sk#67]
 
 (107) HashAggregate [codegen id : 3]
-Input [3]: [ss_quantity#64, ss_sales_price#65, c_customer_sk#68]
-Keys [1]: [c_customer_sk#68]
-Functions [1]: [partial_sum(CheckOverflow((promote_precision(cast(ss_quantity#64 as decimal(12,2))) * promote_precision(cast(ss_sales_price#65 as decimal(12,2)))), DecimalType(18,2)))]
-Aggregate Attributes [2]: [sum#70, isEmpty#71]
-Results [3]: [c_customer_sk#68, sum#72, isEmpty#73]
+Input [3]: [ss_quantity#62, ss_sales_price#63, c_customer_sk#66]
+Keys [1]: [c_customer_sk#66]
+Functions [1]: [partial_sum((cast(ss_quantity#62 as decimal(10,0)) * ss_sales_price#63))]
+Aggregate Attributes [2]: [sum#68, isEmpty#69]
+Results [3]: [c_customer_sk#66, sum#70, isEmpty#71]
 
 (108) Exchange
-Input [3]: [c_customer_sk#68, sum#72, isEmpty#73]
-Arguments: hashpartitioning(c_customer_sk#68, 5), ENSURE_REQUIREMENTS, [plan_id=14]
+Input [3]: [c_customer_sk#66, sum#70, isEmpty#71]
+Arguments: hashpartitioning(c_customer_sk#66, 5), ENSURE_REQUIREMENTS, [plan_id=14]
 
 (109) HashAggregate [codegen id : 4]
-Input [3]: [c_customer_sk#68, sum#72, isEmpty#73]
-Keys [1]: [c_customer_sk#68]
-Functions [1]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#64 as decimal(12,2))) * promote_precision(cast(ss_sales_price#65 as decimal(12,2)))), DecimalType(18,2)))]
-Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#64 as decimal(12,2))) * promote_precision(cast(ss_sales_price#65 as decimal(12,2)))), DecimalType(18,2)))#74]
-Results [1]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#64 as decimal(12,2))) * promote_precision(cast(ss_sales_price#65 as decimal(12,2)))), DecimalType(18,2)))#74 AS csales#75]
+Input [3]: [c_customer_sk#66, sum#70, isEmpty#71]
+Keys [1]: [c_customer_sk#66]
+Functions [1]: [sum((cast(ss_quantity#62 as decimal(10,0)) * ss_sales_price#63))]
+Aggregate Attributes [1]: [sum((cast(ss_quantity#62 as decimal(10,0)) * ss_sales_price#63))#72]
+Results [1]: [sum((cast(ss_quantity#62 as decimal(10,0)) * ss_sales_price#63))#72 AS csales#73]
 
 (110) HashAggregate [codegen id : 4]
-Input [1]: [csales#75]
+Input [1]: [csales#73]
 Keys: []
-Functions [1]: [partial_max(csales#75)]
-Aggregate Attributes [1]: [max#76]
-Results [1]: [max#77]
+Functions [1]: [partial_max(csales#73)]
+Aggregate Attributes [1]: [max#74]
+Results [1]: [max#75]
 
 (111) Exchange
-Input [1]: [max#77]
+Input [1]: [max#75]
 Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=15]
 
 (112) HashAggregate [codegen id : 5]
-Input [1]: [max#77]
+Input [1]: [max#75]
 Keys: []
-Functions [1]: [max(csales#75)]
-Aggregate Attributes [1]: [max(csales#75)#78]
-Results [1]: [max(csales#75)#78 AS tpcds_cmax#79]
+Functions [1]: [max(csales#73)]
+Aggregate Attributes [1]: [max(csales#73)#76]
+Results [1]: [max(csales#73)#76 AS tpcds_cmax#77]
 
-Subquery:4 Hosting operator id = 98 Hosting Expression = ss_sold_date_sk#66 IN dynamicpruning#67
+Subquery:4 Hosting operator id = 98 Hosting Expression = ss_sold_date_sk#64 IN dynamicpruning#65
 BroadcastExchange (117)
 +- * Project (116)
    +- * Filter (115)
@@ -650,25 +650,25 @@ BroadcastExchange (117)
 
 
 (113) Scan parquet default.date_dim
-Output [2]: [d_date_sk#69, d_year#80]
+Output [2]: [d_date_sk#67, d_year#78]
 Batched: true
 Location [not included in comparison]/{warehouse_dir}/date_dim]
 PushedFilters: [In(d_year, [2000,2001,2002,2003]), IsNotNull(d_date_sk)]
 ReadSchema: struct<d_date_sk:int,d_year:int>
 
 (114) ColumnarToRow [codegen id : 1]
-Input [2]: [d_date_sk#69, d_year#80]
+Input [2]: [d_date_sk#67, d_year#78]
 
 (115) Filter [codegen id : 1]
-Input [2]: [d_date_sk#69, d_year#80]
-Condition : (d_year#80 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#69))
+Input [2]: [d_date_sk#67, d_year#78]
+Condition : (d_year#78 IN (2000,2001,2002,2003) AND isnotnull(d_date_sk#67))
 
 (116) Project [codegen id : 1]
-Output [1]: [d_date_sk#69]
-Input [2]: [d_date_sk#69, d_year#80]
+Output [1]: [d_date_sk#67]
+Input [2]: [d_date_sk#67, d_year#78]
 
 (117) BroadcastExchange
-Input [1]: [d_date_sk#69]
+Input [1]: [d_date_sk#67]
 Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=16]
 
 Subquery:5 Hosting operator id = 50 Hosting Expression = ReusedSubquery Subquery scalar-subquery#31, [id=#32]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt
index 19f5b95dce9..ca0801c0a87 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q23b/simplified.txt
@@ -1,7 +1,7 @@
 TakeOrderedAndProject [c_last_name,c_first_name,sales]
   Union
     WholeStageCodegen (18)
-      HashAggregate [c_last_name,c_first_name,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(cs_quantity as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2))),sales,sum,isEmpty]
+      HashAggregate [c_last_name,c_first_name,sum,isEmpty] [sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),sales,sum,isEmpty]
         InputAdapter
           Exchange [c_last_name,c_first_name] #1
             WholeStageCodegen (17)
@@ -78,7 +78,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales]
                                             Exchange #10
                                               WholeStageCodegen (4)
                                                 HashAggregate [csales] [max,max]
-                                                  HashAggregate [c_customer_sk,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(ss_quantity as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2))),csales,sum,isEmpty]
+                                                  HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),csales,sum,isEmpty]
                                                     InputAdapter
                                                       Exchange [c_customer_sk] #11
                                                         WholeStageCodegen (3)
@@ -103,7 +103,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales]
                                                                       ReusedExchange [c_customer_sk] #9
                                                                 InputAdapter
                                                                   ReusedExchange [d_date_sk] #12
-                                    HashAggregate [c_customer_sk,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(ss_quantity as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2))),ssales,sum,isEmpty]
+                                    HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty]
                                       InputAdapter
                                         Exchange [c_customer_sk] #8
                                           WholeStageCodegen (8)
@@ -142,13 +142,13 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales]
                                       Project [c_customer_sk]
                                         Filter [ssales]
                                           ReusedSubquery [tpcds_cmax] #3
-                                          HashAggregate [c_customer_sk,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(ss_quantity as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2))),ssales,sum,isEmpty]
+                                          HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty]
                                             InputAdapter
                                               ReusedExchange [c_customer_sk,sum,isEmpty] #8
                     InputAdapter
                       ReusedExchange [d_date_sk] #3
     WholeStageCodegen (36)
-      HashAggregate [c_last_name,c_first_name,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(ws_quantity as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2))),sales,sum,isEmpty]
+      HashAggregate [c_last_name,c_first_name,sum,isEmpty] [sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),sales,sum,isEmpty]
         InputAdapter
           Exchange [c_last_name,c_first_name] #15
             WholeStageCodegen (35)
@@ -179,7 +179,7 @@ TakeOrderedAndProject [c_last_name,c_first_name,sales]
                                 Project [c_customer_sk]
                                   Filter [ssales]
                                     ReusedSubquery [tpcds_cmax] #3
-                                    HashAggregate [c_customer_sk,sum,isEmpty] [sum(CheckOverflow((promote_precision(cast(ss_quantity as decimal(12,2))) * promote_precision(cast(ss_sales_price as decimal(12,2)))), DecimalType(18,2))),ssales,sum,isEmpty]
+                                    HashAggregate [c_customer_sk,sum,isEmpty] [sum((cast(ss_quantity as decimal(10,0)) * ss_sales_price)),ssales,sum,isEmpty]
                                       InputAdapter
                                         ReusedExchange [c_customer_sk,sum,isEmpty] #8
                         InputAdapter
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/explain.txt
index 5ee962f66e3..b12be952fae 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a.sf100/explain.txt
@@ -536,6 +536,6 @@ Input [2]: [sum#44, count#45]
 Keys: []
 Functions [1]: [avg(netpaid#31)]
 Aggregate Attributes [1]: [avg(netpaid#31)#46]
-Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#31)#46)), DecimalType(24,8)) AS (0.05 * avg(netpaid))#47]
+Results [1]: [(0.05 * avg(netpaid#31)#46) AS (0.05 * avg(netpaid))#47]
 
 
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/explain.txt
index 0ee0f297a3e..a1f8f195c83 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24a/explain.txt
@@ -412,6 +412,6 @@ Input [2]: [sum#44, count#45]
 Keys: []
 Functions [1]: [avg(netpaid#31)]
 Aggregate Attributes [1]: [avg(netpaid#31)#46]
-Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#31)#46)), DecimalType(24,8)) AS (0.05 * avg(netpaid))#47]
+Results [1]: [(0.05 * avg(netpaid#31)#46) AS (0.05 * avg(netpaid))#47]
 
 
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/explain.txt
index 9511d6c4f8e..85ace338124 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b.sf100/explain.txt
@@ -536,6 +536,6 @@ Input [2]: [sum#44, count#45]
 Keys: []
 Functions [1]: [avg(netpaid#31)]
 Aggregate Attributes [1]: [avg(netpaid#31)#46]
-Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#31)#46)), DecimalType(24,8)) AS (0.05 * avg(netpaid))#47]
+Results [1]: [(0.05 * avg(netpaid#31)#46) AS (0.05 * avg(netpaid))#47]
 
 
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/explain.txt
index cb8f5a3ade3..c7ae052a46f 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q24b/explain.txt
@@ -412,6 +412,6 @@ Input [2]: [sum#44, count#45]
 Keys: []
 Functions [1]: [avg(netpaid#31)]
 Aggregate Attributes [1]: [avg(netpaid#31)#46]
-Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#31)#46)), DecimalType(24,8)) AS (0.05 * avg(netpaid))#47]
+Results [1]: [(0.05 * avg(netpaid#31)#46) AS (0.05 * avg(netpaid))#47]
 
 
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.sf100/explain.txt
index 0c596694f83..1ea537db2f4 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30.sf100/explain.txt
@@ -287,7 +287,7 @@ Input [3]: [ctr_state#29, sum#35, count#36]
 Keys [1]: [ctr_state#29]
 Functions [1]: [avg(ctr_total_return#30)]
 Aggregate Attributes [1]: [avg(ctr_total_return#30)#37]
-Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#30)#37) * 1.200000), DecimalType(24,7)) AS (avg(ctr_total_return) * 1.2)#38, ctr_state#29 AS ctr_state#29#39]
+Results [2]: [(avg(ctr_total_return#30)#37 * 1.2) AS (avg(ctr_total_return) * 1.2)#38, ctr_state#29 AS ctr_state#29#39]
 
 (51) Filter [codegen id : 16]
 Input [2]: [(avg(ctr_total_return) * 1.2)#38, ctr_state#29#39]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/explain.txt
index a7256765c84..2a103bbae85 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q30/explain.txt
@@ -199,7 +199,7 @@ Input [3]: [ctr_state#13, sum#19, count#20]
 Keys [1]: [ctr_state#13]
 Functions [1]: [avg(ctr_total_return#14)]
 Aggregate Attributes [1]: [avg(ctr_total_return#14)#21]
-Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#14)#21) * 1.200000), DecimalType(24,7)) AS (avg(ctr_total_return) * 1.2)#22, ctr_state#13 AS ctr_state#13#23]
+Results [2]: [(avg(ctr_total_return#14)#21 * 1.2) AS (avg(ctr_total_return) * 1.2)#22, ctr_state#13 AS ctr_state#13#23]
 
 (32) Filter [codegen id : 8]
 Input [2]: [(avg(ctr_total_return) * 1.2)#22, ctr_state#13#23]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.sf100/explain.txt
index b7d0702c9e9..fded3b97c59 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31.sf100/explain.txt
@@ -599,10 +599,10 @@ Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [pla
 (107) BroadcastHashJoin [codegen id : 42]
 Left keys [1]: [ca_county#34]
 Right keys [1]: [ca_county#45]
-Join condition: ((CASE WHEN (web_sales#49 > 0.00) THEN CheckOverflow((promote_precision(web_sales#60) / promote_precision(web_sales#49)), DecimalType(37,20)) END > CASE WHEN (store_sales#37 > 0.00) THEN CheckOverflow((promote_precision(store_sales#13) / promote_precision(store_sales#37)), DecimalType(37,20)) END) AND (CASE WHEN (web_sales#60 > 0.00) THEN CheckOverflow((promote_precision(web_sales#71) / promote_precision(web_sales#60)), DecimalType(37,20)) END > CASE WHEN (store_sales#13  [...]
+Join condition: ((CASE WHEN (web_sales#49 > 0.00) THEN (web_sales#60 / web_sales#49) END > CASE WHEN (store_sales#37 > 0.00) THEN (store_sales#13 / store_sales#37) END) AND (CASE WHEN (web_sales#60 > 0.00) THEN (web_sales#71 / web_sales#60) END > CASE WHEN (store_sales#13 > 0.00) THEN (store_sales#25 / store_sales#13) END))
 
 (108) Project [codegen id : 42]
-Output [6]: [ca_county#34, d_year#31, CheckOverflow((promote_precision(web_sales#60) / promote_precision(web_sales#49)), DecimalType(37,20)) AS web_q1_q2_increase#72, CheckOverflow((promote_precision(store_sales#13) / promote_precision(store_sales#37)), DecimalType(37,20)) AS store_q1_q2_increase#73, CheckOverflow((promote_precision(web_sales#71) / promote_precision(web_sales#60)), DecimalType(37,20)) AS web_q2_q3_increase#74, CheckOverflow((promote_precision(store_sales#25) / promote_pr [...]
+Output [6]: [ca_county#34, d_year#31, (web_sales#60 / web_sales#49) AS web_q1_q2_increase#72, (store_sales#13 / store_sales#37) AS store_q1_q2_increase#73, (web_sales#71 / web_sales#60) AS web_q2_q3_increase#74, (store_sales#25 / store_sales#13) AS store_q2_q3_increase#75]
 Input [9]: [store_sales#13, store_sales#25, ca_county#34, d_year#31, store_sales#37, ca_county#45, web_sales#49, web_sales#60, web_sales#71]
 
 (109) Exchange
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/explain.txt
index 5c1e8c1c5a9..b55be4c3531 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q31/explain.txt
@@ -429,7 +429,7 @@ Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [pla
 (72) BroadcastHashJoin [codegen id : 24]
 Left keys [1]: [ca_county#45]
 Right keys [1]: [ca_county#57]
-Join condition: (CASE WHEN (web_sales#49 > 0.00) THEN CheckOverflow((promote_precision(web_sales#60) / promote_precision(web_sales#49)), DecimalType(37,20)) END > CASE WHEN (store_sales#13 > 0.00) THEN CheckOverflow((promote_precision(store_sales#25) / promote_precision(store_sales#13)), DecimalType(37,20)) END)
+Join condition: (CASE WHEN (web_sales#49 > 0.00) THEN (web_sales#60 / web_sales#49) END > CASE WHEN (store_sales#13 > 0.00) THEN (store_sales#25 / store_sales#13) END)
 
 (73) Project [codegen id : 24]
 Output [8]: [ca_county#9, d_year#6, store_sales#13, store_sales#25, store_sales#37, ca_county#45, web_sales#49, web_sales#60]
@@ -499,10 +499,10 @@ Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [pla
 (87) BroadcastHashJoin [codegen id : 24]
 Left keys [1]: [ca_county#45]
 Right keys [1]: [ca_county#68]
-Join condition: (CASE WHEN (web_sales#60 > 0.00) THEN CheckOverflow((promote_precision(web_sales#71) / promote_precision(web_sales#60)), DecimalType(37,20)) END > CASE WHEN (store_sales#25 > 0.00) THEN CheckOverflow((promote_precision(store_sales#37) / promote_precision(store_sales#25)), DecimalType(37,20)) END)
+Join condition: (CASE WHEN (web_sales#60 > 0.00) THEN (web_sales#71 / web_sales#60) END > CASE WHEN (store_sales#25 > 0.00) THEN (store_sales#37 / store_sales#25) END)
 
 (88) Project [codegen id : 24]
-Output [6]: [ca_county#9, d_year#6, CheckOverflow((promote_precision(web_sales#60) / promote_precision(web_sales#49)), DecimalType(37,20)) AS web_q1_q2_increase#72, CheckOverflow((promote_precision(store_sales#25) / promote_precision(store_sales#13)), DecimalType(37,20)) AS store_q1_q2_increase#73, CheckOverflow((promote_precision(web_sales#71) / promote_precision(web_sales#60)), DecimalType(37,20)) AS web_q2_q3_increase#74, CheckOverflow((promote_precision(store_sales#37) / promote_prec [...]
+Output [6]: [ca_county#9, d_year#6, (web_sales#60 / web_sales#49) AS web_q1_q2_increase#72, (store_sales#25 / store_sales#13) AS store_q1_q2_increase#73, (web_sales#71 / web_sales#60) AS web_q2_q3_increase#74, (store_sales#37 / store_sales#25) AS store_q2_q3_increase#75]
 Input [10]: [ca_county#9, d_year#6, store_sales#13, store_sales#25, store_sales#37, ca_county#45, web_sales#49, web_sales#60, ca_county#68, web_sales#71]
 
 (89) Exchange
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/explain.txt
index 6961e34c862..3f347759a7a 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32.sf100/explain.txt
@@ -93,7 +93,7 @@ Input [3]: [cs_item_sk#3, sum#12, count#13]
 Keys [1]: [cs_item_sk#3]
 Functions [1]: [avg(UnscaledValue(cs_ext_discount_amt#4))]
 Aggregate Attributes [1]: [avg(UnscaledValue(cs_ext_discount_amt#4))#14]
-Results [2]: [CheckOverflow((1.300000 * promote_precision(cast((avg(UnscaledValue(cs_ext_discount_amt#4))#14 / 100.0) as decimal(11,6)))), DecimalType(14,7)) AS (1.3 * avg(cs_ext_discount_amt))#15, cs_item_sk#3]
+Results [2]: [(1.3 * cast((avg(UnscaledValue(cs_ext_discount_amt#4))#14 / 100.0) as decimal(11,6))) AS (1.3 * avg(cs_ext_discount_amt))#15, cs_item_sk#3]
 
 (15) Filter
 Input [2]: [(1.3 * avg(cs_ext_discount_amt))#15, cs_item_sk#3]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt
index abbb43c8c75..3288793b4dd 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q32/explain.txt
@@ -117,7 +117,7 @@ Input [3]: [cs_item_sk#7, sum#13, count#14]
 Keys [1]: [cs_item_sk#7]
 Functions [1]: [avg(UnscaledValue(cs_ext_discount_amt#8))]
 Aggregate Attributes [1]: [avg(UnscaledValue(cs_ext_discount_amt#8))#15]
-Results [2]: [CheckOverflow((1.300000 * promote_precision(cast((avg(UnscaledValue(cs_ext_discount_amt#8))#15 / 100.0) as decimal(11,6)))), DecimalType(14,7)) AS (1.3 * avg(cs_ext_discount_amt))#16, cs_item_sk#7]
+Results [2]: [(1.3 * cast((avg(UnscaledValue(cs_ext_discount_amt#8))#15 / 100.0) as decimal(11,6))) AS (1.3 * avg(cs_ext_discount_amt))#16, cs_item_sk#7]
 
 (20) Filter [codegen id : 4]
 Input [2]: [(1.3 * avg(cs_ext_discount_amt))#16, cs_item_sk#7]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.sf100/explain.txt
index facdf679490..aff4166d2b2 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36.sf100/explain.txt
@@ -134,7 +134,7 @@ Input [5]: [i_category#13, i_class#14, spark_grouping_id#15, sum#18, sum#19]
 Keys [3]: [i_category#13, i_class#14, spark_grouping_id#15]
 Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))]
 Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#20, sum(UnscaledValue(ss_ext_sales_price#3))#21]
-Results [7]: [CheckOverflow((promote_precision(MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#20,17,2)) / promote_precision(MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#21,17,2))), DecimalType(37,20)) AS gross_margin#22, i_category#13, i_class#14, (cast((shiftright(spark_grouping_id#15, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#15, 0) & 1) as tinyint)) AS lochierarchy#23, (cast((shiftright(spark_grouping_id#15, 1) & 1) as tinyint) + cast((shiftright(spark_grouping [...]
+Results [7]: [(MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#20,17,2) / MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#21,17,2)) AS gross_margin#22, i_category#13, i_class#14, (cast((shiftright(spark_grouping_id#15, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#15, 0) & 1) as tinyint)) AS lochierarchy#23, (cast((shiftright(spark_grouping_id#15, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#15, 0) & 1) as tinyint)) AS _w1#24, CASE WHEN (cast((shiftright(spark [...]
 
 (24) Exchange
 Input [7]: [gross_margin#22, i_category#13, i_class#14, lochierarchy#23, _w1#24, _w2#25, _w3#26]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/explain.txt
index afd243d7564..e45a3c5977c 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q36/explain.txt
@@ -134,7 +134,7 @@ Input [5]: [i_category#13, i_class#14, spark_grouping_id#15, sum#18, sum#19]
 Keys [3]: [i_category#13, i_class#14, spark_grouping_id#15]
 Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))]
 Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#20, sum(UnscaledValue(ss_ext_sales_price#3))#21]
-Results [7]: [CheckOverflow((promote_precision(MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#20,17,2)) / promote_precision(MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#21,17,2))), DecimalType(37,20)) AS gross_margin#22, i_category#13, i_class#14, (cast((shiftright(spark_grouping_id#15, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#15, 0) & 1) as tinyint)) AS lochierarchy#23, (cast((shiftright(spark_grouping_id#15, 1) & 1) as tinyint) + cast((shiftright(spark_grouping [...]
+Results [7]: [(MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#20,17,2) / MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#21,17,2)) AS gross_margin#22, i_category#13, i_class#14, (cast((shiftright(spark_grouping_id#15, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#15, 0) & 1) as tinyint)) AS lochierarchy#23, (cast((shiftright(spark_grouping_id#15, 1) & 1) as tinyint) + cast((shiftright(spark_grouping_id#15, 0) & 1) as tinyint)) AS _w1#24, CASE WHEN (cast((shiftright(spark [...]
 
 (24) Exchange
 Input [7]: [gross_margin#22, i_category#13, i_class#14, lochierarchy#23, _w1#24, _w2#25, _w3#26]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.sf100/explain.txt
index 526b14d5ddd..4ef9e1fd444 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.sf100/explain.txt
@@ -188,7 +188,7 @@ Input [14]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_sales_price#3, ss_e
 (16) HashAggregate [codegen id : 6]
 Input [12]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, ss_ext_discount_amt#2, ss_ext_sales_price#3, ss_ext_wholesale_cost#4, ss_ext_list_price#5, d_year#9]
 Keys [8]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#9]
-Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#5 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#4 as decimal(8,2)))), DecimalType(8,2)) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#2 as decimal(9,2)))), DecimalType(9,2)) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#3 as decimal(10,2)))), Dec [...]
+Functions [1]: [partial_sum(((((ss_ext_list_price#5 - ss_ext_wholesale_cost#4) - ss_ext_discount_amt#2) + ss_ext_sales_price#3) / 2))]
 Aggregate Attributes [2]: [sum#18, isEmpty#19]
 Results [10]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#9, sum#20, isEmpty#21]
 
@@ -199,9 +199,9 @@ Arguments: hashpartitioning(c_customer_id#11, c_first_name#12, c_last_name#13, c
 (18) HashAggregate [codegen id : 7]
 Input [10]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#9, sum#20, isEmpty#21]
 Keys [8]: [c_customer_id#11, c_first_name#12, c_last_name#13, c_preferred_cust_flag#14, c_birth_country#15, c_login#16, c_email_address#17, d_year#9]
-Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#5 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#4 as decimal(8,2)))), DecimalType(8,2)) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#2 as decimal(9,2)))), DecimalType(9,2)) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#3 as decimal(10,2)))), DecimalType [...]
-Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#5 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#4 as decimal(8,2)))), DecimalType(8,2)) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#2 as decimal(9,2)))), DecimalType(9,2)) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#3 as decimal(10,2)))),  [...]
-Results [2]: [c_customer_id#11 AS customer_id#23, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#5 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#4 as decimal(8,2)))), DecimalType(8,2)) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#2 as decimal(9,2)))), DecimalType(9,2)) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price# [...]
+Functions [1]: [sum(((((ss_ext_list_price#5 - ss_ext_wholesale_cost#4) - ss_ext_discount_amt#2) + ss_ext_sales_price#3) / 2))]
+Aggregate Attributes [1]: [sum(((((ss_ext_list_price#5 - ss_ext_wholesale_cost#4) - ss_ext_discount_amt#2) + ss_ext_sales_price#3) / 2))#22]
+Results [2]: [c_customer_id#11 AS customer_id#23, sum(((((ss_ext_list_price#5 - ss_ext_wholesale_cost#4) - ss_ext_discount_amt#2) + ss_ext_sales_price#3) / 2))#22 AS year_total#24]
 
 (19) Filter [codegen id : 7]
 Input [2]: [customer_id#23, year_total#24]
@@ -269,7 +269,7 @@ Input [14]: [ss_customer_sk#25, ss_ext_discount_amt#26, ss_ext_sales_price#27, s
 (34) HashAggregate [codegen id : 14]
 Input [12]: [c_customer_id#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_country#39, c_login#40, c_email_address#41, ss_ext_discount_amt#26, ss_ext_sales_price#27, ss_ext_wholesale_cost#28, ss_ext_list_price#29, d_year#33]
 Keys [8]: [c_customer_id#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_country#39, c_login#40, c_email_address#41, d_year#33]
-Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#29 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#28 as decimal(8,2)))), DecimalType(8,2)) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#26 as decimal(9,2)))), DecimalType(9,2)) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#27 as decimal(10,2)))), [...]
+Functions [1]: [partial_sum(((((ss_ext_list_price#29 - ss_ext_wholesale_cost#28) - ss_ext_discount_amt#26) + ss_ext_sales_price#27) / 2))]
 Aggregate Attributes [2]: [sum#42, isEmpty#43]
 Results [10]: [c_customer_id#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_country#39, c_login#40, c_email_address#41, d_year#33, sum#44, isEmpty#45]
 
@@ -280,9 +280,9 @@ Arguments: hashpartitioning(c_customer_id#35, c_first_name#36, c_last_name#37, c
 (36) HashAggregate [codegen id : 15]
 Input [10]: [c_customer_id#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_country#39, c_login#40, c_email_address#41, d_year#33, sum#44, isEmpty#45]
 Keys [8]: [c_customer_id#35, c_first_name#36, c_last_name#37, c_preferred_cust_flag#38, c_birth_country#39, c_login#40, c_email_address#41, d_year#33]
-Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#29 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#28 as decimal(8,2)))), DecimalType(8,2)) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#26 as decimal(9,2)))), DecimalType(9,2)) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#27 as decimal(10,2)))), Decimal [...]
-Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#29 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#28 as decimal(8,2)))), DecimalType(8,2)) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#26 as decimal(9,2)))), DecimalType(9,2)) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#27 as decimal(10,2)) [...]
-Results [8]: [c_customer_id#35 AS customer_id#46, c_first_name#36 AS customer_first_name#47, c_last_name#37 AS customer_last_name#48, c_preferred_cust_flag#38 AS customer_preferred_cust_flag#49, c_birth_country#39 AS customer_birth_country#50, c_login#40 AS customer_login#51, c_email_address#41 AS customer_email_address#52, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_pri [...]
+Functions [1]: [sum(((((ss_ext_list_price#29 - ss_ext_wholesale_cost#28) - ss_ext_discount_amt#26) + ss_ext_sales_price#27) / 2))]
+Aggregate Attributes [1]: [sum(((((ss_ext_list_price#29 - ss_ext_wholesale_cost#28) - ss_ext_discount_amt#26) + ss_ext_sales_price#27) / 2))#22]
+Results [8]: [c_customer_id#35 AS customer_id#46, c_first_name#36 AS customer_first_name#47, c_last_name#37 AS customer_last_name#48, c_preferred_cust_flag#38 AS customer_preferred_cust_flag#49, c_birth_country#39 AS customer_birth_country#50, c_login#40 AS customer_login#51, c_email_address#41 AS customer_email_address#52, sum(((((ss_ext_list_price#29 - ss_ext_wholesale_cost#28) - ss_ext_discount_amt#26) + ss_ext_sales_price#27) / 2))#22 AS year_total#53]
 
 (37) Exchange
 Input [8]: [customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#53]
@@ -351,7 +351,7 @@ Input [14]: [cs_bill_customer_sk#54, cs_ext_discount_amt#55, cs_ext_sales_price#
 (52) HashAggregate [codegen id : 23]
 Input [12]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, cs_ext_discount_amt#55, cs_ext_sales_price#56, cs_ext_wholesale_cost#57, cs_ext_list_price#58, d_year#61]
 Keys [8]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, d_year#61]
-Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#58 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#57 as decimal(8,2)))), DecimalType(8,2)) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#55 as decimal(9,2)))), DecimalType(9,2)) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#56 as decimal(10,2)))), [...]
+Functions [1]: [partial_sum(((((cs_ext_list_price#58 - cs_ext_wholesale_cost#57) - cs_ext_discount_amt#55) + cs_ext_sales_price#56) / 2))]
 Aggregate Attributes [2]: [sum#70, isEmpty#71]
 Results [10]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, d_year#61, sum#72, isEmpty#73]
 
@@ -362,9 +362,9 @@ Arguments: hashpartitioning(c_customer_id#63, c_first_name#64, c_last_name#65, c
 (54) HashAggregate [codegen id : 24]
 Input [10]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, d_year#61, sum#72, isEmpty#73]
 Keys [8]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, d_year#61]
-Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#58 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#57 as decimal(8,2)))), DecimalType(8,2)) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#55 as decimal(9,2)))), DecimalType(9,2)) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#56 as decimal(10,2)))), Decimal [...]
-Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#58 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#57 as decimal(8,2)))), DecimalType(8,2)) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#55 as decimal(9,2)))), DecimalType(9,2)) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#56 as decimal(10,2)) [...]
-Results [2]: [c_customer_id#63 AS customer_id#75, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#58 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#57 as decimal(8,2)))), DecimalType(8,2)) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#55 as decimal(9,2)))), DecimalType(9,2)) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_pri [...]
+Functions [1]: [sum(((((cs_ext_list_price#58 - cs_ext_wholesale_cost#57) - cs_ext_discount_amt#55) + cs_ext_sales_price#56) / 2))]
+Aggregate Attributes [1]: [sum(((((cs_ext_list_price#58 - cs_ext_wholesale_cost#57) - cs_ext_discount_amt#55) + cs_ext_sales_price#56) / 2))#74]
+Results [2]: [c_customer_id#63 AS customer_id#75, sum(((((cs_ext_list_price#58 - cs_ext_wholesale_cost#57) - cs_ext_discount_amt#55) + cs_ext_sales_price#56) / 2))#74 AS year_total#76]
 
 (55) Filter [codegen id : 24]
 Input [2]: [customer_id#75, year_total#76]
@@ -441,7 +441,7 @@ Input [14]: [cs_bill_customer_sk#77, cs_ext_discount_amt#78, cs_ext_sales_price#
 (72) HashAggregate [codegen id : 32]
 Input [12]: [c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#90, c_login#91, c_email_address#92, cs_ext_discount_amt#78, cs_ext_sales_price#79, cs_ext_wholesale_cost#80, cs_ext_list_price#81, d_year#84]
 Keys [8]: [c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#90, c_login#91, c_email_address#92, d_year#84]
-Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#81 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#80 as decimal(8,2)))), DecimalType(8,2)) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#78 as decimal(9,2)))), DecimalType(9,2)) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#79 as decimal(10,2)))), [...]
+Functions [1]: [partial_sum(((((cs_ext_list_price#81 - cs_ext_wholesale_cost#80) - cs_ext_discount_amt#78) + cs_ext_sales_price#79) / 2))]
 Aggregate Attributes [2]: [sum#93, isEmpty#94]
 Results [10]: [c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#90, c_login#91, c_email_address#92, d_year#84, sum#95, isEmpty#96]
 
@@ -452,9 +452,9 @@ Arguments: hashpartitioning(c_customer_id#86, c_first_name#87, c_last_name#88, c
 (74) HashAggregate [codegen id : 33]
 Input [10]: [c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#90, c_login#91, c_email_address#92, d_year#84, sum#95, isEmpty#96]
 Keys [8]: [c_customer_id#86, c_first_name#87, c_last_name#88, c_preferred_cust_flag#89, c_birth_country#90, c_login#91, c_email_address#92, d_year#84]
-Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#81 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#80 as decimal(8,2)))), DecimalType(8,2)) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#78 as decimal(9,2)))), DecimalType(9,2)) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#79 as decimal(10,2)))), Decimal [...]
-Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#81 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#80 as decimal(8,2)))), DecimalType(8,2)) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#78 as decimal(9,2)))), DecimalType(9,2)) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#79 as decimal(10,2)) [...]
-Results [2]: [c_customer_id#86 AS customer_id#97, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#81 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#80 as decimal(8,2)))), DecimalType(8,2)) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#78 as decimal(9,2)))), DecimalType(9,2)) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_pri [...]
+Functions [1]: [sum(((((cs_ext_list_price#81 - cs_ext_wholesale_cost#80) - cs_ext_discount_amt#78) + cs_ext_sales_price#79) / 2))]
+Aggregate Attributes [1]: [sum(((((cs_ext_list_price#81 - cs_ext_wholesale_cost#80) - cs_ext_discount_amt#78) + cs_ext_sales_price#79) / 2))#74]
+Results [2]: [c_customer_id#86 AS customer_id#97, sum(((((cs_ext_list_price#81 - cs_ext_wholesale_cost#80) - cs_ext_discount_amt#78) + cs_ext_sales_price#79) / 2))#74 AS year_total#98]
 
 (75) Exchange
 Input [2]: [customer_id#97, year_total#98]
@@ -467,7 +467,7 @@ Arguments: [customer_id#97 ASC NULLS FIRST], false, 0
 (77) SortMergeJoin [codegen id : 35]
 Left keys [1]: [customer_id#23]
 Right keys [1]: [customer_id#97]
-Join condition: (CASE WHEN (year_total#76 > 0.000000) THEN CheckOverflow((promote_precision(year_total#98) / promote_precision(year_total#76)), DecimalType(38,14)) END > CASE WHEN (year_total#24 > 0.000000) THEN CheckOverflow((promote_precision(year_total#53) / promote_precision(year_total#24)), DecimalType(38,14)) END)
+Join condition: (CASE WHEN (year_total#76 > 0.000000) THEN (year_total#98 / year_total#76) END > CASE WHEN (year_total#24 > 0.000000) THEN (year_total#53 / year_total#24) END)
 
 (78) Project [codegen id : 35]
 Output [10]: [customer_id#23, customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#76, year_total#98]
@@ -527,7 +527,7 @@ Input [14]: [ws_bill_customer_sk#99, ws_ext_discount_amt#100, ws_ext_sales_price
 (91) HashAggregate [codegen id : 41]
 Input [12]: [c_customer_id#108, c_first_name#109, c_last_name#110, c_preferred_cust_flag#111, c_birth_country#112, c_login#113, c_email_address#114, ws_ext_discount_amt#100, ws_ext_sales_price#101, ws_ext_wholesale_cost#102, ws_ext_list_price#103, d_year#106]
 Keys [8]: [c_customer_id#108, c_first_name#109, c_last_name#110, c_preferred_cust_flag#111, c_birth_country#112, c_login#113, c_email_address#114, d_year#106]
-Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#103 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#102 as decimal(8,2)))), DecimalType(8,2)) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#100 as decimal(9,2)))), DecimalType(9,2)) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#101 as decimal(10,2) [...]
+Functions [1]: [partial_sum(((((ws_ext_list_price#103 - ws_ext_wholesale_cost#102) - ws_ext_discount_amt#100) + ws_ext_sales_price#101) / 2))]
 Aggregate Attributes [2]: [sum#115, isEmpty#116]
 Results [10]: [c_customer_id#108, c_first_name#109, c_last_name#110, c_preferred_cust_flag#111, c_birth_country#112, c_login#113, c_email_address#114, d_year#106, sum#117, isEmpty#118]
 
@@ -538,9 +538,9 @@ Arguments: hashpartitioning(c_customer_id#108, c_first_name#109, c_last_name#110
 (93) HashAggregate [codegen id : 42]
 Input [10]: [c_customer_id#108, c_first_name#109, c_last_name#110, c_preferred_cust_flag#111, c_birth_country#112, c_login#113, c_email_address#114, d_year#106, sum#117, isEmpty#118]
 Keys [8]: [c_customer_id#108, c_first_name#109, c_last_name#110, c_preferred_cust_flag#111, c_birth_country#112, c_login#113, c_email_address#114, d_year#106]
-Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#103 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#102 as decimal(8,2)))), DecimalType(8,2)) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#100 as decimal(9,2)))), DecimalType(9,2)) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#101 as decimal(10,2)))), Dec [...]
-Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#103 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#102 as decimal(8,2)))), DecimalType(8,2)) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#100 as decimal(9,2)))), DecimalType(9,2)) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#101 as decimal(10 [...]
-Results [2]: [c_customer_id#108 AS customer_id#120, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#103 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#102 as decimal(8,2)))), DecimalType(8,2)) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#100 as decimal(9,2)))), DecimalType(9,2)) as decimal(10,2))) + promote_precision(cast(ws_ext_sale [...]
+Functions [1]: [sum(((((ws_ext_list_price#103 - ws_ext_wholesale_cost#102) - ws_ext_discount_amt#100) + ws_ext_sales_price#101) / 2))]
+Aggregate Attributes [1]: [sum(((((ws_ext_list_price#103 - ws_ext_wholesale_cost#102) - ws_ext_discount_amt#100) + ws_ext_sales_price#101) / 2))#119]
+Results [2]: [c_customer_id#108 AS customer_id#120, sum(((((ws_ext_list_price#103 - ws_ext_wholesale_cost#102) - ws_ext_discount_amt#100) + ws_ext_sales_price#101) / 2))#119 AS year_total#121]
 
 (94) Filter [codegen id : 42]
 Input [2]: [customer_id#120, year_total#121]
@@ -617,7 +617,7 @@ Input [14]: [ws_bill_customer_sk#122, ws_ext_discount_amt#123, ws_ext_sales_pric
 (111) HashAggregate [codegen id : 50]
 Input [12]: [c_customer_id#131, c_first_name#132, c_last_name#133, c_preferred_cust_flag#134, c_birth_country#135, c_login#136, c_email_address#137, ws_ext_discount_amt#123, ws_ext_sales_price#124, ws_ext_wholesale_cost#125, ws_ext_list_price#126, d_year#129]
 Keys [8]: [c_customer_id#131, c_first_name#132, c_last_name#133, c_preferred_cust_flag#134, c_birth_country#135, c_login#136, c_email_address#137, d_year#129]
-Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#126 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#125 as decimal(8,2)))), DecimalType(8,2)) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#123 as decimal(9,2)))), DecimalType(9,2)) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#124 as decimal(10,2) [...]
+Functions [1]: [partial_sum(((((ws_ext_list_price#126 - ws_ext_wholesale_cost#125) - ws_ext_discount_amt#123) + ws_ext_sales_price#124) / 2))]
 Aggregate Attributes [2]: [sum#138, isEmpty#139]
 Results [10]: [c_customer_id#131, c_first_name#132, c_last_name#133, c_preferred_cust_flag#134, c_birth_country#135, c_login#136, c_email_address#137, d_year#129, sum#140, isEmpty#141]
 
@@ -628,9 +628,9 @@ Arguments: hashpartitioning(c_customer_id#131, c_first_name#132, c_last_name#133
 (113) HashAggregate [codegen id : 51]
 Input [10]: [c_customer_id#131, c_first_name#132, c_last_name#133, c_preferred_cust_flag#134, c_birth_country#135, c_login#136, c_email_address#137, d_year#129, sum#140, isEmpty#141]
 Keys [8]: [c_customer_id#131, c_first_name#132, c_last_name#133, c_preferred_cust_flag#134, c_birth_country#135, c_login#136, c_email_address#137, d_year#129]
-Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#126 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#125 as decimal(8,2)))), DecimalType(8,2)) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#123 as decimal(9,2)))), DecimalType(9,2)) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#124 as decimal(10,2)))), Dec [...]
-Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#126 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#125 as decimal(8,2)))), DecimalType(8,2)) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#123 as decimal(9,2)))), DecimalType(9,2)) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#124 as decimal(10 [...]
-Results [2]: [c_customer_id#131 AS customer_id#142, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#126 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#125 as decimal(8,2)))), DecimalType(8,2)) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#123 as decimal(9,2)))), DecimalType(9,2)) as decimal(10,2))) + promote_precision(cast(ws_ext_sale [...]
+Functions [1]: [sum(((((ws_ext_list_price#126 - ws_ext_wholesale_cost#125) - ws_ext_discount_amt#123) + ws_ext_sales_price#124) / 2))]
+Aggregate Attributes [1]: [sum(((((ws_ext_list_price#126 - ws_ext_wholesale_cost#125) - ws_ext_discount_amt#123) + ws_ext_sales_price#124) / 2))#119]
+Results [2]: [c_customer_id#131 AS customer_id#142, sum(((((ws_ext_list_price#126 - ws_ext_wholesale_cost#125) - ws_ext_discount_amt#123) + ws_ext_sales_price#124) / 2))#119 AS year_total#143]
 
 (114) Exchange
 Input [2]: [customer_id#142, year_total#143]
@@ -643,7 +643,7 @@ Arguments: [customer_id#142 ASC NULLS FIRST], false, 0
 (116) SortMergeJoin [codegen id : 53]
 Left keys [1]: [customer_id#23]
 Right keys [1]: [customer_id#142]
-Join condition: (CASE WHEN (year_total#76 > 0.000000) THEN CheckOverflow((promote_precision(year_total#98) / promote_precision(year_total#76)), DecimalType(38,14)) END > CASE WHEN (year_total#121 > 0.000000) THEN CheckOverflow((promote_precision(year_total#143) / promote_precision(year_total#121)), DecimalType(38,14)) END)
+Join condition: (CASE WHEN (year_total#76 > 0.000000) THEN (year_total#98 / year_total#76) END > CASE WHEN (year_total#121 > 0.000000) THEN (year_total#143 / year_total#121) END)
 
 (117) Project [codegen id : 53]
 Output [7]: [customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.sf100/simplified.txt
index e8e55fe5757..6aae289dda5 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.sf100/simplified.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4.sf100/simplified.txt
@@ -24,7 +24,7 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom
                                               Exchange [customer_id] #1
                                                 WholeStageCodegen (7)
                                                   Filter [year_total]
-                                                    HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost as decimal(8,2)))), DecimalType(8,2)) as decimal(9,2))) - promote_pre [...]
+                                                    HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((ss_ext_list_price - ss_ext_wholesale_cost) - ss_ext_discount_amt) + ss_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty]
                                                       InputAdapter
                                                         Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #2
                                                           WholeStageCodegen (6)
@@ -68,7 +68,7 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom
                                             InputAdapter
                                               Exchange [customer_id] #6
                                                 WholeStageCodegen (15)
-                                                  HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost as decimal(8,2)))), DecimalType(8,2)) as decimal(9,2))) - promote_preci [...]
+                                                  HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((ss_ext_list_price - ss_ext_wholesale_cost) - ss_ext_discount_amt) + ss_ext_sales_price) / 2)),customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,sum,isEmpty]
                                                     InputAdapter
                                                       Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #7
                                                         WholeStageCodegen (14)
@@ -108,7 +108,7 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom
                                         Exchange [customer_id] #10
                                           WholeStageCodegen (24)
                                             Filter [year_total]
-                                              HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost as decimal(8,2)))), DecimalType(8,2)) as decimal(9,2))) - promote_precision [...]
+                                              HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((cs_ext_list_price - cs_ext_wholesale_cost) - cs_ext_discount_amt) + cs_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty]
                                                 InputAdapter
                                                   Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #11
                                                     WholeStageCodegen (23)
@@ -141,7 +141,7 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom
                               InputAdapter
                                 Exchange [customer_id] #13
                                   WholeStageCodegen (33)
-                                    HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost as decimal(8,2)))), DecimalType(8,2)) as decimal(9,2))) - promote_precision(cast(cs_e [...]
+                                    HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((cs_ext_list_price - cs_ext_wholesale_cost) - cs_ext_discount_amt) + cs_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty]
                                       InputAdapter
                                         Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #14
                                           WholeStageCodegen (32)
@@ -175,7 +175,7 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom
                         Exchange [customer_id] #16
                           WholeStageCodegen (42)
                             Filter [year_total]
-                              HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost as decimal(8,2)))), DecimalType(8,2)) as decimal(9,2))) - promote_precision(cast(ws_ext_dis [...]
+                              HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((ws_ext_list_price - ws_ext_wholesale_cost) - ws_ext_discount_amt) + ws_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty]
                                 InputAdapter
                                   Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #17
                                     WholeStageCodegen (41)
@@ -208,7 +208,7 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom
               InputAdapter
                 Exchange [customer_id] #19
                   WholeStageCodegen (51)
-                    HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost as decimal(8,2)))), DecimalType(8,2)) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt  [...]
+                    HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((ws_ext_list_price - ws_ext_wholesale_cost) - ws_ext_discount_amt) + ws_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty]
                       InputAdapter
                         Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #20
                           WholeStageCodegen (50)
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/explain.txt
index 5ff09a7763c..f0176957230 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/explain.txt
@@ -166,7 +166,7 @@ Input [14]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_fl
 (13) HashAggregate [codegen id : 3]
 Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_sales_price#11, ss_ext_wholesale_cost#12, ss_ext_list_price#13, d_year#17]
 Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17]
-Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#13 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#12 as decimal(8,2)))), DecimalType(8,2)) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#10 as decimal(9,2)))), DecimalType(9,2)) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#11 as decimal(10,2)))), [...]
+Functions [1]: [partial_sum(((((ss_ext_list_price#13 - ss_ext_wholesale_cost#12) - ss_ext_discount_amt#10) + ss_ext_sales_price#11) / 2))]
 Aggregate Attributes [2]: [sum#18, isEmpty#19]
 Results [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17, sum#20, isEmpty#21]
 
@@ -177,9 +177,9 @@ Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, c_pr
 (15) HashAggregate [codegen id : 24]
 Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17, sum#20, isEmpty#21]
 Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, d_year#17]
-Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#13 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#12 as decimal(8,2)))), DecimalType(8,2)) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#10 as decimal(9,2)))), DecimalType(9,2)) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#11 as decimal(10,2)))), Decimal [...]
-Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#13 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#12 as decimal(8,2)))), DecimalType(8,2)) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#10 as decimal(9,2)))), DecimalType(9,2)) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#11 as decimal(10,2)) [...]
-Results [2]: [c_customer_id#2 AS customer_id#23, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#13 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#12 as decimal(8,2)))), DecimalType(8,2)) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#10 as decimal(9,2)))), DecimalType(9,2)) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_pric [...]
+Functions [1]: [sum(((((ss_ext_list_price#13 - ss_ext_wholesale_cost#12) - ss_ext_discount_amt#10) + ss_ext_sales_price#11) / 2))]
+Aggregate Attributes [1]: [sum(((((ss_ext_list_price#13 - ss_ext_wholesale_cost#12) - ss_ext_discount_amt#10) + ss_ext_sales_price#11) / 2))#22]
+Results [2]: [c_customer_id#2 AS customer_id#23, sum(((((ss_ext_list_price#13 - ss_ext_wholesale_cost#12) - ss_ext_discount_amt#10) + ss_ext_sales_price#11) / 2))#22 AS year_total#24]
 
 (16) Filter [codegen id : 24]
 Input [2]: [customer_id#23, year_total#24]
@@ -242,7 +242,7 @@ Input [14]: [c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust
 (29) HashAggregate [codegen id : 6]
 Input [12]: [c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, ss_ext_discount_amt#34, ss_ext_sales_price#35, ss_ext_wholesale_cost#36, ss_ext_list_price#37, d_year#41]
 Keys [8]: [c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, d_year#41]
-Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#37 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#36 as decimal(8,2)))), DecimalType(8,2)) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#34 as decimal(9,2)))), DecimalType(9,2)) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#35 as decimal(10,2)))), [...]
+Functions [1]: [partial_sum(((((ss_ext_list_price#37 - ss_ext_wholesale_cost#36) - ss_ext_discount_amt#34) + ss_ext_sales_price#35) / 2))]
 Aggregate Attributes [2]: [sum#42, isEmpty#43]
 Results [10]: [c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, d_year#41, sum#44, isEmpty#45]
 
@@ -253,9 +253,9 @@ Arguments: hashpartitioning(c_customer_id#26, c_first_name#27, c_last_name#28, c
 (31) HashAggregate [codegen id : 7]
 Input [10]: [c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, d_year#41, sum#44, isEmpty#45]
 Keys [8]: [c_customer_id#26, c_first_name#27, c_last_name#28, c_preferred_cust_flag#29, c_birth_country#30, c_login#31, c_email_address#32, d_year#41]
-Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#37 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#36 as decimal(8,2)))), DecimalType(8,2)) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#34 as decimal(9,2)))), DecimalType(9,2)) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#35 as decimal(10,2)))), Decimal [...]
-Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price#37 as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost#36 as decimal(8,2)))), DecimalType(8,2)) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_amt#34 as decimal(9,2)))), DecimalType(9,2)) as decimal(10,2))) + promote_precision(cast(ss_ext_sales_price#35 as decimal(10,2)) [...]
-Results [8]: [c_customer_id#26 AS customer_id#46, c_first_name#27 AS customer_first_name#47, c_last_name#28 AS customer_last_name#48, c_preferred_cust_flag#29 AS customer_preferred_cust_flag#49, c_birth_country#30 AS customer_birth_country#50, c_login#31 AS customer_login#51, c_email_address#32 AS customer_email_address#52, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_pri [...]
+Functions [1]: [sum(((((ss_ext_list_price#37 - ss_ext_wholesale_cost#36) - ss_ext_discount_amt#34) + ss_ext_sales_price#35) / 2))]
+Aggregate Attributes [1]: [sum(((((ss_ext_list_price#37 - ss_ext_wholesale_cost#36) - ss_ext_discount_amt#34) + ss_ext_sales_price#35) / 2))#22]
+Results [8]: [c_customer_id#26 AS customer_id#46, c_first_name#27 AS customer_first_name#47, c_last_name#28 AS customer_last_name#48, c_preferred_cust_flag#29 AS customer_preferred_cust_flag#49, c_birth_country#30 AS customer_birth_country#50, c_login#31 AS customer_login#51, c_email_address#32 AS customer_email_address#52, sum(((((ss_ext_list_price#37 - ss_ext_wholesale_cost#36) - ss_ext_discount_amt#34) + ss_ext_sales_price#35) / 2))#22 AS year_total#53]
 
 (32) BroadcastExchange
 Input [8]: [customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#53]
@@ -323,7 +323,7 @@ Input [14]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust
 (46) HashAggregate [codegen id : 10]
 Input [12]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, cs_ext_discount_amt#63, cs_ext_sales_price#64, cs_ext_wholesale_cost#65, cs_ext_list_price#66, d_year#69]
 Keys [8]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, d_year#69]
-Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#66 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#65 as decimal(8,2)))), DecimalType(8,2)) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#63 as decimal(9,2)))), DecimalType(9,2)) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#64 as decimal(10,2)))), [...]
+Functions [1]: [partial_sum(((((cs_ext_list_price#66 - cs_ext_wholesale_cost#65) - cs_ext_discount_amt#63) + cs_ext_sales_price#64) / 2))]
 Aggregate Attributes [2]: [sum#70, isEmpty#71]
 Results [10]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, d_year#69, sum#72, isEmpty#73]
 
@@ -334,9 +334,9 @@ Arguments: hashpartitioning(c_customer_id#55, c_first_name#56, c_last_name#57, c
 (48) HashAggregate [codegen id : 11]
 Input [10]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, d_year#69, sum#72, isEmpty#73]
 Keys [8]: [c_customer_id#55, c_first_name#56, c_last_name#57, c_preferred_cust_flag#58, c_birth_country#59, c_login#60, c_email_address#61, d_year#69]
-Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#66 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#65 as decimal(8,2)))), DecimalType(8,2)) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#63 as decimal(9,2)))), DecimalType(9,2)) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#64 as decimal(10,2)))), Decimal [...]
-Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#66 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#65 as decimal(8,2)))), DecimalType(8,2)) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#63 as decimal(9,2)))), DecimalType(9,2)) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#64 as decimal(10,2)) [...]
-Results [2]: [c_customer_id#55 AS customer_id#75, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#66 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#65 as decimal(8,2)))), DecimalType(8,2)) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#63 as decimal(9,2)))), DecimalType(9,2)) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_pri [...]
+Functions [1]: [sum(((((cs_ext_list_price#66 - cs_ext_wholesale_cost#65) - cs_ext_discount_amt#63) + cs_ext_sales_price#64) / 2))]
+Aggregate Attributes [1]: [sum(((((cs_ext_list_price#66 - cs_ext_wholesale_cost#65) - cs_ext_discount_amt#63) + cs_ext_sales_price#64) / 2))#74]
+Results [2]: [c_customer_id#55 AS customer_id#75, sum(((((cs_ext_list_price#66 - cs_ext_wholesale_cost#65) - cs_ext_discount_amt#63) + cs_ext_sales_price#64) / 2))#74 AS year_total#76]
 
 (49) Filter [codegen id : 11]
 Input [2]: [customer_id#75, year_total#76]
@@ -412,7 +412,7 @@ Input [14]: [c_customer_id#78, c_first_name#79, c_last_name#80, c_preferred_cust
 (65) HashAggregate [codegen id : 14]
 Input [12]: [c_customer_id#78, c_first_name#79, c_last_name#80, c_preferred_cust_flag#81, c_birth_country#82, c_login#83, c_email_address#84, cs_ext_discount_amt#86, cs_ext_sales_price#87, cs_ext_wholesale_cost#88, cs_ext_list_price#89, d_year#92]
 Keys [8]: [c_customer_id#78, c_first_name#79, c_last_name#80, c_preferred_cust_flag#81, c_birth_country#82, c_login#83, c_email_address#84, d_year#92]
-Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#89 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#88 as decimal(8,2)))), DecimalType(8,2)) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#86 as decimal(9,2)))), DecimalType(9,2)) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#87 as decimal(10,2)))), [...]
+Functions [1]: [partial_sum(((((cs_ext_list_price#89 - cs_ext_wholesale_cost#88) - cs_ext_discount_amt#86) + cs_ext_sales_price#87) / 2))]
 Aggregate Attributes [2]: [sum#93, isEmpty#94]
 Results [10]: [c_customer_id#78, c_first_name#79, c_last_name#80, c_preferred_cust_flag#81, c_birth_country#82, c_login#83, c_email_address#84, d_year#92, sum#95, isEmpty#96]
 
@@ -423,9 +423,9 @@ Arguments: hashpartitioning(c_customer_id#78, c_first_name#79, c_last_name#80, c
 (67) HashAggregate [codegen id : 15]
 Input [10]: [c_customer_id#78, c_first_name#79, c_last_name#80, c_preferred_cust_flag#81, c_birth_country#82, c_login#83, c_email_address#84, d_year#92, sum#95, isEmpty#96]
 Keys [8]: [c_customer_id#78, c_first_name#79, c_last_name#80, c_preferred_cust_flag#81, c_birth_country#82, c_login#83, c_email_address#84, d_year#92]
-Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#89 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#88 as decimal(8,2)))), DecimalType(8,2)) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#86 as decimal(9,2)))), DecimalType(9,2)) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#87 as decimal(10,2)))), Decimal [...]
-Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#89 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#88 as decimal(8,2)))), DecimalType(8,2)) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#86 as decimal(9,2)))), DecimalType(9,2)) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_price#87 as decimal(10,2)) [...]
-Results [2]: [c_customer_id#78 AS customer_id#97, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price#89 as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost#88 as decimal(8,2)))), DecimalType(8,2)) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_amt#86 as decimal(9,2)))), DecimalType(9,2)) as decimal(10,2))) + promote_precision(cast(cs_ext_sales_pri [...]
+Functions [1]: [sum(((((cs_ext_list_price#89 - cs_ext_wholesale_cost#88) - cs_ext_discount_amt#86) + cs_ext_sales_price#87) / 2))]
+Aggregate Attributes [1]: [sum(((((cs_ext_list_price#89 - cs_ext_wholesale_cost#88) - cs_ext_discount_amt#86) + cs_ext_sales_price#87) / 2))#74]
+Results [2]: [c_customer_id#78 AS customer_id#97, sum(((((cs_ext_list_price#89 - cs_ext_wholesale_cost#88) - cs_ext_discount_amt#86) + cs_ext_sales_price#87) / 2))#74 AS year_total#98]
 
 (68) BroadcastExchange
 Input [2]: [customer_id#97, year_total#98]
@@ -434,7 +434,7 @@ Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [pla
 (69) BroadcastHashJoin [codegen id : 24]
 Left keys [1]: [customer_id#23]
 Right keys [1]: [customer_id#97]
-Join condition: (CASE WHEN (year_total#76 > 0.000000) THEN CheckOverflow((promote_precision(year_total#98) / promote_precision(year_total#76)), DecimalType(38,14)) END > CASE WHEN (year_total#24 > 0.000000) THEN CheckOverflow((promote_precision(year_total#53) / promote_precision(year_total#24)), DecimalType(38,14)) END)
+Join condition: (CASE WHEN (year_total#76 > 0.000000) THEN (year_total#98 / year_total#76) END > CASE WHEN (year_total#24 > 0.000000) THEN (year_total#53 / year_total#24) END)
 
 (70) Project [codegen id : 24]
 Output [10]: [customer_id#23, customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52, year_total#76, year_total#98]
@@ -497,7 +497,7 @@ Input [14]: [c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_c
 (83) HashAggregate [codegen id : 18]
 Input [12]: [c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106, ws_ext_discount_amt#108, ws_ext_sales_price#109, ws_ext_wholesale_cost#110, ws_ext_list_price#111, d_year#114]
 Keys [8]: [c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106, d_year#114]
-Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#111 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#110 as decimal(8,2)))), DecimalType(8,2)) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#108 as decimal(9,2)))), DecimalType(9,2)) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#109 as decimal(10,2) [...]
+Functions [1]: [partial_sum(((((ws_ext_list_price#111 - ws_ext_wholesale_cost#110) - ws_ext_discount_amt#108) + ws_ext_sales_price#109) / 2))]
 Aggregate Attributes [2]: [sum#115, isEmpty#116]
 Results [10]: [c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106, d_year#114, sum#117, isEmpty#118]
 
@@ -508,9 +508,9 @@ Arguments: hashpartitioning(c_customer_id#100, c_first_name#101, c_last_name#102
 (85) HashAggregate [codegen id : 19]
 Input [10]: [c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106, d_year#114, sum#117, isEmpty#118]
 Keys [8]: [c_customer_id#100, c_first_name#101, c_last_name#102, c_preferred_cust_flag#103, c_birth_country#104, c_login#105, c_email_address#106, d_year#114]
-Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#111 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#110 as decimal(8,2)))), DecimalType(8,2)) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#108 as decimal(9,2)))), DecimalType(9,2)) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#109 as decimal(10,2)))), Dec [...]
-Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#111 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#110 as decimal(8,2)))), DecimalType(8,2)) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#108 as decimal(9,2)))), DecimalType(9,2)) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#109 as decimal(10 [...]
-Results [2]: [c_customer_id#100 AS customer_id#120, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#111 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#110 as decimal(8,2)))), DecimalType(8,2)) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#108 as decimal(9,2)))), DecimalType(9,2)) as decimal(10,2))) + promote_precision(cast(ws_ext_sale [...]
+Functions [1]: [sum(((((ws_ext_list_price#111 - ws_ext_wholesale_cost#110) - ws_ext_discount_amt#108) + ws_ext_sales_price#109) / 2))]
+Aggregate Attributes [1]: [sum(((((ws_ext_list_price#111 - ws_ext_wholesale_cost#110) - ws_ext_discount_amt#108) + ws_ext_sales_price#109) / 2))#119]
+Results [2]: [c_customer_id#100 AS customer_id#120, sum(((((ws_ext_list_price#111 - ws_ext_wholesale_cost#110) - ws_ext_discount_amt#108) + ws_ext_sales_price#109) / 2))#119 AS year_total#121]
 
 (86) Filter [codegen id : 19]
 Input [2]: [customer_id#120, year_total#121]
@@ -586,7 +586,7 @@ Input [14]: [c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_c
 (102) HashAggregate [codegen id : 22]
 Input [12]: [c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#127, c_login#128, c_email_address#129, ws_ext_discount_amt#131, ws_ext_sales_price#132, ws_ext_wholesale_cost#133, ws_ext_list_price#134, d_year#137]
 Keys [8]: [c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#127, c_login#128, c_email_address#129, d_year#137]
-Functions [1]: [partial_sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#134 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#133 as decimal(8,2)))), DecimalType(8,2)) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#131 as decimal(9,2)))), DecimalType(9,2)) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#132 as decimal(10,2) [...]
+Functions [1]: [partial_sum(((((ws_ext_list_price#134 - ws_ext_wholesale_cost#133) - ws_ext_discount_amt#131) + ws_ext_sales_price#132) / 2))]
 Aggregate Attributes [2]: [sum#138, isEmpty#139]
 Results [10]: [c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#127, c_login#128, c_email_address#129, d_year#137, sum#140, isEmpty#141]
 
@@ -597,9 +597,9 @@ Arguments: hashpartitioning(c_customer_id#123, c_first_name#124, c_last_name#125
 (104) HashAggregate [codegen id : 23]
 Input [10]: [c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#127, c_login#128, c_email_address#129, d_year#137, sum#140, isEmpty#141]
 Keys [8]: [c_customer_id#123, c_first_name#124, c_last_name#125, c_preferred_cust_flag#126, c_birth_country#127, c_login#128, c_email_address#129, d_year#137]
-Functions [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#134 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#133 as decimal(8,2)))), DecimalType(8,2)) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#131 as decimal(9,2)))), DecimalType(9,2)) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#132 as decimal(10,2)))), Dec [...]
-Aggregate Attributes [1]: [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#134 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#133 as decimal(8,2)))), DecimalType(8,2)) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#131 as decimal(9,2)))), DecimalType(9,2)) as decimal(10,2))) + promote_precision(cast(ws_ext_sales_price#132 as decimal(10 [...]
-Results [2]: [c_customer_id#123 AS customer_id#142, sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price#134 as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost#133 as decimal(8,2)))), DecimalType(8,2)) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt#131 as decimal(9,2)))), DecimalType(9,2)) as decimal(10,2))) + promote_precision(cast(ws_ext_sale [...]
+Functions [1]: [sum(((((ws_ext_list_price#134 - ws_ext_wholesale_cost#133) - ws_ext_discount_amt#131) + ws_ext_sales_price#132) / 2))]
+Aggregate Attributes [1]: [sum(((((ws_ext_list_price#134 - ws_ext_wholesale_cost#133) - ws_ext_discount_amt#131) + ws_ext_sales_price#132) / 2))#119]
+Results [2]: [c_customer_id#123 AS customer_id#142, sum(((((ws_ext_list_price#134 - ws_ext_wholesale_cost#133) - ws_ext_discount_amt#131) + ws_ext_sales_price#132) / 2))#119 AS year_total#143]
 
 (105) BroadcastExchange
 Input [2]: [customer_id#142, year_total#143]
@@ -608,7 +608,7 @@ Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [pla
 (106) BroadcastHashJoin [codegen id : 24]
 Left keys [1]: [customer_id#23]
 Right keys [1]: [customer_id#142]
-Join condition: (CASE WHEN (year_total#76 > 0.000000) THEN CheckOverflow((promote_precision(year_total#98) / promote_precision(year_total#76)), DecimalType(38,14)) END > CASE WHEN (year_total#121 > 0.000000) THEN CheckOverflow((promote_precision(year_total#143) / promote_precision(year_total#121)), DecimalType(38,14)) END)
+Join condition: (CASE WHEN (year_total#76 > 0.000000) THEN (year_total#98 / year_total#76) END > CASE WHEN (year_total#121 > 0.000000) THEN (year_total#143 / year_total#121) END)
 
 (107) Project [codegen id : 24]
 Output [7]: [customer_id#46, customer_first_name#47, customer_last_name#48, customer_preferred_cust_flag#49, customer_birth_country#50, customer_login#51, customer_email_address#52]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/simplified.txt
index 67afe29952d..4b47ade1934 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/simplified.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q4/simplified.txt
@@ -10,7 +10,7 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom
                   BroadcastHashJoin [customer_id,customer_id]
                     BroadcastHashJoin [customer_id,customer_id]
                       Filter [year_total]
-                        HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost as decimal(8,2)))), DecimalType(8,2)) as decimal(9,2))) - promote_precision(cast(ss_ext_discount_ [...]
+                        HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((ss_ext_list_price - ss_ext_wholesale_cost) - ss_ext_discount_amt) + ss_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty]
                           InputAdapter
                             Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #1
                               WholeStageCodegen (3)
@@ -42,7 +42,7 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom
                       InputAdapter
                         BroadcastExchange #4
                           WholeStageCodegen (7)
-                            HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_ext_list_price as decimal(8,2))) - promote_precision(cast(ss_ext_wholesale_cost as decimal(8,2)))), DecimalType(8,2)) as decimal(9,2))) - promote_precision(cast(ss_ext_disco [...]
+                            HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((ss_ext_list_price - ss_ext_wholesale_cost) - ss_ext_discount_amt) + ss_ext_sales_price) / 2)),customer_id,customer_first_name,customer_last_name,customer_preferred_cust_flag,customer_birth_country,customer_login,customer_email_address,year_total,sum,isEmpty]
                               InputAdapter
                                 Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #5
                                   WholeStageCodegen (6)
@@ -75,7 +75,7 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom
                       BroadcastExchange #8
                         WholeStageCodegen (11)
                           Filter [year_total]
-                            HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost as decimal(8,2)))), DecimalType(8,2)) as decimal(9,2))) - promote_precision(cast(cs_ext_disco [...]
+                            HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((cs_ext_list_price - cs_ext_wholesale_cost) - cs_ext_discount_amt) + cs_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty]
                               InputAdapter
                                 Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #9
                                   WholeStageCodegen (10)
@@ -101,7 +101,7 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom
                 InputAdapter
                   BroadcastExchange #11
                     WholeStageCodegen (15)
-                      HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cs_ext_list_price as decimal(8,2))) - promote_precision(cast(cs_ext_wholesale_cost as decimal(8,2)))), DecimalType(8,2)) as decimal(9,2))) - promote_precision(cast(cs_ext_discount_am [...]
+                      HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((cs_ext_list_price - cs_ext_wholesale_cost) - cs_ext_discount_amt) + cs_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty]
                         InputAdapter
                           Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #12
                             WholeStageCodegen (14)
@@ -128,7 +128,7 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom
               BroadcastExchange #14
                 WholeStageCodegen (19)
                   Filter [year_total]
-                    HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost as decimal(8,2)))), DecimalType(8,2)) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt  [...]
+                    HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((ws_ext_list_price - ws_ext_wholesale_cost) - ws_ext_discount_amt) + ws_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty]
                       InputAdapter
                         Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #15
                           WholeStageCodegen (18)
@@ -154,7 +154,7 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom
         InputAdapter
           BroadcastExchange #17
             WholeStageCodegen (23)
-              HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ws_ext_list_price as decimal(8,2))) - promote_precision(cast(ws_ext_wholesale_cost as decimal(8,2)))), DecimalType(8,2)) as decimal(9,2))) - promote_precision(cast(ws_ext_discount_amt as dec [...]
+              HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum,isEmpty] [sum(((((ws_ext_list_price - ws_ext_wholesale_cost) - ws_ext_discount_amt) + ws_ext_sales_price) / 2)),customer_id,year_total,sum,isEmpty]
                 InputAdapter
                   Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #18
                     WholeStageCodegen (22)
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.sf100/explain.txt
index 2f648b3ac91..45c26cccc93 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.sf100/explain.txt
@@ -165,7 +165,7 @@ Input [7]: [cs_warehouse_sk#1, cs_sales_price#4, cr_refunded_cash#11, i_item_id#
 (30) HashAggregate [codegen id : 8]
 Input [5]: [cs_sales_price#4, cr_refunded_cash#11, w_state#19, i_item_id#14, d_date#17]
 Keys [2]: [w_state#19, i_item_id#14]
-Functions [2]: [partial_sum(CASE WHEN (d_date#17 < 2000-03-11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#4 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_refunded_cash#11 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2)) ELSE 0.00 END), partial_sum(CASE WHEN (d_date#17 >= 2000-03-11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#4 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_refunded_cash#11 as decimal(12,2)), 0.00) [...]
+Functions [2]: [partial_sum(CASE WHEN (d_date#17 < 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#11 as decimal(12,2)), 0.00)) ELSE 0.00 END), partial_sum(CASE WHEN (d_date#17 >= 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#11 as decimal(12,2)), 0.00)) ELSE 0.00 END)]
 Aggregate Attributes [4]: [sum#20, isEmpty#21, sum#22, isEmpty#23]
 Results [6]: [w_state#19, i_item_id#14, sum#24, isEmpty#25, sum#26, isEmpty#27]
 
@@ -176,9 +176,9 @@ Arguments: hashpartitioning(w_state#19, i_item_id#14, 5), ENSURE_REQUIREMENTS, [
 (32) HashAggregate [codegen id : 9]
 Input [6]: [w_state#19, i_item_id#14, sum#24, isEmpty#25, sum#26, isEmpty#27]
 Keys [2]: [w_state#19, i_item_id#14]
-Functions [2]: [sum(CASE WHEN (d_date#17 < 2000-03-11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#4 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_refunded_cash#11 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2)) ELSE 0.00 END), sum(CASE WHEN (d_date#17 >= 2000-03-11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#4 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_refunded_cash#11 as decimal(12,2)), 0.00) as decimal(13,2 [...]
-Aggregate Attributes [2]: [sum(CASE WHEN (d_date#17 < 2000-03-11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#4 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_refunded_cash#11 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2)) ELSE 0.00 END)#28, sum(CASE WHEN (d_date#17 >= 2000-03-11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#4 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_refunded_cash#11 as decimal(12,2)), 0.00) a [...]
-Results [4]: [w_state#19, i_item_id#14, sum(CASE WHEN (d_date#17 < 2000-03-11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#4 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_refunded_cash#11 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2)) ELSE 0.00 END)#28 AS sales_before#30, sum(CASE WHEN (d_date#17 >= 2000-03-11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#4 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_refunded_ca [...]
+Functions [2]: [sum(CASE WHEN (d_date#17 < 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#11 as decimal(12,2)), 0.00)) ELSE 0.00 END), sum(CASE WHEN (d_date#17 >= 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#11 as decimal(12,2)), 0.00)) ELSE 0.00 END)]
+Aggregate Attributes [2]: [sum(CASE WHEN (d_date#17 < 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#11 as decimal(12,2)), 0.00)) ELSE 0.00 END)#28, sum(CASE WHEN (d_date#17 >= 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#11 as decimal(12,2)), 0.00)) ELSE 0.00 END)#29]
+Results [4]: [w_state#19, i_item_id#14, sum(CASE WHEN (d_date#17 < 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#11 as decimal(12,2)), 0.00)) ELSE 0.00 END)#28 AS sales_before#30, sum(CASE WHEN (d_date#17 >= 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#11 as decimal(12,2)), 0.00)) ELSE 0.00 END)#29 AS sales_after#31]
 
 (33) TakeOrderedAndProject
 Input [4]: [w_state#19, i_item_id#14, sales_before#30, sales_after#31]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.sf100/simplified.txt
index 5574ad5fa6e..855e9ba4aed 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.sf100/simplified.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40.sf100/simplified.txt
@@ -1,6 +1,6 @@
 TakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after]
   WholeStageCodegen (9)
-    HashAggregate [w_state,i_item_id,sum,isEmpty,sum,isEmpty] [sum(CASE WHEN (d_date < 2000-03-11) THEN CheckOverflow((promote_precision(cast(cs_sales_price as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_refunded_cash as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2)) ELSE 0.00 END),sum(CASE WHEN (d_date >= 2000-03-11) THEN CheckOverflow((promote_precision(cast(cs_sales_price as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_refunded_cash as decimal [...]
+    HashAggregate [w_state,i_item_id,sum,isEmpty,sum,isEmpty] [sum(CASE WHEN (d_date < 2000-03-11) THEN (cs_sales_price - coalesce(cast(cr_refunded_cash as decimal(12,2)), 0.00)) ELSE 0.00 END),sum(CASE WHEN (d_date >= 2000-03-11) THEN (cs_sales_price - coalesce(cast(cr_refunded_cash as decimal(12,2)), 0.00)) ELSE 0.00 END),sales_before,sales_after,sum,isEmpty,sum,isEmpty]
       InputAdapter
         Exchange [w_state,i_item_id] #1
           WholeStageCodegen (8)
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/explain.txt
index 5ea29201e6c..e88e0ee4e2c 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/explain.txt
@@ -165,7 +165,7 @@ Input [7]: [cs_sales_price#4, cs_sold_date_sk#5, cr_refunded_cash#9, w_state#12,
 (30) HashAggregate [codegen id : 8]
 Input [5]: [cs_sales_price#4, cr_refunded_cash#9, w_state#12, i_item_id#14, d_date#17]
 Keys [2]: [w_state#12, i_item_id#14]
-Functions [2]: [partial_sum(CASE WHEN (d_date#17 < 2000-03-11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#4 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2)) ELSE 0.00 END), partial_sum(CASE WHEN (d_date#17 >= 2000-03-11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#4 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00) a [...]
+Functions [2]: [partial_sum(CASE WHEN (d_date#17 < 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END), partial_sum(CASE WHEN (d_date#17 >= 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END)]
 Aggregate Attributes [4]: [sum#18, isEmpty#19, sum#20, isEmpty#21]
 Results [6]: [w_state#12, i_item_id#14, sum#22, isEmpty#23, sum#24, isEmpty#25]
 
@@ -176,9 +176,9 @@ Arguments: hashpartitioning(w_state#12, i_item_id#14, 5), ENSURE_REQUIREMENTS, [
 (32) HashAggregate [codegen id : 9]
 Input [6]: [w_state#12, i_item_id#14, sum#22, isEmpty#23, sum#24, isEmpty#25]
 Keys [2]: [w_state#12, i_item_id#14]
-Functions [2]: [sum(CASE WHEN (d_date#17 < 2000-03-11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#4 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2)) ELSE 0.00 END), sum(CASE WHEN (d_date#17 >= 2000-03-11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#4 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00) as decimal(13,2)) [...]
-Aggregate Attributes [2]: [sum(CASE WHEN (d_date#17 < 2000-03-11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#4 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2)) ELSE 0.00 END)#26, sum(CASE WHEN (d_date#17 >= 2000-03-11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#4 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00) as  [...]
-Results [4]: [w_state#12, i_item_id#14, sum(CASE WHEN (d_date#17 < 2000-03-11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#4 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2)) ELSE 0.00 END)#26 AS sales_before#28, sum(CASE WHEN (d_date#17 >= 2000-03-11) THEN CheckOverflow((promote_precision(cast(cs_sales_price#4 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_refunded_cas [...]
+Functions [2]: [sum(CASE WHEN (d_date#17 < 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END), sum(CASE WHEN (d_date#17 >= 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END)]
+Aggregate Attributes [2]: [sum(CASE WHEN (d_date#17 < 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END)#26, sum(CASE WHEN (d_date#17 >= 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END)#27]
+Results [4]: [w_state#12, i_item_id#14, sum(CASE WHEN (d_date#17 < 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END)#26 AS sales_before#28, sum(CASE WHEN (d_date#17 >= 2000-03-11) THEN (cs_sales_price#4 - coalesce(cast(cr_refunded_cash#9 as decimal(12,2)), 0.00)) ELSE 0.00 END)#27 AS sales_after#29]
 
 (33) TakeOrderedAndProject
 Input [4]: [w_state#12, i_item_id#14, sales_before#28, sales_after#29]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/simplified.txt
index 206317e8a52..a0878fad163 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/simplified.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q40/simplified.txt
@@ -1,6 +1,6 @@
 TakeOrderedAndProject [w_state,i_item_id,sales_before,sales_after]
   WholeStageCodegen (9)
-    HashAggregate [w_state,i_item_id,sum,isEmpty,sum,isEmpty] [sum(CASE WHEN (d_date < 2000-03-11) THEN CheckOverflow((promote_precision(cast(cs_sales_price as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_refunded_cash as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2)) ELSE 0.00 END),sum(CASE WHEN (d_date >= 2000-03-11) THEN CheckOverflow((promote_precision(cast(cs_sales_price as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_refunded_cash as decimal [...]
+    HashAggregate [w_state,i_item_id,sum,isEmpty,sum,isEmpty] [sum(CASE WHEN (d_date < 2000-03-11) THEN (cs_sales_price - coalesce(cast(cr_refunded_cash as decimal(12,2)), 0.00)) ELSE 0.00 END),sum(CASE WHEN (d_date >= 2000-03-11) THEN (cs_sales_price - coalesce(cast(cr_refunded_cash as decimal(12,2)), 0.00)) ELSE 0.00 END),sales_before,sales_after,sum,isEmpty,sum,isEmpty]
       InputAdapter
         Exchange [w_state,i_item_id] #1
           WholeStageCodegen (8)
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.sf100/explain.txt
index afe71668e80..a76d52b8b15 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44.sf100/explain.txt
@@ -70,7 +70,7 @@ Results [2]: [ss_item_sk#1 AS item_sk#10, cast((avg(UnscaledValue(ss_net_profit#
 
 (8) Filter [codegen id : 2]
 Input [2]: [item_sk#10, rank_col#11]
-Condition : (isnotnull(rank_col#11) AND (cast(rank_col#11 as decimal(13,7)) > CheckOverflow((0.900000 * promote_precision(Subquery scalar-subquery#12, [id=#13])), DecimalType(13,7))))
+Condition : (isnotnull(rank_col#11) AND (cast(rank_col#11 as decimal(13,7)) > (0.9 * Subquery scalar-subquery#12, [id=#13])))
 
 (9) Exchange
 Input [2]: [item_sk#10, rank_col#11]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt
index 32534fa455e..2e0534aa11c 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q44/explain.txt
@@ -71,7 +71,7 @@ Results [2]: [ss_item_sk#1 AS item_sk#10, cast((avg(UnscaledValue(ss_net_profit#
 
 (8) Filter [codegen id : 2]
 Input [2]: [item_sk#10, rank_col#11]
-Condition : (isnotnull(rank_col#11) AND (cast(rank_col#11 as decimal(13,7)) > CheckOverflow((0.900000 * promote_precision(Subquery scalar-subquery#12, [id=#13])), DecimalType(13,7))))
+Condition : (isnotnull(rank_col#11) AND (cast(rank_col#11 as decimal(13,7)) > (0.9 * Subquery scalar-subquery#12, [id=#13])))
 
 (9) Exchange
 Input [2]: [item_sk#10, rank_col#11]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.sf100/explain.txt
index 0dad98fba19..b1e3dd728ce 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47.sf100/explain.txt
@@ -186,7 +186,7 @@ Arguments: [avg(_w0#19) windowspecdefinition(i_category#14, i_brand#13, s_store_
 
 (30) Filter [codegen id : 11]
 Input [10]: [i_category#14, i_brand#13, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21]
-Condition : ((isnotnull(avg_monthly_sales#21) AND (avg_monthly_sales#21 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#18 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#21 as decimal(22,6)))), DecimalType(22,6)))) / promote_precision(cast(avg_monthly_sales#21 as decimal(22,6)))), DecimalType(38,16)) > 0.1000000000000000))
+Condition : (((isnotnull(avg_monthly_sales#21) AND isnotnull(sum_sales#18)) AND (avg_monthly_sales#21 > 0.000000)) AND ((abs((sum_sales#18 - avg_monthly_sales#21)) / avg_monthly_sales#21) > 0.1000000000000000))
 
 (31) Project [codegen id : 11]
 Output [9]: [i_category#14, i_brand#13, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#18, avg_monthly_sales#21, rn#20]
@@ -277,7 +277,7 @@ Input [16]: [i_category#14, i_brand#13, s_store_name#10, s_company_name#11, d_ye
 
 (52) TakeOrderedAndProject
 Input [10]: [i_category#14, i_brand#13, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41]
-Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#18 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#21 as decimal(22,6)))), DecimalType(22,6)) ASC NULLS FIRST, s_store_name#10 ASC NULLS FIRST], [i_category#14, i_brand#13, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41]
+Arguments: 100, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, s_store_name#10 ASC NULLS FIRST], [i_category#14, i_brand#13, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41]
 
 ===== Subqueries =====
 
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/explain.txt
index e55defd7ff6..c1a1ee6a4da 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q47/explain.txt
@@ -167,7 +167,7 @@ Arguments: [avg(_w0#19) windowspecdefinition(i_category#3, i_brand#2, s_store_na
 
 (27) Filter [codegen id : 22]
 Input [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21]
-Condition : ((isnotnull(avg_monthly_sales#21) AND (avg_monthly_sales#21 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#18 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#21 as decimal(22,6)))), DecimalType(22,6)))) / promote_precision(cast(avg_monthly_sales#21 as decimal(22,6)))), DecimalType(38,16)) > 0.1000000000000000))
+Condition : (((isnotnull(avg_monthly_sales#21) AND isnotnull(sum_sales#18)) AND (avg_monthly_sales#21 > 0.000000)) AND ((abs((sum_sales#18 - avg_monthly_sales#21)) / avg_monthly_sales#21) > 0.1000000000000000))
 
 (28) Project [codegen id : 22]
 Output [9]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, avg_monthly_sales#21, rn#20]
@@ -242,7 +242,7 @@ Input [16]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year
 
 (45) TakeOrderedAndProject
 Input [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41]
-Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#18 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#21 as decimal(22,6)))), DecimalType(22,6)) ASC NULLS FIRST, s_store_name#13 ASC NULLS FIRST], [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41]
+Arguments: 100, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, s_store_name#13 ASC NULLS FIRST], [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41]
 
 ===== Subqueries =====
 
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.sf100/explain.txt
index a73794d2681..52f0b021ae2 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49.sf100/explain.txt
@@ -177,7 +177,7 @@ Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25
 Keys [1]: [ws_item_sk#1]
 Functions [4]: [sum(coalesce(wr_return_quantity#11, 0)), sum(coalesce(ws_quantity#3, 0)), sum(coalesce(cast(wr_return_amt#12 as decimal(12,2)), 0.00)), sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))]
 Aggregate Attributes [4]: [sum(coalesce(wr_return_quantity#11, 0))#26, sum(coalesce(ws_quantity#3, 0))#27, sum(coalesce(cast(wr_return_amt#12 as decimal(12,2)), 0.00))#28, sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))#29]
-Results [3]: [ws_item_sk#1 AS item#30, CheckOverflow((promote_precision(cast(sum(coalesce(wr_return_quantity#11, 0))#26 as decimal(15,4))) / promote_precision(cast(sum(coalesce(ws_quantity#3, 0))#27 as decimal(15,4)))), DecimalType(35,20)) AS return_ratio#31, CheckOverflow((promote_precision(cast(sum(coalesce(cast(wr_return_amt#12 as decimal(12,2)), 0.00))#28 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))#29 as decimal(15,4)))), Deci [...]
+Results [3]: [ws_item_sk#1 AS item#30, (cast(sum(coalesce(wr_return_quantity#11, 0))#26 as decimal(15,4)) / cast(sum(coalesce(ws_quantity#3, 0))#27 as decimal(15,4))) AS return_ratio#31, (cast(sum(coalesce(cast(wr_return_amt#12 as decimal(12,2)), 0.00))#28 as decimal(15,4)) / cast(sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))#29 as decimal(15,4))) AS currency_ratio#32]
 
 (21) Exchange
 Input [3]: [item#30, return_ratio#31, currency_ratio#32]
@@ -297,7 +297,7 @@ Input [7]: [cs_item_sk#36, sum#54, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#5
 Keys [1]: [cs_item_sk#36]
 Functions [4]: [sum(coalesce(cr_return_quantity#45, 0)), sum(coalesce(cs_quantity#38, 0)), sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))]
 Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#45, 0))#60, sum(coalesce(cs_quantity#38, 0))#61, sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00))#62, sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))#63]
-Results [3]: [cs_item_sk#36 AS item#64, CheckOverflow((promote_precision(cast(sum(coalesce(cr_return_quantity#45, 0))#60 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cs_quantity#38, 0))#61 as decimal(15,4)))), DecimalType(35,20)) AS return_ratio#65, CheckOverflow((promote_precision(cast(sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00))#62 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))#63 as decimal(15,4)))) [...]
+Results [3]: [cs_item_sk#36 AS item#64, (cast(sum(coalesce(cr_return_quantity#45, 0))#60 as decimal(15,4)) / cast(sum(coalesce(cs_quantity#38, 0))#61 as decimal(15,4))) AS return_ratio#65, (cast(sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00))#62 as decimal(15,4)) / cast(sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))#63 as decimal(15,4))) AS currency_ratio#66]
 
 (48) Exchange
 Input [3]: [item#64, return_ratio#65, currency_ratio#66]
@@ -417,7 +417,7 @@ Input [7]: [ss_item_sk#70, sum#88, sum#89, sum#90, isEmpty#91, sum#92, isEmpty#9
 Keys [1]: [ss_item_sk#70]
 Functions [4]: [sum(coalesce(sr_return_quantity#79, 0)), sum(coalesce(ss_quantity#72, 0)), sum(coalesce(cast(sr_return_amt#80 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#73 as decimal(12,2)), 0.00))]
 Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#79, 0))#94, sum(coalesce(ss_quantity#72, 0))#95, sum(coalesce(cast(sr_return_amt#80 as decimal(12,2)), 0.00))#96, sum(coalesce(cast(ss_net_paid#73 as decimal(12,2)), 0.00))#97]
-Results [3]: [ss_item_sk#70 AS item#98, CheckOverflow((promote_precision(cast(sum(coalesce(sr_return_quantity#79, 0))#94 as decimal(15,4))) / promote_precision(cast(sum(coalesce(ss_quantity#72, 0))#95 as decimal(15,4)))), DecimalType(35,20)) AS return_ratio#99, CheckOverflow((promote_precision(cast(sum(coalesce(cast(sr_return_amt#80 as decimal(12,2)), 0.00))#96 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(ss_net_paid#73 as decimal(12,2)), 0.00))#97 as decimal(15,4)))), D [...]
+Results [3]: [ss_item_sk#70 AS item#98, (cast(sum(coalesce(sr_return_quantity#79, 0))#94 as decimal(15,4)) / cast(sum(coalesce(ss_quantity#72, 0))#95 as decimal(15,4))) AS return_ratio#99, (cast(sum(coalesce(cast(sr_return_amt#80 as decimal(12,2)), 0.00))#96 as decimal(15,4)) / cast(sum(coalesce(cast(ss_net_paid#73 as decimal(12,2)), 0.00))#97 as decimal(15,4))) AS currency_ratio#100]
 
 (75) Exchange
 Input [3]: [item#98, return_ratio#99, currency_ratio#100]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt
index 371ed43fb99..b8185bb9ffd 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q49/explain.txt
@@ -156,7 +156,7 @@ Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25
 Keys [1]: [ws_item_sk#1]
 Functions [4]: [sum(coalesce(wr_return_quantity#10, 0)), sum(coalesce(ws_quantity#3, 0)), sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))]
 Aggregate Attributes [4]: [sum(coalesce(wr_return_quantity#10, 0))#26, sum(coalesce(ws_quantity#3, 0))#27, sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00))#28, sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))#29]
-Results [3]: [ws_item_sk#1 AS item#30, CheckOverflow((promote_precision(cast(sum(coalesce(wr_return_quantity#10, 0))#26 as decimal(15,4))) / promote_precision(cast(sum(coalesce(ws_quantity#3, 0))#27 as decimal(15,4)))), DecimalType(35,20)) AS return_ratio#31, CheckOverflow((promote_precision(cast(sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00))#28 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))#29 as decimal(15,4)))), Deci [...]
+Results [3]: [ws_item_sk#1 AS item#30, (cast(sum(coalesce(wr_return_quantity#10, 0))#26 as decimal(15,4)) / cast(sum(coalesce(ws_quantity#3, 0))#27 as decimal(15,4))) AS return_ratio#31, (cast(sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00))#28 as decimal(15,4)) / cast(sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))#29 as decimal(15,4))) AS currency_ratio#32]
 
 (18) Exchange
 Input [3]: [item#30, return_ratio#31, currency_ratio#32]
@@ -264,7 +264,7 @@ Input [7]: [cs_item_sk#36, sum#54, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#5
 Keys [1]: [cs_item_sk#36]
 Functions [4]: [sum(coalesce(cr_return_quantity#44, 0)), sum(coalesce(cs_quantity#38, 0)), sum(coalesce(cast(cr_return_amount#45 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))]
 Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#44, 0))#60, sum(coalesce(cs_quantity#38, 0))#61, sum(coalesce(cast(cr_return_amount#45 as decimal(12,2)), 0.00))#62, sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))#63]
-Results [3]: [cs_item_sk#36 AS item#64, CheckOverflow((promote_precision(cast(sum(coalesce(cr_return_quantity#44, 0))#60 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cs_quantity#38, 0))#61 as decimal(15,4)))), DecimalType(35,20)) AS return_ratio#65, CheckOverflow((promote_precision(cast(sum(coalesce(cast(cr_return_amount#45 as decimal(12,2)), 0.00))#62 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))#63 as decimal(15,4)))) [...]
+Results [3]: [cs_item_sk#36 AS item#64, (cast(sum(coalesce(cr_return_quantity#44, 0))#60 as decimal(15,4)) / cast(sum(coalesce(cs_quantity#38, 0))#61 as decimal(15,4))) AS return_ratio#65, (cast(sum(coalesce(cast(cr_return_amount#45 as decimal(12,2)), 0.00))#62 as decimal(15,4)) / cast(sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))#63 as decimal(15,4))) AS currency_ratio#66]
 
 (42) Exchange
 Input [3]: [item#64, return_ratio#65, currency_ratio#66]
@@ -372,7 +372,7 @@ Input [7]: [ss_item_sk#70, sum#88, sum#89, sum#90, isEmpty#91, sum#92, isEmpty#9
 Keys [1]: [ss_item_sk#70]
 Functions [4]: [sum(coalesce(sr_return_quantity#78, 0)), sum(coalesce(ss_quantity#72, 0)), sum(coalesce(cast(sr_return_amt#79 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#73 as decimal(12,2)), 0.00))]
 Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#78, 0))#94, sum(coalesce(ss_quantity#72, 0))#95, sum(coalesce(cast(sr_return_amt#79 as decimal(12,2)), 0.00))#96, sum(coalesce(cast(ss_net_paid#73 as decimal(12,2)), 0.00))#97]
-Results [3]: [ss_item_sk#70 AS item#98, CheckOverflow((promote_precision(cast(sum(coalesce(sr_return_quantity#78, 0))#94 as decimal(15,4))) / promote_precision(cast(sum(coalesce(ss_quantity#72, 0))#95 as decimal(15,4)))), DecimalType(35,20)) AS return_ratio#99, CheckOverflow((promote_precision(cast(sum(coalesce(cast(sr_return_amt#79 as decimal(12,2)), 0.00))#96 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(ss_net_paid#73 as decimal(12,2)), 0.00))#97 as decimal(15,4)))), D [...]
+Results [3]: [ss_item_sk#70 AS item#98, (cast(sum(coalesce(sr_return_quantity#78, 0))#94 as decimal(15,4)) / cast(sum(coalesce(ss_quantity#72, 0))#95 as decimal(15,4))) AS return_ratio#99, (cast(sum(coalesce(cast(sr_return_amt#79 as decimal(12,2)), 0.00))#96 as decimal(15,4)) / cast(sum(coalesce(cast(ss_net_paid#73 as decimal(12,2)), 0.00))#97 as decimal(15,4))) AS currency_ratio#100]
 
 (66) Exchange
 Input [3]: [item#98, return_ratio#99, currency_ratio#100]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt
index 08dff69c2d6..950ad15aec9 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5.sf100/explain.txt
@@ -173,7 +173,7 @@ Input [5]: [s_store_id#23, sum#29, sum#30, sum#31, sum#32]
 Keys [1]: [s_store_id#23]
 Functions [4]: [sum(UnscaledValue(sales_price#8)), sum(UnscaledValue(return_amt#10)), sum(UnscaledValue(profit#9)), sum(UnscaledValue(net_loss#11))]
 Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#8))#33, sum(UnscaledValue(return_amt#10))#34, sum(UnscaledValue(profit#9))#35, sum(UnscaledValue(net_loss#11))#36]
-Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#8))#33,17,2) AS sales#37, MakeDecimal(sum(UnscaledValue(return_amt#10))#34,17,2) AS returns#38, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#9))#35,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#11))#36,17,2) as decimal(18,2)))), DecimalType(18,2)) AS profit#39, store channel AS channel#40, concat(store, s_store_id#23) AS id#41]
+Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#8))#33,17,2) AS sales#37, MakeDecimal(sum(UnscaledValue(return_amt#10))#34,17,2) AS returns#38, (MakeDecimal(sum(UnscaledValue(profit#9))#35,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#11))#36,17,2)) AS profit#39, store channel AS channel#40, concat(store, s_store_id#23) AS id#41]
 
 (22) Scan parquet default.catalog_sales
 Output [4]: [cs_catalog_page_sk#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45]
@@ -270,7 +270,7 @@ Input [5]: [cp_catalog_page_id#63, sum#69, sum#70, sum#71, sum#72]
 Keys [1]: [cp_catalog_page_id#63]
 Functions [4]: [sum(UnscaledValue(sales_price#48)), sum(UnscaledValue(return_amt#50)), sum(UnscaledValue(profit#49)), sum(UnscaledValue(net_loss#51))]
 Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#48))#73, sum(UnscaledValue(return_amt#50))#74, sum(UnscaledValue(profit#49))#75, sum(UnscaledValue(net_loss#51))#76]
-Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#48))#73,17,2) AS sales#77, MakeDecimal(sum(UnscaledValue(return_amt#50))#74,17,2) AS returns#78, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#49))#75,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#51))#76,17,2) as decimal(18,2)))), DecimalType(18,2)) AS profit#79, catalog channel AS channel#80, concat(catalog_page, cp_catalog_page_id#63) AS id#81]
+Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#48))#73,17,2) AS sales#77, MakeDecimal(sum(UnscaledValue(return_amt#50))#74,17,2) AS returns#78, (MakeDecimal(sum(UnscaledValue(profit#49))#75,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#51))#76,17,2)) AS profit#79, catalog channel AS channel#80, concat(catalog_page, cp_catalog_page_id#63) AS id#81]
 
 (43) Scan parquet default.web_sales
 Output [4]: [ws_web_site_sk#82, ws_ext_sales_price#83, ws_net_profit#84, ws_sold_date_sk#85]
@@ -401,7 +401,7 @@ Input [5]: [web_site_id#108, sum#114, sum#115, sum#116, sum#117]
 Keys [1]: [web_site_id#108]
 Functions [4]: [sum(UnscaledValue(sales_price#88)), sum(UnscaledValue(return_amt#90)), sum(UnscaledValue(profit#89)), sum(UnscaledValue(net_loss#91))]
 Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#88))#118, sum(UnscaledValue(return_amt#90))#119, sum(UnscaledValue(profit#89))#120, sum(UnscaledValue(net_loss#91))#121]
-Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#88))#118,17,2) AS sales#122, MakeDecimal(sum(UnscaledValue(return_amt#90))#119,17,2) AS returns#123, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#89))#120,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#91))#121,17,2) as decimal(18,2)))), DecimalType(18,2)) AS profit#124, web channel AS channel#125, concat(web_site, web_site_id#108) AS id#126]
+Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#88))#118,17,2) AS sales#122, MakeDecimal(sum(UnscaledValue(return_amt#90))#119,17,2) AS returns#123, (MakeDecimal(sum(UnscaledValue(profit#89))#120,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#91))#121,17,2)) AS profit#124, web channel AS channel#125, concat(web_site, web_site_id#108) AS id#126]
 
 (72) Union
 
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt
index ad669da0de9..0d5fc77824e 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q5/explain.txt
@@ -170,7 +170,7 @@ Input [5]: [s_store_id#24, sum#29, sum#30, sum#31, sum#32]
 Keys [1]: [s_store_id#24]
 Functions [4]: [sum(UnscaledValue(sales_price#8)), sum(UnscaledValue(return_amt#10)), sum(UnscaledValue(profit#9)), sum(UnscaledValue(net_loss#11))]
 Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#8))#33, sum(UnscaledValue(return_amt#10))#34, sum(UnscaledValue(profit#9))#35, sum(UnscaledValue(net_loss#11))#36]
-Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#8))#33,17,2) AS sales#37, MakeDecimal(sum(UnscaledValue(return_amt#10))#34,17,2) AS returns#38, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#9))#35,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#11))#36,17,2) as decimal(18,2)))), DecimalType(18,2)) AS profit#39, store channel AS channel#40, concat(store, s_store_id#24) AS id#41]
+Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#8))#33,17,2) AS sales#37, MakeDecimal(sum(UnscaledValue(return_amt#10))#34,17,2) AS returns#38, (MakeDecimal(sum(UnscaledValue(profit#9))#35,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#11))#36,17,2)) AS profit#39, store channel AS channel#40, concat(store, s_store_id#24) AS id#41]
 
 (22) Scan parquet default.catalog_sales
 Output [4]: [cs_catalog_page_sk#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45]
@@ -267,7 +267,7 @@ Input [5]: [cp_catalog_page_id#64, sum#69, sum#70, sum#71, sum#72]
 Keys [1]: [cp_catalog_page_id#64]
 Functions [4]: [sum(UnscaledValue(sales_price#48)), sum(UnscaledValue(return_amt#50)), sum(UnscaledValue(profit#49)), sum(UnscaledValue(net_loss#51))]
 Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#48))#73, sum(UnscaledValue(return_amt#50))#74, sum(UnscaledValue(profit#49))#75, sum(UnscaledValue(net_loss#51))#76]
-Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#48))#73,17,2) AS sales#77, MakeDecimal(sum(UnscaledValue(return_amt#50))#74,17,2) AS returns#78, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#49))#75,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#51))#76,17,2) as decimal(18,2)))), DecimalType(18,2)) AS profit#79, catalog channel AS channel#80, concat(catalog_page, cp_catalog_page_id#64) AS id#81]
+Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#48))#73,17,2) AS sales#77, MakeDecimal(sum(UnscaledValue(return_amt#50))#74,17,2) AS returns#78, (MakeDecimal(sum(UnscaledValue(profit#49))#75,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#51))#76,17,2)) AS profit#79, catalog channel AS channel#80, concat(catalog_page, cp_catalog_page_id#64) AS id#81]
 
 (43) Scan parquet default.web_sales
 Output [4]: [ws_web_site_sk#82, ws_ext_sales_price#83, ws_net_profit#84, ws_sold_date_sk#85]
@@ -386,7 +386,7 @@ Input [5]: [web_site_id#109, sum#114, sum#115, sum#116, sum#117]
 Keys [1]: [web_site_id#109]
 Functions [4]: [sum(UnscaledValue(sales_price#88)), sum(UnscaledValue(return_amt#90)), sum(UnscaledValue(profit#89)), sum(UnscaledValue(net_loss#91))]
 Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#88))#118, sum(UnscaledValue(return_amt#90))#119, sum(UnscaledValue(profit#89))#120, sum(UnscaledValue(net_loss#91))#121]
-Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#88))#118,17,2) AS sales#122, MakeDecimal(sum(UnscaledValue(return_amt#90))#119,17,2) AS returns#123, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#89))#120,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#91))#121,17,2) as decimal(18,2)))), DecimalType(18,2)) AS profit#124, web channel AS channel#125, concat(web_site, web_site_id#109) AS id#126]
+Results [5]: [MakeDecimal(sum(UnscaledValue(sales_price#88))#118,17,2) AS sales#122, MakeDecimal(sum(UnscaledValue(return_amt#90))#119,17,2) AS returns#123, (MakeDecimal(sum(UnscaledValue(profit#89))#120,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#91))#121,17,2)) AS profit#124, web channel AS channel#125, concat(web_site, web_site_id#109) AS id#126]
 
 (69) Union
 
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.sf100/explain.txt
index 507650dfadc..ab10089828e 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53.sf100/explain.txt
@@ -146,7 +146,7 @@ Arguments: [avg(_w0#22) windowspecdefinition(i_manufact_id#5, specifiedwindowfra
 
 (26) Filter [codegen id : 7]
 Input [4]: [i_manufact_id#5, sum_sales#21, _w0#22, avg_quarterly_sales#23]
-Condition : (isnotnull(avg_quarterly_sales#23) AND ((avg_quarterly_sales#23 > 0.000000) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_quarterly_sales#23 as decimal(22,6)))), DecimalType(22,6)))) / promote_precision(cast(avg_quarterly_sales#23 as decimal(22,6)))), DecimalType(38,16)) > 0.1000000000000000)))
+Condition : ((isnotnull(avg_quarterly_sales#23) AND isnotnull(sum_sales#21)) AND ((avg_quarterly_sales#23 > 0.000000) AND ((abs((sum_sales#21 - avg_quarterly_sales#23)) / avg_quarterly_sales#23) > 0.1000000000000000)))
 
 (27) Project [codegen id : 7]
 Output [3]: [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/explain.txt
index 9e09bfb0dfb..b6382074cd4 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q53/explain.txt
@@ -146,7 +146,7 @@ Arguments: [avg(_w0#22) windowspecdefinition(i_manufact_id#5, specifiedwindowfra
 
 (26) Filter [codegen id : 7]
 Input [4]: [i_manufact_id#5, sum_sales#21, _w0#22, avg_quarterly_sales#23]
-Condition : (isnotnull(avg_quarterly_sales#23) AND ((avg_quarterly_sales#23 > 0.000000) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_quarterly_sales#23 as decimal(22,6)))), DecimalType(22,6)))) / promote_precision(cast(avg_quarterly_sales#23 as decimal(22,6)))), DecimalType(38,16)) > 0.1000000000000000)))
+Condition : ((isnotnull(avg_quarterly_sales#23) AND isnotnull(sum_sales#21)) AND ((avg_quarterly_sales#23 > 0.000000) AND ((abs((sum_sales#21 - avg_quarterly_sales#23)) / avg_quarterly_sales#23) > 0.1000000000000000)))
 
 (27) Project [codegen id : 7]
 Output [3]: [i_manufact_id#5, sum_sales#21, avg_quarterly_sales#23]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.sf100/explain.txt
index 9d1a92004c7..ccd1ce78398 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54.sf100/explain.txt
@@ -310,7 +310,7 @@ Input [2]: [c_customer_sk#23, sum#31]
 Keys [1]: [c_customer_sk#23]
 Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#26))]
 Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#26))#32]
-Results [1]: [cast(CheckOverflow((promote_precision(MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#26))#32,17,2)) / 50.00), DecimalType(21,6)) as int) AS segment#33]
+Results [1]: [cast((MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#26))#32,17,2) / 50) as int) AS segment#33]
 
 (56) HashAggregate [codegen id : 15]
 Input [1]: [segment#33]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt
index 3377b58db47..ef46055c089 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q54/explain.txt
@@ -295,7 +295,7 @@ Input [2]: [c_customer_sk#18, sum#31]
 Keys [1]: [c_customer_sk#18]
 Functions [1]: [sum(UnscaledValue(ss_ext_sales_price#21))]
 Aggregate Attributes [1]: [sum(UnscaledValue(ss_ext_sales_price#21))#32]
-Results [1]: [cast(CheckOverflow((promote_precision(MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#21))#32,17,2)) / 50.00), DecimalType(21,6)) as int) AS segment#33]
+Results [1]: [cast((MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#21))#32,17,2) / 50) as int) AS segment#33]
 
 (53) HashAggregate [codegen id : 12]
 Input [1]: [segment#33]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.sf100/explain.txt
index cd93eea6a1c..3e28dcfc498 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57.sf100/explain.txt
@@ -186,7 +186,7 @@ Arguments: [avg(_w0#18) windowspecdefinition(i_category#13, i_brand#12, cc_name#
 
 (30) Filter [codegen id : 11]
 Input [9]: [i_category#13, i_brand#12, cc_name#10, d_year#7, d_moy#8, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20]
-Condition : ((isnotnull(avg_monthly_sales#20) AND (avg_monthly_sales#20 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#17 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#20 as decimal(22,6)))), DecimalType(22,6)))) / promote_precision(cast(avg_monthly_sales#20 as decimal(22,6)))), DecimalType(38,16)) > 0.1000000000000000))
+Condition : (((isnotnull(avg_monthly_sales#20) AND isnotnull(sum_sales#17)) AND (avg_monthly_sales#20 > 0.000000)) AND ((abs((sum_sales#17 - avg_monthly_sales#20)) / avg_monthly_sales#20) > 0.1000000000000000))
 
 (31) Project [codegen id : 11]
 Output [8]: [i_category#13, i_brand#12, cc_name#10, d_year#7, d_moy#8, sum_sales#17, avg_monthly_sales#20, rn#19]
@@ -277,7 +277,7 @@ Input [14]: [i_category#13, i_brand#12, cc_name#10, d_year#7, d_moy#8, sum_sales
 
 (52) TakeOrderedAndProject
 Input [9]: [i_category#13, i_brand#12, cc_name#10, d_year#7, d_moy#8, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38]
-Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#17 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#20 as decimal(22,6)))), DecimalType(22,6)) ASC NULLS FIRST, cc_name#10 ASC NULLS FIRST], [i_category#13, i_brand#12, cc_name#10, d_year#7, d_moy#8, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38]
+Arguments: 100, [(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST, cc_name#10 ASC NULLS FIRST], [i_category#13, i_brand#12, cc_name#10, d_year#7, d_moy#8, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38]
 
 ===== Subqueries =====
 
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/explain.txt
index 7abc61f31e6..5002d36fad5 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q57/explain.txt
@@ -167,7 +167,7 @@ Arguments: [avg(_w0#18) windowspecdefinition(i_category#3, i_brand#2, cc_name#13
 
 (27) Filter [codegen id : 22]
 Input [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20]
-Condition : ((isnotnull(avg_monthly_sales#20) AND (avg_monthly_sales#20 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#17 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#20 as decimal(22,6)))), DecimalType(22,6)))) / promote_precision(cast(avg_monthly_sales#20 as decimal(22,6)))), DecimalType(38,16)) > 0.1000000000000000))
+Condition : (((isnotnull(avg_monthly_sales#20) AND isnotnull(sum_sales#17)) AND (avg_monthly_sales#20 > 0.000000)) AND ((abs((sum_sales#17 - avg_monthly_sales#20)) / avg_monthly_sales#20) > 0.1000000000000000))
 
 (28) Project [codegen id : 22]
 Output [8]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19]
@@ -242,7 +242,7 @@ Input [14]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales
 
 (45) TakeOrderedAndProject
 Input [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38]
-Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#17 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#20 as decimal(22,6)))), DecimalType(22,6)) ASC NULLS FIRST, cc_name#13 ASC NULLS FIRST], [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38]
+Arguments: 100, [(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST, cc_name#13 ASC NULLS FIRST], [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38]
 
 ===== Subqueries =====
 
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt
index 2f88e2378af..1066878061d 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58.sf100/explain.txt
@@ -194,7 +194,7 @@ Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [pla
 (31) BroadcastHashJoin [codegen id : 15]
 Left keys [1]: [item_id#11]
 Right keys [1]: [item_id#22]
-Join condition: ((((cast(ss_item_rev#12 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#23)), DecimalType(19,3))) AND (cast(ss_item_rev#12 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#23)), DecimalType(20,3)))) AND (cast(cs_item_rev#23 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#12)), DecimalType(19,3)))) AND (cast(cs_item_rev#23 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#12)), [...]
+Join condition: ((((cast(ss_item_rev#12 as decimal(19,3)) >= (0.9 * cs_item_rev#23)) AND (cast(ss_item_rev#12 as decimal(20,3)) <= (1.1 * cs_item_rev#23))) AND (cast(cs_item_rev#23 as decimal(19,3)) >= (0.9 * ss_item_rev#12))) AND (cast(cs_item_rev#23 as decimal(20,3)) <= (1.1 * ss_item_rev#12)))
 
 (32) Project [codegen id : 15]
 Output [3]: [item_id#11, ss_item_rev#12, cs_item_rev#23]
@@ -268,10 +268,10 @@ Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [pla
 (47) BroadcastHashJoin [codegen id : 15]
 Left keys [1]: [item_id#11]
 Right keys [1]: [item_id#33]
-Join condition: ((((((((cast(ss_item_rev#12 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#34)), DecimalType(19,3))) AND (cast(ss_item_rev#12 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#34)), DecimalType(20,3)))) AND (cast(cs_item_rev#23 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#34)), DecimalType(19,3)))) AND (cast(cs_item_rev#23 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#3 [...]
+Join condition: ((((((((cast(ss_item_rev#12 as decimal(19,3)) >= (0.9 * ws_item_rev#34)) AND (cast(ss_item_rev#12 as decimal(20,3)) <= (1.1 * ws_item_rev#34))) AND (cast(cs_item_rev#23 as decimal(19,3)) >= (0.9 * ws_item_rev#34))) AND (cast(cs_item_rev#23 as decimal(20,3)) <= (1.1 * ws_item_rev#34))) AND (cast(ws_item_rev#34 as decimal(19,3)) >= (0.9 * ss_item_rev#12))) AND (cast(ws_item_rev#34 as decimal(20,3)) <= (1.1 * ss_item_rev#12))) AND (cast(ws_item_rev#34 as decimal(19,3)) >= (0 [...]
 
 (48) Project [codegen id : 15]
-Output [8]: [item_id#11, ss_item_rev#12, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ss_item_rev#12 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#12 as decimal(18,2))) + promote_precision(cast(cs_item_rev#23 as decimal(18,2)))), DecimalType(18,2)) as decimal(19,2))) + promote_precision(cast(ws_item_rev#34 as decimal(19,2)))), DecimalType(19,2)))), D [...]
+Output [8]: [item_id#11, ss_item_rev#12, (((ss_item_rev#12 / ((ss_item_rev#12 + cs_item_rev#23) + ws_item_rev#34)) / 3) * 100) AS ss_dev#35, cs_item_rev#23, (((cs_item_rev#23 / ((ss_item_rev#12 + cs_item_rev#23) + ws_item_rev#34)) / 3) * 100) AS cs_dev#36, ws_item_rev#34, (((ws_item_rev#34 / ((ss_item_rev#12 + cs_item_rev#23) + ws_item_rev#34)) / 3) * 100) AS ws_dev#37, (((ss_item_rev#12 + cs_item_rev#23) + ws_item_rev#34) / 3) AS average#38]
 Input [5]: [item_id#11, ss_item_rev#12, cs_item_rev#23, item_id#33, ws_item_rev#34]
 
 (49) TakeOrderedAndProject
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt
index 4f2b027a0b0..b6b4673c749 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q58/explain.txt
@@ -194,7 +194,7 @@ Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [pla
 (31) BroadcastHashJoin [codegen id : 15]
 Left keys [1]: [item_id#11]
 Right keys [1]: [item_id#22]
-Join condition: ((((cast(ss_item_rev#12 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(cs_item_rev#23)), DecimalType(19,3))) AND (cast(ss_item_rev#12 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(cs_item_rev#23)), DecimalType(20,3)))) AND (cast(cs_item_rev#23 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ss_item_rev#12)), DecimalType(19,3)))) AND (cast(cs_item_rev#23 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ss_item_rev#12)), [...]
+Join condition: ((((cast(ss_item_rev#12 as decimal(19,3)) >= (0.9 * cs_item_rev#23)) AND (cast(ss_item_rev#12 as decimal(20,3)) <= (1.1 * cs_item_rev#23))) AND (cast(cs_item_rev#23 as decimal(19,3)) >= (0.9 * ss_item_rev#12))) AND (cast(cs_item_rev#23 as decimal(20,3)) <= (1.1 * ss_item_rev#12)))
 
 (32) Project [codegen id : 15]
 Output [3]: [item_id#11, ss_item_rev#12, cs_item_rev#23]
@@ -268,10 +268,10 @@ Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [pla
 (47) BroadcastHashJoin [codegen id : 15]
 Left keys [1]: [item_id#11]
 Right keys [1]: [item_id#33]
-Join condition: ((((((((cast(ss_item_rev#12 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#34)), DecimalType(19,3))) AND (cast(ss_item_rev#12 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#34)), DecimalType(20,3)))) AND (cast(cs_item_rev#23 as decimal(19,3)) >= CheckOverflow((0.90 * promote_precision(ws_item_rev#34)), DecimalType(19,3)))) AND (cast(cs_item_rev#23 as decimal(20,3)) <= CheckOverflow((1.10 * promote_precision(ws_item_rev#3 [...]
+Join condition: ((((((((cast(ss_item_rev#12 as decimal(19,3)) >= (0.9 * ws_item_rev#34)) AND (cast(ss_item_rev#12 as decimal(20,3)) <= (1.1 * ws_item_rev#34))) AND (cast(cs_item_rev#23 as decimal(19,3)) >= (0.9 * ws_item_rev#34))) AND (cast(cs_item_rev#23 as decimal(20,3)) <= (1.1 * ws_item_rev#34))) AND (cast(ws_item_rev#34 as decimal(19,3)) >= (0.9 * ss_item_rev#12))) AND (cast(ws_item_rev#34 as decimal(20,3)) <= (1.1 * ss_item_rev#12))) AND (cast(ws_item_rev#34 as decimal(19,3)) >= (0 [...]
 
 (48) Project [codegen id : 15]
-Output [8]: [item_id#11, ss_item_rev#12, CheckOverflow((promote_precision(CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(ss_item_rev#12 as decimal(19,2))) / promote_precision(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(ss_item_rev#12 as decimal(18,2))) + promote_precision(cast(cs_item_rev#23 as decimal(18,2)))), DecimalType(18,2)) as decimal(19,2))) + promote_precision(cast(ws_item_rev#34 as decimal(19,2)))), DecimalType(19,2)))), D [...]
+Output [8]: [item_id#11, ss_item_rev#12, (((ss_item_rev#12 / ((ss_item_rev#12 + cs_item_rev#23) + ws_item_rev#34)) / 3) * 100) AS ss_dev#35, cs_item_rev#23, (((cs_item_rev#23 / ((ss_item_rev#12 + cs_item_rev#23) + ws_item_rev#34)) / 3) * 100) AS cs_dev#36, ws_item_rev#34, (((ws_item_rev#34 / ((ss_item_rev#12 + cs_item_rev#23) + ws_item_rev#34)) / 3) * 100) AS ws_dev#37, (((ss_item_rev#12 + cs_item_rev#23) + ws_item_rev#34) / 3) AS average#38]
 Input [5]: [item_id#11, ss_item_rev#12, cs_item_rev#23, item_id#33, ws_item_rev#34]
 
 (49) TakeOrderedAndProject
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.sf100/explain.txt
index 551e2666503..eda985e3eb8 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59.sf100/explain.txt
@@ -301,7 +301,7 @@ Right keys [2]: [s_store_id2#73, (d_week_seq2#72 - 52)]
 Join condition: None
 
 (53) Project [codegen id : 10]
-Output [10]: [s_store_name1#42, s_store_id1#44, d_week_seq1#43, CheckOverflow((promote_precision(sun_sales1#45) / promote_precision(sun_sales2#74)), DecimalType(37,20)) AS (sun_sales1 / sun_sales2)#81, CheckOverflow((promote_precision(mon_sales1#46) / promote_precision(mon_sales2#75)), DecimalType(37,20)) AS (mon_sales1 / mon_sales2)#82, CheckOverflow((promote_precision(tue_sales1#47) / promote_precision(tue_sales2#76)), DecimalType(37,20)) AS (tue_sales1 / tue_sales2)#83, CheckOverflow( [...]
+Output [10]: [s_store_name1#42, s_store_id1#44, d_week_seq1#43, (sun_sales1#45 / sun_sales2#74) AS (sun_sales1 / sun_sales2)#81, (mon_sales1#46 / mon_sales2#75) AS (mon_sales1 / mon_sales2)#82, (tue_sales1#47 / tue_sales2#76) AS (tue_sales1 / tue_sales2)#83, (wed_sales1#48 / wed_sales2#77) AS (wed_sales1 / wed_sales2)#84, (thu_sales1#49 / thu_sales2#78) AS (thu_sales1 / thu_sales2)#85, (fri_sales1#50 / fri_sales2#79) AS (fri_sales1 / fri_sales2)#86, (sat_sales1#51 / sat_sales2#80) AS (sa [...]
 Input [19]: [s_store_name1#42, d_week_seq1#43, s_store_id1#44, sun_sales1#45, mon_sales1#46, tue_sales1#47, wed_sales1#48, thu_sales1#49, fri_sales1#50, sat_sales1#51, d_week_seq2#72, s_store_id2#73, sun_sales2#74, mon_sales2#75, tue_sales2#76, wed_sales2#77, thu_sales2#78, fri_sales2#79, sat_sales2#80]
 
 (54) TakeOrderedAndProject
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/explain.txt
index e9788fb36a4..1667bf18678 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q59/explain.txt
@@ -241,7 +241,7 @@ Right keys [2]: [s_store_id2#62, (d_week_seq2#61 - 52)]
 Join condition: None
 
 (43) Project [codegen id : 10]
-Output [10]: [s_store_name1#40, s_store_id1#42, d_week_seq1#41, CheckOverflow((promote_precision(sun_sales1#43) / promote_precision(sun_sales2#63)), DecimalType(37,20)) AS (sun_sales1 / sun_sales2)#70, CheckOverflow((promote_precision(mon_sales1#44) / promote_precision(mon_sales2#64)), DecimalType(37,20)) AS (mon_sales1 / mon_sales2)#71, CheckOverflow((promote_precision(tue_sales1#45) / promote_precision(tue_sales2#65)), DecimalType(37,20)) AS (tue_sales1 / tue_sales2)#72, CheckOverflow( [...]
+Output [10]: [s_store_name1#40, s_store_id1#42, d_week_seq1#41, (sun_sales1#43 / sun_sales2#63) AS (sun_sales1 / sun_sales2)#70, (mon_sales1#44 / mon_sales2#64) AS (mon_sales1 / mon_sales2)#71, (tue_sales1#45 / tue_sales2#65) AS (tue_sales1 / tue_sales2)#72, (wed_sales1#46 / wed_sales2#66) AS (wed_sales1 / wed_sales2)#73, (thu_sales1#47 / thu_sales2#67) AS (thu_sales1 / thu_sales2)#74, (fri_sales1#48 / fri_sales2#68) AS (fri_sales1 / fri_sales2)#75, (sat_sales1#49 / sat_sales2#69) AS (sa [...]
 Input [19]: [s_store_name1#40, d_week_seq1#41, s_store_id1#42, sun_sales1#43, mon_sales1#44, tue_sales1#45, wed_sales1#46, thu_sales1#47, fri_sales1#48, sat_sales1#49, d_week_seq2#61, s_store_id2#62, sun_sales2#63, mon_sales2#64, tue_sales2#65, wed_sales2#66, thu_sales2#67, fri_sales2#68, sat_sales2#69]
 
 (44) TakeOrderedAndProject
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.sf100/explain.txt
index 9fe0cc0d94d..4ad38d3ae9a 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61.sf100/explain.txt
@@ -350,7 +350,7 @@ Arguments: IdentityBroadcastMode, [plan_id=8]
 Join condition: None
 
 (64) Project [codegen id : 15]
-Output [3]: [promotions#24, total#37, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(promotions#24 as decimal(15,4))) / promote_precision(cast(total#37 as decimal(15,4)))), DecimalType(35,20))) * 100.00000000000000000000), DecimalType(38,19)) AS ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#38]
+Output [3]: [promotions#24, total#37, ((cast(promotions#24 as decimal(15,4)) / cast(total#37 as decimal(15,4))) * 100) AS ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#38]
 Input [2]: [promotions#24, total#37]
 
 ===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/explain.txt
index 356747eb87d..7467f83c984 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q61/explain.txt
@@ -365,7 +365,7 @@ Arguments: IdentityBroadcastMode, [plan_id=8]
 Join condition: None
 
 (67) Project [codegen id : 15]
-Output [3]: [promotions#24, total#39, CheckOverflow((promote_precision(CheckOverflow((promote_precision(cast(promotions#24 as decimal(15,4))) / promote_precision(cast(total#39 as decimal(15,4)))), DecimalType(35,20))) * 100.00000000000000000000), DecimalType(38,19)) AS ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#40]
+Output [3]: [promotions#24, total#39, ((cast(promotions#24 as decimal(15,4)) / cast(total#39 as decimal(15,4))) * 100) AS ((CAST(promotions AS DECIMAL(15,4)) / CAST(total AS DECIMAL(15,4))) * 100)#40]
 Input [2]: [promotions#24, total#39]
 
 ===== Subqueries =====
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.sf100/explain.txt
index fdc937b6de9..15a1cd3241c 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63.sf100/explain.txt
@@ -146,7 +146,7 @@ Arguments: [avg(_w0#22) windowspecdefinition(i_manager_id#5, specifiedwindowfram
 
 (26) Filter [codegen id : 7]
 Input [4]: [i_manager_id#5, sum_sales#21, _w0#22, avg_monthly_sales#23]
-Condition : (isnotnull(avg_monthly_sales#23) AND ((avg_monthly_sales#23 > 0.000000) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#23 as decimal(22,6)))), DecimalType(22,6)))) / promote_precision(cast(avg_monthly_sales#23 as decimal(22,6)))), DecimalType(38,16)) > 0.1000000000000000)))
+Condition : ((isnotnull(avg_monthly_sales#23) AND isnotnull(sum_sales#21)) AND ((avg_monthly_sales#23 > 0.000000) AND ((abs((sum_sales#21 - avg_monthly_sales#23)) / avg_monthly_sales#23) > 0.1000000000000000)))
 
 (27) Project [codegen id : 7]
 Output [3]: [i_manager_id#5, sum_sales#21, avg_monthly_sales#23]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/explain.txt
index 51d52f254de..e7a9bc0e95a 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q63/explain.txt
@@ -146,7 +146,7 @@ Arguments: [avg(_w0#22) windowspecdefinition(i_manager_id#5, specifiedwindowfram
 
 (26) Filter [codegen id : 7]
 Input [4]: [i_manager_id#5, sum_sales#21, _w0#22, avg_monthly_sales#23]
-Condition : (isnotnull(avg_monthly_sales#23) AND ((avg_monthly_sales#23 > 0.000000) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#21 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#23 as decimal(22,6)))), DecimalType(22,6)))) / promote_precision(cast(avg_monthly_sales#23 as decimal(22,6)))), DecimalType(38,16)) > 0.1000000000000000)))
+Condition : ((isnotnull(avg_monthly_sales#23) AND isnotnull(sum_sales#21)) AND ((avg_monthly_sales#23 > 0.000000) AND ((abs((sum_sales#21 - avg_monthly_sales#23)) / avg_monthly_sales#23) > 0.1000000000000000)))
 
 (27) Project [codegen id : 7]
 Output [3]: [i_manager_id#5, sum_sales#21, avg_monthly_sales#23]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.sf100/explain.txt
index b18b10c69f2..abb58347a07 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.sf100/explain.txt
@@ -1,13 +1,13 @@
 == Physical Plan ==
-TakeOrderedAndProject (41)
-+- * Project (40)
-   +- * SortMergeJoin Inner (39)
-      :- * Sort (33)
-      :  +- Exchange (32)
-      :     +- * Project (31)
-      :        +- * BroadcastHashJoin Inner BuildRight (30)
-      :           :- * Project (25)
-      :           :  +- * BroadcastHashJoin Inner BuildRight (24)
+TakeOrderedAndProject (42)
++- * Project (41)
+   +- * SortMergeJoin Inner (40)
+      :- * Sort (34)
+      :  +- Exchange (33)
+      :     +- * Project (32)
+      :        +- * BroadcastHashJoin Inner BuildRight (31)
+      :           :- * Project (26)
+      :           :  +- * BroadcastHashJoin Inner BuildRight (25)
       :           :     :- * Filter (10)
       :           :     :  +- * HashAggregate (9)
       :           :     :     +- Exchange (8)
@@ -18,28 +18,29 @@ TakeOrderedAndProject (41)
       :           :     :                 :  +- * ColumnarToRow (2)
       :           :     :                 :     +- Scan parquet default.store_sales (1)
       :           :     :                 +- ReusedExchange (4)
-      :           :     +- BroadcastExchange (23)
-      :           :        +- * HashAggregate (22)
-      :           :           +- Exchange (21)
-      :           :              +- * HashAggregate (20)
-      :           :                 +- * HashAggregate (19)
-      :           :                    +- Exchange (18)
-      :           :                       +- * HashAggregate (17)
-      :           :                          +- * Project (16)
-      :           :                             +- * BroadcastHashJoin Inner BuildRight (15)
-      :           :                                :- * Filter (13)
-      :           :                                :  +- * ColumnarToRow (12)
-      :           :                                :     +- Scan parquet default.store_sales (11)
-      :           :                                +- ReusedExchange (14)
-      :           +- BroadcastExchange (29)
-      :              +- * Filter (28)
-      :                 +- * ColumnarToRow (27)
-      :                    +- Scan parquet default.store (26)
-      +- * Sort (38)
-         +- Exchange (37)
-            +- * Filter (36)
-               +- * ColumnarToRow (35)
-                  +- Scan parquet default.item (34)
+      :           :     +- BroadcastExchange (24)
+      :           :        +- * Filter (23)
+      :           :           +- * HashAggregate (22)
+      :           :              +- Exchange (21)
+      :           :                 +- * HashAggregate (20)
+      :           :                    +- * HashAggregate (19)
+      :           :                       +- Exchange (18)
+      :           :                          +- * HashAggregate (17)
+      :           :                             +- * Project (16)
+      :           :                                +- * BroadcastHashJoin Inner BuildRight (15)
+      :           :                                   :- * Filter (13)
+      :           :                                   :  +- * ColumnarToRow (12)
+      :           :                                   :     +- Scan parquet default.store_sales (11)
+      :           :                                   +- ReusedExchange (14)
+      :           +- BroadcastExchange (30)
+      :              +- * Filter (29)
+      :                 +- * ColumnarToRow (28)
+      :                    +- Scan parquet default.store (27)
+      +- * Sort (39)
+         +- Exchange (38)
+            +- * Filter (37)
+               +- * ColumnarToRow (36)
+                  +- Scan parquet default.item (35)
 
 
 (1) Scan parquet default.store_sales
@@ -57,7 +58,7 @@ Input [4]: [ss_item_sk#1, ss_store_sk#2, ss_sales_price#3, ss_sold_date_sk#4]
 Input [4]: [ss_item_sk#1, ss_store_sk#2, ss_sales_price#3, ss_sold_date_sk#4]
 Condition : (isnotnull(ss_store_sk#2) AND isnotnull(ss_item_sk#1))
 
-(4) ReusedExchange [Reuses operator id: 46]
+(4) ReusedExchange [Reuses operator id: 47]
 Output [1]: [d_date_sk#6]
 
 (5) BroadcastHashJoin [codegen id : 2]
@@ -106,7 +107,7 @@ Input [4]: [ss_item_sk#11, ss_store_sk#12, ss_sales_price#13, ss_sold_date_sk#14
 Input [4]: [ss_item_sk#11, ss_store_sk#12, ss_sales_price#13, ss_sold_date_sk#14]
 Condition : isnotnull(ss_store_sk#12)
 
-(14) ReusedExchange [Reuses operator id: 46]
+(14) ReusedExchange [Reuses operator id: 47]
 Output [1]: [d_date_sk#15]
 
 (15) BroadcastHashJoin [codegen id : 4]
@@ -154,118 +155,122 @@ Functions [1]: [avg(revenue#19)]
 Aggregate Attributes [1]: [avg(revenue#19)#24]
 Results [2]: [ss_store_sk#12, avg(revenue#19)#24 AS ave#25]
 
-(23) BroadcastExchange
+(23) Filter [codegen id : 6]
+Input [2]: [ss_store_sk#12, ave#25]
+Condition : isnotnull(ave#25)
+
+(24) BroadcastExchange
 Input [2]: [ss_store_sk#12, ave#25]
 Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=4]
 
-(24) BroadcastHashJoin [codegen id : 8]
+(25) BroadcastHashJoin [codegen id : 8]
 Left keys [1]: [ss_store_sk#2]
 Right keys [1]: [ss_store_sk#12]
-Join condition: (cast(revenue#10 as decimal(23,7)) <= CheckOverflow((0.100000 * promote_precision(ave#25)), DecimalType(23,7)))
+Join condition: (cast(revenue#10 as decimal(23,7)) <= (0.1 * ave#25))
 
-(25) Project [codegen id : 8]
+(26) Project [codegen id : 8]
 Output [3]: [ss_store_sk#2, ss_item_sk#1, revenue#10]
 Input [5]: [ss_store_sk#2, ss_item_sk#1, revenue#10, ss_store_sk#12, ave#25]
 
-(26) Scan parquet default.store
+(27) Scan parquet default.store
 Output [2]: [s_store_sk#26, s_store_name#27]
 Batched: true
 Location [not included in comparison]/{warehouse_dir}/store]
 PushedFilters: [IsNotNull(s_store_sk)]
 ReadSchema: struct<s_store_sk:int,s_store_name:string>
 
-(27) ColumnarToRow [codegen id : 7]
+(28) ColumnarToRow [codegen id : 7]
 Input [2]: [s_store_sk#26, s_store_name#27]
 
-(28) Filter [codegen id : 7]
+(29) Filter [codegen id : 7]
 Input [2]: [s_store_sk#26, s_store_name#27]
 Condition : isnotnull(s_store_sk#26)
 
-(29) BroadcastExchange
+(30) BroadcastExchange
 Input [2]: [s_store_sk#26, s_store_name#27]
 Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=5]
 
-(30) BroadcastHashJoin [codegen id : 8]
+(31) BroadcastHashJoin [codegen id : 8]
 Left keys [1]: [ss_store_sk#2]
 Right keys [1]: [s_store_sk#26]
 Join condition: None
 
-(31) Project [codegen id : 8]
+(32) Project [codegen id : 8]
 Output [3]: [ss_item_sk#1, revenue#10, s_store_name#27]
 Input [5]: [ss_store_sk#2, ss_item_sk#1, revenue#10, s_store_sk#26, s_store_name#27]
 
-(32) Exchange
+(33) Exchange
 Input [3]: [ss_item_sk#1, revenue#10, s_store_name#27]
 Arguments: hashpartitioning(ss_item_sk#1, 5), ENSURE_REQUIREMENTS, [plan_id=6]
 
-(33) Sort [codegen id : 9]
+(34) Sort [codegen id : 9]
 Input [3]: [ss_item_sk#1, revenue#10, s_store_name#27]
 Arguments: [ss_item_sk#1 ASC NULLS FIRST], false, 0
 
-(34) Scan parquet default.item
+(35) Scan parquet default.item
 Output [5]: [i_item_sk#28, i_item_desc#29, i_current_price#30, i_wholesale_cost#31, i_brand#32]
 Batched: true
 Location [not included in comparison]/{warehouse_dir}/item]
 PushedFilters: [IsNotNull(i_item_sk)]
 ReadSchema: struct<i_item_sk:int,i_item_desc:string,i_current_price:decimal(7,2),i_wholesale_cost:decimal(7,2),i_brand:string>
 
-(35) ColumnarToRow [codegen id : 10]
+(36) ColumnarToRow [codegen id : 10]
 Input [5]: [i_item_sk#28, i_item_desc#29, i_current_price#30, i_wholesale_cost#31, i_brand#32]
 
-(36) Filter [codegen id : 10]
+(37) Filter [codegen id : 10]
 Input [5]: [i_item_sk#28, i_item_desc#29, i_current_price#30, i_wholesale_cost#31, i_brand#32]
 Condition : isnotnull(i_item_sk#28)
 
-(37) Exchange
+(38) Exchange
 Input [5]: [i_item_sk#28, i_item_desc#29, i_current_price#30, i_wholesale_cost#31, i_brand#32]
 Arguments: hashpartitioning(i_item_sk#28, 5), ENSURE_REQUIREMENTS, [plan_id=7]
 
-(38) Sort [codegen id : 11]
+(39) Sort [codegen id : 11]
 Input [5]: [i_item_sk#28, i_item_desc#29, i_current_price#30, i_wholesale_cost#31, i_brand#32]
 Arguments: [i_item_sk#28 ASC NULLS FIRST], false, 0
 
-(39) SortMergeJoin [codegen id : 12]
+(40) SortMergeJoin [codegen id : 12]
 Left keys [1]: [ss_item_sk#1]
 Right keys [1]: [i_item_sk#28]
 Join condition: None
 
-(40) Project [codegen id : 12]
+(41) Project [codegen id : 12]
 Output [6]: [s_store_name#27, i_item_desc#29, revenue#10, i_current_price#30, i_wholesale_cost#31, i_brand#32]
 Input [8]: [ss_item_sk#1, revenue#10, s_store_name#27, i_item_sk#28, i_item_desc#29, i_current_price#30, i_wholesale_cost#31, i_brand#32]
 
-(41) TakeOrderedAndProject
+(42) TakeOrderedAndProject
 Input [6]: [s_store_name#27, i_item_desc#29, revenue#10, i_current_price#30, i_wholesale_cost#31, i_brand#32]
 Arguments: 100, [s_store_name#27 ASC NULLS FIRST, i_item_desc#29 ASC NULLS FIRST], [s_store_name#27, i_item_desc#29, revenue#10, i_current_price#30, i_wholesale_cost#31, i_brand#32]
 
 ===== Subqueries =====
 
 Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#4 IN dynamicpruning#5
-BroadcastExchange (46)
-+- * Project (45)
-   +- * Filter (44)
-      +- * ColumnarToRow (43)
-         +- Scan parquet default.date_dim (42)
+BroadcastExchange (47)
++- * Project (46)
+   +- * Filter (45)
+      +- * ColumnarToRow (44)
+         +- Scan parquet default.date_dim (43)
 
 
-(42) Scan parquet default.date_dim
+(43) Scan parquet default.date_dim
 Output [2]: [d_date_sk#6, d_month_seq#33]
 Batched: true
 Location [not included in comparison]/{warehouse_dir}/date_dim]
 PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1176), LessThanOrEqual(d_month_seq,1187), IsNotNull(d_date_sk)]
 ReadSchema: struct<d_date_sk:int,d_month_seq:int>
 
-(43) ColumnarToRow [codegen id : 1]
+(44) ColumnarToRow [codegen id : 1]
 Input [2]: [d_date_sk#6, d_month_seq#33]
 
-(44) Filter [codegen id : 1]
+(45) Filter [codegen id : 1]
 Input [2]: [d_date_sk#6, d_month_seq#33]
 Condition : (((isnotnull(d_month_seq#33) AND (d_month_seq#33 >= 1176)) AND (d_month_seq#33 <= 1187)) AND isnotnull(d_date_sk#6))
 
-(45) Project [codegen id : 1]
+(46) Project [codegen id : 1]
 Output [1]: [d_date_sk#6]
 Input [2]: [d_date_sk#6, d_month_seq#33]
 
-(46) BroadcastExchange
+(47) BroadcastExchange
 Input [1]: [d_date_sk#6]
 Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=8]
 
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.sf100/simplified.txt
index 08dbe2abd69..c0f5e941dcb 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.sf100/simplified.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65.sf100/simplified.txt
@@ -37,25 +37,26 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes
                             InputAdapter
                               BroadcastExchange #4
                                 WholeStageCodegen (6)
-                                  HashAggregate [ss_store_sk,sum,count] [avg(revenue),ave,sum,count]
-                                    InputAdapter
-                                      Exchange [ss_store_sk] #5
-                                        WholeStageCodegen (5)
-                                          HashAggregate [ss_store_sk,revenue] [sum,count,sum,count]
-                                            HashAggregate [ss_store_sk,ss_item_sk,sum] [sum(UnscaledValue(ss_sales_price)),revenue,sum]
-                                              InputAdapter
-                                                Exchange [ss_store_sk,ss_item_sk] #6
-                                                  WholeStageCodegen (4)
-                                                    HashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] [sum,sum]
-                                                      Project [ss_item_sk,ss_store_sk,ss_sales_price]
-                                                        BroadcastHashJoin [ss_sold_date_sk,d_date_sk]
-                                                          Filter [ss_store_sk]
-                                                            ColumnarToRow
-                                                              InputAdapter
-                                                                Scan parquet default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk]
-                                                                  ReusedSubquery [d_date_sk] #1
-                                                          InputAdapter
-                                                            ReusedExchange [d_date_sk] #3
+                                  Filter [ave]
+                                    HashAggregate [ss_store_sk,sum,count] [avg(revenue),ave,sum,count]
+                                      InputAdapter
+                                        Exchange [ss_store_sk] #5
+                                          WholeStageCodegen (5)
+                                            HashAggregate [ss_store_sk,revenue] [sum,count,sum,count]
+                                              HashAggregate [ss_store_sk,ss_item_sk,sum] [sum(UnscaledValue(ss_sales_price)),revenue,sum]
+                                                InputAdapter
+                                                  Exchange [ss_store_sk,ss_item_sk] #6
+                                                    WholeStageCodegen (4)
+                                                      HashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] [sum,sum]
+                                                        Project [ss_item_sk,ss_store_sk,ss_sales_price]
+                                                          BroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+                                                            Filter [ss_store_sk]
+                                                              ColumnarToRow
+                                                                InputAdapter
+                                                                  Scan parquet default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk]
+                                                                    ReusedSubquery [d_date_sk] #1
+                                                            InputAdapter
+                                                              ReusedExchange [d_date_sk] #3
                         InputAdapter
                           BroadcastExchange #7
                             WholeStageCodegen (7)
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/explain.txt
index dbacad80d20..2aee4f16e67 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/explain.txt
@@ -1,7 +1,7 @@
 == Physical Plan ==
-TakeOrderedAndProject (38)
-+- * Project (37)
-   +- * BroadcastHashJoin Inner BuildRight (36)
+TakeOrderedAndProject (39)
++- * Project (38)
+   +- * BroadcastHashJoin Inner BuildRight (37)
       :- * Project (22)
       :  +- * BroadcastHashJoin Inner BuildRight (21)
       :     :- * Project (16)
@@ -24,19 +24,20 @@ TakeOrderedAndProject (38)
       :        +- * Filter (19)
       :           +- * ColumnarToRow (18)
       :              +- Scan parquet default.item (17)
-      +- BroadcastExchange (35)
-         +- * HashAggregate (34)
-            +- Exchange (33)
-               +- * HashAggregate (32)
-                  +- * HashAggregate (31)
-                     +- Exchange (30)
-                        +- * HashAggregate (29)
-                           +- * Project (28)
-                              +- * BroadcastHashJoin Inner BuildRight (27)
-                                 :- * Filter (25)
-                                 :  +- * ColumnarToRow (24)
-                                 :     +- Scan parquet default.store_sales (23)
-                                 +- ReusedExchange (26)
+      +- BroadcastExchange (36)
+         +- * Filter (35)
+            +- * HashAggregate (34)
+               +- Exchange (33)
+                  +- * HashAggregate (32)
+                     +- * HashAggregate (31)
+                        +- Exchange (30)
+                           +- * HashAggregate (29)
+                              +- * Project (28)
+                                 +- * BroadcastHashJoin Inner BuildRight (27)
+                                    :- * Filter (25)
+                                    :  +- * ColumnarToRow (24)
+                                    :     +- Scan parquet default.store_sales (23)
+                                    +- ReusedExchange (26)
 
 
 (1) Scan parquet default.store
@@ -68,7 +69,7 @@ Input [4]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6]
 Input [4]: [ss_item_sk#3, ss_store_sk#4, ss_sales_price#5, ss_sold_date_sk#6]
 Condition : (isnotnull(ss_store_sk#4) AND isnotnull(ss_item_sk#3))
 
-(7) ReusedExchange [Reuses operator id: 43]
+(7) ReusedExchange [Reuses operator id: 44]
 Output [1]: [d_date_sk#8]
 
 (8) BroadcastHashJoin [codegen id : 2]
@@ -157,7 +158,7 @@ Input [4]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20, ss_sold_date_sk#21
 Input [4]: [ss_item_sk#18, ss_store_sk#19, ss_sales_price#20, ss_sold_date_sk#21]
 Condition : isnotnull(ss_store_sk#19)
 
-(26) ReusedExchange [Reuses operator id: 43]
+(26) ReusedExchange [Reuses operator id: 44]
 Output [1]: [d_date_sk#22]
 
 (27) BroadcastHashJoin [codegen id : 6]
@@ -205,52 +206,56 @@ Functions [1]: [avg(revenue#26)]
 Aggregate Attributes [1]: [avg(revenue#26)#31]
 Results [2]: [ss_store_sk#19, avg(revenue#26)#31 AS ave#32]
 
-(35) BroadcastExchange
+(35) Filter [codegen id : 8]
+Input [2]: [ss_store_sk#19, ave#32]
+Condition : isnotnull(ave#32)
+
+(36) BroadcastExchange
 Input [2]: [ss_store_sk#19, ave#32]
 Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=6]
 
-(36) BroadcastHashJoin [codegen id : 9]
+(37) BroadcastHashJoin [codegen id : 9]
 Left keys [1]: [ss_store_sk#4]
 Right keys [1]: [ss_store_sk#19]
-Join condition: (cast(revenue#12 as decimal(23,7)) <= CheckOverflow((0.100000 * promote_precision(ave#32)), DecimalType(23,7)))
+Join condition: (cast(revenue#12 as decimal(23,7)) <= (0.1 * ave#32))
 
-(37) Project [codegen id : 9]
+(38) Project [codegen id : 9]
 Output [6]: [s_store_name#2, i_item_desc#14, revenue#12, i_current_price#15, i_wholesale_cost#16, i_brand#17]
 Input [9]: [s_store_name#2, ss_store_sk#4, revenue#12, i_item_desc#14, i_current_price#15, i_wholesale_cost#16, i_brand#17, ss_store_sk#19, ave#32]
 
-(38) TakeOrderedAndProject
+(39) TakeOrderedAndProject
 Input [6]: [s_store_name#2, i_item_desc#14, revenue#12, i_current_price#15, i_wholesale_cost#16, i_brand#17]
 Arguments: 100, [s_store_name#2 ASC NULLS FIRST, i_item_desc#14 ASC NULLS FIRST], [s_store_name#2, i_item_desc#14, revenue#12, i_current_price#15, i_wholesale_cost#16, i_brand#17]
 
 ===== Subqueries =====
 
 Subquery:1 Hosting operator id = 4 Hosting Expression = ss_sold_date_sk#6 IN dynamicpruning#7
-BroadcastExchange (43)
-+- * Project (42)
-   +- * Filter (41)
-      +- * ColumnarToRow (40)
-         +- Scan parquet default.date_dim (39)
+BroadcastExchange (44)
++- * Project (43)
+   +- * Filter (42)
+      +- * ColumnarToRow (41)
+         +- Scan parquet default.date_dim (40)
 
 
-(39) Scan parquet default.date_dim
+(40) Scan parquet default.date_dim
 Output [2]: [d_date_sk#8, d_month_seq#33]
 Batched: true
 Location [not included in comparison]/{warehouse_dir}/date_dim]
 PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1176), LessThanOrEqual(d_month_seq,1187), IsNotNull(d_date_sk)]
 ReadSchema: struct<d_date_sk:int,d_month_seq:int>
 
-(40) ColumnarToRow [codegen id : 1]
+(41) ColumnarToRow [codegen id : 1]
 Input [2]: [d_date_sk#8, d_month_seq#33]
 
-(41) Filter [codegen id : 1]
+(42) Filter [codegen id : 1]
 Input [2]: [d_date_sk#8, d_month_seq#33]
 Condition : (((isnotnull(d_month_seq#33) AND (d_month_seq#33 >= 1176)) AND (d_month_seq#33 <= 1187)) AND isnotnull(d_date_sk#8))
 
-(42) Project [codegen id : 1]
+(43) Project [codegen id : 1]
 Output [1]: [d_date_sk#8]
 Input [2]: [d_date_sk#8, d_month_seq#33]
 
-(43) BroadcastExchange
+(44) BroadcastExchange
 Input [1]: [d_date_sk#8]
 Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=7]
 
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/simplified.txt
index 4b519f37a58..5e9a714f60a 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/simplified.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q65/simplified.txt
@@ -45,22 +45,23 @@ TakeOrderedAndProject [s_store_name,i_item_desc,revenue,i_current_price,i_wholes
         InputAdapter
           BroadcastExchange #5
             WholeStageCodegen (8)
-              HashAggregate [ss_store_sk,sum,count] [avg(revenue),ave,sum,count]
-                InputAdapter
-                  Exchange [ss_store_sk] #6
-                    WholeStageCodegen (7)
-                      HashAggregate [ss_store_sk,revenue] [sum,count,sum,count]
-                        HashAggregate [ss_store_sk,ss_item_sk,sum] [sum(UnscaledValue(ss_sales_price)),revenue,sum]
-                          InputAdapter
-                            Exchange [ss_store_sk,ss_item_sk] #7
-                              WholeStageCodegen (6)
-                                HashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] [sum,sum]
-                                  Project [ss_item_sk,ss_store_sk,ss_sales_price]
-                                    BroadcastHashJoin [ss_sold_date_sk,d_date_sk]
-                                      Filter [ss_store_sk]
-                                        ColumnarToRow
-                                          InputAdapter
-                                            Scan parquet default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk]
-                                              ReusedSubquery [d_date_sk] #1
-                                      InputAdapter
-                                        ReusedExchange [d_date_sk] #3
+              Filter [ave]
+                HashAggregate [ss_store_sk,sum,count] [avg(revenue),ave,sum,count]
+                  InputAdapter
+                    Exchange [ss_store_sk] #6
+                      WholeStageCodegen (7)
+                        HashAggregate [ss_store_sk,revenue] [sum,count,sum,count]
+                          HashAggregate [ss_store_sk,ss_item_sk,sum] [sum(UnscaledValue(ss_sales_price)),revenue,sum]
+                            InputAdapter
+                              Exchange [ss_store_sk,ss_item_sk] #7
+                                WholeStageCodegen (6)
+                                  HashAggregate [ss_store_sk,ss_item_sk,ss_sales_price] [sum,sum]
+                                    Project [ss_item_sk,ss_store_sk,ss_sales_price]
+                                      BroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+                                        Filter [ss_store_sk]
+                                          ColumnarToRow
+                                            InputAdapter
+                                              Scan parquet default.store_sales [ss_item_sk,ss_store_sk,ss_sales_price,ss_sold_date_sk]
+                                                ReusedSubquery [d_date_sk] #1
+                                        InputAdapter
+                                          ReusedExchange [d_date_sk] #3
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.sf100/explain.txt
index fbcec9b4464..75ea97a83c2 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.sf100/explain.txt
@@ -172,7 +172,7 @@ Input [13]: [ws_warehouse_sk#3, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid
 (27) HashAggregate [codegen id : 5]
 Input [11]: [ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#17, w_warehouse_sq_ft#18, w_city#19, w_county#20, w_state#21, w_country#22, d_year#14, d_moy#15]
 Keys [7]: [w_warehouse_name#17, w_warehouse_sq_ft#18, w_city#19, w_county#20, w_state#21, w_country#22, d_year#14]
-Functions [24]: [partial_sum(CASE WHEN (d_moy#15 = 1) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#5 as decimal(12,2))) * promote_precision(cast(ws_quantity#4 as decimal(12,2)))), DecimalType(18,2)) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#15 = 2) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#5 as decimal(12,2))) * promote_precision(cast(ws_quantity#4 as decimal(12,2)))), DecimalType(18,2)) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#15 = 3) THEN Che [...]
+Functions [24]: [partial_sum(CASE WHEN (d_moy#15 = 1) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#15 = 2) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#15 = 3) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#15 = 4) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 E [...]
 Aggregate Attributes [48]: [sum#23, isEmpty#24, sum#25, isEmpty#26, sum#27, isEmpty#28, sum#29, isEmpty#30, sum#31, isEmpty#32, sum#33, isEmpty#34, sum#35, isEmpty#36, sum#37, isEmpty#38, sum#39, isEmpty#40, sum#41, isEmpty#42, sum#43, isEmpty#44, sum#45, isEmpty#46, sum#47, isEmpty#48, sum#49, isEmpty#50, sum#51, isEmpty#52, sum#53, isEmpty#54, sum#55, isEmpty#56, sum#57, isEmpty#58, sum#59, isEmpty#60, sum#61, isEmpty#62, sum#63, isEmpty#64, sum#65, isEmpty#66, sum#67, isEmpty#68, sum# [...]
 Results [55]: [w_warehouse_name#17, w_warehouse_sq_ft#18, w_city#19, w_county#20, w_state#21, w_country#22, d_year#14, sum#71, isEmpty#72, sum#73, isEmpty#74, sum#75, isEmpty#76, sum#77, isEmpty#78, sum#79, isEmpty#80, sum#81, isEmpty#82, sum#83, isEmpty#84, sum#85, isEmpty#86, sum#87, isEmpty#88, sum#89, isEmpty#90, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96, sum#97, isEmpty#98, sum#99, isEmpty#100, sum#101, isEmpty#102, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#10 [...]
 
@@ -183,9 +183,9 @@ Arguments: hashpartitioning(w_warehouse_name#17, w_warehouse_sq_ft#18, w_city#19
 (29) HashAggregate [codegen id : 6]
 Input [55]: [w_warehouse_name#17, w_warehouse_sq_ft#18, w_city#19, w_county#20, w_state#21, w_country#22, d_year#14, sum#71, isEmpty#72, sum#73, isEmpty#74, sum#75, isEmpty#76, sum#77, isEmpty#78, sum#79, isEmpty#80, sum#81, isEmpty#82, sum#83, isEmpty#84, sum#85, isEmpty#86, sum#87, isEmpty#88, sum#89, isEmpty#90, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96, sum#97, isEmpty#98, sum#99, isEmpty#100, sum#101, isEmpty#102, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, [...]
 Keys [7]: [w_warehouse_name#17, w_warehouse_sq_ft#18, w_city#19, w_county#20, w_state#21, w_country#22, d_year#14]
-Functions [24]: [sum(CASE WHEN (d_moy#15 = 1) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#5 as decimal(12,2))) * promote_precision(cast(ws_quantity#4 as decimal(12,2)))), DecimalType(18,2)) ELSE 0.00 END), sum(CASE WHEN (d_moy#15 = 2) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#5 as decimal(12,2))) * promote_precision(cast(ws_quantity#4 as decimal(12,2)))), DecimalType(18,2)) ELSE 0.00 END), sum(CASE WHEN (d_moy#15 = 3) THEN CheckOverflow((promote_prec [...]
-Aggregate Attributes [24]: [sum(CASE WHEN (d_moy#15 = 1) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#5 as decimal(12,2))) * promote_precision(cast(ws_quantity#4 as decimal(12,2)))), DecimalType(18,2)) ELSE 0.00 END)#119, sum(CASE WHEN (d_moy#15 = 2) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#5 as decimal(12,2))) * promote_precision(cast(ws_quantity#4 as decimal(12,2)))), DecimalType(18,2)) ELSE 0.00 END)#120, sum(CASE WHEN (d_moy#15 = 3) THEN CheckOve [...]
-Results [32]: [w_warehouse_name#17, w_warehouse_sq_ft#18, w_city#19, w_county#20, w_state#21, w_country#22, DHL,BARIAN AS ship_carriers#143, d_year#14 AS year#144, sum(CASE WHEN (d_moy#15 = 1) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#5 as decimal(12,2))) * promote_precision(cast(ws_quantity#4 as decimal(12,2)))), DecimalType(18,2)) ELSE 0.00 END)#119 AS jan_sales#145, sum(CASE WHEN (d_moy#15 = 2) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#5 as deci [...]
+Functions [24]: [sum(CASE WHEN (d_moy#15 = 1) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#15 = 2) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#15 = 3) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#15 = 4) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#15 = 5 [...]
+Aggregate Attributes [24]: [sum(CASE WHEN (d_moy#15 = 1) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#119, sum(CASE WHEN (d_moy#15 = 2) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#120, sum(CASE WHEN (d_moy#15 = 3) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#121, sum(CASE WHEN (d_moy#15 = 4) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#122,  [...]
+Results [32]: [w_warehouse_name#17, w_warehouse_sq_ft#18, w_city#19, w_county#20, w_state#21, w_country#22, DHL,BARIAN AS ship_carriers#143, d_year#14 AS year#144, sum(CASE WHEN (d_moy#15 = 1) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#119 AS jan_sales#145, sum(CASE WHEN (d_moy#15 = 2) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#120 AS feb_sales#146, sum(CASE WHEN (d_moy#15 = 3) THEN (ws_ext_sales_price#5 * cast [...]
 
 (30) Scan parquet default.catalog_sales
 Output [7]: [cs_sold_time_sk#169, cs_ship_mode_sk#170, cs_warehouse_sk#171, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, cs_sold_date_sk#175]
@@ -253,7 +253,7 @@ Input [13]: [cs_warehouse_sk#171, cs_quantity#172, cs_sales_price#173, cs_net_pa
 (45) HashAggregate [codegen id : 11]
 Input [11]: [cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, d_year#179, d_moy#180]
 Keys [7]: [w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, d_year#179]
-Functions [24]: [partial_sum(CASE WHEN (d_moy#180 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#173 as decimal(12,2))) * promote_precision(cast(cs_quantity#172 as decimal(12,2)))), DecimalType(18,2)) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#180 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#173 as decimal(12,2))) * promote_precision(cast(cs_quantity#172 as decimal(12,2)))), DecimalType(18,2)) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#180 = 3) THEN  [...]
+Functions [24]: [partial_sum(CASE WHEN (d_moy#180 = 1) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#180 = 2) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#180 = 3) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#180 = 4) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0. [...]
 Aggregate Attributes [48]: [sum#188, isEmpty#189, sum#190, isEmpty#191, sum#192, isEmpty#193, sum#194, isEmpty#195, sum#196, isEmpty#197, sum#198, isEmpty#199, sum#200, isEmpty#201, sum#202, isEmpty#203, sum#204, isEmpty#205, sum#206, isEmpty#207, sum#208, isEmpty#209, sum#210, isEmpty#211, sum#212, isEmpty#213, sum#214, isEmpty#215, sum#216, isEmpty#217, sum#218, isEmpty#219, sum#220, isEmpty#221, sum#222, isEmpty#223, sum#224, isEmpty#225, sum#226, isEmpty#227, sum#228, isEmpty#229, su [...]
 Results [55]: [w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, d_year#179, sum#236, isEmpty#237, sum#238, isEmpty#239, sum#240, isEmpty#241, sum#242, isEmpty#243, sum#244, isEmpty#245, sum#246, isEmpty#247, sum#248, isEmpty#249, sum#250, isEmpty#251, sum#252, isEmpty#253, sum#254, isEmpty#255, sum#256, isEmpty#257, sum#258, isEmpty#259, sum#260, isEmpty#261, sum#262, isEmpty#263, sum#264, isEmpty#265, sum#266, isEmpty#267, sum#268, isEmp [...]
 
@@ -264,16 +264,16 @@ Arguments: hashpartitioning(w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#
 (47) HashAggregate [codegen id : 12]
 Input [55]: [w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, d_year#179, sum#236, isEmpty#237, sum#238, isEmpty#239, sum#240, isEmpty#241, sum#242, isEmpty#243, sum#244, isEmpty#245, sum#246, isEmpty#247, sum#248, isEmpty#249, sum#250, isEmpty#251, sum#252, isEmpty#253, sum#254, isEmpty#255, sum#256, isEmpty#257, sum#258, isEmpty#259, sum#260, isEmpty#261, sum#262, isEmpty#263, sum#264, isEmpty#265, sum#266, isEmpty#267, sum#268, isEmpty [...]
 Keys [7]: [w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, d_year#179]
-Functions [24]: [sum(CASE WHEN (d_moy#180 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#173 as decimal(12,2))) * promote_precision(cast(cs_quantity#172 as decimal(12,2)))), DecimalType(18,2)) ELSE 0.00 END), sum(CASE WHEN (d_moy#180 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#173 as decimal(12,2))) * promote_precision(cast(cs_quantity#172 as decimal(12,2)))), DecimalType(18,2)) ELSE 0.00 END), sum(CASE WHEN (d_moy#180 = 3) THEN CheckOverflow((promote_p [...]
-Aggregate Attributes [24]: [sum(CASE WHEN (d_moy#180 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#173 as decimal(12,2))) * promote_precision(cast(cs_quantity#172 as decimal(12,2)))), DecimalType(18,2)) ELSE 0.00 END)#284, sum(CASE WHEN (d_moy#180 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#173 as decimal(12,2))) * promote_precision(cast(cs_quantity#172 as decimal(12,2)))), DecimalType(18,2)) ELSE 0.00 END)#285, sum(CASE WHEN (d_moy#180 = 3) THEN Check [...]
-Results [32]: [w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, DHL,BARIAN AS ship_carriers#308, d_year#179 AS year#309, sum(CASE WHEN (d_moy#180 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#173 as decimal(12,2))) * promote_precision(cast(cs_quantity#172 as decimal(12,2)))), DecimalType(18,2)) ELSE 0.00 END)#284 AS jan_sales#310, sum(CASE WHEN (d_moy#180 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#173  [...]
+Functions [24]: [sum(CASE WHEN (d_moy#180 = 1) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#180 = 2) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#180 = 3) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#180 = 4) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 [...]
+Aggregate Attributes [24]: [sum(CASE WHEN (d_moy#180 = 1) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#284, sum(CASE WHEN (d_moy#180 = 2) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#285, sum(CASE WHEN (d_moy#180 = 3) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#286, sum(CASE WHEN (d_moy#180 = 4) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#2 [...]
+Results [32]: [w_warehouse_name#182, w_warehouse_sq_ft#183, w_city#184, w_county#185, w_state#186, w_country#187, DHL,BARIAN AS ship_carriers#308, d_year#179 AS year#309, sum(CASE WHEN (d_moy#180 = 1) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#284 AS jan_sales#310, sum(CASE WHEN (d_moy#180 = 2) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#285 AS feb_sales#311, sum(CASE WHEN (d_moy#180 = 3) THEN (cs_sales_price#17 [...]
 
 (48) Union
 
 (49) HashAggregate [codegen id : 13]
 Input [32]: [w_warehouse_name#17, w_warehouse_sq_ft#18, w_city#19, w_county#20, w_state#21, w_country#22, ship_carriers#143, year#144, jan_sales#145, feb_sales#146, mar_sales#147, apr_sales#148, may_sales#149, jun_sales#150, jul_sales#151, aug_sales#152, sep_sales#153, oct_sales#154, nov_sales#155, dec_sales#156, jan_net#157, feb_net#158, mar_net#159, apr_net#160, may_net#161, jun_net#162, jul_net#163, aug_net#164, sep_net#165, oct_net#166, nov_net#167, dec_net#168]
 Keys [8]: [w_warehouse_name#17, w_warehouse_sq_ft#18, w_city#19, w_county#20, w_state#21, w_country#22, ship_carriers#143, year#144]
-Functions [36]: [partial_sum(jan_sales#145), partial_sum(feb_sales#146), partial_sum(mar_sales#147), partial_sum(apr_sales#148), partial_sum(may_sales#149), partial_sum(jun_sales#150), partial_sum(jul_sales#151), partial_sum(aug_sales#152), partial_sum(sep_sales#153), partial_sum(oct_sales#154), partial_sum(nov_sales#155), partial_sum(dec_sales#156), partial_sum(CheckOverflow((promote_precision(jan_sales#145) / promote_precision(cast(w_warehouse_sq_ft#18 as decimal(28,2)))), DecimalType( [...]
+Functions [36]: [partial_sum(jan_sales#145), partial_sum(feb_sales#146), partial_sum(mar_sales#147), partial_sum(apr_sales#148), partial_sum(may_sales#149), partial_sum(jun_sales#150), partial_sum(jul_sales#151), partial_sum(aug_sales#152), partial_sum(sep_sales#153), partial_sum(oct_sales#154), partial_sum(nov_sales#155), partial_sum(dec_sales#156), partial_sum((jan_sales#145 / cast(w_warehouse_sq_ft#18 as decimal(10,0)))), partial_sum((feb_sales#146 / cast(w_warehouse_sq_ft#18 as decim [...]
 Aggregate Attributes [72]: [sum#334, isEmpty#335, sum#336, isEmpty#337, sum#338, isEmpty#339, sum#340, isEmpty#341, sum#342, isEmpty#343, sum#344, isEmpty#345, sum#346, isEmpty#347, sum#348, isEmpty#349, sum#350, isEmpty#351, sum#352, isEmpty#353, sum#354, isEmpty#355, sum#356, isEmpty#357, sum#358, isEmpty#359, sum#360, isEmpty#361, sum#362, isEmpty#363, sum#364, isEmpty#365, sum#366, isEmpty#367, sum#368, isEmpty#369, sum#370, isEmpty#371, sum#372, isEmpty#373, sum#374, isEmpty#375, su [...]
 Results [80]: [w_warehouse_name#17, w_warehouse_sq_ft#18, w_city#19, w_county#20, w_state#21, w_country#22, ship_carriers#143, year#144, sum#406, isEmpty#407, sum#408, isEmpty#409, sum#410, isEmpty#411, sum#412, isEmpty#413, sum#414, isEmpty#415, sum#416, isEmpty#417, sum#418, isEmpty#419, sum#420, isEmpty#421, sum#422, isEmpty#423, sum#424, isEmpty#425, sum#426, isEmpty#427, sum#428, isEmpty#429, sum#430, isEmpty#431, sum#432, isEmpty#433, sum#434, isEmpty#435, sum#436, isEmpty#437, sum [...]
 
@@ -284,9 +284,9 @@ Arguments: hashpartitioning(w_warehouse_name#17, w_warehouse_sq_ft#18, w_city#19
 (51) HashAggregate [codegen id : 14]
 Input [80]: [w_warehouse_name#17, w_warehouse_sq_ft#18, w_city#19, w_county#20, w_state#21, w_country#22, ship_carriers#143, year#144, sum#406, isEmpty#407, sum#408, isEmpty#409, sum#410, isEmpty#411, sum#412, isEmpty#413, sum#414, isEmpty#415, sum#416, isEmpty#417, sum#418, isEmpty#419, sum#420, isEmpty#421, sum#422, isEmpty#423, sum#424, isEmpty#425, sum#426, isEmpty#427, sum#428, isEmpty#429, sum#430, isEmpty#431, sum#432, isEmpty#433, sum#434, isEmpty#435, sum#436, isEmpty#437, sum#4 [...]
 Keys [8]: [w_warehouse_name#17, w_warehouse_sq_ft#18, w_city#19, w_county#20, w_state#21, w_country#22, ship_carriers#143, year#144]
-Functions [36]: [sum(jan_sales#145), sum(feb_sales#146), sum(mar_sales#147), sum(apr_sales#148), sum(may_sales#149), sum(jun_sales#150), sum(jul_sales#151), sum(aug_sales#152), sum(sep_sales#153), sum(oct_sales#154), sum(nov_sales#155), sum(dec_sales#156), sum(CheckOverflow((promote_precision(jan_sales#145) / promote_precision(cast(w_warehouse_sq_ft#18 as decimal(28,2)))), DecimalType(38,12))), sum(CheckOverflow((promote_precision(feb_sales#146) / promote_precision(cast(w_warehouse_sq_ft [...]
-Aggregate Attributes [36]: [sum(jan_sales#145)#478, sum(feb_sales#146)#479, sum(mar_sales#147)#480, sum(apr_sales#148)#481, sum(may_sales#149)#482, sum(jun_sales#150)#483, sum(jul_sales#151)#484, sum(aug_sales#152)#485, sum(sep_sales#153)#486, sum(oct_sales#154)#487, sum(nov_sales#155)#488, sum(dec_sales#156)#489, sum(CheckOverflow((promote_precision(jan_sales#145) / promote_precision(cast(w_warehouse_sq_ft#18 as decimal(28,2)))), DecimalType(38,12)))#490, sum(CheckOverflow((promote_prec [...]
-Results [44]: [w_warehouse_name#17, w_warehouse_sq_ft#18, w_city#19, w_county#20, w_state#21, w_country#22, ship_carriers#143, year#144, sum(jan_sales#145)#478 AS jan_sales#514, sum(feb_sales#146)#479 AS feb_sales#515, sum(mar_sales#147)#480 AS mar_sales#516, sum(apr_sales#148)#481 AS apr_sales#517, sum(may_sales#149)#482 AS may_sales#518, sum(jun_sales#150)#483 AS jun_sales#519, sum(jul_sales#151)#484 AS jul_sales#520, sum(aug_sales#152)#485 AS aug_sales#521, sum(sep_sales#153)#486 AS s [...]
+Functions [36]: [sum(jan_sales#145), sum(feb_sales#146), sum(mar_sales#147), sum(apr_sales#148), sum(may_sales#149), sum(jun_sales#150), sum(jul_sales#151), sum(aug_sales#152), sum(sep_sales#153), sum(oct_sales#154), sum(nov_sales#155), sum(dec_sales#156), sum((jan_sales#145 / cast(w_warehouse_sq_ft#18 as decimal(10,0)))), sum((feb_sales#146 / cast(w_warehouse_sq_ft#18 as decimal(10,0)))), sum((mar_sales#147 / cast(w_warehouse_sq_ft#18 as decimal(10,0)))), sum((apr_sales#148 / cast(w_war [...]
+Aggregate Attributes [36]: [sum(jan_sales#145)#478, sum(feb_sales#146)#479, sum(mar_sales#147)#480, sum(apr_sales#148)#481, sum(may_sales#149)#482, sum(jun_sales#150)#483, sum(jul_sales#151)#484, sum(aug_sales#152)#485, sum(sep_sales#153)#486, sum(oct_sales#154)#487, sum(nov_sales#155)#488, sum(dec_sales#156)#489, sum((jan_sales#145 / cast(w_warehouse_sq_ft#18 as decimal(10,0))))#490, sum((feb_sales#146 / cast(w_warehouse_sq_ft#18 as decimal(10,0))))#491, sum((mar_sales#147 / cast(w_ware [...]
+Results [44]: [w_warehouse_name#17, w_warehouse_sq_ft#18, w_city#19, w_county#20, w_state#21, w_country#22, ship_carriers#143, year#144, sum(jan_sales#145)#478 AS jan_sales#514, sum(feb_sales#146)#479 AS feb_sales#515, sum(mar_sales#147)#480 AS mar_sales#516, sum(apr_sales#148)#481 AS apr_sales#517, sum(may_sales#149)#482 AS may_sales#518, sum(jun_sales#150)#483 AS jun_sales#519, sum(jul_sales#151)#484 AS jul_sales#520, sum(aug_sales#152)#485 AS aug_sales#521, sum(sep_sales#153)#486 AS s [...]
 
 (52) TakeOrderedAndProject
 Input [44]: [w_warehouse_name#17, w_warehouse_sq_ft#18, w_city#19, w_county#20, w_state#21, w_country#22, ship_carriers#143, year#144, jan_sales#514, feb_sales#515, mar_sales#516, apr_sales#517, may_sales#518, jun_sales#519, jul_sales#520, aug_sales#521, sep_sales#522, oct_sales#523, nov_sales#524, dec_sales#525, jan_sales_per_sq_foot#526, feb_sales_per_sq_foot#527, mar_sales_per_sq_foot#528, apr_sales_per_sq_foot#529, may_sales_per_sq_foot#530, jun_sales_per_sq_foot#531, jul_sales_per_s [...]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.sf100/simplified.txt
index d9ac8f54234..f6212d810d7 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.sf100/simplified.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66.sf100/simplified.txt
@@ -1,6 +1,6 @@
 TakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_sales_per_sq_foot,feb_sales_per_sq_foot,mar_sales_per_sq_foot,apr_sales_per_sq_foot,may_sales_per_sq_foot,jun_sales_per_sq_foot,jul_sales_per_sq_foot,aug_sales_per_sq_foot,sep_sales_per_sq_foot,oct_sales_per_sq_foot,nov_sales_per_sq_foot,dec_sales_per_sq_fo [...]
   WholeStageCodegen (14)
-    HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,s [...]
+    HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,s [...]
       InputAdapter
         Exchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year] #1
           WholeStageCodegen (13)
@@ -8,7 +8,7 @@ TakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_stat
               InputAdapter
                 Union
                   WholeStageCodegen (6)
-                    HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(CASE WHEN (d_moy = 1) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price as  [...]
+                    HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(CASE WHEN (d_moy = 1) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) EL [...]
                       InputAdapter
                         Exchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #2
                           WholeStageCodegen (5)
@@ -58,7 +58,7 @@ TakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_stat
                                             InputAdapter
                                               Scan parquet default.warehouse [w_warehouse_sk,w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country]
                   WholeStageCodegen (12)
-                    HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(CASE WHEN (d_moy = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price as deci [...]
+                    HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(CASE WHEN (d_moy = 1) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0 [...]
                       InputAdapter
                         Exchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #7
                           WholeStageCodegen (11)
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/explain.txt
index b2e7d4ee9a6..56a9ea6a007 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/explain.txt
@@ -172,7 +172,7 @@ Input [13]: [ws_ship_mode_sk#2, ws_quantity#4, ws_ext_sales_price#5, ws_net_paid
 (27) HashAggregate [codegen id : 5]
 Input [11]: [ws_quantity#4, ws_ext_sales_price#5, ws_net_paid#6, w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, d_moy#18]
 Keys [7]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17]
-Functions [24]: [partial_sum(CASE WHEN (d_moy#18 = 1) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#5 as decimal(12,2))) * promote_precision(cast(ws_quantity#4 as decimal(12,2)))), DecimalType(18,2)) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 2) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#5 as decimal(12,2))) * promote_precision(cast(ws_quantity#4 as decimal(12,2)))), DecimalType(18,2)) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 3) THEN Che [...]
+Functions [24]: [partial_sum(CASE WHEN (d_moy#18 = 1) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 2) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 3) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#18 = 4) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 E [...]
 Aggregate Attributes [48]: [sum#23, isEmpty#24, sum#25, isEmpty#26, sum#27, isEmpty#28, sum#29, isEmpty#30, sum#31, isEmpty#32, sum#33, isEmpty#34, sum#35, isEmpty#36, sum#37, isEmpty#38, sum#39, isEmpty#40, sum#41, isEmpty#42, sum#43, isEmpty#44, sum#45, isEmpty#46, sum#47, isEmpty#48, sum#49, isEmpty#50, sum#51, isEmpty#52, sum#53, isEmpty#54, sum#55, isEmpty#56, sum#57, isEmpty#58, sum#59, isEmpty#60, sum#61, isEmpty#62, sum#63, isEmpty#64, sum#65, isEmpty#66, sum#67, isEmpty#68, sum# [...]
 Results [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, sum#71, isEmpty#72, sum#73, isEmpty#74, sum#75, isEmpty#76, sum#77, isEmpty#78, sum#79, isEmpty#80, sum#81, isEmpty#82, sum#83, isEmpty#84, sum#85, isEmpty#86, sum#87, isEmpty#88, sum#89, isEmpty#90, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96, sum#97, isEmpty#98, sum#99, isEmpty#100, sum#101, isEmpty#102, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#10 [...]
 
@@ -183,9 +183,9 @@ Arguments: hashpartitioning(w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12
 (29) HashAggregate [codegen id : 6]
 Input [55]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17, sum#71, isEmpty#72, sum#73, isEmpty#74, sum#75, isEmpty#76, sum#77, isEmpty#78, sum#79, isEmpty#80, sum#81, isEmpty#82, sum#83, isEmpty#84, sum#85, isEmpty#86, sum#87, isEmpty#88, sum#89, isEmpty#90, sum#91, isEmpty#92, sum#93, isEmpty#94, sum#95, isEmpty#96, sum#97, isEmpty#98, sum#99, isEmpty#100, sum#101, isEmpty#102, sum#103, isEmpty#104, sum#105, isEmpty#106, sum#107, [...]
 Keys [7]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, d_year#17]
-Functions [24]: [sum(CASE WHEN (d_moy#18 = 1) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#5 as decimal(12,2))) * promote_precision(cast(ws_quantity#4 as decimal(12,2)))), DecimalType(18,2)) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 2) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#5 as decimal(12,2))) * promote_precision(cast(ws_quantity#4 as decimal(12,2)))), DecimalType(18,2)) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 3) THEN CheckOverflow((promote_prec [...]
-Aggregate Attributes [24]: [sum(CASE WHEN (d_moy#18 = 1) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#5 as decimal(12,2))) * promote_precision(cast(ws_quantity#4 as decimal(12,2)))), DecimalType(18,2)) ELSE 0.00 END)#119, sum(CASE WHEN (d_moy#18 = 2) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#5 as decimal(12,2))) * promote_precision(cast(ws_quantity#4 as decimal(12,2)))), DecimalType(18,2)) ELSE 0.00 END)#120, sum(CASE WHEN (d_moy#18 = 3) THEN CheckOve [...]
-Results [32]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, DHL,BARIAN AS ship_carriers#143, d_year#17 AS year#144, sum(CASE WHEN (d_moy#18 = 1) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#5 as decimal(12,2))) * promote_precision(cast(ws_quantity#4 as decimal(12,2)))), DecimalType(18,2)) ELSE 0.00 END)#119 AS jan_sales#145, sum(CASE WHEN (d_moy#18 = 2) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price#5 as deci [...]
+Functions [24]: [sum(CASE WHEN (d_moy#18 = 1) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 2) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 3) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 4) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 = 5 [...]
+Aggregate Attributes [24]: [sum(CASE WHEN (d_moy#18 = 1) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#119, sum(CASE WHEN (d_moy#18 = 2) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#120, sum(CASE WHEN (d_moy#18 = 3) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#121, sum(CASE WHEN (d_moy#18 = 4) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#122,  [...]
+Results [32]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, DHL,BARIAN AS ship_carriers#143, d_year#17 AS year#144, sum(CASE WHEN (d_moy#18 = 1) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#119 AS jan_sales#145, sum(CASE WHEN (d_moy#18 = 2) THEN (ws_ext_sales_price#5 * cast(ws_quantity#4 as decimal(10,0))) ELSE 0.00 END)#120 AS feb_sales#146, sum(CASE WHEN (d_moy#18 = 3) THEN (ws_ext_sales_price#5 * cast [...]
 
 (30) Scan parquet default.catalog_sales
 Output [7]: [cs_sold_time_sk#169, cs_ship_mode_sk#170, cs_warehouse_sk#171, cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, cs_sold_date_sk#175]
@@ -253,7 +253,7 @@ Input [13]: [cs_ship_mode_sk#170, cs_quantity#172, cs_sales_price#173, cs_net_pa
 (45) HashAggregate [codegen id : 11]
 Input [11]: [cs_quantity#172, cs_sales_price#173, cs_net_paid_inc_tax#174, w_warehouse_name#177, w_warehouse_sq_ft#178, w_city#179, w_county#180, w_state#181, w_country#182, d_year#184, d_moy#185]
 Keys [7]: [w_warehouse_name#177, w_warehouse_sq_ft#178, w_city#179, w_county#180, w_state#181, w_country#182, d_year#184]
-Functions [24]: [partial_sum(CASE WHEN (d_moy#185 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#173 as decimal(12,2))) * promote_precision(cast(cs_quantity#172 as decimal(12,2)))), DecimalType(18,2)) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#173 as decimal(12,2))) * promote_precision(cast(cs_quantity#172 as decimal(12,2)))), DecimalType(18,2)) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 3) THEN  [...]
+Functions [24]: [partial_sum(CASE WHEN (d_moy#185 = 1) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 2) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 3) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), partial_sum(CASE WHEN (d_moy#185 = 4) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0. [...]
 Aggregate Attributes [48]: [sum#188, isEmpty#189, sum#190, isEmpty#191, sum#192, isEmpty#193, sum#194, isEmpty#195, sum#196, isEmpty#197, sum#198, isEmpty#199, sum#200, isEmpty#201, sum#202, isEmpty#203, sum#204, isEmpty#205, sum#206, isEmpty#207, sum#208, isEmpty#209, sum#210, isEmpty#211, sum#212, isEmpty#213, sum#214, isEmpty#215, sum#216, isEmpty#217, sum#218, isEmpty#219, sum#220, isEmpty#221, sum#222, isEmpty#223, sum#224, isEmpty#225, sum#226, isEmpty#227, sum#228, isEmpty#229, su [...]
 Results [55]: [w_warehouse_name#177, w_warehouse_sq_ft#178, w_city#179, w_county#180, w_state#181, w_country#182, d_year#184, sum#236, isEmpty#237, sum#238, isEmpty#239, sum#240, isEmpty#241, sum#242, isEmpty#243, sum#244, isEmpty#245, sum#246, isEmpty#247, sum#248, isEmpty#249, sum#250, isEmpty#251, sum#252, isEmpty#253, sum#254, isEmpty#255, sum#256, isEmpty#257, sum#258, isEmpty#259, sum#260, isEmpty#261, sum#262, isEmpty#263, sum#264, isEmpty#265, sum#266, isEmpty#267, sum#268, isEmp [...]
 
@@ -264,16 +264,16 @@ Arguments: hashpartitioning(w_warehouse_name#177, w_warehouse_sq_ft#178, w_city#
 (47) HashAggregate [codegen id : 12]
 Input [55]: [w_warehouse_name#177, w_warehouse_sq_ft#178, w_city#179, w_county#180, w_state#181, w_country#182, d_year#184, sum#236, isEmpty#237, sum#238, isEmpty#239, sum#240, isEmpty#241, sum#242, isEmpty#243, sum#244, isEmpty#245, sum#246, isEmpty#247, sum#248, isEmpty#249, sum#250, isEmpty#251, sum#252, isEmpty#253, sum#254, isEmpty#255, sum#256, isEmpty#257, sum#258, isEmpty#259, sum#260, isEmpty#261, sum#262, isEmpty#263, sum#264, isEmpty#265, sum#266, isEmpty#267, sum#268, isEmpty [...]
 Keys [7]: [w_warehouse_name#177, w_warehouse_sq_ft#178, w_city#179, w_county#180, w_state#181, w_country#182, d_year#184]
-Functions [24]: [sum(CASE WHEN (d_moy#185 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#173 as decimal(12,2))) * promote_precision(cast(cs_quantity#172 as decimal(12,2)))), DecimalType(18,2)) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#173 as decimal(12,2))) * promote_precision(cast(cs_quantity#172 as decimal(12,2)))), DecimalType(18,2)) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 3) THEN CheckOverflow((promote_p [...]
-Aggregate Attributes [24]: [sum(CASE WHEN (d_moy#185 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#173 as decimal(12,2))) * promote_precision(cast(cs_quantity#172 as decimal(12,2)))), DecimalType(18,2)) ELSE 0.00 END)#284, sum(CASE WHEN (d_moy#185 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#173 as decimal(12,2))) * promote_precision(cast(cs_quantity#172 as decimal(12,2)))), DecimalType(18,2)) ELSE 0.00 END)#285, sum(CASE WHEN (d_moy#185 = 3) THEN Check [...]
-Results [32]: [w_warehouse_name#177, w_warehouse_sq_ft#178, w_city#179, w_county#180, w_state#181, w_country#182, DHL,BARIAN AS ship_carriers#308, d_year#184 AS year#309, sum(CASE WHEN (d_moy#185 = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price#173 as decimal(12,2))) * promote_precision(cast(cs_quantity#172 as decimal(12,2)))), DecimalType(18,2)) ELSE 0.00 END)#284 AS jan_sales#310, sum(CASE WHEN (d_moy#185 = 2) THEN CheckOverflow((promote_precision(cast(cs_sales_price#173  [...]
+Functions [24]: [sum(CASE WHEN (d_moy#185 = 1) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 2) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 3) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#185 = 4) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END), sum(CASE WHEN (d_moy#18 [...]
+Aggregate Attributes [24]: [sum(CASE WHEN (d_moy#185 = 1) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#284, sum(CASE WHEN (d_moy#185 = 2) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#285, sum(CASE WHEN (d_moy#185 = 3) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#286, sum(CASE WHEN (d_moy#185 = 4) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#2 [...]
+Results [32]: [w_warehouse_name#177, w_warehouse_sq_ft#178, w_city#179, w_county#180, w_state#181, w_country#182, DHL,BARIAN AS ship_carriers#308, d_year#184 AS year#309, sum(CASE WHEN (d_moy#185 = 1) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#284 AS jan_sales#310, sum(CASE WHEN (d_moy#185 = 2) THEN (cs_sales_price#173 * cast(cs_quantity#172 as decimal(10,0))) ELSE 0.00 END)#285 AS feb_sales#311, sum(CASE WHEN (d_moy#185 = 3) THEN (cs_sales_price#17 [...]
 
 (48) Union
 
 (49) HashAggregate [codegen id : 13]
 Input [32]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144, jan_sales#145, feb_sales#146, mar_sales#147, apr_sales#148, may_sales#149, jun_sales#150, jul_sales#151, aug_sales#152, sep_sales#153, oct_sales#154, nov_sales#155, dec_sales#156, jan_net#157, feb_net#158, mar_net#159, apr_net#160, may_net#161, jun_net#162, jul_net#163, aug_net#164, sep_net#165, oct_net#166, nov_net#167, dec_net#168]
 Keys [8]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144]
-Functions [36]: [partial_sum(jan_sales#145), partial_sum(feb_sales#146), partial_sum(mar_sales#147), partial_sum(apr_sales#148), partial_sum(may_sales#149), partial_sum(jun_sales#150), partial_sum(jul_sales#151), partial_sum(aug_sales#152), partial_sum(sep_sales#153), partial_sum(oct_sales#154), partial_sum(nov_sales#155), partial_sum(dec_sales#156), partial_sum(CheckOverflow((promote_precision(jan_sales#145) / promote_precision(cast(w_warehouse_sq_ft#11 as decimal(28,2)))), DecimalType( [...]
+Functions [36]: [partial_sum(jan_sales#145), partial_sum(feb_sales#146), partial_sum(mar_sales#147), partial_sum(apr_sales#148), partial_sum(may_sales#149), partial_sum(jun_sales#150), partial_sum(jul_sales#151), partial_sum(aug_sales#152), partial_sum(sep_sales#153), partial_sum(oct_sales#154), partial_sum(nov_sales#155), partial_sum(dec_sales#156), partial_sum((jan_sales#145 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), partial_sum((feb_sales#146 / cast(w_warehouse_sq_ft#11 as decim [...]
 Aggregate Attributes [72]: [sum#334, isEmpty#335, sum#336, isEmpty#337, sum#338, isEmpty#339, sum#340, isEmpty#341, sum#342, isEmpty#343, sum#344, isEmpty#345, sum#346, isEmpty#347, sum#348, isEmpty#349, sum#350, isEmpty#351, sum#352, isEmpty#353, sum#354, isEmpty#355, sum#356, isEmpty#357, sum#358, isEmpty#359, sum#360, isEmpty#361, sum#362, isEmpty#363, sum#364, isEmpty#365, sum#366, isEmpty#367, sum#368, isEmpty#369, sum#370, isEmpty#371, sum#372, isEmpty#373, sum#374, isEmpty#375, su [...]
 Results [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144, sum#406, isEmpty#407, sum#408, isEmpty#409, sum#410, isEmpty#411, sum#412, isEmpty#413, sum#414, isEmpty#415, sum#416, isEmpty#417, sum#418, isEmpty#419, sum#420, isEmpty#421, sum#422, isEmpty#423, sum#424, isEmpty#425, sum#426, isEmpty#427, sum#428, isEmpty#429, sum#430, isEmpty#431, sum#432, isEmpty#433, sum#434, isEmpty#435, sum#436, isEmpty#437, sum [...]
 
@@ -284,9 +284,9 @@ Arguments: hashpartitioning(w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12
 (51) HashAggregate [codegen id : 14]
 Input [80]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144, sum#406, isEmpty#407, sum#408, isEmpty#409, sum#410, isEmpty#411, sum#412, isEmpty#413, sum#414, isEmpty#415, sum#416, isEmpty#417, sum#418, isEmpty#419, sum#420, isEmpty#421, sum#422, isEmpty#423, sum#424, isEmpty#425, sum#426, isEmpty#427, sum#428, isEmpty#429, sum#430, isEmpty#431, sum#432, isEmpty#433, sum#434, isEmpty#435, sum#436, isEmpty#437, sum#4 [...]
 Keys [8]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144]
-Functions [36]: [sum(jan_sales#145), sum(feb_sales#146), sum(mar_sales#147), sum(apr_sales#148), sum(may_sales#149), sum(jun_sales#150), sum(jul_sales#151), sum(aug_sales#152), sum(sep_sales#153), sum(oct_sales#154), sum(nov_sales#155), sum(dec_sales#156), sum(CheckOverflow((promote_precision(jan_sales#145) / promote_precision(cast(w_warehouse_sq_ft#11 as decimal(28,2)))), DecimalType(38,12))), sum(CheckOverflow((promote_precision(feb_sales#146) / promote_precision(cast(w_warehouse_sq_ft [...]
-Aggregate Attributes [36]: [sum(jan_sales#145)#478, sum(feb_sales#146)#479, sum(mar_sales#147)#480, sum(apr_sales#148)#481, sum(may_sales#149)#482, sum(jun_sales#150)#483, sum(jul_sales#151)#484, sum(aug_sales#152)#485, sum(sep_sales#153)#486, sum(oct_sales#154)#487, sum(nov_sales#155)#488, sum(dec_sales#156)#489, sum(CheckOverflow((promote_precision(jan_sales#145) / promote_precision(cast(w_warehouse_sq_ft#11 as decimal(28,2)))), DecimalType(38,12)))#490, sum(CheckOverflow((promote_prec [...]
-Results [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144, sum(jan_sales#145)#478 AS jan_sales#514, sum(feb_sales#146)#479 AS feb_sales#515, sum(mar_sales#147)#480 AS mar_sales#516, sum(apr_sales#148)#481 AS apr_sales#517, sum(may_sales#149)#482 AS may_sales#518, sum(jun_sales#150)#483 AS jun_sales#519, sum(jul_sales#151)#484 AS jul_sales#520, sum(aug_sales#152)#485 AS aug_sales#521, sum(sep_sales#153)#486 AS s [...]
+Functions [36]: [sum(jan_sales#145), sum(feb_sales#146), sum(mar_sales#147), sum(apr_sales#148), sum(may_sales#149), sum(jun_sales#150), sum(jul_sales#151), sum(aug_sales#152), sum(sep_sales#153), sum(oct_sales#154), sum(nov_sales#155), sum(dec_sales#156), sum((jan_sales#145 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((feb_sales#146 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((mar_sales#147 / cast(w_warehouse_sq_ft#11 as decimal(10,0)))), sum((apr_sales#148 / cast(w_war [...]
+Aggregate Attributes [36]: [sum(jan_sales#145)#478, sum(feb_sales#146)#479, sum(mar_sales#147)#480, sum(apr_sales#148)#481, sum(may_sales#149)#482, sum(jun_sales#150)#483, sum(jul_sales#151)#484, sum(aug_sales#152)#485, sum(sep_sales#153)#486, sum(oct_sales#154)#487, sum(nov_sales#155)#488, sum(dec_sales#156)#489, sum((jan_sales#145 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#490, sum((feb_sales#146 / cast(w_warehouse_sq_ft#11 as decimal(10,0))))#491, sum((mar_sales#147 / cast(w_ware [...]
+Results [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144, sum(jan_sales#145)#478 AS jan_sales#514, sum(feb_sales#146)#479 AS feb_sales#515, sum(mar_sales#147)#480 AS mar_sales#516, sum(apr_sales#148)#481 AS apr_sales#517, sum(may_sales#149)#482 AS may_sales#518, sum(jun_sales#150)#483 AS jun_sales#519, sum(jul_sales#151)#484 AS jul_sales#520, sum(aug_sales#152)#485 AS aug_sales#521, sum(sep_sales#153)#486 AS s [...]
 
 (52) TakeOrderedAndProject
 Input [44]: [w_warehouse_name#10, w_warehouse_sq_ft#11, w_city#12, w_county#13, w_state#14, w_country#15, ship_carriers#143, year#144, jan_sales#514, feb_sales#515, mar_sales#516, apr_sales#517, may_sales#518, jun_sales#519, jul_sales#520, aug_sales#521, sep_sales#522, oct_sales#523, nov_sales#524, dec_sales#525, jan_sales_per_sq_foot#526, feb_sales_per_sq_foot#527, mar_sales_per_sq_foot#528, apr_sales_per_sq_foot#529, may_sales_per_sq_foot#530, jun_sales_per_sq_foot#531, jul_sales_per_s [...]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/simplified.txt
index 17037cfe02c..b5981499990 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/simplified.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q66/simplified.txt
@@ -1,6 +1,6 @@
 TakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year,jan_sales,feb_sales,mar_sales,apr_sales,may_sales,jun_sales,jul_sales,aug_sales,sep_sales,oct_sales,nov_sales,dec_sales,jan_sales_per_sq_foot,feb_sales_per_sq_foot,mar_sales_per_sq_foot,apr_sales_per_sq_foot,may_sales_per_sq_foot,jun_sales_per_sq_foot,jul_sales_per_sq_foot,aug_sales_per_sq_foot,sep_sales_per_sq_foot,oct_sales_per_sq_foot,nov_sales_per_sq_foot,dec_sales_per_sq_fo [...]
   WholeStageCodegen (14)
-    HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,s [...]
+    HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,s [...]
       InputAdapter
         Exchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,ship_carriers,year] #1
           WholeStageCodegen (13)
@@ -8,7 +8,7 @@ TakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_stat
               InputAdapter
                 Union
                   WholeStageCodegen (6)
-                    HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(CASE WHEN (d_moy = 1) THEN CheckOverflow((promote_precision(cast(ws_ext_sales_price as  [...]
+                    HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(CASE WHEN (d_moy = 1) THEN (ws_ext_sales_price * cast(ws_quantity as decimal(10,0))) EL [...]
                       InputAdapter
                         Exchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #2
                           WholeStageCodegen (5)
@@ -58,7 +58,7 @@ TakeOrderedAndProject [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_stat
                                               InputAdapter
                                                 Scan parquet default.ship_mode [sm_ship_mode_sk,sm_carrier]
                   WholeStageCodegen (12)
-                    HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(CASE WHEN (d_moy = 1) THEN CheckOverflow((promote_precision(cast(cs_sales_price as deci [...]
+                    HashAggregate [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty,sum,isEmpty] [sum(CASE WHEN (d_moy = 1) THEN (cs_sales_price * cast(cs_quantity as decimal(10,0))) ELSE 0 [...]
                       InputAdapter
                         Exchange [w_warehouse_name,w_warehouse_sq_ft,w_city,w_county,w_state,w_country,d_year] #7
                           WholeStageCodegen (11)
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.sf100/explain.txt
index e9b22c379e5..e73afae1553 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.sf100/explain.txt
@@ -131,7 +131,7 @@ Arguments: [[ss_quantity#3, ss_sales_price#4, i_category#16, i_class#15, i_brand
 (23) HashAggregate [codegen id : 7]
 Input [11]: [ss_quantity#3, ss_sales_price#4, i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, spark_grouping_id#26]
 Keys [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, spark_grouping_id#26]
-Functions [1]: [partial_sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(ss_quantity#3 as decimal(12,2)))), DecimalType(18,2)), 0.00))]
+Functions [1]: [partial_sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))]
 Aggregate Attributes [2]: [sum#27, isEmpty#28]
 Results [11]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, spark_grouping_id#26, sum#29, isEmpty#30]
 
@@ -142,9 +142,9 @@ Arguments: hashpartitioning(i_category#18, i_class#19, i_brand#20, i_product_nam
 (25) HashAggregate [codegen id : 8]
 Input [11]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, spark_grouping_id#26, sum#29, isEmpty#30]
 Keys [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, spark_grouping_id#26]
-Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(ss_quantity#3 as decimal(12,2)))), DecimalType(18,2)), 0.00))]
-Aggregate Attributes [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(ss_quantity#3 as decimal(12,2)))), DecimalType(18,2)), 0.00))#31]
-Results [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(ss_quantity#3 as decimal(12,2)))), DecimalType(18,2)), 0.00))#31 AS sumsales#32]
+Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))]
+Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#31]
+Results [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#31 AS sumsales#32]
 
 (26) Exchange
 Input [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.sf100/simplified.txt
index 55953a73ff1..bac0b582475 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.sf100/simplified.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67.sf100/simplified.txt
@@ -8,7 +8,7 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_
               InputAdapter
                 Exchange [i_category] #1
                   WholeStageCodegen (8)
-                    HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,sum,isEmpty] [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price as decimal(12,2))) * promote_precision(cast(ss_quantity as decimal(12,2)))), DecimalType(18,2)), 0.00)),sumsales,sum,isEmpty]
+                    HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty]
                       InputAdapter
                         Exchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id] #2
                           WholeStageCodegen (7)
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/explain.txt
index 1619d25d1c1..c1fa9da5284 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/explain.txt
@@ -116,7 +116,7 @@ Arguments: [[ss_quantity#3, ss_sales_price#4, i_category#16, i_class#15, i_brand
 (20) HashAggregate [codegen id : 4]
 Input [11]: [ss_quantity#3, ss_sales_price#4, i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, spark_grouping_id#26]
 Keys [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, spark_grouping_id#26]
-Functions [1]: [partial_sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(ss_quantity#3 as decimal(12,2)))), DecimalType(18,2)), 0.00))]
+Functions [1]: [partial_sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))]
 Aggregate Attributes [2]: [sum#27, isEmpty#28]
 Results [11]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, spark_grouping_id#26, sum#29, isEmpty#30]
 
@@ -127,9 +127,9 @@ Arguments: hashpartitioning(i_category#18, i_class#19, i_brand#20, i_product_nam
 (22) HashAggregate [codegen id : 5]
 Input [11]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, spark_grouping_id#26, sum#29, isEmpty#30]
 Keys [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, spark_grouping_id#26]
-Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(ss_quantity#3 as decimal(12,2)))), DecimalType(18,2)), 0.00))]
-Aggregate Attributes [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(ss_quantity#3 as decimal(12,2)))), DecimalType(18,2)), 0.00))#31]
-Results [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(ss_quantity#3 as decimal(12,2)))), DecimalType(18,2)), 0.00))#31 AS sumsales#32]
+Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))]
+Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#31]
+Results [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#31 AS sumsales#32]
 
 (23) Exchange
 Input [9]: [i_category#18, i_class#19, i_brand#20, i_product_name#21, d_year#22, d_qoy#23, d_moy#24, s_store_id#25, sumsales#32]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/simplified.txt
index 3cb879f7019..36934e4207a 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/simplified.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q67/simplified.txt
@@ -8,7 +8,7 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_
               InputAdapter
                 Exchange [i_category] #1
                   WholeStageCodegen (5)
-                    HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,sum,isEmpty] [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price as decimal(12,2))) * promote_precision(cast(ss_quantity as decimal(12,2)))), DecimalType(18,2)), 0.00)),sumsales,sum,isEmpty]
+                    HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty]
                       InputAdapter
                         Exchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,spark_grouping_id] #2
                           WholeStageCodegen (4)
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.sf100/explain.txt
index ed253eecc37..b9f82e4d7ce 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77.sf100/explain.txt
@@ -225,7 +225,7 @@ Right keys [1]: [s_store_sk#21]
 Join condition: None
 
 (30) Project [codegen id : 8]
-Output [5]: [sales#14, coalesce(returns#28, 0.00) AS returns#30, CheckOverflow((promote_precision(cast(profit#15 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#29, 0.00) as decimal(18,2)))), DecimalType(18,2)) AS profit#31, store channel AS channel#32, s_store_sk#7 AS id#33]
+Output [5]: [sales#14, coalesce(returns#28, 0.00) AS returns#30, (profit#15 - coalesce(profit_loss#29, 0.00)) AS profit#31, store channel AS channel#32, s_store_sk#7 AS id#33]
 Input [6]: [s_store_sk#7, sales#14, profit#15, s_store_sk#21, returns#28, profit_loss#29]
 
 (31) Scan parquet default.catalog_sales
@@ -316,7 +316,7 @@ Arguments: IdentityBroadcastMode, [plan_id=7]
 Join condition: None
 
 (49) Project [codegen id : 14]
-Output [5]: [sales#45, returns#57, CheckOverflow((promote_precision(cast(profit#46 as decimal(18,2))) - promote_precision(cast(profit_loss#58 as decimal(18,2)))), DecimalType(18,2)) AS profit#59, catalog channel AS channel#60, cs_call_center_sk#34 AS id#61]
+Output [5]: [sales#45, returns#57, (profit#46 - profit_loss#58) AS profit#59, catalog channel AS channel#60, cs_call_center_sk#34 AS id#61]
 Input [5]: [cs_call_center_sk#34, sales#45, profit#46, returns#57, profit_loss#58]
 
 (50) Scan parquet default.web_sales
@@ -458,7 +458,7 @@ Right keys [1]: [wp_web_page_sk#81]
 Join condition: None
 
 (79) Project [codegen id : 22]
-Output [5]: [sales#74, coalesce(returns#88, 0.00) AS returns#90, CheckOverflow((promote_precision(cast(profit#75 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#89, 0.00) as decimal(18,2)))), DecimalType(18,2)) AS profit#91, web channel AS channel#92, wp_web_page_sk#67 AS id#93]
+Output [5]: [sales#74, coalesce(returns#88, 0.00) AS returns#90, (profit#75 - coalesce(profit_loss#89, 0.00)) AS profit#91, web channel AS channel#92, wp_web_page_sk#67 AS id#93]
 Input [6]: [wp_web_page_sk#67, sales#74, profit#75, wp_web_page_sk#81, returns#88, profit_loss#89]
 
 (80) Union
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/explain.txt
index 1406aa9f41e..b40a0fcfc34 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q77/explain.txt
@@ -225,7 +225,7 @@ Right keys [1]: [s_store_sk#21]
 Join condition: None
 
 (30) Project [codegen id : 8]
-Output [5]: [sales#14, coalesce(returns#28, 0.00) AS returns#30, CheckOverflow((promote_precision(cast(profit#15 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#29, 0.00) as decimal(18,2)))), DecimalType(18,2)) AS profit#31, store channel AS channel#32, s_store_sk#7 AS id#33]
+Output [5]: [sales#14, coalesce(returns#28, 0.00) AS returns#30, (profit#15 - coalesce(profit_loss#29, 0.00)) AS profit#31, store channel AS channel#32, s_store_sk#7 AS id#33]
 Input [6]: [s_store_sk#7, sales#14, profit#15, s_store_sk#21, returns#28, profit_loss#29]
 
 (31) Scan parquet default.catalog_sales
@@ -316,7 +316,7 @@ Results [2]: [MakeDecimal(sum(UnscaledValue(cr_return_amount#47))#55,17,2) AS re
 Join condition: None
 
 (49) Project [codegen id : 14]
-Output [5]: [sales#45, returns#57, CheckOverflow((promote_precision(cast(profit#46 as decimal(18,2))) - promote_precision(cast(profit_loss#58 as decimal(18,2)))), DecimalType(18,2)) AS profit#59, catalog channel AS channel#60, cs_call_center_sk#34 AS id#61]
+Output [5]: [sales#45, returns#57, (profit#46 - profit_loss#58) AS profit#59, catalog channel AS channel#60, cs_call_center_sk#34 AS id#61]
 Input [5]: [cs_call_center_sk#34, sales#45, profit#46, returns#57, profit_loss#58]
 
 (50) Scan parquet default.web_sales
@@ -458,7 +458,7 @@ Right keys [1]: [wp_web_page_sk#81]
 Join condition: None
 
 (79) Project [codegen id : 22]
-Output [5]: [sales#74, coalesce(returns#88, 0.00) AS returns#90, CheckOverflow((promote_precision(cast(profit#75 as decimal(18,2))) - promote_precision(cast(coalesce(profit_loss#89, 0.00) as decimal(18,2)))), DecimalType(18,2)) AS profit#91, web channel AS channel#92, wp_web_page_sk#67 AS id#93]
+Output [5]: [sales#74, coalesce(returns#88, 0.00) AS returns#90, (profit#75 - coalesce(profit_loss#89, 0.00)) AS profit#91, web channel AS channel#92, wp_web_page_sk#67 AS id#93]
 Input [6]: [wp_web_page_sk#67, sales#74, profit#75, wp_web_page_sk#81, returns#88, profit_loss#89]
 
 (80) Union
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.sf100/explain.txt
index 752e9d19f42..71f81ef7621 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.sf100/explain.txt
@@ -270,7 +270,7 @@ Input [7]: [ss_store_sk#2, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#
 (37) HashAggregate [codegen id : 9]
 Input [5]: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#15, sr_net_loss#16, s_store_id#24]
 Keys [1]: [s_store_id#24]
-Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#5)), partial_sum(coalesce(cast(sr_return_amt#15 as decimal(12,2)), 0.00)), partial_sum(CheckOverflow((promote_precision(cast(ss_net_profit#6 as decimal(13,2))) - promote_precision(cast(coalesce(cast(sr_net_loss#16 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2)))]
+Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#5)), partial_sum(coalesce(cast(sr_return_amt#15 as decimal(12,2)), 0.00)), partial_sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#16 as decimal(12,2)), 0.00)))]
 Aggregate Attributes [5]: [sum#25, sum#26, isEmpty#27, sum#28, isEmpty#29]
 Results [6]: [s_store_id#24, sum#30, sum#31, isEmpty#32, sum#33, isEmpty#34]
 
@@ -281,9 +281,9 @@ Arguments: hashpartitioning(s_store_id#24, 5), ENSURE_REQUIREMENTS, [plan_id=6]
 (39) HashAggregate [codegen id : 10]
 Input [6]: [s_store_id#24, sum#30, sum#31, isEmpty#32, sum#33, isEmpty#34]
 Keys [1]: [s_store_id#24]
-Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#5)), sum(coalesce(cast(sr_return_amt#15 as decimal(12,2)), 0.00)), sum(CheckOverflow((promote_precision(cast(ss_net_profit#6 as decimal(13,2))) - promote_precision(cast(coalesce(cast(sr_net_loss#16 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2)))]
-Aggregate Attributes [3]: [sum(UnscaledValue(ss_ext_sales_price#5))#35, sum(coalesce(cast(sr_return_amt#15 as decimal(12,2)), 0.00))#36, sum(CheckOverflow((promote_precision(cast(ss_net_profit#6 as decimal(13,2))) - promote_precision(cast(coalesce(cast(sr_net_loss#16 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2)))#37]
-Results [5]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#35,17,2) AS sales#38, sum(coalesce(cast(sr_return_amt#15 as decimal(12,2)), 0.00))#36 AS returns#39, sum(CheckOverflow((promote_precision(cast(ss_net_profit#6 as decimal(13,2))) - promote_precision(cast(coalesce(cast(sr_net_loss#16 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2)))#37 AS profit#40, store channel AS channel#41, concat(store, s_store_id#24) AS id#42]
+Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#5)), sum(coalesce(cast(sr_return_amt#15 as decimal(12,2)), 0.00)), sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#16 as decimal(12,2)), 0.00)))]
+Aggregate Attributes [3]: [sum(UnscaledValue(ss_ext_sales_price#5))#35, sum(coalesce(cast(sr_return_amt#15 as decimal(12,2)), 0.00))#36, sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#16 as decimal(12,2)), 0.00)))#37]
+Results [5]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#35,17,2) AS sales#38, sum(coalesce(cast(sr_return_amt#15 as decimal(12,2)), 0.00))#36 AS returns#39, sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#16 as decimal(12,2)), 0.00)))#37 AS profit#40, store channel AS channel#41, concat(store, s_store_id#24) AS id#42]
 
 (40) Scan parquet default.catalog_sales
 Output [7]: [cs_catalog_page_sk#43, cs_item_sk#44, cs_promo_sk#45, cs_order_number#46, cs_ext_sales_price#47, cs_net_profit#48, cs_sold_date_sk#49]
@@ -409,7 +409,7 @@ Input [7]: [cs_catalog_page_sk#43, cs_ext_sales_price#47, cs_net_profit#48, cr_r
 (68) HashAggregate [codegen id : 19]
 Input [5]: [cs_ext_sales_price#47, cs_net_profit#48, cr_return_amount#52, cr_net_loss#53, cp_catalog_page_id#59]
 Keys [1]: [cp_catalog_page_id#59]
-Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#47)), partial_sum(coalesce(cast(cr_return_amount#52 as decimal(12,2)), 0.00)), partial_sum(CheckOverflow((promote_precision(cast(cs_net_profit#48 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#53 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2)))]
+Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#47)), partial_sum(coalesce(cast(cr_return_amount#52 as decimal(12,2)), 0.00)), partial_sum((cs_net_profit#48 - coalesce(cast(cr_net_loss#53 as decimal(12,2)), 0.00)))]
 Aggregate Attributes [5]: [sum#60, sum#61, isEmpty#62, sum#63, isEmpty#64]
 Results [6]: [cp_catalog_page_id#59, sum#65, sum#66, isEmpty#67, sum#68, isEmpty#69]
 
@@ -420,9 +420,9 @@ Arguments: hashpartitioning(cp_catalog_page_id#59, 5), ENSURE_REQUIREMENTS, [pla
 (70) HashAggregate [codegen id : 20]
 Input [6]: [cp_catalog_page_id#59, sum#65, sum#66, isEmpty#67, sum#68, isEmpty#69]
 Keys [1]: [cp_catalog_page_id#59]
-Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#47)), sum(coalesce(cast(cr_return_amount#52 as decimal(12,2)), 0.00)), sum(CheckOverflow((promote_precision(cast(cs_net_profit#48 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#53 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2)))]
-Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#47))#70, sum(coalesce(cast(cr_return_amount#52 as decimal(12,2)), 0.00))#71, sum(CheckOverflow((promote_precision(cast(cs_net_profit#48 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#53 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2)))#72]
-Results [5]: [MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#47))#70,17,2) AS sales#73, sum(coalesce(cast(cr_return_amount#52 as decimal(12,2)), 0.00))#71 AS returns#74, sum(CheckOverflow((promote_precision(cast(cs_net_profit#48 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#53 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2)))#72 AS profit#75, catalog channel AS channel#76, concat(catalog_page, cp_catalog_page_id#59) AS id#77]
+Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#47)), sum(coalesce(cast(cr_return_amount#52 as decimal(12,2)), 0.00)), sum((cs_net_profit#48 - coalesce(cast(cr_net_loss#53 as decimal(12,2)), 0.00)))]
+Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#47))#70, sum(coalesce(cast(cr_return_amount#52 as decimal(12,2)), 0.00))#71, sum((cs_net_profit#48 - coalesce(cast(cr_net_loss#53 as decimal(12,2)), 0.00)))#72]
+Results [5]: [MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#47))#70,17,2) AS sales#73, sum(coalesce(cast(cr_return_amount#52 as decimal(12,2)), 0.00))#71 AS returns#74, sum((cs_net_profit#48 - coalesce(cast(cr_net_loss#53 as decimal(12,2)), 0.00)))#72 AS profit#75, catalog channel AS channel#76, concat(catalog_page, cp_catalog_page_id#59) AS id#77]
 
 (71) Scan parquet default.web_sales
 Output [7]: [ws_item_sk#78, ws_web_site_sk#79, ws_promo_sk#80, ws_order_number#81, ws_ext_sales_price#82, ws_net_profit#83, ws_sold_date_sk#84]
@@ -548,7 +548,7 @@ Input [7]: [ws_web_site_sk#79, ws_ext_sales_price#82, ws_net_profit#83, wr_retur
 (99) HashAggregate [codegen id : 29]
 Input [5]: [ws_ext_sales_price#82, ws_net_profit#83, wr_return_amt#87, wr_net_loss#88, web_site_id#94]
 Keys [1]: [web_site_id#94]
-Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#82)), partial_sum(coalesce(cast(wr_return_amt#87 as decimal(12,2)), 0.00)), partial_sum(CheckOverflow((promote_precision(cast(ws_net_profit#83 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#88 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2)))]
+Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#82)), partial_sum(coalesce(cast(wr_return_amt#87 as decimal(12,2)), 0.00)), partial_sum((ws_net_profit#83 - coalesce(cast(wr_net_loss#88 as decimal(12,2)), 0.00)))]
 Aggregate Attributes [5]: [sum#95, sum#96, isEmpty#97, sum#98, isEmpty#99]
 Results [6]: [web_site_id#94, sum#100, sum#101, isEmpty#102, sum#103, isEmpty#104]
 
@@ -559,9 +559,9 @@ Arguments: hashpartitioning(web_site_id#94, 5), ENSURE_REQUIREMENTS, [plan_id=14
 (101) HashAggregate [codegen id : 30]
 Input [6]: [web_site_id#94, sum#100, sum#101, isEmpty#102, sum#103, isEmpty#104]
 Keys [1]: [web_site_id#94]
-Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#82)), sum(coalesce(cast(wr_return_amt#87 as decimal(12,2)), 0.00)), sum(CheckOverflow((promote_precision(cast(ws_net_profit#83 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#88 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2)))]
-Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#82))#105, sum(coalesce(cast(wr_return_amt#87 as decimal(12,2)), 0.00))#106, sum(CheckOverflow((promote_precision(cast(ws_net_profit#83 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#88 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2)))#107]
-Results [5]: [MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#82))#105,17,2) AS sales#108, sum(coalesce(cast(wr_return_amt#87 as decimal(12,2)), 0.00))#106 AS returns#109, sum(CheckOverflow((promote_precision(cast(ws_net_profit#83 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#88 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2)))#107 AS profit#110, web channel AS channel#111, concat(web_site, web_site_id#94) AS id#112]
+Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#82)), sum(coalesce(cast(wr_return_amt#87 as decimal(12,2)), 0.00)), sum((ws_net_profit#83 - coalesce(cast(wr_net_loss#88 as decimal(12,2)), 0.00)))]
+Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#82))#105, sum(coalesce(cast(wr_return_amt#87 as decimal(12,2)), 0.00))#106, sum((ws_net_profit#83 - coalesce(cast(wr_net_loss#88 as decimal(12,2)), 0.00)))#107]
+Results [5]: [MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#82))#105,17,2) AS sales#108, sum(coalesce(cast(wr_return_amt#87 as decimal(12,2)), 0.00))#106 AS returns#109, sum((ws_net_profit#83 - coalesce(cast(wr_net_loss#88 as decimal(12,2)), 0.00)))#107 AS profit#110, web channel AS channel#111, concat(web_site, web_site_id#94) AS id#112]
 
 (102) Union
 
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.sf100/simplified.txt
index 5a7a2ef572e..c05a12e4612 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.sf100/simplified.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80.sf100/simplified.txt
@@ -9,7 +9,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
                 InputAdapter
                   Union
                     WholeStageCodegen (10)
-                      HashAggregate [s_store_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(CheckOverflow((promote_precision(cast(ss_net_profit as decimal(13,2))) - promote_precision(cast(coalesce(cast(sr_net_loss as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2))),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty]
+                      HashAggregate [s_store_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00))),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty]
                         InputAdapter
                           Exchange [s_store_id] #2
                             WholeStageCodegen (9)
@@ -99,7 +99,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
                                               InputAdapter
                                                 Scan parquet default.store [s_store_sk,s_store_id]
                     WholeStageCodegen (20)
-                      HashAggregate [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(CheckOverflow((promote_precision(cast(cs_net_profit as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2))),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty]
+                      HashAggregate [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00))),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty]
                         InputAdapter
                           Exchange [cp_catalog_page_id] #11
                             WholeStageCodegen (19)
@@ -152,7 +152,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
                                               InputAdapter
                                                 Scan parquet default.catalog_page [cp_catalog_page_sk,cp_catalog_page_id]
                     WholeStageCodegen (30)
-                      HashAggregate [web_site_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(CheckOverflow((promote_precision(cast(ws_net_profit as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2))),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty]
+                      HashAggregate [web_site_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00))),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty]
                         InputAdapter
                           Exchange [web_site_id] #15
                             WholeStageCodegen (29)
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/explain.txt
index daf8834a683..466c18112a3 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/explain.txt
@@ -270,7 +270,7 @@ Input [7]: [ss_promo_sk#3, ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#
 (37) HashAggregate [codegen id : 9]
 Input [5]: [ss_ext_sales_price#5, ss_net_profit#6, sr_return_amt#11, sr_net_loss#12, s_store_id#16]
 Keys [1]: [s_store_id#16]
-Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#5)), partial_sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), partial_sum(CheckOverflow((promote_precision(cast(ss_net_profit#6 as decimal(13,2))) - promote_precision(cast(coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2)))]
+Functions [3]: [partial_sum(UnscaledValue(ss_ext_sales_price#5)), partial_sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), partial_sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))]
 Aggregate Attributes [5]: [sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25]
 Results [6]: [s_store_id#16, sum#26, sum#27, isEmpty#28, sum#29, isEmpty#30]
 
@@ -281,9 +281,9 @@ Arguments: hashpartitioning(s_store_id#16, 5), ENSURE_REQUIREMENTS, [plan_id=6]
 (39) HashAggregate [codegen id : 10]
 Input [6]: [s_store_id#16, sum#26, sum#27, isEmpty#28, sum#29, isEmpty#30]
 Keys [1]: [s_store_id#16]
-Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#5)), sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), sum(CheckOverflow((promote_precision(cast(ss_net_profit#6 as decimal(13,2))) - promote_precision(cast(coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2)))]
-Aggregate Attributes [3]: [sum(UnscaledValue(ss_ext_sales_price#5))#31, sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00))#32, sum(CheckOverflow((promote_precision(cast(ss_net_profit#6 as decimal(13,2))) - promote_precision(cast(coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2)))#33]
-Results [5]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#31,17,2) AS sales#34, sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00))#32 AS returns#35, sum(CheckOverflow((promote_precision(cast(ss_net_profit#6 as decimal(13,2))) - promote_precision(cast(coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2)))#33 AS profit#36, store channel AS channel#37, concat(store, s_store_id#16) AS id#38]
+Functions [3]: [sum(UnscaledValue(ss_ext_sales_price#5)), sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00)), sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))]
+Aggregate Attributes [3]: [sum(UnscaledValue(ss_ext_sales_price#5))#31, sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00))#32, sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))#33]
+Results [5]: [MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#5))#31,17,2) AS sales#34, sum(coalesce(cast(sr_return_amt#11 as decimal(12,2)), 0.00))#32 AS returns#35, sum((ss_net_profit#6 - coalesce(cast(sr_net_loss#12 as decimal(12,2)), 0.00)))#33 AS profit#36, store channel AS channel#37, concat(store, s_store_id#16) AS id#38]
 
 (40) Scan parquet default.catalog_sales
 Output [7]: [cs_catalog_page_sk#39, cs_item_sk#40, cs_promo_sk#41, cs_order_number#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45]
@@ -409,7 +409,7 @@ Input [7]: [cs_promo_sk#41, cs_ext_sales_price#43, cs_net_profit#44, cr_return_a
 (68) HashAggregate [codegen id : 19]
 Input [5]: [cs_ext_sales_price#43, cs_net_profit#44, cr_return_amount#48, cr_net_loss#49, cp_catalog_page_id#53]
 Keys [1]: [cp_catalog_page_id#53]
-Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#43)), partial_sum(coalesce(cast(cr_return_amount#48 as decimal(12,2)), 0.00)), partial_sum(CheckOverflow((promote_precision(cast(cs_net_profit#44 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#49 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2)))]
+Functions [3]: [partial_sum(UnscaledValue(cs_ext_sales_price#43)), partial_sum(coalesce(cast(cr_return_amount#48 as decimal(12,2)), 0.00)), partial_sum((cs_net_profit#44 - coalesce(cast(cr_net_loss#49 as decimal(12,2)), 0.00)))]
 Aggregate Attributes [5]: [sum#56, sum#57, isEmpty#58, sum#59, isEmpty#60]
 Results [6]: [cp_catalog_page_id#53, sum#61, sum#62, isEmpty#63, sum#64, isEmpty#65]
 
@@ -420,9 +420,9 @@ Arguments: hashpartitioning(cp_catalog_page_id#53, 5), ENSURE_REQUIREMENTS, [pla
 (70) HashAggregate [codegen id : 20]
 Input [6]: [cp_catalog_page_id#53, sum#61, sum#62, isEmpty#63, sum#64, isEmpty#65]
 Keys [1]: [cp_catalog_page_id#53]
-Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#43)), sum(coalesce(cast(cr_return_amount#48 as decimal(12,2)), 0.00)), sum(CheckOverflow((promote_precision(cast(cs_net_profit#44 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#49 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2)))]
-Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#43))#66, sum(coalesce(cast(cr_return_amount#48 as decimal(12,2)), 0.00))#67, sum(CheckOverflow((promote_precision(cast(cs_net_profit#44 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#49 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2)))#68]
-Results [5]: [MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#43))#66,17,2) AS sales#69, sum(coalesce(cast(cr_return_amount#48 as decimal(12,2)), 0.00))#67 AS returns#70, sum(CheckOverflow((promote_precision(cast(cs_net_profit#44 as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss#49 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2)))#68 AS profit#71, catalog channel AS channel#72, concat(catalog_page, cp_catalog_page_id#53) AS id#73]
+Functions [3]: [sum(UnscaledValue(cs_ext_sales_price#43)), sum(coalesce(cast(cr_return_amount#48 as decimal(12,2)), 0.00)), sum((cs_net_profit#44 - coalesce(cast(cr_net_loss#49 as decimal(12,2)), 0.00)))]
+Aggregate Attributes [3]: [sum(UnscaledValue(cs_ext_sales_price#43))#66, sum(coalesce(cast(cr_return_amount#48 as decimal(12,2)), 0.00))#67, sum((cs_net_profit#44 - coalesce(cast(cr_net_loss#49 as decimal(12,2)), 0.00)))#68]
+Results [5]: [MakeDecimal(sum(UnscaledValue(cs_ext_sales_price#43))#66,17,2) AS sales#69, sum(coalesce(cast(cr_return_amount#48 as decimal(12,2)), 0.00))#67 AS returns#70, sum((cs_net_profit#44 - coalesce(cast(cr_net_loss#49 as decimal(12,2)), 0.00)))#68 AS profit#71, catalog channel AS channel#72, concat(catalog_page, cp_catalog_page_id#53) AS id#73]
 
 (71) Scan parquet default.web_sales
 Output [7]: [ws_item_sk#74, ws_web_site_sk#75, ws_promo_sk#76, ws_order_number#77, ws_ext_sales_price#78, ws_net_profit#79, ws_sold_date_sk#80]
@@ -548,7 +548,7 @@ Input [7]: [ws_promo_sk#76, ws_ext_sales_price#78, ws_net_profit#79, wr_return_a
 (99) HashAggregate [codegen id : 29]
 Input [5]: [ws_ext_sales_price#78, ws_net_profit#79, wr_return_amt#83, wr_net_loss#84, web_site_id#88]
 Keys [1]: [web_site_id#88]
-Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#78)), partial_sum(coalesce(cast(wr_return_amt#83 as decimal(12,2)), 0.00)), partial_sum(CheckOverflow((promote_precision(cast(ws_net_profit#79 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#84 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2)))]
+Functions [3]: [partial_sum(UnscaledValue(ws_ext_sales_price#78)), partial_sum(coalesce(cast(wr_return_amt#83 as decimal(12,2)), 0.00)), partial_sum((ws_net_profit#79 - coalesce(cast(wr_net_loss#84 as decimal(12,2)), 0.00)))]
 Aggregate Attributes [5]: [sum#91, sum#92, isEmpty#93, sum#94, isEmpty#95]
 Results [6]: [web_site_id#88, sum#96, sum#97, isEmpty#98, sum#99, isEmpty#100]
 
@@ -559,9 +559,9 @@ Arguments: hashpartitioning(web_site_id#88, 5), ENSURE_REQUIREMENTS, [plan_id=14
 (101) HashAggregate [codegen id : 30]
 Input [6]: [web_site_id#88, sum#96, sum#97, isEmpty#98, sum#99, isEmpty#100]
 Keys [1]: [web_site_id#88]
-Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#78)), sum(coalesce(cast(wr_return_amt#83 as decimal(12,2)), 0.00)), sum(CheckOverflow((promote_precision(cast(ws_net_profit#79 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#84 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2)))]
-Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#78))#101, sum(coalesce(cast(wr_return_amt#83 as decimal(12,2)), 0.00))#102, sum(CheckOverflow((promote_precision(cast(ws_net_profit#79 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#84 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2)))#103]
-Results [5]: [MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#78))#101,17,2) AS sales#104, sum(coalesce(cast(wr_return_amt#83 as decimal(12,2)), 0.00))#102 AS returns#105, sum(CheckOverflow((promote_precision(cast(ws_net_profit#79 as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss#84 as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2)))#103 AS profit#106, web channel AS channel#107, concat(web_site, web_site_id#88) AS id#108]
+Functions [3]: [sum(UnscaledValue(ws_ext_sales_price#78)), sum(coalesce(cast(wr_return_amt#83 as decimal(12,2)), 0.00)), sum((ws_net_profit#79 - coalesce(cast(wr_net_loss#84 as decimal(12,2)), 0.00)))]
+Aggregate Attributes [3]: [sum(UnscaledValue(ws_ext_sales_price#78))#101, sum(coalesce(cast(wr_return_amt#83 as decimal(12,2)), 0.00))#102, sum((ws_net_profit#79 - coalesce(cast(wr_net_loss#84 as decimal(12,2)), 0.00)))#103]
+Results [5]: [MakeDecimal(sum(UnscaledValue(ws_ext_sales_price#78))#101,17,2) AS sales#104, sum(coalesce(cast(wr_return_amt#83 as decimal(12,2)), 0.00))#102 AS returns#105, sum((ws_net_profit#79 - coalesce(cast(wr_net_loss#84 as decimal(12,2)), 0.00)))#103 AS profit#106, web channel AS channel#107, concat(web_site, web_site_id#88) AS id#108]
 
 (102) Union
 
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/simplified.txt
index a6fd641bc24..c15d5e1cdc4 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/simplified.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q80/simplified.txt
@@ -9,7 +9,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
                 InputAdapter
                   Union
                     WholeStageCodegen (10)
-                      HashAggregate [s_store_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum(CheckOverflow((promote_precision(cast(ss_net_profit as decimal(13,2))) - promote_precision(cast(coalesce(cast(sr_net_loss as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2))),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty]
+                      HashAggregate [s_store_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ss_ext_sales_price)),sum(coalesce(cast(sr_return_amt as decimal(12,2)), 0.00)),sum((ss_net_profit - coalesce(cast(sr_net_loss as decimal(12,2)), 0.00))),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty]
                         InputAdapter
                           Exchange [s_store_id] #2
                             WholeStageCodegen (9)
@@ -79,7 +79,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
                                                 InputAdapter
                                                   Scan parquet default.promotion [p_promo_sk,p_channel_tv]
                     WholeStageCodegen (20)
-                      HashAggregate [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum(CheckOverflow((promote_precision(cast(cs_net_profit as decimal(13,2))) - promote_precision(cast(coalesce(cast(cr_net_loss as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2))),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty]
+                      HashAggregate [cp_catalog_page_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(cs_ext_sales_price)),sum(coalesce(cast(cr_return_amount as decimal(12,2)), 0.00)),sum((cs_net_profit - coalesce(cast(cr_net_loss as decimal(12,2)), 0.00))),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty]
                         InputAdapter
                           Exchange [cp_catalog_page_id] #9
                             WholeStageCodegen (19)
@@ -130,7 +130,7 @@ TakeOrderedAndProject [channel,id,sales,returns,profit]
                                     InputAdapter
                                       ReusedExchange [p_promo_sk] #8
                     WholeStageCodegen (30)
-                      HashAggregate [web_site_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum(CheckOverflow((promote_precision(cast(ws_net_profit as decimal(13,2))) - promote_precision(cast(coalesce(cast(wr_net_loss as decimal(12,2)), 0.00) as decimal(13,2)))), DecimalType(13,2))),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty]
+                      HashAggregate [web_site_id,sum,sum,isEmpty,sum,isEmpty] [sum(UnscaledValue(ws_ext_sales_price)),sum(coalesce(cast(wr_return_amt as decimal(12,2)), 0.00)),sum((ws_net_profit - coalesce(cast(wr_net_loss as decimal(12,2)), 0.00))),sales,returns,profit,channel,id,sum,sum,isEmpty,sum,isEmpty]
                         InputAdapter
                           Exchange [web_site_id] #13
                             WholeStageCodegen (29)
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.sf100/explain.txt
index d37c984980d..f0a2c9da368 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81.sf100/explain.txt
@@ -297,7 +297,7 @@ Input [3]: [ctr_state#31, sum#37, count#38]
 Keys [1]: [ctr_state#31]
 Functions [1]: [avg(ctr_total_return#32)]
 Aggregate Attributes [1]: [avg(ctr_total_return#32)#39]
-Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#32)#39) * 1.200000), DecimalType(24,7)) AS (avg(ctr_total_return) * 1.2)#40, ctr_state#31 AS ctr_state#31#41]
+Results [2]: [(avg(ctr_total_return#32)#39 * 1.2) AS (avg(ctr_total_return) * 1.2)#40, ctr_state#31 AS ctr_state#31#41]
 
 (53) Filter [codegen id : 19]
 Input [2]: [(avg(ctr_total_return) * 1.2)#40, ctr_state#31#41]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/explain.txt
index 7b23beda1cb..f6be4779952 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q81/explain.txt
@@ -198,7 +198,7 @@ Input [3]: [ctr_state#13, sum#19, count#20]
 Keys [1]: [ctr_state#13]
 Functions [1]: [avg(ctr_total_return#14)]
 Aggregate Attributes [1]: [avg(ctr_total_return#14)#21]
-Results [2]: [CheckOverflow((promote_precision(avg(ctr_total_return#14)#21) * 1.200000), DecimalType(24,7)) AS (avg(ctr_total_return) * 1.2)#22, ctr_state#13 AS ctr_state#13#23]
+Results [2]: [(avg(ctr_total_return#14)#21 * 1.2) AS (avg(ctr_total_return) * 1.2)#22, ctr_state#13 AS ctr_state#13#23]
 
 (32) Filter [codegen id : 8]
 Input [2]: [(avg(ctr_total_return) * 1.2)#22, ctr_state#13#23]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/explain.txt
index a0be704ebd2..2162b9d8d66 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83.sf100/explain.txt
@@ -256,7 +256,7 @@ Right keys [1]: [item_id#33]
 Join condition: None
 
 (45) Project [codegen id : 18]
-Output [8]: [item_id#11, sr_item_qty#12, (((cast(sr_item_qty#12 as double) / cast(((sr_item_qty#12 + cr_item_qty#23) + wr_item_qty#34) as double)) / 3.0) * 100.0) AS sr_dev#35, cr_item_qty#23, (((cast(cr_item_qty#23 as double) / cast(((sr_item_qty#12 + cr_item_qty#23) + wr_item_qty#34) as double)) / 3.0) * 100.0) AS cr_dev#36, wr_item_qty#34, (((cast(wr_item_qty#34 as double) / cast(((sr_item_qty#12 + cr_item_qty#23) + wr_item_qty#34) as double)) / 3.0) * 100.0) AS wr_dev#37, CheckOverfl [...]
+Output [8]: [item_id#11, sr_item_qty#12, (((cast(sr_item_qty#12 as double) / cast(((sr_item_qty#12 + cr_item_qty#23) + wr_item_qty#34) as double)) / 3.0) * 100.0) AS sr_dev#35, cr_item_qty#23, (((cast(cr_item_qty#23 as double) / cast(((sr_item_qty#12 + cr_item_qty#23) + wr_item_qty#34) as double)) / 3.0) * 100.0) AS cr_dev#36, wr_item_qty#34, (((cast(wr_item_qty#34 as double) / cast(((sr_item_qty#12 + cr_item_qty#23) + wr_item_qty#34) as double)) / 3.0) * 100.0) AS wr_dev#37, (cast(((sr_ [...]
 Input [5]: [item_id#11, sr_item_qty#12, cr_item_qty#23, item_id#33, wr_item_qty#34]
 
 (46) TakeOrderedAndProject
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt
index bc6c43f1868..6af1fe03b13 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q83/explain.txt
@@ -256,7 +256,7 @@ Right keys [1]: [item_id#33]
 Join condition: None
 
 (45) Project [codegen id : 18]
-Output [8]: [item_id#11, sr_item_qty#12, (((cast(sr_item_qty#12 as double) / cast(((sr_item_qty#12 + cr_item_qty#23) + wr_item_qty#34) as double)) / 3.0) * 100.0) AS sr_dev#35, cr_item_qty#23, (((cast(cr_item_qty#23 as double) / cast(((sr_item_qty#12 + cr_item_qty#23) + wr_item_qty#34) as double)) / 3.0) * 100.0) AS cr_dev#36, wr_item_qty#34, (((cast(wr_item_qty#34 as double) / cast(((sr_item_qty#12 + cr_item_qty#23) + wr_item_qty#34) as double)) / 3.0) * 100.0) AS wr_dev#37, CheckOverfl [...]
+Output [8]: [item_id#11, sr_item_qty#12, (((cast(sr_item_qty#12 as double) / cast(((sr_item_qty#12 + cr_item_qty#23) + wr_item_qty#34) as double)) / 3.0) * 100.0) AS sr_dev#35, cr_item_qty#23, (((cast(cr_item_qty#23 as double) / cast(((sr_item_qty#12 + cr_item_qty#23) + wr_item_qty#34) as double)) / 3.0) * 100.0) AS cr_dev#36, wr_item_qty#34, (((cast(wr_item_qty#34 as double) / cast(((sr_item_qty#12 + cr_item_qty#23) + wr_item_qty#34) as double)) / 3.0) * 100.0) AS wr_dev#37, (cast(((sr_ [...]
 Input [5]: [item_id#11, sr_item_qty#12, cr_item_qty#23, item_id#33, wr_item_qty#34]
 
 (46) TakeOrderedAndProject
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.sf100/explain.txt
index 6dcaa4bcf93..c7910dfee80 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89.sf100/explain.txt
@@ -141,7 +141,7 @@ Arguments: [avg(_w0#19) windowspecdefinition(i_category#4, i_brand#2, s_store_na
 
 (25) Filter [codegen id : 7]
 Input [9]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, _w0#19, avg_monthly_sales#20]
-Condition : (isnotnull(avg_monthly_sales#20) AND (NOT (avg_monthly_sales#20 = 0.000000) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#18 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#20 as decimal(22,6)))), DecimalType(22,6)))) / promote_precision(cast(avg_monthly_sales#20 as decimal(22,6)))), DecimalType(38,16)) > 0.1000000000000000)))
+Condition : ((isnotnull(avg_monthly_sales#20) AND isnotnull(sum_sales#18)) AND (NOT (avg_monthly_sales#20 = 0.000000) AND ((abs((sum_sales#18 - avg_monthly_sales#20)) / avg_monthly_sales#20) > 0.1000000000000000)))
 
 (26) Project [codegen id : 7]
 Output [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, avg_monthly_sales#20]
@@ -149,7 +149,7 @@ Input [9]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#
 
 (27) TakeOrderedAndProject
 Input [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, avg_monthly_sales#20]
-Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#18 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#20 as decimal(22,6)))), DecimalType(22,6)) ASC NULLS FIRST, s_store_name#13 ASC NULLS FIRST], [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, avg_monthly_sales#20]
+Arguments: 100, [(sum_sales#18 - avg_monthly_sales#20) ASC NULLS FIRST, s_store_name#13 ASC NULLS FIRST], [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, avg_monthly_sales#20]
 
 ===== Subqueries =====
 
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/explain.txt
index fc949bd963e..54aed277fdd 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q89/explain.txt
@@ -141,7 +141,7 @@ Arguments: [avg(_w0#19) windowspecdefinition(i_category#4, i_brand#2, s_store_na
 
 (25) Filter [codegen id : 7]
 Input [9]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, _w0#19, avg_monthly_sales#20]
-Condition : (isnotnull(avg_monthly_sales#20) AND (NOT (avg_monthly_sales#20 = 0.000000) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#18 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#20 as decimal(22,6)))), DecimalType(22,6)))) / promote_precision(cast(avg_monthly_sales#20 as decimal(22,6)))), DecimalType(38,16)) > 0.1000000000000000)))
+Condition : ((isnotnull(avg_monthly_sales#20) AND isnotnull(sum_sales#18)) AND (NOT (avg_monthly_sales#20 = 0.000000) AND ((abs((sum_sales#18 - avg_monthly_sales#20)) / avg_monthly_sales#20) > 0.1000000000000000)))
 
 (26) Project [codegen id : 7]
 Output [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, avg_monthly_sales#20]
@@ -149,7 +149,7 @@ Input [9]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#
 
 (27) TakeOrderedAndProject
 Input [8]: [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, avg_monthly_sales#20]
-Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#18 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#20 as decimal(22,6)))), DecimalType(22,6)) ASC NULLS FIRST, s_store_name#13 ASC NULLS FIRST], [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, avg_monthly_sales#20]
+Arguments: 100, [(sum_sales#18 - avg_monthly_sales#20) ASC NULLS FIRST, s_store_name#13 ASC NULLS FIRST], [i_category#4, i_class#3, i_brand#2, s_store_name#13, s_company_name#14, d_moy#11, sum_sales#18, avg_monthly_sales#20]
 
 ===== Subqueries =====
 
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90.sf100/explain.txt
index e9b15a8928c..bca6ec3cc1b 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90.sf100/explain.txt
@@ -280,6 +280,6 @@ Arguments: IdentityBroadcastMode, [plan_id=7]
 Join condition: None
 
 (51) Project [codegen id : 10]
-Output [1]: [CheckOverflow((promote_precision(cast(amc#14 as decimal(15,4))) / promote_precision(cast(pmc#26 as decimal(15,4)))), DecimalType(35,20)) AS am_pm_ratio#27]
+Output [1]: [(cast(amc#14 as decimal(15,4)) / cast(pmc#26 as decimal(15,4))) AS am_pm_ratio#27]
 Input [2]: [amc#14, pmc#26]
 
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/explain.txt
index c333bed23a0..101b803f6c0 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q90/explain.txt
@@ -280,6 +280,6 @@ Arguments: IdentityBroadcastMode, [plan_id=7]
 Join condition: None
 
 (51) Project [codegen id : 10]
-Output [1]: [CheckOverflow((promote_precision(cast(amc#14 as decimal(15,4))) / promote_precision(cast(pmc#26 as decimal(15,4)))), DecimalType(35,20)) AS am_pm_ratio#27]
+Output [1]: [(cast(amc#14 as decimal(15,4)) / cast(pmc#26 as decimal(15,4))) AS am_pm_ratio#27]
 Input [2]: [amc#14, pmc#26]
 
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/explain.txt
index 601919832b3..df8ad3320a8 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92.sf100/explain.txt
@@ -95,7 +95,7 @@ Input [3]: [ws_item_sk#3, sum#12, count#13]
 Keys [1]: [ws_item_sk#3]
 Functions [1]: [avg(UnscaledValue(ws_ext_discount_amt#4))]
 Aggregate Attributes [1]: [avg(UnscaledValue(ws_ext_discount_amt#4))#14]
-Results [2]: [CheckOverflow((1.300000 * promote_precision(cast((avg(UnscaledValue(ws_ext_discount_amt#4))#14 / 100.0) as decimal(11,6)))), DecimalType(14,7)) AS (1.3 * avg(ws_ext_discount_amt))#15, ws_item_sk#3]
+Results [2]: [(1.3 * cast((avg(UnscaledValue(ws_ext_discount_amt#4))#14 / 100.0) as decimal(11,6))) AS (1.3 * avg(ws_ext_discount_amt))#15, ws_item_sk#3]
 
 (15) Filter
 Input [2]: [(1.3 * avg(ws_ext_discount_amt))#15, ws_item_sk#3]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt
index e4c526881db..b9894bf75bc 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q92/explain.txt
@@ -119,7 +119,7 @@ Input [3]: [ws_item_sk#7, sum#13, count#14]
 Keys [1]: [ws_item_sk#7]
 Functions [1]: [avg(UnscaledValue(ws_ext_discount_amt#8))]
 Aggregate Attributes [1]: [avg(UnscaledValue(ws_ext_discount_amt#8))#15]
-Results [2]: [CheckOverflow((1.300000 * promote_precision(cast((avg(UnscaledValue(ws_ext_discount_amt#8))#15 / 100.0) as decimal(11,6)))), DecimalType(14,7)) AS (1.3 * avg(ws_ext_discount_amt))#16, ws_item_sk#7]
+Results [2]: [(1.3 * cast((avg(UnscaledValue(ws_ext_discount_amt#8))#15 / 100.0) as decimal(11,6))) AS (1.3 * avg(ws_ext_discount_amt))#16, ws_item_sk#7]
 
 (20) Filter [codegen id : 4]
 Input [2]: [(1.3 * avg(ws_ext_discount_amt))#16, ws_item_sk#7]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/explain.txt
index e29bcc99eee..a713399b5d3 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93.sf100/explain.txt
@@ -109,7 +109,7 @@ Right keys [2]: [ss_item_sk#8, ss_ticket_number#10]
 Join condition: None
 
 (20) Project [codegen id : 6]
-Output [2]: [ss_customer_sk#9, CASE WHEN isnotnull(sr_return_quantity#4) THEN CheckOverflow((promote_precision(cast((ss_quantity#11 - sr_return_quantity#4) as decimal(12,2))) * promote_precision(cast(ss_sales_price#12 as decimal(12,2)))), DecimalType(18,2)) ELSE CheckOverflow((promote_precision(cast(ss_quantity#11 as decimal(12,2))) * promote_precision(cast(ss_sales_price#12 as decimal(12,2)))), DecimalType(18,2)) END AS act_sales#14]
+Output [2]: [ss_customer_sk#9, CASE WHEN isnotnull(sr_return_quantity#4) THEN (cast((ss_quantity#11 - sr_return_quantity#4) as decimal(10,0)) * ss_sales_price#12) ELSE (cast(ss_quantity#11 as decimal(10,0)) * ss_sales_price#12) END AS act_sales#14]
 Input [8]: [sr_item_sk#1, sr_ticket_number#3, sr_return_quantity#4, ss_item_sk#8, ss_customer_sk#9, ss_ticket_number#10, ss_quantity#11, ss_sales_price#12]
 
 (21) HashAggregate [codegen id : 6]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt
index 032eb9152cf..41b5654e7f3 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q93/explain.txt
@@ -109,7 +109,7 @@ Right keys [1]: [r_reason_sk#12]
 Join condition: None
 
 (20) Project [codegen id : 6]
-Output [2]: [ss_customer_sk#2, CASE WHEN isnotnull(sr_return_quantity#10) THEN CheckOverflow((promote_precision(cast((ss_quantity#4 - sr_return_quantity#10) as decimal(12,2))) * promote_precision(cast(ss_sales_price#5 as decimal(12,2)))), DecimalType(18,2)) ELSE CheckOverflow((promote_precision(cast(ss_quantity#4 as decimal(12,2))) * promote_precision(cast(ss_sales_price#5 as decimal(12,2)))), DecimalType(18,2)) END AS act_sales#14]
+Output [2]: [ss_customer_sk#2, CASE WHEN isnotnull(sr_return_quantity#10) THEN (cast((ss_quantity#4 - sr_return_quantity#10) as decimal(10,0)) * ss_sales_price#5) ELSE (cast(ss_quantity#4 as decimal(10,0)) * ss_sales_price#5) END AS act_sales#14]
 Input [6]: [ss_customer_sk#2, ss_quantity#4, ss_sales_price#5, sr_reason_sk#8, sr_return_quantity#10, r_reason_sk#12]
 
 (21) HashAggregate [codegen id : 6]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.sf100/explain.txt
index 7f2a84fa037..39acbe56d57 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98.sf100/explain.txt
@@ -123,7 +123,7 @@ Input [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemreve
 Arguments: [sum(_w1#17) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#18], [i_class#9]
 
 (22) Project [codegen id : 9]
-Output [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(_w0#16) * 100.00), DecimalType(21,2)) as decimal(27,2))) / promote_precision(_we0#18)), DecimalType(38,17)) AS revenueratio#19, i_item_id#6]
+Output [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, ((_w0#16 * 100) / _we0#18) AS revenueratio#19, i_item_id#6]
 Input [9]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, _w1#17, i_item_id#6, _we0#18]
 
 (23) Exchange
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/explain.txt
index 4d8874720c8..0bc3366b9a7 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v1_4/q98/explain.txt
@@ -108,7 +108,7 @@ Input [8]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemreve
 Arguments: [sum(_w1#17) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#18], [i_class#9]
 
 (19) Project [codegen id : 6]
-Output [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(_w0#16) * 100.00), DecimalType(21,2)) as decimal(27,2))) / promote_precision(_we0#18)), DecimalType(38,17)) AS revenueratio#19, i_item_id#6]
+Output [7]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, ((_w0#16 * 100) / _we0#18) AS revenueratio#19, i_item_id#6]
 Input [9]: [i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, _w1#17, i_item_id#6, _we0#18]
 
 (20) Exchange
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.sf100/explain.txt
index eef33ed4a97..06683d612fd 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.sf100/explain.txt
@@ -149,7 +149,7 @@ Input [12]: [ss_customer_sk#1, ss_ext_discount_amt#2, ss_ext_list_price#3, d_yea
 (16) HashAggregate [codegen id : 6]
 Input [10]: [c_customer_id#9, c_first_name#10, c_last_name#11, c_preferred_cust_flag#12, c_birth_country#13, c_login#14, c_email_address#15, ss_ext_discount_amt#2, ss_ext_list_price#3, d_year#7]
 Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#7, c_preferred_cust_flag#12, c_birth_country#13, c_login#14, c_email_address#15]
-Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#3 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#2 as decimal(8,2)))), DecimalType(8,2))))]
+Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#3 - ss_ext_discount_amt#2)))]
 Aggregate Attributes [1]: [sum#16]
 Results [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#7, c_preferred_cust_flag#12, c_birth_country#13, c_login#14, c_email_address#15, sum#17]
 
@@ -160,9 +160,9 @@ Arguments: hashpartitioning(c_customer_id#9, c_first_name#10, c_last_name#11, d_
 (18) HashAggregate [codegen id : 7]
 Input [9]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#7, c_preferred_cust_flag#12, c_birth_country#13, c_login#14, c_email_address#15, sum#17]
 Keys [8]: [c_customer_id#9, c_first_name#10, c_last_name#11, d_year#7, c_preferred_cust_flag#12, c_birth_country#13, c_login#14, c_email_address#15]
-Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#3 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#2 as decimal(8,2)))), DecimalType(8,2))))]
-Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#3 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#2 as decimal(8,2)))), DecimalType(8,2))))#18]
-Results [2]: [c_customer_id#9 AS customer_id#19, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#3 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#2 as decimal(8,2)))), DecimalType(8,2))))#18,18,2) AS year_total#20]
+Functions [1]: [sum(UnscaledValue((ss_ext_list_price#3 - ss_ext_discount_amt#2)))]
+Aggregate Attributes [1]: [sum(UnscaledValue((ss_ext_list_price#3 - ss_ext_discount_amt#2)))#18]
+Results [2]: [c_customer_id#9 AS customer_id#19, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#3 - ss_ext_discount_amt#2)))#18,18,2) AS year_total#20]
 
 (19) Filter [codegen id : 7]
 Input [2]: [customer_id#19, year_total#20]
@@ -230,7 +230,7 @@ Input [12]: [ss_customer_sk#21, ss_ext_discount_amt#22, ss_ext_list_price#23, d_
 (34) HashAggregate [codegen id : 14]
 Input [10]: [c_customer_id#29, c_first_name#30, c_last_name#31, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35, ss_ext_discount_amt#22, ss_ext_list_price#23, d_year#27]
 Keys [8]: [c_customer_id#29, c_first_name#30, c_last_name#31, d_year#27, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35]
-Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#23 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#22 as decimal(8,2)))), DecimalType(8,2))))]
+Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#23 - ss_ext_discount_amt#22)))]
 Aggregate Attributes [1]: [sum#36]
 Results [9]: [c_customer_id#29, c_first_name#30, c_last_name#31, d_year#27, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35, sum#37]
 
@@ -241,9 +241,9 @@ Arguments: hashpartitioning(c_customer_id#29, c_first_name#30, c_last_name#31, d
 (36) HashAggregate [codegen id : 15]
 Input [9]: [c_customer_id#29, c_first_name#30, c_last_name#31, d_year#27, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35, sum#37]
 Keys [8]: [c_customer_id#29, c_first_name#30, c_last_name#31, d_year#27, c_preferred_cust_flag#32, c_birth_country#33, c_login#34, c_email_address#35]
-Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#23 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#22 as decimal(8,2)))), DecimalType(8,2))))]
-Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#23 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#22 as decimal(8,2)))), DecimalType(8,2))))#18]
-Results [5]: [c_customer_id#29 AS customer_id#38, c_first_name#30 AS customer_first_name#39, c_last_name#31 AS customer_last_name#40, c_email_address#35 AS customer_email_address#41, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#23 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#22 as decimal(8,2)))), DecimalType(8,2))))#18,18,2) AS year_total#42]
+Functions [1]: [sum(UnscaledValue((ss_ext_list_price#23 - ss_ext_discount_amt#22)))]
+Aggregate Attributes [1]: [sum(UnscaledValue((ss_ext_list_price#23 - ss_ext_discount_amt#22)))#18]
+Results [5]: [c_customer_id#29 AS customer_id#38, c_first_name#30 AS customer_first_name#39, c_last_name#31 AS customer_last_name#40, c_email_address#35 AS customer_email_address#41, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#23 - ss_ext_discount_amt#22)))#18,18,2) AS year_total#42]
 
 (37) Exchange
 Input [5]: [customer_id#38, customer_first_name#39, customer_last_name#40, customer_email_address#41, year_total#42]
@@ -312,7 +312,7 @@ Input [12]: [ws_bill_customer_sk#43, ws_ext_discount_amt#44, ws_ext_list_price#4
 (52) HashAggregate [codegen id : 23]
 Input [10]: [c_customer_id#50, c_first_name#51, c_last_name#52, c_preferred_cust_flag#53, c_birth_country#54, c_login#55, c_email_address#56, ws_ext_discount_amt#44, ws_ext_list_price#45, d_year#48]
 Keys [8]: [c_customer_id#50, c_first_name#51, c_last_name#52, c_preferred_cust_flag#53, c_birth_country#54, c_login#55, c_email_address#56, d_year#48]
-Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#45 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#44 as decimal(8,2)))), DecimalType(8,2))))]
+Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#45 - ws_ext_discount_amt#44)))]
 Aggregate Attributes [1]: [sum#57]
 Results [9]: [c_customer_id#50, c_first_name#51, c_last_name#52, c_preferred_cust_flag#53, c_birth_country#54, c_login#55, c_email_address#56, d_year#48, sum#58]
 
@@ -323,9 +323,9 @@ Arguments: hashpartitioning(c_customer_id#50, c_first_name#51, c_last_name#52, c
 (54) HashAggregate [codegen id : 24]
 Input [9]: [c_customer_id#50, c_first_name#51, c_last_name#52, c_preferred_cust_flag#53, c_birth_country#54, c_login#55, c_email_address#56, d_year#48, sum#58]
 Keys [8]: [c_customer_id#50, c_first_name#51, c_last_name#52, c_preferred_cust_flag#53, c_birth_country#54, c_login#55, c_email_address#56, d_year#48]
-Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#45 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#44 as decimal(8,2)))), DecimalType(8,2))))]
-Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#45 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#44 as decimal(8,2)))), DecimalType(8,2))))#59]
-Results [2]: [c_customer_id#50 AS customer_id#60, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#45 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#44 as decimal(8,2)))), DecimalType(8,2))))#59,18,2) AS year_total#61]
+Functions [1]: [sum(UnscaledValue((ws_ext_list_price#45 - ws_ext_discount_amt#44)))]
+Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#45 - ws_ext_discount_amt#44)))#59]
+Results [2]: [c_customer_id#50 AS customer_id#60, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#45 - ws_ext_discount_amt#44)))#59,18,2) AS year_total#61]
 
 (55) Filter [codegen id : 24]
 Input [2]: [customer_id#60, year_total#61]
@@ -402,7 +402,7 @@ Input [12]: [ws_bill_customer_sk#62, ws_ext_discount_amt#63, ws_ext_list_price#6
 (72) HashAggregate [codegen id : 32]
 Input [10]: [c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#73, c_login#74, c_email_address#75, ws_ext_discount_amt#63, ws_ext_list_price#64, d_year#67]
 Keys [8]: [c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#73, c_login#74, c_email_address#75, d_year#67]
-Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#64 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#63 as decimal(8,2)))), DecimalType(8,2))))]
+Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#64 - ws_ext_discount_amt#63)))]
 Aggregate Attributes [1]: [sum#76]
 Results [9]: [c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#73, c_login#74, c_email_address#75, d_year#67, sum#77]
 
@@ -413,9 +413,9 @@ Arguments: hashpartitioning(c_customer_id#69, c_first_name#70, c_last_name#71, c
 (74) HashAggregate [codegen id : 33]
 Input [9]: [c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#73, c_login#74, c_email_address#75, d_year#67, sum#77]
 Keys [8]: [c_customer_id#69, c_first_name#70, c_last_name#71, c_preferred_cust_flag#72, c_birth_country#73, c_login#74, c_email_address#75, d_year#67]
-Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#64 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#63 as decimal(8,2)))), DecimalType(8,2))))]
-Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#64 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#63 as decimal(8,2)))), DecimalType(8,2))))#59]
-Results [2]: [c_customer_id#69 AS customer_id#78, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#64 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#63 as decimal(8,2)))), DecimalType(8,2))))#59,18,2) AS year_total#79]
+Functions [1]: [sum(UnscaledValue((ws_ext_list_price#64 - ws_ext_discount_amt#63)))]
+Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#64 - ws_ext_discount_amt#63)))#59]
+Results [2]: [c_customer_id#69 AS customer_id#78, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#64 - ws_ext_discount_amt#63)))#59,18,2) AS year_total#79]
 
 (75) Exchange
 Input [2]: [customer_id#78, year_total#79]
@@ -428,7 +428,7 @@ Arguments: [customer_id#78 ASC NULLS FIRST], false, 0
 (77) SortMergeJoin [codegen id : 35]
 Left keys [1]: [customer_id#19]
 Right keys [1]: [customer_id#78]
-Join condition: (CASE WHEN (year_total#61 > 0.00) THEN CheckOverflow((promote_precision(year_total#79) / promote_precision(year_total#61)), DecimalType(38,20)) ELSE 0E-20 END > CASE WHEN (year_total#20 > 0.00) THEN CheckOverflow((promote_precision(year_total#42) / promote_precision(year_total#20)), DecimalType(38,20)) ELSE 0E-20 END)
+Join condition: (CASE WHEN (year_total#61 > 0.00) THEN (year_total#79 / year_total#61) ELSE 0E-20 END > CASE WHEN (year_total#20 > 0.00) THEN (year_total#42 / year_total#20) ELSE 0E-20 END)
 
 (78) Project [codegen id : 35]
 Output [4]: [customer_id#38, customer_first_name#39, customer_last_name#40, customer_email_address#41]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.sf100/simplified.txt
index a97e1ed828a..4755426b39b 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.sf100/simplified.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11.sf100/simplified.txt
@@ -16,7 +16,7 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom
                               Exchange [customer_id] #1
                                 WholeStageCodegen (7)
                                   Filter [year_total]
-                                    HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt as decimal(8,2)))), DecimalType(8,2)))),customer_id,year_total,sum]
+                                    HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt))),customer_id,year_total,sum]
                                       InputAdapter
                                         Exchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #2
                                           WholeStageCodegen (6)
@@ -60,7 +60,7 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom
                             InputAdapter
                               Exchange [customer_id] #6
                                 WholeStageCodegen (15)
-                                  HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt as decimal(8,2)))), DecimalType(8,2)))),customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,sum]
+                                  HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt))),customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,sum]
                                     InputAdapter
                                       Exchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #7
                                         WholeStageCodegen (14)
@@ -100,7 +100,7 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom
                         Exchange [customer_id] #10
                           WholeStageCodegen (24)
                             Filter [year_total]
-                              HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt as decimal(8,2)))), DecimalType(8,2)))),customer_id,year_total,sum]
+                              HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum]
                                 InputAdapter
                                   Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #11
                                     WholeStageCodegen (23)
@@ -133,7 +133,7 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom
               InputAdapter
                 Exchange [customer_id] #13
                   WholeStageCodegen (33)
-                    HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt as decimal(8,2)))), DecimalType(8,2)))),customer_id,year_total,sum]
+                    HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum]
                       InputAdapter
                         Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #14
                           WholeStageCodegen (32)
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/explain.txt
index 2884c8e7ba2..11be07ae31b 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/explain.txt
@@ -129,7 +129,7 @@ Input [12]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_fl
 (13) HashAggregate [codegen id : 3]
 Input [10]: [c_customer_id#2, c_first_name#3, c_last_name#4, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, ss_ext_discount_amt#10, ss_ext_list_price#11, d_year#15]
 Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8]
-Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#11 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#10 as decimal(8,2)))), DecimalType(8,2))))]
+Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#11 - ss_ext_discount_amt#10)))]
 Aggregate Attributes [1]: [sum#16]
 Results [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#17]
 
@@ -140,9 +140,9 @@ Arguments: hashpartitioning(c_customer_id#2, c_first_name#3, c_last_name#4, d_ye
 (15) HashAggregate [codegen id : 16]
 Input [9]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8, sum#17]
 Keys [8]: [c_customer_id#2, c_first_name#3, c_last_name#4, d_year#15, c_preferred_cust_flag#5, c_birth_country#6, c_login#7, c_email_address#8]
-Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#11 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#10 as decimal(8,2)))), DecimalType(8,2))))]
-Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#11 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#10 as decimal(8,2)))), DecimalType(8,2))))#18]
-Results [2]: [c_customer_id#2 AS customer_id#19, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#11 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#10 as decimal(8,2)))), DecimalType(8,2))))#18,18,2) AS year_total#20]
+Functions [1]: [sum(UnscaledValue((ss_ext_list_price#11 - ss_ext_discount_amt#10)))]
+Aggregate Attributes [1]: [sum(UnscaledValue((ss_ext_list_price#11 - ss_ext_discount_amt#10)))#18]
+Results [2]: [c_customer_id#2 AS customer_id#19, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#11 - ss_ext_discount_amt#10)))#18,18,2) AS year_total#20]
 
 (16) Filter [codegen id : 16]
 Input [2]: [customer_id#19, year_total#20]
@@ -205,7 +205,7 @@ Input [12]: [c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust
 (29) HashAggregate [codegen id : 6]
 Input [10]: [c_customer_id#22, c_first_name#23, c_last_name#24, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, ss_ext_discount_amt#30, ss_ext_list_price#31, d_year#35]
 Keys [8]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28]
-Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#31 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#30 as decimal(8,2)))), DecimalType(8,2))))]
+Functions [1]: [partial_sum(UnscaledValue((ss_ext_list_price#31 - ss_ext_discount_amt#30)))]
 Aggregate Attributes [1]: [sum#36]
 Results [9]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, sum#37]
 
@@ -216,9 +216,9 @@ Arguments: hashpartitioning(c_customer_id#22, c_first_name#23, c_last_name#24, d
 (31) HashAggregate [codegen id : 7]
 Input [9]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28, sum#37]
 Keys [8]: [c_customer_id#22, c_first_name#23, c_last_name#24, d_year#35, c_preferred_cust_flag#25, c_birth_country#26, c_login#27, c_email_address#28]
-Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#31 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#30 as decimal(8,2)))), DecimalType(8,2))))]
-Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#31 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#30 as decimal(8,2)))), DecimalType(8,2))))#18]
-Results [5]: [c_customer_id#22 AS customer_id#38, c_first_name#23 AS customer_first_name#39, c_last_name#24 AS customer_last_name#40, c_email_address#28 AS customer_email_address#41, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price#31 as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt#30 as decimal(8,2)))), DecimalType(8,2))))#18,18,2) AS year_total#42]
+Functions [1]: [sum(UnscaledValue((ss_ext_list_price#31 - ss_ext_discount_amt#30)))]
+Aggregate Attributes [1]: [sum(UnscaledValue((ss_ext_list_price#31 - ss_ext_discount_amt#30)))#18]
+Results [5]: [c_customer_id#22 AS customer_id#38, c_first_name#23 AS customer_first_name#39, c_last_name#24 AS customer_last_name#40, c_email_address#28 AS customer_email_address#41, MakeDecimal(sum(UnscaledValue((ss_ext_list_price#31 - ss_ext_discount_amt#30)))#18,18,2) AS year_total#42]
 
 (32) BroadcastExchange
 Input [5]: [customer_id#38, customer_first_name#39, customer_last_name#40, customer_email_address#41, year_total#42]
@@ -286,7 +286,7 @@ Input [12]: [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust
 (46) HashAggregate [codegen id : 10]
 Input [10]: [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, ws_ext_discount_amt#52, ws_ext_list_price#53, d_year#56]
 Keys [8]: [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, d_year#56]
-Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#53 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#52 as decimal(8,2)))), DecimalType(8,2))))]
+Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#53 - ws_ext_discount_amt#52)))]
 Aggregate Attributes [1]: [sum#57]
 Results [9]: [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, d_year#56, sum#58]
 
@@ -297,9 +297,9 @@ Arguments: hashpartitioning(c_customer_id#44, c_first_name#45, c_last_name#46, c
 (48) HashAggregate [codegen id : 11]
 Input [9]: [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, d_year#56, sum#58]
 Keys [8]: [c_customer_id#44, c_first_name#45, c_last_name#46, c_preferred_cust_flag#47, c_birth_country#48, c_login#49, c_email_address#50, d_year#56]
-Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#53 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#52 as decimal(8,2)))), DecimalType(8,2))))]
-Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#53 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#52 as decimal(8,2)))), DecimalType(8,2))))#59]
-Results [2]: [c_customer_id#44 AS customer_id#60, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#53 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#52 as decimal(8,2)))), DecimalType(8,2))))#59,18,2) AS year_total#61]
+Functions [1]: [sum(UnscaledValue((ws_ext_list_price#53 - ws_ext_discount_amt#52)))]
+Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#53 - ws_ext_discount_amt#52)))#59]
+Results [2]: [c_customer_id#44 AS customer_id#60, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#53 - ws_ext_discount_amt#52)))#59,18,2) AS year_total#61]
 
 (49) Filter [codegen id : 11]
 Input [2]: [customer_id#60, year_total#61]
@@ -375,7 +375,7 @@ Input [12]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust
 (65) HashAggregate [codegen id : 14]
 Input [10]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, ws_ext_discount_amt#71, ws_ext_list_price#72, d_year#75]
 Keys [8]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, d_year#75]
-Functions [1]: [partial_sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#72 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#71 as decimal(8,2)))), DecimalType(8,2))))]
+Functions [1]: [partial_sum(UnscaledValue((ws_ext_list_price#72 - ws_ext_discount_amt#71)))]
 Aggregate Attributes [1]: [sum#76]
 Results [9]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, d_year#75, sum#77]
 
@@ -386,9 +386,9 @@ Arguments: hashpartitioning(c_customer_id#63, c_first_name#64, c_last_name#65, c
 (67) HashAggregate [codegen id : 15]
 Input [9]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, d_year#75, sum#77]
 Keys [8]: [c_customer_id#63, c_first_name#64, c_last_name#65, c_preferred_cust_flag#66, c_birth_country#67, c_login#68, c_email_address#69, d_year#75]
-Functions [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#72 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#71 as decimal(8,2)))), DecimalType(8,2))))]
-Aggregate Attributes [1]: [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#72 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#71 as decimal(8,2)))), DecimalType(8,2))))#59]
-Results [2]: [c_customer_id#63 AS customer_id#78, MakeDecimal(sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price#72 as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt#71 as decimal(8,2)))), DecimalType(8,2))))#59,18,2) AS year_total#79]
+Functions [1]: [sum(UnscaledValue((ws_ext_list_price#72 - ws_ext_discount_amt#71)))]
+Aggregate Attributes [1]: [sum(UnscaledValue((ws_ext_list_price#72 - ws_ext_discount_amt#71)))#59]
+Results [2]: [c_customer_id#63 AS customer_id#78, MakeDecimal(sum(UnscaledValue((ws_ext_list_price#72 - ws_ext_discount_amt#71)))#59,18,2) AS year_total#79]
 
 (68) BroadcastExchange
 Input [2]: [customer_id#78, year_total#79]
@@ -397,7 +397,7 @@ Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [pla
 (69) BroadcastHashJoin [codegen id : 16]
 Left keys [1]: [customer_id#19]
 Right keys [1]: [customer_id#78]
-Join condition: (CASE WHEN (year_total#61 > 0.00) THEN CheckOverflow((promote_precision(year_total#79) / promote_precision(year_total#61)), DecimalType(38,20)) ELSE 0E-20 END > CASE WHEN (year_total#20 > 0.00) THEN CheckOverflow((promote_precision(year_total#42) / promote_precision(year_total#20)), DecimalType(38,20)) ELSE 0E-20 END)
+Join condition: (CASE WHEN (year_total#61 > 0.00) THEN (year_total#79 / year_total#61) ELSE 0E-20 END > CASE WHEN (year_total#20 > 0.00) THEN (year_total#42 / year_total#20) ELSE 0E-20 END)
 
 (70) Project [codegen id : 16]
 Output [4]: [customer_id#38, customer_first_name#39, customer_last_name#40, customer_email_address#41]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/simplified.txt
index 91974a295b7..f1cd6022569 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/simplified.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q11/simplified.txt
@@ -6,7 +6,7 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom
           BroadcastHashJoin [customer_id,customer_id]
             BroadcastHashJoin [customer_id,customer_id]
               Filter [year_total]
-                HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt as decimal(8,2)))), DecimalType(8,2)))),customer_id,year_total,sum]
+                HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt))),customer_id,year_total,sum]
                   InputAdapter
                     Exchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #1
                       WholeStageCodegen (3)
@@ -38,7 +38,7 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom
               InputAdapter
                 BroadcastExchange #4
                   WholeStageCodegen (7)
-                    HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ss_ext_list_price as decimal(8,2))) - promote_precision(cast(ss_ext_discount_amt as decimal(8,2)))), DecimalType(8,2)))),customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,sum]
+                    HashAggregate [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,sum] [sum(UnscaledValue((ss_ext_list_price - ss_ext_discount_amt))),customer_id,customer_first_name,customer_last_name,customer_email_address,year_total,sum]
                       InputAdapter
                         Exchange [c_customer_id,c_first_name,c_last_name,d_year,c_preferred_cust_flag,c_birth_country,c_login,c_email_address] #5
                           WholeStageCodegen (6)
@@ -71,7 +71,7 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom
               BroadcastExchange #8
                 WholeStageCodegen (11)
                   Filter [year_total]
-                    HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt as decimal(8,2)))), DecimalType(8,2)))),customer_id,year_total,sum]
+                    HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum]
                       InputAdapter
                         Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #9
                           WholeStageCodegen (10)
@@ -97,7 +97,7 @@ TakeOrderedAndProject [customer_id,customer_first_name,customer_last_name,custom
         InputAdapter
           BroadcastExchange #11
             WholeStageCodegen (15)
-              HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue(CheckOverflow((promote_precision(cast(ws_ext_list_price as decimal(8,2))) - promote_precision(cast(ws_ext_discount_amt as decimal(8,2)))), DecimalType(8,2)))),customer_id,year_total,sum]
+              HashAggregate [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year,sum] [sum(UnscaledValue((ws_ext_list_price - ws_ext_discount_amt))),customer_id,year_total,sum]
                 InputAdapter
                   Exchange [c_customer_id,c_first_name,c_last_name,c_preferred_cust_flag,c_birth_country,c_login,c_email_address,d_year] #12
                     WholeStageCodegen (14)
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.sf100/explain.txt
index 3d13a020acb..6680ce55d06 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12.sf100/explain.txt
@@ -121,7 +121,7 @@ Input [8]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_pric
 Arguments: [sum(_w1#17) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#18], [i_class#9]
 
 (22) Project [codegen id : 9]
-Output [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(_w0#16) * 100.00), DecimalType(21,2)) as decimal(27,2))) / promote_precision(_we0#18)), DecimalType(38,17)) AS revenueratio#19]
+Output [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, ((_w0#16 * 100) / _we0#18) AS revenueratio#19]
 Input [9]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, _w1#17, _we0#18]
 
 (23) TakeOrderedAndProject
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/explain.txt
index 8c652f4782c..79ab439893c 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q12/explain.txt
@@ -106,7 +106,7 @@ Input [8]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_pric
 Arguments: [sum(_w1#17) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#18], [i_class#9]
 
 (19) Project [codegen id : 6]
-Output [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(_w0#16) * 100.00), DecimalType(21,2)) as decimal(27,2))) / promote_precision(_we0#18)), DecimalType(38,17)) AS revenueratio#19]
+Output [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, ((_w0#16 * 100) / _we0#18) AS revenueratio#19]
 Input [9]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, _w1#17, _we0#18]
 
 (20) TakeOrderedAndProject
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt
index 809219a4ee8..cfdc36c7c2b 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/explain.txt
@@ -432,7 +432,7 @@ Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#37, i_brand_
 (75) HashAggregate [codegen id : 43]
 Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#38, i_class_id#39, i_category_id#40]
 Keys [3]: [i_brand_id#38, i_class_id#39, i_category_id#40]
-Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(ss_quantity#2 as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2))), partial_count(1)]
+Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)]
 Aggregate Attributes [3]: [sum#41, isEmpty#42, count#43]
 Results [6]: [i_brand_id#38, i_class_id#39, i_category_id#40, sum#44, isEmpty#45, count#46]
 
@@ -443,9 +443,9 @@ Arguments: hashpartitioning(i_brand_id#38, i_class_id#39, i_category_id#40, 5),
 (77) HashAggregate [codegen id : 88]
 Input [6]: [i_brand_id#38, i_class_id#39, i_category_id#40, sum#44, isEmpty#45, count#46]
 Keys [3]: [i_brand_id#38, i_class_id#39, i_category_id#40]
-Functions [2]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#2 as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2))), count(1)]
-Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#2 as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2)))#47, count(1)#48]
-Results [6]: [store AS channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum(CheckOverflow((promote_precision(cast(ss_quantity#2 as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2)))#47 AS sales#50, count(1)#48 AS number_sales#51]
+Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)]
+Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47, count(1)#48]
+Results [6]: [store AS channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47 AS sales#50, count(1)#48 AS number_sales#51]
 
 (78) Filter [codegen id : 88]
 Input [6]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sales#50, number_sales#51]
@@ -513,7 +513,7 @@ Input [7]: [ss_item_sk#54, ss_quantity#55, ss_list_price#56, i_item_sk#60, i_bra
 (93) HashAggregate [codegen id : 86]
 Input [5]: [ss_quantity#55, ss_list_price#56, i_brand_id#61, i_class_id#62, i_category_id#63]
 Keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63]
-Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(ss_quantity#55 as decimal(12,2))) * promote_precision(cast(ss_list_price#56 as decimal(12,2)))), DecimalType(18,2))), partial_count(1)]
+Functions [2]: [partial_sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56)), partial_count(1)]
 Aggregate Attributes [3]: [sum#64, isEmpty#65, count#66]
 Results [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#67, isEmpty#68, count#69]
 
@@ -524,9 +524,9 @@ Arguments: hashpartitioning(i_brand_id#61, i_class_id#62, i_category_id#63, 5),
 (95) HashAggregate [codegen id : 87]
 Input [6]: [i_brand_id#61, i_class_id#62, i_category_id#63, sum#67, isEmpty#68, count#69]
 Keys [3]: [i_brand_id#61, i_class_id#62, i_category_id#63]
-Functions [2]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#55 as decimal(12,2))) * promote_precision(cast(ss_list_price#56 as decimal(12,2)))), DecimalType(18,2))), count(1)]
-Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#55 as decimal(12,2))) * promote_precision(cast(ss_list_price#56 as decimal(12,2)))), DecimalType(18,2)))#70, count(1)#71]
-Results [6]: [store AS channel#72, i_brand_id#61, i_class_id#62, i_category_id#63, sum(CheckOverflow((promote_precision(cast(ss_quantity#55 as decimal(12,2))) * promote_precision(cast(ss_list_price#56 as decimal(12,2)))), DecimalType(18,2)))#70 AS sales#73, count(1)#71 AS number_sales#74]
+Functions [2]: [sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56)), count(1)]
+Aggregate Attributes [2]: [sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56))#70, count(1)#71]
+Results [6]: [store AS channel#72, i_brand_id#61, i_class_id#62, i_category_id#63, sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56))#70 AS sales#73, count(1)#71 AS number_sales#74]
 
 (96) Filter [codegen id : 87]
 Input [6]: [channel#72, i_brand_id#61, i_class_id#62, i_category_id#63, sales#73, number_sales#74]
@@ -640,7 +640,7 @@ Input [4]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#89, d_date_sk#90]
 (116) HashAggregate [codegen id : 7]
 Input [2]: [quantity#79, list_price#80]
 Keys: []
-Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(quantity#79 as decimal(12,2))) * promote_precision(cast(list_price#80 as decimal(12,2)))), DecimalType(18,2)))]
+Functions [1]: [partial_avg((cast(quantity#79 as decimal(10,0)) * list_price#80))]
 Aggregate Attributes [2]: [sum#93, count#94]
 Results [2]: [sum#95, count#96]
 
@@ -651,9 +651,9 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=17]
 (118) HashAggregate [codegen id : 8]
 Input [2]: [sum#95, count#96]
 Keys: []
-Functions [1]: [avg(CheckOverflow((promote_precision(cast(quantity#79 as decimal(12,2))) * promote_precision(cast(list_price#80 as decimal(12,2)))), DecimalType(18,2)))]
-Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(quantity#79 as decimal(12,2))) * promote_precision(cast(list_price#80 as decimal(12,2)))), DecimalType(18,2)))#97]
-Results [1]: [avg(CheckOverflow((promote_precision(cast(quantity#79 as decimal(12,2))) * promote_precision(cast(list_price#80 as decimal(12,2)))), DecimalType(18,2)))#97 AS average_sales#98]
+Functions [1]: [avg((cast(quantity#79 as decimal(10,0)) * list_price#80))]
+Aggregate Attributes [1]: [avg((cast(quantity#79 as decimal(10,0)) * list_price#80))#97]
+Results [1]: [avg((cast(quantity#79 as decimal(10,0)) * list_price#80))#97 AS average_sales#98]
 
 Subquery:2 Hosting operator id = 100 Hosting Expression = ss_sold_date_sk#77 IN dynamicpruning#12
 
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/simplified.txt
index 82e338515f4..c6d31d04f75 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/simplified.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14.sf100/simplified.txt
@@ -4,7 +4,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
       Filter [sales]
         Subquery #4
           WholeStageCodegen (8)
-            HashAggregate [sum,count] [avg(CheckOverflow((promote_precision(cast(quantity as decimal(12,2))) * promote_precision(cast(list_price as decimal(12,2)))), DecimalType(18,2))),average_sales,sum,count]
+            HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count]
               InputAdapter
                 Exchange #16
                   WholeStageCodegen (7)
@@ -38,7 +38,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
                                       ReusedSubquery [d_date_sk] #3
                                 InputAdapter
                                   ReusedExchange [d_date_sk] #8
-        HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(ss_quantity as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2))),count(1),channel,sales,number_sales,sum,isEmpty,count]
+        HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count]
           InputAdapter
             Exchange [i_brand_id,i_class_id,i_category_id] #1
               WholeStageCodegen (43)
@@ -201,7 +201,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
           WholeStageCodegen (87)
             Filter [sales]
               ReusedSubquery [average_sales] #4
-              HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(ss_quantity as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2))),count(1),channel,sales,number_sales,sum,isEmpty,count]
+              HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count]
                 InputAdapter
                   Exchange [i_brand_id,i_class_id,i_category_id] #18
                     WholeStageCodegen (86)
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt
index 28a695ae202..494ff5cc6c7 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/explain.txt
@@ -369,7 +369,7 @@ Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#37, i_
 (63) HashAggregate [codegen id : 25]
 Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#37, i_class_id#38, i_category_id#39]
 Keys [3]: [i_brand_id#37, i_class_id#38, i_category_id#39]
-Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(ss_quantity#2 as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2))), partial_count(1)]
+Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)]
 Aggregate Attributes [3]: [sum#41, isEmpty#42, count#43]
 Results [6]: [i_brand_id#37, i_class_id#38, i_category_id#39, sum#44, isEmpty#45, count#46]
 
@@ -380,9 +380,9 @@ Arguments: hashpartitioning(i_brand_id#37, i_class_id#38, i_category_id#39, 5),
 (65) HashAggregate [codegen id : 52]
 Input [6]: [i_brand_id#37, i_class_id#38, i_category_id#39, sum#44, isEmpty#45, count#46]
 Keys [3]: [i_brand_id#37, i_class_id#38, i_category_id#39]
-Functions [2]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#2 as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2))), count(1)]
-Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#2 as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2)))#47, count(1)#48]
-Results [6]: [store AS channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum(CheckOverflow((promote_precision(cast(ss_quantity#2 as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2)))#47 AS sales#50, count(1)#48 AS number_sales#51]
+Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)]
+Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47, count(1)#48]
+Results [6]: [store AS channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47 AS sales#50, count(1)#48 AS number_sales#51]
 
 (66) Filter [codegen id : 52]
 Input [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sales#50, number_sales#51]
@@ -438,7 +438,7 @@ Input [7]: [ss_quantity#55, ss_list_price#56, ss_sold_date_sk#57, i_brand_id#60,
 (78) HashAggregate [codegen id : 50]
 Input [5]: [ss_quantity#55, ss_list_price#56, i_brand_id#60, i_class_id#61, i_category_id#62]
 Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62]
-Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(ss_quantity#55 as decimal(12,2))) * promote_precision(cast(ss_list_price#56 as decimal(12,2)))), DecimalType(18,2))), partial_count(1)]
+Functions [2]: [partial_sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56)), partial_count(1)]
 Aggregate Attributes [3]: [sum#64, isEmpty#65, count#66]
 Results [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#67, isEmpty#68, count#69]
 
@@ -449,9 +449,9 @@ Arguments: hashpartitioning(i_brand_id#60, i_class_id#61, i_category_id#62, 5),
 (80) HashAggregate [codegen id : 51]
 Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#67, isEmpty#68, count#69]
 Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62]
-Functions [2]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#55 as decimal(12,2))) * promote_precision(cast(ss_list_price#56 as decimal(12,2)))), DecimalType(18,2))), count(1)]
-Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#55 as decimal(12,2))) * promote_precision(cast(ss_list_price#56 as decimal(12,2)))), DecimalType(18,2)))#70, count(1)#71]
-Results [6]: [store AS channel#72, i_brand_id#60, i_class_id#61, i_category_id#62, sum(CheckOverflow((promote_precision(cast(ss_quantity#55 as decimal(12,2))) * promote_precision(cast(ss_list_price#56 as decimal(12,2)))), DecimalType(18,2)))#70 AS sales#73, count(1)#71 AS number_sales#74]
+Functions [2]: [sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56)), count(1)]
+Aggregate Attributes [2]: [sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56))#70, count(1)#71]
+Results [6]: [store AS channel#72, i_brand_id#60, i_class_id#61, i_category_id#62, sum((cast(ss_quantity#55 as decimal(10,0)) * ss_list_price#56))#70 AS sales#73, count(1)#71 AS number_sales#74]
 
 (81) Filter [codegen id : 51]
 Input [6]: [channel#72, i_brand_id#60, i_class_id#61, i_category_id#62, sales#73, number_sales#74]
@@ -565,7 +565,7 @@ Input [4]: [ws_quantity#87, ws_list_price#88, ws_sold_date_sk#89, d_date_sk#90]
 (101) HashAggregate [codegen id : 7]
 Input [2]: [quantity#79, list_price#80]
 Keys: []
-Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(quantity#79 as decimal(12,2))) * promote_precision(cast(list_price#80 as decimal(12,2)))), DecimalType(18,2)))]
+Functions [1]: [partial_avg((cast(quantity#79 as decimal(10,0)) * list_price#80))]
 Aggregate Attributes [2]: [sum#93, count#94]
 Results [2]: [sum#95, count#96]
 
@@ -576,9 +576,9 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=12]
 (103) HashAggregate [codegen id : 8]
 Input [2]: [sum#95, count#96]
 Keys: []
-Functions [1]: [avg(CheckOverflow((promote_precision(cast(quantity#79 as decimal(12,2))) * promote_precision(cast(list_price#80 as decimal(12,2)))), DecimalType(18,2)))]
-Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(quantity#79 as decimal(12,2))) * promote_precision(cast(list_price#80 as decimal(12,2)))), DecimalType(18,2)))#97]
-Results [1]: [avg(CheckOverflow((promote_precision(cast(quantity#79 as decimal(12,2))) * promote_precision(cast(list_price#80 as decimal(12,2)))), DecimalType(18,2)))#97 AS average_sales#98]
+Functions [1]: [avg((cast(quantity#79 as decimal(10,0)) * list_price#80))]
+Aggregate Attributes [1]: [avg((cast(quantity#79 as decimal(10,0)) * list_price#80))#97]
+Results [1]: [avg((cast(quantity#79 as decimal(10,0)) * list_price#80))#97 AS average_sales#98]
 
 Subquery:2 Hosting operator id = 85 Hosting Expression = ss_sold_date_sk#77 IN dynamicpruning#12
 
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt
index 259178d0e43..13104f1ba31 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14/simplified.txt
@@ -4,7 +4,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
       Filter [sales]
         Subquery #4
           WholeStageCodegen (8)
-            HashAggregate [sum,count] [avg(CheckOverflow((promote_precision(cast(quantity as decimal(12,2))) * promote_precision(cast(list_price as decimal(12,2)))), DecimalType(18,2))),average_sales,sum,count]
+            HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count]
               InputAdapter
                 Exchange #12
                   WholeStageCodegen (7)
@@ -38,7 +38,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
                                       ReusedSubquery [d_date_sk] #3
                                 InputAdapter
                                   ReusedExchange [d_date_sk] #6
-        HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(ss_quantity as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2))),count(1),channel,sales,number_sales,sum,isEmpty,count]
+        HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count]
           InputAdapter
             Exchange [i_brand_id,i_class_id,i_category_id] #1
               WholeStageCodegen (25)
@@ -165,7 +165,7 @@ TakeOrderedAndProject [i_brand_id,i_class_id,i_category_id,channel,sales,number_
           WholeStageCodegen (51)
             Filter [sales]
               ReusedSubquery [average_sales] #4
-              HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(ss_quantity as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2))),count(1),channel,sales,number_sales,sum,isEmpty,count]
+              HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count]
                 InputAdapter
                   Exchange [i_brand_id,i_class_id,i_category_id] #14
                     WholeStageCodegen (50)
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/explain.txt
index 6777e024d93..cb89542ec80 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/explain.txt
@@ -476,7 +476,7 @@ Input [7]: [ss_item_sk#1, ss_quantity#2, ss_list_price#3, i_item_sk#37, i_brand_
 (75) HashAggregate [codegen id : 43]
 Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#38, i_class_id#39, i_category_id#40]
 Keys [3]: [i_brand_id#38, i_class_id#39, i_category_id#40]
-Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(ss_quantity#2 as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2))), partial_count(1)]
+Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)]
 Aggregate Attributes [3]: [sum#41, isEmpty#42, count#43]
 Results [6]: [i_brand_id#38, i_class_id#39, i_category_id#40, sum#44, isEmpty#45, count#46]
 
@@ -487,9 +487,9 @@ Arguments: hashpartitioning(i_brand_id#38, i_class_id#39, i_category_id#40, 5),
 (77) HashAggregate [codegen id : 44]
 Input [6]: [i_brand_id#38, i_class_id#39, i_category_id#40, sum#44, isEmpty#45, count#46]
 Keys [3]: [i_brand_id#38, i_class_id#39, i_category_id#40]
-Functions [2]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#2 as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2))), count(1)]
-Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#2 as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2)))#47, count(1)#48]
-Results [6]: [store AS channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum(CheckOverflow((promote_precision(cast(ss_quantity#2 as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2)))#47 AS sales#50, count(1)#48 AS number_sales#51]
+Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)]
+Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47, count(1)#48]
+Results [6]: [store AS channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47 AS sales#50, count(1)#48 AS number_sales#51]
 
 (78) Filter [codegen id : 44]
 Input [6]: [channel#49, i_brand_id#38, i_class_id#39, i_category_id#40, sales#50, number_sales#51]
@@ -557,7 +557,7 @@ Input [7]: [cs_item_sk#54, cs_quantity#55, cs_list_price#56, i_item_sk#59, i_bra
 (93) HashAggregate [codegen id : 87]
 Input [5]: [cs_quantity#55, cs_list_price#56, i_brand_id#60, i_class_id#61, i_category_id#62]
 Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62]
-Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cs_quantity#55 as decimal(12,2))) * promote_precision(cast(cs_list_price#56 as decimal(12,2)))), DecimalType(18,2))), partial_count(1)]
+Functions [2]: [partial_sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56)), partial_count(1)]
 Aggregate Attributes [3]: [sum#63, isEmpty#64, count#65]
 Results [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#66, isEmpty#67, count#68]
 
@@ -568,9 +568,9 @@ Arguments: hashpartitioning(i_brand_id#60, i_class_id#61, i_category_id#62, 5),
 (95) HashAggregate [codegen id : 88]
 Input [6]: [i_brand_id#60, i_class_id#61, i_category_id#62, sum#66, isEmpty#67, count#68]
 Keys [3]: [i_brand_id#60, i_class_id#61, i_category_id#62]
-Functions [2]: [sum(CheckOverflow((promote_precision(cast(cs_quantity#55 as decimal(12,2))) * promote_precision(cast(cs_list_price#56 as decimal(12,2)))), DecimalType(18,2))), count(1)]
-Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cs_quantity#55 as decimal(12,2))) * promote_precision(cast(cs_list_price#56 as decimal(12,2)))), DecimalType(18,2)))#69, count(1)#70]
-Results [6]: [catalog AS channel#71, i_brand_id#60, i_class_id#61, i_category_id#62, sum(CheckOverflow((promote_precision(cast(cs_quantity#55 as decimal(12,2))) * promote_precision(cast(cs_list_price#56 as decimal(12,2)))), DecimalType(18,2)))#69 AS sales#72, count(1)#70 AS number_sales#73]
+Functions [2]: [sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56)), count(1)]
+Aggregate Attributes [2]: [sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56))#69, count(1)#70]
+Results [6]: [catalog AS channel#71, i_brand_id#60, i_class_id#61, i_category_id#62, sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56))#69 AS sales#72, count(1)#70 AS number_sales#73]
 
 (96) Filter [codegen id : 88]
 Input [6]: [channel#71, i_brand_id#60, i_class_id#61, i_category_id#62, sales#72, number_sales#73]
@@ -638,7 +638,7 @@ Input [7]: [ws_item_sk#74, ws_quantity#75, ws_list_price#76, i_item_sk#79, i_bra
 (111) HashAggregate [codegen id : 131]
 Input [5]: [ws_quantity#75, ws_list_price#76, i_brand_id#80, i_class_id#81, i_category_id#82]
 Keys [3]: [i_brand_id#80, i_class_id#81, i_category_id#82]
-Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(ws_quantity#75 as decimal(12,2))) * promote_precision(cast(ws_list_price#76 as decimal(12,2)))), DecimalType(18,2))), partial_count(1)]
+Functions [2]: [partial_sum((cast(ws_quantity#75 as decimal(10,0)) * ws_list_price#76)), partial_count(1)]
 Aggregate Attributes [3]: [sum#83, isEmpty#84, count#85]
 Results [6]: [i_brand_id#80, i_class_id#81, i_category_id#82, sum#86, isEmpty#87, count#88]
 
@@ -649,9 +649,9 @@ Arguments: hashpartitioning(i_brand_id#80, i_class_id#81, i_category_id#82, 5),
 (113) HashAggregate [codegen id : 132]
 Input [6]: [i_brand_id#80, i_class_id#81, i_category_id#82, sum#86, isEmpty#87, count#88]
 Keys [3]: [i_brand_id#80, i_class_id#81, i_category_id#82]
-Functions [2]: [sum(CheckOverflow((promote_precision(cast(ws_quantity#75 as decimal(12,2))) * promote_precision(cast(ws_list_price#76 as decimal(12,2)))), DecimalType(18,2))), count(1)]
-Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(ws_quantity#75 as decimal(12,2))) * promote_precision(cast(ws_list_price#76 as decimal(12,2)))), DecimalType(18,2)))#89, count(1)#90]
-Results [6]: [web AS channel#91, i_brand_id#80, i_class_id#81, i_category_id#82, sum(CheckOverflow((promote_precision(cast(ws_quantity#75 as decimal(12,2))) * promote_precision(cast(ws_list_price#76 as decimal(12,2)))), DecimalType(18,2)))#89 AS sales#92, count(1)#90 AS number_sales#93]
+Functions [2]: [sum((cast(ws_quantity#75 as decimal(10,0)) * ws_list_price#76)), count(1)]
+Aggregate Attributes [2]: [sum((cast(ws_quantity#75 as decimal(10,0)) * ws_list_price#76))#89, count(1)#90]
+Results [6]: [web AS channel#91, i_brand_id#80, i_class_id#81, i_category_id#82, sum((cast(ws_quantity#75 as decimal(10,0)) * ws_list_price#76))#89 AS sales#92, count(1)#90 AS number_sales#93]
 
 (114) Filter [codegen id : 132]
 Input [6]: [channel#91, i_brand_id#80, i_class_id#81, i_category_id#82, sales#92, number_sales#93]
@@ -908,7 +908,7 @@ Input [4]: [ws_quantity#167, ws_list_price#168, ws_sold_date_sk#169, d_date_sk#1
 (160) HashAggregate [codegen id : 7]
 Input [2]: [quantity#158, list_price#159]
 Keys: []
-Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(quantity#158 as decimal(12,2))) * promote_precision(cast(list_price#159 as decimal(12,2)))), DecimalType(18,2)))]
+Functions [1]: [partial_avg((cast(quantity#158 as decimal(10,0)) * list_price#159))]
 Aggregate Attributes [2]: [sum#173, count#174]
 Results [2]: [sum#175, count#176]
 
@@ -919,9 +919,9 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=24]
 (162) HashAggregate [codegen id : 8]
 Input [2]: [sum#175, count#176]
 Keys: []
-Functions [1]: [avg(CheckOverflow((promote_precision(cast(quantity#158 as decimal(12,2))) * promote_precision(cast(list_price#159 as decimal(12,2)))), DecimalType(18,2)))]
-Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(quantity#158 as decimal(12,2))) * promote_precision(cast(list_price#159 as decimal(12,2)))), DecimalType(18,2)))#177]
-Results [1]: [avg(CheckOverflow((promote_precision(cast(quantity#158 as decimal(12,2))) * promote_precision(cast(list_price#159 as decimal(12,2)))), DecimalType(18,2)))#177 AS average_sales#178]
+Functions [1]: [avg((cast(quantity#158 as decimal(10,0)) * list_price#159))]
+Aggregate Attributes [1]: [avg((cast(quantity#158 as decimal(10,0)) * list_price#159))#177]
+Results [1]: [avg((cast(quantity#158 as decimal(10,0)) * list_price#159))#177 AS average_sales#178]
 
 Subquery:2 Hosting operator id = 144 Hosting Expression = ss_sold_date_sk#156 IN dynamicpruning#12
 
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/simplified.txt
index 856de20a40c..f3afaea9cbf 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/simplified.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a.sf100/simplified.txt
@@ -19,7 +19,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num
                                     Filter [sales]
                                       Subquery #3
                                         WholeStageCodegen (8)
-                                          HashAggregate [sum,count] [avg(CheckOverflow((promote_precision(cast(quantity as decimal(12,2))) * promote_precision(cast(list_price as decimal(12,2)))), DecimalType(18,2))),average_sales,sum,count]
+                                          HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count]
                                             InputAdapter
                                               Exchange #18
                                                 WholeStageCodegen (7)
@@ -60,7 +60,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num
                                                                     ReusedSubquery [d_date_sk] #4
                                                               InputAdapter
                                                                 ReusedExchange [d_date_sk] #19
-                                      HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(ss_quantity as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2))),count(1),channel,sales,number_sales,sum,isEmpty,count]
+                                      HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count]
                                         InputAdapter
                                           Exchange [i_brand_id,i_class_id,i_category_id] #3
                                             WholeStageCodegen (43)
@@ -214,7 +214,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num
                                   WholeStageCodegen (88)
                                     Filter [sales]
                                       ReusedSubquery [average_sales] #3
-                                      HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cs_quantity as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2))),count(1),channel,sales,number_sales,sum,isEmpty,count]
+                                      HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count]
                                         InputAdapter
                                           Exchange [i_brand_id,i_class_id,i_category_id] #20
                                             WholeStageCodegen (87)
@@ -247,7 +247,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num
                                   WholeStageCodegen (132)
                                     Filter [sales]
                                       ReusedSubquery [average_sales] #3
-                                      HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(ws_quantity as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2))),count(1),channel,sales,number_sales,sum,isEmpty,count]
+                                      HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count]
                                         InputAdapter
                                           Exchange [i_brand_id,i_class_id,i_category_id] #22
                                             WholeStageCodegen (131)
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt
index 727c700735c..71a73b4030c 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/explain.txt
@@ -410,7 +410,7 @@ Input [7]: [ss_quantity#2, ss_list_price#3, ss_sold_date_sk#4, i_brand_id#37, i_
 (63) HashAggregate [codegen id : 25]
 Input [5]: [ss_quantity#2, ss_list_price#3, i_brand_id#37, i_class_id#38, i_category_id#39]
 Keys [3]: [i_brand_id#37, i_class_id#38, i_category_id#39]
-Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(ss_quantity#2 as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2))), partial_count(1)]
+Functions [2]: [partial_sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), partial_count(1)]
 Aggregate Attributes [3]: [sum#41, isEmpty#42, count#43]
 Results [6]: [i_brand_id#37, i_class_id#38, i_category_id#39, sum#44, isEmpty#45, count#46]
 
@@ -421,9 +421,9 @@ Arguments: hashpartitioning(i_brand_id#37, i_class_id#38, i_category_id#39, 5),
 (65) HashAggregate [codegen id : 26]
 Input [6]: [i_brand_id#37, i_class_id#38, i_category_id#39, sum#44, isEmpty#45, count#46]
 Keys [3]: [i_brand_id#37, i_class_id#38, i_category_id#39]
-Functions [2]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#2 as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2))), count(1)]
-Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(ss_quantity#2 as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2)))#47, count(1)#48]
-Results [6]: [store AS channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum(CheckOverflow((promote_precision(cast(ss_quantity#2 as decimal(12,2))) * promote_precision(cast(ss_list_price#3 as decimal(12,2)))), DecimalType(18,2)))#47 AS sales#50, count(1)#48 AS number_sales#51]
+Functions [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3)), count(1)]
+Aggregate Attributes [2]: [sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47, count(1)#48]
+Results [6]: [store AS channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sum((cast(ss_quantity#2 as decimal(10,0)) * ss_list_price#3))#47 AS sales#50, count(1)#48 AS number_sales#51]
 
 (66) Filter [codegen id : 26]
 Input [6]: [channel#49, i_brand_id#37, i_class_id#38, i_category_id#39, sales#50, number_sales#51]
@@ -479,7 +479,7 @@ Input [7]: [cs_quantity#55, cs_list_price#56, cs_sold_date_sk#57, i_brand_id#59,
 (78) HashAggregate [codegen id : 51]
 Input [5]: [cs_quantity#55, cs_list_price#56, i_brand_id#59, i_class_id#60, i_category_id#61]
 Keys [3]: [i_brand_id#59, i_class_id#60, i_category_id#61]
-Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(cs_quantity#55 as decimal(12,2))) * promote_precision(cast(cs_list_price#56 as decimal(12,2)))), DecimalType(18,2))), partial_count(1)]
+Functions [2]: [partial_sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56)), partial_count(1)]
 Aggregate Attributes [3]: [sum#63, isEmpty#64, count#65]
 Results [6]: [i_brand_id#59, i_class_id#60, i_category_id#61, sum#66, isEmpty#67, count#68]
 
@@ -490,9 +490,9 @@ Arguments: hashpartitioning(i_brand_id#59, i_class_id#60, i_category_id#61, 5),
 (80) HashAggregate [codegen id : 52]
 Input [6]: [i_brand_id#59, i_class_id#60, i_category_id#61, sum#66, isEmpty#67, count#68]
 Keys [3]: [i_brand_id#59, i_class_id#60, i_category_id#61]
-Functions [2]: [sum(CheckOverflow((promote_precision(cast(cs_quantity#55 as decimal(12,2))) * promote_precision(cast(cs_list_price#56 as decimal(12,2)))), DecimalType(18,2))), count(1)]
-Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(cs_quantity#55 as decimal(12,2))) * promote_precision(cast(cs_list_price#56 as decimal(12,2)))), DecimalType(18,2)))#69, count(1)#70]
-Results [6]: [catalog AS channel#71, i_brand_id#59, i_class_id#60, i_category_id#61, sum(CheckOverflow((promote_precision(cast(cs_quantity#55 as decimal(12,2))) * promote_precision(cast(cs_list_price#56 as decimal(12,2)))), DecimalType(18,2)))#69 AS sales#72, count(1)#70 AS number_sales#73]
+Functions [2]: [sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56)), count(1)]
+Aggregate Attributes [2]: [sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56))#69, count(1)#70]
+Results [6]: [catalog AS channel#71, i_brand_id#59, i_class_id#60, i_category_id#61, sum((cast(cs_quantity#55 as decimal(10,0)) * cs_list_price#56))#69 AS sales#72, count(1)#70 AS number_sales#73]
 
 (81) Filter [codegen id : 52]
 Input [6]: [channel#71, i_brand_id#59, i_class_id#60, i_category_id#61, sales#72, number_sales#73]
@@ -548,7 +548,7 @@ Input [7]: [ws_quantity#75, ws_list_price#76, ws_sold_date_sk#77, i_brand_id#79,
 (93) HashAggregate [codegen id : 77]
 Input [5]: [ws_quantity#75, ws_list_price#76, i_brand_id#79, i_class_id#80, i_category_id#81]
 Keys [3]: [i_brand_id#79, i_class_id#80, i_category_id#81]
-Functions [2]: [partial_sum(CheckOverflow((promote_precision(cast(ws_quantity#75 as decimal(12,2))) * promote_precision(cast(ws_list_price#76 as decimal(12,2)))), DecimalType(18,2))), partial_count(1)]
+Functions [2]: [partial_sum((cast(ws_quantity#75 as decimal(10,0)) * ws_list_price#76)), partial_count(1)]
 Aggregate Attributes [3]: [sum#83, isEmpty#84, count#85]
 Results [6]: [i_brand_id#79, i_class_id#80, i_category_id#81, sum#86, isEmpty#87, count#88]
 
@@ -559,9 +559,9 @@ Arguments: hashpartitioning(i_brand_id#79, i_class_id#80, i_category_id#81, 5),
 (95) HashAggregate [codegen id : 78]
 Input [6]: [i_brand_id#79, i_class_id#80, i_category_id#81, sum#86, isEmpty#87, count#88]
 Keys [3]: [i_brand_id#79, i_class_id#80, i_category_id#81]
-Functions [2]: [sum(CheckOverflow((promote_precision(cast(ws_quantity#75 as decimal(12,2))) * promote_precision(cast(ws_list_price#76 as decimal(12,2)))), DecimalType(18,2))), count(1)]
-Aggregate Attributes [2]: [sum(CheckOverflow((promote_precision(cast(ws_quantity#75 as decimal(12,2))) * promote_precision(cast(ws_list_price#76 as decimal(12,2)))), DecimalType(18,2)))#89, count(1)#90]
-Results [6]: [web AS channel#91, i_brand_id#79, i_class_id#80, i_category_id#81, sum(CheckOverflow((promote_precision(cast(ws_quantity#75 as decimal(12,2))) * promote_precision(cast(ws_list_price#76 as decimal(12,2)))), DecimalType(18,2)))#89 AS sales#92, count(1)#90 AS number_sales#93]
+Functions [2]: [sum((cast(ws_quantity#75 as decimal(10,0)) * ws_list_price#76)), count(1)]
+Aggregate Attributes [2]: [sum((cast(ws_quantity#75 as decimal(10,0)) * ws_list_price#76))#89, count(1)#90]
+Results [6]: [web AS channel#91, i_brand_id#79, i_class_id#80, i_category_id#81, sum((cast(ws_quantity#75 as decimal(10,0)) * ws_list_price#76))#89 AS sales#92, count(1)#90 AS number_sales#93]
 
 (96) Filter [codegen id : 78]
 Input [6]: [channel#91, i_brand_id#79, i_class_id#80, i_category_id#81, sales#92, number_sales#93]
@@ -818,7 +818,7 @@ Input [4]: [ws_quantity#167, ws_list_price#168, ws_sold_date_sk#169, d_date_sk#1
 (142) HashAggregate [codegen id : 7]
 Input [2]: [quantity#158, list_price#159]
 Keys: []
-Functions [1]: [partial_avg(CheckOverflow((promote_precision(cast(quantity#158 as decimal(12,2))) * promote_precision(cast(list_price#159 as decimal(12,2)))), DecimalType(18,2)))]
+Functions [1]: [partial_avg((cast(quantity#158 as decimal(10,0)) * list_price#159))]
 Aggregate Attributes [2]: [sum#173, count#174]
 Results [2]: [sum#175, count#176]
 
@@ -829,9 +829,9 @@ Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=18]
 (144) HashAggregate [codegen id : 8]
 Input [2]: [sum#175, count#176]
 Keys: []
-Functions [1]: [avg(CheckOverflow((promote_precision(cast(quantity#158 as decimal(12,2))) * promote_precision(cast(list_price#159 as decimal(12,2)))), DecimalType(18,2)))]
-Aggregate Attributes [1]: [avg(CheckOverflow((promote_precision(cast(quantity#158 as decimal(12,2))) * promote_precision(cast(list_price#159 as decimal(12,2)))), DecimalType(18,2)))#177]
-Results [1]: [avg(CheckOverflow((promote_precision(cast(quantity#158 as decimal(12,2))) * promote_precision(cast(list_price#159 as decimal(12,2)))), DecimalType(18,2)))#177 AS average_sales#178]
+Functions [1]: [avg((cast(quantity#158 as decimal(10,0)) * list_price#159))]
+Aggregate Attributes [1]: [avg((cast(quantity#158 as decimal(10,0)) * list_price#159))#177]
+Results [1]: [avg((cast(quantity#158 as decimal(10,0)) * list_price#159))#177 AS average_sales#178]
 
 Subquery:2 Hosting operator id = 126 Hosting Expression = ss_sold_date_sk#156 IN dynamicpruning#12
 
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt
index 086c36864eb..dcb7f5cf3f4 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q14a/simplified.txt
@@ -19,7 +19,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num
                                     Filter [sales]
                                       Subquery #3
                                         WholeStageCodegen (8)
-                                          HashAggregate [sum,count] [avg(CheckOverflow((promote_precision(cast(quantity as decimal(12,2))) * promote_precision(cast(list_price as decimal(12,2)))), DecimalType(18,2))),average_sales,sum,count]
+                                          HashAggregate [sum,count] [avg((cast(quantity as decimal(10,0)) * list_price)),average_sales,sum,count]
                                             InputAdapter
                                               Exchange #14
                                                 WholeStageCodegen (7)
@@ -60,7 +60,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num
                                                                     ReusedSubquery [d_date_sk] #4
                                                               InputAdapter
                                                                 ReusedExchange [d_date_sk] #15
-                                      HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(ss_quantity as decimal(12,2))) * promote_precision(cast(ss_list_price as decimal(12,2)))), DecimalType(18,2))),count(1),channel,sales,number_sales,sum,isEmpty,count]
+                                      HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ss_quantity as decimal(10,0)) * ss_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count]
                                         InputAdapter
                                           Exchange [i_brand_id,i_class_id,i_category_id] #3
                                             WholeStageCodegen (25)
@@ -178,7 +178,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num
                                   WholeStageCodegen (52)
                                     Filter [sales]
                                       ReusedSubquery [average_sales] #3
-                                      HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(cs_quantity as decimal(12,2))) * promote_precision(cast(cs_list_price as decimal(12,2)))), DecimalType(18,2))),count(1),channel,sales,number_sales,sum,isEmpty,count]
+                                      HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(cs_quantity as decimal(10,0)) * cs_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count]
                                         InputAdapter
                                           Exchange [i_brand_id,i_class_id,i_category_id] #16
                                             WholeStageCodegen (51)
@@ -202,7 +202,7 @@ TakeOrderedAndProject [channel,i_brand_id,i_class_id,i_category_id,sum_sales,num
                                   WholeStageCodegen (78)
                                     Filter [sales]
                                       ReusedSubquery [average_sales] #3
-                                      HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum(CheckOverflow((promote_precision(cast(ws_quantity as decimal(12,2))) * promote_precision(cast(ws_list_price as decimal(12,2)))), DecimalType(18,2))),count(1),channel,sales,number_sales,sum,isEmpty,count]
+                                      HashAggregate [i_brand_id,i_class_id,i_category_id,sum,isEmpty,count] [sum((cast(ws_quantity as decimal(10,0)) * ws_list_price)),count(1),channel,sales,number_sales,sum,isEmpty,count]
                                         InputAdapter
                                           Exchange [i_brand_id,i_class_id,i_category_id] #17
                                             WholeStageCodegen (77)
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.sf100/explain.txt
index 72200f5f5e0..0050af67525 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20.sf100/explain.txt
@@ -121,7 +121,7 @@ Input [8]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_pric
 Arguments: [sum(_w1#17) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#18], [i_class#9]
 
 (22) Project [codegen id : 9]
-Output [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(_w0#16) * 100.00), DecimalType(21,2)) as decimal(27,2))) / promote_precision(_we0#18)), DecimalType(38,17)) AS revenueratio#19]
+Output [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, ((_w0#16 * 100) / _we0#18) AS revenueratio#19]
 Input [9]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, _w1#17, _we0#18]
 
 (23) TakeOrderedAndProject
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/explain.txt
index fdc4cc9239c..42127b70e87 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q20/explain.txt
@@ -106,7 +106,7 @@ Input [8]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_pric
 Arguments: [sum(_w1#17) windowspecdefinition(i_class#9, specifiedwindowframe(RowFrame, unboundedpreceding$(), unboundedfollowing$())) AS _we0#18], [i_class#9]
 
 (19) Project [codegen id : 6]
-Output [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(_w0#16) * 100.00), DecimalType(21,2)) as decimal(27,2))) / promote_precision(_we0#18)), DecimalType(38,17)) AS revenueratio#19]
+Output [7]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, ((_w0#16 * 100) / _we0#18) AS revenueratio#19]
 Input [9]: [i_item_id#6, i_item_desc#7, i_category#10, i_class#9, i_current_price#8, itemrevenue#15, _w0#16, _w1#17, _we0#18]
 
 (20) TakeOrderedAndProject
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/explain.txt
index 22079f0f10a..d7f26a2d441 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24.sf100/explain.txt
@@ -430,6 +430,6 @@ Input [2]: [sum#46, count#47]
 Keys: []
 Functions [1]: [avg(netpaid#33)]
 Aggregate Attributes [1]: [avg(netpaid#33)#48]
-Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#33)#48)), DecimalType(24,8)) AS (0.05 * avg(netpaid))#49]
+Results [1]: [(0.05 * avg(netpaid#33)#48) AS (0.05 * avg(netpaid))#49]
 
 
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/explain.txt
index ccf92a79559..57ec2c6c35d 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q24/explain.txt
@@ -422,6 +422,6 @@ Input [2]: [sum#46, count#47]
 Keys: []
 Functions [1]: [avg(netpaid#33)]
 Aggregate Attributes [1]: [avg(netpaid#33)#48]
-Results [1]: [CheckOverflow((0.050000 * promote_precision(avg(netpaid#33)#48)), DecimalType(24,8)) AS (0.05 * avg(netpaid))#49]
+Results [1]: [(0.05 * avg(netpaid#33)#48) AS (0.05 * avg(netpaid))#49]
 
 
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.sf100/explain.txt
index 15815ca9b94..92b0cf86563 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a.sf100/explain.txt
@@ -143,7 +143,7 @@ Input [4]: [i_category#12, i_class#11, sum#15, sum#16]
 Keys [2]: [i_category#12, i_class#11]
 Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))]
 Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#17, sum(UnscaledValue(ss_ext_sales_price#3))#18]
-Results [6]: [cast(CheckOverflow((promote_precision(MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#17,17,2)) / promote_precision(MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#18,17,2))), DecimalType(37,20)) as decimal(38,20)) AS gross_margin#19, i_category#12, i_class#11, 0 AS t_category#20, 0 AS t_class#21, 0 AS lochierarchy#22]
+Results [6]: [cast((MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#17,17,2) / MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#18,17,2)) as decimal(38,20)) AS gross_margin#19, i_category#12, i_class#11, 0 AS t_category#20, 0 AS t_class#21, 0 AS lochierarchy#22]
 
 (23) ReusedExchange [Reuses operator id: 21]
 Output [4]: [i_category#12, i_class#11, sum#23, sum#24]
@@ -171,7 +171,7 @@ Input [5]: [i_category#12, sum#33, isEmpty#34, sum#35, isEmpty#36]
 Keys [1]: [i_category#12]
 Functions [2]: [sum(ss_net_profit#27), sum(ss_ext_sales_price#28)]
 Aggregate Attributes [2]: [sum(ss_net_profit#27)#37, sum(ss_ext_sales_price#28)#38]
-Results [6]: [cast(CheckOverflow((promote_precision(sum(ss_net_profit#27)#37) / promote_precision(sum(ss_ext_sales_price#28)#38)), DecimalType(38,11)) as decimal(38,20)) AS gross_margin#39, i_category#12, null AS i_class#40, 0 AS t_category#41, 1 AS t_class#42, 1 AS lochierarchy#43]
+Results [6]: [cast((sum(ss_net_profit#27)#37 / sum(ss_ext_sales_price#28)#38) as decimal(38,20)) AS gross_margin#39, i_category#12, null AS i_class#40, 0 AS t_category#41, 1 AS t_class#42, 1 AS lochierarchy#43]
 
 (28) ReusedExchange [Reuses operator id: 21]
 Output [4]: [i_category#12, i_class#11, sum#44, sum#45]
@@ -199,7 +199,7 @@ Input [4]: [sum#50, isEmpty#51, sum#52, isEmpty#53]
 Keys: []
 Functions [2]: [sum(ss_net_profit#27), sum(ss_ext_sales_price#28)]
 Aggregate Attributes [2]: [sum(ss_net_profit#27)#54, sum(ss_ext_sales_price#28)#55]
-Results [6]: [cast(CheckOverflow((promote_precision(sum(ss_net_profit#27)#54) / promote_precision(sum(ss_ext_sales_price#28)#55)), DecimalType(38,11)) as decimal(38,20)) AS gross_margin#56, null AS i_category#57, null AS i_class#58, 1 AS t_category#59, 1 AS t_class#60, 2 AS lochierarchy#61]
+Results [6]: [cast((sum(ss_net_profit#27)#54 / sum(ss_ext_sales_price#28)#55) as decimal(38,20)) AS gross_margin#56, null AS i_category#57, null AS i_class#58, 1 AS t_category#59, 1 AS t_class#60, 2 AS lochierarchy#61]
 
 (33) Union
 
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/explain.txt
index 65df229e9c1..4689f03b9d4 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q36a/explain.txt
@@ -143,7 +143,7 @@ Input [4]: [i_category#10, i_class#9, sum#15, sum#16]
 Keys [2]: [i_category#10, i_class#9]
 Functions [2]: [sum(UnscaledValue(ss_net_profit#4)), sum(UnscaledValue(ss_ext_sales_price#3))]
 Aggregate Attributes [2]: [sum(UnscaledValue(ss_net_profit#4))#17, sum(UnscaledValue(ss_ext_sales_price#3))#18]
-Results [6]: [cast(CheckOverflow((promote_precision(MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#17,17,2)) / promote_precision(MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#18,17,2))), DecimalType(37,20)) as decimal(38,20)) AS gross_margin#19, i_category#10, i_class#9, 0 AS t_category#20, 0 AS t_class#21, 0 AS lochierarchy#22]
+Results [6]: [cast((MakeDecimal(sum(UnscaledValue(ss_net_profit#4))#17,17,2) / MakeDecimal(sum(UnscaledValue(ss_ext_sales_price#3))#18,17,2)) as decimal(38,20)) AS gross_margin#19, i_category#10, i_class#9, 0 AS t_category#20, 0 AS t_class#21, 0 AS lochierarchy#22]
 
 (23) ReusedExchange [Reuses operator id: 21]
 Output [4]: [i_category#10, i_class#9, sum#23, sum#24]
@@ -171,7 +171,7 @@ Input [5]: [i_category#10, sum#33, isEmpty#34, sum#35, isEmpty#36]
 Keys [1]: [i_category#10]
 Functions [2]: [sum(ss_net_profit#27), sum(ss_ext_sales_price#28)]
 Aggregate Attributes [2]: [sum(ss_net_profit#27)#37, sum(ss_ext_sales_price#28)#38]
-Results [6]: [cast(CheckOverflow((promote_precision(sum(ss_net_profit#27)#37) / promote_precision(sum(ss_ext_sales_price#28)#38)), DecimalType(38,11)) as decimal(38,20)) AS gross_margin#39, i_category#10, null AS i_class#40, 0 AS t_category#41, 1 AS t_class#42, 1 AS lochierarchy#43]
+Results [6]: [cast((sum(ss_net_profit#27)#37 / sum(ss_ext_sales_price#28)#38) as decimal(38,20)) AS gross_margin#39, i_category#10, null AS i_class#40, 0 AS t_category#41, 1 AS t_class#42, 1 AS lochierarchy#43]
 
 (28) ReusedExchange [Reuses operator id: 21]
 Output [4]: [i_category#10, i_class#9, sum#44, sum#45]
@@ -199,7 +199,7 @@ Input [4]: [sum#50, isEmpty#51, sum#52, isEmpty#53]
 Keys: []
 Functions [2]: [sum(ss_net_profit#27), sum(ss_ext_sales_price#28)]
 Aggregate Attributes [2]: [sum(ss_net_profit#27)#54, sum(ss_ext_sales_price#28)#55]
-Results [6]: [cast(CheckOverflow((promote_precision(sum(ss_net_profit#27)#54) / promote_precision(sum(ss_ext_sales_price#28)#55)), DecimalType(38,11)) as decimal(38,20)) AS gross_margin#56, null AS i_category#57, null AS i_class#58, 1 AS t_category#59, 1 AS t_class#60, 2 AS lochierarchy#61]
+Results [6]: [cast((sum(ss_net_profit#27)#54 / sum(ss_ext_sales_price#28)#55) as decimal(38,20)) AS gross_margin#56, null AS i_category#57, null AS i_class#58, 1 AS t_category#59, 1 AS t_class#60, 2 AS lochierarchy#61]
 
 (33) Union
 
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.sf100/explain.txt
index 6d94222679e..1a66fc83800 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47.sf100/explain.txt
@@ -186,7 +186,7 @@ Arguments: [avg(_w0#19) windowspecdefinition(i_category#14, i_brand#13, s_store_
 
 (30) Filter [codegen id : 11]
 Input [10]: [i_category#14, i_brand#13, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21]
-Condition : ((isnotnull(avg_monthly_sales#21) AND (avg_monthly_sales#21 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#18 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#21 as decimal(22,6)))), DecimalType(22,6)))) / promote_precision(cast(avg_monthly_sales#21 as decimal(22,6)))), DecimalType(38,16)) > 0.1000000000000000))
+Condition : (((isnotnull(avg_monthly_sales#21) AND isnotnull(sum_sales#18)) AND (avg_monthly_sales#21 > 0.000000)) AND ((abs((sum_sales#18 - avg_monthly_sales#21)) / avg_monthly_sales#21) > 0.1000000000000000))
 
 (31) Project [codegen id : 11]
 Output [9]: [i_category#14, i_brand#13, s_store_name#10, s_company_name#11, d_year#7, d_moy#8, sum_sales#18, avg_monthly_sales#21, rn#20]
@@ -277,7 +277,7 @@ Input [16]: [i_category#14, i_brand#13, s_store_name#10, s_company_name#11, d_ye
 
 (52) TakeOrderedAndProject
 Input [7]: [i_category#14, d_year#7, d_moy#8, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41]
-Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#18 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#21 as decimal(22,6)))), DecimalType(22,6)) ASC NULLS FIRST, d_moy#8 ASC NULLS FIRST], [i_category#14, d_year#7, d_moy#8, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41]
+Arguments: 100, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, d_moy#8 ASC NULLS FIRST], [i_category#14, d_year#7, d_moy#8, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41]
 
 ===== Subqueries =====
 
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/explain.txt
index 5cd32d2922d..bbf7d425de4 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q47/explain.txt
@@ -167,7 +167,7 @@ Arguments: [avg(_w0#19) windowspecdefinition(i_category#3, i_brand#2, s_store_na
 
 (27) Filter [codegen id : 22]
 Input [10]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, _w0#19, rn#20, avg_monthly_sales#21]
-Condition : ((isnotnull(avg_monthly_sales#21) AND (avg_monthly_sales#21 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#18 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#21 as decimal(22,6)))), DecimalType(22,6)))) / promote_precision(cast(avg_monthly_sales#21 as decimal(22,6)))), DecimalType(38,16)) > 0.1000000000000000))
+Condition : (((isnotnull(avg_monthly_sales#21) AND isnotnull(sum_sales#18)) AND (avg_monthly_sales#21 > 0.000000)) AND ((abs((sum_sales#18 - avg_monthly_sales#21)) / avg_monthly_sales#21) > 0.1000000000000000))
 
 (28) Project [codegen id : 22]
 Output [9]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year#10, d_moy#11, sum_sales#18, avg_monthly_sales#21, rn#20]
@@ -242,7 +242,7 @@ Input [16]: [i_category#3, i_brand#2, s_store_name#13, s_company_name#14, d_year
 
 (45) TakeOrderedAndProject
 Input [7]: [i_category#3, d_year#10, d_moy#11, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41]
-Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#18 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#21 as decimal(22,6)))), DecimalType(22,6)) ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST], [i_category#3, d_year#10, d_moy#11, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41]
+Arguments: 100, [(sum_sales#18 - avg_monthly_sales#21) ASC NULLS FIRST, d_moy#11 ASC NULLS FIRST], [i_category#3, d_year#10, d_moy#11, avg_monthly_sales#21, sum_sales#18, psum#40, nsum#41]
 
 ===== Subqueries =====
 
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.sf100/explain.txt
index 6a546a42ff3..5b500c63e01 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49.sf100/explain.txt
@@ -177,7 +177,7 @@ Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25
 Keys [1]: [ws_item_sk#1]
 Functions [4]: [sum(coalesce(wr_return_quantity#11, 0)), sum(coalesce(ws_quantity#3, 0)), sum(coalesce(cast(wr_return_amt#12 as decimal(12,2)), 0.00)), sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))]
 Aggregate Attributes [4]: [sum(coalesce(wr_return_quantity#11, 0))#26, sum(coalesce(ws_quantity#3, 0))#27, sum(coalesce(cast(wr_return_amt#12 as decimal(12,2)), 0.00))#28, sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))#29]
-Results [3]: [ws_item_sk#1 AS item#30, CheckOverflow((promote_precision(cast(sum(coalesce(wr_return_quantity#11, 0))#26 as decimal(15,4))) / promote_precision(cast(sum(coalesce(ws_quantity#3, 0))#27 as decimal(15,4)))), DecimalType(35,20)) AS return_ratio#31, CheckOverflow((promote_precision(cast(sum(coalesce(cast(wr_return_amt#12 as decimal(12,2)), 0.00))#28 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))#29 as decimal(15,4)))), Deci [...]
+Results [3]: [ws_item_sk#1 AS item#30, (cast(sum(coalesce(wr_return_quantity#11, 0))#26 as decimal(15,4)) / cast(sum(coalesce(ws_quantity#3, 0))#27 as decimal(15,4))) AS return_ratio#31, (cast(sum(coalesce(cast(wr_return_amt#12 as decimal(12,2)), 0.00))#28 as decimal(15,4)) / cast(sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))#29 as decimal(15,4))) AS currency_ratio#32]
 
 (21) Exchange
 Input [3]: [item#30, return_ratio#31, currency_ratio#32]
@@ -297,7 +297,7 @@ Input [7]: [cs_item_sk#36, sum#54, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#5
 Keys [1]: [cs_item_sk#36]
 Functions [4]: [sum(coalesce(cr_return_quantity#45, 0)), sum(coalesce(cs_quantity#38, 0)), sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))]
 Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#45, 0))#60, sum(coalesce(cs_quantity#38, 0))#61, sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00))#62, sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))#63]
-Results [3]: [cs_item_sk#36 AS item#64, CheckOverflow((promote_precision(cast(sum(coalesce(cr_return_quantity#45, 0))#60 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cs_quantity#38, 0))#61 as decimal(15,4)))), DecimalType(35,20)) AS return_ratio#65, CheckOverflow((promote_precision(cast(sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00))#62 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))#63 as decimal(15,4)))) [...]
+Results [3]: [cs_item_sk#36 AS item#64, (cast(sum(coalesce(cr_return_quantity#45, 0))#60 as decimal(15,4)) / cast(sum(coalesce(cs_quantity#38, 0))#61 as decimal(15,4))) AS return_ratio#65, (cast(sum(coalesce(cast(cr_return_amount#46 as decimal(12,2)), 0.00))#62 as decimal(15,4)) / cast(sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))#63 as decimal(15,4))) AS currency_ratio#66]
 
 (48) Exchange
 Input [3]: [item#64, return_ratio#65, currency_ratio#66]
@@ -417,7 +417,7 @@ Input [7]: [ss_item_sk#70, sum#88, sum#89, sum#90, isEmpty#91, sum#92, isEmpty#9
 Keys [1]: [ss_item_sk#70]
 Functions [4]: [sum(coalesce(sr_return_quantity#79, 0)), sum(coalesce(ss_quantity#72, 0)), sum(coalesce(cast(sr_return_amt#80 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#73 as decimal(12,2)), 0.00))]
 Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#79, 0))#94, sum(coalesce(ss_quantity#72, 0))#95, sum(coalesce(cast(sr_return_amt#80 as decimal(12,2)), 0.00))#96, sum(coalesce(cast(ss_net_paid#73 as decimal(12,2)), 0.00))#97]
-Results [3]: [ss_item_sk#70 AS item#98, CheckOverflow((promote_precision(cast(sum(coalesce(sr_return_quantity#79, 0))#94 as decimal(15,4))) / promote_precision(cast(sum(coalesce(ss_quantity#72, 0))#95 as decimal(15,4)))), DecimalType(35,20)) AS return_ratio#99, CheckOverflow((promote_precision(cast(sum(coalesce(cast(sr_return_amt#80 as decimal(12,2)), 0.00))#96 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(ss_net_paid#73 as decimal(12,2)), 0.00))#97 as decimal(15,4)))), D [...]
+Results [3]: [ss_item_sk#70 AS item#98, (cast(sum(coalesce(sr_return_quantity#79, 0))#94 as decimal(15,4)) / cast(sum(coalesce(ss_quantity#72, 0))#95 as decimal(15,4))) AS return_ratio#99, (cast(sum(coalesce(cast(sr_return_amt#80 as decimal(12,2)), 0.00))#96 as decimal(15,4)) / cast(sum(coalesce(cast(ss_net_paid#73 as decimal(12,2)), 0.00))#97 as decimal(15,4))) AS currency_ratio#100]
 
 (75) Exchange
 Input [3]: [item#98, return_ratio#99, currency_ratio#100]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt
index b6cfd8a096c..1a18e3166e4 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q49/explain.txt
@@ -156,7 +156,7 @@ Input [7]: [ws_item_sk#1, sum#20, sum#21, sum#22, isEmpty#23, sum#24, isEmpty#25
 Keys [1]: [ws_item_sk#1]
 Functions [4]: [sum(coalesce(wr_return_quantity#10, 0)), sum(coalesce(ws_quantity#3, 0)), sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00)), sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))]
 Aggregate Attributes [4]: [sum(coalesce(wr_return_quantity#10, 0))#26, sum(coalesce(ws_quantity#3, 0))#27, sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00))#28, sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))#29]
-Results [3]: [ws_item_sk#1 AS item#30, CheckOverflow((promote_precision(cast(sum(coalesce(wr_return_quantity#10, 0))#26 as decimal(15,4))) / promote_precision(cast(sum(coalesce(ws_quantity#3, 0))#27 as decimal(15,4)))), DecimalType(35,20)) AS return_ratio#31, CheckOverflow((promote_precision(cast(sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00))#28 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))#29 as decimal(15,4)))), Deci [...]
+Results [3]: [ws_item_sk#1 AS item#30, (cast(sum(coalesce(wr_return_quantity#10, 0))#26 as decimal(15,4)) / cast(sum(coalesce(ws_quantity#3, 0))#27 as decimal(15,4))) AS return_ratio#31, (cast(sum(coalesce(cast(wr_return_amt#11 as decimal(12,2)), 0.00))#28 as decimal(15,4)) / cast(sum(coalesce(cast(ws_net_paid#4 as decimal(12,2)), 0.00))#29 as decimal(15,4))) AS currency_ratio#32]
 
 (18) Exchange
 Input [3]: [item#30, return_ratio#31, currency_ratio#32]
@@ -264,7 +264,7 @@ Input [7]: [cs_item_sk#36, sum#54, sum#55, sum#56, isEmpty#57, sum#58, isEmpty#5
 Keys [1]: [cs_item_sk#36]
 Functions [4]: [sum(coalesce(cr_return_quantity#44, 0)), sum(coalesce(cs_quantity#38, 0)), sum(coalesce(cast(cr_return_amount#45 as decimal(12,2)), 0.00)), sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))]
 Aggregate Attributes [4]: [sum(coalesce(cr_return_quantity#44, 0))#60, sum(coalesce(cs_quantity#38, 0))#61, sum(coalesce(cast(cr_return_amount#45 as decimal(12,2)), 0.00))#62, sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))#63]
-Results [3]: [cs_item_sk#36 AS item#64, CheckOverflow((promote_precision(cast(sum(coalesce(cr_return_quantity#44, 0))#60 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cs_quantity#38, 0))#61 as decimal(15,4)))), DecimalType(35,20)) AS return_ratio#65, CheckOverflow((promote_precision(cast(sum(coalesce(cast(cr_return_amount#45 as decimal(12,2)), 0.00))#62 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))#63 as decimal(15,4)))) [...]
+Results [3]: [cs_item_sk#36 AS item#64, (cast(sum(coalesce(cr_return_quantity#44, 0))#60 as decimal(15,4)) / cast(sum(coalesce(cs_quantity#38, 0))#61 as decimal(15,4))) AS return_ratio#65, (cast(sum(coalesce(cast(cr_return_amount#45 as decimal(12,2)), 0.00))#62 as decimal(15,4)) / cast(sum(coalesce(cast(cs_net_paid#39 as decimal(12,2)), 0.00))#63 as decimal(15,4))) AS currency_ratio#66]
 
 (42) Exchange
 Input [3]: [item#64, return_ratio#65, currency_ratio#66]
@@ -372,7 +372,7 @@ Input [7]: [ss_item_sk#70, sum#88, sum#89, sum#90, isEmpty#91, sum#92, isEmpty#9
 Keys [1]: [ss_item_sk#70]
 Functions [4]: [sum(coalesce(sr_return_quantity#78, 0)), sum(coalesce(ss_quantity#72, 0)), sum(coalesce(cast(sr_return_amt#79 as decimal(12,2)), 0.00)), sum(coalesce(cast(ss_net_paid#73 as decimal(12,2)), 0.00))]
 Aggregate Attributes [4]: [sum(coalesce(sr_return_quantity#78, 0))#94, sum(coalesce(ss_quantity#72, 0))#95, sum(coalesce(cast(sr_return_amt#79 as decimal(12,2)), 0.00))#96, sum(coalesce(cast(ss_net_paid#73 as decimal(12,2)), 0.00))#97]
-Results [3]: [ss_item_sk#70 AS item#98, CheckOverflow((promote_precision(cast(sum(coalesce(sr_return_quantity#78, 0))#94 as decimal(15,4))) / promote_precision(cast(sum(coalesce(ss_quantity#72, 0))#95 as decimal(15,4)))), DecimalType(35,20)) AS return_ratio#99, CheckOverflow((promote_precision(cast(sum(coalesce(cast(sr_return_amt#79 as decimal(12,2)), 0.00))#96 as decimal(15,4))) / promote_precision(cast(sum(coalesce(cast(ss_net_paid#73 as decimal(12,2)), 0.00))#97 as decimal(15,4)))), D [...]
+Results [3]: [ss_item_sk#70 AS item#98, (cast(sum(coalesce(sr_return_quantity#78, 0))#94 as decimal(15,4)) / cast(sum(coalesce(ss_quantity#72, 0))#95 as decimal(15,4))) AS return_ratio#99, (cast(sum(coalesce(cast(sr_return_amt#79 as decimal(12,2)), 0.00))#96 as decimal(15,4)) / cast(sum(coalesce(cast(ss_net_paid#73 as decimal(12,2)), 0.00))#97 as decimal(15,4))) AS currency_ratio#100]
 
 (66) Exchange
 Input [3]: [item#98, return_ratio#99, currency_ratio#100]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.sf100/explain.txt
index 857e754bf67..809f79575f5 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57.sf100/explain.txt
@@ -186,7 +186,7 @@ Arguments: [avg(_w0#18) windowspecdefinition(i_category#13, i_brand#12, cc_name#
 
 (30) Filter [codegen id : 11]
 Input [9]: [i_category#13, i_brand#12, cc_name#10, d_year#7, d_moy#8, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20]
-Condition : ((isnotnull(avg_monthly_sales#20) AND (avg_monthly_sales#20 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#17 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#20 as decimal(22,6)))), DecimalType(22,6)))) / promote_precision(cast(avg_monthly_sales#20 as decimal(22,6)))), DecimalType(38,16)) > 0.1000000000000000))
+Condition : (((isnotnull(avg_monthly_sales#20) AND isnotnull(sum_sales#17)) AND (avg_monthly_sales#20 > 0.000000)) AND ((abs((sum_sales#17 - avg_monthly_sales#20)) / avg_monthly_sales#20) > 0.1000000000000000))
 
 (31) Project [codegen id : 11]
 Output [8]: [i_category#13, i_brand#12, cc_name#10, d_year#7, d_moy#8, sum_sales#17, avg_monthly_sales#20, rn#19]
@@ -277,7 +277,7 @@ Input [14]: [i_category#13, i_brand#12, cc_name#10, d_year#7, d_moy#8, sum_sales
 
 (52) TakeOrderedAndProject
 Input [8]: [i_category#13, i_brand#12, d_year#7, d_moy#8, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38]
-Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#17 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#20 as decimal(22,6)))), DecimalType(22,6)) ASC NULLS FIRST, d_year#7 ASC NULLS FIRST], [i_category#13, i_brand#12, d_year#7, d_moy#8, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38]
+Arguments: 100, [(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST, d_year#7 ASC NULLS FIRST], [i_category#13, i_brand#12, d_year#7, d_moy#8, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38]
 
 ===== Subqueries =====
 
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/explain.txt
index 0fa6debb223..1e83a06db7e 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q57/explain.txt
@@ -167,7 +167,7 @@ Arguments: [avg(_w0#18) windowspecdefinition(i_category#3, i_brand#2, cc_name#13
 
 (27) Filter [codegen id : 22]
 Input [9]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, _w0#18, rn#19, avg_monthly_sales#20]
-Condition : ((isnotnull(avg_monthly_sales#20) AND (avg_monthly_sales#20 > 0.000000)) AND (CheckOverflow((promote_precision(abs(CheckOverflow((promote_precision(cast(sum_sales#17 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#20 as decimal(22,6)))), DecimalType(22,6)))) / promote_precision(cast(avg_monthly_sales#20 as decimal(22,6)))), DecimalType(38,16)) > 0.1000000000000000))
+Condition : (((isnotnull(avg_monthly_sales#20) AND isnotnull(sum_sales#17)) AND (avg_monthly_sales#20 > 0.000000)) AND ((abs((sum_sales#17 - avg_monthly_sales#20)) / avg_monthly_sales#20) > 0.1000000000000000))
 
 (28) Project [codegen id : 22]
 Output [8]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales#17, avg_monthly_sales#20, rn#19]
@@ -242,7 +242,7 @@ Input [14]: [i_category#3, i_brand#2, cc_name#13, d_year#10, d_moy#11, sum_sales
 
 (45) TakeOrderedAndProject
 Input [8]: [i_category#3, i_brand#2, d_year#10, d_moy#11, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38]
-Arguments: 100, [CheckOverflow((promote_precision(cast(sum_sales#17 as decimal(22,6))) - promote_precision(cast(avg_monthly_sales#20 as decimal(22,6)))), DecimalType(22,6)) ASC NULLS FIRST, d_year#10 ASC NULLS FIRST], [i_category#3, i_brand#2, d_year#10, d_moy#11, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38]
+Arguments: 100, [(sum_sales#17 - avg_monthly_sales#20) ASC NULLS FIRST, d_year#10 ASC NULLS FIRST], [i_category#3, i_brand#2, d_year#10, d_moy#11, avg_monthly_sales#20, sum_sales#17, psum#37, nsum#38]
 
 ===== Subqueries =====
 
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/explain.txt
index 05c7834b2ab..9f493681d1d 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a.sf100/explain.txt
@@ -186,7 +186,7 @@ Input [5]: [s_store_id#23, sum#29, sum#30, sum#31, sum#32]
 Keys [1]: [s_store_id#23]
 Functions [4]: [sum(UnscaledValue(sales_price#8)), sum(UnscaledValue(return_amt#10)), sum(UnscaledValue(profit#9)), sum(UnscaledValue(net_loss#11))]
 Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#8))#33, sum(UnscaledValue(return_amt#10))#34, sum(UnscaledValue(profit#9))#35, sum(UnscaledValue(net_loss#11))#36]
-Results [5]: [store channel AS channel#37, concat(store, s_store_id#23) AS id#38, MakeDecimal(sum(UnscaledValue(sales_price#8))#33,17,2) AS sales#39, MakeDecimal(sum(UnscaledValue(return_amt#10))#34,17,2) AS returns#40, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#9))#35,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#11))#36,17,2) as decimal(18,2)))), DecimalType(18,2)) AS profit#41]
+Results [5]: [store channel AS channel#37, concat(store, s_store_id#23) AS id#38, MakeDecimal(sum(UnscaledValue(sales_price#8))#33,17,2) AS sales#39, MakeDecimal(sum(UnscaledValue(return_amt#10))#34,17,2) AS returns#40, (MakeDecimal(sum(UnscaledValue(profit#9))#35,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#11))#36,17,2)) AS profit#41]
 
 (22) Scan parquet default.catalog_sales
 Output [4]: [cs_catalog_page_sk#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45]
@@ -283,7 +283,7 @@ Input [5]: [cp_catalog_page_id#63, sum#69, sum#70, sum#71, sum#72]
 Keys [1]: [cp_catalog_page_id#63]
 Functions [4]: [sum(UnscaledValue(sales_price#48)), sum(UnscaledValue(return_amt#50)), sum(UnscaledValue(profit#49)), sum(UnscaledValue(net_loss#51))]
 Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#48))#73, sum(UnscaledValue(return_amt#50))#74, sum(UnscaledValue(profit#49))#75, sum(UnscaledValue(net_loss#51))#76]
-Results [5]: [catalog channel AS channel#77, concat(catalog_page, cp_catalog_page_id#63) AS id#78, MakeDecimal(sum(UnscaledValue(sales_price#48))#73,17,2) AS sales#79, MakeDecimal(sum(UnscaledValue(return_amt#50))#74,17,2) AS returns#80, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#49))#75,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#51))#76,17,2) as decimal(18,2)))), DecimalType(18,2)) AS profit#81]
+Results [5]: [catalog channel AS channel#77, concat(catalog_page, cp_catalog_page_id#63) AS id#78, MakeDecimal(sum(UnscaledValue(sales_price#48))#73,17,2) AS sales#79, MakeDecimal(sum(UnscaledValue(return_amt#50))#74,17,2) AS returns#80, (MakeDecimal(sum(UnscaledValue(profit#49))#75,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#51))#76,17,2)) AS profit#81]
 
 (43) Scan parquet default.web_sales
 Output [4]: [ws_web_site_sk#82, ws_ext_sales_price#83, ws_net_profit#84, ws_sold_date_sk#85]
@@ -414,7 +414,7 @@ Input [5]: [web_site_id#108, sum#114, sum#115, sum#116, sum#117]
 Keys [1]: [web_site_id#108]
 Functions [4]: [sum(UnscaledValue(sales_price#88)), sum(UnscaledValue(return_amt#90)), sum(UnscaledValue(profit#89)), sum(UnscaledValue(net_loss#91))]
 Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#88))#118, sum(UnscaledValue(return_amt#90))#119, sum(UnscaledValue(profit#89))#120, sum(UnscaledValue(net_loss#91))#121]
-Results [5]: [web channel AS channel#122, concat(web_site, web_site_id#108) AS id#123, MakeDecimal(sum(UnscaledValue(sales_price#88))#118,17,2) AS sales#124, MakeDecimal(sum(UnscaledValue(return_amt#90))#119,17,2) AS returns#125, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#89))#120,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#91))#121,17,2) as decimal(18,2)))), DecimalType(18,2)) AS profit#126]
+Results [5]: [web channel AS channel#122, concat(web_site, web_site_id#108) AS id#123, MakeDecimal(sum(UnscaledValue(sales_price#88))#118,17,2) AS sales#124, MakeDecimal(sum(UnscaledValue(return_amt#90))#119,17,2) AS returns#125, (MakeDecimal(sum(UnscaledValue(profit#89))#120,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#91))#121,17,2)) AS profit#126]
 
 (72) Union
 
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt
index 225f1d26b0e..e9af0079c31 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q5a/explain.txt
@@ -183,7 +183,7 @@ Input [5]: [s_store_id#24, sum#29, sum#30, sum#31, sum#32]
 Keys [1]: [s_store_id#24]
 Functions [4]: [sum(UnscaledValue(sales_price#8)), sum(UnscaledValue(return_amt#10)), sum(UnscaledValue(profit#9)), sum(UnscaledValue(net_loss#11))]
 Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#8))#33, sum(UnscaledValue(return_amt#10))#34, sum(UnscaledValue(profit#9))#35, sum(UnscaledValue(net_loss#11))#36]
-Results [5]: [store channel AS channel#37, concat(store, s_store_id#24) AS id#38, MakeDecimal(sum(UnscaledValue(sales_price#8))#33,17,2) AS sales#39, MakeDecimal(sum(UnscaledValue(return_amt#10))#34,17,2) AS returns#40, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#9))#35,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#11))#36,17,2) as decimal(18,2)))), DecimalType(18,2)) AS profit#41]
+Results [5]: [store channel AS channel#37, concat(store, s_store_id#24) AS id#38, MakeDecimal(sum(UnscaledValue(sales_price#8))#33,17,2) AS sales#39, MakeDecimal(sum(UnscaledValue(return_amt#10))#34,17,2) AS returns#40, (MakeDecimal(sum(UnscaledValue(profit#9))#35,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#11))#36,17,2)) AS profit#41]
 
 (22) Scan parquet default.catalog_sales
 Output [4]: [cs_catalog_page_sk#42, cs_ext_sales_price#43, cs_net_profit#44, cs_sold_date_sk#45]
@@ -280,7 +280,7 @@ Input [5]: [cp_catalog_page_id#64, sum#69, sum#70, sum#71, sum#72]
 Keys [1]: [cp_catalog_page_id#64]
 Functions [4]: [sum(UnscaledValue(sales_price#48)), sum(UnscaledValue(return_amt#50)), sum(UnscaledValue(profit#49)), sum(UnscaledValue(net_loss#51))]
 Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#48))#73, sum(UnscaledValue(return_amt#50))#74, sum(UnscaledValue(profit#49))#75, sum(UnscaledValue(net_loss#51))#76]
-Results [5]: [catalog channel AS channel#77, concat(catalog_page, cp_catalog_page_id#64) AS id#78, MakeDecimal(sum(UnscaledValue(sales_price#48))#73,17,2) AS sales#79, MakeDecimal(sum(UnscaledValue(return_amt#50))#74,17,2) AS returns#80, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#49))#75,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#51))#76,17,2) as decimal(18,2)))), DecimalType(18,2)) AS profit#81]
+Results [5]: [catalog channel AS channel#77, concat(catalog_page, cp_catalog_page_id#64) AS id#78, MakeDecimal(sum(UnscaledValue(sales_price#48))#73,17,2) AS sales#79, MakeDecimal(sum(UnscaledValue(return_amt#50))#74,17,2) AS returns#80, (MakeDecimal(sum(UnscaledValue(profit#49))#75,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#51))#76,17,2)) AS profit#81]
 
 (43) Scan parquet default.web_sales
 Output [4]: [ws_web_site_sk#82, ws_ext_sales_price#83, ws_net_profit#84, ws_sold_date_sk#85]
@@ -399,7 +399,7 @@ Input [5]: [web_site_id#109, sum#114, sum#115, sum#116, sum#117]
 Keys [1]: [web_site_id#109]
 Functions [4]: [sum(UnscaledValue(sales_price#88)), sum(UnscaledValue(return_amt#90)), sum(UnscaledValue(profit#89)), sum(UnscaledValue(net_loss#91))]
 Aggregate Attributes [4]: [sum(UnscaledValue(sales_price#88))#118, sum(UnscaledValue(return_amt#90))#119, sum(UnscaledValue(profit#89))#120, sum(UnscaledValue(net_loss#91))#121]
-Results [5]: [web channel AS channel#122, concat(web_site, web_site_id#109) AS id#123, MakeDecimal(sum(UnscaledValue(sales_price#88))#118,17,2) AS sales#124, MakeDecimal(sum(UnscaledValue(return_amt#90))#119,17,2) AS returns#125, CheckOverflow((promote_precision(cast(MakeDecimal(sum(UnscaledValue(profit#89))#120,17,2) as decimal(18,2))) - promote_precision(cast(MakeDecimal(sum(UnscaledValue(net_loss#91))#121,17,2) as decimal(18,2)))), DecimalType(18,2)) AS profit#126]
+Results [5]: [web channel AS channel#122, concat(web_site, web_site_id#109) AS id#123, MakeDecimal(sum(UnscaledValue(sales_price#88))#118,17,2) AS sales#124, MakeDecimal(sum(UnscaledValue(return_amt#90))#119,17,2) AS returns#125, (MakeDecimal(sum(UnscaledValue(profit#89))#120,17,2) - MakeDecimal(sum(UnscaledValue(net_loss#91))#121,17,2)) AS profit#126]
 
 (69) Union
 
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/explain.txt
index 2e353046dae..df2efa142cc 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/explain.txt
@@ -11,24 +11,24 @@ TakeOrderedAndProject (45)
                   :     +- * Project (22)
                   :        +- * BroadcastHashJoin Inner BuildRight (21)
                   :           :- * Project (19)
-                  :           :  +- * BroadcastHashJoin Inner BuildRight (18)
-                  :           :     :- * Filter (3)
-                  :           :     :  +- * ColumnarToRow (2)
-                  :           :     :     +- Scan parquet default.store_sales (1)
-                  :           :     +- BroadcastExchange (17)
-                  :           :        +- * Project (16)
-                  :           :           +- * Filter (15)
-                  :           :              +- * BroadcastHashJoin LeftOuter BuildRight (14)
-                  :           :                 :- * Filter (6)
-                  :           :                 :  +- * ColumnarToRow (5)
-                  :           :                 :     +- Scan parquet default.item (4)
-                  :           :                 +- BroadcastExchange (13)
-                  :           :                    +- * HashAggregate (12)
-                  :           :                       +- Exchange (11)
-                  :           :                          +- * HashAggregate (10)
-                  :           :                             +- * Filter (9)
-                  :           :                                +- * ColumnarToRow (8)
-                  :           :                                   +- Scan parquet default.item (7)
+                  :           :  +- * BroadcastHashJoin Inner BuildLeft (18)
+                  :           :     :- BroadcastExchange (14)
+                  :           :     :  +- * Project (13)
+                  :           :     :     +- * BroadcastHashJoin Inner BuildRight (12)
+                  :           :     :        :- * Filter (3)
+                  :           :     :        :  +- * ColumnarToRow (2)
+                  :           :     :        :     +- Scan parquet default.item (1)
+                  :           :     :        +- BroadcastExchange (11)
+                  :           :     :           +- * Filter (10)
+                  :           :     :              +- * HashAggregate (9)
+                  :           :     :                 +- Exchange (8)
+                  :           :     :                    +- * HashAggregate (7)
+                  :           :     :                       +- * Filter (6)
+                  :           :     :                          +- * ColumnarToRow (5)
+                  :           :     :                             +- Scan parquet default.item (4)
+                  :           :     +- * Filter (17)
+                  :           :        +- * ColumnarToRow (16)
+                  :           :           +- Scan parquet default.store_sales (15)
                   :           +- ReusedExchange (20)
                   +- * Sort (38)
                      +- Exchange (37)
@@ -46,116 +46,116 @@ TakeOrderedAndProject (45)
                                           +- Scan parquet default.customer (30)
 
 
-(1) Scan parquet default.store_sales
-Output [3]: [ss_item_sk#1, ss_customer_sk#2, ss_sold_date_sk#3]
-Batched: true
-Location: InMemoryFileIndex []
-PartitionFilters: [isnotnull(ss_sold_date_sk#3), dynamicpruningexpression(ss_sold_date_sk#3 IN dynamicpruning#4)]
-PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)]
-ReadSchema: struct<ss_item_sk:int,ss_customer_sk:int>
-
-(2) ColumnarToRow [codegen id : 5]
-Input [3]: [ss_item_sk#1, ss_customer_sk#2, ss_sold_date_sk#3]
-
-(3) Filter [codegen id : 5]
-Input [3]: [ss_item_sk#1, ss_customer_sk#2, ss_sold_date_sk#3]
-Condition : (isnotnull(ss_customer_sk#2) AND isnotnull(ss_item_sk#1))
-
-(4) Scan parquet default.item
-Output [3]: [i_item_sk#5, i_current_price#6, i_category#7]
+(1) Scan parquet default.item
+Output [3]: [i_item_sk#1, i_current_price#2, i_category#3]
 Batched: true
 Location [not included in comparison]/{warehouse_dir}/item]
-PushedFilters: [IsNotNull(i_current_price), IsNotNull(i_item_sk)]
+PushedFilters: [IsNotNull(i_current_price), IsNotNull(i_category), IsNotNull(i_item_sk)]
 ReadSchema: struct<i_item_sk:int,i_current_price:decimal(7,2),i_category:string>
 
-(5) ColumnarToRow [codegen id : 3]
-Input [3]: [i_item_sk#5, i_current_price#6, i_category#7]
+(2) ColumnarToRow [codegen id : 3]
+Input [3]: [i_item_sk#1, i_current_price#2, i_category#3]
 
-(6) Filter [codegen id : 3]
-Input [3]: [i_item_sk#5, i_current_price#6, i_category#7]
-Condition : (isnotnull(i_current_price#6) AND isnotnull(i_item_sk#5))
+(3) Filter [codegen id : 3]
+Input [3]: [i_item_sk#1, i_current_price#2, i_category#3]
+Condition : ((isnotnull(i_current_price#2) AND isnotnull(i_category#3)) AND isnotnull(i_item_sk#1))
 
-(7) Scan parquet default.item
-Output [2]: [i_current_price#8, i_category#9]
+(4) Scan parquet default.item
+Output [2]: [i_current_price#4, i_category#5]
 Batched: true
 Location [not included in comparison]/{warehouse_dir}/item]
 PushedFilters: [IsNotNull(i_category)]
 ReadSchema: struct<i_current_price:decimal(7,2),i_category:string>
 
-(8) ColumnarToRow [codegen id : 1]
-Input [2]: [i_current_price#8, i_category#9]
-
-(9) Filter [codegen id : 1]
-Input [2]: [i_current_price#8, i_category#9]
-Condition : isnotnull(i_category#9)
-
-(10) HashAggregate [codegen id : 1]
-Input [2]: [i_current_price#8, i_category#9]
-Keys [1]: [i_category#9]
-Functions [1]: [partial_avg(UnscaledValue(i_current_price#8))]
-Aggregate Attributes [2]: [sum#10, count#11]
-Results [3]: [i_category#9, sum#12, count#13]
-
-(11) Exchange
-Input [3]: [i_category#9, sum#12, count#13]
-Arguments: hashpartitioning(i_category#9, 5), ENSURE_REQUIREMENTS, [plan_id=1]
-
-(12) HashAggregate [codegen id : 2]
-Input [3]: [i_category#9, sum#12, count#13]
-Keys [1]: [i_category#9]
-Functions [1]: [avg(UnscaledValue(i_current_price#8))]
-Aggregate Attributes [1]: [avg(UnscaledValue(i_current_price#8))#14]
-Results [2]: [cast((avg(UnscaledValue(i_current_price#8))#14 / 100.0) as decimal(11,6)) AS avg(i_current_price)#15, i_category#9]
-
-(13) BroadcastExchange
-Input [2]: [avg(i_current_price)#15, i_category#9]
+(5) ColumnarToRow [codegen id : 1]
+Input [2]: [i_current_price#4, i_category#5]
+
+(6) Filter [codegen id : 1]
+Input [2]: [i_current_price#4, i_category#5]
+Condition : isnotnull(i_category#5)
+
+(7) HashAggregate [codegen id : 1]
+Input [2]: [i_current_price#4, i_category#5]
+Keys [1]: [i_category#5]
+Functions [1]: [partial_avg(UnscaledValue(i_current_price#4))]
+Aggregate Attributes [2]: [sum#6, count#7]
+Results [3]: [i_category#5, sum#8, count#9]
+
+(8) Exchange
+Input [3]: [i_category#5, sum#8, count#9]
+Arguments: hashpartitioning(i_category#5, 5), ENSURE_REQUIREMENTS, [plan_id=1]
+
+(9) HashAggregate [codegen id : 2]
+Input [3]: [i_category#5, sum#8, count#9]
+Keys [1]: [i_category#5]
+Functions [1]: [avg(UnscaledValue(i_current_price#4))]
+Aggregate Attributes [1]: [avg(UnscaledValue(i_current_price#4))#10]
+Results [2]: [cast((avg(UnscaledValue(i_current_price#4))#10 / 100.0) as decimal(11,6)) AS avg(i_current_price)#11, i_category#5]
+
+(10) Filter [codegen id : 2]
+Input [2]: [avg(i_current_price)#11, i_category#5]
+Condition : isnotnull(avg(i_current_price)#11)
+
+(11) BroadcastExchange
+Input [2]: [avg(i_current_price)#11, i_category#5]
 Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [plan_id=2]
 
-(14) BroadcastHashJoin [codegen id : 3]
-Left keys [1]: [i_category#7]
-Right keys [1]: [i_category#9]
-Join condition: None
-
-(15) Filter [codegen id : 3]
-Input [5]: [i_item_sk#5, i_current_price#6, i_category#7, avg(i_current_price)#15, i_category#9]
-Condition : (cast(i_current_price#6 as decimal(14,7)) > CheckOverflow((1.200000 * promote_precision(avg(i_current_price)#15)), DecimalType(14,7)))
+(12) BroadcastHashJoin [codegen id : 3]
+Left keys [1]: [i_category#3]
+Right keys [1]: [i_category#5]
+Join condition: (cast(i_current_price#2 as decimal(14,7)) > (1.2 * avg(i_current_price)#11))
 
-(16) Project [codegen id : 3]
-Output [1]: [i_item_sk#5]
-Input [5]: [i_item_sk#5, i_current_price#6, i_category#7, avg(i_current_price)#15, i_category#9]
+(13) Project [codegen id : 3]
+Output [1]: [i_item_sk#1]
+Input [5]: [i_item_sk#1, i_current_price#2, i_category#3, avg(i_current_price)#11, i_category#5]
 
-(17) BroadcastExchange
-Input [1]: [i_item_sk#5]
+(14) BroadcastExchange
+Input [1]: [i_item_sk#1]
 Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, true] as bigint)),false), [plan_id=3]
 
+(15) Scan parquet default.store_sales
+Output [3]: [ss_item_sk#12, ss_customer_sk#13, ss_sold_date_sk#14]
+Batched: true
+Location: InMemoryFileIndex []
+PartitionFilters: [isnotnull(ss_sold_date_sk#14), dynamicpruningexpression(ss_sold_date_sk#14 IN dynamicpruning#15)]
+PushedFilters: [IsNotNull(ss_customer_sk), IsNotNull(ss_item_sk)]
+ReadSchema: struct<ss_item_sk:int,ss_customer_sk:int>
+
+(16) ColumnarToRow
+Input [3]: [ss_item_sk#12, ss_customer_sk#13, ss_sold_date_sk#14]
+
+(17) Filter
+Input [3]: [ss_item_sk#12, ss_customer_sk#13, ss_sold_date_sk#14]
+Condition : (isnotnull(ss_customer_sk#13) AND isnotnull(ss_item_sk#12))
+
 (18) BroadcastHashJoin [codegen id : 5]
-Left keys [1]: [ss_item_sk#1]
-Right keys [1]: [i_item_sk#5]
+Left keys [1]: [i_item_sk#1]
+Right keys [1]: [ss_item_sk#12]
 Join condition: None
 
 (19) Project [codegen id : 5]
-Output [2]: [ss_customer_sk#2, ss_sold_date_sk#3]
-Input [4]: [ss_item_sk#1, ss_customer_sk#2, ss_sold_date_sk#3, i_item_sk#5]
+Output [2]: [ss_customer_sk#13, ss_sold_date_sk#14]
+Input [4]: [i_item_sk#1, ss_item_sk#12, ss_customer_sk#13, ss_sold_date_sk#14]
 
 (20) ReusedExchange [Reuses operator id: 50]
 Output [1]: [d_date_sk#16]
 
 (21) BroadcastHashJoin [codegen id : 5]
-Left keys [1]: [ss_sold_date_sk#3]
+Left keys [1]: [ss_sold_date_sk#14]
 Right keys [1]: [d_date_sk#16]
 Join condition: None
 
 (22) Project [codegen id : 5]
-Output [1]: [ss_customer_sk#2]
-Input [3]: [ss_customer_sk#2, ss_sold_date_sk#3, d_date_sk#16]
+Output [1]: [ss_customer_sk#13]
+Input [3]: [ss_customer_sk#13, ss_sold_date_sk#14, d_date_sk#16]
 
 (23) Exchange
-Input [1]: [ss_customer_sk#2]
-Arguments: hashpartitioning(ss_customer_sk#2, 5), ENSURE_REQUIREMENTS, [plan_id=4]
+Input [1]: [ss_customer_sk#13]
+Arguments: hashpartitioning(ss_customer_sk#13, 5), ENSURE_REQUIREMENTS, [plan_id=4]
 
 (24) Sort [codegen id : 6]
-Input [1]: [ss_customer_sk#2]
-Arguments: [ss_customer_sk#2 ASC NULLS FIRST], false, 0
+Input [1]: [ss_customer_sk#13]
+Arguments: [ss_customer_sk#13 ASC NULLS FIRST], false, 0
 
 (25) Scan parquet default.customer_address
 Output [2]: [ca_address_sk#17, ca_state#18]
@@ -219,13 +219,13 @@ Input [2]: [ca_state#18, c_customer_sk#19]
 Arguments: [c_customer_sk#19 ASC NULLS FIRST], false, 0
 
 (39) SortMergeJoin [codegen id : 13]
-Left keys [1]: [ss_customer_sk#2]
+Left keys [1]: [ss_customer_sk#13]
 Right keys [1]: [c_customer_sk#19]
 Join condition: None
 
 (40) Project [codegen id : 13]
 Output [1]: [ca_state#18]
-Input [3]: [ss_customer_sk#2, ca_state#18, c_customer_sk#19]
+Input [3]: [ss_customer_sk#13, ca_state#18, c_customer_sk#19]
 
 (41) HashAggregate [codegen id : 13]
 Input [1]: [ca_state#18]
@@ -255,7 +255,7 @@ Arguments: 100, [cnt#25 ASC NULLS FIRST, ca_state#18 ASC NULLS FIRST], [state#24
 
 ===== Subqueries =====
 
-Subquery:1 Hosting operator id = 1 Hosting Expression = ss_sold_date_sk#3 IN dynamicpruning#4
+Subquery:1 Hosting operator id = 15 Hosting Expression = ss_sold_date_sk#14 IN dynamicpruning#15
 BroadcastExchange (50)
 +- * Project (49)
    +- * Filter (48)
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/simplified.txt
index 700706baf86..2b9aa44c96c 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/simplified.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6.sf100/simplified.txt
@@ -17,13 +17,35 @@ TakeOrderedAndProject [cnt,ca_state,state]
                                 Project [ss_customer_sk]
                                   BroadcastHashJoin [ss_sold_date_sk,d_date_sk]
                                     Project [ss_customer_sk,ss_sold_date_sk]
-                                      BroadcastHashJoin [ss_item_sk,i_item_sk]
+                                      BroadcastHashJoin [i_item_sk,ss_item_sk]
+                                        InputAdapter
+                                          BroadcastExchange #3
+                                            WholeStageCodegen (3)
+                                              Project [i_item_sk]
+                                                BroadcastHashJoin [i_category,i_category,i_current_price,avg(i_current_price)]
+                                                  Filter [i_current_price,i_category,i_item_sk]
+                                                    ColumnarToRow
+                                                      InputAdapter
+                                                        Scan parquet default.item [i_item_sk,i_current_price,i_category]
+                                                  InputAdapter
+                                                    BroadcastExchange #4
+                                                      WholeStageCodegen (2)
+                                                        Filter [avg(i_current_price)]
+                                                          HashAggregate [i_category,sum,count] [avg(UnscaledValue(i_current_price)),avg(i_current_price),sum,count]
+                                                            InputAdapter
+                                                              Exchange [i_category] #5
+                                                                WholeStageCodegen (1)
+                                                                  HashAggregate [i_category,i_current_price] [sum,count,sum,count]
+                                                                    Filter [i_category]
+                                                                      ColumnarToRow
+                                                                        InputAdapter
+                                                                          Scan parquet default.item [i_current_price,i_category]
                                         Filter [ss_customer_sk,ss_item_sk]
                                           ColumnarToRow
                                             InputAdapter
                                               Scan parquet default.store_sales [ss_item_sk,ss_customer_sk,ss_sold_date_sk]
                                                 SubqueryBroadcast [d_date_sk] #1
-                                                  BroadcastExchange #3
+                                                  BroadcastExchange #6
                                                     WholeStageCodegen (1)
                                                       Project [d_date_sk]
                                                         Filter [d_month_seq,d_date_sk]
@@ -31,7 +53,7 @@ TakeOrderedAndProject [cnt,ca_state,state]
                                                             WholeStageCodegen (2)
                                                               HashAggregate [d_month_seq]
                                                                 InputAdapter
-                                                                  Exchange [d_month_seq] #4
+                                                                  Exchange [d_month_seq] #7
                                                                     WholeStageCodegen (1)
                                                                       HashAggregate [d_month_seq]
                                                                         Project [d_month_seq]
@@ -42,30 +64,8 @@ TakeOrderedAndProject [cnt,ca_state,state]
                                                           ColumnarToRow
                                                             InputAdapter
                                                               Scan parquet default.date_dim [d_date_sk,d_month_seq]
-                                        InputAdapter
-                                          BroadcastExchange #5
-                                            WholeStageCodegen (3)
-                                              Project [i_item_sk]
-                                                Filter [i_current_price,avg(i_current_price)]
-                                                  BroadcastHashJoin [i_category,i_category]
-                                                    Filter [i_current_price,i_item_sk]
-                                                      ColumnarToRow
-                                                        InputAdapter
-                                                          Scan parquet default.item [i_item_sk,i_current_price,i_category]
-                                                    InputAdapter
-                                                      BroadcastExchange #6
-                                                        WholeStageCodegen (2)
-                                                          HashAggregate [i_category,sum,count] [avg(UnscaledValue(i_current_price)),avg(i_current_price),sum,count]
-                                                            InputAdapter
-                                                              Exchange [i_category] #7
-                                                                WholeStageCodegen (1)
-                                                                  HashAggregate [i_category,i_current_price] [sum,count,sum,count]
-                                                                    Filter [i_category]
-                                                                      ColumnarToRow
-                                                                        InputAdapter
-                                                                          Scan parquet default.item [i_current_price,i_category]
                                     InputAdapter
-                                      ReusedExchange [d_date_sk] #3
+                                      ReusedExchange [d_date_sk] #6
                     InputAdapter
                       WholeStageCodegen (12)
                         Sort [c_customer_sk]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/explain.txt
index c415966f85b..fc320bc1f78 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/explain.txt
@@ -26,12 +26,12 @@ TakeOrderedAndProject (39)
                   :     +- ReusedExchange (16)
                   +- BroadcastExchange (32)
                      +- * Project (31)
-                        +- * Filter (30)
-                           +- * BroadcastHashJoin LeftOuter BuildRight (29)
-                              :- * Filter (21)
-                              :  +- * ColumnarToRow (20)
-                              :     +- Scan parquet default.item (19)
-                              +- BroadcastExchange (28)
+                        +- * BroadcastHashJoin Inner BuildRight (30)
+                           :- * Filter (21)
+                           :  +- * ColumnarToRow (20)
+                           :     +- Scan parquet default.item (19)
+                           +- BroadcastExchange (29)
+                              +- * Filter (28)
                                  +- * HashAggregate (27)
                                     +- Exchange (26)
                                        +- * HashAggregate (25)
@@ -125,7 +125,7 @@ Input [4]: [ca_state#2, ss_item_sk#5, ss_sold_date_sk#7, d_date_sk#9]
 Output [3]: [i_item_sk#10, i_current_price#11, i_category#12]
 Batched: true
 Location [not included in comparison]/{warehouse_dir}/item]
-PushedFilters: [IsNotNull(i_current_price), IsNotNull(i_item_sk)]
+PushedFilters: [IsNotNull(i_current_price), IsNotNull(i_category), IsNotNull(i_item_sk)]
 ReadSchema: struct<i_item_sk:int,i_current_price:decimal(7,2),i_category:string>
 
 (20) ColumnarToRow [codegen id : 6]
@@ -133,7 +133,7 @@ Input [3]: [i_item_sk#10, i_current_price#11, i_category#12]
 
 (21) Filter [codegen id : 6]
 Input [3]: [i_item_sk#10, i_current_price#11, i_category#12]
-Condition : (isnotnull(i_current_price#11) AND isnotnull(i_item_sk#10))
+Condition : ((isnotnull(i_current_price#11) AND isnotnull(i_category#12)) AND isnotnull(i_item_sk#10))
 
 (22) Scan parquet default.item
 Output [2]: [i_current_price#13, i_category#14]
@@ -167,18 +167,18 @@ Functions [1]: [avg(UnscaledValue(i_current_price#13))]
 Aggregate Attributes [1]: [avg(UnscaledValue(i_current_price#13))#19]
 Results [2]: [cast((avg(UnscaledValue(i_current_price#13))#19 / 100.0) as decimal(11,6)) AS avg(i_current_price)#20, i_category#14]
 
-(28) BroadcastExchange
+(28) Filter [codegen id : 5]
+Input [2]: [avg(i_current_price)#20, i_category#14]
+Condition : isnotnull(avg(i_current_price)#20)
+
+(29) BroadcastExchange
 Input [2]: [avg(i_current_price)#20, i_category#14]
 Arguments: HashedRelationBroadcastMode(List(input[1, string, true]),false), [plan_id=4]
 
-(29) BroadcastHashJoin [codegen id : 6]
+(30) BroadcastHashJoin [codegen id : 6]
 Left keys [1]: [i_category#12]
 Right keys [1]: [i_category#14]
-Join condition: None
-
-(30) Filter [codegen id : 6]
-Input [5]: [i_item_sk#10, i_current_price#11, i_category#12, avg(i_current_price)#20, i_category#14]
-Condition : (cast(i_current_price#11 as decimal(14,7)) > CheckOverflow((1.200000 * promote_precision(avg(i_current_price)#20)), DecimalType(14,7)))
+Join condition: (cast(i_current_price#11 as decimal(14,7)) > (1.2 * avg(i_current_price)#20))
 
 (31) Project [codegen id : 6]
 Output [1]: [i_item_sk#10]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt
index afe91d3ab46..83d6c435fa5 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q6/simplified.txt
@@ -58,15 +58,15 @@ TakeOrderedAndProject [cnt,ca_state,state]
                       BroadcastExchange #6
                         WholeStageCodegen (6)
                           Project [i_item_sk]
-                            Filter [i_current_price,avg(i_current_price)]
-                              BroadcastHashJoin [i_category,i_category]
-                                Filter [i_current_price,i_item_sk]
-                                  ColumnarToRow
-                                    InputAdapter
-                                      Scan parquet default.item [i_item_sk,i_current_price,i_category]
-                                InputAdapter
-                                  BroadcastExchange #7
-                                    WholeStageCodegen (5)
+                            BroadcastHashJoin [i_category,i_category,i_current_price,avg(i_current_price)]
+                              Filter [i_current_price,i_category,i_item_sk]
+                                ColumnarToRow
+                                  InputAdapter
+                                    Scan parquet default.item [i_item_sk,i_current_price,i_category]
+                              InputAdapter
+                                BroadcastExchange #7
+                                  WholeStageCodegen (5)
+                                    Filter [avg(i_current_price)]
                                       HashAggregate [i_category,sum,count] [avg(UnscaledValue(i_current_price)),avg(i_current_price),sum,count]
                                         InputAdapter
                                           Exchange [i_category] #8
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.sf100/explain.txt
index 4dbc69ad5d9..fb33c560d91 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.sf100/explain.txt
@@ -332,7 +332,7 @@ Input [8]: [cs_item_sk#19, cs_order_number#20, cs_ext_list_price#21, cr_item_sk#
 (28) HashAggregate [codegen id : 9]
 Input [5]: [cs_item_sk#19, cs_ext_list_price#21, cr_refunded_cash#25, cr_reversed_charge#26, cr_store_credit#27]
 Keys [1]: [cs_item_sk#19]
-Functions [2]: [partial_sum(UnscaledValue(cs_ext_list_price#21)), partial_sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#25 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#26 as decimal(8,2)))), DecimalType(8,2)) as decimal(9,2))) + promote_precision(cast(cr_store_credit#27 as decimal(9,2)))), DecimalType(9,2)))]
+Functions [2]: [partial_sum(UnscaledValue(cs_ext_list_price#21)), partial_sum(((cr_refunded_cash#25 + cr_reversed_charge#26) + cr_store_credit#27))]
 Aggregate Attributes [3]: [sum#29, sum#30, isEmpty#31]
 Results [4]: [cs_item_sk#19, sum#32, sum#33, isEmpty#34]
 
@@ -343,13 +343,13 @@ Arguments: hashpartitioning(cs_item_sk#19, 5), ENSURE_REQUIREMENTS, [plan_id=5]
 (30) HashAggregate [codegen id : 10]
 Input [4]: [cs_item_sk#19, sum#32, sum#33, isEmpty#34]
 Keys [1]: [cs_item_sk#19]
-Functions [2]: [sum(UnscaledValue(cs_ext_list_price#21)), sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#25 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#26 as decimal(8,2)))), DecimalType(8,2)) as decimal(9,2))) + promote_precision(cast(cr_store_credit#27 as decimal(9,2)))), DecimalType(9,2)))]
-Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_list_price#21))#35, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#25 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#26 as decimal(8,2)))), DecimalType(8,2)) as decimal(9,2))) + promote_precision(cast(cr_store_credit#27 as decimal(9,2)))), DecimalType(9,2)))#36]
-Results [3]: [cs_item_sk#19, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#21))#35,17,2) AS sale#37, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#25 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#26 as decimal(8,2)))), DecimalType(8,2)) as decimal(9,2))) + promote_precision(cast(cr_store_credit#27 as decimal(9,2)))), DecimalType(9,2)))#36 AS refund#38]
+Functions [2]: [sum(UnscaledValue(cs_ext_list_price#21)), sum(((cr_refunded_cash#25 + cr_reversed_charge#26) + cr_store_credit#27))]
+Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_list_price#21))#35, sum(((cr_refunded_cash#25 + cr_reversed_charge#26) + cr_store_credit#27))#36]
+Results [3]: [cs_item_sk#19, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#21))#35,17,2) AS sale#37, sum(((cr_refunded_cash#25 + cr_reversed_charge#26) + cr_store_credit#27))#36 AS refund#38]
 
 (31) Filter [codegen id : 10]
 Input [3]: [cs_item_sk#19, sale#37, refund#38]
-Condition : (isnotnull(sale#37) AND (cast(sale#37 as decimal(21,2)) > CheckOverflow((2.00 * promote_precision(refund#38)), DecimalType(21,2))))
+Condition : ((isnotnull(sale#37) AND isnotnull(refund#38)) AND (cast(sale#37 as decimal(21,2)) > (2 * refund#38)))
 
 (32) Project [codegen id : 10]
 Output [1]: [cs_item_sk#19]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.sf100/simplified.txt
index 4cbab100ef0..3364af91d46 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.sf100/simplified.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64.sf100/simplified.txt
@@ -123,7 +123,7 @@ WholeStageCodegen (88)
                                                                                                                                                                     WholeStageCodegen (10)
                                                                                                                                                                       Project [cs_item_sk]
                                                                                                                                                                         Filter [sale,refund]
-                                                                                                                                                                          HashAggregate [cs_item_sk,sum,sum,isEmpty] [sum(UnscaledValue(cs_ext_list_price)),sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash as decimal(8,2))) + promote_precision(cast(cr_reversed_charge as decimal(8,2)))), DecimalType(8,2)) as decimal(9,2))) + promote_precision(cast( [...]
+                                                                                                                                                                          HashAggregate [cs_item_sk,sum,sum,isEmpty] [sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit)),sale,refund,sum,sum,isEmpty]
                                                                                                                                                                             InputAdapter
                                                                                                                                                                               Exchange [cs_item_sk] #14
                                                                                                                                                                                 WholeStageCodegen (9)
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt
index 28affb7b36a..d65af2c07ea 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/explain.txt
@@ -302,7 +302,7 @@ Input [8]: [cs_item_sk#17, cs_order_number#18, cs_ext_list_price#19, cr_item_sk#
 (27) HashAggregate [codegen id : 8]
 Input [5]: [cs_item_sk#17, cs_ext_list_price#19, cr_refunded_cash#23, cr_reversed_charge#24, cr_store_credit#25]
 Keys [1]: [cs_item_sk#17]
-Functions [2]: [partial_sum(UnscaledValue(cs_ext_list_price#19)), partial_sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#23 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#24 as decimal(8,2)))), DecimalType(8,2)) as decimal(9,2))) + promote_precision(cast(cr_store_credit#25 as decimal(9,2)))), DecimalType(9,2)))]
+Functions [2]: [partial_sum(UnscaledValue(cs_ext_list_price#19)), partial_sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))]
 Aggregate Attributes [3]: [sum#27, sum#28, isEmpty#29]
 Results [4]: [cs_item_sk#17, sum#30, sum#31, isEmpty#32]
 
@@ -313,13 +313,13 @@ Arguments: hashpartitioning(cs_item_sk#17, 5), ENSURE_REQUIREMENTS, [plan_id=5]
 (29) HashAggregate [codegen id : 9]
 Input [4]: [cs_item_sk#17, sum#30, sum#31, isEmpty#32]
 Keys [1]: [cs_item_sk#17]
-Functions [2]: [sum(UnscaledValue(cs_ext_list_price#19)), sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#23 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#24 as decimal(8,2)))), DecimalType(8,2)) as decimal(9,2))) + promote_precision(cast(cr_store_credit#25 as decimal(9,2)))), DecimalType(9,2)))]
-Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_list_price#19))#33, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#23 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#24 as decimal(8,2)))), DecimalType(8,2)) as decimal(9,2))) + promote_precision(cast(cr_store_credit#25 as decimal(9,2)))), DecimalType(9,2)))#34]
-Results [3]: [cs_item_sk#17, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#19))#33,17,2) AS sale#35, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#23 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#24 as decimal(8,2)))), DecimalType(8,2)) as decimal(9,2))) + promote_precision(cast(cr_store_credit#25 as decimal(9,2)))), DecimalType(9,2)))#34 AS refund#36]
+Functions [2]: [sum(UnscaledValue(cs_ext_list_price#19)), sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))]
+Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_list_price#19))#33, sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))#34]
+Results [3]: [cs_item_sk#17, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#19))#33,17,2) AS sale#35, sum(((cr_refunded_cash#23 + cr_reversed_charge#24) + cr_store_credit#25))#34 AS refund#36]
 
 (30) Filter [codegen id : 9]
 Input [3]: [cs_item_sk#17, sale#35, refund#36]
-Condition : (isnotnull(sale#35) AND (cast(sale#35 as decimal(21,2)) > CheckOverflow((2.00 * promote_precision(refund#36)), DecimalType(21,2))))
+Condition : ((isnotnull(sale#35) AND isnotnull(refund#36)) AND (cast(sale#35 as decimal(21,2)) > (2 * refund#36)))
 
 (31) Project [codegen id : 9]
 Output [1]: [cs_item_sk#17]
@@ -739,13 +739,13 @@ Output [4]: [cs_item_sk#122, sum#123, sum#124, isEmpty#125]
 (125) HashAggregate [codegen id : 35]
 Input [4]: [cs_item_sk#122, sum#123, sum#124, isEmpty#125]
 Keys [1]: [cs_item_sk#122]
-Functions [2]: [sum(UnscaledValue(cs_ext_list_price#126)), sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#127 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#128 as decimal(8,2)))), DecimalType(8,2)) as decimal(9,2))) + promote_precision(cast(cr_store_credit#129 as decimal(9,2)))), DecimalType(9,2)))]
-Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_list_price#126))#33, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#127 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#128 as decimal(8,2)))), DecimalType(8,2)) as decimal(9,2))) + promote_precision(cast(cr_store_credit#129 as decimal(9,2)))), DecimalType(9,2)))#34]
-Results [3]: [cs_item_sk#122, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#126))#33,17,2) AS sale#35, sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash#127 as decimal(8,2))) + promote_precision(cast(cr_reversed_charge#128 as decimal(8,2)))), DecimalType(8,2)) as decimal(9,2))) + promote_precision(cast(cr_store_credit#129 as decimal(9,2)))), DecimalType(9,2)))#34 AS refund#36]
+Functions [2]: [sum(UnscaledValue(cs_ext_list_price#126)), sum(((cr_refunded_cash#127 + cr_reversed_charge#128) + cr_store_credit#129))]
+Aggregate Attributes [2]: [sum(UnscaledValue(cs_ext_list_price#126))#33, sum(((cr_refunded_cash#127 + cr_reversed_charge#128) + cr_store_credit#129))#34]
+Results [3]: [cs_item_sk#122, MakeDecimal(sum(UnscaledValue(cs_ext_list_price#126))#33,17,2) AS sale#35, sum(((cr_refunded_cash#127 + cr_reversed_charge#128) + cr_store_credit#129))#34 AS refund#36]
 
 (126) Filter [codegen id : 35]
 Input [3]: [cs_item_sk#122, sale#35, refund#36]
-Condition : (isnotnull(sale#35) AND (cast(sale#35 as decimal(21,2)) > CheckOverflow((2.00 * promote_precision(refund#36)), DecimalType(21,2))))
+Condition : ((isnotnull(sale#35) AND isnotnull(refund#36)) AND (cast(sale#35 as decimal(21,2)) > (2 * refund#36)))
 
 (127) Project [codegen id : 35]
 Output [1]: [cs_item_sk#122]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/simplified.txt
index 859101af5ba..610ca922278 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/simplified.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q64/simplified.txt
@@ -77,7 +77,7 @@ WholeStageCodegen (54)
                                                                                                   Sort [cs_item_sk]
                                                                                                     Project [cs_item_sk]
                                                                                                       Filter [sale,refund]
-                                                                                                        HashAggregate [cs_item_sk,sum,sum,isEmpty] [sum(UnscaledValue(cs_ext_list_price)),sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash as decimal(8,2))) + promote_precision(cast(cr_reversed_charge as decimal(8,2)))), DecimalType(8,2)) as decimal(9,2))) + promote_precision(cast(cr_store_credit as decimal(9,2)))), DecimalType(9,2))),sale,refund [...]
+                                                                                                        HashAggregate [cs_item_sk,sum,sum,isEmpty] [sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit)),sale,refund,sum,sum,isEmpty]
                                                                                                           InputAdapter
                                                                                                             Exchange [cs_item_sk] #6
                                                                                                               WholeStageCodegen (8)
@@ -254,7 +254,7 @@ WholeStageCodegen (54)
                                                                                                   Sort [cs_item_sk]
                                                                                                     Project [cs_item_sk]
                                                                                                       Filter [sale,refund]
-                                                                                                        HashAggregate [cs_item_sk,sum,sum,isEmpty] [sum(UnscaledValue(cs_ext_list_price)),sum(CheckOverflow((promote_precision(cast(CheckOverflow((promote_precision(cast(cr_refunded_cash as decimal(8,2))) + promote_precision(cast(cr_reversed_charge as decimal(8,2)))), DecimalType(8,2)) as decimal(9,2))) + promote_precision(cast(cr_store_credit as decimal(9,2)))), DecimalType(9,2))),sale,refund [...]
+                                                                                                        HashAggregate [cs_item_sk,sum,sum,isEmpty] [sum(UnscaledValue(cs_ext_list_price)),sum(((cr_refunded_cash + cr_reversed_charge) + cr_store_credit)),sale,refund,sum,sum,isEmpty]
                                                                                                           InputAdapter
                                                                                                             ReusedExchange [cs_item_sk,sum,sum,isEmpty] #6
                                                                                           InputAdapter
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.sf100/explain.txt
index 537414941ac..8aa86dfe644 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.sf100/explain.txt
@@ -167,7 +167,7 @@ Input [12]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d
 (22) HashAggregate [codegen id : 7]
 Input [10]: [ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#12, i_brand#14, i_class#15, i_category#16, i_product_name#17]
 Keys [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12]
-Functions [1]: [partial_sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(ss_quantity#3 as decimal(12,2)))), DecimalType(18,2)), 0.00))]
+Functions [1]: [partial_sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))]
 Aggregate Attributes [2]: [sum#18, isEmpty#19]
 Results [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#20, isEmpty#21]
 
@@ -178,9 +178,9 @@ Arguments: hashpartitioning(i_category#16, i_class#15, i_brand#14, i_product_nam
 (24) HashAggregate [codegen id : 8]
 Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#20, isEmpty#21]
 Keys [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12]
-Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(ss_quantity#3 as decimal(12,2)))), DecimalType(18,2)), 0.00))]
-Aggregate Attributes [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(ss_quantity#3 as decimal(12,2)))), DecimalType(18,2)), 0.00))#22]
-Results [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, cast(sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(ss_quantity#3 as decimal(12,2)))), DecimalType(18,2)), 0.00))#22 as decimal(38,2)) AS sumsales#23]
+Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))]
+Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22]
+Results [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, cast(sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22 as decimal(38,2)) AS sumsales#23]
 
 (25) ReusedExchange [Reuses operator id: 23]
 Output [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#24, isEmpty#25]
@@ -188,9 +188,9 @@ Output [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8
 (26) HashAggregate [codegen id : 16]
 Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#24, isEmpty#25]
 Keys [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12]
-Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(ss_quantity#3 as decimal(12,2)))), DecimalType(18,2)), 0.00))]
-Aggregate Attributes [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(ss_quantity#3 as decimal(12,2)))), DecimalType(18,2)), 0.00))#22]
-Results [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(ss_quantity#3 as decimal(12,2)))), DecimalType(18,2)), 0.00))#22 AS sumsales#26]
+Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))]
+Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22]
+Results [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22 AS sumsales#26]
 
 (27) HashAggregate [codegen id : 16]
 Input [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, sumsales#26]
@@ -216,9 +216,9 @@ Output [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8
 (31) HashAggregate [codegen id : 25]
 Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#34, isEmpty#35]
 Keys [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12]
-Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(ss_quantity#3 as decimal(12,2)))), DecimalType(18,2)), 0.00))]
-Aggregate Attributes [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(ss_quantity#3 as decimal(12,2)))), DecimalType(18,2)), 0.00))#22]
-Results [7]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(ss_quantity#3 as decimal(12,2)))), DecimalType(18,2)), 0.00))#22 AS sumsales#26]
+Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))]
+Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22]
+Results [7]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22 AS sumsales#26]
 
 (32) HashAggregate [codegen id : 25]
 Input [7]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, sumsales#26]
@@ -244,9 +244,9 @@ Output [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8
 (36) HashAggregate [codegen id : 34]
 Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#44, isEmpty#45]
 Keys [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12]
-Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(ss_quantity#3 as decimal(12,2)))), DecimalType(18,2)), 0.00))]
-Aggregate Attributes [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(ss_quantity#3 as decimal(12,2)))), DecimalType(18,2)), 0.00))#22]
-Results [6]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(ss_quantity#3 as decimal(12,2)))), DecimalType(18,2)), 0.00))#22 AS sumsales#26]
+Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))]
+Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22]
+Results [6]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22 AS sumsales#26]
 
 (37) HashAggregate [codegen id : 34]
 Input [6]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, sumsales#26]
@@ -272,9 +272,9 @@ Output [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8
 (41) HashAggregate [codegen id : 43]
 Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#55, isEmpty#56]
 Keys [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12]
-Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(ss_quantity#3 as decimal(12,2)))), DecimalType(18,2)), 0.00))]
-Aggregate Attributes [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(ss_quantity#3 as decimal(12,2)))), DecimalType(18,2)), 0.00))#22]
-Results [5]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(ss_quantity#3 as decimal(12,2)))), DecimalType(18,2)), 0.00))#22 AS sumsales#26]
+Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))]
+Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22]
+Results [5]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22 AS sumsales#26]
 
 (42) HashAggregate [codegen id : 43]
 Input [5]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, sumsales#26]
@@ -300,9 +300,9 @@ Output [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8
 (46) HashAggregate [codegen id : 52]
 Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#67, isEmpty#68]
 Keys [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12]
-Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(ss_quantity#3 as decimal(12,2)))), DecimalType(18,2)), 0.00))]
-Aggregate Attributes [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(ss_quantity#3 as decimal(12,2)))), DecimalType(18,2)), 0.00))#22]
-Results [4]: [i_category#16, i_class#15, i_brand#14, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(ss_quantity#3 as decimal(12,2)))), DecimalType(18,2)), 0.00))#22 AS sumsales#26]
+Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))]
+Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22]
+Results [4]: [i_category#16, i_class#15, i_brand#14, sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22 AS sumsales#26]
 
 (47) HashAggregate [codegen id : 52]
 Input [4]: [i_category#16, i_class#15, i_brand#14, sumsales#26]
@@ -328,9 +328,9 @@ Output [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8
 (51) HashAggregate [codegen id : 61]
 Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#80, isEmpty#81]
 Keys [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12]
-Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(ss_quantity#3 as decimal(12,2)))), DecimalType(18,2)), 0.00))]
-Aggregate Attributes [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(ss_quantity#3 as decimal(12,2)))), DecimalType(18,2)), 0.00))#22]
-Results [3]: [i_category#16, i_class#15, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(ss_quantity#3 as decimal(12,2)))), DecimalType(18,2)), 0.00))#22 AS sumsales#26]
+Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))]
+Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22]
+Results [3]: [i_category#16, i_class#15, sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22 AS sumsales#26]
 
 (52) HashAggregate [codegen id : 61]
 Input [3]: [i_category#16, i_class#15, sumsales#26]
@@ -356,9 +356,9 @@ Output [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8
 (56) HashAggregate [codegen id : 70]
 Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#94, isEmpty#95]
 Keys [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12]
-Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(ss_quantity#3 as decimal(12,2)))), DecimalType(18,2)), 0.00))]
-Aggregate Attributes [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(ss_quantity#3 as decimal(12,2)))), DecimalType(18,2)), 0.00))#22]
-Results [2]: [i_category#16, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(ss_quantity#3 as decimal(12,2)))), DecimalType(18,2)), 0.00))#22 AS sumsales#26]
+Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))]
+Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22]
+Results [2]: [i_category#16, sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22 AS sumsales#26]
 
 (57) HashAggregate [codegen id : 70]
 Input [2]: [i_category#16, sumsales#26]
@@ -384,9 +384,9 @@ Output [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8
 (61) HashAggregate [codegen id : 79]
 Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#109, isEmpty#110]
 Keys [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12]
-Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(ss_quantity#3 as decimal(12,2)))), DecimalType(18,2)), 0.00))]
-Aggregate Attributes [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(ss_quantity#3 as decimal(12,2)))), DecimalType(18,2)), 0.00))#22]
-Results [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(ss_quantity#3 as decimal(12,2)))), DecimalType(18,2)), 0.00))#22 AS sumsales#26]
+Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))]
+Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22]
+Results [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22 AS sumsales#26]
 
 (62) HashAggregate [codegen id : 79]
 Input [1]: [sumsales#26]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.sf100/simplified.txt
index 8b39e27c4ca..7c3f8b33610 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.sf100/simplified.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.sf100/simplified.txt
@@ -9,7 +9,7 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_
                 Exchange [i_category] #1
                   Union
                     WholeStageCodegen (8)
-                      HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price as decimal(12,2))) * promote_precision(cast(ss_quantity as decimal(12,2)))), DecimalType(18,2)), 0.00)),sumsales,sum,isEmpty]
+                      HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty]
                         InputAdapter
                           Exchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] #2
                             WholeStageCodegen (7)
@@ -63,7 +63,7 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_
                           Exchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy] #7
                             WholeStageCodegen (16)
                               HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sumsales] [sum,isEmpty,sum,isEmpty]
-                                HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price as decimal(12,2))) * promote_precision(cast(ss_quantity as decimal(12,2)))), DecimalType(18,2)), 0.00)),sumsales,sum,isEmpty]
+                                HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty]
                                   InputAdapter
                                     ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2
                     WholeStageCodegen (26)
@@ -72,7 +72,7 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_
                           Exchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy] #8
                             WholeStageCodegen (25)
                               HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sumsales] [sum,isEmpty,sum,isEmpty]
-                                HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price as decimal(12,2))) * promote_precision(cast(ss_quantity as decimal(12,2)))), DecimalType(18,2)), 0.00)),sumsales,sum,isEmpty]
+                                HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty]
                                   InputAdapter
                                     ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2
                     WholeStageCodegen (35)
@@ -81,7 +81,7 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_
                           Exchange [i_category,i_class,i_brand,i_product_name,d_year] #9
                             WholeStageCodegen (34)
                               HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,sumsales] [sum,isEmpty,sum,isEmpty]
-                                HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price as decimal(12,2))) * promote_precision(cast(ss_quantity as decimal(12,2)))), DecimalType(18,2)), 0.00)),sumsales,sum,isEmpty]
+                                HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty]
                                   InputAdapter
                                     ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2
                     WholeStageCodegen (44)
@@ -90,7 +90,7 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_
                           Exchange [i_category,i_class,i_brand,i_product_name] #10
                             WholeStageCodegen (43)
                               HashAggregate [i_category,i_class,i_brand,i_product_name,sumsales] [sum,isEmpty,sum,isEmpty]
-                                HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price as decimal(12,2))) * promote_precision(cast(ss_quantity as decimal(12,2)))), DecimalType(18,2)), 0.00)),sumsales,sum,isEmpty]
+                                HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty]
                                   InputAdapter
                                     ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2
                     WholeStageCodegen (53)
@@ -99,7 +99,7 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_
                           Exchange [i_category,i_class,i_brand] #11
                             WholeStageCodegen (52)
                               HashAggregate [i_category,i_class,i_brand,sumsales] [sum,isEmpty,sum,isEmpty]
-                                HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price as decimal(12,2))) * promote_precision(cast(ss_quantity as decimal(12,2)))), DecimalType(18,2)), 0.00)),sumsales,sum,isEmpty]
+                                HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty]
                                   InputAdapter
                                     ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2
                     WholeStageCodegen (62)
@@ -108,7 +108,7 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_
                           Exchange [i_category,i_class] #12
                             WholeStageCodegen (61)
                               HashAggregate [i_category,i_class,sumsales] [sum,isEmpty,sum,isEmpty]
-                                HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price as decimal(12,2))) * promote_precision(cast(ss_quantity as decimal(12,2)))), DecimalType(18,2)), 0.00)),sumsales,sum,isEmpty]
+                                HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty]
                                   InputAdapter
                                     ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2
                     WholeStageCodegen (71)
@@ -117,7 +117,7 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_
                           Exchange [i_category] #13
                             WholeStageCodegen (70)
                               HashAggregate [i_category,sumsales] [sum,isEmpty,sum,isEmpty]
-                                HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price as decimal(12,2))) * promote_precision(cast(ss_quantity as decimal(12,2)))), DecimalType(18,2)), 0.00)),sumsales,sum,isEmpty]
+                                HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty]
                                   InputAdapter
                                     ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2
                     WholeStageCodegen (80)
@@ -126,6 +126,6 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_
                           Exchange #14
                             WholeStageCodegen (79)
                               HashAggregate [sumsales] [sum,isEmpty,sum,isEmpty]
-                                HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price as decimal(12,2))) * promote_precision(cast(ss_quantity as decimal(12,2)))), DecimalType(18,2)), 0.00)),sumsales,sum,isEmpty]
+                                HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty]
                                   InputAdapter
                                     ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt
index a8506c6577b..12985f2ffa6 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt
@@ -152,7 +152,7 @@ Input [12]: [ss_item_sk#1, ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d
 (19) HashAggregate [codegen id : 4]
 Input [10]: [ss_quantity#3, ss_sales_price#4, d_year#8, d_moy#9, d_qoy#10, s_store_id#12, i_brand#14, i_class#15, i_category#16, i_product_name#17]
 Keys [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12]
-Functions [1]: [partial_sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(ss_quantity#3 as decimal(12,2)))), DecimalType(18,2)), 0.00))]
+Functions [1]: [partial_sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))]
 Aggregate Attributes [2]: [sum#18, isEmpty#19]
 Results [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#20, isEmpty#21]
 
@@ -163,9 +163,9 @@ Arguments: hashpartitioning(i_category#16, i_class#15, i_brand#14, i_product_nam
 (21) HashAggregate [codegen id : 5]
 Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#20, isEmpty#21]
 Keys [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12]
-Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(ss_quantity#3 as decimal(12,2)))), DecimalType(18,2)), 0.00))]
-Aggregate Attributes [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(ss_quantity#3 as decimal(12,2)))), DecimalType(18,2)), 0.00))#22]
-Results [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, cast(sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(ss_quantity#3 as decimal(12,2)))), DecimalType(18,2)), 0.00))#22 as decimal(38,2)) AS sumsales#23]
+Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))]
+Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22]
+Results [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, cast(sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22 as decimal(38,2)) AS sumsales#23]
 
 (22) ReusedExchange [Reuses operator id: 20]
 Output [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#24, isEmpty#25]
@@ -173,9 +173,9 @@ Output [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8
 (23) HashAggregate [codegen id : 10]
 Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#24, isEmpty#25]
 Keys [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12]
-Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(ss_quantity#3 as decimal(12,2)))), DecimalType(18,2)), 0.00))]
-Aggregate Attributes [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(ss_quantity#3 as decimal(12,2)))), DecimalType(18,2)), 0.00))#22]
-Results [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(ss_quantity#3 as decimal(12,2)))), DecimalType(18,2)), 0.00))#22 AS sumsales#26]
+Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))]
+Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22]
+Results [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22 AS sumsales#26]
 
 (24) HashAggregate [codegen id : 10]
 Input [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, sumsales#26]
@@ -201,9 +201,9 @@ Output [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8
 (28) HashAggregate [codegen id : 16]
 Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#34, isEmpty#35]
 Keys [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12]
-Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(ss_quantity#3 as decimal(12,2)))), DecimalType(18,2)), 0.00))]
-Aggregate Attributes [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(ss_quantity#3 as decimal(12,2)))), DecimalType(18,2)), 0.00))#22]
-Results [7]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(ss_quantity#3 as decimal(12,2)))), DecimalType(18,2)), 0.00))#22 AS sumsales#26]
+Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))]
+Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22]
+Results [7]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22 AS sumsales#26]
 
 (29) HashAggregate [codegen id : 16]
 Input [7]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, sumsales#26]
@@ -229,9 +229,9 @@ Output [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8
 (33) HashAggregate [codegen id : 22]
 Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#44, isEmpty#45]
 Keys [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12]
-Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(ss_quantity#3 as decimal(12,2)))), DecimalType(18,2)), 0.00))]
-Aggregate Attributes [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(ss_quantity#3 as decimal(12,2)))), DecimalType(18,2)), 0.00))#22]
-Results [6]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(ss_quantity#3 as decimal(12,2)))), DecimalType(18,2)), 0.00))#22 AS sumsales#26]
+Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))]
+Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22]
+Results [6]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22 AS sumsales#26]
 
 (34) HashAggregate [codegen id : 22]
 Input [6]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, sumsales#26]
@@ -257,9 +257,9 @@ Output [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8
 (38) HashAggregate [codegen id : 28]
 Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#55, isEmpty#56]
 Keys [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12]
-Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(ss_quantity#3 as decimal(12,2)))), DecimalType(18,2)), 0.00))]
-Aggregate Attributes [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(ss_quantity#3 as decimal(12,2)))), DecimalType(18,2)), 0.00))#22]
-Results [5]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(ss_quantity#3 as decimal(12,2)))), DecimalType(18,2)), 0.00))#22 AS sumsales#26]
+Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))]
+Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22]
+Results [5]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22 AS sumsales#26]
 
 (39) HashAggregate [codegen id : 28]
 Input [5]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, sumsales#26]
@@ -285,9 +285,9 @@ Output [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8
 (43) HashAggregate [codegen id : 34]
 Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#67, isEmpty#68]
 Keys [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12]
-Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(ss_quantity#3 as decimal(12,2)))), DecimalType(18,2)), 0.00))]
-Aggregate Attributes [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(ss_quantity#3 as decimal(12,2)))), DecimalType(18,2)), 0.00))#22]
-Results [4]: [i_category#16, i_class#15, i_brand#14, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(ss_quantity#3 as decimal(12,2)))), DecimalType(18,2)), 0.00))#22 AS sumsales#26]
+Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))]
+Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22]
+Results [4]: [i_category#16, i_class#15, i_brand#14, sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22 AS sumsales#26]
 
 (44) HashAggregate [codegen id : 34]
 Input [4]: [i_category#16, i_class#15, i_brand#14, sumsales#26]
@@ -313,9 +313,9 @@ Output [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8
 (48) HashAggregate [codegen id : 40]
 Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#80, isEmpty#81]
 Keys [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12]
-Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(ss_quantity#3 as decimal(12,2)))), DecimalType(18,2)), 0.00))]
-Aggregate Attributes [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(ss_quantity#3 as decimal(12,2)))), DecimalType(18,2)), 0.00))#22]
-Results [3]: [i_category#16, i_class#15, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(ss_quantity#3 as decimal(12,2)))), DecimalType(18,2)), 0.00))#22 AS sumsales#26]
+Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))]
+Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22]
+Results [3]: [i_category#16, i_class#15, sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22 AS sumsales#26]
 
 (49) HashAggregate [codegen id : 40]
 Input [3]: [i_category#16, i_class#15, sumsales#26]
@@ -341,9 +341,9 @@ Output [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8
 (53) HashAggregate [codegen id : 46]
 Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#94, isEmpty#95]
 Keys [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12]
-Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(ss_quantity#3 as decimal(12,2)))), DecimalType(18,2)), 0.00))]
-Aggregate Attributes [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(ss_quantity#3 as decimal(12,2)))), DecimalType(18,2)), 0.00))#22]
-Results [2]: [i_category#16, sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(ss_quantity#3 as decimal(12,2)))), DecimalType(18,2)), 0.00))#22 AS sumsales#26]
+Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))]
+Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22]
+Results [2]: [i_category#16, sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22 AS sumsales#26]
 
 (54) HashAggregate [codegen id : 46]
 Input [2]: [i_category#16, sumsales#26]
@@ -369,9 +369,9 @@ Output [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8
 (58) HashAggregate [codegen id : 52]
 Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sum#109, isEmpty#110]
 Keys [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12]
-Functions [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(ss_quantity#3 as decimal(12,2)))), DecimalType(18,2)), 0.00))]
-Aggregate Attributes [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(ss_quantity#3 as decimal(12,2)))), DecimalType(18,2)), 0.00))#22]
-Results [1]: [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price#4 as decimal(12,2))) * promote_precision(cast(ss_quantity#3 as decimal(12,2)))), DecimalType(18,2)), 0.00))#22 AS sumsales#26]
+Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))]
+Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22]
+Results [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22 AS sumsales#26]
 
 (59) HashAggregate [codegen id : 52]
 Input [1]: [sumsales#26]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt
index 35d28516561..389318e7c07 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt
@@ -9,7 +9,7 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_
                 Exchange [i_category] #1
                   Union
                     WholeStageCodegen (5)
-                      HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price as decimal(12,2))) * promote_precision(cast(ss_quantity as decimal(12,2)))), DecimalType(18,2)), 0.00)),sumsales,sum,isEmpty]
+                      HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty]
                         InputAdapter
                           Exchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] #2
                             WholeStageCodegen (4)
@@ -54,7 +54,7 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_
                           Exchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy] #6
                             WholeStageCodegen (10)
                               HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,sumsales] [sum,isEmpty,sum,isEmpty]
-                                HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price as decimal(12,2))) * promote_precision(cast(ss_quantity as decimal(12,2)))), DecimalType(18,2)), 0.00)),sumsales,sum,isEmpty]
+                                HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty]
                                   InputAdapter
                                     ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2
                     WholeStageCodegen (17)
@@ -63,7 +63,7 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_
                           Exchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy] #7
                             WholeStageCodegen (16)
                               HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,sumsales] [sum,isEmpty,sum,isEmpty]
-                                HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price as decimal(12,2))) * promote_precision(cast(ss_quantity as decimal(12,2)))), DecimalType(18,2)), 0.00)),sumsales,sum,isEmpty]
+                                HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty]
                                   InputAdapter
                                     ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2
                     WholeStageCodegen (23)
@@ -72,7 +72,7 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_
                           Exchange [i_category,i_class,i_brand,i_product_name,d_year] #8
                             WholeStageCodegen (22)
                               HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,sumsales] [sum,isEmpty,sum,isEmpty]
-                                HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price as decimal(12,2))) * promote_precision(cast(ss_quantity as decimal(12,2)))), DecimalType(18,2)), 0.00)),sumsales,sum,isEmpty]
+                                HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty]
                                   InputAdapter
                                     ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2
                     WholeStageCodegen (29)
@@ -81,7 +81,7 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_
                           Exchange [i_category,i_class,i_brand,i_product_name] #9
                             WholeStageCodegen (28)
                               HashAggregate [i_category,i_class,i_brand,i_product_name,sumsales] [sum,isEmpty,sum,isEmpty]
-                                HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price as decimal(12,2))) * promote_precision(cast(ss_quantity as decimal(12,2)))), DecimalType(18,2)), 0.00)),sumsales,sum,isEmpty]
+                                HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty]
                                   InputAdapter
                                     ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2
                     WholeStageCodegen (35)
@@ -90,7 +90,7 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_
                           Exchange [i_category,i_class,i_brand] #10
                             WholeStageCodegen (34)
                               HashAggregate [i_category,i_class,i_brand,sumsales] [sum,isEmpty,sum,isEmpty]
-                                HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price as decimal(12,2))) * promote_precision(cast(ss_quantity as decimal(12,2)))), DecimalType(18,2)), 0.00)),sumsales,sum,isEmpty]
+                                HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty]
                                   InputAdapter
                                     ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2
                     WholeStageCodegen (41)
@@ -99,7 +99,7 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_
                           Exchange [i_category,i_class] #11
                             WholeStageCodegen (40)
                               HashAggregate [i_category,i_class,sumsales] [sum,isEmpty,sum,isEmpty]
-                                HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price as decimal(12,2))) * promote_precision(cast(ss_quantity as decimal(12,2)))), DecimalType(18,2)), 0.00)),sumsales,sum,isEmpty]
+                                HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty]
                                   InputAdapter
                                     ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2
                     WholeStageCodegen (47)
@@ -108,7 +108,7 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_
                           Exchange [i_category] #12
                             WholeStageCodegen (46)
                               HashAggregate [i_category,sumsales] [sum,isEmpty,sum,isEmpty]
-                                HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price as decimal(12,2))) * promote_precision(cast(ss_quantity as decimal(12,2)))), DecimalType(18,2)), 0.00)),sumsales,sum,isEmpty]
+                                HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty]
                                   InputAdapter
                                     ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2
                     WholeStageCodegen (53)
@@ -117,6 +117,6 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_
                           Exchange #13
                             WholeStageCodegen (52)
                               HashAggregate [sumsales] [sum,isEmpty,sum,isEmpty]
-                                HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce(CheckOverflow((promote_precision(cast(ss_sales_price as decimal(12,2))) * promote_precision(cast(ss_quantity as decimal(12,2)))), DecimalType(18,2)), 0.00)),sumsales,sum,isEmpty]
+                                HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty]
                                   InputAdapter
                                     ReusedExchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] #2
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.sf100/explain.txt
index 8e9e50cc2f0..e52b8d94694 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74.sf100/explain.txt
@@ -428,7 +428,7 @@ Arguments: [customer_id#57 ASC NULLS FIRST], false, 0
 (77) SortMergeJoin [codegen id : 35]
 Left keys [1]: [customer_id#14]
 Right keys [1]: [customer_id#57]
-Join condition: (CASE WHEN (year_total#45 > 0.00) THEN CheckOverflow((promote_precision(year_total#58) / promote_precision(year_total#45)), DecimalType(37,20)) END > CASE WHEN (year_total#15 > 0.00) THEN CheckOverflow((promote_precision(year_total#31) / promote_precision(year_total#15)), DecimalType(37,20)) END)
+Join condition: (CASE WHEN (year_total#45 > 0.00) THEN (year_total#58 / year_total#45) END > CASE WHEN (year_total#15 > 0.00) THEN (year_total#31 / year_total#15) END)
 
 (78) Project [codegen id : 35]
 Output [3]: [customer_id#28, customer_first_name#29, customer_last_name#30]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/explain.txt
index 3c549bb77b1..a58aad9933b 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q74/explain.txt
@@ -397,7 +397,7 @@ Arguments: HashedRelationBroadcastMode(List(input[0, string, true]),false), [pla
 (69) BroadcastHashJoin [codegen id : 16]
 Left keys [1]: [customer_id#14]
 Right keys [1]: [customer_id#57]
-Join condition: (CASE WHEN (year_total#45 > 0.00) THEN CheckOverflow((promote_precision(year_total#58) / promote_precision(year_total#45)), DecimalType(37,20)) END > CASE WHEN (year_total#15 > 0.00) THEN CheckOverflow((promote_precision(year_total#31) / promote_precision(year_total#15)), DecimalType(37,20)) END)
+Join condition: (CASE WHEN (year_total#45 > 0.00) THEN (year_total#58 / year_total#45) END > CASE WHEN (year_total#15 > 0.00) THEN (year_total#31 / year_total#15) END)
 
 (70) Project [codegen id : 16]
 Output [3]: [customer_id#28, customer_first_name#29, customer_last_name#30]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.sf100/explain.txt
index 6bb0d1e9053..505fda5b7ad 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.sf100/explain.txt
@@ -1,131 +1,133 @@
 == Physical Plan ==
-TakeOrderedAndProject (127)
-+- * Project (126)
-   +- * SortMergeJoin Inner (125)
-      :- * Sort (70)
-      :  +- Exchange (69)
-      :     +- * HashAggregate (68)
-      :        +- Exchange (67)
-      :           +- * HashAggregate (66)
-      :              +- * HashAggregate (65)
-      :                 +- Exchange (64)
-      :                    +- * HashAggregate (63)
-      :                       +- Union (62)
-      :                          :- * Project (23)
-      :                          :  +- * SortMergeJoin LeftOuter (22)
-      :                          :     :- * Sort (15)
-      :                          :     :  +- Exchange (14)
-      :                          :     :     +- * Project (13)
-      :                          :     :        +- * BroadcastHashJoin Inner BuildRight (12)
-      :                          :     :           :- * Project (10)
-      :                          :     :           :  +- * BroadcastHashJoin Inner BuildRight (9)
-      :                          :     :           :     :- * Filter (3)
-      :                          :     :           :     :  +- * ColumnarToRow (2)
-      :                          :     :           :     :     +- Scan parquet default.catalog_sales (1)
-      :                          :     :           :     +- BroadcastExchange (8)
-      :                          :     :           :        +- * Project (7)
-      :                          :     :           :           +- * Filter (6)
-      :                          :     :           :              +- * ColumnarToRow (5)
-      :                          :     :           :                 +- Scan parquet default.item (4)
-      :                          :     :           +- ReusedExchange (11)
-      :                          :     +- * Sort (21)
-      :                          :        +- Exchange (20)
-      :                          :           +- * Project (19)
-      :                          :              +- * Filter (18)
-      :                          :                 +- * ColumnarToRow (17)
-      :                          :                    +- Scan parquet default.catalog_returns (16)
-      :                          :- * Project (42)
-      :                          :  +- * SortMergeJoin LeftOuter (41)
-      :                          :     :- * Sort (34)
-      :                          :     :  +- Exchange (33)
-      :                          :     :     +- * Project (32)
-      :                          :     :        +- * BroadcastHashJoin Inner BuildRight (31)
-      :                          :     :           :- * Project (29)
-      :                          :     :           :  +- * BroadcastHashJoin Inner BuildRight (28)
-      :                          :     :           :     :- * Filter (26)
-      :                          :     :           :     :  +- * ColumnarToRow (25)
-      :                          :     :           :     :     +- Scan parquet default.store_sales (24)
-      :                          :     :           :     +- ReusedExchange (27)
-      :                          :     :           +- ReusedExchange (30)
-      :                          :     +- * Sort (40)
-      :                          :        +- Exchange (39)
-      :                          :           +- * Project (38)
-      :                          :              +- * Filter (37)
-      :                          :                 +- * ColumnarToRow (36)
-      :                          :                    +- Scan parquet default.store_returns (35)
-      :                          +- * Project (61)
-      :                             +- * SortMergeJoin LeftOuter (60)
-      :                                :- * Sort (53)
-      :                                :  +- Exchange (52)
-      :                                :     +- * Project (51)
-      :                                :        +- * BroadcastHashJoin Inner BuildRight (50)
-      :                                :           :- * Project (48)
-      :                                :           :  +- * BroadcastHashJoin Inner BuildRight (47)
-      :                                :           :     :- * Filter (45)
-      :                                :           :     :  +- * ColumnarToRow (44)
-      :                                :           :     :     +- Scan parquet default.web_sales (43)
-      :                                :           :     +- ReusedExchange (46)
-      :                                :           +- ReusedExchange (49)
-      :                                +- * Sort (59)
-      :                                   +- Exchange (58)
-      :                                      +- * Project (57)
-      :                                         +- * Filter (56)
-      :                                            +- * ColumnarToRow (55)
-      :                                               +- Scan parquet default.web_returns (54)
-      +- * Sort (124)
-         +- Exchange (123)
-            +- * HashAggregate (122)
-               +- Exchange (121)
-                  +- * HashAggregate (120)
-                     +- * HashAggregate (119)
-                        +- Exchange (118)
-                           +- * HashAggregate (117)
-                              +- Union (116)
-                                 :- * Project (85)
-                                 :  +- * SortMergeJoin LeftOuter (84)
-                                 :     :- * Sort (81)
-                                 :     :  +- Exchange (80)
-                                 :     :     +- * Project (79)
-                                 :     :        +- * BroadcastHashJoin Inner BuildRight (78)
-                                 :     :           :- * Project (76)
-                                 :     :           :  +- * BroadcastHashJoin Inner BuildRight (75)
-                                 :     :           :     :- * Filter (73)
-                                 :     :           :     :  +- * ColumnarToRow (72)
-                                 :     :           :     :     +- Scan parquet default.catalog_sales (71)
-                                 :     :           :     +- ReusedExchange (74)
-                                 :     :           +- ReusedExchange (77)
-                                 :     +- * Sort (83)
-                                 :        +- ReusedExchange (82)
-                                 :- * Project (100)
-                                 :  +- * SortMergeJoin LeftOuter (99)
-                                 :     :- * Sort (96)
-                                 :     :  +- Exchange (95)
-                                 :     :     +- * Project (94)
-                                 :     :        +- * BroadcastHashJoin Inner BuildRight (93)
-                                 :     :           :- * Project (91)
-                                 :     :           :  +- * BroadcastHashJoin Inner BuildRight (90)
-                                 :     :           :     :- * Filter (88)
-                                 :     :           :     :  +- * ColumnarToRow (87)
-                                 :     :           :     :     +- Scan parquet default.store_sales (86)
-                                 :     :           :     +- ReusedExchange (89)
-                                 :     :           +- ReusedExchange (92)
-                                 :     +- * Sort (98)
-                                 :        +- ReusedExchange (97)
-                                 +- * Project (115)
-                                    +- * SortMergeJoin LeftOuter (114)
-                                       :- * Sort (111)
-                                       :  +- Exchange (110)
-                                       :     +- * Project (109)
-                                       :        +- * BroadcastHashJoin Inner BuildRight (108)
-                                       :           :- * Project (106)
-                                       :           :  +- * BroadcastHashJoin Inner BuildRight (105)
-                                       :           :     :- * Filter (103)
-                                       :           :     :  +- * ColumnarToRow (102)
-                                       :           :     :     +- Scan parquet default.web_sales (101)
-                                       :           :     +- ReusedExchange (104)
-                                       :           +- ReusedExchange (107)
-                                       +- * Sort (113)
-                                          +- ReusedExchange (112)
+TakeOrderedAndProject (129)
++- * Project (128)
+   +- * SortMergeJoin Inner (127)
+      :- * Sort (71)
+      :  +- Exchange (70)
+      :     +- * Filter (69)
+      :        +- * HashAggregate (68)
+      :           +- Exchange (67)
+      :              +- * HashAggregate (66)
+      :                 +- * HashAggregate (65)
+      :                    +- Exchange (64)
+      :                       +- * HashAggregate (63)
+      :                          +- Union (62)
+      :                             :- * Project (23)
+      :                             :  +- * SortMergeJoin LeftOuter (22)
+      :                             :     :- * Sort (15)
+      :                             :     :  +- Exchange (14)
+      :                             :     :     +- * Project (13)
+      :                             :     :        +- * BroadcastHashJoin Inner BuildRight (12)
+      :                             :     :           :- * Project (10)
+      :                             :     :           :  +- * BroadcastHashJoin Inner BuildRight (9)
+      :                             :     :           :     :- * Filter (3)
+      :                             :     :           :     :  +- * ColumnarToRow (2)
+      :                             :     :           :     :     +- Scan parquet default.catalog_sales (1)
+      :                             :     :           :     +- BroadcastExchange (8)
+      :                             :     :           :        +- * Project (7)
+      :                             :     :           :           +- * Filter (6)
+      :                             :     :           :              +- * ColumnarToRow (5)
+      :                             :     :           :                 +- Scan parquet default.item (4)
+      :                             :     :           +- ReusedExchange (11)
+      :                             :     +- * Sort (21)
+      :                             :        +- Exchange (20)
+      :                             :           +- * Project (19)
+      :                             :              +- * Filter (18)
+      :                             :                 +- * ColumnarToRow (17)
+      :                             :                    +- Scan parquet default.catalog_returns (16)
+      :                             :- * Project (42)
+      :                             :  +- * SortMergeJoin LeftOuter (41)
+      :                             :     :- * Sort (34)
+      :                             :     :  +- Exchange (33)
+      :                             :     :     +- * Project (32)
+      :                             :     :        +- * BroadcastHashJoin Inner BuildRight (31)
+      :                             :     :           :- * Project (29)
+      :                             :     :           :  +- * BroadcastHashJoin Inner BuildRight (28)
+      :                             :     :           :     :- * Filter (26)
+      :                             :     :           :     :  +- * ColumnarToRow (25)
+      :                             :     :           :     :     +- Scan parquet default.store_sales (24)
+      :                             :     :           :     +- ReusedExchange (27)
+      :                             :     :           +- ReusedExchange (30)
+      :                             :     +- * Sort (40)
+      :                             :        +- Exchange (39)
+      :                             :           +- * Project (38)
+      :                             :              +- * Filter (37)
+      :                             :                 +- * ColumnarToRow (36)
+      :                             :                    +- Scan parquet default.store_returns (35)
+      :                             +- * Project (61)
+      :                                +- * SortMergeJoin LeftOuter (60)
+      :                                   :- * Sort (53)
+      :                                   :  +- Exchange (52)
+      :                                   :     +- * Project (51)
+      :                                   :        +- * BroadcastHashJoin Inner BuildRight (50)
+      :                                   :           :- * Project (48)
+      :                                   :           :  +- * BroadcastHashJoin Inner BuildRight (47)
+      :                                   :           :     :- * Filter (45)
+      :                                   :           :     :  +- * ColumnarToRow (44)
+      :                                   :           :     :     +- Scan parquet default.web_sales (43)
+      :                                   :           :     +- ReusedExchange (46)
+      :                                   :           +- ReusedExchange (49)
+      :                                   +- * Sort (59)
+      :                                      +- Exchange (58)
+      :                                         +- * Project (57)
+      :                                            +- * Filter (56)
+      :                                               +- * ColumnarToRow (55)
+      :                                                  +- Scan parquet default.web_returns (54)
+      +- * Sort (126)
+         +- Exchange (125)
+            +- * Filter (124)
+               +- * HashAggregate (123)
+                  +- Exchange (122)
+                     +- * HashAggregate (121)
+                        +- * HashAggregate (120)
+                           +- Exchange (119)
+                              +- * HashAggregate (118)
+                                 +- Union (117)
+                                    :- * Project (86)
+                                    :  +- * SortMergeJoin LeftOuter (85)
+                                    :     :- * Sort (82)
+                                    :     :  +- Exchange (81)
+                                    :     :     +- * Project (80)
+                                    :     :        +- * BroadcastHashJoin Inner BuildRight (79)
+                                    :     :           :- * Project (77)
+                                    :     :           :  +- * BroadcastHashJoin Inner BuildRight (76)
+                                    :     :           :     :- * Filter (74)
+                                    :     :           :     :  +- * ColumnarToRow (73)
+                                    :     :           :     :     +- Scan parquet default.catalog_sales (72)
+                                    :     :           :     +- ReusedExchange (75)
+                                    :     :           +- ReusedExchange (78)
+                                    :     +- * Sort (84)
+                                    :        +- ReusedExchange (83)
+                                    :- * Project (101)
+                                    :  +- * SortMergeJoin LeftOuter (100)
+                                    :     :- * Sort (97)
+                                    :     :  +- Exchange (96)
+                                    :     :     +- * Project (95)
+                                    :     :        +- * BroadcastHashJoin Inner BuildRight (94)
+                                    :     :           :- * Project (92)
+                                    :     :           :  +- * BroadcastHashJoin Inner BuildRight (91)
+                                    :     :           :     :- * Filter (89)
+                                    :     :           :     :  +- * ColumnarToRow (88)
+                                    :     :           :     :     +- Scan parquet default.store_sales (87)
+                                    :     :           :     +- ReusedExchange (90)
+                                    :     :           +- ReusedExchange (93)
+                                    :     +- * Sort (99)
+                                    :        +- ReusedExchange (98)
+                                    +- * Project (116)
+                                       +- * SortMergeJoin LeftOuter (115)
+                                          :- * Sort (112)
+                                          :  +- Exchange (111)
+                                          :     +- * Project (110)
+                                          :        +- * BroadcastHashJoin Inner BuildRight (109)
+                                          :           :- * Project (107)
+                                          :           :  +- * BroadcastHashJoin Inner BuildRight (106)
+                                          :           :     :- * Filter (104)
+                                          :           :     :  +- * ColumnarToRow (103)
+                                          :           :     :     +- Scan parquet default.web_sales (102)
+                                          :           :     +- ReusedExchange (105)
+                                          :           +- ReusedExchange (108)
+                                          +- * Sort (114)
+                                             +- ReusedExchange (113)
 
 
 (1) Scan parquet default.catalog_sales
@@ -174,7 +176,7 @@ Join condition: None
 Output [9]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12]
 Input [10]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, cs_sold_date_sk#5, i_item_sk#7, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12]
 
-(11) ReusedExchange [Reuses operator id: 131]
+(11) ReusedExchange [Reuses operator id: 133]
 Output [2]: [d_date_sk#13, d_year#14]
 
 (12) BroadcastHashJoin [codegen id : 3]
@@ -226,7 +228,7 @@ Right keys [2]: [cr_order_number#16, cr_item_sk#15]
 Join condition: None
 
 (23) Project [codegen id : 7]
-Output [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (cs_quantity#3 - coalesce(cr_return_quantity#17, 0)) AS sales_cnt#20, CheckOverflow((promote_precision(cast(cs_ext_sales_price#4 as decimal(8,2))) - promote_precision(cast(coalesce(cr_return_amount#18, 0.00) as decimal(8,2)))), DecimalType(8,2)) AS sales_amt#21]
+Output [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, (cs_quantity#3 - coalesce(cr_return_quantity#17, 0)) AS sales_cnt#20, (cs_ext_sales_price#4 - coalesce(cr_return_amount#18, 0.00)) AS sales_amt#21]
 Input [13]: [cs_item_sk#1, cs_order_number#2, cs_quantity#3, cs_ext_sales_price#4, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, d_year#14, cr_item_sk#15, cr_order_number#16, cr_return_quantity#17, cr_return_amount#18]
 
 (24) Scan parquet default.store_sales
@@ -256,7 +258,7 @@ Join condition: None
 Output [9]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31]
 Input [10]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, ss_sold_date_sk#26, i_item_sk#27, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31]
 
-(30) ReusedExchange [Reuses operator id: 131]
+(30) ReusedExchange [Reuses operator id: 133]
 Output [2]: [d_date_sk#32, d_year#33]
 
 (31) BroadcastHashJoin [codegen id : 10]
@@ -308,7 +310,7 @@ Right keys [2]: [sr_ticket_number#35, sr_item_sk#34]
 Join condition: None
 
 (42) Project [codegen id : 14]
-Output [7]: [d_year#33, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, (ss_quantity#24 - coalesce(sr_return_quantity#36, 0)) AS sales_cnt#39, CheckOverflow((promote_precision(cast(ss_ext_sales_price#25 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#37, 0.00) as decimal(8,2)))), DecimalType(8,2)) AS sales_amt#40]
+Output [7]: [d_year#33, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, (ss_quantity#24 - coalesce(sr_return_quantity#36, 0)) AS sales_cnt#39, (ss_ext_sales_price#25 - coalesce(sr_return_amt#37, 0.00)) AS sales_amt#40]
 Input [13]: [ss_item_sk#22, ss_ticket_number#23, ss_quantity#24, ss_ext_sales_price#25, i_brand_id#28, i_class_id#29, i_category_id#30, i_manufact_id#31, d_year#33, sr_item_sk#34, sr_ticket_number#35, sr_return_quantity#36, sr_return_amt#37]
 
 (43) Scan parquet default.web_sales
@@ -338,7 +340,7 @@ Join condition: None
 Output [9]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, ws_sold_date_sk#45, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50]
 Input [10]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, ws_sold_date_sk#45, i_item_sk#46, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50]
 
-(49) ReusedExchange [Reuses operator id: 131]
+(49) ReusedExchange [Reuses operator id: 133]
 Output [2]: [d_date_sk#51, d_year#52]
 
 (50) BroadcastHashJoin [codegen id : 17]
@@ -390,7 +392,7 @@ Right keys [2]: [wr_order_number#54, wr_item_sk#53]
 Join condition: None
 
 (61) Project [codegen id : 21]
-Output [7]: [d_year#52, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, (ws_quantity#43 - coalesce(wr_return_quantity#55, 0)) AS sales_cnt#58, CheckOverflow((promote_precision(cast(ws_ext_sales_price#44 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#56, 0.00) as decimal(8,2)))), DecimalType(8,2)) AS sales_amt#59]
+Output [7]: [d_year#52, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, (ws_quantity#43 - coalesce(wr_return_quantity#55, 0)) AS sales_cnt#58, (ws_ext_sales_price#44 - coalesce(wr_return_amt#56, 0.00)) AS sales_amt#59]
 Input [13]: [ws_item_sk#41, ws_order_number#42, ws_quantity#43, ws_ext_sales_price#44, i_brand_id#47, i_class_id#48, i_category_id#49, i_manufact_id#50, d_year#52, wr_item_sk#53, wr_order_number#54, wr_return_quantity#55, wr_return_amt#56]
 
 (62) Union
@@ -431,15 +433,19 @@ Functions [2]: [sum(sales_cnt#20), sum(UnscaledValue(sales_amt#21))]
 Aggregate Attributes [2]: [sum(sales_cnt#20)#64, sum(UnscaledValue(sales_amt#21))#65]
 Results [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sum(sales_cnt#20)#64 AS sales_cnt#66, MakeDecimal(sum(UnscaledValue(sales_amt#21))#65,18,2) AS sales_amt#67]
 
-(69) Exchange
+(69) Filter [codegen id : 24]
+Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#66, sales_amt#67]
+Condition : isnotnull(sales_cnt#66)
+
+(70) Exchange
 Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#66, sales_amt#67]
 Arguments: hashpartitioning(i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, 5), ENSURE_REQUIREMENTS, [plan_id=10]
 
-(70) Sort [codegen id : 25]
+(71) Sort [codegen id : 25]
 Input [7]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#66, sales_amt#67]
 Arguments: [i_brand_id#8 ASC NULLS FIRST, i_class_id#9 ASC NULLS FIRST, i_category_id#10 ASC NULLS FIRST, i_manufact_id#12 ASC NULLS FIRST], false, 0
 
-(71) Scan parquet default.catalog_sales
+(72) Scan parquet default.catalog_sales
 Output [5]: [cs_item_sk#68, cs_order_number#69, cs_quantity#70, cs_ext_sales_price#71, cs_sold_date_sk#72]
 Batched: true
 Location: InMemoryFileIndex []
@@ -447,62 +453,62 @@ PartitionFilters: [isnotnull(cs_sold_date_sk#72), dynamicpruningexpression(cs_so
 PushedFilters: [IsNotNull(cs_item_sk)]
 ReadSchema: struct<cs_item_sk:int,cs_order_number:int,cs_quantity:int,cs_ext_sales_price:decimal(7,2)>
 
-(72) ColumnarToRow [codegen id : 28]
+(73) ColumnarToRow [codegen id : 28]
 Input [5]: [cs_item_sk#68, cs_order_number#69, cs_quantity#70, cs_ext_sales_price#71, cs_sold_date_sk#72]
 
-(73) Filter [codegen id : 28]
+(74) Filter [codegen id : 28]
 Input [5]: [cs_item_sk#68, cs_order_number#69, cs_quantity#70, cs_ext_sales_price#71, cs_sold_date_sk#72]
 Condition : isnotnull(cs_item_sk#68)
 
-(74) ReusedExchange [Reuses operator id: 8]
+(75) ReusedExchange [Reuses operator id: 8]
 Output [5]: [i_item_sk#74, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78]
 
-(75) BroadcastHashJoin [codegen id : 28]
+(76) BroadcastHashJoin [codegen id : 28]
 Left keys [1]: [cs_item_sk#68]
 Right keys [1]: [i_item_sk#74]
 Join condition: None
 
-(76) Project [codegen id : 28]
+(77) Project [codegen id : 28]
 Output [9]: [cs_item_sk#68, cs_order_number#69, cs_quantity#70, cs_ext_sales_price#71, cs_sold_date_sk#72, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78]
 Input [10]: [cs_item_sk#68, cs_order_number#69, cs_quantity#70, cs_ext_sales_price#71, cs_sold_date_sk#72, i_item_sk#74, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78]
 
-(77) ReusedExchange [Reuses operator id: 135]
+(78) ReusedExchange [Reuses operator id: 137]
 Output [2]: [d_date_sk#79, d_year#80]
 
-(78) BroadcastHashJoin [codegen id : 28]
+(79) BroadcastHashJoin [codegen id : 28]
 Left keys [1]: [cs_sold_date_sk#72]
 Right keys [1]: [d_date_sk#79]
 Join condition: None
 
-(79) Project [codegen id : 28]
+(80) Project [codegen id : 28]
 Output [9]: [cs_item_sk#68, cs_order_number#69, cs_quantity#70, cs_ext_sales_price#71, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, d_year#80]
 Input [11]: [cs_item_sk#68, cs_order_number#69, cs_quantity#70, cs_ext_sales_price#71, cs_sold_date_sk#72, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, d_date_sk#79, d_year#80]
 
-(80) Exchange
+(81) Exchange
 Input [9]: [cs_item_sk#68, cs_order_number#69, cs_quantity#70, cs_ext_sales_price#71, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, d_year#80]
 Arguments: hashpartitioning(cs_order_number#69, cs_item_sk#68, 5), ENSURE_REQUIREMENTS, [plan_id=11]
 
-(81) Sort [codegen id : 29]
+(82) Sort [codegen id : 29]
 Input [9]: [cs_item_sk#68, cs_order_number#69, cs_quantity#70, cs_ext_sales_price#71, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, d_year#80]
 Arguments: [cs_order_number#69 ASC NULLS FIRST, cs_item_sk#68 ASC NULLS FIRST], false, 0
 
-(82) ReusedExchange [Reuses operator id: 20]
+(83) ReusedExchange [Reuses operator id: 20]
 Output [4]: [cr_item_sk#81, cr_order_number#82, cr_return_quantity#83, cr_return_amount#84]
 
-(83) Sort [codegen id : 31]
+(84) Sort [codegen id : 31]
 Input [4]: [cr_item_sk#81, cr_order_number#82, cr_return_quantity#83, cr_return_amount#84]
 Arguments: [cr_order_number#82 ASC NULLS FIRST, cr_item_sk#81 ASC NULLS FIRST], false, 0
 
-(84) SortMergeJoin [codegen id : 32]
+(85) SortMergeJoin [codegen id : 32]
 Left keys [2]: [cs_order_number#69, cs_item_sk#68]
 Right keys [2]: [cr_order_number#82, cr_item_sk#81]
 Join condition: None
 
-(85) Project [codegen id : 32]
-Output [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, (cs_quantity#70 - coalesce(cr_return_quantity#83, 0)) AS sales_cnt#20, CheckOverflow((promote_precision(cast(cs_ext_sales_price#71 as decimal(8,2))) - promote_precision(cast(coalesce(cr_return_amount#84, 0.00) as decimal(8,2)))), DecimalType(8,2)) AS sales_amt#21]
+(86) Project [codegen id : 32]
+Output [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, (cs_quantity#70 - coalesce(cr_return_quantity#83, 0)) AS sales_cnt#20, (cs_ext_sales_price#71 - coalesce(cr_return_amount#84, 0.00)) AS sales_amt#21]
 Input [13]: [cs_item_sk#68, cs_order_number#69, cs_quantity#70, cs_ext_sales_price#71, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, d_year#80, cr_item_sk#81, cr_order_number#82, cr_return_quantity#83, cr_return_amount#84]
 
-(86) Scan parquet default.store_sales
+(87) Scan parquet default.store_sales
 Output [5]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89]
 Batched: true
 Location: InMemoryFileIndex []
@@ -510,62 +516,62 @@ PartitionFilters: [isnotnull(ss_sold_date_sk#89), dynamicpruningexpression(ss_so
 PushedFilters: [IsNotNull(ss_item_sk)]
 ReadSchema: struct<ss_item_sk:int,ss_ticket_number:int,ss_quantity:int,ss_ext_sales_price:decimal(7,2)>
 
-(87) ColumnarToRow [codegen id : 35]
+(88) ColumnarToRow [codegen id : 35]
 Input [5]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89]
 
-(88) Filter [codegen id : 35]
+(89) Filter [codegen id : 35]
 Input [5]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89]
 Condition : isnotnull(ss_item_sk#85)
 
-(89) ReusedExchange [Reuses operator id: 8]
+(90) ReusedExchange [Reuses operator id: 8]
 Output [5]: [i_item_sk#90, i_brand_id#91, i_class_id#92, i_category_id#93, i_manufact_id#94]
 
-(90) BroadcastHashJoin [codegen id : 35]
+(91) BroadcastHashJoin [codegen id : 35]
 Left keys [1]: [ss_item_sk#85]
 Right keys [1]: [i_item_sk#90]
 Join condition: None
 
-(91) Project [codegen id : 35]
+(92) Project [codegen id : 35]
 Output [9]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89, i_brand_id#91, i_class_id#92, i_category_id#93, i_manufact_id#94]
 Input [10]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89, i_item_sk#90, i_brand_id#91, i_class_id#92, i_category_id#93, i_manufact_id#94]
 
-(92) ReusedExchange [Reuses operator id: 135]
+(93) ReusedExchange [Reuses operator id: 137]
 Output [2]: [d_date_sk#95, d_year#96]
 
-(93) BroadcastHashJoin [codegen id : 35]
+(94) BroadcastHashJoin [codegen id : 35]
 Left keys [1]: [ss_sold_date_sk#89]
 Right keys [1]: [d_date_sk#95]
 Join condition: None
 
-(94) Project [codegen id : 35]
+(95) Project [codegen id : 35]
 Output [9]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#91, i_class_id#92, i_category_id#93, i_manufact_id#94, d_year#96]
 Input [11]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, ss_sold_date_sk#89, i_brand_id#91, i_class_id#92, i_category_id#93, i_manufact_id#94, d_date_sk#95, d_year#96]
 
-(95) Exchange
+(96) Exchange
 Input [9]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#91, i_class_id#92, i_category_id#93, i_manufact_id#94, d_year#96]
 Arguments: hashpartitioning(ss_ticket_number#86, ss_item_sk#85, 5), ENSURE_REQUIREMENTS, [plan_id=12]
 
-(96) Sort [codegen id : 36]
+(97) Sort [codegen id : 36]
 Input [9]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#91, i_class_id#92, i_category_id#93, i_manufact_id#94, d_year#96]
 Arguments: [ss_ticket_number#86 ASC NULLS FIRST, ss_item_sk#85 ASC NULLS FIRST], false, 0
 
-(97) ReusedExchange [Reuses operator id: 39]
+(98) ReusedExchange [Reuses operator id: 39]
 Output [4]: [sr_item_sk#97, sr_ticket_number#98, sr_return_quantity#99, sr_return_amt#100]
 
-(98) Sort [codegen id : 38]
+(99) Sort [codegen id : 38]
 Input [4]: [sr_item_sk#97, sr_ticket_number#98, sr_return_quantity#99, sr_return_amt#100]
 Arguments: [sr_ticket_number#98 ASC NULLS FIRST, sr_item_sk#97 ASC NULLS FIRST], false, 0
 
-(99) SortMergeJoin [codegen id : 39]
+(100) SortMergeJoin [codegen id : 39]
 Left keys [2]: [ss_ticket_number#86, ss_item_sk#85]
 Right keys [2]: [sr_ticket_number#98, sr_item_sk#97]
 Join condition: None
 
-(100) Project [codegen id : 39]
-Output [7]: [d_year#96, i_brand_id#91, i_class_id#92, i_category_id#93, i_manufact_id#94, (ss_quantity#87 - coalesce(sr_return_quantity#99, 0)) AS sales_cnt#39, CheckOverflow((promote_precision(cast(ss_ext_sales_price#88 as decimal(8,2))) - promote_precision(cast(coalesce(sr_return_amt#100, 0.00) as decimal(8,2)))), DecimalType(8,2)) AS sales_amt#40]
+(101) Project [codegen id : 39]
+Output [7]: [d_year#96, i_brand_id#91, i_class_id#92, i_category_id#93, i_manufact_id#94, (ss_quantity#87 - coalesce(sr_return_quantity#99, 0)) AS sales_cnt#39, (ss_ext_sales_price#88 - coalesce(sr_return_amt#100, 0.00)) AS sales_amt#40]
 Input [13]: [ss_item_sk#85, ss_ticket_number#86, ss_quantity#87, ss_ext_sales_price#88, i_brand_id#91, i_class_id#92, i_category_id#93, i_manufact_id#94, d_year#96, sr_item_sk#97, sr_ticket_number#98, sr_return_quantity#99, sr_return_amt#100]
 
-(101) Scan parquet default.web_sales
+(102) Scan parquet default.web_sales
 Output [5]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, ws_sold_date_sk#105]
 Batched: true
 Location: InMemoryFileIndex []
@@ -573,144 +579,148 @@ PartitionFilters: [isnotnull(ws_sold_date_sk#105), dynamicpruningexpression(ws_s
 PushedFilters: [IsNotNull(ws_item_sk)]
 ReadSchema: struct<ws_item_sk:int,ws_order_number:int,ws_quantity:int,ws_ext_sales_price:decimal(7,2)>
 
-(102) ColumnarToRow [codegen id : 42]
+(103) ColumnarToRow [codegen id : 42]
 Input [5]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, ws_sold_date_sk#105]
 
-(103) Filter [codegen id : 42]
+(104) Filter [codegen id : 42]
 Input [5]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, ws_sold_date_sk#105]
 Condition : isnotnull(ws_item_sk#101)
 
-(104) ReusedExchange [Reuses operator id: 8]
+(105) ReusedExchange [Reuses operator id: 8]
 Output [5]: [i_item_sk#106, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110]
 
-(105) BroadcastHashJoin [codegen id : 42]
+(106) BroadcastHashJoin [codegen id : 42]
 Left keys [1]: [ws_item_sk#101]
 Right keys [1]: [i_item_sk#106]
 Join condition: None
 
-(106) Project [codegen id : 42]
+(107) Project [codegen id : 42]
 Output [9]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, ws_sold_date_sk#105, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110]
 Input [10]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, ws_sold_date_sk#105, i_item_sk#106, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110]
 
-(107) ReusedExchange [Reuses operator id: 135]
+(108) ReusedExchange [Reuses operator id: 137]
 Output [2]: [d_date_sk#111, d_year#112]
 
-(108) BroadcastHashJoin [codegen id : 42]
+(109) BroadcastHashJoin [codegen id : 42]
 Left keys [1]: [ws_sold_date_sk#105]
 Right keys [1]: [d_date_sk#111]
 Join condition: None
 
-(109) Project [codegen id : 42]
+(110) Project [codegen id : 42]
 Output [9]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112]
 Input [11]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, ws_sold_date_sk#105, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_date_sk#111, d_year#112]
 
-(110) Exchange
+(111) Exchange
 Input [9]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112]
 Arguments: hashpartitioning(ws_order_number#102, ws_item_sk#101, 5), ENSURE_REQUIREMENTS, [plan_id=13]
 
-(111) Sort [codegen id : 43]
+(112) Sort [codegen id : 43]
 Input [9]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112]
 Arguments: [ws_order_number#102 ASC NULLS FIRST, ws_item_sk#101 ASC NULLS FIRST], false, 0
 
-(112) ReusedExchange [Reuses operator id: 58]
+(113) ReusedExchange [Reuses operator id: 58]
 Output [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116]
 
-(113) Sort [codegen id : 45]
+(114) Sort [codegen id : 45]
 Input [4]: [wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116]
 Arguments: [wr_order_number#114 ASC NULLS FIRST, wr_item_sk#113 ASC NULLS FIRST], false, 0
 
-(114) SortMergeJoin [codegen id : 46]
+(115) SortMergeJoin [codegen id : 46]
 Left keys [2]: [ws_order_number#102, ws_item_sk#101]
 Right keys [2]: [wr_order_number#114, wr_item_sk#113]
 Join condition: None
 
-(115) Project [codegen id : 46]
-Output [7]: [d_year#112, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, (ws_quantity#103 - coalesce(wr_return_quantity#115, 0)) AS sales_cnt#58, CheckOverflow((promote_precision(cast(ws_ext_sales_price#104 as decimal(8,2))) - promote_precision(cast(coalesce(wr_return_amt#116, 0.00) as decimal(8,2)))), DecimalType(8,2)) AS sales_amt#59]
+(116) Project [codegen id : 46]
+Output [7]: [d_year#112, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, (ws_quantity#103 - coalesce(wr_return_quantity#115, 0)) AS sales_cnt#58, (ws_ext_sales_price#104 - coalesce(wr_return_amt#116, 0.00)) AS sales_amt#59]
 Input [13]: [ws_item_sk#101, ws_order_number#102, ws_quantity#103, ws_ext_sales_price#104, i_brand_id#107, i_class_id#108, i_category_id#109, i_manufact_id#110, d_year#112, wr_item_sk#113, wr_order_number#114, wr_return_quantity#115, wr_return_amt#116]
 
-(116) Union
+(117) Union
 
-(117) HashAggregate [codegen id : 47]
+(118) HashAggregate [codegen id : 47]
 Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#20, sales_amt#21]
 Keys [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#20, sales_amt#21]
 Functions: []
 Aggregate Attributes: []
 Results [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#20, sales_amt#21]
 
-(118) Exchange
+(119) Exchange
 Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#20, sales_amt#21]
 Arguments: hashpartitioning(d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#20, sales_amt#21, 5), ENSURE_REQUIREMENTS, [plan_id=14]
 
-(119) HashAggregate [codegen id : 48]
+(120) HashAggregate [codegen id : 48]
 Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#20, sales_amt#21]
 Keys [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#20, sales_amt#21]
 Functions: []
 Aggregate Attributes: []
 Results [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#20, sales_amt#21]
 
-(120) HashAggregate [codegen id : 48]
+(121) HashAggregate [codegen id : 48]
 Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#20, sales_amt#21]
 Keys [5]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78]
 Functions [2]: [partial_sum(sales_cnt#20), partial_sum(UnscaledValue(sales_amt#21))]
 Aggregate Attributes [2]: [sum#60, sum#117]
 Results [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sum#62, sum#118]
 
-(121) Exchange
+(122) Exchange
 Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sum#62, sum#118]
 Arguments: hashpartitioning(d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, 5), ENSURE_REQUIREMENTS, [plan_id=15]
 
-(122) HashAggregate [codegen id : 49]
+(123) HashAggregate [codegen id : 49]
 Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sum#62, sum#118]
 Keys [5]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78]
 Functions [2]: [sum(sales_cnt#20), sum(UnscaledValue(sales_amt#21))]
 Aggregate Attributes [2]: [sum(sales_cnt#20)#64, sum(UnscaledValue(sales_amt#21))#65]
 Results [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sum(sales_cnt#20)#64 AS sales_cnt#119, MakeDecimal(sum(UnscaledValue(sales_amt#21))#65,18,2) AS sales_amt#120]
 
-(123) Exchange
+(124) Filter [codegen id : 49]
+Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#119, sales_amt#120]
+Condition : isnotnull(sales_cnt#119)
+
+(125) Exchange
 Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#119, sales_amt#120]
 Arguments: hashpartitioning(i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, 5), ENSURE_REQUIREMENTS, [plan_id=16]
 
-(124) Sort [codegen id : 50]
+(126) Sort [codegen id : 50]
 Input [7]: [d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#119, sales_amt#120]
 Arguments: [i_brand_id#75 ASC NULLS FIRST, i_class_id#76 ASC NULLS FIRST, i_category_id#77 ASC NULLS FIRST, i_manufact_id#78 ASC NULLS FIRST], false, 0
 
-(125) SortMergeJoin [codegen id : 51]
+(127) SortMergeJoin [codegen id : 51]
 Left keys [4]: [i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12]
 Right keys [4]: [i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78]
-Join condition: (CheckOverflow((promote_precision(cast(sales_cnt#66 as decimal(17,2))) / promote_precision(cast(sales_cnt#119 as decimal(17,2)))), DecimalType(37,20)) < 0.90000000000000000000)
+Join condition: ((cast(sales_cnt#66 as decimal(17,2)) / cast(sales_cnt#119 as decimal(17,2))) < 0.90000000000000000000)
 
-(126) Project [codegen id : 51]
-Output [10]: [d_year#80 AS prev_year#121, d_year#14 AS year#122, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#119 AS prev_yr_cnt#123, sales_cnt#66 AS curr_yr_cnt#124, (sales_cnt#66 - sales_cnt#119) AS sales_cnt_diff#125, CheckOverflow((promote_precision(cast(sales_amt#67 as decimal(19,2))) - promote_precision(cast(sales_amt#120 as decimal(19,2)))), DecimalType(19,2)) AS sales_amt_diff#126]
+(128) Project [codegen id : 51]
+Output [10]: [d_year#80 AS prev_year#121, d_year#14 AS year#122, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#119 AS prev_yr_cnt#123, sales_cnt#66 AS curr_yr_cnt#124, (sales_cnt#66 - sales_cnt#119) AS sales_cnt_diff#125, (sales_amt#67 - sales_amt#120) AS sales_amt_diff#126]
 Input [14]: [d_year#14, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, sales_cnt#66, sales_amt#67, d_year#80, i_brand_id#75, i_class_id#76, i_category_id#77, i_manufact_id#78, sales_cnt#119, sales_amt#120]
 
-(127) TakeOrderedAndProject
+(129) TakeOrderedAndProject
 Input [10]: [prev_year#121, year#122, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#123, curr_yr_cnt#124, sales_cnt_diff#125, sales_amt_diff#126]
 Arguments: 100, [sales_cnt_diff#125 ASC NULLS FIRST, sales_amt_diff#126 ASC NULLS FIRST], [prev_year#121, year#122, i_brand_id#8, i_class_id#9, i_category_id#10, i_manufact_id#12, prev_yr_cnt#123, curr_yr_cnt#124, sales_cnt_diff#125, sales_amt_diff#126]
 
 ===== Subqueries =====
 
 Subquery:1 Hosting operator id = 1 Hosting Expression = cs_sold_date_sk#5 IN dynamicpruning#6
-BroadcastExchange (131)
-+- * Filter (130)
-   +- * ColumnarToRow (129)
-      +- Scan parquet default.date_dim (128)
+BroadcastExchange (133)
++- * Filter (132)
+   +- * ColumnarToRow (131)
+      +- Scan parquet default.date_dim (130)
 
 
-(128) Scan parquet default.date_dim
+(130) Scan parquet default.date_dim
 Output [2]: [d_date_sk#13, d_year#14]
 Batched: true
 Location [not included in comparison]/{warehouse_dir}/date_dim]
 PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2002), IsNotNull(d_date_sk)]
 ReadSchema: struct<d_date_sk:int,d_year:int>
 
-(129) ColumnarToRow [codegen id : 1]
+(131) ColumnarToRow [codegen id : 1]
 Input [2]: [d_date_sk#13, d_year#14]
 
-(130) Filter [codegen id : 1]
+(132) Filter [codegen id : 1]
 Input [2]: [d_date_sk#13, d_year#14]
 Condition : ((isnotnull(d_year#14) AND (d_year#14 = 2002)) AND isnotnull(d_date_sk#13))
 
-(131) BroadcastExchange
+(133) BroadcastExchange
 Input [2]: [d_date_sk#13, d_year#14]
 Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=17]
 
@@ -718,33 +728,33 @@ Subquery:2 Hosting operator id = 24 Hosting Expression = ss_sold_date_sk#26 IN d
 
 Subquery:3 Hosting operator id = 43 Hosting Expression = ws_sold_date_sk#45 IN dynamicpruning#6
 
-Subquery:4 Hosting operator id = 71 Hosting Expression = cs_sold_date_sk#72 IN dynamicpruning#73
-BroadcastExchange (135)
-+- * Filter (134)
-   +- * ColumnarToRow (133)
-      +- Scan parquet default.date_dim (132)
+Subquery:4 Hosting operator id = 72 Hosting Expression = cs_sold_date_sk#72 IN dynamicpruning#73
+BroadcastExchange (137)
++- * Filter (136)
+   +- * ColumnarToRow (135)
+      +- Scan parquet default.date_dim (134)
 
 
-(132) Scan parquet default.date_dim
+(134) Scan parquet default.date_dim
 Output [2]: [d_date_sk#79, d_year#80]
 Batched: true
 Location [not included in comparison]/{warehouse_dir}/date_dim]
 PushedFilters: [IsNotNull(d_year), EqualTo(d_year,2001), IsNotNull(d_date_sk)]
 ReadSchema: struct<d_date_sk:int,d_year:int>
 
-(133) ColumnarToRow [codegen id : 1]
+(135) ColumnarToRow [codegen id : 1]
 Input [2]: [d_date_sk#79, d_year#80]
 
-(134) Filter [codegen id : 1]
+(136) Filter [codegen id : 1]
 Input [2]: [d_date_sk#79, d_year#80]
 Condition : ((isnotnull(d_year#80) AND (d_year#80 = 2001)) AND isnotnull(d_date_sk#79))
 
-(135) BroadcastExchange
+(137) BroadcastExchange
 Input [2]: [d_date_sk#79, d_year#80]
 Arguments: HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)),false), [plan_id=18]
 
-Subquery:5 Hosting operator id = 86 Hosting Expression = ss_sold_date_sk#89 IN dynamicpruning#73
+Subquery:5 Hosting operator id = 87 Hosting Expression = ss_sold_date_sk#89 IN dynamicpruning#73
 
-Subquery:6 Hosting operator id = 101 Hosting Expression = ws_sold_date_sk#105 IN dynamicpruning#73
+Subquery:6 Hosting operator id = 102 Hosting Expression = ws_sold_date_sk#105 IN dynamicpruning#73
 
 
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.sf100/simplified.txt
index 4c10fbcb03a..415534326d5 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.sf100/simplified.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q75.sf100/simplified.txt
@@ -8,231 +8,233 @@ TakeOrderedAndProject [sales_cnt_diff,sales_amt_diff,prev_year,year,i_brand_id,i
               InputAdapter
                 Exchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #1
                   WholeStageCodegen (24)
-                    HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] [sum(sales_cnt),sum(UnscaledValue(sales_amt)),sales_cnt,sales_amt,sum,sum]
-                      InputAdapter
-                        Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #2
-                          WholeStageCodegen (23)
-                            HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] [sum,sum,sum,sum]
-                              HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt]
-                                InputAdapter
-                                  Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #3
-                                    WholeStageCodegen (22)
-                                      HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt]
-                                        InputAdapter
-                                          Union
-                                            WholeStageCodegen (7)
-                                              Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount]
-                                                SortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk]
-                                                  InputAdapter
-                                                    WholeStageCodegen (4)
-                                                      Sort [cs_order_number,cs_item_sk]
-                                                        InputAdapter
-                                                          Exchange [cs_order_number,cs_item_sk] #4
-                                                            WholeStageCodegen (3)
-                                                              Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year]
-                                                                BroadcastHashJoin [cs_sold_date_sk,d_date_sk]
-                                                                  Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
-                                                                    BroadcastHashJoin [cs_item_sk,i_item_sk]
-                                                                      Filter [cs_item_sk]
-                                                                        ColumnarToRow
-                                                                          InputAdapter
-                                                                            Scan parquet default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk]
-                                                                              SubqueryBroadcast [d_date_sk] #1
-                                                                                BroadcastExchange #5
-                                                                                  WholeStageCodegen (1)
-                                                                                    Filter [d_year,d_date_sk]
-                                                                                      ColumnarToRow
-                                                                                        InputAdapter
-                                                                                          Scan parquet default.date_dim [d_date_sk,d_year]
-                                                                      InputAdapter
-                                                                        BroadcastExchange #6
-                                                                          WholeStageCodegen (1)
-                                                                            Project [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
-                                                                              Filter [i_category,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
-                                                                                ColumnarToRow
-                                                                                  InputAdapter
-                                                                                    Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id]
-                                                                  InputAdapter
-                                                                    ReusedExchange [d_date_sk,d_year] #5
-                                                  InputAdapter
-                                                    WholeStageCodegen (6)
-                                                      Sort [cr_order_number,cr_item_sk]
-                                                        InputAdapter
-                                                          Exchange [cr_order_number,cr_item_sk] #7
-                                                            WholeStageCodegen (5)
-                                                              Project [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount]
-                                                                Filter [cr_order_number,cr_item_sk]
-                                                                  ColumnarToRow
+                    Filter [sales_cnt]
+                      HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] [sum(sales_cnt),sum(UnscaledValue(sales_amt)),sales_cnt,sales_amt,sum,sum]
+                        InputAdapter
+                          Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #2
+                            WholeStageCodegen (23)
+                              HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] [sum,sum,sum,sum]
+                                HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt]
+                                  InputAdapter
+                                    Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #3
+                                      WholeStageCodegen (22)
+                                        HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt]
+                                          InputAdapter
+                                            Union
+                                              WholeStageCodegen (7)
+                                                Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount]
+                                                  SortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk]
+                                                    InputAdapter
+                                                      WholeStageCodegen (4)
+                                                        Sort [cs_order_number,cs_item_sk]
+                                                          InputAdapter
+                                                            Exchange [cs_order_number,cs_item_sk] #4
+                                                              WholeStageCodegen (3)
+                                                                Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year]
+                                                                  BroadcastHashJoin [cs_sold_date_sk,d_date_sk]
+                                                                    Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
+                                                                      BroadcastHashJoin [cs_item_sk,i_item_sk]
+                                                                        Filter [cs_item_sk]
+                                                                          ColumnarToRow
+                                                                            InputAdapter
+                                                                              Scan parquet default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk]
+                                                                                SubqueryBroadcast [d_date_sk] #1
+                                                                                  BroadcastExchange #5
+                                                                                    WholeStageCodegen (1)
+                                                                                      Filter [d_year,d_date_sk]
+                                                                                        ColumnarToRow
+                                                                                          InputAdapter
+                                                                                            Scan parquet default.date_dim [d_date_sk,d_year]
+                                                                        InputAdapter
+                                                                          BroadcastExchange #6
+                                                                            WholeStageCodegen (1)
+                                                                              Project [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
+                                                                                Filter [i_category,i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
+                                                                                  ColumnarToRow
+                                                                                    InputAdapter
+                                                                                      Scan parquet default.item [i_item_sk,i_brand_id,i_class_id,i_category_id,i_category,i_manufact_id]
                                                                     InputAdapter
-                                                                      Scan parquet default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk]
-                                            WholeStageCodegen (14)
-                                              Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt]
-                                                SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk]
-                                                  InputAdapter
-                                                    WholeStageCodegen (11)
-                                                      Sort [ss_ticket_number,ss_item_sk]
-                                                        InputAdapter
-                                                          Exchange [ss_ticket_number,ss_item_sk] #8
-                                                            WholeStageCodegen (10)
-                                                              Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year]
-                                                                BroadcastHashJoin [ss_sold_date_sk,d_date_sk]
-                                                                  Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
-                                                                    BroadcastHashJoin [ss_item_sk,i_item_sk]
-                                                                      Filter [ss_item_sk]
-                                                                        ColumnarToRow
-                                                                          InputAdapter
-                                                                            Scan parquet default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk]
-                                                                              ReusedSubquery [d_date_sk] #1
+                                                                      ReusedExchange [d_date_sk,d_year] #5
+                                                    InputAdapter
+                                                      WholeStageCodegen (6)
+                                                        Sort [cr_order_number,cr_item_sk]
+                                                          InputAdapter
+                                                            Exchange [cr_order_number,cr_item_sk] #7
+                                                              WholeStageCodegen (5)
+                                                                Project [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount]
+                                                                  Filter [cr_order_number,cr_item_sk]
+                                                                    ColumnarToRow
                                                                       InputAdapter
-                                                                        ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6
-                                                                  InputAdapter
-                                                                    ReusedExchange [d_date_sk,d_year] #5
-                                                  InputAdapter
-                                                    WholeStageCodegen (13)
-                                                      Sort [sr_ticket_number,sr_item_sk]
-                                                        InputAdapter
-                                                          Exchange [sr_ticket_number,sr_item_sk] #9
-                                                            WholeStageCodegen (12)
-                                                              Project [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt]
-                                                                Filter [sr_ticket_number,sr_item_sk]
-                                                                  ColumnarToRow
+                                                                        Scan parquet default.catalog_returns [cr_item_sk,cr_order_number,cr_return_quantity,cr_return_amount,cr_returned_date_sk]
+                                              WholeStageCodegen (14)
+                                                Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ss_quantity,sr_return_quantity,ss_ext_sales_price,sr_return_amt]
+                                                  SortMergeJoin [ss_ticket_number,ss_item_sk,sr_ticket_number,sr_item_sk]
+                                                    InputAdapter
+                                                      WholeStageCodegen (11)
+                                                        Sort [ss_ticket_number,ss_item_sk]
+                                                          InputAdapter
+                                                            Exchange [ss_ticket_number,ss_item_sk] #8
+                                                              WholeStageCodegen (10)
+                                                                Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year]
+                                                                  BroadcastHashJoin [ss_sold_date_sk,d_date_sk]
+                                                                    Project [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
+                                                                      BroadcastHashJoin [ss_item_sk,i_item_sk]
+                                                                        Filter [ss_item_sk]
+                                                                          ColumnarToRow
+                                                                            InputAdapter
+                                                                              Scan parquet default.store_sales [ss_item_sk,ss_ticket_number,ss_quantity,ss_ext_sales_price,ss_sold_date_sk]
+                                                                                ReusedSubquery [d_date_sk] #1
+                                                                        InputAdapter
+                                                                          ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6
                                                                     InputAdapter
-                                                                      Scan parquet default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk]
-                                            WholeStageCodegen (21)
-                                              Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt]
-                                                SortMergeJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk]
-                                                  InputAdapter
-                                                    WholeStageCodegen (18)
-                                                      Sort [ws_order_number,ws_item_sk]
-                                                        InputAdapter
-                                                          Exchange [ws_order_number,ws_item_sk] #10
-                                                            WholeStageCodegen (17)
-                                                              Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year]
-                                                                BroadcastHashJoin [ws_sold_date_sk,d_date_sk]
-                                                                  Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
-                                                                    BroadcastHashJoin [ws_item_sk,i_item_sk]
-                                                                      Filter [ws_item_sk]
-                                                                        ColumnarToRow
-                                                                          InputAdapter
-                                                                            Scan parquet default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk]
-                                                                              ReusedSubquery [d_date_sk] #1
+                                                                      ReusedExchange [d_date_sk,d_year] #5
+                                                    InputAdapter
+                                                      WholeStageCodegen (13)
+                                                        Sort [sr_ticket_number,sr_item_sk]
+                                                          InputAdapter
+                                                            Exchange [sr_ticket_number,sr_item_sk] #9
+                                                              WholeStageCodegen (12)
+                                                                Project [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt]
+                                                                  Filter [sr_ticket_number,sr_item_sk]
+                                                                    ColumnarToRow
                                                                       InputAdapter
-                                                                        ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6
-                                                                  InputAdapter
-                                                                    ReusedExchange [d_date_sk,d_year] #5
-                                                  InputAdapter
-                                                    WholeStageCodegen (20)
-                                                      Sort [wr_order_number,wr_item_sk]
-                                                        InputAdapter
-                                                          Exchange [wr_order_number,wr_item_sk] #11
-                                                            WholeStageCodegen (19)
-                                                              Project [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt]
-                                                                Filter [wr_order_number,wr_item_sk]
-                                                                  ColumnarToRow
+                                                                        Scan parquet default.store_returns [sr_item_sk,sr_ticket_number,sr_return_quantity,sr_return_amt,sr_returned_date_sk]
+                                              WholeStageCodegen (21)
+                                                Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,ws_quantity,wr_return_quantity,ws_ext_sales_price,wr_return_amt]
+                                                  SortMergeJoin [ws_order_number,ws_item_sk,wr_order_number,wr_item_sk]
+                                                    InputAdapter
+                                                      WholeStageCodegen (18)
+                                                        Sort [ws_order_number,ws_item_sk]
+                                                          InputAdapter
+                                                            Exchange [ws_order_number,ws_item_sk] #10
+                                                              WholeStageCodegen (17)
+                                                                Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year]
+                                                                  BroadcastHashJoin [ws_sold_date_sk,d_date_sk]
+                                                                    Project [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
+                                                                      BroadcastHashJoin [ws_item_sk,i_item_sk]
+                                                                        Filter [ws_item_sk]
+                                                                          ColumnarToRow
+                                                                            InputAdapter
+                                                                              Scan parquet default.web_sales [ws_item_sk,ws_order_number,ws_quantity,ws_ext_sales_price,ws_sold_date_sk]
+                                                                                ReusedSubquery [d_date_sk] #1
+                                                                        InputAdapter
+                                                                          ReusedExchange [i_item_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id] #6
                                                                     InputAdapter
-                                                                      Scan parquet default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk]
+                                                                      ReusedExchange [d_date_sk,d_year] #5
+                                                    InputAdapter
+                                                      WholeStageCodegen (20)
+                                                        Sort [wr_order_number,wr_item_sk]
+                                                          InputAdapter
+                                                            Exchange [wr_order_number,wr_item_sk] #11
+                                                              WholeStageCodegen (19)
+                                                                Project [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt]
+                                                                  Filter [wr_order_number,wr_item_sk]
+                                                                    ColumnarToRow
+                                                                      InputAdapter
+                                                                        Scan parquet default.web_returns [wr_item_sk,wr_order_number,wr_return_quantity,wr_return_amt,wr_returned_date_sk]
         InputAdapter
           WholeStageCodegen (50)
             Sort [i_brand_id,i_class_id,i_category_id,i_manufact_id]
               InputAdapter
                 Exchange [i_brand_id,i_class_id,i_category_id,i_manufact_id] #12
                   WholeStageCodegen (49)
-                    HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sum,sum] [sum(sales_cnt),sum(UnscaledValue(sales_amt)),sales_cnt,sales_amt,sum,sum]
-                      InputAdapter
-                        Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id] #13
-                          WholeStageCodegen (48)
-                            HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] [sum,sum,sum,sum]
-                              HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt]
-                                InputAdapter
-                                  Exchange [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt] #14
-                                    WholeStageCodegen (47)
-                                      HashAggregate [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,sales_cnt,sales_amt]
-                                        InputAdapter
-                                          Union
-                                            WholeStageCodegen (32)
-                                              Project [d_year,i_brand_id,i_class_id,i_category_id,i_manufact_id,cs_quantity,cr_return_quantity,cs_ext_sales_price,cr_return_amount]
-                                                SortMergeJoin [cs_order_number,cs_item_sk,cr_order_number,cr_item_sk]
-                                                  InputAdapter
-                                                    WholeStageCodegen (29)
-                                                      Sort [cs_order_number,cs_item_sk]
-                                                        InputAdapter
-                                                          Exchange [cs_order_number,cs_item_sk] #15
-                                                            WholeStageCodegen (28)
-                                                              Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,i_brand_id,i_class_id,i_category_id,i_manufact_id,d_year]
-                                                                BroadcastHashJoin [cs_sold_date_sk,d_date_sk]
-                                                                  Project [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk,i_brand_id,i_class_id,i_category_id,i_manufact_id]
-                                                                    BroadcastHashJoin [cs_item_sk,i_item_sk]
-                                                                      Filter [cs_item_sk]
-                                                                        ColumnarToRow
-                                                                          InputAdapter
-                                                                            Scan parquet default.catalog_sales [cs_item_sk,cs_order_number,cs_quantity,cs_ext_sales_price,cs_sold_date_sk]
-                                                                              SubqueryBroadcast [d_date_sk] #2
... 2322 lines suppressed ...


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org