Skip to content

MERGE INTO fails with INTERNAL_ERROR: AssertionError: No plan for TableReference during query planning #16451

@mrjoe7

Description

@mrjoe7

Apache Iceberg version

1.11.0 (latest release)

Query engine

Spark

Please describe the bug 🐞

When merging temp view data into a table I am getting INTERNAL_ERROR (log included below).
We did not observe this bug with Iceberg version 1.10.1.

The merge statement:

MERGE INTO iceberg.transformed.pdt_id_params target
USING temp_view source ON 
  target.pdt_id IS NOT DISTINCT FROM source.pdt_id 
  AND target.predicta_customer_name IS NOT DISTINCT FROM source.predicta_customer_name
  AND target.hour_ltz_index IS NOT DISTINCT FROM source.hour_ltz_index
  AND target.minute IS NOT DISTINCT FROM source.minute
  AND target.`predicta_customer_name` BETWEEN 'go_energy' AND 'go_energy'
WHEN MATCHED THEN UPDATE SET *
WHEN NOT MATCHED THEN INSERT *

Engine & libs:

  • Spark 4.1.1
  • org.apache.iceberg:iceberg-spark-runtime-4.1_2.13:1.11.0
  • org.apache.iceberg:iceberg-aws-bundle:1.11.0
  • org.apache.hadoop:hadoop-aws:3.5.0
  • org.apache.sedona:sedona-spark-shaded-4.1_2.13:1.9.0
  • org.datasyslab:geotools-wrapper:1.9.0-33.5
  • com.google.cloud.spark:spark-bigquery-with-dependencies_2.13:0.44.1

The table:

CREATE TABLE iceberg.transformed.pdt_id_params (
  predicta_customer_name STRING,
  pdt_id STRING,
  hour_ltz_index BIGINT,
  minute INT,
  max_amount_wh DOUBLE,
  min_amount_wh DOUBLE,
  median_amount_wh DOUBLE,
  mean_amount_wh DOUBLE,
  max_shared_amount_wh DOUBLE,
  min_shared_amount_wh DOUBLE,
  median_shared_amount_wh DOUBLE,
  mean_shared_amount_wh DOUBLE,
  __inserted_on_utc TIMESTAMP)
USING iceberg
PARTITIONED BY (predicta_customer_name)
LOCATION 's3://<redacted>/019bc261-df2c-7692-9f5f-8c53f11bb992'
TBLPROPERTIES (
  'current-snapshot-id' = '345060850875706057',
  'format' = 'iceberg/parquet',
  'format-version' = '2',
  'write.parquet.compression-codec' = 'zstd')

The error:

SparkException: [INTERNAL_ERROR] The Spark SQL phase planning 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. SQLSTATE: XX000

