spark.git
80 min ago[SPARK-26908][SQL] Fix DateTimeUtils.toMillis and millisToDays master
Maxim Gekk [Sat, 23 Feb 2019 17:35:11 +0000 (11:35 -0600)] 
[SPARK-26908][SQL] Fix DateTimeUtils.toMillis and millisToDays

## What changes were proposed in this pull request?

The `DateTimeUtils.toMillis` can produce inaccurate result for some negative values (timestamps before epoch). The error can be around 1ms. In the PR, I propose to use `Math.floorDiv` in casting microseconds to milliseconds, and milliseconds to days since epoch.

## How was this patch tested?

Added new test to `DateTimeUtilsSuite`, and tested by `CastSuite` as well.

Closes #23815 from MaxGekk/micros-to-millis.

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>
112 min ago[MINOR][DOCS] Remove references to Shark
seancxmao [Sat, 23 Feb 2019 17:03:05 +0000 (11:03 -0600)] 
[MINOR][DOCS] Remove references to Shark

## What changes were proposed in this pull request?
This PR aims to remove references to "Shark", which is a precursor to Spark SQL. I searched the whole project for the text "Shark" (ignore case) and just found a single match. Note that occurrences like nickname or test data are irrelevant.

## How was this patch tested?
N/A. Change comments only.

Closes #23876 from seancxmao/remove-Shark.

Authored-by: seancxmao <seancxmao@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
113 min ago[SPARK-26963][MLLIB] SizeEstimator can't make some JDK fields accessible in Java 9+
Sean Owen [Sat, 23 Feb 2019 17:01:47 +0000 (11:01 -0600)] 
[SPARK-26963][MLLIB] SizeEstimator can't make some JDK fields accessible in Java 9+

## What changes were proposed in this pull request?

Don't use inaccessible fields in SizeEstimator, which comes up in Java 9+

## How was this patch tested?

Manually ran tests with Java 11; it causes these tests that failed before to pass.
This ought to pass on Java 8 as there's effectively no change for Java 8.

Closes #23866 from srowen/SPARK-26963.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2 hours ago[SPARK-26939][CORE][DOC] Fix some outdated comments about task schedulers
seancxmao [Sat, 23 Feb 2019 16:52:53 +0000 (10:52 -0600)] 
[SPARK-26939][CORE][DOC] Fix some outdated comments about task schedulers

## What changes were proposed in this pull request?
This PR aims to fix some outdated comments about task schedulers.

1. Change "ClusterScheduler" to "YarnScheduler" in comments of `YarnClusterScheduler`

