Skip to content

Commit 239d77b

Browse files
momcilomrk-dbcloud-fan
authored andcommitted
[SPARK-48338][SQL] Check variable declarations
### What changes were proposed in this pull request? Checking wether variable declaration is only at the beginning of the BEGIN END block. ### Why are the changes needed? SQL standard states that the variables can be declared only immediately after BEGIN. ### Does this PR introduce _any_ user-facing change? Users will get an error if they try to declare variable in the scope that is not started with BEGIN and ended with END or if the declarations are not immediately after BEGIN. ### How was this patch tested? Tests are in SqlScriptingParserSuite. There are 2 tests for now, if declarations are correctly written and if declarations are not written immediately after BEGIN. There is a TODO to write the test if declaration is located in the scope that is not BEGIN END. ### Was this patch authored or co-authored using generative AI tooling? No Closes #47404 from momcilomrk-db/check_variable_declarations. Authored-by: Momcilo Mrkaic <momcilo.mrkaic@databricks.com> Signed-off-by: Wenchen Fan <wenchen@databricks.com>
1 parent 4de4ed1 commit 239d77b

File tree

5 files changed

+106
-6
lines changed

5 files changed

+106
-6
lines changed

common/utils/src/main/resources/error/error-conditions.json

Lines changed: 18 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -2941,6 +2941,24 @@
29412941
],
29422942
"sqlState" : "22029"
29432943
},
2944+
"INVALID_VARIABLE_DECLARATION" : {
2945+
"message" : [
2946+
"Invalid variable declaration."
2947+
],
2948+
"subClass" : {
2949+
"NOT_ALLOWED_IN_SCOPE" : {
2950+
"message" : [
2951+
"Variable <varName> was declared on line <lineNumber>, which is not allowed in this scope."
2952+
]
2953+
},
2954+
"ONLY_AT_BEGINNING" : {
2955+
"message" : [
2956+
"Variable <varName> can only be declared at the beginning of the compound, but it was declared on line <lineNumber>."
2957+
]
2958+
}
2959+
},
2960+
"sqlState" : "42K0M"
2961+
},
29442962
"INVALID_VARIABLE_TYPE_FOR_QUERY_EXECUTE_IMMEDIATE" : {
29452963
"message" : [
29462964
"Variable type must be string type but got <varType>."

common/utils/src/main/resources/error/error-states.json

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -4619,6 +4619,12 @@
46194619
"standard": "N",
46204620
"usedBy": ["Spark"]
46214621
},
4622+
"42K0M": {
4623+
"description": "Invalid variable declaration.",
4624+
"origin": "Spark,",
4625+
"standard": "N",
4626+
"usedBy": ["Spark"]
4627+
},
46224628
"42KD0": {
46234629
"description": "Ambiguous name reference.",
46244630
"origin": "Databricks",

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala

Lines changed: 36 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -48,8 +48,7 @@ import org.apache.spark.sql.catalyst.util.DateTimeUtils.{convertSpecialDate, con
4848
import org.apache.spark.sql.connector.catalog.{CatalogV2Util, SupportsNamespaces, TableCatalog}
4949
import org.apache.spark.sql.connector.catalog.TableChange.ColumnPosition
5050
import org.apache.spark.sql.connector.expressions.{ApplyTransform, BucketTransform, DaysTransform, Expression => V2Expression, FieldReference, HoursTransform, IdentityTransform, LiteralValue, MonthsTransform, Transform, YearsTransform}
51-
import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryParsingErrors, SqlScriptingErrors}
52-
import org.apache.spark.sql.errors.DataTypeErrors.toSQLStmt
51+
import org.apache.spark.sql.errors.{DataTypeErrorsBase, QueryCompilationErrors, QueryParsingErrors, SqlScriptingErrors}
5352
import org.apache.spark.sql.internal.SQLConf
5453
import org.apache.spark.sql.internal.SQLConf.LEGACY_BANG_EQUALS_NOT
5554
import org.apache.spark.sql.types._
@@ -62,7 +61,8 @@ import org.apache.spark.util.random.RandomSampler
6261
* The AstBuilder converts an ANTLR4 ParseTree into a catalyst Expression, LogicalPlan or
6362
* TableIdentifier.
6463
*/
65-
class AstBuilder extends DataTypeAstBuilder with SQLConfHelper with Logging {
64+
class AstBuilder extends DataTypeAstBuilder with SQLConfHelper
65+
with Logging with DataTypeErrorsBase {
6666
import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
6767
import ParserUtils._
6868

@@ -133,12 +133,42 @@ class AstBuilder extends DataTypeAstBuilder with SQLConfHelper with Logging {
133133

134134
private def visitCompoundBodyImpl(
135135
ctx: CompoundBodyContext,
136-
label: Option[String]): CompoundBody = {
136+
label: Option[String],
137+
allowVarDeclare: Boolean): CompoundBody = {
137138
val buff = ListBuffer[CompoundPlanStatement]()
138139
ctx.compoundStatements.forEach(compoundStatement => {
139140
buff += visit(compoundStatement).asInstanceOf[CompoundPlanStatement]
140141
})
141142

143+
val compoundStatements = buff.toList
144+
145+
val candidates = if (allowVarDeclare) {
146+
compoundStatements.dropWhile {
147+
case SingleStatement(_: CreateVariable) => true
148+
case _ => false
149+
}
150+
} else {
151+
compoundStatements
152+
}
153+
154+
val declareVarStatement = candidates.collectFirst {
155+
case SingleStatement(c: CreateVariable) => c
156+
}
157+
158+
declareVarStatement match {
159+
case Some(c: CreateVariable) =>
160+
if (allowVarDeclare) {
161+
throw SqlScriptingErrors.variableDeclarationOnlyAtBeginning(
162+
toSQLId(c.name.asInstanceOf[UnresolvedIdentifier].nameParts),
163+
c.origin.line.get.toString)
164+
} else {
165+
throw SqlScriptingErrors.variableDeclarationNotAllowedInScope(
166+
toSQLId(c.name.asInstanceOf[UnresolvedIdentifier].nameParts),
167+
c.origin.line.get.toString)
168+
}
169+
case _ =>
170+
}
171+
142172
CompoundBody(buff.toSeq, label)
143173
}
144174

@@ -161,11 +191,11 @@ class AstBuilder extends DataTypeAstBuilder with SQLConfHelper with Logging {
161191
val labelText = beginLabelCtx.
162192
map(_.multipartIdentifier().getText).getOrElse(java.util.UUID.randomUUID.toString).
163193
toLowerCase(Locale.ROOT)
164-
visitCompoundBodyImpl(ctx.compoundBody(), Some(labelText))
194+
visitCompoundBodyImpl(ctx.compoundBody(), Some(labelText), allowVarDeclare = true)
165195
}
166196

167197
override def visitCompoundBody(ctx: CompoundBodyContext): CompoundBody = {
168-
visitCompoundBodyImpl(ctx, None)
198+
visitCompoundBodyImpl(ctx, None, allowVarDeclare = false)
169199
}
170200

171201
override def visitCompoundStatement(ctx: CompoundStatementContext): CompoundPlanStatement =

sql/catalyst/src/main/scala/org/apache/spark/sql/errors/SqlScriptingErrors.scala

Lines changed: 14 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -39,4 +39,18 @@ private[sql] object SqlScriptingErrors extends QueryErrorsBase {
3939
messageParameters = Map("endLabel" -> endLabel))
4040
}
4141

42+
def variableDeclarationNotAllowedInScope(varName: String, lineNumber: String): Throwable = {
43+
new SparkException(
44+
errorClass = "INVALID_VARIABLE_DECLARATION.NOT_ALLOWED_IN_SCOPE",
45+
cause = null,
46+
messageParameters = Map("varName" -> varName, "lineNumber" -> lineNumber))
47+
}
48+
49+
def variableDeclarationOnlyAtBeginning(varName: String, lineNumber: String): Throwable = {
50+
new SparkException(
51+
errorClass = "INVALID_VARIABLE_DECLARATION.ONLY_AT_BEGINNING",
52+
cause = null,
53+
messageParameters = Map("varName" -> varName, "lineNumber" -> lineNumber))
54+
}
55+
4256
}

sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/SqlScriptingParserSuite.scala

Lines changed: 32 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -19,6 +19,7 @@ package org.apache.spark.sql.catalyst.parser
1919

2020
import org.apache.spark.{SparkException, SparkFunSuite}
2121
import org.apache.spark.sql.catalyst.plans.SQLHelper
22+
import org.apache.spark.sql.catalyst.plans.logical.CreateVariable
2223

2324
class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper {
2425
import CatalystSqlParser._
@@ -263,6 +264,37 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper {
263264
assert(tree.label.nonEmpty)
264265
}
265266

267+
test("declare at the beginning") {
268+
val sqlScriptText =
269+
"""
270+
|BEGIN
271+
| DECLARE testVariable1 VARCHAR(50);
272+
| DECLARE testVariable2 INTEGER;
273+
|END""".stripMargin
274+
val tree = parseScript(sqlScriptText)
275+
assert(tree.collection.length == 2)
276+
assert(tree.collection.forall(_.isInstanceOf[SingleStatement]))
277+
assert(tree.collection.forall(
278+
_.asInstanceOf[SingleStatement].parsedPlan.isInstanceOf[CreateVariable]))
279+
}
280+
281+
test("declare after beginning") {
282+
val sqlScriptText =
283+
"""
284+
|BEGIN
285+
| SELECT 1;
286+
| DECLARE testVariable INTEGER;
287+
|END""".stripMargin
288+
checkError(
289+
exception = intercept[SparkException] {
290+
parseScript(sqlScriptText)
291+
},
292+
errorClass = "INVALID_VARIABLE_DECLARATION.ONLY_AT_BEGINNING",
293+
parameters = Map("varName" -> "`testVariable`", "lineNumber" -> "4"))
294+
}
295+
296+
// TODO Add test for INVALID_VARIABLE_DECLARATION.NOT_ALLOWED_IN_SCOPE exception
297+
266298
test("SET VAR statement test") {
267299
val sqlScriptText =
268300
"""

0 commit comments

Comments
 (0)