Commit graph

173 commits

Author SHA1 Message Date
Zhenhua Wang 365a29bdbf [SPARK-22100][SQL] Make percentile_approx support date/timestamp type and change the output type to be the same as input type
## What changes were proposed in this pull request?

The `percentile_approx` function previously accepted numeric type input and output double type results.

But since all numeric types, date and timestamp types are represented as numerics internally, `percentile_approx` can support them easily.

After this PR, it supports date type, timestamp type and numeric types as input types. The result type is also changed to be the same as the input type, which is more reasonable for percentiles.

This change is also required when we generate equi-height histograms for these types.

## How was this patch tested?

Added a new test and modified some existing tests.

Author: Zhenhua Wang <wangzhenhua@huawei.com>

Closes #19321 from wzhfy/approx_percentile_support_types.
2017-09-25 09:28:42 -07:00
Liang-Chi Hsieh 3e6a714c9e [SPARK-21766][PYSPARK][SQL] DataFrame toPandas() raises ValueError with nullable int columns
## What changes were proposed in this pull request?

When calling `DataFrame.toPandas()` (without Arrow enabled), if there is a `IntegralType` column (`IntegerType`, `ShortType`, `ByteType`) that has null values the following exception is thrown:

    ValueError: Cannot convert non-finite values (NA or inf) to integer

This is because the null values first get converted to float NaN during the construction of the Pandas DataFrame in `from_records`, and then it is attempted to be converted back to to an integer where it fails.

The fix is going to check if the Pandas DataFrame can cause such failure when converting, if so, we don't do the conversion and use the inferred type by Pandas.

Closes #18945

## How was this patch tested?

Added pyspark test.

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

Closes #19319 from viirya/SPARK-21766.
2017-09-22 22:39:47 +09:00
Yanbo Liang e4d8f9a36a [MINOR][SQL] Correct DataFrame doc.
## What changes were proposed in this pull request?
Correct DataFrame doc.

## How was this patch tested?
Only doc change, no tests.

Author: Yanbo Liang <ybliang8@gmail.com>

Closes #19173 from yanboliang/df-doc.
2017-09-09 09:25:12 -07:00
hyukjinkwon 8598d03a00 [SPARK-15243][ML][SQL][PYTHON] Add missing support for unicode in Param methods & functions in dataframe
## What changes were proposed in this pull request?

This PR proposes to support unicodes in Param methods in ML, other missed functions in DataFrame.

For example, this causes a `ValueError` in Python 2.x when param is a unicode string:

```python
>>> from pyspark.ml.classification import LogisticRegression
>>> lr = LogisticRegression()
>>> lr.hasParam("threshold")
True
>>> lr.hasParam(u"threshold")
Traceback (most recent call last):
 ...
    raise TypeError("hasParam(): paramName must be a string")
TypeError: hasParam(): paramName must be a string
```

This PR is based on https://github.com/apache/spark/pull/13036

## How was this patch tested?

Unit tests in `python/pyspark/ml/tests.py` and `python/pyspark/sql/tests.py`.

Author: hyukjinkwon <gurwls223@gmail.com>
Author: sethah <seth.hendrickson16@gmail.com>

Closes #17096 from HyukjinKwon/SPARK-15243.
2017-09-08 11:57:33 -07:00
hyukjinkwon 07fd68a29f [SPARK-21897][PYTHON][R] Add unionByName API to DataFrame in Python and R
## What changes were proposed in this pull request?

This PR proposes to add a wrapper for `unionByName` API to R and Python as well.

**Python**

```python
df1 = spark.createDataFrame([[1, 2, 3]], ["col0", "col1", "col2"])
df2 = spark.createDataFrame([[4, 5, 6]], ["col1", "col2", "col0"])
df1.unionByName(df2).show()
```

```
+----+----+----+
|col0|col1|col3|
+----+----+----+
|   1|   2|   3|
|   6|   4|   5|
+----+----+----+
```

**R**

```R
df1 <- select(createDataFrame(mtcars), "carb", "am", "gear")
df2 <- select(createDataFrame(mtcars), "am", "gear", "carb")
head(unionByName(limit(df1, 2), limit(df2, 2)))
```

```
  carb am gear
1    4  1    4
2    4  1    4
3    4  1    4
4    4  1    4
```

## How was this patch tested?

Doctests for Python and unit test added in `test_sparkSQL.R` for R.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #19105 from HyukjinKwon/unionByName-r-python.
2017-09-03 21:03:21 +09:00
hyukjinkwon 5cd8ea99f0 [SPARK-21779][PYTHON] Simpler DataFrame.sample API in Python
## What changes were proposed in this pull request?

This PR make `DataFrame.sample(...)` can omit `withReplacement` defaulting `False`, consistently with equivalent Scala / Java API.

In short, the following examples are allowed:

```python
>>> df = spark.range(10)
>>> df.sample(0.5).count()
7
>>> df.sample(fraction=0.5).count()
3
>>> df.sample(0.5, seed=42).count()
5
>>> df.sample(fraction=0.5, seed=42).count()
5
```

In addition, this PR also adds some type checking logics as below:

```python
>>> df = spark.range(10)
>>> df.sample().count()
...
TypeError: withReplacement (optional), fraction (required) and seed (optional) should be a bool, float and number; however, got [].
>>> df.sample(True).count()
...
TypeError: withReplacement (optional), fraction (required) and seed (optional) should be a bool, float and number; however, got [<type 'bool'>].
>>> df.sample(42).count()
...
TypeError: withReplacement (optional), fraction (required) and seed (optional) should be a bool, float and number; however, got [<type 'int'>].
>>> df.sample(fraction=False, seed="a").count()
...
TypeError: withReplacement (optional), fraction (required) and seed (optional) should be a bool, float and number; however, got [<type 'bool'>, <type 'str'>].
>>> df.sample(seed=[1]).count()
...
TypeError: withReplacement (optional), fraction (required) and seed (optional) should be a bool, float and number; however, got [<type 'list'>].
>>> df.sample(withReplacement="a", fraction=0.5, seed=1)
...
TypeError: withReplacement (optional), fraction (required) and seed (optional) should be a bool, float and number; however, got [<type 'str'>, <type 'float'>, <type 'int'>].
```

## How was this patch tested?

Manually tested, unit tests added in doc tests and manually checked the built documentation for Python.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #18999 from HyukjinKwon/SPARK-21779.
2017-09-01 13:01:23 +09:00
Andrew Ray 10be01848e [SPARK-21566][SQL][PYTHON] Python method for summary
## What changes were proposed in this pull request?

