spark.git
4 hours ago[SPARK-23476][CORE] Generate secret in local mode when authentication on master
Gabor Somogyi [Thu, 22 Feb 2018 20:07:51 +0000 (12:07 -0800)] 
[SPARK-23476][CORE] Generate secret in local mode when authentication on

## What changes were proposed in this pull request?

If spark is run with "spark.authenticate=true", then it will fail to start in local mode.

This PR generates secret in local mode when authentication on.

## How was this patch tested?

Modified existing unit test.
Manually started spark-shell.

Author: Gabor Somogyi <gabor.g.somogyi@gmail.com>

Closes #20652 from gaborgsomogyi/SPARK-23476.

5 hours ago[SPARK-23475][UI] Show also skipped stages
Marco Gaido [Thu, 22 Feb 2018 19:00:12 +0000 (11:00 -0800)] 
[SPARK-23475][UI] Show also skipped stages

## What changes were proposed in this pull request?

SPARK-20648 introduced the status `SKIPPED` for the stages. On the UI, previously, skipped stages were shown as `PENDING`; after this change, they are not shown on the UI.

The PR introduce a new section in order to show also `SKIPPED` stages in a proper table.

## How was this patch tested?

manual tests

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #20651 from mgaido91/SPARK-23475.

21 hours ago[SPARK-23475][WEBUI] Skipped stages should be evicted before completed stages
Shixiong Zhu [Thu, 22 Feb 2018 03:43:11 +0000 (19:43 -0800)] 
[SPARK-23475][WEBUI] Skipped stages should be evicted before completed stages

## What changes were proposed in this pull request?

The root cause of missing completed stages is because `cleanupStages` will never remove skipped stages.

This PR changes the logic to always remove skipped stage first. This is safe since  the job itself contains enough information to render skipped stages in the UI.

## How was this patch tested?

The new unit tests.

Author: Shixiong Zhu <zsxwing@gmail.com>

Closes #20656 from zsxwing/SPARK-23475.

25 hours ago[SPARK-23481][WEBUI] lastStageAttempt should fail when a stage doesn't exist
Shixiong Zhu [Wed, 21 Feb 2018 23:37:28 +0000 (15:37 -0800)] 
[SPARK-23481][WEBUI] lastStageAttempt should fail when a stage doesn't exist

## What changes were proposed in this pull request?

The issue here is `AppStatusStore.lastStageAttempt` will return the next available stage in the store when a stage doesn't exist.

This PR adds `last(stageId)` to ensure it returns a correct `StageData`

## How was this patch tested?

The new unit test.

Author: Shixiong Zhu <zsxwing@gmail.com>

Closes #20654 from zsxwing/SPARK-23481.

25 hours ago[SPARK-23484][SS] Fix possible race condition in KafkaContinuousReader
Tathagata Das [Wed, 21 Feb 2018 22:56:13 +0000 (14:56 -0800)] 
[SPARK-23484][SS] Fix possible race condition in KafkaContinuousReader

## What changes were proposed in this pull request?

var `KafkaContinuousReader.knownPartitions` should be threadsafe as it is accessed from multiple threads - the query thread at the time of reader factory creation, and the epoch tracking thread at the time of `needsReconfiguration`.

## How was this patch tested?

Existing tests.

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

Closes #20655 from tdas/SPARK-23484.

30 hours ago[SPARK-23217][ML][PYTHON] Add distanceMeasure param to ClusteringEvaluator Python API
Marco Gaido [Wed, 21 Feb 2018 18:39:36 +0000 (12:39 -0600)] 
[SPARK-23217][ML][PYTHON] Add distanceMeasure param to ClusteringEvaluator Python API

## What changes were proposed in this pull request?

The PR adds the `distanceMeasure` param to ClusteringEvaluator in the Python API. This allows the user to specify `cosine` as distance measure in addition to the default `squaredEuclidean`.

## How was this patch tested?

added UT

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #20627 from mgaido91/SPARK-23217_python.

41 hours ago[SPARK-23418][SQL] Fail DataSourceV2 reads when user schema is passed, but not supported.
Ryan Blue [Wed, 21 Feb 2018 07:10:08 +0000 (15:10 +0800)] 
[SPARK-23418][SQL] Fail DataSourceV2 reads when user schema is passed, but not supported.

## What changes were proposed in this pull request?

DataSourceV2 initially allowed user-supplied schemas when a source doesn't implement `ReadSupportWithSchema`, as long as the schema was identical to the source's schema. This is confusing behavior because changes to an underlying table can cause a previously working job to fail with an exception that user-supplied schemas are not allowed.

This reverts commit adcb25a0624, which was added to #20387 so that it could be removed in a separate JIRA issue and PR.

## How was this patch tested?

Existing tests.

Author: Ryan Blue <blue@apache.org>

Closes #20603 from rdblue/SPARK-23418-revert-adcb25a0624.

45 hours ago[SPARK-23424][SQL] Add codegenStageId in comment
Kazuaki Ishizaki [Wed, 21 Feb 2018 03:26:06 +0000 (11:26 +0800)] 
[SPARK-23424][SQL] Add codegenStageId in comment

## What changes were proposed in this pull request?

This PR always adds `codegenStageId` in comment of the generated class. This is a replication of #20419  for post-Spark 2.3.
Closes #20419

```
/* 001 */ public Object generate(Object[] references) {
/* 002 */   return new GeneratedIteratorForCodegenStage1(references);
/* 003 */ }
/* 004 */
/* 005 */ // codegenStageId=1
/* 006 */ final class GeneratedIteratorForCodegenStage1 extends org.apache.spark.sql.execution.BufferedRowIterator {
/* 007 */   private Object[] references;
...
```

## How was this patch tested?

Existing tests

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

Closes #20612 from kiszk/SPARK-23424.

46 hours ago[SPARK-23454][SS][DOCS] Added trigger information to the Structured Streaming program...
Tathagata Das [Wed, 21 Feb 2018 02:16:10 +0000 (18:16 -0800)] 
[SPARK-23454][SS][DOCS] Added trigger information to the Structured Streaming programming guide

## What changes were proposed in this pull request?

- Added clear information about triggers
- Made the semantics guarantees of watermarks more clear for streaming aggregations and stream-stream joins.

## How was this patch tested?

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

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

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

Closes #20631 from tdas/SPARK-23454.

46 hours ago[SPARK-23468][CORE] Stringify auth secret before storing it in credentials.
Marcelo Vanzin [Wed, 21 Feb 2018 02:06:21 +0000 (18:06 -0800)] 
[SPARK-23468][CORE] Stringify auth secret before storing it in credentials.

The secret is used as a string in many parts of the code, so it has
to be turned into a hex string to avoid issues such as the random
byte sequence not containing a valid UTF8 sequence.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #20643 from vanzin/SPARK-23468.

46 hours ago[SPARK-23470][UI] Use first attempt of last stage to define job description.
Marcelo Vanzin [Wed, 21 Feb 2018 01:54:06 +0000 (17:54 -0800)] 
[SPARK-23470][UI] Use first attempt of last stage to define job description.

This is much faster than finding out what the last attempt is, and the
data should be the same.

There's room for improvement in this page (like only loading data for
the jobs being shown, instead of loading all available jobs and sorting
them), but this should bring performance on par with the 2.2 version.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #20644 from vanzin/SPARK-23470.

2 days ago[SPARK-23434][SQL] Spark should not warn `metadata directory` for a HDFS file path
Dongjoon Hyun [Wed, 21 Feb 2018 00:02:44 +0000 (16:02 -0800)] 
[SPARK-23434][SQL] Spark should not warn `metadata directory` for a HDFS file path

## What changes were proposed in this pull request?

In a kerberized cluster, when Spark reads a file path (e.g. `people.json`), it warns with a wrong warning message during looking up `people.json/_spark_metadata`. The root cause of this situation is the difference between `LocalFileSystem` and `DistributedFileSystem`. `LocalFileSystem.exists()` returns `false`, but `DistributedFileSystem.exists` raises `org.apache.hadoop.security.AccessControlException`.

```scala
scala> spark.version
res0: String = 2.4.0-SNAPSHOT

scala> spark.read.json("file:///usr/hdp/current/spark-client/examples/src/main/resources/people.json").show
+----+-------+
| age|   name|
+----+-------+
|null|Michael|
|  30|   Andy|
|  19| Justin|
+----+-------+

scala> spark.read.json("hdfs:///tmp/people.json")
18/02/15 05:00:48 WARN streaming.FileStreamSink: Error while looking for metadata directory.
18/02/15 05:00:48 WARN streaming.FileStreamSink: Error while looking for metadata directory.
```

After this PR,
```scala
scala> spark.read.json("hdfs:///tmp/people.json").show
+----+-------+
| age|   name|
+----+-------+
|null|Michael|
|  30|   Andy|
|  19| Justin|
+----+-------+
```

## How was this patch tested?

Manual.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #20616 from dongjoon-hyun/SPARK-23434.

2 days ago[SPARK-23456][SPARK-21783] Turn on `native` ORC impl and PPD by default
Dongjoon Hyun [Tue, 20 Feb 2018 17:14:56 +0000 (09:14 -0800)] 
[SPARK-23456][SPARK-21783] Turn on `native` ORC impl and PPD by default

## What changes were proposed in this pull request?

Apache Spark 2.3 introduced `native` ORC supports with vectorization and many fixes. However, it's shipped as a not-default option. This PR enables `native` ORC implementation and predicate-pushdown by default for Apache Spark 2.4. We will improve and stabilize ORC data source before Apache Spark 2.4. And, eventually, Apache Spark will drop old Hive-based ORC code.

## How was this patch tested?

Pass the Jenkins with existing tests.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #20634 from dongjoon-hyun/SPARK-23456.

2 days ago[SPARK-23383][BUILD][MINOR] Make a distribution should exit with usage while detectin...
Kent Yao [Tue, 20 Feb 2018 13:51:30 +0000 (07:51 -0600)] 
[SPARK-23383][BUILD][MINOR] Make a distribution should exit with usage while detecting wrong options

## What changes were proposed in this pull request?
```shell
./dev/make-distribution.sh --name ne-1.0.0-SNAPSHOT xyz --tgz  -Phadoop-2.7
+++ dirname ./dev/make-distribution.sh
++ cd ./dev/..
++ pwd
+ SPARK_HOME=/Users/Kent/Documents/spark
+ DISTDIR=/Users/Kent/Documents/spark/dist
+ MAKE_TGZ=false
+ MAKE_PIP=false
+ MAKE_R=false
+ NAME=none
+ MVN=/Users/Kent/Documents/spark/build/mvn
+ ((  5  ))
+ case $1 in
+ NAME=ne-1.0.0-SNAPSHOT
+ shift
+ shift
+ ((  3  ))
+ case $1 in
+ break
+ '[' -z /Users/Kent/.jenv/candidates/java/current ']'
+ '[' -z /Users/Kent/.jenv/candidates/java/current ']'
++ command -v git
+ '[' /usr/local/bin/git ']'
++ git rev-parse --short HEAD
+ GITREV=98ea6a7
+ '[' '!' -z 98ea6a7 ']'
+ GITREVSTRING=' (git revision 98ea6a7)'
+ unset GITREV
++ command -v /Users/Kent/Documents/spark/build/mvn
+ '[' '!' /Users/Kent/Documents/spark/build/mvn ']'
++ /Users/Kent/Documents/spark/build/mvn help:evaluate -Dexpression=project.version xyz --tgz -Phadoop-2.7
++ grep -v INFO
++ tail -n 1
+ VERSION=' -X,--debug                             Produce execution debug output'
```
It is better to declare the mistakes and exit with usage than `break`

## How was this patch tested?

manually

cc srowen

Author: Kent Yao <yaooqinn@hotmail.com>

Closes #20571 from yaooqinn/SPARK-23383.

2 days ago[SPARK-23240][PYTHON] Better error message when extraneous data in pyspark.daemon...
Bruce Robbins [Tue, 20 Feb 2018 11:26:26 +0000 (20:26 +0900)] 
[SPARK-23240][PYTHON] Better error message when extraneous data in pyspark.daemon's stdout

## What changes were proposed in this pull request?

Print more helpful message when daemon module's stdout is empty or contains a bad port number.

## How was this patch tested?

Manually recreated the environmental issues that caused the mysterious exceptions at one site. Tested that the expected messages are logged.

Also, ran all scala unit tests.

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

Author: Bruce Robbins <bersprockets@gmail.com>

Closes #20424 from bersprockets/SPARK-23240_prop2.

2 days ago[SPARK-23203][SQL] DataSourceV2: Use immutable logical plans.
Ryan Blue [Tue, 20 Feb 2018 08:04:22 +0000 (16:04 +0800)] 
[SPARK-23203][SQL] DataSourceV2: Use immutable logical plans.

## What changes were proposed in this pull request?

SPARK-23203: DataSourceV2 should use immutable catalyst trees instead of wrapping a mutable DataSourceV2Reader. This commit updates DataSourceV2Relation and consolidates much of the DataSourceV2 API requirements for the read path in it. Instead of wrapping a reader that changes, the relation lazily produces a reader from its configuration.

This commit also updates the predicate and projection push-down. Instead of the implementation from SPARK-22197, this reuses the rule matching from the Hive and DataSource read paths (using `PhysicalOperation`) and copies most of the implementation of `SparkPlanner.pruneFilterProject`, with updates for DataSourceV2. By reusing the implementation from other read paths, this should have fewer regressions from other read paths and is less code to maintain.

The new push-down rules also supports the following edge cases:

* The output of DataSourceV2Relation should be what is returned by the reader, in case the reader can only partially satisfy the requested schema projection
* The requested projection passed to the DataSourceV2Reader should include filter columns
* The push-down rule may be run more than once if filters are not pushed through projections

## How was this patch tested?

Existing push-down and read tests.

Author: Ryan Blue <blue@apache.org>

Closes #20387 from rdblue/SPARK-22386-push-down-immutable-trees.

2 days ago[SPARK-23436][SQL] Infer partition as Date only if it can be casted to Date
Marco Gaido [Tue, 20 Feb 2018 05:56:38 +0000 (13:56 +0800)] 
[SPARK-23436][SQL] Infer partition as Date only if it can be casted to Date

## What changes were proposed in this pull request?

Before the patch, Spark could infer as Date a partition value which cannot be casted to Date (this can happen when there are extra characters after a valid date, like `2018-02-15AAA`).

When this happens and the input format has metadata which define the schema of the table, then `null` is returned as a value for the partition column, because the `cast` operator used in (`PartitioningAwareFileIndex.inferPartitioning`) is unable to convert the value.

The PR checks in the partition inference that values can be casted to Date and Timestamp, in order to infer that datatype to them.

## How was this patch tested?

added UT

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #20621 from mgaido91/SPARK-23436.

2 days ago[SPARK-23457][SQL] Register task completion listeners first in ParquetFileFormat
Dongjoon Hyun [Tue, 20 Feb 2018 05:33:03 +0000 (13:33 +0800)] 
[SPARK-23457][SQL] Register task completion listeners first in ParquetFileFormat

## What changes were proposed in this pull request?

ParquetFileFormat leaks opened files in some cases. This PR prevents that by registering task completion listers first before initialization.

- [spark-branch-2.3-test-sbt-hadoop-2.7](https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-branch-2.3-test-sbt-hadoop-2.7/205/testReport/org.apache.spark.sql/FileBasedDataSourceSuite/_It_is_not_a_test_it_is_a_sbt_testing_SuiteSelector_/)
- [spark-master-test-sbt-hadoop-2.6](https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-sbt-hadoop-2.6/4228/testReport/junit/org.apache.spark.sql.execution.datasources.parquet/ParquetQuerySuite/_It_is_not_a_test_it_is_a_sbt_testing_SuiteSelector_/)

```
Caused by: sbt.ForkMain$ForkError: java.lang.Throwable: null
at org.apache.spark.DebugFilesystem$.addOpenStream(DebugFilesystem.scala:36)
at org.apache.spark.DebugFilesystem.open(DebugFilesystem.scala:70)
at org.apache.hadoop.fs.FileSystem.open(FileSystem.java:769)
at org.apache.parquet.hadoop.ParquetFileReader.<init>(ParquetFileReader.java:538)
at org.apache.spark.sql.execution.datasources.parquet.SpecificParquetRecordReaderBase.initialize(SpecificParquetRecordReaderBase.java:149)
at org.apache.spark.sql.execution.datasources.parquet.VectorizedParquetRecordReader.initialize(VectorizedParquetRecordReader.java:133)
at org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat$$anonfun$buildReaderWithPartitionValues$1.apply(ParquetFileFormat.scala:400)
at
```

## How was this patch tested?

Manual. The following test case generates the same leakage.

```scala
  test("SPARK-23457 Register task completion listeners first in ParquetFileFormat") {
    withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_BATCH_SIZE.key -> s"${Int.MaxValue}") {
      withTempDir { dir =>
        val basePath = dir.getCanonicalPath
        Seq(0).toDF("a").write.format("parquet").save(new Path(basePath, "first").toString)
        Seq(1).toDF("a").write.format("parquet").save(new Path(basePath, "second").toString)
        val df = spark.read.parquet(
          new Path(basePath, "first").toString,
          new Path(basePath, "second").toString)
        val e = intercept[SparkException] {
          df.collect()
        }
        assert(e.getCause.isInstanceOf[OutOfMemoryError])
      }
    }
  }
```

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #20619 from dongjoon-hyun/SPARK-23390.

5 days ago[SPARK-23340][SQL] Upgrade Apache ORC to 1.4.3
Dongjoon Hyun [Sat, 17 Feb 2018 08:25:36 +0000 (00:25 -0800)] 
[SPARK-23340][SQL] Upgrade Apache ORC to 1.4.3

## What changes were proposed in this pull request?

