9962390af7
## What changes were proposed in this pull request? Like `Parquet`, users can use `ORC` with Apache Spark structured streaming. This PR adds `orc()` to `DataStreamReader`(Scala/Python) in order to support creating streaming dataset with ORC file format more easily like the other file formats. Also, this adds a test coverage for ORC data source and updates the document. **BEFORE** ```scala scala> spark.readStream.schema("a int").orc("/tmp/orc_ss").writeStream.format("console").start() <console>:24: error: value orc is not a member of org.apache.spark.sql.streaming.DataStreamReader spark.readStream.schema("a int").orc("/tmp/orc_ss").writeStream.format("console").start() ``` **AFTER** ```scala scala> spark.readStream.schema("a int").orc("/tmp/orc_ss").writeStream.format("console").start() res0: org.apache.spark.sql.streaming.StreamingQuery = org.apache.spark.sql.execution.streaming.StreamingQueryWrapper678b3746 scala> ------------------------------------------- Batch: 0 ------------------------------------------- +---+ | a| +---+ | 1| +---+ ``` ## How was this patch tested? Pass the newly added test cases. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #19975 from dongjoon-hyun/SPARK-22781. |
||
---|---|---|
.. | ||
__init__.py | ||
catalog.py | ||
column.py | ||
conf.py | ||
context.py | ||
dataframe.py | ||
functions.py | ||
group.py | ||
readwriter.py | ||
session.py | ||
streaming.py | ||
tests.py | ||
types.py | ||
udf.py | ||
utils.py | ||
window.py |