aboutsummaryrefslogtreecommitdiff
diff options
context:
space:
mode:
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala6
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala (renamed from sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala)6
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala10
-rw-r--r--sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala (renamed from sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala)72
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchema.scala6
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/InferSchema.scala4
6 files changed, 50 insertions, 54 deletions
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
index ce941e3df3..1313a011c6 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
@@ -105,7 +105,7 @@ class Analyzer(
GlobalAggregates ::
ResolveAggregateFunctions ::
TimeWindowing ::
- HiveTypeCoercion.typeCoercionRules ++
+ TypeCoercion.typeCoercionRules ++
extendedResolutionRules : _*),
Batch("Nondeterministic", Once,
PullOutNondeterministic),
@@ -1922,8 +1922,8 @@ class Analyzer(
}
private def illegalNumericPrecedence(from: DataType, to: DataType): Boolean = {
- val fromPrecedence = HiveTypeCoercion.numericPrecedence.indexOf(from)
- val toPrecedence = HiveTypeCoercion.numericPrecedence.indexOf(to)
+ val fromPrecedence = TypeCoercion.numericPrecedence.indexOf(from)
+ val toPrecedence = TypeCoercion.numericPrecedence.indexOf(to)
toPrecedence > 0 && fromPrecedence > toPrecedence
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala
index 537dda60af..91bdcc3b09 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala
@@ -24,7 +24,6 @@ import scala.collection.mutable
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.expressions.aggregate._
-import org.apache.spark.sql.catalyst.expressions.Literal.{FalseLiteral, TrueLiteral}
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.types._
@@ -34,9 +33,6 @@ import org.apache.spark.sql.types._
* A collection of [[Rule]] that can be used to coerce differing types that participate in
* operations into compatible ones.
*
- * Most of these rules are based on Hive semantics, but they do not introduce any dependencies on
- * the hive codebase.
- *
* Notes about type widening / tightest common types: Broadly, there are two cases when we need
* to widen data types (e.g. union, binary comparison). In case 1, we are looking for a common
* data type for two or more data types, and in this case no loss of precision is allowed. Examples
@@ -46,7 +42,7 @@ import org.apache.spark.sql.types._
* double's range is larger than decimal, and yet decimal is more precise than double, but in
* union we would cast the decimal into double).
*/
-object HiveTypeCoercion {
+object TypeCoercion {
val typeCoercionRules =
PropagateTypes ::
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala
index 641c81b247..523fb05397 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala
@@ -18,7 +18,7 @@
package org.apache.spark.sql.catalyst.expressions
import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.catalyst.analysis.{HiveTypeCoercion, TypeCheckResult}
+import org.apache.spark.sql.catalyst.analysis.{TypeCheckResult, TypeCoercion}
import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode}
import org.apache.spark.sql.catalyst.util.TypeUtils
import org.apache.spark.sql.types._
@@ -96,7 +96,7 @@ case class IfNull(left: Expression, right: Expression) extends RuntimeReplaceabl
override def replaceForTypeCoercion(): Expression = {
if (left.dataType != right.dataType) {
- HiveTypeCoercion.findTightestCommonTypeOfTwo(left.dataType, right.dataType).map { dtype =>
+ TypeCoercion.findTightestCommonTypeOfTwo(left.dataType, right.dataType).map { dtype =>
copy(left = Cast(left, dtype), right = Cast(right, dtype))
}.getOrElse(this)
} else {
@@ -116,7 +116,7 @@ case class NullIf(left: Expression, right: Expression) extends RuntimeReplaceabl
override def replaceForTypeCoercion(): Expression = {
if (left.dataType != right.dataType) {
- HiveTypeCoercion.findTightestCommonTypeOfTwo(left.dataType, right.dataType).map { dtype =>
+ TypeCoercion.findTightestCommonTypeOfTwo(left.dataType, right.dataType).map { dtype =>
copy(left = Cast(left, dtype), right = Cast(right, dtype))
}.getOrElse(this)
} else {
@@ -134,7 +134,7 @@ case class Nvl(left: Expression, right: Expression) extends RuntimeReplaceable {
override def replaceForTypeCoercion(): Expression = {
if (left.dataType != right.dataType) {
- HiveTypeCoercion.findTightestCommonTypeOfTwo(left.dataType, right.dataType).map { dtype =>
+ TypeCoercion.findTightestCommonTypeOfTwo(left.dataType, right.dataType).map { dtype =>
copy(left = Cast(left, dtype), right = Cast(right, dtype))
}.getOrElse(this)
} else {
@@ -154,7 +154,7 @@ case class Nvl2(expr1: Expression, expr2: Expression, expr3: Expression)
override def replaceForTypeCoercion(): Expression = {
if (expr2.dataType != expr3.dataType) {
- HiveTypeCoercion.findTightestCommonTypeOfTwo(expr2.dataType, expr3.dataType).map { dtype =>
+ TypeCoercion.findTightestCommonTypeOfTwo(expr2.dataType, expr3.dataType).map { dtype =>
copy(expr2 = Cast(expr2, dtype), expr3 = Cast(expr3, dtype))
}.getOrElse(this)
} else {
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala
index b591861ac0..7435399b14 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala
@@ -26,11 +26,11 @@ import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.types._
import org.apache.spark.unsafe.types.CalendarInterval
-class HiveTypeCoercionSuite extends PlanTest {
+class TypeCoercionSuite extends PlanTest {
test("eligible implicit type cast") {
def shouldCast(from: DataType, to: AbstractDataType, expected: DataType): Unit = {
- val got = HiveTypeCoercion.ImplicitTypeCasts.implicitCast(Literal.create(null, from), to)
+ val got = TypeCoercion.ImplicitTypeCasts.implicitCast(Literal.create(null, from), to)
assert(got.map(_.dataType) == Option(expected),
s"Failed to cast $from to $to")
}
@@ -101,7 +101,7 @@ class HiveTypeCoercionSuite extends PlanTest {
test("ineligible implicit type cast") {
def shouldNotCast(from: DataType, to: AbstractDataType): Unit = {
- val got = HiveTypeCoercion.ImplicitTypeCasts.implicitCast(Literal.create(null, from), to)
+ val got = TypeCoercion.ImplicitTypeCasts.implicitCast(Literal.create(null, from), to)
assert(got.isEmpty, s"Should not be able to cast $from to $to, but got $got")
}
@@ -129,11 +129,11 @@ class HiveTypeCoercionSuite extends PlanTest {
test("tightest common bound for types") {
def widenTest(t1: DataType, t2: DataType, tightestCommon: Option[DataType]) {
- var found = HiveTypeCoercion.findTightestCommonTypeOfTwo(t1, t2)
+ var found = TypeCoercion.findTightestCommonTypeOfTwo(t1, t2)
assert(found == tightestCommon,
s"Expected $tightestCommon as tightest common type for $t1 and $t2, found $found")
// Test both directions to make sure the widening is symmetric.
- found = HiveTypeCoercion.findTightestCommonTypeOfTwo(t2, t1)
+ found = TypeCoercion.findTightestCommonTypeOfTwo(t2, t1)
assert(found == tightestCommon,
s"Expected $tightestCommon as tightest common type for $t2 and $t1, found $found")
}
@@ -206,31 +206,31 @@ class HiveTypeCoercionSuite extends PlanTest {
}
test("cast NullType for expressions that implement ExpectsInputTypes") {
- import HiveTypeCoercionSuite._
+ import TypeCoercionSuite._
- ruleTest(HiveTypeCoercion.ImplicitTypeCasts,
+ ruleTest(TypeCoercion.ImplicitTypeCasts,
AnyTypeUnaryExpression(Literal.create(null, NullType)),
AnyTypeUnaryExpression(Literal.create(null, NullType)))
- ruleTest(HiveTypeCoercion.ImplicitTypeCasts,
+ ruleTest(TypeCoercion.ImplicitTypeCasts,
NumericTypeUnaryExpression(Literal.create(null, NullType)),
NumericTypeUnaryExpression(Literal.create(null, DoubleType)))
}
test("cast NullType for binary operators") {
- import HiveTypeCoercionSuite._
+ import TypeCoercionSuite._
- ruleTest(HiveTypeCoercion.ImplicitTypeCasts,
+ ruleTest(TypeCoercion.ImplicitTypeCasts,
AnyTypeBinaryOperator(Literal.create(null, NullType), Literal.create(null, NullType)),
AnyTypeBinaryOperator(Literal.create(null, NullType), Literal.create(null, NullType)))
- ruleTest(HiveTypeCoercion.ImplicitTypeCasts,
+ ruleTest(TypeCoercion.ImplicitTypeCasts,
NumericTypeBinaryOperator(Literal.create(null, NullType), Literal.create(null, NullType)),
NumericTypeBinaryOperator(Literal.create(null, DoubleType), Literal.create(null, DoubleType)))
}
test("coalesce casts") {
- ruleTest(HiveTypeCoercion.FunctionArgumentConversion,
+ ruleTest(TypeCoercion.FunctionArgumentConversion,
Coalesce(Literal(1.0)
:: Literal(1)
:: Literal.create(1.0, FloatType)
@@ -239,7 +239,7 @@ class HiveTypeCoercionSuite extends PlanTest {
:: Cast(Literal(1), DoubleType)
:: Cast(Literal.create(1.0, FloatType), DoubleType)
:: Nil))
- ruleTest(HiveTypeCoercion.FunctionArgumentConversion,
+ ruleTest(TypeCoercion.FunctionArgumentConversion,
Coalesce(Literal(1L)
:: Literal(1)
:: Literal(new java.math.BigDecimal("1000000000000000000000"))
@@ -251,7 +251,7 @@ class HiveTypeCoercionSuite extends PlanTest {
}
test("CreateArray casts") {
- ruleTest(HiveTypeCoercion.FunctionArgumentConversion,
+ ruleTest(TypeCoercion.FunctionArgumentConversion,
CreateArray(Literal(1.0)
:: Literal(1)
:: Literal.create(1.0, FloatType)
@@ -261,7 +261,7 @@ class HiveTypeCoercionSuite extends PlanTest {
:: Cast(Literal.create(1.0, FloatType), DoubleType)
:: Nil))
- ruleTest(HiveTypeCoercion.FunctionArgumentConversion,
+ ruleTest(TypeCoercion.FunctionArgumentConversion,
CreateArray(Literal(1.0)
:: Literal(1)
:: Literal("a")
@@ -274,7 +274,7 @@ class HiveTypeCoercionSuite extends PlanTest {
test("CreateMap casts") {
// type coercion for map keys
- ruleTest(HiveTypeCoercion.FunctionArgumentConversion,
+ ruleTest(TypeCoercion.FunctionArgumentConversion,
CreateMap(Literal(1)
:: Literal("a")
:: Literal.create(2.0, FloatType)
@@ -286,7 +286,7 @@ class HiveTypeCoercionSuite extends PlanTest {
:: Literal("b")
:: Nil))
// type coercion for map values
- ruleTest(HiveTypeCoercion.FunctionArgumentConversion,
+ ruleTest(TypeCoercion.FunctionArgumentConversion,
CreateMap(Literal(1)
:: Literal("a")
:: Literal(2)
@@ -298,7 +298,7 @@ class HiveTypeCoercionSuite extends PlanTest {
:: Cast(Literal(3.0), StringType)
:: Nil))
// type coercion for both map keys and values
- ruleTest(HiveTypeCoercion.FunctionArgumentConversion,
+ ruleTest(TypeCoercion.FunctionArgumentConversion,
CreateMap(Literal(1)
:: Literal("a")
:: Literal(2.0)
@@ -313,7 +313,7 @@ class HiveTypeCoercionSuite extends PlanTest {
test("greatest/least cast") {
for (operator <- Seq[(Seq[Expression] => Expression)](Greatest, Least)) {
- ruleTest(HiveTypeCoercion.FunctionArgumentConversion,
+ ruleTest(TypeCoercion.FunctionArgumentConversion,
operator(Literal(1.0)
:: Literal(1)
:: Literal.create(1.0, FloatType)
@@ -322,7 +322,7 @@ class HiveTypeCoercionSuite extends PlanTest {
:: Cast(Literal(1), DoubleType)
:: Cast(Literal.create(1.0, FloatType), DoubleType)
:: Nil))
- ruleTest(HiveTypeCoercion.FunctionArgumentConversion,
+ ruleTest(TypeCoercion.FunctionArgumentConversion,
operator(Literal(1L)
:: Literal(1)
:: Literal(new java.math.BigDecimal("1000000000000000000000"))
@@ -335,19 +335,19 @@ class HiveTypeCoercionSuite extends PlanTest {
}
test("nanvl casts") {
- ruleTest(HiveTypeCoercion.FunctionArgumentConversion,
+ ruleTest(TypeCoercion.FunctionArgumentConversion,
NaNvl(Literal.create(1.0, FloatType), Literal.create(1.0, DoubleType)),
NaNvl(Cast(Literal.create(1.0, FloatType), DoubleType), Literal.create(1.0, DoubleType)))
- ruleTest(HiveTypeCoercion.FunctionArgumentConversion,
+ ruleTest(TypeCoercion.FunctionArgumentConversion,
NaNvl(Literal.create(1.0, DoubleType), Literal.create(1.0, FloatType)),
NaNvl(Literal.create(1.0, DoubleType), Cast(Literal.create(1.0, FloatType), DoubleType)))
- ruleTest(HiveTypeCoercion.FunctionArgumentConversion,
+ ruleTest(TypeCoercion.FunctionArgumentConversion,
NaNvl(Literal.create(1.0, DoubleType), Literal.create(1.0, DoubleType)),
NaNvl(Literal.create(1.0, DoubleType), Literal.create(1.0, DoubleType)))
}
test("type coercion for If") {
- val rule = HiveTypeCoercion.IfCoercion
+ val rule = TypeCoercion.IfCoercion
ruleTest(rule,
If(Literal(true), Literal(1), Literal(1L)),
@@ -367,20 +367,20 @@ class HiveTypeCoercionSuite extends PlanTest {
}
test("type coercion for CaseKeyWhen") {
- ruleTest(HiveTypeCoercion.ImplicitTypeCasts,
+ ruleTest(TypeCoercion.ImplicitTypeCasts,
CaseKeyWhen(Literal(1.toShort), Seq(Literal(1), Literal("a"))),
CaseKeyWhen(Cast(Literal(1.toShort), IntegerType), Seq(Literal(1), Literal("a")))
)
- ruleTest(HiveTypeCoercion.CaseWhenCoercion,
+ ruleTest(TypeCoercion.CaseWhenCoercion,
CaseKeyWhen(Literal(true), Seq(Literal(1), Literal("a"))),
CaseKeyWhen(Literal(true), Seq(Literal(1), Literal("a")))
)
- ruleTest(HiveTypeCoercion.CaseWhenCoercion,
+ ruleTest(TypeCoercion.CaseWhenCoercion,
CaseWhen(Seq((Literal(true), Literal(1.2))), Literal.create(1, DecimalType(7, 2))),
CaseWhen(Seq((Literal(true), Literal(1.2))),
Cast(Literal.create(1, DecimalType(7, 2)), DoubleType))
)
- ruleTest(HiveTypeCoercion.CaseWhenCoercion,
+ ruleTest(TypeCoercion.CaseWhenCoercion,
CaseWhen(Seq((Literal(true), Literal(100L))), Literal.create(1, DecimalType(7, 2))),
CaseWhen(Seq((Literal(true), Cast(Literal(100L), DecimalType(22, 2)))),
Cast(Literal.create(1, DecimalType(7, 2)), DecimalType(22, 2)))
@@ -388,7 +388,7 @@ class HiveTypeCoercionSuite extends PlanTest {
}
test("BooleanEquality type cast") {
- val be = HiveTypeCoercion.BooleanEquality
+ val be = TypeCoercion.BooleanEquality
// Use something more than a literal to avoid triggering the simplification rules.
val one = Add(Literal(Decimal(1)), Literal(Decimal(0)))
@@ -414,7 +414,7 @@ class HiveTypeCoercionSuite extends PlanTest {
}
test("BooleanEquality simplification") {
- val be = HiveTypeCoercion.BooleanEquality
+ val be = TypeCoercion.BooleanEquality
ruleTest(be,
EqualTo(Literal(true), Literal(1)),
@@ -473,7 +473,7 @@ class HiveTypeCoercionSuite extends PlanTest {
AttributeReference("f", FloatType)(),
AttributeReference("l", LongType)())
- val wt = HiveTypeCoercion.WidenSetOperationTypes
+ val wt = TypeCoercion.WidenSetOperationTypes
val expectedTypes = Seq(StringType, DecimalType.SYSTEM_DEFAULT, FloatType, DoubleType)
val r1 = wt(Except(firstTable, secondTable)).asInstanceOf[Except]
@@ -512,7 +512,7 @@ class HiveTypeCoercionSuite extends PlanTest {
AttributeReference("p", ByteType)(),
AttributeReference("q", DoubleType)())
- val wt = HiveTypeCoercion.WidenSetOperationTypes
+ val wt = TypeCoercion.WidenSetOperationTypes
val expectedTypes = Seq(StringType, DecimalType.SYSTEM_DEFAULT, FloatType, DoubleType)
val unionRelation = wt(
@@ -536,7 +536,7 @@ class HiveTypeCoercionSuite extends PlanTest {
}
}
- val dp = HiveTypeCoercion.WidenSetOperationTypes
+ val dp = TypeCoercion.WidenSetOperationTypes
val left1 = LocalRelation(
AttributeReference("l", DecimalType(10, 8))())
@@ -584,7 +584,7 @@ class HiveTypeCoercionSuite extends PlanTest {
}
test("rule for date/timestamp operations") {
- val dateTimeOperations = HiveTypeCoercion.DateTimeOperations
+ val dateTimeOperations = TypeCoercion.DateTimeOperations
val date = Literal(new java.sql.Date(0L))
val timestamp = Literal(new Timestamp(0L))
val interval = Literal(new CalendarInterval(0, 0))
@@ -615,7 +615,7 @@ class HiveTypeCoercionSuite extends PlanTest {
*/
test("make sure rules do not fire early") {
// InConversion
- val inConversion = HiveTypeCoercion.InConversion
+ val inConversion = TypeCoercion.InConversion
ruleTest(inConversion,
In(UnresolvedAttribute("a"), Seq(Literal(1))),
In(UnresolvedAttribute("a"), Seq(Literal(1)))
@@ -633,7 +633,7 @@ class HiveTypeCoercionSuite extends PlanTest {
}
-object HiveTypeCoercionSuite {
+object TypeCoercionSuite {
case class AnyTypeUnaryExpression(child: Expression)
extends UnaryExpression with ExpectsInputTypes with Unevaluable {
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchema.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchema.scala
index 05c8d8ee15..de3d889621 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchema.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchema.scala
@@ -25,7 +25,7 @@ import scala.util.control.Exception._
import scala.util.Try
import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.catalyst.analysis.HiveTypeCoercion
+import org.apache.spark.sql.catalyst.analysis.TypeCoercion
import org.apache.spark.sql.catalyst.util.DateTimeUtils
import org.apache.spark.sql.types._
import org.apache.spark.unsafe.types.UTF8String
@@ -171,11 +171,11 @@ private[csv] object CSVInferSchema {
StringType
}
- private val numericPrecedence: IndexedSeq[DataType] = HiveTypeCoercion.numericPrecedence
+ private val numericPrecedence: IndexedSeq[DataType] = TypeCoercion.numericPrecedence
/**
* Copied from internal Spark api
- * [[org.apache.spark.sql.catalyst.analysis.HiveTypeCoercion]]
+ * [[org.apache.spark.sql.catalyst.analysis.TypeCoercion]]
*/
val findTightestCommonType: (DataType, DataType) => Option[DataType] = {
case (t1, t2) if t1 == t2 => Some(t1)
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/InferSchema.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/InferSchema.scala
index 42c82625fa..579b036417 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/InferSchema.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/InferSchema.scala
@@ -22,7 +22,7 @@ import java.util.Comparator
import com.fasterxml.jackson.core._
import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.catalyst.analysis.HiveTypeCoercion
+import org.apache.spark.sql.catalyst.analysis.TypeCoercion
import org.apache.spark.sql.execution.datasources.json.JacksonUtils.nextUntil
import org.apache.spark.sql.types._
import org.apache.spark.util.Utils
@@ -252,7 +252,7 @@ private[sql] object InferSchema {
* Returns the most general data type for two given data types.
*/
def compatibleType(t1: DataType, t2: DataType): DataType = {
- HiveTypeCoercion.findTightestCommonTypeOfTwo(t1, t2).getOrElse {
+ TypeCoercion.findTightestCommonTypeOfTwo(t1, t2).getOrElse {
// t1 or t2 is a StructType, ArrayType, or an unexpected type.
(t1, t2) match {
// Double support larger range than fixed decimal, DecimalType.Maximum should be enough