spark.git
8 hours ago[SPARK-26283][CORE] Enable reading from open frames of zstd, when reading zstd compre... master
Shahid [Sun, 9 Dec 2018 17:44:16 +0000 (11:44 -0600)] 
[SPARK-26283][CORE] Enable reading from open frames of zstd, when reading zstd compressed eventLog

## What changes were proposed in this pull request?
Root cause: Prior to Spark2.4, When we enable zst for eventLog compression, for inprogress application, It always throws exception in the Application UI, when we open from the history server. But after 2.4 it will display the UI information based on the completed frames in the zstd compressed eventLog. But doesn't read incomplete frames for inprogress application.
In this PR, we have added 'setContinous(true)' for reading input stream from eventLog, so that it can read from open frames also. (By default 'isContinous=false' for zstd inputStream and when we try to read an open frame, it throws truncated error)

## How was this patch tested?
Test steps:
1) Add the configurations in the spark-defaults.conf
   (i) spark.eventLog.compress true
   (ii) spark.io.compression.codec zstd
2) Restart history server
3) bin/spark-shell
4) sc.parallelize(1 to 1000, 1000).count
5) Open app UI from the history server UI

**Before fix**
![screenshot from 2018-12-06 00-01-38](https://user-images.githubusercontent.com/23054875/49537340-bfe28b00-f8ee-11e8-9fca-6d42fdc89e1a.png)

**After fix:**
![screenshot from 2018-12-06 00-34-39](https://user-images.githubusercontent.com/23054875/49537353-ca9d2000-f8ee-11e8-803d-645897b9153b.png)

Closes #23241 from shahidki31/zstdEventLog.

Authored-by: Shahid <shahidki31@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
23 hours ago[SPARK-26193][SQL] Implement shuffle write metrics in SQL
Yuanjian Li [Sun, 9 Dec 2018 02:49:15 +0000 (10:49 +0800)] 
[SPARK-26193][SQL] Implement shuffle write metrics in SQL

## What changes were proposed in this pull request?

1. Implement `SQLShuffleWriteMetricsReporter` on the SQL side as the customized `ShuffleWriteMetricsReporter`.
2. Add shuffle write metrics to `ShuffleExchangeExec`, and use these metrics to create corresponding `SQLShuffleWriteMetricsReporter` in shuffle dependency.
3. Rework on `ShuffleMapTask` to add new class named `ShuffleWriteProcessor` which control shuffle write process, we use sql shuffle write metrics by customizing a ShuffleWriteProcessor on SQL side.

## How was this patch tested?
Add UT in SQLMetricsSuite.
Manually test locally, update screen shot to document attached in JIRA.

Closes #23207 from xuanyuanking/SPARK-26193.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
23 hours ago[SPARK-25132][SQL][FOLLOWUP][DOC] Add migration doc for case-insensitive field resolu...
seancxmao [Sun, 9 Dec 2018 01:53:12 +0000 (17:53 -0800)] 
[SPARK-25132][SQL][FOLLOWUP][DOC] Add migration doc for case-insensitive field resolution when reading from Parquet

## What changes were proposed in this pull request?
#22148 introduces a behavior change. According to discussion at #22184, this PR updates migration guide when upgrade from Spark 2.3 to 2.4.

## How was this patch tested?
N/A

Closes #23238 from seancxmao/SPARK-25132-doc-2.4.

Authored-by: seancxmao <seancxmao@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
30 hours ago[SPARK-26021][SQL][FOLLOWUP] only deal with NaN and -0.0 in UnsafeWriter
Wenchen Fan [Sat, 8 Dec 2018 19:18:09 +0000 (11:18 -0800)] 
[SPARK-26021][SQL][FOLLOWUP] only deal with NaN and -0.0 in UnsafeWriter

## What changes were proposed in this pull request?

A followup of https://github.com/apache/spark/pull/23043

There are 4 places we need to deal with NaN and -0.0:
1. comparison expressions. `-0.0` and `0.0` should be treated as same. Different NaNs should be treated as same.
2. Join keys. `-0.0` and `0.0` should be treated as same. Different NaNs should be treated as same.
3. grouping keys. `-0.0` and `0.0` should be assigned to the same group. Different NaNs should be assigned to the same group.
4. window partition keys. `-0.0` and `0.0` should be treated as same. Different NaNs should be treated as same.

The case 1 is OK. Our comparison already handles NaN and -0.0, and for struct/array/map, we will recursively compare the fields/elements.

Case 2, 3 and 4 are problematic, as they compare `UnsafeRow` binary directly, and different NaNs have different binary representation, and the same thing happens for -0.0 and 0.0.

To fix it, a simple solution is: normalize float/double when building unsafe data (`UnsafeRow`, `UnsafeArrayData`, `UnsafeMapData`). Then we don't need to worry about it anymore.

Following this direction, this PR moves the handling of NaN and -0.0 from `Platform` to `UnsafeWriter`, so that places like `UnsafeRow.setFloat` will not handle them, which reduces the perf overhead. It's also easier to add comments explaining why we do it in `UnsafeWriter`.

## How was this patch tested?

existing tests

Closes #23239 from cloud-fan/minor.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
35 hours ago[SPARK-24207][R] follow-up PR for SPARK-24207 to fix code style problems
Huaxin Gao [Sat, 8 Dec 2018 14:23:50 +0000 (22:23 +0800)] 
[SPARK-24207][R] follow-up PR for SPARK-24207 to fix code style problems

## What changes were proposed in this pull request?

follow-up PR for SPARK-24207 to fix code style problems

Closes #23256 from huaxingao/spark-24207-cnt.

Authored-by: Huaxin Gao <huaxing@us.ibm.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
37 hours ago[SPARK-26266][BUILD] Update to Scala 2.12.8
Sean Owen [Sat, 8 Dec 2018 11:59:53 +0000 (05:59 -0600)] 
[SPARK-26266][BUILD] Update to Scala 2.12.8

## What changes were proposed in this pull request?

Update to Scala 2.12.8

## How was this patch tested?

Existing tests.

Closes #23218 from srowen/SPARK-26266.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2 days ago[SPARK-26304][SS] Add default value to spark.kafka.sasl.kerberos.service.name parameter
Gabor Somogyi [Fri, 7 Dec 2018 21:58:02 +0000 (13:58 -0800)] 
[SPARK-26304][SS] Add default value to spark.kafka.sasl.kerberos.service.name parameter

## What changes were proposed in this pull request?

spark.kafka.sasl.kerberos.service.name is an optional parameter but most of the time value `kafka` has to be set. As I've written in the jira the following reasoning is behind:
* Kafka's configuration guide suggest the same value: https://kafka.apache.org/documentation/#security_sasl_kerberos_brokerconfig
* It would be easier for spark users by providing less configuration
* Other streaming engines are doing the same

In this PR I've changed the parameter from optional to `WithDefault` and set `kafka` as default value.

## How was this patch tested?

Available unit tests + on cluster.

Closes #23254 from gaborgsomogyi/SPARK-26304.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2 days ago[SPARK-24333][ML][PYTHON] Add fit with validation set to spark.ml GBT: Python API
Huaxin Gao [Fri, 7 Dec 2018 21:53:35 +0000 (13:53 -0800)] 
[SPARK-24333][ML][PYTHON] Add fit with validation set to spark.ml GBT: Python API

## What changes were proposed in this pull request?

Add validationIndicatorCol and validationTol to GBT Python.

## How was this patch tested?

Add test in doctest to test the new API.

Closes #21465 from huaxingao/spark-24333.

Authored-by: Huaxin Gao <huaxing@us.ibm.com>
Signed-off-by: Bryan Cutler <cutlerb@gmail.com>
2 days ago[SPARK-26196][SPARK-26281][WEBUI] Total tasks title in the stage page is incorrect...
Shahid [Fri, 7 Dec 2018 20:31:35 +0000 (14:31 -0600)] 
[SPARK-26196][SPARK-26281][WEBUI] Total tasks title in the stage page is incorrect when there are failed or killed tasks and update duration metrics

## What changes were proposed in this pull request?
This PR fixes 3 issues
1) Total tasks message in the tasks table is incorrect, when there are failed or killed tasks
2) Sorting of the "Duration" column is not correct
3) Duration in the aggregated tasks summary table and the tasks table and not matching.

Total tasks  = numCompleteTasks +  numActiveTasks + numKilledTasks + numFailedTasks;

Corrected the duration metrics in the tasks table as executorRunTime based on the PR https://github.com/apache/spark/pull/23081

