Commit graph

1728 commits

Author SHA1 Message Date
王晓哲 602c6d82d8 [SPARK-20947][PYTHON] Fix encoding/decoding error in pipe action
## What changes were proposed in this pull request?

Pipe action convert objects into strings using a way that was affected by the default encoding setting of Python environment.

This patch fixed the problem. The detailed description is added here:

https://issues.apache.org/jira/browse/SPARK-20947

## How was this patch tested?

Run the following statement in pyspark-shell, and it will NOT raise exception if this patch is applied:

```python
sc.parallelize([u'\u6d4b\u8bd5']).pipe('cat').collect()
```

Author: 王晓哲 <wxz@linkdoc.com>

Closes #18277 from chaoslawful/fix_pipe_encoding_error.
2018-01-22 10:43:12 +09:00
Takuya UESHIN 568055da93 [SPARK-23054][SQL][PYSPARK][FOLLOWUP] Use sqlType casting when casting PythonUserDefinedType to String.
## What changes were proposed in this pull request?

This is a follow-up of #20246.

If a UDT in Python doesn't have its corresponding Scala UDT, cast to string will be the raw string of the internal value, e.g. `"org.apache.spark.sql.catalyst.expressions.UnsafeArrayDataxxxxxxxx"` if the internal type is `ArrayType`.

This pr fixes it by using its `sqlType` casting.

## How was this patch tested?

Added a test and existing tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #20306 from ueshin/issues/SPARK-23054/fup1.
2018-01-19 11:37:08 +08:00
Tathagata Das 2d41f040a3 [SPARK-23143][SS][PYTHON] Added python API for setting continuous trigger
## What changes were proposed in this pull request?
Self-explanatory.

## How was this patch tested?
New python tests.

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

Closes #20309 from tdas/SPARK-23143.
2018-01-18 12:25:52 -08:00
Takuya UESHIN 5063b74811 [SPARK-23141][SQL][PYSPARK] Support data type string as a returnType for registerJavaFunction.
## What changes were proposed in this pull request?

Currently `UDFRegistration.registerJavaFunction` doesn't support data type string as a `returnType` whereas `UDFRegistration.register`, `udf`, or `pandas_udf` does.
We can support it for `UDFRegistration.registerJavaFunction` as well.

## How was this patch tested?

Added a doctest and existing tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #20307 from ueshin/issues/SPARK-23141.
2018-01-18 22:33:04 +09:00
hyukjinkwon 39d244d921 [SPARK-23122][PYTHON][SQL] Deprecate register* for UDFs in SQLContext and Catalog in PySpark
## What changes were proposed in this pull request?

This PR proposes to deprecate `register*` for UDFs in `SQLContext` and `Catalog` in Spark 2.3.0.

These are inconsistent with Scala / Java APIs and also these basically do the same things with `spark.udf.register*`.

Also, this PR moves the logcis from `[sqlContext|spark.catalog].register*` to `spark.udf.register*` and reuse the docstring.

This PR also handles minor doc corrections. It also includes https://github.com/apache/spark/pull/20158

## How was this patch tested?

Manually tested, manually checked the API documentation and tests added to check if deprecated APIs call the aliases correctly.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #20288 from HyukjinKwon/deprecate-udf.
2018-01-18 14:51:05 +09:00
hyukjinkwon 45ad97df87 [SPARK-23132][PYTHON][ML] Run doctests in ml.image when testing
## What changes were proposed in this pull request?

This PR proposes to actually run the doctests in `ml/image.py`.

## How was this patch tested?

doctests in `python/pyspark/ml/image.py`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #20294 from HyukjinKwon/trigger-image.
2018-01-18 07:30:54 +09:00
Henry Robinson 1f3d933e0b [SPARK-23062][SQL] Improve EXCEPT documentation
## What changes were proposed in this pull request?

Make the default behavior of EXCEPT (i.e. EXCEPT DISTINCT) more
explicit in the documentation, and call out the change in behavior
from 1.x.

Author: Henry Robinson <henry@cloudera.com>

Closes #20254 from henryr/spark-23062.
2018-01-17 16:01:41 +08:00
gatorsmile b85eb946ac [SPARK-22978][PYSPARK] Register Vectorized UDFs for SQL Statement
## What changes were proposed in this pull request?
Register Vectorized UDFs for SQL Statement. For example,

```Python
>>> from pyspark.sql.functions import pandas_udf, PandasUDFType
>>> pandas_udf("integer", PandasUDFType.SCALAR)
... def add_one(x):
...     return x + 1
...
>>> _ = spark.udf.register("add_one", add_one)
>>> spark.sql("SELECT add_one(id) FROM range(3)").collect()
[Row(add_one(id)=1), Row(add_one(id)=2), Row(add_one(id)=3)]
```

## How was this patch tested?
Added test cases

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20171 from gatorsmile/supportVectorizedUDF.
2018-01-16 20:20:33 +09:00
Takeshi Yamamuro b59808385c [SPARK-23023][SQL] Cast field data to strings in showString
## What changes were proposed in this pull request?
The current `Datset.showString` prints rows thru `RowEncoder` deserializers like;
```
scala> Seq(Seq(Seq(1, 2), Seq(3), Seq(4, 5, 6))).toDF("a").show(false)
+------------------------------------------------------------+
|a                                                           |
+------------------------------------------------------------+
|[WrappedArray(1, 2), WrappedArray(3), WrappedArray(4, 5, 6)]|
+------------------------------------------------------------+
```
This result is incorrect because the correct one is;
```
scala> Seq(Seq(Seq(1, 2), Seq(3), Seq(4, 5, 6))).toDF("a").show(false)
+------------------------+
|a                       |
+------------------------+
|[[1, 2], [3], [4, 5, 6]]|
+------------------------+
```
So, this pr fixed code in `showString` to cast field data to strings before printing.

