Apache Spark - A unified analytics engine for large-scale data processing
Go to file
Liang-Chi Hsieh 160c144baa [SPARK-30590][SQL] Untyped select API cannot take typed column expression that needs input type
### What changes were proposed in this pull request?

This patch proposes to throw clear analysis exception if untyped `Dataset.select` takes typed column expression that needs input type.

### Why are the changes needed?

`Dataset` provides few typed `select` helper functions to select typed column expressions. The maximum number of typed columns supported is 5. If wanting to select more than 5 typed columns, it silently calls untyped `Dataset.select` and can causes weird unresolved error, like:

```
org.apache.spark.sql.AnalysisException: unresolved operator 'Aggregate [fooagg(FooAgg(1), None, None, None, input[0, int, false] AS value#114, assertnotnull(cast(value#114 as int)), input[0, int, false] AS value#113, IntegerType, IntegerType, false) AS foo_agg_1#116, fooagg(FooAgg(2), None, None, None, input[0, int, false] AS value#119, assertnotnull(cast(value#119 as int)), input[0, int, false] AS value#118, IntegerType, IntegerType, false) AS foo_agg_2#121, fooagg(FooAgg(3), None, None, None, input[0, int, false] AS value#124, assertnotnull(cast(value#124 as int)), input[0, int, false] AS value#123, IntegerType, IntegerType, false) AS foo_agg_3#126, fooagg(FooAgg(4), None, None, None, input[0, int, false] AS value#129, assertnotnull(cast(value#129 as int)), input[0, int, false] AS value#128, IntegerType, IntegerType, false) AS foo_agg_4#131, fooagg(FooAgg(5), None, None, None, input[0, int, false] AS value#134, assertnotnull(cast(value#134 as int)), input[0, int, false] AS value#133, IntegerType, IntegerType, false) AS foo_agg_5#136, fooagg(FooAgg(6), None, None, None, input[0, int, false] AS value#139, assertnotnull(cast(value#139 as int)), input[0, int, false] AS value#138, IntegerType, IntegerType, false) AS foo_agg_6#141];;
'Aggregate [fooagg(FooAgg(1), None, None, None, input[0, int, false] AS value#114, assertnotnull(cast(value#114 as int)), input[0, int, false] AS value#113, IntegerType, IntegerType, false) AS foo_agg_1#116, fooagg(FooAgg(2), None, None, None, input[0, int, false] AS value#119, assertnotnull(cast(value#119 as int)), input[0, int, false] AS value#118, IntegerType, IntegerType, false) AS foo_agg_2#121, fooagg(FooAgg(3), None, None, None, input[0, int, false] AS value#124, assertnotnull(cast(value#124 as int)), input[0, int, false] AS value#123, IntegerType, IntegerType, false) AS foo_agg_3#126, fooagg(FooAgg(4), None, None, None, input[0, int, false] AS value#129, assertnotnull(cast(value#129 as int)), input[0, int, false] AS value#128, IntegerType, IntegerType, false) AS foo_agg_4#131, fooagg(FooAgg(5), None, None, None, input[0, int, false] AS value#134, assertnotnull(cast(value#134 as int)), input[0, int, false] AS value#133, IntegerType, IntegerType, false) AS foo_agg_5#136, fooagg(FooAgg(6), None, None, None, input[0, int, false] AS value#139, assertnotnull(cast(value#139 as int)), input[0, int, false] AS value#138, IntegerType, IntegerType, false) AS foo_agg_6#141]
+- Project [_1#6 AS a#13, _2#7 AS b#14, _3#8 AS c#15, _4#9 AS d#16, _5#10 AS e#17, _6#11 AS F#18]
 +- LocalRelation [_1#6, _2#7, _3#8, _4#9, _5#10, _6#11]

at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$class.failAnalysis(CheckAnalysis.scala:43)
 at org.apache.spark.sql.catalyst.analysis.Analyzer.failAnalysis(Analyzer.scala:95)
 at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$$anonfun$checkAnalysis$3.apply(CheckAnalysis.scala:431)
 at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$$anonfun$checkAnalysis$3.apply(CheckAnalysis.scala:430)
 at org.apache.spark.sql.catalyst.trees.TreeNode.foreachUp(TreeNode.scala:127)
 at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$class.checkAnalysis(CheckAnalysis.scala:430)
```

However, to fully disallow typed columns as input to untyped `select` API will break current usage like `count` that is a `TypedColumn` in `functions`. In order to keep compatibility, we should allow current usage of certain `TypedColumn`s as input to untyped `select` API. For the `TypedColumn`s that will cause unresolved exception, we should explicitly let users know that they are incorrectly calling untyped `select` with typed columns which need input type.

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

Yes, but this PR only refines the error message.

When users call `Dataset.select` API with typed column that needs input type, an analysis exception will be thrown. Previously an unresolved error will be thrown.

