Commit graph

20733 commits

Author SHA1 Message Date
Marco Gaido e7adb7d7a6 [SPARK-22437][PYSPARK] default mode for jdbc is wrongly set to None
## What changes were proposed in this pull request?

When writing using jdbc with python currently we are wrongly assigning by default None as writing mode. This is due to wrongly calling mode on the `_jwrite` object instead of `self` and it causes an exception.

## How was this patch tested?

manual tests

Author: Marco Gaido <mgaido@hortonworks.com>

Closes #19654 from mgaido91/SPARK-22437.
2017-11-04 16:59:58 +09:00
Kazuaki Ishizaki bc1e101039 [SPARK-22254][CORE] Fix the arrayMax in BufferHolder
## What changes were proposed in this pull request?

This PR replaces the old the maximum array size (`Int.MaxValue`) with the new one (`ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH`).
This PR also refactor the code to calculate the new array size to easily understand why we have to use `newSize - 2` for allocating a new array.

## How was this patch tested?

Used the existing test

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

Closes #19650 from kiszk/SPARK-22254.
2017-11-03 23:35:57 -07:00
Marco Gaido 8915886608 [SPARK-22418][SQL][TEST] Add test cases for NULL Handling
## What changes were proposed in this pull request?

Added a test class to check NULL handling behavior.
The expected behavior is defined as the one of the most well-known databases as specified here: https://sqlite.org/nulls.html.

SparkSQL behaves like other DBs:
 - Adding anything to null gives null -> YES
 - Multiplying null by zero gives null -> YES
 - nulls are distinct in SELECT DISTINCT -> NO
 - nulls are distinct in a UNION -> NO
 - "CASE WHEN null THEN 1 ELSE 0 END" is 0? -> YES
 - "null OR true" is true -> YES
 - "not (null AND false)" is true -> YES
 - null in aggregation are skipped -> YES

## How was this patch tested?

Added test class

Author: Marco Gaido <mgaido@hortonworks.com>

Closes #19653 from mgaido91/SPARK-22418.
2017-11-03 22:03:58 -07:00
zhoukang 51145f1376 [SPARK-22407][WEB-UI] Add rdd id column on storage page to speed up navigating
## What changes were proposed in this pull request?

