Commit graph

828 commits

Author SHA1 Message Date
Tibor Csögör eec1a3c286 [SPARK-23299][SQL][PYSPARK] Fix __repr__ behaviour for Rows
This is PR is meant to replace #20503, which lay dormant for a while.  The solution in the original PR is still valid, so this is just that patch rebased onto the current master.

Original summary follows.

## What changes were proposed in this pull request?

Fix `__repr__` behaviour for Rows.

Rows `__repr__` assumes data is a string when column name is missing.
Examples,

```
>>> from pyspark.sql.types import Row
>>> Row ("Alice", "11")
<Row(Alice, 11)>

>>> Row (name="Alice", age=11)
Row(age=11, name='Alice')

>>> Row ("Alice", 11)
<snip stack trace>
TypeError: sequence item 1: expected string, int found
```

This is because Row () when called without column names assumes everything is a string.

## How was this patch tested?

Manually tested and a unit test was added to `python/pyspark/sql/tests/test_types.py`.

Closes #24448 from tbcs/SPARK-23299.

Lead-authored-by: Tibor Csögör <tibi@tiborius.net>
Co-authored-by: Shashwat Anand <me@shashwat.me>
Signed-off-by: Holden Karau <holden@pigscanfly.ca>
2019-05-06 10:00:49 -07:00
Liang-Chi Hsieh d9bcacf94b [SPARK-27629][PYSPARK] Prevent Unpickler from intervening each unpickling
## What changes were proposed in this pull request?

In SPARK-27612, one correctness issue was reported. When protocol 4 is used to pickle Python objects, we found that unpickled objects were wrong. A temporary fix was proposed by not using highest protocol.

It was found that Opcodes.MEMOIZE was appeared in the opcodes in protocol 4. It is suspect to this issue.

A deeper dive found that Opcodes.MEMOIZE stores objects into internal map of Unpickler object. We use single Unpickler object to unpickle serialized Python bytes. Stored objects intervenes next round of unpickling, if the map is not cleared.

We has two options:

1. Continues to reuse Unpickler, but calls its close after each unpickling.
2. Not to reuse Unpickler and create new Unpickler object in each unpickling.

This patch takes option 1.

## How was this patch tested?