### How was this patch tested?

Unit tests.

Closes #27499 from viirya/SPARK-30590.

Lead-authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Co-authored-by: Liang-Chi Hsieh <liangchi@uber.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-27 14:09:07 +08:00
.github [SPARK-30963][INFRA] Add GitHub Action job for document generation 2020-02-26 19:24:41 -08:00
assembly [SPARK-30950][BUILD] Setting version to 3.1.0-SNAPSHOT 2020-02-25 19:44:31 -08:00
bin [SPARK-30884][PYSPARK] Upgrade to Py4J 0.10.9 2020-02-20 09:09:30 -08:00
build [SPARK-30121][BUILD] Fix memory usage in sbt build script 2019-12-05 11:50:55 -06:00
common [SPARK-30950][BUILD] Setting version to 3.1.0-SNAPSHOT 2020-02-25 19:44:31 -08:00
conf [SPARK-29032][CORE] Add PrometheusServlet to monitor Master/Worker/Driver 2019-09-13 21:31:21 +00:00
core [SPARK-30888][CORE][DOC] Add version information to the configuration of Network 2020-02-27 11:05:11 +09:00
data [SPARK-22666][ML][SQL] Spark datasource for image format 2018-09-05 11:59:00 -07:00
dev [MINOR][BUILD] Fix make-distribution.sh to show usage without 'echo' cmd 2020-02-26 14:40:32 -08:00
docs [SPARK-30888][CORE][DOC] Add version information to the configuration of Network 2020-02-27 11:05:11 +09:00
examples [SPARK-30950][BUILD] Setting version to 3.1.0-SNAPSHOT 2020-02-25 19:44:31 -08:00
external [SPARK-30950][BUILD] Setting version to 3.1.0-SNAPSHOT 2020-02-25 19:44:31 -08:00
graphx [SPARK-30950][BUILD] Setting version to 3.1.0-SNAPSHOT 2020-02-25 19:44:31 -08:00
hadoop-cloud [SPARK-30950][BUILD] Setting version to 3.1.0-SNAPSHOT 2020-02-25 19:44:31 -08:00
launcher [SPARK-30950][BUILD] Setting version to 3.1.0-SNAPSHOT 2020-02-25 19:44:31 -08:00
licenses [SPARK-27557][DOC] Add copy button to Python API docs for easier copying of code-blocks 2019-05-01 11:26:18 -05:00
licenses-binary [SPARK-29308][BUILD] Update deps in dev/deps/spark-deps-hadoop-3.2 for hadoop-3.2 2019-10-13 12:53:12 -05:00
mllib [SPARK-30950][BUILD] Setting version to 3.1.0-SNAPSHOT 2020-02-25 19:44:31 -08:00
mllib-local [SPARK-30950][BUILD] Setting version to 3.1.0-SNAPSHOT 2020-02-25 19:44:31 -08:00
project [SPARK-30928][ML] Remove unnecessary MiMa excludes 2020-02-26 19:45:15 -06:00
python [SPARK-30950][BUILD] Setting version to 3.1.0-SNAPSHOT 2020-02-25 19:44:31 -08:00
R [SPARK-30950][BUILD] Setting version to 3.1.0-SNAPSHOT 2020-02-25 19:44:31 -08:00
repl [SPARK-30950][BUILD] Setting version to 3.1.0-SNAPSHOT 2020-02-25 19:44:31 -08:00
resource-managers [SPARK-30950][BUILD] Setting version to 3.1.0-SNAPSHOT 2020-02-25 19:44:31 -08:00
sbin [SPARK-30884][PYSPARK] Upgrade to Py4J 0.10.9 2020-02-20 09:09:30 -08:00
sql [SPARK-30590][SQL] Untyped select API cannot take typed column expression that needs input type 2020-02-27 14:09:07 +08:00
streaming [SPARK-30950][BUILD] Setting version to 3.1.0-SNAPSHOT 2020-02-25 19:44:31 -08:00
tools [SPARK-30950][BUILD] Setting version to 3.1.0-SNAPSHOT 2020-02-25 19:44:31 -08: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-30084][DOCS] Document how to trigger Jekyll build on Python API doc changes 2019-12-04 17:31:23 -06:00
appveyor.yml [SPARK-23435][SPARKR][TESTS] Update testthat to >= 2.0.0 2020-01-29 10:37:08 +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-29674][CORE] Update dropwizard metrics to 4.1.x for JDK 9+ 2019-11-03 15:13:06 -08:00
LICENSE-binary [SPARK-30695][BUILD] Upgrade Apache ORC to 1.5.9 2020-01-31 17:41:27 -08: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-30950][BUILD] Setting version to 3.1.0-SNAPSHOT 2020-02-25 19:44:31 -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-30030][INFRA] Use RegexChecker instead of TokenChecker to check org.apache.commons.lang. 2019-11-25 12:03:15 -08: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.