[ https://issues.apache.org/jira/browse/SPARK-47463?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17835282#comment-17835282 ]
Zhen Wang edited comment on SPARK-47463 at 4/10/24 2:22 AM: ------------------------------------------------------------ new case: filter with *nvl* expression: {code:java} create table t2(c1 boolean) using iceberg; select * from t2 where nvl(c1, false) = true; {code} was (Author: wforget): new similar failure case: {code:java} create table t2(c1 boolean) using iceberg; select * from t2 where nvl(c1, false) = true; {code} > An error occurred while pushing down the filter of if expression for iceberg > datasource. > ---------------------------------------------------------------------------------------- > > Key: SPARK-47463 > URL: https://issues.apache.org/jira/browse/SPARK-47463 > Project: Spark > Issue Type: Bug > Components: SQL > Affects Versions: 4.0.0 > Environment: Spark 3.5.0 > Iceberg 1.4.3 > Reporter: Zhen Wang > Priority: Major > Labels: pull-request-available > > Reproduce: > {code:java} > create table t1(c1 int) using iceberg; > select * from > (select if(c1 = 1, c1, null) as c1 from t1) t > where t.c1 > 0; {code} > Error: > {code:java} > org.apache.spark.SparkException: [INTERNAL_ERROR] The Spark SQL phase > optimization failed with an internal error. You hit a bug in Spark or the > Spark plugins you use. Please, report this bug to the corresponding > communities or vendors, and provide the full stack trace. > at > org.apache.spark.SparkException$.internalError(SparkException.scala:107) > at > org.apache.spark.sql.execution.QueryExecution$.toInternalError(QueryExecution.scala:536) > at > org.apache.spark.sql.execution.QueryExecution$.withInternalError(QueryExecution.scala:548) > at > org.apache.spark.sql.execution.QueryExecution.$anonfun$executePhase$1(QueryExecution.scala:219) > at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:900) > at > org.apache.spark.sql.execution.QueryExecution.executePhase(QueryExecution.scala:218) > at > org.apache.spark.sql.execution.QueryExecution.optimizedPlan$lzycompute(QueryExecution.scala:148) > at > org.apache.spark.sql.execution.QueryExecution.optimizedPlan(QueryExecution.scala:144) > at > org.apache.spark.sql.execution.QueryExecution.assertOptimized(QueryExecution.scala:162) > at > org.apache.spark.sql.execution.QueryExecution.executedPlan$lzycompute(QueryExecution.scala:182) > at > org.apache.spark.sql.execution.QueryExecution.executedPlan(QueryExecution.scala:179) > at > org.apache.spark.sql.execution.QueryExecution.simpleString(QueryExecution.scala:238) > at > org.apache.spark.sql.execution.QueryExecution.org$apache$spark$sql$execution$QueryExecution$$explainString(QueryExecution.scala:284) > at > org.apache.spark.sql.execution.QueryExecution.explainString(QueryExecution.scala:252) > at > org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$6(SQLExecution.scala:117) > at > org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:201) > at > org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:108) > at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:900) > at > org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:66) > at org.apache.spark.sql.Dataset.withAction(Dataset.scala:4327) > at org.apache.spark.sql.Dataset.collect(Dataset.scala:3580) > at > org.apache.kyuubi.engine.spark.operation.ExecuteStatement.fullCollectResult(ExecuteStatement.scala:72) > at > org.apache.kyuubi.engine.spark.operation.ExecuteStatement.collectAsIterator(ExecuteStatement.scala:164) > at > org.apache.kyuubi.engine.spark.operation.ExecuteStatement.$anonfun$executeStatement$1(ExecuteStatement.scala:87) > at > scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23) > at > org.apache.kyuubi.engine.spark.operation.SparkOperation.$anonfun$withLocalProperties$1(SparkOperation.scala:155) > at > org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:201) > at > org.apache.kyuubi.engine.spark.operation.SparkOperation.withLocalProperties(SparkOperation.scala:139) > at > org.apache.kyuubi.engine.spark.operation.ExecuteStatement.executeStatement(ExecuteStatement.scala:81) > at > org.apache.kyuubi.engine.spark.operation.ExecuteStatement$$anon$1.run(ExecuteStatement.scala:103) > at > java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) > at java.util.concurrent.FutureTask.run(FutureTask.java:266) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) > at java.lang.Thread.run(Thread.java:748) > Caused by: java.lang.AssertionError: assertion failed > at scala.Predef$.assert(Predef.scala:208) > at > org.apache.spark.sql.execution.datasources.v2.PushablePredicate$.$anonfun$unapply$1(DataSourceV2Strategy.scala:649) > at scala.Option.map(Option.scala:230) > at > org.apache.spark.sql.execution.datasources.v2.PushablePredicate$.unapply(DataSourceV2Strategy.scala:648) > at > org.apache.spark.sql.execution.datasources.v2.DataSourceV2Strategy$.translateLeafNodeFilterV2(DataSourceV2Strategy.scala:542) > at > org.apache.spark.sql.execution.datasources.v2.DataSourceV2Strategy$.translateFilterV2WithMapping(DataSourceV2Strategy.scala:595) > at > org.apache.spark.sql.execution.datasources.v2.PushDownUtils$.$anonfun$pushFilters$3(PushDownUtils.scala:88) > at scala.collection.immutable.List.foreach(List.scala:431) > at > org.apache.spark.sql.execution.datasources.v2.PushDownUtils$.pushFilters(PushDownUtils.scala:85) > at > org.apache.spark.sql.execution.datasources.v2.V2ScanRelationPushDown$$anonfun$pushDownFilters$1.applyOrElse(V2ScanRelationPushDown.scala:74) > at > org.apache.spark.sql.execution.datasources.v2.V2ScanRelationPushDown$$anonfun$pushDownFilters$1.applyOrElse(V2ScanRelationPushDown.scala:61) > at > org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDownWithPruning$1(TreeNode.scala:461) > at > org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(origin.scala:76) > at > org.apache.spark.sql.catalyst.trees.TreeNode.transformDownWithPruning(TreeNode.scala:461) > at > org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDownWithPruning(LogicalPlan.scala:32) > at > org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning(AnalysisHelper.scala:267) > at > org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning$(AnalysisHelper.scala:263) > at > org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:32) > at > org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:32) > at > org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDownWithPruning$3(TreeNode.scala:466) > at > org.apache.spark.sql.catalyst.trees.UnaryLike.mapChildren(TreeNode.scala:1215) > at > org.apache.spark.sql.catalyst.trees.UnaryLike.mapChildren$(TreeNode.scala:1214) > at > org.apache.spark.sql.catalyst.plans.logical.Project.mapChildren(basicLogicalOperators.scala:71) > at > org.apache.spark.sql.catalyst.trees.TreeNode.transformDownWithPruning(TreeNode.scala:466) > at > org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDownWithPruning(LogicalPlan.scala:32) > at > org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning(AnalysisHelper.scala:267) > at > org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning$(AnalysisHelper.scala:263) > at > org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:32) > at > org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:32) > at > org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:437) > at > org.apache.spark.sql.catalyst.trees.TreeNode.transform(TreeNode.scala:405) > at > org.apache.spark.sql.execution.datasources.v2.V2ScanRelationPushDown$.pushDownFilters(V2ScanRelationPushDown.scala:61) > at > org.apache.spark.sql.execution.datasources.v2.V2ScanRelationPushDown$.$anonfun$apply$3(V2ScanRelationPushDown.scala:45) > at > org.apache.spark.sql.execution.datasources.v2.V2ScanRelationPushDown$.$anonfun$apply$8(V2ScanRelationPushDown.scala:52) > at > scala.collection.LinearSeqOptimized.foldLeft(LinearSeqOptimized.scala:126) > at > scala.collection.LinearSeqOptimized.foldLeft$(LinearSeqOptimized.scala:122) > at scala.collection.immutable.List.foldLeft(List.scala:91) > at > org.apache.spark.sql.execution.datasources.v2.V2ScanRelationPushDown$.apply(V2ScanRelationPushDown.scala:51) > at > org.apache.spark.sql.execution.datasources.v2.V2ScanRelationPushDown$.apply(V2ScanRelationPushDown.scala:38) > at > org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$2(RuleExecutor.scala:222) > at > scala.collection.LinearSeqOptimized.foldLeft(LinearSeqOptimized.scala:126) > at > scala.collection.LinearSeqOptimized.foldLeft$(LinearSeqOptimized.scala:122) > at scala.collection.immutable.List.foldLeft(List.scala:91) > at > org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1(RuleExecutor.scala:219) > at > org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1$adapted(RuleExecutor.scala:211) > at scala.collection.immutable.List.foreach(List.scala:431) > at > org.apache.spark.sql.catalyst.rules.RuleExecutor.execute(RuleExecutor.scala:211) > at > org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$executeAndTrack$1(RuleExecutor.scala:182) > at > org.apache.spark.sql.catalyst.QueryPlanningTracker$.withTracker(QueryPlanningTracker.scala:89) > at > org.apache.spark.sql.catalyst.rules.RuleExecutor.executeAndTrack(RuleExecutor.scala:182) > at > org.apache.spark.sql.execution.QueryExecution.$anonfun$optimizedPlan$1(QueryExecution.scala:152) > at > org.apache.spark.sql.catalyst.QueryPlanningTracker.measurePhase(QueryPlanningTracker.scala:138) > at > org.apache.spark.sql.execution.QueryExecution.$anonfun$executePhase$2(QueryExecution.scala:219) > at > org.apache.spark.sql.execution.QueryExecution$.withInternalError(QueryExecution.scala:546) > ... 32 more {code} -- This message was sent by Atlassian Jira (v8.20.10#820010) --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org