## How was this patch tested?
test step:
1)
```
bin/spark-shell
scala > sc.parallelize(1 to 100, 10).map{ x => throw new RuntimeException("Bad executor")}.collect()
```
![screenshot from 2018-11-28 07-26-00](https://user-images.githubusercontent.com/23054875/49123523-e2691880-f2de-11e8-9c16-60d1865e6e77.png)

After patch:
![screenshot from 2018-11-28 07-24-31](https://user-images.githubusercontent.com/23054875/49123525-e432dc00-f2de-11e8-89ca-4a53e19c9c18.png)

2)  Duration metrics:
Before patch:
![screenshot from 2018-12-06 03-25-14](https://user-images.githubusercontent.com/23054875/49546591-9e8d9900-f906-11e8-8a0b-157742c47655.png)

After patch:
![screenshot from 2018-12-06 03-23-14](https://user-images.githubusercontent.com/23054875/49546589-9cc3d580-f906-11e8-827f-52ef8ffdeaec.png)

Closes #23160 from shahidki31/totalTasks.

Authored-by: Shahid <shahidki31@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2 days ago[MINOR][SQL][DOC] Correct parquet nullability documentation
dima-asana [Fri, 7 Dec 2018 20:14:43 +0000 (14:14 -0600)] 
[MINOR][SQL][DOC] Correct parquet nullability documentation

## What changes were proposed in this pull request?

Parquet files appear to have nullability info when being written, not being read.

## How was this patch tested?

Some test code: (running spark 2.3, but the relevant code in DataSource looks identical on master)

case class NullTest(bo: Boolean, opbol: Option[Boolean])
val testDf = spark.createDataFrame(Seq(NullTest(true, Some(false))))

defined class NullTest
testDf: org.apache.spark.sql.DataFrame = [bo: boolean, opbol: boolean]

testDf.write.parquet("s3://asana-stats/tmp_dima/parquet_check_schema")

spark.read.parquet("s3://asana-stats/tmp_dima/parquet_check_schema/part-00000-b1bf4a19-d9fe-4ece-a2b4-9bbceb490857-c000.snappy.parquet4").printSchema()
root
 |-- bo: boolean (nullable = true)
 |-- opbol: boolean (nullable = true)

Meanwhile, the parquet file formed does have nullable info:

[]batchprod-report000:/tmp/dimakamalov-batch$ aws s3 ls s3://asana-stats/tmp_dima/parquet_check_schema/
2018-10-17 21:03:52          0 _SUCCESS
2018-10-17 21:03:50        504 part-00000-b1bf4a19-d9fe-4ece-a2b4-9bbceb490857-c000.snappy.parquet
[]batchprod-report000:/tmp/dimakamalov-batch$ aws s3 cp s3://asana-stats/tmp_dima/parquet_check_schema/part-00000-b1bf4a19-d9fe-4ece-a2b4-9bbceb490857-c000.snappy.parquet .
download: s3://asana-stats/tmp_dima/parquet_check_schema/part-00000-b1bf4a19-d9fe-4ece-a2b4-9bbceb490857-c000.snappy.parquet to ./part-00000-b1bf4a19-d9fe-4ece-a2b4-9bbceb490857-c000.snappy.parquet
[]batchprod-report000:/tmp/dimakamalov-batch$ java -jar parquet-tools-1.8.2.jar schema part-00000-b1bf4a19-d9fe-4ece-a2b4-9bbceb490857-c000.snappy.parquet
message spark_schema {
  required boolean bo;
  optional boolean opbol;
}

Closes #22759 from dima-asana/dima-asana-nullable-parquet-doc.

Authored-by: dima-asana <42555784+dima-asana@users.noreply.github.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2 days ago[SPARK-26294][CORE] Delete Unnecessary If statement
10087686 [Fri, 7 Dec 2018 20:11:25 +0000 (14:11 -0600)] 
[SPARK-26294][CORE] Delete Unnecessary If statement

## What changes were proposed in this pull request?
Delete unnecessary If statement, because it Impossible execution when
records less than or equal to zero.it is only execution when records begin zero.
...................
if (inMemSorter == null || inMemSorter.numRecords() <= 0) {
       return 0L;
 }
....................
if (inMemSorter.numRecords() > 0) {
.....................
}
## How was this patch tested?
Existing tests

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

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

Closes #23247 from wangjiaochun/inMemSorter.

Authored-by: 10087686 <wang.jiaochun@zte.com.cn>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2 days ago[SPARK-24243][CORE] Expose exceptions from InProcessAppHandle
Sahil Takiar [Fri, 7 Dec 2018 18:33:42 +0000 (10:33 -0800)] 
[SPARK-24243][CORE] Expose exceptions from InProcessAppHandle

Adds a new method to SparkAppHandle called getError which returns
the exception (if present) that caused the underlying Spark app to
fail.

New tests added to SparkLauncherSuite for the new method.

Closes #21849

Closes #23221 from vanzin/SPARK-24243.

Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2 days ago[SPARK-26060][SQL][FOLLOW-UP] Rename the config name.
Takuya UESHIN [Fri, 7 Dec 2018 15:55:54 +0000 (07:55 -0800)] 
[SPARK-26060][SQL][FOLLOW-UP] Rename the config name.

## What changes were proposed in this pull request?

This is a follow-up of #23031 to rename the config name to `spark.sql.legacy.setCommandRejectsSparkCoreConfs`.

## How was this patch tested?

Existing tests.

Closes #23245 from ueshin/issues/SPARK-26060/rename_config.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2 days ago[SPARK-26298][BUILD] Upgrade Janino to 3.0.11
Dongjoon Hyun [Fri, 7 Dec 2018 04:50:57 +0000 (20:50 -0800)] 
[SPARK-26298][BUILD] Upgrade Janino to 3.0.11

## What changes were proposed in this pull request?

This PR aims to upgrade Janino compiler to the latest version 3.0.11. The followings are the changes from the [release note](http://janino-compiler.github.io/janino/changelog.html).

- Script with many "helper" variables.
- Java 9+ compatibility
- Compilation Error Messages Generated by JDK.
- Added experimental support for the "StackMapFrame" attribute; not active yet.
- Make Unparser more flexible.
- Fixed NPEs in various "toString()" methods.
- Optimize static method invocation with rvalue target expression.
- Added all missing "ClassFile.getConstant*Info()" methods, removing the necessity for many type casts.

## How was this patch tested?

Pass the Jenkins with the existing tests.

Closes #23250 from dongjoon-hyun/SPARK-26298.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2 days ago[SPARK-26263][SQL] Validate partition values with user provided schema
Gengliang Wang [Fri, 7 Dec 2018 03:13:14 +0000 (11:13 +0800)] 
[SPARK-26263][SQL] Validate partition values with user provided schema

## What changes were proposed in this pull request?

Currently if user provides data schema, partition column values are converted as per it. But if the conversion failed, e.g. converting string to int, the column value is null.

This PR proposes to throw exception in such case, instead of converting into null value silently:
1. These null partition column values doesn't make sense to users in most cases. It is better to show the conversion failure, and then users can adjust the schema or ETL jobs to fix it.
2. There are always exceptions on such conversion failure for non-partition data columns. Partition columns should have the same behavior.

We can reproduce the case above as following:
```
/tmp/testDir
├── p=bar
└── p=foo
```
If we run:
```
val schema = StructType(Seq(StructField("p", IntegerType, false)))
spark.read.schema(schema).csv("/tmp/testDir/").show()
```
We will get:
```
+----+
|   p|
+----+
|null|
|null|
+----+
```

## How was this patch tested?

Unit test

Closes #23215 from gengliangwang/SPARK-26263.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2 days ago[SPARK-26289][CORE] cleanup enablePerfMetrics parameter from BytesToBytesMap
caoxuewen [Fri, 7 Dec 2018 01:57:35 +0000 (09:57 +0800)] 
[SPARK-26289][CORE] cleanup enablePerfMetrics parameter from BytesToBytesMap

## What changes were proposed in this pull request?

`enablePerfMetrics `was originally designed in `BytesToBytesMap `to control `getNumHashCollisions  getTimeSpentResizingNs  getAverageProbesPerLookup`.

However, as the Spark version gradual progress.  this parameter is only used for `getAverageProbesPerLookup ` and always given to true when using `BytesToBytesMap`.

 it is also dangerous to determine whether `getAverageProbesPerLookup `opens and throws an `IllegalStateException `exception.
So this pr will be remove `enablePerfMetrics `parameter from `BytesToBytesMap`. thanks.

## How was this patch tested?

the existed test cases.

Closes #23244 from heary-cao/enablePerfMetrics.

Authored-by: caoxuewen <cao.xuewen@zte.com.cn>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
3 days ago[SPARK-26194][K8S] Auto generate auth secret for k8s apps.
Marcelo Vanzin [Thu, 6 Dec 2018 22:17:13 +0000 (14:17 -0800)] 
[SPARK-26194][K8S] Auto generate auth secret for k8s apps.

This change modifies the logic in the SecurityManager to do two
things:

- generate unique app secrets also when k8s is being used
- only store the secret in the user's UGI on YARN

The latter is needed so that k8s won't unnecessarily create
k8s secrets for the UGI credentials when only the auth token
is stored there.

On the k8s side, the secret is propagated to executors using
an environment variable instead. This ensures it works in both
client and cluster mode.

Security doc was updated to mention the feature and clarify that
proper access control in k8s should be enabled for it to be secure.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #23174 from vanzin/SPARK-26194.

3 days ago[SPARK-26236][SS] Add kafka delegation token support documentation.
Gabor Somogyi [Thu, 6 Dec 2018 18:59:20 +0000 (10:59 -0800)] 
[SPARK-26236][SS] Add kafka delegation token support documentation.

## What changes were proposed in this pull request?

Kafka delegation token support implemented in [PR#22598](https://github.com/apache/spark/pull/22598) but that didn't contain documentation because of rapid changes. Because it has been merged in this PR I've documented it.

## How was this patch tested?
jekyll build + manual html check

Closes #23195 from gaborgsomogyi/SPARK-26236.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
3 days ago[SPARK-25274][PYTHON][SQL] In toPandas with Arrow send un-ordered record batches...
Bryan Cutler [Thu, 6 Dec 2018 18:07:28 +0000 (10:07 -0800)] 
[SPARK-25274][PYTHON][SQL] In toPandas with Arrow send un-ordered record batches to improve performance

## What changes were proposed in this pull request?

When executing `toPandas` with Arrow enabled, partitions that arrive in the JVM out-of-order must be buffered before they can be send to Python. This causes an excess of memory to be used in the driver JVM and increases the time it takes to complete because data must sit in the JVM waiting for preceding partitions to come in.

This change sends un-ordered partitions to Python as soon as they arrive in the JVM, followed by a list of partition indices so that Python can assemble the data in the correct order. This way, data is not buffered at the JVM and there is no waiting on particular partitions so performance will be increased.

Followup to #21546

## How was this patch tested?

Added new test with a large number of batches per partition, and test that forces a small delay in the first partition. These test that partitions are collected out-of-order and then are are put in the correct order in Python.

## Performance Tests - toPandas

Tests run on a 4 node standalone cluster with 32 cores total, 14.04.1-Ubuntu and OpenJDK 8
measured wall clock time to execute `toPandas()` and took the average best time of 5 runs/5 loops each.

Test code
```python
df = spark.range(1 << 25, numPartitions=32).toDF("id").withColumn("x1", rand()).withColumn("x2", rand()).withColumn("x3", rand()).withColumn("x4", rand())
for i in range(5):
start = time.time()
_ = df.toPandas()
elapsed = time.time() - start
```

Spark config
```
spark.driver.memory 5g
spark.executor.memory 5g
spark.driver.maxResultSize 2g
spark.sql.execution.arrow.enabled true
```

Current Master w/ Arrow stream | This PR
---------------------|------------
5.16207 | 4.342533
5.133671 | 4.399408
5.147513 | 4.468471
5.105243 | 4.36524
5.018685 | 4.373791

Avg Master | Avg This PR
------------------|--------------
5.1134364 | 4.3898886

Speedup of **1.164821449**

Closes #22275 from BryanCutler/arrow-toPandas-oo-batches-SPARK-25274.

Authored-by: Bryan Cutler <cutlerb@gmail.com>
Signed-off-by: Bryan Cutler <cutlerb@gmail.com>
4 days ago[SPARK-26275][PYTHON][ML] Increases timeout for StreamingLogisticRegressionWithSGDTes...
Hyukjin Kwon [Thu, 6 Dec 2018 01:14:46 +0000 (09:14 +0800)] 
[SPARK-26275][PYTHON][ML] Increases timeout for StreamingLogisticRegressionWithSGDTests.test_training_and_prediction test

## What changes were proposed in this pull request?

Looks this test is flaky

https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/99704/console
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/99569/console
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/99644/console
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/99548/console
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/99454/console
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/99609/console

```
======================================================================
FAIL: test_training_and_prediction (pyspark.mllib.tests.test_streaming_algorithms.StreamingLogisticRegressionWithSGDTests)
Test that the model improves on toy data with no. of batches
----------------------------------------------------------------------
Traceback (most recent call last):
  File "/home/jenkins/workspace/SparkPullRequestBuilder/python/pyspark/mllib/tests/test_streaming_algorithms.py", line 367, in test_training_and_prediction
    self._eventually(condition)
  File "/home/jenkins/workspace/SparkPullRequestBuilder/python/pyspark/mllib/tests/test_streaming_algorithms.py", line 78, in _eventually
    % (timeout, lastValue))
AssertionError: Test failed due to timeout after 30 sec, with last condition returning: Latest errors: 0.67, 0.71, 0.78, 0.7, 0.75, 0.74, 0.73, 0.69, 0.62, 0.71, 0.69, 0.75, 0.72, 0.77, 0.71, 0.74

----------------------------------------------------------------------
Ran 13 tests in 185.051s

FAILED (failures=1, skipped=1)
```

This looks happening after increasing the parallelism in Jenkins to speed up at https://github.com/apache/spark/pull/23111. I am able to reproduce this manually when the resource usage is heavy (with manual decrease of timeout).

## How was this patch tested?

Manually tested by

```
cd python
./run-tests --testnames 'pyspark.mllib.tests.test_streaming_algorithms StreamingLogisticRegressionWithSGDTests.test_training_and_prediction' --python-executables=python
```

Closes #23236 from HyukjinKwon/SPARK-26275.

Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
4 days ago[SPARK-26151][SQL][FOLLOWUP] Return partial results for bad CSV records
Maxim Gekk [Wed, 5 Dec 2018 15:43:03 +0000 (23:43 +0800)] 
[SPARK-26151][SQL][FOLLOWUP] Return partial results for bad CSV records

## What changes were proposed in this pull request?

Updated SQL migration guide according to changes in https://github.com/apache/spark/pull/23120

Closes #23235 from MaxGekk/failuresafe-partial-result-followup.

Lead-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Co-authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
4 days ago[SPARK-26271][FOLLOW-UP][SQL] remove unuse object SparkPlan
caoxuewen [Wed, 5 Dec 2018 15:10:48 +0000 (23:10 +0800)] 
[SPARK-26271][FOLLOW-UP][SQL] remove unuse object SparkPlan

## What changes were proposed in this pull request?

this code come from PR: https://github.com/apache/spark/pull/11190,
but this code has never been used, only since  PR: https://github.com/apache/spark/pull/14548,
Let's continue fix it. thanks.

## How was this patch tested?

N / A

Closes #23227 from heary-cao/unuseSparkPlan.

Authored-by: caoxuewen <cao.xuewen@zte.com.cn>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
4 days ago[SPARK-26133][ML][FOLLOWUP] Fix doc for OneHotEncoder
Liang-Chi Hsieh [Wed, 5 Dec 2018 11:30:25 +0000 (19:30 +0800)] 
[SPARK-26133][ML][FOLLOWUP] Fix doc for OneHotEncoder

## What changes were proposed in this pull request?

This fixes doc of renamed OneHotEncoder in PySpark.

## How was this patch tested?

N/A

Closes #23230 from viirya/remove_one_hot_encoder_followup.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
4 days ago[SPARK-26252][PYTHON] Add support to run specific unittests and/or doctests in python...
Hyukjin Kwon [Wed, 5 Dec 2018 07:22:08 +0000 (15:22 +0800)] 
[SPARK-26252][PYTHON] Add support to run specific unittests and/or doctests in python/run-tests script

## What changes were proposed in this pull request?

This PR proposes add a developer option, `--testnames`, to our testing script to allow run specific set of unittests and doctests.

**1. Run unittests in the class**

```bash
./run-tests --testnames 'pyspark.sql.tests.test_arrow ArrowTests'
```
```
Running PySpark tests. Output is in /.../spark/python/unit-tests.log
Will test against the following Python executables: ['python2.7', 'pypy']
Will test the following Python tests: ['pyspark.sql.tests.test_arrow ArrowTests']
Starting test(python2.7): pyspark.sql.tests.test_arrow ArrowTests
Starting test(pypy): pyspark.sql.tests.test_arrow ArrowTests
Finished test(python2.7): pyspark.sql.tests.test_arrow ArrowTests (14s)
Finished test(pypy): pyspark.sql.tests.test_arrow ArrowTests (14s) ... 22 tests were skipped
Tests passed in 14 seconds

Skipped tests in pyspark.sql.tests.test_arrow ArrowTests with pypy:
    test_createDataFrame_column_name_encoding (pyspark.sql.tests.test_arrow.ArrowTests) ... skipped 'Pandas >= 0.19.2 must be installed; however, it was not found.'
    test_createDataFrame_does_not_modify_input (pyspark.sql.tests.test_arrow.ArrowTests) ... skipped 'Pandas >= 0.19.2 must be installed; however, it was not found.'
    test_createDataFrame_fallback_disabled (pyspark.sql.tests.test_arrow.ArrowTests) ... skipped 'Pandas >= 0.19.2 must be installed; however, it was not found.'
    test_createDataFrame_fallback_enabled (pyspark.sql.tests.test_arrow.ArrowTests) ... skipped
...
```

**2. Run single unittest in the class.**

```bash
./run-tests --testnames 'pyspark.sql.tests.test_arrow ArrowTests.test_null_conversion'
```
```
Running PySpark tests. Output is in /.../spark/python/unit-tests.log
Will test against the following Python executables: ['python2.7', 'pypy']
Will test the following Python tests: ['pyspark.sql.tests.test_arrow ArrowTests.test_null_conversion']
Starting test(pypy): pyspark.sql.tests.test_arrow ArrowTests.test_null_conversion
Starting test(python2.7): pyspark.sql.tests.test_arrow ArrowTests.test_null_conversion
Finished test(pypy): pyspark.sql.tests.test_arrow ArrowTests.test_null_conversion (0s) ... 1 tests were skipped
Finished test(python2.7): pyspark.sql.tests.test_arrow ArrowTests.test_null_conversion (8s)
Tests passed in 8 seconds

Skipped tests in pyspark.sql.tests.test_arrow ArrowTests.test_null_conversion with pypy:
    test_null_conversion (pyspark.sql.tests.test_arrow.ArrowTests) ... skipped 'Pandas >= 0.19.2 must be installed; however, it was not found.'
```

**3. Run doctests in single PySpark module.**

```bash
./run-tests --testnames pyspark.sql.dataframe
```

```
Running PySpark tests. Output is in /.../spark/python/unit-tests.log
Will test against the following Python executables: ['python2.7', 'pypy']
Will test the following Python tests: ['pyspark.sql.dataframe']
Starting test(pypy): pyspark.sql.dataframe
Starting test(python2.7): pyspark.sql.dataframe
Finished test(python2.7): pyspark.sql.dataframe (47s)
Finished test(pypy): pyspark.sql.dataframe (48s)
Tests passed in 48 seconds
```

Of course, you can mix them:

```bash
./run-tests --testnames 'pyspark.sql.tests.test_arrow ArrowTests,pyspark.sql.dataframe'
```

```
Running PySpark tests. Output is in /.../spark/python/unit-tests.log
Will test against the following Python executables: ['python2.7', 'pypy']
Will test the following Python tests: ['pyspark.sql.tests.test_arrow ArrowTests', 'pyspark.sql.dataframe']
Starting test(pypy): pyspark.sql.dataframe
Starting test(pypy): pyspark.sql.tests.test_arrow ArrowTests
Starting test(python2.7): pyspark.sql.dataframe
Starting test(python2.7): pyspark.sql.tests.test_arrow ArrowTests
Finished test(pypy): pyspark.sql.tests.test_arrow ArrowTests (0s) ... 22 tests were skipped
Finished test(python2.7): pyspark.sql.tests.test_arrow ArrowTests (18s)
Finished test(python2.7): pyspark.sql.dataframe (50s)
Finished test(pypy): pyspark.sql.dataframe (52s)
Tests passed in 52 seconds

Skipped tests in pyspark.sql.tests.test_arrow ArrowTests with pypy:
    test_createDataFrame_column_name_encoding (pyspark.sql.tests.test_arrow.ArrowTests) ... skipped 'Pandas >= 0.19.2 must be installed; however, it was not found.'
    test_createDataFrame_does_not_modify_input (pyspark.sql.tests.test_arrow.ArrowTests) ... skipped 'Pandas >= 0.19.2 must be installed; however, it was not found.'
    test_createDataFrame_fallback_disabled (pyspark.sql.tests.test_arrow.ArrowTests) ... skipped 'Pandas >= 0.19.2 must be installed; however, it was not found.'
```

and also you can use all other options (except `--modules`, which will be ignored)

```bash
./run-tests --testnames 'pyspark.sql.tests.test_arrow ArrowTests.test_null_conversion' --python-executables=python
```

```
Running PySpark tests. Output is in /.../spark/python/unit-tests.log
Will test against the following Python executables: ['python']
Will test the following Python tests: ['pyspark.sql.tests.test_arrow ArrowTests.test_null_conversion']
Starting test(python): pyspark.sql.tests.test_arrow ArrowTests.test_null_conversion
Finished test(python): pyspark.sql.tests.test_arrow ArrowTests.test_null_conversion (12s)
Tests passed in 12 seconds
```

See help below:

```bash
 ./run-tests --help
```

```
Usage: run-tests [options]

Options:
...
  Developer Options:
    --testnames=TESTNAMES
                        A comma-separated list of specific modules, classes
                        and functions of doctest or unittest to test. For
                        example, 'pyspark.sql.foo' to run the module as
                        unittests or doctests, 'pyspark.sql.tests FooTests' to
                        run the specific class of unittests,
                        'pyspark.sql.tests FooTests.test_foo' to run the
                        specific unittest in the class. '--modules' option is
                        ignored if they are given.
```

I intentionally grouped it as a developer option to be more conservative.

## How was this patch tested?

Manually tested. Negative tests were also done.

```bash
./run-tests --testnames 'pyspark.sql.tests.test_arrow ArrowTests.test_null_conversion1' --python-executables=python
```

```
...
AttributeError: type object 'ArrowTests' has no attribute 'test_null_conversion1'
...
```

```bash
./run-tests --testnames 'pyspark.sql.tests.test_arrow ArrowT' --python-executables=python
```

```
...
AttributeError: 'module' object has no attribute 'ArrowT'
...
```

```bash
 ./run-tests --testnames 'pyspark.sql.tests.test_ar' --python-executables=python
```
```
...
/.../python2.7: No module named pyspark.sql.tests.test_ar
```

Closes #23203 from HyukjinKwon/SPARK-26252.

Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
5 days ago[SPARK-25829][SQL][FOLLOWUP] Refactor MapConcat in order to check properly the limit...
Marco Gaido [Wed, 5 Dec 2018 01:12:24 +0000 (09:12 +0800)] 
[SPARK-25829][SQL][FOLLOWUP] Refactor MapConcat in order to check properly the limit size

## What changes were proposed in this pull request?

The PR starts from the [comment](https://github.com/apache/spark/pull/23124#discussion_r236112390) in the main one and it aims at:
 - simplifying the code for `MapConcat`;
 - be more precise in checking the limit size.

## How was this patch tested?

existing tests

Closes #23217 from mgaido91/SPARK-25829_followup.

Authored-by: Marco Gaido <marcogaido91@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
5 days ago[SPARK-26094][CORE][STREAMING] createNonEcFile creates parent dirs.
Imran Rashid [Tue, 4 Dec 2018 22:35:04 +0000 (14:35 -0800)] 
[SPARK-26094][CORE][STREAMING] createNonEcFile creates parent dirs.

## What changes were proposed in this pull request?

We explicitly avoid files with hdfs erasure coding for the streaming WAL
and for event logs, as hdfs EC does not support all relevant apis.
However, the new builder api used has different semantics -- it does not
create parent dirs, and it does not resolve relative paths.  This
updates createNonEcFile to have similar semantics to the old api.

## How was this patch tested?

Ran tests with the WAL pointed at a non-existent dir, which failed before this change.  Manually tested the new function with a relative path as well.
Unit tests via jenkins.

Closes #23092 from squito/SPARK-26094.

Authored-by: Imran Rashid <irashid@cloudera.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
5 days ago[SPARK-26119][CORE][WEBUI] Task summary table should contain only successful tasks...
Shahid [Tue, 4 Dec 2018 19:00:58 +0000 (11:00 -0800)] 
[SPARK-26119][CORE][WEBUI] Task summary table should contain only successful tasks' metrics

## What changes were proposed in this pull request?

Task summary table in the stage page currently displays the summary of all the tasks. However, we should display the task summary of only successful tasks, to follow the behavior of previous versions of spark.

## How was this patch tested?
Added UT. attached screenshot
Before patch:
![screenshot from 2018-11-20 00-36-18](https://user-images.githubusercontent.com/23054875/48729339-62e3a580-ec5d-11e8-81f0-0d191a234ffe.png)

![screenshot from 2018-11-20 01-18-37](https://user-images.githubusercontent.com/23054875/48731112-41d18380-ec62-11e8-8c31-1ffbfa04e746.png)

Closes #23088 from shahidki31/summaryMetrics.

Authored-by: Shahid <shahidki31@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
5 days ago[SPARK-26233][SQL] CheckOverflow when encoding a decimal value
Marco Gaido [Tue, 4 Dec 2018 18:33:27 +0000 (10:33 -0800)] 
[SPARK-26233][SQL] CheckOverflow when encoding a decimal value

## What changes were proposed in this pull request?

When we encode a Decimal from external source we don't check for overflow. That method is useful not only in order to enforce that we can represent the correct value in the specified range, but it also changes the underlying data to the right precision/scale. Since in our code generation we assume that a decimal has exactly the same precision and scale of its data type, missing to enforce it can lead to corrupted output/results when there are subsequent transformations.

## How was this patch tested?

added UT

Closes #23210 from mgaido91/SPARK-26233.

Authored-by: Marco Gaido <marcogaido91@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
5 days ago[SPARK-26178][SQL] Use java.time API for parsing timestamps and dates from CSV
Maxim Gekk [Tue, 4 Dec 2018 14:36:33 +0000 (08:36 -0600)] 
[SPARK-26178][SQL] Use java.time API for parsing timestamps and dates from CSV

## What changes were proposed in this pull request?

In the PR, I propose to use **java.time API** for parsing timestamps and dates from CSV content with microseconds precision. The SQL config `spark.sql.legacy.timeParser.enabled` allow to switch back to previous behaviour with using `java.text.SimpleDateFormat`/`FastDateFormat` for parsing/generating timestamps/dates.

## How was this patch tested?

It was tested by `UnivocityParserSuite`, `CsvExpressionsSuite`, `CsvFunctionsSuite` and `CsvSuite`.

Closes #23150 from MaxGekk/time-parser.

Lead-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
5 days ago[SPARK-24423][FOLLOW-UP][SQL] Fix error example
Yuming Wang [Tue, 4 Dec 2018 13:57:58 +0000 (07:57 -0600)] 
[SPARK-24423][FOLLOW-UP][SQL] Fix error example

## What changes were proposed in this pull request?
![image](https://user-images.githubusercontent.com/5399861/49172173-42ad9800-f37b-11e8-8135-7adc323357ae.png)
It will throw:
```
requirement failed: When reading JDBC data sources, users need to specify all or none for the following options: 'partitionColumn', 'lowerBound', 'upperBound', and 'numPartitions'
```
and
```
User-defined partition column subq.c1 not found in the JDBC relation ...
```

This PR fix this error example.

## How was this patch tested?

manual tests

Closes #23170 from wangyum/SPARK-24499.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
5 days ago[MINOR][SQL] Combine the same codes in test cases
彭灿00244106 [Tue, 4 Dec 2018 13:08:16 +0000 (22:08 +0900)] 
[MINOR][SQL] Combine the same codes in test cases

## What changes were proposed in this pull request?

In the DDLSuit, there are four test cases have the same codes , writing a function can combine the same code.

## How was this patch tested?

existing tests.

Closes #23194 from CarolinePeng/Update_temp.

Authored-by: 彭灿00244106 <00244106@zte.intra>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
5 days ago[SPARK-25374][SQL] SafeProjection supports fallback to an interpreted mode
Takeshi Yamamuro [Tue, 4 Dec 2018 12:20:29 +0000 (20:20 +0800)] 
[SPARK-25374][SQL] SafeProjection supports fallback to an interpreted mode

## What changes were proposed in this pull request?
In SPARK-23711, we have implemented the expression fallback logic to an interpreted mode. So, this pr fixed code to support the same fallback mode in `SafeProjection` based on `CodeGeneratorWithInterpretedFallback`.

## How was this patch tested?
Add tests in `CodeGeneratorWithInterpretedFallbackSuite` and `UnsafeRowConverterSuite`.

Closes #22468 from maropu/SPARK-25374-3.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
5 days ago[SPARK-25573] Combine resolveExpression and resolve in the Analyzer
Dilip Biswal [Tue, 4 Dec 2018 07:00:02 +0000 (23:00 -0800)] 
[SPARK-25573] Combine resolveExpression and resolve in the Analyzer

## What changes were proposed in this pull request?
Currently in the Analyzer, we have two methods 1) Resolve 2)ResolveExpressions that are called at different code paths to resolve attributes, column ordinal and extract value expressions. ~~In this PR, we combine the two into one method to make sure, there is only one method that is tasked with resolving the attributes.~~
Update the description of the methods and use better names to make it easier to know when to make use of one method vs the other.

## How was this patch tested?
Existing tests.

Closes #22899 from dilipbiswal/SPARK-25573-final.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
5 days ago[SPARK-25498][SQL][FOLLOW-UP] Return an empty config set when regenerating the golden...
Takeshi Yamamuro [Tue, 4 Dec 2018 04:14:38 +0000 (12:14 +0800)] 
[SPARK-25498][SQL][FOLLOW-UP] Return an empty config set when regenerating the golden files

## What changes were proposed in this pull request?
This pr is to return an empty config set when regenerating the golden files in `SQLQueryTestSuite`.
This is the follow-up of  #22512.

## How was this patch tested?
N/A

Closes #23212 from maropu/SPARK-25498-FOLLOWUP.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
6 days ago[SPARK-26083][K8S] Add Copy pyspark into corresponding dir cmd in pyspark Dockerfile
Qi Shao [Mon, 3 Dec 2018 23:36:41 +0000 (15:36 -0800)] 
[SPARK-26083][K8S] Add Copy pyspark into corresponding dir cmd in pyspark Dockerfile

When I try to run `./bin/pyspark` cmd in a pod in Kubernetes(image built without change from pyspark Dockerfile), I'm getting an error:
```
$SPARK_HOME/bin/pyspark --deploy-mode client --master k8s://https://$KUBERNETES_SERVICE_HOST:$KUBERNETES_SERVICE_PORT_HTTPS ...
Python 2.7.15 (default, Aug 22 2018, 13:24:18)
[GCC 6.4.0] on linux2
Type "help", "copyright", "credits" or "license" for more information.
Could not open PYTHONSTARTUP
IOError: [Errno 2] No such file or directory: '/opt/spark/python/pyspark/shell.py'
```
This is because `pyspark` folder doesn't exist under `/opt/spark/python/`

## What changes were proposed in this pull request?

Added `COPY python/pyspark ${SPARK_HOME}/python/pyspark` to pyspark Dockerfile to resolve issue above.

## How was this patch tested?
Google Kubernetes Engine

Closes #23037 from AzureQ/master.

Authored-by: Qi Shao <qi.shao.nyu@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
6 days ago[SPARK-26256][K8S] Fix labels for pod deletion
Stavros Kontopoulos [Mon, 3 Dec 2018 22:57:18 +0000 (14:57 -0800)] 
[SPARK-26256][K8S] Fix labels for pod deletion

## What changes were proposed in this pull request?
Adds proper labels when deleting executor pods.

## How was this patch tested?
Manually with tests.

Closes #23209 from skonto/fix-deletion-labels.

Authored-by: Stavros Kontopoulos <stavros.kontopoulos@lightbend.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
6 days ago[SPARK-26033][SPARK-26034][PYTHON][FOLLOW-UP] Small cleanup and deduplication in...
Hyukjin Kwon [Mon, 3 Dec 2018 22:03:10 +0000 (14:03 -0800)] 
[SPARK-26033][SPARK-26034][PYTHON][FOLLOW-UP] Small cleanup and deduplication in ml/mllib tests

## What changes were proposed in this pull request?

This PR is a small follow up that puts some logic and functions into smaller scope and make it localized, and deduplicate.

## How was this patch tested?

Manually tested. Jenkins tests as well.

Closes #23200 from HyukjinKwon/followup-SPARK-26034-SPARK-26033.

Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Bryan Cutler <cutlerb@gmail.com>
6 days ago[SPARK-25957][FOLLOWUP] Build python docker image in sbt build too.
Marcelo Vanzin [Mon, 3 Dec 2018 21:54:09 +0000 (13:54 -0800)] 
[SPARK-25957][FOLLOWUP] Build python docker image in sbt build too.

docker-image-tool.sh requires explicit argument to create the python
image now; do that from the sbt integration tests target too.

Closes #23172 from vanzin/SPARK-25957.followup.

Authored-by: Marcelo Vanzin <vanzin@cloudera.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
6 days ago[SPARK-25515][K8S] Adds a config option to keep executor pods for debugging
Stavros Kontopoulos [Mon, 3 Dec 2018 17:02:47 +0000 (09:02 -0800)] 
[SPARK-25515][K8S] Adds a config option to keep executor pods for debugging

## What changes were proposed in this pull request?
Keeps K8s executor resources present if case of failure or normal termination.
Introduces a new boolean config option: `spark.kubernetes.deleteExecutors`, with default value set to true.
The idea is to update Spark K8s backend structures but leave the resources around.
The assumption is that since entries are not removed from the `removedExecutorsCache` we are immune to updates that refer to the the executor resources previously removed.
The only delete operation not touched is the one in the `doKillExecutors` method.
Reason is right now we dont support [blacklisting](https://issues.apache.org/jira/browse/SPARK-23485) and dynamic allocation with Spark on K8s. In both cases in the future we might want to handle these scenarios although its more complicated.
More tests can be added if approach is approved.
## How was this patch tested?
Manually by running a Spark job and verifying pods are not deleted.

Closes #23136 from skonto/keep_pods.

Authored-by: Stavros Kontopoulos <stavros.kontopoulos@lightbend.com>
Signed-off-by: Yinan Li <ynli@google.com>
6 days ago[SPARK-25498][SQL] InterpretedMutableProjection should handle UnsafeRow
Takeshi Yamamuro [Mon, 3 Dec 2018 16:05:15 +0000 (00:05 +0800)] 
[SPARK-25498][SQL] InterpretedMutableProjection should handle UnsafeRow

## What changes were proposed in this pull request?
Since `AggregationIterator` uses `MutableProjection` for `UnsafeRow`, `InterpretedMutableProjection` needs to handle `UnsafeRow` as buffer internally for fixed-length types only.

## How was this patch tested?
Run 'SQLQueryTestSuite' with the interpreted mode.

Closes #22512 from maropu/InterpreterTest.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
6 days ago[SPARK-26177] Config change followup to [] Automated formatting for Scala code
cody koeninger [Mon, 3 Dec 2018 16:03:51 +0000 (10:03 -0600)] 
[SPARK-26177] Config change followup to [] Automated formatting for Scala code

Let's keep this open for a while to see if other configuration tweaks are suggested

## What changes were proposed in this pull request?

Formatting configuration changes following up
https://github.com/apache/spark/pull/23148

## How was this patch tested?

./dev/scalafmt

Closes #23182 from koeninger/scalafmt-config.

Authored-by: cody koeninger <cody@koeninger.org>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
6 days ago[SPARK-26235][CORE] Change log level for ClassNotFoundException/NoClassDefFoundError...
Gengliang Wang [Mon, 3 Dec 2018 16:02:15 +0000 (10:02 -0600)] 
[SPARK-26235][CORE] Change log level for ClassNotFoundException/NoClassDefFoundError in SparkSubmit to Error

## What changes were proposed in this pull request?

In my local setup, I set log4j root category as ERROR (https://stackoverflow.com/questions/27781187/how-to-stop-info-messages-displaying-on-spark-console , first item show up if we google search "set spark log level".) When I run such command
```
spark-submit --class foo bar.jar
```
Nothing shows up, and the script exits.

After quick investigation, I think the log level for ClassNotFoundException/NoClassDefFoundError in SparkSubmit should be ERROR instead of WARN. Since the whole process exit because of the exception/error.

Before https://github.com/apache/spark/pull/20925, the message is not controlled by `log4j.rootCategory`.

## How was this patch tested?

Manual check.

Closes #23189 from gengliangwang/changeLogLevel.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
6 days ago[SPARK-26181][SQL] the `hasMinMaxStats` method of `ColumnStatsMap` is not correct
Daoyuan Wang [Mon, 3 Dec 2018 15:54:26 +0000 (23:54 +0800)] 
[SPARK-26181][SQL] the `hasMinMaxStats` method of `ColumnStatsMap` is not correct

## What changes were proposed in this pull request?

For now the `hasMinMaxStats` will return the same as `hasCountStats`, which is obviously not as expected.

## How was this patch tested?

Existing tests.

Closes #23152 from adrian-wang/minmaxstats.

Authored-by: Daoyuan Wang <me@daoyuan.wang>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
6 days ago[SPARK-26253][WEBUI] Task Summary Metrics Table on Stage Page shows empty table when...
pgandhi [Mon, 3 Dec 2018 13:53:21 +0000 (07:53 -0600)] 
[SPARK-26253][WEBUI] Task Summary Metrics Table on Stage Page shows empty table when no data is present

Task Summary Metrics Table on Stage Page shows empty table when no data is present instead of showing a message.

## What changes were proposed in this pull request?

Added a custom message to show on the task summary metrics table as well as executor summary table when no data is present.

## How was this patch tested?

**Before:**

![49335550-29277d00-f615-11e8-8e62-a953e76bcebf](https://user-images.githubusercontent.com/22228190/49361520-425a2780-f702-11e8-8df4-08862ab6ceb8.png)

**After:**

<img width="1413" alt="screen shot 2018-12-03 at 1 56 09 pm" src="https://user-images.githubusercontent.com/22228190/49362019-8699f780-f703-11e8-93e1-d02df6572923.png">

Closes #23205 from pgandhi999/SPARK-26253.

Authored-by: pgandhi <pgandhi@oath.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
6 days ago[SPARK-26230][SQL] FileIndex: if case sensitive, validate partitions with original...
Gengliang Wang [Mon, 3 Dec 2018 11:53:45 +0000 (19:53 +0800)] 
[SPARK-26230][SQL] FileIndex: if case sensitive, validate partitions with original column names

## What changes were proposed in this pull request?

Partition column name is required to be unique under the same directory. The following paths are invalid partitioned directory:
```
hdfs://host:9000/path/a=1
hdfs://host:9000/path/b=2
```

If case sensitive, the following paths should be invalid too:
```
hdfs://host:9000/path/a=1
hdfs://host:9000/path/A=2
```
Since column 'a' and 'A' are different, and it is wrong to use either one as the column name in partition schema.

Also, there is a `TODO` comment in the code. Currently the Spark doesn't validate such case when `CASE_SENSITIVE` enabled.

This PR is to resolve the problem.

## How was this patch tested?

Add unit test

Closes #23186 from gengliangwang/SPARK-26230.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
6 days ago[SPARK-26151][SQL] Return partial results for bad CSV records
Maxim Gekk [Mon, 3 Dec 2018 10:25:38 +0000 (18:25 +0800)] 
[SPARK-26151][SQL] Return partial results for bad CSV records

## What changes were proposed in this pull request?

In the PR, I propose to change behaviour of `UnivocityParser` and `FailureSafeParser`, and return all fields that were parsed and converted to expected types successfully instead of just returning a row with all `null`s for a bad input in the `PERMISSIVE` mode. For example, for CSV line `0,2013-111-11 12:13:14` and DDL schema `a int, b timestamp`, new result is `Row(0, null)`.

## How was this patch tested?

It was checked by existing tests from `CsvSuite` and `CsvFunctionsSuite`.

Closes #23120 from MaxGekk/failuresafe-partial-result.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
6 days ago[SPARK-26117][FOLLOW-UP][SQL] throw SparkOutOfMemoryError intead of SparkException...
caoxuewen [Mon, 3 Dec 2018 08:18:22 +0000 (16:18 +0800)] 
[SPARK-26117][FOLLOW-UP][SQL] throw SparkOutOfMemoryError intead of SparkException in UnsafeHashedRelation

## What changes were proposed in this pull request?

When build hash Map with one row of data and run out of memory, we should throw a SparkOutOfMemoryError exception, which is more accurate than SparkException. this PR fix it.

## How was this patch tested?

N / A

Closes #23190 from heary-cao/throwUnsafeHashedRelation.

Authored-by: caoxuewen <cao.xuewen@zte.com.cn>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
7 days ago[SPARK-26198][SQL] Fix Metadata serialize null values throw NPE
Yuming Wang [Sun, 2 Dec 2018 14:52:01 +0000 (08:52 -0600)] 
[SPARK-26198][SQL] Fix Metadata serialize null values throw NPE

## What changes were proposed in this pull request?
How to reproduce this issue:
```scala
scala> val meta = new org.apache.spark.sql.types.MetadataBuilder().putNull("key").build().json
java.lang.NullPointerException
  at org.apache.spark.sql.types.Metadata$.org$apache$spark$sql$types$Metadata$$toJsonValue(Metadata.scala:196)
  at org.apache.spark.sql.types.Metadata$$anonfun$1.apply(Metadata.scala:180)
```

This pr fix `NullPointerException` when `Metadata` serialize `null` values.

## How was this patch tested?

unit tests

Closes #23164 from wangyum/SPARK-26198.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
7 days ago[SPARK-26080][PYTHON] Skips Python resource limit on Windows in Python worker
hyukjinkwon [Sun, 2 Dec 2018 09:41:08 +0000 (17:41 +0800)] 
[SPARK-26080][PYTHON] Skips Python resource limit on Windows in Python worker

## What changes were proposed in this pull request?

`resource` package is a Unix specific package. See https://docs.python.org/2/library/resource.html and https://docs.python.org/3/library/resource.html.

Note that we document Windows support:

> Spark runs on both Windows and UNIX-like systems (e.g. Linux, Mac OS).

This should be backported into branch-2.4 to restore Windows support in Spark 2.4.1.

## How was this patch tested?

Manually mocking the changed logics.

Closes #23055 from HyukjinKwon/SPARK-26080.

Lead-authored-by: hyukjinkwon <gurwls223@apache.org>
Co-authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
7 days ago[SPARK-26208][SQL] add headers to empty csv files when header=true
Koert Kuipers [Sun, 2 Dec 2018 09:38:25 +0000 (17:38 +0800)] 
[SPARK-26208][SQL] add headers to empty csv files when header=true

## What changes were proposed in this pull request?

Add headers to empty csv files when header=true, because otherwise these files are invalid when reading.

## How was this patch tested?

Added test for roundtrip of empty dataframe to csv file with headers and back in CSVSuite

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

Closes #23173 from koertkuipers/feat-empty-csv-with-header.

Authored-by: Koert Kuipers <koert@tresata.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
7 days ago[SPARK-26195][SQL] Correct exception messages in some classes
lichaoqun [Sun, 2 Dec 2018 02:55:17 +0000 (10:55 +0800)] 
[SPARK-26195][SQL] Correct exception messages in some classes

## What changes were proposed in this pull request?

UnsupportedOperationException messages are not the same with method name.This PR correct these messages.

## How was this patch tested?
NA

Closes #23154 from lcqzte10192193/wid-lcq-1127.

Authored-by: lichaoqun <li.chaoqun@zte.com.cn>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
7 days ago[SPARK-26216][SQL] Do not use case class as public API (UserDefinedFunction)
Wenchen Fan [Sun, 2 Dec 2018 02:46:17 +0000 (10:46 +0800)] 
[SPARK-26216][SQL] Do not use case class as public API (UserDefinedFunction)

## What changes were proposed in this pull request?

It's a bad idea to use case class as public API, as it has a very wide surface. For example, the `copy` method, its fields, the companion object, etc.

For a particular case, `UserDefinedFunction`. It has a private constructor, and I believe we only want users to access a few methods:`apply`, `nullable`, `asNonNullable`, etc.

However, all its fields, and `copy` method, and the companion object are public unexpectedly. As a result, we made many tricks to work around the binary compatibility issues.

This PR proposes to only make interfaces public, and hide implementations behind with a private class. Now `UserDefinedFunction` is a pure trait, and the concrete implementation is `SparkUserDefinedFunction`, which is private.

Changing class to interface is not binary compatible(but source compatible), so 3.0 is a good chance to do it.

This is the first PR to go with this direction. If it's accepted, I'll create a umbrella JIRA and fix all the public case classes.

## How was this patch tested?

existing tests.

Closes #23178 from cloud-fan/udf.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
7 days ago[SPARK-26161][SQL] Ignore empty files in load
Maxim Gekk [Sun, 2 Dec 2018 02:29:25 +0000 (10:29 +0800)] 
[SPARK-26161][SQL] Ignore empty files in load

## What changes were proposed in this pull request?

In the PR, I propose filtering out all empty files inside of `FileSourceScanExec` and exclude them from file splits. It should reduce overhead of opening and reading files without any data, and as consequence datasources will not produce empty partitions for such files.

## How was this patch tested?

Added a test which creates an empty and non-empty files. If empty files are ignored in load, Text datasource in the `wholetext` mode must create only one partition for non-empty file.

Closes #23130 from MaxGekk/ignore-empty-files.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
7 days ago[SPARK-26211][SQL][TEST][FOLLOW-UP] Combine test cases for `In` and `InSet`.
Takuya UESHIN [Sun, 2 Dec 2018 02:22:22 +0000 (10:22 +0800)] 
[SPARK-26211][SQL][TEST][FOLLOW-UP] Combine test cases for `In` and `InSet`.

## What changes were proposed in this pull request?

This is a follow pr of #23176.

`In` and `InSet` are semantically equal, so the tests for `In` should pass with `InSet`, and vice versa.
This combines those test cases.

## How was this patch tested?

The combined tests and existing tests.

Closes #23187 from ueshin/issues/SPARK-26211/in_inset_tests.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
8 days ago[SPARK-26241][SQL] Add queryId to IncrementalExecution
Reynold Xin [Sun, 2 Dec 2018 00:24:06 +0000 (16:24 -0800)] 
[SPARK-26241][SQL] Add queryId to IncrementalExecution

## What changes were proposed in this pull request?
This is a small change for better debugging: to pass query uuid in IncrementalExecution, when we look at the QueryExecution in isolation to trace back the query.

## How was this patch tested?
N/A - just add some field for better debugging.

Closes #23192 from rxin/SPARK-26241.

Authored-by: Reynold Xin <rxin@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
8 days ago[SPARK-26226][SQL] Track optimization phase for streaming queries
Reynold Xin [Sun, 2 Dec 2018 00:22:38 +0000 (16:22 -0800)] 
[SPARK-26226][SQL] Track optimization phase for streaming queries

## What changes were proposed in this pull request?
In an earlier PR, we missed measuring the optimization phase time for streaming queries. This patch adds it.

## How was this patch tested?
Given this is a debugging feature, and it is very convoluted to add tests to verify the phase is set properly, I am not introducing a streaming specific test.

Closes #23193 from rxin/SPARK-26226-1.

Authored-by: Reynold Xin <rxin@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
8 days ago[MINOR][DOC] Correct some document description errors
liuxian [Sat, 1 Dec 2018 13:11:31 +0000 (07:11 -0600)] 
[MINOR][DOC] Correct some document description errors

## What changes were proposed in this pull request?

Correct some document description errors.

## How was this patch tested?
N/A

Closes #23162 from 10110346/docerror.

Authored-by: liuxian <liu.xian3@zte.com.cn>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
8 days ago[SPARK-26212][BUILD][TEST-MAVEN] Upgrade maven version to 3.6.0
Kazuaki Ishizaki [Sat, 1 Dec 2018 13:06:18 +0000 (07:06 -0600)] 
[SPARK-26212][BUILD][TEST-MAVEN] Upgrade maven version to 3.6.0

## What changes were proposed in this pull request?

This PR updates maven version from 3.5.4 to 3.6.0. The release note of the 3.6.0 is [here](https://maven.apache.org/docs/3.6.0/release-notes.html).

From [the release note of the 3.6.0](https://maven.apache.org/docs/3.6.0/release-notes.html), the followings are new features:
1. There had been issues related to the project discoverytime which has been increased in previous version which influenced some of our users.
1. The output in the reactor summary has been improved.
1. There was an issue related to the classpath ordering.

## How was this patch tested?

Existing tests

Closes #23177 from kiszk/SPARK-26212.

Authored-by: Kazuaki Ishizaki <ishizaki@jp.ibm.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
8 days ago[SPARK-26180][CORE][TEST] Reuse withTempDir function to the SparkCore test case
caoxuewen [Sat, 1 Dec 2018 08:34:11 +0000 (16:34 +0800)] 
[SPARK-26180][CORE][TEST] Reuse withTempDir function to the SparkCore test case

## What changes were proposed in this pull request?

Currently, the common `withTempDir` function is used in Spark SQL test cases. To handle `val dir = Utils. createTempDir()` and `Utils. deleteRecursively (dir)`. Unfortunately, the `withTempDir` function cannot be used in the Spark Core test case. This PR Sharing `withTempDir` function in Spark Sql and SparkCore  to clean up SparkCore test cases. thanks.

## How was this patch tested?

N / A

Closes #23151 from heary-cao/withCreateTempDir.

Authored-by: caoxuewen <cao.xuewen@zte.com.cn>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
8 days ago[SPARK-26189][R] Fix unionAll doc in SparkR
Huaxin Gao [Sat, 1 Dec 2018 07:14:05 +0000 (23:14 -0800)] 
[SPARK-26189][R] Fix unionAll doc in SparkR

## What changes were proposed in this pull request?

Fix unionAll doc in SparkR

## How was this patch tested?

Manually ran test

Author: Huaxin Gao <huaxing@us.ibm.com>

Closes #23161 from huaxingao/spark-26189.

8 days ago[SPARK-23647][PYTHON][SQL] Adds more types for hint in pyspark
DylanGuedes [Sat, 1 Dec 2018 02:37:03 +0000 (10:37 +0800)] 
[SPARK-23647][PYTHON][SQL] Adds more types for hint in pyspark

Signed-off-by: DylanGuedes <djmgguedesgmail.com>
## What changes were proposed in this pull request?

Addition of float, int and list hints for `pyspark.sql` Hint.

## How was this patch tested?

I did manual tests following the same principles used in the Scala version, and also added unit tests.

Closes #20788 from DylanGuedes/jira-21030.

Authored-by: DylanGuedes <djmgguedes@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
9 days ago[SPARK-25876][K8S] Simplify kubernetes configuration types.
Marcelo Vanzin [Sat, 1 Dec 2018 00:23:37 +0000 (16:23 -0800)] 
[SPARK-25876][K8S] Simplify kubernetes configuration types.

There are a few issues with the current configuration types used in
the kubernetes backend:

- they use type parameters for role-specific specialization, which makes
  type signatures really noisy throughout the code base.

- they break encapsulation by forcing the code that creates the config
  object to remove the configuration from SparkConf before creating the
  k8s-specific wrapper.

- they don't provide an easy way for tests to have default values for
  fields they do not use.

This change fixes those problems by:

- creating a base config type with role-specific specialization using
  inheritance

- encapsulating the logic of parsing SparkConf into k8s-specific views
  inside the k8s config classes

- providing some helper code for tests to easily override just the part
  of the configs they want.

Most of the change relates to the above, especially cleaning up the
tests. While doing that, I also made some smaller changes elsewhere:

- removed unnecessary type parameters in KubernetesVolumeSpec

- simplified the error detection logic in KubernetesVolumeUtils; all
  the call sites would just throw the first exception collected by
  that class, since they all called "get" on the "Try" object. Now
  the unnecessary wrapping is gone and the exception is just thrown
  where it occurs.

- removed a lot of unnecessary mocking from tests.

- changed the kerberos-related code so that less logic needs to live
  in the driver builder. In spirit it should be part of the upcoming
  work in this series of cleanups, but it made parts of this change
  simpler.

Tested with existing unit tests and integration tests.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #22959 from vanzin/SPARK-25876.

9 days ago[SPARK-26219][CORE] Executor summary should get updated for failure jobs in the histo...
Shahid [Fri, 30 Nov 2018 23:20:05 +0000 (15:20 -0800)] 
[SPARK-26219][CORE] Executor summary should get updated for failure jobs in the history server UI

The root cause of the problem is, whenever the taskEnd event comes after stageCompleted event, execSummary is updating only for live UI. we need to update for history UI too.

To see the previous discussion, refer: PR for https://github.com/apache/spark/pull/23038, https://issues.apache.org/jira/browse/SPARK-26100.

Added UT. Manually verified

Test step to reproduce:

```
bin/spark-shell --master yarn --conf spark.executor.instances=3
sc.parallelize(1 to 10000, 10).map{ x => throw new RuntimeException("Bad executor")}.collect()
```

Open Executors page from the History UI

Before patch:
![screenshot from 2018-11-29 22-13-34](https://user-images.githubusercontent.com/23054875/49246338-a21ead00-f43a-11e8-8214-f1020420be52.png)

After patch:
![screenshot from 2018-11-30 00-54-49](https://user-images.githubusercontent.com/23054875/49246353-aa76e800-f43a-11e8-98ef-7faecaa7a50e.png)

Closes #23181 from shahidki31/executorUpdate.

Authored-by: Shahid <shahidki31@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
9 days ago[SPARK-26226][SQL] Update query tracker to report timeline for phases
Reynold Xin [Fri, 30 Nov 2018 22:23:18 +0000 (14:23 -0800)] 
[SPARK-26226][SQL] Update query tracker to report timeline for phases

## What changes were proposed in this pull request?
This patch changes the query plan tracker added earlier to report phase timeline, rather than just a duration for each phase. This way, we can easily find time that's unaccounted for.

## How was this patch tested?
Updated test cases to reflect that.

Closes #23183 from rxin/SPARK-26226.

Authored-by: Reynold Xin <rxin@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
9 days ago[SPARK-26201] Fix python broadcast with encryption
schintap [Fri, 30 Nov 2018 18:48:56 +0000 (12:48 -0600)] 
[SPARK-26201] Fix python broadcast with encryption

## What changes were proposed in this pull request?
Python with rpc and disk encryption enabled along with a python broadcast variable and just read the value back on the driver side the job failed with:

Traceback (most recent call last): File "broadcast.py", line 37, in <module> words_new.value File "/pyspark.zip/pyspark/broadcast.py", line 137, in value File "pyspark.zip/pyspark/broadcast.py", line 122, in load_from_path File "pyspark.zip/pyspark/broadcast.py", line 128, in load EOFError: Ran out of input

To reproduce use configs: --conf spark.network.crypto.enabled=true --conf spark.io.encryption.enabled=true

Code:

words_new = sc.broadcast(["scala", "java", "hadoop", "spark", "akka"])
words_new.value
print(words_new.value)

## How was this patch tested?
words_new = sc.broadcast([“scala”, “java”, “hadoop”, “spark”, “akka”])
textFile = sc.textFile(“README.md”)
wordCounts = textFile.flatMap(lambda line: line.split()).map(lambda word: (word + words_new.value[1], 1)).reduceByKey(lambda a, b: a+b)
 count = wordCounts.count()
 print(count)
 words_new.value
 print(words_new.value)

Closes #23166 from redsanket/SPARK-26201.

Authored-by: schintap <schintap@oath.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
9 days ago[MINOR][DOCS] Fix typos
Keiji Yoshida [Fri, 30 Nov 2018 15:03:46 +0000 (09:03 -0600)] 
[MINOR][DOCS] Fix typos

## What changes were proposed in this pull request?
Fix Typos.
This PR is the complete version of https://github.com/apache/spark/pull/23145.

## How was this patch tested?
NA

Closes #23185 from kjmrknsn/docUpdate.

Authored-by: Keiji Yoshida <kjmrknsn@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
9 days ago[SPARK-25528][SQL] data source v2 API refactor (batch read)
Wenchen Fan [Fri, 30 Nov 2018 08:02:43 +0000 (00:02 -0800)] 
[SPARK-25528][SQL] data source v2 API refactor (batch read)

## What changes were proposed in this pull request?

This is the first step of the data source v2 API refactor [proposal](https://docs.google.com/document/d/1uUmKCpWLdh9vHxP7AWJ9EgbwB_U6T3EJYNjhISGmiQg/edit?usp=sharing)

It adds the new API for batch read, without removing the old APIs, as they are still needed for streaming sources.

More concretely, it adds
1. `TableProvider`, works like an anonymous catalog
2. `Table`, represents a structured data set.
3. `ScanBuilder` and `Scan`, a logical represents of data source scan
4. `Batch`, a physical representation of data source batch scan.

## How was this patch tested?

existing tests

Closes #23086 from cloud-fan/refactor-batch.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
9 days ago[SPARK-26188][SQL] FileIndex: don't infer data types of partition columns if user...
Gengliang Wang [Fri, 30 Nov 2018 04:00:55 +0000 (12:00 +0800)] 
[SPARK-26188][SQL] FileIndex: don't infer data types of partition columns if user specifies schema

## What changes were proposed in this pull request?

This PR is to fix a regression introduced in: https://github.com/apache/spark/pull/21004/files#r236998030

If user specifies schema, Spark don't need to infer data type for of partition columns, otherwise the data type might not match with the one user provided.
E.g. for partition directory `p=4d`, after data type inference  the column value will be `4.0`.
See https://issues.apache.org/jira/browse/SPARK-26188 for more details.

Note that user specified schema **might not cover all the data columns**:
```
val schema = new StructType()
  .add("id", StringType)
  .add("ex", ArrayType(StringType))
val df = spark.read
  .schema(schema)
  .format("parquet")
  .load(src.toString)

assert(df.schema.toList === List(
  StructField("ex", ArrayType(StringType)),
  StructField("part", IntegerType), // inferred partitionColumn dataType
  StructField("id", StringType))) // used user provided partitionColumn dataType
```
For the missing columns in user specified schema, Spark still need to infer their data types if `partitionColumnTypeInferenceEnabled` is enabled.

To implement the partially inference, refactor `PartitioningUtils.parsePartitions`  and pass the user specified schema as parameter to cast partition values.

## How was this patch tested?

Add unit test.

Closes #23165 from gengliangwang/fixFileIndex.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
9 days ago[SPARK-26060][SQL] Track SparkConf entries and make SET command reject such entries.
Takuya UESHIN [Fri, 30 Nov 2018 03:56:25 +0000 (11:56 +0800)] 
[SPARK-26060][SQL] Track SparkConf entries and make SET command reject such entries.

## What changes were proposed in this pull request?

Currently the `SET` command works without any warnings even if the specified key is for `SparkConf` entries and it has no effect because the command does not update `SparkConf`, but the behavior might confuse users. We should track `SparkConf` entries and make the command reject for such entries.

## How was this patch tested?

Added a test and existing tests.

Closes #23031 from ueshin/issues/SPARK-26060/set_command.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
9 days ago[SPARK-25446][R] Add schema_of_json() and schema_of_csv() to R
hyukjinkwon [Fri, 30 Nov 2018 02:29:30 +0000 (10:29 +0800)] 
[SPARK-25446][R] Add schema_of_json() and schema_of_csv() to R

## What changes were proposed in this pull request?

This PR proposes to expose `schema_of_json` and `schema_of_csv` at R side.

**`schema_of_json`**:

```r
json <- '{"name":"Bob"}'
df <- sql("SELECT * FROM range(1)")
head(select(df, schema_of_json(json)))
```

```
  schema_of_json({"name":"Bob"})
1            struct<name:string>
```

**`schema_of_csv`**:

```r
csv <- "Amsterdam,2018"
df <- sql("SELECT * FROM range(1)")
head(select(df, schema_of_csv(csv)))
```

```
  schema_of_csv(Amsterdam,2018)
1    struct<_c0:string,_c1:int>
```

## How was this patch tested?

Manually tested, unit tests added, documentation manually built and verified.

Closes #22939 from HyukjinKwon/SPARK-25446.

Authored-by: hyukjinkwon <gurwls223@apache.org>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
9 days ago[SPARK-25501][SS] Add kafka delegation token support.
Gabor Somogyi [Fri, 30 Nov 2018 02:00:47 +0000 (18:00 -0800)] 
[SPARK-25501][SS] Add kafka delegation token support.

## What changes were proposed in this pull request?

It adds kafka delegation token support for structured streaming. Please see the relevant [SPIP](https://docs.google.com/document/d/1ouRayzaJf_N5VQtGhVq9FURXVmRpXzEEWYHob0ne3NY/edit?usp=sharing)

What this PR contains:
* Configuration parameters for the feature
* Delegation token fetching from broker
* Usage of token through dynamic JAAS configuration
* Minor refactoring in the existing code

What this PR doesn't contain:
* Documentation changes because design can change

## How was this patch tested?

Existing tests + added small amount of additional unit tests.

Because it's an external service integration mainly tested on cluster.
* 4 node cluster
* Kafka broker version 1.1.0
* Topic with 4 partitions
* security.protocol = SASL_SSL
* sasl.mechanism = SCRAM-SHA-256

An example of obtaining a token:
```
18/10/01 01:07:49 INFO kafka010.TokenUtil: TOKENID         HMAC                           OWNER           RENEWERS                  ISSUEDATE       EXPIRYDATE      MAXDATE
18/10/01 01:07:49 INFO kafka010.TokenUtil: D1-v__Q5T_uHx55rW16Jwg [hidden] User:user    []                        2018-10-01T01:07 2018-10-02T01:07 2018-10-08T01:07
18/10/01 01:07:49 INFO security.KafkaDelegationTokenProvider: Get token from Kafka: Kind: KAFKA_DELEGATION_TOKEN, Service: kafka.server.delegation.token, Ident: 44 31 2d 76 5f 5f 51 35 54 5f 75 48 78 35 35 72 57 31 36 4a 77 67
```

An example token usage:
```
18/10/01 01:08:07 INFO kafka010.KafkaSecurityHelper: Scram JAAS params: org.apache.kafka.common.security.scram.ScramLoginModule required tokenauth=true serviceName="kafka" username="D1-v__Q5T_uHx55rW16Jwg" password="[hidden]";
18/10/01 01:08:07 INFO kafka010.KafkaSourceProvider: Delegation token detected, using it for login.
```

Closes #22598 from gaborgsomogyi/SPARK-25501.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
10 days ago[SPARK-25977][SQL] Parsing decimals from CSV using locale
Maxim Gekk [Fri, 30 Nov 2018 00:27:55 +0000 (08:27 +0800)] 
[SPARK-25977][SQL] Parsing decimals from CSV using locale

## What changes were proposed in this pull request?

In the PR, I propose using of the locale option to parse decimals from CSV input. After the changes, `UnivocityParser` converts input string to `BigDecimal` and to Spark's Decimal by using `java.text.DecimalFormat`.

## How was this patch tested?

Added a test for the `en-US`, `ko-KR`, `ru-RU`, `de-DE` locales.

Closes #22979 from MaxGekk/decimal-parsing-locale.

Lead-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Co-authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
10 days ago[SPARK-25905][CORE] When getting a remote block, avoid forcing a conversion to a...
Wing Yew Poon [Thu, 29 Nov 2018 20:56:34 +0000 (14:56 -0600)] 
[SPARK-25905][CORE] When getting a remote block, avoid forcing a conversion to a ChunkedByteBuffer

## What changes were proposed in this pull request?

In `BlockManager`, `getRemoteValues` gets a `ChunkedByteBuffer` (by calling `getRemoteBytes`) and creates an `InputStream` from it. `getRemoteBytes`, in turn, gets a `ManagedBuffer` and converts it to a `ChunkedByteBuffer`.
Instead, expose a `getRemoteManagedBuffer` method so `getRemoteValues` can just get this `ManagedBuffer` and use its `InputStream`.
When reading a remote cache block from disk, this reduces heap memory usage significantly.
Retain `getRemoteBytes` for other callers.

## How was this patch tested?

Imran Rashid wrote an application (https://github.com/squito/spark_2gb_test/blob/master/src/main/scala/com/cloudera/sparktest/LargeBlocks.scala), that among other things, tests reading remote cache blocks. I ran this application, using 2500MB blocks, to test reading a cache block on disk. Without this change, with `--executor-memory 5g`, the test fails with `java.lang.OutOfMemoryError: Java heap space`. With the change, the test passes with `--executor-memory 2g`.
I also ran the unit tests in core. In particular, `DistributedSuite` has a set of tests that exercise the `getRemoteValues` code path. `BlockManagerSuite` has several tests that call `getRemoteBytes`; I left these unchanged, so `getRemoteBytes` still gets exercised.

Closes #23058 from wypoon/SPARK-25905.

Authored-by: Wing Yew Poon <wypoon@cloudera.com>
Signed-off-by: Imran Rashid <irashid@cloudera.com>
10 days ago[SPARK-26142] followup: Move sql shuffle read metrics relatives to SQLShuffleMetricsR...
Yuanjian Li [Thu, 29 Nov 2018 20:09:30 +0000 (12:09 -0800)] 
[SPARK-26142] followup: Move sql shuffle read metrics relatives to SQLShuffleMetricsReporter

## What changes were proposed in this pull request?

Follow up for https://github.com/apache/spark/pull/23128, move sql read metrics relatives to `SQLShuffleMetricsReporter`, in order to put sql shuffle read metrics relatives closer and avoid possible problem about forgetting update SQLShuffleMetricsReporter while new metrics added by others.

## How was this patch tested?

Existing tests.

Closes #23175 from xuanyuanking/SPARK-26142-follow.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Reynold Xin <rxin@databricks.com>
10 days ago[SPARK-26158][MLLIB] fix covariance accuracy problem for DenseVector
李亮 [Thu, 29 Nov 2018 19:08:53 +0000 (13:08 -0600)] 
[SPARK-26158][MLLIB] fix covariance accuracy problem for DenseVector

## What changes were proposed in this pull request?
Enhance accuracy of the covariance logic in RowMatrix for function computeCovariance

## How was this patch tested?
Unit test
Accuracy test

Closes #23126 from KyleLi1985/master.

Authored-by: 李亮 <liang.li.work@outlook.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
10 days ago[SPARK-26015][K8S] Set a default UID for Spark on K8S Images
Rob Vesse [Thu, 29 Nov 2018 17:59:38 +0000 (09:59 -0800)] 
[SPARK-26015][K8S] Set a default UID for Spark on K8S Images

Adds USER directives to the Dockerfiles which is configurable via build argument (`spark_uid`) for easy customisation.  A `-u` flag is added to `bin/docker-image-tool.sh` to make it easy to customise this e.g.

```
> bin/docker-image-tool.sh -r rvesse -t uid -u 185 build
> bin/docker-image-tool.sh -r rvesse -t uid push
```

If no UID is explicitly specified it defaults to `185` - this is per skonto's suggestion to align with the OpenShift standard reserved UID for Java apps (
https://lists.openshift.redhat.com/openshift-archives/users/2016-March/msg00283.html)

Notes:
- We have to make the `WORKDIR` writable by the root group or otherwise jobs will fail with `AccessDeniedException`

To Do:
- [x] Debug and resolve issue with client mode test
- [x] Consider whether to always propagate `SPARK_USER_NAME` to environment of driver and executor pods so `entrypoint.sh` can insert that into `/etc/passwd` entry
- [x] Rebase once PR #23013 is merged and update documentation accordingly

Built the Docker images with the new Dockerfiles that include the `USER` directives.  Ran the Spark on K8S integration tests against the new images.  All pass except client mode which I am currently debugging further.

Also manually dropped myself into the resulting container images via `docker run` and checked `id -u` output to see that UID is as expected.

Tried customising the UID from the default via the new `-u` argument to `docker-image-tool.sh` and again checked the resulting image for the correct runtime UID.

cc felixcheung skonto vanzin

Closes #23017 from rvesse/SPARK-26015.

Authored-by: Rob Vesse <rvesse@dotnetrdf.org>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
10 days ago[SPARK-26186][SPARK-26184][CORE] Last updated time is not getting updated for the...
Shahid [Thu, 29 Nov 2018 17:48:18 +0000 (09:48 -0800)] 
[SPARK-26186][SPARK-26184][CORE] Last updated time is not getting updated for the Inprogress application

## What changes were proposed in this pull request?

When the 'spark.history.fs.inProgressOptimization.enabled' is true, inProgress application's last updated time is not getting updated in the History UI. Also, during the cleaning time, InProgress application is getting removed from the listing, even if the last updated time is within the cleaning threshold time.

In this PR, if the fastInprogressOptimization enabled, we update the `lastUpdateTime` of the application as last scan time. This will update the `lastUpdateTime` in the historyUI and also while cleaning, it won't remove if the updateTime is within the cleaning interval

## How was this patch tested?
Added UT, attached screen shot.
Before patch:
![screenshot from 2018-11-27 23-22-38](https://user-images.githubusercontent.com/23054875/49101600-9b5a3380-f29c-11e8-8efc-3fb594e4279a.png)
![screenshot from 2018-11-27 23-20-11](https://user-images.githubusercontent.com/23054875/49101601-9c8b6080-f29c-11e8-928e-643a8c8f4477.png)

After Patch:
![screenshot from 2018-11-27 23-37-10](https://user-images.githubusercontent.com/23054875/49101911-669aac00-f29d-11e8-8181-663e4a08ab0e.png)
![screenshot from 2018-11-27 23-39-04](https://user-images.githubusercontent.com/23054875/49102010-a5306680-f29d-11e8-947a-e8a2a09a785a.png)

Closes #23158 from shahidki31/HistoryLastUpdateTime.

Authored-by: Shahid <shahidki31@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
10 days ago[MINOR][DOCS][WIP] Fix Typos
Keiji Yoshida [Thu, 29 Nov 2018 16:39:00 +0000 (10:39 -0600)] 
[MINOR][DOCS][WIP] Fix Typos

## What changes were proposed in this pull request?
Fix Typos.

## How was this patch tested?
NA

Closes #23145 from kjmrknsn/docUpdate.

Authored-by: Keiji Yoshida <kjmrknsn@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
10 days ago[SPARK-26081][SQL] Prevent empty files for empty partitions in Text datasources
Maxim Gekk [Thu, 29 Nov 2018 16:31:31 +0000 (10:31 -0600)] 
[SPARK-26081][SQL] Prevent empty files for empty partitions in Text datasources

## What changes were proposed in this pull request?

In the PR, I propose to postpone creation of `OutputStream`/`Univocity`/`JacksonGenerator` till the first row should be written. This prevents creation of empty files for empty partitions. So, no need to open and to read such files back while loading data from the location.

## How was this patch tested?

Added tests for Text, JSON and CSV datasource where empty dataset is written but should not produce any files.

Closes #23052 from MaxGekk/text-empty-files.

Lead-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
10 days ago[SPARK-26177] Automated formatting for Scala code
cody koeninger [Thu, 29 Nov 2018 14:54:31 +0000 (08:54 -0600)] 
[SPARK-26177] Automated formatting for Scala code

## What changes were proposed in this pull request?

Add a maven plugin and wrapper script to use scalafmt to format files that differ from git master.

Intention is for contributors to be able to use this to automate fixing code style, not to include it in build pipeline yet.

If this PR is accepted, I'd make a different PR to update the code style section of https://spark.apache.org/contributing.html to mention the script

## How was this patch tested?

Manually tested by modifying a few files and running ./dev/scalafmt then checking that ./dev/scalastyle still passed.

Closes #23148 from koeninger/scalafmt.

Authored-by: cody koeninger <cody@koeninger.org>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
10 days ago[MINOR][ML] add missing params to Instr
zhengruifeng [Thu, 29 Nov 2018 14:53:12 +0000 (08:53 -0600)] 
[MINOR][ML] add missing params to Instr

## What changes were proposed in this pull request?
add following param to instr:
GBTC: validationTol
GBTR: validationTol, validationIndicatorCol
colnames in LiR, LinearSVC, etc

## How was this patch tested?
existing tests

Closes #23122 from zhengruifeng/instr_append_missing_params.

Authored-by: zhengruifeng <ruifengz@foxmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
10 days ago[SPARK-26024][FOLLOWUP][MINOR] Follow-up to remove extra blank lines in R function...
Sean Owen [Thu, 29 Nov 2018 14:48:12 +0000 (08:48 -0600)] 
[SPARK-26024][FOLLOWUP][MINOR] Follow-up to remove extra blank lines in R function descriptions

## What changes were proposed in this pull request?

Follow-up to remove extra blank lines in R function descriptions

## How was this patch tested?

N/A

Closes #23167 from srowen/SPARK-26024.2.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
10 days ago[SPARK-26211][SQL] Fix InSet for binary, and struct and array with null.
Takuya UESHIN [Thu, 29 Nov 2018 14:37:02 +0000 (22:37 +0800)] 
[SPARK-26211][SQL] Fix InSet for binary, and struct and array with null.

## What changes were proposed in this pull request?

Currently `InSet` doesn't work properly for binary type, or struct and array type with null value in the set.
Because, as for binary type, the `HashSet` doesn't work properly for `Array[Byte]`, and as for struct and array type with null value in the set, the `ordering` will throw a `NPE`.

## How was this patch tested?

Added a few tests.

Closes #23176 from ueshin/issues/SPARK-26211/inset.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
10 days ago[SPARK-26163][SQL] Parsing decimals from JSON using locale
Maxim Gekk [Thu, 29 Nov 2018 14:15:12 +0000 (22:15 +0800)] 
[SPARK-26163][SQL] Parsing decimals from JSON using locale

## What changes were proposed in this pull request?

In the PR, I propose using of the locale option to parse (and infer) decimals from JSON input. After the changes, `JacksonParser` converts input string to `BigDecimal` and to Spark's Decimal by using `java.text.DecimalFormat`. New behaviour can be switched off via SQL config `spark.sql.legacy.decimalParsing.enabled`.

## How was this patch tested?

Added 2 tests to `JsonExpressionsSuite` for the `en-US`, `ko-KR`, `ru-RU`, `de-DE` locales:
- Inferring decimal type using locale from JSON field values
- Converting JSON field values to specified decimal type using the locales.

Closes #23132 from MaxGekk/json-decimal-parsing-locale.

Lead-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
10 days ago[SPARK-26133][ML] Remove deprecated OneHotEncoder and rename OneHotEncoderEstimator...
Liang-Chi Hsieh [Thu, 29 Nov 2018 01:54:06 +0000 (01:54 +0000)] 
[SPARK-26133][ML] Remove deprecated OneHotEncoder and rename OneHotEncoderEstimator to OneHotEncoder

## What changes were proposed in this pull request?

We have deprecated `OneHotEncoder` at Spark 2.3.0 and introduced `OneHotEncoderEstimator`. At 3.0.0, we remove deprecated `OneHotEncoder` and rename `OneHotEncoderEstimator` to `OneHotEncoder`.

TODO: According to ML migration guide, we need to keep `OneHotEncoderEstimator` as an alias after renaming. This is not done at this patch in order to facilitate review.

## How was this patch tested?

Existing tests.

Closes #23100 from viirya/remove_one_hot_encoder.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: DB Tsai <d_tsai@apple.com>
11 days ago[SPARK-25829][SQL] remove duplicated map keys with last wins policy
Wenchen Fan [Wed, 28 Nov 2018 15:42:13 +0000 (23:42 +0800)] 
[SPARK-25829][SQL] remove duplicated map keys with last wins policy

## What changes were proposed in this pull request?

Currently duplicated map keys are not handled consistently. For example, map look up respects the duplicated key appears first, `Dataset.collect` only keeps the duplicated key appears last, `MapKeys` returns duplicated keys, etc.

This PR proposes to remove duplicated map keys with last wins policy, to follow Java/Scala and Presto. It only applies to built-in functions, as users can create map with duplicated map keys via private APIs anyway.

updated functions: `CreateMap`, `MapFromArrays`, `MapFromEntries`, `StringToMap`, `MapConcat`, `TransformKeys`.

For other places:
1. data source v1 doesn't have this problem, as users need to provide a java/scala map, which can't have duplicated keys.
2. data source v2 may have this problem. I've added a note to `ArrayBasedMapData` to ask the caller to take care of duplicated keys. In the future we should enforce it in the stable data APIs for data source v2.
3. UDF doesn't have this problem, as users need to provide a java/scala map. Same as data source v1.
4. file format. I checked all of them and only parquet does not enforce it. For backward compatibility reasons I change nothing but leave a note saying that the behavior will be undefined if users write map with duplicated keys to parquet files. Maybe we can add a config and fail by default if parquet files have map with duplicated keys. This can be done in followup.

## How was this patch tested?

updated tests and new tests

Closes #23124 from cloud-fan/map.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
11 days ago[SPARK-25989][ML] OneVsRestModel handle empty outputCols incorrectly
zhengruifeng [Wed, 28 Nov 2018 15:33:34 +0000 (07:33 -0800)] 
[SPARK-25989][ML] OneVsRestModel handle empty outputCols incorrectly

## What changes were proposed in this pull request?
ignore empty output columns

## How was this patch tested?
added tests

Closes #22991 from zhengruifeng/ovrm_empty_outcol.

Authored-by: zhengruifeng <ruifengz@foxmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
11 days ago[SPARK-25998][CORE] Change TorrentBroadcast to hold weak reference of broadcast object
Brandon Krieger [Wed, 28 Nov 2018 15:22:48 +0000 (07:22 -0800)] 
[SPARK-25998][CORE] Change TorrentBroadcast to hold weak reference of broadcast object

## What changes were proposed in this pull request?

This PR changes the broadcast object in TorrentBroadcast from a strong reference to a weak reference. This allows it to be garbage collected even if the Dataset is held in memory. This is ok, because the broadcast object can always be re-read.

## How was this patch tested?

Tested in Spark shell by taking a heap dump, full repro steps listed in https://issues.apache.org/jira/browse/SPARK-25998.

Closes #22995 from bkrieger/bk/torrent-broadcast-weak.

Authored-by: Brandon Krieger <bkrieger@palantir.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
11 days ago[SPARK-26137][CORE] Use Java system property "file.separator" inste…
Mark Pavey [Wed, 28 Nov 2018 15:19:47 +0000 (07:19 -0800)] 
[SPARK-26137][CORE] Use Java system property "file.separator" inste…

… of hard coded "/" in DependencyUtils

## What changes were proposed in this pull request?

Use Java system property "file.separator" instead of hard coded "/" in DependencyUtils.

## How was this patch tested?

Manual test:
Submit Spark application via REST API that reads data from Elasticsearch using spark-elasticsearch library.

Without fix application fails with error:
18/11/22 10:36:20 ERROR Version: Multiple ES-Hadoop versions detected in the classpath; please use only one
jar:file:/C:/<...>/spark-2.4.0-bin-hadoop2.6/work/driver-20181122103610-0001/myApp-assembly-1.0.jar
jar:file:/C:/<...>/myApp-assembly-1.0.jar

18/11/22 10:36:20 ERROR Main: Application [MyApp] failed:
java.lang.Error: Multiple ES-Hadoop versions detected in the classpath; please use only one
jar:file:/C:/<...>/spark-2.4.0-bin-hadoop2.6/work/driver-20181122103610-0001/myApp-assembly-1.0.jar
jar:file:/C:/<...>/myApp-assembly-1.0.jar

at org.elasticsearch.hadoop.util.Version.<clinit>(Version.java:73)
at org.elasticsearch.hadoop.rest.RestService.findPartitions(RestService.java:214)
at org.elasticsearch.spark.rdd.AbstractEsRDD.esPartitions$lzycompute(AbstractEsRDD.scala:73)
at org.elasticsearch.spark.rdd.AbstractEsRDD.esPartitions(AbstractEsRDD.scala:72)
at org.elasticsearch.spark.rdd.AbstractEsRDD.getPartitions(AbstractEsRDD.scala:44)
at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:253)
at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:251)
at scala.Option.getOrElse(Option.scala:121)
at org.apache.spark.rdd.RDD.partitions(RDD.scala:251)
at org.apache.spark.rdd.MapPartitionsRDD.getPartitions(MapPartitionsRDD.scala:49)
at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:253)
at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:251)
at scala.Option.getOrElse(Option.scala:121)
at org.apache.spark.rdd.RDD.partitions(RDD.scala:251)
at org.apache.spark.SparkContext.runJob(SparkContext.scala:2126)
at org.apache.spark.rdd.RDD$$anonfun$collect$1.apply(RDD.scala:945)
at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:112)
at org.apache.spark.rdd.RDD.withScope(RDD.scala:363)
at org.apache.spark.rdd.RDD.collect(RDD.scala:944)
...
at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.lang.reflect.Method.invoke(Method.java:498)
at org.apache.spark.deploy.worker.DriverWrapper$.main(DriverWrapper.scala:65)
at org.apache.spark.deploy.worker.DriverWrapper.main(DriverWrapper.scala)

With fix application runs successfully.

Closes #23102 from markpavey/JIRA_SPARK-26137_DependencyUtilsFileSeparatorFix.

Authored-by: Mark Pavey <markpavey@exabre.co.uk>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
11 days ago[SPARK-26147][SQL] only pull out unevaluable python udf from join condition
Wenchen Fan [Wed, 28 Nov 2018 12:38:42 +0000 (20:38 +0800)] 
[SPARK-26147][SQL] only pull out unevaluable python udf from join condition

## What changes were proposed in this pull request?

https://github.com/apache/spark/pull/22326 made a mistake that, not all python UDFs are unevaluable in join condition. Only python UDFs that refer to attributes from both join side are unevaluable.

This PR fixes this mistake.

## How was this patch tested?

a new test

Closes #23153 from cloud-fan/join.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
11 days ago[SPARK-26114][CORE] ExternalSorter's readingIterator field leak
Sergey Zhemzhitsky [Wed, 28 Nov 2018 12:22:24 +0000 (20:22 +0800)] 
[SPARK-26114][CORE] ExternalSorter's readingIterator field leak

## What changes were proposed in this pull request?

This pull request fixes [SPARK-26114](https://issues.apache.org/jira/browse/SPARK-26114) issue that occurs when trying to reduce the number of partitions by means of coalesce without shuffling after shuffle-based transformations.

The leak occurs because of not cleaning up `ExternalSorter`'s `readingIterator` field as it's done for its `map` and `buffer` fields.
Additionally there are changes to the `CompletionIterator` to prevent capturing its `sub`-iterator and holding it even after the completion iterator completes. It is necessary because in some cases, e.g. in case of standard scala's `flatMap` iterator (which is used is `CoalescedRDD`'s `compute` method) the next value of the main iterator is assigned to `flatMap`'s `cur` field only after it is available.
For DAGs where ShuffledRDD is a parent of CoalescedRDD it means that the data should be fetched from the map-side of the shuffle, but the process of fetching this data consumes quite a lot of memory in addition to the memory already consumed by the iterator held by `flatMap`'s `cur` field (until it is reassigned).

For the following data
```scala
import org.apache.hadoop.io._
import org.apache.hadoop.io.compress._
import org.apache.commons.lang._
import org.apache.spark._

// generate 100M records of sample data
sc.makeRDD(1 to 1000, 1000)
  .flatMap(item => (1 to 100000)
    .map(i => new Text(RandomStringUtils.randomAlphanumeric(3).toLowerCase) -> new Text(RandomStringUtils.randomAlphanumeric(1024))))
  .saveAsSequenceFile("/tmp/random-strings", Some(classOf[GzipCodec]))
```

and the following job
```scala
import org.apache.hadoop.io._
import org.apache.spark._
import org.apache.spark.storage._

val rdd = sc.sequenceFile("/tmp/random-strings", classOf[Text], classOf[Text])
rdd
  .map(item => item._1.toString -> item._2.toString)
  .repartitionAndSortWithinPartitions(new HashPartitioner(1000))
  .coalesce(10,false)
  .count
```

... executed like the following
```bash
spark-shell \
  --num-executors=5 \
  --executor-cores=2 \
  --master=yarn \
  --deploy-mode=client \
  --conf spark.executor.memoryOverhead=512 \
  --conf spark.executor.memory=1g \
  --conf spark.dynamicAllocation.enabled=false \
  --conf spark.executor.extraJavaOptions='-XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=/tmp -Dio.netty.noUnsafe=true'
```

... executors are always failing with OutOfMemoryErrors.

The main issue is multiple leaks of ExternalSorter references.
For example, in case of 2 tasks per executor it is expected to be 2 simultaneous instances of ExternalSorter per executor but heap dump generated on OutOfMemoryError shows that there are more ones.

![run1-noparams-dominator-tree-externalsorter](https://user-images.githubusercontent.com/1523889/48703665-782ce580-ec05-11e8-95a9-d6c94e8285ab.png)

P.S. This PR does not cover cases with CoGroupedRDDs which use ExternalAppendOnlyMap internally, which itself can lead to OutOfMemoryErrors in many places.

## How was this patch tested?

- Existing unit tests
- New unit tests
- Job executions on the live environment

Here is the screenshot before applying this patch
![run3-noparams-failure-ui-5x2-repartition-and-sort](https://user-images.githubusercontent.com/1523889/48700395-f769eb80-ebfc-11e8-831b-e94c757d416c.png)

Here is the screenshot after applying this patch
![run3-noparams-success-ui-5x2-repartition-and-sort](https://user-images.githubusercontent.com/1523889/48700610-7a8b4180-ebfd-11e8-9761-baaf38a58e66.png)
And in case of reducing the number of executors even more the job is still stable
![run3-noparams-success-ui-2x2-repartition-and-sort](https://user-images.githubusercontent.com/1523889/48700619-82e37c80-ebfd-11e8-98ed-a38e1f1f1fd9.png)

Closes #23083 from szhem/SPARK-26114-externalsorter-leak.

Authored-by: Sergey Zhemzhitsky <szhemzhitski@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
11 days ago[SPARK-26142][SQL] Implement shuffle read metrics in SQL
Yuanjian Li [Wed, 28 Nov 2018 12:18:13 +0000 (20:18 +0800)] 
[SPARK-26142][SQL] Implement shuffle read metrics in SQL

## What changes were proposed in this pull request?

Implement `SQLShuffleMetricsReporter` on the sql side as the customized ShuffleMetricsReporter, which extended the `TempShuffleReadMetrics` and update SQLMetrics, in this way shuffle metrics can be reported in the SQL UI.

## How was this patch tested?

Add UT in SQLMetricsSuite.
Manual test locally, before:
![image](https://user-images.githubusercontent.com/4833765/48960517-30f97880-efa8-11e8-982c-92d05938fd1d.png)
after:
![image](https://user-images.githubusercontent.com/4833765/48960587-b54bfb80-efa8-11e8-8e95-7a3c8c74cc5c.png)

Closes #23128 from xuanyuanking/SPARK-26142.

Lead-authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Co-authored-by: liyuanjian <liyuanjian@baidu.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
11 days ago[SPARK-26021][SQL][FOLLOWUP] add test for special floating point values
Wenchen Fan [Wed, 28 Nov 2018 08:21:42 +0000 (16:21 +0800)] 
[SPARK-26021][SQL][FOLLOWUP] add test for special floating point values

## What changes were proposed in this pull request?

a followup of https://github.com/apache/spark/pull/23043 . Add a test to show the minor behavior change introduced by #23043 , and add migration guide.

## How was this patch tested?

a new test

Closes #23141 from cloud-fan/follow.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
11 days ago[SPARK-26159] Codegen for LocalTableScanExec and RDDScanExec
Juliusz Sompolski [Wed, 28 Nov 2018 05:37:11 +0000 (13:37 +0800)] 
[SPARK-26159] Codegen for LocalTableScanExec and RDDScanExec

## What changes were proposed in this pull request?

Implement codegen for `LocalTableScanExec` and `ExistingRDDExec`. Refactor to share code between `LocalTableScanExec`, `ExistingRDDExec`, `InputAdapter` and `RowDataSourceScanExec`.

The difference in `doProduce` between these four was that `ExistingRDDExec` and `RowDataSourceScanExec` triggered adding an `UnsafeProjection`, while `InputAdapter` and `LocalTableScanExec` did not.

In the new trait `InputRDDCodegen` I added a flag `createUnsafeProjection` which the operators set accordingly.

Note: `LocalTableScanExec` explicitly creates its input as `UnsafeRows`, so it was obvious why it doesn't need an `UnsafeProjection`. But if an `InputAdapter` may take input that is `InternalRows` but not `UnsafeRows`, then I think it doesn't need an unsafe projection just because any other operator that is its parent would do that. That assumes that that any parent operator would always result in some `UnsafeProjection` being eventually added, and hence the output of the `WholeStageCodegen` unit would be `UnsafeRows`. If these assumptions hold, I think `createUnsafeProjection` could be set to `(parent == null)`.

Note: Do not codegen `LocalTableScanExec` when it's the only operator. `LocalTableScanExec` has optimized driver-only `executeCollect` and `executeTake` code paths that are used to return `Command` results without starting Spark Jobs. They can no longer be used if the `LocalTableScanExec` gets optimized.

## How was this patch tested?

Covered and used in existing tests.

Closes #23127 from juliuszsompolski/SPARK-26159.

Authored-by: Juliusz Sompolski <julek@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
12 days ago[SPARK-26025][K8S] Speed up docker image build on dev repo.
Marcelo Vanzin [Tue, 27 Nov 2018 17:09:16 +0000 (09:09 -0800)] 
[SPARK-26025][K8S] Speed up docker image build on dev repo.

The "build context" for a docker image - basically the whole contents of the
current directory where "docker" is invoked - can be huge in a dev build,
easily breaking a couple of gigs.

Doing that copy 3 times during the build of docker images severely slows
down the process.

This patch creates a smaller build context - basically mimicking what the
make-distribution.sh script does, so that when building the docker images,
only the necessary bits are in the current directory. For PySpark and R that
is optimized further, since those images are built based on the previously
built Spark main image.

In my current local clone, the dir size is about 2G, but with this script
the "context" sent to docker is about 250M for the main image, 1M for the
pyspark image and 8M for the R image. That speeds up the image builds
considerably.

I also snuck in a fix to the k8s integration test dependencies in the sbt
build, so that the examples are properly built (without having to do it
manually).

Closes #23019 from vanzin/SPARK-26025.

Authored-by: Marcelo Vanzin <vanzin@cloudera.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
12 days ago[SPARK-23356][SQL][TEST] add new test cases for a + 1,a + b and Rand in SetOperationSuite
caoxuewen [Tue, 27 Nov 2018 12:10:34 +0000 (20:10 +0800)] 
[SPARK-23356][SQL][TEST] add new test cases for a + 1,a + b and Rand in SetOperationSuite

## What changes were proposed in this pull request?

The purpose of this PR is supplement new test cases for a + 1,a + b and Rand in SetOperationSuite.
It comes from the comment of closed PR:#20541, thanks.

## How was this patch tested?

add new test cases

Closes #23138 from heary-cao/UnionPushTestCases.

Authored-by: caoxuewen <cao.xuewen@zte.com.cn>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
12 days ago[SPARK-26141] Enable custom metrics implementation in shuffle write
Reynold Xin [Tue, 27 Nov 2018 06:35:52 +0000 (22:35 -0800)] 
[SPARK-26141] Enable custom metrics implementation in shuffle write

## What changes were proposed in this pull request?
This is the write side counterpart to https://github.com/apache/spark/pull/23105

## How was this patch tested?
No behavior change expected, as it is a straightforward refactoring. Updated all existing test cases.

Closes #23106 from rxin/SPARK-26141.

Authored-by: Reynold Xin <rxin@databricks.com>
Signed-off-by: Reynold Xin <rxin@databricks.com>
12 days ago[SPARK-25860][SPARK-26107][FOLLOW-UP] Rule ReplaceNullWithFalseInPredicate
gatorsmile [Tue, 27 Nov 2018 04:51:32 +0000 (04:51 +0000)] 
[SPARK-25860][SPARK-26107][FOLLOW-UP] Rule ReplaceNullWithFalseInPredicate

## What changes were proposed in this pull request?

Based on https://github.com/apache/spark/pull/22857 and https://github.com/apache/spark/pull/23079, this PR did a few updates

- Limit the data types of NULL to Boolean.
- Limit the input data type of replaceNullWithFalse to Boolean; throw an exception in the testing mode.
- Create a new file for the rule ReplaceNullWithFalseInPredicate
- Update the description of this rule.

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

Closes #23139 from gatorsmile/followupSpark-25860.

Authored-by: gatorsmile <gatorsmile@gmail.com>
Signed-off-by: DB Tsai <d_tsai@apple.com>
12 days ago[SPARK-24762][SQL][FOLLOWUP] Enable Option of Product encoders
Liang-Chi Hsieh [Tue, 27 Nov 2018 04:13:48 +0000 (12:13 +0800)] 
[SPARK-24762][SQL][FOLLOWUP] Enable Option of Product encoders

## What changes were proposed in this pull request?

This is follow-up of #21732. This patch inlines `isOptionType` method.

## How was this patch tested?

Existing tests.

Closes #23143 from viirya/SPARK-24762-followup.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
13 days ago[SPARK-26140] followup: rename ShuffleMetricsReporter
Reynold Xin [Tue, 27 Nov 2018 01:01:56 +0000 (17:01 -0800)] 
[SPARK-26140] followup: rename ShuffleMetricsReporter

## What changes were proposed in this pull request?
In https://github.com/apache/spark/pull/23105, due to working on two parallel PRs at once, I made the mistake of committing the copy of the PR that used the name ShuffleMetricsReporter for the interface, rather than the appropriate one ShuffleReadMetricsReporter. This patch fixes that.

## How was this patch tested?
This should be fine as long as compilation passes.

Closes #23147 from rxin/ShuffleReadMetricsReporter.

Authored-by: Reynold Xin <rxin@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>