Adds the recently added `summary` method to the python dataframe interface.

## How was this patch tested?

Additional inline doctests.

Author: Andrew Ray <ray.andrew@gmail.com>

Closes #18762 from aray/summary-py.
2017-08-18 18:10:54 -07:00
byakuinss 0fcde87aad [SPARK-21658][SQL][PYSPARK] Add default None for value in na.replace in PySpark
## What changes were proposed in this pull request?
JIRA issue: https://issues.apache.org/jira/browse/SPARK-21658

Add default None for value in `na.replace` since `Dataframe.replace` and `DataframeNaFunctions.replace` are alias.

The default values are the same now.
```
>>> df = sqlContext.createDataFrame([('Alice', 10, 80.0)])
>>> df.replace({"Alice": "a"}).first()
Row(_1=u'a', _2=10, _3=80.0)
>>> df.na.replace({"Alice": "a"}).first()
Row(_1=u'a', _2=10, _3=80.0)
```

## How was this patch tested?
Existing tests.

cc viirya

Author: byakuinss <grace.chinhanyu@gmail.com>

Closes #18895 from byakuinss/SPARK-21658.
2017-08-15 00:41:01 +09:00
bravo-zhang 84454d7d33 [SPARK-14932][SQL] Allow DataFrame.replace() to replace values with None
## What changes were proposed in this pull request?

Currently `df.na.replace("*", Map[String, String]("NULL" -> null))` will produce exception.
This PR enables passing null/None as value in the replacement map in DataFrame.replace().
Note that the replacement map keys and values should still be the same type, while the values can have a mix of null/None and that type.
This PR enables following operations for example:
`df.na.replace("*", Map[String, String]("NULL" -> null))`(scala)
`df.na.replace("*", Map[Any, Any](60 -> null, 70 -> 80))`(scala)
`df.na.replace('Alice', None)`(python)
`df.na.replace([10, 20])`(python, replacing with None is by default)
One use case could be: I want to replace all the empty strings with null/None because they were incorrectly generated and then drop all null/None data
`df.na.replace("*", Map("" -> null)).na.drop()`(scala)
`df.replace(u'', None).dropna()`(python)

## How was this patch tested?

Scala unit test.
Python doctest and unit test.

Author: bravo-zhang <mzhang1230@gmail.com>

Closes #18820 from bravo-zhang/spark-14932.
2017-08-09 17:42:21 -07:00
Bryan Cutler d03aebbe65 [SPARK-13534][PYSPARK] Using Apache Arrow to increase performance of DataFrame.toPandas
## What changes were proposed in this pull request?
Integrate Apache Arrow with Spark to increase performance of `DataFrame.toPandas`.  This has been done by using Arrow to convert data partitions on the executor JVM to Arrow payload byte arrays where they are then served to the Python process.  The Python DataFrame can then collect the Arrow payloads where they are combined and converted to a Pandas DataFrame.  Data types except complex, date, timestamp, and decimal  are currently supported, otherwise an `UnsupportedOperation` exception is thrown.

Additions to Spark include a Scala package private method `Dataset.toArrowPayload` that will convert data partitions in the executor JVM to `ArrowPayload`s as byte arrays so they can be easily served.  A package private class/object `ArrowConverters` that provide data type mappings and conversion routines.  In Python, a private method `DataFrame._collectAsArrow` is added to collect Arrow payloads and a SQLConf "spark.sql.execution.arrow.enable" can be used in `toPandas()` to enable using Arrow (uses the old conversion by default).

## How was this patch tested?
Added a new test suite `ArrowConvertersSuite` that will run tests on conversion of Datasets to Arrow payloads for supported types.  The suite will generate a Dataset and matching Arrow JSON data, then the dataset is converted to an Arrow payload and finally validated against the JSON data.  This will ensure that the schema and data has been converted correctly.

Added PySpark tests to verify the `toPandas` method is producing equal DataFrames with and without pyarrow.  A roundtrip test to ensure the pandas DataFrame produced by pyspark is equal to a one made directly with pandas.

Author: Bryan Cutler <cutlerb@gmail.com>
Author: Li Jin <ice.xelloss@gmail.com>
Author: Li Jin <li.jin@twosigma.com>
Author: Wes McKinney <wes.mckinney@twosigma.com>

Closes #18459 from BryanCutler/toPandas_with_arrow-SPARK-13534.
2017-07-10 15:21:03 -07:00
hyukjinkwon a848d552ef [SPARK-21264][PYTHON] Call cross join path in join without 'on' and with 'how'
## What changes were proposed in this pull request?

Currently, it throws a NPE when missing columns but join type is speicified in join at PySpark as below:

```python
spark.conf.set("spark.sql.crossJoin.enabled", "false")
spark.range(1).join(spark.range(1), how="inner").show()
```

```
Traceback (most recent call last):
...
py4j.protocol.Py4JJavaError: An error occurred while calling o66.join.
: java.lang.NullPointerException
	at org.apache.spark.sql.Dataset.join(Dataset.scala:931)
	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
...
```

```python
spark.conf.set("spark.sql.crossJoin.enabled", "true")
spark.range(1).join(spark.range(1), how="inner").show()
```

```
...
py4j.protocol.Py4JJavaError: An error occurred while calling o84.join.
: java.lang.NullPointerException
	at org.apache.spark.sql.Dataset.join(Dataset.scala:931)
	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
...
```

This PR suggests to follow Scala's one as below:

```scala
scala> spark.conf.set("spark.sql.crossJoin.enabled", "false")

scala> spark.range(1).join(spark.range(1), Seq.empty[String], "inner").show()
```

```
org.apache.spark.sql.AnalysisException: Detected cartesian product for INNER join between logical plans
Range (0, 1, step=1, splits=Some(8))
and
Range (0, 1, step=1, splits=Some(8))
Join condition is missing or trivial.
Use the CROSS JOIN syntax to allow cartesian products between these relations.;
...
```

```scala
scala> spark.conf.set("spark.sql.crossJoin.enabled", "true")

scala> spark.range(1).join(spark.range(1), Seq.empty[String], "inner").show()
```
```
+---+---+
| id| id|
+---+---+
|  0|  0|
+---+---+
```

**After**

```python
spark.conf.set("spark.sql.crossJoin.enabled", "false")
spark.range(1).join(spark.range(1), how="inner").show()
```

```
Traceback (most recent call last):
...
pyspark.sql.utils.AnalysisException: u'Detected cartesian product for INNER join between logical plans\nRange (0, 1, step=1, splits=Some(8))\nand\nRange (0, 1, step=1, splits=Some(8))\nJoin condition is missing or trivial.\nUse the CROSS JOIN syntax to allow cartesian products between these relations.;'
```

