Apache Spark - A unified analytics engine for large-scale data processing
Go to file
Jungtaek Lim 8d4d433191 [SPARK-33836][SS][PYTHON] Expose DataStreamReader.table and DataStreamWriter.toTable
### What changes were proposed in this pull request?

This PR proposes to expose `DataStreamReader.table` (SPARK-32885) and `DataStreamWriter.toTable` (SPARK-32896) to PySpark, which are the only way to read and write with table in Structured Streaming.

### Why are the changes needed?

Please refer SPARK-32885 and SPARK-32896 for rationalizations of these public APIs. This PR only exposes them to PySpark.

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

Yes, PySpark users will be able to read and write with table in Structured Streaming query.

### How was this patch tested?

Manually tested.

> v1 table

>> create table A and ingest to the table A

```
spark.sql("""
create table table_pyspark_parquet (
    value long,
    `timestamp` timestamp
) USING parquet
""")
df = spark.readStream.format('rate').option('rowsPerSecond', 100).load()
query = df.writeStream.toTable('table_pyspark_parquet', checkpointLocation='/tmp/checkpoint5')
query.lastProgress
query.stop()
```

>> read table A and ingest to the table B which doesn't exist

```
df2 = spark.readStream.table('table_pyspark_parquet')
query2 = df2.writeStream.toTable('table_pyspark_parquet_nonexist', format='parquet', checkpointLocation='/tmp/checkpoint2')
query2.lastProgress
query2.stop()
```

>> select tables

```
spark.sql("DESCRIBE TABLE table_pyspark_parquet").show()
spark.sql("SELECT * FROM table_pyspark_parquet").show()

spark.sql("DESCRIBE TABLE table_pyspark_parquet_nonexist").show()
spark.sql("SELECT * FROM table_pyspark_parquet_nonexist").show()
```

> v2 table (leveraging Apache Iceberg as it provides V2 table and custom catalog as well)

>> create table A and ingest to the table A

```
spark.sql("""
create table iceberg_catalog.default.table_pyspark_v2table (
    value long,
    `timestamp` timestamp
) USING iceberg
""")
df = spark.readStream.format('rate').option('rowsPerSecond', 100).load()
query = df.select('value', 'timestamp').writeStream.toTable('iceberg_catalog.default.table_pyspark_v2table', checkpointLocation='/tmp/checkpoint_v2table_1')
query.lastProgress
query.stop()
```

>> ingest to the non-exist table B

```
df2 = spark.readStream.format('rate').option('rowsPerSecond', 100).load()
query2 = df2.select('value', 'timestamp').writeStream.toTable('iceberg_catalog.default.table_pyspark_v2table_nonexist', checkpointLocation='/tmp/checkpoint_v2table_2')
query2.lastProgress
query2.stop()
```

>> ingest to the non-exist table C partitioned by `value % 10`

```
df3 = spark.readStream.format('rate').option('rowsPerSecond', 100).load()
df3a = df3.selectExpr('value', 'timestamp', 'value % 10 AS partition').repartition('partition')
query3 = df3a.writeStream.partitionBy('partition').toTable('iceberg_catalog.default.table_pyspark_v2table_nonexist_partitioned', checkpointLocation='/tmp/checkpoint_v2table_3')
query3.lastProgress
query3.stop()
```

>> select tables

```
spark.sql("DESCRIBE TABLE iceberg_catalog.default.table_pyspark_v2table").show()
spark.sql("SELECT * FROM iceberg_catalog.default.table_pyspark_v2table").show()

spark.sql("DESCRIBE TABLE iceberg_catalog.default.table_pyspark_v2table_nonexist").show()
spark.sql("SELECT * FROM iceberg_catalog.default.table_pyspark_v2table_nonexist").show()

spark.sql("DESCRIBE TABLE iceberg_catalog.default.table_pyspark_v2table_nonexist_partitioned").show()
spark.sql("SELECT * FROM iceberg_catalog.default.table_pyspark_v2table_nonexist_partitioned").show()
```

Closes #30835 from HeartSaVioR/SPARK-33836.