JVM stacktrace:
org.apache.spark.SparkException
	at org.apache.spark.SparkException$.internalError(SparkException.scala:107)
	at org.apache.spark.sql.execution.QueryExecution$.toInternalError(QueryExecution.scala:706)
	at org.apache.spark.sql.execution.QueryExecution$.withInternalError(QueryExecution.scala:719)
	at org.apache.spark.sql.execution.QueryExecution.$anonfun$executePhase$1(QueryExecution.scala:330)
	at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:804)
	at org.apache.spark.sql.execution.QueryExecution.executePhase(QueryExecution.scala:329)
	at org.apache.spark.sql.execution.QueryExecution.$anonfun$lazyExecutedPlan$1(QueryExecution.scala:285)
	at scala.util.Try$.apply(Try.scala:217)
	at org.apache.spark.util.Utils$.doTryWithCallerStacktrace(Utils.scala:1392)
	at org.apache.spark.util.LazyTry.tryT$lzycompute(LazyTry.scala:46)
	at org.apache.spark.util.LazyTry.tryT(LazyTry.scala:46)
	at org.apache.spark.util.LazyTry.get(LazyTry.scala:58)
	at org.apache.spark.sql.execution.QueryExecution.executedPlan(QueryExecution.scala:295)
	at org.apache.spark.sql.execution.QueryExecution.simpleString(QueryExecution.scala:349)
	at org.apache.spark.sql.execution.QueryExecution.org$apache$spark$sql$execution$QueryExecution$$explainString(QueryExecution.scala:396)
	at org.apache.spark.sql.execution.QueryExecution.explainString(QueryExecution.scala:364)
	at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId0$8(SQLExecution.scala:163)
	at org.apache.spark.sql.execution.SQLExecution$.withSessionTagsApplied(SQLExecution.scala:285)
	at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId0$7(SQLExecution.scala:139)
	at org.apache.spark.JobArtifactSet$.withActiveJobArtifactState(JobArtifactSet.scala:94)
	at org.apache.spark.sql.artifact.ArtifactManager.$anonfun$withResources$1(ArtifactManager.scala:112)
	at org.apache.spark.util.Utils$.withContextClassLoader(Utils.scala:185)
	at org.apache.spark.sql.artifact.ArtifactManager.withClassLoaderIfNeeded(ArtifactManager.scala:102)
	at org.apache.spark.sql.artifact.ArtifactManager.withResources(ArtifactManager.scala:111)
	at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId0$6(SQLExecution.scala:139)
	at org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:308)
	at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId0$1(SQLExecution.scala:138)
	at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:804)
	at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId0(SQLExecution.scala:92)
	at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:250)
	at org.apache.spark.sql.execution.QueryExecution.$anonfun$eagerlyExecuteCommands$1(QueryExecution.scala:185)
	at org.apache.spark.sql.execution.QueryExecution$.withInternalError(QueryExecution.scala:717)
	at org.apache.spark.sql.execution.QueryExecution.org$apache$spark$sql$execution$QueryExecution$$eagerlyExecute$1(QueryExecution.scala:184)
	at org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$3.applyOrElse(QueryExecution.scala:201)
	at org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$3.applyOrElse(QueryExecution.scala:194)
	at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDownWithPruning$1(TreeNode.scala:491)
	at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(origin.scala:107)
	at org.apache.spark.sql.catalyst.trees.TreeNode.transformDownWithPruning(TreeNode.scala:491)
	at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDownWithPruning(LogicalPlan.scala:37)
	at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning(AnalysisHelper.scala:360)
	at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning$(AnalysisHelper.scala:356)
	at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:37)
	at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:37)
	at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:467)
	at org.apache.spark.sql.execution.QueryExecution.eagerlyExecuteCommands(QueryExecution.scala:194)
	at org.apache.spark.sql.execution.QueryExecution.$anonfun$lazyCommandExecuted$1(QueryExecution.scala:155)
	at scala.util.Try$.apply(Try.scala:217)
	at org.apache.spark.util.Utils$.getTryWithCallerStacktrace(Utils.scala:1453)
	at org.apache.spark.util.LazyTry.get(LazyTry.scala:58)
	at org.apache.spark.sql.execution.QueryExecution.commandExecuted(QueryExecution.scala:160)
	at org.apache.spark.sql.classic.Dataset.<init>(Dataset.scala:276)
	at org.apache.spark.sql.classic.Dataset$.$anonfun$ofRows$5(Dataset.scala:139)
	at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:804)
	at org.apache.spark.sql.classic.Dataset$.ofRows(Dataset.scala:135)
	at org.apache.spark.sql.classic.SparkSession.$anonfun$sql$4(SparkSession.scala:584)
	at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:804)
	at org.apache.spark.sql.classic.SparkSession.sql(SparkSession.scala:561)
	at org.apache.spark.sql.connect.planner.SparkConnectPlanner.executeSQL(SparkConnectPlanner.scala:3148)
	at org.apache.spark.sql.connect.planner.SparkConnectPlanner.handleSqlCommand(SparkConnectPlanner.scala:2996)
	at org.apache.spark.sql.connect.planner.SparkConnectPlanner.process(SparkConnectPlanner.scala:2830)
	at org.apache.spark.sql.connect.execution.SparkConnectPlanExecution.handlePlan(SparkConnectPlanExecution.scala:96)
	at org.apache.spark.sql.connect.execution.ExecuteThreadRunner.$anonfun$executeInternal$1(ExecuteThreadRunner.scala:225)
	at org.apache.spark.sql.connect.execution.ExecuteThreadRunner.$anonfun$executeInternal$1$adapted(ExecuteThreadRunner.scala:197)
	at org.apache.spark.sql.connect.service.SessionHolder.$anonfun$withSession$2(SessionHolder.scala:396)
	at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:804)
	at org.apache.spark.sql.connect.service.SessionHolder.$anonfun$withSession$1(SessionHolder.scala:396)
	at org.apache.spark.JobArtifactSet$.withActiveJobArtifactState(JobArtifactSet.scala:94)
	at org.apache.spark.sql.artifact.ArtifactManager.$anonfun$withResources$1(ArtifactManager.scala:112)
	at org.apache.spark.util.Utils$.withContextClassLoader(Utils.scala:185)
	at org.apache.spark.sql.artifact.ArtifactManager.withClassLoaderIfNeeded(ArtifactManager.scala:102)
	at org.apache.spark.sql.artifact.ArtifactManager.withResources(ArtifactManager.scala:111)
	at org.apache.spark.sql.connect.service.SessionHolder.withSession(SessionHolder.scala:395)
	at org.apache.spark.sql.connect.execution.ExecuteThreadRunner.executeInternal(ExecuteThreadRunner.scala:197)
	at org.apache.spark.sql.connect.execution.ExecuteThreadRunner.org$apache$spark$sql$connect$execution$ExecuteThreadRunner$$execute(ExecuteThreadRunner.scala:126)
	at org.apache.spark.sql.connect.execution.ExecuteThreadRunner$ExecutionThread.run(ExecuteThreadRunner.scala:334)