```python
spark.conf.set("spark.sql.crossJoin.enabled", "true")
spark.range(1).join(spark.range(1), how="inner").show()
```
```
+---+---+
| id| id|
+---+---+
|  0|  0|
+---+---+
```

## How was this patch tested?

Added tests in `python/pyspark/sql/tests.py`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #18484 from HyukjinKwon/SPARK-21264.
2017-07-04 11:35:08 +09:00
Wenchen Fan 838effb98a Revert "[SPARK-13534][PYSPARK] Using Apache Arrow to increase performance of DataFrame.toPandas"
This reverts commit e44697606f.
2017-06-28 14:28:40 +08:00
Bryan Cutler e44697606f [SPARK-13534][PYSPARK] Using Apache Arrow to increase performance of DataFrame.toPandas
## What changes were proposed in this pull request?
Integrate Apache Arrow with Spark to increase performance of `DataFrame.toPandas`.  This has been done by using Arrow to convert data partitions on the executor JVM to Arrow payload byte arrays where they are then served to the Python process.  The Python DataFrame can then collect the Arrow payloads where they are combined and converted to a Pandas DataFrame.  All non-complex data types are currently supported, otherwise an `UnsupportedOperation` exception is thrown.

Additions to Spark include a Scala package private method `Dataset.toArrowPayloadBytes` that will convert data partitions in the executor JVM to `ArrowPayload`s as byte arrays so they can be easily served.  A package private class/object `ArrowConverters` that provide data type mappings and conversion routines.  In Python, a public method `DataFrame.collectAsArrow` is added to collect Arrow payloads and an optional flag in `toPandas(useArrow=False)` to enable using Arrow (uses the old conversion by default).

## How was this patch tested?
Added a new test suite `ArrowConvertersSuite` that will run tests on conversion of Datasets to Arrow payloads for supported types.  The suite will generate a Dataset and matching Arrow JSON data, then the dataset is converted to an Arrow payload and finally validated against the JSON data.  This will ensure that the schema and data has been converted correctly.

Added PySpark tests to verify the `toPandas` method is producing equal DataFrames with and without pyarrow.  A roundtrip test to ensure the pandas DataFrame produced by pyspark is equal to a one made directly with pandas.

Author: Bryan Cutler <cutlerb@gmail.com>
Author: Li Jin <ice.xelloss@gmail.com>
Author: Li Jin <li.jin@twosigma.com>
Author: Wes McKinney <wes.mckinney@twosigma.com>

Closes #15821 from BryanCutler/wip-toPandas_with_arrow-SPARK-13534.
2017-06-23 09:01:13 +08:00
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
Reynold Xin b78e3849b2 [SPARK-21042][SQL] Document Dataset.union is resolution by position
## What changes were proposed in this pull request?
Document Dataset.union is resolution by position, not by name, since this has been a confusing point for a lot of users.

## How was this patch tested?
N/A - doc only change.

Author: Reynold Xin <rxin@databricks.com>

Closes #18256 from rxin/SPARK-21042.
2017-06-09 18:29:33 -07: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
gatorsmile de934e6718 [SPARK-19236][SQL][FOLLOW-UP] Added createOrReplaceGlobalTempView method
### What changes were proposed in this pull request?
This PR does the following tasks:
- Added  since
- Added the Python API
- Added test cases

### How was this patch tested?
Added test cases to both Scala and Python

Author: gatorsmile <gatorsmile@gmail.com>

Closes #18147 from gatorsmile/createOrReplaceGlobalTempView.
2017-05-31 11:38:43 -07:00
Michael Armbrust d935e0a9d9 [SPARK-20844] Remove experimental from Structured Streaming APIs
Now that Structured Streaming has been out for several Spark release and has large production use cases, the `Experimental` label is no longer appropriate.  I've left `InterfaceStability.Evolving` however, as I think we may make a few changes to the pluggable Source & Sink API in Spark 2.3.

Author: Michael Armbrust <michael@databricks.com>

Closes #18065 from marmbrus/streamingGA.
2017-05-26 13:33:23 -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
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
Takeshi Yamamuro b4724db19a [SPARK-20425][SQL] Support a vertical display mode for Dataset.show
## What changes were proposed in this pull request?
This pr added a new display mode for `Dataset.show` to print output rows vertically (one line per column value). In the current master, when printing Dataset with many columns, the readability is low like;

