Commit graph

172 commits

Author SHA1 Message Date
hyukjinkwon 67c75021c5 [SPARK-21163][SQL] DataFrame.toPandas should respect the data type
## 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.
2017-06-22 16:22:02 +08:00
zero323 215281d88e [SPARK-20830][PYSPARK][SQL] Add posexplode and posexplode_outer
## What changes were proposed in this pull request?

Add Python wrappers for `o.a.s.sql.functions.explode_outer` and `o.a.s.sql.functions.posexplode_outer`.

## How was this patch tested?

Unit tests, doctests.

Author: zero323 <zero323@users.noreply.github.com>

Closes #18049 from zero323/SPARK-20830.
2017-06-21 14:59:52 -07:00
Xiao Li 2051428173 [SPARK-20980][SQL] Rename wholeFile to multiLine for both CSV and JSON
### What changes were proposed in this pull request?
The current option name `wholeFile` is misleading for CSV users. Currently, it is not representing a record per file. Actually, one file could have multiple records. Thus, we should rename it. Now, the proposal is `multiLine`.

### How was this patch tested?
N/A

Author: Xiao Li <gatorsmile@gmail.com>

Closes #18202 from gatorsmile/renameCVSOption.
2017-06-15 13:18:19 +08:00
Ruben Berenguel Montoro 6cbc61d107 [SPARK-19732][SQL][PYSPARK] Add fill functions for nulls in bool fields of datasets
## 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.
2017-06-03 14:56:42 +09:00
hyukjinkwon 720708ccdd [SPARK-20639][SQL] Add single argument support for to_timestamp in SQL with documentation improvement
## What changes were proposed in this pull request?

This PR proposes three things as below:

- Use casting rules to a timestamp in `to_timestamp` by default (it was `yyyy-MM-dd HH:mm:ss`).

- Support single argument for `to_timestamp` similarly with APIs in other languages.

  For example, the one below works

  ```
  import org.apache.spark.sql.functions._
  Seq("2016-12-31 00:12:00.00").toDF("a").select(to_timestamp(col("a"))).show()
  ```

  prints

  ```
  +----------------------------------------+
  |to_timestamp(`a`, 'yyyy-MM-dd HH:mm:ss')|
  +----------------------------------------+
  |                     2016-12-31 00:12:00|
  +----------------------------------------+
  ```

  whereas this does not work in SQL.

  **Before**

  ```
  spark-sql> SELECT to_timestamp('2016-12-31 00:12:00');
  Error in query: Invalid number of arguments for function to_timestamp; line 1 pos 7
  ```

  **After**

  ```
  spark-sql> SELECT to_timestamp('2016-12-31 00:12:00');
  2016-12-31 00:12:00
  ```

- Related document improvement for SQL function descriptions and other API descriptions accordingly.

  **Before**

  ```
  spark-sql> DESCRIBE FUNCTION extended to_date;
  ...
  Usage: to_date(date_str, fmt) - Parses the `left` expression with the `fmt` expression. Returns null with invalid input.
  Extended Usage:
      Examples:
        > SELECT to_date('2016-12-31', 'yyyy-MM-dd');
         2016-12-31
  ```

  ```
  spark-sql> DESCRIBE FUNCTION extended to_timestamp;
  ...
  Usage: to_timestamp(timestamp, fmt) - Parses the `left` expression with the `format` expression to a timestamp. Returns null with invalid input.
  Extended Usage:
      Examples:
        > SELECT to_timestamp('2016-12-31', 'yyyy-MM-dd');
         2016-12-31 00:00:00.0
  ```

  **After**

  ```
  spark-sql> DESCRIBE FUNCTION extended to_date;
  ...
  Usage:
      to_date(date_str[, fmt]) - Parses the `date_str` expression with the `fmt` expression to
        a date. Returns null with invalid input. By default, it follows casting rules to a date if
        the `fmt` is omitted.

  Extended Usage:
      Examples:
        > SELECT to_date('2009-07-30 04:17:52');
         2009-07-30
        > SELECT to_date('2016-12-31', 'yyyy-MM-dd');
         2016-12-31
  ```

  ```
  spark-sql> DESCRIBE FUNCTION extended to_timestamp;
  ...
   Usage:
      to_timestamp(timestamp[, fmt]) - Parses the `timestamp` expression with the `fmt` expression to
        a timestamp. Returns null with invalid input. By default, it follows casting rules to
        a timestamp if the `fmt` is omitted.

  Extended Usage:
      Examples:
        > SELECT to_timestamp('2016-12-31 00:12:00');
         2016-12-31 00:12:00
        > SELECT to_timestamp('2016-12-31', 'yyyy-MM-dd');
         2016-12-31 00:00:00
  ```

## How was this patch tested?

Added tests in `datetime.sql`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17901 from HyukjinKwon/to_timestamp_arg.
2017-05-12 16:42:58 +08:00
Josh Rosen 8ddbc431d8 [SPARK-20685] Fix BatchPythonEvaluation bug in case of single UDF w/ repeated arg.
## What changes were proposed in this pull request?

There's a latent corner-case bug in PySpark UDF evaluation where executing a `BatchPythonEvaluation` with a single multi-argument UDF where _at least one argument value is repeated_ will crash at execution with a confusing error.

This problem was introduced in #12057: the code there has a fast path for handling a "batch UDF evaluation consisting of a single Python UDF", but that branch incorrectly assumes that a single UDF won't have repeated arguments and therefore skips the code for unpacking arguments from the input row (whose schema may not necessarily match the UDF inputs due to de-duplication of repeated arguments which occurred in the JVM before sending UDF inputs to Python).

This fix here is simply to remove this special-casing: it turns out that the code in the "multiple UDFs" branch just so happens to work for the single-UDF case because Python treats `(x)` as equivalent to `x`, not as a single-argument tuple.

## How was this patch tested?

New regression test in `pyspark.python.sql.tests` module (tested and confirmed that it fails before my fix).

Author: Josh Rosen <joshrosen@databricks.com>

