Serge Rielau created SPARK-46372: ------------------------------------ Summary: "Invalid call to toAttribute on unresolved object" instead UNRESOLVED_COLUMN.WITH_SUGGESTION on INSERT statement Key: SPARK-46372 URL: https://issues.apache.org/jira/browse/SPARK-46372 Project: Spark Issue Type: Bug Components: Spark Core Affects Versions: 3.5.0 Reporter: Serge Rielau
{{CREATE TABLE rec(n INT, sm INT);}} SELECT n + 1, n + 1 + sm FROM rec WHERE rec = 8; [UNRESOLVED_COLUMN.WITH_SUGGESTION] A column, variable, or function parameter with name `rec` cannot be resolved. Did you mean one of the following? [`n`, `sm`]. SQLSTATE: 42703; line 1 pos 40; {{but when placed in an INSERT:}} {{}} INSERT INTO rec SELECT n + 1, n + 1 + sm FROM rec WHERE rec = 8; Invalid call to toAttribute on unresolved object 1. This appears to be a system error and we should raise it as such 2. Clearly we missed or didn't get to the point to raise the proper error. Stacktrace: {quote}scala> spark.sql("INSERT INTO rec SELECT n + 1, n + 1 + sm FROM rec WHERE rec = 8").show(); 23/12/11 18:12:25 WARN ObjectStore: Version information not found in metastore. hive.metastore.schema.verification is not enabled so recording the schema version 2.3.0 23/12/11 18:12:25 WARN ObjectStore: setMetaStoreSchemaVersion called but recording version is disabled: version = 2.3.0, comment = Set by MetaStore serge.rielau@10.240.1.53 org.apache.spark.sql.catalyst.analysis.UnresolvedException: Invalid call to toAttribute on unresolved object at org.apache.spark.sql.catalyst.analysis.UnresolvedAlias.toAttribute(unresolved.scala:707) at org.apache.spark.sql.catalyst.plans.logical.Project.$anonfun$output$1(basicLogicalOperators.scala:74) at scala.collection.immutable.List.map(List.scala:246) at scala.collection.immutable.List.map(List.scala:79) at org.apache.spark.sql.catalyst.plans.logical.Project.output(basicLogicalOperators.scala:74) at org.apache.spark.sql.hive.HiveAnalysis$$anonfun$apply$3.applyOrElse(HiveStrategies.scala:166) at org.apache.spark.sql.hive.HiveAnalysis$$anonfun$apply$3.applyOrElse(HiveStrategies.scala:161) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.$anonfun$resolveOperatorsDownWithPruning$2(AnalysisHelper.scala:170) at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(origin.scala:83) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.$anonfun$resolveOperatorsDownWithPruning$1(AnalysisHelper.scala:170) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper$.allowInvokingTransformsInAnalyzer(AnalysisHelper.scala:323) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.resolveOperatorsDownWithPruning(AnalysisHelper.scala:168) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.resolveOperatorsDownWithPruning$(AnalysisHelper.scala:164) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolveOperatorsDownWithPruning(LogicalPlan.scala:33) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.resolveOperatorsWithPruning(AnalysisHelper.scala:99) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.resolveOperatorsWithPruning$(AnalysisHelper.scala:96) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolveOperatorsWithPruning(LogicalPlan.scala:33) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.resolveOperators(AnalysisHelper.scala:76) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.resolveOperators$(AnalysisHelper.scala:75) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolveOperators(LogicalPlan.scala:33) at org.apache.spark.sql.hive.HiveAnalysis$.apply(HiveStrategies.scala:161) at org.apache.spark.sql.hive.HiveAnalysis$.apply(HiveStrategies.scala:160) at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$2(RuleExecutor.scala:222) 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.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:333) at org.apache.spark.sql.catalyst.rules.RuleExecutor.execute(RuleExecutor.scala:211) at org.apache.spark.sql.catalyst.analysis.Analyzer.org$apache$spark$sql$catalyst$analysis$Analyzer$$executeSameContext(Analyzer.scala:224) at org.apache.spark.sql.catalyst.analysis.Analyzer.$anonfun$execute$1(Analyzer.scala:220) at org.apache.spark.sql.catalyst.analysis.AnalysisContext$.withNewAnalysisContext(Analyzer.scala:176) at org.apache.spark.sql.catalyst.analysis.Analyzer.execute(Analyzer.scala:220) at org.apache.spark.sql.catalyst.analysis.Analyzer.execute(Analyzer.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:89) at org.apache.spark.sql.catalyst.rules.RuleExecutor.executeAndTrack(RuleExecutor.scala:182) at org.apache.spark.sql.catalyst.analysis.Analyzer.$anonfun$executeAndCheck$1(Analyzer.scala:212) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper$.markInAnalyzer(AnalysisHelper.scala:330) at org.apache.spark.sql.catalyst.analysis.Analyzer.executeAndCheck(Analyzer.scala:211) at org.apache.spark.sql.execution.QueryExecution.$anonfun$analyzed$1(QueryExecution.scala:88) at org.apache.spark.sql.catalyst.QueryPlanningTracker.measurePhase(QueryPlanningTracker.scala:138) at org.apache.spark.sql.execution.QueryExecution.$anonfun$executePhase$2(QueryExecution.scala:230) at org.apache.spark.sql.execution.QueryExecution$.withInternalError(QueryExecution.scala:557) at org.apache.spark.sql.execution.QueryExecution.$anonfun$executePhase$1(QueryExecution.scala:230) at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:918) at org.apache.spark.sql.execution.QueryExecution.executePhase(QueryExecution.scala:229) at org.apache.spark.sql.execution.QueryExecution.analyzed$lzycompute(QueryExecution.scala:88) at org.apache.spark.sql.execution.QueryExecution.analyzed(QueryExecution.scala:85) at org.apache.spark.sql.execution.QueryExecution.assertAnalyzed(QueryExecution.scala:69) at org.apache.spark.sql.Dataset$.$anonfun$ofRows$2(Dataset.scala:101) at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:918) at org.apache.spark.sql.Dataset$.ofRows(Dataset.scala:99) at org.apache.spark.sql.SparkSession.$anonfun$sql$4(SparkSession.scala:708) at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:918) at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:699) at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:730) at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:761) ... 42 elided {quote} -- 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