Apache Spark - A unified analytics engine for large-scale data processing
Go to file
Kent Yao cdd8e51742 [SPARK-33419][SQL] Unexpected behavior when using SET commands before a query in SparkSession.sql
### What changes were proposed in this pull request?

SparkSession.sql converts a string value to a DataFrame, and the string value should be one single SQL statement ending up w/ or w/o one or more semicolons. e.g.

```sql
scala> spark.sql(" select 2").show
+---+
|  2|
+---+
|  2|
+---+
scala> spark.sql(" select 2;").show
+---+
|  2|
+---+
|  2|
+---+

scala> spark.sql(" select 2;;;;").show
+---+
|  2|
+---+
|  2|
+---+
```
If we put 2 or more statements in, it fails in the parser as expected, e.g.

```sql
scala> spark.sql(" select 2; select 1;").show
org.apache.spark.sql.catalyst.parser.ParseException:
extraneous input 'select' expecting {<EOF>, ';'}(line 1, pos 11)

== SQL ==
 select 2; select 1;
-----------^^^

  at org.apache.spark.sql.catalyst.parser.ParseException.withCommand(ParseDriver.scala:263)
  at org.apache.spark.sql.catalyst.parser.AbstractSqlParser.parse(ParseDriver.scala:130)
  at org.apache.spark.sql.execution.SparkSqlParser.parse(SparkSqlParser.scala:51)
  at org.apache.spark.sql.catalyst.parser.AbstractSqlParser.parsePlan(ParseDriver.scala:81)
  at org.apache.spark.sql.SparkSession.$anonfun$sql$2(SparkSession.scala:610)
  at org.apache.spark.sql.catalyst.QueryPlanningTracker.measurePhase(QueryPlanningTracker.scala:111)
  at org.apache.spark.sql.SparkSession.$anonfun$sql$1(SparkSession.scala:610)
  at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:769)
  at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:607)
  ... 47 elided
```

As a very generic user scenario, users may want to change some settings before they execute
the queries. They may pass a string value like `set spark.sql.abc=2; select 1;` into this API, which creates a confusing gap between the actual effect and the user's expectations.

The user may want the query to be executed with spark.sql.abc=2, but Spark actually treats the whole part of `2; select 1;` as the value of the property 'spark.sql.abc',
e.g.

```
scala> spark.sql("set spark.sql.abc=2; select 1;").show
+-------------+------------+
|          key|       value|
+-------------+------------+
|spark.sql.abc|2; select 1;|
+-------------+------------+
```

What's more, the SET symbol could digest everything behind it, which makes it unstable from version to version, e.g.

