From 2ed877c8774ac72124c7f445446c3150e7db2bc8 Mon Sep 17 00:00:00 2001 From: Serge Rielau Date: Thu, 28 Aug 2025 17:17:23 -0700 Subject: [PATCH 01/32] Redesign EXECUTE IMMEDIATE --- .../catalyst/analysis/executeImmediate.scala | 98 ++++----- .../sql/catalyst/rules/RuleIdCollection.scala | 1 + .../catalyst/analysis/ExecuteImmediate.scala | 207 ++++++++++++++++++ .../command/ExecuteImmediateCommandExec.scala | 174 +++++++++++++++ .../internal/BaseSessionStateBuilder.scala | 3 +- 5 files changed, 430 insertions(+), 53 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediate.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/command/ExecuteImmediateCommandExec.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala index b926cdf57f169..54ebb66489dba 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala @@ -19,9 +19,9 @@ package org.apache.spark.sql.catalyst.analysis import scala.util.{Either, Left, Right} -import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, VariableReference} +import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Expression, VariableReference} import org.apache.spark.sql.catalyst.parser.ParseException -import org.apache.spark.sql.catalyst.plans.logical.{CompoundBody, LogicalPlan, SetVariable} +import org.apache.spark.sql.catalyst.plans.logical.{CompoundBody, ExecutableDuringAnalysis, LocalRelation, LogicalPlan, UnaryNode} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.trees.TreePattern.{EXECUTE_IMMEDIATE, TreePattern} import org.apache.spark.sql.connector.catalog.CatalogManager @@ -44,8 +44,39 @@ case class ExecuteImmediateQuery( } /** - * This rule substitutes execute immediate query node with fully analyzed - * plan that is passed as string literal or session parameter. + * Logical plan representing a resolved execute immediate command that will recursively + * invoke SQL execution. + * + * @param args parameters of query + * @param query query string or variable + * @param targetVariables variables to store the result of the query + */ +case class ExecuteImmediateCommand( + args: Seq[Expression], + query: Either[String, VariableReference], + targetVariables: Seq[VariableReference]) + extends UnaryNode with ExecutableDuringAnalysis { + + final override val nodePatterns: Seq[TreePattern] = Seq(EXECUTE_IMMEDIATE) + + override def child: LogicalPlan = LocalRelation(Nil, Nil) + + override def output: Seq[Attribute] = child.output + + override def stageForExplain(): LogicalPlan = { + // For EXPLAIN, just show the command without executing it + copy() + } + + override protected def withNewChildInternal( + newChild: LogicalPlan): ExecuteImmediateCommand = { + copy() + } +} + +/** + * This rule substitutes execute immediate query node with a command node + * that will handle recursive SQL execution. */ class SubstituteExecuteImmediate( val catalogManager: CatalogManager, @@ -114,57 +145,20 @@ class SubstituteExecuteImmediate( case ExecuteImmediateQuery(expressions, query, targetVariables) if expressions.forall(_.resolved) => - - val queryString = extractQueryString(query) - val plan = parseStatement(queryString, targetVariables) - - val posNodes = plan.collect { case p: LogicalPlan => - p.expressions.flatMap(_.collect { case n: PosParameter => n }) - }.flatten - val namedNodes = plan.collect { case p: LogicalPlan => - p.expressions.flatMap(_.collect { case n: NamedParameter => n }) - }.flatten - - val queryPlan = if (expressions.isEmpty || (posNodes.isEmpty && namedNodes.isEmpty)) { - plan - } else if (posNodes.nonEmpty && namedNodes.nonEmpty) { - throw QueryCompilationErrors.invalidQueryMixedQueryParameters() - } else { - if (posNodes.nonEmpty) { - PosParameterizedQuery(plan, expressions) - } else { - val aliases = expressions.collect { - case e: Alias => e - case u: VariableReference => Alias(u, u.identifier.name())() - } - - if (aliases.size != expressions.size) { - val nonAliases = expressions.filter(attr => - !attr.isInstanceOf[Alias] && !attr.isInstanceOf[VariableReference]) - - throw QueryCompilationErrors.invalidQueryAllParametersMustBeNamed(nonAliases) - } - - NameParameterizedQuery( - plan, - aliases.map(_.name), - // We need to resolve arguments before Resolution batch to make sure - // that some rule does not accidentally resolve our parameters. - // We do not want this as they can resolve some unsupported parameters. - aliases) - } + // Resolve query variable reference if it's a variable + val resolvedQuery = query match { + case Left(str) => Left(str) + case Right(attr) => + val variable = getVariableReference(attr, attr.nameParts) + Right(variable) } - // Fully analyze the generated plan. AnalysisContext.withExecuteImmediateContext makes sure - // that SQL scripting local variables will not be accessed from the plan. - val finalPlan = AnalysisContext.withExecuteImmediateContext { - resolveChild(queryPlan) - } - checkAnalysis(finalPlan) + // Resolve target variables + val resolvedTargetVars = targetVariables.map(attr => + getVariableReference(attr, attr.nameParts)) - if (targetVariables.nonEmpty) { - SetVariable(targetVariables, finalPlan) - } else { finalPlan } + // Transform into a command that will handle recursive execution + ExecuteImmediateCommand(expressions, resolvedQuery, resolvedTargetVars) } private def parseStatement( diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleIdCollection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleIdCollection.scala index e7b59af5e7766..a5ed8d730740a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleIdCollection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleIdCollection.scala @@ -87,6 +87,7 @@ object RuleIdCollection { "org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases" :: "org.apache.spark.sql.catalyst.analysis.EliminateUnions" :: "org.apache.spark.sql.catalyst.analysis.ResolveDefaultColumns" :: + "org.apache.spark.sql.catalyst.analysis.ExecuteExecutableDuringAnalysis" :: "org.apache.spark.sql.catalyst.analysis.ResolveExpressionsWithNamePlaceholders" :: "org.apache.spark.sql.catalyst.analysis.ResolveHints$ResolveCoalesceHints" :: "org.apache.spark.sql.catalyst.analysis.ResolveHints$ResolveJoinStrategyHints" :: diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediate.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediate.scala new file mode 100644 index 0000000000000..ea9e8b429337a --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediate.scala @@ -0,0 +1,207 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.analysis + +import scala.util.{Either, Left, Right} + +import org.apache.spark.sql.{AnalysisException, SparkSession} +import org.apache.spark.sql.catalyst.expressions.VariableReference +import org.apache.spark.sql.catalyst.plans.logical.{CompoundBody, LocalRelation, LogicalPlan} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.catalyst.trees.TreePattern.EXECUTE_IMMEDIATE +import org.apache.spark.sql.errors.QueryCompilationErrors +import org.apache.spark.sql.types.StringType + +/** + * Analysis rule that executes ExecuteImmediateCommand during analysis and replaces it + * with the results, similar to how CALL statements work. + */ +case class ExecuteExecutableDuringAnalysis(sparkSession: SparkSession) extends Rule[LogicalPlan] { + + override def apply(plan: LogicalPlan): LogicalPlan = { + plan.resolveOperatorsWithPruning(_.containsPattern(EXECUTE_IMMEDIATE), ruleId) { + case cmd: ExecuteImmediateCommand => + executeImmediate(cmd) + } + } + + private def executeImmediate(cmd: ExecuteImmediateCommand): LogicalPlan = { + try { + // Extract the query string + val queryString = extractQueryString(cmd.query) + + // Parse and validate the query + val parsedPlan = sparkSession.sessionState.sqlParser.parsePlan(queryString) + validateQuery(queryString, parsedPlan) + + // Execute the query recursively + val result = if (cmd.args.isEmpty) { + // No parameters - execute directly + AnalysisContext.withExecuteImmediateContext { + sparkSession.sql(queryString) + } + } else { + // For parameterized queries, substitute parameters manually + val paramValues = cmd.args.map(_.eval(null)) + val substitutedQuery = substituteParameters(queryString, paramValues) + AnalysisContext.withExecuteImmediateContext { + sparkSession.sql(substitutedQuery) + } + } + + // Handle target variables if specified + if (cmd.targetVariables.nonEmpty) { + handleTargetVariables(result, cmd.targetVariables) + // Return empty relation for INTO queries + LocalRelation(Nil, Nil) + } else { + // Return the query results as a LocalRelation + val internalRows = result.queryExecution.executedPlan.executeCollect() + LocalRelation(result.queryExecution.analyzed.output, internalRows.toSeq) + } + + } catch { + case e: AnalysisException => + // Re-throw AnalysisException as-is to preserve error type for tests + throw e + case e: Exception => + throw new RuntimeException(s"Failed to execute immediate query: ${e.getMessage}", e) + } + } + + private def extractQueryString(queryExpr: Either[String, VariableReference]): String = { + queryExpr match { + case Left(literal) => literal + case Right(variable) => + // Evaluate the variable reference + if (!variable.dataType.sameType(StringType)) { + throw QueryCompilationErrors.invalidExecuteImmediateVariableType(variable.dataType) + } + + val value = variable.eval(null) + if (value == null) { + throw QueryCompilationErrors.nullSQLStringExecuteImmediate(variable.identifier.name()) + } + + value.toString + } + } + + private def validateQuery(queryString: String, parsedPlan: LogicalPlan): Unit = { + // Check for compound bodies (SQL scripting) + if (parsedPlan.isInstanceOf[CompoundBody]) { + throw QueryCompilationErrors.sqlScriptInExecuteImmediate(queryString) + } + + // Check for nested EXECUTE IMMEDIATE + if (parsedPlan.containsPattern(EXECUTE_IMMEDIATE)) { + throw QueryCompilationErrors.nestedExecuteImmediate(queryString) + } + } + + private def substituteParameters(queryString: String, paramValues: Seq[Any]): String = { + var substituted = queryString + var paramIndex = 0 + + // Handle positional parameters (?) + while (substituted.contains("?") && paramIndex < paramValues.length) { + val value = paramValues(paramIndex) + val sqlLiteral = if (value == null) { + "NULL" + } else { + value match { + case s: String => s"'$s'" + case n: Number => n.toString + case b: Boolean => b.toString + case _ => s"'$value'" + } + } + substituted = substituted.replaceFirst("\\?", sqlLiteral) + paramIndex += 1 + } + + substituted + } + + private def handleTargetVariables( + result: org.apache.spark.sql.DataFrame, + targetVariables: Seq[VariableReference]): Unit = { + // Collect the results from the query + val values = result.queryExecution.executedPlan.executeCollect() + + if (values.length == 0) { + // No rows: Set all variables to null + targetVariables.foreach { variable => + setVariable(variable, null) + } + } else if (values.length > 1) { + // Multiple rows: Error + throw new RuntimeException( + "EXECUTE IMMEDIATE ... INTO query returned more than one row") + } else { + // Exactly one row: Set each variable to the corresponding column value + val row = values(0) + targetVariables.zipWithIndex.foreach { case (variable, index) => + val value = row.get(index, variable.dataType) + setVariable(variable, value) + } + } + } + + private def setVariable(variable: VariableReference, value: Any): Unit = { + import java.util.Locale + import org.apache.spark.sql.catalyst.{SqlScriptingContextManager} + import org.apache.spark.sql.catalyst.analysis.{FakeLocalCatalog, FakeSystemCatalog} + import org.apache.spark.sql.catalyst.catalog.VariableDefinition + import org.apache.spark.sql.catalyst.expressions.Literal + import org.apache.spark.sql.errors.QueryCompilationErrors.unresolvedVariableError + + val namePartsCaseAdjusted = if (sparkSession.sessionState.conf.caseSensitiveAnalysis) { + variable.originalNameParts + } else { + variable.originalNameParts.map(_.toLowerCase(Locale.ROOT)) + } + + val tempVariableManager = sparkSession.sessionState.catalogManager.tempVariableManager + val scriptingVariableManager = SqlScriptingContextManager.get().map(_.getVariableManager) + + val variableManager = variable.catalog match { + case FakeLocalCatalog if scriptingVariableManager.isEmpty => + throw new RuntimeException("SetVariable: Variable has FakeLocalCatalog, " + + "but ScriptingVariableManager is None.") + + case FakeLocalCatalog if scriptingVariableManager.get.get(namePartsCaseAdjusted).isEmpty => + throw new RuntimeException("Local variable should be present in SetVariable" + + "because ResolveSetVariable has already determined it exists.") + + case FakeLocalCatalog => scriptingVariableManager.get + + case FakeSystemCatalog if tempVariableManager.get(namePartsCaseAdjusted).isEmpty => + throw unresolvedVariableError(namePartsCaseAdjusted, Seq("SYSTEM", "SESSION")) + + case FakeSystemCatalog => tempVariableManager + + case c => throw new RuntimeException("Unexpected catalog in SetVariable: " + c) + } + + val varDef = VariableDefinition( + variable.identifier, variable.varDef.defaultValueSQL, Literal(value, variable.dataType)) + + variableManager.set(namePartsCaseAdjusted, varDef) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ExecuteImmediateCommandExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ExecuteImmediateCommandExec.scala new file mode 100644 index 0000000000000..14d136564260b --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ExecuteImmediateCommandExec.scala @@ -0,0 +1,174 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.command + +import scala.util.{Either, Left, Right} + +import org.apache.spark.sql.{AnalysisException, DataFrame, Row, SparkSession} +import org.apache.spark.sql.catalyst.analysis.AnalysisContext +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, VariableReference} +import org.apache.spark.sql.catalyst.plans.logical.{CompoundBody, LogicalPlan} +import org.apache.spark.sql.catalyst.trees.TreePattern.EXECUTE_IMMEDIATE +import org.apache.spark.sql.errors.QueryCompilationErrors +import org.apache.spark.sql.execution.command.LeafRunnableCommand +import org.apache.spark.sql.types.StringType + +/** + * Runnable command that executes an EXECUTE IMMEDIATE statement by recursively + * invoking the SQL execution pipeline. + */ +case class ExecuteImmediateCommandExec( + args: Seq[Expression], + query: Either[String, VariableReference], + targetVariables: Seq[VariableReference]) extends LeafRunnableCommand { + + override def output: Seq[Attribute] = { + if (targetVariables.nonEmpty) { + // For EXECUTE IMMEDIATE ... INTO, return empty output + Seq.empty + } else { + // For regular EXECUTE IMMEDIATE, get the schema from the executed query + // This is a bit of a hack, but necessary for the type system + try { + val queryString = extractQueryString(null, query) // Use null SparkSession for schema only + // We can't execute the query here due to circular dependencies + // Return a placeholder that will be corrected during execution + Seq.empty + } catch { + case _: Exception => Seq.empty + } + } + } + + override def run(sparkSession: SparkSession): Seq[Row] = { + // Extract the query string + val queryString = extractQueryString(sparkSession, query) + + // Prepare arguments for parameterized queries + val resolvedArgs = resolveArguments(sparkSession, args) + + try { + // Parse the query to understand its structure + val parsedPlan = sparkSession.sessionState.sqlParser.parsePlan(queryString) + + // Check for unsupported features + validateQuery(queryString, parsedPlan) + + // Execute the query recursively through the SQL execution pipeline + val result = executeQuery(sparkSession, queryString, parsedPlan, resolvedArgs) + + // Handle target variables if specified + if (targetVariables.nonEmpty) { + handleTargetVariables(sparkSession, result, targetVariables) + Seq.empty // No output for INTO queries + } else { + // For regular EXECUTE IMMEDIATE, we can't return rows via RunnableCommand + // because of schema mismatch issues. The CommandResult mechanism will handle this. + // For now, return empty and let the test fail with a clearer error + throw new UnsupportedOperationException( + "EXECUTE IMMEDIATE with result return is not yet fully implemented. " + + "The query was executed but results cannot be returned via RunnableCommand " + + "due to schema limitations.") + } + } catch { + case e: AnalysisException => + // Re-throw AnalysisException as-is to preserve error type for tests + throw e + case e: Exception => + throw new RuntimeException(s"Failed to execute immediate query: ${e.getMessage}", e) + } + } + + private def extractQueryString( + sparkSession: SparkSession, + queryExpr: Either[String, VariableReference]): String = { + queryExpr match { + case Left(literal) => literal + case Right(variable) => + // Evaluate the variable reference + if (!variable.dataType.sameType(StringType)) { + throw QueryCompilationErrors.invalidExecuteImmediateVariableType(variable.dataType) + } + + val value = variable.eval(null) + if (value == null) { + throw QueryCompilationErrors.nullSQLStringExecuteImmediate(variable.identifier.name()) + } + + value.toString + } + } + + private def resolveArguments( + sparkSession: SparkSession, + expressions: Seq[Expression]): Seq[Expression] = { + expressions.map { expr => + if (expr.resolved) { + expr + } else { + // For now, just return the expression as-is + // In a complete implementation, you would need to resolve variables/parameters + expr + } + } + } + + private def validateQuery(queryString: String, parsedPlan: LogicalPlan): Unit = { + // Check for compound bodies (SQL scripting) + if (parsedPlan.isInstanceOf[CompoundBody]) { + throw QueryCompilationErrors.sqlScriptInExecuteImmediate(queryString) + } + + // Check for nested EXECUTE IMMEDIATE + if (parsedPlan.containsPattern(EXECUTE_IMMEDIATE)) { + throw QueryCompilationErrors.nestedExecuteImmediate(queryString) + } + } + + private def executeQuery( + sparkSession: SparkSession, + queryString: String, + parsedPlan: LogicalPlan, + resolvedArgs: Seq[Expression]): DataFrame = { + + // For now, use the SparkSession.sql method which handles parameterization + // This is the recursive SQL execution we want + if (resolvedArgs.isEmpty) { + // Execute within EXECUTE IMMEDIATE context to isolate variables + AnalysisContext.withExecuteImmediateContext { + sparkSession.sql(queryString) + } + } else { + // For parameterized queries, convert resolved args to values and pass them to sql() + val paramValues = resolvedArgs.map(_.eval(null)) + AnalysisContext.withExecuteImmediateContext { + sparkSession.sql(queryString, paramValues.toArray) + } + } + } + + private def handleTargetVariables( + sparkSession: SparkSession, + result: DataFrame, + targetVars: Seq[VariableReference]): Unit = { + // This would need to set session variables with the query results + // For now, we'll throw an error as this functionality would need additional implementation + throw new UnsupportedOperationException( + "EXECUTE IMMEDIATE ... INTO with recursive execution is not yet implemented") + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala index ada30cde27cd0..21bd337200c9c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.internal import org.apache.spark.annotation.Unstable import org.apache.spark.sql.{DataSourceRegistration, ExperimentalMethods, SparkSessionExtensions, UDTFRegistration} import org.apache.spark.sql.artifact.ArtifactManager -import org.apache.spark.sql.catalyst.analysis.{Analyzer, EvalSubqueriesForTimeTravel, FunctionRegistry, InvokeProcedures, ReplaceCharWithVarchar, ResolveDataSource, ResolveSessionCatalog, ResolveTranspose, TableFunctionRegistry} +import org.apache.spark.sql.catalyst.analysis.{Analyzer, EvalSubqueriesForTimeTravel, ExecuteExecutableDuringAnalysis, FunctionRegistry, InvokeProcedures, ReplaceCharWithVarchar, ResolveDataSource, ResolveSessionCatalog, ResolveTranspose, TableFunctionRegistry} import org.apache.spark.sql.catalyst.analysis.resolver.ResolverExtension import org.apache.spark.sql.catalyst.catalog.{FunctionExpressionBuilder, SessionCatalog} import org.apache.spark.sql.catalyst.expressions.{Expression, ExtractSemiStructuredFields} @@ -244,6 +244,7 @@ abstract class BaseSessionStateBuilder( new EvalSubqueriesForTimeTravel +: new ResolveTranspose(session) +: new InvokeProcedures(session) +: + ExecuteExecutableDuringAnalysis(session) +: ExtractSemiStructuredFields +: customResolutionRules From eb92a0cefba079e6536e0bb13593131e5479f581 Mon Sep 17 00:00:00 2001 From: Serge Rielau Date: Thu, 28 Aug 2025 21:17:27 -0700 Subject: [PATCH 02/32] Named parameter support --- .../catalyst/analysis/ExecuteImmediate.scala | 123 +++++++++++++++--- 1 file changed, 108 insertions(+), 15 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediate.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediate.scala index ea9e8b429337a..3283f2ad98786 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediate.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.analysis import scala.util.{Either, Left, Right} import org.apache.spark.sql.{AnalysisException, SparkSession} -import org.apache.spark.sql.catalyst.expressions.VariableReference +import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, VariableReference} import org.apache.spark.sql.catalyst.plans.logical.{CompoundBody, LocalRelation, LogicalPlan} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.trees.TreePattern.EXECUTE_IMMEDIATE @@ -56,11 +56,25 @@ case class ExecuteExecutableDuringAnalysis(sparkSession: SparkSession) extends R sparkSession.sql(queryString) } } else { - // For parameterized queries, substitute parameters manually - val paramValues = cmd.args.map(_.eval(null)) - val substitutedQuery = substituteParameters(queryString, paramValues) + // For parameterized queries, build parameter map from USING clause + // The args are already resolved by outer parameter resolution + val (positionalParams, namedParams) = separateParameters(cmd.args) + AnalysisContext.withExecuteImmediateContext { - sparkSession.sql(substitutedQuery) + if (namedParams.nonEmpty && positionalParams.isEmpty) { + // Only named parameters: use Map overload + sparkSession.sql(queryString, namedParams) + } else if (positionalParams.nonEmpty && namedParams.isEmpty) { + // Only positional parameters: use Array overload + sparkSession.sql(queryString, positionalParams.toArray) + } else if (namedParams.isEmpty && positionalParams.isEmpty) { + // No parameters + sparkSession.sql(queryString) + } else { + // Mixed parameters: not directly supported, need manual substitution + val substitutedQuery = substituteNamedParameters(queryString, namedParams) + sparkSession.sql(substitutedQuery, positionalParams.toArray) + } } } @@ -115,22 +129,15 @@ case class ExecuteExecutableDuringAnalysis(sparkSession: SparkSession) extends R } private def substituteParameters(queryString: String, paramValues: Seq[Any]): String = { + // For now, just handle positional parameters + // Named parameters require more complex resolution that involves the original args expressions var substituted = queryString var paramIndex = 0 // Handle positional parameters (?) while (substituted.contains("?") && paramIndex < paramValues.length) { val value = paramValues(paramIndex) - val sqlLiteral = if (value == null) { - "NULL" - } else { - value match { - case s: String => s"'$s'" - case n: Number => n.toString - case b: Boolean => b.toString - case _ => s"'$value'" - } - } + val sqlLiteral = formatSqlLiteral(value) substituted = substituted.replaceFirst("\\?", sqlLiteral) paramIndex += 1 } @@ -138,6 +145,92 @@ case class ExecuteExecutableDuringAnalysis(sparkSession: SparkSession) extends R substituted } + private def substituteParametersWithNames( + queryString: String, + args: Seq[Expression]): String = { + try { + var substituted = queryString + val paramMap = scala.collection.mutable.Map[String, Any]() + var positionalIndex = 0 + + // Build parameter map from args + args.foreach { + case alias: Alias => + // Named parameter: "value AS paramName" + val paramName = alias.name + val paramValue = alias.child.eval(null) + paramMap(paramName) = paramValue + case expr => + // Positional parameter: just a value + val paramValue = expr.eval(null) + // Handle positional parameters first + if (substituted.contains("?")) { + val sqlLiteral = formatSqlLiteral(paramValue) + substituted = substituted.replaceFirst("\\?", sqlLiteral) + } + positionalIndex += 1 + } + + // Substitute named parameters (:paramName) + paramMap.foreach { case (paramName, paramValue) => + val sqlLiteral = formatSqlLiteral(paramValue) + val pattern = s":$paramName\\b" // Use word boundary to avoid partial matches + substituted = substituted.replaceAll(pattern, sqlLiteral) + } + + substituted + } catch { + case e: Exception => + throw new RuntimeException( + s"Error in parameter substitution for query '$queryString'", e) + } + } + + private def separateParameters(args: Seq[Expression]): (Seq[Any], Map[String, Any]) = { + val positionalParams = scala.collection.mutable.ListBuffer[Any]() + val namedParams = scala.collection.mutable.Map[String, Any]() + + args.foreach { + case alias: Alias => + // Named parameter: "value AS paramName" + val paramName = alias.name + val paramValue = alias.child.eval(null) + namedParams(paramName) = paramValue + case expr => + // Positional parameter: just a value + val paramValue = expr.eval(null) + positionalParams += paramValue + } + + (positionalParams.toSeq, namedParams.toMap) + } + + private def substituteNamedParameters( + queryString: String, namedParams: Map[String, Any]): String = { + var substituted = queryString + // Substitute named parameters (:paramName) + namedParams.foreach { case (paramName, paramValue) => + val sqlLiteral = formatSqlLiteral(paramValue) + val pattern = s":$paramName\\b" // Use word boundary to avoid partial matches + substituted = substituted.replaceAll(pattern, sqlLiteral) + } + + substituted + } + + private def formatSqlLiteral(value: Any): String = { + if (value == null) { + "NULL" + } else { + value match { + case s: String => s"'$s'" + case n: Number => n.toString + case b: Boolean => b.toString + case _ => s"'$value'" + } + } + } + private def handleTargetVariables( result: org.apache.spark.sql.DataFrame, targetVariables: Seq[VariableReference]): Unit = { From ab76d3ffbd7e7b8e0c4334e16a94d3d02a0b2158 Mon Sep 17 00:00:00 2001 From: Serge Rielau Date: Thu, 28 Aug 2025 21:39:14 -0700 Subject: [PATCH 03/32] INTO clause support --- .../catalyst/analysis/ExecuteImmediate.scala | 43 ++++++++++--------- 1 file changed, 22 insertions(+), 21 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediate.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediate.scala index 3283f2ad98786..e5b00d0c4cdf5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediate.scala @@ -234,6 +234,14 @@ case class ExecuteExecutableDuringAnalysis(sparkSession: SparkSession) extends R private def handleTargetVariables( result: org.apache.spark.sql.DataFrame, targetVariables: Seq[VariableReference]): Unit = { + // Ensure all target variables are resolved + targetVariables.foreach { variable => + if (!variable.resolved) { + throw org.apache.spark.SparkException.internalError( + s"Target variable ${variable.identifier} is not resolved") + } + } + // Collect the results from the query val values = result.queryExecution.executedPlan.executeCollect() @@ -244,8 +252,10 @@ case class ExecuteExecutableDuringAnalysis(sparkSession: SparkSession) extends R } } else if (values.length > 1) { // Multiple rows: Error - throw new RuntimeException( - "EXECUTE IMMEDIATE ... INTO query returned more than one row") + throw new org.apache.spark.SparkException( + errorClass = "ROW_SUBQUERY_TOO_MANY_ROWS", + messageParameters = Map.empty, + cause = null) } else { // Exactly one row: Set each variable to the corresponding column value val row = values(0) @@ -258,11 +268,10 @@ case class ExecuteExecutableDuringAnalysis(sparkSession: SparkSession) extends R private def setVariable(variable: VariableReference, value: Any): Unit = { import java.util.Locale - import org.apache.spark.sql.catalyst.{SqlScriptingContextManager} + import org.apache.spark.sql.catalyst.SqlScriptingContextManager import org.apache.spark.sql.catalyst.analysis.{FakeLocalCatalog, FakeSystemCatalog} import org.apache.spark.sql.catalyst.catalog.VariableDefinition import org.apache.spark.sql.catalyst.expressions.Literal - import org.apache.spark.sql.errors.QueryCompilationErrors.unresolvedVariableError val namePartsCaseAdjusted = if (sparkSession.sessionState.conf.caseSensitiveAnalysis) { variable.originalNameParts @@ -270,26 +279,18 @@ case class ExecuteExecutableDuringAnalysis(sparkSession: SparkSession) extends R variable.originalNameParts.map(_.toLowerCase(Locale.ROOT)) } - val tempVariableManager = sparkSession.sessionState.catalogManager.tempVariableManager - val scriptingVariableManager = SqlScriptingContextManager.get().map(_.getVariableManager) - + // Variable should already be resolved, so we can trust its catalog information val variableManager = variable.catalog match { - case FakeLocalCatalog if scriptingVariableManager.isEmpty => - throw new RuntimeException("SetVariable: Variable has FakeLocalCatalog, " + - "but ScriptingVariableManager is None.") - - case FakeLocalCatalog if scriptingVariableManager.get.get(namePartsCaseAdjusted).isEmpty => - throw new RuntimeException("Local variable should be present in SetVariable" + - "because ResolveSetVariable has already determined it exists.") - - case FakeLocalCatalog => scriptingVariableManager.get - - case FakeSystemCatalog if tempVariableManager.get(namePartsCaseAdjusted).isEmpty => - throw unresolvedVariableError(namePartsCaseAdjusted, Seq("SYSTEM", "SESSION")) + case FakeLocalCatalog => + SqlScriptingContextManager.get().map(_.getVariableManager).getOrElse( + throw org.apache.spark.SparkException.internalError( + "Variable has FakeLocalCatalog but ScriptingVariableManager is None")) - case FakeSystemCatalog => tempVariableManager + case FakeSystemCatalog => + sparkSession.sessionState.catalogManager.tempVariableManager - case c => throw new RuntimeException("Unexpected catalog in SetVariable: " + c) + case c => + throw org.apache.spark.SparkException.internalError(s"Unexpected catalog: $c") } val varDef = VariableDefinition( From 0e151c80b43f906b760c52540748596342a451c8 Mon Sep 17 00:00:00 2001 From: Serge Rielau Date: Fri, 29 Aug 2025 10:16:38 -0700 Subject: [PATCH 04/32] Phase 3 --- .../sql/catalyst/parser/SqlBaseParser.g4 | 15 +---- .../sql/catalyst/analysis/Analyzer.scala | 7 +- .../analysis/ColumnResolutionHelper.scala | 64 +++++++++++++++++++ .../catalyst/analysis/executeImmediate.scala | 55 +++++++++------- .../sql/catalyst/parser/AstBuilder.scala | 11 +--- .../analysis/AnalysisErrorSuite.scala | 10 +-- .../sql/catalyst/analysis/AnalysisSuite.scala | 32 +--------- .../catalyst/analysis/ExecuteImmediate.scala | 53 +++++++++------ .../sql/hive/HiveSessionStateBuilder.scala | 7 +- 9 files changed, 145 insertions(+), 109 deletions(-) diff --git a/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 b/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 index e63a229a32078..8efab99d4ec83 100644 --- a/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 +++ b/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 @@ -397,7 +397,7 @@ setResetStatement ; executeImmediate - : EXECUTE IMMEDIATE queryParam=executeImmediateQueryParam (INTO targetVariable=multipartIdentifierList)? executeImmediateUsing? + : EXECUTE IMMEDIATE queryParam=expression (INTO targetVariable=multipartIdentifierList)? executeImmediateUsing? ; executeImmediateUsing @@ -405,19 +405,6 @@ executeImmediateUsing | USING params=namedExpressionSeq ; -executeImmediateQueryParam - : stringLit - | multipartIdentifier - ; - -executeImmediateArgument - : (constant|multipartIdentifier) (AS name=errorCapturingIdentifier)? - ; - -executeImmediateArgumentSeq - : executeImmediateArgument (COMMA executeImmediateArgument)* - ; - timezone : stringLit | LOCAL diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 1896a1c7ac279..8ff7215697aaf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -495,10 +495,6 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor RewriteMergeIntoTable :: MoveParameterizedQueriesDown :: BindParameters :: - new SubstituteExecuteImmediate( - catalogManager, - resolveChild = executeSameContext, - checkAnalysis = checkAnalysis) :: typeCoercionRules() ++ Seq( ResolveWithCTE, @@ -1802,8 +1798,7 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor case s: Sort if !s.resolved || s.missingInput.nonEmpty => resolveReferencesInSort(s) - // Pass for Execute Immediate as arguments will be resolved by [[SubstituteExecuteImmediate]]. - case e : ExecuteImmediateQuery => e + // Remove this case - let ResolveReferences handle ExecuteImmediateQuery expressions normally case d: DataFrameDropColumns if !d.resolved => resolveDataFrameDropColumns(d) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ColumnResolutionHelper.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ColumnResolutionHelper.scala index bade5f0bee9d6..a880c7913bfba 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ColumnResolutionHelper.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ColumnResolutionHelper.scala @@ -233,6 +233,70 @@ trait ColumnResolutionHelper extends Logging with DataTypeErrorsBase { } } + /** + * Look up variable by nameParts. + * If in SQL Script, first check local variables, unless in EXECUTE IMMEDIATE + * (EXECUTE IMMEDIATE generated query cannot access local variables). + * if not found fall back to session variables. + * @param nameParts NameParts of the variable. + * @return Reference to the variable. + */ + def lookupVariable(nameParts: Seq[String]): Option[VariableReference] = { + // The temp variables live in `SYSTEM.SESSION`, and the name can be qualified or not. + def maybeTempVariableName(nameParts: Seq[String]): Boolean = { + nameParts.length == 1 || { + if (nameParts.length == 2) { + nameParts.head.equalsIgnoreCase(CatalogManager.SESSION_NAMESPACE) + } else if (nameParts.length == 3) { + nameParts(0).equalsIgnoreCase(CatalogManager.SYSTEM_CATALOG_NAME) && + nameParts(1).equalsIgnoreCase(CatalogManager.SESSION_NAMESPACE) + } else { + false + } + } + } + + val namePartsCaseAdjusted = if (conf.caseSensitiveAnalysis) { + nameParts + } else { + nameParts.map(_.toLowerCase(Locale.ROOT)) + } + + SqlScriptingContextManager.get().map(_.getVariableManager) + // In EXECUTE IMMEDIATE context, only allow session variables (system.session.X), + // not local variables + .filterNot { _ => + AnalysisContext.get.isExecuteImmediate && !maybeTempVariableName(nameParts) + } + // If variable name is qualified with session. treat it as a session variable. + .filterNot(_ => + nameParts.length > 2 + || (nameParts.length == 2 && isForbiddenLabelOrForVariableName(nameParts.head))) + .flatMap(_.get(namePartsCaseAdjusted)) + .map { varDef => + VariableReference( + nameParts, + FakeLocalCatalog, + Identifier.of(Array(varDef.identifier.namespace().last), namePartsCaseAdjusted.last), + varDef) + } + .orElse( + if (maybeTempVariableName(nameParts)) { + catalogManager.tempVariableManager + .get(namePartsCaseAdjusted) + .map { varDef => + VariableReference( + nameParts, + FakeSystemCatalog, + Identifier.of(Array(CatalogManager.SESSION_NAMESPACE), namePartsCaseAdjusted.last), + varDef + )} + } else { + None + } + ) + } + // Resolves `UnresolvedAttribute` to its value. protected def resolveVariables(e: Expression): Expression = { val variableResolution = new VariableResolution(catalogManager.tempVariableManager) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala index 54ebb66489dba..663f5bc4d1485 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala @@ -31,15 +31,16 @@ import org.apache.spark.sql.types.StringType /** * Logical plan representing execute immediate query. * - * @param args parameters of query - * @param query query string or variable + * @param queryParam the query expression (first child) + * @param args parameters from USING clause (subsequent children) * @param targetVariables variables to store the result of the query */ case class ExecuteImmediateQuery( + queryParam: Expression, args: Seq[Expression], - query: Either[String, UnresolvedAttribute], targetVariables: Seq[UnresolvedAttribute]) extends UnresolvedLeafNode { + final override val nodePatterns: Seq[TreePattern] = Seq(EXECUTE_IMMEDIATE) } @@ -47,13 +48,13 @@ case class ExecuteImmediateQuery( * Logical plan representing a resolved execute immediate command that will recursively * invoke SQL execution. * - * @param args parameters of query - * @param query query string or variable + * @param queryParam the resolved query expression + * @param args parameters from USING clause * @param targetVariables variables to store the result of the query */ case class ExecuteImmediateCommand( + queryParam: Expression, args: Seq[Expression], - query: Either[String, VariableReference], targetVariables: Seq[VariableReference]) extends UnaryNode with ExecutableDuringAnalysis { @@ -140,25 +141,22 @@ class SubstituteExecuteImmediate( override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsWithPruning(_.containsPattern(EXECUTE_IMMEDIATE), ruleId) { - case e @ ExecuteImmediateQuery(expressions, _, _) if expressions.exists(!_.resolved) => - e.copy(args = resolveArguments(expressions)) - - case ExecuteImmediateQuery(expressions, query, targetVariables) - if expressions.forall(_.resolved) => - // Resolve query variable reference if it's a variable - val resolvedQuery = query match { - case Left(str) => Left(str) - case Right(attr) => - val variable = getVariableReference(attr, attr.nameParts) - Right(variable) + case e @ ExecuteImmediateQuery(queryParam, args, targetVariables) => + // Check if all expressions are resolved (they should be resolved by ResolveReferences now) + val queryParamResolved = queryParam.resolved + val allArgsResolved = args.forall(_.resolved) + val targetVariablesResolved = targetVariables.forall(_.resolved) + + if (queryParamResolved && allArgsResolved && targetVariablesResolved) { + // All resolved - transform to command + val finalTargetVars = targetVariables.map(attr => + getVariableReference(attr, attr.nameParts)) + + ExecuteImmediateCommand(queryParam, args, finalTargetVars) + } else { + // Not all resolved yet - wait for next iteration + e } - - // Resolve target variables - val resolvedTargetVars = targetVariables.map(attr => - getVariableReference(attr, attr.nameParts)) - - // Transform into a command that will handle recursive execution - ExecuteImmediateCommand(expressions, resolvedQuery, resolvedTargetVars) } private def parseStatement( @@ -195,6 +193,15 @@ class SubstituteExecuteImmediate( plan } + + + private def isQueryResolved(query: Either[String, UnresolvedAttribute]): Boolean = { + query match { + case Left(_) => true // String literals are always resolved + case Right(attr) => attr.resolved // Check if the attribute is resolved + } + } + private def getVariableReference(expr: Expression, nameParts: Seq[String]): VariableReference = { variableResolution.lookupVariable( nameParts = nameParts, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 4e930280381c1..5872119b0e9e1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -22,7 +22,6 @@ import java.util.concurrent.TimeUnit import scala.collection.mutable.{ArrayBuffer, HashMap, ListBuffer, Set} import scala.jdk.CollectionConverters._ -import scala.util.{Left, Right} import org.antlr.v4.runtime.{ParserRuleContext, RuleContext, Token} import org.antlr.v4.runtime.tree.{ParseTree, RuleNode, TerminalNode} @@ -1157,11 +1156,8 @@ class AstBuilder extends DataTypeAstBuilder * }}} */ override def visitExecuteImmediate(ctx: ExecuteImmediateContext): LogicalPlan = withOrigin(ctx) { - // Because of how parsing rules are written, we know that either - // queryParam or targetVariable is non null - hence use Either to represent this. - val queryString = Option(ctx.queryParam.stringLit()).map(sl => Left(string(visitStringLit(sl)))) - val queryVariable = Option(ctx.queryParam.multipartIdentifier) - .map(mpi => Right(UnresolvedAttribute(visitMultipartIdentifier(mpi)))) + // With the new grammar, queryParam is now an expression + val queryParam = expression(ctx.queryParam) val targetVars = Option(ctx.targetVariable).toSeq .flatMap(v => visitMultipartIdentifierList(v)) @@ -1169,8 +1165,7 @@ class AstBuilder extends DataTypeAstBuilder visitExecuteImmediateUsing(_) }.getOrElse{ Seq.empty } - - ExecuteImmediateQuery(exprs, queryString.getOrElse(queryVariable.get), targetVars) + ExecuteImmediateQuery(queryParam, exprs, targetVars) } override def visitExecuteImmediateUsing( diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala index a301f77cf0c02..f551037e7a1a2 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala @@ -850,8 +850,8 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { test("EXEC IMMEDIATE - non string variable as sqlString parameter") { val execImmediatePlan = ExecuteImmediateQuery( + UnresolvedAttribute("testVarA"), Seq.empty, - scala.util.Right(UnresolvedAttribute("testVarA")), Seq(UnresolvedAttribute("testVarA"))) assertAnalysisErrorCondition( @@ -864,8 +864,8 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { test("EXEC IMMEDIATE - Null string as sqlString parameter") { val execImmediatePlan = ExecuteImmediateQuery( + UnresolvedAttribute("testVarNull"), Seq.empty, - scala.util.Right(UnresolvedAttribute("testVarNull")), Seq(UnresolvedAttribute("testVarNull"))) assertAnalysisErrorCondition( @@ -877,8 +877,8 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { test("EXEC IMMEDIATE - Unsupported expr for parameter") { val execImmediatePlan: LogicalPlan = ExecuteImmediateQuery( + Literal("SELECT ?"), Seq(UnresolvedAttribute("testVarA"), NaNvl(Literal(1), Literal(1))), - scala.util.Left("SELECT ?"), Seq.empty) assertAnalysisErrorCondition( @@ -891,8 +891,8 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { test("EXEC IMMEDIATE - Name Parametrize query with non named parameters") { val execImmediateSetVariablePlan = ExecuteImmediateQuery( + Literal("SELECT :first"), Seq(Literal(2), new Alias(UnresolvedAttribute("testVarA"), "first")(), Literal(3)), - scala.util.Left("SELECT :first"), Seq.empty) assertAnalysisErrorCondition( @@ -905,8 +905,8 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { test("EXEC IMMEDIATE - INTO specified for COMMAND query") { val execImmediateSetVariablePlan = ExecuteImmediateQuery( + Literal("SET VAR testVarA = 1"), Seq.empty, - scala.util.Left("SET VAR testVarA = 1"), Seq(UnresolvedAttribute("testVarA"))) assertAnalysisErrorCondition( diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala index 0c8d2bae418ad..5992f2d099bdc 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala @@ -30,7 +30,7 @@ import org.apache.spark.SparkException import org.apache.spark.api.python.PythonEvalType import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.{AliasIdentifier, QueryPlanningTracker, TableIdentifier} -import org.apache.spark.sql.catalyst.catalog.{InMemoryCatalog, SessionCatalog, VariableDefinition} +import org.apache.spark.sql.catalyst.catalog.{InMemoryCatalog, SessionCatalog} import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder @@ -42,7 +42,7 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, RangePartitioning, RoundRobinPartitioning} import org.apache.spark.sql.catalyst.types.DataTypeUtils import org.apache.spark.sql.catalyst.util._ -import org.apache.spark.sql.connector.catalog.{Identifier, InMemoryTable} +import org.apache.spark.sql.connector.catalog.InMemoryTable import org.apache.spark.sql.errors.QueryCompilationErrors import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.internal.SQLConf @@ -1520,34 +1520,6 @@ class AnalysisSuite extends AnalysisTest with Matchers { assertAnalysisSuccess(finalPlan) } - test("Execute Immediate plan transformation") { - try { - val varDef1 = VariableDefinition(Identifier.of(Array("res"), "res"), "1", Literal(1)) - SimpleAnalyzer.catalogManager.tempVariableManager.create( - Seq("res", "res"), varDef1, overrideIfExists = true) - - val varDef2 = VariableDefinition(Identifier.of(Array("res2"), "res2"), "1", Literal(1)) - SimpleAnalyzer.catalogManager.tempVariableManager.create( - Seq("res2", "res2"), varDef2, overrideIfExists = true) - val actual1 = parsePlan("EXECUTE IMMEDIATE 'SELECT 42 WHERE ? = 1' USING 2").analyze - val expected1 = parsePlan("SELECT 42 where 2 = 1").analyze - comparePlans(actual1, expected1) - val actual2 = parsePlan( - "EXECUTE IMMEDIATE 'SELECT 42 WHERE :first = 1' USING 2 as first").analyze - val expected2 = parsePlan("SELECT 42 where 2 = 1").analyze - comparePlans(actual2, expected2) - // Test that plan is transformed to SET operation - val actual3 = parsePlan( - "EXECUTE IMMEDIATE 'SELECT 17, 7 WHERE ? = 1' INTO res, res2 USING 2").analyze - // Normalize to make the plan equivalent to the below set statement. - val expected3 = parsePlan("SET var (res, res2) = (SELECT 17, 7 where 2 = 1)").analyze - comparePlans(actual3, expected3) - } finally { - SimpleAnalyzer.catalogManager.tempVariableManager.remove(Seq("res")) - SimpleAnalyzer.catalogManager.tempVariableManager.remove(Seq("res2")) - } - } - test("SPARK-41271: bind named parameters to literals") { CTERelationDef.curId.set(0) val actual1 = NameParameterizedQuery( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediate.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediate.scala index e5b00d0c4cdf5..14080127ae933 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediate.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.catalyst.analysis -import scala.util.{Either, Left, Right} - import org.apache.spark.sql.{AnalysisException, SparkSession} import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, VariableReference} import org.apache.spark.sql.catalyst.plans.logical.{CompoundBody, LocalRelation, LogicalPlan} @@ -42,8 +40,8 @@ case class ExecuteExecutableDuringAnalysis(sparkSession: SparkSession) extends R private def executeImmediate(cmd: ExecuteImmediateCommand): LogicalPlan = { try { - // Extract the query string - val queryString = extractQueryString(cmd.query) + // Extract the query string from the queryParam expression + val queryString = extractQueryString(cmd.queryParam) // Parse and validate the query val parsedPlan = sparkSession.sessionState.sqlParser.parsePlan(queryString) @@ -98,22 +96,24 @@ case class ExecuteExecutableDuringAnalysis(sparkSession: SparkSession) extends R } } - private def extractQueryString(queryExpr: Either[String, VariableReference]): String = { - queryExpr match { - case Left(literal) => literal - case Right(variable) => - // Evaluate the variable reference - if (!variable.dataType.sameType(StringType)) { - throw QueryCompilationErrors.invalidExecuteImmediateVariableType(variable.dataType) - } + private def extractQueryString(queryExpr: Expression): String = { + // Ensure the expression resolves to string type + if (!queryExpr.dataType.sameType(StringType)) { + throw QueryCompilationErrors.invalidExecuteImmediateVariableType(queryExpr.dataType) + } - val value = variable.eval(null) - if (value == null) { + // Evaluate the expression to get the query string + val value = queryExpr.eval(null) + if (value == null) { + queryExpr match { + case variable: VariableReference => throw QueryCompilationErrors.nullSQLStringExecuteImmediate(variable.identifier.name()) - } - - value.toString + case _ => + throw QueryCompilationErrors.nullSQLStringExecuteImmediate("query expression") + } } + + value.toString } private def validateQuery(queryString: String, parsedPlan: LogicalPlan): Unit = { @@ -187,6 +187,7 @@ case class ExecuteExecutableDuringAnalysis(sparkSession: SparkSession) extends R } private def separateParameters(args: Seq[Expression]): (Seq[Any], Map[String, Any]) = { + import org.apache.spark.sql.catalyst.expressions.{EmptyRow, Literal} val positionalParams = scala.collection.mutable.ListBuffer[Any]() val namedParams = scala.collection.mutable.Map[String, Any]() @@ -194,11 +195,25 @@ case class ExecuteExecutableDuringAnalysis(sparkSession: SparkSession) extends R case alias: Alias => // Named parameter: "value AS paramName" val paramName = alias.name - val paramValue = alias.child.eval(null) + // Evaluate the expression (should already be resolved by analyzer) + val paramValue = if (alias.child.foldable) { + // For foldable expressions, create a literal (similar to ConstantFolding) + Literal.create(alias.child.eval(EmptyRow), alias.child.dataType).value + } else { + // For non-foldable expressions, just evaluate + alias.child.eval(EmptyRow) + } namedParams(paramName) = paramValue case expr => // Positional parameter: just a value - val paramValue = expr.eval(null) + // Evaluate the expression (should already be resolved by analyzer) + val paramValue = if (expr.foldable) { + // For foldable expressions, create a literal (similar to ConstantFolding) + Literal.create(expr.eval(EmptyRow), expr.dataType).value + } else { + // For non-foldable expressions, just evaluate + expr.eval(EmptyRow) + } positionalParams += paramValue } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala index f9bebce7cbfab..ee625dd25aeb3 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala @@ -25,7 +25,7 @@ import org.apache.hadoop.hive.ql.exec.{UDAF, UDF} import org.apache.hadoop.hive.ql.udf.generic.{AbstractGenericUDAFResolver, GenericUDF, GenericUDTF} import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.analysis.{Analyzer, EvalSubqueriesForTimeTravel, InvokeProcedures, ReplaceCharWithVarchar, ResolveDataSource, ResolveSessionCatalog, ResolveTranspose} +import org.apache.spark.sql.catalyst.analysis.{Analyzer, EvalSubqueriesForTimeTravel, ExecuteExecutableDuringAnalysis, InvokeProcedures, ReplaceCharWithVarchar, ResolveDataSource, ResolveSessionCatalog, ResolveTranspose, SubstituteExecuteImmediate} import org.apache.spark.sql.catalyst.analysis.resolver.ResolverExtension import org.apache.spark.sql.catalyst.catalog.{ExternalCatalogWithListener, InvalidUDFClassException} import org.apache.spark.sql.catalyst.expressions.{Expression, ExtractSemiStructuredFields} @@ -132,8 +132,9 @@ class HiveSessionStateBuilder( new EvalSubqueriesForTimeTravel +: new DetermineTableStats(session) +: new ResolveTranspose(session) +: - new InvokeProcedures(session) +: - ExtractSemiStructuredFields +: + new InvokeProcedures(session) +: + ExecuteExecutableDuringAnalysis(session) +: + ExtractSemiStructuredFields +: customResolutionRules override val postHocResolutionRules: Seq[Rule[LogicalPlan]] = From f8d1920cf5052a30e10155e39f0531323a4d7780 Mon Sep 17 00:00:00 2001 From: Serge Rielau Date: Fri, 29 Aug 2025 20:03:09 -0700 Subject: [PATCH 05/32] Complete function for EXECUTE IMMEDIATE REWORK --- .../resources/error/error-conditions.json | 13 +- .../org/apache/spark/sql/SparkSession.scala | 18 ++ .../analysis/ColumnResolutionHelper.scala | 1 + .../catalyst/analysis/executeImmediate.scala | 113 ++++++- .../sql/catalyst/analysis/parameters.scala | 104 +++++++ .../sql/errors/QueryCompilationErrors.scala | 6 +- .../analysis/AnalysisErrorSuite.scala | 4 +- .../spark/sql/connect/SparkSession.scala | 7 + .../catalyst/analysis/ExecuteImmediate.scala | 241 ++++++++++++--- .../spark/sql/classic/SparkSession.scala | 42 ++- .../command/ExecuteImmediateCommandExec.scala | 2 +- .../execute-immediate.sql.out | 282 ++++++------------ .../sql-tests/inputs/execute-immediate.sql | 12 +- .../results/execute-immediate.sql.out | 85 ++++-- .../sql/hive/HiveSessionStateBuilder.scala | 2 +- 15 files changed, 642 insertions(+), 290 deletions(-) diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 167c460536ac9..bbbda04a644ac 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -3862,6 +3862,12 @@ ], "sqlState" : "42604" }, + "INVALID_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE" : { + "message" : [ + "Expression type must be string type but got ." + ], + "sqlState" : "42K09" + }, "INVALID_UDF_IMPLEMENTATION" : { "message" : [ "Function does not implement a ScalarFunction or AggregateFunction." @@ -3914,12 +3920,7 @@ }, "sqlState" : "42K0M" }, - "INVALID_VARIABLE_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE" : { - "message" : [ - "Variable type must be string type but got ." - ], - "sqlState" : "42K09" - }, + "INVALID_VARIANT_CAST" : { "message" : [ "The variant value `` cannot be cast into ``. Please use `try_variant_get` instead." diff --git a/sql/api/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/api/src/main/scala/org/apache/spark/sql/SparkSession.scala index 8e7ae51e998f7..c557bb1fd1d6e 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -523,6 +523,24 @@ abstract class SparkSession extends Serializable with Closeable { sql(sqlText, args.asScala.toMap) } + /** + * Executes a SQL query substituting parameters by the given arguments with optional names, + * returning the result as a `DataFrame`. This API eagerly runs DDL/DML commands, but not for + * SELECT queries. This method allows the inner query to determine whether to use positional + * or named parameters based on its parameter markers. + * + * @param sqlText + * A SQL statement with named or positional parameters to execute. + * @param args + * An array of Java/Scala objects that can be converted to SQL literal expressions. + * @param paramNames + * An optional array of parameter names corresponding to args. If provided, enables named + * parameter binding where parameter names are available. If None or shorter than args, + * remaining parameters are treated as positional. + * @since 4.0.0 + */ + def sql(sqlText: String, args: Array[_], paramNames: Array[String]): DataFrame + /** * Executes a SQL query using Spark, returning the result as a `DataFrame`. This API eagerly * runs DDL/DML commands, but not for SELECT queries. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ColumnResolutionHelper.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ColumnResolutionHelper.scala index a880c7913bfba..4cd27444b6866 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ColumnResolutionHelper.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ColumnResolutionHelper.scala @@ -242,6 +242,7 @@ trait ColumnResolutionHelper extends Logging with DataTypeErrorsBase { * @return Reference to the variable. */ def lookupVariable(nameParts: Seq[String]): Option[VariableReference] = { + // The temp variables live in `SYSTEM.SESSION`, and the name can be qualified or not. def maybeTempVariableName(nameParts: Seq[String]): Boolean = { nameParts.length == 1 || { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala index 663f5bc4d1485..1b3c96b83808d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala @@ -21,7 +21,7 @@ import scala.util.{Either, Left, Right} import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Expression, VariableReference} import org.apache.spark.sql.catalyst.parser.ParseException -import org.apache.spark.sql.catalyst.plans.logical.{CompoundBody, ExecutableDuringAnalysis, LocalRelation, LogicalPlan, UnaryNode} +import org.apache.spark.sql.catalyst.plans.logical.{CompoundBody, ExecutableDuringAnalysis, LocalRelation, LogicalPlan, SetVariable, UnaryNode} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.trees.TreePattern.{EXECUTE_IMMEDIATE, TreePattern} import org.apache.spark.sql.connector.catalog.CatalogManager @@ -38,7 +38,7 @@ import org.apache.spark.sql.types.StringType case class ExecuteImmediateQuery( queryParam: Expression, args: Seq[Expression], - targetVariables: Seq[UnresolvedAttribute]) + targetVariables: Seq[Expression]) extends UnresolvedLeafNode { final override val nodePatterns: Seq[TreePattern] = Seq(EXECUTE_IMMEDIATE) @@ -50,12 +50,10 @@ case class ExecuteImmediateQuery( * * @param queryParam the resolved query expression * @param args parameters from USING clause - * @param targetVariables variables to store the result of the query */ case class ExecuteImmediateCommand( queryParam: Expression, - args: Seq[Expression], - targetVariables: Seq[VariableReference]) + args: Seq[Expression]) extends UnaryNode with ExecutableDuringAnalysis { final override val nodePatterns: Seq[TreePattern] = Seq(EXECUTE_IMMEDIATE) @@ -64,6 +62,13 @@ case class ExecuteImmediateCommand( override def output: Seq[Attribute] = child.output + override lazy val resolved: Boolean = { + // ExecuteImmediateCommand should not be considered resolved until it has been + // executed and replaced by ExecuteExecutableDuringAnalysis rule. + // This ensures that SetVariable waits for execution to complete. + false + } + override def stageForExplain(): LogicalPlan = { // For EXPLAIN, just show the command without executing it copy() @@ -101,6 +106,7 @@ class SubstituteExecuteImmediate( case a: Alias => Alias(resolveVariable(a.child), a.name)() case p: Parameter => p + case varRef: VariableReference => varRef // VariableReference is already resolved case other => throw QueryCompilationErrors.unsupportedParameterExpression(other) } @@ -123,7 +129,7 @@ class SubstituteExecuteImmediate( val varReference = getVariableReference(u, u.nameParts) if (!varReference.dataType.sameType(StringType)) { - throw QueryCompilationErrors.invalidExecuteImmediateVariableType(varReference.dataType) + throw QueryCompilationErrors.invalidExecuteImmediateExpressionType(varReference.dataType) } // Call eval with null value passed instead of a row. @@ -145,14 +151,52 @@ class SubstituteExecuteImmediate( // Check if all expressions are resolved (they should be resolved by ResolveReferences now) val queryParamResolved = queryParam.resolved val allArgsResolved = args.forall(_.resolved) - val targetVariablesResolved = targetVariables.forall(_.resolved) + val targetVariablesResolved = targetVariables.forall { + case _: UnresolvedAttribute => false // Unresolved attributes are not resolved + case alias: Alias => alias.child.resolved // For aliases, check if child is resolved + case _: VariableReference => true // VariableReference is already resolved + case expr => expr.resolved // For other expressions, use standard resolved check + } - if (queryParamResolved && allArgsResolved && targetVariablesResolved) { - // All resolved - transform to command - val finalTargetVars = targetVariables.map(attr => - getVariableReference(attr, attr.nameParts)) + // Validate that USING clause expressions don't contain unsupported constructs + validateUsingClauseExpressions(args) - ExecuteImmediateCommand(queryParam, args, finalTargetVars) + if (queryParamResolved && allArgsResolved && targetVariablesResolved) { + // All resolved - transform based on whether we have target variables + if (targetVariables.nonEmpty) { + // EXECUTE IMMEDIATE ... INTO should generate SetVariable plan + // SetVariable expects UnresolvedAttribute objects that ResolveSetVariable will resolve + val finalTargetVars = targetVariables.map { + case attr: UnresolvedAttribute => + // Keep as UnresolvedAttribute for ResolveSetVariable to handle + attr + case alias: Alias => + // Extract the UnresolvedAttribute from the alias + alias.child match { + case attr: UnresolvedAttribute => + attr + case varRef: VariableReference => + // Convert back to UnresolvedAttribute for ResolveSetVariable + UnresolvedAttribute(varRef.originalNameParts) + case _ => + throw QueryCompilationErrors.unsupportedParameterExpression(alias.child) + } + case varRef: VariableReference => + // Convert back to UnresolvedAttribute for ResolveSetVariable + UnresolvedAttribute(varRef.originalNameParts) + case other => + throw QueryCompilationErrors.unsupportedParameterExpression(other) + } + + // Validate that the query is suitable for INTO clause + validateQueryForInto(queryParam) + // Create SetVariable plan with the execute immediate query as source + val sourceQuery = ExecuteImmediateCommand(queryParam, args) + SetVariable(finalTargetVars, sourceQuery) + } else { + // Regular EXECUTE IMMEDIATE without INTO + ExecuteImmediateCommand(queryParam, args) + } } else { // Not all resolved yet - wait for next iteration e @@ -216,4 +260,49 @@ class SubstituteExecuteImmediate( expr.origin) } } + + private def validateQueryForInto(queryParam: Expression): Unit = { + // Extract the query string to validate + val queryString = queryParam.eval(null) match { + case null => return // Will be caught later by other validation + case value => value.toString + } + + // If targetVariables is defined, statement needs to be a query. + // Try to parse as query first, then as general plan + try { + catalogManager.v1SessionCatalog.parser.parseQuery(queryString) + // Success - it's a valid query, proceed + } catch { + case e: ParseException => + // parseQuery failed, try parsePlan to see if it's valid SQL but not a query + try { + catalogManager.v1SessionCatalog.parser.parsePlan(queryString) + // Plan was successfully parsed, but query wasn't - throw error + throw QueryCompilationErrors.invalidStatementForExecuteInto(queryString) + } catch { + case _: ParseException => + // Both failed - let the original parse error propagate + throw e + } + } + } + + private def validateUsingClauseExpressions(args: Seq[Expression]): Unit = { + import org.apache.spark.sql.catalyst.expressions.{ScalarSubquery, Exists, ListQuery, InSubquery} + args.foreach { expr => + // Check the expression and its children for unsupported constructs + expr.foreach { + case subquery: ScalarSubquery => + throw QueryCompilationErrors.unsupportedParameterExpression(subquery) + case exists: Exists => + throw QueryCompilationErrors.unsupportedParameterExpression(exists) + case listQuery: ListQuery => + throw QueryCompilationErrors.unsupportedParameterExpression(listQuery) + case inSubquery: InSubquery => + throw QueryCompilationErrors.unsupportedParameterExpression(inSubquery) + case _ => // Other expressions are fine + } + } + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/parameters.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/parameters.scala index 2cfc2a8c90dc5..b3ffb97032673 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/parameters.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/parameters.scala @@ -104,6 +104,28 @@ case class PosParameterizedQuery(child: LogicalPlan, args: Seq[Expression]) copy(child = newChild) } +/** + * The logical plan representing a parameterized query with unified parameter support. + * This allows the query to use either positional or named parameters based on the + * parameter markers found in the query, with optional parameter names provided. + * + * @param child The parameterized logical plan. + * @param args The literal values or collection constructor functions such as `map()`, + * `array()`, `struct()` of parameters. + * @param paramNames Optional parameter names corresponding to args. If provided for an argument, + * that argument can be used for named parameter binding. If not provided or + * shorter than args, remaining parameters are treated as positional. + */ +case class UnifiedParameterizedQuery( + child: LogicalPlan, + args: Seq[Expression], + paramNames: Seq[String]) + extends ParameterizedQuery(child) { + assert(args.nonEmpty) + override protected def withNewChildInternal(newChild: LogicalPlan): LogicalPlan = + copy(child = newChild) +} + /** * Moves `ParameterizedQuery` inside `SupervisingCommand` for their supervised plans to be * resolved later by the analyzer. @@ -203,6 +225,88 @@ object BindParameters extends Rule[LogicalPlan] with QueryErrorsBase { args(posToIndex(pos)) } + case UnifiedParameterizedQuery(child, args, paramNames) + if !child.containsPattern(UNRESOLVED_WITH) && + args.forall(_.resolved) => + + // First pass: collect all parameter types used in the query to determine strategy + val namedParams = scala.collection.mutable.Set.empty[String] + val positionalParams = scala.collection.mutable.Set.empty[Int] + bind(child) { + case NamedParameter(name) => namedParams.add(name); NamedParameter(name) + case p @ PosParameter(pos) => positionalParams.add(pos); p + } + + // Build parameter maps based on what the query actually uses + val namedArgsMap = scala.collection.mutable.Map[String, Expression]() + val positionalArgs = scala.collection.mutable.ListBuffer[Expression]() + + if (namedParams.nonEmpty && positionalParams.isEmpty) { + // Query uses only named parameters - try to match args to named parameters + args.zipWithIndex.foreach { case (arg, index) => + val paramName = if (index < paramNames.length && paramNames(index).nonEmpty) { + paramNames(index) + } else { + // For session variables without explicit AS clause, try to infer the name + // from the parameter names found in the query + namedParams.toSeq.lift(index).getOrElse(s"param_$index") + } + namedArgsMap(paramName) = arg + } + } else if (positionalParams.nonEmpty && namedParams.isEmpty) { + // Query uses only positional parameters - use all args as positional + positionalArgs ++= args + } else { + // Mixed or no parameters - build both maps + args.zipWithIndex.foreach { case (arg, index) => + if (index < paramNames.length && paramNames(index).nonEmpty) { + namedArgsMap(paramNames(index)) = arg + } else { + positionalArgs += arg + } + } + } + + // Check all arguments for validity + val allArgs = namedArgsMap.toSeq ++ positionalArgs.zipWithIndex.map { + case (arg, idx) => (s"_$idx", arg) + } + checkArgs(allArgs) + + // Bind named parameters by converting expressions to literals + val boundWithNamed = if (namedArgsMap.nonEmpty) { + bind(child) { + case NamedParameter(name) if namedArgsMap.contains(name) => + val expr = namedArgsMap(name) + if (expr.foldable) { + Literal.create(expr.eval(), expr.dataType) + } else { + // For non-foldable expressions, try to convert to SQL and re-parse + expr + } + } + } else { + child + } + + // Bind positional parameters + if (positionalArgs.nonEmpty) { + val posToIndex = positionalParams.toSeq.sorted.zipWithIndex.toMap + bind(boundWithNamed) { + case PosParameter(pos) if posToIndex.contains(pos) && + positionalArgs.size > posToIndex(pos) => + val expr = positionalArgs(posToIndex(pos)) + if (expr.foldable) { + Literal.create(expr.eval(), expr.dataType) + } else { + // For non-foldable expressions, try to convert to SQL and re-parse + expr + } + } + } else { + boundWithNamed + } + case other => other } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala index 74eed741622f1..c298229e80f1a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala @@ -4131,10 +4131,10 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat messageParameters = Map.empty) } - def invalidExecuteImmediateVariableType(dataType: DataType): Throwable = { + def invalidExecuteImmediateExpressionType(dataType: DataType): Throwable = { throw new AnalysisException( - errorClass = "INVALID_VARIABLE_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE", - messageParameters = Map("varType" -> toSQLType(dataType))) + errorClass = "INVALID_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE", + messageParameters = Map("exprType" -> toSQLType(dataType))) } def nullSQLStringExecuteImmediate(varName: String): Throwable = { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala index f551037e7a1a2..9588e9ede55be 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala @@ -856,9 +856,9 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { assertAnalysisErrorCondition( inputPlan = execImmediatePlan, - expectedErrorCondition = "INVALID_VARIABLE_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE", + expectedErrorCondition = "INVALID_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE", expectedMessageParameters = Map( - "varType" -> "\"INT\"" + "exprType" -> "\"INT\"" )) } diff --git a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/SparkSession.scala b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/SparkSession.scala index 646db83981fe2..591e3a57739c6 100644 --- a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/SparkSession.scala +++ b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/SparkSession.scala @@ -250,6 +250,13 @@ class SparkSession private[sql] ( sql(query, Array.empty) } + /** @inheritdoc */ + def sql(sqlText: String, args: Array[_], paramNames: Array[String]): DataFrame = { + // For Connect, delegate to existing positional parameter implementation + // The unified parameter logic is handled on the server side + sql(sqlText, args) + } + private def sql(sqlCommand: proto.SqlCommand): DataFrame = { // Send the SQL once to the server and then check the output. executeCommandWithDataFrameReturn(newCommand(_.setSqlCommand(sqlCommand))) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediate.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediate.scala index 14080127ae933..70b3682c5415d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediate.scala @@ -54,38 +54,24 @@ case class ExecuteExecutableDuringAnalysis(sparkSession: SparkSession) extends R sparkSession.sql(queryString) } } else { - // For parameterized queries, build parameter map from USING clause + // For parameterized queries, build unified parameter arrays // The args are already resolved by outer parameter resolution - val (positionalParams, namedParams) = separateParameters(cmd.args) + val (paramValues, paramNames) = buildUnifiedParameters(cmd.args) + + // Validate parameter usage patterns + validateParameterUsage(cmd.queryParam, cmd.args, paramNames.toSeq) AnalysisContext.withExecuteImmediateContext { - if (namedParams.nonEmpty && positionalParams.isEmpty) { - // Only named parameters: use Map overload - sparkSession.sql(queryString, namedParams) - } else if (positionalParams.nonEmpty && namedParams.isEmpty) { - // Only positional parameters: use Array overload - sparkSession.sql(queryString, positionalParams.toArray) - } else if (namedParams.isEmpty && positionalParams.isEmpty) { - // No parameters - sparkSession.sql(queryString) - } else { - // Mixed parameters: not directly supported, need manual substitution - val substitutedQuery = substituteNamedParameters(queryString, namedParams) - sparkSession.sql(substitutedQuery, positionalParams.toArray) - } + // Use the new unified parameter API - let the inner query decide + // whether to use positional or named parameters based on its markers + sparkSession.sql(queryString, paramValues, paramNames) } } - // Handle target variables if specified - if (cmd.targetVariables.nonEmpty) { - handleTargetVariables(result, cmd.targetVariables) - // Return empty relation for INTO queries - LocalRelation(Nil, Nil) - } else { - // Return the query results as a LocalRelation - val internalRows = result.queryExecution.executedPlan.executeCollect() - LocalRelation(result.queryExecution.analyzed.output, internalRows.toSeq) - } + // Return the query results as a LocalRelation + // ExecuteImmediateCommand returns query results; SetVariable handles variable assignment + val internalRows = result.queryExecution.executedPlan.executeCollect() + LocalRelation(result.queryExecution.analyzed.output, internalRows.toSeq) } catch { case e: AnalysisException => @@ -99,23 +85,33 @@ case class ExecuteExecutableDuringAnalysis(sparkSession: SparkSession) extends R private def extractQueryString(queryExpr: Expression): String = { // Ensure the expression resolves to string type if (!queryExpr.dataType.sameType(StringType)) { - throw QueryCompilationErrors.invalidExecuteImmediateVariableType(queryExpr.dataType) + throw QueryCompilationErrors.invalidExecuteImmediateExpressionType(queryExpr.dataType) } // Evaluate the expression to get the query string val value = queryExpr.eval(null) if (value == null) { - queryExpr match { - case variable: VariableReference => - throw QueryCompilationErrors.nullSQLStringExecuteImmediate(variable.identifier.name()) - case _ => - throw QueryCompilationErrors.nullSQLStringExecuteImmediate("query expression") - } + // Extract the original text from the expression's origin for the error message + val originalText = extractOriginalText(queryExpr) + throw QueryCompilationErrors.nullSQLStringExecuteImmediate(originalText) } value.toString } + private def extractOriginalText(queryExpr: Expression): String = { + val origin = queryExpr.origin + // Try to extract the original text from the origin information + (origin.sqlText, origin.startIndex, origin.stopIndex) match { + case (Some(sqlText), Some(startIndex), Some(stopIndex)) => + // Extract the substring from the original SQL text + sqlText.substring(startIndex, stopIndex + 1) + case _ => + // Fallback to the SQL representation if origin information is not available + queryExpr.sql + } + } + private def validateQuery(queryString: String, parsedPlan: LogicalPlan): Unit = { // Check for compound bodies (SQL scripting) if (parsedPlan.isInstanceOf[CompoundBody]) { @@ -186,24 +182,181 @@ case class ExecuteExecutableDuringAnalysis(sparkSession: SparkSession) extends R } } + /** + * Builds unified parameter arrays for the new sql() API. + * Returns (values, names) where values contains all parameter values + * and names contains corresponding parameter names (or empty string for positional). + */ + private def buildUnifiedParameters(args: Seq[Expression]): (Array[Any], Array[String]) = { + import org.apache.spark.sql.catalyst.expressions.{EmptyRow, Literal} + import org.apache.spark.sql.catalyst.expressions.VariableReference + val values = scala.collection.mutable.ListBuffer[Any]() + val names = scala.collection.mutable.ListBuffer[String]() + + + + args.foreach { + case alias: Alias => + // Check if this is an auto-generated alias from variable resolution + // or an explicit "value AS paramName" from the user + val isAutoGeneratedAlias = alias.child match { + case varRef: VariableReference => + // If the alias name matches the variable name, it's auto-generated + alias.name == varRef.identifier.name() + case _ => false + } + + val paramValue = alias.child match { + case varRef: VariableReference => + // Variable references should be evaluated to their values + varRef.eval(EmptyRow) + case foldable if foldable.foldable => + Literal.create(foldable.eval(EmptyRow), foldable.dataType).value + case other => + // Expression is not foldable - this is not supported for parameters + // Check for specific unsupported expression types to provide better error messages + import org.apache.spark.sql.catalyst.expressions.{ScalarSubquery, Exists, ListQuery, InSubquery} + other match { + case _: ScalarSubquery | _: Exists | _: ListQuery | _: InSubquery => + throw QueryCompilationErrors.unsupportedParameterExpression(other) + case _ if !other.foldable => + throw QueryCompilationErrors.unsupportedParameterExpression(other) + case _ => + // This should not happen, but fallback to evaluation + other.eval(EmptyRow) + } + } + + if (isAutoGeneratedAlias) { + // This is a session variable without explicit AS clause + // Pass the variable name so the inner query can use it for named parameters + val varName = alias.child.asInstanceOf[VariableReference].identifier.name() + + values += paramValue + names += varName // Use the variable name for named parameter binding + } else { + // This is a true named parameter: "value AS paramName" + val paramName = alias.name + values += paramValue + names += paramName + } + case expr => + // Positional parameter: just a value + val paramValue = expr match { + case varRef: VariableReference => + // Variable references should be evaluated to their values + varRef.eval(EmptyRow) + case foldable if foldable.foldable => + Literal.create(foldable.eval(EmptyRow), foldable.dataType).value + case other => + // Expression is not foldable - this is not supported for parameters + // Check for specific unsupported expression types to provide better error messages + import org.apache.spark.sql.catalyst.expressions.{ScalarSubquery, Exists, ListQuery, InSubquery} + other match { + case _: ScalarSubquery | _: Exists | _: ListQuery | _: InSubquery => + throw QueryCompilationErrors.unsupportedParameterExpression(other) + case _ if !other.foldable => + throw QueryCompilationErrors.unsupportedParameterExpression(other) + case _ => + // This should not happen, but fallback to evaluation + other.eval(EmptyRow) + } + } + values += paramValue + names += null // null indicates unnamed expression (hole) + } + + (values.toArray, names.toArray) + } + + private def validateParameterUsage( + queryParam: Expression, + args: Seq[Expression], + names: Seq[String]): Unit = { + // Extract the query string to check for parameter patterns + val queryString = queryParam.eval(null) match { + case null => return // Will be caught later by other validation + case value => value.toString + } + + // Check what types of parameters the query uses + val positionalParameterPattern = "\\?".r + val namedParameterPattern = ":[a-zA-Z_][a-zA-Z0-9_]*".r + + val queryUsesPositionalParameters = + positionalParameterPattern.findFirstIn(queryString).isDefined + val queryUsesNamedParameters = namedParameterPattern.findFirstIn(queryString).isDefined + + // First check: Does the query itself mix positional and named parameters? + if (queryUsesPositionalParameters && queryUsesNamedParameters) { + throw QueryCompilationErrors.invalidQueryMixedQueryParameters() + } + + // Second check: If query uses ONLY named parameters, all USING expressions must have names + if (queryUsesNamedParameters && !queryUsesPositionalParameters) { + val unnamedExpressions = names.zipWithIndex.collect { + case (null, index) => index + case ("", index) => index // Also catch empty strings as unnamed + } + if (unnamedExpressions.nonEmpty) { + // Get the actual expressions that don't have names for error reporting + val unnamedExprs = unnamedExpressions.map(args(_)) + throw QueryCompilationErrors.invalidQueryAllParametersMustBeNamed(unnamedExprs) + } + } + } + private def separateParameters(args: Seq[Expression]): (Seq[Any], Map[String, Any]) = { import org.apache.spark.sql.catalyst.expressions.{EmptyRow, Literal} + import org.apache.spark.sql.catalyst.expressions.VariableReference val positionalParams = scala.collection.mutable.ListBuffer[Any]() val namedParams = scala.collection.mutable.Map[String, Any]() + // scalastyle:off println + System.err.println(s"DEBUG: separateParameters called with ${args.length} args") + args.zipWithIndex.foreach { case (arg, i) => + System.err.println(s"DEBUG: arg[$i]: ${arg.getClass.getSimpleName} = $arg") + System.err.println(s"DEBUG: arg[$i].resolved: ${arg.resolved}") + System.err.println(s"DEBUG: arg[$i].foldable: ${arg.foldable}") + } + // scalastyle:on println + args.foreach { case alias: Alias => - // Named parameter: "value AS paramName" - val paramName = alias.name - // Evaluate the expression (should already be resolved by analyzer) - val paramValue = if (alias.child.foldable) { - // For foldable expressions, create a literal (similar to ConstantFolding) - Literal.create(alias.child.eval(EmptyRow), alias.child.dataType).value + // Check if this is an auto-generated alias from variable resolution + // or an explicit "value AS paramName" from the user + val isAutoGeneratedAlias = alias.child match { + case varRef: VariableReference => + // If the alias name matches the variable name, it's auto-generated + alias.name == varRef.identifier.name() + case _ => false + } + + if (isAutoGeneratedAlias) { + // This is actually a positional parameter (session variable without AS) + val paramValue = if (alias.child.foldable) { + Literal.create(alias.child.eval(EmptyRow), alias.child.dataType).value + } else { + alias.child.eval(EmptyRow) + } + // scalastyle:off println + System.err.println( + s"DEBUG: Positional param = $paramValue (from auto-generated alias ${alias.name})") + // scalastyle:on println + positionalParams += paramValue } else { - // For non-foldable expressions, just evaluate - alias.child.eval(EmptyRow) + // This is a true named parameter: "value AS paramName" + val paramName = alias.name + val paramValue = if (alias.child.foldable) { + Literal.create(alias.child.eval(EmptyRow), alias.child.dataType).value + } else { + alias.child.eval(EmptyRow) + } + // scalastyle:off println + System.err.println(s"DEBUG: Named param '$paramName' = $paramValue") + // scalastyle:on println + namedParams(paramName) = paramValue } - namedParams(paramName) = paramValue case expr => // Positional parameter: just a value // Evaluate the expression (should already be resolved by analyzer) @@ -214,6 +367,10 @@ case class ExecuteExecutableDuringAnalysis(sparkSession: SparkSession) extends R // For non-foldable expressions, just evaluate expr.eval(EmptyRow) } + // scalastyle:off println + System.err.println( + s"DEBUG: Positional param = $paramValue (from ${expr.getClass.getSimpleName})") + // scalastyle:on println positionalParams += paramValue } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/classic/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/classic/SparkSession.scala index 681e1b16af590..97473b5d3fef0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/classic/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/classic/SparkSession.scala @@ -40,7 +40,7 @@ import org.apache.spark.sql import org.apache.spark.sql.{Artifact, DataSourceRegistration, Encoder, Encoders, ExperimentalMethods, Row, SparkSessionBuilder, SparkSessionCompanion, SparkSessionExtensions, SparkSessionExtensionsProvider, UDTFRegistration} import org.apache.spark.sql.artifact.ArtifactManager import org.apache.spark.sql.catalyst._ -import org.apache.spark.sql.catalyst.analysis.{NameParameterizedQuery, PosParameterizedQuery, UnresolvedRelation} +import org.apache.spark.sql.catalyst.analysis.{NameParameterizedQuery, PosParameterizedQuery, UnifiedParameterizedQuery, UnresolvedRelation} import org.apache.spark.sql.catalyst.encoders._ import org.apache.spark.sql.catalyst.expressions.AttributeReference import org.apache.spark.sql.catalyst.parser.ParserInterface @@ -509,6 +509,46 @@ class SparkSession private( sql(sqlText, args.asScala.toMap) } + /** + * Executes a SQL query substituting parameters by the given arguments with optional names, + * returning the result as a `DataFrame`. This method allows the inner query to determine + * whether to use positional or named parameters based on its parameter markers. + */ + def sql(sqlText: String, args: Array[_], paramNames: Array[String]): DataFrame = { + sql(sqlText, args, paramNames, new QueryPlanningTracker) + } + + /** + * Internal implementation of unified parameter API with tracker. + */ + private[sql] def sql( + sqlText: String, + args: Array[_], + paramNames: Array[String], + tracker: QueryPlanningTracker): DataFrame = + withActive { + val plan = tracker.measurePhase(QueryPlanningTracker.PARSING) { + val parsedPlan = sessionState.sqlParser.parsePlan(sqlText) + if (args.nonEmpty) { + if (parsedPlan.isInstanceOf[CompoundBody]) { + // Positional parameters are not supported for SQL scripting. + throw SqlScriptingErrors.positionalParametersAreNotSupportedWithSqlScripting() + } + + // Create a unified parameter query that can handle both positional and named parameters + // The query itself will determine which type to use based on its parameter markers + UnifiedParameterizedQuery( + parsedPlan, + args.map(lit(_).expr).toImmutableArraySeq, + paramNames.toImmutableArraySeq + ) + } else { + parsedPlan + } + } + Dataset.ofRows(self, plan, tracker) + } + /** @inheritdoc */ override def sql(sqlText: String): DataFrame = sql(sqlText, Map.empty[String, Any]) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ExecuteImmediateCommandExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ExecuteImmediateCommandExec.scala index 14d136564260b..020c99732812e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ExecuteImmediateCommandExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ExecuteImmediateCommandExec.scala @@ -102,7 +102,7 @@ case class ExecuteImmediateCommandExec( case Right(variable) => // Evaluate the variable reference if (!variable.dataType.sameType(StringType)) { - throw QueryCompilationErrors.invalidExecuteImmediateVariableType(variable.dataType) + throw QueryCompilationErrors.invalidExecuteImmediateExpressionType(variable.dataType) } val value = variable.eval(null) diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/execute-immediate.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/execute-immediate.sql.out index d575cac56d28b..7c2307d2d81b3 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/execute-immediate.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/execute-immediate.sql.out @@ -48,68 +48,37 @@ SetVariable [variablereference(system.session.sql_string=CAST(NULL AS STRING))] -- !query EXECUTE IMMEDIATE 'SET spark.sql.ansi.enabled=true' -- !query analysis -SetCommand (spark.sql.ansi.enabled,Some(true)) +LocalRelation [key#x, value#x] -- !query EXECUTE IMMEDIATE 'CREATE TEMPORARY VIEW IDENTIFIER(:tblName) AS SELECT id, name FROM tbl_view' USING 'tbl_view_tmp' as tblName -- !query analysis -CreateViewCommand `tbl_view_tmp`, SELECT id, name FROM tbl_view, false, false, LocalTempView, UNSUPPORTED, true - +- Project [id#x, name#x] - +- SubqueryAlias tbl_view - +- View (`tbl_view`, [id#x, name#x, data#x]) - +- Project [cast(id#x as int) AS id#x, cast(name#x as string) AS name#x, cast(data#x as struct>) AS data#x] - +- Project [id#x, name#x, data#x] - +- SubqueryAlias tbl_view - +- LocalRelation [id#x, name#x, data#x] +LocalRelation -- !query EXECUTE IMMEDIATE 'SELECT * FROM tbl_view_tmp' -- !query analysis -Project [id#x, name#x] -+- SubqueryAlias tbl_view_tmp - +- View (`tbl_view_tmp`, [id#x, name#x]) - +- Project [cast(id#x as int) AS id#x, cast(name#x as string) AS name#x] - +- Project [id#x, name#x] - +- SubqueryAlias tbl_view - +- View (`tbl_view`, [id#x, name#x, data#x]) - +- Project [cast(id#x as int) AS id#x, cast(name#x as string) AS name#x, cast(data#x as struct>) AS data#x] - +- Project [id#x, name#x, data#x] - +- SubqueryAlias tbl_view - +- LocalRelation [id#x, name#x, data#x] +LocalRelation [id#x, name#x] -- !query EXECUTE IMMEDIATE 'REFRESH TABLE IDENTIFIER(:tblName)' USING 'x' as tblName -- !query analysis -RefreshTableCommand `spark_catalog`.`default`.`x` +LocalRelation -- !query EXECUTE IMMEDIATE sql_string -- !query analysis -Project [id#x, name#x, data#x] -+- Filter (name#x = name1) - +- SubqueryAlias tbl_view - +- View (`tbl_view`, [id#x, name#x, data#x]) - +- Project [cast(id#x as int) AS id#x, cast(name#x as string) AS name#x, cast(data#x as struct>) AS data#x] - +- Project [id#x, name#x, data#x] - +- SubqueryAlias tbl_view - +- LocalRelation [id#x, name#x, data#x] +LocalRelation [id#x, name#x, data#x] -- !query EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = \'name1\'' -- !query analysis -Project [id#x, name#x, data#x] -+- Filter (name#x = name1) - +- SubqueryAlias tbl_view - +- View (`tbl_view`, [id#x, name#x, data#x]) - +- Project [cast(id#x as int) AS id#x, cast(name#x as string) AS name#x, cast(data#x as struct>) AS data#x] - +- Project [id#x, name#x, data#x] - +- SubqueryAlias tbl_view - +- LocalRelation [id#x, name#x, data#x] +LocalRelation [id#x, name#x, data#x] -- !query @@ -138,74 +107,37 @@ SetVariable [variablereference(system.session.a=CAST(NULL AS STRING))] -- !query EXECUTE IMMEDIATE sql_string USING 'name1', 'name3' -- !query analysis -Project [id#x, name#x, data#x] -+- Filter ((name#x = name1) OR (name#x = name3)) - +- SubqueryAlias tbl_view - +- View (`tbl_view`, [id#x, name#x, data#x]) - +- Project [cast(id#x as int) AS id#x, cast(name#x as string) AS name#x, cast(data#x as struct>) AS data#x] - +- Project [id#x, name#x, data#x] - +- SubqueryAlias tbl_view - +- LocalRelation [id#x, name#x, data#x] +LocalRelation [id#x, name#x, data#x] -- !query EXECUTE IMMEDIATE sql_string USING a, 'name2' -- !query analysis -Project [id#x, name#x, data#x] -+- Filter ((name#x = variablereference(system.session.a='name1')) OR (name#x = name2)) - +- SubqueryAlias tbl_view - +- View (`tbl_view`, [id#x, name#x, data#x]) - +- Project [cast(id#x as int) AS id#x, cast(name#x as string) AS name#x, cast(data#x as struct>) AS data#x] - +- Project [id#x, name#x, data#x] - +- SubqueryAlias tbl_view - +- LocalRelation [id#x, name#x, data#x] +LocalRelation [id#x, name#x, data#x] -- !query EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = ? or name = ?' USING 'name1', 'name3' -- !query analysis -Project [id#x, name#x, data#x] -+- Filter ((name#x = name1) OR (name#x = name3)) - +- SubqueryAlias tbl_view - +- View (`tbl_view`, [id#x, name#x, data#x]) - +- Project [cast(id#x as int) AS id#x, cast(name#x as string) AS name#x, cast(data#x as struct>) AS data#x] - +- Project [id#x, name#x, data#x] - +- SubqueryAlias tbl_view - +- LocalRelation [id#x, name#x, data#x] +LocalRelation [id#x, name#x, data#x] -- !query EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = ? or name = ?' USING a, 'name2' -- !query analysis -Project [id#x, name#x, data#x] -+- Filter ((name#x = variablereference(system.session.a='name1')) OR (name#x = name2)) - +- SubqueryAlias tbl_view - +- View (`tbl_view`, [id#x, name#x, data#x]) - +- Project [cast(id#x as int) AS id#x, cast(name#x as string) AS name#x, cast(data#x as struct>) AS data#x] - +- Project [id#x, name#x, data#x] - +- SubqueryAlias tbl_view - +- LocalRelation [id#x, name#x, data#x] +LocalRelation [id#x, name#x, data#x] -- !query EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = ? or name = ?' USING (a, 'name2') -- !query analysis -Project [id#x, name#x, data#x] -+- Filter ((name#x = variablereference(system.session.a='name1')) OR (name#x = name2)) - +- SubqueryAlias tbl_view - +- View (`tbl_view`, [id#x, name#x, data#x]) - +- Project [cast(id#x as int) AS id#x, cast(name#x as string) AS name#x, cast(data#x as struct>) AS data#x] - +- Project [id#x, name#x, data#x] - +- SubqueryAlias tbl_view - +- LocalRelation [id#x, name#x, data#x] +LocalRelation [id#x, name#x, data#x] -- !query EXECUTE IMMEDIATE 'INSERT INTO x VALUES(?)' USING 1 -- !query analysis -InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/x, false, CSV, [path=file:[not included in comparison]/{warehouse_dir}/x], Append, `spark_catalog`.`default`.`x`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/x), [id] -+- Project [col1#x AS id#x] - +- LocalRelation [col1#x] +LocalRelation -- !query @@ -242,74 +174,37 @@ SetVariable [variablereference(system.session.b=CAST(NULL AS INT))] -- !query EXECUTE IMMEDIATE sql_string USING 40 as second, 'name7' as first -- !query analysis -Project [id#x, name#x, data#x] -+- Filter ((name#x = name7) OR (id#x = 40)) - +- SubqueryAlias tbl_view - +- View (`tbl_view`, [id#x, name#x, data#x]) - +- Project [cast(id#x as int) AS id#x, cast(name#x as string) AS name#x, cast(data#x as struct>) AS data#x] - +- Project [id#x, name#x, data#x] - +- SubqueryAlias tbl_view - +- LocalRelation [id#x, name#x, data#x] +LocalRelation [id#x, name#x, data#x] -- !query EXECUTE IMMEDIATE sql_string USING b as second, 'name7' as first -- !query analysis -Project [id#x, name#x, data#x] -+- Filter ((name#x = name7) OR (id#x = variablereference(system.session.b=40))) - +- SubqueryAlias tbl_view - +- View (`tbl_view`, [id#x, name#x, data#x]) - +- Project [cast(id#x as int) AS id#x, cast(name#x as string) AS name#x, cast(data#x as struct>) AS data#x] - +- Project [id#x, name#x, data#x] - +- SubqueryAlias tbl_view - +- LocalRelation [id#x, name#x, data#x] +LocalRelation [id#x, name#x, data#x] -- !query EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = :first or id = :second' USING 40 as second, 'name7' as first -- !query analysis -Project [id#x, name#x, data#x] -+- Filter ((name#x = name7) OR (id#x = 40)) - +- SubqueryAlias tbl_view - +- View (`tbl_view`, [id#x, name#x, data#x]) - +- Project [cast(id#x as int) AS id#x, cast(name#x as string) AS name#x, cast(data#x as struct>) AS data#x] - +- Project [id#x, name#x, data#x] - +- SubqueryAlias tbl_view - +- LocalRelation [id#x, name#x, data#x] +LocalRelation [id#x, name#x, data#x] -- !query EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = :first or id = :second' USING 'name7' as first, b as second -- !query analysis -Project [id#x, name#x, data#x] -+- Filter ((name#x = name7) OR (id#x = variablereference(system.session.b=40))) - +- SubqueryAlias tbl_view - +- View (`tbl_view`, [id#x, name#x, data#x]) - +- Project [cast(id#x as int) AS id#x, cast(name#x as string) AS name#x, cast(data#x as struct>) AS data#x] - +- Project [id#x, name#x, data#x] - +- SubqueryAlias tbl_view - +- LocalRelation [id#x, name#x, data#x] +LocalRelation [id#x, name#x, data#x] -- !query EXECUTE IMMEDIATE 'SELECT tbl_view.*, :first as p FROM tbl_view WHERE name = :first' USING 'name7' as first -- !query analysis -Project [id#x, name#x, data#x, name7 AS p#x] -+- Filter (name#x = name7) - +- SubqueryAlias tbl_view - +- View (`tbl_view`, [id#x, name#x, data#x]) - +- Project [cast(id#x as int) AS id#x, cast(name#x as string) AS name#x, cast(data#x as struct>) AS data#x] - +- Project [id#x, name#x, data#x] - +- SubqueryAlias tbl_view - +- LocalRelation [id#x, name#x, data#x] +LocalRelation [id#x, name#x, data#x, p#x] -- !query EXECUTE IMMEDIATE 'SET VAR sql_string = ?' USING 'SELECT id from tbl_view where name = :first' -- !query analysis -SetVariable [variablereference(system.session.sql_string='SELECT * from tbl_view where name = :first or id = :second')] -+- Project [SELECT id from tbl_view where name = :first AS sql_string#x] - +- OneRowRelation +LocalRelation -- !query @@ -330,16 +225,7 @@ CreateVariable defaultvalueexpression(null, null), false EXECUTE IMMEDIATE sql_string INTO res_id USING 'name7' as first -- !query analysis SetVariable [variablereference(system.session.res_id=CAST(NULL AS INT))] -+- GlobalLimit 2 - +- LocalLimit 2 - +- Project [id#x] - +- Filter (name#x = name7) - +- SubqueryAlias tbl_view - +- View (`tbl_view`, [id#x, name#x, data#x]) - +- Project [cast(id#x as int) AS id#x, cast(name#x as string) AS name#x, cast(data#x as struct>) AS data#x] - +- Project [id#x, name#x, data#x] - +- SubqueryAlias tbl_view - +- LocalRelation [id#x, name#x, data#x] ++- LocalRelation [id#x] -- !query @@ -353,16 +239,7 @@ Project [variablereference(system.session.res_id=70) AS res_id#x] EXECUTE IMMEDIATE sql_string INTO res_id USING a as first -- !query analysis SetVariable [variablereference(system.session.res_id=70)] -+- GlobalLimit 2 - +- LocalLimit 2 - +- Project [id#x] - +- Filter (name#x = variablereference(system.session.a='name1')) - +- SubqueryAlias tbl_view - +- View (`tbl_view`, [id#x, name#x, data#x]) - +- Project [cast(id#x as int) AS id#x, cast(name#x as string) AS name#x, cast(data#x as struct>) AS data#x] - +- Project [id#x, name#x, data#x] - +- SubqueryAlias tbl_view - +- LocalRelation [id#x, name#x, data#x] ++- LocalRelation [id#x] -- !query @@ -384,8 +261,7 @@ SetVariable [variablereference(system.session.sql_string='SELECT id from tbl_vie EXECUTE IMMEDIATE 'SELECT 42' INTO res_id -- !query analysis SetVariable [variablereference(system.session.res_id=10)] -+- Project [42 AS 42#x] - +- OneRowRelation ++- LocalRelation [42#x] -- !query @@ -399,16 +275,7 @@ Project [variablereference(system.session.res_id=42) AS res_id#x] EXECUTE IMMEDIATE 'SELECT id, name FROM tbl_view WHERE id = ?' INTO b, a USING 10 -- !query analysis SetVariable [variablereference(system.session.b=40), variablereference(system.session.a='name1')] -+- GlobalLimit 2 - +- LocalLimit 2 - +- Project [id#x, name#x] - +- Filter (id#x = 10) - +- SubqueryAlias tbl_view - +- View (`tbl_view`, [id#x, name#x, data#x]) - +- Project [cast(id#x as int) AS id#x, cast(name#x as string) AS name#x, cast(data#x as struct>) AS data#x] - +- Project [id#x, name#x, data#x] - +- SubqueryAlias tbl_view - +- LocalRelation [id#x, name#x, data#x] ++- LocalRelation [id#x, name#x] -- !query @@ -421,23 +288,14 @@ Project [variablereference(system.session.b=10) AS b#x, variablereference(system -- !query EXECUTE IMMEDIATE 'SELECT * FROM tbl_view where id = ? AND name = ?' USING b as first, a -- !query analysis -Project [id#x, name#x, data#x] -+- Filter ((id#x = variablereference(system.session.b=10)) AND (name#x = variablereference(system.session.a='name1'))) - +- SubqueryAlias tbl_view - +- View (`tbl_view`, [id#x, name#x, data#x]) - +- Project [cast(id#x as int) AS id#x, cast(name#x as string) AS name#x, cast(data#x as struct>) AS data#x] - +- Project [id#x, name#x, data#x] - +- SubqueryAlias tbl_view - +- LocalRelation [id#x, name#x, data#x] +LocalRelation [id#x, name#x, data#x] -- !query EXECUTE IMMEDIATE 'SELECT 42 WHERE 2 = 1' INTO res_id -- !query analysis SetVariable [variablereference(system.session.res_id=42)] -+- Project [42 AS 42#x] - +- Filter (2 = 1) - +- OneRowRelation ++- LocalRelation , [42#x] -- !query @@ -452,8 +310,7 @@ EXECUTE IMMEDIATE 'SELECT \'1707\'' INTO res_id -- !query analysis SetVariable [variablereference(system.session.res_id=CAST(NULL AS INT))] +- Project [cast(1707#x as int) AS res_id#x] - +- Project [1707 AS 1707#x] - +- OneRowRelation + +- LocalRelation [1707#x] -- !query @@ -502,13 +359,12 @@ org.apache.spark.sql.AnalysisException -- !query EXECUTE IMMEDIATE 'SELECT * FROM tbl_view WHERE ? = id' USING id -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "UNRESOLVED_VARIABLE", - "sqlState" : "42883", + "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + "sqlState" : "42703", "messageParameters" : { - "searchPath" : "`system`.`session`", - "variableName" : "`id`" + "objectName" : "`id`" }, "queryContext" : [ { "objectType" : "", @@ -590,10 +446,10 @@ EXECUTE IMMEDIATE b -- !query analysis org.apache.spark.sql.AnalysisException { - "errorClass" : "INVALID_VARIABLE_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE", + "errorClass" : "INVALID_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE", "sqlState" : "42K09", "messageParameters" : { - "varType" : "\"INT\"" + "exprType" : "\"INT\"" } } @@ -617,21 +473,7 @@ SetVariable [variablereference(system.session.a='name1')] -- !query EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = :first' USING CONCAT(a , "me1") as first -- !query analysis -org.apache.spark.sql.AnalysisException -{ - "errorClass" : "UNSUPPORTED_EXPR_FOR_PARAMETER", - "sqlState" : "42K0E", - "messageParameters" : { - "invalidExprSql" : "\"CONCAT(a, me1)\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 70, - "stopIndex" : 86, - "fragment" : "CONCAT(a , \"me1\")" - } ] -} +LocalRelation [id#x, name#x, data#x] -- !query @@ -759,14 +601,7 @@ CreateVariable defaultvalueexpression(10, 10), false -- !query EXECUTE IMMEDIATE 'SELECT id FROM tbl_view WHERE id = :p' USING p -- !query analysis -Project [id#x] -+- Filter (id#x = variablereference(system.session.p=10)) - +- SubqueryAlias tbl_view - +- View (`tbl_view`, [id#x, name#x, data#x]) - +- Project [cast(id#x as int) AS id#x, cast(name#x as string) AS name#x, cast(data#x as struct>) AS data#x] - +- Project [id#x, name#x, data#x] - +- SubqueryAlias tbl_view - +- LocalRelation [id#x, name#x, data#x] +LocalRelation [id#x] -- !query @@ -830,6 +665,57 @@ org.apache.spark.sql.AnalysisException } +-- !query +SET VAR sql_string = 5 +-- !query analysis +SetVariable [variablereference(system.session.sql_string=CAST(NULL AS STRING))] ++- Project [cast(sql_string#x as string) AS sql_string#x] + +- Project [5 AS sql_string#x] + +- OneRowRelation + + +-- !query +EXECUTE IMMEDIATE sql_string +-- !query analysis +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'5'", + "hint" : "" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 28, + "fragment" : "EXECUTE IMMEDIATE sql_string" + } ] +} + + +-- !query +SET VAR sql_string = 'hello' +-- !query analysis +SetVariable [variablereference(system.session.sql_string='5')] ++- Project [hello AS sql_string#x] + +- OneRowRelation + + +-- !query +EXECUTE IMMEDIATE length(sql_string) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE", + "sqlState" : "42K09", + "messageParameters" : { + "exprType" : "\"INT\"" + } +} + + -- !query DROP TABLE x -- !query analysis diff --git a/sql/core/src/test/resources/sql-tests/inputs/execute-immediate.sql b/sql/core/src/test/resources/sql-tests/inputs/execute-immediate.sql index f7d27c6c0b039..402f3bfdddcec 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/execute-immediate.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/execute-immediate.sql @@ -109,8 +109,10 @@ EXECUTE IMMEDIATE b; SET VAR sql_string = 'SELECT * from tbl_view where name = :first or id = :second'; SET VAR a = 'na'; --- expressions not supported - feature not supported +-- constant expressions are supported EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = :first' USING CONCAT(a , "me1") as first; + +-- subquery in using not supported EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = :first' USING (SELECT 42) as first, 'name2' as second; -- INTO variables not matching scalar types @@ -146,4 +148,12 @@ EXECUTE IMMEDIATE 'EXECUTE IMMEDIATE \'SELECT id FROM tbl_view WHERE id = ? USIN SET VAR sql_string = null; EXECUTE IMMEDIATE sql_string; +-- sqlString is null +SET VAR sql_string = 5; +EXECUTE IMMEDIATE sql_string; + +-- sqlString is null +SET VAR sql_string = 'hello'; +EXECUTE IMMEDIATE length(sql_string); + DROP TABLE x; \ No newline at end of file diff --git a/sql/core/src/test/resources/sql-tests/results/execute-immediate.sql.out b/sql/core/src/test/resources/sql-tests/results/execute-immediate.sql.out index 9249d7eb3e517..c3150ed270613 100644 --- a/sql/core/src/test/resources/sql-tests/results/execute-immediate.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/execute-immediate.sql.out @@ -427,13 +427,12 @@ EXECUTE IMMEDIATE 'SELECT * FROM tbl_view WHERE ? = id' USING id -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "UNRESOLVED_VARIABLE", - "sqlState" : "42883", + "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + "sqlState" : "42703", "messageParameters" : { - "searchPath" : "`system`.`session`", - "variableName" : "`id`" + "objectName" : "`id`" }, "queryContext" : [ { "objectType" : "", @@ -525,10 +524,10 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "INVALID_VARIABLE_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE", + "errorClass" : "INVALID_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE", "sqlState" : "42K09", "messageParameters" : { - "varType" : "\"INT\"" + "exprType" : "\"INT\"" } } @@ -552,23 +551,9 @@ struct<> -- !query EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = :first' USING CONCAT(a , "me1") as first -- !query schema -struct<> +struct>> -- !query output -org.apache.spark.sql.AnalysisException -{ - "errorClass" : "UNSUPPORTED_EXPR_FOR_PARAMETER", - "sqlState" : "42K0E", - "messageParameters" : { - "invalidExprSql" : "\"CONCAT(a, me1)\"" - }, - "queryContext" : [ { - "objectType" : "", - "objectName" : "", - "startIndex" : 70, - "stopIndex" : 86, - "fragment" : "CONCAT(a , \"me1\")" - } ] -} +10 name1 {"f1":1,"s2":{"f2":101,"f3":"a"}} -- !query @@ -784,6 +769,60 @@ org.apache.spark.sql.AnalysisException } +-- !query +SET VAR sql_string = 5 +-- !query schema +struct<> +-- !query output + + + +-- !query +EXECUTE IMMEDIATE sql_string +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'5'", + "hint" : "" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 28, + "fragment" : "EXECUTE IMMEDIATE sql_string" + } ] +} + + +-- !query +SET VAR sql_string = 'hello' +-- !query schema +struct<> +-- !query output + + + +-- !query +EXECUTE IMMEDIATE length(sql_string) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE", + "sqlState" : "42K09", + "messageParameters" : { + "exprType" : "\"INT\"" + } +} + + -- !query DROP TABLE x -- !query schema diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala index ee625dd25aeb3..e7da11f2be494 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala @@ -25,7 +25,7 @@ import org.apache.hadoop.hive.ql.exec.{UDAF, UDF} import org.apache.hadoop.hive.ql.udf.generic.{AbstractGenericUDAFResolver, GenericUDF, GenericUDTF} import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.analysis.{Analyzer, EvalSubqueriesForTimeTravel, ExecuteExecutableDuringAnalysis, InvokeProcedures, ReplaceCharWithVarchar, ResolveDataSource, ResolveSessionCatalog, ResolveTranspose, SubstituteExecuteImmediate} +import org.apache.spark.sql.catalyst.analysis.{Analyzer, EvalSubqueriesForTimeTravel, ExecuteExecutableDuringAnalysis, InvokeProcedures, ReplaceCharWithVarchar, ResolveDataSource, ResolveSessionCatalog, ResolveTranspose} import org.apache.spark.sql.catalyst.analysis.resolver.ResolverExtension import org.apache.spark.sql.catalyst.catalog.{ExternalCatalogWithListener, InvalidUDFClassException} import org.apache.spark.sql.catalyst.expressions.{Expression, ExtractSemiStructuredFields} From 8a8b3e4740764eec6151867dc21c518aad91f92a Mon Sep 17 00:00:00 2001 From: Serge Rielau Date: Fri, 29 Aug 2025 21:18:19 -0700 Subject: [PATCH 06/32] Update common/utils/src/main/resources/error/error-conditions.json --- common/utils/src/main/resources/error/error-conditions.json | 1 - 1 file changed, 1 deletion(-) diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index bbbda04a644ac..37e0aaeda6125 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -3920,7 +3920,6 @@ }, "sqlState" : "42K0M" }, - "INVALID_VARIANT_CAST" : { "message" : [ "The variant value `` cannot be cast into ``. Please use `try_variant_get` instead." From 1aeaac93fb67fdd6cd4dc0d8d890e2c04c1346b0 Mon Sep 17 00:00:00 2001 From: Serge Rielau Date: Fri, 29 Aug 2025 22:53:39 -0700 Subject: [PATCH 07/32] Rework --- .../org/apache/spark/sql/catalyst/analysis/Analyzer.scala | 4 ++++ .../sql/catalyst/analysis/ColumnResolutionHelper.scala | 4 +++- .../spark/sql/catalyst/analysis/executeImmediate.scala | 6 +++--- .../apache/spark/sql/catalyst/rules/RuleIdCollection.scala | 4 ++-- ...xecuteImmediate.scala => ExecuteImmediateCommands.scala} | 2 +- .../apache/spark/sql/internal/BaseSessionStateBuilder.scala | 4 ++-- .../org/apache/spark/sql/hive/HiveSessionStateBuilder.scala | 4 ++-- 7 files changed, 17 insertions(+), 11 deletions(-) rename sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/{ExecuteImmediate.scala => ExecuteImmediateCommands.scala} (99%) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 8ff7215697aaf..d699a373aae61 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -444,6 +444,10 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor AddMetadataColumns :: DeduplicateRelations :: ResolveCollationName :: + new ResolveExecuteImmediate( + catalogManager, + resolveChild = (plan: LogicalPlan) => plan, + checkAnalysis = (_: LogicalPlan) => ()) :: ResolveMergeIntoSchemaEvolution :: new ResolveReferences(catalogManager) :: // Please do not insert any other rules in between. See the TODO comments in rule diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ColumnResolutionHelper.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ColumnResolutionHelper.scala index 4cd27444b6866..b19ef981a651a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ColumnResolutionHelper.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ColumnResolutionHelper.scala @@ -23,12 +23,14 @@ import scala.collection.mutable import org.apache.spark.internal.Logging import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.SqlScriptingContextManager import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.SubExprUtils.wrapOuterReference +import org.apache.spark.sql.catalyst.parser.SqlScriptingLabelContext.isForbiddenLabelOrForVariableName import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.trees.CurrentOrigin.withOrigin import org.apache.spark.sql.catalyst.trees.TreePattern._ -import org.apache.spark.sql.connector.catalog.CatalogManager +import org.apache.spark.sql.connector.catalog.{CatalogManager, Identifier} import org.apache.spark.sql.errors.{DataTypeErrorsBase, QueryCompilationErrors} import org.apache.spark.sql.internal.SQLConf diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala index 1b3c96b83808d..1a574c64c8edb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala @@ -64,7 +64,7 @@ case class ExecuteImmediateCommand( override lazy val resolved: Boolean = { // ExecuteImmediateCommand should not be considered resolved until it has been - // executed and replaced by ExecuteExecutableDuringAnalysis rule. + // executed and replaced by ExecuteImmediateCommands rule. // This ensures that SetVariable waits for execution to complete. false } @@ -81,10 +81,10 @@ case class ExecuteImmediateCommand( } /** - * This rule substitutes execute immediate query node with a command node + * This rule resolves execute immediate query node into a command node * that will handle recursive SQL execution. */ -class SubstituteExecuteImmediate( +class ResolveExecuteImmediate( val catalogManager: CatalogManager, resolveChild: LogicalPlan => LogicalPlan, checkAnalysis: LogicalPlan => Unit) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleIdCollection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleIdCollection.scala index a5ed8d730740a..24394a9989bab 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleIdCollection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleIdCollection.scala @@ -87,7 +87,7 @@ object RuleIdCollection { "org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases" :: "org.apache.spark.sql.catalyst.analysis.EliminateUnions" :: "org.apache.spark.sql.catalyst.analysis.ResolveDefaultColumns" :: - "org.apache.spark.sql.catalyst.analysis.ExecuteExecutableDuringAnalysis" :: + "org.apache.spark.sql.catalyst.analysis.ExecuteImmediateCommands" :: "org.apache.spark.sql.catalyst.analysis.ResolveExpressionsWithNamePlaceholders" :: "org.apache.spark.sql.catalyst.analysis.ResolveHints$ResolveCoalesceHints" :: "org.apache.spark.sql.catalyst.analysis.ResolveHints$ResolveJoinStrategyHints" :: @@ -98,7 +98,7 @@ object RuleIdCollection { "org.apache.spark.sql.catalyst.analysis.ResolveOrderByAll" :: "org.apache.spark.sql.catalyst.analysis.ResolveRowLevelCommandAssignments" :: "org.apache.spark.sql.catalyst.analysis.ResolveSetVariable" :: - "org.apache.spark.sql.catalyst.analysis.SubstituteExecuteImmediate" :: + "org.apache.spark.sql.catalyst.analysis.ResolveExecuteImmediate" :: "org.apache.spark.sql.catalyst.analysis.ResolveTableSpec" :: "org.apache.spark.sql.catalyst.analysis.ResolveTimeZone" :: "org.apache.spark.sql.catalyst.analysis.ResolveUnion" :: diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediate.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediateCommands.scala similarity index 99% rename from sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediate.scala rename to sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediateCommands.scala index 70b3682c5415d..29396d8fa1aea 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediateCommands.scala @@ -29,7 +29,7 @@ import org.apache.spark.sql.types.StringType * Analysis rule that executes ExecuteImmediateCommand during analysis and replaces it * with the results, similar to how CALL statements work. */ -case class ExecuteExecutableDuringAnalysis(sparkSession: SparkSession) extends Rule[LogicalPlan] { +case class ExecuteImmediateCommands(sparkSession: SparkSession) extends Rule[LogicalPlan] { override def apply(plan: LogicalPlan): LogicalPlan = { plan.resolveOperatorsWithPruning(_.containsPattern(EXECUTE_IMMEDIATE), ruleId) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala index 21bd337200c9c..097614f8a3a71 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.internal import org.apache.spark.annotation.Unstable import org.apache.spark.sql.{DataSourceRegistration, ExperimentalMethods, SparkSessionExtensions, UDTFRegistration} import org.apache.spark.sql.artifact.ArtifactManager -import org.apache.spark.sql.catalyst.analysis.{Analyzer, EvalSubqueriesForTimeTravel, ExecuteExecutableDuringAnalysis, FunctionRegistry, InvokeProcedures, ReplaceCharWithVarchar, ResolveDataSource, ResolveSessionCatalog, ResolveTranspose, TableFunctionRegistry} +import org.apache.spark.sql.catalyst.analysis.{Analyzer, EvalSubqueriesForTimeTravel, ExecuteImmediateCommands, FunctionRegistry, InvokeProcedures, ReplaceCharWithVarchar, ResolveDataSource, ResolveSessionCatalog, ResolveTranspose, TableFunctionRegistry} import org.apache.spark.sql.catalyst.analysis.resolver.ResolverExtension import org.apache.spark.sql.catalyst.catalog.{FunctionExpressionBuilder, SessionCatalog} import org.apache.spark.sql.catalyst.expressions.{Expression, ExtractSemiStructuredFields} @@ -244,7 +244,7 @@ abstract class BaseSessionStateBuilder( new EvalSubqueriesForTimeTravel +: new ResolveTranspose(session) +: new InvokeProcedures(session) +: - ExecuteExecutableDuringAnalysis(session) +: + ExecuteImmediateCommands(session) +: ExtractSemiStructuredFields +: customResolutionRules diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala index e7da11f2be494..1f593165d4e27 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala @@ -25,7 +25,7 @@ import org.apache.hadoop.hive.ql.exec.{UDAF, UDF} import org.apache.hadoop.hive.ql.udf.generic.{AbstractGenericUDAFResolver, GenericUDF, GenericUDTF} import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.analysis.{Analyzer, EvalSubqueriesForTimeTravel, ExecuteExecutableDuringAnalysis, InvokeProcedures, ReplaceCharWithVarchar, ResolveDataSource, ResolveSessionCatalog, ResolveTranspose} +import org.apache.spark.sql.catalyst.analysis.{Analyzer, EvalSubqueriesForTimeTravel, ExecuteImmediateCommands, InvokeProcedures, ReplaceCharWithVarchar, ResolveDataSource, ResolveSessionCatalog, ResolveTranspose} import org.apache.spark.sql.catalyst.analysis.resolver.ResolverExtension import org.apache.spark.sql.catalyst.catalog.{ExternalCatalogWithListener, InvalidUDFClassException} import org.apache.spark.sql.catalyst.expressions.{Expression, ExtractSemiStructuredFields} @@ -133,7 +133,7 @@ class HiveSessionStateBuilder( new DetermineTableStats(session) +: new ResolveTranspose(session) +: new InvokeProcedures(session) +: - ExecuteExecutableDuringAnalysis(session) +: + ExecuteImmediateCommands(session) +: ExtractSemiStructuredFields +: customResolutionRules From 7bf196c9d7c048ab6b14144e266446acc6c3b181 Mon Sep 17 00:00:00 2001 From: Serge Rielau Date: Sat, 30 Aug 2025 11:20:37 -0700 Subject: [PATCH 08/32] Fix testcases --- .../catalyst/analysis/executeImmediate.scala | 43 +++++- .../analysis/AnalysisErrorSuite.scala | 65 --------- .../analysis/ExecuteImmediateCommands.scala | 5 +- .../command/ExecuteImmediateCommandExec.scala | 6 +- .../execute-immediate.sql.out | 111 ++++++++++++++-- ...rse-query-correctness-old-behavior.sql.out | 111 +++++++++++----- .../parse-query-correctness.sql.out | 49 +------ .../sql-tests/inputs/execute-immediate.sql | 21 ++- .../results/execute-immediate.sql.out | 124 ++++++++++++++++-- ...rse-query-correctness-old-behavior.sql.out | 80 +++++++++-- .../sql/errors/QueryParsingErrorsSuite.scala | 16 +-- 11 files changed, 441 insertions(+), 190 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala index 1a574c64c8edb..17f07906c3abf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala @@ -162,6 +162,8 @@ class ResolveExecuteImmediate( validateUsingClauseExpressions(args) if (queryParamResolved && allArgsResolved && targetVariablesResolved) { + // Validate parameter usage patterns + validateParameterUsage(queryParam, args) // All resolved - transform based on whether we have target variables if (targetVariables.nonEmpty) { // EXECUTE IMMEDIATE ... INTO should generate SetVariable plan @@ -229,11 +231,6 @@ class ResolveExecuteImmediate( throw QueryCompilationErrors.sqlScriptInExecuteImmediate(queryString) } - // do not allow nested execute immediate - if (plan.containsPattern(EXECUTE_IMMEDIATE)) { - throw QueryCompilationErrors.nestedExecuteImmediate(queryString) - } - plan } @@ -305,4 +302,40 @@ class ResolveExecuteImmediate( } } } + + private def validateParameterUsage(queryParam: Expression, args: Seq[Expression]): Unit = { + // Extract the query string to validate parameter patterns + val queryString = queryParam.eval(null) match { + case null => return // Will be caught later by other validation + case value => value.toString + } + + // Check for positional and named parameter patterns in the query string + val positionalParameterPattern = "\\?".r + val namedParameterPattern = ":[a-zA-Z_][a-zA-Z0-9_]*".r + val queryUsesPositionalParameters = + positionalParameterPattern.findFirstIn(queryString).isDefined + val queryUsesNamedParameters = + namedParameterPattern.findFirstIn(queryString).isDefined + + // Error if query mixes positional and named parameters + if (queryUsesPositionalParameters && queryUsesNamedParameters) { + throw QueryCompilationErrors.invalidQueryMixedQueryParameters() + } + + // If query uses only named parameters, ensure all USING expressions have names + if (queryUsesNamedParameters && !queryUsesPositionalParameters) { + val unnamedExpressions = args.zipWithIndex.collect { + case (expr, index) if !hasName(expr) => expr + } + if (unnamedExpressions.nonEmpty) { + throw QueryCompilationErrors.invalidQueryAllParametersMustBeNamed(unnamedExpressions) + } + } + } + + private def hasName(expr: Expression): Boolean = expr match { + case _: Alias => true + case _ => false + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala index 9588e9ede55be..9fee25aded08d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala @@ -834,74 +834,9 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { """"explode(array(min(a)))", "explode(array(max(a)))"""" :: Nil ) - errorConditionTest( - "EXEC IMMEDIATE - nested execute immediate not allowed", - CatalystSqlParser.parsePlan("EXECUTE IMMEDIATE 'EXECUTE IMMEDIATE \\\'SELECT 42\\\''"), - "NESTED_EXECUTE_IMMEDIATE", - Map( - "sqlString" -> "EXECUTE IMMEDIATE 'SELECT 42'")) - - errorConditionTest( - "EXEC IMMEDIATE - both positional and named used", - CatalystSqlParser.parsePlan("EXECUTE IMMEDIATE 'SELECT 42 where ? = :first'" + - " USING 1, 2 as first"), - "INVALID_QUERY_MIXED_QUERY_PARAMETERS", - Map.empty) - - test("EXEC IMMEDIATE - non string variable as sqlString parameter") { - val execImmediatePlan = ExecuteImmediateQuery( - UnresolvedAttribute("testVarA"), - Seq.empty, - Seq(UnresolvedAttribute("testVarA"))) - - assertAnalysisErrorCondition( - inputPlan = execImmediatePlan, - expectedErrorCondition = "INVALID_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE", - expectedMessageParameters = Map( - "exprType" -> "\"INT\"" - )) - } - - test("EXEC IMMEDIATE - Null string as sqlString parameter") { - val execImmediatePlan = ExecuteImmediateQuery( - UnresolvedAttribute("testVarNull"), - Seq.empty, - Seq(UnresolvedAttribute("testVarNull"))) - - assertAnalysisErrorCondition( - inputPlan = execImmediatePlan, - expectedErrorCondition = "NULL_QUERY_STRING_EXECUTE_IMMEDIATE", - expectedMessageParameters = Map("varName" -> "`testVarNull`")) - } - - - test("EXEC IMMEDIATE - Unsupported expr for parameter") { - val execImmediatePlan: LogicalPlan = ExecuteImmediateQuery( - Literal("SELECT ?"), - Seq(UnresolvedAttribute("testVarA"), NaNvl(Literal(1), Literal(1))), - Seq.empty) - assertAnalysisErrorCondition( - inputPlan = execImmediatePlan, - expectedErrorCondition = "UNSUPPORTED_EXPR_FOR_PARAMETER", - expectedMessageParameters = Map( - "invalidExprSql" -> "\"nanvl(1, 1)\"" - )) - } - test("EXEC IMMEDIATE - Name Parametrize query with non named parameters") { - val execImmediateSetVariablePlan = ExecuteImmediateQuery( - Literal("SELECT :first"), - Seq(Literal(2), new Alias(UnresolvedAttribute("testVarA"), "first")(), Literal(3)), - Seq.empty) - assertAnalysisErrorCondition( - inputPlan = execImmediateSetVariablePlan, - expectedErrorCondition = "ALL_PARAMETERS_MUST_BE_NAMED", - expectedMessageParameters = Map( - "exprs" -> "\"2\", \"3\"" - )) - } test("EXEC IMMEDIATE - INTO specified for COMMAND query") { val execImmediateSetVariablePlan = ExecuteImmediateQuery( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediateCommands.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediateCommands.scala index 29396d8fa1aea..fd9eb967fd957 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediateCommands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediateCommands.scala @@ -118,10 +118,7 @@ case class ExecuteImmediateCommands(sparkSession: SparkSession) extends Rule[Log throw QueryCompilationErrors.sqlScriptInExecuteImmediate(queryString) } - // Check for nested EXECUTE IMMEDIATE - if (parsedPlan.containsPattern(EXECUTE_IMMEDIATE)) { - throw QueryCompilationErrors.nestedExecuteImmediate(queryString) - } + } private def substituteParameters(queryString: String, paramValues: Seq[Any]): String = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ExecuteImmediateCommandExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ExecuteImmediateCommandExec.scala index 020c99732812e..0e561f2cf892b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ExecuteImmediateCommandExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ExecuteImmediateCommandExec.scala @@ -23,7 +23,6 @@ import org.apache.spark.sql.{AnalysisException, DataFrame, Row, SparkSession} import org.apache.spark.sql.catalyst.analysis.AnalysisContext import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, VariableReference} import org.apache.spark.sql.catalyst.plans.logical.{CompoundBody, LogicalPlan} -import org.apache.spark.sql.catalyst.trees.TreePattern.EXECUTE_IMMEDIATE import org.apache.spark.sql.errors.QueryCompilationErrors import org.apache.spark.sql.execution.command.LeafRunnableCommand import org.apache.spark.sql.types.StringType @@ -134,10 +133,7 @@ case class ExecuteImmediateCommandExec( throw QueryCompilationErrors.sqlScriptInExecuteImmediate(queryString) } - // Check for nested EXECUTE IMMEDIATE - if (parsedPlan.containsPattern(EXECUTE_IMMEDIATE)) { - throw QueryCompilationErrors.nestedExecuteImmediate(queryString) - } + } private def executeQuery( diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/execute-immediate.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/execute-immediate.sql.out index 7c2307d2d81b3..efd18e3478553 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/execute-immediate.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/execute-immediate.sql.out @@ -631,16 +631,9 @@ org.apache.spark.sql.AnalysisException -- !query -EXECUTE IMMEDIATE 'EXECUTE IMMEDIATE \'SELECT id FROM tbl_view WHERE id = ? USING 10\'' +EXECUTE IMMEDIATE 'EXECUTE IMMEDIATE \'SELECT id FROM tbl_view WHERE id = ?\' USING 10' -- !query analysis -org.apache.spark.sql.AnalysisException -{ - "errorClass" : "NESTED_EXECUTE_IMMEDIATE", - "sqlState" : "07501", - "messageParameters" : { - "sqlString" : "EXECUTE IMMEDIATE 'SELECT ID FROM TBL_VIEW WHERE ID = ? USING 10'" - } -} +LocalRelation [id#x] -- !query @@ -716,6 +709,106 @@ org.apache.spark.sql.AnalysisException } +-- !query +EXECUTE IMMEDIATE 'SELECT 42 where ? = :first' USING 1, 2 as first +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_QUERY_MIXED_QUERY_PARAMETERS", + "sqlState" : "42613" +} + + +-- !query +DECLARE int_var INT +-- !query analysis +CreateVariable defaultvalueexpression(null, null), false ++- ResolvedIdentifier org.apache.spark.sql.catalyst.analysis.FakeSystemCatalog$@xxxxxxxx, session.int_var + + +-- !query +SET VAR int_var = 42 +-- !query analysis +SetVariable [variablereference(system.session.int_var=CAST(NULL AS INT))] ++- Project [42 AS int_var#x] + +- OneRowRelation + + +-- !query +EXECUTE IMMEDIATE int_var +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE", + "sqlState" : "42K09", + "messageParameters" : { + "exprType" : "\"INT\"" + } +} + + +-- !query +DECLARE null_var STRING +-- !query analysis +CreateVariable defaultvalueexpression(null, null), false ++- ResolvedIdentifier org.apache.spark.sql.catalyst.analysis.FakeSystemCatalog$@xxxxxxxx, session.null_var + + +-- !query +SET VAR null_var = null +-- !query analysis +SetVariable [variablereference(system.session.null_var=CAST(NULL AS STRING))] ++- Project [cast(null_var#x as string) AS null_var#x] + +- Project [null AS null_var#x] + +- OneRowRelation + + +-- !query +EXECUTE IMMEDIATE null_var +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NULL_QUERY_STRING_EXECUTE_IMMEDIATE", + "sqlState" : "22004", + "messageParameters" : { + "varName" : "`null_var`" + } +} + + +-- !query +EXECUTE IMMEDIATE 'SELECT ?' USING (SELECT 1) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_EXPR_FOR_PARAMETER", + "sqlState" : "42K0E", + "messageParameters" : { + "invalidExprSql" : "\"scalarsubquery()\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 36, + "stopIndex" : 45, + "fragment" : "(SELECT 1)" + } ] +} + + +-- !query +EXECUTE IMMEDIATE 'SELECT :first' USING 2, 3 +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "ALL_PARAMETERS_MUST_BE_NAMED", + "sqlState" : "07001", + "messageParameters" : { + "exprs" : "\"2\", \"3\"" + } +} + + -- !query DROP TABLE x -- !query analysis diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/parse-query-correctness-old-behavior.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/parse-query-correctness-old-behavior.sql.out index 1c9db4eaed6f6..eca46dd524367 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/parse-query-correctness-old-behavior.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/parse-query-correctness-old-behavior.sql.out @@ -994,24 +994,25 @@ CreateVariable defaultvalueexpression(null, null), false -- !query EXECUTE IMMEDIATE 'SELECT 1 UNION SELECT 2 UNION SELECT 3' INTO v1 -- !query analysis -SetVariable [variablereference(system.session.v1=CAST(NULL AS INT))] -+- Project [1 AS UNION#x] - +- OneRowRelation +org.apache.spark.SparkException +{ + "errorClass" : "ROW_SUBQUERY_TOO_MANY_ROWS", + "sqlState" : "21000" +} -- !query SELECT v1 -- !query analysis -Project [variablereference(system.session.v1=1) AS v1#x] +Project [variablereference(system.session.v1=CAST(NULL AS INT)) AS v1#x] +- OneRowRelation -- !query EXECUTE IMMEDIATE 'SELECT 1 UNION SELECT 1 UNION SELECT 1' INTO v1 -- !query analysis -SetVariable [variablereference(system.session.v1=1)] -+- Project [1 AS UNION#x] - +- OneRowRelation +SetVariable [variablereference(system.session.v1=CAST(NULL AS INT))] ++- LocalRelation [1#x] -- !query @@ -1025,8 +1026,7 @@ Project [variablereference(system.session.v1=1) AS v1#x] EXECUTE IMMEDIATE 'SELECT 1 EXCEPT SELECT 2 EXCEPT SELECT 3' INTO v1 -- !query analysis SetVariable [variablereference(system.session.v1=1)] -+- Project [1 AS EXCEPT#x] - +- OneRowRelation ++- LocalRelation [1#x] -- !query @@ -1040,38 +1040,35 @@ Project [variablereference(system.session.v1=1) AS v1#x] EXECUTE IMMEDIATE 'SELECT 1 EXCEPT SELECT 1 EXCEPT SELECT 1' INTO v1 -- !query analysis SetVariable [variablereference(system.session.v1=1)] -+- Project [1 AS EXCEPT#x] - +- OneRowRelation ++- LocalRelation , [1#x] -- !query SELECT v1 -- !query analysis -Project [variablereference(system.session.v1=1) AS v1#x] +Project [variablereference(system.session.v1=CAST(NULL AS INT)) AS v1#x] +- OneRowRelation -- !query EXECUTE IMMEDIATE 'SELECT 1 INTERSECT SELECT 2 INTERSECT SELECT 3' INTO v1 -- !query analysis -SetVariable [variablereference(system.session.v1=1)] -+- Project [1 AS INTERSECT#x] - +- OneRowRelation +SetVariable [variablereference(system.session.v1=CAST(NULL AS INT))] ++- LocalRelation , [1#x] -- !query SELECT v1 -- !query analysis -Project [variablereference(system.session.v1=1) AS v1#x] +Project [variablereference(system.session.v1=CAST(NULL AS INT)) AS v1#x] +- OneRowRelation -- !query EXECUTE IMMEDIATE 'SELECT 1 INTERSECT SELECT 1 INTERSECT SELECT 1' INTO v1 -- !query analysis -SetVariable [variablereference(system.session.v1=1)] -+- Project [1 AS INTERSECT#x] - +- OneRowRelation +SetVariable [variablereference(system.session.v1=CAST(NULL AS INT))] ++- LocalRelation [1#x] -- !query @@ -1084,9 +1081,22 @@ Project [variablereference(system.session.v1=1) AS v1#x] -- !query EXECUTE IMMEDIATE 'SELECT 1 JOIN SELECT 2' INTO v1 -- !query analysis -SetVariable [variablereference(system.session.v1=1)] -+- Project [1 AS JOIN#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'SELECT'", + "hint" : "" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 50, + "fragment" : "EXECUTE IMMEDIATE 'SELECT 1 JOIN SELECT 2' INTO v1" + } ] +} -- !query @@ -1099,9 +1109,22 @@ Project [variablereference(system.session.v1=1) AS v1#x] -- !query EXECUTE IMMEDIATE 'SELECT 1 VALUES (1)' INTO v1 -- !query analysis -SetVariable [variablereference(system.session.v1=1)] -+- Project [1 AS VALUES#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 47, + "fragment" : "EXECUTE IMMEDIATE 'SELECT 1 VALUES (1)' INTO v1" + } ] +} -- !query @@ -1114,9 +1137,22 @@ Project [variablereference(system.session.v1=1) AS v1#x] -- !query EXECUTE IMMEDIATE 'SELECT 1 alias garbage garbage garbage' INTO v1 -- !query analysis -SetVariable [variablereference(system.session.v1=1)] -+- Project [1 AS alias#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'garbage'", + "hint" : "" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 66, + "fragment" : "EXECUTE IMMEDIATE 'SELECT 1 alias garbage garbage garbage' INTO v1" + } ] +} -- !query @@ -1129,9 +1165,22 @@ Project [variablereference(system.session.v1=1) AS v1#x] -- !query EXECUTE IMMEDIATE 'SELECT 1 WITH abc' INTO v1 -- !query analysis -SetVariable [variablereference(system.session.v1=1)] -+- Project [1 AS WITH#x] - +- OneRowRelation +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'abc'", + "hint" : ": extra input 'abc'" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 45, + "fragment" : "EXECUTE IMMEDIATE 'SELECT 1 WITH abc' INTO v1" + } ] +} -- !query diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/parse-query-correctness.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/parse-query-correctness.sql.out index 3abcad1b3251d..3c5aab62938f9 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/parse-query-correctness.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/parse-query-correctness.sql.out @@ -1140,18 +1140,7 @@ Project [variablereference(system.session.v1=CAST(NULL AS INT)) AS v1#x] EXECUTE IMMEDIATE 'SELECT 1 UNION SELECT 1 UNION SELECT 1' INTO v1 -- !query analysis SetVariable [variablereference(system.session.v1=CAST(NULL AS INT))] -+- GlobalLimit 2 - +- LocalLimit 2 - +- Distinct - +- Union false, false - :- Distinct - : +- Union false, false - : :- Project [1 AS 1#x] - : : +- OneRowRelation - : +- Project [1 AS 1#x] - : +- OneRowRelation - +- Project [1 AS 1#x] - +- OneRowRelation ++- LocalRelation [1#x] -- !query @@ -1165,14 +1154,7 @@ Project [variablereference(system.session.v1=1) AS v1#x] EXECUTE IMMEDIATE 'SELECT 1 EXCEPT SELECT 2 EXCEPT SELECT 3' INTO v1 -- !query analysis SetVariable [variablereference(system.session.v1=1)] -+- Except false - :- Except false - : :- Project [1 AS 1#x] - : : +- OneRowRelation - : +- Project [2 AS 2#x] - : +- OneRowRelation - +- Project [3 AS 3#x] - +- OneRowRelation ++- LocalRelation [1#x] -- !query @@ -1186,14 +1168,7 @@ Project [variablereference(system.session.v1=1) AS v1#x] EXECUTE IMMEDIATE 'SELECT 1 EXCEPT SELECT 1 EXCEPT SELECT 1' INTO v1 -- !query analysis SetVariable [variablereference(system.session.v1=1)] -+- Except false - :- Except false - : :- Project [1 AS 1#x] - : : +- OneRowRelation - : +- Project [1 AS 1#x] - : +- OneRowRelation - +- Project [1 AS 1#x] - +- OneRowRelation ++- LocalRelation , [1#x] -- !query @@ -1207,14 +1182,7 @@ Project [variablereference(system.session.v1=CAST(NULL AS INT)) AS v1#x] EXECUTE IMMEDIATE 'SELECT 1 INTERSECT SELECT 2 INTERSECT SELECT 3' INTO v1 -- !query analysis SetVariable [variablereference(system.session.v1=CAST(NULL AS INT))] -+- Intersect false - :- Intersect false - : :- Project [1 AS 1#x] - : : +- OneRowRelation - : +- Project [2 AS 2#x] - : +- OneRowRelation - +- Project [3 AS 3#x] - +- OneRowRelation ++- LocalRelation , [1#x] -- !query @@ -1228,14 +1196,7 @@ Project [variablereference(system.session.v1=CAST(NULL AS INT)) AS v1#x] EXECUTE IMMEDIATE 'SELECT 1 INTERSECT SELECT 1 INTERSECT SELECT 1' INTO v1 -- !query analysis SetVariable [variablereference(system.session.v1=CAST(NULL AS INT))] -+- Intersect false - :- Intersect false - : :- Project [1 AS 1#x] - : : +- OneRowRelation - : +- Project [1 AS 1#x] - : +- OneRowRelation - +- Project [1 AS 1#x] - +- OneRowRelation ++- LocalRelation [1#x] -- !query diff --git a/sql/core/src/test/resources/sql-tests/inputs/execute-immediate.sql b/sql/core/src/test/resources/sql-tests/inputs/execute-immediate.sql index 402f3bfdddcec..5ba3b9771d422 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/execute-immediate.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/execute-immediate.sql @@ -142,7 +142,7 @@ EXECUTE IMMEDIATE 'SELECT id FROM tbl_view WHERE id = :p' USING p, 'p'; EXECUTE IMMEDIATE 'SELECT id, data.f1 FROM tbl_view WHERE id = 10' INTO res_id, res_id; -- nested execute immediate -EXECUTE IMMEDIATE 'EXECUTE IMMEDIATE \'SELECT id FROM tbl_view WHERE id = ? USING 10\''; +EXECUTE IMMEDIATE 'EXECUTE IMMEDIATE \'SELECT id FROM tbl_view WHERE id = ?\' USING 10'; -- sqlString is null SET VAR sql_string = null; @@ -156,4 +156,23 @@ EXECUTE IMMEDIATE sql_string; SET VAR sql_string = 'hello'; EXECUTE IMMEDIATE length(sql_string); +-- mixed positional and named parameters in query +EXECUTE IMMEDIATE 'SELECT 42 where ? = :first' USING 1, 2 as first; + +-- non-string variable as sqlString parameter +DECLARE int_var INT; +SET VAR int_var = 42; +EXECUTE IMMEDIATE int_var; + +-- null string as sqlString parameter +DECLARE null_var STRING; +SET VAR null_var = null; +EXECUTE IMMEDIATE null_var; + +-- unsupported expression for parameter (subquery) +EXECUTE IMMEDIATE 'SELECT ?' USING (SELECT 1); + +-- named query with unnamed parameters +EXECUTE IMMEDIATE 'SELECT :first' USING 2, 3; + DROP TABLE x; \ No newline at end of file diff --git a/sql/core/src/test/resources/sql-tests/results/execute-immediate.sql.out b/sql/core/src/test/resources/sql-tests/results/execute-immediate.sql.out index c3150ed270613..29ba68b1f43f5 100644 --- a/sql/core/src/test/resources/sql-tests/results/execute-immediate.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/execute-immediate.sql.out @@ -732,18 +732,11 @@ org.apache.spark.sql.AnalysisException -- !query -EXECUTE IMMEDIATE 'EXECUTE IMMEDIATE \'SELECT id FROM tbl_view WHERE id = ? USING 10\'' +EXECUTE IMMEDIATE 'EXECUTE IMMEDIATE \'SELECT id FROM tbl_view WHERE id = ?\' USING 10' -- !query schema -struct<> +struct -- !query output -org.apache.spark.sql.AnalysisException -{ - "errorClass" : "NESTED_EXECUTE_IMMEDIATE", - "sqlState" : "07501", - "messageParameters" : { - "sqlString" : "EXECUTE IMMEDIATE 'SELECT ID FROM TBL_VIEW WHERE ID = ? USING 10'" - } -} +10 -- !query @@ -823,6 +816,117 @@ org.apache.spark.sql.AnalysisException } +-- !query +EXECUTE IMMEDIATE 'SELECT 42 where ? = :first' USING 1, 2 as first +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_QUERY_MIXED_QUERY_PARAMETERS", + "sqlState" : "42613" +} + + +-- !query +DECLARE int_var INT +-- !query schema +struct<> +-- !query output + + + +-- !query +SET VAR int_var = 42 +-- !query schema +struct<> +-- !query output + + + +-- !query +EXECUTE IMMEDIATE int_var +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE", + "sqlState" : "42K09", + "messageParameters" : { + "exprType" : "\"INT\"" + } +} + + +-- !query +DECLARE null_var STRING +-- !query schema +struct<> +-- !query output + + + +-- !query +SET VAR null_var = null +-- !query schema +struct<> +-- !query output + + + +-- !query +EXECUTE IMMEDIATE null_var +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "NULL_QUERY_STRING_EXECUTE_IMMEDIATE", + "sqlState" : "22004", + "messageParameters" : { + "varName" : "`null_var`" + } +} + + +-- !query +EXECUTE IMMEDIATE 'SELECT ?' USING (SELECT 1) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_EXPR_FOR_PARAMETER", + "sqlState" : "42K0E", + "messageParameters" : { + "invalidExprSql" : "\"scalarsubquery()\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 36, + "stopIndex" : 45, + "fragment" : "(SELECT 1)" + } ] +} + + +-- !query +EXECUTE IMMEDIATE 'SELECT :first' USING 2, 3 +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "ALL_PARAMETERS_MUST_BE_NAMED", + "sqlState" : "07001", + "messageParameters" : { + "exprs" : "\"2\", \"3\"" + } +} + + -- !query DROP TABLE x -- !query schema diff --git a/sql/core/src/test/resources/sql-tests/results/parse-query-correctness-old-behavior.sql.out b/sql/core/src/test/resources/sql-tests/results/parse-query-correctness-old-behavior.sql.out index 6c9751864324d..9a523e9562f96 100644 --- a/sql/core/src/test/resources/sql-tests/results/parse-query-correctness-old-behavior.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/parse-query-correctness-old-behavior.sql.out @@ -614,7 +614,11 @@ EXECUTE IMMEDIATE 'SELECT 1 UNION SELECT 2 UNION SELECT 3' INTO v1 -- !query schema struct<> -- !query output - +org.apache.spark.SparkException +{ + "errorClass" : "ROW_SUBQUERY_TOO_MANY_ROWS", + "sqlState" : "21000" +} -- !query @@ -622,7 +626,7 @@ SELECT v1 -- !query schema struct -- !query output -1 +NULL -- !query @@ -670,7 +674,7 @@ SELECT v1 -- !query schema struct -- !query output -1 +NULL -- !query @@ -686,7 +690,7 @@ SELECT v1 -- !query schema struct -- !query output -1 +NULL -- !query @@ -710,7 +714,22 @@ EXECUTE IMMEDIATE 'SELECT 1 JOIN SELECT 2' INTO v1 -- !query schema struct<> -- !query output - +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'SELECT'", + "hint" : "" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 50, + "fragment" : "EXECUTE IMMEDIATE 'SELECT 1 JOIN SELECT 2' INTO v1" + } ] +} -- !query @@ -726,7 +745,22 @@ EXECUTE IMMEDIATE 'SELECT 1 VALUES (1)' INTO v1 -- !query schema struct<> -- !query output - +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'('", + "hint" : "" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 47, + "fragment" : "EXECUTE IMMEDIATE 'SELECT 1 VALUES (1)' INTO v1" + } ] +} -- !query @@ -742,7 +776,22 @@ EXECUTE IMMEDIATE 'SELECT 1 alias garbage garbage garbage' INTO v1 -- !query schema struct<> -- !query output - +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'garbage'", + "hint" : "" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 66, + "fragment" : "EXECUTE IMMEDIATE 'SELECT 1 alias garbage garbage garbage' INTO v1" + } ] +} -- !query @@ -758,7 +807,22 @@ EXECUTE IMMEDIATE 'SELECT 1 WITH abc' INTO v1 -- !query schema struct<> -- !query output - +org.apache.spark.sql.catalyst.parser.ParseException +{ + "errorClass" : "PARSE_SYNTAX_ERROR", + "sqlState" : "42601", + "messageParameters" : { + "error" : "'abc'", + "hint" : ": extra input 'abc'" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 1, + "stopIndex" : 45, + "fragment" : "EXECUTE IMMEDIATE 'SELECT 1 WITH abc' INTO v1" + } ] +} -- !query diff --git a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryParsingErrorsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryParsingErrorsSuite.scala index 666f85e19c1c3..629d85f19b0a2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryParsingErrorsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryParsingErrorsSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.errors import org.apache.spark.SparkThrowable -import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.{AnalysisException, QueryTest} import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.catalyst.plans.SQLHelper import org.apache.spark.sql.catalyst.util.TypeUtils.toSQLId @@ -62,16 +62,16 @@ class QueryParsingErrorsSuite extends QueryTest with SharedSparkSession with SQL ) } - test("PARSE_SYNTAX_ERROR: Execute immediate syntax error with INTO specified") { + test("UNRESOLVED_COLUMN: Execute immediate with unresolved column in INTO clause") { val query = "EXECUTE IMMEDIATE 'SELCT 1707 WHERE ? = 1' INTO a USING 1" checkError( - exception = parseException(query), - condition = "PARSE_SYNTAX_ERROR", - parameters = Map("error" -> "'SELCT'", "hint" -> ""), + exception = intercept[AnalysisException](sql(query)), + condition = "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + parameters = Map("objectName" -> "`a`"), context = ExpectedContext( - start = 0, - stop = 56, - fragment = query) + start = 48, + stop = 48, + fragment = "a") ) } From 0198a3d158f82e3a16896add3161f5a1e0c7d200 Mon Sep 17 00:00:00 2001 From: Serge Rielau Date: Sat, 30 Aug 2025 14:16:28 -0700 Subject: [PATCH 09/32] Fix more testcases --- .../analysis/ExecuteImmediateCommands.scala | 2 +- .../command/ExecuteImmediateCommandExec.scala | 170 ------------------ .../apache/spark/sql/ParametersSuite.scala | 17 +- 3 files changed, 14 insertions(+), 175 deletions(-) delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/command/ExecuteImmediateCommandExec.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediateCommands.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediateCommands.scala index fd9eb967fd957..1a185dabe5599 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediateCommands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediateCommands.scala @@ -117,9 +117,9 @@ case class ExecuteImmediateCommands(sparkSession: SparkSession) extends Rule[Log if (parsedPlan.isInstanceOf[CompoundBody]) { throw QueryCompilationErrors.sqlScriptInExecuteImmediate(queryString) } + } - } private def substituteParameters(queryString: String, paramValues: Seq[Any]): String = { // For now, just handle positional parameters diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ExecuteImmediateCommandExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ExecuteImmediateCommandExec.scala deleted file mode 100644 index 0e561f2cf892b..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ExecuteImmediateCommandExec.scala +++ /dev/null @@ -1,170 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.execution.command - -import scala.util.{Either, Left, Right} - -import org.apache.spark.sql.{AnalysisException, DataFrame, Row, SparkSession} -import org.apache.spark.sql.catalyst.analysis.AnalysisContext -import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, VariableReference} -import org.apache.spark.sql.catalyst.plans.logical.{CompoundBody, LogicalPlan} -import org.apache.spark.sql.errors.QueryCompilationErrors -import org.apache.spark.sql.execution.command.LeafRunnableCommand -import org.apache.spark.sql.types.StringType - -/** - * Runnable command that executes an EXECUTE IMMEDIATE statement by recursively - * invoking the SQL execution pipeline. - */ -case class ExecuteImmediateCommandExec( - args: Seq[Expression], - query: Either[String, VariableReference], - targetVariables: Seq[VariableReference]) extends LeafRunnableCommand { - - override def output: Seq[Attribute] = { - if (targetVariables.nonEmpty) { - // For EXECUTE IMMEDIATE ... INTO, return empty output - Seq.empty - } else { - // For regular EXECUTE IMMEDIATE, get the schema from the executed query - // This is a bit of a hack, but necessary for the type system - try { - val queryString = extractQueryString(null, query) // Use null SparkSession for schema only - // We can't execute the query here due to circular dependencies - // Return a placeholder that will be corrected during execution - Seq.empty - } catch { - case _: Exception => Seq.empty - } - } - } - - override def run(sparkSession: SparkSession): Seq[Row] = { - // Extract the query string - val queryString = extractQueryString(sparkSession, query) - - // Prepare arguments for parameterized queries - val resolvedArgs = resolveArguments(sparkSession, args) - - try { - // Parse the query to understand its structure - val parsedPlan = sparkSession.sessionState.sqlParser.parsePlan(queryString) - - // Check for unsupported features - validateQuery(queryString, parsedPlan) - - // Execute the query recursively through the SQL execution pipeline - val result = executeQuery(sparkSession, queryString, parsedPlan, resolvedArgs) - - // Handle target variables if specified - if (targetVariables.nonEmpty) { - handleTargetVariables(sparkSession, result, targetVariables) - Seq.empty // No output for INTO queries - } else { - // For regular EXECUTE IMMEDIATE, we can't return rows via RunnableCommand - // because of schema mismatch issues. The CommandResult mechanism will handle this. - // For now, return empty and let the test fail with a clearer error - throw new UnsupportedOperationException( - "EXECUTE IMMEDIATE with result return is not yet fully implemented. " + - "The query was executed but results cannot be returned via RunnableCommand " + - "due to schema limitations.") - } - } catch { - case e: AnalysisException => - // Re-throw AnalysisException as-is to preserve error type for tests - throw e - case e: Exception => - throw new RuntimeException(s"Failed to execute immediate query: ${e.getMessage}", e) - } - } - - private def extractQueryString( - sparkSession: SparkSession, - queryExpr: Either[String, VariableReference]): String = { - queryExpr match { - case Left(literal) => literal - case Right(variable) => - // Evaluate the variable reference - if (!variable.dataType.sameType(StringType)) { - throw QueryCompilationErrors.invalidExecuteImmediateExpressionType(variable.dataType) - } - - val value = variable.eval(null) - if (value == null) { - throw QueryCompilationErrors.nullSQLStringExecuteImmediate(variable.identifier.name()) - } - - value.toString - } - } - - private def resolveArguments( - sparkSession: SparkSession, - expressions: Seq[Expression]): Seq[Expression] = { - expressions.map { expr => - if (expr.resolved) { - expr - } else { - // For now, just return the expression as-is - // In a complete implementation, you would need to resolve variables/parameters - expr - } - } - } - - private def validateQuery(queryString: String, parsedPlan: LogicalPlan): Unit = { - // Check for compound bodies (SQL scripting) - if (parsedPlan.isInstanceOf[CompoundBody]) { - throw QueryCompilationErrors.sqlScriptInExecuteImmediate(queryString) - } - - - } - - private def executeQuery( - sparkSession: SparkSession, - queryString: String, - parsedPlan: LogicalPlan, - resolvedArgs: Seq[Expression]): DataFrame = { - - // For now, use the SparkSession.sql method which handles parameterization - // This is the recursive SQL execution we want - if (resolvedArgs.isEmpty) { - // Execute within EXECUTE IMMEDIATE context to isolate variables - AnalysisContext.withExecuteImmediateContext { - sparkSession.sql(queryString) - } - } else { - // For parameterized queries, convert resolved args to values and pass them to sql() - val paramValues = resolvedArgs.map(_.eval(null)) - AnalysisContext.withExecuteImmediateContext { - sparkSession.sql(queryString, paramValues.toArray) - } - } - } - - private def handleTargetVariables( - sparkSession: SparkSession, - result: DataFrame, - targetVars: Seq[VariableReference]): Unit = { - // This would need to set session variables with the query results - // For now, we'll throw an error as this functionality would need additional implementation - throw new UnsupportedOperationException( - "EXECUTE IMMEDIATE ... INTO with recursive execution is not yet implemented") - } -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ParametersSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ParametersSuite.scala index 0c2380a2d90e3..c8af349977c31 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ParametersSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ParametersSuite.scala @@ -21,7 +21,7 @@ import java.time.{Instant, LocalDate, LocalDateTime, ZoneId} import org.apache.spark.sql.catalyst.expressions.Literal import org.apache.spark.sql.catalyst.parser.ParseException -import org.apache.spark.sql.catalyst.plans.logical.Limit +import org.apache.spark.sql.catalyst.plans.logical.{Limit, LocalRelation} import org.apache.spark.sql.functions.{array, call_function, lit, map, map_from_arrays, map_from_entries, str_to_map, struct} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession @@ -616,12 +616,21 @@ class ParametersSuite extends QueryTest with SharedSparkSession { val variableDirectly = sql("SELECT 42 WHERE 1 = a").queryExecution.optimizedPlan val parameterizedSpark = spark.sql("SELECT 42 WHERE 1 = ?", Array(1)).queryExecution.optimizedPlan - val parameterizedSql = - spark.sql("EXECUTE IMMEDIATE 'SELECT 42 WHERE 1 = ?' USING a").queryExecution.optimizedPlan + // Test that variable folding works consistently for regular queries and parameterized queries comparePlans(expected, variableDirectly) comparePlans(expected, parameterizedSpark) - comparePlans(expected, parameterizedSql) + + // EXECUTE IMMEDIATE has different semantics - it executes during analysis and returns + // materialized results as LocalRelation, not logical plans for optimization + val executeImmediateResult = + spark.sql("EXECUTE IMMEDIATE 'SELECT 42 WHERE 1 = ?' USING a") + + // Verify EXECUTE IMMEDIATE produces the correct result (42) + checkAnswer(executeImmediateResult, Row(42)) + + // Verify EXECUTE IMMEDIATE returns LocalRelation (materialized results), not logical plan + assert(executeImmediateResult.queryExecution.optimizedPlan.isInstanceOf[LocalRelation]) } test("SPARK-49017: bind named parameters with IDENTIFIER clause") { From dd3d2bc236b9452e02ada2365634173030dca492 Mon Sep 17 00:00:00 2001 From: Serge Rielau Date: Sat, 30 Aug 2025 14:41:51 -0700 Subject: [PATCH 10/32] Cleanup --- .../catalyst/analysis/executeImmediate.scala | 80 +----- .../analysis/ExecuteImmediateCommands.scala | 261 +----------------- 2 files changed, 13 insertions(+), 328 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala index 17f07906c3abf..4d7c2d3217e38 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala @@ -17,11 +17,9 @@ package org.apache.spark.sql.catalyst.analysis -import scala.util.{Either, Left, Right} - import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Expression, VariableReference} import org.apache.spark.sql.catalyst.parser.ParseException -import org.apache.spark.sql.catalyst.plans.logical.{CompoundBody, ExecutableDuringAnalysis, LocalRelation, LogicalPlan, SetVariable, UnaryNode} +import org.apache.spark.sql.catalyst.plans.logical.{ExecutableDuringAnalysis, LocalRelation, LogicalPlan, SetVariable, UnaryNode} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.trees.TreePattern.{EXECUTE_IMMEDIATE, TreePattern} import org.apache.spark.sql.connector.catalog.CatalogManager @@ -162,8 +160,6 @@ class ResolveExecuteImmediate( validateUsingClauseExpressions(args) if (queryParamResolved && allArgsResolved && targetVariablesResolved) { - // Validate parameter usage patterns - validateParameterUsage(queryParam, args) // All resolved - transform based on whether we have target variables if (targetVariables.nonEmpty) { // EXECUTE IMMEDIATE ... INTO should generate SetVariable plan @@ -205,44 +201,6 @@ class ResolveExecuteImmediate( } } - private def parseStatement( - queryString: String, - targetVariables: Seq[Expression]): LogicalPlan = { - // If targetVariables is defined, statement needs to be a query. - // Otherwise, it can be anything. - val plan = if (targetVariables.nonEmpty) { - try { - catalogManager.v1SessionCatalog.parser.parseQuery(queryString) - } catch { - case e: ParseException => - // Since we do not have a way of telling that parseQuery failed because of - // actual parsing error or because statement was passed where query was expected, - // we need to make sure that parsePlan wouldn't throw - catalogManager.v1SessionCatalog.parser.parsePlan(queryString) - - // Plan was successfully parsed, but query wasn't - throw. - throw QueryCompilationErrors.invalidStatementForExecuteInto(queryString) - } - } else { - catalogManager.v1SessionCatalog.parser.parsePlan(queryString) - } - - if (plan.isInstanceOf[CompoundBody]) { - throw QueryCompilationErrors.sqlScriptInExecuteImmediate(queryString) - } - - plan - } - - - - private def isQueryResolved(query: Either[String, UnresolvedAttribute]): Boolean = { - query match { - case Left(_) => true // String literals are always resolved - case Right(attr) => attr.resolved // Check if the attribute is resolved - } - } - private def getVariableReference(expr: Expression, nameParts: Seq[String]): VariableReference = { variableResolution.lookupVariable( nameParts = nameParts, @@ -302,40 +260,4 @@ class ResolveExecuteImmediate( } } } - - private def validateParameterUsage(queryParam: Expression, args: Seq[Expression]): Unit = { - // Extract the query string to validate parameter patterns - val queryString = queryParam.eval(null) match { - case null => return // Will be caught later by other validation - case value => value.toString - } - - // Check for positional and named parameter patterns in the query string - val positionalParameterPattern = "\\?".r - val namedParameterPattern = ":[a-zA-Z_][a-zA-Z0-9_]*".r - val queryUsesPositionalParameters = - positionalParameterPattern.findFirstIn(queryString).isDefined - val queryUsesNamedParameters = - namedParameterPattern.findFirstIn(queryString).isDefined - - // Error if query mixes positional and named parameters - if (queryUsesPositionalParameters && queryUsesNamedParameters) { - throw QueryCompilationErrors.invalidQueryMixedQueryParameters() - } - - // If query uses only named parameters, ensure all USING expressions have names - if (queryUsesNamedParameters && !queryUsesPositionalParameters) { - val unnamedExpressions = args.zipWithIndex.collect { - case (expr, index) if !hasName(expr) => expr - } - if (unnamedExpressions.nonEmpty) { - throw QueryCompilationErrors.invalidQueryAllParametersMustBeNamed(unnamedExpressions) - } - } - } - - private def hasName(expr: Expression): Boolean = expr match { - case _: Alias => true - case _ => false - } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediateCommands.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediateCommands.scala index 1a185dabe5599..2e3ebf0ec1d48 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediateCommands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediateCommands.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.{AnalysisException, SparkSession} -import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, VariableReference} +import org.apache.spark.sql.catalyst.expressions.{Alias, Expression} import org.apache.spark.sql.catalyst.plans.logical.{CompoundBody, LocalRelation, LogicalPlan} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.trees.TreePattern.EXECUTE_IMMEDIATE @@ -55,21 +55,17 @@ case class ExecuteImmediateCommands(sparkSession: SparkSession) extends Rule[Log } } else { // For parameterized queries, build unified parameter arrays - // The args are already resolved by outer parameter resolution val (paramValues, paramNames) = buildUnifiedParameters(cmd.args) // Validate parameter usage patterns validateParameterUsage(cmd.queryParam, cmd.args, paramNames.toSeq) AnalysisContext.withExecuteImmediateContext { - // Use the new unified parameter API - let the inner query decide - // whether to use positional or named parameters based on its markers sparkSession.sql(queryString, paramValues, paramNames) } } // Return the query results as a LocalRelation - // ExecuteImmediateCommand returns query results; SetVariable handles variable assignment val internalRows = result.queryExecution.executedPlan.executeCollect() LocalRelation(result.queryExecution.analyzed.output, internalRows.toSeq) @@ -119,70 +115,8 @@ case class ExecuteImmediateCommands(sparkSession: SparkSession) extends Rule[Log } } - - - private def substituteParameters(queryString: String, paramValues: Seq[Any]): String = { - // For now, just handle positional parameters - // Named parameters require more complex resolution that involves the original args expressions - var substituted = queryString - var paramIndex = 0 - - // Handle positional parameters (?) - while (substituted.contains("?") && paramIndex < paramValues.length) { - val value = paramValues(paramIndex) - val sqlLiteral = formatSqlLiteral(value) - substituted = substituted.replaceFirst("\\?", sqlLiteral) - paramIndex += 1 - } - - substituted - } - - private def substituteParametersWithNames( - queryString: String, - args: Seq[Expression]): String = { - try { - var substituted = queryString - val paramMap = scala.collection.mutable.Map[String, Any]() - var positionalIndex = 0 - - // Build parameter map from args - args.foreach { - case alias: Alias => - // Named parameter: "value AS paramName" - val paramName = alias.name - val paramValue = alias.child.eval(null) - paramMap(paramName) = paramValue - case expr => - // Positional parameter: just a value - val paramValue = expr.eval(null) - // Handle positional parameters first - if (substituted.contains("?")) { - val sqlLiteral = formatSqlLiteral(paramValue) - substituted = substituted.replaceFirst("\\?", sqlLiteral) - } - positionalIndex += 1 - } - - // Substitute named parameters (:paramName) - paramMap.foreach { case (paramName, paramValue) => - val sqlLiteral = formatSqlLiteral(paramValue) - val pattern = s":$paramName\\b" // Use word boundary to avoid partial matches - substituted = substituted.replaceAll(pattern, sqlLiteral) - } - - substituted - } catch { - case e: Exception => - throw new RuntimeException( - s"Error in parameter substitution for query '$queryString'", e) - } - } - /** - * Builds unified parameter arrays for the new sql() API. - * Returns (values, names) where values contains all parameter values - * and names contains corresponding parameter names (or empty string for positional). + * Builds unified parameter arrays for the sql() API. */ private def buildUnifiedParameters(args: Seq[Expression]): (Array[Any], Array[String]) = { import org.apache.spark.sql.catalyst.expressions.{EmptyRow, Literal} @@ -190,12 +124,9 @@ case class ExecuteImmediateCommands(sparkSession: SparkSession) extends Rule[Log val values = scala.collection.mutable.ListBuffer[Any]() val names = scala.collection.mutable.ListBuffer[String]() - - args.foreach { case alias: Alias => - // Check if this is an auto-generated alias from variable resolution - // or an explicit "value AS paramName" from the user + // Check if this is an auto-generated alias or explicit "value AS paramName" val isAutoGeneratedAlias = alias.child match { case varRef: VariableReference => // If the alias name matches the variable name, it's auto-generated @@ -210,8 +141,7 @@ case class ExecuteImmediateCommands(sparkSession: SparkSession) extends Rule[Log case foldable if foldable.foldable => Literal.create(foldable.eval(EmptyRow), foldable.dataType).value case other => - // Expression is not foldable - this is not supported for parameters - // Check for specific unsupported expression types to provide better error messages + // Expression is not foldable - not supported for parameters import org.apache.spark.sql.catalyst.expressions.{ScalarSubquery, Exists, ListQuery, InSubquery} other match { case _: ScalarSubquery | _: Exists | _: ListQuery | _: InSubquery => @@ -225,14 +155,13 @@ case class ExecuteImmediateCommands(sparkSession: SparkSession) extends Rule[Log } if (isAutoGeneratedAlias) { - // This is a session variable without explicit AS clause - // Pass the variable name so the inner query can use it for named parameters + // Session variable without explicit AS clause val varName = alias.child.asInstanceOf[VariableReference].identifier.name() values += paramValue - names += varName // Use the variable name for named parameter binding + names += varName } else { - // This is a true named parameter: "value AS paramName" + // Named parameter: "value AS paramName" val paramName = alias.name values += paramValue names += paramName @@ -246,8 +175,7 @@ case class ExecuteImmediateCommands(sparkSession: SparkSession) extends Rule[Log case foldable if foldable.foldable => Literal.create(foldable.eval(EmptyRow), foldable.dataType).value case other => - // Expression is not foldable - this is not supported for parameters - // Check for specific unsupported expression types to provide better error messages + // Expression is not foldable - not supported for parameters import org.apache.spark.sql.catalyst.expressions.{ScalarSubquery, Exists, ListQuery, InSubquery} other match { case _: ScalarSubquery | _: Exists | _: ListQuery | _: InSubquery => @@ -260,7 +188,7 @@ case class ExecuteImmediateCommands(sparkSession: SparkSession) extends Rule[Log } } values += paramValue - names += null // null indicates unnamed expression (hole) + names += null // unnamed expression } (values.toArray, names.toArray) @@ -284,16 +212,16 @@ case class ExecuteImmediateCommands(sparkSession: SparkSession) extends Rule[Log positionalParameterPattern.findFirstIn(queryString).isDefined val queryUsesNamedParameters = namedParameterPattern.findFirstIn(queryString).isDefined - // First check: Does the query itself mix positional and named parameters? + // Check: Does the query mix positional and named parameters? if (queryUsesPositionalParameters && queryUsesNamedParameters) { throw QueryCompilationErrors.invalidQueryMixedQueryParameters() } - // Second check: If query uses ONLY named parameters, all USING expressions must have names + // If query uses only named parameters, all USING expressions must have names if (queryUsesNamedParameters && !queryUsesPositionalParameters) { val unnamedExpressions = names.zipWithIndex.collect { case (null, index) => index - case ("", index) => index // Also catch empty strings as unnamed + case ("", index) => index // empty strings are unnamed } if (unnamedExpressions.nonEmpty) { // Get the actual expressions that don't have names for error reporting @@ -302,169 +230,4 @@ case class ExecuteImmediateCommands(sparkSession: SparkSession) extends Rule[Log } } } - - private def separateParameters(args: Seq[Expression]): (Seq[Any], Map[String, Any]) = { - import org.apache.spark.sql.catalyst.expressions.{EmptyRow, Literal} - import org.apache.spark.sql.catalyst.expressions.VariableReference - val positionalParams = scala.collection.mutable.ListBuffer[Any]() - val namedParams = scala.collection.mutable.Map[String, Any]() - - // scalastyle:off println - System.err.println(s"DEBUG: separateParameters called with ${args.length} args") - args.zipWithIndex.foreach { case (arg, i) => - System.err.println(s"DEBUG: arg[$i]: ${arg.getClass.getSimpleName} = $arg") - System.err.println(s"DEBUG: arg[$i].resolved: ${arg.resolved}") - System.err.println(s"DEBUG: arg[$i].foldable: ${arg.foldable}") - } - // scalastyle:on println - - args.foreach { - case alias: Alias => - // Check if this is an auto-generated alias from variable resolution - // or an explicit "value AS paramName" from the user - val isAutoGeneratedAlias = alias.child match { - case varRef: VariableReference => - // If the alias name matches the variable name, it's auto-generated - alias.name == varRef.identifier.name() - case _ => false - } - - if (isAutoGeneratedAlias) { - // This is actually a positional parameter (session variable without AS) - val paramValue = if (alias.child.foldable) { - Literal.create(alias.child.eval(EmptyRow), alias.child.dataType).value - } else { - alias.child.eval(EmptyRow) - } - // scalastyle:off println - System.err.println( - s"DEBUG: Positional param = $paramValue (from auto-generated alias ${alias.name})") - // scalastyle:on println - positionalParams += paramValue - } else { - // This is a true named parameter: "value AS paramName" - val paramName = alias.name - val paramValue = if (alias.child.foldable) { - Literal.create(alias.child.eval(EmptyRow), alias.child.dataType).value - } else { - alias.child.eval(EmptyRow) - } - // scalastyle:off println - System.err.println(s"DEBUG: Named param '$paramName' = $paramValue") - // scalastyle:on println - namedParams(paramName) = paramValue - } - case expr => - // Positional parameter: just a value - // Evaluate the expression (should already be resolved by analyzer) - val paramValue = if (expr.foldable) { - // For foldable expressions, create a literal (similar to ConstantFolding) - Literal.create(expr.eval(EmptyRow), expr.dataType).value - } else { - // For non-foldable expressions, just evaluate - expr.eval(EmptyRow) - } - // scalastyle:off println - System.err.println( - s"DEBUG: Positional param = $paramValue (from ${expr.getClass.getSimpleName})") - // scalastyle:on println - positionalParams += paramValue - } - - (positionalParams.toSeq, namedParams.toMap) - } - - private def substituteNamedParameters( - queryString: String, namedParams: Map[String, Any]): String = { - var substituted = queryString - // Substitute named parameters (:paramName) - namedParams.foreach { case (paramName, paramValue) => - val sqlLiteral = formatSqlLiteral(paramValue) - val pattern = s":$paramName\\b" // Use word boundary to avoid partial matches - substituted = substituted.replaceAll(pattern, sqlLiteral) - } - - substituted - } - - private def formatSqlLiteral(value: Any): String = { - if (value == null) { - "NULL" - } else { - value match { - case s: String => s"'$s'" - case n: Number => n.toString - case b: Boolean => b.toString - case _ => s"'$value'" - } - } - } - - private def handleTargetVariables( - result: org.apache.spark.sql.DataFrame, - targetVariables: Seq[VariableReference]): Unit = { - // Ensure all target variables are resolved - targetVariables.foreach { variable => - if (!variable.resolved) { - throw org.apache.spark.SparkException.internalError( - s"Target variable ${variable.identifier} is not resolved") - } - } - - // Collect the results from the query - val values = result.queryExecution.executedPlan.executeCollect() - - if (values.length == 0) { - // No rows: Set all variables to null - targetVariables.foreach { variable => - setVariable(variable, null) - } - } else if (values.length > 1) { - // Multiple rows: Error - throw new org.apache.spark.SparkException( - errorClass = "ROW_SUBQUERY_TOO_MANY_ROWS", - messageParameters = Map.empty, - cause = null) - } else { - // Exactly one row: Set each variable to the corresponding column value - val row = values(0) - targetVariables.zipWithIndex.foreach { case (variable, index) => - val value = row.get(index, variable.dataType) - setVariable(variable, value) - } - } - } - - private def setVariable(variable: VariableReference, value: Any): Unit = { - import java.util.Locale - import org.apache.spark.sql.catalyst.SqlScriptingContextManager - import org.apache.spark.sql.catalyst.analysis.{FakeLocalCatalog, FakeSystemCatalog} - import org.apache.spark.sql.catalyst.catalog.VariableDefinition - import org.apache.spark.sql.catalyst.expressions.Literal - - val namePartsCaseAdjusted = if (sparkSession.sessionState.conf.caseSensitiveAnalysis) { - variable.originalNameParts - } else { - variable.originalNameParts.map(_.toLowerCase(Locale.ROOT)) - } - - // Variable should already be resolved, so we can trust its catalog information - val variableManager = variable.catalog match { - case FakeLocalCatalog => - SqlScriptingContextManager.get().map(_.getVariableManager).getOrElse( - throw org.apache.spark.SparkException.internalError( - "Variable has FakeLocalCatalog but ScriptingVariableManager is None")) - - case FakeSystemCatalog => - sparkSession.sessionState.catalogManager.tempVariableManager - - case c => - throw org.apache.spark.SparkException.internalError(s"Unexpected catalog: $c") - } - - val varDef = VariableDefinition( - variable.identifier, variable.varDef.defaultValueSQL, Literal(value, variable.dataType)) - - variableManager.set(namePartsCaseAdjusted, varDef) - } } From 65b31dd8aa85a539f330acd3a680ad64214bf84b Mon Sep 17 00:00:00 2001 From: Serge Rielau Date: Sat, 30 Aug 2025 15:51:33 -0700 Subject: [PATCH 11/32] Cleanup 2 --- .../analysis/ColumnResolutionHelper.scala | 69 +------------------ .../catalyst/analysis/executeImmediate.scala | 32 --------- 2 files changed, 1 insertion(+), 100 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ColumnResolutionHelper.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ColumnResolutionHelper.scala index b19ef981a651a..bade5f0bee9d6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ColumnResolutionHelper.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ColumnResolutionHelper.scala @@ -23,14 +23,12 @@ import scala.collection.mutable import org.apache.spark.internal.Logging import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.SqlScriptingContextManager import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.SubExprUtils.wrapOuterReference -import org.apache.spark.sql.catalyst.parser.SqlScriptingLabelContext.isForbiddenLabelOrForVariableName import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.trees.CurrentOrigin.withOrigin import org.apache.spark.sql.catalyst.trees.TreePattern._ -import org.apache.spark.sql.connector.catalog.{CatalogManager, Identifier} +import org.apache.spark.sql.connector.catalog.CatalogManager import org.apache.spark.sql.errors.{DataTypeErrorsBase, QueryCompilationErrors} import org.apache.spark.sql.internal.SQLConf @@ -235,71 +233,6 @@ trait ColumnResolutionHelper extends Logging with DataTypeErrorsBase { } } - /** - * Look up variable by nameParts. - * If in SQL Script, first check local variables, unless in EXECUTE IMMEDIATE - * (EXECUTE IMMEDIATE generated query cannot access local variables). - * if not found fall back to session variables. - * @param nameParts NameParts of the variable. - * @return Reference to the variable. - */ - def lookupVariable(nameParts: Seq[String]): Option[VariableReference] = { - - // The temp variables live in `SYSTEM.SESSION`, and the name can be qualified or not. - def maybeTempVariableName(nameParts: Seq[String]): Boolean = { - nameParts.length == 1 || { - if (nameParts.length == 2) { - nameParts.head.equalsIgnoreCase(CatalogManager.SESSION_NAMESPACE) - } else if (nameParts.length == 3) { - nameParts(0).equalsIgnoreCase(CatalogManager.SYSTEM_CATALOG_NAME) && - nameParts(1).equalsIgnoreCase(CatalogManager.SESSION_NAMESPACE) - } else { - false - } - } - } - - val namePartsCaseAdjusted = if (conf.caseSensitiveAnalysis) { - nameParts - } else { - nameParts.map(_.toLowerCase(Locale.ROOT)) - } - - SqlScriptingContextManager.get().map(_.getVariableManager) - // In EXECUTE IMMEDIATE context, only allow session variables (system.session.X), - // not local variables - .filterNot { _ => - AnalysisContext.get.isExecuteImmediate && !maybeTempVariableName(nameParts) - } - // If variable name is qualified with session. treat it as a session variable. - .filterNot(_ => - nameParts.length > 2 - || (nameParts.length == 2 && isForbiddenLabelOrForVariableName(nameParts.head))) - .flatMap(_.get(namePartsCaseAdjusted)) - .map { varDef => - VariableReference( - nameParts, - FakeLocalCatalog, - Identifier.of(Array(varDef.identifier.namespace().last), namePartsCaseAdjusted.last), - varDef) - } - .orElse( - if (maybeTempVariableName(nameParts)) { - catalogManager.tempVariableManager - .get(namePartsCaseAdjusted) - .map { varDef => - VariableReference( - nameParts, - FakeSystemCatalog, - Identifier.of(Array(CatalogManager.SESSION_NAMESPACE), namePartsCaseAdjusted.last), - varDef - )} - } else { - None - } - ) - } - // Resolves `UnresolvedAttribute` to its value. protected def resolveVariables(e: Expression): Expression = { val variableResolution = new VariableResolution(catalogManager.tempVariableManager) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala index 4d7c2d3217e38..47de85727b452 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala @@ -24,7 +24,6 @@ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.trees.TreePattern.{EXECUTE_IMMEDIATE, TreePattern} import org.apache.spark.sql.connector.catalog.CatalogManager import org.apache.spark.sql.errors.QueryCompilationErrors -import org.apache.spark.sql.types.StringType /** * Logical plan representing execute immediate query. @@ -110,38 +109,7 @@ class ResolveExecuteImmediate( } } - def resolveArguments(expressions: Seq[Expression]): Seq[Expression] = { - expressions.map { exp => - if (exp.resolved) { - exp - } else { - resolveVariable(exp) - } - } - } - - def extractQueryString(either: Either[String, UnresolvedAttribute]): String = { - either match { - case Left(v) => v - case Right(u) => - val varReference = getVariableReference(u, u.nameParts) - - if (!varReference.dataType.sameType(StringType)) { - throw QueryCompilationErrors.invalidExecuteImmediateExpressionType(varReference.dataType) - } - // Call eval with null value passed instead of a row. - // This is ok as this is variable and invoking eval should - // be independent of row value. - val varReferenceValue = varReference.eval(null) - - if (varReferenceValue == null) { - throw QueryCompilationErrors.nullSQLStringExecuteImmediate(u.name) - } - - varReferenceValue.toString - } - } override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsWithPruning(_.containsPattern(EXECUTE_IMMEDIATE), ruleId) { From b7c3aef4bb5986f791bece879fc607f30c1d5f96 Mon Sep 17 00:00:00 2001 From: Serge Rielau Date: Sat, 30 Aug 2025 17:50:23 -0700 Subject: [PATCH 12/32] re-add columnResolutionHelper --- .../analysis/ColumnResolutionHelper.scala | 69 ++++++++++++++++++- 1 file changed, 68 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ColumnResolutionHelper.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ColumnResolutionHelper.scala index bade5f0bee9d6..b19ef981a651a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ColumnResolutionHelper.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ColumnResolutionHelper.scala @@ -23,12 +23,14 @@ import scala.collection.mutable import org.apache.spark.internal.Logging import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.SqlScriptingContextManager import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.SubExprUtils.wrapOuterReference +import org.apache.spark.sql.catalyst.parser.SqlScriptingLabelContext.isForbiddenLabelOrForVariableName import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.trees.CurrentOrigin.withOrigin import org.apache.spark.sql.catalyst.trees.TreePattern._ -import org.apache.spark.sql.connector.catalog.CatalogManager +import org.apache.spark.sql.connector.catalog.{CatalogManager, Identifier} import org.apache.spark.sql.errors.{DataTypeErrorsBase, QueryCompilationErrors} import org.apache.spark.sql.internal.SQLConf @@ -233,6 +235,71 @@ trait ColumnResolutionHelper extends Logging with DataTypeErrorsBase { } } + /** + * Look up variable by nameParts. + * If in SQL Script, first check local variables, unless in EXECUTE IMMEDIATE + * (EXECUTE IMMEDIATE generated query cannot access local variables). + * if not found fall back to session variables. + * @param nameParts NameParts of the variable. + * @return Reference to the variable. + */ + def lookupVariable(nameParts: Seq[String]): Option[VariableReference] = { + + // The temp variables live in `SYSTEM.SESSION`, and the name can be qualified or not. + def maybeTempVariableName(nameParts: Seq[String]): Boolean = { + nameParts.length == 1 || { + if (nameParts.length == 2) { + nameParts.head.equalsIgnoreCase(CatalogManager.SESSION_NAMESPACE) + } else if (nameParts.length == 3) { + nameParts(0).equalsIgnoreCase(CatalogManager.SYSTEM_CATALOG_NAME) && + nameParts(1).equalsIgnoreCase(CatalogManager.SESSION_NAMESPACE) + } else { + false + } + } + } + + val namePartsCaseAdjusted = if (conf.caseSensitiveAnalysis) { + nameParts + } else { + nameParts.map(_.toLowerCase(Locale.ROOT)) + } + + SqlScriptingContextManager.get().map(_.getVariableManager) + // In EXECUTE IMMEDIATE context, only allow session variables (system.session.X), + // not local variables + .filterNot { _ => + AnalysisContext.get.isExecuteImmediate && !maybeTempVariableName(nameParts) + } + // If variable name is qualified with session. treat it as a session variable. + .filterNot(_ => + nameParts.length > 2 + || (nameParts.length == 2 && isForbiddenLabelOrForVariableName(nameParts.head))) + .flatMap(_.get(namePartsCaseAdjusted)) + .map { varDef => + VariableReference( + nameParts, + FakeLocalCatalog, + Identifier.of(Array(varDef.identifier.namespace().last), namePartsCaseAdjusted.last), + varDef) + } + .orElse( + if (maybeTempVariableName(nameParts)) { + catalogManager.tempVariableManager + .get(namePartsCaseAdjusted) + .map { varDef => + VariableReference( + nameParts, + FakeSystemCatalog, + Identifier.of(Array(CatalogManager.SESSION_NAMESPACE), namePartsCaseAdjusted.last), + varDef + )} + } else { + None + } + ) + } + // Resolves `UnresolvedAttribute` to its value. protected def resolveVariables(e: Expression): Expression = { val variableResolution = new VariableResolution(catalogManager.tempVariableManager) From ea5d87eabd3f39334c7bea5f9220a64fd8e898fa Mon Sep 17 00:00:00 2001 From: Serge Rielau Date: Sat, 30 Aug 2025 22:14:57 -0700 Subject: [PATCH 13/32] Enable SQL Scripting with EXECUTE IMMEDIATE --- .../analysis/ExecuteImmediateCommands.scala | 44 +++++++++++++++---- 1 file changed, 35 insertions(+), 9 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediateCommands.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediateCommands.scala index 2e3ebf0ec1d48..ebd6725389445 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediateCommands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediateCommands.scala @@ -18,8 +18,9 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.{AnalysisException, SparkSession} +import org.apache.spark.sql.catalyst.{SqlScriptingContextManager} import org.apache.spark.sql.catalyst.expressions.{Alias, Expression} -import org.apache.spark.sql.catalyst.plans.logical.{CompoundBody, LocalRelation, LogicalPlan} +import org.apache.spark.sql.catalyst.plans.logical.{Command, CommandResult, CompoundBody, LocalRelation, LogicalPlan} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.trees.TreePattern.EXECUTE_IMMEDIATE import org.apache.spark.sql.errors.QueryCompilationErrors @@ -47,11 +48,13 @@ case class ExecuteImmediateCommands(sparkSession: SparkSession) extends Rule[Log val parsedPlan = sparkSession.sessionState.sqlParser.parsePlan(queryString) validateQuery(queryString, parsedPlan) - // Execute the query recursively + // Execute the query recursively with isolated local variable context val result = if (cmd.args.isEmpty) { // No parameters - execute directly - AnalysisContext.withExecuteImmediateContext { - sparkSession.sql(queryString) + withIsolatedLocalVariableContext { + AnalysisContext.withExecuteImmediateContext { + sparkSession.sql(queryString) + } } } else { // For parameterized queries, build unified parameter arrays @@ -60,14 +63,25 @@ case class ExecuteImmediateCommands(sparkSession: SparkSession) extends Rule[Log // Validate parameter usage patterns validateParameterUsage(cmd.queryParam, cmd.args, paramNames.toSeq) - AnalysisContext.withExecuteImmediateContext { - sparkSession.sql(queryString, paramValues, paramNames) + withIsolatedLocalVariableContext { + AnalysisContext.withExecuteImmediateContext { + sparkSession.sql(queryString, paramValues, paramNames) + } } } - // Return the query results as a LocalRelation - val internalRows = result.queryExecution.executedPlan.executeCollect() - LocalRelation(result.queryExecution.analyzed.output, internalRows.toSeq) + // Check if the executed statement is a Command (like DECLARE, SET VARIABLE, etc.) + // Commands should not return result sets + result.queryExecution.analyzed match { + case cmd: Command => + // Commands don't produce output - return CommandResult to indicate this is a command + val executedRows = result.queryExecution.executedPlan.executeCollect() + CommandResult(cmd.output, cmd, result.queryExecution.executedPlan, executedRows.toSeq) + case _ => + // Regular queries - return the results as a LocalRelation + val internalRows = result.queryExecution.executedPlan.executeCollect() + LocalRelation(result.queryExecution.analyzed.output, internalRows.toSeq) + } } catch { case e: AnalysisException => @@ -230,4 +244,16 @@ case class ExecuteImmediateCommands(sparkSession: SparkSession) extends Rule[Log } } } + + /** + * Temporarily isolates the SQL scripting context during EXECUTE IMMEDIATE execution. + * This makes withinSqlScript() return false, ensuring that statements within EXECUTE IMMEDIATE + * are not affected by the outer SQL script context (e.g., local variables, script-specific + * errors). + */ + private def withIsolatedLocalVariableContext[A](f: => A): A = { + // Completely clear the SQL scripting context to make withinSqlScript() return false + val handle = SqlScriptingContextManager.create(null) + handle.runWith(f) + } } From 2e44eb038062e837fed187f510a9546207da073d Mon Sep 17 00:00:00 2001 From: Serge Rielau Date: Sun, 31 Aug 2025 11:00:13 -0700 Subject: [PATCH 14/32] Add another testcase --- .../catalyst/analysis/ResolveCatalogs.scala | 3 +- .../catalyst/analysis/executeImmediate.scala | 21 +++++++++ .../analysis/ExecuteImmediateCommands.scala | 4 +- .../execute-immediate.sql.out | 46 +++++++++++++++++-- .../sql-tests/inputs/execute-immediate.sql | 3 ++ .../results/execute-immediate.sql.out | 22 +++++++++ 6 files changed, 90 insertions(+), 9 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala index 851db598c4e39..6307ccd5b975d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala @@ -142,8 +142,7 @@ class ResolveCatalogs(val catalogManager: CatalogManager) } private def withinSqlScript: Boolean = - SqlScriptingContextManager.get().map(_.getVariableManager).isDefined && - !AnalysisContext.get.isExecuteImmediate + SqlScriptingContextManager.get().map(_.getVariableManager).isDefined private def assertValidSessionVariableNameParts( nameParts: Seq[String], diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala index 47de85727b452..c82b151d13e4b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala @@ -127,6 +127,9 @@ class ResolveExecuteImmediate( // Validate that USING clause expressions don't contain unsupported constructs validateUsingClauseExpressions(args) + // Validate that query parameter is foldable (constant expression) + validateQueryParameter(queryParam) + if (queryParamResolved && allArgsResolved && targetVariablesResolved) { // All resolved - transform based on whether we have target variables if (targetVariables.nonEmpty) { @@ -228,4 +231,22 @@ class ResolveExecuteImmediate( } } } + + private def validateQueryParameter(queryParam: Expression): Unit = { + import org.apache.spark.sql.catalyst.expressions.{ScalarSubquery, Exists, ListQuery, InSubquery} + + // Only check for specific unsupported constructs like subqueries + // Variable references and expressions like stringvar || 'hello' should be allowed + queryParam.foreach { + case subquery: ScalarSubquery => + throw QueryCompilationErrors.unsupportedParameterExpression(subquery) + case exists: Exists => + throw QueryCompilationErrors.unsupportedParameterExpression(exists) + case listQuery: ListQuery => + throw QueryCompilationErrors.unsupportedParameterExpression(listQuery) + case inSubquery: InSubquery => + throw QueryCompilationErrors.unsupportedParameterExpression(inSubquery) + case _ => // Other expressions including variables and concatenations are fine + } + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediateCommands.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediateCommands.scala index ebd6725389445..4a044903678eb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediateCommands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediateCommands.scala @@ -75,8 +75,8 @@ case class ExecuteImmediateCommands(sparkSession: SparkSession) extends Rule[Log result.queryExecution.analyzed match { case cmd: Command => // Commands don't produce output - return CommandResult to indicate this is a command - val executedRows = result.queryExecution.executedPlan.executeCollect() - CommandResult(cmd.output, cmd, result.queryExecution.executedPlan, executedRows.toSeq) + // For analyzer tests, we don't need the actual executed rows, just the structure + CommandResult(cmd.output, cmd, result.queryExecution.executedPlan, Seq.empty) case _ => // Regular queries - return the results as a LocalRelation val internalRows = result.queryExecution.executedPlan.executeCollect() diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/execute-immediate.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/execute-immediate.sql.out index efd18e3478553..3321a31c567c0 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/execute-immediate.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/execute-immediate.sql.out @@ -48,13 +48,22 @@ SetVariable [variablereference(system.session.sql_string=CAST(NULL AS STRING))] -- !query EXECUTE IMMEDIATE 'SET spark.sql.ansi.enabled=true' -- !query analysis -LocalRelation [key#x, value#x] +CommandResult [key#x, value#x], CommandResult [key#x, value#x] + +- SetCommand (spark.sql.ansi.enabled,Some(true)) -- !query EXECUTE IMMEDIATE 'CREATE TEMPORARY VIEW IDENTIFIER(:tblName) AS SELECT id, name FROM tbl_view' USING 'tbl_view_tmp' as tblName -- !query analysis -LocalRelation +CommandResult CommandResult + +- CreateViewCommand `tbl_view_tmp`, SELECT id, name FROM tbl_view, false, false, LocalTempView, UNSUPPORTED, true + +- Project [id#x, name#x] + +- SubqueryAlias tbl_view + +- View (`tbl_view`, [id#x, name#x, data#x]) + +- Project [cast(id#x as int) AS id#x, cast(name#x as string) AS name#x, cast(data#x as struct>) AS data#x] + +- Project [id#x, name#x, data#x] + +- SubqueryAlias tbl_view + +- LocalRelation [id#x, name#x, data#x] -- !query @@ -66,7 +75,8 @@ LocalRelation [id#x, name#x] -- !query EXECUTE IMMEDIATE 'REFRESH TABLE IDENTIFIER(:tblName)' USING 'x' as tblName -- !query analysis -LocalRelation +CommandResult CommandResult + +- RefreshTableCommand `spark_catalog`.`default`.`x` -- !query @@ -137,7 +147,10 @@ LocalRelation [id#x, name#x, data#x] -- !query EXECUTE IMMEDIATE 'INSERT INTO x VALUES(?)' USING 1 -- !query analysis -LocalRelation +CommandResult CommandResult + +- InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/x, false, CSV, [path=file:[not included in comparison]/{warehouse_dir}/x], Append, `spark_catalog`.`default`.`x`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/x), [id] + +- Project [col1#x AS id#x] + +- LocalRelation [col1#x] -- !query @@ -204,7 +217,10 @@ LocalRelation [id#x, name#x, data#x, p#x] -- !query EXECUTE IMMEDIATE 'SET VAR sql_string = ?' USING 'SELECT id from tbl_view where name = :first' -- !query analysis -LocalRelation +CommandResult CommandResult + +- SetVariable [variablereference(system.session.sql_string='SELECT * from tbl_view where name = :first or id = :second')] + +- Project [SELECT id from tbl_view where name = :first AS sql_string#x] + +- OneRowRelation -- !query @@ -809,6 +825,26 @@ org.apache.spark.sql.AnalysisException } +-- !query +EXECUTE IMMEDIATE (SELECT c FROM (VALUES(1)) AS T(c)) +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_EXPR_FOR_PARAMETER", + "sqlState" : "42K0E", + "messageParameters" : { + "invalidExprSql" : "\"scalarsubquery()\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 19, + "stopIndex" : 53, + "fragment" : "(SELECT c FROM (VALUES(1)) AS T(c))" + } ] +} + + -- !query DROP TABLE x -- !query analysis diff --git a/sql/core/src/test/resources/sql-tests/inputs/execute-immediate.sql b/sql/core/src/test/resources/sql-tests/inputs/execute-immediate.sql index 5ba3b9771d422..46da0f4509dd7 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/execute-immediate.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/execute-immediate.sql @@ -175,4 +175,7 @@ EXECUTE IMMEDIATE 'SELECT ?' USING (SELECT 1); -- named query with unnamed parameters EXECUTE IMMEDIATE 'SELECT :first' USING 2, 3; +-- Query is not a constant +EXECUTE IMMEDIATE (SELECT c FROM (VALUES(1)) AS T(c)); + DROP TABLE x; \ No newline at end of file diff --git a/sql/core/src/test/resources/sql-tests/results/execute-immediate.sql.out b/sql/core/src/test/resources/sql-tests/results/execute-immediate.sql.out index 29ba68b1f43f5..18e7ec0e19b5e 100644 --- a/sql/core/src/test/resources/sql-tests/results/execute-immediate.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/execute-immediate.sql.out @@ -927,6 +927,28 @@ org.apache.spark.sql.AnalysisException } +-- !query +EXECUTE IMMEDIATE (SELECT c FROM (VALUES(1)) AS T(c)) +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "UNSUPPORTED_EXPR_FOR_PARAMETER", + "sqlState" : "42K0E", + "messageParameters" : { + "invalidExprSql" : "\"scalarsubquery()\"" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 19, + "stopIndex" : 53, + "fragment" : "(SELECT c FROM (VALUES(1)) AS T(c))" + } ] +} + + -- !query DROP TABLE x -- !query schema From e69e214e4e566780fc9c4aea64614d3201cff212 Mon Sep 17 00:00:00 2001 From: Serge Rielau Date: Sun, 31 Aug 2025 11:29:02 -0700 Subject: [PATCH 15/32] Clean up --- .../catalyst/analysis/executeImmediate.scala | 31 +--- .../analysis/ExecuteImmediateCommands.scala | 149 ++++++++---------- 2 files changed, 65 insertions(+), 115 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala index c82b151d13e4b..da5caa1d6da7c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.analysis -import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Expression, VariableReference} +import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Exists, Expression, InSubquery, ListQuery, ScalarSubquery, VariableReference} import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.catalyst.plans.logical.{ExecutableDuringAnalysis, LocalRelation, LogicalPlan, SetVariable, UnaryNode} import org.apache.spark.sql.catalyst.rules.Rule @@ -84,33 +84,9 @@ case class ExecuteImmediateCommand( class ResolveExecuteImmediate( val catalogManager: CatalogManager, resolveChild: LogicalPlan => LogicalPlan, - checkAnalysis: LogicalPlan => Unit) - extends Rule[LogicalPlan] { + checkAnalysis: LogicalPlan => Unit) extends Rule[LogicalPlan] { private val variableResolution = new VariableResolution(catalogManager.tempVariableManager) - def resolveVariable(e: Expression): Expression = { - - /** - * We know that the expression is either UnresolvedAttribute, Alias or Parameter, as passed from - * the parser. If it is an UnresolvedAttribute, we look it up in the catalog and return it. If - * it is an Alias, we resolve the child and return an Alias with the same name. If it is - * a Parameter, we leave it as is because the parameter belongs to another parameterized - * query and should be resolved later. - */ - e match { - case u: UnresolvedAttribute => - getVariableReference(u, u.nameParts) - case a: Alias => - Alias(resolveVariable(a.child), a.name)() - case p: Parameter => p - case varRef: VariableReference => varRef // VariableReference is already resolved - case other => - throw QueryCompilationErrors.unsupportedParameterExpression(other) - } - } - - - override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsWithPruning(_.containsPattern(EXECUTE_IMMEDIATE), ruleId) { case e @ ExecuteImmediateQuery(queryParam, args, targetVariables) => @@ -215,7 +191,6 @@ class ResolveExecuteImmediate( } private def validateUsingClauseExpressions(args: Seq[Expression]): Unit = { - import org.apache.spark.sql.catalyst.expressions.{ScalarSubquery, Exists, ListQuery, InSubquery} args.foreach { expr => // Check the expression and its children for unsupported constructs expr.foreach { @@ -233,8 +208,6 @@ class ResolveExecuteImmediate( } private def validateQueryParameter(queryParam: Expression): Unit = { - import org.apache.spark.sql.catalyst.expressions.{ScalarSubquery, Exists, ListQuery, InSubquery} - // Only check for specific unsupported constructs like subqueries // Variable references and expressions like stringvar || 'hello' should be allowed queryParam.foreach { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediateCommands.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediateCommands.scala index 4a044903678eb..6eb9e03bc0926 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediateCommands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediateCommands.scala @@ -17,9 +17,9 @@ package org.apache.spark.sql.catalyst.analysis -import org.apache.spark.sql.{AnalysisException, SparkSession} +import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.{SqlScriptingContextManager} -import org.apache.spark.sql.catalyst.expressions.{Alias, Expression} +import org.apache.spark.sql.catalyst.expressions.{Alias, EmptyRow, Exists, Expression, InSubquery, ListQuery, Literal, ScalarSubquery, VariableReference} import org.apache.spark.sql.catalyst.plans.logical.{Command, CommandResult, CompoundBody, LocalRelation, LogicalPlan} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.trees.TreePattern.EXECUTE_IMMEDIATE @@ -40,55 +40,46 @@ case class ExecuteImmediateCommands(sparkSession: SparkSession) extends Rule[Log } private def executeImmediate(cmd: ExecuteImmediateCommand): LogicalPlan = { - try { - // Extract the query string from the queryParam expression - val queryString = extractQueryString(cmd.queryParam) - - // Parse and validate the query - val parsedPlan = sparkSession.sessionState.sqlParser.parsePlan(queryString) - validateQuery(queryString, parsedPlan) - - // Execute the query recursively with isolated local variable context - val result = if (cmd.args.isEmpty) { - // No parameters - execute directly - withIsolatedLocalVariableContext { - AnalysisContext.withExecuteImmediateContext { - sparkSession.sql(queryString) - } + // Extract the query string from the queryParam expression + val queryString = extractQueryString(cmd.queryParam) + + // Parse and validate the query + val parsedPlan = sparkSession.sessionState.sqlParser.parsePlan(queryString) + validateQuery(queryString, parsedPlan) + + // Execute the query recursively with isolated local variable context + val result = if (cmd.args.isEmpty) { + // No parameters - execute directly + withIsolatedLocalVariableContext { + AnalysisContext.withExecuteImmediateContext { + sparkSession.sql(queryString) } - } else { - // For parameterized queries, build unified parameter arrays - val (paramValues, paramNames) = buildUnifiedParameters(cmd.args) + } + } else { + // For parameterized queries, build unified parameter arrays + val (paramValues, paramNames) = buildUnifiedParameters(cmd.args) - // Validate parameter usage patterns - validateParameterUsage(cmd.queryParam, cmd.args, paramNames.toSeq) + // Validate parameter usage patterns + validateParameterUsage(cmd.queryParam, cmd.args, paramNames.toSeq) - withIsolatedLocalVariableContext { - AnalysisContext.withExecuteImmediateContext { - sparkSession.sql(queryString, paramValues, paramNames) - } + withIsolatedLocalVariableContext { + AnalysisContext.withExecuteImmediateContext { + sparkSession.sql(queryString, paramValues, paramNames) } } + } - // Check if the executed statement is a Command (like DECLARE, SET VARIABLE, etc.) - // Commands should not return result sets - result.queryExecution.analyzed match { - case cmd: Command => - // Commands don't produce output - return CommandResult to indicate this is a command - // For analyzer tests, we don't need the actual executed rows, just the structure - CommandResult(cmd.output, cmd, result.queryExecution.executedPlan, Seq.empty) - case _ => - // Regular queries - return the results as a LocalRelation - val internalRows = result.queryExecution.executedPlan.executeCollect() - LocalRelation(result.queryExecution.analyzed.output, internalRows.toSeq) - } - - } catch { - case e: AnalysisException => - // Re-throw AnalysisException as-is to preserve error type for tests - throw e - case e: Exception => - throw new RuntimeException(s"Failed to execute immediate query: ${e.getMessage}", e) + // Check if the executed statement is a Command (like DECLARE, SET VARIABLE, etc.) + // Commands should not return result sets + result.queryExecution.analyzed match { + case cmd: Command => + // Commands don't produce output - return CommandResult to indicate this is a command + // For analyzer tests, we don't need the actual executed rows, just the structure + CommandResult(cmd.output, cmd, result.queryExecution.executedPlan, Seq.empty) + case _ => + // Regular queries - return the results as a LocalRelation + val internalRows = result.queryExecution.executedPlan.executeCollect() + LocalRelation(result.queryExecution.analyzed.output, internalRows.toSeq) } } @@ -133,8 +124,6 @@ case class ExecuteImmediateCommands(sparkSession: SparkSession) extends Rule[Log * Builds unified parameter arrays for the sql() API. */ private def buildUnifiedParameters(args: Seq[Expression]): (Array[Any], Array[String]) = { - import org.apache.spark.sql.catalyst.expressions.{EmptyRow, Literal} - import org.apache.spark.sql.catalyst.expressions.VariableReference val values = scala.collection.mutable.ListBuffer[Any]() val names = scala.collection.mutable.ListBuffer[String]() @@ -148,30 +137,11 @@ case class ExecuteImmediateCommands(sparkSession: SparkSession) extends Rule[Log case _ => false } - val paramValue = alias.child match { - case varRef: VariableReference => - // Variable references should be evaluated to their values - varRef.eval(EmptyRow) - case foldable if foldable.foldable => - Literal.create(foldable.eval(EmptyRow), foldable.dataType).value - case other => - // Expression is not foldable - not supported for parameters - import org.apache.spark.sql.catalyst.expressions.{ScalarSubquery, Exists, ListQuery, InSubquery} - other match { - case _: ScalarSubquery | _: Exists | _: ListQuery | _: InSubquery => - throw QueryCompilationErrors.unsupportedParameterExpression(other) - case _ if !other.foldable => - throw QueryCompilationErrors.unsupportedParameterExpression(other) - case _ => - // This should not happen, but fallback to evaluation - other.eval(EmptyRow) - } - } + val paramValue = evaluateParameterExpression(alias.child) if (isAutoGeneratedAlias) { // Session variable without explicit AS clause val varName = alias.child.asInstanceOf[VariableReference].identifier.name() - values += paramValue names += varName } else { @@ -182,25 +152,7 @@ case class ExecuteImmediateCommands(sparkSession: SparkSession) extends Rule[Log } case expr => // Positional parameter: just a value - val paramValue = expr match { - case varRef: VariableReference => - // Variable references should be evaluated to their values - varRef.eval(EmptyRow) - case foldable if foldable.foldable => - Literal.create(foldable.eval(EmptyRow), foldable.dataType).value - case other => - // Expression is not foldable - not supported for parameters - import org.apache.spark.sql.catalyst.expressions.{ScalarSubquery, Exists, ListQuery, InSubquery} - other match { - case _: ScalarSubquery | _: Exists | _: ListQuery | _: InSubquery => - throw QueryCompilationErrors.unsupportedParameterExpression(other) - case _ if !other.foldable => - throw QueryCompilationErrors.unsupportedParameterExpression(other) - case _ => - // This should not happen, but fallback to evaluation - other.eval(EmptyRow) - } - } + val paramValue = evaluateParameterExpression(expr) values += paramValue names += null // unnamed expression } @@ -208,6 +160,31 @@ case class ExecuteImmediateCommands(sparkSession: SparkSession) extends Rule[Log (values.toArray, names.toArray) } + /** + * Evaluates a parameter expression, ensuring it's foldable and doesn't contain + * unsupported constructs. + */ + private def evaluateParameterExpression(expr: Expression): Any = { + expr match { + case varRef: VariableReference => + // Variable references should be evaluated to their values + varRef.eval(EmptyRow) + case foldable if foldable.foldable => + Literal.create(foldable.eval(EmptyRow), foldable.dataType).value + case other => + // Expression is not foldable - not supported for parameters + other match { + case _: ScalarSubquery | _: Exists | _: ListQuery | _: InSubquery => + throw QueryCompilationErrors.unsupportedParameterExpression(other) + case _ if !other.foldable => + throw QueryCompilationErrors.unsupportedParameterExpression(other) + case _ => + // This should not happen, but fallback to evaluation + other.eval(EmptyRow) + } + } + } + private def validateParameterUsage( queryParam: Expression, args: Seq[Expression], From 0c39ea2a9db2e5bf6d3079c72f0b8ffffc1b7eab Mon Sep 17 00:00:00 2001 From: Serge Rielau Date: Mon, 1 Sep 2025 18:24:08 -0700 Subject: [PATCH 16/32] Update sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala --- .../scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index d699a373aae61..3f5f005eb7835 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -1802,7 +1802,6 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor case s: Sort if !s.resolved || s.missingInput.nonEmpty => resolveReferencesInSort(s) - // Remove this case - let ResolveReferences handle ExecuteImmediateQuery expressions normally case d: DataFrameDropColumns if !d.resolved => resolveDataFrameDropColumns(d) From 9b67877ad58b139de993cebeb9b8f59f6a1926f5 Mon Sep 17 00:00:00 2001 From: Serge Rielau Date: Mon, 1 Sep 2025 18:33:30 -0700 Subject: [PATCH 17/32] Update sql/core/src/test/resources/sql-tests/inputs/execute-immediate.sql --- .../src/test/resources/sql-tests/inputs/execute-immediate.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/test/resources/sql-tests/inputs/execute-immediate.sql b/sql/core/src/test/resources/sql-tests/inputs/execute-immediate.sql index 46da0f4509dd7..9eaf59b6009cd 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/execute-immediate.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/execute-immediate.sql @@ -148,7 +148,7 @@ EXECUTE IMMEDIATE 'EXECUTE IMMEDIATE \'SELECT id FROM tbl_view WHERE id = ?\' US SET VAR sql_string = null; EXECUTE IMMEDIATE sql_string; --- sqlString is null +-- sqlString is not a string SET VAR sql_string = 5; EXECUTE IMMEDIATE sql_string; From d3d4511242dbd89ee86e50b66a892addc5e7d4b3 Mon Sep 17 00:00:00 2001 From: Serge Rielau Date: Mon, 1 Sep 2025 18:34:35 -0700 Subject: [PATCH 18/32] Update sql/core/src/test/resources/sql-tests/inputs/execute-immediate.sql --- .../src/test/resources/sql-tests/inputs/execute-immediate.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/test/resources/sql-tests/inputs/execute-immediate.sql b/sql/core/src/test/resources/sql-tests/inputs/execute-immediate.sql index 9eaf59b6009cd..748f05f71c64e 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/execute-immediate.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/execute-immediate.sql @@ -178,4 +178,4 @@ EXECUTE IMMEDIATE 'SELECT :first' USING 2, 3; -- Query is not a constant EXECUTE IMMEDIATE (SELECT c FROM (VALUES(1)) AS T(c)); -DROP TABLE x; \ No newline at end of file +DROP TABLE x; From c1270d6e0c4149b8794684f22372ba17277ac4ff Mon Sep 17 00:00:00 2001 From: Serge Rielau Date: Mon, 1 Sep 2025 18:35:22 -0700 Subject: [PATCH 19/32] Update sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala --- .../org/apache/spark/sql/hive/HiveSessionStateBuilder.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala index 1f593165d4e27..6a8463340383e 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala @@ -132,9 +132,9 @@ class HiveSessionStateBuilder( new EvalSubqueriesForTimeTravel +: new DetermineTableStats(session) +: new ResolveTranspose(session) +: - new InvokeProcedures(session) +: - ExecuteImmediateCommands(session) +: - ExtractSemiStructuredFields +: + new InvokeProcedures(session) +: + ExecuteImmediateCommands(session) +: + ExtractSemiStructuredFields +: customResolutionRules override val postHocResolutionRules: Seq[Rule[LogicalPlan]] = From b18c99c2a334ce88a346301ad4d94ece4c14efda Mon Sep 17 00:00:00 2001 From: Serge Rielau Date: Mon, 1 Sep 2025 18:37:34 -0700 Subject: [PATCH 20/32] Update sql/core/src/test/resources/sql-tests/inputs/execute-immediate.sql --- .../src/test/resources/sql-tests/inputs/execute-immediate.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/test/resources/sql-tests/inputs/execute-immediate.sql b/sql/core/src/test/resources/sql-tests/inputs/execute-immediate.sql index 748f05f71c64e..9f87955896a85 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/execute-immediate.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/execute-immediate.sql @@ -152,7 +152,7 @@ EXECUTE IMMEDIATE sql_string; SET VAR sql_string = 5; EXECUTE IMMEDIATE sql_string; --- sqlString is null +-- sqlString is not a well formed SQL statement. SET VAR sql_string = 'hello'; EXECUTE IMMEDIATE length(sql_string); From 04f91ab6d7c9f90cd214cd25ce4b89c2b654ae83 Mon Sep 17 00:00:00 2001 From: Serge Rielau Date: Mon, 1 Sep 2025 18:38:09 -0700 Subject: [PATCH 21/32] Update sql/core/src/test/resources/sql-tests/inputs/execute-immediate.sql --- .../src/test/resources/sql-tests/inputs/execute-immediate.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/sql/core/src/test/resources/sql-tests/inputs/execute-immediate.sql b/sql/core/src/test/resources/sql-tests/inputs/execute-immediate.sql index 9f87955896a85..78ae4cca89199 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/execute-immediate.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/execute-immediate.sql @@ -179,3 +179,4 @@ EXECUTE IMMEDIATE 'SELECT :first' USING 2, 3; EXECUTE IMMEDIATE (SELECT c FROM (VALUES(1)) AS T(c)); DROP TABLE x; + From bde8617360f6f79f6c08131fee3100350d5fb850 Mon Sep 17 00:00:00 2001 From: Serge Rielau Date: Tue, 2 Sep 2025 10:47:46 -0700 Subject: [PATCH 22/32] Address code review comments by David and Wenchen --- .../sql/catalyst/analysis/Analyzer.scala | 5 +- .../analysis/ColumnResolutionHelper.scala | 69 +------- .../catalyst/analysis/executeImmediate.scala | 147 +++++------------- .../analysis/ExecuteImmediateCommands.scala | 14 +- .../execute-immediate.sql.out | 49 +++--- .../results/execute-immediate.sql.out | 49 +++--- 6 files changed, 102 insertions(+), 231 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 3f5f005eb7835..85c587b7a2d0e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -444,10 +444,7 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor AddMetadataColumns :: DeduplicateRelations :: ResolveCollationName :: - new ResolveExecuteImmediate( - catalogManager, - resolveChild = (plan: LogicalPlan) => plan, - checkAnalysis = (_: LogicalPlan) => ()) :: + new ResolveExecuteImmediate(catalogManager):: ResolveMergeIntoSchemaEvolution :: new ResolveReferences(catalogManager) :: // Please do not insert any other rules in between. See the TODO comments in rule diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ColumnResolutionHelper.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ColumnResolutionHelper.scala index b19ef981a651a..bade5f0bee9d6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ColumnResolutionHelper.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ColumnResolutionHelper.scala @@ -23,14 +23,12 @@ import scala.collection.mutable import org.apache.spark.internal.Logging import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.SqlScriptingContextManager import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.SubExprUtils.wrapOuterReference -import org.apache.spark.sql.catalyst.parser.SqlScriptingLabelContext.isForbiddenLabelOrForVariableName import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.trees.CurrentOrigin.withOrigin import org.apache.spark.sql.catalyst.trees.TreePattern._ -import org.apache.spark.sql.connector.catalog.{CatalogManager, Identifier} +import org.apache.spark.sql.connector.catalog.CatalogManager import org.apache.spark.sql.errors.{DataTypeErrorsBase, QueryCompilationErrors} import org.apache.spark.sql.internal.SQLConf @@ -235,71 +233,6 @@ trait ColumnResolutionHelper extends Logging with DataTypeErrorsBase { } } - /** - * Look up variable by nameParts. - * If in SQL Script, first check local variables, unless in EXECUTE IMMEDIATE - * (EXECUTE IMMEDIATE generated query cannot access local variables). - * if not found fall back to session variables. - * @param nameParts NameParts of the variable. - * @return Reference to the variable. - */ - def lookupVariable(nameParts: Seq[String]): Option[VariableReference] = { - - // The temp variables live in `SYSTEM.SESSION`, and the name can be qualified or not. - def maybeTempVariableName(nameParts: Seq[String]): Boolean = { - nameParts.length == 1 || { - if (nameParts.length == 2) { - nameParts.head.equalsIgnoreCase(CatalogManager.SESSION_NAMESPACE) - } else if (nameParts.length == 3) { - nameParts(0).equalsIgnoreCase(CatalogManager.SYSTEM_CATALOG_NAME) && - nameParts(1).equalsIgnoreCase(CatalogManager.SESSION_NAMESPACE) - } else { - false - } - } - } - - val namePartsCaseAdjusted = if (conf.caseSensitiveAnalysis) { - nameParts - } else { - nameParts.map(_.toLowerCase(Locale.ROOT)) - } - - SqlScriptingContextManager.get().map(_.getVariableManager) - // In EXECUTE IMMEDIATE context, only allow session variables (system.session.X), - // not local variables - .filterNot { _ => - AnalysisContext.get.isExecuteImmediate && !maybeTempVariableName(nameParts) - } - // If variable name is qualified with session. treat it as a session variable. - .filterNot(_ => - nameParts.length > 2 - || (nameParts.length == 2 && isForbiddenLabelOrForVariableName(nameParts.head))) - .flatMap(_.get(namePartsCaseAdjusted)) - .map { varDef => - VariableReference( - nameParts, - FakeLocalCatalog, - Identifier.of(Array(varDef.identifier.namespace().last), namePartsCaseAdjusted.last), - varDef) - } - .orElse( - if (maybeTempVariableName(nameParts)) { - catalogManager.tempVariableManager - .get(namePartsCaseAdjusted) - .map { varDef => - VariableReference( - nameParts, - FakeSystemCatalog, - Identifier.of(Array(CatalogManager.SESSION_NAMESPACE), namePartsCaseAdjusted.last), - varDef - )} - } else { - None - } - ) - } - // Resolves `UnresolvedAttribute` to its value. protected def resolveVariables(e: Expression): Expression = { val variableResolution = new VariableResolution(catalogManager.tempVariableManager) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala index da5caa1d6da7c..b63110ac57005 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala @@ -18,7 +18,6 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Exists, Expression, InSubquery, ListQuery, ScalarSubquery, VariableReference} -import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.catalyst.plans.logical.{ExecutableDuringAnalysis, LocalRelation, LogicalPlan, SetVariable, UnaryNode} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.trees.TreePattern.{EXECUTE_IMMEDIATE, TreePattern} @@ -28,12 +27,12 @@ import org.apache.spark.sql.errors.QueryCompilationErrors /** * Logical plan representing execute immediate query. * - * @param queryParam the query expression (first child) + * @param sqlStmtStr the query expression (first child) * @param args parameters from USING clause (subsequent children) * @param targetVariables variables to store the result of the query */ case class ExecuteImmediateQuery( - queryParam: Expression, + sqlStmtStr: Expression, args: Seq[Expression], targetVariables: Seq[Expression]) extends UnresolvedLeafNode { @@ -45,11 +44,11 @@ case class ExecuteImmediateQuery( * Logical plan representing a resolved execute immediate command that will recursively * invoke SQL execution. * - * @param queryParam the resolved query expression + * @param sqlStmtStr the resolved query expression * @param args parameters from USING clause */ case class ExecuteImmediateCommand( - queryParam: Expression, + sqlStmtStr: Expression, args: Seq[Expression]) extends UnaryNode with ExecutableDuringAnalysis { @@ -82,113 +81,53 @@ case class ExecuteImmediateCommand( * that will handle recursive SQL execution. */ class ResolveExecuteImmediate( - val catalogManager: CatalogManager, - resolveChild: LogicalPlan => LogicalPlan, - checkAnalysis: LogicalPlan => Unit) extends Rule[LogicalPlan] { - private val variableResolution = new VariableResolution(catalogManager.tempVariableManager) - + val catalogManager: CatalogManager) extends Rule[LogicalPlan] { override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsWithPruning(_.containsPattern(EXECUTE_IMMEDIATE), ruleId) { - case e @ ExecuteImmediateQuery(queryParam, args, targetVariables) => - // Check if all expressions are resolved (they should be resolved by ResolveReferences now) - val queryParamResolved = queryParam.resolved - val allArgsResolved = args.forall(_.resolved) - val targetVariablesResolved = targetVariables.forall { - case _: UnresolvedAttribute => false // Unresolved attributes are not resolved - case alias: Alias => alias.child.resolved // For aliases, check if child is resolved - case _: VariableReference => true // VariableReference is already resolved - case expr => expr.resolved // For other expressions, use standard resolved check - } - + case ExecuteImmediateQuery(sqlStmtStr, args, targetVariables) + if sqlStmtStr.resolved && targetVariables.forall(_.resolved) && args.forall(_.resolved) => // Validate that USING clause expressions don't contain unsupported constructs validateUsingClauseExpressions(args) // Validate that query parameter is foldable (constant expression) - validateQueryParameter(queryParam) - - if (queryParamResolved && allArgsResolved && targetVariablesResolved) { - // All resolved - transform based on whether we have target variables - if (targetVariables.nonEmpty) { - // EXECUTE IMMEDIATE ... INTO should generate SetVariable plan - // SetVariable expects UnresolvedAttribute objects that ResolveSetVariable will resolve - val finalTargetVars = targetVariables.map { - case attr: UnresolvedAttribute => - // Keep as UnresolvedAttribute for ResolveSetVariable to handle - attr - case alias: Alias => - // Extract the UnresolvedAttribute from the alias - alias.child match { - case attr: UnresolvedAttribute => - attr - case varRef: VariableReference => - // Convert back to UnresolvedAttribute for ResolveSetVariable - UnresolvedAttribute(varRef.originalNameParts) - case _ => - throw QueryCompilationErrors.unsupportedParameterExpression(alias.child) - } - case varRef: VariableReference => - // Convert back to UnresolvedAttribute for ResolveSetVariable - UnresolvedAttribute(varRef.originalNameParts) - case other => - throw QueryCompilationErrors.unsupportedParameterExpression(other) - } - - // Validate that the query is suitable for INTO clause - validateQueryForInto(queryParam) - // Create SetVariable plan with the execute immediate query as source - val sourceQuery = ExecuteImmediateCommand(queryParam, args) - SetVariable(finalTargetVars, sourceQuery) - } else { - // Regular EXECUTE IMMEDIATE without INTO - ExecuteImmediateCommand(queryParam, args) + validateSqlStmt(sqlStmtStr) + + // All resolved - transform based on whether we have target variables + if (targetVariables.nonEmpty) { + // EXECUTE IMMEDIATE ... INTO should generate SetVariable plan + // SetVariable expects UnresolvedAttribute objects that ResolveSetVariable will resolve + val finalTargetVars = targetVariables.map { + case attr: UnresolvedAttribute => + // Keep as UnresolvedAttribute for ResolveSetVariable to handle + attr + case alias: Alias => + // Extract the UnresolvedAttribute from the alias + alias.child match { + case attr: UnresolvedAttribute => + attr + case varRef: VariableReference => + // Convert back to UnresolvedAttribute for ResolveSetVariable + UnresolvedAttribute(varRef.originalNameParts) + case _ => + throw QueryCompilationErrors.unsupportedParameterExpression(alias.child) + } + case varRef: VariableReference => + // Convert back to UnresolvedAttribute for ResolveSetVariable + UnresolvedAttribute(varRef.originalNameParts) + case other => + throw QueryCompilationErrors.unsupportedParameterExpression(other) } - } else { - // Not all resolved yet - wait for next iteration - e - } - } - private def getVariableReference(expr: Expression, nameParts: Seq[String]): VariableReference = { - variableResolution.lookupVariable( - nameParts = nameParts, - resolvingExecuteImmediate = AnalysisContext.get.isExecuteImmediate - ) match { - case Some(variable) => variable - case _ => - throw QueryCompilationErrors - .unresolvedVariableError( - nameParts, - Seq(CatalogManager.SYSTEM_CATALOG_NAME, CatalogManager.SESSION_NAMESPACE), - expr.origin) - } - } - - private def validateQueryForInto(queryParam: Expression): Unit = { - // Extract the query string to validate - val queryString = queryParam.eval(null) match { - case null => return // Will be caught later by other validation - case value => value.toString - } - - // If targetVariables is defined, statement needs to be a query. - // Try to parse as query first, then as general plan - try { - catalogManager.v1SessionCatalog.parser.parseQuery(queryString) - // Success - it's a valid query, proceed - } catch { - case e: ParseException => - // parseQuery failed, try parsePlan to see if it's valid SQL but not a query - try { - catalogManager.v1SessionCatalog.parser.parsePlan(queryString) - // Plan was successfully parsed, but query wasn't - throw error - throw QueryCompilationErrors.invalidStatementForExecuteInto(queryString) - } catch { - case _: ParseException => - // Both failed - let the original parse error propagate - throw e + // Create SetVariable plan with the execute immediate query as source + val sourceQuery = ExecuteImmediateCommand(sqlStmtStr, args) + SetVariable(finalTargetVars, sourceQuery) + } else { + // Regular EXECUTE IMMEDIATE without INTO + ExecuteImmediateCommand(sqlStmtStr, args) } + case other => other + // Not all resolved yet - wait for next iteration } - } private def validateUsingClauseExpressions(args: Seq[Expression]): Unit = { args.foreach { expr => @@ -207,10 +146,10 @@ class ResolveExecuteImmediate( } } - private def validateQueryParameter(queryParam: Expression): Unit = { + private def validateSqlStmt(sqlStmtStr: Expression): Unit = { // Only check for specific unsupported constructs like subqueries // Variable references and expressions like stringvar || 'hello' should be allowed - queryParam.foreach { + sqlStmtStr.foreach { case subquery: ScalarSubquery => throw QueryCompilationErrors.unsupportedParameterExpression(subquery) case exists: Exists => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediateCommands.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediateCommands.scala index 6eb9e03bc0926..ec80620065cdd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediateCommands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediateCommands.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.{SqlScriptingContextManager} +import org.apache.spark.sql.catalyst.SqlScriptingContextManager import org.apache.spark.sql.catalyst.expressions.{Alias, EmptyRow, Exists, Expression, InSubquery, ListQuery, Literal, ScalarSubquery, VariableReference} import org.apache.spark.sql.catalyst.plans.logical.{Command, CommandResult, CompoundBody, LocalRelation, LogicalPlan} import org.apache.spark.sql.catalyst.rules.Rule @@ -41,18 +41,18 @@ case class ExecuteImmediateCommands(sparkSession: SparkSession) extends Rule[Log private def executeImmediate(cmd: ExecuteImmediateCommand): LogicalPlan = { // Extract the query string from the queryParam expression - val queryString = extractQueryString(cmd.queryParam) + val sqlStmtStr = extractQueryString(cmd.sqlStmtStr) // Parse and validate the query - val parsedPlan = sparkSession.sessionState.sqlParser.parsePlan(queryString) - validateQuery(queryString, parsedPlan) + val parsedPlan = sparkSession.sessionState.sqlParser.parsePlan(sqlStmtStr) + validateQuery(sqlStmtStr, parsedPlan) // Execute the query recursively with isolated local variable context val result = if (cmd.args.isEmpty) { // No parameters - execute directly withIsolatedLocalVariableContext { AnalysisContext.withExecuteImmediateContext { - sparkSession.sql(queryString) + sparkSession.sql(sqlStmtStr) } } } else { @@ -60,11 +60,11 @@ case class ExecuteImmediateCommands(sparkSession: SparkSession) extends Rule[Log val (paramValues, paramNames) = buildUnifiedParameters(cmd.args) // Validate parameter usage patterns - validateParameterUsage(cmd.queryParam, cmd.args, paramNames.toSeq) + validateParameterUsage(cmd.sqlStmtStr, cmd.args, paramNames.toSeq) withIsolatedLocalVariableContext { AnalysisContext.withExecuteImmediateContext { - sparkSession.sql(queryString, paramValues, paramNames) + sparkSession.sql(sqlStmtStr, paramValues, paramNames) } } } diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/execute-immediate.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/execute-immediate.sql.out index 3321a31c567c0..82bbc43247580 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/execute-immediate.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/execute-immediate.sql.out @@ -364,10 +364,11 @@ EXECUTE IMMEDIATE 'INSERT INTO x VALUES (?)' INTO res_id USING 1 -- !query analysis org.apache.spark.sql.AnalysisException { - "errorClass" : "INVALID_STATEMENT_FOR_EXECUTE_INTO", - "sqlState" : "07501", + "errorClass" : "ASSIGNMENT_ARITY_MISMATCH", + "sqlState" : "42802", "messageParameters" : { - "sqlString" : "INSERT INTO X VALUES (?)" + "numExpr" : "0", + "numTarget" : "1" } } @@ -495,19 +496,19 @@ LocalRelation [id#x, name#x, data#x] -- !query EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = :first' USING (SELECT 42) as first, 'name2' as second -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.SparkException { - "errorClass" : "UNSUPPORTED_EXPR_FOR_PARAMETER", - "sqlState" : "42K0E", + "errorClass" : "INTERNAL_ERROR", + "sqlState" : "XX000", "messageParameters" : { - "invalidExprSql" : "\"scalarsubquery()\"" + "message" : "Found the unresolved operator: 'Project [unresolvedalias(42)]" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 70, - "stopIndex" : 80, - "fragment" : "(SELECT 42)" + "startIndex" : 71, + "stopIndex" : 79, + "fragment" : "SELECT 42" } ] } @@ -795,19 +796,19 @@ org.apache.spark.sql.AnalysisException -- !query EXECUTE IMMEDIATE 'SELECT ?' USING (SELECT 1) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.SparkException { - "errorClass" : "UNSUPPORTED_EXPR_FOR_PARAMETER", - "sqlState" : "42K0E", + "errorClass" : "INTERNAL_ERROR", + "sqlState" : "XX000", "messageParameters" : { - "invalidExprSql" : "\"scalarsubquery()\"" + "message" : "Found the unresolved operator: 'Project [unresolvedalias(1)]" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 36, - "stopIndex" : 45, - "fragment" : "(SELECT 1)" + "startIndex" : 37, + "stopIndex" : 44, + "fragment" : "SELECT 1" } ] } @@ -828,19 +829,19 @@ org.apache.spark.sql.AnalysisException -- !query EXECUTE IMMEDIATE (SELECT c FROM (VALUES(1)) AS T(c)) -- !query analysis -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "UNSUPPORTED_EXPR_FOR_PARAMETER", - "sqlState" : "42K0E", + "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + "sqlState" : "42703", "messageParameters" : { - "invalidExprSql" : "\"scalarsubquery()\"" + "objectName" : "`c`" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 19, - "stopIndex" : 53, - "fragment" : "(SELECT c FROM (VALUES(1)) AS T(c))" + "startIndex" : 27, + "stopIndex" : 27, + "fragment" : "c" } ] } diff --git a/sql/core/src/test/resources/sql-tests/results/execute-immediate.sql.out b/sql/core/src/test/resources/sql-tests/results/execute-immediate.sql.out index 18e7ec0e19b5e..2cc746a10ca4b 100644 --- a/sql/core/src/test/resources/sql-tests/results/execute-immediate.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/execute-immediate.sql.out @@ -414,10 +414,11 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "INVALID_STATEMENT_FOR_EXECUTE_INTO", - "sqlState" : "07501", + "errorClass" : "ASSIGNMENT_ARITY_MISMATCH", + "sqlState" : "42802", "messageParameters" : { - "sqlString" : "INSERT INTO X VALUES (?)" + "numExpr" : "0", + "numTarget" : "1" } } @@ -561,19 +562,19 @@ EXECUTE IMMEDIATE 'SELECT * from tbl_view where name = :first' USING (SELECT 42) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.SparkException { - "errorClass" : "UNSUPPORTED_EXPR_FOR_PARAMETER", - "sqlState" : "42K0E", + "errorClass" : "INTERNAL_ERROR", + "sqlState" : "XX000", "messageParameters" : { - "invalidExprSql" : "\"scalarsubquery()\"" + "message" : "Found the unresolved operator: 'Project [unresolvedalias(42)]" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 70, - "stopIndex" : 80, - "fragment" : "(SELECT 42)" + "startIndex" : 71, + "stopIndex" : 79, + "fragment" : "SELECT 42" } ] } @@ -895,19 +896,19 @@ EXECUTE IMMEDIATE 'SELECT ?' USING (SELECT 1) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.SparkException { - "errorClass" : "UNSUPPORTED_EXPR_FOR_PARAMETER", - "sqlState" : "42K0E", + "errorClass" : "INTERNAL_ERROR", + "sqlState" : "XX000", "messageParameters" : { - "invalidExprSql" : "\"scalarsubquery()\"" + "message" : "Found the unresolved operator: 'Project [unresolvedalias(1)]" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 36, - "stopIndex" : 45, - "fragment" : "(SELECT 1)" + "startIndex" : 37, + "stopIndex" : 44, + "fragment" : "SELECT 1" } ] } @@ -932,19 +933,19 @@ EXECUTE IMMEDIATE (SELECT c FROM (VALUES(1)) AS T(c)) -- !query schema struct<> -- !query output -org.apache.spark.sql.AnalysisException +org.apache.spark.sql.catalyst.ExtendedAnalysisException { - "errorClass" : "UNSUPPORTED_EXPR_FOR_PARAMETER", - "sqlState" : "42K0E", + "errorClass" : "UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", + "sqlState" : "42703", "messageParameters" : { - "invalidExprSql" : "\"scalarsubquery()\"" + "objectName" : "`c`" }, "queryContext" : [ { "objectType" : "", "objectName" : "", - "startIndex" : 19, - "stopIndex" : 53, - "fragment" : "(SELECT c FROM (VALUES(1)) AS T(c))" + "startIndex" : 27, + "stopIndex" : 27, + "fragment" : "c" } ] } From a7ab0d47875cdbf5848a0114a8e3d960d29e6257 Mon Sep 17 00:00:00 2001 From: Serge Rielau Date: Tue, 2 Sep 2025 14:25:30 -0700 Subject: [PATCH 23/32] More code review comments --- common/utils/src/main/resources/error/error-conditions.json | 2 +- .../spark/sql/catalyst/analysis/executeImmediate.scala | 4 ++-- .../org/apache/spark/sql/catalyst/analysis/parameters.scala | 2 +- .../org/apache/spark/sql/catalyst/parser/AstBuilder.scala | 4 ++-- .../apache/spark/sql/errors/QueryCompilationErrors.scala | 2 +- .../spark/sql/catalyst/analysis/AnalysisErrorSuite.scala | 2 +- .../sql-tests/analyzer-results/execute-immediate.sql.out | 6 +++--- .../resources/sql-tests/results/execute-immediate.sql.out | 6 +++--- 8 files changed, 14 insertions(+), 14 deletions(-) diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 37e0aaeda6125..2f7a8bced7832 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -3862,7 +3862,7 @@ ], "sqlState" : "42604" }, - "INVALID_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE" : { + "INVALID_EXPR_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE" : { "message" : [ "Expression type must be string type but got ." ], diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala index b63110ac57005..f5fd11b375059 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala @@ -31,7 +31,7 @@ import org.apache.spark.sql.errors.QueryCompilationErrors * @param args parameters from USING clause (subsequent children) * @param targetVariables variables to store the result of the query */ -case class ExecuteImmediateQuery( +case class UnresolvedExecuteImmediate( sqlStmtStr: Expression, args: Seq[Expression], targetVariables: Seq[Expression]) @@ -84,7 +84,7 @@ class ResolveExecuteImmediate( val catalogManager: CatalogManager) extends Rule[LogicalPlan] { override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsWithPruning(_.containsPattern(EXECUTE_IMMEDIATE), ruleId) { - case ExecuteImmediateQuery(sqlStmtStr, args, targetVariables) + case UnresolvedExecuteImmediate(sqlStmtStr, args, targetVariables) if sqlStmtStr.resolved && targetVariables.forall(_.resolved) && args.forall(_.resolved) => // Validate that USING clause expressions don't contain unsupported constructs validateUsingClauseExpressions(args) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/parameters.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/parameters.scala index b3ffb97032673..9c20306ff80a5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/parameters.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/parameters.scala @@ -165,7 +165,7 @@ object MoveParameterizedQueriesDown extends Rule[LogicalPlan] { } /** - * Finds all named parameters in `ParameterizedQuery` and substitutes them by literals or + * Binds all named parameters in `ParameterizedQuery` and substitutes them by literals or * by collection constructor functions such as `map()`, `array()`, `struct()` * from the user-specified arguments. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 5872119b0e9e1..b42cba86e0fb9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -1148,7 +1148,7 @@ class AstBuilder extends DataTypeAstBuilder } /** - * Returns the parameters for [[ExecuteImmediateQuery]] logical plan. + * Returns the parameters for [[UnresolvedExecuteImmediate]] logical plan. * Expected format: * {{{ * EXECUTE IMMEDIATE {query_string|string_literal} @@ -1165,7 +1165,7 @@ class AstBuilder extends DataTypeAstBuilder visitExecuteImmediateUsing(_) }.getOrElse{ Seq.empty } - ExecuteImmediateQuery(queryParam, exprs, targetVars) + UnresolvedExecuteImmediate(queryParam, exprs, targetVars) } override def visitExecuteImmediateUsing( diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala index c298229e80f1a..0efb61c133f76 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala @@ -4133,7 +4133,7 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat def invalidExecuteImmediateExpressionType(dataType: DataType): Throwable = { throw new AnalysisException( - errorClass = "INVALID_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE", + errorClass = "INVALID_EXPR_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE", messageParameters = Map("exprType" -> toSQLType(dataType))) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala index 9fee25aded08d..c9892742722e1 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala @@ -839,7 +839,7 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { test("EXEC IMMEDIATE - INTO specified for COMMAND query") { - val execImmediateSetVariablePlan = ExecuteImmediateQuery( + val execImmediateSetVariablePlan = UnresolvedExecuteImmediate( Literal("SET VAR testVarA = 1"), Seq.empty, Seq(UnresolvedAttribute("testVarA"))) diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/execute-immediate.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/execute-immediate.sql.out index 82bbc43247580..da14427837080 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/execute-immediate.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/execute-immediate.sql.out @@ -463,7 +463,7 @@ EXECUTE IMMEDIATE b -- !query analysis org.apache.spark.sql.AnalysisException { - "errorClass" : "INVALID_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE", + "errorClass" : "INVALID_EXPR_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE", "sqlState" : "42K09", "messageParameters" : { "exprType" : "\"INT\"" @@ -718,7 +718,7 @@ EXECUTE IMMEDIATE length(sql_string) -- !query analysis org.apache.spark.sql.AnalysisException { - "errorClass" : "INVALID_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE", + "errorClass" : "INVALID_EXPR_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE", "sqlState" : "42K09", "messageParameters" : { "exprType" : "\"INT\"" @@ -756,7 +756,7 @@ EXECUTE IMMEDIATE int_var -- !query analysis org.apache.spark.sql.AnalysisException { - "errorClass" : "INVALID_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE", + "errorClass" : "INVALID_EXPR_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE", "sqlState" : "42K09", "messageParameters" : { "exprType" : "\"INT\"" diff --git a/sql/core/src/test/resources/sql-tests/results/execute-immediate.sql.out b/sql/core/src/test/resources/sql-tests/results/execute-immediate.sql.out index 2cc746a10ca4b..d35b11e3f0bf0 100644 --- a/sql/core/src/test/resources/sql-tests/results/execute-immediate.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/execute-immediate.sql.out @@ -525,7 +525,7 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "INVALID_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE", + "errorClass" : "INVALID_EXPR_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE", "sqlState" : "42K09", "messageParameters" : { "exprType" : "\"INT\"" @@ -809,7 +809,7 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "INVALID_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE", + "errorClass" : "INVALID_EXPR_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE", "sqlState" : "42K09", "messageParameters" : { "exprType" : "\"INT\"" @@ -852,7 +852,7 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "INVALID_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE", + "errorClass" : "INVALID_EXPR_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE", "sqlState" : "42K09", "messageParameters" : { "exprType" : "\"INT\"" From bf1e7c83806e74d779a6d35c44f1a538889df79d Mon Sep 17 00:00:00 2001 From: Serge Rielau Date: Tue, 2 Sep 2025 20:12:11 -0700 Subject: [PATCH 24/32] Fox error-condiditions --- .../src/main/resources/error/error-conditions.json | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 2f7a8bced7832..044c8662dd23d 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -2737,6 +2737,12 @@ ], "sqlState" : "42001" }, + "INVALID_EXPR_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE" : { + "message" : [ + "Expression type must be string type but got ." + ], + "sqlState" : "42K09" + }, "INVALID_EXTERNAL_TYPE" : { "message" : [ "The external type is not valid for the type at the expression ." @@ -3862,12 +3868,6 @@ ], "sqlState" : "42604" }, - "INVALID_EXPR_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE" : { - "message" : [ - "Expression type must be string type but got ." - ], - "sqlState" : "42K09" - }, "INVALID_UDF_IMPLEMENTATION" : { "message" : [ "Function does not implement a ScalarFunction or AggregateFunction." From 928c5d9dda4a7273c081efe2de3d673894ef8368 Mon Sep 17 00:00:00 2001 From: Serge Rielau Date: Thu, 4 Sep 2025 09:57:38 -0700 Subject: [PATCH 25/32] Code cleanup --- .../sql/catalyst/analysis/Analyzer.scala | 12 -- .../analysis/ColumnResolutionHelper.scala | 1 - .../analysis/ResolveSetVariable.scala | 3 +- .../analysis/VariableResolution.scala | 22 +--- .../catalyst/analysis/executeImmediate.scala | 104 +++++++++++------- .../sql/errors/QueryCompilationErrors.scala | 6 - .../analysis/ExecuteImmediateCommands.scala | 24 +--- .../sql/execution/command/SetCommand.scala | 5 +- .../execute-immediate.sql.out | 7 +- .../results/execute-immediate.sql.out | 7 +- 10 files changed, 83 insertions(+), 108 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 85c587b7a2d0e..a77732c841dd5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -135,9 +135,6 @@ object FakeV2SessionCatalog extends TableCatalog with FunctionCatalog with Suppo * if `t` was a permanent table when the current view was created, it * should still be a permanent table when resolving the current view, * even if a temp view `t` has been created. - * @param isExecuteImmediate Whether the current plan is created by EXECUTE IMMEDIATE. Used when - * resolving variables, as SQL Scripting local variables should not be - * visible from EXECUTE IMMEDIATE. * @param outerPlan The query plan from the outer query that can be used to resolve star * expressions in a subquery. */ @@ -155,7 +152,6 @@ case class AnalysisContext( referredTempFunctionNames: mutable.Set[String] = mutable.Set.empty, referredTempVariableNames: Seq[Seq[String]] = Seq.empty, outerPlan: Option[LogicalPlan] = None, - isExecuteImmediate: Boolean = false, collation: Option[String] = None, /** @@ -212,19 +208,11 @@ object AnalysisContext { viewDesc.viewReferredTempViewNames, mutable.Set(viewDesc.viewReferredTempFunctionNames: _*), viewDesc.viewReferredTempVariableNames, - isExecuteImmediate = originContext.isExecuteImmediate, collation = viewDesc.collation) set(context) try f finally { set(originContext) } } - def withExecuteImmediateContext[A](f: => A): A = { - val originContext = value.get() - val context = originContext.copy(isExecuteImmediate = true) - - set(context) - try f finally { set(originContext) } - } def withNewAnalysisContext[A](f: => A): A = { val originContext = value.get() diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ColumnResolutionHelper.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ColumnResolutionHelper.scala index bade5f0bee9d6..3224ccafafec3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ColumnResolutionHelper.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ColumnResolutionHelper.scala @@ -241,7 +241,6 @@ trait ColumnResolutionHelper extends Logging with DataTypeErrorsBase { variableResolution.resolveMultipartName( nameParts = nameParts, resolvingView = AnalysisContext.get.catalogAndNamespace.nonEmpty, - resolvingExecuteImmediate = AnalysisContext.get.isExecuteImmediate, referredTempVariableNames = AnalysisContext.get.referredTempVariableNames ).map(e => Alias(e, nameParts.last)()) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSetVariable.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSetVariable.scala index ab44084357670..8247454188c6b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSetVariable.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSetVariable.scala @@ -42,8 +42,7 @@ class ResolveSetVariable(val catalogManager: CatalogManager) extends Rule[Logica val resolvedVars = setVariable.targetVariables.map { case u: UnresolvedAttribute => variableResolution.lookupVariable( - nameParts = u.nameParts, - resolvingExecuteImmediate = AnalysisContext.get.isExecuteImmediate + nameParts = u.nameParts ) match { case Some(variable) => variable.copy(canFold = false) case _ => throw unresolvedVariableError(u.nameParts, Seq("SYSTEM", "SESSION")) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/VariableResolution.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/VariableResolution.scala index 72af7c619a083..2a1f35b0859d3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/VariableResolution.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/VariableResolution.scala @@ -52,9 +52,6 @@ class VariableResolution(tempVariableManager: TempVariableManager) extends SQLCo * (e.g., ["catalog", "schema", "variable", "field1", "field2"]) * @param resolvingView Whether this resolution is happening within a view context. * When true, only variables explicitly referred to in the view definition are accessible. - * @param resolvingExecuteImmediate Whether this resolution is happening within an - * EXECUTE IMMEDIATE context. When true, local variables are not accessible, only session - * variables. * @param referredTempVariableNames When resolving within a view, this contains the list of * variable names that the view explicitly references and should have access to. * @@ -65,7 +62,6 @@ class VariableResolution(tempVariableManager: TempVariableManager) extends SQLCo def resolveMultipartName( nameParts: Seq[String], resolvingView: Boolean, - resolvingExecuteImmediate: Boolean, referredTempVariableNames: Seq[Seq[String]]): Option[Expression] = { var resolvedVariable: Option[Expression] = None // We only support temp variables for now, so the variable name can at most have 3 parts. @@ -76,7 +72,6 @@ class VariableResolution(tempVariableManager: TempVariableManager) extends SQLCo resolvedVariable = resolveVariable( nameParts = nameParts.dropRight(numInnerFields), resolvingView = resolvingView, - resolvingExecuteImmediate = resolvingExecuteImmediate, referredTempVariableNames = referredTempVariableNames ) @@ -99,16 +94,12 @@ class VariableResolution(tempVariableManager: TempVariableManager) extends SQLCo /** * Look up variable by nameParts. - * If in SQL Script, first check local variables, unless in EXECUTE IMMEDIATE - * (EXECUTE IMMEDIATE generated query cannot access local variables). - * if not found fall back to session variables. + * If in SQL Script, first check local variables. + * If not found fall back to session variables. * @param nameParts NameParts of the variable. - * @param resolvingExecuteImmediate Whether the current context is in EXECUTE IMMEDIATE. * @return Reference to the variable. */ - def lookupVariable( - nameParts: Seq[String], - resolvingExecuteImmediate: Boolean): Option[VariableReference] = { + def lookupVariable(nameParts: Seq[String]): Option[VariableReference] = { val namePartsCaseAdjusted = if (conf.caseSensitiveAnalysis) { nameParts } else { @@ -118,8 +109,6 @@ class VariableResolution(tempVariableManager: TempVariableManager) extends SQLCo SqlScriptingContextManager .get() .map(_.getVariableManager) - // If we are in EXECUTE IMMEDIATE lookup only session variables. - .filterNot(_ => resolvingExecuteImmediate) // If variable name is qualified with session. treat it as a session variable. .filterNot( _ => @@ -156,16 +145,15 @@ class VariableResolution(tempVariableManager: TempVariableManager) extends SQLCo private def resolveVariable( nameParts: Seq[String], resolvingView: Boolean, - resolvingExecuteImmediate: Boolean, referredTempVariableNames: Seq[Seq[String]]): Option[Expression] = { if (resolvingView) { if (referredTempVariableNames.contains(nameParts)) { - lookupVariable(nameParts = nameParts, resolvingExecuteImmediate = resolvingExecuteImmediate) + lookupVariable(nameParts = nameParts) } else { None } } else { - lookupVariable(nameParts = nameParts, resolvingExecuteImmediate = resolvingExecuteImmediate) + lookupVariable(nameParts = nameParts) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala index f5fd11b375059..a0ace607e38e1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala @@ -17,12 +17,15 @@ package org.apache.spark.sql.catalyst.analysis -import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Exists, Expression, InSubquery, ListQuery, ScalarSubquery, VariableReference} +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, EmptyRow, Expression, InSubquery, SubqueryExpression, VariableReference} import org.apache.spark.sql.catalyst.plans.logical.{ExecutableDuringAnalysis, LocalRelation, LogicalPlan, SetVariable, UnaryNode} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.trees.TreePattern.{EXECUTE_IMMEDIATE, TreePattern} +import org.apache.spark.sql.catalyst.util.TypeUtils.toSQLId import org.apache.spark.sql.connector.catalog.CatalogManager import org.apache.spark.sql.errors.QueryCompilationErrors +import org.apache.spark.unsafe.types.UTF8String /** * Logical plan representing execute immediate query. @@ -84,40 +87,44 @@ class ResolveExecuteImmediate( val catalogManager: CatalogManager) extends Rule[LogicalPlan] { override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsWithPruning(_.containsPattern(EXECUTE_IMMEDIATE), ruleId) { - case UnresolvedExecuteImmediate(sqlStmtStr, args, targetVariables) - if sqlStmtStr.resolved && targetVariables.forall(_.resolved) && args.forall(_.resolved) => - // Validate that USING clause expressions don't contain unsupported constructs - validateUsingClauseExpressions(args) - - // Validate that query parameter is foldable (constant expression) - validateSqlStmt(sqlStmtStr) + case node @ UnresolvedExecuteImmediate(sqlStmtStr, args, targetVariables) => + if (sqlStmtStr.resolved && targetVariables.forall(_.resolved) && args.forall(_.resolved)) { + // Validate that expressions don't contain unsupported constructs (like subqueries) + validateExpressions(args :+ sqlStmtStr) // All resolved - transform based on whether we have target variables if (targetVariables.nonEmpty) { // EXECUTE IMMEDIATE ... INTO should generate SetVariable plan - // SetVariable expects UnresolvedAttribute objects that ResolveSetVariable will resolve + // First validate that the SQL statement is not a command (commands don't return results) + validateNotCommandForInto(sqlStmtStr) + // At this point, all targetVariables are resolved, so we only expect VariableReference + // or Alias containing VariableReference val finalTargetVars = targetVariables.map { - case attr: UnresolvedAttribute => - // Keep as UnresolvedAttribute for ResolveSetVariable to handle - attr case alias: Alias => - // Extract the UnresolvedAttribute from the alias + // Extract the VariableReference from the alias alias.child match { - case attr: UnresolvedAttribute => - attr case varRef: VariableReference => - // Convert back to UnresolvedAttribute for ResolveSetVariable - UnresolvedAttribute(varRef.originalNameParts) + // Use resolved VariableReference directly with canFold = false + varRef.copy(canFold = false) case _ => throw QueryCompilationErrors.unsupportedParameterExpression(alias.child) } case varRef: VariableReference => - // Convert back to UnresolvedAttribute for ResolveSetVariable - UnresolvedAttribute(varRef.originalNameParts) + // Use resolved VariableReference directly with canFold = false + varRef.copy(canFold = false) case other => throw QueryCompilationErrors.unsupportedParameterExpression(other) } + // Check for duplicate variable names (same logic as ResolveSetVariable) + val dups = finalTargetVars.groupBy(_.identifier).filter(kv => kv._2.length > 1) + if (dups.nonEmpty) { + throw new AnalysisException( + errorClass = "DUPLICATE_ASSIGNMENTS", + messageParameters = Map("nameList" -> + dups.keys.map(key => toSQLId(key.name())).mkString(", "))) + } + // Create SetVariable plan with the execute immediate query as source val sourceQuery = ExecuteImmediateCommand(sqlStmtStr, args) SetVariable(finalTargetVars, sourceQuery) @@ -125,40 +132,55 @@ class ResolveExecuteImmediate( // Regular EXECUTE IMMEDIATE without INTO ExecuteImmediateCommand(sqlStmtStr, args) } - case other => other + } else { // Not all resolved yet - wait for next iteration + node + } } - private def validateUsingClauseExpressions(args: Seq[Expression]): Unit = { - args.foreach { expr => + /** + * Validates that expressions don't contain unsupported constructs like subqueries. + * Variable references and expressions like string concatenation are allowed. + */ + private def validateExpressions(expressions: Seq[Expression]): Unit = { + expressions.foreach { expr => // Check the expression and its children for unsupported constructs expr.foreach { - case subquery: ScalarSubquery => + case subquery: SubqueryExpression => throw QueryCompilationErrors.unsupportedParameterExpression(subquery) - case exists: Exists => - throw QueryCompilationErrors.unsupportedParameterExpression(exists) - case listQuery: ListQuery => - throw QueryCompilationErrors.unsupportedParameterExpression(listQuery) case inSubquery: InSubquery => + // InSubquery doesn't extend SubqueryExpression directly but contains a subquery throw QueryCompilationErrors.unsupportedParameterExpression(inSubquery) - case _ => // Other expressions are fine + case _ => // Other expressions including variables and concatenations are fine } } } - private def validateSqlStmt(sqlStmtStr: Expression): Unit = { - // Only check for specific unsupported constructs like subqueries - // Variable references and expressions like stringvar || 'hello' should be allowed - sqlStmtStr.foreach { - case subquery: ScalarSubquery => - throw QueryCompilationErrors.unsupportedParameterExpression(subquery) - case exists: Exists => - throw QueryCompilationErrors.unsupportedParameterExpression(exists) - case listQuery: ListQuery => - throw QueryCompilationErrors.unsupportedParameterExpression(listQuery) - case inSubquery: InSubquery => - throw QueryCompilationErrors.unsupportedParameterExpression(inSubquery) - case _ => // Other expressions including variables and concatenations are fine + private def validateNotCommandForInto(sqlStmtStr: Expression): Unit = { + // Extract the query string to check if it's a command + val queryString = sqlStmtStr.eval(EmptyRow) match { + case null => + // Use a generic AnalysisException for null query + throw new AnalysisException("NULL_QUERY_STRING_EXECUTE_IMMEDIATE", + Map.empty[String, String]) + case s: UTF8String => + // scalastyle:off caselocale + s.toString.trim.toUpperCase + // scalastyle:on caselocale + case other => + // Use a generic AnalysisException for wrong type + throw new AnalysisException("INVALID_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE", + Map("actualType" -> other.getClass.getSimpleName)) + } + // Check if the SQL statement starts with common command keywords + // Commands don't return results, so INTO clause is invalid + val commandKeywords = Set( + "SET", "CREATE", "DROP", "ALTER", "INSERT", "UPDATE", "DELETE", + "DECLARE", "CALL", "GRANT", "REVOKE", "TRUNCATE", "MERGE", "USE" + ) + val firstWord = queryString.split("\\s+").headOption.getOrElse("") + if (commandKeywords.contains(firstWord)) { + throw QueryCompilationErrors.invalidStatementForExecuteInto(queryString) } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala index 0efb61c133f76..d1c81990f4f42 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala @@ -4149,12 +4149,6 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat messageParameters = Map("sqlString" -> toSQLStmt(queryString))) } - def nestedExecuteImmediate(queryString: String): Throwable = { - throw new AnalysisException( - errorClass = "NESTED_EXECUTE_IMMEDIATE", - messageParameters = Map("sqlString" -> toSQLStmt(queryString))) - } - def sqlScriptInExecuteImmediate(sqlScriptString: String): Throwable = { throw new AnalysisException( errorClass = "SQL_SCRIPT_IN_EXECUTE_IMMEDIATE", diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediateCommands.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediateCommands.scala index ec80620065cdd..eaf2296feb34b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediateCommands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediateCommands.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.SqlScriptingContextManager -import org.apache.spark.sql.catalyst.expressions.{Alias, EmptyRow, Exists, Expression, InSubquery, ListQuery, Literal, ScalarSubquery, VariableReference} +import org.apache.spark.sql.catalyst.expressions.{Alias, EmptyRow, Expression, Literal, VariableReference} import org.apache.spark.sql.catalyst.plans.logical.{Command, CommandResult, CompoundBody, LocalRelation, LogicalPlan} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.trees.TreePattern.EXECUTE_IMMEDIATE @@ -51,9 +51,7 @@ case class ExecuteImmediateCommands(sparkSession: SparkSession) extends Rule[Log val result = if (cmd.args.isEmpty) { // No parameters - execute directly withIsolatedLocalVariableContext { - AnalysisContext.withExecuteImmediateContext { - sparkSession.sql(sqlStmtStr) - } + sparkSession.sql(sqlStmtStr) } } else { // For parameterized queries, build unified parameter arrays @@ -63,9 +61,7 @@ case class ExecuteImmediateCommands(sparkSession: SparkSession) extends Rule[Log validateParameterUsage(cmd.sqlStmtStr, cmd.args, paramNames.toSeq) withIsolatedLocalVariableContext { - AnalysisContext.withExecuteImmediateContext { - sparkSession.sql(sqlStmtStr, paramValues, paramNames) - } + sparkSession.sql(sqlStmtStr, paramValues, paramNames) } } @@ -161,8 +157,8 @@ case class ExecuteImmediateCommands(sparkSession: SparkSession) extends Rule[Log } /** - * Evaluates a parameter expression, ensuring it's foldable and doesn't contain - * unsupported constructs. + * Evaluates a parameter expression. Validation for unsupported constructs like subqueries + * is already done during analysis in ResolveExecuteImmediate.validateExpressions(). */ private def evaluateParameterExpression(expr: Expression): Any = { expr match { @@ -173,15 +169,7 @@ case class ExecuteImmediateCommands(sparkSession: SparkSession) extends Rule[Log Literal.create(foldable.eval(EmptyRow), foldable.dataType).value case other => // Expression is not foldable - not supported for parameters - other match { - case _: ScalarSubquery | _: Exists | _: ListQuery | _: InSubquery => - throw QueryCompilationErrors.unsupportedParameterExpression(other) - case _ if !other.foldable => - throw QueryCompilationErrors.unsupportedParameterExpression(other) - case _ => - // This should not happen, but fallback to evaluation - other.eval(EmptyRow) - } + throw QueryCompilationErrors.unsupportedParameterExpression(other) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala index a3591ff89e5cf..e31e7e8d704ca 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.command import org.apache.spark.internal.Logging import org.apache.spark.internal.LogKeys.{CONFIG, CONFIG2, KEY, VALUE} import org.apache.spark.sql.{AnalysisException, Row, SparkSession} -import org.apache.spark.sql.catalyst.analysis.{AnalysisContext, VariableResolution} +import org.apache.spark.sql.catalyst.analysis.VariableResolution import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.catalyst.plans.logical.IgnoreCachedData @@ -112,8 +112,7 @@ case class SetCommand(kv: Option[(String, Option[String])]) sparkSession.sessionState.analyzer.catalogManager.tempVariableManager ) val variable = variableResolution.lookupVariable( - nameParts = varName, - resolvingExecuteImmediate = AnalysisContext.get.isExecuteImmediate + nameParts = varName ) if (variable.isDefined) { throw new AnalysisException( diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/execute-immediate.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/execute-immediate.sql.out index da14427837080..6238852b4efa9 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/execute-immediate.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/execute-immediate.sql.out @@ -364,11 +364,10 @@ EXECUTE IMMEDIATE 'INSERT INTO x VALUES (?)' INTO res_id USING 1 -- !query analysis org.apache.spark.sql.AnalysisException { - "errorClass" : "ASSIGNMENT_ARITY_MISMATCH", - "sqlState" : "42802", + "errorClass" : "INVALID_STATEMENT_FOR_EXECUTE_INTO", + "sqlState" : "07501", "messageParameters" : { - "numExpr" : "0", - "numTarget" : "1" + "sqlString" : "INSERT INTO X VALUES (?)" } } diff --git a/sql/core/src/test/resources/sql-tests/results/execute-immediate.sql.out b/sql/core/src/test/resources/sql-tests/results/execute-immediate.sql.out index d35b11e3f0bf0..3e398ce38815c 100644 --- a/sql/core/src/test/resources/sql-tests/results/execute-immediate.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/execute-immediate.sql.out @@ -414,11 +414,10 @@ struct<> -- !query output org.apache.spark.sql.AnalysisException { - "errorClass" : "ASSIGNMENT_ARITY_MISMATCH", - "sqlState" : "42802", + "errorClass" : "INVALID_STATEMENT_FOR_EXECUTE_INTO", + "sqlState" : "07501", "messageParameters" : { - "numExpr" : "0", - "numTarget" : "1" + "sqlString" : "INSERT INTO X VALUES (?)" } } From 77d7268131b48ef0a62fd034e554903eeed8d85b Mon Sep 17 00:00:00 2001 From: Serge Rielau Date: Thu, 4 Sep 2025 11:07:18 -0700 Subject: [PATCH 26/32] Reduce complexity around INTO clause --- .../catalyst/analysis/executeImmediate.scala | 40 +++---------------- .../analysis/ExecuteImmediateCommands.scala | 8 +++- 2 files changed, 12 insertions(+), 36 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala index a0ace607e38e1..5a89f35a89cf0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala @@ -18,14 +18,13 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, EmptyRow, Expression, InSubquery, SubqueryExpression, VariableReference} +import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Expression, InSubquery, SubqueryExpression, VariableReference} import org.apache.spark.sql.catalyst.plans.logical.{ExecutableDuringAnalysis, LocalRelation, LogicalPlan, SetVariable, UnaryNode} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.trees.TreePattern.{EXECUTE_IMMEDIATE, TreePattern} import org.apache.spark.sql.catalyst.util.TypeUtils.toSQLId import org.apache.spark.sql.connector.catalog.CatalogManager import org.apache.spark.sql.errors.QueryCompilationErrors -import org.apache.spark.unsafe.types.UTF8String /** * Logical plan representing execute immediate query. @@ -49,10 +48,12 @@ case class UnresolvedExecuteImmediate( * * @param sqlStmtStr the resolved query expression * @param args parameters from USING clause + * @param hasIntoClause whether this EXECUTE IMMEDIATE has an INTO clause */ case class ExecuteImmediateCommand( sqlStmtStr: Expression, - args: Seq[Expression]) + args: Seq[Expression], + hasIntoClause: Boolean = false) extends UnaryNode with ExecutableDuringAnalysis { final override val nodePatterns: Seq[TreePattern] = Seq(EXECUTE_IMMEDIATE) @@ -95,8 +96,6 @@ class ResolveExecuteImmediate( // All resolved - transform based on whether we have target variables if (targetVariables.nonEmpty) { // EXECUTE IMMEDIATE ... INTO should generate SetVariable plan - // First validate that the SQL statement is not a command (commands don't return results) - validateNotCommandForInto(sqlStmtStr) // At this point, all targetVariables are resolved, so we only expect VariableReference // or Alias containing VariableReference val finalTargetVars = targetVariables.map { @@ -126,11 +125,11 @@ class ResolveExecuteImmediate( } // Create SetVariable plan with the execute immediate query as source - val sourceQuery = ExecuteImmediateCommand(sqlStmtStr, args) + val sourceQuery = ExecuteImmediateCommand(sqlStmtStr, args, hasIntoClause = true) SetVariable(finalTargetVars, sourceQuery) } else { // Regular EXECUTE IMMEDIATE without INTO - ExecuteImmediateCommand(sqlStmtStr, args) + ExecuteImmediateCommand(sqlStmtStr, args, hasIntoClause = false) } } else { // Not all resolved yet - wait for next iteration @@ -156,31 +155,4 @@ class ResolveExecuteImmediate( } } - private def validateNotCommandForInto(sqlStmtStr: Expression): Unit = { - // Extract the query string to check if it's a command - val queryString = sqlStmtStr.eval(EmptyRow) match { - case null => - // Use a generic AnalysisException for null query - throw new AnalysisException("NULL_QUERY_STRING_EXECUTE_IMMEDIATE", - Map.empty[String, String]) - case s: UTF8String => - // scalastyle:off caselocale - s.toString.trim.toUpperCase - // scalastyle:on caselocale - case other => - // Use a generic AnalysisException for wrong type - throw new AnalysisException("INVALID_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE", - Map("actualType" -> other.getClass.getSimpleName)) - } - // Check if the SQL statement starts with common command keywords - // Commands don't return results, so INTO clause is invalid - val commandKeywords = Set( - "SET", "CREATE", "DROP", "ALTER", "INSERT", "UPDATE", "DELETE", - "DECLARE", "CALL", "GRANT", "REVOKE", "TRUNCATE", "MERGE", "USE" - ) - val firstWord = queryString.split("\\s+").headOption.getOrElse("") - if (commandKeywords.contains(firstWord)) { - throw QueryCompilationErrors.invalidStatementForExecuteInto(queryString) - } - } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediateCommands.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediateCommands.scala index eaf2296feb34b..71a0ca26c7012 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediateCommands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediateCommands.scala @@ -68,10 +68,14 @@ case class ExecuteImmediateCommands(sparkSession: SparkSession) extends Rule[Log // Check if the executed statement is a Command (like DECLARE, SET VARIABLE, etc.) // Commands should not return result sets result.queryExecution.analyzed match { - case cmd: Command => + case command: Command => + // If this EXECUTE IMMEDIATE has an INTO clause, commands are not allowed + if (cmd.hasIntoClause) { + throw QueryCompilationErrors.invalidStatementForExecuteInto(sqlStmtStr) + } // Commands don't produce output - return CommandResult to indicate this is a command // For analyzer tests, we don't need the actual executed rows, just the structure - CommandResult(cmd.output, cmd, result.queryExecution.executedPlan, Seq.empty) + CommandResult(command.output, command, result.queryExecution.executedPlan, Seq.empty) case _ => // Regular queries - return the results as a LocalRelation val internalRows = result.queryExecution.executedPlan.executeCollect() From 0588b932400c37000f2f4ad7ff678c45fbab4160 Mon Sep 17 00:00:00 2001 From: Serge Rielau Date: Thu, 4 Sep 2025 12:01:55 -0700 Subject: [PATCH 27/32] Fixed parameter tests in parameters.scala Please enter the commit message for your changes. Lines starting --- .../sql/catalyst/analysis/parameters.scala | 34 ++++++++++------ .../analysis/ExecuteImmediateCommands.scala | 39 ------------------- 2 files changed, 23 insertions(+), 50 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/parameters.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/parameters.scala index 9c20306ff80a5..3ddc1b42ed0ed 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/parameters.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/parameters.scala @@ -22,7 +22,7 @@ import org.apache.spark.sql.catalyst.expressions.{Alias, CreateArray, CreateMap, import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SupervisingCommand} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.trees.TreePattern.{COMMAND, PARAMETER, PARAMETERIZED_QUERY, TreePattern, UNRESOLVED_WITH} -import org.apache.spark.sql.errors.QueryErrorsBase +import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryErrorsBase} import org.apache.spark.sql.types.DataType sealed trait Parameter extends LeafExpression with Unevaluable { @@ -113,8 +113,8 @@ case class PosParameterizedQuery(child: LogicalPlan, args: Seq[Expression]) * @param args The literal values or collection constructor functions such as `map()`, * `array()`, `struct()` of parameters. * @param paramNames Optional parameter names corresponding to args. If provided for an argument, - * that argument can be used for named parameter binding. If not provided or - * shorter than args, remaining parameters are treated as positional. + * that argument can be used for named parameter binding. If not provided + * parameters are treated as positional. */ case class UnifiedParameterizedQuery( child: LogicalPlan, @@ -237,6 +237,24 @@ object BindParameters extends Rule[LogicalPlan] with QueryErrorsBase { case p @ PosParameter(pos) => positionalParams.add(pos); p } + // Check: Does the query mix positional and named parameters? + if (namedParams.nonEmpty && positionalParams.nonEmpty) { + throw QueryCompilationErrors.invalidQueryMixedQueryParameters() + } + + // If query uses only named parameters, all USING expressions must have names + if (namedParams.nonEmpty && positionalParams.isEmpty) { + val unnamedExpressions = paramNames.zipWithIndex.collect { + case (null, index) => index + case ("", index) => index // empty strings are unnamed + } + if (unnamedExpressions.nonEmpty) { + // Get the actual expressions that don't have names for error reporting + val unnamedExprs = unnamedExpressions.map(args(_)) + throw QueryCompilationErrors.invalidQueryAllParametersMustBeNamed(unnamedExprs) + } + } + // Build parameter maps based on what the query actually uses val namedArgsMap = scala.collection.mutable.Map[String, Expression]() val positionalArgs = scala.collection.mutable.ListBuffer[Expression]() @@ -257,14 +275,8 @@ object BindParameters extends Rule[LogicalPlan] with QueryErrorsBase { // Query uses only positional parameters - use all args as positional positionalArgs ++= args } else { - // Mixed or no parameters - build both maps - args.zipWithIndex.foreach { case (arg, index) => - if (index < paramNames.length && paramNames(index).nonEmpty) { - namedArgsMap(paramNames(index)) = arg - } else { - positionalArgs += arg - } - } + // No parameters in query - treat all args as positional for backward compatibility + positionalArgs ++= args } // Check all arguments for validity diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediateCommands.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediateCommands.scala index 71a0ca26c7012..40f2aec61d15b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediateCommands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediateCommands.scala @@ -57,9 +57,6 @@ case class ExecuteImmediateCommands(sparkSession: SparkSession) extends Rule[Log // For parameterized queries, build unified parameter arrays val (paramValues, paramNames) = buildUnifiedParameters(cmd.args) - // Validate parameter usage patterns - validateParameterUsage(cmd.sqlStmtStr, cmd.args, paramNames.toSeq) - withIsolatedLocalVariableContext { sparkSession.sql(sqlStmtStr, paramValues, paramNames) } @@ -177,42 +174,6 @@ case class ExecuteImmediateCommands(sparkSession: SparkSession) extends Rule[Log } } - private def validateParameterUsage( - queryParam: Expression, - args: Seq[Expression], - names: Seq[String]): Unit = { - // Extract the query string to check for parameter patterns - val queryString = queryParam.eval(null) match { - case null => return // Will be caught later by other validation - case value => value.toString - } - - // Check what types of parameters the query uses - val positionalParameterPattern = "\\?".r - val namedParameterPattern = ":[a-zA-Z_][a-zA-Z0-9_]*".r - - val queryUsesPositionalParameters = - positionalParameterPattern.findFirstIn(queryString).isDefined - val queryUsesNamedParameters = namedParameterPattern.findFirstIn(queryString).isDefined - - // Check: Does the query mix positional and named parameters? - if (queryUsesPositionalParameters && queryUsesNamedParameters) { - throw QueryCompilationErrors.invalidQueryMixedQueryParameters() - } - - // If query uses only named parameters, all USING expressions must have names - if (queryUsesNamedParameters && !queryUsesPositionalParameters) { - val unnamedExpressions = names.zipWithIndex.collect { - case (null, index) => index - case ("", index) => index // empty strings are unnamed - } - if (unnamedExpressions.nonEmpty) { - // Get the actual expressions that don't have names for error reporting - val unnamedExprs = unnamedExpressions.map(args(_)) - throw QueryCompilationErrors.invalidQueryAllParametersMustBeNamed(unnamedExprs) - } - } - } /** * Temporarily isolates the SQL scripting context during EXECUTE IMMEDIATE execution. From b501c5127f9582b0a78d1810a4be7d224dee0b4e Mon Sep 17 00:00:00 2001 From: Serge Rielau Date: Thu, 4 Sep 2025 12:18:25 -0700 Subject: [PATCH 28/32] simplify parameters.scala --- .../sql/catalyst/analysis/parameters.scala | 86 ++++++------------- 1 file changed, 25 insertions(+), 61 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/parameters.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/parameters.scala index 3ddc1b42ed0ed..8838c1632bda0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/parameters.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/parameters.scala @@ -229,7 +229,7 @@ object BindParameters extends Rule[LogicalPlan] with QueryErrorsBase { if !child.containsPattern(UNRESOLVED_WITH) && args.forall(_.resolved) => - // First pass: collect all parameter types used in the query to determine strategy + // Collect parameter types used in the query and validate no mixing val namedParams = scala.collection.mutable.Set.empty[String] val positionalParams = scala.collection.mutable.Set.empty[Int] bind(child) { @@ -237,86 +237,50 @@ object BindParameters extends Rule[LogicalPlan] with QueryErrorsBase { case p @ PosParameter(pos) => positionalParams.add(pos); p } - // Check: Does the query mix positional and named parameters? + // Validate: no mixing of positional and named parameters if (namedParams.nonEmpty && positionalParams.nonEmpty) { throw QueryCompilationErrors.invalidQueryMixedQueryParameters() } - // If query uses only named parameters, all USING expressions must have names + // Validate: if query uses named parameters, all USING expressions must have names if (namedParams.nonEmpty && positionalParams.isEmpty) { val unnamedExpressions = paramNames.zipWithIndex.collect { case (null, index) => index case ("", index) => index // empty strings are unnamed } if (unnamedExpressions.nonEmpty) { - // Get the actual expressions that don't have names for error reporting val unnamedExprs = unnamedExpressions.map(args(_)) throw QueryCompilationErrors.invalidQueryAllParametersMustBeNamed(unnamedExprs) } } - // Build parameter maps based on what the query actually uses - val namedArgsMap = scala.collection.mutable.Map[String, Expression]() - val positionalArgs = scala.collection.mutable.ListBuffer[Expression]() - - if (namedParams.nonEmpty && positionalParams.isEmpty) { - // Query uses only named parameters - try to match args to named parameters - args.zipWithIndex.foreach { case (arg, index) => - val paramName = if (index < paramNames.length && paramNames(index).nonEmpty) { - paramNames(index) - } else { - // For session variables without explicit AS clause, try to infer the name - // from the parameter names found in the query - namedParams.toSeq.lift(index).getOrElse(s"param_$index") - } - namedArgsMap(paramName) = arg + // Check all arguments for validity (args are already evaluated expressions/literals) + val allArgs = args.zipWithIndex.map { case (arg, idx) => + val name = if (idx < paramNames.length && paramNames(idx) != null) { + paramNames(idx) + } else { + s"_$idx" } - } else if (positionalParams.nonEmpty && namedParams.isEmpty) { - // Query uses only positional parameters - use all args as positional - positionalArgs ++= args - } else { - // No parameters in query - treat all args as positional for backward compatibility - positionalArgs ++= args - } - - // Check all arguments for validity - val allArgs = namedArgsMap.toSeq ++ positionalArgs.zipWithIndex.map { - case (arg, idx) => (s"_$idx", arg) + (name, arg) } checkArgs(allArgs) - // Bind named parameters by converting expressions to literals - val boundWithNamed = if (namedArgsMap.nonEmpty) { - bind(child) { - case NamedParameter(name) if namedArgsMap.contains(name) => - val expr = namedArgsMap(name) - if (expr.foldable) { - Literal.create(expr.eval(), expr.dataType) - } else { - // For non-foldable expressions, try to convert to SQL and re-parse - expr - } - } - } else { - child - } + // Single pass binding - args are already literals/evaluated expressions + val namedArgsMap = paramNames.zipWithIndex.collect { + case (name, index) if name != null => name -> args(index) + }.toMap + val positionalArgsMap = if (positionalParams.nonEmpty) { + val sortedPositions = positionalParams.toSeq.sorted + sortedPositions.zipWithIndex.map { case (pos, index) => + pos -> (if (index < args.length) args(index) else args.last) + }.toMap + } else Map.empty[Int, Expression] - // Bind positional parameters - if (positionalArgs.nonEmpty) { - val posToIndex = positionalParams.toSeq.sorted.zipWithIndex.toMap - bind(boundWithNamed) { - case PosParameter(pos) if posToIndex.contains(pos) && - positionalArgs.size > posToIndex(pos) => - val expr = positionalArgs(posToIndex(pos)) - if (expr.foldable) { - Literal.create(expr.eval(), expr.dataType) - } else { - // For non-foldable expressions, try to convert to SQL and re-parse - expr - } - } - } else { - boundWithNamed + bind(child) { + case NamedParameter(name) if namedArgsMap.contains(name) => + namedArgsMap(name) + case PosParameter(pos) if positionalArgsMap.contains(pos) => + positionalArgsMap(pos) } case other => other From af6f40309190a6e7c9ac83f3b62e41256296b7d7 Mon Sep 17 00:00:00 2001 From: Serge Rielau Date: Thu, 4 Sep 2025 17:32:42 -0700 Subject: [PATCH 29/32] more fixes --- .../org/apache/spark/sql/SparkSession.scala | 17 ----------------- .../sql/catalyst/analysis/parameters.scala | 4 ++-- .../apache/spark/sql/connect/SparkSession.scala | 6 ------ .../analysis/ExecuteImmediateCommands.scala | 3 ++- .../apache/spark/sql/classic/SparkSession.scala | 2 +- 5 files changed, 5 insertions(+), 27 deletions(-) diff --git a/sql/api/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/api/src/main/scala/org/apache/spark/sql/SparkSession.scala index c557bb1fd1d6e..99290eef50b2b 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -523,23 +523,6 @@ abstract class SparkSession extends Serializable with Closeable { sql(sqlText, args.asScala.toMap) } - /** - * Executes a SQL query substituting parameters by the given arguments with optional names, - * returning the result as a `DataFrame`. This API eagerly runs DDL/DML commands, but not for - * SELECT queries. This method allows the inner query to determine whether to use positional - * or named parameters based on its parameter markers. - * - * @param sqlText - * A SQL statement with named or positional parameters to execute. - * @param args - * An array of Java/Scala objects that can be converted to SQL literal expressions. - * @param paramNames - * An optional array of parameter names corresponding to args. If provided, enables named - * parameter binding where parameter names are available. If None or shorter than args, - * remaining parameters are treated as positional. - * @since 4.0.0 - */ - def sql(sqlText: String, args: Array[_], paramNames: Array[String]): DataFrame /** * Executes a SQL query using Spark, returning the result as a `DataFrame`. This API eagerly diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/parameters.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/parameters.scala index 8838c1632bda0..cf1ffef76fdfc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/parameters.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/parameters.scala @@ -271,8 +271,8 @@ object BindParameters extends Rule[LogicalPlan] with QueryErrorsBase { }.toMap val positionalArgsMap = if (positionalParams.nonEmpty) { val sortedPositions = positionalParams.toSeq.sorted - sortedPositions.zipWithIndex.map { case (pos, index) => - pos -> (if (index < args.length) args(index) else args.last) + sortedPositions.zipWithIndex.collect { case (pos, index) if index < args.length => + pos -> args(index) }.toMap } else Map.empty[Int, Expression] diff --git a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/SparkSession.scala b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/SparkSession.scala index 591e3a57739c6..97ce547f03b23 100644 --- a/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/SparkSession.scala +++ b/sql/connect/common/src/main/scala/org/apache/spark/sql/connect/SparkSession.scala @@ -250,12 +250,6 @@ class SparkSession private[sql] ( sql(query, Array.empty) } - /** @inheritdoc */ - def sql(sqlText: String, args: Array[_], paramNames: Array[String]): DataFrame = { - // For Connect, delegate to existing positional parameter implementation - // The unified parameter logic is handled on the server side - sql(sqlText, args) - } private def sql(sqlCommand: proto.SqlCommand): DataFrame = { // Send the SQL once to the server and then check the output. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediateCommands.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediateCommands.scala index 40f2aec61d15b..0d14408ad268e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediateCommands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediateCommands.scala @@ -58,7 +58,8 @@ case class ExecuteImmediateCommands(sparkSession: SparkSession) extends Rule[Log val (paramValues, paramNames) = buildUnifiedParameters(cmd.args) withIsolatedLocalVariableContext { - sparkSession.sql(sqlStmtStr, paramValues, paramNames) + sparkSession.asInstanceOf[org.apache.spark.sql.classic.SparkSession] + .sql(sqlStmtStr, paramValues, paramNames) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/classic/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/classic/SparkSession.scala index 97473b5d3fef0..35542338656de 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/classic/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/classic/SparkSession.scala @@ -514,7 +514,7 @@ class SparkSession private( * returning the result as a `DataFrame`. This method allows the inner query to determine * whether to use positional or named parameters based on its parameter markers. */ - def sql(sqlText: String, args: Array[_], paramNames: Array[String]): DataFrame = { + private[sql] def sql(sqlText: String, args: Array[_], paramNames: Array[String]): DataFrame = { sql(sqlText, args, paramNames, new QueryPlanningTracker) } From 769cadc1832df718d8f446826149aa8f216d6f96 Mon Sep 17 00:00:00 2001 From: Serge Rielau Date: Thu, 4 Sep 2025 21:23:47 -0700 Subject: [PATCH 30/32] Move testcases --- .../analysis/AnalysisErrorSuite.scala | 13 ----------- .../execute-immediate.sql.out | 20 ++++++++++++++++ .../sql-tests/inputs/execute-immediate.sql | 4 ++++ .../results/execute-immediate.sql.out | 23 +++++++++++++++++++ 4 files changed, 47 insertions(+), 13 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala index c9892742722e1..94f650bc35c77 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala @@ -838,19 +838,6 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { - test("EXEC IMMEDIATE - INTO specified for COMMAND query") { - val execImmediateSetVariablePlan = UnresolvedExecuteImmediate( - Literal("SET VAR testVarA = 1"), - Seq.empty, - Seq(UnresolvedAttribute("testVarA"))) - - assertAnalysisErrorCondition( - inputPlan = execImmediateSetVariablePlan, - expectedErrorCondition = "INVALID_STATEMENT_FOR_EXECUTE_INTO", - expectedMessageParameters = Map( - "sqlString" -> "SET VAR TESTVARA = 1" - )) - } test("SPARK-6452 regression test") { // CheckAnalysis should throw AnalysisException when Aggregate contains missing attribute(s) diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/execute-immediate.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/execute-immediate.sql.out index 6238852b4efa9..e5cdff605cdff 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/execute-immediate.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/execute-immediate.sql.out @@ -372,6 +372,26 @@ org.apache.spark.sql.AnalysisException } +-- !query +DECLARE OR REPLACE testvarA INT +-- !query analysis +CreateVariable defaultvalueexpression(null, null), true ++- ResolvedIdentifier org.apache.spark.sql.catalyst.analysis.FakeSystemCatalog$@xxxxxxxx, session.testvarA + + +-- !query +EXECUTE IMMEDIATE 'SET VAR testVarA = 1' INTO testVarA +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_STATEMENT_FOR_EXECUTE_INTO", + "sqlState" : "07501", + "messageParameters" : { + "sqlString" : "SET VAR TESTVARA = 1" + } +} + + -- !query EXECUTE IMMEDIATE 'SELECT * FROM tbl_view WHERE ? = id' USING id -- !query analysis diff --git a/sql/core/src/test/resources/sql-tests/inputs/execute-immediate.sql b/sql/core/src/test/resources/sql-tests/inputs/execute-immediate.sql index 78ae4cca89199..3eaacbd37abc6 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/execute-immediate.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/execute-immediate.sql @@ -87,6 +87,10 @@ EXECUTE IMMEDIATE 'SELECT \'invalid_cast_error_expected\'' INTO res_id; -- require query when using INTO EXECUTE IMMEDIATE 'INSERT INTO x VALUES (?)' INTO res_id USING 1; +-- require query when using INTO with SET VAR command +DECLARE OR REPLACE testvarA INT; +EXECUTE IMMEDIATE 'SET VAR testVarA = 1' INTO testVarA; + -- use column in using - should fail as we expect variable here EXECUTE IMMEDIATE 'SELECT * FROM tbl_view WHERE ? = id' USING id; diff --git a/sql/core/src/test/resources/sql-tests/results/execute-immediate.sql.out b/sql/core/src/test/resources/sql-tests/results/execute-immediate.sql.out index 3e398ce38815c..f2f2bb661e30e 100644 --- a/sql/core/src/test/resources/sql-tests/results/execute-immediate.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/execute-immediate.sql.out @@ -422,6 +422,29 @@ org.apache.spark.sql.AnalysisException } +-- !query +DECLARE OR REPLACE testvarA INT +-- !query schema +struct<> +-- !query output + + + +-- !query +EXECUTE IMMEDIATE 'SET VAR testVarA = 1' INTO testVarA +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_STATEMENT_FOR_EXECUTE_INTO", + "sqlState" : "07501", + "messageParameters" : { + "sqlString" : "SET VAR TESTVARA = 1" + } +} + + -- !query EXECUTE IMMEDIATE 'SELECT * FROM tbl_view WHERE ? = id' USING id -- !query schema From ecbc859b28d7db68d77441d42a068abf13bdf68f Mon Sep 17 00:00:00 2001 From: Serge Rielau Date: Fri, 5 Sep 2025 09:07:54 -0700 Subject: [PATCH 31/32] Fixes --- .../org/apache/spark/sql/SparkSession.scala | 1 - .../sql/catalyst/analysis/parameters.scala | 21 ++++++-- .../analysis/ExecuteImmediateCommands.scala | 48 +++++++------------ .../plans/logical/CommandResult.scala | 19 ++++++++ .../spark/sql/classic/SparkSession.scala | 16 ++----- .../execute-immediate.sql.out | 10 ++-- 6 files changed, 59 insertions(+), 56 deletions(-) diff --git a/sql/api/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/api/src/main/scala/org/apache/spark/sql/SparkSession.scala index 99290eef50b2b..8e7ae51e998f7 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -523,7 +523,6 @@ abstract class SparkSession extends Serializable with Closeable { sql(sqlText, args.asScala.toMap) } - /** * Executes a SQL query using Spark, returning the result as a `DataFrame`. This API eagerly * runs DDL/DML commands, but not for SELECT queries. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/parameters.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/parameters.scala index cf1ffef76fdfc..d6997bdaf103a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/parameters.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/parameters.scala @@ -19,10 +19,10 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.SparkException import org.apache.spark.sql.catalyst.expressions.{Alias, CreateArray, CreateMap, CreateNamedStruct, Expression, LeafExpression, Literal, MapFromArrays, MapFromEntries, SubqueryExpression, Unevaluable, VariableReference} -import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SupervisingCommand} +import org.apache.spark.sql.catalyst.plans.logical.{CompoundBody, LogicalPlan, SupervisingCommand} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.trees.TreePattern.{COMMAND, PARAMETER, PARAMETERIZED_QUERY, TreePattern, UNRESOLVED_WITH} -import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryErrorsBase} +import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryErrorsBase, SqlScriptingErrors} import org.apache.spark.sql.types.DataType sealed trait Parameter extends LeafExpression with Unevaluable { @@ -105,7 +105,7 @@ case class PosParameterizedQuery(child: LogicalPlan, args: Seq[Expression]) } /** - * The logical plan representing a parameterized query with unified parameter support. + * The logical plan representing a parameterized query with general parameter support. * This allows the query to use either positional or named parameters based on the * parameter markers found in the query, with optional parameter names provided. * @@ -116,7 +116,7 @@ case class PosParameterizedQuery(child: LogicalPlan, args: Seq[Expression]) * that argument can be used for named parameter binding. If not provided * parameters are treated as positional. */ -case class UnifiedParameterizedQuery( +case class GeneralParameterizedQuery( child: LogicalPlan, args: Seq[Expression], paramNames: Seq[String]) @@ -213,6 +213,12 @@ object BindParameters extends Rule[LogicalPlan] with QueryErrorsBase { case PosParameterizedQuery(child, args) if !child.containsPattern(UNRESOLVED_WITH) && args.forall(_.resolved) => + + // Validate: positional parameters are not supported for SQL scripting + if (child.isInstanceOf[CompoundBody]) { + throw SqlScriptingErrors.positionalParametersAreNotSupportedWithSqlScripting() + } + val indexedArgs = args.zipWithIndex checkArgs(indexedArgs.map(arg => (s"_${arg._2}", arg._1))) @@ -225,7 +231,7 @@ object BindParameters extends Rule[LogicalPlan] with QueryErrorsBase { args(posToIndex(pos)) } - case UnifiedParameterizedQuery(child, args, paramNames) + case GeneralParameterizedQuery(child, args, paramNames) if !child.containsPattern(UNRESOLVED_WITH) && args.forall(_.resolved) => @@ -242,6 +248,11 @@ object BindParameters extends Rule[LogicalPlan] with QueryErrorsBase { throw QueryCompilationErrors.invalidQueryMixedQueryParameters() } + // Validate: positional parameters are not supported for SQL scripting + if (positionalParams.nonEmpty && child.isInstanceOf[CompoundBody]) { + throw SqlScriptingErrors.positionalParametersAreNotSupportedWithSqlScripting() + } + // Validate: if query uses named parameters, all USING expressions must have names if (namedParams.nonEmpty && positionalParams.isEmpty) { val unnamedExpressions = paramNames.zipWithIndex.collect { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediateCommands.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediateCommands.scala index 0d14408ad268e..b131519401a23 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediateCommands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediateCommands.scala @@ -40,22 +40,22 @@ case class ExecuteImmediateCommands(sparkSession: SparkSession) extends Rule[Log } private def executeImmediate(cmd: ExecuteImmediateCommand): LogicalPlan = { - // Extract the query string from the queryParam expression + // Extract the query string from the queryParam expression val sqlStmtStr = extractQueryString(cmd.sqlStmtStr) - // Parse and validate the query + // Parse and validate the query val parsedPlan = sparkSession.sessionState.sqlParser.parsePlan(sqlStmtStr) validateQuery(sqlStmtStr, parsedPlan) // Execute the query recursively with isolated local variable context - val result = if (cmd.args.isEmpty) { - // No parameters - execute directly + val result = if (cmd.args.isEmpty) { + // No parameters - execute directly withIsolatedLocalVariableContext { sparkSession.sql(sqlStmtStr) - } - } else { - // For parameterized queries, build unified parameter arrays - val (paramValues, paramNames) = buildUnifiedParameters(cmd.args) + } + } else { + // For parameterized queries, build unified parameter arrays + val (paramValues, paramNames) = buildUnifiedParameters(cmd.args) withIsolatedLocalVariableContext { sparkSession.asInstanceOf[org.apache.spark.sql.classic.SparkSession] @@ -71,9 +71,11 @@ case class ExecuteImmediateCommands(sparkSession: SparkSession) extends Rule[Log if (cmd.hasIntoClause) { throw QueryCompilationErrors.invalidStatementForExecuteInto(sqlStmtStr) } - // Commands don't produce output - return CommandResult to indicate this is a command - // For analyzer tests, we don't need the actual executed rows, just the structure - CommandResult(command.output, command, result.queryExecution.executedPlan, Seq.empty) + // Commands may produce output (e.g., SET commands) - collect the actual results + // The Thrift Server relies on CommandResult containing the actual output data + val commandRows = result.queryExecution.executedPlan.executeCollect() + CommandResult( + command.output, command, result.queryExecution.executedPlan, commandRows.toSeq) case _ => // Regular queries - return the results as a LocalRelation val internalRows = result.queryExecution.executedPlan.executeCollect() @@ -127,27 +129,9 @@ case class ExecuteImmediateCommands(sparkSession: SparkSession) extends Rule[Log args.foreach { case alias: Alias => - // Check if this is an auto-generated alias or explicit "value AS paramName" - val isAutoGeneratedAlias = alias.child match { - case varRef: VariableReference => - // If the alias name matches the variable name, it's auto-generated - alias.name == varRef.identifier.name() - case _ => false - } - val paramValue = evaluateParameterExpression(alias.child) - - if (isAutoGeneratedAlias) { - // Session variable without explicit AS clause - val varName = alias.child.asInstanceOf[VariableReference].identifier.name() - values += paramValue - names += varName - } else { - // Named parameter: "value AS paramName" - val paramName = alias.name - values += paramValue - names += paramName - } + values += paramValue + names += alias.name case expr => // Positional parameter: just a value val paramValue = evaluateParameterExpression(expr) @@ -164,7 +148,7 @@ case class ExecuteImmediateCommands(sparkSession: SparkSession) extends Rule[Log */ private def evaluateParameterExpression(expr: Expression): Any = { expr match { - case varRef: VariableReference => + case varRef: VariableReference => // Variable references should be evaluated to their values varRef.eval(EmptyRow) case foldable if foldable.foldable => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CommandResult.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CommandResult.scala index 2ef342227833d..467a77f99c309 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CommandResult.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/CommandResult.scala @@ -40,4 +40,23 @@ case class CommandResult( override def computeStats(): Statistics = Statistics(sizeInBytes = EstimationUtils.getSizePerRow(output) * rows.length) + + override def simpleString(maxFields: Int): String = { + val outputStr = output.map(_.name).mkString("[", ", ", "]") + val dataStr = if (rows.nonEmpty) { + val formattedRows = rows.take(maxFields).map { row => + output.zipWithIndex.map { case (attr, i) => + val value = row.get(i, attr.dataType) + if (value == null) "null" else value.toString + }.mkString("[", ", ", "]") + } + s", data=[${formattedRows.mkString(", ")}]" + } else { + "" + } + s"CommandResult $outputStr$dataStr" + } + + // Don't override toString - let the default TreeNode.toString handle tree structure + // override def toString: String = simpleString(conf.maxToStringFields) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/classic/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/classic/SparkSession.scala index 35542338656de..a8e18a9cce165 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/classic/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/classic/SparkSession.scala @@ -40,15 +40,14 @@ import org.apache.spark.sql import org.apache.spark.sql.{Artifact, DataSourceRegistration, Encoder, Encoders, ExperimentalMethods, Row, SparkSessionBuilder, SparkSessionCompanion, SparkSessionExtensions, SparkSessionExtensionsProvider, UDTFRegistration} import org.apache.spark.sql.artifact.ArtifactManager import org.apache.spark.sql.catalyst._ -import org.apache.spark.sql.catalyst.analysis.{NameParameterizedQuery, PosParameterizedQuery, UnifiedParameterizedQuery, UnresolvedRelation} +import org.apache.spark.sql.catalyst.analysis.{GeneralParameterizedQuery, NameParameterizedQuery, PosParameterizedQuery, UnresolvedRelation} import org.apache.spark.sql.catalyst.encoders._ import org.apache.spark.sql.catalyst.expressions.AttributeReference import org.apache.spark.sql.catalyst.parser.ParserInterface -import org.apache.spark.sql.catalyst.plans.logical.{CompoundBody, LocalRelation, Range} +import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, Range} import org.apache.spark.sql.catalyst.types.DataTypeUtils.toAttributes import org.apache.spark.sql.catalyst.util.CharVarcharUtils import org.apache.spark.sql.classic.SparkSession.applyAndLoadExtensions -import org.apache.spark.sql.errors.SqlScriptingErrors import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.command.ExternalCommandExecutor import org.apache.spark.sql.execution.datasources.LogicalRelation @@ -450,10 +449,6 @@ class SparkSession private( val plan = tracker.measurePhase(QueryPlanningTracker.PARSING) { val parsedPlan = sessionState.sqlParser.parsePlan(sqlText) if (args.nonEmpty) { - if (parsedPlan.isInstanceOf[CompoundBody]) { - // Positional parameters are not supported for SQL scripting. - throw SqlScriptingErrors.positionalParametersAreNotSupportedWithSqlScripting() - } PosParameterizedQuery(parsedPlan, args.map(lit(_).expr).toImmutableArraySeq) } else { parsedPlan @@ -530,14 +525,9 @@ class SparkSession private( val plan = tracker.measurePhase(QueryPlanningTracker.PARSING) { val parsedPlan = sessionState.sqlParser.parsePlan(sqlText) if (args.nonEmpty) { - if (parsedPlan.isInstanceOf[CompoundBody]) { - // Positional parameters are not supported for SQL scripting. - throw SqlScriptingErrors.positionalParametersAreNotSupportedWithSqlScripting() - } - // Create a unified parameter query that can handle both positional and named parameters // The query itself will determine which type to use based on its parameter markers - UnifiedParameterizedQuery( + GeneralParameterizedQuery( parsedPlan, args.map(lit(_).expr).toImmutableArraySeq, paramNames.toImmutableArraySeq diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/execute-immediate.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/execute-immediate.sql.out index e5cdff605cdff..609dea79bf3fa 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/execute-immediate.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/execute-immediate.sql.out @@ -48,14 +48,14 @@ SetVariable [variablereference(system.session.sql_string=CAST(NULL AS STRING))] -- !query EXECUTE IMMEDIATE 'SET spark.sql.ansi.enabled=true' -- !query analysis -CommandResult [key#x, value#x], CommandResult [key#x, value#x] +CommandResult [key, value], data=[[spark.sql.ansi.enabled, true]] +- SetCommand (spark.sql.ansi.enabled,Some(true)) -- !query EXECUTE IMMEDIATE 'CREATE TEMPORARY VIEW IDENTIFIER(:tblName) AS SELECT id, name FROM tbl_view' USING 'tbl_view_tmp' as tblName -- !query analysis -CommandResult CommandResult +CommandResult [] +- CreateViewCommand `tbl_view_tmp`, SELECT id, name FROM tbl_view, false, false, LocalTempView, UNSUPPORTED, true +- Project [id#x, name#x] +- SubqueryAlias tbl_view @@ -75,7 +75,7 @@ LocalRelation [id#x, name#x] -- !query EXECUTE IMMEDIATE 'REFRESH TABLE IDENTIFIER(:tblName)' USING 'x' as tblName -- !query analysis -CommandResult CommandResult +CommandResult [] +- RefreshTableCommand `spark_catalog`.`default`.`x` @@ -147,7 +147,7 @@ LocalRelation [id#x, name#x, data#x] -- !query EXECUTE IMMEDIATE 'INSERT INTO x VALUES(?)' USING 1 -- !query analysis -CommandResult CommandResult +CommandResult [] +- InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/x, false, CSV, [path=file:[not included in comparison]/{warehouse_dir}/x], Append, `spark_catalog`.`default`.`x`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/x), [id] +- Project [col1#x AS id#x] +- LocalRelation [col1#x] @@ -217,7 +217,7 @@ LocalRelation [id#x, name#x, data#x, p#x] -- !query EXECUTE IMMEDIATE 'SET VAR sql_string = ?' USING 'SELECT id from tbl_view where name = :first' -- !query analysis -CommandResult CommandResult +CommandResult [] +- SetVariable [variablereference(system.session.sql_string='SELECT * from tbl_view where name = :first or id = :second')] +- Project [SELECT id from tbl_view where name = :first AS sql_string#x] +- OneRowRelation From 37e927f272c64404c76d7d204f10e8db0b002cb3 Mon Sep 17 00:00:00 2001 From: Serge Rielau Date: Fri, 5 Sep 2025 09:51:00 -0700 Subject: [PATCH 32/32] code review --- .../catalyst/analysis/executeImmediate.scala | 86 ++++++++++--------- .../analysis/ExecuteImmediateCommands.scala | 21 +++-- .../spark/sql/classic/SparkSession.scala | 2 +- 3 files changed, 57 insertions(+), 52 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala index 5a89f35a89cf0..9016ffa9a1567 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/executeImmediate.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.analysis.UnresolvedPlanId import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Expression, InSubquery, SubqueryExpression, VariableReference} import org.apache.spark.sql.catalyst.plans.logical.{ExecutableDuringAnalysis, LocalRelation, LogicalPlan, SetVariable, UnaryNode} import org.apache.spark.sql.catalyst.rules.Rule @@ -90,47 +91,47 @@ class ResolveExecuteImmediate( plan.resolveOperatorsWithPruning(_.containsPattern(EXECUTE_IMMEDIATE), ruleId) { case node @ UnresolvedExecuteImmediate(sqlStmtStr, args, targetVariables) => if (sqlStmtStr.resolved && targetVariables.forall(_.resolved) && args.forall(_.resolved)) { - // Validate that expressions don't contain unsupported constructs (like subqueries) - validateExpressions(args :+ sqlStmtStr) - - // All resolved - transform based on whether we have target variables - if (targetVariables.nonEmpty) { - // EXECUTE IMMEDIATE ... INTO should generate SetVariable plan - // At this point, all targetVariables are resolved, so we only expect VariableReference - // or Alias containing VariableReference - val finalTargetVars = targetVariables.map { - case alias: Alias => - // Extract the VariableReference from the alias - alias.child match { - case varRef: VariableReference => - // Use resolved VariableReference directly with canFold = false - varRef.copy(canFold = false) - case _ => - throw QueryCompilationErrors.unsupportedParameterExpression(alias.child) - } - case varRef: VariableReference => - // Use resolved VariableReference directly with canFold = false - varRef.copy(canFold = false) - case other => - throw QueryCompilationErrors.unsupportedParameterExpression(other) + // Validate expressions before transformation + validateExpressions(args :+ sqlStmtStr) + + // All resolved - transform based on whether we have target variables + if (targetVariables.nonEmpty) { + // EXECUTE IMMEDIATE ... INTO should generate SetVariable plan + // At this point, all targetVariables are resolved, so we only expect VariableReference + // or Alias containing VariableReference + val finalTargetVars = targetVariables.map { + case alias: Alias => + // Extract the VariableReference from the alias + alias.child match { + case varRef: VariableReference => + // Use resolved VariableReference directly with canFold = false + varRef.copy(canFold = false) + case _ => + throw QueryCompilationErrors.unsupportedParameterExpression(alias.child) + } + case varRef: VariableReference => + // Use resolved VariableReference directly with canFold = false + varRef.copy(canFold = false) + case other => + throw QueryCompilationErrors.unsupportedParameterExpression(other) + } + + // Check for duplicate variable names (same logic as ResolveSetVariable) + val dups = finalTargetVars.groupBy(_.identifier).filter(kv => kv._2.length > 1) + if (dups.nonEmpty) { + throw new AnalysisException( + errorClass = "DUPLICATE_ASSIGNMENTS", + messageParameters = Map("nameList" -> + dups.keys.map(key => toSQLId(key.name())).mkString(", "))) + } + + // Create SetVariable plan with the execute immediate query as source + val sourceQuery = ExecuteImmediateCommand(sqlStmtStr, args, hasIntoClause = true) + SetVariable(finalTargetVars, sourceQuery) + } else { + // Regular EXECUTE IMMEDIATE without INTO + ExecuteImmediateCommand(sqlStmtStr, args, hasIntoClause = false) } - - // Check for duplicate variable names (same logic as ResolveSetVariable) - val dups = finalTargetVars.groupBy(_.identifier).filter(kv => kv._2.length > 1) - if (dups.nonEmpty) { - throw new AnalysisException( - errorClass = "DUPLICATE_ASSIGNMENTS", - messageParameters = Map("nameList" -> - dups.keys.map(key => toSQLId(key.name())).mkString(", "))) - } - - // Create SetVariable plan with the execute immediate query as source - val sourceQuery = ExecuteImmediateCommand(sqlStmtStr, args, hasIntoClause = true) - SetVariable(finalTargetVars, sourceQuery) - } else { - // Regular EXECUTE IMMEDIATE without INTO - ExecuteImmediateCommand(sqlStmtStr, args, hasIntoClause = false) - } } else { // Not all resolved yet - wait for next iteration node @@ -140,16 +141,21 @@ class ResolveExecuteImmediate( /** * Validates that expressions don't contain unsupported constructs like subqueries. * Variable references and expressions like string concatenation are allowed. + * This validation catches both resolved and unresolved subqueries. */ private def validateExpressions(expressions: Seq[Expression]): Unit = { expressions.foreach { expr => // Check the expression and its children for unsupported constructs expr.foreach { case subquery: SubqueryExpression => + // Resolved subqueries (ScalarSubquery, ListQuery, etc.) throw QueryCompilationErrors.unsupportedParameterExpression(subquery) case inSubquery: InSubquery => // InSubquery doesn't extend SubqueryExpression directly but contains a subquery throw QueryCompilationErrors.unsupportedParameterExpression(inSubquery) + case unresolvedPlanId: UnresolvedPlanId => + // Unresolved subqueries (UnresolvedScalarSubqueryPlanId, etc.) + throw QueryCompilationErrors.unsupportedParameterExpression(unresolvedPlanId) case _ => // Other expressions including variables and concatenations are fine } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediateCommands.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediateCommands.scala index b131519401a23..b94b428a22bfa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediateCommands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ExecuteImmediateCommands.scala @@ -40,22 +40,22 @@ case class ExecuteImmediateCommands(sparkSession: SparkSession) extends Rule[Log } private def executeImmediate(cmd: ExecuteImmediateCommand): LogicalPlan = { - // Extract the query string from the queryParam expression + // Extract the query string from the queryParam expression val sqlStmtStr = extractQueryString(cmd.sqlStmtStr) - // Parse and validate the query + // Parse and validate the query val parsedPlan = sparkSession.sessionState.sqlParser.parsePlan(sqlStmtStr) validateQuery(sqlStmtStr, parsedPlan) // Execute the query recursively with isolated local variable context - val result = if (cmd.args.isEmpty) { - // No parameters - execute directly + val result = if (cmd.args.isEmpty) { + // No parameters - execute directly withIsolatedLocalVariableContext { sparkSession.sql(sqlStmtStr) - } - } else { - // For parameterized queries, build unified parameter arrays - val (paramValues, paramNames) = buildUnifiedParameters(cmd.args) + } + } else { + // For parameterized queries, build parameter arrays + val (paramValues, paramNames) = buildUnifiedParameters(cmd.args) withIsolatedLocalVariableContext { sparkSession.asInstanceOf[org.apache.spark.sql.classic.SparkSession] @@ -121,7 +121,7 @@ case class ExecuteImmediateCommands(sparkSession: SparkSession) extends Rule[Log } /** - * Builds unified parameter arrays for the sql() API. + * Builds parameter arrays for the sql() API. */ private def buildUnifiedParameters(args: Seq[Expression]): (Array[Any], Array[String]) = { val values = scala.collection.mutable.ListBuffer[Any]() @@ -148,7 +148,7 @@ case class ExecuteImmediateCommands(sparkSession: SparkSession) extends Rule[Log */ private def evaluateParameterExpression(expr: Expression): Any = { expr match { - case varRef: VariableReference => + case varRef: VariableReference => // Variable references should be evaluated to their values varRef.eval(EmptyRow) case foldable if foldable.foldable => @@ -159,7 +159,6 @@ case class ExecuteImmediateCommands(sparkSession: SparkSession) extends Rule[Log } } - /** * Temporarily isolates the SQL scripting context during EXECUTE IMMEDIATE execution. * This makes withinSqlScript() return false, ensuring that statements within EXECUTE IMMEDIATE diff --git a/sql/core/src/main/scala/org/apache/spark/sql/classic/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/classic/SparkSession.scala index a8e18a9cce165..c33f930e5a013 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/classic/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/classic/SparkSession.scala @@ -525,7 +525,7 @@ class SparkSession private( val plan = tracker.measurePhase(QueryPlanningTracker.PARSING) { val parsedPlan = sessionState.sqlParser.parsePlan(sqlText) if (args.nonEmpty) { - // Create a unified parameter query that can handle both positional and named parameters + // Create a general parameter query that can handle both positional and named parameters // The query itself will determine which type to use based on its parameter markers GeneralParameterizedQuery( parsedPlan,