Commit graph

23693 commits

Author SHA1 Message Date
schintap 13c56346f4 [SPARK-25692][CORE] Remove static initialization of worker eventLoop handling chunk fetch requests within TransportContext. This fixes ChunkFetchIntegrationSuite as well
## What changes were proposed in this pull request?

How to reproduce
./build/mvn test -Dtest=org.apache.spark.network.RequestTimeoutIntegrationSuite,org.apache.spark.network.ChunkFetchIntegrationSuite -DwildcardSuites=None test
furtherRequestsDelay Test within RequestTimeoutIntegrationSuite was holding onto buffer references within worker threads. The test does close the server context but since the threads are global and there is sleep of 60 secs to fetch a specific chunk within this test, it grabs on it and waits for the client to consume but however the test is testing for a request timeout and it times out after 10 secs, so the workers are just waiting there for the buffer to be consumed by client as per my understanding.

This tends to happen if you dont have enough IO threads available on the specific system and also the order of the tests being run determines its flakyness like if ChunkFetchIntegrationSuite runs first then there is no issue. For example on mac with 8 threads these tests run fine but on my vm with 4 threads it fails. It matches the number of fetch calls in RequestTimeoutIntegrationSuite.

So do we really need it to be static?

I dont think this requires a global declaration as these threads are only required on the shuffle server end and on the client TransportContext initialization i.e the Client don't initialize these threads. The Shuffle Server initializes one TransportContext object. So, I think this is fine to be an instance variable and I see no harm.

## How was this patch tested?
Integration tests, manual tests

Closes #23700 from redsanket/SPARK-25692.

Authored-by: schintap <schintap@oath.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-02-05 10:43:43 -08:00
sandeep-katta 1dd7419702 [SPARK-26758][CORE] Idle Executors are not getting killed after spark.dynamiAllocation.executorIdleTimeout value
## What changes were proposed in this pull request?

**updateAndSyncNumExecutorsTarget**  API should be called after **initializing** flag is unset
## How was this patch tested?
Added UT and also manually tested

After Fix
![afterfix](https://user-images.githubusercontent.com/35216143/51983136-ed4a5000-24bd-11e9-90c8-c4a562c17a4b.png)

Closes #23697 from sandeep-katta/executorIssue.

Authored-by: sandeep-katta <sandeep.katta2007@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-02-04 20:13:22 -08:00
seancxmao 755f9c2076 [SPARK-26813][BUILD] Consolidate java version across language compilers and build tools
## What changes were proposed in this pull request?
The java version here means versions of javac source, javac target, scalac target. They could be consolidated as a single version (currently 1.8)

|      |javac|scalac   |
|------|-----|---------|
|source|1.8  |2.12/2.11|
|target|1.8  |1.8      |

The current issues are as follows

* Maven build defines a single property (`java.version`) to specify java version while SBT build defines different properties for javac (`javacJVMVersion`) and scalac (`scalacJVMVersion`). SBT build should use a single property as Maven build does.
* Furthermore, it's better for SBT build to refer to `java.version` defined by Maven build. This is possible since we've already been using sbt-pom-reader.

## How was this patch tested?
Tested locally.

```
build/mvn clean compile
build/sbt clean compile
```

Closes #23724 from seancxmao/specify-java-version-once.

Authored-by: seancxmao <seancxmao@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-02-04 08:56:24 -06:00
Stavros Kontopoulos 196ca0c8f5 [SPARK-26603][K8S] Update minikube backend
## What changes were proposed in this pull request?

- Covers latest minikube versions.
- keeps the older version support

Note: While I was facing disk pressure issues locally on machine, I noticed minikube status command would report that everything was working fine even if some kube-system pods were not up. I don't think the output is 100% reliable but it is good enough for most cases.

## How was this patch tested?

Run it against latest version of minikube (v0.32.0).

Author: Stavros Kontopoulos <stavros.kontopoulos@lightbend.com>

Closes #23520 from skonto/update-mini-backend.
2019-02-03 17:15:20 -08:00
zhoukang 255faaf343 [SPARK-26751][SQL] Fix memory leak when statement run in background and throw exception which is not HiveSQLException
## What changes were proposed in this pull request?
When we run in background and we get exception which is not HiveSQLException,
we may encounter memory leak since handleToOperation will not removed correctly.
The reason is below:
1. When calling operation.run() in HiveSessionImpl#executeStatementInternal we throw an exception which is not HiveSQLException
2. Then the opHandle generated by SparkSQLOperationManager will not be added into opHandleSet of HiveSessionImpl , and operationManager.closeOperation(opHandle) will not be called
3. When we close the session we will also call operationManager.closeOperation(opHandle),since we did not add this opHandle into the opHandleSet.

For the reasons above,the opHandled will always in SparkSQLOperationManager#handleToOperation,which will cause memory leak.
More details and a case has attached on https://issues.apache.org/jira/browse/SPARK-26751
This patch will always throw HiveSQLException when running in background

## How was this patch tested?
Exist UT

Closes #23673 from caneGuy/zhoukang/fix-hivesessionimpl-leak.

Authored-by: zhoukang <zhoukang199191@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-02-03 08:45:57 -06:00
Hyukjin Kwon dfb880951a [SPARK-26818][ML] Make MLEvents JSON ser/de safe
## What changes were proposed in this pull request?

Currently, it looks it's not going to cause any virtually effective problem apparently (if I didn't misread the codes).

I see one place that JSON formatted events are being used.

ec506bd30c/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala (L148)

It's okay because it just logs when the exception is ignorable

9690eba16e/core/src/main/scala/org/apache/spark/util/ListenerBus.scala (L111)

I guess it should be best to stay safe - I don't want this unstable experimental feature breaks anything in any case. It also disables `logEvent` in `SparkListenerEvent` for the same reason.