Passing the test added in SPARK-27612 (#24519).

Closes #24521 from viirya/SPARK-27629.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-05-04 13:21:08 +09:00
HyukjinKwon 5c479243de [SPARK-27612][PYTHON] Use Python's default protocol instead of highest protocol
## What changes were proposed in this pull request?

This PR partially reverts https://github.com/apache/spark/pull/20691

After we changed the Python protocol to highest ones, seems like it introduced a correctness bug. This potentially affects all Python related code paths.

I suspect a bug related to Pryolite (maybe opcodes `MEMOIZE`, `FRAME` and/or our `RowPickler`). I would like to stick to default protocol for now and investigate the issue separately.

I will separately investigate later to bring highest protocol back.

## How was this patch tested?

Unittest was added.

```bash
./run-tests --python-executables=python3.7 --testname "pyspark.sql.tests.test_serde SerdeTests.test_int_array_serialization"
```

Closes #24519 from HyukjinKwon/SPARK-27612.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-05-03 14:40:13 +09:00
Gabor Somogyi fb6b19ab7c [SPARK-23014][SS] Fully remove V1 memory sink.
## What changes were proposed in this pull request?

There is a MemorySink v2 already so v1 can be removed. In this PR I've removed it completely.
What this PR contains:
* V1 memory sink removal
* V2 memory sink renamed to become the only implementation
* Since DSv2 sends exceptions in a chained format (linking them with cause field) I've made python side compliant
* Adapted all the tests

## How was this patch tested?

Existing unit tests.

Closes #24403 from gaborgsomogyi/SPARK-23014.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-04-29 09:44:23 -07:00
Jash Gala 90085a1847 [SPARK-23619][DOCS] Add output description for some generator expressions / functions
## What changes were proposed in this pull request?

This PR addresses SPARK-23619: https://issues.apache.org/jira/browse/SPARK-23619

It adds additional comments indicating the default column names for the `explode` and `posexplode`
functions in Spark-SQL.

Functions for which comments have been updated so far:
* stack
* inline
* explode
* posexplode
* explode_outer
* posexplode_outer

## How was this patch tested?

This is just a change in the comments. The package builds and tests successfullly after the change.

Closes #23748 from jashgala/SPARK-23619.

Authored-by: Jash Gala <jashgala@amazon.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-04-27 10:30:12 +09:00
Bryan Cutler d36cce18e2 [SPARK-27276][PYTHON][SQL] Increase minimum version of pyarrow to 0.12.1 and remove prior workarounds
## What changes were proposed in this pull request?

This increases the minimum support version of pyarrow to 0.12.1 and removes workarounds in pyspark to remain compatible with prior versions. This means that users will need to have at least pyarrow 0.12.1 installed and available in the cluster or an `ImportError` will be raised to indicate an upgrade is needed.

## How was this patch tested?

Existing tests using:
Python 2.7.15, pyarrow 0.12.1, pandas 0.24.2
Python 3.6.7, pyarrow 0.12.1, pandas 0.24.0

Closes #24298 from BryanCutler/arrow-bump-min-pyarrow-SPARK-27276.

Authored-by: Bryan Cutler <cutlerb@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-04-22 19:30:31 +09:00
Bryan Cutler f62f44f2a2 [SPARK-27387][PYTHON][TESTS] Replace sqlutils.assertPandasEqual with Pandas assert_frame_equals
## What changes were proposed in this pull request?

Running PySpark tests with Pandas 0.24.x causes a failure in `test_pandas_udf_grouped_map` test_supported_types:
`ValueError: The truth value of an array with more than one element is ambiguous. Use a.any() or a.all()`

This is because a column is an ArrayType and the method `sqlutils ReusedSQLTestCase.assertPandasEqual ` does not properly check this.

This PR removes `assertPandasEqual` and replaces it with the built-in `pandas.util.testing.assert_frame_equal` which can properly handle columns of ArrayType and also prints out better diff between the DataFrames when an error occurs.

Additionally, imports of pandas and pyarrow were moved to the top of related test files to avoid duplicating the same import many times.

## How was this patch tested?

Existing tests

Closes #24306 from BryanCutler/python-pandas-assert_frame_equal-SPARK-27387.

Authored-by: Bryan Cutler <cutlerb@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-04-10 07:50:25 +09:00
Liang-Chi Hsieh d04a7371da [MINOR][DOC][SQL] Remove out-of-date doc about ORC in DataFrameReader and Writer
## What changes were proposed in this pull request?

According to current status, `orc` is available even Hive support isn't enabled. This is a minor doc change to reflect it.

## How was this patch tested?

Doc only change.

Closes #24280 from viirya/fix-orc-doc.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-04-03 09:11:09 -07:00
Maxim Gekk 1d20d13149 [SPARK-25496][SQL] Deprecate from_utc_timestamp and to_utc_timestamp
## What changes were proposed in this pull request?

In the PR, I propose to deprecate the `from_utc_timestamp()` and `to_utc_timestamp`, and disable them by default. The functions can be enabled back via the SQL config `spark.sql.legacy.utcTimestampFunc.enabled`. By default, any calls of the functions throw an analysis exception.

One of the reason for deprecation is functions violate semantic of `TimestampType` which is number of microseconds since epoch in UTC time zone. Shifting microseconds since epoch by time zone offset doesn't make sense because the result doesn't represent microseconds since epoch in UTC time zone any more, and cannot be considered as `TimestampType`.

## How was this patch tested?

The changes were tested by `DateExpressionsSuite` and `DateFunctionsSuite`.

Closes #24195 from MaxGekk/conv-utc-timestamp-deprecate.

Lead-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Co-authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-03 10:55:56 +08:00
Hyukjin Kwon d7dd59a6b4 [SPARK-26224][SQL][PYTHON][R][FOLLOW-UP] Add notes about many projects in withColumn at SparkR and PySpark as well
## What changes were proposed in this pull request?

This is a followup of https://github.com/apache/spark/pull/23285. This PR adds the notes into PySpark and SparkR documentation as well.

While I am here, I revised the doc a bit to make it sound a bit more neutral

## How was this patch tested?

Manually built the doc and verified.

Closes #24272 from HyukjinKwon/SPARK-26224.

Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-04-03 08:30:24 +09:00
Dongjoon Hyun d575a453db Revert "[SPARK-25496][SQL] Deprecate from_utc_timestamp and to_utc_timestamp"
This reverts commit c5e83ab92c.
2019-04-02 01:05:54 -07:00
Dongjoon Hyun a0d807d5ab [SPARK-26856][PYSPARK][FOLLOWUP] Fix UT failure due to wrong patterns for Kinesis assembly
## What changes were proposed in this pull request?

After [SPARK-26856](https://github.com/apache/spark/pull/23797), `Kinesis` Python UT fails with `Found multiple JARs` exception due to a wrong pattern.

- https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/104171/console
```
Exception: Found multiple JARs:
.../spark-streaming-kinesis-asl-assembly-3.0.0-SNAPSHOT.jar,
.../spark-streaming-kinesis-asl-assembly_2.12-3.0.0-SNAPSHOT.jar;
please remove all but one
```

It's because the pattern was changed in a wrong way.

**Original**
```python
kinesis_asl_assembly_dir, "target/scala-*/%s-*.jar" % name_prefix))
kinesis_asl_assembly_dir, "target/%s_*.jar" % name_prefix))
```
**After SPARK-26856**
```python
project_full_path, "target/scala-*/%s*.jar" % jar_name_prefix))
project_full_path, "target/%s*.jar" % jar_name_prefix))
```

The actual kinesis assembly jar files look like the followings.

**SBT Build**
```
-rw-r--r--  1 dongjoon  staff  87459461 Apr  1 19:01 spark-streaming-kinesis-asl-assembly-3.0.0-SNAPSHOT.jar
-rw-r--r--  1 dongjoon  staff       309 Apr  1 18:58 spark-streaming-kinesis-asl-assembly_2.12-3.0.0-SNAPSHOT-tests.jar
-rw-r--r--  1 dongjoon  staff       309 Apr  1 18:58 spark-streaming-kinesis-asl-assembly_2.12-3.0.0-SNAPSHOT.jar
```

**MAVEN Build**
```
-rw-r--r--   1 dongjoon  staff   8.6K Apr  1 18:55 spark-streaming-kinesis-asl-assembly_2.12-3.0.0-SNAPSHOT-sources.jar
-rw-r--r--   1 dongjoon  staff   8.6K Apr  1 18:55 spark-streaming-kinesis-asl-assembly_2.12-3.0.0-SNAPSHOT-test-sources.jar
-rw-r--r--   1 dongjoon  staff   8.7K Apr  1 18:55 spark-streaming-kinesis-asl-assembly_2.12-3.0.0-SNAPSHOT-tests.jar
-rw-r--r--   1 dongjoon  staff    21M Apr  1 18:55 spark-streaming-kinesis-asl-assembly_2.12-3.0.0-SNAPSHOT.jar
```

In addition, after SPARK-26856, the utility function `search_jar` is shared to find `avro` jar files which are identical for both `sbt` and `mvn`. To sum up, The current jar pattern parameter cannot handle both `kinesis` and `avro` jars. This PR splits the single pattern into two patterns.

## How was this patch tested?

Manual. Please note that this will remove only `Found multiple JARs` exception. Kinesis tests need more configurations to run locally.
```
$ build/sbt -Pkinesis-asl test:package streaming-kinesis-asl-assembly/assembly
$ export ENABLE_KINESIS_TESTS=1
$ python/run-tests.py --python-executables python2.7 --module pyspark-streaming
```

Closes #24268 from dongjoon-hyun/SPARK-26856.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-04-02 14:52:56 +09:00
Maxim Gekk c5e83ab92c [SPARK-25496][SQL] Deprecate from_utc_timestamp and to_utc_timestamp
## What changes were proposed in this pull request?

In the PR, I propose to deprecate the `from_utc_timestamp()` and `to_utc_timestamp`, and disable them by default. The functions can be enabled back via the SQL config `spark.sql.legacy.utcTimestampFunc.enabled`. By default, any calls of the functions throw an analysis exception.

One of the reason for deprecation is functions violate semantic of `TimestampType` which is number of microseconds since epoch in UTC time zone. Shifting microseconds since epoch by time zone offset doesn't make sense because the result doesn't represent microseconds since epoch in UTC time zone any more, and cannot be considered as `TimestampType`.

## How was this patch tested?

The changes were tested by `DateExpressionsSuite` and `DateFunctionsSuite`.

Closes #24195 from MaxGekk/conv-utc-timestamp-deprecate.

Lead-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Co-authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-02 10:20:06 +08:00
Takuya UESHIN 594be7a911 [SPARK-27240][PYTHON] Use pandas DataFrame for struct type argument in Scalar Pandas UDF.
## What changes were proposed in this pull request?

Now that we support returning pandas DataFrame for struct type in Scalar Pandas UDF.

If we chain another Pandas UDF after the Scalar Pandas UDF returning pandas DataFrame, the argument of the chained UDF will be pandas DataFrame, but currently we don't support pandas DataFrame as an argument of Scalar Pandas UDF. That means there is an inconsistency between the chained UDF and the single UDF.

We should support taking pandas DataFrame for struct type argument in Scalar Pandas UDF to be consistent.
Currently pyarrow >=0.11 is supported.

## How was this patch tested?

Modified and added some tests.

Closes #24177 from ueshin/issues/SPARK-27240/structtype_argument.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Bryan Cutler <cutlerb@gmail.com>
2019-03-25 11:26:09 -07:00
Maxim Gekk 027ed2d11b [SPARK-23643][CORE][SQL][ML] Shrinking the buffer in hashSeed up to size of the seed parameter
## What changes were proposed in this pull request?

The hashSeed method allocates 64 bytes instead of 8. Other bytes are always zeros (thanks to default behavior of ByteBuffer). And they could be excluded from hash calculation because they don't differentiate inputs.

## How was this patch tested?

By running the existing tests - XORShiftRandomSuite

Closes #20793 from MaxGekk/hash-buff-size.

Lead-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Co-authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-23 11:26:09 -05:00
Bryan Cutler be08b415da [SPARK-27163][PYTHON] Cleanup and consolidate Pandas UDF functionality
## What changes were proposed in this pull request?

This change is a cleanup and consolidation of 3 areas related to Pandas UDFs:

1) `ArrowStreamPandasSerializer` now inherits from `ArrowStreamSerializer` and uses the base class `dump_stream`, `load_stream` to create Arrow reader/writer and send Arrow record batches.  `ArrowStreamPandasSerializer` makes the conversions to/from Pandas and converts to Arrow record batch iterators. This change removed duplicated creation of Arrow readers/writers.

2) `createDataFrame` with Arrow now uses `ArrowStreamPandasSerializer` instead of doing its own conversions from Pandas to Arrow and sending record batches through `ArrowStreamSerializer`.