Lead-authored-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
Co-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-12-21 19:42:59 +09:00
.github [MINOR][INFRA] Add -Pspark-ganglia-lgpl to the build definition with Scala 2.13 on GitHub Actions 2020-12-18 15:10:13 +09:00
assembly [SPARK-33662][BUILD] Setting version to 3.2.0-SNAPSHOT 2020-12-04 14:10:42 -08:00
bin [MINOR] Spelling bin core docs external mllib repl 2020-11-30 13:59:51 +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-33662][BUILD] Setting version to 3.2.0-SNAPSHOT 2020-12-04 14:10:42 -08:00
conf [SPARK-32004][ALL] Drop references to slave 2020-07-13 14:05:33 -07:00
core [SPARK-26341][WEBUI][FOLLOWUP] Update stage memory metrics on stage end 2020-12-20 17:38:47 -08:00
data [SPARK-22666][ML][SQL] Spark datasource for image format 2018-09-05 11:59:00 -07:00
dev [SPARK-33843][BUILD] Upgrade to Zstd 1.4.8 2020-12-19 06:59:44 -08:00
docs [SPARK-33840][DOCS] Add spark.sql.files.minPartitionNum to performence tuning doc 2020-12-18 20:27:16 +09:00
examples [SPARK-33662][BUILD] Setting version to 3.2.0-SNAPSHOT 2020-12-04 14:10:42 -08:00
external [SPARK-33810][TESTS] Reenable test cases disabled in SPARK-31732 2020-12-16 08:34:22 -08:00
graphx [SPARK-33662][BUILD] Setting version to 3.2.0-SNAPSHOT 2020-12-04 14:10:42 -08:00
hadoop-cloud [SPARK-33662][BUILD] Setting version to 3.2.0-SNAPSHOT 2020-12-04 14:10:42 -08:00
launcher [SPARK-32447][CORE][PYTHON][FOLLOW-UP] Fix other occurrences of 'python' to 'python3' 2020-12-13 10:41:47 +09: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-33518][ML] Improve performance of ML ALS recommendForAll by GEMV 2020-12-19 08:43:48 -06:00
mllib-local [SPARK-33662][BUILD] Setting version to 3.2.0-SNAPSHOT 2020-12-04 14:10:42 -08:00
project [SPARK-33854][BUILD] Use ListBuffer instead of Stack in SparkBuild.scala 2020-12-19 14:19:44 -08:00
python [SPARK-33836][SS][PYTHON] Expose DataStreamReader.table and DataStreamWriter.toTable 2020-12-21 19:42:59 +09:00
R [SPARK-26199][SPARK-31517][R] Fix strategy for handling ... names in mutate 2020-12-17 17:20:45 +09:00
repl [SPARK-33662][BUILD] Setting version to 3.2.0-SNAPSHOT 2020-12-04 14:10:42 -08:00
resource-managers [SPARK-22256][MESOS] Introduce spark.mesos.driver.memoryOverhead 2020-12-15 14:00:38 -08:00
sbin [MINOR][DOCS] fix typo for docs,log message and comments 2020-08-22 06:45:35 +09:00
sql [SPARK-33849][SQL][TESTS] Unify v1 and v2 DROP TABLE tests 2020-12-21 08:34:12 +00:00
streaming [SPARK-33810][TESTS] Reenable test cases disabled in SPARK-31732 2020-12-16 08:34:22 -08:00
tools [SPARK-33662][BUILD] Setting version to 3.2.0-SNAPSHOT 2020-12-04 14:10:42 -08: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-33757][INFRA][R][FOLLOWUP] Provide more simple solution 2020-12-13 17:27:39 -08: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-33705][SQL][TEST] Fix HiveThriftHttpServerSuite flakiness 2020-12-14 05:14:38 +00: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-33843][BUILD] Upgrade to Zstd 1.4.8 2020-12-19 06:59:44 -08:00
README.md [MINOR][DOCS] Fix Jenkins job badge image and link in README.md 2020-12-16 00:10:13 -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.