## How was this patch tested?
Added tests in `DataFrameSuite`.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #20214 from maropu/SPARK-23023.
2018-01-15 16:26:52 +08:00
hyukjinkwon cd9f49a2ae [SPARK-22980][PYTHON][SQL] Clarify the length of each series is of each batch within scalar Pandas UDF
## What changes were proposed in this pull request?

This PR proposes to add a note that saying the length of a scalar Pandas UDF's `Series` is not of the whole input column but of the batch.

We are fine for a group map UDF because the usage is different from our typical UDF but scalar UDFs might cause confusion with the normal UDF.

For example, please consider this example:

```python
from pyspark.sql.functions import pandas_udf, col, lit

df = spark.range(1)
f = pandas_udf(lambda x, y: len(x) + y, LongType())
df.select(f(lit('text'), col('id'))).show()
```

```
+------------------+
|<lambda>(text, id)|
+------------------+
|                 1|
+------------------+
```

```python
from pyspark.sql.functions import udf, col, lit

df = spark.range(1)
f = udf(lambda x, y: len(x) + y, "long")
df.select(f(lit('text'), col('id'))).show()
```

```
+------------------+
|<lambda>(text, id)|
+------------------+
|                 4|
+------------------+
```

## How was this patch tested?

Manually built the doc and checked the output.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #20237 from HyukjinKwon/SPARK-22980.
2018-01-13 16:13:44 +09:00
gatorsmile 651f76153f [SPARK-23028] Bump master branch version to 2.4.0-SNAPSHOT
## What changes were proposed in this pull request?
This patch bumps the master branch version to `2.4.0-SNAPSHOT`.

## How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20222 from gatorsmile/bump24.
2018-01-13 00:37:59 +08:00
WeichenXu a7d98d53ce [SPARK-23008][ML][FOLLOW-UP] mark OneHotEncoder python API deprecated
## What changes were proposed in this pull request?

mark OneHotEncoder python API deprecated

## How was this patch tested?

N/A

Author: WeichenXu <weichen.xu@databricks.com>

Closes #20241 from WeichenXu123/mark_ohe_deprecated.
2018-01-12 11:27:02 +02:00
WeichenXu b5042d75c2 [SPARK-23008][ML] OnehotEncoderEstimator python API
## What changes were proposed in this pull request?

OnehotEncoderEstimator python API.

## How was this patch tested?

doctest

Author: WeichenXu <weichen.xu@databricks.com>

Closes #20209 from WeichenXu123/ohe_py.
2018-01-11 16:20:30 -08:00
sethah 70bcc9d5ae [SPARK-22993][ML] Clarify HasCheckpointInterval param doc
## What changes were proposed in this pull request?

Add a note to the `HasCheckpointInterval` parameter doc that clarifies that this setting is ignored when no checkpoint directory has been set on the spark context.

## How was this patch tested?

No tests necessary, just a doc update.

Author: sethah <shendrickson@cloudera.com>

Closes #20188 from sethah/als_checkpoint_doc.
2018-01-09 23:32:47 -08:00
Bryan Cutler e599837248 [SPARK-23009][PYTHON] Fix for non-str col names to createDataFrame from Pandas
## What changes were proposed in this pull request?

This the case when calling `SparkSession.createDataFrame` using a Pandas DataFrame that has non-str column labels.

The column name conversion logic to handle non-string or unicode in python2 is:
```
if column is not any type of string:
    name = str(column)
else if column is unicode in Python 2:
    name = column.encode('utf-8')
```

## How was this patch tested?

Added a new test with a Pandas DataFrame that has int column labels

Author: Bryan Cutler <cutlerb@gmail.com>

Closes #20210 from BryanCutler/python-createDataFrame-int-col-error-SPARK-23009.
2018-01-10 14:55:24 +09:00
Bryan Cutler 7bcc266681 [SPARK-23018][PYTHON] Fix createDataFrame from Pandas timestamp series assignment
## What changes were proposed in this pull request?

This fixes createDataFrame from Pandas to only assign modified timestamp series back to a copied version of the Pandas DataFrame.  Previously, if the Pandas DataFrame was only a reference (e.g. a slice of another) each series will still get assigned back to the reference even if it is not a modified timestamp column.  This caused the following warning "SettingWithCopyWarning: A value is trying to be set on a copy of a slice from a DataFrame."

## How was this patch tested?

existing tests

Author: Bryan Cutler <cutlerb@gmail.com>

Closes #20213 from BryanCutler/pyspark-createDataFrame-copy-slice-warn-SPARK-23018.
2018-01-10 14:00:07 +09:00
Guilherme Berger 3e40eb3f1f [SPARK-22566][PYTHON] Better error message for _merge_type in Pandas to Spark DF conversion
## What changes were proposed in this pull request?

It provides a better error message when doing `spark_session.createDataFrame(pandas_df)` with no schema and an error occurs in the schema inference due to incompatible types.

The Pandas column names are propagated down and the error message mentions which column had the merging error.

https://issues.apache.org/jira/browse/SPARK-22566

## How was this patch tested?

Manually in the `./bin/pyspark` console, and with new tests: `./python/run-tests`

<img width="873" alt="screen shot 2017-11-21 at 13 29 49" src="https://user-images.githubusercontent.com/3977115/33080121-382274e0-cecf-11e7-808f-057a65bb7b00.png">

I state that the contribution is my original work and that I license the work to the Apache Spark project under the project’s open source license.

Author: Guilherme Berger <gberger@palantir.com>

Closes #19792 from gberger/master.
2018-01-08 14:32:05 +09:00
hyukjinkwon 993f21567a [SPARK-22901][PYTHON][FOLLOWUP] Adds the doc for asNondeterministic for wrapped UDF function
## What changes were proposed in this pull request?

This PR wraps the `asNondeterministic` attribute in the wrapped UDF function to set the docstring properly.