#### 3.1
```sql
scala> spark.sql("set;").show
org.apache.spark.sql.catalyst.parser.ParseException:
Expected format is 'SET', 'SET key', or 'SET key=value'. If you want to include special characters in key, please use quotes, e.g., SET `ke y`=value.(line 1, pos 0)

== SQL ==
set;
^^^

  at org.apache.spark.sql.execution.SparkSqlAstBuilder.$anonfun$visitSetConfiguration$1(SparkSqlParser.scala:83)
  at org.apache.spark.sql.catalyst.parser.ParserUtils$.withOrigin(ParserUtils.scala:113)
  at org.apache.spark.sql.execution.SparkSqlAstBuilder.visitSetConfiguration(SparkSqlParser.scala:72)
  at org.apache.spark.sql.execution.SparkSqlAstBuilder.visitSetConfiguration(SparkSqlParser.scala:58)
  at org.apache.spark.sql.catalyst.parser.SqlBaseParser$SetConfigurationContext.accept(SqlBaseParser.java:2161)
  at org.antlr.v4.runtime.tree.AbstractParseTreeVisitor.visit(AbstractParseTreeVisitor.java:18)
  at org.apache.spark.sql.catalyst.parser.AstBuilder.$anonfun$visitSingleStatement$1(AstBuilder.scala:77)
  at org.apache.spark.sql.catalyst.parser.ParserUtils$.withOrigin(ParserUtils.scala:113)
  at org.apache.spark.sql.catalyst.parser.AstBuilder.visitSingleStatement(AstBuilder.scala:77)
  at org.apache.spark.sql.catalyst.parser.AbstractSqlParser.$anonfun$parsePlan$1(ParseDriver.scala:82)
  at org.apache.spark.sql.catalyst.parser.AbstractSqlParser.parse(ParseDriver.scala:113)
  at org.apache.spark.sql.execution.SparkSqlParser.parse(SparkSqlParser.scala:51)
  at org.apache.spark.sql.catalyst.parser.AbstractSqlParser.parsePlan(ParseDriver.scala:81)
  at org.apache.spark.sql.SparkSession.$anonfun$sql$2(SparkSession.scala:610)
  at org.apache.spark.sql.catalyst.QueryPlanningTracker.measurePhase(QueryPlanningTracker.scala:111)
  at org.apache.spark.sql.SparkSession.$anonfun$sql$1(SparkSession.scala:610)
  at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:769)
  at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:607)
  ... 47 elided

scala> spark.sql("set a;").show
org.apache.spark.sql.catalyst.parser.ParseException:
Expected format is 'SET', 'SET key', or 'SET key=value'. If you want to include special characters in key, please use quotes, e.g., SET `ke y`=value.(line 1, pos 0)

== SQL ==
set a;
^^^

  at org.apache.spark.sql.execution.SparkSqlAstBuilder.$anonfun$visitSetConfiguration$1(SparkSqlParser.scala:83)
  at org.apache.spark.sql.catalyst.parser.ParserUtils$.withOrigin(ParserUtils.scala:113)
  at org.apache.spark.sql.execution.SparkSqlAstBuilder.visitSetConfiguration(SparkSqlParser.scala:72)
  at org.apache.spark.sql.execution.SparkSqlAstBuilder.visitSetConfiguration(SparkSqlParser.scala:58)
  at org.apache.spark.sql.catalyst.parser.SqlBaseParser$SetConfigurationContext.accept(SqlBaseParser.java:2161)
  at org.antlr.v4.runtime.tree.AbstractParseTreeVisitor.visit(AbstractParseTreeVisitor.java:18)
  at org.apache.spark.sql.catalyst.parser.AstBuilder.$anonfun$visitSingleStatement$1(AstBuilder.scala:77)
  at org.apache.spark.sql.catalyst.parser.ParserUtils$.withOrigin(ParserUtils.scala:113)
  at org.apache.spark.sql.catalyst.parser.AstBuilder.visitSingleStatement(AstBuilder.scala:77)
  at org.apache.spark.sql.catalyst.parser.AbstractSqlParser.$anonfun$parsePlan$1(ParseDriver.scala:82)
  at org.apache.spark.sql.catalyst.parser.AbstractSqlParser.parse(ParseDriver.scala:113)
  at org.apache.spark.sql.execution.SparkSqlParser.parse(SparkSqlParser.scala:51)
  at org.apache.spark.sql.catalyst.parser.AbstractSqlParser.parsePlan(ParseDriver.scala:81)
  at org.apache.spark.sql.SparkSession.$anonfun$sql$2(SparkSession.scala:610)
  at org.apache.spark.sql.catalyst.QueryPlanningTracker.measurePhase(QueryPlanningTracker.scala:111)
  at org.apache.spark.sql.SparkSession.$anonfun$sql$1(SparkSession.scala:610)
  at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:769)
  at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:607)
  ... 47 elided
```

#### 2.4

```sql
scala> spark.sql("set;").show
+---+-----------+
|key|      value|
+---+-----------+
|  ;|<undefined>|
+---+-----------+

scala> spark.sql("set a;").show
+---+-----------+
|key|      value|
+---+-----------+
| a;|<undefined>|
+---+-----------+
```

In this PR,
1.  make `set spark.sql.abc=2; select 1;` in `SparkSession.sql` fail directly, user should call `.sql` for each statement separately.
2. make the semicolon as the separator of statements, and if users want to use it as part of the property value, shall use quotes too.

### Why are the changes needed?

1. disambiguation for  `SparkSession.sql`
2. make semicolon work same both w/ `SET` and other statements

### Does this PR introduce _any_ user-facing change?

yes,
the semicolon works as a separator of statements now, it will be trimmed if it is at the end of the statement and fail the statement if it is in the middle. you need to use quotes if you want it to be part of the property value

### How was this patch tested?

new tests