This is also to match SQL execution events side:

ca545f7941/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala (L41-L57)

to make ML events JSON ser/de safe.

## How was this patch tested?

Manually tested, and unit tests were added.

Closes #23728 from HyukjinKwon/SPARK-26818.

Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-02-03 21:19:35 +08:00
Maxim Gekk 96c6c295cc [SPARK-26805][SQL] Eliminate double checking of stringToDate and stringToTimestamp inputs
## What changes were proposed in this pull request?

In the PR, I propose to eliminate checking of parsed segments inside of the `stringToDate` and `stringToTimestamp` because such checking is already performed while constructing *java.time* classes, in particular inside of `LocalDate` and `LocalTime`. As a consequence of removing the explicit checks, the `isInvalidDate` method is not needed any more, and it was removed from `DateTimeUtils`.

## How was this patch tested?

This was tested by `DateExpressionsSuite`, `DateFunctionsSuite`, `DateTimeUtilsSuite` and `CastSuite`.

Closes #23717 from MaxGekk/datetimeutils-refactoring.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-02-02 18:20:16 -06:00
Ryan Blue f72d217788
[SPARK-26677][BUILD] Update Parquet to 1.10.1 with notEq pushdown fix.
## What changes were proposed in this pull request?

Update to Parquet Java 1.10.1.

## How was this patch tested?

Added a test from HyukjinKwon that validates the notEq case from SPARK-26677.

Closes #23704 from rdblue/SPARK-26677-fix-noteq-parquet-bug.

Lead-authored-by: Ryan Blue <blue@apache.org>
Co-authored-by: Hyukjin Kwon <gurwls223@apache.org>
Co-authored-by: Ryan Blue <rdblue@users.noreply.github.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2019-02-02 09:17:52 -08:00
Hyukjin Kwon a5427a0067 [MINOR][SQL][DOCS] Reformat the tables in SQL migration guide
## What changes were proposed in this pull request?

1. Reformat the tables to be located with a proper indentation under the corresponding item to be consistent.

2. Fix **Table 2**'s contents to be more readable with code blocks.

### Table 1

**Before:**