```python
from pyspark.sql.functions import udf
help(udf(lambda x: x).asNondeterministic)
```

Before:

```
Help on function <lambda> in module pyspark.sql.udf:

<lambda> lambda
(END
```

After:

```
Help on function asNondeterministic in module pyspark.sql.udf:

asNondeterministic()
    Updates UserDefinedFunction to nondeterministic.

    .. versionadded:: 2.3
(END)
```

## How was this patch tested?

Manually tested and a simple test was added.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #20173 from HyukjinKwon/SPARK-22901-followup.
2018-01-06 23:08:26 +08:00
Li Jin f2dd8b9237 [SPARK-22930][PYTHON][SQL] Improve the description of Vectorized UDFs for non-deterministic cases
## What changes were proposed in this pull request?

Add tests for using non deterministic UDFs in aggregate.

Update pandas_udf docstring w.r.t to determinism.

## How was this patch tested?
test_nondeterministic_udf_in_aggregate

Author: Li Jin <ice.xelloss@gmail.com>

Closes #20142 from icexelloss/SPARK-22930-pandas-udf-deterministic.
2018-01-06 16:11:20 +08:00
gatorsmile 5aadbc929c [SPARK-22939][PYSPARK] Support Spark UDF in registerFunction
## What changes were proposed in this pull request?
```Python
import random
from pyspark.sql.functions import udf
from pyspark.sql.types import IntegerType, StringType
random_udf = udf(lambda: int(random.random() * 100), IntegerType()).asNondeterministic()
spark.catalog.registerFunction("random_udf", random_udf, StringType())
spark.sql("SELECT random_udf()").collect()
```

We will get the following error.
```
Py4JError: An error occurred while calling o29.__getnewargs__. Trace:
py4j.Py4JException: Method __getnewargs__([]) does not exist
	at py4j.reflection.ReflectionEngine.getMethod(ReflectionEngine.java:318)
	at py4j.reflection.ReflectionEngine.getMethod(ReflectionEngine.java:326)
	at py4j.Gateway.invoke(Gateway.java:274)
	at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)
	at py4j.commands.CallCommand.execute(CallCommand.java:79)
	at py4j.GatewayConnection.run(GatewayConnection.java:214)
	at java.lang.Thread.run(Thread.java:745)
```

This PR is to support it.

## How was this patch tested?
WIP

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20137 from gatorsmile/registerFunction.
2018-01-04 21:07:31 +08:00
Felix Cheung df95a908ba [SPARK-22933][SPARKR] R Structured Streaming API for withWatermark, trigger, partitionBy
## What changes were proposed in this pull request?

R Structured Streaming API for withWatermark, trigger, partitionBy

## How was this patch tested?

manual, unit tests

Author: Felix Cheung <felixcheung_m@hotmail.com>

Closes #20129 from felixcheung/rwater.
2018-01-03 21:43:14 -08:00
Bryan Cutler 1c9f95cb77 [SPARK-22530][PYTHON][SQL] Adding Arrow support for ArrayType
## What changes were proposed in this pull request?

This change adds `ArrayType` support for working with Arrow in pyspark when creating a DataFrame, calling `toPandas()`, and using vectorized `pandas_udf`.

## How was this patch tested?

Added new Python unit tests using Array data.

Author: Bryan Cutler <cutlerb@gmail.com>

Closes #20114 from BryanCutler/arrow-ArrayType-support-SPARK-22530.
2018-01-02 07:13:27 +09:00
Sean Owen c284c4e1f6 [MINOR] Fix a bunch of typos 2018-01-02 07:10:19 +09:00
Nick Pentreath 028ee40165 [SPARK-22801][ML][PYSPARK] Allow FeatureHasher to treat numeric columns as categorical
Previously, `FeatureHasher` always treats numeric type columns as numbers and never as categorical features. It is quite common to have categorical features represented as numbers or codes in data sources.

In order to hash these features as categorical, users must first explicitly convert them to strings which is cumbersome.

Add a new param `categoricalCols` which specifies the numeric columns that should be treated as categorical features.

## How was this patch tested?

New unit tests.

Author: Nick Pentreath <nickp@za.ibm.com>

Closes #19991 from MLnick/hasher-num-cat.
2017-12-31 14:51:38 +02:00
Takeshi Yamamuro f2b3525c17 [SPARK-22771][SQL] Concatenate binary inputs into a binary output
## What changes were proposed in this pull request?
This pr modified `concat` to concat binary inputs into a single binary output.
`concat` in the current master always output data as a string. But, in some databases (e.g., PostgreSQL), if all inputs are binary, `concat` also outputs binary.

## How was this patch tested?
Added tests in `SQLQueryTestSuite` and `TypeCoercionSuite`.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #19977 from maropu/SPARK-22771.
2017-12-30 14:09:56 +08:00
Bago Amirbekian 816963043a [SPARK-22734][ML][PYSPARK] Added Python API for VectorSizeHint.
(Please fill in changes proposed in this fix)

Python API for VectorSizeHint Transformer.

(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)

doc-tests.

Author: Bago Amirbekian <bago@databricks.com>

Closes #20112 from MrBago/vectorSizeHint-PythonAPI.
2017-12-29 19:45:14 -08:00
Bago Amirbekian 30fcdc0380 [SPARK-22922][ML][PYSPARK] Pyspark portion of the fit-multiple API
## What changes were proposed in this pull request?

Adding fitMultiple API to `Estimator` with default implementation. Also update have ml.tuning meta-estimators use this API.

## How was this patch tested?

Unit tests.

Author: Bago Amirbekian <bago@databricks.com>

Closes #20058 from MrBago/python-fitMultiple.
2017-12-29 16:31:25 -08:00
Takuya UESHIN 11a849b3a7 [SPARK-22370][SQL][PYSPARK][FOLLOW-UP] Fix a test failure when xmlrunner is installed.
## What changes were proposed in this pull request?

