-
Notifications
You must be signed in to change notification settings - Fork 254
fix: Fall back to Spark for MakeDecimal with unsupported input type #2815
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
base: main
Are you sure you want to change the base?
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change | ||||
|---|---|---|---|---|---|---|
|
|
@@ -38,6 +38,14 @@ object CometUnscaledValue extends CometExpressionSerde[UnscaledValue] { | |||||
| } | ||||||
|
|
||||||
| object CometMakeDecimal extends CometExpressionSerde[MakeDecimal] { | ||||||
|
|
||||||
| override def getSupportLevel(expr: MakeDecimal): SupportLevel = { | ||||||
| expr.child.dataType match { | ||||||
| case _: LongType => Compatible() | ||||||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Suggested change
because LongType is an object/singleton |
||||||
| case other => Unsupported(Some(s"Unsupported input data type: $other")) | ||||||
| } | ||||||
| } | ||||||
|
|
||||||
| override def convert( | ||||||
| expr: MakeDecimal, | ||||||
| inputs: Seq[Attribute], | ||||||
|
|
||||||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -29,6 +29,7 @@ import org.scalatest.Tag | |
| import org.apache.hadoop.fs.Path | ||
| import org.apache.spark.sql.{CometTestBase, DataFrame, Row} | ||
| import org.apache.spark.sql.catalyst.expressions.{Alias, Cast, FromUnixTime, Literal, TruncDate, TruncTimestamp} | ||
| import org.apache.spark.sql.catalyst.expressions.aggregate.Sum | ||
| import org.apache.spark.sql.catalyst.optimizer.SimplifyExtractValueOps | ||
| import org.apache.spark.sql.comet.{CometColumnarToRowExec, CometProjectExec} | ||
| import org.apache.spark.sql.execution.{InputAdapter, ProjectExec, SparkPlan, WholeStageCodegenExec} | ||
|
|
@@ -3187,4 +3188,30 @@ class CometExpressionSuite extends CometTestBase with AdaptiveSparkPlanHelper { | |
| CometConcat.unsupportedReason) | ||
| } | ||
| } | ||
|
|
||
| // https://github.com/apache/datafusion-comet/issues/2813 | ||
| test("make decimal using DataFrame API") { | ||
| withTable("t1") { | ||
| sql("create table t1 using parquet as select 123456 as c1 from range(1)") | ||
|
|
||
| withSQLConf( | ||
| CometConf.COMET_EXEC_ENABLED.key -> "true", | ||
| SQLConf.USE_V1_SOURCE_LIST.key -> "parquet", | ||
| CometConf.COMET_ENABLED.key -> "true", | ||
| CometConf.COMET_EXPLAIN_FALLBACK_ENABLED.key -> "true", | ||
| SQLConf.ANSI_ENABLED.key -> "false", | ||
| SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false", | ||
| CometConf.getExprAllowIncompatConfigKey(classOf[Sum]) -> "true", | ||
| CometConf.COMET_NATIVE_SCAN_IMPL.key -> CometConf.SCAN_NATIVE_ICEBERG_COMPAT, | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Are all settings needed for this test ? E.g. |
||
| SQLConf.ADAPTIVE_OPTIMIZER_EXCLUDED_RULES.key -> "org.apache.spark.sql.catalyst.optimizer.ConstantFolding") { | ||
|
|
||
| val df = sql("select * from t1") | ||
| val makeDecimalColumn = createMakeDecimalColumn(df.col("c1").expr, 3, 0) | ||
| val df1 = df.withColumn("result", makeDecimalColumn) | ||
|
|
||
| checkSparkAnswerAndFallbackReason(df1, "Unsupported input data type: IntegerType") | ||
| } | ||
| } | ||
| } | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. IMO it would be useful to add a positive test too - with LongType |
||
|
|
||
| } | ||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
should we also support Int32? 🤔