This is an automated email from the ASF dual-hosted git repository.
wenchen pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git
The following commit(s) were added to refs/heads/master by this push:
new 9898e9d3e4a5 [SPARK-48342][SQL] Introduction of SQL Scripting Parser
9898e9d3e4a5 is described below
commit 9898e9d3e4a5cf805652b1aa06e38d702249f168
Author: David Milicevic <[email protected]>
AuthorDate: Tue Jun 18 15:55:27 2024 +0800
[SPARK-48342][SQL] Introduction of SQL Scripting Parser
### What changes were proposed in this pull request?
This PR proposes changes to SQL parser to introduce support for SQL
scripting statements:
- Introduces `BEGIN` keyword to the lexer.
- Changes grammar to support compound statements.
- Exposes new `parseScript` method in `ParserInterface`.
- Implements visitor functions for new nodes in the syntax tree in
`AstBuilder`.
- Introduces new logical operators in `SqlScriptingLogicalOperators` for
compound statements that are created by visiting functions and that will be
used during interpretation phase.
In order to simplify the process, in this PR we only introduce the support
for compound statements to the SQL parser.
Follow-up PRs will introduce interpreter, further statements, support for
exceptions thrown from parser/interpreter, etc.
More details can be found in [Jira
item](https://issues.apache.org/jira/browse/SPARK-48342) for this task and its
parent (where the design doc is uploaded as well).
### Why are the changes needed?
The intent is to add support for SQL scripting (and stored procedures down
the line). It gives users the ability to develop complex logic and ETL entirely
in SQL.
Until now, users had to write verbose SQL statements or combine SQL +
Python to efficiently write the logic. This is an effort to breach that gap and
enable complex logic to be written entirely in SQL.
### Does this PR introduce _any_ user-facing change?
No.
This PR is a first in series of PRs that will introduce changes to `sql()`
API to add support for SQL scripting, but for now, the API remains unchanged.
In the future, the API will remain the same as well, but it will have new
possibility to execute SQL scripts.
### How was this patch tested?
There are tests in `SqlScriptingParserSuite` that test the newly introduced
parser changes.
### Was this patch authored or co-authored using generative AI tooling?
No.
Closes #46665 from davidm-db/sql_batch_parser.
Lead-authored-by: David Milicevic <[email protected]>
Co-authored-by: David Milicevic
<[email protected]>
Signed-off-by: Wenchen Fan <[email protected]>
---
.../SparkConnectWithSessionExtensionSuite.scala | 5 +-
docs/sql-ref-ansi-compliance.md | 1 +
.../spark/sql/catalyst/parser/SqlBaseLexer.g4 | 1 +
.../spark/sql/catalyst/parser/SqlBaseParser.g4 | 24 +++
.../sql/catalyst/parser/AbstractSqlParser.scala | 13 ++
.../spark/sql/catalyst/parser/AstBuilder.scala | 42 ++++-
.../sql/catalyst/parser/ParserInterface.scala | 6 +
.../parser/SqlScriptingLogicalOperators.scala | 52 +++++++
.../catalyst/parser/SqlScriptingParserSuite.scala | 172 +++++++++++++++++++++
.../sql-tests/results/ansi/keywords.sql.out | 1 +
.../resources/sql-tests/results/keywords.sql.out | 1 +
.../spark/sql/SparkSessionExtensionSuite.scala | 5 +-
.../ThriftServerWithSparkContextSuite.scala | 2 +-
13 files changed, 321 insertions(+), 4 deletions(-)
diff --git
a/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectWithSessionExtensionSuite.scala
b/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectWithSessionExtensionSuite.scala
index 37c7fe25097c..c234b4f068bc 100644
---
a/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectWithSessionExtensionSuite.scala
+++
b/connector/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectWithSessionExtensionSuite.scala
@@ -23,7 +23,7 @@ import org.apache.spark.sql._
import org.apache.spark.sql.catalyst._
import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
import org.apache.spark.sql.catalyst.expressions.Expression
-import org.apache.spark.sql.catalyst.parser.ParserInterface
+import org.apache.spark.sql.catalyst.parser.{CompoundBody, ParserInterface}
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.connect.service.SessionHolder
import org.apache.spark.sql.types.{DataType, StructType}
@@ -54,6 +54,9 @@ class SparkConnectWithSessionExtensionSuite extends
SparkFunSuite {
override def parseQuery(sqlText: String): LogicalPlan =
delegate.parseQuery(sqlText)
+
+ override def parseScript(sqlScriptText: String): CompoundBody =
+ delegate.parseScript(sqlScriptText)
}
test("Parse table name with test parser") {
diff --git a/docs/sql-ref-ansi-compliance.md b/docs/sql-ref-ansi-compliance.md
index 587b1cc366bb..40d7d7dd4003 100644
--- a/docs/sql-ref-ansi-compliance.md
+++ b/docs/sql-ref-ansi-compliance.md
@@ -414,6 +414,7 @@ Below is a list of all the keywords in Spark SQL.
|ASC|non-reserved|non-reserved|non-reserved|
|AT|non-reserved|non-reserved|reserved|
|AUTHORIZATION|reserved|non-reserved|reserved|
+|BEGIN|non-reserved|non-reserved|non-reserved|
|BETWEEN|non-reserved|non-reserved|reserved|
|BIGINT|non-reserved|non-reserved|reserved|
|BINARY|non-reserved|non-reserved|reserved|
diff --git
a/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseLexer.g4
b/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseLexer.g4
index a9705c1733df..5753b153de30 100644
---
a/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseLexer.g4
+++
b/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseLexer.g4
@@ -134,6 +134,7 @@ AS: 'AS';
ASC: 'ASC';
AT: 'AT';
AUTHORIZATION: 'AUTHORIZATION';
+BEGIN: 'BEGIN';
BETWEEN: 'BETWEEN';
BIGINT: 'BIGINT';
BINARY: 'BINARY';
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 4552c17e0cf1..37671fc735c3 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
@@ -42,6 +42,28 @@ options { tokenVocab = SqlBaseLexer; }
public boolean double_quoted_identifiers = false;
}
+compoundOrSingleStatement
+ : singleStatement
+ | singleCompoundStatement
+ ;
+
+singleCompoundStatement
+ : beginEndCompoundBlock SEMICOLON? EOF
+ ;
+
+beginEndCompoundBlock
+ : BEGIN compoundBody END
+ ;
+
+compoundBody
+ : (compoundStatements+=compoundStatement SEMICOLON)*
+ ;
+
+compoundStatement
+ : statement
+ | beginEndCompoundBlock
+ ;
+
singleStatement
: statement SEMICOLON* EOF
;
@@ -1360,6 +1382,7 @@ ansiNonReserved
| ARRAY
| ASC
| AT
+ | BEGIN
| BETWEEN
| BIGINT
| BINARY
@@ -1662,6 +1685,7 @@ nonReserved
| ASC
| AT
| AUTHORIZATION
+ | BEGIN
| BETWEEN
| BIGINT
| BINARY
diff --git
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AbstractSqlParser.scala
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AbstractSqlParser.scala
index 96b9b9006c9c..1c477964a689 100644
---
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AbstractSqlParser.scala
+++
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AbstractSqlParser.scala
@@ -91,6 +91,19 @@ abstract class AbstractSqlParser extends AbstractParser with
ParserInterface {
}
}
+ /** Creates [[CompoundBody]] for a given SQL script string. */
+ override def parseScript(sqlScriptText: String): CompoundBody =
parse(sqlScriptText) { parser =>
+ val ctx = parser.compoundOrSingleStatement()
+ withErrorHandling(ctx, Some(sqlScriptText)) {
+ astBuilder.visitCompoundOrSingleStatement(ctx) match {
+ case body: CompoundBody => body
+ case _ =>
+ val position = Origin(None, None)
+ throw QueryParsingErrors.sqlStatementUnsupportedError(sqlScriptText,
position)
+ }
+ }
+ }
+
def withErrorHandling[T](ctx: ParserRuleContext, sqlText:
Option[String])(toResult: => T): T = {
withOrigin(ctx, sqlText) {
try {
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 243ff9e8a6a4..f716d632df8a 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
@@ -20,7 +20,8 @@ package org.apache.spark.sql.catalyst.parser
import java.util.Locale
import java.util.concurrent.TimeUnit
-import scala.collection.mutable.{ArrayBuffer, Set}
+import scala.collection.immutable.Seq
+import scala.collection.mutable.{ArrayBuffer, ListBuffer, Set}
import scala.jdk.CollectionConverters._
import scala.util.{Left, Right}
@@ -115,6 +116,45 @@ class AstBuilder extends DataTypeAstBuilder with
SQLConfHelper with Logging {
}
}
+ override def visitCompoundOrSingleStatement(
+ ctx: CompoundOrSingleStatementContext): CompoundBody = withOrigin(ctx) {
+ Option(ctx.singleCompoundStatement()).map { s =>
+ visit(s).asInstanceOf[CompoundBody]
+ }.getOrElse {
+ val logicalPlan = visitSingleStatement(ctx.singleStatement())
+ CompoundBody(Seq(SingleStatement(parsedPlan = logicalPlan)))
+ }
+ }
+
+ override def visitSingleCompoundStatement(ctx:
SingleCompoundStatementContext): CompoundBody = {
+ visit(ctx.beginEndCompoundBlock()).asInstanceOf[CompoundBody]
+ }
+
+ private def visitCompoundBodyImpl(ctx: CompoundBodyContext): CompoundBody = {
+ val buff = ListBuffer[CompoundPlanStatement]()
+ ctx.compoundStatements.forEach(compoundStatement => {
+ buff += visit(compoundStatement).asInstanceOf[CompoundPlanStatement]
+ })
+ CompoundBody(buff.toSeq)
+ }
+
+ override def visitBeginEndCompoundBlock(ctx: BeginEndCompoundBlockContext):
CompoundBody = {
+ visitCompoundBodyImpl(ctx.compoundBody())
+ }
+
+ override def visitCompoundBody(ctx: CompoundBodyContext): CompoundBody = {
+ visitCompoundBodyImpl(ctx)
+ }
+
+ override def visitCompoundStatement(ctx: CompoundStatementContext):
CompoundPlanStatement =
+ withOrigin(ctx) {
+ Option(ctx.statement()).map {s =>
+ SingleStatement(parsedPlan = visit(s).asInstanceOf[LogicalPlan])
+ }.getOrElse {
+ visit(ctx.beginEndCompoundBlock()).asInstanceOf[CompoundPlanStatement]
+ }
+ }
+
override def visitSingleStatement(ctx: SingleStatementContext): LogicalPlan
= withOrigin(ctx) {
visit(ctx.statement).asInstanceOf[LogicalPlan]
}
diff --git
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserInterface.scala
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserInterface.scala
index 3aec1dd43113..04edb0f75c46 100644
---
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserInterface.scala
+++
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserInterface.scala
@@ -62,4 +62,10 @@ trait ParserInterface extends DataTypeParserInterface {
*/
@throws[ParseException]("Text cannot be parsed to a LogicalPlan")
def parseQuery(sqlText: String): LogicalPlan
+
+ /**
+ * Parse a SQL script string to a [[CompoundBody]].
+ */
+ @throws[ParseException]("Text cannot be parsed to a CompoundBody")
+ def parseScript(sqlScriptText: String): CompoundBody
}
diff --git
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/SqlScriptingLogicalOperators.scala
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/SqlScriptingLogicalOperators.scala
new file mode 100644
index 000000000000..816ef82a3d8e
--- /dev/null
+++
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/SqlScriptingLogicalOperators.scala
@@ -0,0 +1,52 @@
+/*
+ * 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.parser
+
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.trees.{CurrentOrigin, Origin, WithOrigin}
+
+/**
+ * Trait for all SQL Scripting logical operators that are product of parsing
phase.
+ * These operators will be used by the SQL Scripting interpreter to generate
execution nodes.
+ */
+sealed trait CompoundPlanStatement
+
+/**
+ * Logical operator representing result of parsing a single SQL statement
+ * that is supposed to be executed against Spark.
+ * @param parsedPlan Result of SQL statement parsing.
+ */
+case class SingleStatement(parsedPlan: LogicalPlan)
+ extends CompoundPlanStatement
+ with WithOrigin {
+
+ override val origin: Origin = CurrentOrigin.get
+
+ def getText(sqlScriptText: String): String = {
+ if (origin.startIndex.isEmpty || origin.stopIndex.isEmpty) {
+ return null
+ }
+ sqlScriptText.substring(origin.startIndex.get, origin.stopIndex.get + 1)
+ }
+}
+
+/**
+ * Logical operator for a compound body. Contains all statements within the
compound body.
+ * @param collection Collection of statements within the compound body.
+ */
+case class CompoundBody(collection: Seq[CompoundPlanStatement]) extends
CompoundPlanStatement
diff --git
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/SqlScriptingParserSuite.scala
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/SqlScriptingParserSuite.scala
new file mode 100644
index 000000000000..657e4b2232ee
--- /dev/null
+++
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/SqlScriptingParserSuite.scala
@@ -0,0 +1,172 @@
+/*
+ * 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.parser
+
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.sql.catalyst.plans.SQLHelper
+
+class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper {
+ import CatalystSqlParser._
+
+ test("single select") {
+ val sqlScriptText = "SELECT 1;"
+ val tree = parseScript(sqlScriptText)
+ assert(tree.collection.length == 1)
+ assert(tree.collection.head.isInstanceOf[SingleStatement])
+ val sparkStatement = tree.collection.head.asInstanceOf[SingleStatement]
+ assert(sparkStatement.getText(sqlScriptText) == "SELECT 1;")
+ }
+
+ test("single select without ;") {
+ val sqlScriptText = "SELECT 1"
+ val tree = parseScript(sqlScriptText)
+ assert(tree.collection.length == 1)
+ assert(tree.collection.head.isInstanceOf[SingleStatement])
+ val sparkStatement = tree.collection.head.asInstanceOf[SingleStatement]
+ assert(sparkStatement.getText(sqlScriptText) == "SELECT 1")
+ }
+
+ test("multi select without ; - should fail") {
+ val sqlScriptText = "SELECT 1 SELECT 1"
+ val e = intercept[ParseException] {
+ parseScript(sqlScriptText)
+ }
+ assert(e.getErrorClass === "PARSE_SYNTAX_ERROR")
+ assert(e.getMessage.contains("Syntax error"))
+ assert(e.getMessage.contains("SELECT 1 SELECT 1"))
+ }
+
+ test("multi select") {
+ val sqlScriptText = "BEGIN SELECT 1;SELECT 2; END"
+ val tree = parseScript(sqlScriptText)
+ assert(tree.collection.length == 2)
+ assert(tree.collection.forall(_.isInstanceOf[SingleStatement]))
+
+ sqlScriptText.split(";")
+ .map(cleanupStatementString)
+ .zip(tree.collection)
+ .foreach { case (expected, statement) =>
+ val sparkStatement = statement.asInstanceOf[SingleStatement]
+ val statementText = sparkStatement.getText(sqlScriptText)
+ assert(statementText == expected)
+ }
+ }
+
+ test("empty BEGIN END block") {
+ val sqlScriptText =
+ """
+ |BEGIN
+ |END""".stripMargin
+ val tree = parseScript(sqlScriptText)
+ assert(tree.collection.isEmpty)
+ }
+
+ test("multiple ; in row - should fail") {
+ val sqlScriptText =
+ """
+ |BEGIN
+ | SELECT 1;;
+ | SELECT 2;
+ |END""".stripMargin
+ val e = intercept[ParseException] {
+ parseScript(sqlScriptText)
+ }
+ assert(e.getErrorClass === "PARSE_SYNTAX_ERROR")
+ assert(e.getMessage.contains("Syntax error"))
+ assert(e.getMessage.contains("at or near ';'"))
+ }
+
+ test("without ; in last statement - should fail") {
+ val sqlScriptText =
+ """
+ |BEGIN
+ | SELECT 1;
+ | SELECT 2
+ |END""".stripMargin
+ val e = intercept[ParseException] {
+ parseScript(sqlScriptText)
+ }
+ assert(e.getErrorClass === "PARSE_SYNTAX_ERROR")
+ assert(e.getMessage.contains("Syntax error"))
+ assert(e.getMessage.contains("at or near end of input"))
+ }
+
+ test("multi statement") {
+ val sqlScriptText =
+ """
+ |BEGIN
+ | SELECT 1;
+ | SELECT 2;
+ | INSERT INTO A VALUES (a, b, 3);
+ | SELECT a, b, c FROM T;
+ | SELECT * FROM T;
+ |END""".stripMargin
+ val tree = parseScript(sqlScriptText)
+ assert(tree.collection.length == 5)
+ assert(tree.collection.forall(_.isInstanceOf[SingleStatement]))
+ sqlScriptText.split(";")
+ .map(cleanupStatementString)
+ .zip(tree.collection)
+ .foreach { case (expected, statement) =>
+ val sparkStatement = statement.asInstanceOf[SingleStatement]
+ val statementText = sparkStatement.getText(sqlScriptText)
+ assert(statementText == expected)
+ }
+ }
+
+ test("nested begin end") {
+ val sqlScriptText =
+ """
+ |BEGIN
+ | BEGIN
+ | SELECT 1;
+ | END;
+ | BEGIN
+ | BEGIN
+ | SELECT 2;
+ | SELECT 3;
+ | END;
+ | END;
+ |END""".stripMargin
+ val tree = parseScript(sqlScriptText)
+ assert(tree.collection.length == 2)
+ assert(tree.collection.head.isInstanceOf[CompoundBody])
+ val body1 = tree.collection.head.asInstanceOf[CompoundBody]
+ assert(body1.collection.length == 1)
+
assert(body1.collection.head.asInstanceOf[SingleStatement].getText(sqlScriptText)
+ == "SELECT 1")
+
+ val body2 = tree.collection(1).asInstanceOf[CompoundBody]
+ assert(body2.collection.length == 1)
+ assert(body2.collection.head.isInstanceOf[CompoundBody])
+ val nestedBody = body2.collection.head.asInstanceOf[CompoundBody]
+
assert(nestedBody.collection.head.asInstanceOf[SingleStatement].getText(sqlScriptText)
+ == "SELECT 2")
+
assert(nestedBody.collection(1).asInstanceOf[SingleStatement].getText(sqlScriptText)
+ == "SELECT 3")
+ }
+
+ // Helper methods
+ def cleanupStatementString(statementStr: String): String = {
+ statementStr
+ .replace("\n", "")
+ .replace("BEGIN", "")
+ .replace("END", "")
+ .trim
+ }
+}
diff --git
a/sql/core/src/test/resources/sql-tests/results/ansi/keywords.sql.out
b/sql/core/src/test/resources/sql-tests/results/ansi/keywords.sql.out
index 5c8cd52eb4ee..4e928562aa5d 100644
--- a/sql/core/src/test/resources/sql-tests/results/ansi/keywords.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/ansi/keywords.sql.out
@@ -20,6 +20,7 @@ AS true
ASC false
AT false
AUTHORIZATION true
+BEGIN false
BETWEEN false
BIGINT false
BINARY false
diff --git a/sql/core/src/test/resources/sql-tests/results/keywords.sql.out
b/sql/core/src/test/resources/sql-tests/results/keywords.sql.out
index 9138fcd552d8..e036c6620776 100644
--- a/sql/core/src/test/resources/sql-tests/results/keywords.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/keywords.sql.out
@@ -20,6 +20,7 @@ AS false
ASC false
AT false
AUTHORIZATION false
+BEGIN false
BETWEEN false
BIGINT false
BINARY false
diff --git
a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala
b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala
index 4d38e360f438..1f0033a0efcd 100644
---
a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala
+++
b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionExtensionSuite.scala
@@ -30,7 +30,7 @@ import org.apache.spark.sql.catalyst.catalog.BucketSpec
import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.expressions.aggregate.{Final, Partial}
-import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser,
ParserInterface}
+import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, CompoundBody,
ParserInterface}
import org.apache.spark.sql.catalyst.plans.SQLHelper
import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, Limit,
LocalRelation, LogicalPlan, Statistics, UnresolvedHint}
import org.apache.spark.sql.catalyst.plans.physical.{Partitioning,
SinglePartition}
@@ -581,6 +581,9 @@ case class MyParser(spark: SparkSession, delegate:
ParserInterface) extends Pars
override def parseQuery(sqlText: String): LogicalPlan =
delegate.parseQuery(sqlText)
+
+ override def parseScript(sqlScriptText: String): CompoundBody =
+ delegate.parseScript(sqlScriptText)
}
object MyExtensions {
diff --git
a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala
b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala
index 1b13c7c5f89a..43d3532ab78c 100644
---
a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala
+++
b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala
@@ -214,7 +214,7 @@ trait ThriftServerWithSparkContextSuite extends
SharedThriftServer {
val sessionHandle = client.openSession(user, "")
val infoValue = client.getInfo(sessionHandle,
GetInfoType.CLI_ODBC_KEYWORDS)
// scalastyle:off line.size.limit
- assert(infoValue.getStringValue ==
"ADD,AFTER,ALL,ALTER,ALWAYS,ANALYZE,AND,ANTI,ANY,ANY_VALUE,ARCHIVE,ARRAY,AS,ASC,AT,AUTHORIZATION,BETWEEN,BIGINT,BINARY,BINDING,BOOLEAN,BOTH,BUCKET,BUCKETS,BY,BYTE,CACHE,CASCADE,CASE,CAST,CATALOG,CATALOGS,CHANGE,CHAR,CHARACTER,CHECK,CLEAR,CLUSTER,CLUSTERED,CODEGEN,COLLATE,COLLATION,COLLECTION,COLUMN,COLUMNS,COMMENT,COMMIT,COMPACT,COMPACTIONS,COMPENSATION,COMPUTE,CONCATENATE,CONSTRAINT,COST,CREATE,CROSS,CUBE,CURRENT,CURRENT_DATE,CURRENT_TIME,CURRENT
[...]
+ assert(infoValue.getStringValue ==
"ADD,AFTER,ALL,ALTER,ALWAYS,ANALYZE,AND,ANTI,ANY,ANY_VALUE,ARCHIVE,ARRAY,AS,ASC,AT,AUTHORIZATION,BEGIN,BETWEEN,BIGINT,BINARY,BINDING,BOOLEAN,BOTH,BUCKET,BUCKETS,BY,BYTE,CACHE,CASCADE,CASE,CAST,CATALOG,CATALOGS,CHANGE,CHAR,CHARACTER,CHECK,CLEAR,CLUSTER,CLUSTERED,CODEGEN,COLLATE,COLLATION,COLLECTION,COLUMN,COLUMNS,COMMENT,COMMIT,COMPACT,COMPACTIONS,COMPENSATION,COMPUTE,CONCATENATE,CONSTRAINT,COST,CREATE,CROSS,CUBE,CURRENT,CURRENT_DATE,CURRENT_TIME,C
[...]
// scalastyle:on line.size.limit
}
}
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]