This is a follow-up pr of #19587.

If `xmlrunner` is installed, `VectorizedUDFTests.test_vectorized_udf_check_config` fails by the following error because the `self` which is a subclass of `unittest.TestCase` in the UDF `check_records_per_batch` can't be pickled anymore.

```
PicklingError: Cannot pickle files that are not opened for reading: w
```

This changes the UDF not to refer the `self`.

## How was this patch tested?

Tested locally.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #20115 from ueshin/issues/SPARK-22370_fup1.
2017-12-29 23:04:28 +09:00
hyukjinkwon 796e48c60e [SPARK-22313][PYTHON][FOLLOWUP] Explicitly import warnings namespace in flume.py
## What changes were proposed in this pull request?

This PR explicitly imports the missing `warnings` in `flume.py`.

## How was this patch tested?

Manually tested.

```python
>>> import warnings
>>> warnings.simplefilter('always', DeprecationWarning)
>>> from pyspark.streaming import flume
>>> flume.FlumeUtils.createStream(None, None, None)
Traceback (most recent call last):
  File "<stdin>", line 1, in <module>
  File "/.../spark/python/pyspark/streaming/flume.py", line 60, in createStream
    warnings.warn(
NameError: global name 'warnings' is not defined
```

```python
>>> import warnings
>>> warnings.simplefilter('always', DeprecationWarning)
>>> from pyspark.streaming import flume
>>> flume.FlumeUtils.createStream(None, None, None)
/.../spark/python/pyspark/streaming/flume.py:65: DeprecationWarning: Deprecated in 2.3.0. Flume support is deprecated as of Spark 2.3.0. See SPARK-22142.
  DeprecationWarning)
...
```

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #20110 from HyukjinKwon/SPARK-22313-followup.
2017-12-29 14:46:03 +09:00
soonmok-kwon ffe6fd77a4 [SPARK-22818][SQL] csv escape of quote escape
## What changes were proposed in this pull request?

Escape of escape should be considered when using the UniVocity csv encoding/decoding library.

Ref: https://github.com/uniVocity/univocity-parsers#escaping-quote-escape-characters

One option is added for reading and writing CSV: `escapeQuoteEscaping`

## How was this patch tested?

Unit test added.

Author: soonmok-kwon <soonmok.kwon@navercorp.com>

Closes #20004 from ep1804/SPARK-22818.
2017-12-29 07:30:06 +08:00
Marco Gaido ff48b1b338 [SPARK-22901][PYTHON] Add deterministic flag to pyspark UDF
## What changes were proposed in this pull request?

In SPARK-20586 the flag `deterministic` was added to Scala UDF, but it is not available for python UDF. This flag is useful for cases when the UDF's code can return different result with the same input. Due to optimization, duplicate invocations may be eliminated or the function may even be invoked more times than it is present in the query. This can lead to unexpected behavior.

This PR adds the deterministic flag, via the `asNondeterministic` method, to let the user mark the function as non-deterministic and therefore avoid the optimizations which might lead to strange behaviors.

## How was this patch tested?

Manual tests:
```
>>> from pyspark.sql.functions import *
>>> from pyspark.sql.types import *
>>> df_br = spark.createDataFrame([{'name': 'hello'}])
>>> import random
>>> udf_random_col =  udf(lambda: int(100*random.random()), IntegerType()).asNondeterministic()
>>> df_br = df_br.withColumn('RAND', udf_random_col())
>>> random.seed(1234)
>>> udf_add_ten =  udf(lambda rand: rand + 10, IntegerType())
>>> df_br.withColumn('RAND_PLUS_TEN', udf_add_ten('RAND')).show()
+-----+----+-------------+
| name|RAND|RAND_PLUS_TEN|
+-----+----+-------------+
|hello|   3|           13|
+-----+----+-------------+

```

Author: Marco Gaido <marcogaido91@gmail.com>
Author: Marco Gaido <mgaido@hortonworks.com>

Closes #19929 from mgaido91/SPARK-22629.
2017-12-26 06:39:40 -08:00
Takuya UESHIN eb386be1ed [SPARK-21552][SQL] Add DecimalType support to ArrowWriter.
## What changes were proposed in this pull request?

Decimal type is not yet supported in `ArrowWriter`.
This is adding the decimal type support.

## How was this patch tested?

Added a test to `ArrowConvertersSuite`.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #18754 from ueshin/issues/SPARK-21552.
2017-12-26 21:37:25 +09:00
Takuya UESHIN 12d20dd75b [SPARK-22874][PYSPARK][SQL][FOLLOW-UP] Modify error messages to show actual versions.
## What changes were proposed in this pull request?

This is a follow-up pr of #20054 modifying error messages for both pandas and pyarrow to show actual versions.

## How was this patch tested?

Existing tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #20074 from ueshin/issues/SPARK-22874_fup1.
2017-12-25 20:29:10 +09:00
Takuya UESHIN 13190a4f60 [SPARK-22874][PYSPARK][SQL] Modify checking pandas version to use LooseVersion.
## What changes were proposed in this pull request?

Currently we check pandas version by capturing if `ImportError` for the specific imports is raised or not but we can compare `LooseVersion` of the version strings as the same as we're checking pyarrow version.

## How was this patch tested?

Existing tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #20054 from ueshin/issues/SPARK-22874.
2017-12-22 20:09:51 +09:00
Bryan Cutler 59d52631eb [SPARK-22324][SQL][PYTHON] Upgrade Arrow to 0.8.0
## What changes were proposed in this pull request?

Upgrade Spark to Arrow 0.8.0 for Java and Python.  Also includes an upgrade of Netty to 4.1.17 to resolve dependency requirements.

The highlights that pertain to Spark for the update from Arrow versoin 0.4.1 to 0.8.0 include:

* Java refactoring for more simple API
* Java reduced heap usage and streamlined hot code paths
* Type support for DecimalType, ArrayType
* Improved type casting support in Python
* Simplified type checking in Python

## How was this patch tested?

Existing tests

Author: Bryan Cutler <cutlerb@gmail.com>
Author: Shixiong Zhu <zsxwing@gmail.com>

Closes #19884 from BryanCutler/arrow-upgrade-080-SPARK-22324.
2017-12-21 20:43:56 +09:00
Yanbo Liang fb0562f346 [SPARK-22810][ML][PYSPARK] Expose Python API for LinearRegression with huber loss.
## What changes were proposed in this pull request?
Expose Python API for _LinearRegression_ with _huber_ loss.

## How was this patch tested?
Unit test.

Author: Yanbo Liang <ybliang8@gmail.com>

Closes #19994 from yanboliang/spark-22810.
2017-12-20 17:51:42 -08:00
Dongjoon Hyun 9962390af7 [SPARK-22781][SS] Support creating streaming dataset with ORC files
## What changes were proposed in this pull request?

Like `Parquet`, users can use `ORC` with Apache Spark structured streaming. This PR adds `orc()` to `DataStreamReader`(Scala/Python) in order to support creating streaming dataset with ORC file format more easily like the other file formats. Also, this adds a test coverage for ORC data source and updates the document.

**BEFORE**

```scala
scala> spark.readStream.schema("a int").orc("/tmp/orc_ss").writeStream.format("console").start()
<console>:24: error: value orc is not a member of org.apache.spark.sql.streaming.DataStreamReader
       spark.readStream.schema("a int").orc("/tmp/orc_ss").writeStream.format("console").start()
```

**AFTER**
```scala
scala> spark.readStream.schema("a int").orc("/tmp/orc_ss").writeStream.format("console").start()
res0: org.apache.spark.sql.streaming.StreamingQuery = org.apache.spark.sql.execution.streaming.StreamingQueryWrapper678b3746

scala>
-------------------------------------------
Batch: 0
-------------------------------------------
+---+
|  a|
+---+
|  1|
+---+
```

## How was this patch tested?

Pass the newly added test cases.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #19975 from dongjoon-hyun/SPARK-22781.
2017-12-19 23:50:06 -08:00
Fernando Pereira 13268a58f8 [SPARK-22649][PYTHON][SQL] Adding localCheckpoint to Dataset API
## What changes were proposed in this pull request?

This change adds local checkpoint support to datasets and respective bind from Python Dataframe API.

