cdd8e51742
### 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> |
||
---|---|---|
.github | ||
assembly | ||
bin | ||
binder | ||
build | ||
common | ||
conf | ||
core | ||
data | ||
dev | ||
docs | ||
examples | ||
external | ||
graphx | ||
hadoop-cloud | ||
launcher | ||
licenses | ||
licenses-binary | ||
mllib | ||
mllib-local | ||
project | ||
python | ||
R | ||
repl | ||
resource-managers | ||
sbin | ||
sql | ||
streaming | ||
tools | ||
.asf.yaml | ||
.gitattributes | ||
.gitignore | ||
.sbtopts | ||
appveyor.yml | ||
CONTRIBUTING.md | ||
LICENSE | ||
LICENSE-binary | ||
NOTICE | ||
NOTICE-binary | ||
pom.xml | ||
README.md | ||
scalastyle-config.xml |
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.
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.