|
17 | 17 |
|
18 | 18 | package org.apache.spark.sql.catalyst.analysis
|
19 | 19 |
|
20 |
| -import scala.util.{Either, Left, Right} |
21 |
| - |
22 | 20 | import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Expression, VariableReference}
|
23 | 21 | import org.apache.spark.sql.catalyst.parser.ParseException
|
24 |
| -import org.apache.spark.sql.catalyst.plans.logical.{CompoundBody, ExecutableDuringAnalysis, LocalRelation, LogicalPlan, SetVariable, UnaryNode} |
| 22 | +import org.apache.spark.sql.catalyst.plans.logical.{ExecutableDuringAnalysis, LocalRelation, LogicalPlan, SetVariable, UnaryNode} |
25 | 23 | import org.apache.spark.sql.catalyst.rules.Rule
|
26 | 24 | import org.apache.spark.sql.catalyst.trees.TreePattern.{EXECUTE_IMMEDIATE, TreePattern}
|
27 | 25 | import org.apache.spark.sql.connector.catalog.CatalogManager
|
@@ -162,8 +160,6 @@ class ResolveExecuteImmediate(
|
162 | 160 | validateUsingClauseExpressions(args)
|
163 | 161 |
|
164 | 162 | if (queryParamResolved && allArgsResolved && targetVariablesResolved) {
|
165 |
| - // Validate parameter usage patterns |
166 |
| - validateParameterUsage(queryParam, args) |
167 | 163 | // All resolved - transform based on whether we have target variables
|
168 | 164 | if (targetVariables.nonEmpty) {
|
169 | 165 | // EXECUTE IMMEDIATE ... INTO should generate SetVariable plan
|
@@ -205,44 +201,6 @@ class ResolveExecuteImmediate(
|
205 | 201 | }
|
206 | 202 | }
|
207 | 203 |
|
208 |
| - private def parseStatement( |
209 |
| - queryString: String, |
210 |
| - targetVariables: Seq[Expression]): LogicalPlan = { |
211 |
| - // If targetVariables is defined, statement needs to be a query. |
212 |
| - // Otherwise, it can be anything. |
213 |
| - val plan = if (targetVariables.nonEmpty) { |
214 |
| - try { |
215 |
| - catalogManager.v1SessionCatalog.parser.parseQuery(queryString) |
216 |
| - } catch { |
217 |
| - case e: ParseException => |
218 |
| - // Since we do not have a way of telling that parseQuery failed because of |
219 |
| - // actual parsing error or because statement was passed where query was expected, |
220 |
| - // we need to make sure that parsePlan wouldn't throw |
221 |
| - catalogManager.v1SessionCatalog.parser.parsePlan(queryString) |
222 |
| - |
223 |
| - // Plan was successfully parsed, but query wasn't - throw. |
224 |
| - throw QueryCompilationErrors.invalidStatementForExecuteInto(queryString) |
225 |
| - } |
226 |
| - } else { |
227 |
| - catalogManager.v1SessionCatalog.parser.parsePlan(queryString) |
228 |
| - } |
229 |
| - |
230 |
| - if (plan.isInstanceOf[CompoundBody]) { |
231 |
| - throw QueryCompilationErrors.sqlScriptInExecuteImmediate(queryString) |
232 |
| - } |
233 |
| - |
234 |
| - plan |
235 |
| - } |
236 |
| - |
237 |
| - |
238 |
| - |
239 |
| - private def isQueryResolved(query: Either[String, UnresolvedAttribute]): Boolean = { |
240 |
| - query match { |
241 |
| - case Left(_) => true // String literals are always resolved |
242 |
| - case Right(attr) => attr.resolved // Check if the attribute is resolved |
243 |
| - } |
244 |
| - } |
245 |
| - |
246 | 204 | private def getVariableReference(expr: Expression, nameParts: Seq[String]): VariableReference = {
|
247 | 205 | variableResolution.lookupVariable(
|
248 | 206 | nameParts = nameParts,
|
@@ -302,40 +260,4 @@ class ResolveExecuteImmediate(
|
302 | 260 | }
|
303 | 261 | }
|
304 | 262 | }
|
305 |
| - |
306 |
| - private def validateParameterUsage(queryParam: Expression, args: Seq[Expression]): Unit = { |
307 |
| - // Extract the query string to validate parameter patterns |
308 |
| - val queryString = queryParam.eval(null) match { |
309 |
| - case null => return // Will be caught later by other validation |
310 |
| - case value => value.toString |
311 |
| - } |
312 |
| - |
313 |
| - // Check for positional and named parameter patterns in the query string |
314 |
| - val positionalParameterPattern = "\\?".r |
315 |
| - val namedParameterPattern = ":[a-zA-Z_][a-zA-Z0-9_]*".r |
316 |
| - val queryUsesPositionalParameters = |
317 |
| - positionalParameterPattern.findFirstIn(queryString).isDefined |
318 |
| - val queryUsesNamedParameters = |
319 |
| - namedParameterPattern.findFirstIn(queryString).isDefined |
320 |
| - |
321 |
| - // Error if query mixes positional and named parameters |
322 |
| - if (queryUsesPositionalParameters && queryUsesNamedParameters) { |
323 |
| - throw QueryCompilationErrors.invalidQueryMixedQueryParameters() |
324 |
| - } |
325 |
| - |
326 |
| - // If query uses only named parameters, ensure all USING expressions have names |
327 |
| - if (queryUsesNamedParameters && !queryUsesPositionalParameters) { |
328 |
| - val unnamedExpressions = args.zipWithIndex.collect { |
329 |
| - case (expr, index) if !hasName(expr) => expr |
330 |
| - } |
331 |
| - if (unnamedExpressions.nonEmpty) { |
332 |
| - throw QueryCompilationErrors.invalidQueryAllParametersMustBeNamed(unnamedExpressions) |
333 |
| - } |
334 |
| - } |
335 |
| - } |
336 |
| - |
337 |
| - private def hasName(expr: Expression): Boolean = expr match { |
338 |
| - case _: Alias => true |
339 |
| - case _ => false |
340 |
| - } |
341 | 263 | }
|
0 commit comments