3) Grouped Map UDFs now reuse existing logic in `ArrowStreamPandasSerializer` to send Pandas DataFrame results as a `StructType` instead of separating each column from the DataFrame. This makes the code a little more consistent with the Python worker, but does require that the returned StructType column is flattened out in `FlatMapGroupsInPandasExec` in Scala.

## How was this patch tested?

Existing tests and ran tests with pyarrow 0.12.0

Closes #24095 from BryanCutler/arrow-refactor-cleanup-UDFs.

Authored-by: Bryan Cutler <cutlerb@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-21 17:44:51 +09:00
Huon Wilson b67d369572 [SPARK-27099][SQL] Add 'xxhash64' for hashing arbitrary columns to Long
## What changes were proposed in this pull request?

This introduces a new SQL function 'xxhash64' for getting a 64-bit hash of an arbitrary number of columns.

This is designed to exactly mimic the 32-bit `hash`, which uses
MurmurHash3. The name is designed to be more future-proof than the
'hash', by indicating the exact algorithm used, similar to md5 and the
sha hashes.

## How was this patch tested?

The tests for the existing `hash` function were duplicated to run with `xxhash64`.

Closes #24019 from huonw/hash64.

Authored-by: Huon Wilson <Huon.Wilson@data61.csiro.au>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-20 16:34:34 +08:00
Hyukjin Kwon c99463d4cf [SPARK-26979][PYTHON][FOLLOW-UP] Make binary math/string functions take string as columns as well
## What changes were proposed in this pull request?

This is a followup of https://github.com/apache/spark/pull/23882 to handle binary math/string functions. For instance, see the cases below:

**Before:**

```python
>>> from pyspark.sql.functions import lit, ascii
>>> spark.range(1).select(lit('a').alias("value")).select(ascii("value"))
Traceback (most recent call last):
  File "<stdin>", line 1, in <module>
  File "/.../spark/python/pyspark/sql/functions.py", line 51, in _
    jc = getattr(sc._jvm.functions, name)(col._jc if isinstance(col, Column) else col)
  File "/.../spark/python/lib/py4j-0.10.8.1-src.zip/py4j/java_gateway.py", line 1286, in __call__
  File "/.../spark/python/pyspark/sql/utils.py", line 63, in deco
    return f(*a, **kw)
  File "/.../spark/python/lib/py4j-0.10.8.1-src.zip/py4j/protocol.py", line 332, in get_return_value
py4j.protocol.Py4JError: An error occurred while calling z:org.apache.spark.sql.functions.ascii. Trace:
py4j.Py4JException: Method ascii([class java.lang.String]) does not exist
	at py4j.reflection.ReflectionEngine.getMethod(ReflectionEngine.java:318)
	at py4j.reflection.ReflectionEngine.getMethod(ReflectionEngine.java:339)
	at py4j.Gateway.invoke(Gateway.java:276)
	at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)
	at py4j.commands.CallCommand.execute(CallCommand.java:79)
	at py4j.GatewayConnection.run(GatewayConnection.java:238)
	at java.lang.Thread.run(Thread.java:748)
```

```python
>>> from pyspark.sql.functions import atan2
>>> spark.range(1).select(atan2("id", "id"))
Traceback (most recent call last):
  File "<stdin>", line 1, in <module>
  File "/.../spark/python/pyspark/sql/functions.py", line 78, in _
    jc = getattr(sc._jvm.functions, name)(col1._jc if isinstance(col1, Column) else float(col1),
ValueError: could not convert string to float: id
```

**After:**

```python
>>> from pyspark.sql.functions import lit, ascii
>>> spark.range(1).select(lit('a').alias("value")).select(ascii("value"))
DataFrame[ascii(value): int]
```

```python
>>> from pyspark.sql.functions import atan2
>>> spark.range(1).select(atan2("id", "id"))
DataFrame[ATAN2(id, id): double]
```

Note that,

- This PR causes a slight behaviour changes for math functions. For instance, numbers as strings (e.g., `"1"`) were supported as arguments of binary math functions before. After this PR, it recognises it as column names.

- I also intentionally didn't document this behaviour changes since we're going ahead for Spark 3.0 and I don't think numbers as strings make much sense in math functions.

- There is another exception `when`, which takes string as literal values as below. This PR doeesn't fix this ambiguity.
  ```python
  >>> spark.range(1).select(when(lit(True), col("id"))).show()
  ```

  ```
  +--------------------------+
  |CASE WHEN true THEN id END|
  +--------------------------+
  |                         0|
  +--------------------------+
  ```

  ```python
  >>> spark.range(1).select(when(lit(True), "id")).show()
  ```

  ```
  +--------------------------+
  |CASE WHEN true THEN id END|
  +--------------------------+
  |                        id|
  +--------------------------+
  ```

This PR also fixes as below:

https://github.com/apache/spark/pull/23882 fixed it to:

- Rename `_create_function` to `_create_name_function`
- Define new `_create_function` to take strings as column names.

This PR, I proposes to:

- Revert `_create_name_function` name to `_create_function`.
- Define new `_create_function_over_column` to take strings as column names.

## How was this patch tested?

Some unit tests were added for binary math / string functions.

Closes #24121 from HyukjinKwon/SPARK-26979.

Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-20 08:06:10 +09:00
André Sá de Mello f9180f8752 [SPARK-26979][PYTHON] Add missing string column name support for some SQL functions
## What changes were proposed in this pull request?

Most SQL functions defined in `spark.sql.functions` have two calling patterns, one with a Column object as input, and another with a string representing a column name, which is then converted into a Column object internally.

There are, however, a few notable exceptions:

- lower()
- upper()
- abs()
- bitwiseNOT()
- ltrim()
- rtrim()
- trim()
- ascii()
- base64()
- unbase64()

While this doesn't break anything, as you can easily create a Column object yourself prior to passing it to one of these functions, it has two undesirable consequences:

1. It is surprising - it breaks coder's expectations when they are first starting with Spark. Every API should be as consistent as possible, so as to make the learning curve smoother and to reduce causes for human error;