If reliability requirements can be lowered to favor performance, as in cases of further quick transformations followed by a reliable save, localCheckpoints() fit very well.
Furthermore, at the moment Reliable checkpoints still incur double computation (see #9428)
In general it makes the API more complete as well.

## How was this patch tested?

Python land quick use case:

```python
>>> from time import sleep
>>> from pyspark.sql import types as T
>>> from pyspark.sql import functions as F

>>> def f(x):
    sleep(1)
    return x*2
   ...:

>>> df1 = spark.range(30, numPartitions=6)
>>> df2 = df1.select(F.udf(f, T.LongType())("id"))

>>> %time _ = df2.collect()
CPU times: user 7.79 ms, sys: 5.84 ms, total: 13.6 ms
Wall time: 12.2 s

>>> %time df3 = df2.localCheckpoint()
CPU times: user 2.38 ms, sys: 2.3 ms, total: 4.68 ms
Wall time: 10.3 s

>>> %time _ = df3.collect()
CPU times: user 5.09 ms, sys: 410 µs, total: 5.5 ms
Wall time: 148 ms

>>> sc.setCheckpointDir(".")
>>> %time df3 = df2.checkpoint()
CPU times: user 4.04 ms, sys: 1.63 ms, total: 5.67 ms
Wall time: 20.3 s
```

Author: Fernando Pereira <fernando.pereira@epfl.ch>

Closes #19805 from ferdonline/feature_dataset_localCheckpoint.
2017-12-19 20:47:12 -08:00
Youngbin Kim 6e36d8d562 [SPARK-22829] Add new built-in function date_trunc()
## What changes were proposed in this pull request?

Adding date_trunc() as a built-in function.
`date_trunc` is common in other databases, but Spark or Hive does not have support for this. `date_trunc` is commonly used by data scientists and business intelligence application such as Superset (https://github.com/apache/incubator-superset).
We do have `trunc` but this only works with 'MONTH' and 'YEAR' level on the DateType input.

date_trunc() in other databases:
AWS Redshift: http://docs.aws.amazon.com/redshift/latest/dg/r_DATE_TRUNC.html
PostgreSQL: https://www.postgresql.org/docs/9.1/static/functions-datetime.html
Presto: https://prestodb.io/docs/current/functions/datetime.html

## How was this patch tested?

Unit tests

(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.

Author: Youngbin Kim <ykim828@hotmail.com>

Closes #20015 from youngbink/date_trunc.
2017-12-19 20:22:33 -08:00
Bago Amirbekian 0c8fca4608 [SPARK-22811][PYSPARK][ML] Fix pyspark.ml.tests failure when Hive is not available.
## What changes were proposed in this pull request?

pyspark.ml.tests is missing a py4j import. I've added the import and fixed the test that uses it. This test was only failing when testing without Hive.

## How was this patch tested?

Existing tests.

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

Author: Bago Amirbekian <bago@databricks.com>

Closes #19997 from MrBago/fix-ImageReaderTest2.
2017-12-16 10:57:35 +09:00
Prashant Sharma 40de176c93 [SPARK-16496][SQL] Add wholetext as option for reading text in SQL.
## What changes were proposed in this pull request?

In multiple text analysis problems, it is not often desirable for the rows to be split by "\n". There exists a wholeText reader for RDD API, and this JIRA just adds the same support for Dataset API.
## How was this patch tested?

Added relevant new tests for both scala and Java APIs

Author: Prashant Sharma <prashsh1@in.ibm.com>
Author: Prashant Sharma <prashant@apache.org>

Closes #14151 from ScrapCodes/SPARK-16496/wholetext.
2017-12-14 11:19:34 -08:00
hyukjinkwon aa4cf2b19e [SPARK-22651][PYTHON][ML] Prevent initiating multiple Hive clients for ImageSchema.readImages
## What changes were proposed in this pull request?

Calling `ImageSchema.readImages` multiple times as below in PySpark shell:

```python
from pyspark.ml.image import ImageSchema
data_path = 'data/mllib/images/kittens'
_ = ImageSchema.readImages(data_path, recursive=True, dropImageFailures=True).collect()
_ = ImageSchema.readImages(data_path, recursive=True, dropImageFailures=True).collect()
```

throws an error as below:

```
...
org.datanucleus.exceptions.NucleusDataStoreException: Unable to open a test connection to the given database. JDBC url = jdbc:derby:;databaseName=metastore_db;create=true, username = APP. Terminating connection pool (set lazyInit to true if you expect to start your database after your app). Original Exception: ------
java.sql.SQLException: Failed to start database 'metastore_db' with class loader org.apache.spark.sql.hive.client.IsolatedClientLoader$$anon$1742f639f, see the next exception for details.
...
	at org.apache.derby.jdbc.AutoloadedDriver.connect(Unknown Source)
...
	at org.apache.hadoop.hive.metastore.HiveMetaStore.newRetryingHMSHandler(HiveMetaStore.java:5762)
...
	at org.apache.spark.sql.hive.client.HiveClientImpl.newState(HiveClientImpl.scala:180)
...
	at org.apache.spark.sql.hive.HiveExternalCatalog$$anonfun$databaseExists$1.apply$mcZ$sp(HiveExternalCatalog.scala:195)
	at org.apache.spark.sql.hive.HiveExternalCatalog$$anonfun$databaseExists$1.apply(HiveExternalCatalog.scala:195)
	at org.apache.spark.sql.hive.HiveExternalCatalog$$anonfun$databaseExists$1.apply(HiveExternalCatalog.scala:195)
	at org.apache.spark.sql.hive.HiveExternalCatalog.withClient(HiveExternalCatalog.scala:97)
	at org.apache.spark.sql.hive.HiveExternalCatalog.databaseExists(HiveExternalCatalog.scala:194)
	at org.apache.spark.sql.internal.SharedState.externalCatalog$lzycompute(SharedState.scala:100)
	at org.apache.spark.sql.internal.SharedState.externalCatalog(SharedState.scala:88)
	at org.apache.spark.sql.hive.HiveSessionStateBuilder.externalCatalog(HiveSessionStateBuilder.scala:39)
	at org.apache.spark.sql.hive.HiveSessionStateBuilder.catalog$lzycompute(HiveSessionStateBuilder.scala:54)
	at org.apache.spark.sql.hive.HiveSessionStateBuilder.catalog(HiveSessionStateBuilder.scala:52)
	at org.apache.spark.sql.hive.HiveSessionStateBuilder$$anon$1.<init>(HiveSessionStateBuilder.scala:69)
	at org.apache.spark.sql.hive.HiveSessionStateBuilder.analyzer(HiveSessionStateBuilder.scala:69)
	at org.apache.spark.sql.internal.BaseSessionStateBuilder$$anonfun$build$2.apply(BaseSessionStateBuilder.scala:293)
	at org.apache.spark.sql.internal.BaseSessionStateBuilder$$anonfun$build$2.apply(BaseSessionStateBuilder.scala:293)
	at org.apache.spark.sql.internal.SessionState.analyzer$lzycompute(SessionState.scala:79)
	at org.apache.spark.sql.internal.SessionState.analyzer(SessionState.scala:79)
	at org.apache.spark.sql.execution.QueryExecution.analyzed$lzycompute(QueryExecution.scala:70)
	at org.apache.spark.sql.execution.QueryExecution.analyzed(QueryExecution.scala:68)
	at org.apache.spark.sql.execution.QueryExecution.assertAnalyzed(QueryExecution.scala:51)
	at org.apache.spark.sql.Dataset$.ofRows(Dataset.scala:70)
	at org.apache.spark.sql.SparkSession.internalCreateDataFrame(SparkSession.scala:574)
	at org.apache.spark.sql.SparkSession.createDataFrame(SparkSession.scala:593)
	at org.apache.spark.sql.SparkSession.createDataFrame(SparkSession.scala:348)
	at org.apache.spark.sql.SparkSession.createDataFrame(SparkSession.scala:348)
	at org.apache.spark.ml.image.ImageSchema$$anonfun$readImages$2$$anonfun$apply$1.apply(ImageSchema.scala:253)
...
Caused by: ERROR XJ040: Failed to start database 'metastore_db' with class loader org.apache.spark.sql.hive.client.IsolatedClientLoader$$anon$1742f639f, see the next exception for details.
	at org.apache.derby.iapi.error.StandardException.newException(Unknown Source)
	at org.apache.derby.impl.jdbc.SQLExceptionFactory.wrapArgsForTransportAcrossDRDA(Unknown Source)
	... 121 more
Caused by: ERROR XSDB6: Another instance of Derby may have already booted the database /.../spark/metastore_db.
...
Traceback (most recent call last):
  File "<stdin>", line 1, in <module>
  File "/.../spark/python/pyspark/ml/image.py", line 190, in readImages
    dropImageFailures, float(sampleRatio), seed)
  File "/.../spark/python/lib/py4j-0.10.6-src.zip/py4j/java_gateway.py", line 1160, in __call__
  File "/.../spark/python/pyspark/sql/utils.py", line 69, in deco
    raise AnalysisException(s.split(': ', 1)[1], stackTrace)
pyspark.sql.utils.AnalysisException: u'java.lang.RuntimeException: java.lang.RuntimeException: Unable to instantiate org.apache.hadoop.hive.ql.metadata.SessionHiveMetaStoreClient;'
```

Seems we better stick to `SparkSession.builder.getOrCreate()` like:

51620e288b/python/pyspark/sql/streaming.py (L329)

dc5d34d8dc/python/pyspark/sql/column.py (L541)

33d43bf1b6/python/pyspark/sql/readwriter.py (L105)

## How was this patch tested?

This was tested as below in PySpark shell:

```python
from pyspark.ml.image import ImageSchema
data_path = 'data/mllib/images/kittens'
_ = ImageSchema.readImages(data_path, recursive=True, dropImageFailures=True).collect()
_ = ImageSchema.readImages(data_path, recursive=True, dropImageFailures=True).collect()
```

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #19845 from HyukjinKwon/SPARK-22651.
2017-12-02 11:55:43 +09:00
hyukjinkwon 92cfbeeb5c [SPARK-21866][ML][PYTHON][FOLLOWUP] Few cleanups and fix image test failure in Python 3.6.0 / NumPy 1.13.3
## What changes were proposed in this pull request?

Image test seems failed in Python 3.6.0 / NumPy 1.13.3. I manually tested as below:

```
======================================================================
ERROR: test_read_images (pyspark.ml.tests.ImageReaderTest)
----------------------------------------------------------------------
Traceback (most recent call last):
  File "/.../spark/python/pyspark/ml/tests.py", line 1831, in test_read_images
    self.assertEqual(ImageSchema.toImage(array, origin=first_row[0]), first_row)
  File "/.../spark/python/pyspark/ml/image.py", line 149, in toImage
    data = bytearray(array.astype(dtype=np.uint8).ravel())
TypeError: only integer scalar arrays can be converted to a scalar index

----------------------------------------------------------------------
Ran 1 test in 7.606s
```

To be clear, I think the error seems from NumPy - 75b2d5d427/numpy/core/src/multiarray/number.c (L947)

For a smaller scope:

```python
>>> import numpy as np
>>> bytearray(np.array([1]).astype(dtype=np.uint8))
Traceback (most recent call last):
  File "<stdin>", line 1, in <module>
TypeError: only integer scalar arrays can be converted to a scalar index
```

In Python 2.7 / NumPy 1.13.1, it prints:

```
bytearray(b'\x01')
```

So, here, I simply worked around it by converting it to bytes as below:

```python
>>> bytearray(np.array([1]).astype(dtype=np.uint8).tobytes())
bytearray(b'\x01')
```

Also, while looking into it again, I realised few arguments could be quite confusing, for example, `Row` that needs some specific attributes and `numpy.ndarray`. I added few type checking and added some tests accordingly. So, it shows an error message as below:

```
TypeError: array argument should be numpy.ndarray; however, it got [<class 'str'>].
```

## How was this patch tested?

Manually tested with `./python/run-tests`.

And also:

```
PYSPARK_PYTHON=python3 SPARK_TESTING=1 bin/pyspark pyspark.ml.tests ImageReaderTest
```

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #19835 from HyukjinKwon/SPARK-21866-followup.
2017-11-30 10:26:55 +09:00
Takuya UESHIN 64817c423c [SPARK-22395][SQL][PYTHON] Fix the behavior of timestamp values for Pandas to respect session timezone
## What changes were proposed in this pull request?

When converting Pandas DataFrame/Series from/to Spark DataFrame using `toPandas()` or pandas udfs, timestamp values behave to respect Python system timezone instead of session timezone.

For example, let's say we use `"America/Los_Angeles"` as session timezone and have a timestamp value `"1970-01-01 00:00:01"` in the timezone. Btw, I'm in Japan so Python timezone would be `"Asia/Tokyo"`.

The timestamp value from current `toPandas()` will be the following:

```
>>> spark.conf.set("spark.sql.session.timeZone", "America/Los_Angeles")
>>> df = spark.createDataFrame([28801], "long").selectExpr("timestamp(value) as ts")
>>> df.show()
+-------------------+
|                 ts|
+-------------------+
|1970-01-01 00:00:01|
+-------------------+

>>> df.toPandas()
                   ts
0 1970-01-01 17:00:01
```

As you can see, the value becomes `"1970-01-01 17:00:01"` because it respects Python timezone.
As we discussed in #18664, we consider this behavior is a bug and the value should be `"1970-01-01 00:00:01"`.

## How was this patch tested?

Added tests and existing tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #19607 from ueshin/issues/SPARK-22395.
2017-11-28 16:45:22 +08:00
gaborgsomogyi 33d43bf1b6 [SPARK-22484][DOC] Document PySpark DataFrame csv writer behavior whe…
## What changes were proposed in this pull request?

In PySpark API Document, DataFrame.write.csv() says that setting the quote parameter to an empty string should turn off quoting. Instead, it uses the [null character](https://en.wikipedia.org/wiki/Null_character) as the quote.

This PR fixes the doc.

## How was this patch tested?

Manual.

```
cd python/docs
make html
open _build/html/pyspark.sql.html
```

Author: gaborgsomogyi <gabor.g.somogyi@gmail.com>

Closes #19814 from gaborgsomogyi/SPARK-22484.
2017-11-28 10:14:35 +09:00
Ilya Matiach 1edb3175d8 [SPARK-21866][ML][PYSPARK] Adding spark image reader
## What changes were proposed in this pull request?
Adding spark image reader, an implementation of schema for representing images in spark DataFrames

The code is taken from the spark package located here:
(https://github.com/Microsoft/spark-images)

Please see the JIRA for more information (https://issues.apache.org/jira/browse/SPARK-21866)

Please see mailing list for SPIP vote and approval information:
(http://apache-spark-developers-list.1001551.n3.nabble.com/VOTE-SPIP-SPARK-21866-Image-support-in-Apache-Spark-td22510.html)

# Background and motivation
As Apache Spark is being used more and more in the industry, some new use cases are emerging for different data formats beyond the traditional SQL types or the numerical types (vectors and matrices). Deep Learning applications commonly deal with image processing. A number of projects add some Deep Learning capabilities to Spark (see list below), but they struggle to communicate with each other or with MLlib pipelines because there is no standard way to represent an image in Spark DataFrames. We propose to federate efforts for representing images in Spark by defining a representation that caters to the most common needs of users and library developers.
This SPIP proposes a specification to represent images in Spark DataFrames and Datasets (based on existing industrial standards), and an interface for loading sources of images. It is not meant to be a full-fledged image processing library, but rather the core description that other libraries and users can rely on. Several packages already offer various processing facilities for transforming images or doing more complex operations, and each has various design tradeoffs that make them better as standalone solutions.
This project is a joint collaboration between Microsoft and Databricks, which have been testing this design in two open source packages: MMLSpark and Deep Learning Pipelines.
The proposed image format is an in-memory, decompressed representation that targets low-level applications. It is significantly more liberal in memory usage than compressed image representations such as JPEG, PNG, etc., but it allows easy communication with popular image processing libraries and has no decoding overhead.

## How was this patch tested?

Unit tests in scala ImageSchemaSuite, unit tests in python

Author: Ilya Matiach <ilmat@microsoft.com>
Author: hyukjinkwon <gurwls223@gmail.com>

Closes #19439 from imatiach-msft/ilmat/spark-images.
2017-11-22 15:45:45 -08:00
WeichenXu 2d868d9398 [SPARK-22521][ML] VectorIndexerModel support handle unseen categories via handleInvalid: Python API
## What changes were proposed in this pull request?

Add python api for VectorIndexerModel support handle unseen categories via handleInvalid.

## How was this patch tested?

doctest added.

Author: WeichenXu <weichen.xu@databricks.com>

Closes #19753 from WeichenXu123/vector_indexer_invalid_py.
2017-11-21 10:53:53 -08:00
Liang-Chi Hsieh 9d45e675e2 [SPARK-22541][SQL] Explicitly claim that Python udfs can't be conditionally executed with short-curcuit evaluation
## What changes were proposed in this pull request?

Besides conditional expressions such as `when` and `if`, users may want to conditionally execute python udfs by short-curcuit evaluation. We should also explicitly note that python udfs don't support this kind of conditional execution too.

## How was this patch tested?

N/A, just document change.

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

Closes #19787 from viirya/SPARK-22541.
2017-11-21 09:36:37 +01:00
Li Jin 7d039e0c0a [SPARK-22409] Introduce function type argument in pandas_udf
## What changes were proposed in this pull request?

* Add a "function type" argument to pandas_udf.
* Add a new public enum class `PandasUdfType` in pyspark.sql.functions
* Refactor udf related code from pyspark.sql.functions to pyspark.sql.udf
* Merge "PythonUdfType" and "PythonEvalType" into a single enum class "PythonEvalType"

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

pandas_udf('double', PandasUDFType.SCALAR):
def plus_one(v):
    return v + 1
```

## Design doc
https://docs.google.com/document/d/1KlLaa-xJ3oz28xlEJqXyCAHU3dwFYkFs_ixcUXrJNTc/edit

## How was this patch tested?

Added PandasUDFTests

## TODO:
* [x] Implement proper enum type for `PandasUDFType`
* [x] Update documentation
* [x] Add more tests in PandasUDFTests

Author: Li Jin <ice.xelloss@gmail.com>

Closes #19630 from icexelloss/spark-22409-pandas-udf-type.
2017-11-17 16:43:08 +01:00
Dongjoon Hyun aa88b8dbbb [SPARK-22490][DOC] Add PySpark doc for SparkSession.builder
## What changes were proposed in this pull request?

In PySpark API Document, [SparkSession.build](http://spark.apache.org/docs/2.2.0/api/python/pyspark.sql.html) is not documented and shows default value description.
```
SparkSession.builder = <pyspark.sql.session.Builder object ...
```

This PR adds the doc.

![screen](https://user-images.githubusercontent.com/9700541/32705514-1bdcafaa-c7ca-11e7-88bf-05566fea42de.png)

The following is the diff of the generated result.

```
$ diff old.html new.html
95a96,101
> <dl class="attribute">
> <dt id="pyspark.sql.SparkSession.builder">
> <code class="descname">builder</code><a class="headerlink" href="#pyspark.sql.SparkSession.builder" title="Permalink to this definition">¶</a></dt>
> <dd><p>A class attribute having a <a class="reference internal" href="#pyspark.sql.SparkSession.Builder" title="pyspark.sql.SparkSession.Builder"><code class="xref py py-class docutils literal"><span class="pre">Builder</span></code></a> to construct <a class="reference internal" href="#pyspark.sql.SparkSession" title="pyspark.sql.SparkSession"><code class="xref py py-class docutils literal"><span class="pre">SparkSession</span></code></a> instances</p>
> </dd></dl>
>
212,216d217
< <dt id="pyspark.sql.SparkSession.builder">
< <code class="descname">builder</code><em class="property"> = &lt;pyspark.sql.session.SparkSession.Builder object&gt;</em><a class="headerlink" href="#pyspark.sql.SparkSession.builder" title="Permalink to this definition">¶</a></dt>
< <dd></dd></dl>
<
< <dl class="attribute">
```

## How was this patch tested?

Manual.

```
cd python/docs
make html
open _build/html/pyspark.sql.html
```

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #19726 from dongjoon-hyun/SPARK-22490.
2017-11-15 08:59:29 -08:00