Add rdd id column on storage page to speed up navigating.
Example has attached on [SPARK-22407](https://issues.apache.org/jira/browse/SPARK-22407)
An example below:
![add-rddid](https://user-images.githubusercontent.com/26762018/32361127-da0758ac-c097-11e7-9f8c-0ea7ffb87e12.png)
![rdd-cache](https://user-images.githubusercontent.com/26762018/32361128-da3c1574-c097-11e7-8ab1-2def66466f33.png)
## How was this patch tested?

Current unit test and manually deploy an history server for testing

Author: zhoukang <zhoukang199191@gmail.com>

Closes #19625 from caneGuy/zhoukang/add-rddid.
2017-11-03 12:20:17 +00:00
Wenchen Fan 2fd12af437 [SPARK-22306][SQL] alter table schema should not erase the bucketing metadata at hive side
forward-port https://github.com/apache/spark/pull/19622 to master branch.

This bug doesn't exist in master because we've added hive bucketing support and the hive bucketing metadata can be recognized by Spark, but we should still port it to master: 1) there may be other unsupported hive metadata removed by Spark. 2) reduce code difference between master and 2.2 to ease the backport in the feature.

***

When we alter table schema, we set the new schema to spark `CatalogTable`, convert it to hive table, and finally call `hive.alterTable`. This causes a problem in Spark 2.2, because hive bucketing metedata is not recognized by Spark, which means a Spark `CatalogTable` representing a hive table is always non-bucketed, and when we convert it to hive table and call `hive.alterTable`, the original hive bucketing metadata will be removed.

To fix this bug, we should read out the raw hive table metadata, update its schema, and call `hive.alterTable`. By doing this we can guarantee only the schema is changed, and nothing else.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19644 from cloud-fan/infer.
2017-11-02 23:41:16 +01:00
ZouChenjun 882079f5c6 [SPARK-22243][DSTREAM] spark.yarn.jars should reload from config when checkpoint recovery
## What changes were proposed in this pull request?
the previous [PR](https://github.com/apache/spark/pull/19469) is deleted by mistake.
the solution is straight forward.
adding  "spark.yarn.jars" to propertiesToReload so this property will load from config.

## How was this patch tested?

manual tests

Author: ZouChenjun <zouchenjun@youzan.com>

Closes #19637 from ChenjunZou/checkpoint-yarn-jars.
2017-11-02 11:06:37 -07:00
Dongjoon Hyun e3f67a97f1 [SPARK-22416][SQL] Move OrcOptions from sql/hive to sql/core
## What changes were proposed in this pull request?

According to the [discussion](https://github.com/apache/spark/pull/19571#issuecomment-339472976) on SPARK-15474, we will add new OrcFileFormat in `sql/core` module and allow users to use both old and new OrcFileFormat.

To do that, `OrcOptions` should be visible in `sql/core` module, too. Previously, it was `private[orc]` in `sql/hive`. This PR removes `private[orc]` because we don't use `private[sql]` in `sql/execution` package after [SPARK-16964](https://github.com/apache/spark/pull/14554).

## How was this patch tested?

Pass the Jenkins with the existing tests.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #19636 from dongjoon-hyun/SPARK-22416.
2017-11-02 18:28:56 +01:00
hyukjinkwon 41b60125b6 [SPARK-22369][PYTHON][DOCS] Exposes catalog API documentation in PySpark
## What changes were proposed in this pull request?

This PR proposes to add a link from `spark.catalog(..)` to `Catalog` and expose Catalog APIs in PySpark as below:

<img width="740" alt="2017-10-29 12 25 46" src="https://user-images.githubusercontent.com/6477701/32135863-f8e9b040-bc40-11e7-92ad-09c8043a1295.png">

<img width="1131" alt="2017-10-29 12 26 33" src="https://user-images.githubusercontent.com/6477701/32135849-bb257b86-bc40-11e7-9eda-4d58fc1301c2.png">

Note that this is not shown in the list on the top - https://spark.apache.org/docs/latest/api/python/pyspark.sql.html#module-pyspark.sql

<img width="674" alt="2017-10-29 12 30 58" src="https://user-images.githubusercontent.com/6477701/32135854-d50fab16-bc40-11e7-9181-812c56fd22f5.png">

This is basically similar with `DataFrameReader` and `DataFrameWriter`.

## How was this patch tested?

Manually built the doc.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #19596 from HyukjinKwon/SPARK-22369.
2017-11-02 15:22:52 +01:00
Stavros Kontopoulos b2463fad71 [SPARK-22145][MESOS] fix supervise with checkpointing on mesos
## What changes were proposed in this pull request?

- Fixes the issue with the frameworkId being recovered by checkpointed data overwriting the one sent by the dipatcher.
- Keeps submission driver id as the only index for all data structures in the dispatcher.
Allocates a different task id per driver retry to satisfy the mesos requirements. Check the relevant ticket for the details on that.
## How was this patch tested?

Manually tested this with DC/OS 1.10. Launched a streaming job with checkpointing to hdfs, made the driver fail several times and observed behavior:
![image](https://user-images.githubusercontent.com/7945591/30940500-f7d2a744-a3e9-11e7-8c56-f2ccbb271e80.png)

![image](https://user-images.githubusercontent.com/7945591/30940550-19bc15de-a3ea-11e7-8a11-f48abfe36720.png)

![image](https://user-images.githubusercontent.com/7945591/30940524-083ea308-a3ea-11e7-83ae-00d3fa17b928.png)

![image](https://user-images.githubusercontent.com/7945591/30940579-2f0fb242-a3ea-11e7-82f9-86179da28b8c.png)

![image](https://user-images.githubusercontent.com/7945591/30940591-3b561b0e-a3ea-11e7-9dbd-e71912bb2ef3.png)

![image](https://user-images.githubusercontent.com/7945591/30940605-49c810ca-a3ea-11e7-8af5-67930851fd38.png)

![image](https://user-images.githubusercontent.com/7945591/30940631-59f4a288-a3ea-11e7-88cb-c3741b72bb13.png)

![image](https://user-images.githubusercontent.com/7945591/30940642-62346c9e-a3ea-11e7-8935-82e494925f67.png)

![image](https://user-images.githubusercontent.com/7945591/30940653-6c46d53c-a3ea-11e7-8dd1-5840d484d28c.png)

Author: Stavros Kontopoulos <st.kontopoulos@gmail.com>

Closes #19374 from skonto/fix_retry.
2017-11-02 13:25:48 +00:00
Patrick Woody 277b1924b4 [SPARK-22408][SQL] RelationalGroupedDataset's distinct pivot value calculation launches unnecessary stages
## What changes were proposed in this pull request?

Adding a global limit on top of the distinct values before sorting and collecting will reduce the overall work in the case where we have more distinct values. We will also eagerly perform a collect rather than a take because we know we only have at most (maxValues + 1) rows.

## How was this patch tested?

Existing tests cover sorted order

Author: Patrick Woody <pwoody@palantir.com>

Closes #19629 from pwoody/SPARK-22408.
2017-11-02 14:19:21 +01:00
Sean Owen 849b465bbf [SPARK-14650][REPL][BUILD] Compile Spark REPL for Scala 2.12
## What changes were proposed in this pull request?

Spark REPL changes for Scala 2.12.4: use command(), not processLine() in ILoop; remove direct dependence on older jline. Not sure whether this became needed in 2.12.4 or just missed this before. This makes spark-shell work in 2.12.

## How was this patch tested?

Existing tests; manual run of spark-shell in 2.11, 2.12 builds

Author: Sean Owen <sowen@cloudera.com>

Closes #19612 from srowen/SPARK-14650.2.
2017-11-02 09:45:34 +00:00
Marco Gaido b04eefae49 [MINOR][DOC] automatic type inference supports also Date and Timestamp
## What changes were proposed in this pull request?

Easy fix in the documentation, which is reporting that only numeric types and string are supported in type inference for partition columns, while Date and Timestamp are supported too since 2.1.0, thanks to SPARK-17388.

## How was this patch tested?

n/a

Author: Marco Gaido <mgaido@hortonworks.com>

Closes #19628 from mgaido91/SPARK-22398.
2017-11-02 09:30:03 +09:00
Reynold Xin d43e1f06bd [MINOR] Data source v2 docs update.
## What changes were proposed in this pull request?
This patch includes some doc updates for data source API v2. I was reading the code and noticed some minor issues.

## How was this patch tested?
This is a doc only change.

Author: Reynold Xin <rxin@databricks.com>

Closes #19626 from rxin/dsv2-update.
2017-11-01 18:39:15 +01:00
LucaCanali 1ffe03d9e8 [SPARK-22190][CORE] Add Spark executor task metrics to Dropwizard metrics
## What changes were proposed in this pull request?

This proposed patch is about making Spark executor task metrics available as Dropwizard metrics. This is intended to be of aid in monitoring Spark jobs and when drilling down on performance troubleshooting issues.

## How was this patch tested?

Manually tested on a Spark cluster (see JIRA for an example screenshot).

Author: LucaCanali <luca.canali@cern.ch>

Closes #19426 from LucaCanali/SparkTaskMetricsDropWizard.
2017-11-01 15:40:25 +01:00
Sital Kedia 444bce1c98 [SPARK-19112][CORE] Support for ZStandard codec
## What changes were proposed in this pull request?

Using zstd compression for Spark jobs spilling 100s of TBs of data, we could reduce the amount of data written to disk by as much as 50%. This translates to significant latency gain because of reduced disk io operations. There is a degradation CPU time by 2 - 5% because of zstd compression overhead, but for jobs which are bottlenecked by disk IO, this hit can be taken.

## Benchmark
Please note that this benchmark is using real world compute heavy production workload spilling TBs of data to disk

|         | zstd performance as compred to LZ4   |
| ------------- | -----:|
| spill/shuffle bytes    | -48% |
| cpu time    |    + 3% |
| cpu reservation time       |    -40%|
| latency     |     -40% |

## How was this patch tested?

Tested by running few jobs spilling large amount of data on the cluster and amount of intermediate data written to disk reduced by as much as 50%.

Author: Sital Kedia <skedia@fb.com>

Closes #18805 from sitalkedia/skedia/upstream_zstd.
2017-11-01 14:54:08 +01:00
Liang-Chi Hsieh 07f390a27d [SPARK-22347][PYSPARK][DOC] Add document to notice users for using udfs with conditional expressions
## What changes were proposed in this pull request?

Under the current execution mode of Python UDFs, we don't well support Python UDFs as branch values or else value in CaseWhen expression.

Since to fix it might need the change not small (e.g., #19592) and this issue has simpler workaround. We should just notice users in the document about this.

## How was this patch tested?

Only document change.

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

Closes #19617 from viirya/SPARK-22347-3.
2017-11-01 13:09:35 +01:00
Devaraj K 96798d14f0 [SPARK-22172][CORE] Worker hangs when the external shuffle service port is already in use
## What changes were proposed in this pull request?

Handling the NonFatal exceptions while starting the external shuffle service, if there are any NonFatal exceptions it logs and continues without the external shuffle service.

## How was this patch tested?

I verified it manually, it logs the exception and continues to serve without external shuffle service when BindException occurs.

Author: Devaraj K <devaraj@apache.org>

Closes #19396 from devaraj-kavali/SPARK-22172.
2017-11-01 18:07:39 +08:00
Zheng RuiFeng 556b5d2151 [SPARK-5484][FOLLOWUP] PeriodicRDDCheckpointer doc cleanup
## What changes were proposed in this pull request?
PeriodicRDDCheckpointer was already moved out of mllib in Spark-5484

## How was this patch tested?
existing tests

Author: Zheng RuiFeng <ruifengz@foxmail.com>

Closes #19618 from zhengruifeng/checkpointer_doc.
2017-11-01 08:45:11 +00:00
Jose Torres 73231860ba [SPARK-22305] Write HDFSBackedStateStoreProvider.loadMap non-recursively
## What changes were proposed in this pull request?
Write HDFSBackedStateStoreProvider.loadMap non-recursively. This prevents stack overflow if too many deltas stack up in a low memory environment.

## How was this patch tested?

existing unit tests for functional equivalence, new unit test to check for stack overflow

Author: Jose Torres <jose@databricks.com>

Closes #19611 from joseph-torres/SPARK-22305.
2017-10-31 11:53:50 -07:00
Sital Kedia 7986cc09b1 [SPARK-11334][CORE] Fix bug in Executor allocation manager in running tasks calculation
## What changes were proposed in this pull request?

We often see the issue of Spark jobs stuck because the Executor Allocation Manager does not ask for any executor even if there are pending tasks in case dynamic allocation is turned on. Looking at the logic in Executor Allocation Manager, which calculates the running tasks, it can happen that the calculation will be wrong and the number of running tasks can become negative.

## How was this patch tested?

Added unit test

Author: Sital Kedia <skedia@fb.com>

Closes #19580 from sitalkedia/skedia/fix_stuck_job.
2017-10-31 09:49:58 -07:00
Wenchen Fan 4d9ebf3835 [SPARK-19611][SQL][FOLLOWUP] set dataSchema correctly in HiveMetastoreCatalog.convertToLogicalRelation
## What changes were proposed in this pull request?

We made a mistake in https://github.com/apache/spark/pull/16944 . In `HiveMetastoreCatalog#inferIfNeeded` we infer the data schema, merge with full schema, and return the new full schema. At caller side we treat the full schema as data schema and set it to `HadoopFsRelation`.

This doesn't cause any problem because both parquet and orc can work with a wrong data schema that has extra columns, but it's better to fix this mistake.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19615 from cloud-fan/infer.
2017-10-31 11:35:32 +01:00
Zhenhua Wang 59589bc654 [SPARK-22310][SQL] Refactor join estimation to incorporate estimation logic for different kinds of statistics
## What changes were proposed in this pull request?

The current join estimation logic is only based on basic column statistics (such as ndv, etc). If we want to add estimation for other kinds of statistics (such as histograms), it's not easy to incorporate into the current algorithm:
1. When we have multiple pairs of join keys, the current algorithm computes cardinality in a single formula. But if different join keys have different kinds of stats, the computation logic for each pair of join keys become different, so the previous formula does not apply.
2. Currently it computes cardinality and updates join keys' column stats separately. It's better to do these two steps together, since both computation and update logic are different for different kinds of stats.

## How was this patch tested?

Only refactor, covered by existing tests.

Author: Zhenhua Wang <wangzhenhua@huawei.com>

Closes #19531 from wzhfy/join_est_refactor.
2017-10-31 11:13:48 +01:00
bomeng aa6db57e39 [SPARK-22399][ML] update the location of reference paper
## What changes were proposed in this pull request?
Update the url of reference paper.

## How was this patch tested?
It is comments, so nothing tested.

Author: bomeng <bmeng@us.ibm.com>

Closes #19614 from bomeng/22399.
2017-10-31 08:20:23 +00:00
pj.fanning 1ff41d8693 [SPARK-21708][BUILD] update some sbt plugins
## What changes were proposed in this pull request?

These are just some straightforward upgrades to use the latest versions of some sbt plugins that also support sbt 1.0.
The remaining sbt plugins that need upgrading will require bigger changes.

## How was this patch tested?

Tested sbt use manually.

Author: pj.fanning <pj.fanning@workday.com>

Closes #19609 from pjfanning/SPARK-21708.
2017-10-31 08:16:54 +00:00
Felix Cheung ded3ed9733 [SPARK-22327][SPARKR][TEST] check for version warning
## What changes were proposed in this pull request?

Will need to port to this to branch-1.6, -2.0, -2.1, -2.2

## How was this patch tested?

manually
Jenkins, AppVeyor

Author: Felix Cheung <felixcheung_m@hotmail.com>

Closes #19549 from felixcheung/rcranversioncheck.
2017-10-30 21:44:24 -07:00
Zhenhua Wang 44c4003155 [SPARK-22400][SQL] rename some APIs and classes to make their meaning clearer
## What changes were proposed in this pull request?

Both `ReadSupport` and `ReadTask` have a method called `createReader`, but they create different things. This could cause some confusion for data source developers. The same issue exists between `WriteSupport` and `DataWriterFactory`, both of which have a method called `createWriter`. This PR renames the method of `ReadTask`/`DataWriterFactory` to `createDataReader`/`createDataWriter`.

Besides, the name of `RowToInternalRowDataWriterFactory` is not correct, because it actually converts `InternalRow`s to `Row`s. It should be renamed `InternalRowDataWriterFactory`.

## How was this patch tested?

Only renaming, should be covered by existing tests.

Author: Zhenhua Wang <wzh_zju@163.com>

Closes #19610 from wzhfy/rename.
2017-10-30 10:21:05 -07:00
gatorsmile 65338de5fb [SPARK-22396][SQL] Better Error Message for InsertIntoDir using Hive format without enabling Hive Support
## What changes were proposed in this pull request?
When Hive support is not on, users can hit unresolved plan node when trying to call `INSERT OVERWRITE DIRECTORY` using Hive format.
```
"unresolved operator 'InsertIntoDir true, Storage(Location: /private/var/folders/vx/j0ydl5rn0gd9mgrh1pljnw900000gn/T/spark-b4227606-9311-46a8-8c02-56355bf0e2bc, Serde Library: org.apache.hadoop.hive.ql.io.orc.OrcSerde, InputFormat: org.apache.hadoop.hive.ql.io.orc.OrcInputFormat, OutputFormat: org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat), hive, true;;
```

This PR is to issue a better error message.
## How was this patch tested?
Added a test case.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #19608 from gatorsmile/hivesupportInsertOverwrite.
2017-10-30 10:19:34 -07:00
Wenchen Fan 079a2609d7 [SPARK-17788][SPARK-21033][SQL] fix the potential OOM in UnsafeExternalSorter and ShuffleExternalSorter
## What changes were proposed in this pull request?

In `UnsafeInMemorySorter`, one record may take 32 bytes: 1 `long` for pointer, 1 `long` for key-prefix, and another 2 `long`s as the temporary buffer for radix sort.

In `UnsafeExternalSorter`, we set the `DEFAULT_NUM_ELEMENTS_FOR_SPILL_THRESHOLD` to be `1024 * 1024 * 1024 / 2`, and hoping the max size of point array to be 8 GB. However this is wrong, `1024 * 1024 * 1024 / 2 * 32` is actually 16 GB, and if we grow the point array before reach this limitation, we may hit the max-page-size error.

Users may see exception like this on large dataset:
```
Caused by: java.lang.IllegalArgumentException: Cannot allocate a page with more than 17179869176 bytes
at org.apache.spark.memory.TaskMemoryManager.allocatePage(TaskMemoryManager.java:241)
at org.apache.spark.memory.MemoryConsumer.allocatePage(MemoryConsumer.java:121)
at org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.acquireNewPageIfNecessary(UnsafeExternalSorter.java:374)
at org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.insertRecord(UnsafeExternalSorter.java:396)
at org.apache.spark.sql.execution.UnsafeExternalRowSorter.insertRow(UnsafeExternalRowSorter.java:94)
...
```

Setting `DEFAULT_NUM_ELEMENTS_FOR_SPILL_THRESHOLD` to a smaller number is not enough, users can still set the config to a big number and trigger the too large page size issue. This PR fixes it by explicitly handling the too large page size exception in the sorter and spill.

This PR also change the type of `spark.shuffle.spill.numElementsForceSpillThreshold` to int, because it's only compared with `numRecords`, which is an int. This is an internal conf so we don't have a serious compatibility issue.

## How was this patch tested?

TODO

Author: Wenchen Fan <wenchen@databricks.com>

Closes #18251 from cloud-fan/sort.
2017-10-30 17:53:06 +01:00
guoxiaolong 3663764254 [WEB-UI] Add count in fair scheduler pool page
## What changes were proposed in this pull request?

Add count in fair scheduler pool page. The purpose is to know the statistics clearly.
For specific reasons, please refer to PR of https://github.com/apache/spark/pull/18525

fix before:
![1](https://user-images.githubusercontent.com/26266482/31641589-4b17b970-b318-11e7-97eb-f5a36db428f6.png)

![2](https://user-images.githubusercontent.com/26266482/31641643-97b6345a-b318-11e7-8c20-4b164ade228d.png)

fix after:
![3](https://user-images.githubusercontent.com/26266482/31641688-e6ceacb6-b318-11e7-8204-6a816c581a29.png)

![4](https://user-images.githubusercontent.com/26266482/31641766-7310b0c0-b319-11e7-871d-a57f874f1e8b.png)

## How was this patch tested?

(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)
(If this patch involves UI changes, please attach a screenshot; otherwise, remove this)

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

Author: guoxiaolong <guo.xiaolong1@zte.com.cn>

Closes #19507 from guoxiaolongzte/add_count_in_fair_scheduler_pool_page.
2017-10-30 12:14:38 +00:00
Wenchen Fan 9f02d7dc53 [SPARK-22385][SQL] MapObjects should not access list element by index
## What changes were proposed in this pull request?

This issue was discovered and investigated by Ohad Raviv and Sean Owen in https://issues.apache.org/jira/browse/SPARK-21657. The input data of `MapObjects` may be a `List` which has O(n) complexity for accessing by index. When converting input data to catalyst array, `MapObjects` gets element by index in each loop, and results to bad performance.

This PR fixes this issue by accessing elements via Iterator.

## How was this patch tested?

using the test script in https://issues.apache.org/jira/browse/SPARK-21657
```
val BASE = 100000000
val N = 100000
val df = sc.parallelize(List(("1234567890", (BASE to (BASE+N)).map(x => (x.toString, (x+1).toString, (x+2).toString, (x+3).toString)).toList ))).toDF("c1", "c_arr")
spark.time(df.queryExecution.toRdd.foreach(_ => ()))
```

We can see 50x speed up.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19603 from cloud-fan/map-objects.
2017-10-30 11:00:44 +01:00
Henry Robinson 9f5c77ae32 [SPARK-21983][SQL] Fix Antlr 4.7 deprecation warnings
## What changes were proposed in this pull request?

Fix three deprecation warnings introduced by move to ANTLR 4.7:

* Use ParserRuleContext.addChild(TerminalNode) in preference to
  deprecated ParserRuleContext.addChild(Token) interface.
* TokenStream.reset() is deprecated in favour of seek(0)
* Replace use of deprecated ANTLRInputStream with stream returned by
  CharStreams.fromString()

The last item changed the way we construct ANTLR's input stream (from
direct instantiation to factory construction), so necessitated a change
to how we override the LA() method to always return an upper-case
char. The ANTLR object is now wrapped, rather than inherited-from.

* Also fix incorrect usage of CharStream.getText() which expects the rhs
  of the supplied interval to be the last char to be returned, i.e. the
  interval is inclusive, and work around bug in ANTLR 4.7 where empty
  streams or intervals may cause getText() to throw an error.

## How was this patch tested?

Ran all the sql tests. Confirmed that LA() override has coverage by
breaking it, and noting that tests failed.

Author: Henry Robinson <henry@apache.org>

Closes #19578 from henryr/spark-21983.
2017-10-30 07:45:54 +00:00
tengpeng 6eda55f728 Added more information to Imputer
Often times we want to impute custom values other than 'NaN'. My addition helps people locate this function without reading the API.

## What changes were proposed in this pull request?

(Please fill in changes proposed in this fix)

## How was this patch tested?

(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)
(If this patch involves UI changes, please attach a screenshot; otherwise, remove this)

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

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

Closes #19600 from tengpeng/patch-5.
2017-10-30 07:24:55 +00:00
hyukjinkwon 188b47e683 [SPARK-22379][PYTHON] Reduce duplication setUpClass and tearDownClass in PySpark SQL tests
## What changes were proposed in this pull request?

This PR propose to add `ReusedSQLTestCase` which deduplicate `setUpClass` and  `tearDownClass` in `sql/tests.py`.

## How was this patch tested?

Jenkins tests and manual tests.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #19595 from HyukjinKwon/reduce-dupe.
2017-10-30 11:50:22 +09:00
Shivaram Venkataraman 1fe27612d7 [SPARK-22344][SPARKR] Set java.io.tmpdir for SparkR tests
This PR sets the java.io.tmpdir for CRAN checks and also disables the hsperfdata for the JVM when running CRAN checks. Together this prevents files from being left behind in `/tmp`

## How was this patch tested?
Tested manually on a clean EC2 machine

Author: Shivaram Venkataraman <shivaram@cs.berkeley.edu>

Closes #19589 from shivaram/sparkr-tmpdir-clean.
2017-10-29 18:53:47 -07:00
gatorsmile 659acf18da Revert "[SPARK-22308] Support alternative unit testing styles in external applications"
This reverts commit 592cfeab9c.
2017-10-29 10:37:25 -07:00
Jen-Ming Chung bc7ca9786e [SPARK-22291][SQL] Conversion error when transforming array types of uuid, inet and cidr to StingType in PostgreSQL
## What changes were proposed in this pull request?

This PR fixes the conversion error when reads data from a PostgreSQL table that contains columns of `uuid[]`, `inet[]` and `cidr[]` data types.

For example, create a table with the uuid[] data type, and insert the test data.
```SQL
CREATE TABLE users
(
    id smallint NOT NULL,
    name character varying(50),
    user_ids uuid[],
    PRIMARY KEY (id)
)

INSERT INTO users ("id", "name","user_ids")
VALUES (1, 'foo', ARRAY
    ['7be8aaf8-650e-4dbb-8186-0a749840ecf2'
    ,'205f9bfc-018c-4452-a605-609c0cfad228']::UUID[]
)
```
Then it will throw the following exceptions when trying to load the data.
```
java.lang.ClassCastException: [Ljava.util.UUID; cannot be cast to [Ljava.lang.String;
    at org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils$$anonfun$14.apply(JdbcUtils.scala:459)
    at org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils$$anonfun$14.apply(JdbcUtils.scala:458)
...
```

## How was this patch tested?

Added test in `PostgresIntegrationSuite`.

Author: Jen-Ming Chung <jenmingisme@gmail.com>

Closes #19567 from jmchung/SPARK-22291.
2017-10-29 18:11:48 +01:00
Xin Lu 544a1ba678 [SPARK-22375][TEST] Test script can fail if eggs are installed by set…
…up.py during test process

## What changes were proposed in this pull request?

Ignore the python/.eggs folder when running lint-python

## How was this patch tested?
1) put a bad python file in python/.eggs and ran the original script.  results were:

xins-MBP:spark xinlu$ dev/lint-python
PEP8 checks failed.
./python/.eggs/worker.py:33:4: E121 continuation line under-indented for hanging indent
./python/.eggs/worker.py:34:5: E131 continuation line unaligned for hanging indent

2) test same situation with change:

xins-MBP:spark xinlu$ dev/lint-python
PEP8 checks passed.
The sphinx-build command was not found. Skipping pydoc checks for now

Author: Xin Lu <xlu@salesforce.com>

Closes #19597 from xynny/SPARK-22375.
2017-10-29 15:29:23 +09:00
Wenchen Fan 7fdacbc77b [SPARK-19727][SQL][FOLLOWUP] Fix for round function that modifies original column
## What changes were proposed in this pull request?

This is a followup of https://github.com/apache/spark/pull/17075 , to fix the bug in codegen path.

## How was this patch tested?

new regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19576 from cloud-fan/bug.
2017-10-28 18:24:18 -07:00
Juliusz Sompolski e80da8129a [MINOR] Remove false comment from planStreamingAggregation
## What changes were proposed in this pull request?

AggUtils.planStreamingAggregation has some comments about DISTINCT aggregates,
while streaming aggregation does not support DISTINCT.
This seems to have been wrongly copy-pasted over.

## How was this patch tested?

Only a comment change.

Author: Juliusz Sompolski <julek@databricks.com>

Closes #18937 from juliuszsompolski/streaming-agg-doc.
2017-10-28 17:20:35 -07:00
Takuya UESHIN 4c5269f1aa [SPARK-22370][SQL][PYSPARK] Config values should be captured in Driver.
## What changes were proposed in this pull request?

`ArrowEvalPythonExec` and `FlatMapGroupsInPandasExec` are refering config values of `SQLConf` in function for `mapPartitions`/`mapPartitionsInternal`, but we should capture them in Driver.

## How was this patch tested?

Added a test and existing tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #19587 from ueshin/issues/SPARK-22370.
2017-10-28 18:33:09 +01:00
Liang-Chi Hsieh 683ffe0620 [SPARK-22335][SQL] Clarify union behavior on Dataset of typed objects in the document
## What changes were proposed in this pull request?

Seems that end users can be confused by the union's behavior on Dataset of typed objects. We can clarity it more in the document of `union` function.

## How was this patch tested?

Only document change.

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

Closes #19570 from viirya/SPARK-22335.
2017-10-28 21:47:15 +09:00
Reynold Xin d28d5732ae [SPARK-21619][SQL] Fail the execution of canonicalized plans explicitly
## What changes were proposed in this pull request?
Canonicalized plans are not supposed to be executed. I ran into a case in which there's some code that accidentally calls execute on a canonicalized plan. This patch throws a more explicit exception when that happens.

## How was this patch tested?
Added a test case in SparkPlanSuite.

Author: Reynold Xin <rxin@databricks.com>

Closes #18828 from rxin/SPARK-21619.
2017-10-27 23:44:24 -07:00
donnyzone c42d208e19 [SPARK-22333][SQL] timeFunctionCall(CURRENT_DATE, CURRENT_TIMESTAMP) has conflicts with columnReference
## What changes were proposed in this pull request?
https://issues.apache.org/jira/browse/SPARK-22333

In current version, users can use CURRENT_DATE() and CURRENT_TIMESTAMP() without specifying braces.
However, when a table has columns named as "current_date" or "current_timestamp", it will still be parsed as function call.

There are many such cases in our production cluster. We get the wrong answer due to this inappropriate behevior. In general, ColumnReference should get higher priority than timeFunctionCall.

## How was this patch tested?
unit test
manul test

Author: donnyzone <wellfengzhu@gmail.com>

Closes #19559 from DonnyZone/master.
2017-10-27 23:40:59 -07:00
Sathiya 01f6ba0e7a [SPARK-22181][SQL] Adds ReplaceExceptWithFilter rule
## What changes were proposed in this pull request?

Adds a new optimisation rule 'ReplaceExceptWithNotFilter' that replaces Except logical with Filter operator and schedule it before applying 'ReplaceExceptWithAntiJoin' rule. This way we can avoid expensive join operation if one or both of the datasets of the Except operation are fully derived out of Filters from a same parent.

## How was this patch tested?

The patch is tested locally using spark-shell + unit test.

Author: Sathiya <sathiya.kumar@polytechnique.edu>

Closes #19451 from sathiyapk/SPARK-22181-optimize-exceptWithFilter.
2017-10-27 18:57:08 -07:00
WeichenXu 20eb95e5e9 [SPARK-21911][ML][PYSPARK] Parallel Model Evaluation for ML Tuning in PySpark
## What changes were proposed in this pull request?

Add parallelism support for ML tuning in pyspark.

## How was this patch tested?

Test updated.

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

Closes #19122 from WeichenXu123/par-ml-tuning-py.
2017-10-27 15:19:27 -07:00
Marco Gaido b3d8fc3dc4 [SPARK-22226][SQL] splitExpression can create too many method calls in the outer class
## What changes were proposed in this pull request?

SPARK-18016 introduced `NestedClass` to avoid that the many methods generated by `splitExpressions` contribute to the outer class' constant pool, making it growing too much. Unfortunately, despite their definition is stored in the `NestedClass`, they all are invoked in the outer class and for each method invocation, there are two entries added to the constant pool: a `Methodref` and a `Utf8` entry (you can easily check this compiling a simple sample class with `janinoc` and looking at its Constant Pool). This limits the scalability of the solution with very large methods which are split in a lot of small ones. This means that currently we are generating classes like this one:

```
class SpecificUnsafeProjection extends org.apache.spark.sql.catalyst.expressions.UnsafeProjection {
...
  public UnsafeRow apply(InternalRow i) {
     rowWriter.zeroOutNullBytes();
     apply_0(i);
     apply_1(i);
...
    nestedClassInstance.apply_862(i);
    nestedClassInstance.apply_863(i);
...
    nestedClassInstance1.apply_1612(i);
    nestedClassInstance1.apply_1613(i);
...
  }
...
  private class NestedClass {
    private void apply_862(InternalRow i) { ... }
    private void apply_863(InternalRow i) { ... }
...
  }
  private class NestedClass1 {
    private void apply_1612(InternalRow i) { ... }
    private void apply_1613(InternalRow i) { ... }
...
  }
}
```

This PR reduce the Constant Pool size of the outer class by adding a new method to each nested class: in this method we invoke all the small methods generated by `splitExpression` in that nested class. In this way, in the outer class there is only one method invocation per nested class, reducing by orders of magnitude the entries in its constant pool because of method invocations. This means that after the patch the generated code becomes:

```
class SpecificUnsafeProjection extends org.apache.spark.sql.catalyst.expressions.UnsafeProjection {
...
  public UnsafeRow apply(InternalRow i) {
     rowWriter.zeroOutNullBytes();
     apply_0(i);
     apply_1(i);
     ...
     nestedClassInstance.apply(i);
     nestedClassInstance1.apply(i);
     ...
  }
...
  private class NestedClass {
    private void apply_862(InternalRow i) { ... }
    private void apply_863(InternalRow i) { ... }
...
    private void apply(InternalRow i) {
      apply_862(i);
      apply_863(i);
      ...
    }
  }
  private class NestedClass1 {
    private void apply_1612(InternalRow i) { ... }
    private void apply_1613(InternalRow i) { ... }
...
    private void apply(InternalRow i) {
      apply_1612(i);
      apply_1613(i);
      ...
    }
  }
}
```

## How was this patch tested?

Added UT and existing UTs

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

Closes #19480 from mgaido91/SPARK-22226.
2017-10-27 13:43:09 -07:00
gatorsmile 36b826f5d1 [TRIVIAL][SQL] Code cleaning in ResolveReferences
## What changes were proposed in this pull request?
This PR is to clean the related codes majorly based on the today's code review on  https://github.com/apache/spark/pull/19559

## How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #19585 from gatorsmile/trivialFixes.
2017-10-27 07:52:10 -07:00
Bryan Cutler 17af727e38 [SPARK-21375][PYSPARK][SQL] Add Date and Timestamp support to ArrowConverters for toPandas() Conversion
## What changes were proposed in this pull request?

Adding date and timestamp support with Arrow for `toPandas()` and `pandas_udf`s.  Timestamps are stored in Arrow as UTC and manifested to the user as timezone-naive localized to the Python system timezone.

## How was this patch tested?

Added Scala tests for date and timestamp types under ArrowConverters, ArrowUtils, and ArrowWriter suites.  Added Python tests for `toPandas()` and `pandas_udf`s with date and timestamp types.

Author: Bryan Cutler <cutlerb@gmail.com>
Author: Takuya UESHIN <ueshin@databricks.com>

Closes #18664 from BryanCutler/arrow-date-timestamp-SPARK-21375.
2017-10-26 23:02:46 -07:00
Wenchen Fan 5c3a1f3fad [SPARK-22355][SQL] Dataset.collect is not threadsafe
## What changes were proposed in this pull request?

It's possible that users create a `Dataset`, and call `collect` of this `Dataset` in many threads at the same time. Currently `Dataset#collect` just call `encoder.fromRow` to convert spark rows to objects of type T, and this encoder is per-dataset. This means `Dataset#collect` is not thread-safe, because the encoder uses a projection to output the object to a re-usable row.

This PR fixes this problem, by creating a new projection when calling `Dataset#collect`, so that we have the re-usable row for each method call, instead of each Dataset.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19577 from cloud-fan/encoder.
2017-10-26 17:51:16 -07:00
Wenchen Fan 9b262f6a08 [SPARK-22356][SQL] data source table should support overlapped columns between data and partition schema
## What changes were proposed in this pull request?

This is a regression introduced by #14207. After Spark 2.1, we store the inferred schema when creating the table, to avoid inferring schema again at read path. However, there is one special case: overlapped columns between data and partition. For this case, it breaks the assumption of table schema that there is on ovelap between data and partition schema, and partition columns should be at the end. The result is, for Spark 2.1, the table scan has incorrect schema that puts partition columns at the end. For Spark 2.2, we add a check in CatalogTable to validate table schema, which fails at this case.

To fix this issue, a simple and safe approach is to fallback to old behavior when overlapeed columns detected, i.e. store empty schema in metastore.

## How was this patch tested?

new regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #19579 from cloud-fan/bug2.
2017-10-26 17:39:53 -07:00