2. It gets in the way of stylistic conventions. Most of the time it makes Python code more readable to use literal names, and the API provides ample support for that, but these few exceptions prevent this pattern from being universally applicable.

This patch is meant to fix the aforementioned problem.

### Effect

This patch **enables** support for passing column names as input to those functions mentioned above.

### Side effects

This PR also **fixes** an issue with some functions being defined multiple times by using `_create_function()`.

### How it works

`_create_function()` was redefined to always convert the argument to a Column object. The old implementation has been kept under `_create_name_function()`, and is still being used to generate the following special functions:

- lit()
- col()
- column()
- asc()
- desc()
- asc_nulls_first()
- asc_nulls_last()
- desc_nulls_first()
- desc_nulls_last()

This is because these functions can only take a column name as their argument. This is not a problem, as their semantics require so.

## How was this patch tested?

Ran ./dev/run-tests and tested it manually.

Closes #23882 from asmello/col-name-support-pyspark.

Authored-by: André Sá de Mello <amello@palantir.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-17 12:58:16 -05:00
Dilip Biswal 7a136f8670 [SPARK-27096][SQL][FOLLOWUP] Do the correct validation of join types in R side and fix join docs for scala, python and r
## What changes were proposed in this pull request?
This is a minor follow-up PR for SPARK-27096. The original PR reconciled the join types supported between dataset and sql interface. In case of R, we do the join type validation in the R side. In this PR we do the correct validation and adds tests in R to test all the join types along with the error condition. Along with this, i made the necessary doc correction.

## How was this patch tested?
Add R tests.

Closes #24087 from dilipbiswal/joinfix_followup.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-16 13:04:54 +09:00
TigerYang414 60a899b8c3 [SPARK-27041][PYSPARK] Use imap() for python 2.x to resolve oom issue
## What changes were proposed in this pull request?

With large partition, pyspark may exceeds executor memory limit and trigger out of memory for python 2.7.
This is because map() is used. Unlike in python3.x, python 2.7 map() will generate a list and need to read all data into memory.

The proposed fix will use imap in python 2.7 and it has been verified.