Closes #17927 from JoshRosen/SPARK-20685.
2017-05-10 16:50:57 -07:00
zero323 f53a820721 [SPARK-16931][PYTHON][SQL] Add Python wrapper for bucketBy
## What changes were proposed in this pull request?

Adds Python wrappers for `DataFrameWriter.bucketBy` and `DataFrameWriter.sortBy` ([SPARK-16931](https://issues.apache.org/jira/browse/SPARK-16931))

## How was this patch tested?

Unit tests covering new feature.

__Note__: Based on work of GregBowyer (f49b9a23468f7af32cb53d2b654272757c151725)

CC HyukjinKwon

Author: zero323 <zero323@users.noreply.github.com>
Author: Greg Bowyer <gbowyer@fastmail.co.uk>

Closes #17077 from zero323/SPARK-16931.
2017-05-08 10:58:27 +08:00
zero323 63d90e7da4 [SPARK-18777][PYTHON][SQL] Return UDF from udf.register
## What changes were proposed in this pull request?

- Move udf wrapping code from `functions.udf` to `functions.UserDefinedFunction`.
- Return wrapped udf from `catalog.registerFunction` and dependent methods.
- Update docstrings in `catalog.registerFunction` and `SQLContext.registerFunction`.
- Unit tests.

## How was this patch tested?

- Existing unit tests and docstests.
- Additional tests covering new feature.

Author: zero323 <zero323@users.noreply.github.com>

Closes #17831 from zero323/SPARK-18777.
2017-05-06 22:28:42 -07:00
zero323 02bbe73118 [SPARK-20584][PYSPARK][SQL] Python generic hint support
## 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.
2017-05-03 19:15:28 -07:00
zero323 f0169a1c6a [SPARK-20290][MINOR][PYTHON][SQL] Add PySpark wrapper for eqNullSafe
## What changes were proposed in this pull request?

Adds Python bindings for `Column.eqNullSafe`

## How was this patch tested?

Manual tests, existing unit tests, doc build.

Author: zero323 <zero323@users.noreply.github.com>

Closes #17605 from zero323/SPARK-20290.
2017-05-01 09:43:32 -07:00
Srinivasa Reddy Vundela 6613046c8c [MINOR][DOCS][PYTHON] Adding missing boolean type for replacement value in fillna
## 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.
2017-04-30 21:42:05 -07:00
zero323 e2773996b8 [SPARK-19454][PYTHON][SQL] DataFrame.replace improvements
## 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.
2017-04-05 11:47:40 -07:00
hyukjinkwon 3fbf0a5f92 [MINOR][DOCS] Match several documentation changes in Scala to R/Python
## What changes were proposed in this pull request?

This PR proposes to match minor documentations changes in https://github.com/apache/spark/pull/17399 and https://github.com/apache/spark/pull/17380 to R/Python.

## How was this patch tested?

Manual tests in Python , Python tests via `./python/run-tests.py --module=pyspark-sql` and lint-checks for Python/R.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17429 from HyukjinKwon/minor-match-doc.
2017-03-26 18:40:00 -07:00
Tyson Condie 746a558de2 [SPARK-19876][SS][WIP] OneTime Trigger Executor
## What changes were proposed in this pull request?

An additional trigger and trigger executor that will execute a single trigger only. One can use this OneTime trigger to have more control over the scheduling of triggers.

In addition, this patch requires an optimization to StreamExecution that logs a commit record at the end of successfully processing a batch. This new commit log will be used to determine the next batch (offsets) to process after a restart, instead of using the offset log itself to determine what batch to process next after restart; using the offset log to determine this would process the previously logged batch, always, thus not permitting a OneTime trigger feature.

## How was this patch tested?

A number of existing tests have been revised. These tests all assumed that when restarting a stream, the last batch in the offset log is to be re-processed. Given that we now have a commit log that will tell us if that last batch was processed successfully, the results/assumptions of those tests needed to be revised accordingly.

In addition, a OneTime trigger test was added to StreamingQuerySuite, which tests:
- The semantics of OneTime trigger (i.e., on start, execute a single batch, then stop).
- The case when the commit log was not able to successfully log the completion of a batch before restart, which would mean that we should fall back to what's in the offset log.
- A OneTime trigger execution that results in an exception being thrown.

marmbrus tdas zsxwing

Please review http://spark.apache.org/contributing.html before opening a pull request.

Author: Tyson Condie <tcondie@gmail.com>
Author: Tathagata Das <tathagata.das1565@gmail.com>

Closes #17219 from tcondie/stream-commit.
2017-03-23 14:32:05 -07:00
hyukjinkwon 07c12c09a7 [SPARK-18579][SQL] Use ignoreLeadingWhiteSpace and ignoreTrailingWhiteSpace options in CSV writing
## What changes were proposed in this pull request?

This PR proposes to support _not_ trimming the white spaces when writing out. These are `false` by default in CSV reading path but these are `true` by default in CSV writing in univocity parser.

Both `ignoreLeadingWhiteSpace` and `ignoreTrailingWhiteSpace` options are not being used for writing and therefore, we are always trimming the white spaces.

It seems we should provide a way to keep this white spaces easily.

WIth the data below:

```scala
val df = spark.read.csv(Seq("a , b  , c").toDS)
df.show()
```

```
+---+----+---+
|_c0| _c1|_c2|
+---+----+---+
| a | b  |  c|
+---+----+---+
```

**Before**

```scala
df.write.csv("/tmp/text.csv")
spark.read.text("/tmp/text.csv").show()
```

```
+-----+
|value|
+-----+
|a,b,c|
+-----+
```

It seems this can't be worked around via `quoteAll` too.

```scala
df.write.option("quoteAll", true).csv("/tmp/text.csv")
spark.read.text("/tmp/text.csv").show()
```
```
+-----------+
|      value|
+-----------+
|"a","b","c"|
+-----------+
```

**After**

```scala
df.write.option("ignoreLeadingWhiteSpace", false).option("ignoreTrailingWhiteSpace", false).csv("/tmp/text.csv")
spark.read.text("/tmp/text.csv").show()
```

```
+----------+
|     value|
+----------+
|a , b  , c|
+----------+
```

Note that this case is possible in R

```r
> system("cat text.csv")
f1,f2,f3
a , b  , c
> df <- read.csv(file="text.csv")
> df
  f1   f2 f3
1 a   b    c
> write.csv(df, file="text1.csv", quote=F, row.names=F)
> system("cat text1.csv")
f1,f2,f3
a , b  , c
```

## How was this patch tested?

Unit tests in `CSVSuite` and manual tests for Python.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17310 from HyukjinKwon/SPARK-18579.
2017-03-23 00:25:01 -07:00
Jeff Zhang cabe1df860 [SPARK-12334][SQL][PYSPARK] Support read from multiple input paths for orc file in DataFrameReader.orc
Beside the issue in spark api, also fix 2 minor issues in pyspark
- support read from multiple input paths for orc
- support read from multiple input paths for text

Author: Jeff Zhang <zjffdu@apache.org>

Closes #10307 from zjffdu/SPARK-12334.
2017-03-09 11:44:34 -08:00
Jason White 206030bd12 [SPARK-19561][SQL] add int case handling for TimestampType
## What changes were proposed in this pull request?

Add handling of input of type `Int` for dataType `TimestampType` to `EvaluatePython.scala`. Py4J serializes ints smaller than MIN_INT or larger than MAX_INT to Long, which are handled correctly already, but values between MIN_INT and MAX_INT are serialized to Int.

These range limits correspond to roughly half an hour on either side of the epoch. As a result, PySpark doesn't allow TimestampType values to be created in this range.

Alternatives attempted: patching the `TimestampType.toInternal` function to cast return values to `long`, so Py4J would always serialize them to Scala Long. Python3 does not have a `long` type, so this approach failed on Python3.

## How was this patch tested?

Added a new PySpark-side test that fails without the change.

The contribution is my original work and I license the work to the project under the project’s open source license.

Resubmission of https://github.com/apache/spark/pull/16896. The original PR didn't go through Jenkins and broke the build. davies dongjoon-hyun

cloud-fan Could you kick off a Jenkins run for me? It passed everything for me locally, but it's possible something has changed in the last few weeks.

Author: Jason White <jason.white@shopify.com>

Closes #17200 from JasonMWhite/SPARK-19561.
2017-03-09 10:34:54 -08:00
Wenchen Fan 47b2f68a88 Revert "[SPARK-19561] [PYTHON] cast TimestampType.toInternal output to long"
This reverts commit 711addd46e.
2017-03-07 17:14:26 -08:00
Jason White 6f4684622a [SPARK-19561] [PYTHON] cast TimestampType.toInternal output to long
## What changes were proposed in this pull request?

Cast the output of `TimestampType.toInternal` to long to allow for proper Timestamp creation in DataFrames near the epoch.

## How was this patch tested?

Added a new test that fails without the change.

dongjoon-hyun davies Mind taking a look?

The contribution is my original work and I license the work to the project under the project’s open source license.

Author: Jason White <jason.white@shopify.com>

Closes #16896 from JasonMWhite/SPARK-19561.
2017-03-07 13:14:37 -08:00
hyukjinkwon 224e0e785b [SPARK-19701][SQL][PYTHON] Throws a correct exception for 'in' operator against column
## What changes were proposed in this pull request?

This PR proposes to remove incorrect implementation that has been not executed so far (at least from Spark 1.5.2) for `in` operator and throw a correct exception rather than saying it is a bool. I tested the codes above in 1.5.2, 1.6.3, 2.1.0 and in the master branch as below:

**1.5.2**

```python
>>> df = sqlContext.createDataFrame([[1]])
>>> 1 in df._1
Traceback (most recent call last):
  File "<stdin>", line 1, in <module>
  File ".../spark-1.5.2-bin-hadoop2.6/python/pyspark/sql/column.py", line 418, in __nonzero__
    raise ValueError("Cannot convert column into bool: please use '&' for 'and', '|' for 'or', "
ValueError: Cannot convert column into bool: please use '&' for 'and', '|' for 'or', '~' for 'not' when building DataFrame boolean expressions.
```

**1.6.3**

```python
>>> 1 in sqlContext.range(1).id
Traceback (most recent call last):
  File "<stdin>", line 1, in <module>
  File ".../spark-1.6.3-bin-hadoop2.6/python/pyspark/sql/column.py", line 447, in __nonzero__
    raise ValueError("Cannot convert column into bool: please use '&' for 'and', '|' for 'or', "
ValueError: Cannot convert column into bool: please use '&' for 'and', '|' for 'or', '~' for 'not' when building DataFrame boolean expressions.
```

**2.1.0**

```python
>>> 1 in spark.range(1).id
Traceback (most recent call last):
  File "<stdin>", line 1, in <module>
  File ".../spark-2.1.0-bin-hadoop2.7/python/pyspark/sql/column.py", line 426, in __nonzero__
    raise ValueError("Cannot convert column into bool: please use '&' for 'and', '|' for 'or', "
ValueError: Cannot convert column into bool: please use '&' for 'and', '|' for 'or', '~' for 'not' when building DataFrame boolean expressions.
```

**Current Master**

```python
>>> 1 in spark.range(1).id
Traceback (most recent call last):
  File "<stdin>", line 1, in <module>
  File ".../spark/python/pyspark/sql/column.py", line 452, in __nonzero__
    raise ValueError("Cannot convert column into bool: please use '&' for 'and', '|' for 'or', "
ValueError: Cannot convert column into bool: please use '&' for 'and', '|' for 'or', '~' for 'not' when building DataFrame boolean expressions.
```

**After**

```python
>>> 1 in spark.range(1).id
Traceback (most recent call last):
  File "<stdin>", line 1, in <module>
  File ".../spark/python/pyspark/sql/column.py", line 184, in __contains__
    raise ValueError("Cannot apply 'in' operator against a column: please use 'contains' "
ValueError: Cannot apply 'in' operator against a column: please use 'contains' in a string column or 'array_contains' function for an array column.
```

In more details,

It seems the implementation intended to support this

```python
1 in df.column
```

However, currently, it throws an exception as below:

```python
Traceback (most recent call last):
  File "<stdin>", line 1, in <module>
  File ".../spark/python/pyspark/sql/column.py", line 426, in __nonzero__
    raise ValueError("Cannot convert column into bool: please use '&' for 'and', '|' for 'or', "
ValueError: Cannot convert column into bool: please use '&' for 'and', '|' for 'or', '~' for 'not' when building DataFrame boolean expressions.
```

What happens here is as below:

```python
class Column(object):
    def __contains__(self, item):
        print "I am contains"
        return Column()
    def __nonzero__(self):
        raise Exception("I am nonzero.")

>>> 1 in Column()
I am contains
Traceback (most recent call last):
  File "<stdin>", line 1, in <module>
  File "<stdin>", line 6, in __nonzero__
Exception: I am nonzero.
```

It seems it calls `__contains__` first and then `__nonzero__` or `__bool__` is being called against `Column()` to make this a bool (or int to be specific).

It seems `__nonzero__` (for Python 2), `__bool__` (for Python 3) and `__contains__` forcing the the return into a bool unlike other operators. There are few references about this as below:

https://bugs.python.org/issue16011
http://stackoverflow.com/questions/12244074/python-source-code-for-built-in-in-operator/12244378#12244378
http://stackoverflow.com/questions/38542543/functionality-of-python-in-vs-contains/38542777

It seems we can't overwrite `__nonzero__` or `__bool__` as a workaround to make this working because these force the return type as a bool as below:

```python
class Column(object):
    def __contains__(self, item):
        print "I am contains"
        return Column()
    def __nonzero__(self):
        return "a"

>>> 1 in Column()
I am contains
Traceback (most recent call last):
  File "<stdin>", line 1, in <module>
TypeError: __nonzero__ should return bool or int, returned str
```

## How was this patch tested?

Added unit tests in `tests.py`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17160 from HyukjinKwon/SPARK-19701.
2017-03-05 18:04:52 -08:00
hyukjinkwon 7e5359be5c [SPARK-19610][SQL] Support parsing multiline CSV files
## What changes were proposed in this pull request?

This PR proposes the support for multiple lines for CSV by resembling the multiline supports in JSON datasource (in case of JSON, per file).

So, this PR introduces `wholeFile` option which makes the format not splittable and reads each whole file. Since Univocity parser can produces each row from a stream, it should be capable of parsing very large documents when the internal rows are fix in the memory.

## How was this patch tested?

Unit tests in `CSVSuite` and `tests.py`

Manual tests with a single 9GB CSV file in local file system, for example,

```scala
spark.read.option("wholeFile", true).option("inferSchema", true).csv("tmp.csv").count()
```

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16976 from HyukjinKwon/SPARK-19610.
2017-02-28 13:34:33 -08:00
zero323 4a5e38f574 [SPARK-19161][PYTHON][SQL] Improving UDF Docstrings
## What changes were proposed in this pull request?

Replaces `UserDefinedFunction` object returned from `udf` with a function wrapper providing docstring and arguments information as proposed in [SPARK-19161](https://issues.apache.org/jira/browse/SPARK-19161).

### Backward incompatible changes:

- `pyspark.sql.functions.udf` will return a `function` instead of `UserDefinedFunction`. To ensure backward compatible public API we use function attributes to mimic  `UserDefinedFunction` API (`func` and `returnType` attributes).  This should have a minimal impact on the user code.

  An alternative implementation could use dynamical sub-classing. This would ensure full backward compatibility but is more fragile in practice.

### Limitations:

Full functionality (retained docstring and argument list) is achieved only in the recent Python version. Legacy Python version will preserve only docstrings, but not argument list. This should be an acceptable trade-off between achieved improvements and overall complexity.

### Possible impact on other tickets:

This can affect [SPARK-18777](https://issues.apache.org/jira/browse/SPARK-18777).

## How was this patch tested?

Existing unit tests to ensure backward compatibility, additional tests targeting proposed changes.

Author: zero323 <zero323@users.noreply.github.com>

Closes #16534 from zero323/SPARK-19161.
2017-02-24 08:22:30 -08:00
Wenchen Fan 4fa4cf1d4c [SPARK-19706][PYSPARK] add Column.contains in pyspark
## What changes were proposed in this pull request?

to be consistent with the scala API, we should also add `contains` to `Column` in pyspark.

## How was this patch tested?

updated unit test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #17036 from cloud-fan/pyspark.
2017-02-23 13:22:39 -08:00
Nathan Howell 21fde57f15 [SPARK-18352][SQL] Support parsing multiline json files
## What changes were proposed in this pull request?

If a new option `wholeFile` is set to `true` the JSON reader will parse each file (instead of a single line) as a value. This is done with Jackson streaming and it should be capable of parsing very large documents, assuming the row will fit in memory.

Because the file is not buffered in memory the corrupt record handling is also slightly different when `wholeFile` is enabled: the corrupt column will contain the filename instead of the literal JSON if there is a parsing failure. It would be easy to extend this to add the parser location (line, column and byte offsets) to the output if desired.

These changes have allowed types other than `String` to be parsed. Support for `UTF8String` and `Text` have been added (alongside `String` and `InputFormat`) and no longer require a conversion to `String` just for parsing.

I've also included a few other changes that generate slightly better bytecode and (imo) make it more obvious when and where boxing is occurring in the parser. These are included as separate commits, let me know if they should be flattened into this PR or moved to a new one.

## How was this patch tested?

New and existing unit tests. No performance or load tests have been run.

Author: Nathan Howell <nhowell@godaddy.com>

Closes #16386 from NathanHowell/SPARK-18352.
2017-02-16 20:51:19 -08:00
zero323 c97f4e17de [SPARK-19160][PYTHON][SQL] Add udf decorator
## What changes were proposed in this pull request?

This PR adds `udf` decorator syntax as proposed in [SPARK-19160](https://issues.apache.org/jira/browse/SPARK-19160).

This allows users to define UDF using simplified syntax:

```python
from pyspark.sql.decorators import udf

udf(IntegerType())
def add_one(x):
    """Adds one"""
    if x is not None:
        return x + 1
 ```

without need to define a separate function and udf.

## How was this patch tested?

Existing unit tests to ensure backward compatibility and additional unit tests covering new functionality.

Author: zero323 <zero323@users.noreply.github.com>

Closes #16533 from zero323/SPARK-19160.
2017-02-15 10:16:34 -08:00
Sheamus K. Parkes 7b64f7aa03 [SPARK-18541][PYTHON] Add metadata parameter to pyspark.sql.Column.alias()
## What changes were proposed in this pull request?

Add a `metadata` keyword parameter to `pyspark.sql.Column.alias()` to allow users to mix-in metadata while manipulating `DataFrame`s in `pyspark`.  Without this, I believe it was necessary to pass back through `SparkSession.createDataFrame` each time a user wanted to manipulate `StructField.metadata` in `pyspark`.

This pull request also improves consistency between the Scala and Python APIs (i.e. I did not add any functionality that was not already in the Scala API).

Discussed ahead of time on JIRA with marmbrus

## How was this patch tested?

Added unit tests (and doc tests).  Ran the pertinent tests manually.

Author: Sheamus K. Parkes <shea.parkes@milliman.com>

Closes #16094 from shea-parkes/pyspark-column-alias-metadata.
2017-02-14 09:57:43 -08:00
zero323 e0eeb0f89f [SPARK-19162][PYTHON][SQL] UserDefinedFunction should validate that func is callable
## What changes were proposed in this pull request?

UDF constructor checks if `func` argument is callable and if it is not, fails fast instead of waiting for an action.

## How was this patch tested?

Unit tests.

Author: zero323 <zero323@users.noreply.github.com>

Closes #16535 from zero323/SPARK-19162.
2017-02-14 09:46:22 -08:00
zero323 e02ac303c6 [SPARK-19429][PYTHON][SQL] Support slice arguments in Column.__getitem__
## What changes were proposed in this pull request?

- Add support for `slice` arguments in `Column.__getitem__`.
- Remove obsolete `__getslice__` bindings.

## How was this patch tested?

Existing unit tests, additional tests covering `[]` with `slice`.

Author: zero323 <zero323@users.noreply.github.com>

Closes #16771 from zero323/SPARK-19429.
2017-02-13 15:23:56 -08:00
zero323 ab88b24106 [SPARK-19427][PYTHON][SQL] Support data type string as a returnType argument of UDF
## What changes were proposed in this pull request?

Add support for data type string as a return type argument of `UserDefinedFunction`:

```python
f = udf(lambda x: x, "integer")
 f.returnType

## IntegerType
```

## How was this patch tested?

Existing unit tests, additional unit tests covering new feature.

Author: zero323 <zero323@users.noreply.github.com>

Closes #16769 from zero323/SPARK-19427.
2017-02-13 10:37:34 -08:00
anabranch 7a7ce272fe [SPARK-16609] Add to_date/to_timestamp with format functions
## What changes were proposed in this pull request?

This pull request adds two new user facing functions:
- `to_date` which accepts an expression and a format and returns a date.
- `to_timestamp` which accepts an expression and a format and returns a timestamp.

For example, Given a date in format: `2016-21-05`. (YYYY-dd-MM)

### Date Function
*Previously*
```
to_date(unix_timestamp(lit("2016-21-05"), "yyyy-dd-MM").cast("timestamp"))
```
*Current*
```
to_date(lit("2016-21-05"), "yyyy-dd-MM")
```

### Timestamp Function
*Previously*
```
unix_timestamp(lit("2016-21-05"), "yyyy-dd-MM").cast("timestamp")
```
*Current*
```
to_timestamp(lit("2016-21-05"), "yyyy-dd-MM")
```
### Tasks

- [X] Add `to_date` to Scala Functions
- [x] Add `to_date` to Python Functions
- [x] Add `to_date` to SQL Functions
- [X] Add `to_timestamp` to Scala Functions
- [x] Add `to_timestamp` to Python Functions
- [x] Add `to_timestamp` to SQL Functions
- [x] Add function to R

## How was this patch tested?

- [x] Add Functions to `DateFunctionsSuite`
- Test new `ParseToTimestamp` Expression (*not necessary*)
- Test new `ParseToDate` Expression (*not necessary*)
- [x] Add test for R
- [x] Add test for Python in test.py

Please review http://spark.apache.org/contributing.html before opening a pull request.

Author: anabranch <wac.chambers@gmail.com>
Author: Bill Chambers <bill@databricks.com>
Author: anabranch <bill@databricks.com>

Closes #16138 from anabranch/SPARK-16609.
2017-02-07 15:50:30 +01:00
Zheng RuiFeng b0985764f0 [SPARK-14352][SQL] approxQuantile should support multi columns
## 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.
2017-02-01 14:11:28 -08:00
zero323 9063835803 [SPARK-19163][PYTHON][SQL] Delay _judf initialization to the __call__
## What changes were proposed in this pull request?

Defer `UserDefinedFunction._judf` initialization to the first call. This prevents unintended `SparkSession` initialization.  This allows users to define and import UDF without creating a context / session as a side effect.

[SPARK-19163](https://issues.apache.org/jira/browse/SPARK-19163)

## How was this patch tested?

Unit tests.

Author: zero323 <zero323@users.noreply.github.com>

Closes #16536 from zero323/SPARK-19163.
2017-01-31 18:03:39 -08:00
gatorsmile 772035e771 [SPARK-19229][SQL] Disallow Creating Hive Source Tables when Hive Support is Not Enabled
### What changes were proposed in this pull request?
It is weird to create Hive source tables when using InMemoryCatalog. We are unable to operate it. This PR is to block users to create Hive source tables.

### How was this patch tested?
Fixed the test cases

Author: gatorsmile <gatorsmile@gmail.com>

Closes #16587 from gatorsmile/blockHiveTable.
2017-01-22 20:37:37 -08:00
Davies Liu 9b7a03f15a [SPARK-18589][SQL] Fix Python UDF accessing attributes from both side of join
## What changes were proposed in this pull request?

PythonUDF is unevaluable, which can not be used inside a join condition, currently the optimizer will push a PythonUDF which accessing both side of join into the join condition, then the query will fail to plan.

This PR fix this issue by checking the expression is evaluable  or not before pushing it into Join.

## How was this patch tested?

Add a regression test.

Author: Davies Liu <davies@databricks.com>

Closes #16581 from davies/pyudf_join.
2017-01-20 16:11:40 -08:00
Liang-Chi Hsieh d06172b88e [SPARK-19223][SQL][PYSPARK] Fix InputFileBlockHolder for datasources which are based on HadoopRDD or NewHadoopRDD
## What changes were proposed in this pull request?

For some datasources which are based on HadoopRDD or NewHadoopRDD, such as spark-xml, InputFileBlockHolder doesn't work with Python UDF.

The method to reproduce it is, running the following codes with `bin/pyspark --packages com.databricks:spark-xml_2.11:0.4.1`:

    from pyspark.sql.functions import udf,input_file_name
    from pyspark.sql.types import StringType
    from pyspark.sql import SparkSession

    def filename(path):
        return path

    session = SparkSession.builder.appName('APP').getOrCreate()

    session.udf.register('sameText', filename)
    sameText = udf(filename, StringType())

    df = session.read.format('xml').load('a.xml', rowTag='root').select('*', input_file_name().alias('file'))
    df.select('file').show() # works
    df.select(sameText(df['file'])).show()   # returns empty content

The issue is because in `HadoopRDD` and `NewHadoopRDD` we set the file block's info in `InputFileBlockHolder` before the returned iterator begins consuming. `InputFileBlockHolder` will record this info into thread local variable. When running Python UDF in batch, we set up another thread to consume the iterator from child plan's output rdd, so we can't read the info back in another thread.

To fix this, we have to set the info in `InputFileBlockHolder` after the iterator begins consuming. So the info can be read in correct thread.

## How was this patch tested?

Manual test with above example codes for spark-xml package on pyspark: `bin/pyspark --packages com.databricks:spark-xml_2.11:0.4.1`.

Added pyspark test.

Please review http://spark.apache.org/contributing.html before opening a pull request.

Author: Liang-Chi Hsieh <viirya@gmail.com>

Closes #16585 from viirya/fix-inputfileblock-hadooprdd.
2017-01-18 23:06:44 +08:00
Vinayak 285a7798e2 [SPARK-18687][PYSPARK][SQL] Backward compatibility - creating a Dataframe on a new SQLContext object fails with a Derby error
Change is for SQLContext to reuse the active SparkSession during construction if the sparkContext supplied is the same as the currently active SparkContext. Without this change, a new SparkSession is instantiated that results in a Derby error when attempting to create a dataframe using a new SQLContext object even though the SparkContext supplied to the new SQLContext is same as the currently active one. Refer https://issues.apache.org/jira/browse/SPARK-18687 for details on the error and a repro.

Existing unit tests and a new unit test added to pyspark-sql:

/python/run-tests --python-executables=python --modules=pyspark-sql

Please review http://spark.apache.org/contributing.html before opening a pull request.

Author: Vinayak <vijoshi5@in.ibm.com>
Author: Vinayak Joshi <vijoshi@users.noreply.github.com>

Closes #16119 from vijoshi/SPARK-18687_master.
2017-01-13 18:35:51 +08:00
Liang-Chi Hsieh c6c37b8af7 [SPARK-19055][SQL][PYSPARK] Fix SparkSession initialization when SparkContext is stopped
## What changes were proposed in this pull request?

In SparkSession initialization, we store created the instance of SparkSession into a class variable _instantiatedContext. Next time we can use SparkSession.builder.getOrCreate() to retrieve the existing SparkSession instance.

However, when the active SparkContext is stopped and we create another new SparkContext to use, the existing SparkSession is still associated with the stopped SparkContext. So the operations with this existing SparkSession will be failed.

We need to detect such case in SparkSession and renew the class variable _instantiatedContext if needed.

## How was this patch tested?

New test added in PySpark.

Please review http://spark.apache.org/contributing.html before opening a pull request.

Author: Liang-Chi Hsieh <viirya@gmail.com>

Closes #16454 from viirya/fix-pyspark-sparksession.
2017-01-12 20:53:31 +08:00
Burak Yavuz 0917c8ee07 [SPARK-18888] partitionBy in DataStreamWriter in Python throws _to_seq not defined
## What changes were proposed in this pull request?

`_to_seq` wasn't imported.

## How was this patch tested?

Added partitionBy to existing write path unit test

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #16297 from brkyvz/SPARK-18888.
2016-12-15 14:26:54 -08:00
Shixiong Zhu 1ac6567bdb [SPARK-18852][SS] StreamingQuery.lastProgress should be null when recentProgress is empty
## What changes were proposed in this pull request?

Right now `StreamingQuery.lastProgress` throws NoSuchElementException and it's hard to be used in Python since Python user will just see Py4jError.

This PR just makes it return null instead.

## How was this patch tested?

`test("lastProgress should be null when recentProgress is empty")`

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #16273 from zsxwing/SPARK-18852.
2016-12-14 13:36:41 -08:00
gatorsmile 422a45cf04 [SPARK-18766][SQL] Push Down Filter Through BatchEvalPython (Python UDF)
### What changes were proposed in this pull request?
Currently, when users use Python UDF in Filter, BatchEvalPython is always generated below FilterExec. However, not all the predicates need to be evaluated after Python UDF execution. Thus, this PR is to push down the determinisitc predicates through `BatchEvalPython`.
```Python
>>> df = spark.createDataFrame([(1, "1"), (2, "2"), (1, "2"), (1, "2")], ["key", "value"])
>>> from pyspark.sql.functions import udf, col
>>> from pyspark.sql.types import BooleanType
>>> my_filter = udf(lambda a: a < 2, BooleanType())
>>> sel = df.select(col("key"), col("value")).filter((my_filter(col("key"))) & (df.value < "2"))
>>> sel.explain(True)
```
Before the fix, the plan looks like
```
== Optimized Logical Plan ==
Filter ((isnotnull(value#1) && <lambda>(key#0L)) && (value#1 < 2))
+- LogicalRDD [key#0L, value#1]

== Physical Plan ==
*Project [key#0L, value#1]
+- *Filter ((isnotnull(value#1) && pythonUDF0#9) && (value#1 < 2))
   +- BatchEvalPython [<lambda>(key#0L)], [key#0L, value#1, pythonUDF0#9]
      +- Scan ExistingRDD[key#0L,value#1]
```

After the fix, the plan looks like
```
== Optimized Logical Plan ==
Filter ((isnotnull(value#1) && <lambda>(key#0L)) && (value#1 < 2))
+- LogicalRDD [key#0L, value#1]

== Physical Plan ==
*Project [key#0L, value#1]
+- *Filter pythonUDF0#9: boolean
   +- BatchEvalPython [<lambda>(key#0L)], [key#0L, value#1, pythonUDF0#9]
      +- *Filter (isnotnull(value#1) && (value#1 < 2))
         +- Scan ExistingRDD[key#0L,value#1]
```

### How was this patch tested?
Added both unit test cases for `BatchEvalPythonExec` and also add an end-to-end test case in Python test suite.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #16193 from gatorsmile/pythonUDFPredicatePushDown.
2016-12-10 08:47:45 -08:00
Liang-Chi Hsieh 6a5a7254dc [SPARK-18667][PYSPARK][SQL] Change the way to group row in BatchEvalPythonExec so input_file_name function can work with UDF in pyspark
## What changes were proposed in this pull request?

`input_file_name` doesn't return filename when working with UDF in PySpark. An example shows the problem:

    from pyspark.sql.functions import *
    from pyspark.sql.types import *

    def filename(path):
        return path

    sourceFile = udf(filename, StringType())
    spark.read.json("tmp.json").select(sourceFile(input_file_name())).show()

    +---------------------------+
    |filename(input_file_name())|
    +---------------------------+
    |                           |
    +---------------------------+

The cause of this issue is, we group rows in `BatchEvalPythonExec` for batching processing of PythonUDF. Currently we group rows first and then evaluate expressions on the rows. If the data is less than the required number of rows for a group, the iterator will be consumed to the end before the evaluation. However, once the iterator reaches the end, we will unset input filename. So the input_file_name expression can't return correct filename.

This patch fixes the approach to group the batch of rows. We evaluate the expression first and then group evaluated results to batch.

## How was this patch tested?

Added unit test to PySpark.

Please review http://spark.apache.org/contributing.html before opening a pull request.

Author: Liang-Chi Hsieh <viirya@gmail.com>

Closes #16115 from viirya/fix-py-udf-input-filename.
2016-12-08 23:22:18 +08:00
Michael Armbrust 70b2bf717d [SPARK-18754][SS] Rename recentProgresses to recentProgress
Based on an informal survey, users find this option easier to understand / remember.

Author: Michael Armbrust <michael@databricks.com>

Closes #16182 from marmbrus/renameRecentProgress.
2016-12-07 15:36:29 -08:00
Liang-Chi Hsieh 3ba69b6485 [SPARK-18634][PYSPARK][SQL] Corruption and Correctness issues with exploding Python UDFs
## What changes were proposed in this pull request?

As reported in the Jira, there are some weird issues with exploding Python UDFs in SparkSQL.

The following test code can reproduce it. Notice: the following test code is reported to return wrong results in the Jira. However, as I tested on master branch, it causes exception and so can't return any result.

    >>> from pyspark.sql.functions import *
    >>> from pyspark.sql.types import *
    >>>
    >>> df = spark.range(10)
    >>>
    >>> def return_range(value):
    ...   return [(i, str(i)) for i in range(value - 1, value + 1)]
    ...
    >>> range_udf = udf(return_range, ArrayType(StructType([StructField("integer_val", IntegerType()),
    ...                                                     StructField("string_val", StringType())])))
    >>>
    >>> df.select("id", explode(range_udf(df.id))).show()
    Traceback (most recent call last):
      File "<stdin>", line 1, in <module>
      File "/spark/python/pyspark/sql/dataframe.py", line 318, in show
        print(self._jdf.showString(n, 20))
      File "/spark/python/lib/py4j-0.10.4-src.zip/py4j/java_gateway.py", line 1133, in __call__
      File "/spark/python/pyspark/sql/utils.py", line 63, in deco
        return f(*a, **kw)
      File "/spark/python/lib/py4j-0.10.4-src.zip/py4j/protocol.py", line 319, in get_return_value py4j.protocol.Py4JJavaError: An error occurred while calling o126.showString.: java.lang.AssertionError: assertion failed
        at scala.Predef$.assert(Predef.scala:156)
        at org.apache.spark.sql.execution.CodegenSupport$class.consume(WholeStageCodegenExec.scala:120)
        at org.apache.spark.sql.execution.GenerateExec.consume(GenerateExec.scala:57)

The cause of this issue is, in `ExtractPythonUDFs` we insert `BatchEvalPythonExec` to run PythonUDFs in batch. `BatchEvalPythonExec` will add extra outputs (e.g., `pythonUDF0`) to original plan. In above case, the original `Range` only has one output `id`. After `ExtractPythonUDFs`, the added `BatchEvalPythonExec` has two outputs `id` and `pythonUDF0`.

Because the output of `GenerateExec` is given after analysis phase, in above case, it is the combination of `id`, i.e., the output of `Range`, and `col`. But in planning phase, we change `GenerateExec`'s child plan to `BatchEvalPythonExec` with additional output attributes.

It will cause no problem in non wholestage codegen. Because when evaluating the additional attributes are projected out the final output of `GenerateExec`.

However, as `GenerateExec` now supports wholestage codegen, the framework will input all the outputs of the child plan to `GenerateExec`. Then when consuming `GenerateExec`'s output data (i.e., calling `consume`), the number of output attributes is different to the output variables in wholestage codegen.

To solve this issue, this patch only gives the generator's output to `GenerateExec` after analysis phase. `GenerateExec`'s output is the combination of its child plan's output and the generator's output. So when we change `GenerateExec`'s child, its output is still correct.

## How was this patch tested?

Added test cases to PySpark.

Please review http://spark.apache.org/contributing.html before opening a pull request.

Author: Liang-Chi Hsieh <viirya@gmail.com>

Closes #16120 from viirya/fix-py-udf-with-generator.
2016-12-05 17:50:43 -08:00
Shixiong Zhu 246012859f [SPARK-18694][SS] Add StreamingQuery.explain and exception to Python and fix StreamingQueryException
## What changes were proposed in this pull request?

- Add StreamingQuery.explain and exception to Python.
- Fix StreamingQueryException to not expose `OffsetSeq`.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #16125 from zsxwing/py-streaming-explain.
2016-12-05 11:36:11 -08:00
zero323 a9cbfc4f6a [SPARK-18690][PYTHON][SQL] Backward compatibility of unbounded frames
## What changes were proposed in this pull request?

Makes `Window.unboundedPreceding` and `Window.unboundedFollowing` backward compatible.

## How was this patch tested?

Pyspark SQL unittests.

Please review http://spark.apache.org/contributing.html before opening a pull request.

Author: zero323 <zero323@users.noreply.github.com>

Closes #16123 from zero323/SPARK-17845-follow-up.
2016-12-02 17:39:28 -08:00
Tathagata Das bc09a2b8c3 [SPARK-18516][STRUCTURED STREAMING] Follow up PR to add StreamingQuery.status to Python
## What changes were proposed in this pull request?
- Add StreamingQueryStatus.json
- Make it not case class (to avoid unnecessarily exposing implicit object StreamingQueryStatus, consistent with StreamingQueryProgress)
- Add StreamingQuery.status to Python
- Fix post-termination status

## How was this patch tested?
New unit tests

Author: Tathagata Das <tathagata.das1565@gmail.com>

Closes #16075 from tdas/SPARK-18516-1.
2016-11-29 23:08:56 -08:00
Tathagata Das c3d08e2f29 [SPARK-18516][SQL] Split state and progress in streaming
This PR separates the status of a `StreamingQuery` into two separate APIs:
 - `status` - describes the status of a `StreamingQuery` at this moment, including what phase of processing is currently happening and if data is available.
 - `recentProgress` - an array of statistics about the most recent microbatches that have executed.

A recent progress contains the following information:
```
{
  "id" : "2be8670a-fce1-4859-a530-748f29553bb6",
  "name" : "query-29",
  "timestamp" : 1479705392724,
  "inputRowsPerSecond" : 230.76923076923077,
  "processedRowsPerSecond" : 10.869565217391303,
  "durationMs" : {
    "triggerExecution" : 276,
    "queryPlanning" : 3,
    "getBatch" : 5,
    "getOffset" : 3,
    "addBatch" : 234,
    "walCommit" : 30
  },
  "currentWatermark" : 0,
  "stateOperators" : [ ],
  "sources" : [ {
    "description" : "KafkaSource[Subscribe[topic-14]]",
    "startOffset" : {
      "topic-14" : {
        "2" : 0,
        "4" : 1,
        "1" : 0,
        "3" : 0,
        "0" : 0
      }
    },
    "endOffset" : {
      "topic-14" : {
        "2" : 1,
        "4" : 2,
        "1" : 0,
        "3" : 0,
        "0" : 1
      }
    },
    "numRecords" : 3,
    "inputRowsPerSecond" : 230.76923076923077,
    "processedRowsPerSecond" : 10.869565217391303
  } ]
}
```

Additionally, in order to make it possible to correlate progress updates across restarts, we change the `id` field from an integer that is unique with in the JVM to a `UUID` that is globally unique.

Author: Tathagata Das <tathagata.das1565@gmail.com>
Author: Michael Armbrust <michael@databricks.com>

Closes #15954 from marmbrus/queryProgress.
2016-11-29 17:24:17 -08:00
Srinath Shankar 2d96d35dc0 [SPARK-17946][PYSPARK] Python crossJoin API similar to Scala
## 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.
2016-10-14 18:24:47 -07:00
Reynold Xin 6f20a92ca3 [SPARK-17845] [SQL] More self-evident window function frame boundary API
## What changes were proposed in this pull request?
This patch improves the window function frame boundary API to make it more obvious to read and to use. The two high level changes are:

1. Create Window.currentRow, Window.unboundedPreceding, Window.unboundedFollowing to indicate the special values in frame boundaries. These methods map to the special integral values so we are not breaking backward compatibility here. This change makes the frame boundaries more self-evident (instead of Long.MinValue, it becomes Window.unboundedPreceding).

2. In Python, for any value less than or equal to JVM's Long.MinValue, treat it as Window.unboundedPreceding. For any value larger than or equal to JVM's Long.MaxValue, treat it as Window.unboundedFollowing. Before this change, if the user specifies any value that is less than Long.MinValue but not -sys.maxsize (e.g. -sys.maxsize + 1), the number we pass over to the JVM would overflow, resulting in a frame that does not make sense.

Code example required to specify a frame before this patch:
```
Window.rowsBetween(-Long.MinValue, 0)
```

While the above code should still work, the new way is more obvious to read:
```
Window.rowsBetween(Window.unboundedPreceding, Window.currentRow)
```

## How was this patch tested?
- Updated DataFrameWindowSuite (for Scala/Java)
- Updated test_window_functions_cumulative_sum (for Python)
- Renamed DataFrameWindowSuite DataFrameWindowFunctionsSuite to better reflect its purpose

Author: Reynold Xin <rxin@databricks.com>

Closes #15438 from rxin/SPARK-17845.
2016-10-12 16:45:10 -07:00
Bijay Pathak 8880fd13ef [SPARK-14761][SQL] Reject invalid join methods when join columns are not specified in PySpark DataFrame join.
## 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.
2016-10-12 10:09:49 -07:00