According to [SPARK-1140 Remove references to ClusterScheduler](https://issues.apache.org/jira/browse/SPARK-1140), ClusterScheduler is not used anymore.

I also searched "ClusterScheduler" within the whole project, no other occurrences are found in comments or test cases. Note classes like `YarnClusterSchedulerBackend` or `MesosClusterScheduler` are not relevant.

2. Update comments about `statusUpdate` from `TaskSetManager`
`statusUpdate` has been moved to `TaskSchedulerImpl`. StatusUpdate event handling is delegated to `handleSuccessfulTask`/`handleFailedTask`.

## How was this patch tested?
N/A. Fix comments only.

Closes #23844 from seancxmao/taskscheduler-comments.

Authored-by: seancxmao <seancxmao@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
3 hours ago[SPARK-26903][SQL] Remove the TimeZone cache
Maxim Gekk [Sat, 23 Feb 2019 15:44:22 +0000 (09:44 -0600)] 
[SPARK-26903][SQL] Remove the TimeZone cache

## What changes were proposed in this pull request?

In the PR, I propose to convert time zone string to `TimeZone` by converting it to `ZoneId` which uses `ZoneOffset` internally. The `ZoneOffset` class of JDK 8 has a cache already: http://hg.openjdk.java.net/jdk8/jdk8/jdk/file/687fd7c7986d/src/share/classes/java/time/ZoneOffset.java#l205 . In this way, there is no need to support cache of time zones in Spark.

The PR removes `computedTimeZones` from `DateTimeUtils`, and uses `ZoneId.of` to convert time zone id string to `ZoneId` and to `TimeZone` at the end.

## How was this patch tested?

The changes were tested by

Closes #23812 from MaxGekk/timezone-cache.

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>
11 hours ago[SPARK-26945][PYTHON][SS][TESTS] Fix flaky test_*_await_termination in PySpark SS...
Hyukjin Kwon [Sat, 23 Feb 2019 06:57:04 +0000 (14:57 +0800)] 
[SPARK-26945][PYTHON][SS][TESTS] Fix flaky test_*_await_termination in PySpark SS tests

## What changes were proposed in this pull request?

This PR proposes to make sure processing all available data before stopping and delete the temp directory.

See https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/102518/console

```
ERROR: test_query_manager_await_termination (pyspark.sql.tests.test_streaming.StreamingTests)
----------------------------------------------------------------------
Traceback (most recent call last):
 File "/home/jenkins/workspace/SparkPullRequestBuilder/python/pyspark/sql/tests/test_streaming.py", line 259, in test_query_manager_await_termination
 shutil.rmtree(tmpPath)
 File "/home/anaconda/lib/python2.7/shutil.py", line 256, in rmtree
 onerror(os.rmdir, path, sys.exc_info())
 File "/home/anaconda/lib/python2.7/shutil.py", line 254, in rmtree
 os.rmdir(path)
OSError: [Errno 39] Directory not empty: '/home/jenkins/workspace/SparkPullRequestBuilder/python/target/072153bd-f981-47be-bda2-e2b657a16f65/tmp4WGp7n'
```

See https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/102311/console

```
ERROR: test_stream_await_termination (pyspark.sql.tests.test_streaming.StreamingTests)
----------------------------------------------------------------------
Traceback (most recent call last):
  File "/home/jenkins/workspace/SparkPullRequestBuilder2/python/pyspark/sql/tests/test_streaming.py", line 202, in test_stream_await_termination
    shutil.rmtree(tmpPath)
  File "/usr/lib64/pypy-2.5.1/lib-python/2.7/shutil.py", line 256, in rmtree
    onerror(os.rmdir, path, sys.exc_info())
  File "/usr/lib64/pypy-2.5.1/lib-python/2.7/shutil.py", line 254, in rmtree
    os.rmdir(path)
OSError: [Errno 39] Directory not empty: '/home/jenkins/workspace/SparkPullRequestBuilder2/python/target/7244f4ff-6b60-4f6c-b787-de4f15922bf5/tmpQbMZSo'
```

## How was this patch tested?

Jenkins tests - I should run multiple times to see if there are other flaky tests + if this PR really fixes it.

Closes #23870 from HyukjinKwon/SPARK-26945.

Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
19 hours ago[SPARK-26215][SQL] Define reserved/non-reserved keywords based on the ANSI SQL standard
Takeshi Yamamuro [Fri, 22 Feb 2019 23:38:47 +0000 (08:38 +0900)] 
[SPARK-26215][SQL] Define reserved/non-reserved keywords based on the ANSI SQL standard

## What changes were proposed in this pull request?
This pr targeted to define reserved/non-reserved keywords for Spark SQL based on the ANSI SQL standards and the other database-like systems (e.g., PostgreSQL). We assume that they basically follow the ANSI SQL-2011 standard, but it is slightly different between each other. Therefore, this pr documented all the keywords in `docs/sql-reserved-and-non-reserved-key-words.md`.

NOTE: This pr only added a small set of keywords as reserved ones and these keywords are reserved in all the ANSI SQL standards (SQL-92, SQL-99, SQL-2003, SQL-2008, SQL-2011, and SQL-2016) and PostgreSQL. This is because there is room to discuss which keyword should be reserved or not, .e.g., interval units (day, hour, minute, second, ...) are reserved in the ANSI SQL standards though, they are not reserved in PostgreSQL. Therefore, we need more researches about the other database-like systems (e.g., Oracle Databases, DB2, SQL server) in follow-up activities.

References:
 - The reserved/non-reserved SQL keywords in the ANSI SQL standards: https://developer.mimer.com/wp-content/uploads/2018/05/Standard-SQL-Reserved-Words-Summary.pdf
 - SQL Key Words in PostgreSQL: https://www.postgresql.org/docs/current/sql-keywords-appendix.html

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

Closes #23259 from maropu/SPARK-26215-WIP.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
23 hours ago[SPARK-26895][CORE] prepareSubmitEnvironment should be called within doAs for proxy...
Alessandro Bellina [Fri, 22 Feb 2019 19:15:20 +0000 (11:15 -0800)] 
[SPARK-26895][CORE] prepareSubmitEnvironment should be called within doAs for proxy users

## What changes were proposed in this pull request?

`prepareSubmitEnvironment` performs globbing that will fail in the case where a proxy user (`--proxy-user`) doesn't have permission to the file. This is a bug also with 2.3, so we should backport, as currently you can't launch an application that for instance is passing a file under `--archives`, and that file is owned by the target user.

The solution is to call `prepareSubmitEnvironment` within a doAs context if proxying.

## How was this patch tested?

Manual tests running with `--proxy-user` and `--archives`, before and after, showing that the globbing is successful when the resource is owned by the target user.

I've looked at writing unit tests, but I am not sure I can do that cleanly (perhaps with a custom FileSystem). Open to ideas.

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

Closes #23806 from abellina/SPARK-26895_prepareSubmitEnvironment_from_doAs.

Lead-authored-by: Alessandro Bellina <abellina@gmail.com>
Co-authored-by: Alessandro Bellina <abellina@yahoo-inc.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
36 hours ago[SPARK-26851][SQL][FOLLOWUP] Fix cachedColumnBuffers field for Scala 2.11 build
Sean Owen [Fri, 22 Feb 2019 06:22:52 +0000 (15:22 +0900)] 
[SPARK-26851][SQL][FOLLOWUP] Fix cachedColumnBuffers field for Scala 2.11 build

## What changes were proposed in this pull request?

Per https://github.com/apache/spark/pull/23768/files#r259083019 the last change to this line here caused the 2.11 build to fail. It's worked around by making `_cachedColumnBuffers` a field, as it was never set by callers to anything other than its default of null.

## How was this patch tested?

Existing tests.

Closes #23864 from srowen/SPARK-26851.2.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
36 hours ago[SPARK-26930][SQL] Tests in ParquetFilterSuite don't verify filter class
nandorKollar [Fri, 22 Feb 2019 06:07:55 +0000 (14:07 +0800)] 
[SPARK-26930][SQL] Tests in ParquetFilterSuite don't verify filter class

## What changes were proposed in this pull request?

Add assert to verify predicate class in ParquetFilterSuite

## How was this patch tested?

Ran ParquetFilterSuite, tests passed

Closes #23855 from nandorKollar/SPARK-26930.

Lead-authored-by: nandorKollar <nandorKollar@users.noreply.github.com>
Co-authored-by: Hyukjin Kwon <gurwls223@gmail.com>
Co-authored-by: Nandor Kollar <nkollar@cloudera.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
38 hours ago[SPARK-26950][SQL][TEST] Make RandomDataGenerator use Float.NaN or Double.NaN for...
Dongjoon Hyun [Fri, 22 Feb 2019 04:25:26 +0000 (12:25 +0800)] 
[SPARK-26950][SQL][TEST] Make RandomDataGenerator use Float.NaN or Double.NaN for all NaN values

## What changes were proposed in this pull request?

Apache Spark uses the predefined `Float.NaN` and `Double.NaN` for NaN values, but there exists more NaN values with different binary presentations.

```scala
scala> java.nio.ByteBuffer.allocate(4).putFloat(Float.NaN).array
res1: Array[Byte] = Array(127, -64, 0, 0)

scala> val x = java.lang.Float.intBitsToFloat(-6966608)
x: Float = NaN

scala> java.nio.ByteBuffer.allocate(4).putFloat(x).array
res2: Array[Byte] = Array(-1, -107, -78, -80)
```

Since users can have these values, `RandomDataGenerator` generates these NaN values. However, this causes `checkEvaluationWithUnsafeProjection` failures due to the difference between `UnsafeRow` binary presentation. The following is the UT failure instance. This PR aims to fix this UT flakiness.

- https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/102528/testReport/

## How was this patch tested?

Pass the Jenkins with the newly added test cases.

Closes #23851 from dongjoon-hyun/SPARK-26950.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
38 hours ago[SPARK-25097][ML] Support prediction on single instance in KMeans/BiKMeans/GMM
zhengruifeng [Fri, 22 Feb 2019 04:21:28 +0000 (22:21 -0600)] 
[SPARK-25097][ML] Support prediction on single instance in KMeans/BiKMeans/GMM

## What changes were proposed in this pull request?
expose method `predict` in KMeans/BiKMeans/GMM

## How was this patch tested?
added testsuites

Closes #22087 from zhengruifeng/clu_pre_instance.

Authored-by: zhengruifeng <ruifengz@foxmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
38 hours ago[SPARK-26955][CORE] Align Spark's TimSort to jdk11 implementation
Maxim Gekk [Fri, 22 Feb 2019 04:18:23 +0000 (22:18 -0600)] 
[SPARK-26955][CORE] Align Spark's TimSort to jdk11 implementation

## What changes were proposed in this pull request?

Spark's TimSort deviates from JDK 11 TimSort in a couple places:
- `stackLen` was increased in jdk
- additional cases for break in `mergeCollapse`: `n < 0`

In the PR, I propose to align Spark TimSort to jdk implementation.

## How was this patch tested?

By existing test suites, in particular, `SorterSuite`.

Closes #23858 from MaxGekk/timsort-java-alignment.

Authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
38 hours ago[MINOR][SQL] Fix typo in exception about set table properties.
gengjiaan [Fri, 22 Feb 2019 04:13:47 +0000 (22:13 -0600)] 
[MINOR][SQL] Fix typo in exception about set table properties.

## What changes were proposed in this pull request?

The function of the method named verifyTableProperties is

`If the given table properties contains datasource properties, throw an exception. We will do this check when create or alter a table, i.e. when we try to write table metadata to Hive metastore.`

But the message of AnalysisException in verifyTableProperties contains one typo and one unsuited word.
So I change the exception from

`Cannot persistent ${table.qualifiedName} into hive metastore`

to

`Cannot persist ${table.qualifiedName} into Hive metastore`

## How was this patch tested?

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

Closes #23574 from beliefer/incorrect-analysis-exception.

Authored-by: gengjiaan <gengjiaan@360.cn>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
40 hours ago[SPARK-26960][ML] Wait for listener bus to clear in MLEventsSuite to reduce test...
Joseph K. Bradley [Fri, 22 Feb 2019 02:08:16 +0000 (10:08 +0800)] 
[SPARK-26960][ML] Wait for listener bus to clear in MLEventsSuite to reduce test flakiness

## What changes were proposed in this pull request?

This patch aims to address flakiness I've observed in MLEventsSuite in these tests:
*  test("pipeline read/write events")
*  test("pipeline model read/write events")

The issue is in the "read/write events" tests, which work as follows:
* write
* wait until we see at least 1 write-related SparkListenerEvent
* read
* wait until we see at least 1 read-related SparkListenerEvent

The problem is that the last step does NOT allow any write-related SparkListenerEvents, but some of those events may be delayed enough that they are seen in this last step. We should ideally add logic before "read" to wait until the listener events are cleared/complete. Looking into other SparkListener tests, we need to use `sc.listenerBus.waitUntilEmpty(TIMEOUT)`.

This patch adds the waitUntilEmpty() call.

## How was this patch tested?

It's a test!

Closes #23863 from jkbradley/SPARK-26960.

Authored-by: Joseph K. Bradley <joseph@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
43 hours ago[SPARK-26958][SQL][TEST] Add NestedSchemaPruningBenchmark
Dongjoon Hyun [Thu, 21 Feb 2019 23:39:36 +0000 (23:39 +0000)] 
[SPARK-26958][SQL][TEST] Add NestedSchemaPruningBenchmark

## What changes were proposed in this pull request?

This adds `NestedSchemaPruningBenchmark` to show the nested schema pruning performance clearly and to verify new PR's performance benefit and to prevent the future performance degradation.

## How was this patch tested?

Manually run the benchmark.

```
SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain org.apache.spark.sql.execution.benchmark.NestedSchemaPruningBenchmark"
```

Closes #23862 from dongjoon-hyun/SPARK-NESTED-SCHEMA-PRUNING-BM.

Lead-authored-by: Dongjoon Hyun <dhyun@apple.com>
Co-authored-by: DB Tsai <d_tsai@apple.com>
Signed-off-by: DB Tsai <d_tsai@apple.com>
2 days ago[DOCS] MINOR Complement the document of stringOrderType for StringIndexer in PySpark
Liang-Chi Hsieh [Thu, 21 Feb 2019 16:36:48 +0000 (08:36 -0800)] 
[DOCS] MINOR Complement the document of stringOrderType for StringIndexer in PySpark

## What changes were proposed in this pull request?

We revised the behavior of the param `stringOrderType` of `StringIndexer` in case of equal frequency when under frequencyDesc/Asc. This isn't reflected in PySpark's document. We should do it.

## How was this patch tested?

Only document change.

Closes #23849 from viirya/py-stringindexer-doc.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Holden Karau <holden@pigscanfly.ca>
2 days ago[SPARK-26917][SQL] Cache lock recache by condition
Dave DeCaprio [Thu, 21 Feb 2019 15:04:50 +0000 (09:04 -0600)] 
[SPARK-26917][SQL] Cache lock recache by condition

## What changes were proposed in this pull request?

Related to SPARK-26617 and SPARK-26548.  There was a new location we found where we were still seeing the locks.  We traced it to the recacheByCondition function.  In this PR I have changed that function so that the writeLock is not held while the condition is being evaluated.

cloud-fan & gatorsmile This is a further tweak to the other cache PRs we have done (which have helped us tremendously).

## How was this patch tested?

Has been tested on a live system where the blocking was causing major issues and it is working well.
CacheManager has no explicit unit test but is used in many places internally as part of the SharedState.

Closes #23833 from DaveDeCaprio/cache-lock-recacheByCondition.

Lead-authored-by: Dave DeCaprio <daved@alum.mit.edu>
Co-authored-by: David DeCaprio <daved@alum.mit.edu>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2 days ago[R] update package description
Felix Cheung [Thu, 21 Feb 2019 11:00:36 +0000 (19:00 +0800)] 
[R] update package description

## What changes were proposed in this pull request?

update package description

Closes #23852 from felixcheung/rdesccran.

Authored-by: Felix Cheung <felixcheung_m@hotmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2 days ago[SPARK-26824][SS] Fix the checkpoint location and _spark_metadata when it contains...
Shixiong Zhu [Wed, 20 Feb 2019 23:44:20 +0000 (15:44 -0800)] 
[SPARK-26824][SS] Fix the checkpoint location and _spark_metadata when it contains special chars

## What changes were proposed in this pull request?

When a user specifies a checkpoint location or a file sink output using a path containing special chars that need to be escaped in a path, the streaming query will store checkpoint and file sink metadata in a wrong place. In this PR, I uploaded a checkpoint that was generated by the following codes using Spark 2.4.0 to show this issue:

```
implicit val s = spark.sqlContext
val input = org.apache.spark.sql.execution.streaming.MemoryStream[Int]
input.addData(1, 2, 3)
val q = input.toDF.writeStream.format("parquet").option("checkpointLocation", ".../chk %#chk").start(".../output %#output")
q.stop()
```
Here is the structure of the directory:
```
sql/core/src/test/resources/structured-streaming/escaped-path-2.4.0
├── chk%252520%252525%252523chk
│   ├── commits
│   │   └── 0
│   ├── metadata
│   └── offsets
│       └── 0
├── output %#output
│   └── part-00000-97f675a2-bb82-4201-8245-05f3dae4c372-c000.snappy.parquet
└── output%20%25%23output
    └── _spark_metadata
        └── 0
```

In this checkpoint, the user specified checkpoint location is `.../chk %#chk` but the real path to store the checkpoint is `.../chk%252520%252525%252523chk` (this is generated by escaping the original path three times). The user specified output path is `.../output %#output` but the path to store `_spark_metadata` is `.../output%20%25%23output/_spark_metadata` (this is generated by escaping the original path once). The data files are still in the correct path (such as `.../output %#output/part-00000-97f675a2-bb82-4201-8245-05f3dae4c372-c000.snappy.parquet`).

This checkpoint will be used in unit tests in this PR.

The fix is just simply removing improper `Path.toUri` calls to fix the issue.

However, as the user may not read the release note and is not aware of this checkpoint location change, if they upgrade Spark without moving checkpoint to the new location, their query will just start from the scratch. In order to not surprise the users, this PR also adds a check to **detect the impacted paths and throws an error** to include the migration guide. This check can be turned off by an internal sql conf `spark.sql.streaming.checkpoint.escapedPathCheck.enabled`. Here are examples of errors that will be reported:

- Streaming checkpoint error:
```
Error: we detected a possible problem with the location of your checkpoint and you
likely need to move it before restarting this query.

Earlier version of Spark incorrectly escaped paths when writing out checkpoints for
structured streaming. While this was corrected in Spark 3.0, it appears that your
query was started using an earlier version that incorrectly handled the checkpoint
path.

Correct Checkpoint Directory: /.../chk %#chk
Incorrect Checkpoint Directory: /.../chk%252520%252525%252523chk

Please move the data from the incorrect directory to the correct one, delete the
incorrect directory, and then restart this query. If you believe you are receiving
this message in error, you can disable it with the SQL conf
spark.sql.streaming.checkpoint.escapedPathCheck.enabled.
```

- File sink error (`_spark_metadata`):
```
Error: we detected a possible problem with the location of your "_spark_metadata"
directory and you likely need to move it before restarting this query.

Earlier version of Spark incorrectly escaped paths when writing out the
"_spark_metadata" directory for structured streaming. While this was corrected in
Spark 3.0, it appears that your query was started using an earlier version that
incorrectly handled the "_spark_metadata" path.

Correct "_spark_metadata" Directory: /.../output %#output/_spark_metadata
Incorrect "_spark_metadata" Directory: /.../output%20%25%23output/_spark_metadata

Please move the data from the incorrect directory to the correct one, delete the
incorrect directory, and then restart this query. If you believe you are receiving
this message in error, you can disable it with the SQL conf
spark.sql.streaming.checkpoint.escapedPathCheck.enabled.
```

## How was this patch tested?

The new unit tests.

Closes #23733 from zsxwing/path-fix.

Authored-by: Shixiong Zhu <zsxwing@gmail.com>
Signed-off-by: Shixiong Zhu <zsxwing@gmail.com>
2 days ago[SPARK-26892][CORE] Fix saveAsTextFile throws NullPointerException when null row...
liupengcheng [Wed, 20 Feb 2019 22:42:55 +0000 (16:42 -0600)] 
[SPARK-26892][CORE] Fix saveAsTextFile throws NullPointerException when null row present

## What changes were proposed in this pull request?

Currently, RDD.saveAsTextFile may throw NullPointerException then null row is present.
```
scala> sc.parallelize(Seq(1,null),1).saveAsTextFile("/tmp/foobar.dat")
19/02/15 21:39:17 ERROR Utils: Aborting task
java.lang.NullPointerException
at org.apache.spark.rdd.RDD.$anonfun$saveAsTextFile$3(RDD.scala:1510)
at scala.collection.Iterator$$anon$10.next(Iterator.scala:459)
at org.apache.spark.internal.io.SparkHadoopWriter$.$anonfun$executeTask$1(SparkHadoopWriter.scala:129)
at org.apache.spark.util.Utils$.tryWithSafeFinallyAndFailureCallbacks(Utils.scala:1352)
at org.apache.spark.internal.io.SparkHadoopWriter$.executeTask(SparkHadoopWriter.scala:127)
at org.apache.spark.internal.io.SparkHadoopWriter$.$anonfun$write$1(SparkHadoopWriter.scala:83)
at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
at org.apache.spark.scheduler.Task.run(Task.scala:121)
at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:425)
at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1318)
at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:428)
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
at java.lang.Thread.run(Thread.java:748)
```

This PR write "Null" for null row to avoid NPE and fix it.

## How was this patch tested?

NA

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

Closes #23799 from liupc/Fix-saveAsTextFile-throws-NullPointerException-when-null-row-present.

Lead-authored-by: liupengcheng <liupengcheng@xiaomi.com>
Co-authored-by: Liupengcheng <liupengcheng@xiaomi.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2 days ago[SPARK-26877][YARN] Support user-level app staging directory in yarn mode when spark...
Liupengcheng [Wed, 20 Feb 2019 19:45:12 +0000 (11:45 -0800)] 
[SPARK-26877][YARN] Support user-level app staging directory in yarn mode when spark.yarn…

Currently, when running applications on yarn mode, the app staging directory of  is controlled by `spark.yarn.stagingDir` config if specified, and this directory cannot separate different users, sometimes, it's inconvenient for file and quota management for users.

Sometimes, there might be an unexpected increasing of the staging files, two possible reasons are:
1. The `spark.yarn.preserve.staging.files` provided can be misused by users
2. cron task constantly starting new applications on non-existent yarn queue(wrong configuration).

But now, we are not easy to find out the which user obtains the most HDFS files or spaces.
what's more, even we want set HDFS name quota or space quota for each user to limit the increase is impossible.

So I propose to add user sub directories under this app staging directory which is more clear.

existing UT

Closes #23786 from liupc/Support-user-level-app-staging-dir.

Authored-by: Liupengcheng <liupengcheng@xiaomi.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
3 days ago[SPARK-26900][SQL] Simplify truncation to quarter of year
Maxim Gekk [Wed, 20 Feb 2019 14:55:08 +0000 (08:55 -0600)] 
[SPARK-26900][SQL] Simplify truncation to quarter of year

## What changes were proposed in this pull request?

In the PR, I propose to simplify timestamp truncation to quarter of year by using *java.time* API directly. The `LocalDate` instance can be truncation to quarter timestamp via adjusting by chrono field `IsoFields.DAY_OF_QUARTER`.

## How was this patch tested?

This was checked by existing test suite - `DateTimeUtilsSuite`.

Closes #23808 from MaxGekk/date-quarter-of-year.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
3 days ago[SPARK-22798][PYTHON][ML] Add multiple column support to PySpark StringIndexer
Huaxin Gao [Wed, 20 Feb 2019 14:52:46 +0000 (08:52 -0600)] 
[SPARK-22798][PYTHON][ML] Add multiple column support to PySpark StringIndexer

## What changes were proposed in this pull request?

Add multiple column support to PySpark StringIndexer

## How was this patch tested?

Add doctest

Closes #23741 from huaxingao/spark-22798.

Authored-by: Huaxin Gao <huaxing@us.ibm.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
3 days ago[SPARK-26859][SQL] Fix field writer index bug in non-vectorized ORC deserializer
Ivan Vergiliev [Wed, 20 Feb 2019 13:49:38 +0000 (21:49 +0800)] 
[SPARK-26859][SQL] Fix field writer index bug in non-vectorized ORC deserializer

## What changes were proposed in this pull request?

This happens in a schema evolution use case only when a user specifies the schema manually and use non-vectorized ORC deserializer code path.

There is a bug in `OrcDeserializer.scala` that results in `null`s being set at the wrong column position, and for state from previous records to remain uncleared in next records. There are more details for when exactly the bug gets triggered and what the outcome is in the [JIRA issue](https://jira.apache.org/jira/browse/SPARK-26859).

The high-level summary is that this bug results in severe data correctness issues, but fortunately the set of conditions to expose the bug are complicated and make the surface area somewhat small.

This change fixes the problem and adds a respective test.

## How was this patch tested?

Pass the Jenkins with the newly added test cases.

Closes #23766 from IvanVergiliev/fix-orc-deserializer.

Lead-authored-by: Ivan Vergiliev <ivan.vergiliev@gmail.com>
Co-authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
3 days ago[SPARK-26916][SS] Upgrade to Kafka 2.1.1
Dongjoon Hyun [Wed, 20 Feb 2019 04:29:11 +0000 (20:29 -0800)] 
[SPARK-26916][SS] Upgrade to Kafka 2.1.1

## What changes were proposed in this pull request?

As a part of preparing the official JDK 11 support ([SPARK-24417](https://issues.apache.org/jira/browse/SPARK-24417)), Spark 3.0.0 upgraded KAFKA version to 2.1.0. This PR updates Kafka dependency to 2.1.1 to bring the following 42 bug fixes.
- https://issues.apache.org/jira/projects/KAFKA/versions/12344250

## How was this patch tested?

Pass the Jenkins with the existing tests.

Closes #23831 from dongjoon-hyun/SPARK-26916.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
3 days ago[SPARK-26762][SQL][R] Arrow optimization for conversion from Spark DataFrame to R...
Hyukjin Kwon [Wed, 20 Feb 2019 03:35:17 +0000 (11:35 +0800)] 
[SPARK-26762][SQL][R] Arrow optimization for conversion from Spark DataFrame to R DataFrame

## What changes were proposed in this pull request?

This PR targets to support Arrow optimization for conversion from Spark DataFrame to R DataFrame.
Like PySpark side, it falls back to non-optimization code path when it's unable to use Arrow optimization.

This can be tested as below:

```bash
$ ./bin/sparkR --conf spark.sql.execution.arrow.enabled=true
```

```r
collect(createDataFrame(mtcars))
```

### Requirements
  - R 3.5.x
  - Arrow package 0.12+
    ```bash
    Rscript -e 'remotes::install_github("apache/arrowapache-arrow-0.12.0", subdir = "r")'
    ```

**Note:** currently, Arrow R package is not in CRAN. Please take a look at ARROW-3204.
**Note:** currently, Arrow R package seems not supporting Windows. Please take a look at ARROW-3204.

### Benchmarks

**Shall**

```bash
sync && sudo purge
./bin/sparkR --conf spark.sql.execution.arrow.enabled=false --driver-memory 4g
```

```bash
sync && sudo purge
./bin/sparkR --conf spark.sql.execution.arrow.enabled=true --driver-memory 4g
```

**R code**

```r
df <- cache(createDataFrame(read.csv("500000.csv")))
count(df)

test <- function() {
  options(digits.secs = 6) # milliseconds
  start.time <- Sys.time()
  collect(df)
  end.time <- Sys.time()
  time.taken <- end.time - start.time
  print(time.taken)
}

test()
```

**Data (350 MB):**

```r
object.size(read.csv("500000.csv"))
350379504 bytes
```

"500000 Records"  http://eforexcel.com/wp/downloads-16-sample-csv-files-data-sets-for-testing/

**Results**

```
Time difference of 221.32014 secs
```

```
Time difference of 15.51145 secs
```

The performance improvement was around **1426%**.

### Limitations:

- For now, Arrow optimization with R does not support when the data is `raw`, and when user explicitly gives float type in the schema. They produce corrupt values. In this case, we decide to fall back to non-optimization code path.

- Due to ARROW-4512, it cannot send and receive batch by batch. It has to send all batches in Arrow stream format at once. It needs improvement later.

## How was this patch tested?

Existing tests related with Arrow optimization cover this change. Also, manually tested.

Closes #23760 from HyukjinKwon/SPARK-26762.

Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
3 days ago[SPARK-26901][SQL][R] Adds child's output into references to avoid column-pruning...
Hyukjin Kwon [Wed, 20 Feb 2019 02:24:40 +0000 (10:24 +0800)] 
[SPARK-26901][SQL][R] Adds child's output into references to avoid column-pruning for vectorized gapply()

## What changes were proposed in this pull request?

Currently, looks column pruning is done to vectorized `gapply()`. Given R native function could use all referred fields so it shouldn't be pruned. To avoid this, it adds child's output into `references` like `OutputConsumer`.

```
$ ./bin/sparkR --conf spark.sql.execution.arrow.enabled=true
```

```r
df <- createDataFrame(mtcars)
explain(count(groupBy(gapply(df,
                             "gear",
                             function(key, group) {
                               data.frame(gear = key[[1]], disp = mean(group$disp))
                             },
                             structType("gear double, disp double")))), TRUE)
```

**Before:**

```
== Optimized Logical Plan ==
Aggregate [count(1) AS count#41L]
+- Project
   +- FlatMapGroupsInRWithArrow [...]
      +- Project [gear#9]
         +- LogicalRDD [mpg#0, cyl#1, disp#2, hp#3, drat#4, wt#5, qsec#6, vs#7, am#8, gear#9, carb#10], false

== Physical Plan ==
*(4) HashAggregate(keys=[], functions=[count(1)], output=[count#41L])
+- Exchange SinglePartition
   +- *(3) HashAggregate(keys=[], functions=[partial_count(1)], output=[count#44L])
      +- *(3) Project
         +- FlatMapGroupsInRWithArrow [...]
            +- *(2) Sort [gear#9 ASC NULLS FIRST], false, 0
               +- Exchange hashpartitioning(gear#9, 200)
                  +- *(1) Project [gear#9]
                     +- *(1) Scan ExistingRDD arrow[mpg#0,cyl#1,disp#2,hp#3,drat#4,wt#5,qsec#6,vs#7,am#8,gear#9,carb#10]
```

**After:**

```
== Optimized Logical Plan ==
Aggregate [count(1) AS count#91L]
+- Project
   +- FlatMapGroupsInRWithArrow [...]
      +- LogicalRDD [mpg#0, cyl#1, disp#2, hp#3, drat#4, wt#5, qsec#6, vs#7, am#8, gear#9, carb#10], false

== Physical Plan ==
*(4) HashAggregate(keys=[], functions=[count(1)], output=[count#91L])
+- Exchange SinglePartition
   +- *(3) HashAggregate(keys=[], functions=[partial_count(1)], output=[count#94L])
      +- *(3) Project
         +- FlatMapGroupsInRWithArrow [...]
            +- *(2) Sort [gear#9 ASC NULLS FIRST], false, 0
               +- Exchange hashpartitioning(gear#9, 200)
                  +- *(1) Scan ExistingRDD arrow[mpg#0,cyl#1,disp#2,hp#3,drat#4,wt#5,qsec#6,vs#7,am#8,gear#9,carb#10]
```

Currently, it adds corrupt values for missing columns (via pruned columnar batches to Arrow writers that requires non-pruned columns) such as:

```r
...
  c(7.90505033345994e-323, 7.90505033345994e-323, 7.90505033345994e-323, 7.90505033345994e-323, 7.90505033345994e-323, 7.90505033345994e-323, 7.90505033345994e-323, 7.90505033345994e-323, 7.90505033345994e-323, 7.90505033345994e-323, 7.90505033345994e-323, 7.90505033345994e-323, 0, 0, 4.17777978645388e-314)
  c(0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, -1.04669129845114e+219)
  c(3.4482690635875e-313, 3.4482690635875e-313, 3.4482690635875e-313,
  c(0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 2.47032822920623e-323)
...
```

which should be something like:

```r
...
  c(4, 4, 1, 2, 2, 4, 4, 1, 2, 1, 1, 2)
  c(26, 30.4, 15.8, 19.7, 15)
  c(4, 4, 8, 6, 8)
  c(120.3, 95.1, 351, 145, 301)
...
```

## How was this patch tested?

Manually tested, and unit tests were added.

The test code is basiaclly:

```r
df <- createDataFrame(mtcars)
count(gapply(df,
             c("gear"),
             function(key, group) {
                stopifnot(all(group$hp > 50))
                group
             },
             schema(df)))
```

`mtcars`'s hp is all more then 50.

```r
> mtcars$hp > 50
 [1] TRUE TRUE TRUE TRUE TRUE TRUE TRUE TRUE TRUE TRUE TRUE TRUE TRUE TRUE TRUE
[16] TRUE TRUE TRUE TRUE TRUE TRUE TRUE TRUE TRUE TRUE TRUE TRUE TRUE TRUE TRUE
[31] TRUE TRUE
```

However, due to corrpt value, (like 0 or 7.xxxxx), werid values were found. So, it's currently being failed as below in the master:

```
Error in handleErrors(returnStatus, conn) :
  org.apache.spark.SparkException: Job aborted due to stage failure: Task 82 in stage 1.0 failed 1 times, most recent failure: Lost task 82.0 in stage 1.0 (TID 198, localhost, executor driver): org.apache.spark.SparkException: R worker exited unexpectedly (crashed)
 Error in computeFunc(key, inputData) : all(group$hp > 50) is not TRUE
Error in computeFunc(key, inputData) : all(group$hp > 50) is not TRUE
Error in computeFunc(key, inputData) : all(group$hp > 50) is not TRUE
```

I also compared the total length while I am here. Regular `gapply` without Arrow has some holes .. so I had to compare the results with R data frame.

Closes #23810 from HyukjinKwon/SPARK-26901.

Lead-authored-by: Hyukjin Kwon <gurwls223@apache.org>
Co-authored-by: Hyukjin Kwon <gurwls223@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
3 days ago[SPARK-24894][K8S] Make sure valid host names are created for executors.
Marcelo Vanzin [Tue, 19 Feb 2019 23:19:59 +0000 (15:19 -0800)] 
[SPARK-24894][K8S] Make sure valid host names are created for executors.

Since the host name is derived from the app name, which can contain arbitrary
characters, it needs to be sanitized so that only valid characters are allowed.

On top of that, take extra care that truncation doesn't leave characters that
are valid except at the start of a host name.

Closes #23781 from vanzin/SPARK-24894.

Authored-by: Marcelo Vanzin <vanzin@cloudera.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
3 days ago[SPARK-26882] Check the Kubernetes integration tests scalatyle
Holden Karau [Tue, 19 Feb 2019 21:49:47 +0000 (13:49 -0800)] 
[SPARK-26882] Check the Kubernetes integration tests scalatyle

## What changes were proposed in this pull request?

Add the kubernetes integration tests to the scalastyle profiles.

## How was this patch tested?

Run ./dev/scalastyle with a bad change manually

## Follow on work

See SPARK-26898 to add scalastyle for k8s integration to the CI

Closes #23792 from holdenk/SPARK-26882-check-k8s-integration-tests-when-linting.

Authored-by: Holden Karau <holden@pigscanfly.ca>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
3 days ago[SPARK-26891][YARN] Fixing flaky test in YarnSchedulerBackendSuite
“attilapiros” [Tue, 19 Feb 2019 21:22:40 +0000 (13:22 -0800)] 
[SPARK-26891][YARN] Fixing flaky test in YarnSchedulerBackendSuite

The test "RequestExecutors reflects node blacklist and is serializable" is flaky because of multi threaded access of the mock task scheduler. For details check [Mockito FAQ (occasional exceptions like: WrongTypeOfReturnValue)](https://github.com/mockito/mockito/wiki/FAQ#is-mockito-thread-safe). So instead of mocking the task scheduler in the test TaskSchedulerImpl is simply subclassed.

This multithreaded access of the `nodeBlacklist()` method is coming from:
1) the unit test thread via calling of the method `prepareRequestExecutors()`
2) the `DriverEndpoint.onStart` which runs a periodic task that ends up calling this method

Existing unittest.

Closes #23801 from attilapiros/SPARK-26891.

Authored-by: “attilapiros” <piros.attila.zsolt@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
4 days ago[MINOR][DOCS] Fix the update rule in StreamingKMeansModel documentation
joelgenter [Tue, 19 Feb 2019 14:40:59 +0000 (08:40 -0600)] 
[MINOR][DOCS] Fix the update rule in StreamingKMeansModel documentation

## What changes were proposed in this pull request?
The formatting for the update rule (in the documentation) now appears as
![image](https://user-images.githubusercontent.com/14948437/52933807-5a0c7980-3309-11e9-8573-642a73e77c26.png)
instead of
![image](https://user-images.githubusercontent.com/14948437/52933897-a8ba1380-3309-11e9-8e16-e47c27b4a044.png)

Closes #23819 from joelgenter/patch-1.

Authored-by: joelgenter <joelgenter@outlook.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
4 days ago[SPARK-26909][FOLLOWUP][SQL] use unsafeRow.hashCode() as hash value in HashAggregate
yucai [Tue, 19 Feb 2019 05:01:10 +0000 (13:01 +0800)] 
[SPARK-26909][FOLLOWUP][SQL] use unsafeRow.hashCode() as hash value in HashAggregate

## What changes were proposed in this pull request?

This is a followup PR for #21149.

New way uses unsafeRow.hashCode() as hash value in HashAggregate.
The unsafe row has [null bit set] etc., so the hash should be different from shuffle hash, and then we don't need a special seed.

## How was this patch tested?

UTs.

Closes #23821 from yucai/unsafe_hash.

Authored-by: yucai <yyu1@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
4 days ago[MINOR][DOC] Add note regarding proper usage of QueryExecution.toRdd
Jungtaek Lim (HeartSaVioR) [Tue, 19 Feb 2019 01:42:21 +0000 (09:42 +0800)] 
[MINOR][DOC] Add note regarding proper usage of QueryExecution.toRdd

## What changes were proposed in this pull request?

This proposes adding a note on `QueryExecution.toRdd` regarding Spark's internal optimization callers would need to indicate.

## How was this patch tested?

This patch is a documentation change.

Closes #23822 from HeartSaVioR/MINOR-doc-add-note-query-execution-to-rdd.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
4 days ago[SPARK-26785][SQL] data source v2 API refactor: streaming write
Wenchen Fan [Tue, 19 Feb 2019 00:17:24 +0000 (16:17 -0800)] 
[SPARK-26785][SQL] data source v2 API refactor: streaming write

## What changes were proposed in this pull request?

Continue the API refactor for streaming write, according to the [doc](https://docs.google.com/document/d/1vI26UEuDpVuOjWw4WPoH2T6y8WAekwtI7qoowhOFnI4/edit?usp=sharing).

The major changes:
1. rename `StreamingWriteSupport` to `StreamingWrite`
2. add `WriteBuilder.buildForStreaming`
3. update existing sinks, to move the creation of `StreamingWrite` to `Table`

## How was this patch tested?

existing tests

Closes #23702 from cloud-fan/stream-write.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
5 days ago[SPARK-26744][SPARK-26744][SQL][HOTFOX] Disable schema validation tests for FileDataS...
Hyukjin Kwon [Mon, 18 Feb 2019 13:13:00 +0000 (21:13 +0800)] 
[SPARK-26744][SPARK-26744][SQL][HOTFOX] Disable schema validation tests for FileDataSourceV2 (partially revert )

## What changes were proposed in this pull request?

This PR partially revert SPARK-26744.

https://github.com/apache/spark/commit/60caa92deaf6941f58da82dcc0962ebf3a598ced and https://github.com/apache/spark/commit/4dce45a5992e6a89a26b5a0739b33cfeaf979208 were merged at similar time range independently. So the test failures were not caught.

- https://github.com/apache/spark/commit/60caa92deaf6941f58da82dcc0962ebf3a598ced happened to add a schema reading logic in writing path for overwrite mode as well.

- https://github.com/apache/spark/commit/4dce45a5992e6a89a26b5a0739b33cfeaf979208 added some tests with overwrite modes with migrated ORC v2.

And the tests looks starting to fail.

I guess the discussion won't be short (see https://github.com/apache/spark/pull/23606#discussion_r257675083) and this PR proposes to disable the tests added at https://github.com/apache/spark/commit/4dce45a5992e6a89a26b5a0739b33cfeaf979208 to unblock other PRs for now.

## How was this patch tested?

Existing tests.

Closes #23828 from HyukjinKwon/SPARK-26744.

Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
5 days ago[SPARK-26889][SS][DOCS] Fix timestamp type in Structured Streaming + Kafka Integratio...
Gabor Somogyi [Mon, 18 Feb 2019 09:22:06 +0000 (17:22 +0800)] 
[SPARK-26889][SS][DOCS] Fix timestamp type in Structured Streaming + Kafka Integration Guide

## What changes were proposed in this pull request?

```
$ spark-shell --packages org.apache.spark:spark-sql-kafka-0-10_2.11:3.0.0-SNAPSHOT
...
scala> val df = spark.read.format("kafka").option("kafka.bootstrap.servers", "foo").option("subscribe", "bar").load().printSchema()
root
 |-- key: binary (nullable = true)
 |-- value: binary (nullable = true)
 |-- topic: string (nullable = true)
 |-- partition: integer (nullable = true)
 |-- offset: long (nullable = true)
 |-- timestamp: timestamp (nullable = true)
 |-- timestampType: integer (nullable = true)

df: Unit = ()
```
In the doc timestamp type is `long` and in this PR I've changed it to `timestamp`.

## How was this patch tested?

cd docs/
SKIP_API=1 jekyll build
Manual webpage check.

Closes #23796 from gaborgsomogyi/SPARK-26889.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
5 days ago[SPARK-26353][SQL] Add typed aggregate functions(max/min) to the example module.
liuxian [Mon, 18 Feb 2019 09:20:58 +0000 (17:20 +0800)] 
[SPARK-26353][SQL] Add typed aggregate functions(max/min) to the example module.

## What changes were proposed in this pull request?

Add typed aggregate functions(max/min) to the example module.

## How was this patch tested?
Manual testing:
running typed minimum:

```
+-----+----------------------+
|value|TypedMin(scala.Tuple2)|
+-----+----------------------+
|    0|                 [0.0]|
|    2|                 [2.0]|
|    1|                 [1.0]|
+-----+----------------------+
```

running typed maximum:

```
+-----+----------------------+
|value|TypedMax(scala.Tuple2)|
+-----+----------------------+
|    0|                  [18]|
|    2|                  [17]|
|    1|                  [19]|
+-----+----------------------+
```

Closes #23304 from 10110346/typedminmax.

Authored-by: liuxian <liu.xian3@zte.com.cn>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
5 days ago[SPARK-24570][SQL] Implement Spark own GetTablesOperation to fix SQL client tools...
Yuming Wang [Mon, 18 Feb 2019 07:35:45 +0000 (23:35 -0800)] 
[SPARK-24570][SQL] Implement Spark own GetTablesOperation to fix SQL client tools cannot show tables

## What changes were proposed in this pull request?

For SQL client tools([DBeaver](https://dbeaver.io/))'s Navigator use [`GetTablesOperation`](https://github.com/apache/spark/blob/a7444570764b0a08b7e908dc7931744f9dbdf3c6/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java) to obtain table names.

We should use [`metadataHive`](https://github.com/apache/spark/blob/95d172da2b370ff6257bfd6fcd102ac553f6f6af/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala#L52-L53), but it use [`executionHive`](https://github.com/apache/spark/blob/24f5bbd770033dacdea62555488bfffb61665279/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala#L93-L95).

This PR implement Spark own `GetTablesOperation` to use `metadataHive`.

## How was this patch tested?

unit test and manual tests

![image](https://user-images.githubusercontent.com/5399861/47430696-acf77980-d7cc-11e8-824d-f28d78f60a00.png)
![image](https://user-images.githubusercontent.com/5399861/47440576-09649400-d7e1-11e8-97a8-a96f73f70361.png)

Closes #22794 from wangyum/SPARK-24570.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
5 days ago[SPARK-26666][SQL] Support DSv2 overwrite and dynamic partition overwrite.
Ryan Blue [Mon, 18 Feb 2019 05:16:28 +0000 (13:16 +0800)] 
[SPARK-26666][SQL] Support DSv2 overwrite and dynamic partition overwrite.

## What changes were proposed in this pull request?

This adds two logical plans that implement the ReplaceData operation from the [logical plans SPIP](https://docs.google.com/document/d/1gYm5Ji2Mge3QBdOliFV5gSPTKlX4q1DCBXIkiyMv62A/edit?ts=5a987801#heading=h.m45webtwxf2d). These two plans will be used to implement Spark's `INSERT OVERWRITE` behavior for v2.

Specific changes:
* Add `SupportsTruncate`, `SupportsOverwrite`, and `SupportsDynamicOverwrite` to DSv2 write API
* Add `OverwriteByExpression` and `OverwritePartitionsDynamic` plans (logical and physical)
* Add new plans to DSv2 write validation rule `ResolveOutputRelation`
* Refactor `WriteToDataSourceV2Exec` into trait used by all DSv2 write exec nodes

## How was this patch tested?

* The v2 analysis suite has been updated to validate the new overwrite plans
* The analysis suite for `OverwriteByExpression` checks that the delete expression is resolved using the table's columns
* Existing tests validate that overwrite exec plan works
* Updated existing v2 test because schema is used to validate overwrite

Closes #23606 from rdblue/SPARK-26666-add-overwrite.

Authored-by: Ryan Blue <blue@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
5 days ago[SPARK-26887][SQL][PYTHON][NS] Create datetime.date directly instead of creating...
Takuya UESHIN [Mon, 18 Feb 2019 03:48:10 +0000 (11:48 +0800)] 
[SPARK-26887][SQL][PYTHON][NS] Create datetime.date directly instead of creating datetime64 as intermediate data.

## What changes were proposed in this pull request?

Currently `DataFrame.toPandas()` with arrow enabled or `ArrowStreamPandasSerializer` for pandas UDF with pyarrow<0.12 creates `datetime64[ns]` type series as intermediate data and then convert to `datetime.date` series, but the intermediate `datetime64[ns]` might cause an overflow even if the date is valid.

```
>>> import datetime
>>>
>>> t = [datetime.date(2262, 4, 12), datetime.date(2263, 4, 12)]
>>>
>>> df = spark.createDataFrame(t, 'date')
>>> df.show()
+----------+
|     value|
+----------+
|2262-04-12|
|2263-04-12|
+----------+

>>>
>>> spark.conf.set("spark.sql.execution.arrow.enabled", "true")
>>>
>>> df.toPandas()
        value
0  1677-09-21
1  1678-09-21
```

We should avoid creating such intermediate data and create `datetime.date` series directly instead.

## How was this patch tested?

Modified some tests to include the date which overflow caused by the intermediate conversion.
Run tests with pyarrow 0.8, 0.10, 0.11, 0.12 in my local environment.

Closes #23795 from ueshin/issues/SPARK-26887/date_as_object.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
5 days ago[SPARK-26897][SQL][TEST][FOLLOW-UP] Remove workaround for 2.2.0 and 2.1.x in HiveExte...
Takeshi Yamamuro [Mon, 18 Feb 2019 03:24:36 +0000 (11:24 +0800)] 
[SPARK-26897][SQL][TEST][FOLLOW-UP] Remove workaround for 2.2.0 and 2.1.x in HiveExternalCatalogVersionsSuite

## What changes were proposed in this pull request?
This pr just removed workaround for 2.2.0 and 2.1.x in HiveExternalCatalogVersionsSuite.

## How was this patch tested?
Pass the Jenkins.

Closes #23817 from maropu/SPARK-26607-FOLLOWUP.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
5 days ago[SPARK-26878] QueryTest.compare() does not handle maps with array keys correctly
Ala Luszczak [Mon, 18 Feb 2019 02:39:31 +0000 (10:39 +0800)] 
[SPARK-26878] QueryTest.compare() does not handle maps with array keys correctly

## What changes were proposed in this pull request?

The previous strategy for comparing Maps leveraged sorting (key, value) tuples by their _.toString. However, the _.toString representation of an arrays has nothing to do with it's content. If a map has array keys, it's (key, value) pairs would be compared with other maps essentially at random. This could results in false negatives in tests.

This changes first compares keys together to find the matching ones, and then compares associated values.

## How was this patch tested?

New unit test added.

Closes #23789 from ala/compare-map.

Authored-by: Ala Luszczak <ala@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
5 days ago[SPARK-26897][SQL][TEST] Update Spark 2.3.x testing from HiveExternalCatalogVersionsSuite
Takeshi Yamamuro [Sun, 17 Feb 2019 23:05:49 +0000 (08:05 +0900)] 
[SPARK-26897][SQL][TEST] Update Spark 2.3.x testing from HiveExternalCatalogVersionsSuite

## What changes were proposed in this pull request?
The maintenance release of `branch-2.3` (v2.3.3) vote passed, so this issue updates PROCESS_TABLES.testingVersions in HiveExternalCatalogVersionsSuite

## How was this patch tested?
Pass the Jenkins.

Closes #23807 from maropu/SPARK-26897.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
6 days ago[SPARK-26721][ML] Avoid per-tree normalization in featureImportance for GBT
Marco Gaido [Sat, 16 Feb 2019 22:51:01 +0000 (16:51 -0600)] 
[SPARK-26721][ML] Avoid per-tree normalization in featureImportance for GBT

## What changes were proposed in this pull request?

Our feature importance calculation is taken from sklearn's one, which has been recently fixed (in https://github.com/scikit-learn/scikit-learn/pull/11176). Citing the description of that PR:

> Because the feature importances are (currently, by default) normalized and then averaged, feature importances from later stages are overweighted.

The PR performs a fix similar to sklearn's one. The per-tree normalization of the feature importance is skipped and GBT.

Credits for pointing out clearly the issue and the sklearn's PR to Daniel Jumper.

## How was this patch tested?

modified UT, checked that the computed `featureImportance` in that test is similar to sklearn's one (ti can't be the same, because the trees may be slightly different)

Closes #23773 from mgaido91/SPARK-26721.

Authored-by: Marco Gaido <marcogaido91@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
7 days ago[SPARK-26744][SQL] Support schema validation in FileDataSourceV2 framework
Gengliang Wang [Sat, 16 Feb 2019 09:11:36 +0000 (17:11 +0800)] 
[SPARK-26744][SQL] Support schema validation in FileDataSourceV2 framework

## What changes were proposed in this pull request?

The file source has a schema validation feature, which validates 2 schemas:
1. the user-specified schema when reading.
2. the schema of input data when writing.

If a file source doesn't support the schema, we can fail the query earlier.

This PR is to implement the same feature  in the `FileDataSourceV2` framework. Comparing to `FileFormat`, `FileDataSourceV2` has multiple layers. The API is added in two places:
1. Read path: the table schema is determined in `TableProvider.getTable`. The actual read schema can be a subset of the table schema.  This PR proposes to validate the actual read schema in  `FileScan`.
2.  Write path: validate the actual output schema in `FileWriteBuilder`.

## How was this patch tested?

Unit test

Closes #23714 from gengliangwang/schemaValidationV2.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
7 days ago[SPARK-26673][FOLLOWUP][SQL] File source V2: remove duplicated broadcast object in...
Gengliang Wang [Sat, 16 Feb 2019 06:44:37 +0000 (14:44 +0800)] 
[SPARK-26673][FOLLOWUP][SQL] File source V2: remove duplicated broadcast object in FileWriterFactory

## What changes were proposed in this pull request?

This is a followup PR to fix two issues in #23601:
1.  the class `FileWriterFactory` contains `conf: SerializableConfiguration` as a member, which is duplicated with `WriteJobDescription. serializableHadoopConf `. By removing it we can reduce the broadcast task binary size by around 70KB
2. The test suite `OrcV1QuerySuite`/`OrcV1QuerySuite`/`OrcV1PartitionDiscoverySuite` didn't change the configuration `SQLConf.USE_V1_SOURCE_WRITER_LIST` to `"orc"`. We should set the conf.

## How was this patch tested?

Unit test

Closes #23800 from gengliangwang/reduceWriteTaskSize.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
7 days ago[SPARK-21094][PYTHON] Add popen_kwargs to launch_gateway
Peter Parente [Sat, 16 Feb 2019 02:08:06 +0000 (18:08 -0800)] 
[SPARK-21094][PYTHON] Add popen_kwargs to launch_gateway

## What changes were proposed in this pull request?

Allow the caller to customize the py4j JVM subprocess pipes and buffers for programmatic capturing of its output.

https://issues.apache.org/jira/browse/SPARK-21094 has more detail about the use case.

## How was this patch tested?

Tested by running the pyspark unit tests locally.

Closes #18339 from parente/feature/SPARK-21094-popen-args.

Lead-authored-by: Peter Parente <parente@cs.unc.edu>
Co-authored-by: Peter Parente <peter.parente@maxpoint.com>
Signed-off-by: Holden Karau <holden@pigscanfly.ca>
7 days ago[SPARK-26772][YARN] Delete ServiceCredentialProvider and make HadoopDelegationTokenPr...
Gabor Somogyi [Fri, 15 Feb 2019 22:43:13 +0000 (14:43 -0800)] 
[SPARK-26772][YARN] Delete ServiceCredentialProvider and make HadoopDelegationTokenProvider a developer API

## What changes were proposed in this pull request?

`HadoopDelegationTokenProvider` has basically the same functionality just like `ServiceCredentialProvider` so the interfaces can be merged.

`YARNHadoopDelegationTokenManager` now loads `ServiceCredentialProvider`s in one step. The drawback of this if one provider fails all others are not loaded. `HadoopDelegationTokenManager` loads `HadoopDelegationTokenProvider`s independently so it provides more robust behaviour.

In this PR I've I've made the following changes:
* Deleted `YARNHadoopDelegationTokenManager` and `ServiceCredentialProvider`
* Made `HadoopDelegationTokenProvider` a `DeveloperApi`

## How was this patch tested?

Existing unit tests.

Closes #23686 from gaborgsomogyi/SPARK-26772.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
7 days ago[SPARK-26790][CORE] Change approach for retrieving executor logs and attributes:...
Jungtaek Lim (HeartSaVioR) [Fri, 15 Feb 2019 20:44:14 +0000 (12:44 -0800)] 
[SPARK-26790][CORE] Change approach for retrieving executor logs and attributes: self-retrieve

## What changes were proposed in this pull request?

This patch proposes to change the approach on extracting log urls as well as attributes from YARN executor:

 - AS-IS: extract information from `Container` API and include them to container launch context
- TO-BE: let YARN executor self-extracting information

This approach leads us to populate more attributes like nodemanager's IPC port which can let us configure custom log url to JHS log url directly.

## How was this patch tested?

Existing unit tests.

Closes #23706 from HeartSaVioR/SPARK-26790.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
8 days ago[SPARK-26871][SQL] File Source V2: avoid creating unnecessary FileIndex in the write...
Gengliang Wang [Fri, 15 Feb 2019 06:57:23 +0000 (14:57 +0800)] 
[SPARK-26871][SQL] File Source V2: avoid creating unnecessary FileIndex in the write path

## What changes were proposed in this pull request?

In https://github.com/apache/spark/pull/23383, the file source V2 framework is implemented. In the PR, `FileIndex` is created as a member of `FileTable`, so that we can implement partition pruning like https://github.com/apache/spark/commit/0f9fcabb4ac2e8afec14d010e86467372a85d334 in the future(As data source V2 catalog is under development, partition pruning is removed from the PR)

However, after write path of file source V2 is implemented, I find that a simple write will create an unnecessary `FileIndex`, which is required by `FileTable`. This is a sort of regression. And we can see there is a warning message when writing to ORC files
```
WARN InMemoryFileIndex: The directory file:/tmp/foo was not found. Was it deleted very recently?
```
This PR is to make `FileIndex` as a lazy value in `FileTable`, so that we can avoid creating unnecessary `FileIndex` in the write path.

## How was this patch tested?

Existing unit test

Closes #23774 from gengliangwang/moveFileIndexInV2.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
8 days ago[SPARK-26870][SQL] Move to_avro/from_avro into functions object due to Java compatibility
Hyukjin Kwon [Fri, 15 Feb 2019 02:24:35 +0000 (10:24 +0800)] 
[SPARK-26870][SQL] Move to_avro/from_avro into functions object due to Java compatibility

## What changes were proposed in this pull request?

Currently, looks, to use `from_avro` and `to_avro` in Java APIs side,

```java
import static org.apache.spark.sql.avro.package$.MODULE$;

MODULE$.to_avro
MODULE$.from_avro
```

This PR targets to deprecate and move both functions under `avro` package into `functions` object like the way of our `org.apache.spark.sql.functions`.

Therefore, Java side can import:

```java
import static org.apache.spark.sql.avro.functions.*;
```

and Scala side can import:

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

## How was this patch tested?

Manually tested, and unit tests for Java APIs were added.

Closes #23784 from HyukjinKwon/SPARK-26870.

Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
8 days ago[SPARK-26840][SQL] Avoid cost-based join reorder in presence of join hints
maryannxue [Fri, 15 Feb 2019 00:56:55 +0000 (16:56 -0800)] 
[SPARK-26840][SQL] Avoid cost-based join reorder in presence of join hints

## What changes were proposed in this pull request?

This is a fix for https://github.com/apache/spark/pull/23524, which did not stop cost-based join reorder when the CostBasedJoinReorder rule recurses down the tree and applies join reorder for nested joins with hints.

The issue had not been detected by the existing tests because CBO is disabled by default.

## How was this patch tested?

Enabled CBO for JoinHintSuite.

Closes #23759 from maryannxue/spark-26840.

Lead-authored-by: maryannxue <maryannxue@apache.org>
Co-authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
8 days ago[SPARK-26861][SQL] deprecate typed sum/count/average
Wenchen Fan [Fri, 15 Feb 2019 00:54:39 +0000 (16:54 -0800)] 
[SPARK-26861][SQL] deprecate typed sum/count/average

## What changes were proposed in this pull request?

These builtin typed aggregate functions are not very useful:
1. users can just call the untyped ones and turn the resulting dataframe to a dataset. It has better performance.
2. the typed aggregate functions have subtle different behaviors regarding empty input.

I think we should get rid of these builtin typed agg functions and suggest users to use the untyped ones.

However, these functions are still useful as a demo of the `Aggregator` API, so I copied them to the example module.

## How was this patch tested?

N/A

Closes #23763 from cloud-fan/example.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
9 days ago[SPARK-26873][SQL] Use a consistent timestamp to build Hadoop Job IDs.
Ryan Blue [Thu, 14 Feb 2019 16:25:33 +0000 (08:25 -0800)] 
[SPARK-26873][SQL] Use a consistent timestamp to build Hadoop Job IDs.

## What changes were proposed in this pull request?

Updates FileFormatWriter to create a consistent Hadoop Job ID for a write.

## How was this patch tested?

Existing tests for regressions.

Closes #23777 from rdblue/SPARK-26873-fix-file-format-writer-job-ids.

Authored-by: Ryan Blue <blue@apache.org>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
9 days ago[SPARK-26572][SQL] fix aggregate codegen result evaluation
Peter Toth [Thu, 14 Feb 2019 15:02:56 +0000 (23:02 +0800)] 
[SPARK-26572][SQL] fix aggregate codegen result evaluation

## What changes were proposed in this pull request?

This PR is a correctness fix in `HashAggregateExec` code generation. It forces evaluation of result expressions before calling `consume()` to avoid multiple executions.

This PR fixes a use case where an aggregate is nested into a broadcast join and appears on the "stream" side. The issue is that Broadcast join generates it's own loop. And without forcing evaluation of `resultExpressions` of `HashAggregateExec` before the join's loop these expressions can be executed multiple times giving incorrect results.

## How was this patch tested?

New UT was added.

Closes #23731 from peter-toth/SPARK-26572.

Authored-by: Peter Toth <peter.toth@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
9 days ago[SPARK-26794][SQL] SparkSession enableHiveSupport does not point to hive but in-memor...
Kent Yao [Thu, 14 Feb 2019 07:07:22 +0000 (15:07 +0800)] 
[SPARK-26794][SQL] SparkSession enableHiveSupport does not point to hive but in-memory while the SparkContext exists

## What changes were proposed in this pull request?

```java
public class SqlDemo {
    public static void main(final String[] args) throws Exception {
        SparkConf conf = new SparkConf().setAppName("spark-sql-demo");
        JavaSparkContext sc = new JavaSparkContext(conf);
        SparkSession ss = SparkSession.builder().enableHiveSupport().getOrCreate();
        ss.sql("show databases").show();
    }
}
```
Before https://issues.apache.org/jira/browse/SPARK-20946, the demo above point to the right hive metastore if the hive-site.xml is present. But now it can only point to the default in-memory one.

Catalog is now as a variable shared across SparkSessions, it is instantiated with SparkContext's conf. After https://issues.apache.org/jira/browse/SPARK-20946, Session level configs are not pass to SparkContext's conf anymore, so the enableHiveSupport API takes no affect on the catalog instance.

You can set spark.sql.catalogImplementation=hive application wide to solve the problem, or never create a sc before you call SparkSession.builder().enableHiveSupport().getOrCreate()

Here we respect the SparkSession level configuration at the first time to generate catalog within SharedState

## How was this patch tested?

1. add ut
2. manually
```scala
test("enableHiveSupport has right to determine the catalog while using an existing sc") {
    val conf = new SparkConf().setMaster("local").setAppName("SharedState Test")
    val sc = SparkContext.getOrCreate(conf)
    val ss = SparkSession.builder().enableHiveSupport().getOrCreate()
    assert(ss.sharedState.externalCatalog.unwrapped.isInstanceOf[HiveExternalCatalog],
      "The catalog should be hive ")

    val ss2 = SparkSession.builder().getOrCreate()
    assert(ss2.sharedState.externalCatalog.unwrapped.isInstanceOf[HiveExternalCatalog],
      "The catalog should be shared across sessions")
  }
```

Without this fix, the above test will fail.
You can apply it to `org.apache.spark.sql.hive.HiveSharedStateSuite`,
and run,
```sbt
./build/sbt  -Phadoop-2.7 -Phive  "hive/testOnly org.apache.spark.sql.hive.HiveSharedStateSuite"
```
to verify.

Closes #23709 from yaooqinn/SPARK-26794.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
9 days ago[SPARK-26851][SQL] Fix double-checked locking in CachedRDDBuilder
Bruce Robbins [Thu, 14 Feb 2019 06:57:25 +0000 (14:57 +0800)] 
[SPARK-26851][SQL] Fix double-checked locking in CachedRDDBuilder

## What changes were proposed in this pull request?

According to Brian Goetz et al in Java Concurrency in Practice, the double checked locking pattern has worked since Java 5, but only if the resource is declared volatile:

> Subsequent changes in the JMM (Java 5.0 and later) have enabled DCL to work if resource is made volatile, and the performance impact of this is small since volatile reads are usually only slightly more expensive than nonvolatile reads.

CachedRDDBuilder. cachedColumnBuffers and CachedRDDBuilder.clearCache both use DCL to manage the resource ``_cachedColumnBuffers``. The missing ingredient is that ``_cachedColumnBuffers`` is not volatile.

Because of this, clearCache may see ``_cachedColumnBuffers`` as null, when in fact it is not, and therefore fail to un-cache the RDD. There may be other, more subtle bugs due to visibility issues.

To avoid these issues, this PR makes ``_cachedColumnBuffers`` volatile.

## How was this patch tested?

- Existing SQL unit tests
- Existing pyspark-sql tests

Closes #23768 from bersprockets/SPARK-26851.

Authored-by: Bruce Robbins <bersprockets@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
9 days ago[SPARK-26650][CORE] Demote noisy HBase-related log message.
Marcelo Vanzin [Thu, 14 Feb 2019 03:40:10 +0000 (11:40 +0800)] 
[SPARK-26650][CORE] Demote noisy HBase-related log message.

Make it a debug message so that it doesn't show up in the vast
majority of cases, where HBase classes are not available.

Closes #23776 from vanzin/SPARK-26650.

Authored-by: Marcelo Vanzin <vanzin@cloudera.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
9 days ago[SPARK-26865][SQL] DataSourceV2Strategy should push normalized filters
Dongjoon Hyun [Thu, 14 Feb 2019 00:04:27 +0000 (16:04 -0800)] 
[SPARK-26865][SQL] DataSourceV2Strategy should push normalized filters

## What changes were proposed in this pull request?

This PR aims to make `DataSourceV2Strategy` normalize filters like [FileSourceStrategy](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala#L150-L158) when it pushes them into `SupportsPushDownFilters.pushFilters`.

## How was this patch tested?

Pass the Jenkins with the newly added test case.

Closes #23770 from dongjoon-hyun/SPARK-26865.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
9 days ago[SPARK-26817][CORE] Use System.nanoTime to measure time intervals
Maxim Gekk [Wed, 13 Feb 2019 19:12:16 +0000 (13:12 -0600)] 
[SPARK-26817][CORE] Use System.nanoTime to measure time intervals

## What changes were proposed in this pull request?

In the PR, I propose to use `System.nanoTime()` instead of `System.currentTimeMillis()` in measurements of time intervals.

`System.currentTimeMillis()` returns current wallclock time and will follow changes to the system clock. Thus, negative wallclock adjustments can cause timeouts to "hang" for a long time (until wallclock time has caught up to its previous value again). This can happen when ntpd does a "step" after the network has been disconnected for some time. The most canonical example is during system bootup when DHCP takes longer than usual. This can lead to failures that are really hard to understand/reproduce. `System.nanoTime()` is guaranteed to be monotonically increasing irrespective of wallclock changes.

## How was this patch tested?

By existing test suites.

Closes #23727 from MaxGekk/system-nanotime.

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-26798][SQL] HandleNullInputsForUDF should trust nullability
Wenchen Fan [Wed, 13 Feb 2019 15:22:11 +0000 (00:22 +0900)] 
[SPARK-26798][SQL] HandleNullInputsForUDF should trust nullability

## What changes were proposed in this pull request?

There is a very old TODO in `HandleNullInputsForUDF`, saying that we can skip the null check if input is not nullable. We leverage the nullability info at many places, we can trust it here too.

## How was this patch tested?

re-enable an ignored test

Closes #23712 from cloud-fan/minor.

Lead-authored-by: Wenchen Fan <wenchen@databricks.com>
Co-authored-by: Xiao Li <gatorsmile@gmail.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
10 days ago[SPARK-26835][DOCS] Notes API documentation for available options of Data sources...
Peter G. Horvath [Wed, 13 Feb 2019 14:02:51 +0000 (08:02 -0600)] 
[SPARK-26835][DOCS] Notes API documentation for available options of Data sources in SparkSQL guide

## What changes were proposed in this pull request?

This PR proposes to add some pointers of available options of Data source in Spark SQL guide.

## How was this patch tested?
N/A: documentation change

Closes #23742 from peter-gergely-horvath/SPARK-26835.

Authored-by: Peter G. Horvath <peter.gergely.horvath@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
10 days ago[MINOR][DOCS] Fix for contradiction in condition formula of keeping intermediate...
Viktor Tarasenko [Wed, 13 Feb 2019 14:01:20 +0000 (08:01 -0600)] 
[MINOR][DOCS] Fix for contradiction in condition formula of keeping intermediate state of window in structured streaming docs

This change solves contradiction in structured streaming documentation in formula which tests if specific window will be updated by calculating watermark and comparing with "T" parameter(intermediate state is cleared as (max event time seen by the engine - late threshold > T), otherwise kept(written as "until")). By further examples the "T" seems to be the end of the window, not start like documentation says firstly. For more information please take a look at my question in stackoverflow https://stackoverflow.com/questions/54599594/understanding-window-with-watermark-in-apache-spark-structured-streaming

Can be tested by building documentation.

Closes #23765 from vitektarasenko/master.

Authored-by: Viktor Tarasenko <v.tarasenko@vezet.ru>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
10 days ago[SPARK-26864][SQL] Query may return incorrect result when python udf is used as a...
Dilip Biswal [Wed, 13 Feb 2019 13:14:19 +0000 (21:14 +0800)] 
[SPARK-26864][SQL] Query may return incorrect result when python udf is used as a join condition and the udf uses attributes from both legs of left semi join.

## What changes were proposed in this pull request?
In SPARK-25314, we supported the scenario of having a python UDF that refers to attributes from both legs of a join condition by rewriting the plan to convert an inner join or left semi join to a filter over a cross join. In case of left semi join, this transformation may cause incorrect results when the right leg of join condition produces duplicate rows based on the join condition. This fix disallows the rewrite for left semi join and raises an error in the case like we do for other types of join. In future, we should have separate rule in optimizer to convert left semi join to inner join (I am aware of one case we could do it if we leverage informational constraint i.e when we know the right side does not produce duplicates).

**Python**

```SQL
>>> from pyspark import SparkContext
>>> from pyspark.sql import SparkSession, Column, Row
>>> from pyspark.sql.functions import UserDefinedFunction, udf
>>> from pyspark.sql.types import *
>>> from pyspark.sql.utils import AnalysisException
>>>
>>> spark.conf.set("spark.sql.crossJoin.enabled", "True")
>>> left = spark.createDataFrame([Row(lc1=1, lc2=1), Row(lc1=2, lc2=2)])
>>> right = spark.createDataFrame([Row(rc1=1, rc2=1), Row(rc1=1, rc2=1)])
>>> func = udf(lambda a, b: a == b, BooleanType())
>>> df = left.join(right, func("lc1", "rc1"), "leftsemi").show()
19/02/12 16:07:10 WARN PullOutPythonUDFInJoinCondition: The join condition:<lambda>(lc1#0L, rc1#4L) of the join plan contains PythonUDF only, it will be moved out and the join plan will be turned to cross join.
+---+---+
|lc1|lc2|
+---+---+
|  1|  1|
|  1|  1|
+---+---+
```

**Scala**

```SQL
scala> val left = Seq((1, 1), (2, 2)).toDF("lc1", "lc2")
left: org.apache.spark.sql.DataFrame = [lc1: int, lc2: int]

scala> val right = Seq((1, 1), (1, 1)).toDF("rc1", "rc2")
right: org.apache.spark.sql.DataFrame = [rc1: int, rc2: int]

scala> val equal = udf((p1: Integer, p2: Integer) => {
     |   p1 == p2
     | })
equal: org.apache.spark.sql.expressions.UserDefinedFunction = SparkUserDefinedFunction($Lambda$2141/11016292394666f1b5,BooleanType,List(Some(Schema(IntegerType,true)), Some(Schema(IntegerType,true))),None,false,true)

scala> val df = left.join(right, equal(col("lc1"), col("rc1")), "leftsemi")
df: org.apache.spark.sql.DataFrame = [lc1: int, lc2: int]

scala> df.show()
+---+---+
|lc1|lc2|
+---+---+
|  1|  1|
+---+---+

```

## How was this patch tested?
Modified existing tests.

Closes #23769 from dilipbiswal/dkb_python_udf_in_join.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
10 days ago[SPARK-26761][SQL][R] Vectorized R gapply() implementation
Hyukjin Kwon [Wed, 13 Feb 2019 03:19:58 +0000 (11:19 +0800)] 
[SPARK-26761][SQL][R] Vectorized R gapply() implementation

## What changes were proposed in this pull request?

This PR targets to add vectorized `gapply()` in R, Arrow optimization.

This can be tested as below:

```bash
$ ./bin/sparkR --conf spark.sql.execution.arrow.enabled=true
```

```r
df <- createDataFrame(mtcars)
collect(gapply(df,
               "gear",
               function(key, group) {
                 data.frame(gear = key[[1]], disp = mean(group$disp) > group$disp)
               },
               structType("gear double, disp boolean")))
```

### Requirements
  - R 3.5.x
  - Arrow package 0.12+
    ```bash
    Rscript -e 'remotes::install_github("apache/arrowapache-arrow-0.12.0", subdir = "r")'
    ```

**Note:** currently, Arrow R package is not in CRAN. Please take a look at ARROW-3204.
**Note:** currently, Arrow R package seems not supporting Windows. Please take a look at ARROW-3204.

### Benchmarks

**Shall**

```bash
sync && sudo purge
./bin/sparkR --conf spark.sql.execution.arrow.enabled=false
```

```bash
sync && sudo purge
./bin/sparkR --conf spark.sql.execution.arrow.enabled=true
```

**R code**

```r
rdf <- read.csv("500000.csv")
rdf <- rdf[, c("Month.of.Joining", "Weight.in.Kgs.")]  # We're only interested in the key and values to calculate.
df <- cache(createDataFrame(rdf))
count(df)

test <- function() {
  options(digits.secs = 6) # milliseconds
  start.time <- Sys.time()
  count(gapply(df,
               "Month_of_Joining",
               function(key, group) {
                 data.frame(Month_of_Joining = key[[1]], Weight_in_Kgs_ = mean(group$Weight_in_Kgs_) > group$Weight_in_Kgs_)
               },
               structType("Month_of_Joining integer, Weight_in_Kgs_ boolean")))
  end.time <- Sys.time()
  time.taken <- end.time - start.time
  print(time.taken)
}

test()
```

**Data (350 MB):**

```r
object.size(read.csv("500000.csv"))
350379504 bytes
```

"500000 Records"  http://eforexcel.com/wp/downloads-16-sample-csv-files-data-sets-for-testing/

**Results**

```
Time difference of 35.67459 secs
```

```
Time difference of 4.301399 secs
```

The performance improvement was around **829%**.

**Note that** I am 100% sure this PR improves more then 829% because I gave up testing it with non-Arrow optimization because it took super super super long when the data size becomes bigger.

### Limitations

- For now, Arrow optimization with R does not support when the data is `raw`, and when user explicitly gives float type in the schema. They produce corrupt values.

- Due to ARROW-4512, it cannot send and receive batch by batch. It has to send all batches in Arrow stream format at once. It needs improvement later.

## How was this patch tested?

Unit tests were added

**TODOs:**
- [x] Draft codes
- [x] make the tests passed
- [x] make the CRAN check pass
- [x] Performance measurement
- [x] Supportability investigation (for instance types)

Closes #23746 from HyukjinKwon/SPARK-26759.

Authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
10 days ago[SPARK-26857][SQL] Return UnsafeArrayData for date/timestamp type in ColumnarArray...
Gengliang Wang [Wed, 13 Feb 2019 01:23:31 +0000 (10:23 +0900)] 
[SPARK-26857][SQL] Return UnsafeArrayData for date/timestamp type in ColumnarArray.copy()

## What changes were proposed in this pull request?

In https://github.com/apache/spark/issues/23569, the copy method of `ColumnarArray` is implemented.
To further improve it, we can return `UnsafeArrayData` for `date`/`timestamp` type in `ColumnarArray.copy()`.

## How was this patch tested?

Unit test

Closes #23761 from gengliangwang/copyDateAndTS.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
11 days ago[SPARK-26853][SQL] Add example and version for commonly used aggregate function descr...
Dilip Biswal [Tue, 12 Feb 2019 07:24:54 +0000 (23:24 -0800)] 
[SPARK-26853][SQL] Add example and version for commonly used aggregate function descriptions

## What changes were proposed in this pull request?
This improves the expression description for commonly used aggregate functions such as Max, Min, Count, etc.

## How was this patch tested?
Verified the function description manually from the shell.

Closes #23756 from dilipbiswal/dkb_expr_description_aggregate.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
11 days ago[SPARK-25158][SQL] Executor accidentally exit because ScriptTransformationWriterThrea...
yangjie01 [Tue, 12 Feb 2019 04:16:33 +0000 (12:16 +0800)] 
[SPARK-25158][SQL] Executor accidentally exit because ScriptTransformationWriterThread throw Exception.

## What changes were proposed in this pull request?

Run Spark-Sql job use transform features(`ScriptTransformationExec`) with config `spark.speculation = true`, sometimes job fails and we found many Executor Dead through `Executor Tab`, through analysis log and code we found :

`ScriptTransformationExec` start a new thread(`ScriptTransformationWriterThread`), the new thread is very likely to throw `TaskKilledException`(from iter.map.foreach part) when speculation is on, this exception will captured by `SparkUncaughtExceptionHandler` which registered during Executor start, `SparkUncaughtExceptionHandler` will call `System.exit (SparkExitCode.UNCAUGHT_EXCEPTION)` to shutdown `Executor`, this is unexpected.

We should not kill the executor just because `ScriptTransformationWriterThread` fails. log the error(not only `TaskKilledException`) instead of throwing it is enough, Exception already pass to `ScriptTransformationExec` and handle by `TaskRunner`.

## How was this patch tested?

Register `TestUncaughtExceptionHandler` to test case in `ScriptTransformationSuite`, then assert there is no Uncaught Exception handled.

Before this patch "script transformation should not swallow errors from upstream operators (no serde)" and "script transformation should not swallow errors from upstream operators (with serde)"  throwing `IllegalArgumentException` and handle by `TestUncaughtExceptionHandler` .

Closes #22149 from LuciferYang/fix-transformation-task-kill.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
11 days ago[SPARK-26696][SQL] Makes Dataset encoder public
Simeon Simeonov [Tue, 12 Feb 2019 03:04:26 +0000 (11:04 +0800)] 
[SPARK-26696][SQL] Makes Dataset encoder public

## What changes were proposed in this pull request?

Implements the solution proposed in [SPARK-26696](https://issues.apache.org/jira/browse/SPARK-26696), a minor refactoring that allows frameworks to perform advanced type-preserving dataset transformations without carrying `Encoder` implicits from user code.

The change allows

```scala
def foo[A](ds: Dataset[A]): Dataset[A] =
  ds.toDF().as[A](ds.encoder)
```

instead of

```scala
def foo[A: Encoder](ds: Dataset[A]): Dataset[A] =
  ds.toDF().as[A](implicitly[Encoder[A]])
```

## How was this patch tested?

This patch was tested with an automated test that was later removed as it was deemed unnecessary per the discussion in this PR.

Closes #23620 from ssimeonov/ss_SPARK-26696.

Authored-by: Simeon Simeonov <sim@fastignite.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
11 days ago[SPARK-26740][SPARK-26654][SQL] Make statistics of timestamp/date columns independent...
Maxim Gekk [Tue, 12 Feb 2019 02:58:00 +0000 (10:58 +0800)] 
[SPARK-26740][SPARK-26654][SQL] Make statistics of timestamp/date columns independent from system time zones

## What changes were proposed in this pull request?

In the PR, I propose to covert underlying types of timestamp/date columns to strings, and store the converted values as column statistics. This makes statistics for timestamp/date columns independent from system time zone while saving and retrieving such statistics.

I bumped versions of stored statistics from 1 to 2 since the PR changes the format.

## How was this patch tested?

The changes were tested by `StatisticsCollectionSuite` and by `StatisticsSuite`.

Closes #23662 from MaxGekk/column-stats-time-date.

Lead-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Co-authored-by: Hyukjin Kwon <gurwls223@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
12 days ago[SPARK-26843][MESOS] Use ConfigEntry for hardcoded configs for "mesos" resource manager
Jungtaek Lim (HeartSaVioR) [Mon, 11 Feb 2019 00:34:33 +0000 (16:34 -0800)] 
[SPARK-26843][MESOS] Use ConfigEntry for hardcoded configs for "mesos" resource manager

## What changes were proposed in this pull request?

This patch makes hardcoded configs in "mesos" module to use ConfigEntry, avoiding issues on mistake like SPARK-26082.

Please note that there're some changes on type while migrating to ConfigEntry: specifically "comma-separated list on a string" becomes "sequence of strings". While SparkConf smoothly handles on the change (comma-separated list on a string is still supported so backward compatible), there're some methods in utility class (`mesos` package private) to depend on the type change, so this patch also modifies the method signature for them a bit.

## How was this patch tested?

Existing tests.

Closes #23743 from HeartSaVioR/SPARK-26843.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
12 days ago[SPARK-26816][CORE][TEST] Add XORShiftRandom Benchmark
Maxim Gekk [Sun, 10 Feb 2019 21:52:24 +0000 (13:52 -0800)] 
[SPARK-26816][CORE][TEST] Add XORShiftRandom Benchmark

## What changes were proposed in this pull request?

- The benchmark of `XORShiftRandom.nextInt` vis-a-vis `java.util.Random.nextInt` is moved from the `XORShiftRandom` object to `XORShiftRandomBenchmark`.
- Added benchmarks for `nextLong`, `nextDouble` and `nextGaussian` that are used in Spark as well.
- Added a separate benchmark for `XORShiftRandom.hashSeed`.

Closes #23752 from MaxGekk/xorshiftrandom-benchmark.

Lead-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Co-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
13 days ago[SPARK-26822] Upgrade the deprecated module 'optparse'
cchung100m [Sun, 10 Feb 2019 06:36:22 +0000 (00:36 -0600)] 
[SPARK-26822] Upgrade the deprecated module 'optparse'

Follow the [official document](https://docs.python.org/2/library/argparse.html#upgrading-optparse-code)  to upgrade the deprecated module 'optparse' to  'argparse'.

## What changes were proposed in this pull request?

This PR proposes to replace 'optparse' module with 'argparse' module.

## How was this patch tested?

Follow the [previous testing](https://github.com/apache/spark/commit/7e3eb3cd209d83394ca2b2cec79b26b1bbe9d7ea), manually tested and negative tests were also done. My [test results](https://gist.github.com/cchung100m/1661e7df6e8b66940a6e52a20861f61d)

Closes #23730 from cchung100m/solve_deprecated_module_optparse.

Authored-by: cchung100m <cchung100m@cs.ccu.edu.tw>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
13 days ago[SPARK-26082][MESOS][FOLLOWUP] Fix Scala-2.11 build
Jungtaek Lim (HeartSaVioR) [Sat, 9 Feb 2019 21:46:52 +0000 (13:46 -0800)] 
[SPARK-26082][MESOS][FOLLOWUP] Fix Scala-2.11 build

## What changes were proposed in this pull request?

#23744 added a UT to prevent a future regression. However, it breaks Scala-2.11 build. This fixes that.

## How was this patch tested?

Manual test with Scala-2.11 profile.

Closes #23755 from HeartSaVioR/SPARK-26082-FOLLOW-UP-V2.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2 weeks ago[SPARK-26766][CORE] Remove the list of filesystems from HadoopDelegationTokenProvider...
Gabor Somogyi [Fri, 8 Feb 2019 21:41:52 +0000 (13:41 -0800)] 
[SPARK-26766][CORE] Remove the list of filesystems from HadoopDelegationTokenProvider.obtainDelegationTokens

## What changes were proposed in this pull request?

Delegation token providers interface now has a parameter `fileSystems` but this is needed only for `HadoopFSDelegationTokenProvider`.

In this PR I've addressed this issue in the following way:
* Removed `fileSystems` parameter from `HadoopDelegationTokenProvider`
* Moved `YarnSparkHadoopUtil.hadoopFSsToAccess` into `HadoopFSDelegationTokenProvider`
* Moved `spark.yarn.stagingDir` into core
* Moved `spark.yarn.access.namenodes` into core and renamed to `spark.kerberos.access.namenodes`
* Moved `spark.yarn.access.hadoopFileSystems` into core and renamed to `spark.kerberos.access.hadoopFileSystems`

## How was this patch tested?

Existing unit tests.

Closes #23698 from gaborgsomogyi/SPARK-26766.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2 weeks ago[SPARK-26389][SS] Add force delete temp checkpoint configuration
Gabor Somogyi [Fri, 8 Feb 2019 18:22:51 +0000 (10:22 -0800)] 
[SPARK-26389][SS] Add force delete temp checkpoint configuration

## What changes were proposed in this pull request?

Not all users wants to keep temporary checkpoint directories. Additionally hard to restore from it.

In this PR I've added a force delete flag which is default `false`. Additionally not clear for users when temporary checkpoint directory deleted so added log messages to explain this a bit more.

## How was this patch tested?

Existing + additional unit tests.

Closes #23732 from gaborgsomogyi/SPARK-26389.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2 weeks ago[SPARK-26185][PYTHON] add weightCol in python MulticlassClassificationEvaluator
Huaxin Gao [Fri, 8 Feb 2019 17:46:54 +0000 (09:46 -0800)] 
[SPARK-26185][PYTHON] add weightCol in python MulticlassClassificationEvaluator

## What changes were proposed in this pull request?

add weightCol for python version of MulticlassClassificationEvaluator and MulticlassMetrics

## How was this patch tested?

add doc test

Closes #23157 from huaxingao/spark-26185.

Authored-by: Huaxin Gao <huaxing@us.ibm.com>
Signed-off-by: Holden Karau <holden@pigscanfly.ca>
2 weeks ago[SPARK-26831][PYTHON] Eliminates Python version check for executor at driver side...
Stefaan Lippens [Fri, 8 Feb 2019 02:43:17 +0000 (10:43 +0800)] 
[SPARK-26831][PYTHON] Eliminates Python version check for executor at driver side when using IPython

## What changes were proposed in this pull request?

I was trying out pyspark on a system with only a `python3` command but no `python` command and got this error:

```bash
/opt/spark/bin/pyspark: line 45: python: command not found
```

I also noticed the bash syntax for the IPython version check is wrong:  `[[ !  $WORKS_WITH_IPYTHON ]]`  always evaluates to false when `$WORKS_WITH_IPYTHON` is non-empty

This PR simply eliminates the Python version check for executor at driver side when using IPython.

## How was this patch tested?

I manually tested the `pyspark` launch script and bash syntax stuff

Closes #23736 from soxofaan/master.

Authored-by: Stefaan Lippens <soxofaan@gmail.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2 weeks ago[SPARK-26082][MESOS][FOLLOWUP] Add UT on fetcher cache option on MesosClusterScheduler
Jungtaek Lim (HeartSaVioR) [Thu, 7 Feb 2019 16:51:55 +0000 (08:51 -0800)] 
[SPARK-26082][MESOS][FOLLOWUP] Add UT on fetcher cache option on MesosClusterScheduler

## What changes were proposed in this pull request?

This patch adds UT on testing SPARK-26082 to avoid regression. While #23743 reduces the possibility to make a similar mistake, the needed lines of code for adding tests are not that huge, so I guess it might be worth to add them.

## How was this patch tested?

Newly added UTs. Test "supports setting fetcher cache" fails when #23743 is not applied and succeeds when #23743 is applied.

Closes #23744 from HeartSaVioR/SPARK-26082-add-unit-test.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2 weeks ago[SPARK-26082][MESOS] Fix mesos fetch cache config name
mwlon [Thu, 7 Feb 2019 09:21:31 +0000 (01:21 -0800)] 
[SPARK-26082][MESOS] Fix mesos fetch cache config name

## What changes were proposed in this pull request?

* change MesosClusterScheduler to use correct argument name for Mesos fetch cache (spark.mesos.fetchCache.enable -> spark.mesos.fetcherCache.enable)

## How was this patch tested?

Not sure this requires a test, since it's just a string change.

Closes #23734 from mwlon/SPARK-26082.

Authored-by: mwlon <mloncaric@hmc.edu>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2 weeks ago[SPARK-26734][STREAMING] Fix StackOverflowError with large block queue
Ross Lodge [Wed, 6 Feb 2019 16:43:40 +0000 (08:43 -0800)] 
[SPARK-26734][STREAMING] Fix StackOverflowError with large block queue

## What changes were proposed in this pull request?

SPARK-23991 introduced a bug in `ReceivedBlockTracker#allocateBlocksToBatch`: when a queue with more than a few thousand blocks are in the queue, serializing the queue throws a StackOverflowError.  This change just adds `dequeueAll` to the new `clone` operation on the queue so that the fix in 23991 is preserved but the serialized data comes from an ArrayBuffer which doesn't have the serialization problems that mutable.Queue has.

## How was this patch tested?

A unit test was added.

Closes #23716 from rlodge/SPARK-26734.

Authored-by: Ross Lodge <rlodge@concentricsky.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2 weeks ago[SPARK-26745][SQL][TESTS] JsonSuite test case: empty line -> 0 record count
Branden Smith [Wed, 6 Feb 2019 05:55:19 +0000 (13:55 +0800)] 
[SPARK-26745][SQL][TESTS] JsonSuite test case: empty line -> 0 record count

## What changes were proposed in this pull request?

This PR consists of the `test` components of #23665 only, minus the associated patch from that PR.

It adds a new unit test to `JsonSuite` which verifies that the `count()` returned from a `DataFrame` loaded from JSON containing empty lines does not include those empty lines in the record count. The test runs `count` prior to otherwise reading data from the `DataFrame`, so as to catch future cases where a pre-parsing optimization might result in `count` results inconsistent with existing behavior.

This PR is intended to be deployed alongside #23667; `master` currently causes the test to fail, as described in [SPARK-26745](https://issues.apache.org/jira/browse/SPARK-26745).

## How was this patch tested?

Manual testing, existing `JsonSuite` unit tests.

Closes #23674 from sumitsu/json_emptyline_count_test.

Authored-by: Branden Smith <branden.smith@publicismedia.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2 weeks ago[SPARK-26733][K8S] Cleanup entrypoint.sh.
Marcelo Vanzin [Wed, 6 Feb 2019 00:00:18 +0000 (16:00 -0800)] 
[SPARK-26733][K8S] Cleanup entrypoint.sh.

Merge both case statements, and remove unused variables that
are not set by the Scala code anymore.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #23655 from vanzin/SPARK-26733.

2 weeks ago[SPARK-26768][CORE] Remove useless code in BlockManager
Liupengcheng [Tue, 5 Feb 2019 18:47:21 +0000 (10:47 -0800)] 
[SPARK-26768][CORE] Remove useless code in BlockManager

## What changes were proposed in this pull request?

Recently, when I was reading some code of `BlockManager.getBlockData`, I found that there are useless code that would never reach. The related codes is as below:

```
override def getBlockData(blockId: BlockId): ManagedBuffer = {
  if (blockId.isShuffle) {
    shuffleManager.shuffleBlockResolver.getBlockData(blockId.asInstanceOf[ShuffleBlockId])
  } else {
    getLocalBytes(blockId) match {
      case Some(blockData) =>
        new BlockManagerManagedBuffer(blockInfoManager, blockId, blockData, true)
      case None =>
        // If this block manager receives a request for a block that it doesn't have then it's
        // likely that the master has outdated block statuses for this block. Therefore, we send
        // an RPC so that this block is marked as being unavailable from this block manager.
        reportBlockStatus(blockId, BlockStatus.empty)
        throw new BlockNotFoundException(blockId.toString)
    }
  }
}
```
```
def getLocalBytes(blockId: BlockId): Option[BlockData] = {
  logDebug(s"Getting local block $blockId as bytes")
  // As an optimization for map output fetches, if the block is for a shuffle, return it
  // without acquiring a lock; the disk store never deletes (recent) items so this should work
  if (blockId.isShuffle) {
    val shuffleBlockResolver = shuffleManager.shuffleBlockResolver
    // TODO: This should gracefully handle case where local block is not available. Currently
    // downstream code will throw an exception.
    val buf = new ChunkedByteBuffer(
      shuffleBlockResolver.getBlockData(blockId.asInstanceOf[ShuffleBlockId]).nioByteBuffer())
    Some(new ByteBufferBlockData(buf, true))
  } else {
    blockInfoManager.lockForReading(blockId).map { info => doGetLocalBytes(blockId, info) }
  }
}
```
the `blockId.isShuffle` is checked twice, but however it seems that in the method calling hierarchy of `BlockManager.getLocalBytes`, the another callsite of the `BlockManager.getLocalBytes` is at `TorrentBroadcast.readBlocks` where the blockId can never be a `ShuffleBlockId`.

![image](https://user-images.githubusercontent.com/6747355/51963980-1fe55000-24a0-11e9-961a-e10fe67f8119.png)

So I think we should remove these useless code for easy reading.

## How was this patch tested?

NA

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

Closes #23693 from liupc/Remove-useless-code-in-BlockManager.

Authored-by: Liupengcheng <liupengcheng@xiaomi.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2 weeks ago[SPARK-25692][CORE] Remove static initialization of worker eventLoop handling chunk...
schintap [Tue, 5 Feb 2019 18:43:43 +0000 (10:43 -0800)] 
[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>
2 weeks ago[SPARK-26758][CORE] Idle Executors are not getting killed after spark.dynamiAllocatio...
sandeep-katta [Tue, 5 Feb 2019 04:13:22 +0000 (20:13 -0800)] 
[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>
2 weeks ago[SPARK-26813][BUILD] Consolidate java version across language compilers and build...
seancxmao [Mon, 4 Feb 2019 14:56:24 +0000 (08:56 -0600)] 
[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>
2 weeks ago[SPARK-26603][K8S] Update minikube backend
Stavros Kontopoulos [Mon, 4 Feb 2019 01:15:20 +0000 (17:15 -0800)] 
[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.

2 weeks ago[SPARK-26751][SQL] Fix memory leak when statement run in background and throw excepti...
zhoukang [Sun, 3 Feb 2019 14:45:57 +0000 (08:45 -0600)] 
[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>
2 weeks ago[SPARK-26818][ML] Make MLEvents JSON ser/de safe
Hyukjin Kwon [Sun, 3 Feb 2019 13:19:35 +0000 (21:19 +0800)] 
[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.

https://github.com/apache/spark/blob/ec506bd30c2ca324c12c9ec811764081c2eb8c42/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala#L148

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

https://github.com/apache/spark/blob/9690eba16efe6d25261934d8b73a221972b684f3/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:

https://github.com/apache/spark/blob/ca545f79410a464ef24e3986fac225f53bb2ef02/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>
2 weeks ago[SPARK-26805][SQL] Eliminate double checking of stringToDate and stringToTimestamp...
Maxim Gekk [Sun, 3 Feb 2019 00:20:16 +0000 (18:20 -0600)] 
[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>
3 weeks ago[SPARK-26677][BUILD] Update Parquet to 1.10.1 with notEq pushdown fix.
Ryan Blue [Sat, 2 Feb 2019 17:17:52 +0000 (09:17 -0800)] 
[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>
3 weeks ago[MINOR][SQL][DOCS] Reformat the tables in SQL migration guide
Hyukjin Kwon [Sat, 2 Feb 2019 15:45:46 +0000 (23:45 +0800)] 
[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>
3 weeks ago[SPARK-26651][SQL][DOC] Collapse notes related to java.time API
Maxim Gekk [Sat, 2 Feb 2019 03:17:33 +0000 (11:17 +0800)] 
[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>
3 weeks ago[SPARK-18161][PYTHON] Update cloudpickle to v0.6.1
Boris Shminke [Sat, 2 Feb 2019 02:49:45 +0000 (10:49 +0800)] 
[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>
3 weeks ago[SPARK-26714][CORE][WEBUI] Show 0 partition job in WebUI
xiaodeshan [Sat, 2 Feb 2019 00:38:27 +0000 (18:38 -0600)] 
[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>
3 weeks ago[MINOR][DOC] Writing to partitioned Hive metastore Parquet tables is not supported...
liuxian [Sat, 2 Feb 2019 00:34:13 +0000 (18:34 -0600)] 
[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:
 https://github.com/apache/spark/blob/d53e11ffce3f721886918c1cb4525478971f02bc/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>
3 weeks ago[SPARK-26771][CORE][GRAPHX] Make .unpersist(), .destroy() consistently non-blocking...
Sean Owen [Sat, 2 Feb 2019 00:29:55 +0000 (18:29 -0600)] 
[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>
3 weeks ago[SPARK-26754][PYTHON] Add hasTrainingSummary to replace duplicate code in PySpark
Huaxin Gao [Fri, 1 Feb 2019 23:29:58 +0000 (17:29 -0600)] 
[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>