```
scala> val df = spark.range(100).selectExpr((0 until 100).map(i => s"rand() AS c$i"): _*)
scala> df.show(3, 0)
+------------------+------------------+------------------+-------------------+------------------+------------------+-------------------+------------------+------------------+------------------+------------------+-------------------+------------------+------------------+------------------+-------------------+-------------------+-------------------+------------------+------------------+-------------------+------------------+-------------------+------------------+-------------------+-------------------+-------------------+--------------------+-------------------+------------------+-------------------+--------------------+------------------+------------------+-------------------+-------------------+-------------------+------------------+------------------+-------------------+------------------+------------------+-------------------+-------------------+-------------------+------------------+--------------------+--------------------+-------------------+-------------------+-------------------+-------------------+-------------------+-------------------+--------------------+-------------------+-------------------+-------------------+-------------------+------------------+------------------+-------------------+-------------------+------------------+-------------------+------------------+------------------+-----------------+-------------------+-------------------+------------------+-------------------+------------------+-------------------+-------------------+-------------------+------------------+-------------------+------------------+-------------------+-------------------+-------------------+-------------------+-------------------+-------------------+-------------------+-------------------+------------------+-------------------+-------------------+------------------+------------------+------------------+-------------------+------------------+-------------------+------------------+-------------------+-------------------+-------------------+
|c0                |c1                |c2                |c3                 |c4                |c5                |c6                 |c7                |c8                |c9                |c10               |c11                |c12               |c13               |c14               |c15                |c16                |c17                |c18               |c19               |c20                |c21               |c22                |c23               |c24                |c25                |c26                |c27                 |c28                |c29               |c30                |c31                 |c32               |c33               |c34                |c35                |c36                |c37               |c38               |c39                |c40               |c41               |c42                |c43                |c44                |c45               |c46                 |c47                 |c48                |c49                |c50                |c51                |c52                |c53                |c54                 |c55                |c56                |c57                |c58                |c59               |c60               |c61                |c62                |c63               |c64                |c65               |c66               |c67              |c68                |c69                |c70               |c71                |c72               |c73                |c74                |c75                |c76               |c77                |c78               |c79                |c80                |c81                |c82                |c83                |c84                |c85                |c86                |c87               |c88                |c89                |c90               |c91               |c92               |c93                |c94               |c95                |c96               |c97                |c98                |c99                |
+------------------+------------------+------------------+-------------------+------------------+------------------+-------------------+------------------+------------------+------------------+------------------+-------------------+------------------+------------------+------------------+-------------------+-------------------+-------------------+------------------+------------------+-------------------+------------------+-------------------+------------------+-------------------+-------------------+-------------------+--------------------+-------------------+------------------+-------------------+--------------------+------------------+------------------+-------------------+-------------------+-------------------+------------------+------------------+-------------------+------------------+------------------+-------------------+-------------------+-------------------+------------------+--------------------+--------------------+-------------------+-------------------+-------------------+-------------------+-------------------+-------------------+--------------------+-------------------+-------------------+-------------------+-------------------+------------------+------------------+-------------------+-------------------+------------------+-------------------+------------------+------------------+-----------------+-------------------+-------------------+------------------+-------------------+------------------+-------------------+-------------------+-------------------+------------------+-------------------+------------------+-------------------+-------------------+-------------------+-------------------+-------------------+-------------------+-------------------+-------------------+------------------+-------------------+-------------------+------------------+------------------+------------------+-------------------+------------------+-------------------+------------------+-------------------+-------------------+-------------------+
|0.6306087152476858|0.9174349686288383|0.5511324165035159|0.3320844128641819 |0.7738486877101489|0.2154915886962553|0.4754997600674299 |0.922780639280355 |0.7136894772661909|0.2277580838165979|0.5926874459847249|0.40311408392226633|0.467830264333843 |0.8330466896984213|0.1893258482389527|0.6320849515511165 |0.7530911056912044 |0.06700254871955424|0.370528597355559 |0.2755437445193154|0.23704391110980128|0.8067400174905822|0.13597793616251852|0.1708888820162453|0.01672725007605702|0.983118121881555  |0.25040195628629924|0.060537253723083384|0.20000530582637488|0.3400572407133511|0.9375689433322597 |0.057039316954370256|0.8053269714347623|0.5247817572228813|0.28419308820527944|0.9798908885194533 |0.31805988175678146|0.7034448027077574|0.5400575751346084|0.25336322371116216|0.9361634546853429|0.6118681368289798|0.6295081549153907 |0.13417468943957422|0.41617137072255794|0.7267230869252035|0.023792726137561115|0.5776157058356362  |0.04884204913195467|0.26728716103441275|0.646680370807925  |0.9782712690657244 |0.16434031314818154|0.20985522381321275|0.24739842475440077 |0.26335189682977334|0.19604841662422068|0.10742950487300651|0.20283136488091502|0.3100312319723688|0.886959006630645 |0.25157102269776244|0.34428775168410786|0.3500506818575777|0.3781142441912052 |0.8560316444386715|0.4737104888956839|0.735903101602148|0.02236617130529006|0.8769074095835873 |0.2001426662503153|0.5534032319238532 |0.7289496620397098|0.41955191309992157|0.9337700133660436 |0.34059094378451005|0.6419144759403556|0.08167496930341167|0.9947099478497635|0.48010888605366586|0.22314796858167918|0.17786598882331306|0.7351521162297135 |0.5422057170020095 |0.9521927872726792 |0.7459825486368227 |0.40907708791990627|0.8903819313311575|0.7251413746923618 |0.2977174938745204 |0.9515209660203555|0.9375968604766713|0.5087851740042524|0.4255237544908751 |0.8023768698664653|0.48003189618006703|0.1775841829745185|0.09050775629268382|0.6743909291138167 |0.2498415755876865 |
|0.6866473844170801|0.4774360641212433|0.631696201340726 |0.33979113021468343|0.5663049010847052|0.7280190472258865|0.41370958502324806|0.9977433873622218|0.7671957338989901|0.2788708556233931|0.3355106391656496|0.88478952319287   |0.0333974166999893|0.6061744715862606|0.9617779139652359|0.22484954822341863|0.12770906021550898|0.5577789629508672 |0.2877649024640704|0.5566577406549361|0.9334933255278052 |0.9166720585157266|0.9689249324600591 |0.6367502457478598|0.7993572745928459 |0.23213222324218108|0.11928284054154137|0.6173493362456599  |0.0505122058694798 |0.9050228629552983|0.17112767911121707|0.47395598348370005 |0.5820498657823081|0.6241124650645072|0.18587258258036776|0.14987593554122225|0.3079446253653946 |0.9414228822867968|0.8362276265462365|0.9155655305576353 |0.5121559807153562|0.8963362656525707|0.22765970274318037|0.8177039187132797 |0.8190326635933787 |0.5256005177032199|0.8167598457269669  |0.030936807130934496|0.6733006585281015 |0.4208049626816347 |0.24603085738518538|0.22719198954208153|0.1622280557565281 |0.22217325159218038|0.014684419513742553|0.08987111517447499|0.2157764759142622 |0.8223414104088321 |0.4868624404491777 |0.4016191733088167|0.6169281906889263|0.15603611040433385|0.18289285085714913|0.9538408988218972|0.15037154865295121|0.5364516961987454|0.8077254873163031|0.712600478545675|0.7277477241003857 |0.19822912960348305|0.8305051199208777|0.18631911396566114|0.8909532487898342|0.3470409226992506 |0.35306974180587636|0.9107058868891469 |0.3321327206004986|0.48952332459050607|0.3630403307479373|0.5400046826340376 |0.5387377194310529 |0.42860539421837585|0.23214101630985995|0.21438968839794847|0.15370603160082352|0.04355605642700022|0.6096006707067466 |0.6933354157094292|0.06302172470859002|0.03174631856164001|0.664243581650643 |0.7833239547446621|0.696884598352864 |0.34626385933237736|0.9263495598791336|0.404818892816584  |0.2085585394755507|0.6150004897990109 |0.05391193524302473|0.28188484028329097|
+------------------+------------------+------------------+-------------------+------------------+------------------+-------------------+------------------+------------------+------------------+------------------+-------------------+------------------+------------------+------------------+-------------------+-------------------+-------------------+------------------+------------------+-------------------+------------------+-------------------+------------------+-------------------+-------------------+-------------------+--------------------+-------------------+------------------+-------------------+--------------------+------------------+------------------+-------------------+-------------------+-------------------+------------------+------------------+-------------------+------------------+------------------+-------------------+-------------------+-------------------+------------------+--------------------+--------------------+-------------------+-------------------+-------------------+-------------------+-------------------+-------------------+--------------------+-------------------+-------------------+-------------------+-------------------+------------------+------------------+-------------------+-------------------+------------------+-------------------+------------------+------------------+-----------------+-------------------+-------------------+------------------+-------------------+------------------+-------------------+-------------------+-------------------+------------------+-------------------+------------------+-------------------+-------------------+-------------------+-------------------+-------------------+-------------------+-------------------+-------------------+------------------+-------------------+-------------------+------------------+------------------+------------------+-------------------+------------------+-------------------+------------------+-------------------+-------------------+-------------------+
only showing top 2 rows
```