This PR updates Apache ORC dependencies to 1.4.3 released on February 9th. Apache ORC 1.4.2 release removes unnecessary dependencies and 1.4.3 has 5 more patches (https://s.apache.org/Fll8).

Especially, the following ORC-285 is fixed at 1.4.3.

```scala
scala> val df = Seq(Array.empty[Float]).toDF()

scala> df.write.format("orc").save("/tmp/floatarray")

scala> spark.read.orc("/tmp/floatarray")
res1: org.apache.spark.sql.DataFrame = [value: array<float>]

scala> spark.read.orc("/tmp/floatarray").show()
18/02/12 22:09:10 ERROR Executor: Exception in task 0.0 in stage 1.0 (TID 1)
java.io.IOException: Error reading file: file:/tmp/floatarray/part-00000-9c0b461b-4df1-4c23-aac1-3e4f349ac7d6-c000.snappy.orc
at org.apache.orc.impl.RecordReaderImpl.nextBatch(RecordReaderImpl.java:1191)
at org.apache.orc.mapreduce.OrcMapreduceRecordReader.ensureBatch(OrcMapreduceRecordReader.java:78)
...
Caused by: java.io.EOFException: Read past EOF for compressed stream Stream for column 2 kind DATA position: 0 length: 0 range: 0 offset: 0 limit: 0
```

## How was this patch tested?

Pass the Jenkins test.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #20511 from dongjoon-hyun/SPARK-23340.

5 days ago[SPARK-23447][SQL] Cleanup codegen template for Literal
Kris Mok [Sat, 17 Feb 2018 02:54:14 +0000 (10:54 +0800)] 
[SPARK-23447][SQL] Cleanup codegen template for Literal

## What changes were proposed in this pull request?

Cleaned up the codegen templates for `Literal`s, to make sure that the `ExprCode` returned from `Literal.doGenCode()` has:
1. an empty `code` field;
2. an `isNull` field of either literal `true` or `false`;
3. a `value` field that is just a simple literal/constant.

Before this PR, there are a couple of paths that would return a non-trivial `code` and all of them are actually unnecessary. The `NaN` and `Infinity` constants for `double` and `float` can be accessed through constants directly available so there's no need to add a reference for them.

Also took the opportunity to add a new util method for ease of creating `ExprCode` for inline-able non-null values.

## How was this patch tested?

Existing tests.

Author: Kris Mok <kris.mok@databricks.com>

Closes #20626 from rednaxelafx/codegen-literal.

5 days ago[SPARK-23381][CORE] Murmur3 hash generates a different value from other implementations
Shintaro Murakami [Sat, 17 Feb 2018 01:17:55 +0000 (17:17 -0800)] 
[SPARK-23381][CORE] Murmur3 hash generates a different value from other implementations

## What changes were proposed in this pull request?
Murmur3 hash generates a different value from the original and other implementations (like Scala standard library and Guava or so) when the length of a bytes array is not multiple of 4.

## How was this patch tested?
Added a unit test.

**Note: When we merge this PR, please give all the credits to Shintaro Murakami.**

Author: Shintaro Murakami <mrkm4ntrgmail.com>

Author: gatorsmile <gatorsmile@gmail.com>
Author: Shintaro Murakami <mrkm4ntr@gmail.com>

Closes #20630 from gatorsmile/pr-20568.

6 days ago[SPARK-23362][SS] Migrate Kafka Microbatch source to v2
Tathagata Das [Fri, 16 Feb 2018 22:30:19 +0000 (14:30 -0800)] 
[SPARK-23362][SS] Migrate Kafka Microbatch source to v2

## What changes were proposed in this pull request?
Migrating KafkaSource (with data source v1) to KafkaMicroBatchReader (with data source v2).

Performance comparison:
In a unit test with in-process Kafka broker, I tested the read throughput of V1 and V2 using 20M records in a single partition. They were comparable.

## How was this patch tested?
Existing tests, few modified to be better tests than the existing ones.

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

Closes #20554 from tdas/SPARK-23362.

6 days ago[SPARK-23446][PYTHON] Explicitly check supported types in toPandas
hyukjinkwon [Fri, 16 Feb 2018 17:41:17 +0000 (09:41 -0800)] 
[SPARK-23446][PYTHON] Explicitly check supported types in toPandas

## What changes were proposed in this pull request?

This PR explicitly specifies and checks the types we supported in `toPandas`. This was a hole. For example, we haven't finished the binary type support in Python side yet but now it allows as below:

```python
spark.conf.set("spark.sql.execution.arrow.enabled", "false")
df = spark.createDataFrame([[bytearray("a")]])
df.toPandas()
spark.conf.set("spark.sql.execution.arrow.enabled", "true")
df.toPandas()
```

```
     _1
0  [97]
  _1
0  a
```

This should be disallowed. I think the same things also apply to nested timestamps too.

I also added some nicer message about `spark.sql.execution.arrow.enabled` in the error message.

## How was this patch tested?

Manually tested and tests added in `python/pyspark/sql/tests.py`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #20625 from HyukjinKwon/pandas_convertion_supported_type.

7 days ago[SPARK-23413][UI] Fix sorting tasks by Host / Executor ID at the Stage page
“attilapiros” [Thu, 15 Feb 2018 19:51:24 +0000 (13:51 -0600)] 
[SPARK-23413][UI] Fix sorting tasks by Host / Executor ID at the Stage page

## What changes were proposed in this pull request?

Fixing exception got at sorting tasks by Host / Executor ID:
```
        java.lang.IllegalArgumentException: Invalid sort column: Host
at org.apache.spark.ui.jobs.ApiHelper$.indexName(StagePage.scala:1017)
at org.apache.spark.ui.jobs.TaskDataSource.sliceData(StagePage.scala:694)
at org.apache.spark.ui.PagedDataSource.pageData(PagedTable.scala:61)
at org.apache.spark.ui.PagedTable$class.table(PagedTable.scala:96)
at org.apache.spark.ui.jobs.TaskPagedTable.table(StagePage.scala:708)
at org.apache.spark.ui.jobs.StagePage.liftedTree1$1(StagePage.scala:293)
at org.apache.spark.ui.jobs.StagePage.render(StagePage.scala:282)
at org.apache.spark.ui.WebUI$$anonfun$2.apply(WebUI.scala:82)
at org.apache.spark.ui.WebUI$$anonfun$2.apply(WebUI.scala:82)
at org.apache.spark.ui.JettyUtils$$anon$3.doGet(JettyUtils.scala:90)
at javax.servlet.http.HttpServlet.service(HttpServlet.java:687)
at javax.servlet.http.HttpServlet.service(HttpServlet.java:790)
at org.spark_project.jetty.servlet.ServletHolder.handle(ServletHolder.java:848)
at org.spark_project.jetty.servlet.ServletHandler.doHandle(ServletHandler.java:584)
```

Moreover some refactoring to avoid similar problems by introducing constants for each header name and reusing them at the identification of the corresponding sorting index.

## How was this patch tested?

Manually:

![screen shot 2018-02-13 at 18 57 10](https://user-images.githubusercontent.com/2017933/36166532-1cfdf3b8-10f3-11e8-8d32-5fcaad2af214.png)

Author: “attilapiros” <piros.attila.zsolt@gmail.com>

Closes #20601 from attilapiros/SPARK-23413.

7 days ago[SPARK-23377][ML] Fixes Bucketizer with multiple columns persistence bug
Liang-Chi Hsieh [Thu, 15 Feb 2018 17:54:39 +0000 (09:54 -0800)] 
[SPARK-23377][ML] Fixes Bucketizer with multiple columns persistence bug

## What changes were proposed in this pull request?

#### Problem:

Since 2.3, `Bucketizer` supports multiple input/output columns. We will check if exclusive params are set during transformation. E.g., if `inputCols` and `outputCol` are both set, an error will be thrown.

However, when we write `Bucketizer`, looks like the default params and user-supplied params are merged during writing. All saved params are loaded back and set to created model instance. So the default `outputCol` param in `HasOutputCol` trait will be set in `paramMap` and become an user-supplied param. That makes the check of exclusive params failed.

#### Fix:

This changes the saving logic of Bucketizer to handle this case. This is a quick fix to catch the time of 2.3. We should consider modify the persistence mechanism later.

Please see the discussion in the JIRA.

Note: The multi-column `QuantileDiscretizer` also has the same issue.

## How was this patch tested?

Modified tests.

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

Closes #20594 from viirya/SPARK-23377-2.

7 days ago[MINOR][SQL] Fix an error message about inserting into bucketed tables
Dongjoon Hyun [Thu, 15 Feb 2018 17:40:08 +0000 (09:40 -0800)] 
[MINOR][SQL] Fix an error message about inserting into bucketed tables

## What changes were proposed in this pull request?

This replaces `Sparkcurrently` to `Spark currently` in the following error message.

```scala
scala> sql("insert into t2 select * from v1")
org.apache.spark.sql.AnalysisException: Output Hive table `default`.`t2`
is bucketed but Sparkcurrently does NOT populate bucketed ...
```

## How was this patch tested?

Manual.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #20617 from dongjoon-hyun/SPARK-ERROR-MSG.

7 days ago[SPARK-23426][SQL] Use `hive` ORC impl and disable PPD for Spark 2.3.0
Dongjoon Hyun [Thu, 15 Feb 2018 16:55:39 +0000 (08:55 -0800)] 
[SPARK-23426][SQL] Use `hive` ORC impl and disable PPD for Spark 2.3.0

## What changes were proposed in this pull request?

To prevent any regressions, this PR changes ORC implementation to `hive` by default like Spark 2.2.X.
Users can enable `native` ORC. Also, ORC PPD is also restored to `false` like Spark 2.2.X.

![orc_section](https://user-images.githubusercontent.com/9700541/36221575-57a1d702-1173-11e8-89fe-dca5842f4ca7.png)

## How was this patch tested?

Pass all test cases.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #20610 from dongjoon-hyun/SPARK-ORC-DISABLE.

7 days ago[INFRA] Close stale PRs.
Marcelo Vanzin [Thu, 15 Feb 2018 15:47:40 +0000 (07:47 -0800)] 
[INFRA] Close stale PRs.

Closes #20587
Closes #20586

7 days ago[SPARK-23422][CORE] YarnShuffleIntegrationSuite fix when SPARK_PREPEN…
Gabor Somogyi [Thu, 15 Feb 2018 11:52:40 +0000 (03:52 -0800)] 
[SPARK-23422][CORE] YarnShuffleIntegrationSuite fix when SPARK_PREPEN…

…D_CLASSES set to 1

## What changes were proposed in this pull request?

YarnShuffleIntegrationSuite fails when SPARK_PREPEND_CLASSES set to 1.

Normally mllib built before yarn module. When SPARK_PREPEND_CLASSES used mllib classes are on yarn test classpath.

Before 2.3 that did not cause issues. But 2.3 has SPARK-22450, which registered some mllib classes with the kryo serializer. Now it dies with the following error:

`
18/02/13 07:33:29 INFO SparkContext: Starting job: collect at YarnShuffleIntegrationSuite.scala:143
Exception in thread "dag-scheduler-event-loop" java.lang.NoClassDefFoundError: breeze/linalg/DenseMatrix
`

In this PR NoClassDefFoundError caught only in case of testing and then do nothing.

## How was this patch tested?

Automated: Pass the Jenkins.

Author: Gabor Somogyi <gabor.g.somogyi@gmail.com>

Closes #20608 from gaborgsomogyi/SPARK-23422.

7 days ago[SPARK-23359][SQL] Adds an alias 'names' of 'fieldNames' in Scala's StructType
hyukjinkwon [Thu, 15 Feb 2018 09:13:05 +0000 (17:13 +0800)] 
[SPARK-23359][SQL] Adds an alias 'names' of 'fieldNames' in Scala's StructType

## What changes were proposed in this pull request?

This PR proposes to add an alias 'names' of  'fieldNames' in Scala. Please see the discussion in [SPARK-20090](https://issues.apache.org/jira/browse/SPARK-20090).

## How was this patch tested?

Unit tests added in `DataTypeSuite.scala`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #20545 from HyukjinKwon/SPARK-23359.

7 days ago[SPARK-23366] Improve hot reading path in ReadAheadInputStream
Juliusz Sompolski [Thu, 15 Feb 2018 09:09:06 +0000 (17:09 +0800)] 
[SPARK-23366] Improve hot reading path in ReadAheadInputStream

## What changes were proposed in this pull request?

`ReadAheadInputStream` was introduced in https://github.com/apache/spark/pull/18317/ to optimize reading spill files from disk.
However, from the profiles it seems that the hot path of reading small amounts of data (like readInt) is inefficient - it involves taking locks, and multiple checks.

Optimize locking: Lock is not needed when simply accessing the active buffer. Only lock when needing to swap buffers or trigger async reading, or get information about the async state.

Optimize short-path single byte reads, that are used e.g. by Java library DataInputStream.readInt.

The asyncReader used to call "read" only once on the underlying stream, that never filled the underlying buffer when it was wrapping an LZ4BlockInputStream. If the buffer was returned unfilled, that would trigger the async reader to be triggered to fill the read ahead buffer on each call, because the reader would see that the active buffer is below the refill threshold all the time.

However, filling the full buffer all the time could introduce increased latency, so also add an `AtomicBoolean` flag for the async reader to return earlier if there is a reader waiting for data.

Remove `readAheadThresholdInBytes` and instead immediately trigger async read when switching the buffers. It allows to simplify code paths, especially the hot one that then only has to check if there is available data in the active buffer, without worrying if it needs to retrigger async read. It seems to have positive effect on perf.

## How was this patch tested?

It was noticed as a regression in some workloads after upgrading to Spark 2.3. 

It was particularly visible on TPCDS Q95 running on instances with fast disk (i3 AWS instances).
Running with profiling:
* Spark 2.2 - 5.2-5.3 minutes 9.5% in LZ4BlockInputStream.read
* Spark 2.3 - 6.4-6.6 minutes 31.1% in ReadAheadInputStream.read
* Spark 2.3 + fix - 5.3-5.4 minutes 13.3% in ReadAheadInputStream.read - very slightly slower, practically within noise.

We didn't see other regressions, and many workloads in general seem to be faster with Spark 2.3 (not investigated if thanks to async readed, or unrelated).

Author: Juliusz Sompolski <julek@databricks.com>

Closes #20555 from juliuszsompolski/SPARK-23366.

7 days ago[SPARK-23419][SPARK-23416][SS] data source v2 write path should re-throw interruption...
Wenchen Fan [Thu, 15 Feb 2018 08:59:44 +0000 (16:59 +0800)] 
[SPARK-23419][SPARK-23416][SS] data source v2 write path should re-throw interruption exceptions directly

## What changes were proposed in this pull request?

Streaming execution has a list of exceptions that means interruption, and handle them specially. `WriteToDataSourceV2Exec` should also respect this list and not wrap them with `SparkException`.

## How was this patch tested?

existing test.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20605 from cloud-fan/write.

7 days ago[SPARK-23094] Revert [] Fix invalid character handling in JsonDataSource
gatorsmile [Thu, 15 Feb 2018 07:56:02 +0000 (23:56 -0800)] 
[SPARK-23094] Revert [] Fix invalid character handling in JsonDataSource

## What changes were proposed in this pull request?
This PR is to revert the PR https://github.com/apache/spark/pull/20302, because it causes a regression.

## How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20614 from gatorsmile/revertJsonFix.

7 days ago[SPARK-23421][SPARK-22356][SQL] Document the behavior change in
gatorsmile [Thu, 15 Feb 2018 07:52:59 +0000 (23:52 -0800)] 
[SPARK-23421][SPARK-22356][SQL] Document the behavior change in

## What changes were proposed in this pull request?
https://github.com/apache/spark/pull/19579 introduces a behavior change. We need to document it in the migration guide.

## How was this patch tested?
Also update the HiveExternalCatalogVersionsSuite to verify it.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20606 from gatorsmile/addMigrationGuide.

8 days ago[SPARK-23406][SS] Enable stream-stream self-joins
Tathagata Das [Wed, 14 Feb 2018 22:27:02 +0000 (14:27 -0800)] 
[SPARK-23406][SS] Enable stream-stream self-joins

## What changes were proposed in this pull request?

Solved two bugs to enable stream-stream self joins.

### Incorrect analysis due to missing MultiInstanceRelation trait
Streaming leaf nodes did not extend MultiInstanceRelation, which is necessary for the catalyst analyzer to convert the self-join logical plan DAG into a tree (by creating new instances of the leaf relations). This was causing the error `Failure when resolving conflicting references in Join:` (see JIRA for details).

### Incorrect attribute rewrite when splicing batch plans in MicroBatchExecution
When splicing the source's batch plan into the streaming plan (by replacing the StreamingExecutionPlan), we were rewriting the attribute reference in the streaming plan with the new attribute references from the batch plan. This was incorrectly handling the scenario when multiple StreamingExecutionRelation point to the same source, and therefore eventually point to the same batch plan returned by the source. Here is an example query, and its corresponding plan transformations.
```
val df = input.toDF
val join =
      df.select('value % 5 as "key", 'value).join(
        df.select('value % 5 as "key", 'value), "key")
```
Streaming logical plan before splicing the batch plan
```
Project [key#6, value#1, value#12]
+- Join Inner, (key#6 = key#9)
   :- Project [(value#1 % 5) AS key#6, value#1]
   :  +- StreamingExecutionRelation Memory[#1], value#1
   +- Project [(value#12 % 5) AS key#9, value#12]
      +- StreamingExecutionRelation Memory[#1], value#12  // two different leaves pointing to same source
```
Batch logical plan after splicing the batch plan and before rewriting
```
Project [key#6, value#1, value#12]
+- Join Inner, (key#6 = key#9)
   :- Project [(value#1 % 5) AS key#6, value#1]
   :  +- LocalRelation [value#66]           // replaces StreamingExecutionRelation Memory[#1], value#1
   +- Project [(value#12 % 5) AS key#9, value#12]
      +- LocalRelation [value#66]           // replaces StreamingExecutionRelation Memory[#1], value#12
```
Batch logical plan after rewriting the attributes. Specifically, for spliced, the new output attributes (value#66) replace the earlier output attributes (value#12, and value#1, one for each StreamingExecutionRelation).
```
Project [key#6, value#66, value#66]       // both value#1 and value#12 replaces by value#66
+- Join Inner, (key#6 = key#9)
   :- Project [(value#66 % 5) AS key#6, value#66]
   :  +- LocalRelation [value#66]
   +- Project [(value#66 % 5) AS key#9, value#66]
      +- LocalRelation [value#66]
```
This causes the optimizer to eliminate value#66 from one side of the join.
```
Project [key#6, value#66, value#66]
+- Join Inner, (key#6 = key#9)
   :- Project [(value#66 % 5) AS key#6, value#66]
   :  +- LocalRelation [value#66]
   +- Project [(value#66 % 5) AS key#9]   // this does not generate value, incorrect join results
      +- LocalRelation [value#66]
```

**Solution**: Instead of rewriting attributes, use a Project to introduce aliases between the output attribute references and the new reference generated by the spliced plans. The analyzer and optimizer will take care of the rest.
```
Project [key#6, value#1, value#12]
+- Join Inner, (key#6 = key#9)
   :- Project [(value#1 % 5) AS key#6, value#1]
   :  +- Project [value#66 AS value#1]   // solution: project with aliases
   :     +- LocalRelation [value#66]
   +- Project [(value#12 % 5) AS key#9, value#12]
      +- Project [value#66 AS value#12]    // solution: project with aliases
         +- LocalRelation [value#66]
```

## How was this patch tested?
New unit test

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

Closes #20598 from tdas/SPARK-23406.

8 days agoRevert "[SPARK-23249][SQL] Improved block merging logic for partitions"
gatorsmile [Wed, 14 Feb 2018 18:57:12 +0000 (10:57 -0800)] 
Revert "[SPARK-23249][SQL] Improved block merging logic for partitions"

This reverts commit 8c21170decfb9ca4d3233e1ea13bd1b6e3199ed9.

8 days ago[SPARK-23394][UI] In RDD storage page show the executor addresses instead of the IDs
“attilapiros” [Wed, 14 Feb 2018 14:45:54 +0000 (06:45 -0800)] 
[SPARK-23394][UI] In RDD storage page show the executor addresses instead of the IDs

## What changes were proposed in this pull request?

Extending RDD storage page to show executor addresses in the block table.

## How was this patch tested?

Manually:

![screen shot 2018-02-13 at 10 30 59](https://user-images.githubusercontent.com/2017933/36142668-0b3578f8-10a9-11e8-95ea-2f57703ee4af.png)

Author: “attilapiros” <piros.attila.zsolt@gmail.com>

Closes #20589 from attilapiros/SPARK-23394.

8 days ago[SPARK-23399][SQL] Register a task completion listener first for OrcColumnarBatchReader
Dongjoon Hyun [Wed, 14 Feb 2018 02:55:24 +0000 (10:55 +0800)] 
[SPARK-23399][SQL] Register a task completion listener first for OrcColumnarBatchReader

## What changes were proposed in this pull request?

This PR aims to resolve an open file leakage issue reported at [SPARK-23390](https://issues.apache.org/jira/browse/SPARK-23390) by moving the listener registration position. Currently, the sequence is like the following.

1. Create `batchReader`
2. `batchReader.initialize` opens a ORC file.
3. `batchReader.initBatch` may take a long time to alloc memory in some environment and cause errors.
4. `Option(TaskContext.get()).foreach(_.addTaskCompletionListener(_ => iter.close()))`

This PR moves 4 before 2 and 3. To sum up, the new sequence is 1 -> 4 -> 2 -> 3.

## How was this patch tested?

Manual. The following test case makes OOM intentionally to cause leaked filesystem connection in the current code base. With this patch, leakage doesn't occurs.

```scala
  // This should be tested manually because it raises OOM intentionally
  // in order to cause `Leaked filesystem connection`.
  test("SPARK-23399 Register a task completion listener first for OrcColumnarBatchReader") {
    withSQLConf(SQLConf.ORC_VECTORIZED_READER_BATCH_SIZE.key -> s"${Int.MaxValue}") {
      withTempDir { dir =>
        val basePath = dir.getCanonicalPath
        Seq(0).toDF("a").write.format("orc").save(new Path(basePath, "first").toString)
        Seq(1).toDF("a").write.format("orc").save(new Path(basePath, "second").toString)
        val df = spark.read.orc(
          new Path(basePath, "first").toString,
          new Path(basePath, "second").toString)
        val e = intercept[SparkException] {
          df.collect()
        }
        assert(e.getCause.isInstanceOf[OutOfMemoryError])
      }
    }
  }
```

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #20590 from dongjoon-hyun/SPARK-23399.

9 days agoRevert "[SPARK-23303][SQL] improve the explain result for data source v2 relations"
gatorsmile [Wed, 14 Feb 2018 00:21:17 +0000 (16:21 -0800)] 
Revert "[SPARK-23303][SQL] improve the explain result for data source v2 relations"

This reverts commit f17b936f0ddb7d46d1349bd42f9a64c84c06e48d.

9 days ago[SPARK-23235][CORE] Add executor Threaddump to api
“attilapiros” [Tue, 13 Feb 2018 22:46:43 +0000 (16:46 -0600)] 
[SPARK-23235][CORE] Add executor Threaddump to api

## What changes were proposed in this pull request?

Extending api with the executor thread dump data.

For this new REST URL is introduced:
- GET http://localhost:4040/api/v1/applications/{applicationId}/executors/{executorId}/threads

<details>
<summary>Example response:</summary>

``` javascript
[ {
  "threadId" : 52,
  "threadName" : "context-cleaner-periodic-gc",
  "threadState" : "TIMED_WAITING",
  "stackTrace" : "sun.misc.Unsafe.park(Native Method)\njava.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)\njava.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)\njava.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:1093)\njava.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:809)\njava.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1074)\njava.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)\njava.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\njava.lang.Thread.run(Thread.java:748)",
  "blockedByLock" : "Lock(java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject1385411893})",
  "holdingLocks" : [ ]
}, {
  "threadId" : 48,
  "threadName" : "dag-scheduler-event-loop",
  "threadState" : "WAITING",
  "stackTrace" : "sun.misc.Unsafe.park(Native Method)\njava.util.concurrent.locks.LockSupport.park(LockSupport.java:175)\njava.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)\njava.util.concurrent.LinkedBlockingDeque.takeFirst(LinkedBlockingDeque.java:492)\njava.util.concurrent.LinkedBlockingDeque.take(LinkedBlockingDeque.java:680)\norg.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:46)",
  "blockedByLock" : "Lock(java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject1138053349})",
  "holdingLocks" : [ ]
}, {
  "threadId" : 17,
  "threadName" : "dispatcher-event-loop-0",
  "threadState" : "WAITING",
  "stackTrace" : "sun.misc.Unsafe.park(Native Method)\njava.util.concurrent.locks.LockSupport.park(LockSupport.java:175)\njava.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)\njava.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)\norg.apache.spark.rpc.netty.Dispatcher$MessageLoop.run(Dispatcher.scala:215)\njava.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\njava.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\njava.lang.Thread.run(Thread.java:748)",
  "blockedByLock" : "Lock(java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject1764626380})",
  "holdingLocks" : [ "Lock(java.util.concurrent.ThreadPoolExecutor$Worker832743930})" ]
}, {
  "threadId" : 18,
  "threadName" : "dispatcher-event-loop-1",
  "threadState" : "WAITING",
  "stackTrace" : "sun.misc.Unsafe.park(Native Method)\njava.util.concurrent.locks.LockSupport.park(LockSupport.java:175)\njava.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)\njava.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)\norg.apache.spark.rpc.netty.Dispatcher$MessageLoop.run(Dispatcher.scala:215)\njava.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\njava.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\njava.lang.Thread.run(Thread.java:748)",
  "blockedByLock" : "Lock(java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject1764626380})",
  "holdingLocks" : [ "Lock(java.util.concurrent.ThreadPoolExecutor$Worker834153999})" ]
}, {
  "threadId" : 19,
  "threadName" : "dispatcher-event-loop-2",
  "threadState" : "WAITING",
  "stackTrace" : "sun.misc.Unsafe.park(Native Method)\njava.util.concurrent.locks.LockSupport.park(LockSupport.java:175)\njava.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)\njava.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)\norg.apache.spark.rpc.netty.Dispatcher$MessageLoop.run(Dispatcher.scala:215)\njava.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\njava.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\njava.lang.Thread.run(Thread.java:748)",
  "blockedByLock" : "Lock(java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject1764626380})",
  "holdingLocks" : [ "Lock(java.util.concurrent.ThreadPoolExecutor$Worker664836465})" ]
}, {
  "threadId" : 20,
  "threadName" : "dispatcher-event-loop-3",
  "threadState" : "WAITING",
  "stackTrace" : "sun.misc.Unsafe.park(Native Method)\njava.util.concurrent.locks.LockSupport.park(LockSupport.java:175)\njava.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)\njava.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)\norg.apache.spark.rpc.netty.Dispatcher$MessageLoop.run(Dispatcher.scala:215)\njava.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\njava.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\njava.lang.Thread.run(Thread.java:748)",
  "blockedByLock" : "Lock(java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject1764626380})",
  "holdingLocks" : [ "Lock(java.util.concurrent.ThreadPoolExecutor$Worker1645557354})" ]
}, {
  "threadId" : 21,
  "threadName" : "dispatcher-event-loop-4",
  "threadState" : "WAITING",
  "stackTrace" : "sun.misc.Unsafe.park(Native Method)\njava.util.concurrent.locks.LockSupport.park(LockSupport.java:175)\njava.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)\njava.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)\norg.apache.spark.rpc.netty.Dispatcher$MessageLoop.run(Dispatcher.scala:215)\njava.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\njava.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\njava.lang.Thread.run(Thread.java:748)",
  "blockedByLock" : "Lock(java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject1764626380})",
  "holdingLocks" : [ "Lock(java.util.concurrent.ThreadPoolExecutor$Worker1188871851})" ]
}, {
  "threadId" : 22,
  "threadName" : "dispatcher-event-loop-5",
  "threadState" : "WAITING",
  "stackTrace" : "sun.misc.Unsafe.park(Native Method)\njava.util.concurrent.locks.LockSupport.park(LockSupport.java:175)\njava.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)\njava.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)\norg.apache.spark.rpc.netty.Dispatcher$MessageLoop.run(Dispatcher.scala:215)\njava.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\njava.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\njava.lang.Thread.run(Thread.java:748)",
  "blockedByLock" : "Lock(java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject1764626380})",
  "holdingLocks" : [ "Lock(java.util.concurrent.ThreadPoolExecutor$Worker920926249})" ]
}, {
  "threadId" : 23,
  "threadName" : "dispatcher-event-loop-6",
  "threadState" : "WAITING",
  "stackTrace" : "sun.misc.Unsafe.park(Native Method)\njava.util.concurrent.locks.LockSupport.park(LockSupport.java:175)\njava.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)\njava.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)\norg.apache.spark.rpc.netty.Dispatcher$MessageLoop.run(Dispatcher.scala:215)\njava.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\njava.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\njava.lang.Thread.run(Thread.java:748)",
  "blockedByLock" : "Lock(java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject1764626380})",
  "holdingLocks" : [ "Lock(java.util.concurrent.ThreadPoolExecutor$Worker355222677})" ]
}, {
  "threadId" : 24,
  "threadName" : "dispatcher-event-loop-7",
  "threadState" : "WAITING",
  "stackTrace" : "sun.misc.Unsafe.park(Native Method)\njava.util.concurrent.locks.LockSupport.park(LockSupport.java:175)\njava.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)\njava.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)\norg.apache.spark.rpc.netty.Dispatcher$MessageLoop.run(Dispatcher.scala:215)\njava.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\njava.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\njava.lang.Thread.run(Thread.java:748)",
  "blockedByLock" : "Lock(java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject1764626380})",
  "holdingLocks" : [ "Lock(java.util.concurrent.ThreadPoolExecutor$Worker1589745212})" ]
}, {
  "threadId" : 49,
  "threadName" : "driver-heartbeater",
  "threadState" : "TIMED_WAITING",
  "stackTrace" : "sun.misc.Unsafe.park(Native Method)\njava.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)\njava.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)\njava.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:1093)\njava.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:809)\njava.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1074)\njava.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)\njava.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\njava.lang.Thread.run(Thread.java:748)",
  "blockedByLock" : "Lock(java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject1602885835})",
  "holdingLocks" : [ ]
}, {
  "threadId" : 53,
  "threadName" : "element-tracking-store-worker",
  "threadState" : "WAITING",
  "stackTrace" : "sun.misc.Unsafe.park(Native Method)\njava.util.concurrent.locks.LockSupport.park(LockSupport.java:175)\njava.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)\njava.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)\njava.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1074)\njava.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)\njava.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\njava.lang.Thread.run(Thread.java:748)",
  "blockedByLock" : "Lock(java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject1439439099})",
  "holdingLocks" : [ ]
}, {
  "threadId" : 3,
  "threadName" : "Finalizer",
  "threadState" : "WAITING",
  "stackTrace" : "java.lang.Object.wait(Native Method)\njava.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:143)\njava.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:164)\njava.lang.ref.Finalizer$FinalizerThread.run(Finalizer.java:209)",
  "blockedByLock" : "Lock(java.lang.ref.ReferenceQueue$Lock1213098236})",
  "holdingLocks" : [ ]
}, {
  "threadId" : 15,
  "threadName" : "ForkJoinPool-1-worker-13",
  "threadState" : "WAITING",
  "stackTrace" : "sun.misc.Unsafe.park(Native Method)\nscala.concurrent.forkjoin.ForkJoinPool.scan(ForkJoinPool.java:2075)\nscala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)\nscala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)",
  "blockedByLock" : "Lock(scala.concurrent.forkjoin.ForkJoinPool380286413})",
  "holdingLocks" : [ ]
}, {
  "threadId" : 45,
  "threadName" : "heartbeat-receiver-event-loop-thread",
  "threadState" : "TIMED_WAITING",
  "stackTrace" : "sun.misc.Unsafe.park(Native Method)\njava.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)\njava.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)\njava.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:1093)\njava.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:809)\njava.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1074)\njava.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)\njava.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\njava.lang.Thread.run(Thread.java:748)",
  "blockedByLock" : "Lock(java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject715135812})",
  "holdingLocks" : [ ]
}, {
  "threadId" : 1,
  "threadName" : "main",
  "threadState" : "RUNNABLE",
  "stackTrace" : "java.io.FileInputStream.read0(Native Method)\njava.io.FileInputStream.read(FileInputStream.java:207)\nscala.tools.jline_embedded.internal.NonBlockingInputStream.read(NonBlockingInputStream.java:169) => holding Monitor(scala.tools.jline_embedded.internal.NonBlockingInputStream46248392})\nscala.tools.jline_embedded.internal.NonBlockingInputStream.read(NonBlockingInputStream.java:137)\nscala.tools.jline_embedded.internal.NonBlockingInputStream.read(NonBlockingInputStream.java:246)\nscala.tools.jline_embedded.internal.InputStreamReader.read(InputStreamReader.java:261) => holding Monitor(scala.tools.jline_embedded.internal.NonBlockingInputStream46248392})\nscala.tools.jline_embedded.internal.InputStreamReader.read(InputStreamReader.java:198) => holding Monitor(scala.tools.jline_embedded.internal.NonBlockingInputStream46248392})\nscala.tools.jline_embedded.console.ConsoleReader.readCharacter(ConsoleReader.java:2145)\nscala.tools.jline_embedded.console.ConsoleReader.readLine(ConsoleReader.java:2349)\nscala.tools.jline_embedded.console.ConsoleReader.readLine(ConsoleReader.java:2269)\nscala.tools.nsc.interpreter.jline_embedded.InteractiveReader.readOneLine(JLineReader.scala:57)\nscala.tools.nsc.interpreter.InteractiveReader$$anonfun$readLine$2.apply(InteractiveReader.scala:37)\nscala.tools.nsc.interpreter.InteractiveReader$$anonfun$readLine$2.apply(InteractiveReader.scala:37)\nscala.tools.nsc.interpreter.InteractiveReader$.restartSysCalls(InteractiveReader.scala:44)\nscala.tools.nsc.interpreter.InteractiveReader$class.readLine(InteractiveReader.scala:37)\nscala.tools.nsc.interpreter.jline_embedded.InteractiveReader.readLine(JLineReader.scala:28)\nscala.tools.nsc.interpreter.ILoop.readOneLine(ILoop.scala:404)\nscala.tools.nsc.interpreter.ILoop.loop(ILoop.scala:413)\nscala.tools.nsc.interpreter.ILoop$$anonfun$process$1.apply$mcZ$sp(ILoop.scala:923)\nscala.tools.nsc.interpreter.ILoop$$anonfun$process$1.apply(ILoop.scala:909)\nscala.tools.nsc.interpreter.ILoop$$anonfun$process$1.apply(ILoop.scala:909)\nscala.reflect.internal.util.ScalaClassLoader$.savingContextLoader(ScalaClassLoader.scala:97)\nscala.tools.nsc.interpreter.ILoop.process(ILoop.scala:909)\norg.apache.spark.repl.Main$.doMain(Main.scala:76)\norg.apache.spark.repl.Main$.main(Main.scala:56)\norg.apache.spark.repl.Main.main(Main.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)\norg.apache.spark.deploy.SparkSubmit$.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:879)\norg.apache.spark.deploy.SparkSubmit$.doRunMain$1(SparkSubmit.scala:197)\norg.apache.spark.deploy.SparkSubmit$.submit(SparkSubmit.scala:227)\norg.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:136)\norg.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)",
  "blockedByLock" : "",
  "holdingLocks" : [ "Monitor(scala.tools.jline_embedded.internal.NonBlockingInputStream46248392})" ]
}, {
  "threadId" : 26,
  "threadName" : "map-output-dispatcher-0",
  "threadState" : "WAITING",
  "stackTrace" : "sun.misc.Unsafe.park(Native Method)\njava.util.concurrent.locks.LockSupport.park(LockSupport.java:175)\njava.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)\njava.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)\norg.apache.spark.MapOutputTrackerMaster$MessageLoop.run(MapOutputTracker.scala:384)\njava.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\njava.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\njava.lang.Thread.run(Thread.java:748)",
  "blockedByLock" : "Lock(java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject350285679})",
  "holdingLocks" : [ "Lock(java.util.concurrent.ThreadPoolExecutor$Worker1791280119})" ]
}, {
  "threadId" : 27,
  "threadName" : "map-output-dispatcher-1",
  "threadState" : "WAITING",
  "stackTrace" : "sun.misc.Unsafe.park(Native Method)\njava.util.concurrent.locks.LockSupport.park(LockSupport.java:175)\njava.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)\njava.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)\norg.apache.spark.MapOutputTrackerMaster$MessageLoop.run(MapOutputTracker.scala:384)\njava.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\njava.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\njava.lang.Thread.run(Thread.java:748)",
  "blockedByLock" : "Lock(java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject350285679})",
  "holdingLocks" : [ "Lock(java.util.concurrent.ThreadPoolExecutor$Worker1947378744})" ]
}, {
  "threadId" : 28,
  "threadName" : "map-output-dispatcher-2",
  "threadState" : "WAITING",
  "stackTrace" : "sun.misc.Unsafe.park(Native Method)\njava.util.concurrent.locks.LockSupport.park(LockSupport.java:175)\njava.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)\njava.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)\norg.apache.spark.MapOutputTrackerMaster$MessageLoop.run(MapOutputTracker.scala:384)\njava.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\njava.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\njava.lang.Thread.run(Thread.java:748)",
  "blockedByLock" : "Lock(java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject350285679})",
  "holdingLocks" : [ "Lock(java.util.concurrent.ThreadPoolExecutor$Worker507507251})" ]
}, {
  "threadId" : 29,
  "threadName" : "map-output-dispatcher-3",
  "threadState" : "WAITING",
  "stackTrace" : "sun.misc.Unsafe.park(Native Method)\njava.util.concurrent.locks.LockSupport.park(LockSupport.java:175)\njava.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)\njava.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)\norg.apache.spark.MapOutputTrackerMaster$MessageLoop.run(MapOutputTracker.scala:384)\njava.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\njava.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\njava.lang.Thread.run(Thread.java:748)",
  "blockedByLock" : "Lock(java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject350285679})",
  "holdingLocks" : [ "Lock(java.util.concurrent.ThreadPoolExecutor$Worker1016408627})" ]
}, {
  "threadId" : 30,
  "threadName" : "map-output-dispatcher-4",
  "threadState" : "WAITING",
  "stackTrace" : "sun.misc.Unsafe.park(Native Method)\njava.util.concurrent.locks.LockSupport.park(LockSupport.java:175)\njava.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)\njava.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)\norg.apache.spark.MapOutputTrackerMaster$MessageLoop.run(MapOutputTracker.scala:384)\njava.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\njava.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\njava.lang.Thread.run(Thread.java:748)",
  "blockedByLock" : "Lock(java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject350285679})",
  "holdingLocks" : [ "Lock(java.util.concurrent.ThreadPoolExecutor$Worker1879219501})" ]
}, {
  "threadId" : 31,
  "threadName" : "map-output-dispatcher-5",
  "threadState" : "WAITING",
  "stackTrace" : "sun.misc.Unsafe.park(Native Method)\njava.util.concurrent.locks.LockSupport.park(LockSupport.java:175)\njava.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)\njava.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)\norg.apache.spark.MapOutputTrackerMaster$MessageLoop.run(MapOutputTracker.scala:384)\njava.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\njava.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\njava.lang.Thread.run(Thread.java:748)",
  "blockedByLock" : "Lock(java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject350285679})",
  "holdingLocks" : [ "Lock(java.util.concurrent.ThreadPoolExecutor$Worker290509937})" ]
}, {
  "threadId" : 32,
  "threadName" : "map-output-dispatcher-6",
  "threadState" : "WAITING",
  "stackTrace" : "sun.misc.Unsafe.park(Native Method)\njava.util.concurrent.locks.LockSupport.park(LockSupport.java:175)\njava.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)\njava.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)\norg.apache.spark.MapOutputTrackerMaster$MessageLoop.run(MapOutputTracker.scala:384)\njava.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\njava.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\njava.lang.Thread.run(Thread.java:748)",
  "blockedByLock" : "Lock(java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject350285679})",
  "holdingLocks" : [ "Lock(java.util.concurrent.ThreadPoolExecutor$Worker1889468930})" ]
}, {
  "threadId" : 33,
  "threadName" : "map-output-dispatcher-7",
  "threadState" : "WAITING",
  "stackTrace" : "sun.misc.Unsafe.park(Native Method)\njava.util.concurrent.locks.LockSupport.park(LockSupport.java:175)\njava.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)\njava.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)\norg.apache.spark.MapOutputTrackerMaster$MessageLoop.run(MapOutputTracker.scala:384)\njava.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\njava.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\njava.lang.Thread.run(Thread.java:748)",
  "blockedByLock" : "Lock(java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject350285679})",
  "holdingLocks" : [ "Lock(java.util.concurrent.ThreadPoolExecutor$Worker1699637904})" ]
}, {
  "threadId" : 47,
  "threadName" : "netty-rpc-env-timeout",
  "threadState" : "TIMED_WAITING",
  "stackTrace" : "sun.misc.Unsafe.park(Native Method)\njava.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)\njava.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)\njava.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:1093)\njava.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:809)\njava.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1074)\njava.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)\njava.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\njava.lang.Thread.run(Thread.java:748)",
  "blockedByLock" : "Lock(java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject977194847})",
  "holdingLocks" : [ ]
}, {
  "threadId" : 14,
  "threadName" : "NonBlockingInputStreamThread",
  "threadState" : "WAITING",
  "stackTrace" : "java.lang.Object.wait(Native Method)\nscala.tools.jline_embedded.internal.NonBlockingInputStream.run(NonBlockingInputStream.java:278)\njava.lang.Thread.run(Thread.java:748)",
  "blockedByThreadId" : 1,
  "blockedByLock" : "Lock(scala.tools.jline_embedded.internal.NonBlockingInputStream46248392})",
  "holdingLocks" : [ ]
}, {
  "threadId" : 2,
  "threadName" : "Reference Handler",
  "threadState" : "WAITING",
  "stackTrace" : "java.lang.Object.wait(Native Method)\njava.lang.Object.wait(Object.java:502)\njava.lang.ref.Reference.tryHandlePending(Reference.java:191)\njava.lang.ref.Reference$ReferenceHandler.run(Reference.java:153)",
  "blockedByLock" : "Lock(java.lang.ref.Reference$Lock1359433302})",
  "holdingLocks" : [ ]
}, {
  "threadId" : 35,
  "threadName" : "refresh progress",
  "threadState" : "TIMED_WAITING",
  "stackTrace" : "java.lang.Object.wait(Native Method)\njava.util.TimerThread.mainLoop(Timer.java:552)\njava.util.TimerThread.run(Timer.java:505)",
  "blockedByLock" : "Lock(java.util.TaskQueue44276328})",
  "holdingLocks" : [ ]
}, {
  "threadId" : 34,
  "threadName" : "RemoteBlock-temp-file-clean-thread",
  "threadState" : "TIMED_WAITING",
  "stackTrace" : "java.lang.Object.wait(Native Method)\njava.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:143)\norg.apache.spark.storage.BlockManager$RemoteBlockTempFileManager.org$apache$spark$storage$BlockManager$RemoteBlockTempFileManager$$keepCleaning(BlockManager.scala:1630)\norg.apache.spark.storage.BlockManager$RemoteBlockTempFileManager$$anon$1.run(BlockManager.scala:1608)",
  "blockedByLock" : "Lock(java.lang.ref.ReferenceQueue$Lock391748181})",
  "holdingLocks" : [ ]
}, {
  "threadId" : 25,
  "threadName" : "rpc-server-3-1",
  "threadState" : "RUNNABLE",
  "stackTrace" : "sun.nio.ch.KQueueArrayWrapper.kevent0(Native Method)\nsun.nio.ch.KQueueArrayWrapper.poll(KQueueArrayWrapper.java:198)\nsun.nio.ch.KQueueSelectorImpl.doSelect(KQueueSelectorImpl.java:117)\nsun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86) => holding Monitor(sun.nio.ch.KQueueSelectorImpl2057702496})\nsun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)\nio.netty.channel.nio.SelectedSelectionKeySetSelector.select(SelectedSelectionKeySetSelector.java:62)\nio.netty.channel.nio.NioEventLoop.select(NioEventLoop.java:753)\nio.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:409)\nio.netty.util.concurrent.SingleThreadEventExecutor$5.run(SingleThreadEventExecutor.java:858)\nio.netty.util.concurrent.DefaultThreadFactory$DefaultRunnableDecorator.run(DefaultThreadFactory.java:138)\njava.lang.Thread.run(Thread.java:748)",
  "blockedByLock" : "",
  "holdingLocks" : [ "Monitor(io.netty.channel.nio.SelectedSelectionKeySet1066929256})", "Monitor(java.util.Collections$UnmodifiableSet561426729})", "Monitor(sun.nio.ch.KQueueSelectorImpl2057702496})" ]
}, {
  "threadId" : 50,
  "threadName" : "shuffle-server-5-1",
  "threadState" : "RUNNABLE",
  "stackTrace" : "sun.nio.ch.KQueueArrayWrapper.kevent0(Native Method)\nsun.nio.ch.KQueueArrayWrapper.poll(KQueueArrayWrapper.java:198)\nsun.nio.ch.KQueueSelectorImpl.doSelect(KQueueSelectorImpl.java:117)\nsun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86) => holding Monitor(sun.nio.ch.KQueueSelectorImpl1401522546})\nsun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)\nio.netty.channel.nio.SelectedSelectionKeySetSelector.select(SelectedSelectionKeySetSelector.java:62)\nio.netty.channel.nio.NioEventLoop.select(NioEventLoop.java:753)\nio.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:409)\nio.netty.util.concurrent.SingleThreadEventExecutor$5.run(SingleThreadEventExecutor.java:858)\nio.netty.util.concurrent.DefaultThreadFactory$DefaultRunnableDecorator.run(DefaultThreadFactory.java:138)\njava.lang.Thread.run(Thread.java:748)",
  "blockedByLock" : "",
  "holdingLocks" : [ "Monitor(io.netty.channel.nio.SelectedSelectionKeySet385972319})", "Monitor(java.util.Collections$UnmodifiableSet477937109})", "Monitor(sun.nio.ch.KQueueSelectorImpl1401522546})" ]
}, {
  "threadId" : 4,
  "threadName" : "Signal Dispatcher",
  "threadState" : "RUNNABLE",
  "stackTrace" : "",
  "blockedByLock" : "",
  "holdingLocks" : [ ]
}, {
  "threadId" : 51,
  "threadName" : "Spark Context Cleaner",
  "threadState" : "TIMED_WAITING",
  "stackTrace" : "java.lang.Object.wait(Native Method)\njava.lang.ref.ReferenceQueue.remove(ReferenceQueue.java:143)\norg.apache.spark.ContextCleaner$$anonfun$org$apache$spark$ContextCleaner$$keepCleaning$1.apply$mcV$sp(ContextCleaner.scala:181)\norg.apache.spark.util.Utils$.tryOrStopSparkContext(Utils.scala:1319)\norg.apache.spark.ContextCleaner.org$apache$spark$ContextCleaner$$keepCleaning(ContextCleaner.scala:178)\norg.apache.spark.ContextCleaner$$anon$1.run(ContextCleaner.scala:73)",
  "blockedByLock" : "Lock(java.lang.ref.ReferenceQueue$Lock1739420764})",
  "holdingLocks" : [ ]
}, {
  "threadId" : 16,
  "threadName" : "spark-listener-group-appStatus",
  "threadState" : "WAITING",
  "stackTrace" : "sun.misc.Unsafe.park(Native Method)\njava.util.concurrent.locks.LockSupport.park(LockSupport.java:175)\njava.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)\njava.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)\norg.apache.spark.scheduler.AsyncEventQueue$$anonfun$org$apache$spark$scheduler$AsyncEventQueue$$dispatch$1.apply(AsyncEventQueue.scala:94)\nscala.util.DynamicVariable.withValue(DynamicVariable.scala:58)\norg.apache.spark.scheduler.AsyncEventQueue.org$apache$spark$scheduler$AsyncEventQueue$$dispatch(AsyncEventQueue.scala:83)\norg.apache.spark.scheduler.AsyncEventQueue$$anon$1$$anonfun$run$1.apply$mcV$sp(AsyncEventQueue.scala:79)\norg.apache.spark.util.Utils$.tryOrStopSparkContext(Utils.scala:1319)\norg.apache.spark.scheduler.AsyncEventQueue$$anon$1.run(AsyncEventQueue.scala:78)",
  "blockedByLock" : "Lock(java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject1287190987})",
  "holdingLocks" : [ ]
}, {
  "threadId" : 44,
  "threadName" : "spark-listener-group-executorManagement",
  "threadState" : "WAITING",
  "stackTrace" : "sun.misc.Unsafe.park(Native Method)\njava.util.concurrent.locks.LockSupport.park(LockSupport.java:175)\njava.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)\njava.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)\norg.apache.spark.scheduler.AsyncEventQueue$$anonfun$org$apache$spark$scheduler$AsyncEventQueue$$dispatch$1.apply(AsyncEventQueue.scala:94)\nscala.util.DynamicVariable.withValue(DynamicVariable.scala:58)\norg.apache.spark.scheduler.AsyncEventQueue.org$apache$spark$scheduler$AsyncEventQueue$$dispatch(AsyncEventQueue.scala:83)\norg.apache.spark.scheduler.AsyncEventQueue$$anon$1$$anonfun$run$1.apply$mcV$sp(AsyncEventQueue.scala:79)\norg.apache.spark.util.Utils$.tryOrStopSparkContext(Utils.scala:1319)\norg.apache.spark.scheduler.AsyncEventQueue$$anon$1.run(AsyncEventQueue.scala:78)",
  "blockedByLock" : "Lock(java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject943262890})",
  "holdingLocks" : [ ]
}, {
  "threadId" : 54,
  "threadName" : "spark-listener-group-shared",
  "threadState" : "WAITING",
  "stackTrace" : "sun.misc.Unsafe.park(Native Method)\njava.util.concurrent.locks.LockSupport.park(LockSupport.java:175)\njava.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)\njava.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)\norg.apache.spark.scheduler.AsyncEventQueue$$anonfun$org$apache$spark$scheduler$AsyncEventQueue$$dispatch$1.apply(AsyncEventQueue.scala:94)\nscala.util.DynamicVariable.withValue(DynamicVariable.scala:58)\norg.apache.spark.scheduler.AsyncEventQueue.org$apache$spark$scheduler$AsyncEventQueue$$dispatch(AsyncEventQueue.scala:83)\norg.apache.spark.scheduler.AsyncEventQueue$$anon$1$$anonfun$run$1.apply$mcV$sp(AsyncEventQueue.scala:79)\norg.apache.spark.util.Utils$.tryOrStopSparkContext(Utils.scala:1319)\norg.apache.spark.scheduler.AsyncEventQueue$$anon$1.run(AsyncEventQueue.scala:78)",
  "blockedByLock" : "Lock(java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject334604425})",
  "holdingLocks" : [ ]
}, {
  "threadId" : 37,
  "threadName" : "SparkUI-37",
  "threadState" : "TIMED_WAITING",
  "stackTrace" : "sun.misc.Unsafe.park(Native Method)\njava.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)\njava.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)\norg.spark_project.jetty.util.BlockingArrayQueue.poll(BlockingArrayQueue.java:392)\norg.spark_project.jetty.util.thread.QueuedThreadPool.idleJobPoll(QueuedThreadPool.java:563)\norg.spark_project.jetty.util.thread.QueuedThreadPool.access$800(QueuedThreadPool.java:48)\norg.spark_project.jetty.util.thread.QueuedThreadPool$2.run(QueuedThreadPool.java:626)\njava.lang.Thread.run(Thread.java:748)",
  "blockedByLock" : "Lock(java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject1503479572})",
  "holdingLocks" : [ ]
}, {
  "threadId" : 38,
  "threadName" : "SparkUI-38",
  "threadState" : "RUNNABLE",
  "stackTrace" : "sun.nio.ch.KQueueArrayWrapper.kevent0(Native Method)\nsun.nio.ch.KQueueArrayWrapper.poll(KQueueArrayWrapper.java:198)\nsun.nio.ch.KQueueSelectorImpl.doSelect(KQueueSelectorImpl.java:117)\nsun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86) => holding Monitor(sun.nio.ch.KQueueSelectorImpl841741934})\nsun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)\nsun.nio.ch.SelectorImpl.select(SelectorImpl.java:101)\norg.spark_project.jetty.io.ManagedSelector$SelectorProducer.select(ManagedSelector.java:243)\norg.spark_project.jetty.io.ManagedSelector$SelectorProducer.produce(ManagedSelector.java:191)\norg.spark_project.jetty.util.thread.strategy.ExecuteProduceConsume.executeProduceConsume(ExecuteProduceConsume.java:249)\norg.spark_project.jetty.util.thread.strategy.ExecuteProduceConsume.produceConsume(ExecuteProduceConsume.java:148)\norg.spark_project.jetty.util.thread.strategy.ExecuteProduceConsume.run(ExecuteProduceConsume.java:136)\norg.spark_project.jetty.util.thread.QueuedThreadPool.runJob(QueuedThreadPool.java:671)\norg.spark_project.jetty.util.thread.QueuedThreadPool$2.run(QueuedThreadPool.java:589)\njava.lang.Thread.run(Thread.java:748)",
  "blockedByLock" : "",
  "holdingLocks" : [ "Monitor(sun.nio.ch.Util$3873523986})", "Monitor(java.util.Collections$UnmodifiableSet1769333189})", "Monitor(sun.nio.ch.KQueueSelectorImpl841741934})" ]
}, {
  "threadId" : 40,
  "threadName" : "SparkUI-40-acceptor-034929380-Spark3a557b62{HTTP/1.1,[http/1.1]}{0.0.0.0:4040}",
  "threadState" : "RUNNABLE",
  "stackTrace" : "sun.nio.ch.ServerSocketChannelImpl.accept0(Native Method)\nsun.nio.ch.ServerSocketChannelImpl.accept(ServerSocketChannelImpl.java:422)\nsun.nio.ch.ServerSocketChannelImpl.accept(ServerSocketChannelImpl.java:250) => holding Monitor(java.lang.Object1134240909})\norg.spark_project.jetty.server.ServerConnector.accept(ServerConnector.java:371)\norg.spark_project.jetty.server.AbstractConnector$Acceptor.run(AbstractConnector.java:601)\norg.spark_project.jetty.util.thread.QueuedThreadPool.runJob(QueuedThreadPool.java:671)\norg.spark_project.jetty.util.thread.QueuedThreadPool$2.run(QueuedThreadPool.java:589)\njava.lang.Thread.run(Thread.java:748)",
  "blockedByLock" : "",
  "holdingLocks" : [ "Monitor(java.lang.Object1134240909})" ]
}, {
  "threadId" : 43,
  "threadName" : "SparkUI-43",
  "threadState" : "RUNNABLE",
  "stackTrace" : "sun.management.ThreadImpl.dumpThreads0(Native Method)\nsun.management.ThreadImpl.dumpAllThreads(ThreadImpl.java:454)\norg.apache.spark.util.Utils$.getThreadDump(Utils.scala:2170)\norg.apache.spark.SparkContext.getExecutorThreadDump(SparkContext.scala:596)\norg.apache.spark.status.api.v1.AbstractApplicationResource$$anonfun$threadDump$1$$anonfun$apply$1.apply(OneApplicationResource.scala:66)\norg.apache.spark.status.api.v1.AbstractApplicationResource$$anonfun$threadDump$1$$anonfun$apply$1.apply(OneApplicationResource.scala:65)\nscala.Option.flatMap(Option.scala:171)\norg.apache.spark.status.api.v1.AbstractApplicationResource$$anonfun$threadDump$1.apply(OneApplicationResource.scala:65)\norg.apache.spark.status.api.v1.AbstractApplicationResource$$anonfun$threadDump$1.apply(OneApplicationResource.scala:58)\norg.apache.spark.status.api.v1.BaseAppResource$$anonfun$withUI$1.apply(ApiRootResource.scala:139)\norg.apache.spark.status.api.v1.BaseAppResource$$anonfun$withUI$1.apply(ApiRootResource.scala:134)\norg.apache.spark.ui.SparkUI.withSparkUI(SparkUI.scala:106)\norg.apache.spark.status.api.v1.BaseAppResource$class.withUI(ApiRootResource.scala:134)\norg.apache.spark.status.api.v1.AbstractApplicationResource.withUI(OneApplicationResource.scala:32)\norg.apache.spark.status.api.v1.AbstractApplicationResource.threadDump(OneApplicationResource.scala:58)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.glassfish.jersey.server.model.internal.ResourceMethodInvocationHandlerFactory$1.invoke(ResourceMethodInvocationHandlerFactory.java:81)\norg.glassfish.jersey.server.model.internal.AbstractJavaResourceMethodDispatcher$1.run(AbstractJavaResourceMethodDispatcher.java:144)\norg.glassfish.jersey.server.model.internal.AbstractJavaResourceMethodDispatcher.invoke(AbstractJavaResourceMethodDispatcher.java:161)\norg.glassfish.jersey.server.model.internal.JavaResourceMethodDispatcherProvider$TypeOutInvoker.doDispatch(JavaResourceMethodDispatcherProvider.java:205)\norg.glassfish.jersey.server.model.internal.AbstractJavaResourceMethodDispatcher.dispatch(AbstractJavaResourceMethodDispatcher.java:99)\norg.glassfish.jersey.server.model.ResourceMethodInvoker.invoke(ResourceMethodInvoker.java:389)\norg.glassfish.jersey.server.model.ResourceMethodInvoker.apply(ResourceMethodInvoker.java:347)\norg.glassfish.jersey.server.model.ResourceMethodInvoker.apply(ResourceMethodInvoker.java:102)\norg.glassfish.jersey.server.ServerRuntime$2.run(ServerRuntime.java:326)\norg.glassfish.jersey.internal.Errors$1.call(Errors.java:271)\norg.glassfish.jersey.internal.Errors$1.call(Errors.java:267)\norg.glassfish.jersey.internal.Errors.process(Errors.java:315)\norg.glassfish.jersey.internal.Errors.process(Errors.java:297)\norg.glassfish.jersey.internal.Errors.process(Errors.java:267)\norg.glassfish.jersey.process.internal.RequestScope.runInScope(RequestScope.java:317)\norg.glassfish.jersey.server.ServerRuntime.process(ServerRuntime.java:305)\norg.glassfish.jersey.server.ApplicationHandler.handle(ApplicationHandler.java:1154)\norg.glassfish.jersey.servlet.WebComponent.serviceImpl(WebComponent.java:473)\norg.glassfish.jersey.servlet.WebComponent.service(WebComponent.java:427)\norg.glassfish.jersey.servlet.ServletContainer.service(ServletContainer.java:388)\norg.glassfish.jersey.servlet.ServletContainer.service(ServletContainer.java:341)\norg.glassfish.jersey.servlet.ServletContainer.service(ServletContainer.java:228)\norg.spark_project.jetty.servlet.ServletHolder.handle(ServletHolder.java:848)\norg.spark_project.jetty.servlet.ServletHandler.doHandle(ServletHandler.java:584)\norg.spark_project.jetty.server.handler.ContextHandler.doHandle(ContextHandler.java:1180)\norg.spark_project.jetty.servlet.ServletHandler.doScope(ServletHandler.java:512)\norg.spark_project.jetty.server.handler.ContextHandler.doScope(ContextHandler.java:1112)\norg.spark_project.jetty.server.handler.ScopedHandler.handle(ScopedHandler.java:141)\norg.spark_project.jetty.server.handler.gzip.GzipHandler.handle(GzipHandler.java:493)\norg.spark_project.jetty.server.handler.ContextHandlerCollection.handle(ContextHandlerCollection.java:213)\norg.spark_project.jetty.server.handler.HandlerWrapper.handle(HandlerWrapper.java:134)\norg.spark_project.jetty.server.Server.handle(Server.java:534)\norg.spark_project.jetty.server.HttpChannel.handle(HttpChannel.java:320)\norg.spark_project.jetty.server.HttpConnection.onFillable(HttpConnection.java:251)\norg.spark_project.jetty.io.AbstractConnection$ReadCallback.succeeded(AbstractConnection.java:283)\norg.spark_project.jetty.io.FillInterest.fillable(FillInterest.java:108)\norg.spark_project.jetty.io.SelectChannelEndPoint$2.run(SelectChannelEndPoint.java:93)\norg.spark_project.jetty.util.thread.strategy.ExecuteProduceConsume.executeProduceConsume(ExecuteProduceConsume.java:303)\norg.spark_project.jetty.util.thread.strategy.ExecuteProduceConsume.produceConsume(ExecuteProduceConsume.java:148)\norg.spark_project.jetty.util.thread.strategy.ExecuteProduceConsume.run(ExecuteProduceConsume.java:136)\norg.spark_project.jetty.util.thread.QueuedThreadPool.runJob(QueuedThreadPool.java:671)\norg.spark_project.jetty.util.thread.QueuedThreadPool$2.run(QueuedThreadPool.java:589)\njava.lang.Thread.run(Thread.java:748)",
  "blockedByLock" : "",
  "holdingLocks" : [ ]
}, {
  "threadId" : 67,
  "threadName" : "SparkUI-67",
  "threadState" : "RUNNABLE",
  "stackTrace" : "sun.nio.ch.KQueueArrayWrapper.kevent0(Native Method)\nsun.nio.ch.KQueueArrayWrapper.poll(KQueueArrayWrapper.java:198)\nsun.nio.ch.KQueueSelectorImpl.doSelect(KQueueSelectorImpl.java:117)\nsun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86) => holding Monitor(sun.nio.ch.KQueueSelectorImpl1837806480})\nsun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)\nsun.nio.ch.SelectorImpl.select(SelectorImpl.java:101)\norg.spark_project.jetty.io.ManagedSelector$SelectorProducer.select(ManagedSelector.java:243)\norg.spark_project.jetty.io.ManagedSelector$SelectorProducer.produce(ManagedSelector.java:191)\norg.spark_project.jetty.util.thread.strategy.ExecuteProduceConsume.executeProduceConsume(ExecuteProduceConsume.java:249)\norg.spark_project.jetty.util.thread.strategy.ExecuteProduceConsume.produceConsume(ExecuteProduceConsume.java:148)\norg.spark_project.jetty.util.thread.strategy.ExecuteProduceConsume.run(ExecuteProduceConsume.java:136)\norg.spark_project.jetty.util.thread.QueuedThreadPool.runJob(QueuedThreadPool.java:671)\norg.spark_project.jetty.util.thread.QueuedThreadPool$2.run(QueuedThreadPool.java:589)\njava.lang.Thread.run(Thread.java:748)",
  "blockedByLock" : "",
  "holdingLocks" : [ "Monitor(sun.nio.ch.Util$3881415814})", "Monitor(java.util.Collections$UnmodifiableSet62050480})", "Monitor(sun.nio.ch.KQueueSelectorImpl1837806480})" ]
}, {
  "threadId" : 68,
  "threadName" : "SparkUI-68",
  "threadState" : "RUNNABLE",
  "stackTrace" : "sun.nio.ch.KQueueArrayWrapper.kevent0(Native Method)\nsun.nio.ch.KQueueArrayWrapper.poll(KQueueArrayWrapper.java:198)\nsun.nio.ch.KQueueSelectorImpl.doSelect(KQueueSelectorImpl.java:117)\nsun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86) => holding Monitor(sun.nio.ch.KQueueSelectorImpl223607814})\nsun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)\nsun.nio.ch.SelectorImpl.select(SelectorImpl.java:101)\norg.spark_project.jetty.io.ManagedSelector$SelectorProducer.select(ManagedSelector.java:243)\norg.spark_project.jetty.io.ManagedSelector$SelectorProducer.produce(ManagedSelector.java:191)\norg.spark_project.jetty.util.thread.strategy.ExecuteProduceConsume.executeProduceConsume(ExecuteProduceConsume.java:249)\norg.spark_project.jetty.util.thread.strategy.ExecuteProduceConsume.produceConsume(ExecuteProduceConsume.java:148)\norg.spark_project.jetty.util.thread.strategy.ExecuteProduceConsume.run(ExecuteProduceConsume.java:136)\norg.spark_project.jetty.util.thread.QueuedThreadPool.runJob(QueuedThreadPool.java:671)\norg.spark_project.jetty.util.thread.QueuedThreadPool$2.run(QueuedThreadPool.java:589)\njava.lang.Thread.run(Thread.java:748)",
  "blockedByLock" : "",
  "holdingLocks" : [ "Monitor(sun.nio.ch.Util$3543145185})", "Monitor(java.util.Collections$UnmodifiableSet897441546})", "Monitor(sun.nio.ch.KQueueSelectorImpl223607814})" ]
}, {
  "threadId" : 71,
  "threadName" : "SparkUI-71",
  "threadState" : "TIMED_WAITING",
  "stackTrace" : "sun.misc.Unsafe.park(Native Method)\njava.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)\njava.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)\norg.spark_project.jetty.util.BlockingArrayQueue.poll(BlockingArrayQueue.java:392)\norg.spark_project.jetty.util.thread.QueuedThreadPool.idleJobPoll(QueuedThreadPool.java:563)\norg.spark_project.jetty.util.thread.QueuedThreadPool.access$800(QueuedThreadPool.java:48)\norg.spark_project.jetty.util.thread.QueuedThreadPool$2.run(QueuedThreadPool.java:626)\njava.lang.Thread.run(Thread.java:748)",
  "blockedByLock" : "Lock(java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject1503479572})",
  "holdingLocks" : [ ]
}, {
  "threadId" : 77,
  "threadName" : "SparkUI-77",
  "threadState" : "TIMED_WAITING",
  "stackTrace" : "sun.misc.Unsafe.park(Native Method)\njava.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)\njava.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)\norg.spark_project.jetty.util.BlockingArrayQueue.poll(BlockingArrayQueue.java:392)\norg.spark_project.jetty.util.thread.QueuedThreadPool.idleJobPoll(QueuedThreadPool.java:563)\norg.spark_project.jetty.util.thread.QueuedThreadPool.access$800(QueuedThreadPool.java:48)\norg.spark_project.jetty.util.thread.QueuedThreadPool$2.run(QueuedThreadPool.java:626)\njava.lang.Thread.run(Thread.java:748)",
  "blockedByLock" : "Lock(java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject1503479572})",
  "holdingLocks" : [ ]
}, {
  "threadId" : 78,
  "threadName" : "SparkUI-78",
  "threadState" : "RUNNABLE",
  "stackTrace" : "sun.nio.ch.KQueueArrayWrapper.kevent0(Native Method)\nsun.nio.ch.KQueueArrayWrapper.poll(KQueueArrayWrapper.java:198)\nsun.nio.ch.KQueueSelectorImpl.doSelect(KQueueSelectorImpl.java:117)\nsun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86) => holding Monitor(sun.nio.ch.KQueueSelectorImpl403077801})\nsun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)\nsun.nio.ch.SelectorImpl.select(SelectorImpl.java:101)\norg.spark_project.jetty.io.ManagedSelector$SelectorProducer.select(ManagedSelector.java:243)\norg.spark_project.jetty.io.ManagedSelector$SelectorProducer.produce(ManagedSelector.java:191)\norg.spark_project.jetty.util.thread.strategy.ExecuteProduceConsume.executeProduceConsume(ExecuteProduceConsume.java:249)\norg.spark_project.jetty.util.thread.strategy.ExecuteProduceConsume.produceConsume(ExecuteProduceConsume.java:148)\norg.spark_project.jetty.util.thread.strategy.ExecuteProduceConsume.run(ExecuteProduceConsume.java:136)\norg.spark_project.jetty.util.thread.QueuedThreadPool.runJob(QueuedThreadPool.java:671)\norg.spark_project.jetty.util.thread.QueuedThreadPool$2.run(QueuedThreadPool.java:589)\njava.lang.Thread.run(Thread.java:748)",
  "blockedByLock" : "",
  "holdingLocks" : [ "Monitor(sun.nio.ch.Util$3261312406})", "Monitor(java.util.Collections$UnmodifiableSet852901260})", "Monitor(sun.nio.ch.KQueueSelectorImpl403077801})" ]
}, {
  "threadId" : 72,
  "threadName" : "SparkUI-JettyScheduler",
  "threadState" : "TIMED_WAITING",
  "stackTrace" : "sun.misc.Unsafe.park(Native Method)\njava.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)\njava.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.awaitNanos(AbstractQueuedSynchronizer.java:2078)\njava.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:1093)\njava.util.concurrent.ScheduledThreadPoolExecutor$DelayedWorkQueue.take(ScheduledThreadPoolExecutor.java:809)\njava.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1074)\njava.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)\njava.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\njava.lang.Thread.run(Thread.java:748)",
  "blockedByLock" : "Lock(java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject1587346642})",
  "holdingLocks" : [ ]
}, {
  "threadId" : 63,
  "threadName" : "task-result-getter-0",
  "threadState" : "WAITING",
  "stackTrace" : "sun.misc.Unsafe.park(Native Method)\njava.util.concurrent.locks.LockSupport.park(LockSupport.java:175)\njava.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)\njava.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)\njava.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1074)\njava.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)\njava.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\njava.lang.Thread.run(Thread.java:748)",
  "blockedByLock" : "Lock(java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject537563105})",
  "holdingLocks" : [ ]
}, {
  "threadId" : 64,
  "threadName" : "task-result-getter-1",
  "threadState" : "WAITING",
  "stackTrace" : "sun.misc.Unsafe.park(Native Method)\njava.util.concurrent.locks.LockSupport.park(LockSupport.java:175)\njava.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)\njava.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)\njava.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1074)\njava.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)\njava.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\njava.lang.Thread.run(Thread.java:748)",
  "blockedByLock" : "Lock(java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject537563105})",
  "holdingLocks" : [ ]
}, {
  "threadId" : 65,
  "threadName" : "task-result-getter-2",
  "threadState" : "WAITING",
  "stackTrace" : "sun.misc.Unsafe.park(Native Method)\njava.util.concurrent.locks.LockSupport.park(LockSupport.java:175)\njava.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)\njava.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)\njava.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1074)\njava.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)\njava.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\njava.lang.Thread.run(Thread.java:748)",
  "blockedByLock" : "Lock(java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject537563105})",
  "holdingLocks" : [ ]
}, {
  "threadId" : 66,
  "threadName" : "task-result-getter-3",
  "threadState" : "WAITING",
  "stackTrace" : "sun.misc.Unsafe.park(Native Method)\njava.util.concurrent.locks.LockSupport.park(LockSupport.java:175)\njava.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)\njava.util.concurrent.LinkedBlockingQueue.take(LinkedBlockingQueue.java:442)\njava.util.concurrent.ThreadPoolExecutor.getTask(ThreadPoolExecutor.java:1074)\njava.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1134)\njava.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\njava.lang.Thread.run(Thread.java:748)",
  "blockedByLock" : "Lock(java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject537563105})",
  "holdingLocks" : [ ]
}, {
  "threadId" : 46,
  "threadName" : "Timer-0",
  "threadState" : "WAITING",
  "stackTrace" : "java.lang.Object.wait(Native Method)\njava.lang.Object.wait(Object.java:502)\njava.util.TimerThread.mainLoop(Timer.java:526)\njava.util.TimerThread.run(Timer.java:505)",
  "blockedByLock" : "Lock(java.util.TaskQueue635634547})",
  "holdingLocks" : [ ]
} ]
```
</details>

## How was this patch tested?

It was tested manually.

Old executor page with thread dumps:

<img width="1632" alt="screen shot 2018-02-01 at 14 31 19" src="https://user-images.githubusercontent.com/2017933/35682124-e2ec5d96-075f-11e8-9713-a502e12d05c2.png">

New api:

<img width="1669" alt="screen shot 2018-02-01 at 14 31 56" src="https://user-images.githubusercontent.com/2017933/35682149-f75b80d6-075f-11e8-95b0-c75d048f0b04.png">

Testing error cases.

Initial state:

![screen shot 2018-02-06 at 13 05 05](https://user-images.githubusercontent.com/2017933/35858990-ad2982be-0b3e-11e8-879b-656112065c7f.png)

Dead executor:

```bash
$ curl -o - -s -w "\n%{http_code}\n"   http://localhost:4040/api/v1/applications/app-20180206122543-0000/executors/1/threads

Executor is not active.
400
```

Never existed (but well formatted: number) executor ID:

```bash
$ curl -o - -s -w "\n%{http_code}\n"   http://localhost:4040/api/v1/applications/app-20180206122543-0000/executors/42/threads

Executor does not exist.
404
```

Not available stacktrace (dead executor but UI has not registered as dead yet):
```bash
$ kill -9 <PID of CoarseGrainedExecutorBackend for executor 2> ;  curl -o - -s -w "\n%{http_code}\n"   http://localhost:4040/api/v1/applications/app-20180206122543-0000/executors/2/threads

No thread dump is available.
404
```

Invalid executor ID format:

```bash
$ curl -o - -s -w "\n%{http_code}\n"   http://localhost:4040/api/v1/applications/app-20180206122543-0000/executors/something6/threads

Invalid executorId: neither 'driver' nor number.
400
```

Author: “attilapiros” <piros.attila.zsolt@gmail.com>

Closes #20474 from attilapiros/SPARK-23235.

9 days ago[SPARK-23400][SQL] Add a constructors for ScalaUDF
gatorsmile [Tue, 13 Feb 2018 19:56:49 +0000 (11:56 -0800)] 
[SPARK-23400][SQL] Add a constructors for ScalaUDF

## What changes were proposed in this pull request?

In this upcoming 2.3 release, we changed the interface of `ScalaUDF`. Unfortunately, some Spark packages (e.g., spark-deep-learning) are using our internal class `ScalaUDF`. In the release 2.3, we added new parameters into this class. The users hit the binary compatibility issues and got the exception:

```
> java.lang.NoSuchMethodError: org.apache.spark.sql.catalyst.expressions.ScalaUDF.&lt;init&gt;(Ljava/lang/Object;Lorg/apache/spark/sql/types/DataType;Lscala/collection/Seq;Lscala/collection/Seq;Lscala/Option;)V
```

This PR is to improve the backward compatibility. However, we definitely should not encourage the external packages to use our internal classes. This might make us hard to maintain/develop the codes in Spark.

## How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20591 from gatorsmile/scalaUDF.

9 days ago[SPARK-23154][ML][DOC] Document backwards compatibility guarantees for ML persistence
Joseph K. Bradley [Tue, 13 Feb 2018 19:18:45 +0000 (11:18 -0800)] 
[SPARK-23154][ML][DOC] Document backwards compatibility guarantees for ML persistence

## What changes were proposed in this pull request?

Added documentation about what MLlib guarantees in terms of loading ML models and Pipelines from old Spark versions.  Discussed & confirmed on linked JIRA.

Author: Joseph K. Bradley <joseph@databricks.com>

Closes #20592 from jkbradley/SPARK-23154-backwards-compat-doc.

9 days ago[SPARK-23217][ML] Add cosine distance measure to ClusteringEvaluator
Marco Gaido [Tue, 13 Feb 2018 17:51:19 +0000 (11:51 -0600)] 
[SPARK-23217][ML] Add cosine distance measure to ClusteringEvaluator

## What changes were proposed in this pull request?

The PR provided an implementation of ClusteringEvaluator using the cosine distance measure.
This allows to evaluate clustering results created using the cosine distance, introduced in SPARK-22119.

In the corresponding JIRA, there is a design document for the algorithm implemented here.

## How was this patch tested?

Added UT which compares the result to the one provided by python sklearn.

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #20396 from mgaido91/SPARK-23217.

9 days ago[SPARK-23316][SQL] AnalysisException after max iteration reached for IN query
Bogdan Raducanu [Tue, 13 Feb 2018 17:49:52 +0000 (09:49 -0800)] 
[SPARK-23316][SQL] AnalysisException after max iteration reached for IN query

## What changes were proposed in this pull request?
Added flag ignoreNullability to DataType.equalsStructurally.
The previous semantic is for ignoreNullability=false.
When ignoreNullability=true equalsStructurally ignores nullability of contained types (map key types, value types, array element types, structure field types).
In.checkInputTypes calls equalsStructurally to check if the children types match. They should match regardless of nullability (which is just a hint), so it is now called with ignoreNullability=true.

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

Author: Bogdan Raducanu <bogdan@databricks.com>

Closes #20548 from bogdanrdc/SPARK-23316.

9 days ago[SPARK-23392][TEST] Add some test cases for images feature
xubo245 [Tue, 13 Feb 2018 17:45:20 +0000 (11:45 -0600)] 
[SPARK-23392][TEST] Add some test cases for images feature

## What changes were proposed in this pull request?

Add some test cases for images feature

## How was this patch tested?
Add some test cases in ImageSchemaSuite

Author: xubo245 <601450868@qq.com>

Closes #20583 from xubo245/CARBONDATA23392_AddTestForImage.

9 days ago[SPARK-23382][WEB-UI] Spark Streaming ui about the contents of the for need to have...
guoxiaolong [Tue, 13 Feb 2018 17:39:33 +0000 (11:39 -0600)] 
[SPARK-23382][WEB-UI] Spark Streaming ui about the contents of the for need to have hidden and show features, when the table records very much.

## What changes were proposed in this pull request?
Spark Streaming ui about the contents of the for need to have hidden and show features, when the table records very much.
please refer to https://github.com/apache/spark/pull/20216

fix after:
![1](https://user-images.githubusercontent.com/26266482/36068644-df029328-0f14-11e8-8350-cfdde9733ffc.png)

## How was this patch tested?

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

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

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

Closes #20570 from guoxiaolongzte/SPARK-23382.

9 days ago[SPARK-23053][CORE] taskBinarySerialization and task partitions calculate in DagSched...
huangtengfei [Tue, 13 Feb 2018 15:59:21 +0000 (09:59 -0600)] 
[SPARK-23053][CORE] taskBinarySerialization and task partitions calculate in DagScheduler.submitMissingTasks should keep the same RDD checkpoint status

## What changes were proposed in this pull request?

When we run concurrent jobs using the same rdd which is marked to do checkpoint. If one job has finished running the job, and start the process of RDD.doCheckpoint, while another job is submitted, then submitStage and submitMissingTasks will be called. In [submitMissingTasks](https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala#L961), will serialize taskBinaryBytes and calculate task partitions which are both affected by the status of checkpoint, if the former is calculated before doCheckpoint finished, while the latter is calculated after doCheckpoint finished, when run task, rdd.compute will be called, for some rdds with particular partition type such as [UnionRDD](https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala) who will do partition type cast, will get a ClassCastException because the part params is actually a CheckpointRDDPartition.
This error occurs  because rdd.doCheckpoint occurs in the same thread that called sc.runJob, while the task serialization occurs in the DAGSchedulers event loop.

## How was this patch tested?

the exist uts and also add a test case in DAGScheduerSuite to show the exception case.

Author: huangtengfei <huangtengfei@huangtengfeideMacBook-Pro.local>

Closes #20244 from ivoson/branch-taskpart-mistype.

9 days ago[SPARK-23189][CORE][WEB UI] Reflect stage level blacklisting on executor tab
“attilapiros” [Tue, 13 Feb 2018 15:54:52 +0000 (09:54 -0600)] 
[SPARK-23189][CORE][WEB UI] Reflect stage level blacklisting on executor tab

## What changes were proposed in this pull request?

The purpose of this PR to reflect the stage level blacklisting on the executor tab for the currently active stages.

After this change in the executor tab at the Status column one of the following label will be:

- "Blacklisted" when the executor is blacklisted application level (old flag)
- "Dead" when the executor is not Blacklisted and not Active
- "Blacklisted in Stages: [...]" when the executor is Active but the there are active blacklisted stages for the executor. Within the [] coma separated active stageIDs are listed.
- "Active" when the executor is Active and there is no active blacklisted stages for the executor

## How was this patch tested?

Both with unit tests and manually.

#### Manual test

Spark was started as:

```bash
 bin/spark-shell --master "local-cluster[2,1,1024]" --conf "spark.blacklist.enabled=true" --conf "spark.blacklist.stage.maxFailedTasksPerExecutor=1" --conf "spark.blacklist.application.maxFailedTasksPerExecutor=10"
```

And the job was:
```scala
import org.apache.spark.SparkEnv

val pairs = sc.parallelize(1 to 10000, 10).map { x =>
  if (SparkEnv.get.executorId.toInt == 0) throw new RuntimeException("Bad executor")
  else  {
    Thread.sleep(10)
    (x % 10, x)
  }
}

val all = pairs.cogroup(pairs)

all.collect()
```

UI screenshots about the running:

- One executor is blacklisted in the two stages:

![One executor is blacklisted in two stages](https://issues.apache.org/jira/secure/attachment/12908314/multiple_stages_1.png)

- One stage completes the other one is still running:

![One stage completes the other is still running](https://issues.apache.org/jira/secure/attachment/12908315/multiple_stages_2.png)

- Both stages are completed:

![Both stages are completed](https://issues.apache.org/jira/secure/attachment/12908316/multiple_stages_3.png)

### Unit tests

In AppStatusListenerSuite.scala both the node blacklisting for a stage and the executor blacklisting for stage are tested.

Author: “attilapiros” <piros.attila.zsolt@gmail.com>

Closes #20408 from attilapiros/SPARK-23189.

9 days ago[SPARK-20659][CORE] Removing sc.getExecutorStorageStatus and making StorageStatus...
“attilapiros” [Tue, 13 Feb 2018 14:54:15 +0000 (06:54 -0800)] 
[SPARK-20659][CORE] Removing sc.getExecutorStorageStatus and making StorageStatus private

## What changes were proposed in this pull request?

In this PR StorageStatus is made to private and simplified a bit moreover SparkContext.getExecutorStorageStatus method is removed. The reason of keeping StorageStatus is that it is usage from SparkContext.getRDDStorageInfo.

Instead of the method SparkContext.getExecutorStorageStatus executor infos are extended with additional memory metrics such as usedOnHeapStorageMemory, usedOffHeapStorageMemory, totalOnHeapStorageMemory, totalOffHeapStorageMemory.

## How was this patch tested?

By running existing unit tests.

Author: “attilapiros” <piros.attila.zsolt@gmail.com>
Author: Attila Zsolt Piros <2017933+attilapiros@users.noreply.github.com>

Closes #20546 from attilapiros/SPARK-20659.

9 days ago[SPARK-23384][WEB-UI] When it has no incomplete(completed) applications found, the...
guoxiaolong [Tue, 13 Feb 2018 12:23:10 +0000 (06:23 -0600)] 
[SPARK-23384][WEB-UI] When it has no incomplete(completed) applications found, the last updated time is not formatted and client local time zone is not show in history server web ui.

## What changes were proposed in this pull request?

When it has no incomplete(completed) applications found, the last updated time is not formatted and client local time zone is not show in history server web ui. It is a bug.

fix before:
![1](https://user-images.githubusercontent.com/26266482/36070635-264d7cf0-0f3a-11e8-8426-14135ffedb16.png)

fix after:
![2](https://user-images.githubusercontent.com/26266482/36070651-8ec3800e-0f3a-11e8-991c-6122cc9539fe.png)

## How was this patch tested?

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

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

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

Closes #20573 from guoxiaolongzte/SPARK-23384.

9 days ago[SPARK-23318][ML] FP-growth: WARN FPGrowth: Input data is not cached
Arseniy Tashoyan [Tue, 13 Feb 2018 12:20:34 +0000 (06:20 -0600)] 
[SPARK-23318][ML] FP-growth: WARN FPGrowth: Input data is not cached

## What changes were proposed in this pull request?

Cache the RDD of items in ml.FPGrowth before passing it to mllib.FPGrowth. Cache only when the user did not cache the input dataset of transactions. This fixes the warning about uncached data emerging from mllib.FPGrowth.

## How was this patch tested?

Manually:
1. Run ml.FPGrowthExample - warning is there
2. Apply the fix
3. Run ml.FPGrowthExample again - no warning anymore

Author: Arseniy Tashoyan <tashoyan@gmail.com>

Closes #20578 from tashoyan/SPARK-23318.

9 days ago[SPARK-20090][FOLLOW-UP] Revert the deprecation of `names` in PySpark
gatorsmile [Tue, 13 Feb 2018 06:05:13 +0000 (15:05 +0900)] 
[SPARK-20090][FOLLOW-UP] Revert the deprecation of `names` in PySpark

## What changes were proposed in this pull request?
Deprecating the field `name` in PySpark is not expected. This PR is to revert the change.

## How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20595 from gatorsmile/removeDeprecate.

9 days ago[SPARK-23303][SQL] improve the explain result for data source v2 relations
Wenchen Fan [Tue, 13 Feb 2018 05:12:22 +0000 (21:12 -0800)] 
[SPARK-23303][SQL] improve the explain result for data source v2 relations

## What changes were proposed in this pull request?

The current explain result for data source v2 relation is unreadable:
```
== Parsed Logical Plan ==
'Filter ('i > 6)
+- AnalysisBarrier
      +- Project [j#1]
         +- DataSourceV2Relation [i#0, j#1], org.apache.spark.sql.sources.v2.AdvancedDataSourceV2$Reader3b415940

== Analyzed Logical Plan ==
j: int
Project [j#1]
+- Filter (i#0 > 6)
   +- Project [j#1, i#0]
      +- DataSourceV2Relation [i#0, j#1], org.apache.spark.sql.sources.v2.AdvancedDataSourceV2$Reader3b415940

== Optimized Logical Plan ==
Project [j#1]
+- Filter isnotnull(i#0)
   +- DataSourceV2Relation [i#0, j#1], org.apache.spark.sql.sources.v2.AdvancedDataSourceV2$Reader3b415940

== Physical Plan ==
*(1) Project [j#1]
+- *(1) Filter isnotnull(i#0)
   +- *(1) DataSourceV2Scan [i#0, j#1], org.apache.spark.sql.sources.v2.AdvancedDataSourceV2$Reader3b415940
```

after this PR
```
== Parsed Logical Plan ==
'Project [unresolvedalias('j, None)]
+- AnalysisBarrier
      +- Relation AdvancedDataSourceV2[i#0, j#1]

== Analyzed Logical Plan ==
j: int
Project [j#1]
+- Relation AdvancedDataSourceV2[i#0, j#1]

== Optimized Logical Plan ==
Relation AdvancedDataSourceV2[j#1]

== Physical Plan ==
*(1) Scan AdvancedDataSourceV2[j#1]
```
-------
```
== Analyzed Logical Plan ==
i: int, j: int
Filter (i#88 > 3)
+- Relation JavaAdvancedDataSourceV2[i#88, j#89]

== Optimized Logical Plan ==
Filter isnotnull(i#88)
+- Relation JavaAdvancedDataSourceV2[i#88, j#89] (PushedFilter: [GreaterThan(i,3)])

== Physical Plan ==
*(1) Filter isnotnull(i#88)
+- *(1) Scan JavaAdvancedDataSourceV2[i#88, j#89] (PushedFilter: [GreaterThan(i,3)])
```

an example for streaming query
```
== Parsed Logical Plan ==
Aggregate [value#6], [value#6, count(1) AS count(1)#11L]
+- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#6]
   +- MapElements <function1>, class java.lang.String, [StructField(value,StringType,true)], obj#5: java.lang.String
      +- DeserializeToObject cast(value#25 as string).toString, obj#4: java.lang.String
         +- Streaming Relation FakeDataSourceV2$[value#25]

== Analyzed Logical Plan ==
value: string, count(1): bigint
Aggregate [value#6], [value#6, count(1) AS count(1)#11L]
+- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#6]
   +- MapElements <function1>, class java.lang.String, [StructField(value,StringType,true)], obj#5: java.lang.String
      +- DeserializeToObject cast(value#25 as string).toString, obj#4: java.lang.String
         +- Streaming Relation FakeDataSourceV2$[value#25]

== Optimized Logical Plan ==
Aggregate [value#6], [value#6, count(1) AS count(1)#11L]
+- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#6]
   +- MapElements <function1>, class java.lang.String, [StructField(value,StringType,true)], obj#5: java.lang.String
      +- DeserializeToObject value#25.toString, obj#4: java.lang.String
         +- Streaming Relation FakeDataSourceV2$[value#25]

== Physical Plan ==
*(4) HashAggregate(keys=[value#6], functions=[count(1)], output=[value#6, count(1)#11L])
+- StateStoreSave [value#6], state info [ checkpoint = *********(redacted)/cloud/dev/spark/target/tmp/temporary-549f264b-2531-4fcb-a52f-433c77347c12/state, runId = f84d9da9-2f8c-45c1-9ea1-70791be684de, opId = 0, ver = 0, numPartitions = 5], Complete, 0
   +- *(3) HashAggregate(keys=[value#6], functions=[merge_count(1)], output=[value#6, count#16L])
      +- StateStoreRestore [value#6], state info [ checkpoint = *********(redacted)/cloud/dev/spark/target/tmp/temporary-549f264b-2531-4fcb-a52f-433c77347c12/state, runId = f84d9da9-2f8c-45c1-9ea1-70791be684de, opId = 0, ver = 0, numPartitions = 5]
         +- *(2) HashAggregate(keys=[value#6], functions=[merge_count(1)], output=[value#6, count#16L])
            +- Exchange hashpartitioning(value#6, 5)
               +- *(1) HashAggregate(keys=[value#6], functions=[partial_count(1)], output=[value#6, count#16L])
                  +- *(1) SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true, false) AS value#6]
                     +- *(1) MapElements <function1>, obj#5: java.lang.String
                        +- *(1) DeserializeToObject value#25.toString, obj#4: java.lang.String
                           +- *(1) Scan FakeDataSourceV2$[value#25]
```
## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20477 from cloud-fan/explain.

9 days ago[SPARK-23379][SQL] skip when setting the same current database in HiveClientImpl
Feng Liu [Tue, 13 Feb 2018 04:57:26 +0000 (20:57 -0800)] 
[SPARK-23379][SQL] skip when setting the same current database in HiveClientImpl

## What changes were proposed in this pull request?

If the target database name is as same as the current database, we should be able to skip one metastore access.

## How was this patch tested?

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

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

Author: Feng Liu <fengliu@databricks.com>

Closes #20565 from liufengdb/remove-redundant.

9 days ago[SPARK-23323][SQL] Support commit coordinator for DataSourceV2 writes
Ryan Blue [Tue, 13 Feb 2018 03:40:34 +0000 (11:40 +0800)] 
[SPARK-23323][SQL] Support commit coordinator for DataSourceV2 writes

## What changes were proposed in this pull request?

DataSourceV2 batch writes should use the output commit coordinator if it is required by the data source. This adds a new method, `DataWriterFactory#useCommitCoordinator`, that determines whether the coordinator will be used. If the write factory returns true, `WriteToDataSourceV2` will use the coordinator for batch writes.

## How was this patch tested?

This relies on existing write tests, which now use the commit coordinator.

Author: Ryan Blue <blue@apache.org>

Closes #20490 from rdblue/SPARK-23323-add-commit-coordinator.

10 days ago[SPARK-23230][SQL] When hive.default.fileformat is other kinds of file types, create...
sychen [Tue, 13 Feb 2018 00:00:47 +0000 (16:00 -0800)] 
[SPARK-23230][SQL] When hive.default.fileformat is other kinds of file types, create textfile table cause a serde error

When hive.default.fileformat is other kinds of file types, create textfile table cause a serde error.
We should take the default type of textfile and sequencefile both as org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe.

```
set hive.default.fileformat=orc;
create table tbl( i string ) stored as textfile;
desc formatted tbl;

Serde Library org.apache.hadoop.hive.ql.io.orc.OrcSerde
InputFormat  org.apache.hadoop.mapred.TextInputFormat
OutputFormat  org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
```

Author: sychen <sychen@ctrip.com>

Closes #20406 from cxzl25/default_serde.

10 days ago[SPARK-23313][DOC] Add a migration guide for ORC
Dongjoon Hyun [Mon, 12 Feb 2018 23:26:37 +0000 (15:26 -0800)] 
[SPARK-23313][DOC] Add a migration guide for ORC

## What changes were proposed in this pull request?

This PR adds a migration guide documentation for ORC.

![orc-guide](https://user-images.githubusercontent.com/9700541/36123859-ec165cae-1002-11e8-90b7-7313be7a81a5.png)

## How was this patch tested?

N/A.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #20484 from dongjoon-hyun/SPARK-23313.

10 days ago[SPARK-23378][SQL] move setCurrentDatabase from HiveExternalCatalog to HiveClientImpl
Feng Liu [Mon, 12 Feb 2018 22:58:31 +0000 (14:58 -0800)] 
[SPARK-23378][SQL] move setCurrentDatabase from HiveExternalCatalog to HiveClientImpl

## What changes were proposed in this pull request?

This removes the special case that `alterPartitions` call from `HiveExternalCatalog` can reset the current database in the hive client as a side effect.

## How was this patch tested?

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

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

Author: Feng Liu <fengliu@databricks.com>

Closes #20564 from liufengdb/move.

10 days ago[SPARK-22002][SQL][FOLLOWUP][TEST] Add a test to check if the original schema doesn...
Takuya UESHIN [Mon, 12 Feb 2018 20:20:29 +0000 (12:20 -0800)] 
[SPARK-22002][SQL][FOLLOWUP][TEST] Add a test to check if the original schema doesn't have metadata.

## What changes were proposed in this pull request?

This is a follow-up pr of #19231 which modified the behavior to remove metadata from JDBC table schema.
This pr adds a test to check if the schema doesn't have metadata.

## How was this patch tested?

Added a test and existing tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #20585 from ueshin/issues/SPARK-22002/fup1.

10 days ago[SPARK-23388][SQL] Support for Parquet Binary DecimalType in VectorizedColumnReader
James Thompson [Mon, 12 Feb 2018 19:34:56 +0000 (11:34 -0800)] 
[SPARK-23388][SQL] Support for Parquet Binary DecimalType in VectorizedColumnReader

## What changes were proposed in this pull request?

Re-add support for parquet binary DecimalType in VectorizedColumnReader

## How was this patch tested?

Existing test suite

Author: James Thompson <jamesthomp@users.noreply.github.com>

Closes #20580 from jamesthomp/jt/add-back-binary-decimal.

10 days ago[SPARK-23391][CORE] It may lead to overflow for some integer multiplication
liuxian [Mon, 12 Feb 2018 14:49:45 +0000 (08:49 -0600)] 
[SPARK-23391][CORE] It may lead to overflow for some integer multiplication

## What changes were proposed in this pull request?
In the `getBlockData`,`blockId.reduceId` is the `Int` type, when it is greater than 2^28, `blockId.reduceId*8` will overflow
In the `decompress0`, `len` and  `unitSize` are  Int type, so `len * unitSize` may lead to  overflow
## How was this patch tested?
N/A

Author: liuxian <liu.xian3@zte.com.cn>

Closes #20581 from 10110346/overflow2.

10 days ago[SPARK-22977][SQL] fix web UI SQL tab for CTAS
Wenchen Fan [Mon, 12 Feb 2018 14:07:59 +0000 (22:07 +0800)] 
[SPARK-22977][SQL] fix web UI SQL tab for CTAS

## What changes were proposed in this pull request?

This is a regression in Spark 2.3.

In Spark 2.2, we have a fragile UI support for SQL data writing commands. We only track the input query plan of `FileFormatWriter` and display its metrics. This is not ideal because we don't know who triggered the writing(can be table insertion, CTAS, etc.), but it's still useful to see the metrics of the input query.

In Spark 2.3, we introduced a new mechanism: `DataWritigCommand`, to fix the UI issue entirely. Now these writing commands have real children, and we don't need to hack into the `FileFormatWriter` for the UI. This also helps with `explain`, now `explain` can show the physical plan of the input query, while in 2.2 the physical writing plan is simply `ExecutedCommandExec` and it has no child.

However there is a regression in CTAS. CTAS commands don't extend `DataWritigCommand`, and we don't have the UI hack in `FileFormatWriter` anymore, so the UI for CTAS is just an empty node. See https://issues.apache.org/jira/browse/SPARK-22977 for more information about this UI issue.

To fix it, we should apply the `DataWritigCommand` mechanism to CTAS commands.

TODO: In the future, we should refactor this part and create some physical layer code pieces for data writing, and reuse them in different writing commands. We should have different logical nodes for different operators, even some of them share some same logic, e.g. CTAS, CREATE TABLE, INSERT TABLE. Internally we can share the same physical logic.

## How was this patch tested?

manually tested.
For data source table
<img width="644" alt="1" src="https://user-images.githubusercontent.com/3182036/35874155-bdffab28-0ba6-11e8-94a8-e32e106ba069.png">
For hive table
<img width="666" alt="2" src="https://user-images.githubusercontent.com/3182036/35874161-c437e2a8-0ba6-11e8-98ed-7930f01432c5.png">

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20521 from cloud-fan/UI.

10 days ago[MINOR][TEST] spark.testing` No effect on the SparkFunSuite unit test
caoxuewen [Mon, 12 Feb 2018 14:05:27 +0000 (22:05 +0800)] 
[MINOR][TEST] spark.testing` No effect on the SparkFunSuite unit test

## What changes were proposed in this pull request?

Currently, we use SBT and MAVN to spark unit test, are affected by the parameters of `spark.testing`. However, when using the IDE test tool, `spark.testing` support is not very good, sometimes need to be manually added to the beforeEach. example: HiveSparkSubmitSuite RPackageUtilsSuite SparkSubmitSuite. The PR unified `spark.testing` parameter extraction to SparkFunSuite, support IDE test tool, and the test code is more compact.

## How was this patch tested?

the existed test cases.

Author: caoxuewen <cao.xuewen@zte.com.cn>

Closes #20582 from heary-cao/sparktesting.

10 days ago[SPARK-23352][PYTHON] Explicitly specify supported types in Pandas UDFs
hyukjinkwon [Mon, 12 Feb 2018 11:49:36 +0000 (20:49 +0900)] 
[SPARK-23352][PYTHON] Explicitly specify supported types in Pandas UDFs

## What changes were proposed in this pull request?

This PR targets to explicitly specify supported types in Pandas UDFs.
The main change here is to add a deduplicated and explicit type checking in `returnType` ahead with documenting this; however, it happened to fix multiple things.

1. Currently, we don't support `BinaryType` in Pandas UDFs, for example, see:

    ```python
    from pyspark.sql.functions import pandas_udf
    pudf = pandas_udf(lambda x: x, "binary")
    df = spark.createDataFrame([[bytearray(1)]])
    df.select(pudf("_1")).show()
    ```
    ```
    ...
    TypeError: Unsupported type in conversion to Arrow: BinaryType
    ```

    We can document this behaviour for its guide.

2. Also, the grouped aggregate Pandas UDF fails fast on `ArrayType` but seems we can support this case.

    ```python
    from pyspark.sql.functions import pandas_udf, PandasUDFType
    foo = pandas_udf(lambda v: v.mean(), 'array<double>', PandasUDFType.GROUPED_AGG)
    df = spark.range(100).selectExpr("id", "array(id) as value")
    df.groupBy("id").agg(foo("value")).show()
    ```

    ```
    ...
     NotImplementedError: ArrayType, StructType and MapType are not supported with PandasUDFType.GROUPED_AGG
    ```

3. Since we can check the return type ahead, we can fail fast before actual execution.

    ```python
    # we can fail fast at this stage because we know the schema ahead
    pandas_udf(lambda x: x, BinaryType())
    ```

## How was this patch tested?

Manually tested and unit tests for `BinaryType` and `ArrayType(...)` were added.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #20531 from HyukjinKwon/pudf-cleanup.

10 days ago[SPARK-23390][SQL] Flaky Test Suite: FileBasedDataSourceSuite in Spark 2.3/hadoop 2.7
Wenchen Fan [Mon, 12 Feb 2018 07:46:23 +0000 (23:46 -0800)] 
[SPARK-23390][SQL] Flaky Test Suite: FileBasedDataSourceSuite in Spark 2.3/hadoop 2.7

## What changes were proposed in this pull request?

This test only fails with sbt on Hadoop 2.7, I can't reproduce it locally, but here is my speculation by looking at the code:
1. FileSystem.delete doesn't delete the directory entirely, somehow we can still open the file as a 0-length empty file.(just speculation)
2. ORC intentionally allow empty files, and the reader fails during reading without closing the file stream.

This PR improves the test to make sure all files are deleted and can't be opened.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20584 from cloud-fan/flaky-test.

10 days ago[SPARK-22119][FOLLOWUP][ML] Use spherical KMeans with cosine distance
Marco Gaido [Mon, 12 Feb 2018 02:15:30 +0000 (20:15 -0600)] 
[SPARK-22119][FOLLOWUP][ML] Use spherical KMeans with cosine distance

## What changes were proposed in this pull request?

In #19340 some comments considered needed to use spherical KMeans when cosine distance measure is specified, as Matlab does; instead of the implementation based on the behavior of other tools/libraries like Rapidminer, nltk and ELKI, ie. the centroids are computed as the mean of all the points in the clusters.

The PR introduce the approach used in spherical KMeans. This behavior has the nice feature to minimize the within-cluster cosine distance.

## How was this patch tested?

existing/improved UTs

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #20518 from mgaido91/SPARK-22119_followup.

11 days ago[SPARK-23376][SQL] creating UnsafeKVExternalSorter with BytesToBytesMap may fail
Wenchen Fan [Sun, 11 Feb 2018 16:03:49 +0000 (00:03 +0800)] 
[SPARK-23376][SQL] creating UnsafeKVExternalSorter with BytesToBytesMap may fail

## What changes were proposed in this pull request?

This is a long-standing bug in `UnsafeKVExternalSorter` and was reported in the dev list multiple times.

When creating `UnsafeKVExternalSorter` with `BytesToBytesMap`, we need to create a `UnsafeInMemorySorter` to sort the data in `BytesToBytesMap`. The data format of the sorter and the map is same, so no data movement is required. However, both the sorter and the map need a point array for some bookkeeping work.

There is an optimization in `UnsafeKVExternalSorter`: reuse the point array between the sorter and the map, to avoid an extra memory allocation. This sounds like a reasonable optimization, the length of the `BytesToBytesMap` point array is at least 4 times larger than the number of keys(to avoid hash collision, the hash table size should be at least 2 times larger than the number of keys, and each key occupies 2 slots). `UnsafeInMemorySorter` needs the pointer array size to be 4 times of the number of entries, so we are safe to reuse the point array.

However, the number of keys of the map doesn't equal to the number of entries in the map, because `BytesToBytesMap` supports duplicated keys. This breaks the assumption of the above optimization and we may run out of space when inserting data into the sorter, and hit error
```
java.lang.IllegalStateException: There is no space for new record
   at org.apache.spark.util.collection.unsafe.sort.UnsafeInMemorySorter.insertRecord(UnsafeInMemorySorter.java:239)
   at org.apache.spark.sql.execution.UnsafeKVExternalSorter.<init>(UnsafeKVExternalSorter.java:149)
...
```

This PR fixes this bug by creating a new point array if the existing one is not big enough.

## How was this patch tested?

a new test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20561 from cloud-fan/bug.

11 days ago[SPARK-22624][PYSPARK] Expose range partitioning shuffle introduced by spark-22614
xubo245 [Sun, 11 Feb 2018 10:23:15 +0000 (19:23 +0900)] 
[SPARK-22624][PYSPARK] Expose range partitioning shuffle introduced by spark-22614

## What changes were proposed in this pull request?

 Expose range partitioning shuffle introduced by spark-22614

## How was this patch tested?

Unit test in dataframe.py

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

Author: xubo245 <601450868@qq.com>

Closes #20456 from xubo245/SPARK22624_PysparkRangePartition.

11 days ago[SPARK-23084][PYTHON] Add unboundedPreceding(), unboundedFollowing() and currentRow...
Huaxin Gao [Sun, 11 Feb 2018 09:55:38 +0000 (18:55 +0900)] 
[SPARK-23084][PYTHON] Add unboundedPreceding(), unboundedFollowing() and currentRow() to PySpark

## What changes were proposed in this pull request?

Added unboundedPreceding(), unboundedFollowing() and currentRow() to PySpark, also updated the rangeBetween API

## How was this patch tested?

did unit test on my local. Please let me know if I need to add unit test in tests.py

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

Closes #20400 from huaxingao/spark_23084.

11 days ago[SPARK-23314][PYTHON] Add ambiguous=False when localizing tz-naive timestamps in...
Li Jin [Sun, 11 Feb 2018 08:31:35 +0000 (17:31 +0900)] 
[SPARK-23314][PYTHON] Add ambiguous=False when localizing tz-naive timestamps in Arrow codepath to deal with dst

## What changes were proposed in this pull request?
When tz_localize a tz-naive timetamp, pandas will throw exception if the timestamp is during daylight saving time period, e.g., `2015-11-01 01:30:00`. This PR fixes this issue by setting `ambiguous=False` when calling tz_localize, which is the same default behavior of pytz.

## How was this patch tested?
Add `test_timestamp_dst`

Author: Li Jin <ice.xelloss@gmail.com>

Closes #20537 from icexelloss/SPARK-23314.

12 days ago[SPARK-23344][PYTHON][ML] Add distanceMeasure param to KMeans
Marco Gaido [Sat, 10 Feb 2018 16:46:45 +0000 (10:46 -0600)] 
[SPARK-23344][PYTHON][ML] Add distanceMeasure param to KMeans

## What changes were proposed in this pull request?

SPARK-22119 introduced a new parameter for KMeans, ie. `distanceMeasure`. The PR adds it also to the Python interface.

## How was this patch tested?

added UTs

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #20520 from mgaido91/SPARK-23344.

12 days ago[SPARK-23360][SQL][PYTHON] Get local timezone from environment via pytz, or dateutil.
Takuya UESHIN [Sat, 10 Feb 2018 16:08:02 +0000 (01:08 +0900)] 
[SPARK-23360][SQL][PYTHON] Get local timezone from environment via pytz, or dateutil.

## What changes were proposed in this pull request?

Currently we use `tzlocal()` to get Python local timezone, but it sometimes causes unexpected behavior.
I changed the way to get Python local timezone to use pytz if the timezone is specified in environment variable, or timezone file via dateutil .

## How was this patch tested?

Added a test and existing tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #20559 from ueshin/issues/SPARK-23360/master.

13 days ago[SPARK-23275][SQL] fix the thread leaking in hive/tests
Feng Liu [Sat, 10 Feb 2018 00:21:47 +0000 (16:21 -0800)] 
[SPARK-23275][SQL] fix the thread leaking in hive/tests

## What changes were proposed in this pull request?

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

The two lines actually can trigger the hive metastore bug: https://issues.apache.org/jira/browse/HIVE-16844

The two configs are not in the default `ObjectStore` properties, so any run hive commands after these two lines will set the `propsChanged` flag in the `ObjectStore.setConf` and then cause thread leaks.

I don't think the two lines are very useful. They can be removed safely.

## How was this patch tested?

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

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

Author: Feng Liu <fengliu@databricks.com>

Closes #20562 from liufengdb/fix-omm.

13 days ago[MINOR][HIVE] Typo fixes
Jacek Laskowski [Sat, 10 Feb 2018 00:18:30 +0000 (18:18 -0600)] 
[MINOR][HIVE] Typo fixes

## What changes were proposed in this pull request?

Typo fixes (with expanding a Hive property)

## How was this patch tested?

local build. Awaiting Jenkins

Author: Jacek Laskowski <jacek@japila.pl>

Closes #20550 from jaceklaskowski/hiveutils-typos.

13 days ago[SPARK-16501][MESOS] Allow providing Mesos principal & secret via files
Rob Vesse [Fri, 9 Feb 2018 19:21:20 +0000 (11:21 -0800)] 
[SPARK-16501][MESOS] Allow providing Mesos principal & secret via files

This commit modifies the Mesos submission client to allow the principal
and secret to be provided indirectly via files.  The path to these files
can be specified either via Spark configuration or via environment
variable.

Assuming these files are appropriately protected by FS/OS permissions
this means we don't ever leak the actual values in process info like ps

Environment variable specification is useful because it allows you to
interpolate the location of this file when using per-user Mesos
credentials.

For some background as to why we have taken this approach I will briefly describe our set up.  On our systems we provide each authorised user account with their own Mesos credentials to provide certain security and audit guarantees to our customers. These credentials are managed by a central Secret management service. In our `spark-env.sh` we determine the appropriate secret and principal files to use depending on the user who is invoking Spark hence the need to inject these via environment variables as well as by configuration properties. So we set these environment variables appropriately and our Spark read in the contents of those files to authenticate itself with Mesos.

This is functionality we have been using it in production across multiple customer sites for some time. This has been in the field for around 18 months with no reported issues. These changes have been sufficient to meet our customer security and audit requirements.

We have been building and deploying custom builds of Apache Spark with various minor tweaks like this which we are now looking to contribute back into the community in order that we can rely upon stock Apache Spark builds and stop maintaining our own internal fork.

Author: Rob Vesse <rvesse@dotnetrdf.org>

Closes #20167 from rvesse/SPARK-16501.

13 days ago[SPARK-21860][CORE][FOLLOWUP] fix java style error
Marco Gaido [Fri, 9 Feb 2018 14:46:27 +0000 (08:46 -0600)] 
[SPARK-21860][CORE][FOLLOWUP] fix java style error

## What changes were proposed in this pull request?

#19077 introduced a Java style error (too long line). Quick fix.

## How was this patch tested?

running `./dev/lint-java`

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #20558 from mgaido91/SPARK-21860.

13 days ago[SPARK-23358][CORE] When the number of partitions is greater than 2^28, it will resul...
liuxian [Fri, 9 Feb 2018 14:45:06 +0000 (08:45 -0600)] 
[SPARK-23358][CORE] When the number of partitions is greater than 2^28, it will result in an error result

## What changes were proposed in this pull request?
In the `checkIndexAndDataFile`,the `blocks` is the ` Int` type,  when it is greater than 2^28, `blocks*8` will overflow, and this will result in an error result.
In fact, `blocks` is actually the number of partitions.

## How was this patch tested?
Manual test

Author: liuxian <liu.xian3@zte.com.cn>

Closes #20544 from 10110346/overflow.

13 days ago[SPARK-23328][PYTHON] Disallow default value None in na.replace/replace when 'to_repl...
hyukjinkwon [Fri, 9 Feb 2018 06:21:10 +0000 (14:21 +0800)] 
[SPARK-23328][PYTHON] Disallow default value None in na.replace/replace when 'to_replace' is not a dictionary

## What changes were proposed in this pull request?

This PR proposes to disallow default value None when 'to_replace' is not a dictionary.

It seems weird we set the default value of `value` to `None` and we ended up allowing the case as below:

```python
>>> df.show()
```
```
+----+------+-----+
| age|height| name|
+----+------+-----+
|  10|    80|Alice|
...
```

```python
>>> df.na.replace('Alice').show()
```
```
+----+------+----+
| age|height|name|
+----+------+----+
|  10|    80|null|
...
```

**After**

This PR targets to disallow the case above:

```python
>>> df.na.replace('Alice').show()
```
```
...
TypeError: value is required when to_replace is not a dictionary.
```

while we still allow when `to_replace` is a dictionary:

```python
>>> df.na.replace({'Alice': None}).show()
```
```
+----+------+----+
| age|height|name|
+----+------+----+
|  10|    80|null|
...
```

## How was this patch tested?

Manually tested, tests were added in `python/pyspark/sql/tests.py` and doctests were fixed.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #20499 from HyukjinKwon/SPARK-19454-followup.

13 days ago[SPARK-23186][SQL] Initialize DriverManager first before loading JDBC Drivers
Dongjoon Hyun [Fri, 9 Feb 2018 04:54:57 +0000 (12:54 +0800)] 
[SPARK-23186][SQL] Initialize DriverManager first before loading JDBC Drivers

## What changes were proposed in this pull request?

Since some JDBC Drivers have class initialization code to call `DriverManager`, we need to initialize `DriverManager` first in order to avoid potential executor-side **deadlock** situations like the following (or [STORM-2527](https://issues.apache.org/jira/browse/STORM-2527)).

```
Thread 9587: (state = BLOCKED)
 - sun.reflect.NativeConstructorAccessorImpl.newInstance0(java.lang.reflect.Constructor, java.lang.Object[]) bci=0 (Compiled frame; information may be imprecise)
 - sun.reflect.NativeConstructorAccessorImpl.newInstance(java.lang.Object[]) bci=85, line=62 (Compiled frame)
 - sun.reflect.DelegatingConstructorAccessorImpl.newInstance(java.lang.Object[]) bci=5, line=45 (Compiled frame)
 - java.lang.reflect.Constructor.newInstance(java.lang.Object[]) bci=79, line=423 (Compiled frame)
 - java.lang.Class.newInstance() bci=138, line=442 (Compiled frame)
 - java.util.ServiceLoader$LazyIterator.nextService() bci=119, line=380 (Interpreted frame)
 - java.util.ServiceLoader$LazyIterator.next() bci=11, line=404 (Interpreted frame)
 - java.util.ServiceLoader$1.next() bci=37, line=480 (Interpreted frame)
 - java.sql.DriverManager$2.run() bci=21, line=603 (Interpreted frame)
 - java.sql.DriverManager$2.run() bci=1, line=583 (Interpreted frame)
 - java.security.AccessController.doPrivileged(java.security.PrivilegedAction) bci=0 (Compiled frame)
 - java.sql.DriverManager.loadInitialDrivers() bci=27, line=583 (Interpreted frame)
 - java.sql.DriverManager.<clinit>() bci=32, line=101 (Interpreted frame)
 - org.apache.phoenix.mapreduce.util.ConnectionUtil.getConnection(java.lang.String, java.lang.Integer, java.lang.String, java.util.Properties) bci=12, line=98 (Interpreted frame)
 - org.apache.phoenix.mapreduce.util.ConnectionUtil.getInputConnection(org.apache.hadoop.conf.Configuration, java.util.Properties) bci=22, line=57 (Interpreted frame)
 - org.apache.phoenix.mapreduce.PhoenixInputFormat.getQueryPlan(org.apache.hadoop.mapreduce.JobContext, org.apache.hadoop.conf.Configuration) bci=61, line=116 (Interpreted frame)
 - org.apache.phoenix.mapreduce.PhoenixInputFormat.createRecordReader(org.apache.hadoop.mapreduce.InputSplit, org.apache.hadoop.mapreduce.TaskAttemptContext) bci=10, line=71 (Interpreted frame)
 - org.apache.spark.rdd.NewHadoopRDD$$anon$1.<init>(org.apache.spark.rdd.NewHadoopRDD, org.apache.spark.Partition, org.apache.spark.TaskContext) bci=233, line=156 (Interpreted frame)

Thread 9170: (state = BLOCKED)
 - org.apache.phoenix.jdbc.PhoenixDriver.<clinit>() bci=35, line=125 (Interpreted frame)
 - sun.reflect.NativeConstructorAccessorImpl.newInstance0(java.lang.reflect.Constructor, java.lang.Object[]) bci=0 (Compiled frame)
 - sun.reflect.NativeConstructorAccessorImpl.newInstance(java.lang.Object[]) bci=85, line=62 (Compiled frame)
 - sun.reflect.DelegatingConstructorAccessorImpl.newInstance(java.lang.Object[]) bci=5, line=45 (Compiled frame)
 - java.lang.reflect.Constructor.newInstance(java.lang.Object[]) bci=79, line=423 (Compiled frame)
 - java.lang.Class.newInstance() bci=138, line=442 (Compiled frame)
 - org.apache.spark.sql.execution.datasources.jdbc.DriverRegistry$.register(java.lang.String) bci=89, line=46 (Interpreted frame)
 - org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils$$anonfun$createConnectionFactory$2.apply() bci=7, line=53 (Interpreted frame)
 - org.apache.spark.sql.execution.datasources.jdbc.JdbcUtils$$anonfun$createConnectionFactory$2.apply() bci=1, line=52 (Interpreted frame)
 - org.apache.spark.sql.execution.datasources.jdbc.JDBCRDD$$anon$1.<init>(org.apache.spark.sql.execution.datasources.jdbc.JDBCRDD, org.apache.spark.Partition, org.apache.spark.TaskContext) bci=81, line=347 (Interpreted frame)
 - org.apache.spark.sql.execution.datasources.jdbc.JDBCRDD.compute(org.apache.spark.Partition, org.apache.spark.TaskContext) bci=7, line=339 (Interpreted frame)
```

## How was this patch tested?

N/A

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #20359 from dongjoon-hyun/SPARK-23186.

2 weeks ago[SPARK-23336][BUILD] Upgrade snappy-java to 1.1.7.1
Yuming Wang [Thu, 8 Feb 2018 18:52:08 +0000 (12:52 -0600)] 
[SPARK-23336][BUILD] Upgrade snappy-java to 1.1.7.1

## What changes were proposed in this pull request?

This PR upgrade snappy-java from 1.1.2.6 to 1.1.7.1.
1.1.7.1 release notes:
- Improved performance for big-endian architecture
- The other performance improvement in [snappy-1.1.5](https://github.com/google/snappy/releases/tag/1.1.5)

1.1.4 release notes:
- Fix a 1% performance regression when snappy is used in PIE executables.
- Improve compression performance by 5%.
- Improve decompression performance by 20%.

More details:
https://github.com/xerial/snappy-java/blob/master/Milestone.md

## How was this patch tested?

manual tests

Author: Yuming Wang <wgyumg@gmail.com>

Closes #20510 from wangyum/SPARK-23336.

2 weeks ago[SPARK-21860][CORE] Improve memory reuse for heap memory in `HeapMemoryAllocator`
liuxian [Thu, 8 Feb 2018 15:41:30 +0000 (23:41 +0800)] 
[SPARK-21860][CORE] Improve memory reuse for heap memory in `HeapMemoryAllocator`

## What changes were proposed in this pull request?
In `HeapMemoryAllocator`, when allocating memory from pool, and the key of pool is memory size.
Actually some size of memory ,such as 1025bytes,1026bytes,......1032bytes, we can think they are the same,because we allocate memory in multiples of 8 bytes.
In this case, we can improve memory reuse.

## How was this patch tested?
Existing tests and added unit tests

Author: liuxian <liu.xian3@zte.com.cn>

Closes #19077 from 10110346/headmemoptimize.

2 weeks ago[SPARK-23268][SQL][FOLLOWUP] Reorganize packages in data source V2
Wenchen Fan [Thu, 8 Feb 2018 11:20:11 +0000 (19:20 +0800)] 
[SPARK-23268][SQL][FOLLOWUP] Reorganize packages in data source V2

## What changes were proposed in this pull request?

This is a followup of https://github.com/apache/spark/pull/20435.

While reorganizing the packages for streaming data source v2, the top level stream read/write support interfaces should not be in the reader/writer package, but should be in the `sources.v2` package, to follow the `ReadSupport`, `WriteSupport`, etc.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20509 from cloud-fan/followup.

2 weeks ago[SPARK-23348][SQL] append data using saveAsTable should adjust the data types
Wenchen Fan [Thu, 8 Feb 2018 08:08:54 +0000 (00:08 -0800)] 
[SPARK-23348][SQL] append data using saveAsTable should adjust the data types

## What changes were proposed in this pull request?

For inserting/appending data to an existing table, Spark should adjust the data types of the input query according to the table schema, or fail fast if it's uncastable.

There are several ways to insert/append data: SQL API, `DataFrameWriter.insertInto`, `DataFrameWriter.saveAsTable`. The first 2 ways create `InsertIntoTable` plan, and the last way creates `CreateTable` plan. However, we only adjust input query data types for `InsertIntoTable`, and users may hit weird errors when appending data using `saveAsTable`. See the JIRA for the error case.

This PR fixes this bug by adjusting data types for `CreateTable` too.

## How was this patch tested?

new test.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20527 from cloud-fan/saveAsTable.

2 weeks agoRevert [SPARK-22279][SQL] Turn on spark.sql.hive.convertMetastoreOrc by default
gatorsmile [Thu, 8 Feb 2018 04:21:18 +0000 (12:21 +0800)] 
Revert [SPARK-22279][SQL] Turn on spark.sql.hive.convertMetastoreOrc by default

## What changes were proposed in this pull request?

This is to revert the changes made in https://github.com/apache/spark/pull/19499 , because this causes a regression. We should not ignore the table-specific compression conf when the Hive serde tables are converted to the data source tables.

## How was this patch tested?

The existing tests.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20536 from gatorsmile/revert22279.

2 weeks ago[SPARK-23319][TESTS][FOLLOWUP] Fix a test for Python 3 without pandas.
Takuya UESHIN [Thu, 8 Feb 2018 03:46:10 +0000 (12:46 +0900)] 
[SPARK-23319][TESTS][FOLLOWUP] Fix a test for Python 3 without pandas.

## What changes were proposed in this pull request?

This is a followup pr of #20487.

When importing module but it doesn't exists, the error message is slightly different between Python 2 and 3.

E.g., in Python 2:

```
No module named pandas
```

in Python 3:

```
No module named 'pandas'
```

So, one test to check an import error fails in Python 3 without pandas.

This pr fixes it.

## How was this patch tested?

Tested manually in my local environment.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #20538 from ueshin/issues/SPARK-23319/fup1.

2 weeks ago[SPARK-23092][SQL] Migrate MemoryStream to DataSourceV2 APIs
Tathagata Das [Wed, 7 Feb 2018 23:22:53 +0000 (15:22 -0800)] 
[SPARK-23092][SQL] Migrate MemoryStream to DataSourceV2 APIs

## What changes were proposed in this pull request?

This PR migrates the MemoryStream to DataSourceV2 APIs.

One additional change is in the reported keys in StreamingQueryProgress.durationMs. "getOffset" and "getBatch" replaced with "setOffsetRange" and "getEndOffset" as tracking these make more sense. Unit tests changed accordingly.

## How was this patch tested?
Existing unit tests, few updated unit tests.

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

Closes #20445 from tdas/SPARK-23092.

2 weeks ago[SPARK-23345][SQL] Remove open stream record even closing it fails
Liang-Chi Hsieh [Wed, 7 Feb 2018 17:48:49 +0000 (09:48 -0800)] 
[SPARK-23345][SQL] Remove open stream record even closing it fails

## What changes were proposed in this pull request?

When `DebugFilesystem` closes opened stream, if any exception occurs, we still need to remove the open stream record from `DebugFilesystem`. Otherwise, it goes to report leaked filesystem connection.

## How was this patch tested?

Existing tests.

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

Closes #20524 from viirya/SPARK-23345.

2 weeks ago[SPARK-23319][TESTS] Explicitly specify Pandas and PyArrow versions in PySpark tests...
hyukjinkwon [Wed, 7 Feb 2018 14:28:10 +0000 (23:28 +0900)] 
[SPARK-23319][TESTS] Explicitly specify Pandas and PyArrow versions in PySpark tests (to skip or test)

## What changes were proposed in this pull request?

This PR proposes to explicitly specify Pandas and PyArrow versions in PySpark tests to skip or test.

We declared the extra dependencies:

https://github.com/apache/spark/blob/b8bfce51abf28c66ba1fc67b0f25fe1617c81025/python/setup.py#L204

In case of PyArrow:

Currently we only check if pyarrow is installed or not without checking the version. It already fails to run tests. For example, if PyArrow 0.7.0 is installed:

```
======================================================================
ERROR: test_vectorized_udf_wrong_return_type (pyspark.sql.tests.ScalarPandasUDF)
----------------------------------------------------------------------
Traceback (most recent call last):
  File "/.../spark/python/pyspark/sql/tests.py", line 4019, in test_vectorized_udf_wrong_return_type
    f = pandas_udf(lambda x: x * 1.0, MapType(LongType(), LongType()))
  File "/.../spark/python/pyspark/sql/functions.py", line 2309, in pandas_udf
    return _create_udf(f=f, returnType=return_type, evalType=eval_type)
  File "/.../spark/python/pyspark/sql/udf.py", line 47, in _create_udf
    require_minimum_pyarrow_version()
  File "/.../spark/python/pyspark/sql/utils.py", line 132, in require_minimum_pyarrow_version
    "however, your version was %s." % pyarrow.__version__)
ImportError: pyarrow >= 0.8.0 must be installed on calling Python process; however, your version was 0.7.0.

----------------------------------------------------------------------
Ran 33 tests in 8.098s

FAILED (errors=33)
```

In case of Pandas:

There are few tests for old Pandas which were tested only when Pandas version was lower, and I rewrote them to be tested when both Pandas version is lower and missing.

## How was this patch tested?

Manually tested by modifying the condition:

```
test_createDataFrame_column_name_encoding (pyspark.sql.tests.ArrowTests) ... skipped 'Pandas >= 1.19.2 must be installed; however, your version was 0.19.2.'
test_createDataFrame_does_not_modify_input (pyspark.sql.tests.ArrowTests) ... skipped 'Pandas >= 1.19.2 must be installed; however, your version was 0.19.2.'
test_createDataFrame_respect_session_timezone (pyspark.sql.tests.ArrowTests) ... skipped 'Pandas >= 1.19.2 must be installed; however, your version was 0.19.2.'
```

```
test_createDataFrame_column_name_encoding (pyspark.sql.tests.ArrowTests) ... skipped 'Pandas >= 0.19.2 must be installed; however, it was not found.'
test_createDataFrame_does_not_modify_input (pyspark.sql.tests.ArrowTests) ... skipped 'Pandas >= 0.19.2 must be installed; however, it was not found.'
test_createDataFrame_respect_session_timezone (pyspark.sql.tests.ArrowTests) ... skipped 'Pandas >= 0.19.2 must be installed; however, it was not found.'
```

```
test_createDataFrame_column_name_encoding (pyspark.sql.tests.ArrowTests) ... skipped 'PyArrow >= 1.8.0 must be installed; however, your version was 0.8.0.'
test_createDataFrame_does_not_modify_input (pyspark.sql.tests.ArrowTests) ... skipped 'PyArrow >= 1.8.0 must be installed; however, your version was 0.8.0.'
test_createDataFrame_respect_session_timezone (pyspark.sql.tests.ArrowTests) ... skipped 'PyArrow >= 1.8.0 must be installed; however, your version was 0.8.0.'
```

```
test_createDataFrame_column_name_encoding (pyspark.sql.tests.ArrowTests) ... skipped 'PyArrow >= 0.8.0 must be installed; however, it was not found.'
test_createDataFrame_does_not_modify_input (pyspark.sql.tests.ArrowTests) ... skipped 'PyArrow >= 0.8.0 must be installed; however, it was not found.'
test_createDataFrame_respect_session_timezone (pyspark.sql.tests.ArrowTests) ... skipped 'PyArrow >= 0.8.0 must be installed; however, it was not found.'
```

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #20487 from HyukjinKwon/pyarrow-pandas-skip.

2 weeks ago[SPARK-23122][PYSPARK][FOLLOWUP] Replace registerTempTable by createOrReplaceTempView
gatorsmile [Wed, 7 Feb 2018 14:24:16 +0000 (23:24 +0900)] 
[SPARK-23122][PYSPARK][FOLLOWUP] Replace registerTempTable by createOrReplaceTempView

## What changes were proposed in this pull request?
Replace `registerTempTable` by `createOrReplaceTempView`.

## How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20523 from gatorsmile/updateExamples.

2 weeks ago[SPARK-23327][SQL] Update the description and tests of three external API or functions
gatorsmile [Wed, 7 Feb 2018 00:46:43 +0000 (16:46 -0800)] 
[SPARK-23327][SQL] Update the description and tests of three external API or functions

## What changes were proposed in this pull request?
Update the description and tests of three external API or functions `createFunction `, `length` and `repartitionByRange `

## How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20495 from gatorsmile/updateFunc.

2 weeks ago[SPARK-23315][SQL] failed to get output from canonicalized data source v2 related...
Wenchen Fan [Tue, 6 Feb 2018 20:43:45 +0000 (12:43 -0800)] 
[SPARK-23315][SQL] failed to get output from canonicalized data source v2 related plans

## What changes were proposed in this pull request?

`DataSourceV2Relation`  keeps a `fullOutput` and resolves the real output on demand by column name lookup. i.e.
```
lazy val output: Seq[Attribute] = reader.readSchema().map(_.name).map { name =>
  fullOutput.find(_.name == name).get
}
```

This will be broken after we canonicalize the plan, because all attribute names become "None", see https://github.com/apache/spark/blob/v2.3.0-rc1/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Canonicalize.scala#L42

To fix this, `DataSourceV2Relation` should just keep `output`, and update the `output` when doing column pruning.

## How was this patch tested?

a new test case

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20485 from cloud-fan/canonicalize.

2 weeks ago[MINOR][TEST] Fix class name for Pandas UDF tests
Li Jin [Tue, 6 Feb 2018 20:30:04 +0000 (12:30 -0800)] 
[MINOR][TEST] Fix class name for Pandas UDF tests

## What changes were proposed in this pull request?

In https://github.com/apache/spark/commit/b2ce17b4c9fea58140a57ca1846b2689b15c0d61, I mistakenly renamed `VectorizedUDFTests` to `ScalarPandasUDF`. This PR fixes the mistake.

## How was this patch tested?

Existing tests.

Author: Li Jin <ice.xelloss@gmail.com>

Closes #20489 from icexelloss/fix-scalar-udf-tests.

2 weeks ago[SPARK-23312][SQL][FOLLOWUP] add a config to turn off vectorized cache reader
Wenchen Fan [Tue, 6 Feb 2018 20:27:37 +0000 (12:27 -0800)] 
[SPARK-23312][SQL][FOLLOWUP] add a config to turn off vectorized cache reader

## What changes were proposed in this pull request?

https://github.com/apache/spark/pull/20483 tried to provide a way to turn off the new columnar cache reader, to restore the behavior in 2.2. However even we turn off that config, the behavior is still different than 2.2.

If the output data are rows, we still enable whole stage codegen for the scan node, which is different with 2.2, we should also fix it.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #20513 from cloud-fan/cache.

2 weeks ago[SPARK-23310][CORE][FOLLOWUP] Fix Java style check issues.
Takuya UESHIN [Tue, 6 Feb 2018 18:46:48 +0000 (10:46 -0800)] 
[SPARK-23310][CORE][FOLLOWUP] Fix Java style check issues.

## What changes were proposed in this pull request?

This is a follow-up of #20492 which broke lint-java checks.
This pr fixes the lint-java issues.

```
[ERROR] src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillReader.java:[79] (sizes) LineLength: Line is longer than 100 characters (found 114).
```

## How was this patch tested?

Checked manually in my local environment.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #20514 from ueshin/issues/SPARK-23310/fup1.

2 weeks ago[SPARK-23334][SQL][PYTHON] Fix pandas_udf with return type StringType() to handle...
Takuya UESHIN [Tue, 6 Feb 2018 09:30:50 +0000 (18:30 +0900)] 
[SPARK-23334][SQL][PYTHON] Fix pandas_udf with return type StringType() to handle str type properly in Python 2.

## What changes were proposed in this pull request?

In Python 2, when `pandas_udf` tries to return string type value created in the udf with `".."`, the execution fails. E.g.,

```python
from pyspark.sql.functions import pandas_udf, col
import pandas as pd

df = spark.range(10)
str_f = pandas_udf(lambda x: pd.Series(["%s" % i for i in x]), "string")
df.select(str_f(col('id'))).show()
```

raises the following exception:

```
...

java.lang.AssertionError: assertion failed: Invalid schema from pandas_udf: expected StringType, got BinaryType
at scala.Predef$.assert(Predef.scala:170)
at org.apache.spark.sql.execution.python.ArrowEvalPythonExec$$anon$2.<init>(ArrowEvalPythonExec.scala:93)

...
```

Seems like pyarrow ignores `type` parameter for `pa.Array.from_pandas()` and consider it as binary type when the type is string type and the string values are `str` instead of `unicode` in Python 2.

This pr adds a workaround for the case.

## How was this patch tested?

Added a test and existing tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #20507 from ueshin/issues/SPARK-23334.

2 weeks ago[SPARK-23300][TESTS] Prints out if Pandas and PyArrow are installed or not in PySpark...
hyukjinkwon [Tue, 6 Feb 2018 07:08:15 +0000 (16:08 +0900)] 
[SPARK-23300][TESTS] Prints out if Pandas and PyArrow are installed or not in PySpark SQL tests

## What changes were proposed in this pull request?

This PR proposes to log if PyArrow and Pandas are installed or not so we can check if related tests are going to be skipped or not.

## How was this patch tested?

Manually tested:

I don't have PyArrow installed in PyPy.
```bash
$ ./run-tests --python-executables=python3
```

```
...
Will test against the following Python executables: ['python3']
Will test the following Python modules: ['pyspark-core', 'pyspark-ml', 'pyspark-mllib', 'pyspark-sql', 'pyspark-streaming']
Will test PyArrow related features against Python executable 'python3' in 'pyspark-sql' module.
Will test Pandas related features against Python executable 'python3' in 'pyspark-sql' module.
Starting test(python3): pyspark.mllib.tests
Starting test(python3): pyspark.sql.tests
Starting test(python3): pyspark.streaming.tests
Starting test(python3): pyspark.tests
```

```bash
$ ./run-tests --modules=pyspark-streaming
```

```
...
Will test against the following Python executables: ['python2.7', 'pypy']
Will test the following Python modules: ['pyspark-streaming']
Starting test(pypy): pyspark.streaming.tests
Starting test(pypy): pyspark.streaming.util
Starting test(python2.7): pyspark.streaming.tests
Starting test(python2.7): pyspark.streaming.util
```

```bash
$ ./run-tests
```

```
...
Will test against the following Python executables: ['python2.7', 'pypy']
Will test the following Python modules: ['pyspark-core', 'pyspark-ml', 'pyspark-mllib', 'pyspark-sql', 'pyspark-streaming']
Will test PyArrow related features against Python executable 'python2.7' in 'pyspark-sql' module.
Will test Pandas related features against Python executable 'python2.7' in 'pyspark-sql' module.
Will skip PyArrow related features against Python executable 'pypy' in 'pyspark-sql' module. PyArrow >= 0.8.0 is required; however, PyArrow was not found.
Will test Pandas related features against Python executable 'pypy' in 'pyspark-sql' module.
Starting test(pypy): pyspark.streaming.tests
Starting test(pypy): pyspark.sql.tests
Starting test(pypy): pyspark.tests
Starting test(python2.7): pyspark.mllib.tests
```

```bash
$ ./run-tests --modules=pyspark-sql --python-executables=pypy
```

```
...
Will test against the following Python executables: ['pypy']
Will test the following Python modules: ['pyspark-sql']
Will skip PyArrow related features against Python executable 'pypy' in 'pyspark-sql' module. PyArrow >= 0.8.0 is required; however, PyArrow was not found.
Will test Pandas related features against Python executable 'pypy' in 'pyspark-sql' module.
Starting test(pypy): pyspark.sql.tests
Starting test(pypy): pyspark.sql.catalog
Starting test(pypy): pyspark.sql.column
Starting test(pypy): pyspark.sql.conf
```

After some modification to produce other cases:

```bash
$ ./run-tests
```

```
...
Will test against the following Python executables: ['python2.7', 'pypy']
Will test the following Python modules: ['pyspark-core', 'pyspark-ml', 'pyspark-mllib', 'pyspark-sql', 'pyspark-streaming']
Will skip PyArrow related features against Python executable 'python2.7' in 'pyspark-sql' module. PyArrow >= 20.0.0 is required; however, PyArrow 0.8.0 was found.
Will skip Pandas related features against Python executable 'python2.7' in 'pyspark-sql' module. Pandas >= 20.0.0 is required; however, Pandas 0.20.2 was found.
Will skip PyArrow related features against Python executable 'pypy' in 'pyspark-sql' module. PyArrow >= 20.0.0 is required; however, PyArrow was not found.
Will skip Pandas related features against Python executable 'pypy' in 'pyspark-sql' module. Pandas >= 20.0.0 is required; however, Pandas 0.22.0 was found.
Starting test(pypy): pyspark.sql.tests
Starting test(pypy): pyspark.streaming.tests
Starting test(pypy): pyspark.tests
Starting test(python2.7): pyspark.mllib.tests
```

```bash
./run-tests-with-coverage
```
```
...
Will test against the following Python executables: ['python2.7', 'pypy']
Will test the following Python modules: ['pyspark-core', 'pyspark-ml', 'pyspark-mllib', 'pyspark-sql', 'pyspark-streaming']
Will test PyArrow related features against Python executable 'python2.7' in 'pyspark-sql' module.
Will test Pandas related features against Python executable 'python2.7' in 'pyspark-sql' module.
Coverage is not installed in Python executable 'pypy' but 'COVERAGE_PROCESS_START' environment variable is set, exiting.
```

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #20473 from HyukjinKwon/SPARK-23300.

2 weeks ago[SPARK-23290][SQL][PYTHON] Use datetime.date for date type when converting Spark...
Takuya UESHIN [Tue, 6 Feb 2018 06:52:25 +0000 (14:52 +0800)] 
[SPARK-23290][SQL][PYTHON] Use datetime.date for date type when converting Spark DataFrame to Pandas DataFrame.

## What changes were proposed in this pull request?

In #18664, there was a change in how `DateType` is being returned to users ([line 1968 in dataframe.py](https://github.com/apache/spark/pull/18664/files#diff-6fc344560230bf0ef711bb9b5573f1faR1968)). This can cause client code which works in Spark 2.2 to fail.
See [SPARK-23290](https://issues.apache.org/jira/browse/SPARK-23290?focusedCommentId=16350917&page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-16350917) for an example.

This pr modifies to use `datetime.date` for date type as Spark 2.2 does.

## How was this patch tested?

Tests modified to fit the new behavior and existing tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #20506 from ueshin/issues/SPARK-23290.

2 weeks ago[SPARK-23326][WEBUI] schedulerDelay should return 0 when the task is running
Shixiong Zhu [Tue, 6 Feb 2018 06:42:42 +0000 (14:42 +0800)] 
[SPARK-23326][WEBUI] schedulerDelay should return 0 when the task is running

## What changes were proposed in this pull request?

When a task is still running, metrics like executorRunTime are not available. Then `schedulerDelay` will be almost the same as `duration` and that's confusing.

This PR makes `schedulerDelay` return 0 when the task is running which is the same behavior as 2.2.

## How was this patch tested?

`AppStatusUtilsSuite.schedulerDelay`

Author: Shixiong Zhu <zsxwing@gmail.com>

Closes #20493 from zsxwing/SPARK-23326.

2 weeks ago[SPARK-23330][WEBUI] Spark UI SQL executions page throws NPE
Xingbo Jiang [Mon, 5 Feb 2018 22:17:11 +0000 (14:17 -0800)] 
[SPARK-23330][WEBUI] Spark UI SQL executions page throws NPE

## What changes were proposed in this pull request?

Spark SQL executions page throws the following error and the page crashes:
```
HTTP ERROR 500
Problem accessing /SQL/. Reason:

Server Error
Caused by:
java.lang.NullPointerException
at scala.collection.immutable.StringOps$.length$extension(StringOps.scala:47)
at scala.collection.immutable.StringOps.length(StringOps.scala:47)
at scala.collection.IndexedSeqOptimized$class.isEmpty(IndexedSeqOptimized.scala:27)
at scala.collection.immutable.StringOps.isEmpty(StringOps.scala:29)
at scala.collection.TraversableOnce$class.nonEmpty(TraversableOnce.scala:111)
at scala.collection.immutable.StringOps.nonEmpty(StringOps.scala:29)
at org.apache.spark.sql.execution.ui.ExecutionTable.descriptionCell(AllExecutionsPage.scala:182)
at org.apache.spark.sql.execution.ui.ExecutionTable.row(AllExecutionsPage.scala:155)
at org.apache.spark.sql.execution.ui.ExecutionTable$$anonfun$8.apply(AllExecutionsPage.scala:204)
at org.apache.spark.sql.execution.ui.ExecutionTable$$anonfun$8.apply(AllExecutionsPage.scala:204)
at org.apache.spark.ui.UIUtils$$anonfun$listingTable$2.apply(UIUtils.scala:339)
at org.apache.spark.ui.UIUtils$$anonfun$listingTable$2.apply(UIUtils.scala:339)
at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)
at scala.collection.TraversableLike$class.map(TraversableLike.scala:234)
at scala.collection.AbstractTraversable.map(Traversable.scala:104)
at org.apache.spark.ui.UIUtils$.listingTable(UIUtils.scala:339)
at org.apache.spark.sql.execution.ui.ExecutionTable.toNodeSeq(AllExecutionsPage.scala:203)
at org.apache.spark.sql.execution.ui.AllExecutionsPage.render(AllExecutionsPage.scala:67)
at org.apache.spark.ui.WebUI$$anonfun$2.apply(WebUI.scala:82)
at org.apache.spark.ui.WebUI$$anonfun$2.apply(WebUI.scala:82)
at org.apache.spark.ui.JettyUtils$$anon$3.doGet(JettyUtils.scala:90)
at javax.servlet.http.HttpServlet.service(HttpServlet.java:687)
at javax.servlet.http.HttpServlet.service(HttpServlet.java:790)
at org.eclipse.jetty.servlet.ServletHolder.handle(ServletHolder.java:848)
at org.eclipse.jetty.servlet.ServletHandler.doHandle(ServletHandler.java:584)
at org.eclipse.jetty.server.handler.ContextHandler.doHandle(ContextHandler.java:1180)
at org.eclipse.jetty.servlet.ServletHandler.doScope(ServletHandler.java:512)
at org.eclipse.jetty.server.handler.ContextHandler.doScope(ContextHandler.java:1112)
at org.eclipse.jetty.server.handler.ScopedHandler.handle(ScopedHandler.java:141)
at org.eclipse.jetty.server.handler.ContextHandlerCollection.handle(ContextHandlerCollection.java:213)
at org.eclipse.jetty.server.handler.HandlerWrapper.handle(HandlerWrapper.java:134)
at org.eclipse.jetty.server.Server.handle(Server.java:534)
at org.eclipse.jetty.server.HttpChannel.handle(HttpChannel.java:320)
at org.eclipse.jetty.server.HttpConnection.onFillable(HttpConnection.java:251)
at org.eclipse.jetty.io.AbstractConnection$ReadCallback.succeeded(AbstractConnection.java:283)
at org.eclipse.jetty.io.FillInterest.fillable(FillInterest.java:108)
at org.eclipse.jetty.io.SelectChannelEndPoint$2.run(SelectChannelEndPoint.java:93)
at org.eclipse.jetty.util.thread.strategy.ExecuteProduceConsume.executeProduceConsume(ExecuteProduceConsume.java:303)
at org.eclipse.jetty.util.thread.strategy.ExecuteProduceConsume.produceConsume(ExecuteProduceConsume.java:148)
at org.eclipse.jetty.util.thread.strategy.ExecuteProduceConsume.run(ExecuteProduceConsume.java:136)
at org.eclipse.jetty.util.thread.QueuedThreadPool.runJob(QueuedThreadPool.java:671)
at org.eclipse.jetty.util.thread.QueuedThreadPool$2.run(QueuedThreadPool.java:589)
at java.lang.Thread.run(Thread.java:748)
```

One of the possible reason that this page fails may be the `SparkListenerSQLExecutionStart` event get dropped before processed, so the execution description and details don't get updated.
This was not a issue in 2.2 because it would ignore any job start event that arrives before the corresponding execution start event, which doesn't sound like a good decision.

We shall try to handle the null values in the front page side, that is, try to give a default value when `execution.details` or `execution.description` is null.
Another possible approach is not to spill the `LiveExecutionData` in `SQLAppStatusListener.update(exec: LiveExecutionData)` if `exec.details` is null. This is not ideal because this way you will not see the execution if `SparkListenerSQLExecutionStart` event is lost, because `AllExecutionsPage` only read executions from KVStore.

## How was this patch tested?

After the change, the page shows the following:
![image](https://user-images.githubusercontent.com/4784782/35775480-28cc5fde-093e-11e8-8ccc-f58c2ef4a514.png)

Author: Xingbo Jiang <xingbo.jiang@databricks.com>

Closes #20502 from jiangxb1987/executionPage.

2 weeks ago[SPARK-23310][CORE] Turn off read ahead input stream for unshafe shuffle reader
Sital Kedia [Mon, 5 Feb 2018 18:19:18 +0000 (10:19 -0800)] 
[SPARK-23310][CORE] Turn off read ahead input stream for unshafe shuffle reader

To fix regression for TPC-DS queries

Author: Sital Kedia <skedia@fb.com>

Closes #20492 from sitalkedia/turn_off_async_inputstream.