Commit graph

2987 commits

Author SHA1 Message Date
Liwei Lin 27f543b15f [SPARK-20441][SPARK-20432][SS] Within the same streaming query, one StreamingRelation should only be transformed to one StreamingExecutionRelation
## What changes were proposed in this pull request?

Within the same streaming query, when one `StreamingRelation` is referred multiple times – e.g. `df.union(df)` – we should transform it only to one `StreamingExecutionRelation`, instead of two or more different `StreamingExecutionRelation`s (each of which would have a separate set of source, source logs, ...).

## How was this patch tested?

Added two test cases, each of which would fail without this patch.

Author: Liwei Lin <lwlin7@gmail.com>

Closes #17735 from lw-lin/SPARK-20441.
2017-05-03 08:55:02 -07:00
Sean Owen 16fab6b0ef [SPARK-20523][BUILD] Clean up build warnings for 2.2.0 release
## What changes were proposed in this pull request?

Fix build warnings primarily related to Breeze 0.13 operator changes, Java style problems

## How was this patch tested?

Existing tests

Author: Sean Owen <sowen@cloudera.com>

Closes #17803 from srowen/SPARK-20523.
2017-05-03 10:18:35 +01:00
Michael Armbrust 6235132a8c [SPARK-20567] Lazily bind in GenerateExec
It is not valid to eagerly bind with the child's output as this causes failures when we attempt to canonicalize the plan (replacing the attribute references with dummies).

Author: Michael Armbrust <michael@databricks.com>

Closes #17838 from marmbrus/fixBindExplode.
2017-05-02 22:44:27 -07:00
Kazuaki Ishizaki afb21bf22a [SPARK-20537][CORE] Fixing OffHeapColumnVector reallocation
## What changes were proposed in this pull request?

As #17773 revealed `OnHeapColumnVector` may copy a part of the original storage.

`OffHeapColumnVector` reallocation also copies to the new storage data up to 'elementsAppended'. This variable is only updated when using the `ColumnVector.appendX` API, while `ColumnVector.putX` is more commonly used.
This PR copies the new storage data up to the previously-allocated size in`OffHeapColumnVector`.

## How was this patch tested?

Existing test suites

Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com>

Closes #17811 from kiszk/SPARK-20537.
2017-05-02 13:56:41 +08:00
Sean Owen af726cd611 [SPARK-20459][SQL] JdbcUtils throws IllegalStateException: Cause already initialized after getting SQLException
## What changes were proposed in this pull request?

Avoid failing to initCause on JDBC exception with cause initialized to null

## How was this patch tested?

Existing tests

Author: Sean Owen <sowen@cloudera.com>

Closes #17800 from srowen/SPARK-20459.
2017-05-01 17:01:05 -07:00
Kunal Khamar 6fc6cf88d8 [SPARK-20464][SS] Add a job group and description for streaming queries and fix cancellation of running jobs using the job group
## What changes were proposed in this pull request?

Job group: adding a job group is required to properly cancel running jobs related to a query.
Description: the new description makes it easier to group the batches of a query by sorting by name in the Spark Jobs UI.

## How was this patch tested?

- Unit tests
- UI screenshot

  - Order by job id:
![screen shot 2017-04-27 at 5 10 09 pm](https://cloud.githubusercontent.com/assets/7865120/25509468/15452274-2b6e-11e7-87ba-d929816688cf.png)

  - Order by description:
![screen shot 2017-04-27 at 5 10 22 pm](https://cloud.githubusercontent.com/assets/7865120/25509474/1c298512-2b6e-11e7-99b8-fef1ef7665c1.png)

  - Order by job id (no query name):
![screen shot 2017-04-27 at 5 21 33 pm](https://cloud.githubusercontent.com/assets/7865120/25509482/28c96dc8-2b6e-11e7-8df0-9d3cdbb05e36.png)

  - Order by description (no query name):
![screen shot 2017-04-27 at 5 21 44 pm](https://cloud.githubusercontent.com/assets/7865120/25509489/37674742-2b6e-11e7-9357-b5c38ec16ac4.png)

Author: Kunal Khamar <kkhamar@outlook.com>

Closes #17765 from kunalkhamar/sc-6696.
2017-05-01 11:37:30 -07:00
Herman van Hovell 6b44c4d63a [SPARK-20534][SQL] Make outer generate exec return empty rows
## What changes were proposed in this pull request?
Generate exec does not produce `null` values if the generator for the input row is empty and the generate operates in outer mode without join. This is caused by the fact that the `join=false` code path is different from the `join=true` code path, and that the `join=false` code path did deal with outer properly. This PR addresses this issue.

## How was this patch tested?
Updated `outer*` tests in `GeneratorFunctionSuite`.

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #17810 from hvanhovell/SPARK-20534.
2017-05-01 09:46:35 -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
hyukjinkwon 70f1bcd7bc [SPARK-20493][R] De-duplicate parse logics for DDL-like type strings in R
## What changes were proposed in this pull request?

It seems we are using `SQLUtils.getSQLDataType` for type string in structField. It looks we can replace this with `CatalystSqlParser.parseDataType`.

They look similar DDL-like type definitions as below:

```scala
scala> Seq(Tuple1(Tuple1("a"))).toDF.show()
```
```
+---+
| _1|
+---+
|[a]|
+---+
```

```scala
scala> Seq(Tuple1(Tuple1("a"))).toDF.select($"_1".cast("struct<_1:string>")).show()
```
```
+---+
| _1|
+---+
|[a]|
+---+
```

Such type strings looks identical when R’s one as below:

```R
> write.df(sql("SELECT named_struct('_1', 'a') as struct"), "/tmp/aa", "parquet")
> collect(read.df("/tmp/aa", "parquet", structType(structField("struct", "struct<_1:string>"))))
  struct
1      a
```

R’s one is stricter because we are checking the types via regular expressions in R side ahead.

Actual logics there look a bit different but as we check it ahead in R side, it looks replacing it would not introduce (I think) no behaviour changes. To make this sure, the tests dedicated for it were added in SPARK-20105. (It looks `structField` is the only place that calls this method).

## How was this patch tested?

Existing tests - https://github.com/apache/spark/blob/master/R/pkg/inst/tests/testthat/test_sparkSQL.R#L143-L194 should cover this.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17785 from HyukjinKwon/SPARK-20493.
2017-04-29 11:02:17 -07:00
Wenchen Fan b90bf520fd [SPARK-12837][CORE] Do not send the name of internal accumulator to executor side
## What changes were proposed in this pull request?

When sending accumulator updates back to driver, the network overhead is pretty big as there are a lot of accumulators, e.g. `TaskMetrics` will send about 20 accumulators everytime, there may be a lot of `SQLMetric` if the query plan is complicated.

Therefore, it's critical to reduce the size of serialized accumulator. A simple way is to not send the name of internal accumulators to executor side, as it's unnecessary. When executor sends accumulator updates back to driver, we can look up the accumulator name in `AccumulatorContext` easily. Note that, we still need to send names of normal accumulators, as the user code run at executor side may rely on accumulator names.

In the future, we should reimplement `TaskMetrics` to not rely on accumulators and use custom serialization.

Tried on the example in https://issues.apache.org/jira/browse/SPARK-12837, the size of serialized accumulator has been cut down by about 40%.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #17596 from cloud-fan/oom.
2017-04-27 19:38:14 -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
Weiqing Yang 2ba1eba371 [SPARK-12868][SQL] Allow adding jars from hdfs
## What changes were proposed in this pull request?
Spark 2.2 is going to be cut, it'll be great if SPARK-12868 can be resolved before that. There have been several PRs for this like [PR#16324](https://github.com/apache/spark/pull/16324) , but all of them are inactivity for a long time or have been closed.

This PR added a SparkUrlStreamHandlerFactory, which relies on 'protocol' to choose the appropriate
UrlStreamHandlerFactory like FsUrlStreamHandlerFactory to create URLStreamHandler.

## How was this patch tested?
1. Add a new unit test.
2. Check manually.
Before: throw an exception with " failed unknown protocol: hdfs"
<img width="914" alt="screen shot 2017-03-17 at 9 07 36 pm" src="https://cloud.githubusercontent.com/assets/8546874/24075277/5abe0a7c-0bd5-11e7-900e-ec3d3105da0b.png">

After:
<img width="1148" alt="screen shot 2017-03-18 at 11 42 18 am" src="https://cloud.githubusercontent.com/assets/8546874/24075283/69382a60-0bd5-11e7-8d30-d9405c3aaaba.png">

Author: Weiqing Yang <yangweiqing001@gmail.com>

Closes #17342 from weiqingy/SPARK-18910.
2017-04-26 13:54:40 -07:00
Michal Szafranski a277ae80a2 [SPARK-20474] Fixing OnHeapColumnVector reallocation
## What changes were proposed in this pull request?
OnHeapColumnVector reallocation copies to the new storage data up to 'elementsAppended'. This variable is only updated when using the ColumnVector.appendX API, while ColumnVector.putX is more commonly used.

## How was this patch tested?
Tested using existing unit tests.

Author: Michal Szafranski <michal@databricks.com>

Closes #17773 from michal-databricks/spark-20474.
2017-04-26 12:47:37 -07:00
Michal Szafranski 99c6cf9ef1 [SPARK-20473] Enabling missing types in ColumnVector.Array
## What changes were proposed in this pull request?
ColumnVector implementations originally did not support some Catalyst types (float, short, and boolean). Now that they do, those types should be also added to the ColumnVector.Array.

## How was this patch tested?
Tested using existing unit tests.

Author: Michal Szafranski <michal@databricks.com>

Closes #17772 from michal-databricks/spark-20473.
2017-04-26 11:21:25 -07:00
Sameer Agarwal caf392025c [SPARK-18127] Add hooks and extension points to Spark
## What changes were proposed in this pull request?

This patch adds support for customizing the spark session by injecting user-defined custom extensions. This allows a user to add custom analyzer rules/checks, optimizer rules, planning strategies or even a customized parser.

## How was this patch tested?

Unit Tests in SparkSessionExtensionSuite

Author: Sameer Agarwal <sameerag@cs.berkeley.edu>

Closes #17724 from sameeragarwal/session-extensions.
2017-04-25 17:05:20 -07:00
Sameer Agarwal 31345fde82 [SPARK-20451] Filter out nested mapType datatypes from sort order in randomSplit
## What changes were proposed in this pull request?

In `randomSplit`, It is possible that the underlying dataset doesn't guarantee the ordering of rows in its constituent partitions each time a split is materialized which could result in overlapping
splits.

To prevent this, as part of SPARK-12662, we explicitly sort each input partition to make the ordering deterministic. Given that `MapTypes` cannot be sorted this patch explicitly prunes them out from the sort order. Additionally, if the resulting sort order is empty, this patch then materializes the dataset to guarantee determinism.

## How was this patch tested?

Extended `randomSplit on reordered partitions` in `DataFrameStatSuite` to also test for dataframes with mapTypes nested mapTypes.

Author: Sameer Agarwal <sameerag@cs.berkeley.edu>

Closes #17751 from sameeragarwal/randomsplit2.
2017-04-25 13:05:20 +08:00
Xiao Li 776a2c0e91 [SPARK-20439][SQL] Fix Catalog API listTables and getTable when failed to fetch table metadata
### What changes were proposed in this pull request?

`spark.catalog.listTables` and `spark.catalog.getTable` does not work if we are unable to retrieve table metadata due to any reason (e.g., table serde class is not accessible or the table type is not accepted by Spark SQL). After this PR, the APIs still return the corresponding Table without the description and tableType)

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

Author: Xiao Li <gatorsmile@gmail.com>

Closes #17730 from gatorsmile/listTables.
2017-04-24 17:21:42 +08:00
Takeshi Yamamuro b3c572a6b3 [SPARK-20430][SQL] Initialise RangeExec parameters in a driver side
## What changes were proposed in this pull request?
This pr initialised `RangeExec` parameters in a driver side.
In the current master, a query below throws `NullPointerException`;
```
sql("SET spark.sql.codegen.wholeStage=false")
sql("SELECT * FROM range(1)").show

17/04/20 17:11:05 ERROR Executor: Exception in task 0.0 in stage 0.0 (TID 0)
java.lang.NullPointerException
        at org.apache.spark.sql.execution.SparkPlan.sparkContext(SparkPlan.scala:54)
        at org.apache.spark.sql.execution.RangeExec.numSlices(basicPhysicalOperators.scala:343)
        at org.apache.spark.sql.execution.RangeExec$$anonfun$20.apply(basicPhysicalOperators.scala:506)
        at org.apache.spark.sql.execution.RangeExec$$anonfun$20.apply(basicPhysicalOperators.scala:505)
        at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsWithIndex$1$$anonfun$apply$26.apply(RDD.scala:844)
        at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsWithIndex$1$$anonfun$apply$26.apply(RDD.scala:844)
        at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
        at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:323)
        at org.apache.spark.rdd.RDD.iterator(RDD.scala:287)
        at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
        at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:323)
        at org.apache.spark.rdd.RDD.iterator(RDD.scala:287)
        at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)
        at org.apache.spark.scheduler.Task.run(Task.scala:108)
        at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:320)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
```

## How was this patch tested?
Added a test in `DataFrameRangeSuite`.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #17717 from maropu/SPARK-20430.
2017-04-22 09:41:58 -07:00
Herman van Hovell e2b3d2367a [SPARK-20420][SQL] Add events to the external catalog
## What changes were proposed in this pull request?
It is often useful to be able to track changes to the `ExternalCatalog`. This PR makes the `ExternalCatalog` emit events when a catalog object is changed. Events are fired before and after the change.

The following events are fired per object:

- Database
  - CreateDatabasePreEvent: event fired before the database is created.
  - CreateDatabaseEvent: event fired after the database has been created.
  - DropDatabasePreEvent: event fired before the database is dropped.
  - DropDatabaseEvent: event fired after the database has been dropped.
- Table
  - CreateTablePreEvent: event fired before the table is created.
  - CreateTableEvent: event fired after the table has been created.
  - RenameTablePreEvent: event fired before the table is renamed.
  - RenameTableEvent: event fired after the table has been renamed.
  - DropTablePreEvent: event fired before the table is dropped.
  - DropTableEvent: event fired after the table has been dropped.
- Function
  - CreateFunctionPreEvent: event fired before the function is created.
  - CreateFunctionEvent: event fired after the function has been created.
  - RenameFunctionPreEvent: event fired before the function is renamed.
  - RenameFunctionEvent: event fired after the function has been renamed.
  - DropFunctionPreEvent: event fired before the function is dropped.
  - DropFunctionPreEvent: event fired after the function has been dropped.

The current events currently only contain the names of the object modified. We add more events, and more details at a later point.

A user can monitor changes to the external catalog by adding a listener to the Spark listener bus checking for `ExternalCatalogEvent`s using the `SparkListener.onOtherEvent` hook. A more direct approach is add listener directly to the `ExternalCatalog`.

## How was this patch tested?
Added the `ExternalCatalogEventSuite`.

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #17710 from hvanhovell/SPARK-20420.
2017-04-21 00:05:03 -07:00
Takeshi Yamamuro 48d760d028 [SPARK-20281][SQL] Print the identical Range parameters of SparkContext APIs and SQL in explain
## What changes were proposed in this pull request?
This pr modified code to print the identical `Range` parameters of SparkContext APIs and SQL in `explain` output. In the current master, they internally use `defaultParallelism` for `splits` by default though, they print different strings in explain output;

```
scala> spark.range(4).explain
== Physical Plan ==
*Range (0, 4, step=1, splits=Some(8))

scala> sql("select * from range(4)").explain
== Physical Plan ==
*Range (0, 4, step=1, splits=None)
```

## How was this patch tested?
Added tests in `SQLQuerySuite` and modified some results in the existing tests.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #17670 from maropu/SPARK-20281.
2017-04-20 19:40:21 -07:00
Herman van Hovell 760c8d088d [SPARK-20329][SQL] Make timezone aware expression without timezone unresolved
## What changes were proposed in this pull request?
A cast expression with a resolved time zone is not equal to a cast expression without a resolved time zone. The `ResolveAggregateFunction` assumed that these expression were the same, and would fail to resolve `HAVING` clauses which contain a `Cast` expression.

This is in essence caused by the fact that a `TimeZoneAwareExpression` can be resolved without a set time zone. This PR fixes this, and makes a `TimeZoneAwareExpression` unresolved as long as it has no TimeZone set.

## How was this patch tested?
Added a regression test to the `SQLQueryTestSuite.having` file.

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #17641 from hvanhovell/SPARK-20329.
2017-04-21 10:06:12 +08:00
Juliusz Sompolski 0368eb9d86 [SPARK-20367] Properly unescape column names of partitioning columns parsed from paths.
## What changes were proposed in this pull request?

When infering partitioning schema from paths, the column in parsePartitionColumn should be unescaped with unescapePathName, just like it is being done in e.g. parsePathFragmentAsSeq.

## How was this patch tested?

Added a test to FileIndexSuite.

Author: Juliusz Sompolski <julek@databricks.com>

Closes #17703 from juliuszsompolski/SPARK-20367.
2017-04-21 09:49:42 +08:00
Reynold Xin c6f62c5b81 [SPARK-20405][SQL] Dataset.withNewExecutionId should be private
## What changes were proposed in this pull request?
Dataset.withNewExecutionId is only used in Dataset itself and should be private.

## How was this patch tested?
N/A - this is a simple visibility change.

Author: Reynold Xin <rxin@databricks.com>

Closes #17699 from rxin/SPARK-20405.
2017-04-20 14:29:59 +02:00
Xiao Li 55bea56911 [SPARK-20156][SQL][FOLLOW-UP] Java String toLowerCase "Turkish locale bug" in Database and Table DDLs
### What changes were proposed in this pull request?
Database and Table names conform the Hive standard ("[a-zA-z_0-9]+"), i.e. if this name only contains characters, numbers, and _.

When calling `toLowerCase` on the names, we should add `Locale.ROOT` to the `toLowerCase`for avoiding inadvertent locale-sensitive variation in behavior (aka the "Turkish locale problem").

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

Author: Xiao Li <gatorsmile@gmail.com>

Closes #17655 from gatorsmile/locale.
2017-04-20 11:13:48 +01:00
Eric Liang dd6d55d5de [SPARK-20398][SQL] range() operator should include cancellation reason when killed
## What changes were proposed in this pull request?

https://issues.apache.org/jira/browse/SPARK-19820 adds a reason field for why tasks were killed. However, for backwards compatibility it left the old TaskKilledException constructor which defaults to "unknown reason".
The range() operator should use the constructor that fills in the reason rather than dropping it on task kill.

## How was this patch tested?

Existing tests, and I tested this manually.

Author: Eric Liang <ekl@databricks.com>

Closes #17692 from ericl/fix-kill-reason-in-range.
2017-04-19 19:53:40 -07:00
Liang-Chi Hsieh 773754b6c1 [SPARK-20356][SQL] Pruned InMemoryTableScanExec should have correct output partitioning and ordering
## What changes were proposed in this pull request?

The output of `InMemoryTableScanExec` can be pruned and mismatch with `InMemoryRelation` and its child plan's output. This causes wrong output partitioning and ordering.

## How was this patch tested?

Jenkins tests.

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

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

Closes #17679 from viirya/SPARK-20356.
2017-04-19 16:01:28 +08:00
Xiao Li 01ff0350a8 [SPARK-20349][SQL] ListFunctions returns duplicate functions after using persistent functions
### What changes were proposed in this pull request?
The session catalog caches some persistent functions in the `FunctionRegistry`, so there can be duplicates. Our Catalog API `listFunctions` does not handle it.

It would be better if `SessionCatalog` API can de-duplciate the records, instead of doing it by each API caller. In `FunctionRegistry`, our functions are identified by the unquoted string. Thus, this PR is try to parse it using our parser interface and then de-duplicate the names.

### How was this patch tested?
Added test cases.

Author: Xiao Li <gatorsmile@gmail.com>

Closes #17646 from gatorsmile/showFunctions.
2017-04-17 09:50:20 -07:00
wangzhenhua fb036c4413 [SPARK-20318][SQL] Use Catalyst type for min/max in ColumnStat for ease of estimation
## What changes were proposed in this pull request?

Currently when estimating predicates like col > literal or col = literal, we will update min or max in column stats based on literal value. However, literal value is of Catalyst type (internal type), while min/max is of external type. Then for the next predicate, we again need to do type conversion to compare and update column stats. This is awkward and causes many unnecessary conversions in estimation.

To solve this, we use Catalyst type for min/max in `ColumnStat`. Note that the persistent format in metastore is still of external type, so there's no inconsistency for statistics in metastore.

This pr also fixes a bug for boolean type in `IN` condition.

## How was this patch tested?

The changes for ColumnStat are covered by existing tests.
For bug fix, a new test for boolean type in IN condition is added

Author: wangzhenhua <wangzhenhua@huawei.com>

Closes #17630 from wzhfy/refactorColumnStat.
2017-04-14 19:16:47 +08:00
Steve Loughran 7536e2849d [SPARK-20038][SQL] FileFormatWriter.ExecuteWriteTask.releaseResources() implementations to be re-entrant
## What changes were proposed in this pull request?

have the`FileFormatWriter.ExecuteWriteTask.releaseResources()` implementations  set `currentWriter=null` in a finally clause. This guarantees that if the first call to `currentWriter()` throws an exception, the second releaseResources() call made during the task cancel process will not trigger a second attempt to close the stream.

## How was this patch tested?

Tricky. I've been fixing the underlying cause when I saw the problem [HADOOP-14204](https://issues.apache.org/jira/browse/HADOOP-14204), but SPARK-10109 shows I'm not the first to have seen this. I can't replicate it locally any more, my code no longer being broken.

code review, however, should be straightforward

Author: Steve Loughran <stevel@hortonworks.com>

Closes #17364 from steveloughran/stevel/SPARK-20038-close.
2017-04-13 15:30:44 -05:00
Burak Yavuz 924c42477b [SPARK-20301][FLAKY-TEST] Fix Hadoop Shell.runCommand flakiness in Structured Streaming tests
## What changes were proposed in this pull request?

Some Structured Streaming tests show flakiness such as:
```
[info] - prune results by current_date, complete mode - 696 *** FAILED *** (10 seconds, 937 milliseconds)
[info]   Timed out while stopping and waiting for microbatchthread to terminate.: The code passed to failAfter did not complete within 10 seconds.
```

This happens when we wait for the stream to stop, but it doesn't. The reason it doesn't stop is that we interrupt the microBatchThread, but Hadoop's `Shell.runCommand` swallows the interrupt exception, and the exception is not propagated upstream to the microBatchThread. Then this thread continues to run, only to start blocking on the `streamManualClock`.

## How was this patch tested?

Thousand retries locally and [Jenkins](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/75720/testReport) of the flaky tests

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #17613 from brkyvz/flaky-stream-agg.
2017-04-12 11:24:59 -07:00
Xiao Li 504e62e2f4 [SPARK-20303][SQL] Rename createTempFunction to registerFunction
### What changes were proposed in this pull request?
Session catalog API `createTempFunction` is being used by Hive build-in functions, persistent functions, and temporary functions. Thus, the name is confusing. This PR is to rename it by `registerFunction`. Also we can move construction of `FunctionBuilder` and `ExpressionInfo` into the new `registerFunction`, instead of duplicating the logics everywhere.

In the next PRs, the remaining Function-related APIs also need cleanups.

### How was this patch tested?
Existing test cases.

Author: Xiao Li <gatorsmile@gmail.com>

Closes #17615 from gatorsmile/cleanupCreateTempFunction.
2017-04-12 09:01:26 -07:00
hyukjinkwon ceaf77ae43 [SPARK-18692][BUILD][DOCS] Test Java 8 unidoc build on Jenkins
## What changes were proposed in this pull request?

This PR proposes to run Spark unidoc to test Javadoc 8 build as Javadoc 8 is easily re-breakable.

There are several problems with it:

- It introduces little extra bit of time to run the tests. In my case, it took 1.5 mins more (`Elapsed :[94.8746569157]`). How it was tested is described in "How was this patch tested?".

- > One problem that I noticed was that Unidoc appeared to be processing test sources: if we can find a way to exclude those from being processed in the first place then that might significantly speed things up.

  (see  joshrosen's [comment](https://issues.apache.org/jira/browse/SPARK-18692?focusedCommentId=15947627&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15947627))

To complete this automated build, It also suggests to fix existing Javadoc breaks / ones introduced by test codes as described above.

There fixes are similar instances that previously fixed. Please refer https://github.com/apache/spark/pull/15999 and https://github.com/apache/spark/pull/16013

Note that this only fixes **errors** not **warnings**. Please see my observation https://github.com/apache/spark/pull/17389#issuecomment-288438704 for spurious errors by warnings.

## How was this patch tested?

Manually via `jekyll build` for building tests. Also, tested via running `./dev/run-tests`.

This was tested via manually adding `time.time()` as below:

```diff
     profiles_and_goals = build_profiles + sbt_goals

     print("[info] Building Spark unidoc (w/Hive 1.2.1) using SBT with these arguments: ",
           " ".join(profiles_and_goals))

+    import time
+    st = time.time()
     exec_sbt(profiles_and_goals)
+    print("Elapsed :[%s]" % str(time.time() - st))
```

produces

```
...
========================================================================
Building Unidoc API Documentation
========================================================================
...
[info] Main Java API documentation successful.
...
Elapsed :[94.8746569157]
...

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17477 from HyukjinKwon/SPARK-18692.
2017-04-12 12:38:48 +01: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
Dilip Biswal b14bfc3f8e [SPARK-19993][SQL] Caching logical plans containing subquery expressions does not work.
## What changes were proposed in this pull request?
The sameResult() method does not work when the logical plan contains subquery expressions.

**Before the fix**
```SQL
scala> val ds = spark.sql("select * from s1 where s1.c1 in (select s2.c1 from s2 where s1.c1 = s2.c1)")
ds: org.apache.spark.sql.DataFrame = [c1: int]

scala> ds.cache
res13: ds.type = [c1: int]

scala> spark.sql("select * from s1 where s1.c1 in (select s2.c1 from s2 where s1.c1 = s2.c1)").explain(true)
== Analyzed Logical Plan ==
c1: int
Project [c1#86]
+- Filter c1#86 IN (list#78 [c1#86])
   :  +- Project [c1#87]
   :     +- Filter (outer(c1#86) = c1#87)
   :        +- SubqueryAlias s2
   :           +- Relation[c1#87] parquet
   +- SubqueryAlias s1
      +- Relation[c1#86] parquet

== Optimized Logical Plan ==
Join LeftSemi, ((c1#86 = c1#87) && (c1#86 = c1#87))
:- Relation[c1#86] parquet
+- Relation[c1#87] parquet
```
**Plan after fix**
```SQL
== Analyzed Logical Plan ==
c1: int
Project [c1#22]
+- Filter c1#22 IN (list#14 [c1#22])
   :  +- Project [c1#23]
   :     +- Filter (outer(c1#22) = c1#23)
   :        +- SubqueryAlias s2
   :           +- Relation[c1#23] parquet
   +- SubqueryAlias s1
      +- Relation[c1#22] parquet

== Optimized Logical Plan ==
InMemoryRelation [c1#22], true, 10000, StorageLevel(disk, memory, deserialized, 1 replicas)
   +- *BroadcastHashJoin [c1#1, c1#1], [c1#2, c1#2], LeftSemi, BuildRight
      :- *FileScan parquet default.s1[c1#1] Batched: true, Format: Parquet, Location: InMemoryFileIndex[file:/Users/dbiswal/mygit/apache/spark/bin/spark-warehouse/s1], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<c1:int>
      +- BroadcastExchange HashedRelationBroadcastMode(List((shiftleft(cast(input[0, int, true] as bigint), 32) | (cast(input[0, int, true] as bigint) & 4294967295))))
         +- *FileScan parquet default.s2[c1#2] Batched: true, Format: Parquet, Location: InMemoryFileIndex[file:/Users/dbiswal/mygit/apache/spark/bin/spark-warehouse/s2], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<c1:int>
```
## How was this patch tested?
New tests are added to CachedTableSuite.

Author: Dilip Biswal <dbiswal@us.ibm.com>

Closes #17330 from dilipbiswal/subquery_cache_final.
2017-04-12 12:18:01 +08:00
DB Tsai 8ad63ee158 [SPARK-20291][SQL] NaNvl(FloatType, NullType) should not be cast to NaNvl(DoubleType, DoubleType)
## What changes were proposed in this pull request?

`NaNvl(float value, null)` will be converted into `NaNvl(float value, Cast(null, DoubleType))` and finally `NaNvl(Cast(float value, DoubleType), Cast(null, DoubleType))`.

This will cause mismatching in the output type when the input type is float.

By adding extra rule in TypeCoercion can resolve this issue.

## How was this patch tested?

unite tests.

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

Author: DB Tsai <dbt@netflix.com>

Closes #17606 from dbtsai/fixNaNvl.
2017-04-12 11:19:20 +08:00
Reynold Xin 379b0b0bbd [SPARK-20283][SQL] Add preOptimizationBatches
## What changes were proposed in this pull request?
We currently have postHocOptimizationBatches, but not preOptimizationBatches. This patch adds preOptimizationBatches so the optimizer debugging extensions are symmetric.

## How was this patch tested?
N/A

Author: Reynold Xin <rxin@databricks.com>

Closes #17595 from rxin/SPARK-20283.
2017-04-10 14:14:09 -07:00
Shixiong Zhu a35b9d9712 [SPARK-20282][SS][TESTS] Write the commit log first to fix a race contion in tests
## What changes were proposed in this pull request?

This PR fixes the following failure:
```
sbt.ForkMain$ForkError: org.scalatest.exceptions.TestFailedException:
Assert on query failed:

== Progress ==
   AssertOnQuery(<condition>, )
   StopStream
   AddData to MemoryStream[value#30891]: 1,2
   StartStream(OneTimeTrigger,org.apache.spark.util.SystemClock35cdc93a,Map())
   CheckAnswer: [6],[3]
   StopStream
=> AssertOnQuery(<condition>, )
   AssertOnQuery(<condition>, )
   StartStream(OneTimeTrigger,org.apache.spark.util.SystemClockcdb247d,Map())
   CheckAnswer: [6],[3]
   StopStream
   AddData to MemoryStream[value#30891]: 3
   StartStream(OneTimeTrigger,org.apache.spark.util.SystemClock55394e4d,Map())
   CheckLastBatch: [2]
   StopStream
   AddData to MemoryStream[value#30891]: 0
   StartStream(OneTimeTrigger,org.apache.spark.util.SystemClock749aa997,Map())
   ExpectFailure[org.apache.spark.SparkException, isFatalError: false]
   AssertOnQuery(<condition>, )
   AssertOnQuery(<condition>, incorrect start offset or end offset on exception)

== Stream ==
Output Mode: Append
Stream state: not started
Thread state: dead

== Sink ==
0: [6] [3]

== Plan ==

	at org.scalatest.Assertions$class.newAssertionFailedException(Assertions.scala:495)
	at org.scalatest.FunSuite.newAssertionFailedException(FunSuite.scala:1555)
	at org.scalatest.Assertions$class.fail(Assertions.scala:1328)
	at org.scalatest.FunSuite.fail(FunSuite.scala:1555)
	at org.apache.spark.sql.streaming.StreamTest$class.failTest$1(StreamTest.scala:347)
	at org.apache.spark.sql.streaming.StreamTest$class.verify$1(StreamTest.scala:318)
	at org.apache.spark.sql.streaming.StreamTest$$anonfun$liftedTree1$1$1.apply(StreamTest.scala:483)
	at org.apache.spark.sql.streaming.StreamTest$$anonfun$liftedTree1$1$1.apply(StreamTest.scala:357)
	at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
	at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)
	at org.apache.spark.sql.streaming.StreamTest$class.liftedTree1$1(StreamTest.scala:357)
	at org.apache.spark.sql.streaming.StreamTest$class.testStream(StreamTest.scala:356)
	at org.apache.spark.sql.streaming.StreamingQuerySuite.testStream(StreamingQuerySuite.scala:41)
	at org.apache.spark.sql.streaming.StreamingQuerySuite$$anonfun$6.apply$mcV$sp(StreamingQuerySuite.scala:166)
	at org.apache.spark.sql.streaming.StreamingQuerySuite$$anonfun$6.apply(StreamingQuerySuite.scala:161)
	at org.apache.spark.sql.streaming.StreamingQuerySuite$$anonfun$6.apply(StreamingQuerySuite.scala:161)
	at org.apache.spark.sql.catalyst.util.package$.quietly(package.scala:42)
	at org.apache.spark.sql.test.SQLTestUtils$$anonfun$testQuietly$1.apply$mcV$sp(SQLTestUtils.scala:268)
	at org.apache.spark.sql.test.SQLTestUtils$$anonfun$testQuietly$1.apply(SQLTestUtils.scala:268)
	at org.apache.spark.sql.test.SQLTestUtils$$anonfun$testQuietly$1.apply(SQLTestUtils.scala:268)
	at org.scalatest.Transformer$$anonfun$apply$1.apply$mcV$sp(Transformer.scala:22)
	at org.scalatest.OutcomeOf$class.outcomeOf(OutcomeOf.scala:85)
	at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104)
	at org.scalatest.Transformer.apply(Transformer.scala:22)
	at org.scalatest.Transformer.apply(Transformer.scala:20)
	at org.scalatest.FunSuiteLike$$anon$1.apply(FunSuiteLike.scala:166)
	at org.apache.spark.SparkFunSuite.withFixture(SparkFunSuite.scala:68)
	at org.scalatest.FunSuiteLike$class.invokeWithFixture$1(FunSuiteLike.scala:163)
	at org.scalatest.FunSuiteLike$$anonfun$runTest$1.apply(FunSuiteLike.scala:175)
	at org.scalatest.FunSuiteLike$$anonfun$runTest$1.apply(FunSuiteLike.scala:175)
	at org.scalatest.SuperEngine.runTestImpl(Engine.scala:306)
	at org.scalatest.FunSuiteLike$class.runTest(FunSuiteLike.scala:175)
	at org.apache.spark.sql.streaming.StreamingQuerySuite.org$scalatest$BeforeAndAfterEach$$super$runTest(StreamingQuerySuite.scala:41)
	at org.scalatest.BeforeAndAfterEach$class.runTest(BeforeAndAfterEach.scala:255)
	at org.apache.spark.sql.streaming.StreamingQuerySuite.org$scalatest$BeforeAndAfter$$super$runTest(StreamingQuerySuite.scala:41)
	at org.scalatest.BeforeAndAfter$class.runTest(BeforeAndAfter.scala:200)
	at org.apache.spark.sql.streaming.StreamingQuerySuite.runTest(StreamingQuerySuite.scala:41)
	at org.scalatest.FunSuiteLike$$anonfun$runTests$1.apply(FunSuiteLike.scala:208)
	at org.scalatest.FunSuiteLike$$anonfun$runTests$1.apply(FunSuiteLike.scala:208)
	at org.scalatest.SuperEngine$$anonfun$traverseSubNodes$1$1.apply(Engine.scala:413)
	at org.scalatest.SuperEngine$$anonfun$traverseSubNodes$1$1.apply(Engine.scala:401)
	at scala.collection.immutable.List.foreach(List.scala:381)
	at org.scalatest.SuperEngine.traverseSubNodes$1(Engine.scala:401)
	at org.scalatest.SuperEngine.org$scalatest$SuperEngine$$runTestsInBranch(Engine.scala:396)
	at org.scalatest.SuperEngine.runTestsImpl(Engine.scala:483)
	at org.scalatest.FunSuiteLike$class.runTests(FunSuiteLike.scala:208)
	at org.scalatest.FunSuite.runTests(FunSuite.scala:1555)
	at org.scalatest.Suite$class.run(Suite.scala:1424)
	at org.scalatest.FunSuite.org$scalatest$FunSuiteLike$$super$run(FunSuite.scala:1555)
	at org.scalatest.FunSuiteLike$$anonfun$run$1.apply(FunSuiteLike.scala:212)
	at org.scalatest.FunSuiteLike$$anonfun$run$1.apply(FunSuiteLike.scala:212)
	at org.scalatest.SuperEngine.runImpl(Engine.scala:545)
	at org.scalatest.FunSuiteLike$class.run(FunSuiteLike.scala:212)
	at org.apache.spark.SparkFunSuite.org$scalatest$BeforeAndAfterAll$$super$run(SparkFunSuite.scala:31)
	at org.scalatest.BeforeAndAfterAll$class.liftedTree1$1(BeforeAndAfterAll.scala:257)
	at org.scalatest.BeforeAndAfterAll$class.run(BeforeAndAfterAll.scala:256)
	at org.apache.spark.sql.streaming.StreamingQuerySuite.org$scalatest$BeforeAndAfter$$super$run(StreamingQuerySuite.scala:41)
	at org.scalatest.BeforeAndAfter$class.run(BeforeAndAfter.scala:241)
	at org.apache.spark.sql.streaming.StreamingQuerySuite.run(StreamingQuerySuite.scala:41)
	at org.scalatest.tools.Framework.org$scalatest$tools$Framework$$runSuite(Framework.scala:357)
	at org.scalatest.tools.Framework$ScalaTestTask.execute(Framework.scala:502)
	at sbt.ForkMain$Run$2.call(ForkMain.java:296)
	at sbt.ForkMain$Run$2.call(ForkMain.java:286)
	at java.util.concurrent.FutureTask.run(FutureTask.java:266)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
	at java.lang.Thread.run(Thread.java:745)
```

The failure is because `CheckAnswer` will run once `committedOffsets` is updated. Then writing the commit log may be interrupted by the following `StopStream`.

This PR just change the order to write the commit log first.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #17594 from zsxwing/SPARK-20282.
2017-04-10 14:09:32 -07:00
Bogdan Raducanu f6dd8e0e16 [SPARK-20280][CORE] FileStatusCache Weigher integer overflow
## What changes were proposed in this pull request?

Weigher.weigh needs to return Int but it is possible for an Array[FileStatus] to have size > Int.maxValue. To avoid this, the size is scaled down by a factor of 32. The maximumWeight of the cache is also scaled down by the same factor.

## How was this patch tested?
New test in FileIndexSuite

Author: Bogdan Raducanu <bogdan@databricks.com>

Closes #17591 from bogdanrdc/SPARK-20280.
2017-04-10 21:56:21 +02:00
Sean Owen a26e3ed5e4 [SPARK-20156][CORE][SQL][STREAMING][MLLIB] Java String toLowerCase "Turkish locale bug" causes Spark problems
## What changes were proposed in this pull request?

Add Locale.ROOT to internal calls to String `toLowerCase`, `toUpperCase`, to avoid inadvertent locale-sensitive variation in behavior (aka the "Turkish locale problem").

The change looks large but it is just adding `Locale.ROOT` (the locale with no country or language specified) to every call to these methods.

## How was this patch tested?

Existing tests.

Author: Sean Owen <sowen@cloudera.com>

Closes #17527 from srowen/SPARK-20156.
2017-04-10 20:11:56 +01:00
Wenchen Fan 3d7f201f2a [SPARK-20229][SQL] add semanticHash to QueryPlan
## What changes were proposed in this pull request?

Like `Expression`, `QueryPlan` should also have a `semanticHash` method, then we can put plans to a hash map and look it up fast. This PR refactors `QueryPlan` to follow `Expression` and put all the normalization logic in `QueryPlan.canonicalized`, so that it's very natural to implement `semanticHash`.

follow-up: improve `CacheManager` to leverage this `semanticHash` and speed up plan lookup, instead of iterating all cached plans.

## How was this patch tested?

existing tests. Note that we don't need to test the `semanticHash` method, once the existing tests prove `sameResult` is correct, we are good.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #17541 from cloud-fan/plan-semantic.
2017-04-10 13:36:08 +08:00
DB Tsai 1a0bc41659
[SPARK-20270][SQL] na.fill should not change the values in long or integer when the default value is in double
## What changes were proposed in this pull request?

This bug was partially addressed in SPARK-18555 https://github.com/apache/spark/pull/15994, but the root cause isn't completely solved. This bug is pretty critical since it changes the member id in Long in our application if the member id can not be represented by Double losslessly when the member id is very big.

Here is an example how this happens, with
```
      Seq[(java.lang.Long, java.lang.Double)]((null, 3.14), (9123146099426677101L, null),
        (9123146560113991650L, 1.6), (null, null)).toDF("a", "b").na.fill(0.2),
```
the logical plan will be
```
== Analyzed Logical Plan ==
a: bigint, b: double
Project [cast(coalesce(cast(a#232L as double), cast(0.2 as double)) as bigint) AS a#240L, cast(coalesce(nanvl(b#233, cast(null as double)), 0.2) as double) AS b#241]
+- Project [_1#229L AS a#232L, _2#230 AS b#233]
   +- LocalRelation [_1#229L, _2#230]
```

Note that even the value is not null, Spark will cast the Long into Double first. Then if it's not null, Spark will cast it back to Long which results in losing precision.

The behavior should be that the original value should not be changed if it's not null, but Spark will change the value which is wrong.

With the PR, the logical plan will be
```
== Analyzed Logical Plan ==
a: bigint, b: double
Project [coalesce(a#232L, cast(0.2 as bigint)) AS a#240L, coalesce(nanvl(b#233, cast(null as double)), cast(0.2 as double)) AS b#241]
+- Project [_1#229L AS a#232L, _2#230 AS b#233]
   +- LocalRelation [_1#229L, _2#230]
```
which behaves correctly without changing the original Long values and also avoids extra cost of unnecessary casting.

## How was this patch tested?

unit test added.

+cc srowen rxin cloud-fan gatorsmile

Thanks.

Author: DB Tsai <dbt@netflix.com>

Closes #17577 from dbtsai/fixnafill.
2017-04-10 05:16:34 +00:00
Adrian Ionescu 589f3edb82 [SPARK-20255] Move listLeafFiles() to InMemoryFileIndex
## What changes were proposed in this pull request

Trying to get a grip on the `FileIndex` hierarchy, I was confused by the following inconsistency:

On the one hand, `PartitioningAwareFileIndex` defines `leafFiles` and `leafDirToChildrenFiles` as abstract, but on the other it fully implements `listLeafFiles` which does all the listing of files. However, the latter is only used by `InMemoryFileIndex`.

I'm hereby proposing to move this method (and all its dependencies) to the implementation class that actually uses it, and thus unclutter the `PartitioningAwareFileIndex` interface.

## How was this patch tested?

`./build/sbt sql/test`

Author: Adrian Ionescu <adrian@databricks.com>

Closes #17570 from adrian-ionescu/list-leaf-files.
2017-04-07 14:00:23 -07:00
Wenchen Fan ad3cc1312d [SPARK-20245][SQL][MINOR] pass output to LogicalRelation directly
## What changes were proposed in this pull request?

Currently `LogicalRelation` has a `expectedOutputAttributes` parameter, which makes it hard to reason about what the actual output is. Like other leaf nodes, `LogicalRelation` should also take `output` as a parameter, to simplify the logic

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #17552 from cloud-fan/minor.
2017-04-07 15:58:50 +08: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
Tathagata Das 9543fc0e08 [SPARK-20224][SS] Updated docs for streaming dropDuplicates and mapGroupsWithState
## What changes were proposed in this pull request?

- Fixed bug in Java API not passing timeout conf to scala API
- Updated markdown docs
- Updated scala docs
- Added scala and Java example

## How was this patch tested?
Manually ran examples.

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

Closes #17539 from tdas/SPARK-20224.
2017-04-05 16:03:04 -07:00
Tathagata Das dad499f324 [SPARK-20209][SS] Execute next trigger immediately if previous batch took longer than trigger interval
## What changes were proposed in this pull request?

For large trigger intervals (e.g. 10 minutes), if a batch takes 11 minutes, then it will wait for 9 mins before starting the next batch. This does not make sense. The processing time based trigger policy should be to do process batches as fast as possible, but no faster than 1 in every trigger interval. If batches are taking longer than trigger interval anyways, then no point waiting extra trigger interval.

In this PR, I modified the ProcessingTimeExecutor to do so. Another minor change I did was to extract our StreamManualClock into a separate class so that it can be used outside subclasses of StreamTest. For example, ProcessingTimeExecutorSuite does not need to create any context for testing, just needs the StreamManualClock.

## How was this patch tested?
Added new unit tests to comprehensively test this behavior.

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

Closes #17525 from tdas/SPARK-20209.
2017-04-04 23:20:17 -07:00
Reynold Xin b6e71032d9 Small doc fix for ReuseSubquery. 2017-04-04 22:46:42 -07:00
Wenchen Fan 295747e597 [SPARK-19716][SQL] support by-name resolution for struct type elements in array
## What changes were proposed in this pull request?

Previously when we construct deserializer expression for array type, we will first cast the corresponding field to expected array type and then apply `MapObjects`.

However, by doing that, we lose the opportunity to do by-name resolution for struct type inside array type. In this PR, I introduce a `UnresolvedMapObjects` to hold the lambda function and the input array expression. Then during analysis, after the input array expression is resolved, we get the actual array element type and apply by-name resolution. Then we don't need to add `Cast` for array type when constructing the deserializer expression, as the element type is determined later at analyzer.

## How was this patch tested?

new regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #17398 from cloud-fan/dataset.
2017-04-04 16:38:32 -07:00
Wenchen Fan 402bf2a50d [SPARK-20204][SQL] remove SimpleCatalystConf and CatalystConf type alias
## What changes were proposed in this pull request?

This is a follow-up of https://github.com/apache/spark/pull/17285 .

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #17521 from cloud-fan/conf.
2017-04-04 11:56:21 -07:00
Xiao Li 26e7bca229 [SPARK-20198][SQL] Remove the inconsistency in table/function name conventions in SparkSession.Catalog APIs
### What changes were proposed in this pull request?
Observed by felixcheung , in `SparkSession`.`Catalog` APIs, we have different conventions/rules for table/function identifiers/names. Most APIs accept the qualified name (i.e., `databaseName`.`tableName` or `databaseName`.`functionName`). However, the following five APIs do not accept it.
- def listColumns(tableName: String): Dataset[Column]
- def getTable(tableName: String): Table
- def getFunction(functionName: String): Function
- def tableExists(tableName: String): Boolean
- def functionExists(functionName: String): Boolean

To make them consistent with the other Catalog APIs, this PR does the changes, updates the function/API comments and adds the `params` to clarify the inputs we allow.

### How was this patch tested?
Added the test cases .

Author: Xiao Li <gatorsmile@gmail.com>

Closes #17518 from gatorsmile/tableIdentifier.
2017-04-04 18:57:46 +08:00