`psql`, CLI for PostgreSQL, supports a vertical display mode for this case like:
http://stackoverflow.com/questions/9604723/alternate-output-format-for-psql

```
-RECORD 0-------------------
 c0  | 0.6306087152476858
 c1  | 0.9174349686288383
 c2  | 0.5511324165035159
...
 c98 | 0.05391193524302473
 c99 | 0.28188484028329097
-RECORD 1-------------------
 c0  | 0.6866473844170801
 c1  | 0.4774360641212433
 c2  | 0.631696201340726
...
 c98 | 0.05391193524302473
 c99 | 0.28188484028329097
only showing top 2 rows
```

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #17733 from maropu/SPARK-20425.
2017-04-26 22:18:01 -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
Zheng RuiFeng 63f077fbe5 [SPARK-20041][DOC] Update docs for NaN handling in approxQuantile
## What changes were proposed in this pull request?
Update docs for NaN handling in approxQuantile.

## How was this patch tested?
existing tests.

Author: Zheng RuiFeng <ruifengz@foxmail.com>

Closes #17369 from zhengruifeng/doc_quantiles_nan.
2017-03-21 08:45:59 -07:00
Shixiong Zhu 9bf4e2baad [SPARK-19497][SS] Implement streaming deduplication
## What changes were proposed in this pull request?

This PR adds a special streaming deduplication operator to support `dropDuplicates` with `aggregation` and watermark. It reuses the `dropDuplicates` API but creates new logical plan `Deduplication` and new physical plan `DeduplicationExec`.

The following cases are supported:

- one or multiple `dropDuplicates()` without aggregation (with or without watermark)
- `dropDuplicates` before aggregation

Not supported cases:

- `dropDuplicates` after aggregation

Breaking changes:
- `dropDuplicates` without aggregation doesn't work with `complete` or `update` mode.

## How was this patch tested?

The new unit tests.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #16970 from zsxwing/dedup.
2017-02-23 11:25:39 -08:00
Felix Cheung 671bc08ed5 [SPARK-19399][SPARKR] Add R coalesce API for DataFrame and Column
## What changes were proposed in this pull request?

Add coalesce on DataFrame for down partitioning without shuffle and coalesce on Column

## How was this patch tested?

manual, unit tests

Author: Felix Cheung <felixcheung_m@hotmail.com>

Closes #16739 from felixcheung/rcoalesce.
2017-02-15 10:45:37 -08:00
zero323 9c4405e8e8 [SPARK-19453][PYTHON][SQL][DOC] Correct and extend DataFrame.replace docstring
## What changes were proposed in this pull request?

- Provides correct description of the semantics of a `dict` argument passed as `to_replace`.
- Describes type requirements for collection arguments.
- Describes behavior with `to_replace: List[T]` and `value: T`

## How was this patch tested?

Manual testing, documentation build.

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

Closes #16792 from zero323/SPARK-19453.
2017-02-14 09:42:24 -08: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
anabranch 19d9d4c855 [SPARK-19126][DOCS] Update Join Documentation Across Languages
## What changes were proposed in this pull request?

- [X] Make sure all join types are clearly mentioned
- [X] Make join labeling/style consistent
- [X] Make join label ordering docs the same
- [X] Improve join documentation according to above for Scala
- [X] Improve join documentation according to above for Python
- [X] Improve join documentation according to above for R

## How was this patch tested?
No tests b/c docs.

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

Author: anabranch <wac.chambers@gmail.com>

Closes #16504 from anabranch/SPARK-19126.
2017-01-08 20:37:46 -08:00
hyukjinkwon 933a6548d4
[SPARK-18447][DOCS] Fix the markdown for Note:/NOTE:/Note that across Python API documentation
## What changes were proposed in this pull request?

It seems in Python, there are

- `Note:`
- `NOTE:`
- `Note that`
- `.. note::`

This PR proposes to fix those to `.. note::` to be consistent.

**Before**

<img width="567" alt="2016-11-21 1 18 49" src="https://cloud.githubusercontent.com/assets/6477701/20464305/85144c86-af88-11e6-8ee9-90f584dd856c.png">

<img width="617" alt="2016-11-21 12 42 43" src="https://cloud.githubusercontent.com/assets/6477701/20464263/27be5022-af88-11e6-8577-4bbca7cdf36c.png">

**After**

<img width="554" alt="2016-11-21 1 18 42" src="https://cloud.githubusercontent.com/assets/6477701/20464306/8fe48932-af88-11e6-83e1-fc3cbf74407d.png">

<img width="628" alt="2016-11-21 12 42 51" src="https://cloud.githubusercontent.com/assets/6477701/20464264/2d3e156e-af88-11e6-93f3-cab8d8d02983.png">

## How was this patch tested?

The notes were found via

```bash
grep -r "Note: " .
grep -r "NOTE: " .
grep -r "Note that " .
```

And then fixed one by one comparing with API documentation.

After that, manually tested via `make html` under `./python/docs`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #15947 from HyukjinKwon/SPARK-18447.
2016-11-22 11:40:18 +00:00
Burak Yavuz 97a8239a62 [SPARK-18493] Add missing python APIs: withWatermark and checkpoint to dataframe
## What changes were proposed in this pull request?

This PR adds two of the newly added methods of `Dataset`s to Python:
`withWatermark` and `checkpoint`

## How was this patch tested?

Doc tests

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #15921 from brkyvz/py-watermark.
2016-11-21 17:24:02 -08:00
anabranch 49b6f456ac
[SPARK-18365][DOCS] Improve Sample Method Documentation
## What changes were proposed in this pull request?

I found the documentation for the sample method to be confusing, this adds more clarification across all languages.

- [x] Scala
- [x] Python
- [x] R
- [x] RDD Scala
- [ ] RDD Python with SEED
- [X] RDD Java
- [x] RDD Java with SEED
- [x] RDD Python

## How was this patch tested?

NA

Please review https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark before opening a pull request.

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

