## What changes were proposed in this pull request?
Integrate Apache Arrow with Spark to increase performance of `DataFrame.toPandas`. This has been done by using Arrow to convert data partitions on the executor JVM to Arrow payload byte arrays where they are then served to the Python process. The Python DataFrame can then collect the Arrow payloads where they are combined and converted to a Pandas DataFrame. Data types except complex, date, timestamp, and decimal are currently supported, otherwise an `UnsupportedOperation` exception is thrown.
Additions to Spark include a Scala package private method `Dataset.toArrowPayload` that will convert data partitions in the executor JVM to `ArrowPayload`s as byte arrays so they can be easily served. A package private class/object `ArrowConverters` that provide data type mappings and conversion routines. In Python, a private method `DataFrame._collectAsArrow` is added to collect Arrow payloads and a SQLConf "spark.sql.execution.arrow.enable" can be used in `toPandas()` to enable using Arrow (uses the old conversion by default).
## How was this patch tested?
Added a new test suite `ArrowConvertersSuite` that will run tests on conversion of Datasets to Arrow payloads for supported types. The suite will generate a Dataset and matching Arrow JSON data, then the dataset is converted to an Arrow payload and finally validated against the JSON data. This will ensure that the schema and data has been converted correctly.
Added PySpark tests to verify the `toPandas` method is producing equal DataFrames with and without pyarrow. A roundtrip test to ensure the pandas DataFrame produced by pyspark is equal to a one made directly with pandas.
Author: Bryan Cutler <cutlerb@gmail.com>
Author: Li Jin <ice.xelloss@gmail.com>
Author: Li Jin <li.jin@twosigma.com>
Author: Wes McKinney <wes.mckinney@twosigma.com>
Closes#18459 from BryanCutler/toPandas_with_arrow-SPARK-13534.
## What changes were proposed in this pull request?
Currently, it throws a NPE when missing columns but join type is speicified in join at PySpark as below:
```python
spark.conf.set("spark.sql.crossJoin.enabled", "false")
spark.range(1).join(spark.range(1), how="inner").show()
```
```
Traceback (most recent call last):
...
py4j.protocol.Py4JJavaError: An error occurred while calling o66.join.
: java.lang.NullPointerException
at org.apache.spark.sql.Dataset.join(Dataset.scala:931)
at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
...
```
```python
spark.conf.set("spark.sql.crossJoin.enabled", "true")
spark.range(1).join(spark.range(1), how="inner").show()
```
```
...
py4j.protocol.Py4JJavaError: An error occurred while calling o84.join.
: java.lang.NullPointerException
at org.apache.spark.sql.Dataset.join(Dataset.scala:931)
at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
...
```
This PR suggests to follow Scala's one as below:
```scala
scala> spark.conf.set("spark.sql.crossJoin.enabled", "false")
scala> spark.range(1).join(spark.range(1), Seq.empty[String], "inner").show()
```
```
org.apache.spark.sql.AnalysisException: Detected cartesian product for INNER join between logical plans
Range (0, 1, step=1, splits=Some(8))
and
Range (0, 1, step=1, splits=Some(8))
Join condition is missing or trivial.
Use the CROSS JOIN syntax to allow cartesian products between these relations.;
...
```
```scala
scala> spark.conf.set("spark.sql.crossJoin.enabled", "true")
scala> spark.range(1).join(spark.range(1), Seq.empty[String], "inner").show()
```
```
+---+---+
| id| id|
+---+---+
| 0| 0|
+---+---+
```
**After**
```python
spark.conf.set("spark.sql.crossJoin.enabled", "false")
spark.range(1).join(spark.range(1), how="inner").show()
```
```
Traceback (most recent call last):
...
pyspark.sql.utils.AnalysisException: u'Detected cartesian product for INNER join between logical plans\nRange (0, 1, step=1, splits=Some(8))\nand\nRange (0, 1, step=1, splits=Some(8))\nJoin condition is missing or trivial.\nUse the CROSS JOIN syntax to allow cartesian products between these relations.;'
```
```python
spark.conf.set("spark.sql.crossJoin.enabled", "true")
spark.range(1).join(spark.range(1), how="inner").show()
```
```
+---+---+
| id| id|
+---+---+
| 0| 0|
+---+---+
```
## How was this patch tested?
Added tests in `python/pyspark/sql/tests.py`.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#18484 from HyukjinKwon/SPARK-21264.
## What changes were proposed in this pull request?
Integrate Apache Arrow with Spark to increase performance of `DataFrame.toPandas`. This has been done by using Arrow to convert data partitions on the executor JVM to Arrow payload byte arrays where they are then served to the Python process. The Python DataFrame can then collect the Arrow payloads where they are combined and converted to a Pandas DataFrame. All non-complex data types are currently supported, otherwise an `UnsupportedOperation` exception is thrown.
Additions to Spark include a Scala package private method `Dataset.toArrowPayloadBytes` that will convert data partitions in the executor JVM to `ArrowPayload`s as byte arrays so they can be easily served. A package private class/object `ArrowConverters` that provide data type mappings and conversion routines. In Python, a public method `DataFrame.collectAsArrow` is added to collect Arrow payloads and an optional flag in `toPandas(useArrow=False)` to enable using Arrow (uses the old conversion by default).
## How was this patch tested?
Added a new test suite `ArrowConvertersSuite` that will run tests on conversion of Datasets to Arrow payloads for supported types. The suite will generate a Dataset and matching Arrow JSON data, then the dataset is converted to an Arrow payload and finally validated against the JSON data. This will ensure that the schema and data has been converted correctly.
Added PySpark tests to verify the `toPandas` method is producing equal DataFrames with and without pyarrow. A roundtrip test to ensure the pandas DataFrame produced by pyspark is equal to a one made directly with pandas.
Author: Bryan Cutler <cutlerb@gmail.com>
Author: Li Jin <ice.xelloss@gmail.com>
Author: Li Jin <li.jin@twosigma.com>
Author: Wes McKinney <wes.mckinney@twosigma.com>
Closes#15821 from BryanCutler/wip-toPandas_with_arrow-SPARK-13534.
## What changes were proposed in this pull request?
Currently we convert a spark DataFrame to Pandas Dataframe by `pd.DataFrame.from_records`. It infers the data type from the data and doesn't respect the spark DataFrame Schema. This PR fixes it.
## How was this patch tested?
a new regression test
Author: hyukjinkwon <gurwls223@gmail.com>
Author: Wenchen Fan <wenchen@databricks.com>
Author: Wenchen Fan <cloud0fan@gmail.com>
Closes#18378 from cloud-fan/to_pandas.
## What changes were proposed in this pull request?
Document Dataset.union is resolution by position, not by name, since this has been a confusing point for a lot of users.
## How was this patch tested?
N/A - doc only change.
Author: Reynold Xin <rxin@databricks.com>
Closes#18256 from rxin/SPARK-21042.
## What changes were proposed in this pull request?
Allow fill/replace of NAs with booleans, both in Python and Scala
## How was this patch tested?
Unit tests, doctests
This PR is original work from me and I license this work to the Spark project
Author: Ruben Berenguel Montoro <ruben@mostlymaths.net>
Author: Ruben Berenguel <ruben@mostlymaths.net>
Closes#18164 from rberenguel/SPARK-19732-fillna-bools.
### What changes were proposed in this pull request?
This PR does the following tasks:
- Added since
- Added the Python API
- Added test cases
### How was this patch tested?
Added test cases to both Scala and Python
Author: gatorsmile <gatorsmile@gmail.com>
Closes#18147 from gatorsmile/createOrReplaceGlobalTempView.
Now that Structured Streaming has been out for several Spark release and has large production use cases, the `Experimental` label is no longer appropriate. I've left `InterfaceStability.Evolving` however, as I think we may make a few changes to the pluggable Source & Sink API in Spark 2.3.
Author: Michael Armbrust <michael@databricks.com>
Closes#18065 from marmbrus/streamingGA.
## What changes were proposed in this pull request?
Adds `hint` method to PySpark `DataFrame`.
## How was this patch tested?
Unit tests, doctests.
Author: zero323 <zero323@users.noreply.github.com>
Closes#17850 from zero323/SPARK-20584.
## What changes were proposed in this pull request?
Currently pyspark Dataframe.fillna API supports boolean type when we pass dict, but it is missing in documentation.
## How was this patch tested?
>>> spark.createDataFrame([Row(a=True),Row(a=None)]).fillna({"a" : True}).show()
+----+
| a|
+----+
|true|
|true|
+----+
Please review http://spark.apache.org/contributing.html before opening a pull request.
Author: Srinivasa Reddy Vundela <vsr@cloudera.com>
Closes#17688 from vundela/fillna_doc_fix.
## What changes were proposed in this pull request?
- Allows skipping `value` argument if `to_replace` is a `dict`:
```python
df = sc.parallelize([("Alice", 1, 3.0)]).toDF()
df.replace({"Alice": "Bob"}).show()
````
- Adds validation step to ensure homogeneous values / replacements.
- Simplifies internal control flow.
- Improves unit tests coverage.
## How was this patch tested?
Existing unit tests, additional unit tests, manual testing.
Author: zero323 <zero323@users.noreply.github.com>
Closes#16793 from zero323/SPARK-19454.
## What changes were proposed in this pull request?
Update docs for NaN handling in approxQuantile.
## How was this patch tested?
existing tests.
Author: Zheng RuiFeng <ruifengz@foxmail.com>
Closes#17369 from zhengruifeng/doc_quantiles_nan.
## What changes were proposed in this pull request?
This PR adds a special streaming deduplication operator to support `dropDuplicates` with `aggregation` and watermark. It reuses the `dropDuplicates` API but creates new logical plan `Deduplication` and new physical plan `DeduplicationExec`.
The following cases are supported:
- one or multiple `dropDuplicates()` without aggregation (with or without watermark)
- `dropDuplicates` before aggregation
Not supported cases:
- `dropDuplicates` after aggregation
Breaking changes:
- `dropDuplicates` without aggregation doesn't work with `complete` or `update` mode.
## How was this patch tested?
The new unit tests.
Author: Shixiong Zhu <shixiong@databricks.com>
Closes#16970 from zsxwing/dedup.
## What changes were proposed in this pull request?
Add coalesce on DataFrame for down partitioning without shuffle and coalesce on Column
## How was this patch tested?
manual, unit tests
Author: Felix Cheung <felixcheung_m@hotmail.com>
Closes#16739 from felixcheung/rcoalesce.
## What changes were proposed in this pull request?
- Provides correct description of the semantics of a `dict` argument passed as `to_replace`.
- Describes type requirements for collection arguments.
- Describes behavior with `to_replace: List[T]` and `value: T`
## How was this patch tested?
Manual testing, documentation build.
Author: zero323 <zero323@users.noreply.github.com>
Closes#16792 from zero323/SPARK-19453.
## What changes were proposed in this pull request?
1, add the multi-cols support based on current private api
2, add the multi-cols support to pyspark
## How was this patch tested?
unit tests
Author: Zheng RuiFeng <ruifengz@foxmail.com>
Author: Ruifeng Zheng <ruifengz@foxmail.com>
Closes#12135 from zhengruifeng/quantile4multicols.
## What changes were proposed in this pull request?
- [X] Make sure all join types are clearly mentioned
- [X] Make join labeling/style consistent
- [X] Make join label ordering docs the same
- [X] Improve join documentation according to above for Scala
- [X] Improve join documentation according to above for Python
- [X] Improve join documentation according to above for R
## How was this patch tested?
No tests b/c docs.
Please review http://spark.apache.org/contributing.html before opening a pull request.
Author: anabranch <wac.chambers@gmail.com>
Closes#16504 from anabranch/SPARK-19126.
## What changes were proposed in this pull request?
This PR adds two of the newly added methods of `Dataset`s to Python:
`withWatermark` and `checkpoint`
## How was this patch tested?
Doc tests
Author: Burak Yavuz <brkyvz@gmail.com>
Closes#15921 from brkyvz/py-watermark.
## What changes were proposed in this pull request?
I found the documentation for the sample method to be confusing, this adds more clarification across all languages.
- [x] Scala
- [x] Python
- [x] R
- [x] RDD Scala
- [ ] RDD Python with SEED
- [X] RDD Java
- [x] RDD Java with SEED
- [x] RDD Python
## How was this patch tested?
NA
Please review https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark before opening a pull request.
Author: anabranch <wac.chambers@gmail.com>
Author: Bill Chambers <bill@databricks.com>
Closes#15815 from anabranch/SPARK-18365.
## What changes were proposed in this pull request?
Add a crossJoin function to the DataFrame API similar to that in Scala. Joins with no condition (cartesian products) must be specified with the crossJoin API
## How was this patch tested?
Added python tests to ensure that an AnalysisException if a cartesian product is specified without crossJoin(), and that cartesian products can execute if specified via crossJoin()
(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)
(If this patch involves UI changes, please attach a screenshot; otherwise, remove this)
Please review https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark before opening a pull request.
Author: Srinath Shankar <srinath@databricks.com>
Closes#15493 from srinathshankar/crosspython.
[SPARK-11905](https://issues.apache.org/jira/browse/SPARK-11905) added support for `persist`/`cache` for `Dataset`. However, there is no user-facing API to check if a `Dataset` is cached and if so what the storage level is. This PR adds `getStorageLevel` to `Dataset`, analogous to `RDD.getStorageLevel`.
Updated `DatasetCacheSuite`.
Author: Nick Pentreath <nickp@za.ibm.com>
Closes#13780 from MLnick/ds-storagelevel.
Signed-off-by: Michael Armbrust <michael@databricks.com>
## What changes were proposed in this pull request?
In PySpark, the invalid join type will not throw error for the following join:
```df1.join(df2, how='not-a-valid-join-type')```
The signature of the join is:
```def join(self, other, on=None, how=None):```
The existing code completely ignores the `how` parameter when `on` is `None`. This patch will process the arguments passed to join and pass in to JVM Spark SQL Analyzer, which will validate the join type passed.
## How was this patch tested?
Used manual and existing test suites.
Author: Bijay Pathak <bkpathak@mtu.edu>
Closes#15409 from bkpathak/SPARK-14761.
## What changes were proposed in this pull request?
Global temporary view is a cross-session temporary view, which means it's shared among all sessions. Its lifetime is the lifetime of the Spark application, i.e. it will be automatically dropped when the application terminates. It's tied to a system preserved database `global_temp`(configurable via SparkConf), and we must use the qualified name to refer a global temp view, e.g. SELECT * FROM global_temp.view1.
changes for `SessionCatalog`:
1. add a new field `gloabalTempViews: GlobalTempViewManager`, to access the shared global temp views, and the global temp db name.
2. `createDatabase` will fail if users wanna create `global_temp`, which is system preserved.
3. `setCurrentDatabase` will fail if users wanna set `global_temp`, which is system preserved.
4. add `createGlobalTempView`, which is used in `CreateViewCommand` to create global temp views.
5. add `dropGlobalTempView`, which is used in `CatalogImpl` to drop global temp view.
6. add `alterTempViewDefinition`, which is used in `AlterViewAsCommand` to update the view definition for local/global temp views.
7. `renameTable`/`dropTable`/`isTemporaryTable`/`lookupRelation`/`getTempViewOrPermanentTableMetadata`/`refreshTable` will handle global temp views.
changes for SQL commands:
1. `CreateViewCommand`/`AlterViewAsCommand` is updated to support global temp views
2. `ShowTablesCommand` outputs a new column `database`, which is used to distinguish global and local temp views.
3. other commands can also handle global temp views if they call `SessionCatalog` APIs which accepts global temp views, e.g. `DropTableCommand`, `AlterTableRenameCommand`, `ShowColumnsCommand`, etc.
changes for other public API
1. add a new method `dropGlobalTempView` in `Catalog`
2. `Catalog.findTable` can find global temp view
3. add a new method `createGlobalTempView` in `Dataset`
## How was this patch tested?
new tests in `SQLViewSuite`
Author: Wenchen Fan <wenchen@databricks.com>
Closes#14897 from cloud-fan/global-temp-view.
## What changes were proposed in this pull request?
In PySpark, `df.take(1)` runs a single-stage job which computes only one partition of the DataFrame, while `df.limit(1).collect()` computes all partitions and runs a two-stage job. This difference in performance is confusing.
The reason why `limit(1).collect()` is so much slower is that `collect()` internally maps to `df.rdd.<some-pyspark-conversions>.toLocalIterator`, which causes Spark SQL to build a query where a global limit appears in the middle of the plan; this, in turn, ends up being executed inefficiently because limits in the middle of plans are now implemented by repartitioning to a single task rather than by running a `take()` job on the driver (this was done in #7334, a patch which was a prerequisite to allowing partition-local limits to be pushed beneath unions, etc.).
In order to fix this performance problem I think that we should generalize the fix from SPARK-10731 / #8876 so that `DataFrame.collect()` also delegates to the Scala implementation and shares the same performance properties. This patch modifies `DataFrame.collect()` to first collect all results to the driver and then pass them to Python, allowing this query to be planned using Spark's `CollectLimit` optimizations.
## How was this patch tested?
Added a regression test in `sql/tests.py` which asserts that the expected number of jobs, stages, and tasks are run for both queries.
Author: Josh Rosen <joshrosen@databricks.com>
Closes#15068 from JoshRosen/pyspark-collect-limit.
## What changes were proposed in this pull request?
Require the use of CROSS join syntax in SQL (and a new crossJoin
DataFrame API) to specify explicit cartesian products between relations.
By cartesian product we mean a join between relations R and S where
there is no join condition involving columns from both R and S.
If a cartesian product is detected in the absence of an explicit CROSS
join, an error must be thrown. Turning on the
"spark.sql.crossJoin.enabled" configuration flag will disable this check
and allow cartesian products without an explicit CROSS join.
The new crossJoin DataFrame API must be used to specify explicit cross
joins. The existing join(DataFrame) method will produce a INNER join
that will require a subsequent join condition.
That is df1.join(df2) is equivalent to select * from df1, df2.
## How was this patch tested?
Added cross-join.sql to the SQLQueryTestSuite to test the check for cartesian products. Added a couple of tests to the DataFrameJoinSuite to test the crossJoin API. Modified various other test suites to explicitly specify a cross join where an INNER join or a comma-separated list was previously used.
Author: Srinath Shankar <srinath@databricks.com>
Closes#14866 from srinathshankar/crossjoin.
## What changes were proposed in this pull request?
`withColumnRenamed` and `drop` is a no-op if the given column name does not exists. Python documentation also describe that, but this PR adds more explicit line consistently with Scala to reduce the ambiguity.
## How was this patch tested?
It's about docs.
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#14288 from dongjoon-hyun/SPARK-16651.
## What changes were proposed in this pull request?
doc change only
## How was this patch tested?
doc change only
Author: Mortada Mehyar <mortada.mehyar@gmail.com>
Closes#14253 from mortada/histogram_typos.
## What changes were proposed in this pull request?
Make `dataframe.drop` API in python support multi-columns parameters,
so that it is the same with scala API.
## How was this patch tested?
The doc test.
Author: WeichenXu <WeichenXu123@outlook.com>
Closes#14203 from WeichenXu123/drop_python_api.
## What changes were proposed in this pull request?
- Moved DataStreamReader/Writer from pyspark.sql to pyspark.sql.streaming to make them consistent with scala packaging
- Exposed the necessary classes in sql.streaming package so that they appear in the docs
- Added pyspark.sql.streaming module to the docs
## How was this patch tested?
- updated unit tests.
- generated docs for testing visibility of pyspark.sql.streaming classes.
Author: Tathagata Das <tathagata.das1565@gmail.com>
Closes#13955 from tdas/SPARK-16266.
## What changes were proposed in this pull request?
Fixes a couple old references to `DataFrameWriter.startStream` to `DataStreamWriter.start
Author: Burak Yavuz <brkyvz@gmail.com>
Closes#13952 from brkyvz/minor-doc-fix.
## What changes were proposed in this pull request?
Allowing truncate to a specific number of character is convenient at times, especially while operating from the REPL. Sometimes those last few characters make all the difference, and showing everything brings in whole lot of noise.
## How was this patch tested?
Existing tests. + 1 new test in DataFrameSuite.
For SparkR and pyspark, existing tests and manual testing.
Author: Prashant Sharma <prashsh1@in.ibm.com>
Author: Prashant Sharma <prashant@apache.org>
Closes#13839 from ScrapCodes/add_truncateTo_DF.show.
Renamed for simplicity, so that its obvious that its related to streaming.
Existing unit tests.
Author: Tathagata Das <tathagata.das1565@gmail.com>
Closes#13673 from tdas/SPARK-15953.
## What changes were proposed in this pull request?
Currently, the DataFrameReader/Writer has method that are needed for streaming and non-streaming DFs. This is quite awkward because each method in them through runtime exception for one case or the other. So rather having half the methods throw runtime exceptions, its just better to have a different reader/writer API for streams.
- [x] Python API!!
## How was this patch tested?
Existing unit tests + two sets of unit tests for DataFrameReader/Writer and DataStreamReader/Writer.
Author: Tathagata Das <tathagata.das1565@gmail.com>
Closes#13653 from tdas/SPARK-15933.
## What changes were proposed in this pull request?
We use autoBroadcastJoinThreshold + 1L as the default value of size estimation, that is not good in 2.0, because we will calculate the size based on size of schema, then the estimation could be less than autoBroadcastJoinThreshold if you have an SELECT on top of an DataFrame created from RDD.
This PR change the default value to Long.MaxValue.
## How was this patch tested?
Added regression tests.
Author: Davies Liu <davies@databricks.com>
Closes#13183 from davies/fix_default_size.
#### What changes were proposed in this pull request?
This follow-up PR is to address the remaining comments in https://github.com/apache/spark/pull/12385
The major change in this PR is to issue better error messages in PySpark by using the mechanism that was proposed by davies in https://github.com/apache/spark/pull/7135
For example, in PySpark, if we input the following statement:
```python
>>> l = [('Alice', 1)]
>>> df = sqlContext.createDataFrame(l)
>>> df.createTempView("people")
>>> df.createTempView("people")
```
Before this PR, the exception we will get is like
```
Traceback (most recent call last):
File "<stdin>", line 1, in <module>
File "/Users/xiaoli/IdeaProjects/sparkDelivery/python/pyspark/sql/dataframe.py", line 152, in createTempView
self._jdf.createTempView(name)
File "/Users/xiaoli/IdeaProjects/sparkDelivery/python/lib/py4j-0.10.1-src.zip/py4j/java_gateway.py", line 933, in __call__
File "/Users/xiaoli/IdeaProjects/sparkDelivery/python/pyspark/sql/utils.py", line 63, in deco
return f(*a, **kw)
File "/Users/xiaoli/IdeaProjects/sparkDelivery/python/lib/py4j-0.10.1-src.zip/py4j/protocol.py", line 312, in get_return_value
py4j.protocol.Py4JJavaError: An error occurred while calling o35.createTempView.
: org.apache.spark.sql.catalyst.analysis.TempTableAlreadyExistsException: Temporary table 'people' already exists;
at org.apache.spark.sql.catalyst.catalog.SessionCatalog.createTempView(SessionCatalog.scala:324)
at org.apache.spark.sql.SparkSession.createTempView(SparkSession.scala:523)
at org.apache.spark.sql.Dataset.createTempView(Dataset.scala:2328)
at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.lang.reflect.Method.invoke(Method.java:606)
at py4j.reflection.MethodInvoker.invoke(MethodInvoker.java:237)
at py4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)
at py4j.Gateway.invoke(Gateway.java:280)
at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:128)
at py4j.commands.CallCommand.execute(CallCommand.java:79)
at py4j.GatewayConnection.run(GatewayConnection.java:211)
at java.lang.Thread.run(Thread.java:745)
```
After this PR, the exception we will get become cleaner:
```
Traceback (most recent call last):
File "<stdin>", line 1, in <module>
File "/Users/xiaoli/IdeaProjects/sparkDelivery/python/pyspark/sql/dataframe.py", line 152, in createTempView
self._jdf.createTempView(name)
File "/Users/xiaoli/IdeaProjects/sparkDelivery/python/lib/py4j-0.10.1-src.zip/py4j/java_gateway.py", line 933, in __call__
File "/Users/xiaoli/IdeaProjects/sparkDelivery/python/pyspark/sql/utils.py", line 75, in deco
raise AnalysisException(s.split(': ', 1)[1], stackTrace)
pyspark.sql.utils.AnalysisException: u"Temporary table 'people' already exists;"
```
#### How was this patch tested?
Fixed an existing PySpark test case
Author: gatorsmile <gatorsmile@gmail.com>
Closes#13126 from gatorsmile/followup-14684.
## What changes were proposed in this pull request?
Deprecates registerTempTable and add dataset.createTempView, dataset.createOrReplaceTempView.
## How was this patch tested?
Unit tests.
Author: Sean Zhong <seanzhong@databricks.com>
Closes#12945 from clockfly/spark-15171.
## What changes were proposed in this pull request?
Earlier we removed experimental tag for Scala/Java DataFrames, but haven't done so for Python. This patch removes the experimental flag for Python and declares them stable.
## How was this patch tested?
N/A.
Author: Reynold Xin <rxin@databricks.com>
Closes#13062 from rxin/SPARK-15278.
## What changes were proposed in this pull request?
This patch provides a first cut of python APIs for structured streaming. This PR provides the new classes:
- ContinuousQuery
- Trigger
- ProcessingTime
in pyspark under `pyspark.sql.streaming`.
In addition, it contains the new methods added under:
- `DataFrameWriter`
a) `startStream`
b) `trigger`
c) `queryName`
- `DataFrameReader`
a) `stream`
- `DataFrame`
a) `isStreaming`
This PR doesn't contain all methods exposed for `ContinuousQuery`, for example:
- `exception`
- `sourceStatuses`
- `sinkStatus`
They may be added in a follow up.
This PR also contains some very minor doc fixes in the Scala side.
## How was this patch tested?
Python doc tests
TODO:
- [ ] verify Python docs look good
Author: Burak Yavuz <brkyvz@gmail.com>
Author: Burak Yavuz <burak@databricks.com>
Closes#12320 from brkyvz/stream-python.
## What changes were proposed in this pull request?
Change unpersist blocking parameter default value to match Scala
## How was this patch tested?
unit tests, manual tests
jkbradley davies
Author: felixcheung <felixcheung_m@hotmail.com>
Closes#12507 from felixcheung/pyunpersist.
## What changes were proposed in this pull request?
The PyDoc Makefile used "=" rather than "?=" for setting env variables so it overwrote the user values. This ignored the environment variables we set for linting allowing warnings through. This PR also fixes the warnings that had been introduced.
## How was this patch tested?
manual local export & make
Author: Holden Karau <holden@us.ibm.com>
Closes#12336 from holdenk/SPARK-14573-fix-pydoc-makefile.
## What changes were proposed in this pull request?
RDD.toLocalIterator() could be used to fetch one partition at a time to reduce the memory usage. Right now, for Dataset/Dataframe we have to use df.rdd.toLocalIterator, which is super slow also requires lots of memory (because of the Java serializer or even Kyro serializer).
This PR introduce an optimized toLocalIterator for Dataset/DataFrame, which is much faster and requires much less memory. For a partition with 5 millions rows, `df.rdd.toIterator` took about 100 seconds, but df.toIterator took less than 7 seconds. For 10 millions row, rdd.toIterator will crash (not enough memory) with 4G heap, but df.toLocalIterator could finished in 12 seconds.
The JDBC server has been updated to use DataFrame.toIterator.
## How was this patch tested?
Existing tests.
Author: Davies Liu <davies@databricks.com>
Closes#12114 from davies/local_iterator.