![screen shot 2019-02-02 at 11 37 30 am](https://user-images.githubusercontent.com/6477701/52159396-f1a18380-26de-11e9-9dca-f56b19f22bb4.png)

**After:**

![screen shot 2019-02-02 at 11 32 39 am](https://user-images.githubusercontent.com/6477701/52159370-7d66e000-26de-11e9-9e6d-81cf73691c05.png)

### Table 2

**Before:**

![screen shot 2019-02-02 at 11 35 51 am](https://user-images.githubusercontent.com/6477701/52159401-0ed65200-26df-11e9-8b0e-86d005c233b5.png)

**After:**

![screen shot 2019-02-02 at 11 32 44 am](https://user-images.githubusercontent.com/6477701/52159372-7f30a380-26de-11e9-8c04-a88c74b78cff.png)

## How was this patch tested?

Manually built the doc.

Closes #23723 from HyukjinKwon/minor-doc-fix.

Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-02-02 23:45:46 +08:00
Maxim Gekk b85974db85 [SPARK-26651][SQL][DOC] Collapse notes related to java.time API
## What changes were proposed in this pull request?

Collapsed notes about using Java 8 API for date/timestamp manipulations and Proleptic Gregorian calendar in the SQL migration guide.

Closes #23722 from MaxGekk/collapse-notes.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-02-02 11:17:33 +08:00
Boris Shminke 75ea89ad94 [SPARK-18161][PYTHON] Update cloudpickle to v0.6.1
## What changes were proposed in this pull request?

In this PR we've done two things:
1) updated the Spark's copy of cloudpickle to 0.6.1 (current stable)
The main reason Spark stayed with cloudpickle 0.4.x was that the default pickle protocol was changed in later versions.

2) started using pickle.HIGHEST_PROTOCOL for both Python 2 and Python 3 for serializers and broadcast
[Pyrolite](https://github.com/irmen/Pyrolite) has such Pickle protocol version support: reading: 0,1,2,3,4; writing: 2.

## How was this patch tested?

Jenkins tests.

Authors: Sloane Simmons, Boris Shminke

This contribution is original work of Sloane Simmons and Boris Shminke and they licensed it to the project under the project's open source license.

Closes #20691 from inpefess/pickle_protocol_4.

Lead-authored-by: Boris Shminke <boris@shminke.me>
Co-authored-by: singularperturbation <sloanes.k@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-02-02 10:49:45 +08:00
xiaodeshan a0faabf7b5 [SPARK-26714][CORE][WEBUI] Show 0 partition job in WebUI
## What changes were proposed in this pull request?

When the job's partiton is zero, it will still get a jobid but not shown in ui. It's strange. This PR is to show this job in ui.

Example:
In bash:
mkdir -p /home/test/testdir

sc.textFile("/home/test/testdir")

Some logs:

```
19/01/24 17:26:19 INFO FileInputFormat: Total input paths to process : 0
19/01/24 17:26:19 INFO SparkContext: Starting job: collect at WordCount.scala:9
19/01/24 17:26:19 INFO DAGScheduler: Job 0 finished: collect at WordCount.scala:9, took 0.003735 s
```

## How was this patch tested?

UT

Closes #23637 from deshanxiao/spark-26714.

Authored-by: xiaodeshan <xiaodeshan@xiaomi.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-02-01 18:38:27 -06:00
liuxian 421ff6f60e [MINOR][DOC] Writing to partitioned Hive metastore Parquet tables is not supported for Spark SQL
## What changes were proposed in this pull request?

Even if `spark.sql.hive.convertMetastoreParquet` is true,  when writing to partitioned Hive metastore
Parquet tables,  Spark SQL still  can not use its own Parquet support instead of Hive SerDe.

Related code:
 d53e11ffce/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala (L198)
## How was this patch tested?
N/A

Closes #23671 from 10110346/parquetdoc.

Authored-by: liuxian <liu.xian3@zte.com.cn>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-02-01 18:34:13 -06:00
Sean Owen 8171b156eb [SPARK-26771][CORE][GRAPHX] Make .unpersist(), .destroy() consistently non-blocking by default
## What changes were proposed in this pull request?

Make .unpersist(), .destroy() non-blocking by default and adjust callers to request blocking only where important.

This also adds an optional blocking argument to Pyspark's RDD.unpersist(), which never had one.

## How was this patch tested?

Existing tests.

Closes #23685 from srowen/SPARK-26771.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-02-01 18:29:55 -06:00
Huaxin Gao 5bb9647e10 [SPARK-26754][PYTHON] Add hasTrainingSummary to replace duplicate code in PySpark
## What changes were proposed in this pull request?

Python version of https://github.com/apache/spark/pull/17654

## How was this patch tested?

Existing Python unit test

Closes #23676 from huaxingao/spark26754.

Authored-by: Huaxin Gao <huaxing@us.ibm.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-02-01 17:29:58 -06:00
Shixiong Zhu 03a928cbec
[SPARK-26806][SS] EventTimeStats.merge should handle zeros correctly
## What changes were proposed in this pull request?

Right now, EventTimeStats.merge doesn't handle `zero.merge(zero)` correctly. This will make `avg` become `NaN`. And whatever gets merged with the result of `zero.merge(zero)`, `avg` will still be `NaN`. Then finally, we call `NaN.toLong` and get `0`, and the user will see the following incorrect report:

```
"eventTime" : {
    "avg" : "1970-01-01T00:00:00.000Z",
    "max" : "2019-01-31T12:57:00.000Z",
    "min" : "2019-01-30T18:44:04.000Z",
    "watermark" : "1970-01-01T00:00:00.000Z"
  }
```

This issue was reported by liancheng .

This PR fixes the above issue.

## How was this patch tested?

The new unit tests.

Closes #23718 from zsxwing/merge-zero.

Authored-by: Shixiong Zhu <zsxwing@gmail.com>
Signed-off-by: Shixiong Zhu <zsxwing@gmail.com>
2019-02-01 11:15:05 -08:00
Liupengcheng aea5f50646 [SPARK-26525][SHUFFLE] Fast release ShuffleBlockFetcherIterator on completion of the iteration
## What changes were proposed in this pull request?

Currently, spark would not release ShuffleBlockFetcherIterator until the whole task finished.In some conditions, it incurs memory leak.

An example is `rdd.repartition(m).coalesce(n, shuffle = false).save`, each `ShuffleBlockFetcherIterator` contains  some metas about mapStatus(`blocksByAddress`) and each resultTask will keep n(max to shuffle partitions) shuffleBlockFetcherIterator and the memory would never released until the task completion, for they are referenced by the completion callbacks of TaskContext. In some case, it may take huge memory and incurs OOM.

Actually, We can release ShuffleBlockFetcherIterator as soon as it's consumed.
This PR is to resolve this problem.

## How was this patch tested?

unittest

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

Closes #23438 from liupc/Fast-release-shuffleblockfetcheriterator.

Lead-authored-by: Liupengcheng <liupengcheng@xiaomi.com>
Co-authored-by: liupengcheng <liupengcheng@xiaomi.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-02-01 13:47:14 +08:00
wuyi 8f968b4c06 [SPARK-26730][SQL] Strip redundant AssertNotNull for ExpressionEncoder's serializer
## What changes were proposed in this pull request?

For types like Product, we've already add AssertNotNull when we construct serializer(see code below), so we could strip redundant AssertNotNull for those types.

```
val fieldValue = Invoke(
    AssertNotNull(inputObject, walkedTypePath), fieldName, dataTypeFor(fieldType),
    returnNullable = !fieldType.typeSymbol.asClass.isPrimitive)
```
## How was this patch tested?

Existed.

Closes #23651 from Ngone51/dev-strip-redundant-assertnotnull-for-ecnoder-serializer.

Authored-by: wuyi <ngone_5451@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-02-01 10:48:37 +08:00
Hyukjin Kwon cdd694c52b [SPARK-7721][INFRA] Run and generate test coverage report from Python via Jenkins
## What changes were proposed in this pull request?

### Background

For the current status, the test script that generates coverage information was merged
into Spark, https://github.com/apache/spark/pull/20204

So, we can generate the coverage report and site by, for example:

```
run-tests-with-coverage --python-executables=python3 --modules=pyspark-sql
```

like `run-tests` script in `./python`.

### Proposed change

The next step is to host this coverage report via `github.io` automatically
by Jenkins (see https://spark-test.github.io/pyspark-coverage-site/).

This uses my testing account for Spark, spark-test, which is shared to Felix and Shivaram a long time ago for testing purpose including AppVeyor.

To cut this short, this PR targets to run the coverage in
[spark-master-test-sbt-hadoop-2.7](https://amplab.cs.berkeley.edu/jenkins/job/spark-master-test-sbt-hadoop-2.7/)

In the specific job, it will clone the page, and rebase the up-to-date PySpark test coverage from the latest commit. For instance as below:

```bash
# Clone PySpark coverage site.
git clone https://github.com/spark-test/pyspark-coverage-site.git

# Remove existing HTMLs.
rm -fr pyspark-coverage-site/*

# Copy generated coverage HTMLs.
cp -r .../python/test_coverage/htmlcov/* pyspark-coverage-site/

# Check out to a temporary branch.
git symbolic-ref HEAD refs/heads/latest_branch

# Add all the files.
git add -A

# Commit current HTMLs.
git commit -am "Coverage report at latest commit in Apache Spark"

# Delete the old branch.
git branch -D gh-pages

# Rename the temporary branch to master.
git branch -m gh-pages

# Finally, force update to our repository.
git push -f origin gh-pages
```

So, it is a one single up-to-date coverage can be shown in the `github-io` page. The commands above were manually tested.

### TODOs

- [x] Write a draft HyukjinKwon
- [x] `pip install coverage` to all python implementations (pypy, python2, python3) in Jenkins workers  - shaneknapp
- [x] Set hidden `SPARK_TEST_KEY` for spark-test's password in Jenkins via Jenkins's feature
 This should be set in both PR builder and `spark-master-test-sbt-hadoop-2.7` so that later other PRs can test and fix the bugs - shaneknapp
- [x] Set an environment variable that indicates `spark-master-test-sbt-hadoop-2.7` so that that specific build can report and update the coverage site - shaneknapp
- [x] Make PR builder's test passed HyukjinKwon
- [x] Fix flaky test related with coverage HyukjinKwon
  -  6 consecutive passes out of 7 runs

This PR will be co-authored with me and shaneknapp

## How was this patch tested?

It will be tested via Jenkins.

Closes #23117 from HyukjinKwon/SPARK-7721.

Lead-authored-by: Hyukjin Kwon <gurwls223@apache.org>
Co-authored-by: hyukjinkwon <gurwls223@apache.org>
Co-authored-by: shane knapp <incomplete@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-02-01 10:18:08 +08:00
bscan e44f308593 [SPARK-26787] Fix standardizeLabels error message in WeightedLeastSquares
Error message falsely states standardization=True is causing a problem, even when standardization=False. The real issue is standardizeLabels=True, which is set automatically in LinearRegression and not currently available in the Public API.

## What changes were proposed in this pull request?

A simple change to an error message. More details here: https://jira.apache.org/jira/browse/SPARK-26787

## How was this patch tested?

This does not change any functionality.

Closes #23705 from bscan/bscan-errormsg-1.

Authored-by: bscan <brianjscannell@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-01-31 19:50:18 -06:00
Huaxin Gao f7d87b1685 [SPARK-25997][ML] add Python example code for Power Iteration Clustering in spark.ml
## What changes were proposed in this pull request?

Add python example for Power Iteration Clustering in spark.ml

## How was this patch tested?

Manually tested

Closes #22996 from huaxingao/spark-25997.

Authored-by: Huaxin Gao <huaxing@us.ibm.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-01-31 19:33:44 -06:00
SongYadong 0fe9c144fd [DOC][MINOR] Add metrics instance 'mesos_cluster' to monitoring doc
## What changes were proposed in this pull request?

Metrics instance "mesos_cluster" exists in spark, but not mentioned in monitoring.md. This PR add it.

## How was this patch tested?

Manually test.

Closes #23691 from SongYadong/doc_mesos_metrics_inst.

Authored-by: SongYadong <song.yadong1@zte.com.cn>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-01-31 18:30:17 -06:00
Huon Wilson da526985c7 [SPARK-26757][GRAPHX] Return 0 for count on empty Edge/Vertex RDDs
## What changes were proposed in this pull request?

Previously a "java.lang.UnsupportedOperationException: empty
collection" exception would be thrown due to using `reduce`, rather
than `fold` or similar that can tolerate empty RDDs.

This behaviour has existed for the Vertex RDDs since it was introduced
in b30e0ae035. It seems this behaviour
was inherited by the Edge RDDs via copy-paste in
ee29ef3800.

## How was this patch tested?

Two new unit tests.

Closes #23681 from huonw/empty-graphx.

Authored-by: Huon Wilson <Huon.Wilson@data61.csiro.au>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-01-31 17:27:11 -06:00
seancxmao 2514163366 [SPARK-26799][BUILD] Make ANTLR v4 version consistent between Maven and SBT
## What changes were proposed in this pull request?
Currently ANTLR v4 versions used by Maven and SBT are slightly different. Maven uses `4.7.1` while SBT uses `4.7`.

* Maven(`pom.xml`): `<antlr4.version>4.7.1</antlr4.version>`
* SBT(`project/SparkBuild.scala`): `antlr4Version in Antlr4 := "4.7"`

We should make Maven and SBT use a single version. Furthermore we'd better specify antlr4 version in one place to avoid mismatch between Maven and SBT in the future.

This PR lets SBT use antlr4 version specified in Maven POM file, rather than specify its own antlr4 version. This is in the same as how `hadoop.version` is specified in `project/SparkBuild.scala`

## How was this patch tested?
Test locally.

After run `sbt compile`, Java files generated by ANTLR are located at:

```
sql/catalyst/target/scala-2.12/src_managed/main/antlr4/org/apache/spark/sql/catalyst/parser/*.java
```

These Java files have a comment at the head. We can see now SBT uses ANTLR `4.7.1`.

```
// Generated from .../spark/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 by ANTLR 4.7.1
```

Closes #23713 from seancxmao/antlr4-version-consistent.

Authored-by: seancxmao <seancxmao@gmail.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-01-31 14:39:32 -08:00
韩田田00222924 f4a17e916b [SPARK-26726] Synchronize the amount of memory used by the broadcast variable to the UI display
…not synchronized to the UI display

## What changes were proposed in this pull request?
The amount of memory used by the broadcast variable is not synchronized to the UI display.
I added the case for BroadcastBlockId and updated the memory usage.

## How was this patch tested?

We can test this patch with unit tests.

Closes #23649 from httfighter/SPARK-26726.

Lead-authored-by: 韩田田00222924 <han.tiantian@zte.com.cn>
Co-authored-by: han.tiantian@zte.com.cn <han.tiantian@zte.com.cn>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-01-31 09:17:33 -08:00
Gengliang Wang df4c53e44b [SPARK-26673][SQL] File source V2 writes: create framework and migrate ORC
## What changes were proposed in this pull request?

Create a framework for write path of File Source V2.
Also, migrate write path of ORC to V2.

Supported:
* Write to file as Dataframe

Not Supported:
* Partitioning, which is still under development in the data source V2 project.
* Bucketing, which is still under development in the data source V2 project.
* Catalog.

## How was this patch tested?

Unit test

Closes #23601 from gengliangwang/orc_write.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-01-31 21:29:01 +08:00
Ilya Matiach b3b62ba303 [SPARK-19591][ML][MLLIB][FOLLOWUP] Add sample weights to decision trees - fix tolerance
This is a follow-up to PR:
https://github.com/apache/spark/pull/21632

## What changes were proposed in this pull request?

This PR tunes the tolerance used for deciding whether to add zero feature values to a value-count map (where the key is the feature value and the value is the weighted count of those feature values).
In the previous PR the tolerance scaled by the square of the unweighted number of samples, which is too aggressive for a large number of unweighted samples.  Unfortunately using just "Utils.EPSILON * unweightedNumSamples" is not enough either, so I multiplied that by a factor tuned by the testing procedure below.

## How was this patch tested?

This involved manually running the sample weight tests for decision tree regressor to see whether the tolerance was large enough to exclude zero feature values.

Eg in SBT:
```
./build/sbt
> project mllib
> testOnly *DecisionTreeRegressorSuite -- -z "training with sample weights"
```

For validation, I added a print inside the if in the code below and validated that the tolerance was large enough so that we would not include zero features (which don't exist in that test):
```
      val valueCountMap = if (weightedNumSamples - partNumSamples > tolerance) {
        print("should not print this")
        partValueCountMap + (0.0 -> (weightedNumSamples - partNumSamples))
      } else {
        partValueCountMap
      }
```

Closes #23682 from imatiach-msft/ilmat/sample-weights-tol.

Authored-by: Ilya Matiach <ilmat@microsoft.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-01-31 05:44:55 -06:00
Huaxin Gao bc6f191451 [SPARK-24779][R] Add map_concat / map_from_entries / an option in months_between UDF to disable rounding-off
## What changes were proposed in this pull request?

Add the R version of map_concat / map_from_entries / an option in months_between UDF to disable rounding-off

## How was this patch tested?

Add test in test_sparkSQL.R

Closes #21835 from huaxingao/spark-24779.

Authored-by: Huaxin Gao <huaxing@us.ibm.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-01-31 19:38:32 +08:00
Wenchen Fan 0e2c487459 [SPARK-26448][SQL][FOLLOWUP] should not normalize grouping expressions for final aggregate
## What changes were proposed in this pull request?

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

`AggUtils.createAggregate` is not the right place to normalize the grouping expressions, as final aggregate is also created by it. The grouping expressions of final aggregate should be attributes which refer to the grouping expressions in partial aggregate.

This PR moves the normalization to the caller side of `AggUtils`.

## How was this patch tested?

existing tests

Closes #23692 from cloud-fan/follow.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-01-31 16:20:18 +08:00
Hyukjin Kwon 0d77d575e1 [MINOR][DOCS] Add a note that 'spark.executor.pyspark.memory' is dependent on 'resource'
## What changes were proposed in this pull request?

This PR adds a note that explicitly `spark.executor.pyspark.memory` is dependent on resource module's behaviours at Python memory usage.

For instance, I at least see some difference at https://github.com/apache/spark/pull/21977#discussion_r251220966

## How was this patch tested?

Manually built the doc.

Closes #23664 from HyukjinKwon/note-resource-dependent.

Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-01-31 15:51:40 +08:00
Gengliang Wang 308996bc72 [SPARK-26716][SPARK-26765][FOLLOWUP][SQL] Clean up schema validation methods and override toString method in Avro
## What changes were proposed in this pull request?

In #23639, the API `supportDataType` is refactored. We should also remove the method `verifyWriteSchema` and `verifyReadSchema` in `DataSourceUtils`.

Since the error message use `FileFormat.toString` to specify the data source naming,  this PR also overriding the `toString` method in `AvroFileFormat`.

## How was this patch tested?

Unit test.

Closes #23699 from gengliangwang/SPARK-26716-followup.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-01-31 15:44:44 +08:00
Hyukjin Kwon d4d6df2f7d [SPARK-26745][SQL] Revert count optimization in JSON datasource by SPARK-24959
## What changes were proposed in this pull request?

This PR reverts JSON count optimization part of #21909.

We cannot distinguish the cases below without parsing:

```
[{...}, {...}]
```

```
[]
```

```
{...}
```

```bash
# empty string
```

when we `count()`. One line (input: IN) can be, 0 record, 1 record and multiple records and this is dependent on each input.

See also https://github.com/apache/spark/pull/23665#discussion_r251276720.

## How was this patch tested?

Manually tested.

Closes #23667 from HyukjinKwon/revert-SPARK-24959.

Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-01-31 14:32:31 +08:00
Dongjoon Hyun aeff69bd87
[SPARK-24360][SQL] Support Hive 3.1 metastore
## What changes were proposed in this pull request?

Hive 3.1.1 is released. This PR aims to support Hive 3.1.x metastore.
Please note that Hive 3.0.0 Metastore is skipped intentionally.

## How was this patch tested?

Pass the Jenkins with the updated test cases including 3.1.

Closes #23694 from dongjoon-hyun/SPARK-24360-3.1.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2019-01-30 20:33:21 -08:00
Wenchen Fan d8d2736fd1 [SPARK-26708][SQL][FOLLOWUP] put the special handling of non-cascade uncache in the uncache method
## What changes were proposed in this pull request?

This is a follow up of https://github.com/apache/spark/pull/23644/files , to make these methods less coupled with each other.

## How was this patch tested?

existing tests

Closes #23687 from cloud-fan/cache.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-01-31 11:04:33 +08:00
Jungtaek Lim (HeartSaVioR) ae5b2a6a92 [SPARK-26311][CORE] New feature: apply custom log URL pattern for executor log URLs in SHS
## What changes were proposed in this pull request?

This patch proposes adding a new configuration on SHS: custom executor log URL pattern. This will enable end users to replace executor logs to other than RM provide, like external log service, which enables to serve executor logs when NodeManager becomes unavailable in case of YARN.

End users can build their own of custom executor log URLs with pre-defined patterns which would be vary on each resource manager. This patch adds some patterns to YARN resource manager. (For others, there's even no executor log url available so cannot define patterns as well.)

Please refer the doc change as well as added UTs in this patch to see how to set up the feature.

## How was this patch tested?

Added UT, as well as manual test with YARN cluster

Closes #23260 from HeartSaVioR/SPARK-26311.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-01-30 11:52:30 -08:00
ankurgupta 25b97a41ce [SPARK-26753][CORE] Fixed custom log levels for spark-shell by using Filter instead of Threshold
This fix replaces the Threshold with a Filter for ConsoleAppender which checks
to ensure that either the logLevel is greater than thresholdLevel (shell log
level) or the log originated from a custom defined logger. In these cases, it
lets a log event go through, otherwise it doesn't.

1. Ensured that custom log level works when set by default (via log4j.properties)
2. Ensured that logs are not printed twice when log level is changed by setLogLevel
3. Ensured that custom logs are printed when log level is changed back by setLogLevel

Closes #23675 from ankuriitg/ankurgupta/SPARK-26753.

Authored-by: ankurgupta <ankur.gupta@cloudera.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-01-30 10:54:24 -08:00
Marcelo Vanzin a8da41061f [SPARK-25689][CORE] Follow up: don't get delegation tokens when kerberos not available.
This avoids trying to get delegation tokens when a TGT is not available, e.g.
when running in yarn-cluster mode without a keytab. That would result in an
error since that is not allowed.

Tested with some (internal) integration tests that started failing with the
patch for SPARK-25689.

Closes #23689 from vanzin/SPARK-25689.followup.

Authored-by: Marcelo Vanzin <vanzin@cloudera.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-01-30 09:52:50 -08:00
Marcelo Vanzin 6a2f3dcc2b [SPARK-26732][CORE][TEST] Wait for listener bus to process events in SparkContextInfoSuite.
Otherwise the RDD data may be out of date by the time the test tries to check it.

Tested with an artificial delay inserted in AppStatusListener.

Closes #23654 from vanzin/SPARK-26732.

Authored-by: Marcelo Vanzin <vanzin@cloudera.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2019-01-31 00:10:23 +09:00
Bruce Robbins 7781c6fd73 [SPARK-26378][SQL] Restore performance of queries against wide CSV/JSON tables
## What changes were proposed in this pull request?

After [recent changes](11e5f1bcd4) to CSV parsing to return partial results for bad CSV records, queries of wide CSV tables slowed considerably. That recent change resulted in every row being recreated, even when the associated input record had no parsing issues and the user specified no corrupt record field in his/her schema.

The change to FailureSafeParser.scala also impacted queries against wide JSON tables as well.

In this PR, I propose that a row should be recreated only if columns need to be shifted due to the existence of a corrupt column field in the user-supplied schema. Otherwise, the code should use the row as-is (For CSV input, it will have values for the columns that could be converted, and also null values for columns that could not be converted).

See benchmarks below. The CSV benchmark for 1000 columns went from 120144 ms to 89069 ms, a savings of 25% (this only brings the cost down to baseline levels. Again, see benchmarks below).

Similarly, the JSON benchmark for 1000 columns (added in this PR) went from 109621 ms to 80871 ms, also a savings of 25%.

Still, partial results functionality is preserved:

<pre>
bash-3.2$ cat test2.csv
"hello",1999-08-01,"last"
"there","bad date","field"
"again","2017-11-22","in file"
bash-3.2$ bin/spark-shell
...etc...
scala> val df = spark.read.schema("a string, b date, c string").csv("test2.csv")
df: org.apache.spark.sql.DataFrame = [a: string, b: date ... 1 more field]
scala> df.show
+-----+----------+-------+
|    a|         b|      c|
+-----+----------+-------+
|hello|1999-08-01|   last|
|there|      null|  field|
|again|2017-11-22|in file|
+-----+----------+-------+
scala> val df = spark.read.schema("badRecord string, a string, b date, c string").
     | option("columnNameOfCorruptRecord", "badRecord").
     | csv("test2.csv")
df: org.apache.spark.sql.DataFrame = [badRecord: string, a: string ... 2 more fields]
scala> df.show
+--------------------+-----+----------+-------+
|           badRecord|    a|         b|      c|
+--------------------+-----+----------+-------+
|                null|hello|1999-08-01|   last|
|"there","bad date...|there|      null|  field|
|                null|again|2017-11-22|in file|
+--------------------+-----+----------+-------+
scala>
</pre>

### CSVBenchmark Benchmarks:

baseline = commit before partial results change
PR = this PR
master = master branch

[baseline_CSVBenchmark-results.txt](https://github.com/apache/spark/files/2697109/baseline_CSVBenchmark-results.txt)
[pr_CSVBenchmark-results.txt](https://github.com/apache/spark/files/2697110/pr_CSVBenchmark-results.txt)
[master_CSVBenchmark-results.txt](https://github.com/apache/spark/files/2697111/master_CSVBenchmark-results.txt)

### JSONBenchmark Benchmarks:

baseline = commit before partial results change
PR = this PR
master = master branch

[baseline_JSONBenchmark-results.txt](https://github.com/apache/spark/files/2711040/baseline_JSONBenchmark-results.txt)
[pr_JSONBenchmark-results.txt](https://github.com/apache/spark/files/2711041/pr_JSONBenchmark-results.txt)
[master_JSONBenchmark-results.txt](https://github.com/apache/spark/files/2711042/master_JSONBenchmark-results.txt)

## How was this patch tested?

- All SQL unit tests.
- Added 2 CSV benchmarks
- Python core and SQL tests

Closes #23336 from bersprockets/csv-wide-row-opt2.

Authored-by: Bruce Robbins <bersprockets@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-01-30 15:15:29 +08:00
Hyukjin Kwon c08021cd87 [SPARK-26776][PYTHON] Reduce Py4J communication cost in PySpark's execution barrier check
## What changes were proposed in this pull request?

I am investigating flaky tests. I realised that:

```
      File "/home/jenkins/workspace/SparkPullRequestBuilder/python/pyspark/rdd.py", line 2512, in __init__
        self.is_barrier = prev._is_barrier() or isFromBarrier
      File "/home/jenkins/workspace/SparkPullRequestBuilder/python/pyspark/rdd.py", line 2412, in _is_barrier
        return self._jrdd.rdd().isBarrier()
      File "/home/jenkins/workspace/SparkPullRequestBuilder/python/lib/py4j-0.10.8.1-src.zip/py4j/java_gateway.py", line 1286, in __call__
        answer, self.gateway_client, self.target_id, self.name)
      File "/home/jenkins/workspace/SparkPullRequestBuilder/python/lib/py4j-0.10.8.1-src.zip/py4j/protocol.py", line 342, in get_return_value
        return OUTPUT_CONVERTER[type](answer[2:], gateway_client)
      File "/home/jenkins/workspace/SparkPullRequestBuilder/python/lib/py4j-0.10.8.1-src.zip/py4j/java_gateway.py", line 2492, in <lambda>
        lambda target_id, gateway_client: JavaObject(target_id, gateway_client))
      File "/home/jenkins/workspace/SparkPullRequestBuilder/python/lib/py4j-0.10.8.1-src.zip/py4j/java_gateway.py", line 1324, in __init__
        ThreadSafeFinalizer.add_finalizer(key, value)
      File "/home/jenkins/workspace/SparkPullRequestBuilder/python/lib/py4j-0.10.8.1-src.zip/py4j/finalizer.py", line 43, in add_finalizer
        cls.finalizers[id] = weak_ref
      File "/usr/lib64/pypy-2.5.1/lib-python/2.7/threading.py", line 216, in __exit__
        self.release()
      File "/usr/lib64/pypy-2.5.1/lib-python/2.7/threading.py", line 208, in release
        self.__block.release()
    error: release unlocked lock
```

I assume it might not be directly related with the test itself but I noticed that it `prev._is_barrier()` attempts to access via Py4J.

Accessing via Py4J is expensive. Therefore, this PR proposes to avoid Py4J access when `isFromBarrier` is `True`.

## How was this patch tested?

Unittests should cover this.

Closes #23690 from HyukjinKwon/minor-barrier.

Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-01-30 12:24:27 +08:00
ryne.yang fbc3c5e8a3
[SPARK-26718][SS] Fixed integer overflow in SS kafka rateLimit calculation
## What changes were proposed in this pull request?

Fix the integer overflow issue in rateLimit.

## How was this patch tested?

Pass the Jenkins with newly added UT for the possible case where integer could be overflowed.

Closes #23666 from linehrr/master.

Authored-by: ryne.yang <ryne.yang@acuityads.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2019-01-29 10:58:10 -08:00
Gengliang Wang 1beed0d7c2 [SPARK-26765][SQL] Avro: Validate input and output schema
## What changes were proposed in this pull request?

The API `supportDataType` in `FileFormat` helps to validate the output/input schema before exection starts. So that we can avoid some invalid data source IO, and users can see clean error messages.

This PR is to override the validation API in Avro data source.
Also, as per the spec of Avro(https://avro.apache.org/docs/1.8.2/spec.html), `NullType` is supported. This PR fixes the handling of `NullType`.

## How was this patch tested?

Unit test

Closes #23684 from gengliangwang/avroSupportDataType.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-01-30 00:17:33 +08:00
Liang-Chi Hsieh 66afd869d1
[SPARK-26702][SQL][TEST] Create a test trait for Parquet and Orc test
## What changes were proposed in this pull request?

For making test suite supporting both Parquet and Orc by reusing test cases, this patch extracts the methods for testing. For example, if we need to test a common feature shared by Parquet and Orc, we should be able to write test cases once and reuse them to test both formats.

This patch extracts the methods for testing and uses a variable `dataSourceName` to set up data format to test against with.

## How was this patch tested?

Existing tests.

Closes #23628 from viirya/datasource-test.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2019-01-29 07:31:42 -08:00
Liang-Chi Hsieh 33107897ad [SPARK-11215][ML] Add multiple columns support to StringIndexer
## What changes were proposed in this pull request?

This takes over #19621 to add multi-column support to StringIndexer:

1. Supports encoding multiple columns.
2. Previously, when specifying `frequencyDesc` or `frequencyAsc` as `stringOrderType` param in `StringIndexer`, in case of equal frequency, the order of strings is undefined. After this change, the strings with equal frequency are further sorted alphabetically.

## How was this patch tested?

Added tests.

Closes #20146 from viirya/SPARK-11215.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-01-29 09:21:25 -06:00
Xianyang Liu 5d672b7f3e [SPARK-26763][SQL] Using fileStatus cache when filterPartitions
## What changes were proposed in this pull request?

We should pass the existed `fileStatusCache` to `InMemoryFileIndex` even though there aren't partition columns.

## How was this patch tested?

Existed test. Extra tests can be added if there is a requirement.

Closes #23683 from ConeyLiu/filestatuscache.

Authored-by: Xianyang Liu <xianyang.liu@intel.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-01-29 23:11:11 +08:00
Wenchen Fan e97ab1d980 [SPARK-26695][SQL] data source v2 API refactor - continuous read
## What changes were proposed in this pull request?

Following https://github.com/apache/spark/pull/23430, this PR does the API refactor for continuous read, w.r.t. the [doc](https://docs.google.com/document/d/1uUmKCpWLdh9vHxP7AWJ9EgbwB_U6T3EJYNjhISGmiQg/edit?usp=sharing)

The major changes:
1. rename `XXXContinuousReadSupport` to `XXXContinuousStream`
2. at the beginning of continuous streaming execution, convert `StreamingRelationV2` to `StreamingDataSourceV2Relation` directly, instead of `StreamingExecutionRelation`.
3. remove all the hacks as we have finished all the read side API refactor

## How was this patch tested?

existing tests

Closes #23619 from cloud-fan/continuous.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-01-29 00:07:27 -08:00
Bryan Cutler 16990f9299 [SPARK-26566][PYTHON][SQL] Upgrade Apache Arrow to version 0.12.0
## What changes were proposed in this pull request?

Upgrade Apache Arrow to version 0.12.0. This includes the Java artifacts and fixes to enable usage with pyarrow 0.12.0

Version 0.12.0 includes the following selected fixes/improvements relevant to Spark users:

* Safe cast fails from numpy float64 array with nans to integer, ARROW-4258
* Java, Reduce heap usage for variable width vectors, ARROW-4147
* Binary identity cast not implemented, ARROW-4101
* pyarrow open_stream deprecated, use ipc.open_stream, ARROW-4098
* conversion to date object no longer needed, ARROW-3910
* Error reading IPC file with no record batches, ARROW-3894
* Signed to unsigned integer cast yields incorrect results when type sizes are the same, ARROW-3790
* from_pandas gives incorrect results when converting floating point to bool, ARROW-3428
* Import pyarrow fails if scikit-learn is installed from conda (boost-cpp / libboost issue), ARROW-3048
* Java update to official Flatbuffers version 1.9.0, ARROW-3175

complete list [here](https://issues.apache.org/jira/issues/?jql=project%20%3D%20ARROW%20AND%20status%20in%20(Resolved%2C%20Closed)%20AND%20fixVersion%20%3D%200.12.0)

PySpark requires the following fixes to work with PyArrow 0.12.0

* Encrypted pyspark worker fails due to ChunkedStream missing closed property
* pyarrow now converts dates as objects by default, which causes error because type is assumed datetime64
* ArrowTests fails due to difference in raised error message
* pyarrow.open_stream deprecated
* tests fail because groupby adds index column with duplicate name

## How was this patch tested?

Ran unit tests with pyarrow versions 0.8.0, 0.10.0, 0.11.1, 0.12.0

Closes #23657 from BryanCutler/arrow-upgrade-012.

Authored-by: Bryan Cutler <cutlerb@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-01-29 14:18:45 +08:00
Takeshi Yamamuro 92706e6576
[SPARK-26747][SQL] Makes GetMapValue nullability more precise
## What changes were proposed in this pull request?
In master, `GetMapValue` nullable is always true;
cf133e6110/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeExtractors.scala (L371)

But, If input key is foldable, we could make its nullability more precise.
This fix is the same with SPARK-26637(#23566).

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

Closes #23669 from maropu/SPARK-26747.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2019-01-28 13:39:50 -08:00
Marcelo Vanzin 2a67dbfbd3 [SPARK-26595][CORE] Allow credential renewal based on kerberos ticket cache.
This change addes a new mode for credential renewal that does not require
a keytab; it uses the local ticket cache instead, so it works while the
user keeps the cache valid.

This can be useful for, e.g., people running long spark-shell sessions where
their kerberos login is kept up-to-date.

The main change to enable this behavior is in HadoopDelegationTokenManager,
with a small change in the HDFS token provider. The other changes are to avoid
creating duplicate tokens when submitting the application to YARN; they allow
the tokens from the scheduler to be sent to the YARN AM, reducing the round trips
to HDFS.

For that, the scheduler initialization code was changed a little bit so that
the tokens are available when the YARN client is initialized. That basically
takes care of a long-standing TODO that was in the code to clean up configuration
propagation to the driver's RPC endpoint (in CoarseGrainedSchedulerBackend).

Tested with an app designed to stress this functionality, with both keytab and
cache-based logins. Some basic kerberos tests on k8s also.

Closes #23525 from vanzin/SPARK-26595.

Authored-by: Marcelo Vanzin <vanzin@cloudera.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-01-28 13:32:34 -08:00
Sean Owen 8baf3ba35b [SPARK-26660][FOLLOWUP] Add warning logs when broadcasting large task binary
## What changes were proposed in this pull request?

The warning introduced in https://github.com/apache/spark/pull/23580 has a bug: https://github.com/apache/spark/pull/23580#issuecomment-458000380 This just fixes the logic.

## How was this patch tested?

N/A

Closes #23668 from srowen/SPARK-26660.2.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-01-28 13:47:32 -06:00