Closes #15815 from anabranch/SPARK-18365.
2016-11-17 11:34:55 +00: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
Nick Pentreath 5aeb7384c7 [SPARK-16063][SQL] Add storageLevel to Dataset
[SPARK-11905](https://issues.apache.org/jira/browse/SPARK-11905) added support for `persist`/`cache` for `Dataset`. However, there is no user-facing API to check if a `Dataset` is cached and if so what the storage level is. This PR adds `getStorageLevel` to `Dataset`, analogous to `RDD.getStorageLevel`.

Updated `DatasetCacheSuite`.

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

Closes #13780 from MLnick/ds-storagelevel.

Signed-off-by: Michael Armbrust <michael@databricks.com>
2016-10-14 15:09:49 -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
Wenchen Fan 23ddff4b2b [SPARK-17338][SQL] add global temp view
## What changes were proposed in this pull request?

Global temporary view is a cross-session temporary view, which means it's shared among all sessions. Its lifetime is the lifetime of the Spark application, i.e. it will be automatically dropped when the application terminates. It's tied to a system preserved database `global_temp`(configurable via SparkConf), and we must use the qualified name to refer a global temp view, e.g. SELECT * FROM global_temp.view1.

changes for `SessionCatalog`:

1. add a new field `gloabalTempViews: GlobalTempViewManager`, to access the shared global temp views, and the global temp db name.
2. `createDatabase` will fail if users wanna create `global_temp`, which is system preserved.
3. `setCurrentDatabase` will fail if users wanna set `global_temp`, which is system preserved.
4. add `createGlobalTempView`, which is used in `CreateViewCommand` to create global temp views.
5. add `dropGlobalTempView`, which is used in `CatalogImpl` to drop global temp view.
6. add `alterTempViewDefinition`, which is used in `AlterViewAsCommand` to update the view definition for local/global temp views.
7. `renameTable`/`dropTable`/`isTemporaryTable`/`lookupRelation`/`getTempViewOrPermanentTableMetadata`/`refreshTable` will handle global temp views.

changes for SQL commands:

1. `CreateViewCommand`/`AlterViewAsCommand` is updated to support global temp views
2. `ShowTablesCommand` outputs a new column `database`, which is used to distinguish global and local temp views.
3. other commands can also handle global temp views if they call `SessionCatalog` APIs which accepts global temp views, e.g. `DropTableCommand`, `AlterTableRenameCommand`, `ShowColumnsCommand`, etc.

changes for other public API

1. add a new method `dropGlobalTempView` in `Catalog`
2. `Catalog.findTable` can find global temp view
3. add a new method `createGlobalTempView` in `Dataset`

## How was this patch tested?

new tests in `SQLViewSuite`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #14897 from cloud-fan/global-temp-view.
2016-10-10 15:48:57 +08:00
hyukjinkwon 2190037757
[MINOR][PYSPARK][DOCS] Fix examples in PySpark documentation
## What changes were proposed in this pull request?

This PR proposes to fix wrongly indented examples in PySpark documentation

```
-        >>> json_sdf = spark.readStream.format("json")\
-                                       .schema(sdf_schema)\
-                                       .load(tempfile.mkdtemp())
+        >>> json_sdf = spark.readStream.format("json") \\
+        ...     .schema(sdf_schema) \\
+        ...     .load(tempfile.mkdtemp())
```

```
-        people.filter(people.age > 30).join(department, people.deptId == department.id)\
+        people.filter(people.age > 30).join(department, people.deptId == department.id) \\
```

```
-        >>> examples = [LabeledPoint(1.1, Vectors.sparse(3, [(0, 1.23), (2, 4.56)])), \
-                        LabeledPoint(0.0, Vectors.dense([1.01, 2.02, 3.03]))]
+        >>> examples = [LabeledPoint(1.1, Vectors.sparse(3, [(0, 1.23), (2, 4.56)])),
+        ...             LabeledPoint(0.0, Vectors.dense([1.01, 2.02, 3.03]))]
```

```
-        >>> examples = [LabeledPoint(1.1, Vectors.sparse(3, [(0, -1.23), (2, 4.56e-7)])), \
-                        LabeledPoint(0.0, Vectors.dense([1.01, 2.02, 3.03]))]
+        >>> examples = [LabeledPoint(1.1, Vectors.sparse(3, [(0, -1.23), (2, 4.56e-7)])),
+        ...             LabeledPoint(0.0, Vectors.dense([1.01, 2.02, 3.03]))]
```

```
-        ...      for x in iterator:
-        ...           print(x)
+        ...     for x in iterator:
+        ...          print(x)
```

## How was this patch tested?

Manually tested.

**Before**

![2016-09-26 8 36 02](https://cloud.githubusercontent.com/assets/6477701/18834471/05c7a478-8431-11e6-94bb-09aa37b12ddb.png)

![2016-09-26 9 22 16](https://cloud.githubusercontent.com/assets/6477701/18834472/06c8735c-8431-11e6-8775-78631eab0411.png)

<img width="601" alt="2016-09-27 2 29 27" src="https://cloud.githubusercontent.com/assets/6477701/18861294/29c0d5b4-84bf-11e6-99c5-3c9d913c125d.png">

<img width="1056" alt="2016-09-27 2 29 58" src="https://cloud.githubusercontent.com/assets/6477701/18861298/31694cd8-84bf-11e6-9e61-9888cb8c2089.png">

<img width="1079" alt="2016-09-27 2 30 05" src="https://cloud.githubusercontent.com/assets/6477701/18861301/359722da-84bf-11e6-97f9-5f5365582d14.png">

**After**

![2016-09-26 9 29 47](https://cloud.githubusercontent.com/assets/6477701/18834467/0367f9da-8431-11e6-86d9-a490d3297339.png)

![2016-09-26 9 30 24](https://cloud.githubusercontent.com/assets/6477701/18834463/f870fae0-8430-11e6-9482-01fc47898492.png)

<img width="515" alt="2016-09-27 2 28 19" src="https://cloud.githubusercontent.com/assets/6477701/18861305/3ff88b88-84bf-11e6-902c-9f725e8a8b10.png">

<img width="652" alt="2016-09-27 3 50 59" src="https://cloud.githubusercontent.com/assets/6477701/18863053/592fbc74-84ca-11e6-8dbf-99cf57947de8.png">

<img width="709" alt="2016-09-27 3 51 03" src="https://cloud.githubusercontent.com/assets/6477701/18863060/601607be-84ca-11e6-80aa-a401df41c321.png">

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #15242 from HyukjinKwon/minor-example-pyspark.
2016-09-28 06:19:04 -04:00
Josh Rosen 6d06ff6f7e [SPARK-17514] df.take(1) and df.limit(1).collect() should perform the same in Python
## What changes were proposed in this pull request?

In PySpark, `df.take(1)` runs a single-stage job which computes only one partition of the DataFrame, while `df.limit(1).collect()` computes all partitions and runs a two-stage job. This difference in performance is confusing.

The reason why `limit(1).collect()` is so much slower is that `collect()` internally maps to `df.rdd.<some-pyspark-conversions>.toLocalIterator`, which causes Spark SQL to build a query where a global limit appears in the middle of the plan; this, in turn, ends up being executed inefficiently because limits in the middle of plans are now implemented by repartitioning to a single task rather than by running a `take()` job on the driver (this was done in #7334, a patch which was a prerequisite to allowing partition-local limits to be pushed beneath unions, etc.).

In order to fix this performance problem I think that we should generalize the fix from SPARK-10731 / #8876 so that `DataFrame.collect()` also delegates to the Scala implementation and shares the same performance properties. This patch modifies `DataFrame.collect()` to first collect all results to the driver and then pass them to Python, allowing this query to be planned using Spark's `CollectLimit` optimizations.

## How was this patch tested?

Added a regression test in `sql/tests.py` which asserts that the expected number of jobs, stages, and tasks are run for both queries.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #15068 from JoshRosen/pyspark-collect-limit.
2016-09-14 10:10:01 -07:00
Srinath Shankar e6132a6cf1 [SPARK-17298][SQL] Require explicit CROSS join for cartesian products
## What changes were proposed in this pull request?

Require the use of CROSS join syntax in SQL (and a new crossJoin
DataFrame API) to specify explicit cartesian products between relations.
By cartesian product we mean a join between relations R and S where
there is no join condition involving columns from both R and S.

If a cartesian product is detected in the absence of an explicit CROSS
join, an error must be thrown. Turning on the
"spark.sql.crossJoin.enabled" configuration flag will disable this check
and allow cartesian products without an explicit CROSS join.

The new crossJoin DataFrame API must be used to specify explicit cross
joins. The existing join(DataFrame) method will produce a INNER join
that will require a subsequent join condition.
That is df1.join(df2) is equivalent to select * from df1, df2.

## How was this patch tested?

Added cross-join.sql to the SQLQueryTestSuite to test the check for cartesian products. Added a couple of tests to the DataFrameJoinSuite to test the crossJoin API. Modified various other test suites to explicitly specify a cross join where an INNER join or a comma-separated list was previously used.

Author: Srinath Shankar <srinath@databricks.com>

Closes #14866 from srinathshankar/crossjoin.
2016-09-03 00:20:43 +02:00
Nicholas Chammas 274f3b9ec8 [SPARK-16772] Correct API doc references to PySpark classes + formatting fixes
## What's Been Changed

The PR corrects several broken or missing class references in the Python API docs. It also correct formatting problems.

For example, you can see [here](http://spark.apache.org/docs/2.0.0/api/python/pyspark.sql.html#pyspark.sql.SQLContext.registerFunction) how Sphinx is not picking up the reference to `DataType`. That's because the reference is relative to the current module, whereas `DataType` is in a different module.

You can also see [here](http://spark.apache.org/docs/2.0.0/api/python/pyspark.sql.html#pyspark.sql.SQLContext.createDataFrame) how the formatting for byte, tinyint, and so on is italic instead of monospace. That's because in ReST single backticks just make things italic, unlike in Markdown.

## Testing

I tested this PR by [building the Python docs](https://github.com/apache/spark/tree/master/docs#generating-the-documentation-html) and reviewing the results locally in my browser. I confirmed that the broken or missing class references were resolved, and that the formatting was corrected.

Author: Nicholas Chammas <nicholas.chammas@gmail.com>

Closes #14393 from nchammas/python-docstring-fixes.
2016-07-28 14:57:15 -07:00
Dongjoon Hyun 47f5b88db4 [SPARK-16651][PYSPARK][DOC] Make withColumnRenamed/drop description more consistent with Scala API
## What changes were proposed in this pull request?

`withColumnRenamed` and `drop` is a no-op if the given column name does not exists. Python documentation also describe that, but this PR adds more explicit line consistently with Scala to reduce the ambiguity.

## How was this patch tested?

It's about docs.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #14288 from dongjoon-hyun/SPARK-16651.
2016-07-22 13:20:06 +01:00
Mortada Mehyar 6ee40d2cc5 [DOC] improve python doc for rdd.histogram and dataframe.join
## What changes were proposed in this pull request?

doc change only

## How was this patch tested?

doc change only

Author: Mortada Mehyar <mortada.mehyar@gmail.com>

Closes #14253 from mortada/histogram_typos.
2016-07-18 23:49:47 -07:00
WeichenXu 1832423827 [SPARK-16546][SQL][PYSPARK] update python dataframe.drop
## What changes were proposed in this pull request?

Make `dataframe.drop` API in python support multi-columns parameters,
so that it is the same with scala API.

## How was this patch tested?

The doc test.

Author: WeichenXu <WeichenXu123@outlook.com>

Closes #14203 from WeichenXu123/drop_python_api.
2016-07-14 22:55:49 -07:00
Dongjoon Hyun 142df4834b [SPARK-16429][SQL] Include StringType columns in describe()
## What changes were proposed in this pull request?

Currently, Spark `describe` supports `StringType`. However, `describe()` returns a dataset for only all numeric columns. This PR aims to include `StringType` columns in `describe()`, `describe` without argument.

**Background**
```scala
scala> spark.read.json("examples/src/main/resources/people.json").describe("age", "name").show()
+-------+------------------+-------+
|summary|               age|   name|
+-------+------------------+-------+
|  count|                 2|      3|
|   mean|              24.5|   null|
| stddev|7.7781745930520225|   null|
|    min|                19|   Andy|
|    max|                30|Michael|
+-------+------------------+-------+
```

**Before**
```scala
scala> spark.read.json("examples/src/main/resources/people.json").describe().show()
+-------+------------------+
|summary|               age|
+-------+------------------+
|  count|                 2|
|   mean|              24.5|
| stddev|7.7781745930520225|
|    min|                19|
|    max|                30|
+-------+------------------+
```

**After**
```scala
scala> spark.read.json("examples/src/main/resources/people.json").describe().show()
+-------+------------------+-------+
|summary|               age|   name|
+-------+------------------+-------+
|  count|                 2|      3|
|   mean|              24.5|   null|
| stddev|7.7781745930520225|   null|
|    min|                19|   Andy|
|    max|                30|Michael|
+-------+------------------+-------+
```

## How was this patch tested?

Pass the Jenkins with a update testcase.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #14095 from dongjoon-hyun/SPARK-16429.
2016-07-08 14:36:50 -07:00
Dongjoon Hyun dff73bfa5e [SPARK-16052][SQL] Improve CollapseRepartition optimizer for Repartition/RepartitionBy
## What changes were proposed in this pull request?

This PR improves `CollapseRepartition` to optimize the adjacent combinations of **Repartition** and **RepartitionBy**. Also, this PR adds a testsuite for this optimizer.

**Target Scenario**
```scala
scala> val dsView1 = spark.range(8).repartition(8, $"id")
scala> dsView1.createOrReplaceTempView("dsView1")
scala> sql("select id from dsView1 distribute by id").explain(true)
```

**Before**
```scala
scala> sql("select id from dsView1 distribute by id").explain(true)
== Parsed Logical Plan ==
'RepartitionByExpression ['id]
+- 'Project ['id]
   +- 'UnresolvedRelation `dsView1`

== Analyzed Logical Plan ==
id: bigint
RepartitionByExpression [id#0L]
+- Project [id#0L]
   +- SubqueryAlias dsview1
      +- RepartitionByExpression [id#0L], 8
         +- Range (0, 8, splits=8)

== Optimized Logical Plan ==
RepartitionByExpression [id#0L]
+- RepartitionByExpression [id#0L], 8
   +- Range (0, 8, splits=8)

== Physical Plan ==
Exchange hashpartitioning(id#0L, 200)
+- Exchange hashpartitioning(id#0L, 8)
   +- *Range (0, 8, splits=8)
```

**After**
```scala
scala> sql("select id from dsView1 distribute by id").explain(true)
== Parsed Logical Plan ==
'RepartitionByExpression ['id]
+- 'Project ['id]
   +- 'UnresolvedRelation `dsView1`

== Analyzed Logical Plan ==
id: bigint
RepartitionByExpression [id#0L]
+- Project [id#0L]
   +- SubqueryAlias dsview1
      +- RepartitionByExpression [id#0L], 8
         +- Range (0, 8, splits=8)

== Optimized Logical Plan ==
RepartitionByExpression [id#0L]
+- Range (0, 8, splits=8)

== Physical Plan ==
Exchange hashpartitioning(id#0L, 200)
+- *Range (0, 8, splits=8)
```

## How was this patch tested?

Pass the Jenkins tests (including a new testsuite).

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #13765 from dongjoon-hyun/SPARK-16052.
2016-07-08 16:44:53 +08:00
hyukjinkwon 4e14199ff7 [MINOR][PYSPARK][DOC] Fix wrongly formatted examples in PySpark documentation
## What changes were proposed in this pull request?

This PR fixes wrongly formatted examples in PySpark documentation as below:

- **`SparkSession`**

  - **Before**

    ![2016-07-06 11 34 41](https://cloud.githubusercontent.com/assets/6477701/16605847/ae939526-436d-11e6-8ab8-6ad578362425.png)

  - **After**

    ![2016-07-06 11 33 56](https://cloud.githubusercontent.com/assets/6477701/16605845/ace9ee78-436d-11e6-8923-b76d4fc3e7c3.png)

- **`Builder`**

  - **Before**
    ![2016-07-06 11 34 44](https://cloud.githubusercontent.com/assets/6477701/16605844/aba60dbc-436d-11e6-990a-c87bc0281c6b.png)

  - **After**
    ![2016-07-06 1 26 37](https://cloud.githubusercontent.com/assets/6477701/16607562/586704c0-437d-11e6-9483-e0af93d8f74e.png)

This PR also fixes several similar instances across the documentation in `sql` PySpark module.

## How was this patch tested?

N/A

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #14063 from HyukjinKwon/minor-pyspark-builder.
2016-07-06 10:45:51 -07:00
Tathagata Das f454a7f9f0 [SPARK-16266][SQL][STREAING] Moved DataStreamReader/Writer from pyspark.sql to pyspark.sql.streaming
## What changes were proposed in this pull request?

- Moved DataStreamReader/Writer from pyspark.sql to pyspark.sql.streaming to make them consistent with scala packaging
- Exposed the necessary classes in sql.streaming package so that they appear in the docs
- Added pyspark.sql.streaming module to the docs

## How was this patch tested?
- updated unit tests.
- generated docs for testing visibility of pyspark.sql.streaming classes.

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

Closes #13955 from tdas/SPARK-16266.
2016-06-28 22:07:11 -07:00
Burak Yavuz 5545b79109 [MINOR][DOCS][STRUCTURED STREAMING] Minor doc fixes around DataFrameWriter and DataStreamWriter
## What changes were proposed in this pull request?

Fixes a couple old references to `DataFrameWriter.startStream` to `DataStreamWriter.start

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #13952 from brkyvz/minor-doc-fix.
2016-06-28 17:02:16 -07:00
Prashant Sharma f6b497fcdd [SPARK-16128][SQL] Allow setting length of characters to be truncated to, in Dataset.show function.
## What changes were proposed in this pull request?

Allowing truncate to a specific number of character is convenient at times, especially while operating from the REPL. Sometimes those last few characters make all the difference, and showing everything brings in whole lot of noise.

## How was this patch tested?
Existing tests. + 1 new test in DataFrameSuite.

For SparkR and pyspark, existing tests and manual testing.

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

Closes #13839 from ScrapCodes/add_truncateTo_DF.show.
2016-06-28 17:11:06 +05:30
Tathagata Das 9a5071996b [SPARK-15953][WIP][STREAMING] Renamed ContinuousQuery to StreamingQuery
Renamed for simplicity, so that its obvious that its related to streaming.

Existing unit tests.

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

Closes #13673 from tdas/SPARK-15953.
2016-06-15 10:46:07 -07:00
Tathagata Das 214adb14b8 [SPARK-15933][SQL][STREAMING] Refactored DF reader-writer to use readStream and writeStream for streaming DFs
## What changes were proposed in this pull request?
Currently, the DataFrameReader/Writer has method that are needed for streaming and non-streaming DFs. This is quite awkward because each method in them through runtime exception for one case or the other. So rather having half the methods throw runtime exceptions, its just better to have a different reader/writer API for streams.

- [x] Python API!!

## How was this patch tested?
Existing unit tests + two sets of unit tests for DataFrameReader/Writer and DataStreamReader/Writer.

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

Closes #13653 from tdas/SPARK-15933.
2016-06-14 17:58:45 -07:00