Commit graph

501 commits

Author SHA1 Message Date
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
hyukjinkwon d228cd0b02 [SPARK-20442][PYTHON][DOCS] Fill up documentations for functions in Column API in PySpark
## What changes were proposed in this pull request?

This PR proposes to fill up the documentation with examples for `bitwiseOR`, `bitwiseAND`, `bitwiseXOR`. `contains`, `asc` and `desc` in `Column` API.

Also, this PR fixes minor typos in the documentation and matches some of the contents between Scala doc and Python doc.

Lastly, this PR suggests to use `spark` rather than `sc` in doc tests in `Column` for Python documentation.

## How was this patch tested?

Doc tests were added and manually tested with the commands below:

`./python/run-tests.py --module pyspark-sql`
`./python/run-tests.py --module pyspark-sql --python-executable python3`
`./dev/lint-python`

Output was checked via `make html` under `./python/docs`. The snapshots will be left on the codes with comments.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17737 from HyukjinKwon/SPARK-20442.
2017-04-29 13:46:40 -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
Michael Patterson 8765bc17d0 [SPARK-20132][DOCS] Add documentation for column string functions
## What changes were proposed in this pull request?
Add docstrings to column.py for the Column functions `rlike`, `like`, `startswith`, and `endswith`. Pass these docstrings through `_bin_op`

There may be a better place to put the docstrings. I put them immediately above the Column class.

## How was this patch tested?

I ran `make html` on my local computer to remake the documentation, and verified that the html pages were displaying the docstrings correctly. I tried running `dev-tests`, and the formatting tests passed. However, my mvn build didn't work I think due to issues on my computer.

These docstrings are my original work and free license.

davies has done the most recent work reorganizing `_bin_op`

Author: Michael Patterson <map222@gmail.com>

Closes #17469 from map222/patterson-documentation.
2017-04-22 19:58:54 -07:00
Kyle Kelley f654b39a63 [SPARK-20360][PYTHON] reprs for interpreters
## What changes were proposed in this pull request?

Establishes a very minimal `_repr_html_` for PySpark's `SparkContext`.

## How was this patch tested?

nteract:

![screen shot 2017-04-17 at 3 41 29 pm](https://cloud.githubusercontent.com/assets/836375/25107701/d57090ba-2385-11e7-8147-74bc2c50a41b.png)

Jupyter:

![screen shot 2017-04-17 at 3 53 19 pm](https://cloud.githubusercontent.com/assets/836375/25107725/05bf1fe8-2386-11e7-93e1-07a20c917dde.png)

Hydrogen:

![screen shot 2017-04-17 at 3 49 55 pm](https://cloud.githubusercontent.com/assets/836375/25107664/a75e1ddc-2385-11e7-8477-258661833007.png)

Author: Kyle Kelley <rgbkrk@gmail.com>

Closes #17662 from rgbkrk/repr.
2017-04-18 12:35:27 -07:00
hyukjinkwon bca4259f12 [MINOR][DOCS] JSON APIs related documentation fixes
## What changes were proposed in this pull request?

This PR proposes corrections related to JSON APIs as below:

- Rendering links in Python documentation
- Replacing `RDD` to `Dataset` in programing guide
- Adding missing description about JSON Lines consistently in `DataFrameReader.json` in Python API
- De-duplicating little bit of `DataFrameReader.json` in Scala/Java API

## How was this patch tested?

Manually build the documentation via `jekyll build`. Corresponding snapstops will be left on the codes.

Note that currently there are Javadoc8 breaks in several places. These are proposed to be handled in https://github.com/apache/spark/pull/17477. So, this PR does not fix those.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17602 from HyukjinKwon/minor-json-documentation.
2017-04-12 09:16:39 +01:00
Felix Cheung bccc330193 [SPARK-20196][PYTHON][SQL] update doc for catalog functions for all languages, add pyspark refreshByPath API
## What changes were proposed in this pull request?

Update doc to remove external for createTable, add refreshByPath in python

## How was this patch tested?

manual

Author: Felix Cheung <felixcheung_m@hotmail.com>

Closes #17512 from felixcheung/catalogdoc.
2017-04-06 09:09:43 -07:00
zero323 e2773996b8 [SPARK-19454][PYTHON][SQL] DataFrame.replace improvements
## What changes were proposed in this pull request?

- Allows skipping `value` argument if `to_replace` is a `dict`:
	```python
	df = sc.parallelize([("Alice", 1, 3.0)]).toDF()
	df.replace({"Alice": "Bob"}).show()
	````
- Adds validation step to ensure homogeneous values / replacements.
- Simplifies internal control flow.
- Improves unit tests coverage.

## How was this patch tested?

Existing unit tests, additional unit tests, manual testing.

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

Closes #16793 from zero323/SPARK-19454.
2017-04-05 11:47:40 -07:00
hyukjinkwon cff11fd20e [SPARK-20166][SQL] Use XXX for ISO 8601 timezone instead of ZZ (FastDateFormat specific) in CSV/JSON timeformat options
## What changes were proposed in this pull request?

This PR proposes to use `XXX` format instead of `ZZ`. `ZZ` seems a `FastDateFormat` specific.

`ZZ` supports "ISO 8601 extended format time zones" but it seems `FastDateFormat` specific option.
I misunderstood this is compatible format with `SimpleDateFormat` when this change is introduced.
Please see [SimpleDateFormat documentation]( https://docs.oracle.com/javase/7/docs/api/java/text/SimpleDateFormat.html#iso8601timezone) and [FastDateFormat documentation](https://commons.apache.org/proper/commons-lang/apidocs/org/apache/commons/lang3/time/FastDateFormat.html).

It seems we better replace `ZZ` to `XXX` because they look using the same strategy - [FastDateParser.java#L930](8767cd4f1a/src/main/java/org/apache/commons/lang3/time/FastDateParser.java (L930)), [FastDateParser.java#L932-L951 ](8767cd4f1a/src/main/java/org/apache/commons/lang3/time/FastDateParser.java (L932-L951)) and [FastDateParser.java#L596-L601](8767cd4f1a/src/main/java/org/apache/commons/lang3/time/FastDateParser.java (L596-L601)).

I also checked the codes and manually debugged it for sure. It seems both cases use the same pattern `( Z|(?:[+-]\\d{2}(?::)\\d{2}))`.

_Note that this should be rather a fix about documentation and not the behaviour change because `ZZ` seems invalid date format in `SimpleDateFormat` as documented in `DataFrameReader` and etc, and both `ZZ` and `XXX` look identically working with `FastDateFormat`_

Current documentation is as below:

```
   * <li>`timestampFormat` (default `yyyy-MM-dd'T'HH:mm:ss.SSSZZ`): sets the string that
   * indicates a timestamp format. Custom date formats follow the formats at
   * `java.text.SimpleDateFormat`. This applies to timestamp type.</li>
```

## How was this patch tested?

Existing tests should cover this. Also, manually tested as below (BTW, I don't think these are worth being added as tests within Spark):

**Parse**

```scala
scala> new java.text.SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSXXX").parse("2017-03-21T00:00:00.000-11:00")
res4: java.util.Date = Tue Mar 21 20:00:00 KST 2017

scala>  new java.text.SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSXXX").parse("2017-03-21T00:00:00.000Z")
res10: java.util.Date = Tue Mar 21 09:00:00 KST 2017

scala> new java.text.SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSZZ").parse("2017-03-21T00:00:00.000-11:00")
java.text.ParseException: Unparseable date: "2017-03-21T00:00:00.000-11:00"
  at java.text.DateFormat.parse(DateFormat.java:366)
  ... 48 elided
scala>  new java.text.SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSZZ").parse("2017-03-21T00:00:00.000Z")
java.text.ParseException: Unparseable date: "2017-03-21T00:00:00.000Z"
  at java.text.DateFormat.parse(DateFormat.java:366)
  ... 48 elided
```

```scala
scala> org.apache.commons.lang3.time.FastDateFormat.getInstance("yyyy-MM-dd'T'HH:mm:ss.SSSXXX").parse("2017-03-21T00:00:00.000-11:00")
res7: java.util.Date = Tue Mar 21 20:00:00 KST 2017

scala> org.apache.commons.lang3.time.FastDateFormat.getInstance("yyyy-MM-dd'T'HH:mm:ss.SSSXXX").parse("2017-03-21T00:00:00.000Z")
res1: java.util.Date = Tue Mar 21 09:00:00 KST 2017

scala> org.apache.commons.lang3.time.FastDateFormat.getInstance("yyyy-MM-dd'T'HH:mm:ss.SSSZZ").parse("2017-03-21T00:00:00.000-11:00")
res8: java.util.Date = Tue Mar 21 20:00:00 KST 2017

scala> org.apache.commons.lang3.time.FastDateFormat.getInstance("yyyy-MM-dd'T'HH:mm:ss.SSSZZ").parse("2017-03-21T00:00:00.000Z")
res2: java.util.Date = Tue Mar 21 09:00:00 KST 2017
```

**Format**

```scala
scala> new java.text.SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSXXX").format(new java.text.SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSXXX").parse("2017-03-21T00:00:00.000-11:00"))
res6: String = 2017-03-21T20:00:00.000+09:00
```

```scala
scala> val fd = org.apache.commons.lang3.time.FastDateFormat.getInstance("yyyy-MM-dd'T'HH:mm:ss.SSSZZ")
fd: org.apache.commons.lang3.time.FastDateFormat = FastDateFormat[yyyy-MM-dd'T'HH:mm:ss.SSSZZ,ko_KR,Asia/Seoul]

scala> fd.format(fd.parse("2017-03-21T00:00:00.000-11:00"))
res1: String = 2017-03-21T20:00:00.000+09:00

scala> val fd = org.apache.commons.lang3.time.FastDateFormat.getInstance("yyyy-MM-dd'T'HH:mm:ss.SSSXXX")
fd: org.apache.commons.lang3.time.FastDateFormat = FastDateFormat[yyyy-MM-dd'T'HH:mm:ss.SSSXXX,ko_KR,Asia/Seoul]

scala> fd.format(fd.parse("2017-03-21T00:00:00.000-11:00"))
res2: String = 2017-03-21T20:00:00.000+09:00
```

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17489 from HyukjinKwon/SPARK-20166.
2017-04-03 10:07:41 +01:00
hyukjinkwon 3fbf0a5f92 [MINOR][DOCS] Match several documentation changes in Scala to R/Python
## What changes were proposed in this pull request?

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

## How was this patch tested?

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

Author: hyukjinkwon <gurwls223@gmail.com>

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

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

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

## How was this patch tested?

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

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

marmbrus tdas zsxwing

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

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

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

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

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

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

WIth the data below:

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

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

**Before**

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

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

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

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

**After**

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

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

Note that this case is possible in R

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

## How was this patch tested?

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

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17310 from HyukjinKwon/SPARK-18579.
2017-03-23 00:25:01 -07:00
hyukjinkwon 465818389a [SPARK-19949][SQL][FOLLOW-UP] Clean up parse modes and update related comments
## What changes were proposed in this pull request?

This PR proposes to make `mode` options in both CSV and JSON to use `cass object` and fix some related comments related previous fix.

Also, this PR modifies some tests related parse modes.

## How was this patch tested?

Modified unit tests in both `CSVSuite.scala` and `JsonSuite.scala`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17377 from HyukjinKwon/SPARK-19949.
2017-03-22 09:52:37 -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
hyukjinkwon 0cdcf91145 [SPARK-19849][SQL] Support ArrayType in to_json to produce JSON array
## What changes were proposed in this pull request?

This PR proposes to support an array of struct type in `to_json` as below:

```scala
import org.apache.spark.sql.functions._

val df = Seq(Tuple1(Tuple1(1) :: Nil)).toDF("a")
df.select(to_json($"a").as("json")).show()
```

```
+----------+
|      json|
+----------+
|[{"_1":1}]|
+----------+
```

Currently, it throws an exception as below (a newline manually inserted for readability):

```
org.apache.spark.sql.AnalysisException: cannot resolve 'structtojson(`array`)' due to data type
mismatch: structtojson requires that the expression is a struct expression.;;
```

This allows the roundtrip with `from_json` as below:

```scala
import org.apache.spark.sql.functions._
import org.apache.spark.sql.types._

val schema = ArrayType(StructType(StructField("a", IntegerType) :: Nil))
val df = Seq("""[{"a":1}, {"a":2}]""").toDF("json").select(from_json($"json", schema).as("array"))
df.show()

// Read back.
df.select(to_json($"array").as("json")).show()
```

```
+----------+
|     array|
+----------+
|[[1], [2]]|
+----------+

+-----------------+
|             json|
+-----------------+
|[{"a":1},{"a":2}]|
+-----------------+
```

Also, this PR proposes to rename from `StructToJson` to `StructsToJson ` and `JsonToStruct` to `JsonToStructs`.

## How was this patch tested?

Unit tests in `JsonFunctionsSuite` and `JsonExpressionsSuite` for Scala, doctest for Python and test in `test_sparkSQL.R` for R.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17192 from HyukjinKwon/SPARK-19849.
2017-03-19 22:33:01 -07:00
Liwei Lin e1ac553402 [SPARK-19817][SS] Make it clear that timeZone is a general option in DataStreamReader/Writer
## What changes were proposed in this pull request?

As timezone setting can also affect partition values, it works for all formats, we should make it clear.

## How was this patch tested?

N/A

Author: Liwei Lin <lwlin7@gmail.com>

Closes #17299 from lw-lin/timezone.
2017-03-14 22:30:16 -07:00
Takuya UESHIN 7ded39c223 [SPARK-19817][SQL] Make it clear that timeZone option is a general option in DataFrameReader/Writer.
## What changes were proposed in this pull request?

As timezone setting can also affect partition values, it works for all formats, we should make it clear.

## How was this patch tested?

Existing tests.

Author: Takuya UESHIN <ueshin@databricks.com>

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

Author: Jeff Zhang <zjffdu@apache.org>

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

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

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

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

## How was this patch tested?

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

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

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

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

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

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

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

## How was this patch tested?

Added a new test that fails without the change.

dongjoon-hyun davies Mind taking a look?

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

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

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

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

**1.5.2**

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

**1.6.3**

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

**2.1.0**

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

**Current Master**

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

**After**

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

In more details,

It seems the implementation intended to support this

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

However, currently, it throws an exception as below:

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

What happens here is as below:

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

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

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

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

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

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

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

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

## How was this patch tested?

Added unit tests in `tests.py`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17160 from HyukjinKwon/SPARK-19701.
2017-03-05 18:04:52 -08:00
hyukjinkwon 369a148e59 [SPARK-19595][SQL] Support json array in from_json
## What changes were proposed in this pull request?

This PR proposes to both,

**Do not allow json arrays with multiple elements and return null in `from_json` with `StructType` as the schema.**

Currently, it only reads the single row when the input is a json array. So, the codes below:

```scala
import org.apache.spark.sql.functions._
import org.apache.spark.sql.types._
val schema = StructType(StructField("a", IntegerType) :: Nil)
Seq(("""[{"a": 1}, {"a": 2}]""")).toDF("struct").select(from_json(col("struct"), schema)).show()
```
prints

```
+--------------------+
|jsontostruct(struct)|
+--------------------+
|                 [1]|
+--------------------+
```

This PR simply suggests to print this as `null` if the schema is `StructType` and input is json array.with multiple elements

```
+--------------------+
|jsontostruct(struct)|
+--------------------+
|                null|
+--------------------+
```

**Support json arrays in `from_json` with `ArrayType` as the schema.**

```scala
import org.apache.spark.sql.functions._
import org.apache.spark.sql.types._
val schema = ArrayType(StructType(StructField("a", IntegerType) :: Nil))
Seq(("""[{"a": 1}, {"a": 2}]""")).toDF("array").select(from_json(col("array"), schema)).show()
```

prints

```
+-------------------+
|jsontostruct(array)|
+-------------------+
|         [[1], [2]]|
+-------------------+
```

## How was this patch tested?

Unit test in `JsonExpressionsSuite`, `JsonFunctionsSuite`, Python doctests and manual test.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16929 from HyukjinKwon/disallow-array.
2017-03-05 14:35:06 -08:00
Felix Cheung 8d6ef895ee [SPARK-18352][DOCS] wholeFile JSON update doc and programming guide
## What changes were proposed in this pull request?

Update doc for R, programming guide. Clarify default behavior for all languages.

## How was this patch tested?

manually

Author: Felix Cheung <felixcheung_m@hotmail.com>

Closes #17128 from felixcheung/jsonwholefiledoc.
2017-03-02 01:02:38 -08:00
hyukjinkwon 7e5359be5c [SPARK-19610][SQL] Support parsing multiline CSV files
## What changes were proposed in this pull request?

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

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

## How was this patch tested?

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

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

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

Author: hyukjinkwon <gurwls223@gmail.com>

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

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

### Backward incompatible changes:

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

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

### Limitations:

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

### Possible impact on other tickets:

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

## How was this patch tested?

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

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

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

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

## How was this patch tested?

updated unit test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #17036 from cloud-fan/pyspark.
2017-02-23 13:22:39 -08:00
Takeshi Yamamuro 09ed6e7711 [SPARK-18699][SQL] Put malformed tokens into a new field when parsing CSV data
## What changes were proposed in this pull request?
This pr added a logic to put malformed tokens into a new field when parsing CSV data  in case of permissive modes. In the current master, if the CSV parser hits these malformed ones, it throws an exception below (and then a job fails);
```
Caused by: java.lang.IllegalArgumentException
	at java.sql.Date.valueOf(Date.java:143)
	at org.apache.spark.sql.catalyst.util.DateTimeUtils$.stringToTime(DateTimeUtils.scala:137)
	at org.apache.spark.sql.execution.datasources.csv.CSVTypeCast$$anonfun$castTo$6.apply$mcJ$sp(CSVInferSchema.scala:272)
	at org.apache.spark.sql.execution.datasources.csv.CSVTypeCast$$anonfun$castTo$6.apply(CSVInferSchema.scala:272)
	at org.apache.spark.sql.execution.datasources.csv.CSVTypeCast$$anonfun$castTo$6.apply(CSVInferSchema.scala:272)
	at scala.util.Try.getOrElse(Try.scala:79)
	at org.apache.spark.sql.execution.datasources.csv.CSVTypeCast$.castTo(CSVInferSchema.scala:269)
	at
```
In case that users load large CSV-formatted data, the job failure makes users get some confused. So, this fix set NULL for original columns and put malformed tokens in a new field.

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

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #16928 from maropu/SPARK-18699-2.
2017-02-23 12:09:36 -08: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
Nathan Howell 21fde57f15 [SPARK-18352][SQL] Support parsing multiline json files
## What changes were proposed in this pull request?

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

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

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

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

## How was this patch tested?

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

Author: Nathan Howell <nhowell@godaddy.com>

Closes #16386 from NathanHowell/SPARK-18352.
2017-02-16 20:51:19 -08:00
Takuya UESHIN 865b2fd84c [SPARK-18937][SQL] Timezone support in CSV/JSON parsing
## What changes were proposed in this pull request?

This is a follow-up pr of #16308.

This pr enables timezone support in CSV/JSON parsing.

We should introduce `timeZone` option for CSV/JSON datasources (the default value of the option is session local timezone).

The datasources should use the `timeZone` option to format/parse to write/read timestamp values.
Notice that while reading, if the timestampFormat has the timezone info, the timezone will not be used because we should respect the timezone in the values.

For example, if you have timestamp `"2016-01-01 00:00:00"` in `GMT`, the values written with the default timezone option, which is `"GMT"` because session local timezone is `"GMT"` here, are:

```scala
scala> spark.conf.set("spark.sql.session.timeZone", "GMT")

scala> val df = Seq(new java.sql.Timestamp(1451606400000L)).toDF("ts")
df: org.apache.spark.sql.DataFrame = [ts: timestamp]

scala> df.show()
+-------------------+
|ts                 |
+-------------------+
|2016-01-01 00:00:00|
+-------------------+

scala> df.write.json("/path/to/gmtjson")
```

```sh
$ cat /path/to/gmtjson/part-*
{"ts":"2016-01-01T00:00:00.000Z"}
```

whereas setting the option to `"PST"`, they are:

```scala
scala> df.write.option("timeZone", "PST").json("/path/to/pstjson")
```

```sh
$ cat /path/to/pstjson/part-*
{"ts":"2015-12-31T16:00:00.000-08:00"}
```

We can properly read these files even if the timezone option is wrong because the timestamp values have timezone info:

```scala
scala> val schema = new StructType().add("ts", TimestampType)
schema: org.apache.spark.sql.types.StructType = StructType(StructField(ts,TimestampType,true))

scala> spark.read.schema(schema).json("/path/to/gmtjson").show()
+-------------------+
|ts                 |
+-------------------+
|2016-01-01 00:00:00|
+-------------------+

scala> spark.read.schema(schema).option("timeZone", "PST").json("/path/to/gmtjson").show()
+-------------------+
|ts                 |
+-------------------+
|2016-01-01 00:00:00|
+-------------------+
```

And even if `timezoneFormat` doesn't contain timezone info, we can properly read the values with setting correct timezone option:

```scala
scala> df.write.option("timestampFormat", "yyyy-MM-dd'T'HH:mm:ss").option("timeZone", "JST").json("/path/to/jstjson")
```

```sh
$ cat /path/to/jstjson/part-*
{"ts":"2016-01-01T09:00:00"}
```

```scala
// wrong result
scala> spark.read.schema(schema).option("timestampFormat", "yyyy-MM-dd'T'HH:mm:ss").json("/path/to/jstjson").show()
+-------------------+
|ts                 |
+-------------------+
|2016-01-01 09:00:00|
+-------------------+

// correct result
scala> spark.read.schema(schema).option("timestampFormat", "yyyy-MM-dd'T'HH:mm:ss").option("timeZone", "JST").json("/path/to/jstjson").show()
+-------------------+
|ts                 |
+-------------------+
|2016-01-01 00:00:00|
+-------------------+
```

This pr also makes `JsonToStruct` and `StructToJson` `TimeZoneAwareExpression` to be able to evaluate values with timezone option.

## How was this patch tested?

Existing tests and added some tests.

Author: Takuya UESHIN <ueshin@happy-camper.st>

Closes #16750 from ueshin/issues/SPARK-18937.
2017-02-15 13:26:34 -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 c97f4e17de [SPARK-19160][PYTHON][SQL] Add udf decorator
## What changes were proposed in this pull request?

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

This allows users to define UDF using simplified syntax:

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

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

without need to define a separate function and udf.

## How was this patch tested?

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

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

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

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

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

Discussed ahead of time on JIRA with marmbrus

## How was this patch tested?

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

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

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

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

## How was this patch tested?

Unit tests.

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

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

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

## How was this patch tested?

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

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

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

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

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

## IntegerType
```

## How was this patch tested?

Existing unit tests, additional unit tests covering new feature.

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

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

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

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

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

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

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

## How was this patch tested?

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

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

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

Closes #16138 from anabranch/SPARK-16609.
2017-02-07 15:50:30 +01:00
Zheng RuiFeng b0985764f0 [SPARK-14352][SQL] approxQuantile should support multi columns
## What changes were proposed in this pull request?

1, add the multi-cols support based on current private api
2, add the multi-cols support to pyspark
## How was this patch tested?

unit tests

Author: Zheng RuiFeng <ruifengz@foxmail.com>
Author: Ruifeng Zheng <ruifengz@foxmail.com>

Closes #12135 from zhengruifeng/quantile4multicols.
2017-02-01 14:11:28 -08:00
zero323 9063835803 [SPARK-19163][PYTHON][SQL] Delay _judf initialization to the __call__
## What changes were proposed in this pull request?

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

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

## How was this patch tested?

Unit tests.

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

Closes #16536 from zero323/SPARK-19163.
2017-01-31 18:03:39 -08:00
zero323 06fbc35549 [SPARK-19403][PYTHON][SQL] Correct pyspark.sql.column.__all__ list.
## What changes were proposed in this pull request?

This removes from the `__all__` list class names that are not defined (visible) in the `pyspark.sql.column`.

## How was this patch tested?

Existing unit tests.

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

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

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

Author: gatorsmile <gatorsmile@gmail.com>

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

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

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

## How was this patch tested?

Add a regression test.

Author: Davies Liu <davies@databricks.com>

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

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

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

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

    def filename(path):
        return path

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

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

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

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

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

## How was this patch tested?

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

Added pyspark test.

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

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

Closes #16585 from viirya/fix-inputfileblock-hadooprdd.
2017-01-18 23:06:44 +08:00
DjvuLee 843ec8ec42 [SPARK-19239][PYSPARK] Check parameters whether equals None when specify the column in jdbc API
## What changes were proposed in this pull request?

The `jdbc` API do not check the `lowerBound` and `upperBound` when we
specified the ``column``, and just throw the following exception:

>```int() argument must be a string or a number, not 'NoneType'```

If we check the parameter, we can give a more friendly suggestion.

## How was this patch tested?
Test using the pyspark shell, without the lowerBound and upperBound parameters.

Author: DjvuLee <lihu@bytedance.com>

Closes #16599 from djvulee/pysparkFix.
2017-01-17 10:37:29 -08:00
Wenchen Fan 18ee55dd5d [SPARK-19148][SQL] do not expose the external table concept in Catalog
## What changes were proposed in this pull request?

In https://github.com/apache/spark/pull/16296 , we reached a consensus that we should hide the external/managed table concept to users and only expose custom table path.

This PR renames `Catalog.createExternalTable` to `createTable`(still keep the old versions for backward compatibility), and only set the table type to EXTERNAL if `path` is specified in options.

## How was this patch tested?

new tests in `CatalogSuite`

Author: Wenchen Fan <wenchen@databricks.com>

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

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

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

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

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

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

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

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

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

## How was this patch tested?

New test added in PySpark.

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

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

Closes #16454 from viirya/fix-pyspark-sparksession.
2017-01-12 20:53:31 +08:00
zero323 5db35b312e [SPARK-19164][PYTHON][SQL] Remove unused UserDefinedFunction._broadcast
## What changes were proposed in this pull request?

Removes `UserDefinedFunction._broadcast` and `UserDefinedFunction.__del__` method.

## How was this patch tested?

Existing unit tests.

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

Closes #16538 from zero323/SPARK-19164.
2017-01-12 01:05:02 -08:00