Details
-
Bug
-
Status: Open
-
Minor
-
Resolution: Unresolved
-
3.5.0
-
None
-
None
Description
When performing a query with a nullif in the where clause such as "select * from table where nullif(column, '') is null" I get the following stack trace:
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:615) at org.apache.spark.sql.execution.QueryExecution$.withInternalError(QueryExecution.scala:627) at org.apache.spark.sql.execution.QueryExecution.$anonfun$executePhase$1(QueryExecution.scala:256) at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:900) at org.apache.spark.sql.execution.QueryExecution.executePhase(QueryExecution.scala:255) at org.apache.spark.sql.execution.QueryExecution.optimizedPlan$lzycompute(QueryExecution.scala:157) at org.apache.spark.sql.execution.QueryExecution.optimizedPlan(QueryExecution.scala:153) at org.apache.spark.sql.execution.QueryExecution.$anonfun$writePlans$4(QueryExecution.scala:340) at org.apache.spark.sql.catalyst.plans.QueryPlan$.append(QueryPlan.scala:716) at org.apache.spark.sql.execution.QueryExecution.writePlans(QueryExecution.scala:340) at org.apache.spark.sql.execution.QueryExecution.toString(QueryExecution.scala:357) at org.apache.spark.sql.execution.QueryExecution.org$apache$spark$sql$execution$QueryExecution$$explainString(QueryExecution.scala:311) at org.apache.spark.sql.execution.QueryExecution.explainString(QueryExecution.scala:289) at org.apache.spark.sql.execution.SQLExecution$.executeQuery$1(SQLExecution.scala:121) at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$9(SQLExecution.scala:165) at org.apache.spark.sql.catalyst.QueryPlanningTracker$.withTracker(QueryPlanningTracker.scala:108) at org.apache.spark.sql.execution.SQLExecution$.withTracker(SQLExecution.scala:255) at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$8(SQLExecution.scala:165) at org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:276) at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:164) at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:900) at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:70) at org.apache.spark.sql.Dataset.withAction(Dataset.scala:4399) at org.apache.spark.sql.Dataset.head(Dataset.scala:3370) at org.apache.spark.sql.Dataset.take(Dataset.scala:3593) at org.apache.spark.sql.Dataset.getRows(Dataset.scala:282) at org.apache.spark.sql.Dataset.showString(Dataset.scala:317) at org.apache.spark.sql.Dataset.show(Dataset.scala:883) at org.apache.spark.sql.Dataset.show(Dataset.scala:842) at org.apache.spark.sql.Dataset.show(Dataset.scala:851) ... 47 elided 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:667) at scala.Option.map(Option.scala:230) at org.apache.spark.sql.execution.datasources.v2.PushablePredicate$.unapply(DataSourceV2Strategy.scala:666) at org.apache.spark.sql.execution.datasources.v2.DataSourceV2Strategy$.translateLeafNodeFilterV2(DataSourceV2Strategy.scala:560) at org.apache.spark.sql.execution.datasources.v2.DataSourceV2Strategy$.translateFilterV2WithMapping(DataSourceV2Strategy.scala:613) 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:503) at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(origin.scala:76) at org.apache.spark.sql.catalyst.trees.TreeNode.transformDownWithPruning(TreeNode.scala:503) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDownWithPruning(LogicalPlan.scala:33) 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:33) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:33) at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDownWithPruning$3(TreeNode.scala:508) at org.apache.spark.sql.catalyst.trees.UnaryLike.mapChildren(TreeNode.scala:1288) at org.apache.spark.sql.catalyst.trees.UnaryLike.mapChildren$(TreeNode.scala:1287) at org.apache.spark.sql.catalyst.plans.logical.Project.mapChildren(basicLogicalOperators.scala:99) at org.apache.spark.sql.catalyst.trees.TreeNode.transformDownWithPruning(TreeNode.scala:508) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDownWithPruning(LogicalPlan.scala:33) 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:33) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:33) at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDownWithPruning$3(TreeNode.scala:508) at org.apache.spark.sql.catalyst.trees.UnaryLike.mapChildren(TreeNode.scala:1288) at org.apache.spark.sql.catalyst.trees.UnaryLike.mapChildren$(TreeNode.scala:1287) at org.apache.spark.sql.catalyst.plans.logical.LocalLimit.mapChildren(basicLogicalOperators.scala:1952) at org.apache.spark.sql.catalyst.trees.TreeNode.transformDownWithPruning(TreeNode.scala:508) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDownWithPruning(LogicalPlan.scala:33) 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:33) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:33) at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDownWithPruning$3(TreeNode.scala:508) at org.apache.spark.sql.catalyst.trees.UnaryLike.mapChildren(TreeNode.scala:1288) at org.apache.spark.sql.catalyst.trees.UnaryLike.mapChildren$(TreeNode.scala:1287) at org.apache.spark.sql.catalyst.plans.logical.GlobalLimit.mapChildren(basicLogicalOperators.scala:1931) at org.apache.spark.sql.catalyst.trees.TreeNode.transformDownWithPruning(TreeNode.scala:508) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDownWithPruning(LogicalPlan.scala:33) 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:33) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:33) at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:479) at org.apache.spark.sql.catalyst.trees.TreeNode.transform(TreeNode.scala:447) 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$1(RuleExecutor.scala:239) 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.executeBatch$1(RuleExecutor.scala:236) at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$6(RuleExecutor.scala:319) at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23) at org.apache.spark.sql.catalyst.rules.RuleExecutor$RuleExecutionContext$.withContext(RuleExecutor.scala:368) at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$5(RuleExecutor.scala:319) at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$5$adapted(RuleExecutor.scala:309) at scala.collection.immutable.List.foreach(List.scala:431) at org.apache.spark.sql.catalyst.rules.RuleExecutor.execute(RuleExecutor.scala:309) at org.apache.spark.sql.catalyst.rules.RuleExecutor.execute(RuleExecutor.scala:195) at org.apache.spark.sql.catalyst.optimizer.BaseOptimizer.super$execute(BaseOptimizer.scala:28) at org.apache.spark.sql.catalyst.optimizer.BaseOptimizer.$anonfun$execute$1(BaseOptimizer.scala:28) at org.apache.spark.sql.catalyst.optimizer.OptimizationContext$.withOptimizationContext(OptimizationContext.scala:80) at org.apache.spark.sql.catalyst.optimizer.BaseOptimizer.execute(BaseOptimizer.scala:28) at org.apache.spark.sql.catalyst.optimizer.BaseOptimizer.execute(BaseOptimizer.scala:23) at org.apache.spark.sql.catalyst.rules.RuleExecutor.execute(RuleExecutor.scala:191) at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$executeAndTrack$1(RuleExecutor.scala:182) at org.apache.spark.sql.catalyst.QueryPlanningTracker$.withTracker(QueryPlanningTracker.scala:108) at org.apache.spark.sql.catalyst.rules.RuleExecutor.executeAndTrack(RuleExecutor.scala:182) at org.apache.spark.sql.execution.QueryExecution.$anonfun$optimizedPlan$1(QueryExecution.scala:161) at org.apache.spark.sql.catalyst.QueryPlanningTracker.measurePhase(QueryPlanningTracker.scala:219) at org.apache.spark.sql.execution.QueryExecution.$anonfun$executePhase$2(QueryExecution.scala:256) at org.apache.spark.sql.execution.QueryExecution$.withInternalError(QueryExecution.scala:625) ... 75 more
It is possible to work around this bug by using "column = '' or column is null"
I had originally opened this as an issue with Iceberg, but was re-directed to this being a Spark issue. User singhpk234 commented with the specific location of the issue here.
Attachments
Attachments
Issue Links
- is related to
-
SPARK-47463 An error occurred while pushing down the filter of if expression for iceberg datasource.
- Resolved