## How was this patch tested?
Manual test.
(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 http://spark.apache.org/contributing.html before opening a pull request.

Closes #23954 from TigerYang414/patch-1.

Lead-authored-by: TigerYang414 <39265202+TigerYang414@users.noreply.github.com>
Co-authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-12 10:23:26 -05:00
Jagadesh Kiran d9978fb4e4 [SPARK-26860][PYSPARK][SPARKR] Fix for RangeBetween and RowsBetween docs to be in sync with spark documentation
The docs describing RangeBetween & RowsBetween for pySpark & SparkR are not in sync with Spark description.

a. Edited PySpark and SparkR docs  and made description same for both RangeBetween and RowsBetween
b. created executable examples in both pySpark and SparkR documentation
c. Locally tested the patch for scala Style checks and UT for checking no testcase failures

Closes #23946 from jagadesh-kiran/master.

Authored-by: Jagadesh Kiran <jagadesh.n@in.verizon.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-11 08:53:09 -05:00
Gabor Somogyi 3729efb4d0 [SPARK-26856][PYSPARK] Python support for from_avro and to_avro APIs
## What changes were proposed in this pull request?

Avro is built-in but external data source module since Spark 2.4 but  `from_avro` and `to_avro` APIs not yet supported in pyspark.

In this PR I've made them available from pyspark.

## How was this patch tested?

Please see the python API examples what I've added.

cd docs/
SKIP_SCALADOC=1 SKIP_RDOC=1 SKIP_SQLDOC=1 jekyll build
Manual webpage check.

Closes #23797 from gaborgsomogyi/SPARK-26856.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-11 10:15:07 +09:00
sandeep-katta 14f2286e56 [SPARK-27101][PYTHON] Drop the created database after the test in test_session
## What changes were proposed in this pull request?

Cleaning the testcase, drop the database after use

## How was this patch tested?

existing UT

Closes #24021 from sandeep-katta/cleanPythonTest.

Authored-by: sandeep-katta <sandeep.katta2007@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-09 09:12:33 +09:00
Bryan Cutler ddc2052ebd [SPARK-23836][PYTHON] Add support for StructType return in Scalar Pandas UDF
## What changes were proposed in this pull request?

This change adds support for returning StructType from a scalar Pandas UDF, where the return value of the function is a pandas.DataFrame. Nested structs are not supported and an error will be raised, child types can be any other type currently supported.

## How was this patch tested?

Added additional unit tests to `test_pandas_udf_scalar`

Closes #23900 from BryanCutler/pyspark-support-scalar_udf-StructType-SPARK-23836.

Authored-by: Bryan Cutler <cutlerb@gmail.com>
Signed-off-by: Bryan Cutler <cutlerb@gmail.com>
2019-03-07 08:52:24 -08:00
Sean Owen 0deebd3820 [SPARK-26016][DOCS] Clarify that text DataSource read/write, and RDD methods that read text, always use UTF-8
## What changes were proposed in this pull request?

Clarify that text DataSource read/write, and RDD methods that read text, always use UTF-8 as they use Hadoop's implementation underneath. I think these are all the places that this needs a mention in the user-facing docs.

## How was this patch tested?

Doc tests.

Closes #23962 from srowen/SPARK-26016.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-03-05 08:03:39 +09:00
Hellsen83 387efe29b7 [SPARK-26449][PYTHON] Add transform method to DataFrame API
## What changes were proposed in this pull request?

Added .transform() method to Python DataFrame API to be in sync with Scala API.

## How was this patch tested?

Addition has been tested manually.

Closes #23877 from Hellsen83/pyspark-dataframe-transform.

Authored-by: Hellsen83 <erik.christiansen83@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-02-26 18:22:36 -06:00
Hyukjin Kwon a56b3511fc [SPARK-26945][PYTHON][SS][TESTS] Fix flaky test_*_await_termination in PySpark SS tests
## What changes were proposed in this pull request?

This PR proposes to make sure processing all available data before stopping and delete the temp directory.

See https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/102518/console

```
ERROR: test_query_manager_await_termination (pyspark.sql.tests.test_streaming.StreamingTests)
----------------------------------------------------------------------
Traceback (most recent call last):
 File "/home/jenkins/workspace/SparkPullRequestBuilder/python/pyspark/sql/tests/test_streaming.py", line 259, in test_query_manager_await_termination
 shutil.rmtree(tmpPath)
 File "/home/anaconda/lib/python2.7/shutil.py", line 256, in rmtree
 onerror(os.rmdir, path, sys.exc_info())
 File "/home/anaconda/lib/python2.7/shutil.py", line 254, in rmtree
 os.rmdir(path)
OSError: [Errno 39] Directory not empty: '/home/jenkins/workspace/SparkPullRequestBuilder/python/target/072153bd-f981-47be-bda2-e2b657a16f65/tmp4WGp7n'
```

See https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/102311/console

```
ERROR: test_stream_await_termination (pyspark.sql.tests.test_streaming.StreamingTests)
----------------------------------------------------------------------
Traceback (most recent call last):
  File "/home/jenkins/workspace/SparkPullRequestBuilder2/python/pyspark/sql/tests/test_streaming.py", line 202, in test_stream_await_termination
    shutil.rmtree(tmpPath)
  File "/usr/lib64/pypy-2.5.1/lib-python/2.7/shutil.py", line 256, in rmtree
    onerror(os.rmdir, path, sys.exc_info())
  File "/usr/lib64/pypy-2.5.1/lib-python/2.7/shutil.py", line 254, in rmtree
    os.rmdir(path)
OSError: [Errno 39] Directory not empty: '/home/jenkins/workspace/SparkPullRequestBuilder2/python/target/7244f4ff-6b60-4f6c-b787-de4f15922bf5/tmpQbMZSo'
```

## How was this patch tested?

Jenkins tests - I should run multiple times to see if there are other flaky tests + if this PR really fixes it.

Closes #23870 from HyukjinKwon/SPARK-26945.

Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-02-23 14:57:04 +08:00
Takuya UESHIN 4a4e7aeca7 [SPARK-26887][SQL][PYTHON][NS] Create datetime.date directly instead of creating datetime64 as intermediate data.
## What changes were proposed in this pull request?

Currently `DataFrame.toPandas()` with arrow enabled or `ArrowStreamPandasSerializer` for pandas UDF with pyarrow<0.12 creates `datetime64[ns]` type series as intermediate data and then convert to `datetime.date` series, but the intermediate `datetime64[ns]` might cause an overflow even if the date is valid.

```
>>> import datetime
>>>
>>> t = [datetime.date(2262, 4, 12), datetime.date(2263, 4, 12)]
>>>
>>> df = spark.createDataFrame(t, 'date')
>>> df.show()
+----------+
|     value|
+----------+
|2262-04-12|
|2263-04-12|
+----------+

>>>
>>> spark.conf.set("spark.sql.execution.arrow.enabled", "true")
>>>
>>> df.toPandas()
        value
0  1677-09-21
1  1678-09-21
```

We should avoid creating such intermediate data and create `datetime.date` series directly instead.

## How was this patch tested?

Modified some tests to include the date which overflow caused by the intermediate conversion.
Run tests with pyarrow 0.8, 0.10, 0.11, 0.12 in my local environment.

Closes #23795 from ueshin/issues/SPARK-26887/date_as_object.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-02-18 11:48:10 +08:00
Dilip Biswal 7f44c9a252 [SPARK-26864][SQL] Query may return incorrect result when python udf is used as a join condition and the udf uses attributes from both legs of left semi join.
## What changes were proposed in this pull request?
In SPARK-25314, we supported the scenario of having a python UDF that refers to attributes from both legs of a join condition by rewriting the plan to convert an inner join or left semi join to a filter over a cross join. In case of left semi join, this transformation may cause incorrect results when the right leg of join condition produces duplicate rows based on the join condition. This fix disallows the rewrite for left semi join and raises an error in the case like we do for other types of join. In future, we should have separate rule in optimizer to convert left semi join to inner join (I am aware of one case we could do it if we leverage informational constraint i.e when we know the right side does not produce duplicates).

**Python**

```SQL
>>> from pyspark import SparkContext
>>> from pyspark.sql import SparkSession, Column, Row
>>> from pyspark.sql.functions import UserDefinedFunction, udf
>>> from pyspark.sql.types import *
>>> from pyspark.sql.utils import AnalysisException
>>>
>>> spark.conf.set("spark.sql.crossJoin.enabled", "True")
>>> left = spark.createDataFrame([Row(lc1=1, lc2=1), Row(lc1=2, lc2=2)])
>>> right = spark.createDataFrame([Row(rc1=1, rc2=1), Row(rc1=1, rc2=1)])
>>> func = udf(lambda a, b: a == b, BooleanType())
>>> df = left.join(right, func("lc1", "rc1"), "leftsemi").show()
19/02/12 16:07:10 WARN PullOutPythonUDFInJoinCondition: The join condition:<lambda>(lc1#0L, rc1#4L) of the join plan contains PythonUDF only, it will be moved out and the join plan will be turned to cross join.
+---+---+
|lc1|lc2|
+---+---+
|  1|  1|
|  1|  1|
+---+---+
```

**Scala**

```SQL
scala> val left = Seq((1, 1), (2, 2)).toDF("lc1", "lc2")
left: org.apache.spark.sql.DataFrame = [lc1: int, lc2: int]

scala> val right = Seq((1, 1), (1, 1)).toDF("rc1", "rc2")
right: org.apache.spark.sql.DataFrame = [rc1: int, rc2: int]

scala> val equal = udf((p1: Integer, p2: Integer) => {
     |   p1 == p2
     | })
equal: org.apache.spark.sql.expressions.UserDefinedFunction = SparkUserDefinedFunction($Lambda$2141/11016292394666f1b5,BooleanType,List(Some(Schema(IntegerType,true)), Some(Schema(IntegerType,true))),None,false,true)

scala> val df = left.join(right, equal(col("lc1"), col("rc1")), "leftsemi")
df: org.apache.spark.sql.DataFrame = [lc1: int, lc2: int]

scala> df.show()
+---+---+
|lc1|lc2|
+---+---+
|  1|  1|
+---+---+

```

## How was this patch tested?
Modified existing tests.

Closes #23769 from dilipbiswal/dkb_python_udf_in_join.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-02-13 21:14:19 +08:00
Bryan Cutler 16990f9299 [SPARK-26566][PYTHON][SQL] Upgrade Apache Arrow to version 0.12.0
## What changes were proposed in this pull request?

Upgrade Apache Arrow to version 0.12.0. This includes the Java artifacts and fixes to enable usage with pyarrow 0.12.0

Version 0.12.0 includes the following selected fixes/improvements relevant to Spark users:

* Safe cast fails from numpy float64 array with nans to integer, ARROW-4258
* Java, Reduce heap usage for variable width vectors, ARROW-4147
* Binary identity cast not implemented, ARROW-4101
* pyarrow open_stream deprecated, use ipc.open_stream, ARROW-4098
* conversion to date object no longer needed, ARROW-3910
* Error reading IPC file with no record batches, ARROW-3894
* Signed to unsigned integer cast yields incorrect results when type sizes are the same, ARROW-3790
* from_pandas gives incorrect results when converting floating point to bool, ARROW-3428
* Import pyarrow fails if scikit-learn is installed from conda (boost-cpp / libboost issue), ARROW-3048
* Java update to official Flatbuffers version 1.9.0, ARROW-3175

complete list [here](https://issues.apache.org/jira/issues/?jql=project%20%3D%20ARROW%20AND%20status%20in%20(Resolved%2C%20Closed)%20AND%20fixVersion%20%3D%200.12.0)

PySpark requires the following fixes to work with PyArrow 0.12.0

* Encrypted pyspark worker fails due to ChunkedStream missing closed property
* pyarrow now converts dates as objects by default, which causes error because type is assumed datetime64
* ArrowTests fails due to difference in raised error message
* pyarrow.open_stream deprecated
* tests fail because groupby adds index column with duplicate name

## How was this patch tested?

Ran unit tests with pyarrow versions 0.8.0, 0.10.0, 0.11.1, 0.12.0

Closes #23657 from BryanCutler/arrow-upgrade-012.

Authored-by: Bryan Cutler <cutlerb@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-01-29 14:18:45 +08:00
Liang-Chi Hsieh f92d276653 [SPARK-25811][PYSPARK] Raise a proper error when unsafe cast is detected by PyArrow
## What changes were proposed in this pull request?

Since 0.11.0, PyArrow supports to raise an error for unsafe cast ([PR](https://github.com/apache/arrow/pull/2504)). We should use it to raise a proper error for pandas udf users when such cast is detected.

Added a SQL config `spark.sql.execution.pandas.arrowSafeTypeConversion` to disable Arrow safe type check.

## How was this patch tested?

Added test and manually test.

Closes #22807 from viirya/SPARK-25811.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-01-22 14:54:41 +08:00
Hyukjin Kwon 75d84498a4 [SPARK-26676][PYTHON] Make HiveContextSQLTests.test_unbounded_frames test compatible with Python 2 and PyPy
## What changes were proposed in this pull request?

This particular test is being skipped at PyPy and Python 2.

```
Skipped tests in pyspark.sql.tests.test_context with pypy:
    test_unbounded_frames (pyspark.sql.tests.test_context.HiveContextSQLTests) ... skipped "Unittest < 3.3 doesn't support mocking"

Skipped tests in pyspark.sql.tests.test_context with python2.7:
    test_unbounded_frames (pyspark.sql.tests.test_context.HiveContextSQLTests) ... skipped "Unittest < 3.3 doesn't support mocking"
```

We don't have to use unittest 3.3 module to mock. And looks the test itself isn't compatible with Python 2.

This PR makes:
 - Manually monkey-patch `sys.maxsize` to get rid of unittest 3.3 condition
 - Use the built-in `reload` in Python 2, and `importlib.reload` in Python 3

## How was this patch tested?

Manually tested, and unit test is fixed.

Closes #23604 from HyukjinKwon/test-window.

Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Bryan Cutler <cutlerb@gmail.com>
2019-01-21 14:27:17 -08:00
Marco Gaido 6d9c54b62c [SPARK-26645][PYTHON] Support decimals with negative scale when parsing datatype
## What changes were proposed in this pull request?

When parsing datatypes from the json internal representation, PySpark doesn't support decimals with negative scales. Since they are allowed and can actually happen, PySpark should be able to successfully parse them.

## How was this patch tested?

added test

Closes #23575 from mgaido91/SPARK-26645.

Authored-by: Marco Gaido <marcogaido91@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-01-20 17:43:50 +08:00
Sean Owen c2d0d700b5 [SPARK-26640][CORE][ML][SQL][STREAMING][PYSPARK] Code cleanup from lgtm.com analysis
## What changes were proposed in this pull request?

Misc code cleanup from lgtm.com analysis. See comments below for details.

## How was this patch tested?

Existing tests.

Closes #23571 from srowen/SPARK-26640.

Lead-authored-by: Sean Owen <sean.owen@databricks.com>
Co-authored-by: Hyukjin Kwon <gurwls223@apache.org>
Co-authored-by: Sean Owen <srowen@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-01-17 19:40:39 -06:00
deepyaman 68496c1af3 [SPARK-26451][SQL] Change lead/lag argument name from count to offset
## What changes were proposed in this pull request?

Change aligns argument name with that in Scala version and documentation.

## How was this patch tested?

(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 http://spark.apache.org/contributing.html before opening a pull request.

Closes #23357 from deepyaman/patch-1.

Authored-by: deepyaman <deepyaman.datta@utexas.edu>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2018-12-28 00:02:41 +08:00
Maxim Gekk 7c7fccfeb5 [SPARK-26424][SQL] Use java.time API in date/timestamp expressions
## What changes were proposed in this pull request?

In the PR, I propose to switch the `DateFormatClass`, `ToUnixTimestamp`, `FromUnixTime`, `UnixTime` on java.time API for parsing/formatting dates and timestamps. The API has been already implemented by the `Timestamp`/`DateFormatter` classes. One of benefit is those classes support parsing timestamps with microsecond precision. Old behaviour can be switched on via SQL config: `spark.sql.legacy.timeParser.enabled` (`false` by default).

## How was this patch tested?

It was tested by existing test suites - `DateFunctionsSuite`, `DateExpressionsSuite`, `JsonSuite`, `CsvSuite`, `SQLQueryTestSuite` as well as PySpark tests.

Closes #23358 from MaxGekk/new-time-cast.

Lead-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Co-authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-12-27 11:09:50 +08:00
Maxim Gekk 1008ab0801 [SPARK-26178][SPARK-26243][SQL][FOLLOWUP] Replacing SimpleDateFormat by DateTimeFormatter in comments
## What changes were proposed in this pull request?

The PRs #23150 and #23196 switched JSON and CSV datasources on new formatter for dates/timestamps which is based on `DateTimeFormatter`. In this PR, I replaced `SimpleDateFormat` by `DateTimeFormatter` to reflect the changes.

Closes #23374 from MaxGekk/java-time-docs.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2018-12-24 10:47:47 +08:00
Li Jin 86100df54b [SPARK-24561][SQL][PYTHON] User-defined window aggregation functions with Pandas UDF (bounded window)
## What changes were proposed in this pull request?

This PR implements a new feature - window aggregation Pandas UDF for bounded window.

#### Doc:
https://docs.google.com/document/d/14EjeY5z4-NC27-SmIP9CsMPCANeTcvxN44a7SIJtZPc/edit#heading=h.c87w44wcj3wj

#### Example:
```
from pyspark.sql.functions import pandas_udf, PandasUDFType
from pyspark.sql.window import Window

df = spark.range(0, 10, 2).toDF('v')
w1 = Window.partitionBy().orderBy('v').rangeBetween(-2, 4)
w2 = Window.partitionBy().orderBy('v').rowsBetween(-2, 2)

pandas_udf('double', PandasUDFType.GROUPED_AGG)
def avg(v):
    return v.mean()

df.withColumn('v_mean', avg(df['v']).over(w1)).show()
# +---+------+
# |  v|v_mean|
# +---+------+
# |  0|   1.0|
# |  2|   2.0|
# |  4|   4.0|
# |  6|   6.0|
# |  8|   7.0|
# +---+------+

df.withColumn('v_mean', avg(df['v']).over(w2)).show()
# +---+------+
# |  v|v_mean|
# +---+------+
# |  0|   2.0|
# |  2|   3.0|
# |  4|   4.0|
# |  6|   5.0|
# |  8|   6.0|
# +---+------+

```

#### High level changes:

This PR modifies the existing WindowInPandasExec physical node to deal with unbounded (growing, shrinking and sliding) windows.

* `WindowInPandasExec` now share the same base class as `WindowExec` and share utility functions. See `WindowExecBase`
* `WindowFunctionFrame` now has two new functions `currentLowerBound` and `currentUpperBound` - to return the lower and upper window bound for the current output row. It is also modified to allow `AggregateProcessor` == null. Null aggregator processor is used for `WindowInPandasExec` where we don't have an aggregator and only uses lower and upper bound functions from `WindowFunctionFrame`
* The biggest change is in `WindowInPandasExec`, where it is modified to take `currentLowerBound` and `currentUpperBound` and write those values together with the input data to the python process for rolling window aggregation. See `WindowInPandasExec` for more details.

#### Discussion
In benchmarking, I found numpy variant of the rolling window UDF is much faster than the pandas version:

Spark SQL window function: 20s
Pandas variant: ~80s
Numpy variant: 10s
Numpy variant with numba: 4s

Allowing numpy variant of the vectorized UDFs is something I want to discuss because of the performance improvement, but doesn't have to be in this PR.

## How was this patch tested?

New tests

Closes #22305 from icexelloss/SPARK-24561-bounded-window-udf.

Authored-by: Li Jin <ice.xelloss@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2018-12-18 09:15:21 +08:00
Li Jin 160e583a17 [SPARK-26364][PYTHON][TESTING] Clean up imports in test_pandas_udf*
## What changes were proposed in this pull request?

Clean up unconditional import statements and move them to the top.

Conditional imports (pandas, numpy, pyarrow) are left as-is.

## How was this patch tested?

Exising tests.

Closes #23314 from icexelloss/clean-up-test-imports.

Authored-by: Li Jin <ice.xelloss@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2018-12-14 10:45:24 +08:00
Takuya UESHIN 8edae94fa7 [SPARK-26355][PYSPARK] Add a workaround for PyArrow 0.11.
## What changes were proposed in this pull request?

In PyArrow 0.11, there is a API breaking change.

- [ARROW-1949](https://issues.apache.org/jira/browse/ARROW-1949) - [Python/C++] Add option to Array.from_pandas and pyarrow.array to perform unsafe casts.

This causes test failures in `ScalarPandasUDFTests.test_vectorized_udf_null_(byte|short|int|long)`:

```
  File "/Users/ueshin/workspace/apache-spark/spark/python/pyspark/worker.py", line 377, in main
    process()
  File "/Users/ueshin/workspace/apache-spark/spark/python/pyspark/worker.py", line 372, in process
    serializer.dump_stream(func(split_index, iterator), outfile)
  File "/Users/ueshin/workspace/apache-spark/spark/python/pyspark/serializers.py", line 317, in dump_stream
    batch = _create_batch(series, self._timezone)
  File "/Users/ueshin/workspace/apache-spark/spark/python/pyspark/serializers.py", line 286, in _create_batch
    arrs = [create_array(s, t) for s, t in series]
  File "/Users/ueshin/workspace/apache-spark/spark/python/pyspark/serializers.py", line 284, in create_array
    return pa.Array.from_pandas(s, mask=mask, type=t)
  File "pyarrow/array.pxi", line 474, in pyarrow.lib.Array.from_pandas
    return array(obj, mask=mask, type=type, safe=safe, from_pandas=True,
  File "pyarrow/array.pxi", line 169, in pyarrow.lib.array
    return _ndarray_to_array(values, mask, type, from_pandas, safe,
  File "pyarrow/array.pxi", line 69, in pyarrow.lib._ndarray_to_array
    check_status(NdarrayToArrow(pool, values, mask, from_pandas,
  File "pyarrow/error.pxi", line 81, in pyarrow.lib.check_status
    raise ArrowInvalid(message)
ArrowInvalid: Floating point value truncated
```

We should add a workaround to support PyArrow 0.11.

## How was this patch tested?

In my local environment.

Closes #23305 from ueshin/issues/SPARK-26355/pyarrow_0.11.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2018-12-13 13:14:59 +08:00
Maxim Gekk 4e1d859c19 [SPARK-26303][SQL] Return partial results for bad JSON records
## What changes were proposed in this pull request?

In the PR, I propose to return partial results from JSON datasource and JSON functions in the PERMISSIVE mode if some of JSON fields are parsed and converted to desired types successfully. The changes are made only for `StructType`. Whole bad JSON records are placed into the corrupt column specified by the `columnNameOfCorruptRecord` option or SQL config.

Partial results are not returned for malformed JSON input.

## How was this patch tested?

Added new UT which checks converting JSON strings with one invalid and one valid field at the end of the string.

Closes #23253 from MaxGekk/json-bad-record.

Lead-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2018-12-11 16:06:57 +08:00
Wenchen Fan 7d5f6e8c49 [SPARK-26293][SQL] Cast exception when having python udf in subquery
## What changes were proposed in this pull request?

This is a regression introduced by https://github.com/apache/spark/pull/22104 at Spark 2.4.0.

When we have Python UDF in subquery, we will hit an exception
```
Caused by: java.lang.ClassCastException: org.apache.spark.sql.catalyst.expressions.AttributeReference cannot be cast to org.apache.spark.sql.catalyst.expressions.PythonUDF
	at scala.collection.immutable.Stream.map(Stream.scala:414)
	at org.apache.spark.sql.execution.python.EvalPythonExec.$anonfun$doExecute$2(EvalPythonExec.scala:98)
	at org.apache.spark.rdd.RDD.$anonfun$mapPartitions$2(RDD.scala:815)
...
```

https://github.com/apache/spark/pull/22104 turned `ExtractPythonUDFs` from a physical rule to optimizer rule. However, there is a difference between a physical rule and optimizer rule. A physical rule always runs once, an optimizer rule may be applied twice on a query tree even the rule is located in a batch that only runs once.

For a subquery, the `OptimizeSubqueries` rule will execute the entire optimizer on the query plan inside subquery. Later on subquery will be turned to joins, and the optimizer rules will be applied to it again.

Unfortunately, the `ExtractPythonUDFs` rule is not idempotent. When it's applied twice on a query plan inside subquery, it will produce a malformed plan. It extracts Python UDF from Python exec plans.

This PR proposes 2 changes to be double safe:
1. `ExtractPythonUDFs` should skip python exec plans, to make the rule idempotent
2. `ExtractPythonUDFs` should skip subquery

## How was this patch tested?

a new test.

Closes #23248 from cloud-fan/python.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-12-11 14:16:51 +08:00
Bryan Cutler ecaa495b1f [SPARK-25274][PYTHON][SQL] In toPandas with Arrow send un-ordered record batches to improve performance
## What changes were proposed in this pull request?

When executing `toPandas` with Arrow enabled, partitions that arrive in the JVM out-of-order must be buffered before they can be send to Python. This causes an excess of memory to be used in the driver JVM and increases the time it takes to complete because data must sit in the JVM waiting for preceding partitions to come in.

This change sends un-ordered partitions to Python as soon as they arrive in the JVM, followed by a list of partition indices so that Python can assemble the data in the correct order. This way, data is not buffered at the JVM and there is no waiting on particular partitions so performance will be increased.

Followup to #21546

## How was this patch tested?

Added new test with a large number of batches per partition, and test that forces a small delay in the first partition. These test that partitions are collected out-of-order and then are are put in the correct order in Python.

## Performance Tests - toPandas

Tests run on a 4 node standalone cluster with 32 cores total, 14.04.1-Ubuntu and OpenJDK 8
measured wall clock time to execute `toPandas()` and took the average best time of 5 runs/5 loops each.

Test code
```python
df = spark.range(1 << 25, numPartitions=32).toDF("id").withColumn("x1", rand()).withColumn("x2", rand()).withColumn("x3", rand()).withColumn("x4", rand())
for i in range(5):
	start = time.time()
	_ = df.toPandas()
	elapsed = time.time() - start
```

Spark config
```
spark.driver.memory 5g
spark.executor.memory 5g
spark.driver.maxResultSize 2g
spark.sql.execution.arrow.enabled true
```

Current Master w/ Arrow stream | This PR
---------------------|------------
5.16207 | 4.342533
5.133671 | 4.399408
5.147513 | 4.468471
5.105243 | 4.36524
5.018685 | 4.373791

Avg Master | Avg This PR
------------------|--------------
5.1134364 | 4.3898886

Speedup of **1.164821449**

Closes #22275 from BryanCutler/arrow-toPandas-oo-batches-SPARK-25274.

Authored-by: Bryan Cutler <cutlerb@gmail.com>
Signed-off-by: Bryan Cutler <cutlerb@gmail.com>
2018-12-06 10:07:28 -08:00
DylanGuedes 28d3374407 [SPARK-23647][PYTHON][SQL] Adds more types for hint in pyspark
Signed-off-by: DylanGuedes <djmgguedesgmail.com>

## What changes were proposed in this pull request?

Addition of float, int and list hints for `pyspark.sql` Hint.

## How was this patch tested?

I did manual tests following the same principles used in the Scala version, and also added unit tests.

Closes #20788 from DylanGuedes/jira-21030.

Authored-by: DylanGuedes <djmgguedes@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2018-12-01 10:37:03 +08:00
Wenchen Fan fa0d4bf699 [SPARK-25829][SQL] remove duplicated map keys with last wins policy
## What changes were proposed in this pull request?

Currently duplicated map keys are not handled consistently. For example, map look up respects the duplicated key appears first, `Dataset.collect` only keeps the duplicated key appears last, `MapKeys` returns duplicated keys, etc.

This PR proposes to remove duplicated map keys with last wins policy, to follow Java/Scala and Presto. It only applies to built-in functions, as users can create map with duplicated map keys via private APIs anyway.

updated functions: `CreateMap`, `MapFromArrays`, `MapFromEntries`, `StringToMap`, `MapConcat`, `TransformKeys`.

For other places:
1. data source v1 doesn't have this problem, as users need to provide a java/scala map, which can't have duplicated keys.
2. data source v2 may have this problem. I've added a note to `ArrayBasedMapData` to ask the caller to take care of duplicated keys. In the future we should enforce it in the stable data APIs for data source v2.
3. UDF doesn't have this problem, as users need to provide a java/scala map. Same as data source v1.
4. file format. I checked all of them and only parquet does not enforce it. For backward compatibility reasons I change nothing but leave a note saying that the behavior will be undefined if users write map with duplicated keys to parquet files. Maybe we can add a config and fail by default if parquet files have map with duplicated keys. This can be done in followup.

## How was this patch tested?

updated tests and new tests

Closes #23124 from cloud-fan/map.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-11-28 23:42:13 +08:00
Wenchen Fan affe80958d [SPARK-26147][SQL] only pull out unevaluable python udf from join condition
## What changes were proposed in this pull request?

https://github.com/apache/spark/pull/22326 made a mistake that, not all python UDFs are unevaluable in join condition. Only python UDFs that refer to attributes from both join side are unevaluable.

This PR fixes this mistake.

## How was this patch tested?

a new test

Closes #23153 from cloud-fan/join.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-11-28 20:38:42 +08:00
Juliusz Sompolski 8c6871828e [SPARK-26159] Codegen for LocalTableScanExec and RDDScanExec
## What changes were proposed in this pull request?

Implement codegen for `LocalTableScanExec` and `ExistingRDDExec`. Refactor to share code between `LocalTableScanExec`, `ExistingRDDExec`, `InputAdapter` and `RowDataSourceScanExec`.

The difference in `doProduce` between these four was that `ExistingRDDExec` and `RowDataSourceScanExec` triggered adding an `UnsafeProjection`, while `InputAdapter` and `LocalTableScanExec` did not.

In the new trait `InputRDDCodegen` I added a flag `createUnsafeProjection` which the operators set accordingly.

Note: `LocalTableScanExec` explicitly creates its input as `UnsafeRows`, so it was obvious why it doesn't need an `UnsafeProjection`. But if an `InputAdapter` may take input that is `InternalRows` but not `UnsafeRows`, then I think it doesn't need an unsafe projection just because any other operator that is its parent would do that. That assumes that that any parent operator would always result in some `UnsafeProjection` being eventually added, and hence the output of the `WholeStageCodegen` unit would be `UnsafeRows`. If these assumptions hold, I think `createUnsafeProjection` could be set to `(parent == null)`.

Note: Do not codegen `LocalTableScanExec` when it's the only operator. `LocalTableScanExec` has optimized driver-only `executeCollect` and `executeTake` code paths that are used to return `Command` results without starting Spark Jobs. They can no longer be used if the `LocalTableScanExec` gets optimized.

## How was this patch tested?

Covered and used in existing tests.

Closes #23127 from juliuszsompolski/SPARK-26159.

Authored-by: Juliusz Sompolski <julek@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-11-28 13:37:11 +08:00
gatorsmile 94145786a5 [SPARK-25908][SQL][FOLLOW-UP] Add back unionAll
## What changes were proposed in this pull request?
This PR is to add back `unionAll`, which is widely used. The name is also consistent with our ANSI SQL. We also have the corresponding `intersectAll` and `exceptAll`, which were introduced in Spark 2.4.

## How was this patch tested?
Added a test case in DataFrameSuite

Closes #23131 from gatorsmile/addBackUnionAll.

Authored-by: gatorsmile <gatorsmile@gmail.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2018-11-25 15:53:07 -08:00
Katrin Leinweber c5daccb1da [MINOR] Update all DOI links to preferred resolver
## What changes were proposed in this pull request?

The DOI foundation recommends [this new resolver](https://www.doi.org/doi_handbook/3_Resolution.html#3.8). Accordingly, this PR re`sed`s all static DOI links ;-)

## How was this patch tested?

It wasn't, since it seems as safe as a "[typo fix](https://spark.apache.org/contributing.html)".

In case any of the files is included from other projects, and should be updated there, please let me know.

Closes #23129 from katrinleinweber/resolve-DOIs-securely.

Authored-by: Katrin Leinweber <9948149+katrinleinweber@users.noreply.github.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-11-25 17:43:55 -06:00