Closes #30332 from yaooqinn/SPARK-33419.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-11-13 06:58:16 +00:00
.github [SPARK-33439][INFRA] Use SERIAL_SBT_TESTS=1 for SQL modules 2020-11-12 21:19:51 -08:00
assembly [SPARK-30950][BUILD] Setting version to 3.1.0-SNAPSHOT 2020-02-25 19:44:31 -08:00
bin [SPARK-32839][WINDOWS] Make Spark scripts working with the spaces in paths on Windows 2020-09-14 13:15:14 +09:00
binder [SPARK-32204][SPARK-32182][DOCS] Add a quickstart page with Binder integration in PySpark documentation 2020-08-26 12:23:24 +09:00
build [SPARK-32998][BUILD] Add ability to override default remote repos with internal one 2020-10-22 16:35:55 -07:00
common [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode 2020-11-09 11:00:52 -06:00
conf [SPARK-32004][ALL] Drop references to slave 2020-07-13 14:05:33 -07:00
core [MINOR][DOC] spark.executor.memoryOverhead is not cluster-mode only 2020-11-12 18:53:06 +09:00
data [SPARK-22666][ML][SQL] Spark datasource for image format 2018-09-05 11:59:00 -07:00
dev [SPARK-33213][BUILD] Upgrade Apache Arrow to 2.0.0 2020-11-09 19:07:16 -08:00
docs [SPARK-33259][SS] Disable streaming query with possible correctness issue by default 2020-11-12 15:31:57 -08:00
examples [MINOR][GRAPHX] Correct typos in the sub-modules: graphx, external, and examples 2020-11-12 08:29:22 +09:00
external [MINOR][GRAPHX] Correct typos in the sub-modules: graphx, external, and examples 2020-11-12 08:29:22 +09:00
graphx [MINOR][GRAPHX] Correct typos in the sub-modules: graphx, external, and examples 2020-11-12 08:29:22 +09:00
hadoop-cloud [SPARK-33212][BUILD] Move to shaded clients for Hadoop 3.x profile 2020-10-22 03:21:34 +00:00
launcher [SPARK-33212][BUILD] Move to shaded clients for Hadoop 3.x profile 2020-10-22 03:21:34 +00:00
licenses [SPARK-32435][PYTHON] Remove heapq3 port from Python 3 2020-07-27 20:10:13 +09:00
licenses-binary [SPARK-32435][PYTHON] Remove heapq3 port from Python 3 2020-07-27 20:10:13 +09:00
mllib [SPARK-32907][ML][PYTHON] adaptively blockify instances - LinearSVC 2020-11-12 19:14:07 +08:00
mllib-local [SPARK-32907][ML] adaptively blockify instances - revert blockify gmm 2020-09-23 15:54:56 +08:00
project [SPARK-33365][BUILD] Update SBT to 1.4.2 2020-11-05 17:37:44 -08:00
python [SPARK-32907][ML][PYTHON] adaptively blockify instances - LinearSVC 2020-11-12 19:14:07 +08:00
R [SPARK-33363] Add prompt information related to the current task when pyspark/sparkR starts 2020-11-10 11:12:19 +09:00
repl [SPARK-30090][SHELL] Adapt Spark REPL to Scala 2.13 2020-09-12 18:15:15 -05:00
resource-managers [SPARK-33408][SPARK-32354][K8S][R] Use R 3.6.3 in K8s R image and re-enable RTestsSuite 2020-11-12 15:36:31 +09:00
sbin [MINOR][DOCS] fix typo for docs,log message and comments 2020-08-22 06:45:35 +09:00
sql [SPARK-33419][SQL] Unexpected behavior when using SET commands before a query in SparkSession.sql 2020-11-13 06:58:16 +00:00
streaming [SPARK-32850][CORE][K8S] Simplify the RPC message flow of decommission 2020-10-23 13:58:44 +09:00
tools [SPARK-21708][BUILD] Migrate build to sbt 1.x 2020-10-07 15:28:00 -07:00
.asf.yaml [SPARK-31352] Add .asf.yaml to control Github settings 2020-04-06 09:06:01 -05:00
.gitattributes [SPARK-30653][INFRA][SQL] EOL character enforcement for java/scala/xml/py/R files 2020-01-27 10:20:51 -08:00
.gitignore [SPARK-33269][INFRA] Ignore ".bsp/" directory in Git 2020-10-28 21:32:09 +09:00
.sbtopts [SPARK-21708][BUILD] Migrate build to sbt 1.x 2020-10-07 15:28:00 -07:00
appveyor.yml [SPARK-32647][INFRA] Report SparkR test results with JUnit reporter 2020-08-18 19:35:15 +09:00
CONTRIBUTING.md [MINOR][DOCS] Tighten up some key links to the project and download pages to use HTTPS 2019-05-21 10:56:42 -07:00
LICENSE [SPARK-32435][PYTHON] Remove heapq3 port from Python 3 2020-07-27 20:10:13 +09:00
LICENSE-binary [SPARK-32435][PYTHON] Remove heapq3 port from Python 3 2020-07-27 20:10:13 +09:00
NOTICE [SPARK-29674][CORE] Update dropwizard metrics to 4.1.x for JDK 9+ 2019-11-03 15:13:06 -08:00
NOTICE-binary [SPARK-29674][CORE] Update dropwizard metrics to 4.1.x for JDK 9+ 2019-11-03 15:13:06 -08:00
pom.xml [SPARK-33213][BUILD] Upgrade Apache Arrow to 2.0.0 2020-11-09 19:07:16 -08:00
README.md [MINOR][DOCS] Fix Jenkins build image and link in README.md 2020-01-20 23:08:24 -08:00
scalastyle-config.xml [SPARK-32539][INFRA] Disallow FileSystem.get(Configuration conf) in style check by default 2020-08-06 05:56:59 +00:00

Apache Spark

Spark is a unified analytics engine for large-scale data processing. It provides high-level APIs in Scala, Java, Python, and R, and an optimized engine that supports general computation graphs for data analysis. It also supports a rich set of higher-level tools including Spark SQL for SQL and DataFrames, MLlib for machine learning, GraphX for graph processing, and Structured Streaming for stream processing.

https://spark.apache.org/

Jenkins Build AppVeyor Build PySpark Coverage

Online Documentation

You can find the latest Spark documentation, including a programming guide, on the project web page. This README file only contains basic setup instructions.

Building Spark

Spark is built using Apache Maven. To build Spark and its example programs, run:

./build/mvn -DskipTests clean package

(You do not need to do this if you downloaded a pre-built package.)

More detailed documentation is available from the project site, at "Building Spark".

For general development tips, including info on developing Spark using an IDE, see "Useful Developer Tools".

Interactive Scala Shell

The easiest way to start using Spark is through the Scala shell:

./bin/spark-shell

Try the following command, which should return 1,000,000,000:

scala> spark.range(1000 * 1000 * 1000).count()

Interactive Python Shell

Alternatively, if you prefer Python, you can use the Python shell:

./bin/pyspark

And run the following command, which should also return 1,000,000,000:

>>> spark.range(1000 * 1000 * 1000).count()

Example Programs

Spark also comes with several sample programs in the examples directory. To run one of them, use ./bin/run-example <class> [params]. For example:

./bin/run-example SparkPi

will run the Pi example locally.

You can set the MASTER environment variable when running examples to submit examples to a cluster. This can be a mesos:// or spark:// URL, "yarn" to run on YARN, and "local" to run locally with one thread, or "local[N]" to run locally with N threads. You can also use an abbreviated class name if the class is in the examples package. For instance:

MASTER=spark://host:7077 ./bin/run-example SparkPi

Many of the example programs print usage help if no params are given.

Running Tests

Testing first requires building Spark. Once Spark is built, tests can be run using:

./dev/run-tests

Please see the guidance on how to run tests for a module, or individual tests.

There is also a Kubernetes integration test, see resource-managers/kubernetes/integration-tests/README.md

A Note About Hadoop Versions

Spark uses the Hadoop core library to talk to HDFS and other Hadoop-supported storage systems. Because the protocols have changed in different versions of Hadoop, you must build Spark against the same version that your cluster runs.

Please refer to the build documentation at "Specifying the Hadoop Version and Enabling YARN" for detailed guidance on building for a particular distribution of Hadoop, including building for particular Hive and Hive Thriftserver distributions.

Configuration

Please refer to the Configuration Guide in the online documentation for an overview on how to configure Spark.

Contributing

Please review the Contribution to Spark guide for information on how to get started contributing to the project.