Caused by: java.lang.AssertionError: assertion failed: No plan for TableReference[predicta_customer_name#474367, pdt_id#474368, hour_ltz_index#474369L, minute#474370, valid_from_trading_date#474371, max_amount_wh#474372, min_amount_wh#474373, median_amount_wh#474374, mean_amount_wh#474375, max_shared_amount_wh#474376, min_shared_amount_wh#474377, median_shared_amount_wh#474378, mean_shared_amount_wh#474379, __inserted_on_utc#474380] iceberg.transformed.pdt_id_params_history

	at scala.Predef$.assert(Predef.scala:279)
	at org.apache.spark.sql.catalyst.planning.QueryPlanner.plan(QueryPlanner.scala:93)
	at org.apache.spark.sql.execution.SparkStrategies.plan(SparkStrategies.scala:79)
	at org.apache.spark.sql.catalyst.planning.QueryPlanner.$anonfun$plan$3(QueryPlanner.scala:78)
	at scala.collection.IterableOnceOps.foldLeft(IterableOnce.scala:738)
	at scala.collection.IterableOnceOps.foldLeft$(IterableOnce.scala:732)
	at scala.collection.AbstractIterator.foldLeft(Iterator.scala:1313)
	at org.apache.spark.sql.catalyst.planning.QueryPlanner.$anonfun$plan$2(QueryPlanner.scala:75)
	at scala.collection.Iterator$$anon$10.nextCur(Iterator.scala:604)
	at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:618)
	at org.apache.spark.sql.catalyst.planning.QueryPlanner.plan(QueryPlanner.scala:93)
	at org.apache.spark.sql.execution.SparkStrategies.plan(SparkStrategies.scala:79)
	at org.apache.spark.sql.catalyst.planning.QueryPlanner.$anonfun$plan$3(QueryPlanner.scala:78)
	at scala.collection.IterableOnceOps.foldLeft(IterableOnce.scala:738)
	at scala.collection.IterableOnceOps.foldLeft$(IterableOnce.scala:732)
	at scala.collection.AbstractIterator.foldLeft(Iterator.scala:1313)
	at org.apache.spark.sql.catalyst.planning.QueryPlanner.$anonfun$plan$2(QueryPlanner.scala:75)
	at scala.collection.Iterator$$anon$10.nextCur(Iterator.scala:604)
	at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:618)
	at org.apache.spark.sql.catalyst.planning.QueryPlanner.plan(QueryPlanner.scala:93)
	at org.apache.spark.sql.execution.SparkStrategies.plan(SparkStrategies.scala:79)
	at org.apache.spark.sql.catalyst.planning.QueryPlanner.$anonfun$plan$3(QueryPlanner.scala:78)
	at scala.collection.IterableOnceOps.foldLeft(IterableOnce.scala:738)
	at scala.collection.IterableOnceOps.foldLeft$(IterableOnce.scala:732)
	at scala.collection.AbstractIterator.foldLeft(Iterator.scala:1313)
	at org.apache.spark.sql.catalyst.planning.QueryPlanner.$anonfun$plan$2(QueryPlanner.scala:75)
	at scala.collection.Iterator$$anon$10.nextCur(Iterator.scala:604)
	at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:618)
	at org.apache.spark.sql.catalyst.planning.QueryPlanner.plan(QueryPlanner.scala:93)
	at org.apache.spark.sql.execution.SparkStrategies.plan(SparkStrategies.scala:79)
	at org.apache.spark.sql.catalyst.planning.QueryPlanner.$anonfun$plan$3(QueryPlanner.scala:78)
	at scala.collection.IterableOnceOps.foldLeft(IterableOnce.scala:738)
	at scala.collection.IterableOnceOps.foldLeft$(IterableOnce.scala:732)
	at scala.collection.AbstractIterator.foldLeft(Iterator.scala:1313)
	at org.apache.spark.sql.catalyst.planning.QueryPlanner.$anonfun$plan$2(QueryPlanner.scala:75)
	at scala.collection.Iterator$$anon$10.nextCur(Iterator.scala:604)
	at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:618)
	at org.apache.spark.sql.catalyst.planning.QueryPlanner.plan(QueryPlanner.scala:93)
	at org.apache.spark.sql.execution.SparkStrategies.plan(SparkStrategies.scala:79)
	at org.apache.spark.sql.catalyst.planning.QueryPlanner.$anonfun$plan$3(QueryPlanner.scala:78)
	at scala.collection.IterableOnceOps.foldLeft(IterableOnce.scala:738)
	at scala.collection.IterableOnceOps.foldLeft$(IterableOnce.scala:732)
	at scala.collection.AbstractIterator.foldLeft(Iterator.scala:1313)
	at org.apache.spark.sql.catalyst.planning.QueryPlanner.$anonfun$plan$2(QueryPlanner.scala:75)
	at scala.collection.Iterator$$anon$10.nextCur(Iterator.scala:604)
	at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:618)
	at org.apache.spark.sql.catalyst.planning.QueryPlanner.plan(QueryPlanner.scala:93)
	at org.apache.spark.sql.execution.SparkStrategies.plan(SparkStrategies.scala:79)
	at org.apache.spark.sql.catalyst.planning.QueryPlanner.$anonfun$plan$3(QueryPlanner.scala:78)
	at scala.collection.IterableOnceOps.foldLeft(IterableOnce.scala:738)
	at scala.collection.IterableOnceOps.foldLeft$(IterableOnce.scala:732)
	at scala.collection.AbstractIterator.foldLeft(Iterator.scala:1313)
	at org.apache.spark.sql.catalyst.planning.QueryPlanner.$anonfun$plan$2(QueryPlanner.scala:75)
	at scala.collection.Iterator$$anon$10.nextCur(Iterator.scala:604)
	at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:618)
	at org.apache.spark.sql.catalyst.planning.QueryPlanner.plan(QueryPlanner.scala:93)
	at org.apache.spark.sql.execution.SparkStrategies.plan(SparkStrategies.scala:79)
	at org.apache.spark.sql.catalyst.planning.QueryPlanner.$anonfun$plan$3(QueryPlanner.scala:78)
	at scala.collection.IterableOnceOps.foldLeft(IterableOnce.scala:738)
	at scala.collection.IterableOnceOps.foldLeft$(IterableOnce.scala:732)
	at scala.collection.AbstractIterator.foldLeft(Iterator.scala:1313)
	at org.apache.spark.sql.catalyst.planning.QueryPlanner.$anonfun$plan$2(QueryPlanner.scala:75)
	at scala.collection.Iterator$$anon$10.nextCur(Iterator.scala:604)
	at scala.collection.Iterator$$anon$10.hasNext(Iterator.scala:618)
	at org.apache.spark.sql.catalyst.planning.QueryPlanner.plan(QueryPlanner.scala:93)
	at org.apache.spark.sql.execution.SparkStrategies.plan(SparkStrategies.scala:79)
	at org.apache.spark.sql.execution.QueryExecution$.createSparkPlan(QueryExecution.scala:656)
	at org.apache.spark.sql.execution.adaptive.InsertAdaptiveSparkPlan.compileSubquery(InsertAdaptiveSparkPlan.scala:167)
	at org.apache.spark.sql.execution.adaptive.InsertAdaptiveSparkPlan.$anonfun$buildSubqueryMap$4(InsertAdaptiveSparkPlan.scala:152)
	at org.apache.spark.sql.execution.adaptive.InsertAdaptiveSparkPlan.$anonfun$buildSubqueryMap$4$adapted(InsertAdaptiveSparkPlan.scala:148)
	at org.apache.spark.sql.catalyst.trees.TreeNode.foreach(TreeNode.scala:263)
	at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$foreach$1(TreeNode.scala:264)
	at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$foreach$1$adapted(TreeNode.scala:264)
	at scala.collection.IterableOnceOps.foreach(IterableOnce.scala:630)
	at scala.collection.IterableOnceOps.foreach$(IterableOnce.scala:628)
	at scala.collection.AbstractIterable.foreach(Iterable.scala:936)
	at org.apache.spark.sql.catalyst.trees.TreeNode.foreach(TreeNode.scala:264)
	at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$foreach$1(TreeNode.scala:264)
	at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$foreach$1$adapted(TreeNode.scala:264)
	at scala.collection.immutable.Vector.foreach(Vector.scala:2125)
	at org.apache.spark.sql.catalyst.trees.TreeNode.foreach(TreeNode.scala:264)
	at org.apache.spark.sql.execution.adaptive.InsertAdaptiveSparkPlan.$anonfun$buildSubqueryMap$3(InsertAdaptiveSparkPlan.scala:148)
	at org.apache.spark.sql.execution.adaptive.InsertAdaptiveSparkPlan.$anonfun$buildSubqueryMap$3$adapted(InsertAdaptiveSparkPlan.scala:148)
	at scala.collection.immutable.List.foreach(List.scala:323)
	at org.apache.spark.sql.execution.adaptive.InsertAdaptiveSparkPlan.$anonfun$buildSubqueryMap$1(InsertAdaptiveSparkPlan.scala:148)
	at org.apache.spark.sql.execution.adaptive.InsertAdaptiveSparkPlan.$anonfun$buildSubqueryMap$1$adapted(InsertAdaptiveSparkPlan.scala:148)
	at org.apache.spark.sql.catalyst.trees.TreeNode.foreach(TreeNode.scala:263)
	at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$foreach$1(TreeNode.scala:264)
	at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$foreach$1$adapted(TreeNode.scala:264)
	at scala.collection.immutable.Vector.foreach(Vector.scala:2125)
	at org.apache.spark.sql.catalyst.trees.TreeNode.foreach(TreeNode.scala:264)
	at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$foreach$1(TreeNode.scala:264)
	at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$foreach$1$adapted(TreeNode.scala:264)
	at scala.collection.immutable.Vector.foreach(Vector.scala:2125)
	at org.apache.spark.sql.catalyst.trees.TreeNode.foreach(TreeNode.scala:264)
	at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$foreach$1(TreeNode.scala:264)
	at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$foreach$1$adapted(TreeNode.scala:264)
	at scala.collection.immutable.Vector.foreach(Vector.scala:2125)
	at org.apache.spark.sql.catalyst.trees.TreeNode.foreach(TreeNode.scala:264)
	at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$foreach$1(TreeNode.scala:264)
	at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$foreach$1$adapted(TreeNode.scala:264)
	at scala.collection.immutable.Vector.foreach(Vector.scala:2125)
	at org.apache.spark.sql.catalyst.trees.TreeNode.foreach(TreeNode.scala:264)
	at org.apache.spark.sql.execution.adaptive.InsertAdaptiveSparkPlan.buildSubqueryMap(InsertAdaptiveSparkPlan.scala:148)
	at org.apache.spark.sql.execution.adaptive.InsertAdaptiveSparkPlan.applyInternal(InsertAdaptiveSparkPlan.scala:76)
	at org.apache.spark.sql.execution.adaptive.InsertAdaptiveSparkPlan.apply(InsertAdaptiveSparkPlan.scala:49)
	at org.apache.spark.sql.execution.adaptive.InsertAdaptiveSparkPlan.$anonfun$applyInternal$1(InsertAdaptiveSparkPlan.scala:55)
	at scala.collection.immutable.Vector1.map(Vector.scala:2141)
	at scala.collection.immutable.Vector1.map(Vector.scala:386)
	at org.apache.spark.sql.execution.adaptive.InsertAdaptiveSparkPlan.applyInternal(InsertAdaptiveSparkPlan.scala:55)
	at org.apache.spark.sql.execution.adaptive.InsertAdaptiveSparkPlan.apply(InsertAdaptiveSparkPlan.scala:49)
	at org.apache.spark.sql.execution.adaptive.InsertAdaptiveSparkPlan.apply(InsertAdaptiveSparkPlan.scala:44)
	at org.apache.spark.sql.execution.QueryExecution$.$anonfun$prepareForExecution$1(QueryExecution.scala:638)
	at scala.collection.LinearSeqOps.foldLeft(LinearSeq.scala:183)
	at scala.collection.LinearSeqOps.foldLeft$(LinearSeq.scala:179)
	at scala.collection.immutable.List.foldLeft(List.scala:79)
	at org.apache.spark.sql.execution.QueryExecution$.prepareForExecution(QueryExecution.scala:637)
	at org.apache.spark.sql.execution.QueryExecution.$anonfun$lazyExecutedPlan$2(QueryExecution.scala:285)
	at org.apache.spark.sql.catalyst.QueryPlanningTracker.measurePhase(QueryPlanningTracker.scala:148)
	at org.apache.spark.sql.execution.QueryExecution.$anonfun$executePhase$2(QueryExecution.scala:330)
	at org.apache.spark.sql.execution.QueryExecution$.withInternalError(QueryExecution.scala:717)
	at org.apache.spark.sql.execution.QueryExecution.$anonfun$executePhase$1(QueryExecution.scala:330)
	at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:804)
	at org.apache.spark.sql.execution.QueryExecution.executePhase(QueryExecution.scala:329)
	at org.apache.spark.sql.execution.QueryExecution.$anonfun$lazyExecutedPlan$1(QueryExecution.scala:285)
	at scala.util.Try$.apply(Try.scala:217)
	at org.apache.spark.util.Utils$.doTryWithCallerStacktrace(Utils.scala:1392)
	at org.apache.spark.util.LazyTry.tryT$lzycompute(LazyTry.scala:46)
	at org.apache.spark.util.LazyTry.tryT(LazyTry.scala:46)
	at org.apache.spark.util.LazyTry.get(LazyTry.scala:58)
	at org.apache.spark.sql.execution.QueryExecution.executedPlan(QueryExecution.scala:295)
	at org.apache.spark.sql.execution.QueryExecution.simpleString(QueryExecution.scala:349)
	at org.apache.spark.sql.execution.QueryExecution.org$apache$spark$sql$execution$QueryExecution$$explainString(QueryExecution.scala:396)
	at org.apache.spark.sql.execution.QueryExecution.explainString(QueryExecution.scala:364)
	at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId0$8(SQLExecution.scala:163)
	at org.apache.spark.sql.execution.SQLExecution$.withSessionTagsApplied(SQLExecution.scala:285)
	at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId0$7(SQLExecution.scala:139)
	at org.apache.spark.JobArtifactSet$.withActiveJobArtifactState(JobArtifactSet.scala:94)
	at org.apache.spark.sql.artifact.ArtifactManager.$anonfun$withResources$1(ArtifactManager.scala:112)
	at org.apache.spark.util.Utils$.withContextClassLoader(Utils.scala:185)
	at org.apache.spark.sql.artifact.ArtifactManager.withClassLoaderIfNeeded(ArtifactManager.scala:102)
	at org.apache.spark.sql.artifact.ArtifactManager.withResources(ArtifactManager.scala:111)
	at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId0$6(SQLExecution.scala:139)
	at org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:308)
	at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId0$1(SQLExecution.scala:138)
	at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:804)
	at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId0(SQLExecution.scala:92)
	at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:250)
	at org.apache.spark.sql.execution.QueryExecution.$anonfun$eagerlyExecuteCommands$1(QueryExecution.scala:185)
	at org.apache.spark.sql.execution.QueryExecution$.withInternalError(QueryExecution.scala:717)
	at org.apache.spark.sql.execution.QueryExecution.org$apache$spark$sql$execution$QueryExecution$$eagerlyExecute$1(QueryExecution.scala:184)
	at org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$3.applyOrElse(QueryExecution.scala:201)
	at org.apache.spark.sql.execution.QueryExecution$$anonfun$eagerlyExecuteCommands$3.applyOrElse(QueryExecution.scala:194)
	at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDownWithPruning$1(TreeNode.scala:491)
	at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(origin.scala:107)
	at org.apache.spark.sql.catalyst.trees.TreeNode.transformDownWithPruning(TreeNode.scala:491)
	at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDownWithPruning(LogicalPlan.scala:37)
	at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning(AnalysisHelper.scala:360)
	at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning$(AnalysisHelper.scala:356)
	at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:37)
	at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:37)
	at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:467)
	at org.apache.spark.sql.execution.QueryExecution.eagerlyExecuteCommands(QueryExecution.scala:194)
	at org.apache.spark.sql.execution.QueryExecution.$anonfun$lazyCommandExecuted$1(QueryExecution.scala:155)
	at scala.util.Try$.apply(Try.scala:217)
	at org.apache.spark.util.Utils$.doTryWithCallerStacktrace(Utils.scala:1392)
	at org.apache.spark.util.LazyTry.tryT$lzycompute(LazyTry.scala:46)
	at org.apache.spark.util.LazyTry.tryT(LazyTry.scala:46)
	at org.apache.spark.util.LazyTry.get(LazyTry.scala:58)
	at org.apache.spark.sql.execution.QueryExecution.commandExecuted(QueryExecution.scala:160)
	at org.apache.spark.sql.classic.Dataset.<init>(Dataset.scala:276)
	at org.apache.spark.sql.classic.Dataset$.$anonfun$ofRows$5(Dataset.scala:139)
	at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:804)
	at org.apache.spark.sql.classic.Dataset$.ofRows(Dataset.scala:135)
	at org.apache.spark.sql.classic.SparkSession.$anonfun$sql$4(SparkSession.scala:584)
	at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:804)
	at org.apache.spark.sql.classic.SparkSession.sql(SparkSession.scala:561)
	at org.apache.spark.sql.connect.planner.SparkConnectPlanner.executeSQL(SparkConnectPlanner.scala:3148)
	at org.apache.spark.sql.connect.planner.SparkConnectPlanner.handleSqlCommand(SparkConnectPlanner.scala:2996)
	at org.apache.spark.sql.connect.planner.SparkConnectPlanner.process(SparkConnectPlanner.scala:2830)
	at org.apache.spark.sql.connect.execution.SparkConnectPlanExecution.handlePlan(SparkConnectPlanExecution.scala:96)
	at org.apache.spark.sql.connect.execution.ExecuteThreadRunner.$anonfun$executeInternal$1(ExecuteThreadRunner.scala:225)
	at org.apache.spark.sql.connect.execution.ExecuteThreadRunner.$anonfun$executeInternal$1$adapted(ExecuteThreadRunner.scala:197)
	at org.apache.spark.sql.connect.service.SessionHolder.$anonfun$withSession$2(SessionHolder.scala:396)
	at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:804)
	at org.apache.spark.sql.connect.service.SessionHolder.$anonfun$withSession$1(SessionHolder.scala:396)
	at org.apache.spark.JobArtifactSet$.withActiveJobArtifactState(JobArtifactSet.scala:94)
	at org.apache.spark.sql.artifact.ArtifactManager.$anonfun$withResources$1(ArtifactManager.scala:112)
	at org.apache.spark.util.Utils$.withContextClassLoader(Utils.scala:185)
	at org.apache.spark.sql.artifact.ArtifactManager.withClassLoaderIfNeeded(ArtifactManager.scala:102)
	at org.apache.spark.sql.artifact.ArtifactManager.withResources(ArtifactManager.scala:111)
	at org.apache.spark.sql.connect.service.SessionHolder.withSession(SessionHolder.scala:395)
	at org.apache.spark.sql.connect.execution.ExecuteThreadRunner.executeInternal(ExecuteThreadRunner.scala:197)
	at org.apache.spark.sql.connect.execution.ExecuteThreadRunner.org$apache$spark$sql$connect$execution$ExecuteThreadRunner$$execute(ExecuteThreadRunner.scala:126)
	at org.apache.spark.sql.connect.execution.ExecuteThreadRunner$ExecutionThread.run(ExecuteThreadRunner.scala:334)

Willingness to contribute

  • I can contribute a fix for this bug independently
  • I would be willing to contribute a fix for this bug with guidance from the Iceberg community
  • I cannot contribute a fix for this bug at this time

Metadata

Metadata

Assignees

No one assigned

    Labels

    bugSomething isn't working

    Type

    No type
    No fields configured for issues without a type.

    Projects

    No projects

    Milestone

    No milestone

    Relationships

    None yet

    Development

    No branches or pull requests

    Issue actions