Commit graph

2396 commits

Author SHA1 Message Date
jerryshao 5854f77ce1 [SPARK-20244][CORE] Handle incorrect bytesRead metrics when using PySpark
## What changes were proposed in this pull request?

Hadoop FileSystem's statistics in based on thread local variables, this is ok if the RDD computation chain is running in the same thread. But if child RDD creates another thread to consume the iterator got from Hadoop RDDs, the bytesRead computation will be error, because now the iterator's `next()` and `close()` may run in different threads. This could be happened when using PySpark with PythonRDD.

So here building a map to track the `bytesRead` for different thread and add them together. This method will be used in three RDDs, `HadoopRDD`, `NewHadoopRDD` and `FileScanRDD`. I assume `FileScanRDD` cannot be called directly, so I only fixed `HadoopRDD` and `NewHadoopRDD`.

## How was this patch tested?

Unit test and local cluster verification.

Author: jerryshao <sshao@hortonworks.com>

Closes #17617 from jerryshao/SPARK-20244.
2017-05-31 22:34:53 -07:00
Shixiong Zhu 24db35826a [SPARK-20940][CORE] Replace IllegalAccessError with IllegalStateException
## What changes were proposed in this pull request?

`IllegalAccessError` is a fatal error (a subclass of LinkageError) and its meaning is `Thrown if an application attempts to access or modify a field, or to call a method that it does not have access to`. Throwing a fatal error for AccumulatorV2 is not necessary and is pretty bad because it usually will just kill executors or SparkContext ([SPARK-20666](https://issues.apache.org/jira/browse/SPARK-20666) is an example of killing SparkContext due to `IllegalAccessError`). I think the correct type of exception in AccumulatorV2 should be `IllegalStateException`.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #18168 from zsxwing/SPARK-20940.
2017-05-31 17:26:18 -07:00
jinxing ac7fc3075b [SPARK-20288] Avoid generating the MapStatus by stageId in BasicSchedulerIntegrationSuite
## What changes were proposed in this pull request?

ShuffleId is determined before job submitted. But it's hard to predict stageId by shuffleId.
Stage is created in DAGScheduler(
https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala#L381), but the order is n
ot determined in `HashSet`.
I added a log(println(s"Creating ShufflMapStage-$id on shuffle-${shuffleDep.shuffleId}")) after (https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala#L331), when testing BasicSchedulerIntegrationSuite:"multi-stage job". It will print:
Creating ShufflMapStage-0 on shuffle-0
Creating ShufflMapStage-1 on shuffle-2
Creating ShufflMapStage-2 on shuffle-1
Creating ShufflMapStage-3 on shuffle-3
or
Creating ShufflMapStage-0 on shuffle-1
Creating ShufflMapStage-1 on shuffle-3
Creating ShufflMapStage-2 on shuffle-0
Creating ShufflMapStage-3 on shuffle-2
It might be better to avoid generating the MapStatus by stageId.

Author: jinxing <jinxing6042@126.com>

Closes #17603 from jinxing64/SPARK-20288.
2017-05-31 10:46:23 -05:00
jinxing de953c214c [SPARK-20333] HashPartitioner should be compatible with num of child RDD's partitions.
## What changes were proposed in this pull request?

Fix test
"don't submit stage until its dependencies map outputs are registered (SPARK-5259)" ,
"run trivial shuffle with out-of-band executor failure and retry",
"reduce tasks should be placed locally with map output"
in DAGSchedulerSuite.

Author: jinxing <jinxing6042@126.com>

Closes #17634 from jinxing64/SPARK-20333.
2017-05-30 14:02:33 -05:00
Wenchen Fan 1d62f8aca8 [SPARK-19659][CORE][FOLLOW-UP] Fetch big blocks to disk when shuffle-read
## What changes were proposed in this pull request?

This PR includes some minor improvement for the comments and tests in https://github.com/apache/spark/pull/16989

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #18117 from cloud-fan/follow.
2017-05-27 10:57:43 +08:00
Yu Peng 4af3781291 [SPARK-10643][CORE] Make spark-submit download remote files to local in client mode
## What changes were proposed in this pull request?

This PR makes spark-submit script download remote files to local file system for local/standalone client mode.

## How was this patch tested?

- Unit tests
- Manual tests by adding s3a jar and testing against file on s3.

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

Author: Yu Peng <loneknightpy@gmail.com>

Closes #18078 from loneknightpy/download-jar-in-spark-submit.
2017-05-26 16:28:36 -07:00
Wenchen Fan 629f38e171 [SPARK-20887][CORE] support alternative keys in ConfigBuilder
## What changes were proposed in this pull request?

`ConfigBuilder` builds `ConfigEntry` which can only read value with one key, if we wanna change the config name but still keep the old one, it's hard to do.

This PR introduce `ConfigBuilder.withAlternative`, to support reading config value with alternative keys. And also rename `spark.scheduler.listenerbus.eventqueue.size` to `spark.scheduler.listenerbus.eventqueue.capacity` with this feature, according to https://github.com/apache/spark/pull/14269#discussion_r118432313

## How was this patch tested?

a new test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #18110 from cloud-fan/config.
2017-05-26 21:13:38 +08:00
hyukjinkwon e9f983df27 [SPARK-19707][SPARK-18922][TESTS][SQL][CORE] Fix test failures/the invalid path check for sc.addJar on Windows
## What changes were proposed in this pull request?

This PR proposes two things:

- A follow up for SPARK-19707 (Improving the invalid path check for sc.addJar on Windows as well).

```
org.apache.spark.SparkContextSuite:
 - add jar with invalid path *** FAILED *** (32 milliseconds)
   2 was not equal to 1 (SparkContextSuite.scala:309)
   ...
```

- Fix path vs URI related test failures on Windows.

```
org.apache.spark.storage.LocalDirsSuite:
 - SPARK_LOCAL_DIRS override also affects driver *** FAILED *** (0 milliseconds)
   new java.io.File("/NONEXISTENT_PATH").exists() was true (LocalDirsSuite.scala:50)
   ...

 - Utils.getLocalDir() throws an exception if any temporary directory cannot be retrieved *** FAILED *** (15 milliseconds)
   Expected exception java.io.IOException to be thrown, but no exception was thrown. (LocalDirsSuite.scala:64)
   ...
```

```
org.apache.spark.sql.hive.HiveSchemaInferenceSuite:
 - orc: schema should be inferred and saved when INFER_AND_SAVE is specified *** FAILED *** (203 milliseconds)
   java.net.URISyntaxException: Illegal character in opaque part at index 2: C:\projects\spark\target\tmp\spark-dae61ab3-a851-4dd3-bf4e-be97c501f254
   ...

 - parquet: schema should be inferred and saved when INFER_AND_SAVE is specified *** FAILED *** (203 milliseconds)
   java.net.URISyntaxException: Illegal character in opaque part at index 2: C:\projects\spark\target\tmp\spark-fa3aff89-a66e-4376-9a37-2a9b87596939
   ...

 - orc: schema should be inferred but not stored when INFER_ONLY is specified *** FAILED *** (141 milliseconds)
   java.net.URISyntaxException: Illegal character in opaque part at index 2: C:\projects\spark\target\tmp\spark-fb464e59-b049-481b-9c75-f53295c9fc2c
   ...

 - parquet: schema should be inferred but not stored when INFER_ONLY is specified *** FAILED *** (125 milliseconds)
   java.net.URISyntaxException: Illegal character in opaque part at index 2: C:\projects\spark\target\tmp\spark-9487568e-80a4-42b3-b0a5-d95314c4ccbc
   ...

 - orc: schema should not be inferred when NEVER_INFER is specified *** FAILED *** (156 milliseconds)
   java.net.URISyntaxException: Illegal character in opaque part at index 2: C:\projects\spark\target\tmp\spark-0d2dfa45-1b0f-4958-a8be-1074ed0135a
   ...

 - parquet: schema should not be inferred when NEVER_INFER is specified *** FAILED *** (547 milliseconds)
   java.net.URISyntaxException: Illegal character in opaque part at index 2: C:\projects\spark\target\tmp\spark-6d95d64e-613e-4a59-a0f6-d198c5aa51ee
   ...
```

```
org.apache.spark.sql.execution.command.DDLSuite:
 - create temporary view using *** FAILED *** (15 milliseconds)
   org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-3881d9ca-561b-488d-90b9-97587472b853	mp;
   ...

 - insert data to a data source table which has a non-existing location should succeed *** FAILED *** (109 milliseconds)
   file:/C:projectsspark%09arget%09mpspark-4cad3d19-6085-4b75-b407-fe5e9d21df54 did not equal file:///C:/projects/spark/target/tmp/spark-4cad3d19-6085-4b75-b407-fe5e9d21df54 (DDLSuite.scala:1869)
   ...

 - insert into a data source table with a non-existing partition location should succeed *** FAILED *** (94 milliseconds)
   file:/C:projectsspark%09arget%09mpspark-4b52e7de-e3aa-42fd-95d4-6d4d58d1d95d did not equal file:///C:/projects/spark/target/tmp/spark-4b52e7de-e3aa-42fd-95d4-6d4d58d1d95d (DDLSuite.scala:1910)
   ...

 - read data from a data source table which has a non-existing location should succeed *** FAILED *** (93 milliseconds)
   file:/C:projectsspark%09arget%09mpspark-f8c281e2-08c2-4f73-abbf-f3865b702c34 did not equal file:///C:/projects/spark/target/tmp/spark-f8c281e2-08c2-4f73-abbf-f3865b702c34 (DDLSuite.scala:1937)
   ...

 - read data from a data source table with non-existing partition location should succeed *** FAILED *** (110 milliseconds)
   java.lang.IllegalArgumentException: Can not create a Path from an empty string
   ...

 - create datasource table with a non-existing location *** FAILED *** (94 milliseconds)
   file:/C:projectsspark%09arget%09mpspark-387316ae-070c-4e78-9b78-19ebf7b29ec8 did not equal file:///C:/projects/spark/target/tmp/spark-387316ae-070c-4e78-9b78-19ebf7b29ec8 (DDLSuite.scala:1982)
   ...

 - CTAS for external data source table with a non-existing location *** FAILED *** (16 milliseconds)
   java.lang.IllegalArgumentException: Can not create a Path from an empty string
   ...

 - CTAS for external data source table with a existed location *** FAILED *** (15 milliseconds)
   java.lang.IllegalArgumentException: Can not create a Path from an empty string
   ...

 - data source table:partition column name containing a b *** FAILED *** (125 milliseconds)
   java.lang.IllegalArgumentException: Can not create a Path from an empty string
   ...

 - data source table:partition column name containing a:b *** FAILED *** (143 milliseconds)
   java.lang.IllegalArgumentException: Can not create a Path from an empty string
   ...

 - data source table:partition column name containing a%b *** FAILED *** (109 milliseconds)
   java.lang.IllegalArgumentException: Can not create a Path from an empty string
   ...

 - data source table:partition column name containing a,b *** FAILED *** (109 milliseconds)
   java.lang.IllegalArgumentException: Can not create a Path from an empty string
   ...

 - location uri contains a b for datasource table *** FAILED *** (94 milliseconds)
   file:/C:projectsspark%09arget%09mpspark-5739cda9-b702-4e14-932c-42e8c4174480a%20b did not equal file:///C:/projects/spark/target/tmp/spark-5739cda9-b702-4e14-932c-42e8c4174480/a%20b (DDLSuite.scala:2084)
   ...

 - location uri contains a:b for datasource table *** FAILED *** (78 milliseconds)
   file:/C:projectsspark%09arget%09mpspark-9bdd227c-840f-4f08-b7c5-4036638f098da:b did not equal file:///C:/projects/spark/target/tmp/spark-9bdd227c-840f-4f08-b7c5-4036638f098d/a:b (DDLSuite.scala:2084)
   ...

 - location uri contains a%b for datasource table *** FAILED *** (78 milliseconds)
   file:/C:projectsspark%09arget%09mpspark-62bb5f1d-fa20-460a-b534-cb2e172a3640a%25b did not equal file:///C:/projects/spark/target/tmp/spark-62bb5f1d-fa20-460a-b534-cb2e172a3640/a%25b (DDLSuite.scala:2084)
   ...

 - location uri contains a b for database *** FAILED *** (16 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
   ...

 - location uri contains a:b for database *** FAILED *** (15 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
   ...

 - location uri contains a%b for database *** FAILED *** (0 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
   ...
```

```
org.apache.spark.sql.hive.execution.HiveDDLSuite:
 - create hive table with a non-existing location *** FAILED *** (16 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
   ...

 - CTAS for external hive table with a non-existing location *** FAILED *** (16 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
   ...

 - CTAS for external hive table with a existed location *** FAILED *** (16 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
   ...

 - partition column name of parquet table containing a b *** FAILED *** (156 milliseconds)
   java.lang.IllegalArgumentException: Can not create a Path from an empty string
   ...

 - partition column name of parquet table containing a:b *** FAILED *** (94 milliseconds)
   java.lang.IllegalArgumentException: Can not create a Path from an empty string
   ...

 - partition column name of parquet table containing a%b *** FAILED *** (125 milliseconds)
   java.lang.IllegalArgumentException: Can not create a Path from an empty string
   ...

 - partition column name of parquet table containing a,b *** FAILED *** (110 milliseconds)
   java.lang.IllegalArgumentException: Can not create a Path from an empty string
   ...

 - partition column name of hive table containing a b *** FAILED *** (15 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
   ...

 - partition column name of hive table containing a:b *** FAILED *** (16 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
   ...

 - partition column name of hive table containing a%b *** FAILED *** (16 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
   ...

 - partition column name of hive table containing a,b *** FAILED *** (0 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
   ...

 - hive table: location uri contains a b *** FAILED *** (0 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
   ...

 - hive table: location uri contains a:b *** FAILED *** (0 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
   ...

 - hive table: location uri contains a%b *** FAILED *** (0 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
   ...
```

```
org.apache.spark.sql.sources.PathOptionSuite:
 - path option also exist for write path *** FAILED *** (94 milliseconds)
   file:/C:projectsspark%09arget%09mpspark-2870b281-7ac0-43d6-b6b6-134e01ab6fdc did not equal file:///C:/projects/spark/target/tmp/spark-2870b281-7ac0-43d6-b6b6-134e01ab6fdc (PathOptionSuite.scala:98)
   ...
```

```
org.apache.spark.sql.CachedTableSuite:
 - SPARK-19765: UNCACHE TABLE should un-cache all cached plans that refer to this table *** FAILED *** (110 milliseconds)
   java.lang.IllegalArgumentException: Can not create a Path from an empty string
   ...
```

```
org.apache.spark.sql.execution.DataSourceScanExecRedactionSuite:
 - treeString is redacted *** FAILED *** (250 milliseconds)
   "file:/C:/projects/spark/target/tmp/spark-3ecc1fa4-3e76-489c-95f4-f0b0500eae28" did not contain "C:\projects\spark\target\tmp\spark-3ecc1fa4-3e76-489c-95f4-f0b0500eae28" (DataSourceScanExecRedactionSuite.scala:46)
   ...
```

## How was this patch tested?

Tested via AppVeyor for each and checked it passed once each. These should be retested via AppVeyor in this PR.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17987 from HyukjinKwon/windows-20170515.
2017-05-25 17:10:30 +01:00
jinxing 3f94e64aa8 [SPARK-19659] Fetch big blocks to disk when shuffle-read.
## What changes were proposed in this pull request?

Currently the whole block is fetched into memory(off heap by default) when shuffle-read. A block is defined by (shuffleId, mapId, reduceId). Thus it can be large when skew situations. If OOM happens during shuffle read, job will be killed and users will be notified to "Consider boosting spark.yarn.executor.memoryOverhead". Adjusting parameter and allocating more memory can resolve the OOM. However the approach is not perfectly suitable for production environment, especially for data warehouse.
Using Spark SQL as data engine in warehouse, users hope to have a unified parameter(e.g. memory) but less resource wasted(resource is allocated but not used). The hope is strong especially when migrating data engine to Spark from another one(e.g. Hive). Tuning the parameter for thousands of SQLs one by one is very time consuming.
It's not always easy to predict skew situations, when happen, it make sense to fetch remote blocks to disk for shuffle-read, rather than kill the job because of OOM.

In this pr, I propose to fetch big blocks to disk(which is also mentioned in SPARK-3019):

1. Track average size and also the outliers(which are larger than 2*avgSize) in MapStatus;
2. Request memory from `MemoryManager` before fetch blocks and release the memory to `MemoryManager` when `ManagedBuffer` is released.
3. Fetch remote blocks to disk when failing acquiring memory from `MemoryManager`, otherwise fetch to memory.

This is an improvement for memory control when shuffle blocks and help to avoid OOM in scenarios like below:
1. Single huge block;
2. Sizes of many blocks are underestimated in `MapStatus` and the actual footprint of blocks is much larger than the estimated.

## How was this patch tested?
Added unit test in `MapStatusSuite` and `ShuffleBlockFetcherIteratorSuite`.

Author: jinxing <jinxing6042@126.com>

Closes #16989 from jinxing64/SPARK-19659.
2017-05-25 16:11:30 +08:00
Xingbo Jiang d76633e3ca [SPARK-18406][CORE] Race between end-of-task and completion iterator read lock release
## What changes were proposed in this pull request?

When a TaskContext is not propagated properly to all child threads for the task, just like the reported cases in this issue, we fail to get to TID from TaskContext and that causes unable to release the lock and assertion failures. To resolve this, we have to explicitly pass the TID value to the `unlock` method.

## How was this patch tested?

Add new failing regression test case in `RDDSuite`.

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

Closes #18076 from jiangxb1987/completion-iterator.
2017-05-24 15:43:23 +08:00
James Shuster 4dbb63f085 [SPARK-20815][SPARKR] NullPointerException in RPackageUtils#checkManifestForR
## What changes were proposed in this pull request?

- Add a null check to RPackageUtils#checkManifestForR so that jars w/o manifests don't NPE.

## How was this patch tested?

- Unit tests and manual tests.

Author: James Shuster <jshuster@palantir.com>

Closes #18040 from jrshust/feature/r-package-utils.
2017-05-22 21:41:11 -07:00
jinxing 2597674bcc [SPARK-20801] Record accurate size of blocks in MapStatus when it's above threshold.
## What changes were proposed in this pull request?

Currently, when number of reduces is above 2000, HighlyCompressedMapStatus is used to store size of blocks. in HighlyCompressedMapStatus, only average size is stored for non empty blocks. Which is not good for memory control when we shuffle blocks. It makes sense to store the accurate size of block when it's above threshold.

## How was this patch tested?

Added test in MapStatusSuite.

Author: jinxing <jinxing6042@126.com>

Closes #18031 from jinxing64/SPARK-20801.
2017-05-22 22:09:49 +08:00
caoxuewen f1ffc6e71f [SPARK-20609][CORE] Run the SortShuffleSuite unit tests have residual spark_* system directory
## What changes were proposed in this pull request?
This PR solution to run the SortShuffleSuite unit tests have residual spark_* system directory
For example:
OS:Windows 7
After the running SortShuffleSuite unit tests,
the system of TMP directory have '..\AppData\Local\Temp\spark-f64121f9-11b4-4ffd-a4f0-cfca66643503' not deleted

## How was this patch tested?
Run SortShuffleSuite unit test.

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

Closes #17869 from heary-cao/SortShuffleSuite.
2017-05-22 14:23:23 +01:00
fjh100456 190d8b0b63 [SPARK-20591][WEB UI] Succeeded tasks num not equal in all jobs page and job detail page on spark web ui when speculative task(s) exist.
## What changes were proposed in this pull request?

Modified succeeded num in job detail page from "completed = stageData.completedIndices.size" to "completed = stageData.numCompleteTasks",which making succeeded tasks num in all jobs page and job detail page look more consistent, and more easily to find which stages the speculative task(s) were in.

## How was this patch tested?

manual tests

Author: fjh100456 <fu.jinhua6@zte.com.cn>

Closes #17923 from fjh100456/master.
2017-05-22 13:58:42 +01:00
caoxuewen f398640daa [SPARK-20607][CORE] Add new unit tests to ShuffleSuite
## What changes were proposed in this pull request?

This PR update to two:
1.adds the new unit tests.
  testing would be performed when there is no shuffle stage,
  shuffle will not generate the data file and the index files.
2.Modify the '[SPARK-4085] rerun map stage if reduce stage cannot find its local shuffle file' unit test,
  parallelize is 1 but not is 2, Check the index file and delete.

## How was this patch tested?
The new unit test.

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

Closes #17868 from heary-cao/ShuffleSuite.
2017-05-19 15:25:03 +01:00
Shixiong Zhu 324a904d8e [SPARK-13747][CORE] Add ThreadUtils.awaitReady and disallow Await.ready
## What changes were proposed in this pull request?

Add `ThreadUtils.awaitReady` similar to `ThreadUtils.awaitResult` and disallow `Await.ready`.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #17763 from zsxwing/awaitready.
2017-05-17 17:21:46 -07:00
Josh Rosen 30e0557dbc [SPARK-20776] Fix perf. problems in JobProgressListener caused by TaskMetrics construction
## What changes were proposed in this pull request?

In

```
./bin/spark-shell --master=local[64]
```

I ran

```
sc.parallelize(1 to 100000, 100000).count()
```
and profiled the time spend in the LiveListenerBus event processing thread. I discovered that the majority of the time was being spent in `TaskMetrics.empty` calls in `JobProgressListener.onTaskStart`. It turns out that we can slightly refactor to remove the need to construct one empty instance per call, greatly improving the performance of this code.

The performance gains here help to avoid an issue where listener events would be dropped because the JobProgressListener couldn't keep up with the throughput.

**Before:**

![image](https://cloud.githubusercontent.com/assets/50748/26133095/95bcd42a-3a59-11e7-8051-a50550e447b8.png)

**After:**

![image](https://cloud.githubusercontent.com/assets/50748/26133070/7935e148-3a59-11e7-8c2d-73d5aa5a2397.png)

## How was this patch tested?

Benchmarks described above.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #18008 from JoshRosen/nametoaccums-improvements.
2017-05-17 13:04:21 +08:00
Shixiong Zhu 9150bca47e [SPARK-20529][CORE] Allow worker and master work with a proxy server
## What changes were proposed in this pull request?

In the current codes, when worker connects to master, master will send its address to the worker. Then worker will save this address and use it to reconnect in case of failure. However, sometimes, this address is not correct. If there is a proxy between master and worker, the address master sent is not the address of proxy.

In this PR, the master address used by the worker will be sent to the master, then master just replies this address back, worker will use this address to reconnect in case of failure. In other words, the worker will use the config master address set in the worker side if possible rather than the master address set in the master side.

There is still one potential issue though. When a master is restarted or takes over leadership, the work will use the address sent from the master to connect. If there is still a proxy between  master and worker, the address may be wrong. However, there is no way to figure it out just in the worker.

## How was this patch tested?

The new added unit test.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #17821 from zsxwing/SPARK-20529.
2017-05-16 10:35:51 -07:00
Shixiong Zhu 7d6ff39106 [SPARK-20702][CORE] TaskContextImpl.markTaskCompleted should not hide the original error
## What changes were proposed in this pull request?

This PR adds an `error` parameter to `TaskContextImpl.markTaskCompleted` to propagate the original error.

It also fixes an issue that `TaskCompletionListenerException.getMessage` doesn't include `previousError`.

## How was this patch tested?

New unit tests.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #17942 from zsxwing/SPARK-20702.
2017-05-12 10:46:44 -07:00
Sean Owen fc8a2b6ee6 [SPARK-20554][BUILD] Remove usage of scala.language.reflectiveCalls
## What changes were proposed in this pull request?

Remove uses of scala.language.reflectiveCalls that are either unnecessary or probably resulting in more complex code. This turned out to be less significant than I thought, but, still worth a touch-up.

## How was this patch tested?

Existing tests.

Author: Sean Owen <sowen@cloudera.com>

Closes #17949 from srowen/SPARK-20554.
2017-05-12 09:55:04 +01:00
NICHOLAS T. MARION b512233a45 [SPARK-20393][WEBU UI] Strengthen Spark to prevent XSS vulnerabilities
## What changes were proposed in this pull request?

Add stripXSS and stripXSSMap to Spark Core's UIUtils. Calling these functions at any point that getParameter is called against a HttpServletRequest.

## How was this patch tested?

Unit tests, IBM Security AppScan Standard no longer showing vulnerabilities, manual verification of WebUI pages.

Author: NICHOLAS T. MARION <nmarion@us.ibm.com>

Closes #17686 from n-marion/xss-fix.
2017-05-10 10:59:57 +01:00
Michael Mior a4cbf26bca [SPARK-20637][CORE] Remove mention of old RDD classes from comments
## What changes were proposed in this pull request?

A few comments around the code mention RDD classes that do not exist anymore. I'm not sure of the best way to replace these, so I've just removed them here.

## How was this patch tested?

Only changes code comments, no testing required

Author: Michael Mior <mmior@uwaterloo.ca>

Closes #17900 from michaelmior/remove-old-rdds.
2017-05-10 10:21:43 +01:00
Sanket 181261a81d [SPARK-20355] Add per application spark version on the history server headerpage
## What changes were proposed in this pull request?

Spark Version for a specific application is not displayed on the history page now. It should be nice to switch the spark version on the UI when we click on the specific application.
Currently there seems to be way as SparkListenerLogStart records the application version. So, it should be trivial to listen to this event and provision this change on the UI.
For Example
<img width="1439" alt="screen shot 2017-04-06 at 3 23 41 pm" src="https://cloud.githubusercontent.com/assets/8295799/25092650/41f3970a-2354-11e7-9b0d-4646d0adeb61.png">
<img width="1399" alt="screen shot 2017-04-17 at 9 59 33 am" src="https://cloud.githubusercontent.com/assets/8295799/25092743/9f9e2f28-2354-11e7-9605-f2f1c63f21fe.png">

{"Event":"SparkListenerLogStart","Spark Version":"2.0.0"}
(Please fill in changes proposed in this fix)
Modified the SparkUI for History server to listen to SparkLogListenerStart event and extract the version and print it.

## How was this patch tested?
Manual testing of UI page. Attaching the UI screenshot changes here

(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: Sanket <schintap@untilservice-lm>

Closes #17658 from redsanket/SPARK-20355.
2017-05-09 09:30:09 -05:00
Xianyang Liu 15526653a9 [SPARK-19956][CORE] Optimize a location order of blocks with topology information
## What changes were proposed in this pull request?

When call the method getLocations of BlockManager, we only compare the data block host. Random selection for non-local data blocks, this may cause the selected data block to be in a different rack. So in this patch to increase the sort of the rack.

## How was this patch tested?

New test case.

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

Author: Xianyang Liu <xianyang.liu@intel.com>

Closes #17300 from ConeyLiu/blockmanager.
2017-05-08 17:33:47 +08:00
caoxuewen 37f963ac13 [SPARK-20518][CORE] Supplement the new blockidsuite unit tests
## What changes were proposed in this pull request?

This PR adds the new unit tests to support ShuffleDataBlockId , ShuffleIndexBlockId , TempShuffleBlockId , TempLocalBlockId

## How was this patch tested?

The new unit test.

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

Closes #17794 from heary-cao/blockidsuite.
2017-05-07 10:08:06 +01:00
Sean Owen 16fab6b0ef [SPARK-20523][BUILD] Clean up build warnings for 2.2.0 release
## What changes were proposed in this pull request?

Fix build warnings primarily related to Breeze 0.13 operator changes, Java style problems

## How was this patch tested?

Existing tests

Author: Sean Owen <sowen@cloudera.com>

Closes #17803 from srowen/SPARK-20523.
2017-05-03 10:18:35 +01:00
Ryan Blue 2b2dd08e97 [SPARK-20540][CORE] Fix unstable executor requests.
There are two problems fixed in this commit. First, the
ExecutorAllocationManager sets a timeout to avoid requesting executors
too often. However, the timeout is always updated based on its value and
a timeout, not the current time. If the call is delayed by locking for
more than the ongoing scheduler timeout, the manager will request more
executors on every run. This seems to be the main cause of SPARK-20540.

The second problem is that the total number of requested executors is
not tracked by the CoarseGrainedSchedulerBackend. Instead, it calculates
the value based on the current status of 3 variables: the number of
known executors, the number of executors that have been killed, and the
number of pending executors. But, the number of pending executors is
never less than 0, even though there may be more known than requested.
When executors are killed and not replaced, this can cause the request
sent to YARN to be incorrect because there were too many executors due
to the scheduler's state being slightly out of date. This is fixed by tracking
the currently requested size explicitly.

## How was this patch tested?

Existing tests.

Author: Ryan Blue <blue@apache.org>

Closes #17813 from rdblue/SPARK-20540-fix-dynamic-allocation.
2017-05-01 14:48:02 -07:00
Aaditya Ramesh 77bcd77ed5 [SPARK-19525][CORE] Add RDD checkpoint compression support
## What changes were proposed in this pull request?

This PR adds RDD checkpoint compression support and add a new config `spark.checkpoint.compress` to enable/disable it. Credit goes to aramesh117

Closes #17024

## How was this patch tested?

The new unit test.

Author: Shixiong Zhu <shixiong@databricks.com>
Author: Aaditya Ramesh <aramesh@conviva.com>

Closes #17789 from zsxwing/pr17024.
2017-04-28 15:28:56 -07:00
hyukjinkwon 8c911adac5 [SPARK-20465][CORE] Throws a proper exception when any temp directory could not be got
## What changes were proposed in this pull request?

This PR proposes to throw an exception with better message rather than `ArrayIndexOutOfBoundsException` when temp directories could not be created.

Running the commands below:

```bash
./bin/spark-shell --conf spark.local.dir=/NONEXISTENT_DIR_ONE,/NONEXISTENT_DIR_TWO
```

produces ...

**Before**

```
Exception in thread "main" java.lang.ExceptionInInitializerError
        ...
Caused by: java.lang.ArrayIndexOutOfBoundsException: 0
        ...
```

**After**

```
Exception in thread "main" java.lang.ExceptionInInitializerError
        ...
Caused by: java.io.IOException: Failed to get a temp directory under [/NONEXISTENT_DIR_ONE,/NONEXISTENT_DIR_TWO].
        ...
```

## How was this patch tested?

Unit tests in `LocalDirsSuite.scala`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17768 from HyukjinKwon/throws-temp-dir-exception.
2017-04-28 08:49:35 +01:00
Wenchen Fan b90bf520fd [SPARK-12837][CORE] Do not send the name of internal accumulator to executor side
## What changes were proposed in this pull request?

When sending accumulator updates back to driver, the network overhead is pretty big as there are a lot of accumulators, e.g. `TaskMetrics` will send about 20 accumulators everytime, there may be a lot of `SQLMetric` if the query plan is complicated.

Therefore, it's critical to reduce the size of serialized accumulator. A simple way is to not send the name of internal accumulators to executor side, as it's unnecessary. When executor sends accumulator updates back to driver, we can look up the accumulator name in `AccumulatorContext` easily. Note that, we still need to send names of normal accumulators, as the user code run at executor side may rely on accumulator names.

In the future, we should reimplement `TaskMetrics` to not rely on accumulators and use custom serialization.

Tried on the example in https://issues.apache.org/jira/browse/SPARK-12837, the size of serialized accumulator has been cut down by about 40%.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #17596 from cloud-fan/oom.
2017-04-27 19:38:14 -07:00
Shixiong Zhu 01c999e7f9 [SPARK-20461][CORE][SS] Use UninterruptibleThread for Executor and fix the potential hang in CachedKafkaConsumer
## What changes were proposed in this pull request?

This PR changes Executor's threads to `UninterruptibleThread` so that we can use `runUninterruptibly` in `CachedKafkaConsumer`. However, this is just best effort to avoid hanging forever. If the user uses`CachedKafkaConsumer` in another thread (e.g., create a new thread or Future), the potential hang may still happen.

## How was this patch tested?

The new added test.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #17761 from zsxwing/int.
2017-04-27 13:55:03 -07:00
Mark Grover 66636ef0b0 [SPARK-20435][CORE] More thorough redaction of sensitive information
This change does a more thorough redaction of sensitive information from logs and UI
Add unit tests that ensure that no regressions happen that leak sensitive information to the logs.

The motivation for this change was appearance of password like so in `SparkListenerEnvironmentUpdate` in event logs under some JVM configurations:
`"sun.java.command":"org.apache.spark.deploy.SparkSubmit ... --conf spark.executorEnv.HADOOP_CREDSTORE_PASSWORD=secret_password ..."
`
Previously redaction logic was only checking if the key matched the secret regex pattern, it'd redact it's value. That worked for most cases. However, in the above case, the key (sun.java.command) doesn't tell much, so the value needs to be searched. This PR expands the check to check for values as well.

## How was this patch tested?

New unit tests added that ensure that no sensitive information is present in the event logs or the yarn logs. Old unit test in UtilsSuite was modified because the test was asserting that a non-sensitive property's value won't be redacted. However, the non-sensitive value had the literal "secret" in it which was causing it to redact. Simply updating the non-sensitive property's value to another arbitrary value (that didn't have "secret" in it) fixed it.

Author: Mark Grover <mark@apache.org>

Closes #17725 from markgrover/spark-20435.
2017-04-26 17:06:21 -07:00
jerryshao 66dd5b83ff [SPARK-20391][CORE] Rename memory related fields in ExecutorSummay
## What changes were proposed in this pull request?

This is a follow-up of #14617 to make the name of memory related fields more meaningful.

Here  for the backward compatibility, I didn't change `maxMemory` and `memoryUsed` fields.

## How was this patch tested?

Existing UT and local verification.

CC squito and tgravescs .

Author: jerryshao <sshao@hortonworks.com>

Closes #17700 from jerryshao/SPARK-20391.
2017-04-26 09:01:50 -05:00
ding 0a7f5f2798 [SPARK-5484][GRAPHX] Periodically do checkpoint in Pregel
## What changes were proposed in this pull request?

Pregel-based iterative algorithms with more than ~50 iterations begin to slow down and eventually fail with a StackOverflowError due to Spark's lack of support for long lineage chains.

This PR causes Pregel to checkpoint the graph periodically if the checkpoint directory is set.
This PR moves PeriodicGraphCheckpointer.scala from mllib to graphx, moves PeriodicRDDCheckpointer.scala, PeriodicCheckpointer.scala from mllib to core
## How was this patch tested?

unit tests, manual tests
(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)

(If this patch involves UI changes, please attach a screenshot; otherwise, remove this)

Author: ding <ding@localhost.localdomain>
Author: dding3 <ding.ding@intel.com>
Author: Michael Allman <michael@videoamp.com>

Closes #15125 from dding3/cp2_pregel.
2017-04-25 11:20:32 -07:00
jerryshao 5280d93e6e [SPARK-20239][CORE] Improve HistoryServer's ACL mechanism
## What changes were proposed in this pull request?

Current SHS (Spark History Server) two different ACLs:

* ACL of base URL, it is controlled by "spark.acls.enabled" or "spark.ui.acls.enabled", and with this enabled, only user configured with "spark.admin.acls" (or group) or "spark.ui.view.acls" (or group), or the user who started SHS could list all the applications, otherwise none of them can be listed. This will also affect REST APIs which listing the summary of all apps and one app.
* Per application ACL. This is controlled by "spark.history.ui.acls.enabled". With this enabled only history admin user and user/group who ran this app can access the details of this app.

With this two ACLs, we may encounter several unexpected behaviors:

1. if base URL's ACL (`spark.acls.enable`) is enabled but user A has no view permission. User "A" cannot see the app list but could still access details of it's own app.
2. if ACLs of base URL (`spark.acls.enable`) is disabled, then user "A" could download any application's event log, even it is not run by user "A".
3. The changes of Live UI's ACL will affect History UI's ACL which share the same conf file.

The unexpected behaviors is mainly because we have two different ACLs, ideally we should have only one to manage all.

So to improve SHS's ACL mechanism, here in this PR proposed to:

1. Disable "spark.acls.enable" and only use "spark.history.ui.acls.enable" for history server.
2. Check permission for event-log download REST API.

With this PR:

1. Admin user could see/download the list of all applications, as well as application details.
2. Normal user could see the list of all applications, but can only download and check the details of applications accessible to him.

## How was this patch tested?

New UTs are added, also verified in real cluster.

CC tgravescs vanzin please help to review, this PR changes the semantics you did previously. Thanks a lot.

Author: jerryshao <sshao@hortonworks.com>

Closes #17582 from jerryshao/SPARK-20239.
2017-04-24 18:18:59 -07:00
jerryshao 592f5c8934 [SPARK-20172][CORE] Add file permission check when listing files in FsHistoryProvider
## What changes were proposed in this pull request?

In the current Spark's HistoryServer we expected to get `AccessControlException` during listing all the files, but unfortunately it was not worked because we actually doesn't check the access permission and no other calls will throw such exception. What was worse is that this check will be deferred until reading files, which is not necessary and quite verbose, since it will be printed out the exception in every 10 seconds when checking the files.

So here with this fix, we actually check the read permission during listing the files, which could avoid unnecessary file read later on and suppress the verbose log.

## How was this patch tested?

Add unit test to verify.

Author: jerryshao <sshao@hortonworks.com>

Closes #17495 from jerryshao/SPARK-20172.
2017-04-20 16:02:09 -07:00
Eric Liang b2ebadfd55 [SPARK-20358][CORE] Executors failing stage on interrupted exception thrown by cancelled tasks
## What changes were proposed in this pull request?

This was a regression introduced by my earlier PR here: https://github.com/apache/spark/pull/17531

It turns out NonFatal() does not in fact catch InterruptedException.

## How was this patch tested?

Extended cancellation unit test coverage. The first test fails before this patch.

cc JoshRosen mridulm

Author: Eric Liang <ekl@databricks.com>

Closes #17659 from ericl/spark-20358.
2017-04-20 09:55:10 -07:00
Wenchen Fan a7b430b571 [SPARK-15354][FLAKY-TEST] TopologyAwareBlockReplicationPolicyBehavior.Peers in 2 racks
## What changes were proposed in this pull request?

`TopologyAwareBlockReplicationPolicyBehavior.Peers in 2 racks` is failing occasionally: https://spark-tests.appspot.com/test-details?suite_name=org.apache.spark.storage.TopologyAwareBlockReplicationPolicyBehavior&test_name=Peers+in+2+racks.

This is because, when we generate 10 block manager id to test, they may all belong to the same rack, as the rack is randomly picked. This PR fixes this problem by forcing each rack to be picked at least once.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #17624 from cloud-fan/test.
2017-04-13 08:38:24 +08:00
hyukjinkwon ceaf77ae43 [SPARK-18692][BUILD][DOCS] Test Java 8 unidoc build on Jenkins
## What changes were proposed in this pull request?

This PR proposes to run Spark unidoc to test Javadoc 8 build as Javadoc 8 is easily re-breakable.

There are several problems with it:

- It introduces little extra bit of time to run the tests. In my case, it took 1.5 mins more (`Elapsed :[94.8746569157]`). How it was tested is described in "How was this patch tested?".

- > One problem that I noticed was that Unidoc appeared to be processing test sources: if we can find a way to exclude those from being processed in the first place then that might significantly speed things up.

  (see  joshrosen's [comment](https://issues.apache.org/jira/browse/SPARK-18692?focusedCommentId=15947627&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15947627))

To complete this automated build, It also suggests to fix existing Javadoc breaks / ones introduced by test codes as described above.

There fixes are similar instances that previously fixed. Please refer https://github.com/apache/spark/pull/15999 and https://github.com/apache/spark/pull/16013

Note that this only fixes **errors** not **warnings**. Please see my observation https://github.com/apache/spark/pull/17389#issuecomment-288438704 for spurious errors by warnings.

## How was this patch tested?

Manually via `jekyll build` for building tests. Also, tested via running `./dev/run-tests`.

This was tested via manually adding `time.time()` as below:

```diff
     profiles_and_goals = build_profiles + sbt_goals

     print("[info] Building Spark unidoc (w/Hive 1.2.1) using SBT with these arguments: ",
           " ".join(profiles_and_goals))

+    import time
+    st = time.time()
     exec_sbt(profiles_and_goals)
+    print("Elapsed :[%s]" % str(time.time() - st))
```

produces

```
...
========================================================================
Building Unidoc API Documentation
========================================================================
...
[info] Main Java API documentation successful.
...
Elapsed :[94.8746569157]
...

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17477 from HyukjinKwon/SPARK-18692.
2017-04-12 12:38:48 +01:00
Sean Owen a26e3ed5e4 [SPARK-20156][CORE][SQL][STREAMING][MLLIB] Java String toLowerCase "Turkish locale bug" causes Spark problems
## What changes were proposed in this pull request?

Add Locale.ROOT to internal calls to String `toLowerCase`, `toUpperCase`, to avoid inadvertent locale-sensitive variation in behavior (aka the "Turkish locale problem").

The change looks large but it is just adding `Locale.ROOT` (the locale with no country or language specified) to every call to these methods.

## How was this patch tested?

Existing tests.

Author: Sean Owen <sowen@cloudera.com>

Closes #17527 from srowen/SPARK-20156.
2017-04-10 20:11:56 +01:00
Bogdan Raducanu 4f7d49b955 [SPARK-20243][TESTS] DebugFilesystem.assertNoOpenStreams thread race
## What changes were proposed in this pull request?

Synchronize access to openStreams map.

## How was this patch tested?

Existing tests.

Author: Bogdan Raducanu <bogdan@databricks.com>

Closes #17592 from bogdanrdc/SPARK-20243.
2017-04-10 17:34:15 +02:00
jerryshao a4491626ed [SPARK-17019][CORE] Expose on-heap and off-heap memory usage in various places
## What changes were proposed in this pull request?

With [SPARK-13992](https://issues.apache.org/jira/browse/SPARK-13992), Spark supports persisting data into off-heap memory, but the usage of on-heap and off-heap memory is not exposed currently, it is not so convenient for user to monitor and profile, so here propose to expose off-heap memory as well as on-heap memory usage in various places:
1. Spark UI's executor page will display both on-heap and off-heap memory usage.
2. REST request returns both on-heap and off-heap memory.
3. Also this can be gotten from MetricsSystem.
4. Last this usage can be obtained programmatically from SparkListener.

Attach the UI changes:

![screen shot 2016-08-12 at 11 20 44 am](https://cloud.githubusercontent.com/assets/850797/17612032/6c2f4480-607f-11e6-82e8-a27fb8cbb4ae.png)

Backward compatibility is also considered for event-log and REST API. Old event log can still be replayed with off-heap usage displayed as 0. For REST API, only adds the new fields, so JSON backward compatibility can still be kept.
## How was this patch tested?

Unit test added and manual verification.

Author: jerryshao <sshao@hortonworks.com>

Closes #14617 from jerryshao/SPARK-17019.
2017-04-06 13:23:54 -05:00
Eric Liang 5142e5d4e0 [SPARK-20217][CORE] Executor should not fail stage if killed task throws non-interrupted exception
## What changes were proposed in this pull request?

If tasks throw non-interrupted exceptions on kill (e.g. java.nio.channels.ClosedByInterruptException), their death is reported back as TaskFailed instead of TaskKilled. This causes stage failure in some cases.

This is reproducible as follows. Run the following, and then use SparkContext.killTaskAttempt to kill one of the tasks. The entire stage will fail since we threw a RuntimeException instead of InterruptedException.

```
spark.range(100).repartition(100).foreach { i =>
  try {
    Thread.sleep(10000000)
  } catch {
    case t: InterruptedException =>
      throw new RuntimeException(t)
  }
}
```
Based on the code in TaskSetManager, I think this also affects kills of speculative tasks. However, since the number of speculated tasks is few, and usually you need to fail a task a few times before the stage is cancelled, it unlikely this would be noticed in production unless both speculation was enabled and the num allowed task failures was = 1.

We should probably unconditionally return TaskKilled instead of TaskFailed if the task was killed by the driver, regardless of the actual exception thrown.

## How was this patch tested?

Unit test. The test fails before the change in Executor.scala

cc JoshRosen

Author: Eric Liang <ekl@databricks.com>

Closes #17531 from ericl/fix-task-interrupt.
2017-04-05 19:37:21 -07:00
Dilip Biswal 9d68c67235 [SPARK-20204][SQL][FOLLOWUP] SQLConf should react to change in default timezone settings
## What changes were proposed in this pull request?
Make sure SESSION_LOCAL_TIMEZONE reflects the change in JVM's default timezone setting. Currently several timezone related tests fail as the change to default timezone is not picked up by SQLConf.

## How was this patch tested?
Added an unit test in ConfigEntrySuite

Author: Dilip Biswal <dbiswal@us.ibm.com>

Closes #17537 from dilipbiswal/timezone_debug.
2017-04-06 08:33:14 +08:00
Denis Bolshakov fb5869f2cf [SPARK-9002][CORE] KryoSerializer initialization does not include 'Array[Int]'
[SPARK-9002][CORE] KryoSerializer initialization does not include 'Array[Int]'

## What changes were proposed in this pull request?

Array[Int] has been registered in KryoSerializer.
The following file has been changed
core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala

## How was this patch tested?

First, the issue was reproduced by new unit test.
Then, the issue was fixed to pass the failed test.

Author: Denis Bolshakov <denis.bolshakov@onefactor.com>

Closes #17482 from dbolshak/SPARK-9002.
2017-04-03 10:16:07 +01:00
Kent Yao e9d268f63e [SPARK-20096][SPARK SUBMIT][MINOR] Expose the right queue name not null if set by --conf or configure file
## What changes were proposed in this pull request?

while submit apps with -v or --verbose, we can print the right queue name, but if we set a queue name with `spark.yarn.queue` by --conf or in the spark-default.conf, we just got `null`  for the queue in Parsed arguments.
```
bin/spark-shell -v --conf spark.yarn.queue=thequeue
Using properties file: /home/hadoop/spark-2.1.0-bin-apache-hdp2.7.3/conf/spark-defaults.conf
....
Adding default property: spark.yarn.queue=default
Parsed arguments:
  master                  yarn
  deployMode              client
  ...
  queue                   null
  ....
  verbose                 true
Spark properties used, including those specified through
 --conf and those from the properties file /home/hadoop/spark-2.1.0-bin-apache-hdp2.7.3/conf/spark-defaults.conf:
  spark.yarn.queue -> thequeue
  ....
```
## How was this patch tested?

ut and local verify

Author: Kent Yao <yaooqinn@hotmail.com>

Closes #17430 from yaooqinn/SPARK-20096.
2017-03-30 16:11:03 +01:00
Shubham Chopra b454d4402e [SPARK-15354][CORE] Topology aware block replication strategies
## What changes were proposed in this pull request?

Implementations of strategies for resilient block replication for different resource managers that replicate the 3-replica strategy used by HDFS, where the first replica is on an executor, the second replica within the same rack as the executor and a third replica on a different rack.
The implementation involves providing two pluggable classes, one running in the driver that provides topology information for every host at cluster start and the second prioritizing a list of peer BlockManagerIds.

The prioritization itself can be thought of an optimization problem to find a minimal set of peers that satisfy certain objectives and replicating to these peers first. The objectives can be used to express richer constraints over and above HDFS like 3-replica strategy.
## How was this patch tested?

This patch was tested with unit tests for storage, along with new unit tests to verify prioritization behaviour.

Author: Shubham Chopra <schopra31@bloomberg.net>

Closes #13932 from shubhamchopra/PrioritizerStrategy.
2017-03-30 22:21:57 +08:00
jerryshao c622a87c44 [SPARK-20059][YARN] Use the correct classloader for HBaseCredentialProvider
## What changes were proposed in this pull request?

Currently we use system classloader to find HBase jars, if it is specified by `--jars`, then it will be failed with ClassNotFound issue. So here changing to use child classloader.

Also putting added jars and main jar into classpath of submitted application in yarn cluster mode, otherwise HBase jars specified with `--jars` will never be honored in cluster mode, and fetching tokens in client side will always be failed.

## How was this patch tested?

Unit test and local verification.

Author: jerryshao <sshao@hortonworks.com>

Closes #17388 from jerryshao/SPARK-20059.
2017-03-29 10:09:58 -07:00
Marcelo Vanzin b56ad2b1ec [SPARK-19556][CORE] Do not encrypt block manager data in memory.
This change modifies the way block data is encrypted to make the more
common cases faster, while penalizing an edge case. As a side effect
of the change, all data that goes through the block manager is now
encrypted only when needed, including the previous path (broadcast
variables) where that did not happen.

The way the change works is by not encrypting data that is stored in
memory; so if a serialized block is in memory, it will only be encrypted
once it is evicted to disk.

The penalty comes when transferring that encrypted data from disk. If the
data ends up in memory again, it is as efficient as before; but if the
evicted block needs to be transferred directly to a remote executor, then
there's now a performance penalty, since the code now uses a custom
FileRegion implementation to decrypt the data before transferring.

This also means that block data transferred between executors now is
not encrypted (and thus relies on the network library encryption support
for secrecy). Shuffle blocks are still transferred in encrypted form,
since they're handled in a slightly different way by the code. This also
keeps compatibility with existing external shuffle services, which transfer
encrypted shuffle blocks, and avoids having to make the external service
aware of encryption at all.

The serialization and deserialization APIs in the SerializerManager now
do not do encryption automatically; callers need to explicitly wrap their
streams with an appropriate crypto stream before using those.

As a result of these changes, some of the workarounds added in SPARK-19520
are removed here.

Testing: a new trait ("EncryptionFunSuite") was added that provides an easy
way to run a test twice, with encryption on and off; broadcast, block manager
and caching tests were modified to use this new trait so that the existing
tests exercise both encrypted and non-encrypted paths. I also ran some
applications with encryption turned on to verify that they still work,
including streaming tests that failed without the fix for SPARK-19520.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #17295 from vanzin/SPARK-19556.
2017-03-29 20:27:41 +08:00
liujianhui 92e385e0b5 [SPARK-19868] conflict TasksetManager lead to spark stopped
## What changes were proposed in this pull request?

We must set the taskset to zombie before the DAGScheduler handles the taskEnded event. It's possible the taskEnded event will cause the DAGScheduler to launch a new stage attempt (this happens when map output data was lost), and if this happens before the taskSet has been set to zombie, it will appear that we have conflicting task sets.

Author: liujianhui <liujianhui@didichuxing>

Closes #17208 from liujianhuiouc/spark-19868.
2017-03-28 12:13:45 -07:00
Shubham Chopra a250933c62 [SPARK-19803][CORE][TEST] Proactive replication test failures
## What changes were proposed in this pull request?
Executors cache a list of their peers that is refreshed by default every minute. The cached stale references were randomly being used for replication. Since those executors were removed from the master, they did not occur in the block locations as reported by the master. This was fixed by
1. Refreshing peer cache in the block manager before trying to pro-actively replicate. This way the probability of replicating to a failed executor is eliminated.
2. Explicitly stopping the block manager in the tests. This shuts down the RPC endpoint use by the block manager. This way, even if a block manager tries to replicate using a stale reference, the replication logic should take care of refreshing the list of peers after failure.

## How was this patch tested?
Tested manually

Author: Shubham Chopra <schopra31@bloomberg.net>
Author: Kay Ousterhout <kayousterhout@gmail.com>
Author: Shubham Chopra <shubhamchopra@users.noreply.github.com>

Closes #17325 from shubhamchopra/SPARK-19803.
2017-03-28 09:47:29 +08:00
Herman van Hovell 0a6c50711b [SPARK-20070][SQL] Fix 2.10 build
## What changes were proposed in this pull request?
Commit 91fa80fe8a broke the build for scala 2.10. The commit uses `Regex.regex` field which is not available in Scala 2.10. This PR fixes this.

## How was this patch tested?
Existing tests.

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #17420 from hvanhovell/SPARK-20070-2.0.
2017-03-25 01:07:50 +01:00
Herman van Hovell 91fa80fe8a [SPARK-20070][SQL] Redact DataSourceScanExec treeString
## What changes were proposed in this pull request?
The explain output of `DataSourceScanExec` can contain sensitive information (like Amazon keys). Such information should not end up in logs, or be exposed to non privileged users.

This PR addresses this by adding a redaction facility for the `DataSourceScanExec.treeString`. A user can enable this by setting a regex in the `spark.redaction.string.regex` configuration.

## How was this patch tested?
Added a unit test to check the output of DataSourceScanExec.

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #17397 from hvanhovell/SPARK-20070.
2017-03-24 15:52:48 -07:00
Eric Liang 8e558041aa [SPARK-19820][CORE] Add interface to kill tasks w/ a reason
This commit adds a killTaskAttempt method to SparkContext, to allow users to
kill tasks so that they can be re-scheduled elsewhere.

This also refactors the task kill path to allow specifying a reason for the task kill. The reason is propagated opaquely through events, and will show up in the UI automatically as `(N killed: $reason)` and `TaskKilled: $reason`. Without this change, there is no way to provide the user feedback through the UI.

Currently used reasons are "stage cancelled", "another attempt succeeded", and "killed via SparkContext.killTask". The user can also specify a custom reason through `SparkContext.killTask`.

cc rxin

In the stage overview UI the reasons are summarized:
![1](https://cloud.githubusercontent.com/assets/14922/23929209/a83b2862-08e1-11e7-8b3e-ae1967bbe2e5.png)

Within the stage UI you can see individual task kill reasons:
![2](https://cloud.githubusercontent.com/assets/14922/23929200/9a798692-08e1-11e7-8697-72b27ad8a287.png)

Existing tests, tried killing some stages in the UI and verified the messages are as expected.

Author: Eric Liang <ekl@databricks.com>
Author: Eric Liang <ekl@google.com>

Closes #17166 from ericl/kill-reason.
2017-03-23 23:30:44 -07:00
jinxing 19596c28b6 [SPARK-16929] Improve performance when check speculatable tasks.
## What changes were proposed in this pull request?
1. Use a MedianHeap to record durations of successful tasks.  When check speculatable tasks, we can get the median duration with O(1) time complexity.

2. `checkSpeculatableTasks` will synchronize `TaskSchedulerImpl`. If `checkSpeculatableTasks` doesn't finish with 100ms, then the possibility exists for that thread to release and then immediately re-acquire the lock. Change `scheduleAtFixedRate` to be `scheduleWithFixedDelay` when call method of `checkSpeculatableTasks`.
## How was this patch tested?
Added MedianHeapSuite.

Author: jinxing <jinxing6042@126.com>

Closes #16867 from jinxing64/SPARK-16929.
2017-03-23 23:25:56 -07:00
erenavsarogullari b7be05a203 [SPARK-19567][CORE][SCHEDULER] Support some Schedulable variables immutability and access
## What changes were proposed in this pull request?
Some `Schedulable` Entities(`Pool` and `TaskSetManager`) variables need refactoring for _immutability_ and _access modifiers_ levels as follows:
- From `var` to `val` (if there is no requirement): This is important to support immutability as much as possible.
  - Sample => `Pool`: `weight`, `minShare`, `priority`, `name` and `taskSetSchedulingAlgorithm`.
- Access modifiers: Specially, `var`s access needs to be restricted from other parts of codebase to prevent potential side effects.
  - `TaskSetManager`: `tasksSuccessful`, `totalResultSize`, `calculatedTasks` etc...

This PR is related with #15604 and has been created seperatedly to keep patch content as isolated and to help the reviewers.

## How was this patch tested?
Added new UTs and existing UT coverage.

Author: erenavsarogullari <erenavsarogullari@gmail.com>

Closes #16905 from erenavsarogullari/SPARK-19567.
2017-03-23 17:20:52 -07:00
Michael Allman 7fa116f8fc [SPARK-17204][CORE] Fix replicated off heap storage
(Jira: https://issues.apache.org/jira/browse/SPARK-17204)

## What changes were proposed in this pull request?

There are a couple of bugs in the `BlockManager` with respect to support for replicated off-heap storage. First, the locally-stored off-heap byte buffer is disposed of when it is replicated. It should not be. Second, the replica byte buffers are stored as heap byte buffers instead of direct byte buffers even when the storage level memory mode is off-heap. This PR addresses both of these problems.

## How was this patch tested?

`BlockManagerReplicationSuite` was enhanced to fill in the coverage gaps. It now fails if either of the bugs in this PR exist.

Author: Michael Allman <michael@videoamp.com>

Closes #16499 from mallman/spark-17204-replicated_off_heap_storage.
2017-03-21 11:51:22 +08:00
Sital Kedia 7b5d873aef [SPARK-13369] Add config for number of consecutive fetch failures
The previously hardcoded max 4 retries per stage is not suitable for all cluster configurations. Since spark retries a stage at the sign of the first fetch failure, you can easily end up with many stage retries to discover all the failures. In particular, two scenarios this value should change are (1) if there are more than 4 executors per node; in that case, it may take 4 retries to discover the problem with each executor on the node and (2) during cluster maintenance on large clusters, where multiple machines are serviced at once, but you also cannot afford total cluster downtime. By making this value configurable, cluster managers can tune this value to something more appropriate to their cluster configuration.

Unit tests

Author: Sital Kedia <skedia@fb.com>

Closes #17307 from sitalkedia/SPARK-13369.
2017-03-17 09:33:58 -05:00
Bogdan Raducanu ee91a0decc [SPARK-19946][TESTING] DebugFilesystem.assertNoOpenStreams should report the open streams to help debugging
## What changes were proposed in this pull request?

DebugFilesystem.assertNoOpenStreams throws an exception with a cause exception that actually shows the code line which leaked the stream.

## How was this patch tested?
New test in SparkContextSuite to check there is a cause exception.

Author: Bogdan Raducanu <bogdan@databricks.com>

Closes #17292 from bogdanrdc/SPARK-19946.
2017-03-16 15:25:45 +01:00
erenavsarogullari 046b8d4aef [SPARK-18066][CORE][TESTS] Add Pool usage policies test coverage for FIFO & FAIR Schedulers
## What changes were proposed in this pull request?

The following FIFO & FAIR Schedulers Pool usage cases need to have unit test coverage :
- FIFO Scheduler just uses **root pool** so even if `spark.scheduler.pool` property is set, related pool is not created and `TaskSetManagers` are added to **root pool**.
- FAIR Scheduler uses `default pool` when `spark.scheduler.pool` property is not set. This can be happened when
  - `Properties` object is **null**,
  - `Properties` object is **empty**(`new Properties()`),
  - **default pool** is set(`spark.scheduler.pool=default`).
- FAIR Scheduler creates a **new pool** with **default values** when `spark.scheduler.pool` property points a **non-existent** pool. This can be happened when **scheduler allocation file** is not set or it does not contain related pool.
## How was this patch tested?

New Unit tests are added.

Author: erenavsarogullari <erenavsarogullari@gmail.com>

Closes #15604 from erenavsarogullari/SPARK-18066.
2017-03-15 15:57:51 -07:00
jiangxingbo 97cc5e5a55 [SPARK-19960][CORE] Move SparkHadoopWriter to internal/io/
## What changes were proposed in this pull request?

This PR introduces the following changes:
1. Move `SparkHadoopWriter` to `core/internal/io/`, so that it's in the same directory with `SparkHadoopMapReduceWriter`;
2. Move `SparkHadoopWriterUtils` to a separated file.

After this PR is merged, we may consolidate `SparkHadoopWriter` and `SparkHadoopMapReduceWriter`, and make the new commit protocol support the old `mapred` package's committer;

## How was this patch tested?

Tested by existing test cases.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #17304 from jiangxb1987/writer.
2017-03-15 14:58:19 -07:00
Herman van Hovell 9ff85be3bd [SPARK-19889][SQL] Make TaskContext callbacks thread safe
## What changes were proposed in this pull request?
It is sometimes useful to use multiple threads in a task to parallelize tasks. These threads might register some completion/failure listeners to clean up when the task completes or fails. We currently cannot register such a callback and be sure that it will get called, because the context might be in the process of invoking its callbacks, when the the callback gets registered.

This PR improves this by making sure that you cannot add a completion/failure listener from a different thread when the context is being marked as completed/failed in another thread. This is done by synchronizing these methods on the task context itself.

Failure listeners were called only once. Completion listeners now follow the same pattern; this lifts the idempotency requirement for completion listeners and makes it easier to implement them. In some cases we can (accidentally) add a completion/failure listener after the fact, these listeners will be called immediately in order make sure we can safely clean-up after a task.

As a result of this change we could make the `failure` and `completed` flags non-volatile. The `isCompleted()` method now uses synchronization to ensure that updates are visible across threads.

## How was this patch tested?
Adding tests to `TaskContestSuite` to test adding listeners to a completed/failed context.

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #17244 from hvanhovell/SPARK-19889.
2017-03-15 10:46:05 +01:00
jinxing 3232e54f2f [SPARK-19793] Use clock.getTimeMillis when mark task as finished in TaskSetManager.
## What changes were proposed in this pull request?

TaskSetManager is now using `System.getCurrentTimeMillis` when mark task as finished in `handleSuccessfulTask` and `handleFailedTask`. Thus developer cannot set the tasks finishing time in unit test. When `handleSuccessfulTask`, task's duration = `System.getCurrentTimeMillis` - launchTime(which can be set by `clock`), the result is not correct.

## How was this patch tested?
Existing tests.

Author: jinxing <jinxing6042@126.com>

Closes #17133 from jinxing64/SPARK-19793.
2017-03-09 10:56:19 -08:00
uncleGen 49570ed05d [SPARK-19803][TEST] flaky BlockManagerReplicationSuite test failure
## What changes were proposed in this pull request?

200ms may be too short. Give more time for replication to happen and new block be reported to master

## How was this patch tested?

test manully

Author: uncleGen <hustyugm@gmail.com>
Author: dylon <hustyugm@gmail.com>

Closes #17144 from uncleGen/SPARK-19803.
2017-03-07 12:24:53 -08:00
Imran Rashid 12bf832407 [SPARK-19796][CORE] Fix serialization of long property values in TaskDescription
## What changes were proposed in this pull request?

The properties that are serialized with a TaskDescription can have very long values (eg. "spark.job.description" which is set to the full sql statement with the thrift-server).  DataOutputStream.writeUTF() does not work well for long strings, so this changes the way those values are serialized to handle longer strings.

## How was this patch tested?

Updated existing unit test to reproduce the issue.  All unit tests via jenkins.

Author: Imran Rashid <irashid@cloudera.com>

Closes #17140 from squito/SPARK-19796.
2017-03-06 14:06:11 -06:00
Imran Rashid 8417a7ae6c [SPARK-19276][CORE] Fetch Failure handling robust to user error handling
## What changes were proposed in this pull request?

Fault-tolerance in spark requires special handling of shuffle fetch
failures.  The Executor would catch FetchFailedException and send a
special msg back to the driver.

However, intervening user code could intercept that exception, and wrap
it with something else.  This even happens in SparkSQL.  So rather than
checking the thrown exception only, we'll store the fetch failure directly
in the TaskContext, where users can't touch it.

## How was this patch tested?

Added a test case which failed before the fix.  Full test suite via jenkins.

Author: Imran Rashid <irashid@cloudera.com>

Closes #16639 from squito/SPARK-19276.
2017-03-02 16:46:01 -08:00
Patrick Woody 433d9eb615 [SPARK-19631][CORE] OutputCommitCoordinator should not allow commits for already failed tasks
## What changes were proposed in this pull request?

Previously it was possible for there to be a race between a task failure and committing the output of a task. For example, the driver may mark a task attempt as failed due to an executor heartbeat timeout (possibly due to GC), but the task attempt actually ends up coordinating with the OutputCommitCoordinator once the executor recovers and committing its result. This will lead to any retry attempt failing because the task result has already been committed despite the original attempt failing.

This ensures that any previously failed task attempts cannot enter the commit protocol.

## How was this patch tested?

Added a unit test

Author: Patrick Woody <pwoody@palantir.com>

Closes #16959 from pwoody/pw/recordFailuresForCommitter.
2017-03-02 15:55:32 -08:00
GavinGavinNo1 89990a0109 [SPARK-13931] Stage can hang if an executor fails while speculated tasks are running
## What changes were proposed in this pull request?
When function 'executorLost' is invoked in class 'TaskSetManager', it's significant to judge whether variable 'isZombie' is set to true.

This pull request fixes the following hang:

1.Open speculation switch in the application.
2.Run this app and suppose last task of shuffleMapStage 1 finishes. Let's get the record straight, from the eyes of DAG, this stage really finishes, and from the eyes of TaskSetManager, variable 'isZombie' is set to true, but variable runningTasksSet isn't empty because of speculation.
3.Suddenly, executor 3 is lost. TaskScheduler receiving this signal, invokes all executorLost functions of rootPool's taskSetManagers. DAG receiving this signal, removes all this executor's outputLocs.
4.TaskSetManager adds all this executor's tasks to pendingTasks and tells DAG they will be resubmitted (Attention: possibly not on time).
5.DAG starts to submit a new waitingStage, let's say shuffleMapStage 2, and going to find that shuffleMapStage 1 is its missing parent because some outputLocs are removed due to executor lost. Then DAG submits shuffleMapStage 1 again.
6.DAG still receives Task 'Resubmitted' signal from old taskSetManager, and increases the number of pendingTasks of shuffleMapStage 1 each time. However, old taskSetManager won't resolve new task to submit because its variable 'isZombie' is set to true.
7.Finally shuffleMapStage 1 never finishes in DAG together with all stages depending on it.

## How was this patch tested?

It's quite difficult to construct test cases.

Author: GavinGavinNo1 <gavingavinno1@gmail.com>
Author: 16092929 <16092929@cnsuning.com>

Closes #16855 from GavinGavinNo1/resolve-stage-blocked2.
2017-03-01 21:40:41 -08:00
Yuming Wang 9b8eca65dc [SPARK-19660][CORE][SQL] Replace the configuration property names that are deprecated in the version of Hadoop 2.6
## What changes were proposed in this pull request?

Replace all the Hadoop deprecated configuration property names according to [DeprecatedProperties](https://hadoop.apache.org/docs/r2.6.0/hadoop-project-dist/hadoop-common/DeprecatedProperties.html).

except:
https://github.com/apache/spark/blob/v2.1.0/python/pyspark/sql/tests.py#L1533
https://github.com/apache/spark/blob/v2.1.0/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala#L987
https://github.com/apache/spark/blob/v2.1.0/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala#L45
https://github.com/apache/spark/blob/v2.1.0/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala#L614

## How was this patch tested?

Existing tests

Author: Yuming Wang <wgyumg@gmail.com>

Closes #16990 from wangyum/HadoopDeprecatedProperties.
2017-02-28 10:13:42 +00:00
hyukjinkwon 4ba9c6c453 [MINOR][BUILD] Fix lint-java breaks in Java
## What changes were proposed in this pull request?

This PR proposes to fix the lint-breaks as below:

```
[ERROR] src/test/java/org/apache/spark/network/TransportResponseHandlerSuite.java:[29,8] (imports) UnusedImports: Unused import - org.apache.spark.network.buffer.ManagedBuffer.
[ERROR] src/main/java/org/apache/spark/unsafe/types/UTF8String.java:[156,10] (modifier) ModifierOrder: 'Nonnull' annotation modifier does not precede non-annotation modifiers.
[ERROR] src/main/java/org/apache/spark/SparkFirehoseListener.java:[122] (sizes) LineLength: Line is longer than 100 characters (found 105).
[ERROR] src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java:[164,78] (coding) OneStatementPerLine: Only one statement per line allowed.
[ERROR] src/test/java/test/org/apache/spark/JavaAPISuite.java:[1157] (sizes) LineLength: Line is longer than 100 characters (found 121).
[ERROR] src/test/java/org/apache/spark/streaming/JavaMapWithStateSuite.java:[149] (sizes) LineLength: Line is longer than 100 characters (found 113).
[ERROR] src/test/java/test/org/apache/spark/streaming/Java8APISuite.java:[146] (sizes) LineLength: Line is longer than 100 characters (found 122).
[ERROR] src/test/java/test/org/apache/spark/streaming/JavaAPISuite.java:[32,8] (imports) UnusedImports: Unused import - org.apache.spark.streaming.Time.
[ERROR] src/test/java/test/org/apache/spark/streaming/JavaAPISuite.java:[611] (sizes) LineLength: Line is longer than 100 characters (found 101).
[ERROR] src/test/java/test/org/apache/spark/streaming/JavaAPISuite.java:[1317] (sizes) LineLength: Line is longer than 100 characters (found 102).
[ERROR] src/test/java/test/org/apache/spark/sql/JavaDatasetAggregatorSuite.java:[91] (sizes) LineLength: Line is longer than 100 characters (found 102).
[ERROR] src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java:[113] (sizes) LineLength: Line is longer than 100 characters (found 101).
[ERROR] src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java:[164] (sizes) LineLength: Line is longer than 100 characters (found 110).
[ERROR] src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java:[212] (sizes) LineLength: Line is longer than 100 characters (found 114).
[ERROR] src/test/java/org/apache/spark/mllib/tree/JavaDecisionTreeSuite.java:[36] (sizes) LineLength: Line is longer than 100 characters (found 101).
[ERROR] src/main/java/org/apache/spark/examples/streaming/JavaKinesisWordCountASL.java:[26,8] (imports) UnusedImports: Unused import - com.amazonaws.regions.RegionUtils.
[ERROR] src/test/java/org/apache/spark/streaming/kinesis/JavaKinesisStreamSuite.java:[20,8] (imports) UnusedImports: Unused import - com.amazonaws.regions.RegionUtils.
[ERROR] src/test/java/org/apache/spark/streaming/kinesis/JavaKinesisStreamSuite.java:[94] (sizes) LineLength: Line is longer than 100 characters (found 103).
[ERROR] src/main/java/org/apache/spark/examples/ml/JavaTokenizerExample.java:[30,8] (imports) UnusedImports: Unused import - org.apache.spark.sql.api.java.UDF1.
[ERROR] src/main/java/org/apache/spark/examples/ml/JavaTokenizerExample.java:[72] (sizes) LineLength: Line is longer than 100 characters (found 104).
[ERROR] src/main/java/org/apache/spark/examples/mllib/JavaRankingMetricsExample.java:[121] (sizes) LineLength: Line is longer than 100 characters (found 101).
[ERROR] src/main/java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java:[28,8] (imports) UnusedImports: Unused import - org.apache.spark.api.java.JavaRDD.
[ERROR] src/main/java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java:[29,8] (imports) UnusedImports: Unused import - org.apache.spark.api.java.JavaSparkContext.
```

## How was this patch tested?

Manually via

```bash
./dev/lint-java
```

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17072 from HyukjinKwon/java-lint.
2017-02-27 08:44:26 +00:00
Shubham Chopra fa7c582e94 [SPARK-15355][CORE] Proactive block replication
## What changes were proposed in this pull request?

We are proposing addition of pro-active block replication in case of executor failures. BlockManagerMasterEndpoint does all the book-keeping to keep a track of all the executors and the blocks they hold. It also keeps a track of which executors are alive through heartbeats. When an executor is removed, all this book-keeping state is updated to reflect the lost executor. This step can be used to identify executors that are still in possession of a copy of the cached data and a message could be sent to them to use the existing "replicate" function to find and place new replicas on other suitable hosts. Blocks replicated this way will let the master know of their existence.

This can happen when an executor is lost, and would that way be pro-active as opposed be being done at query time.
## How was this patch tested?

This patch was tested with existing unit tests along with new unit tests added to test the functionality.

Author: Shubham Chopra <schopra31@bloomberg.net>

Closes #14412 from shubhamchopra/ProactiveBlockReplication.
2017-02-24 15:40:01 -08:00
Imran Rashid 5f74148bb4 [SPARK-19597][CORE] test case for task deserialization errors
Adds a test case that ensures that Executors gracefully handle a task that fails to deserialize, by sending back a reasonable failure message.  This does not change any behavior (the prior behavior was already correct), it just adds a test case to prevent regression.

Author: Imran Rashid <irashid@cloudera.com>

Closes #16930 from squito/executor_task_deserialization.
2017-02-24 13:03:37 -08:00
Kay Ousterhout 5cbd3b59ba [SPARK-19560] Improve DAGScheduler tests.
This commit improves the tests that check the case when a
ShuffleMapTask completes successfully on an executor that has
failed.  This commit improves the commenting around the existing
test for this, and adds some additional checks to make it more
clear what went wrong if the tests fail (the fact that these
tests are hard to understand came up in the context of markhamstra's
proposed fix for #16620).

This commit also removes a test that I realized tested exactly
the same functionality.

markhamstra, I verified that the new version of the test still fails (and
in a more helpful way) for your proposed change for #16620.

Author: Kay Ousterhout <kayousterhout@gmail.com>

Closes #16892 from kayousterhout/SPARK-19560.
2017-02-24 11:42:45 -08:00
wangzhenhua 69d0da6373 [SPARK-17078][SQL] Show stats when explain
## What changes were proposed in this pull request?

Currently we can only check the estimated stats in logical plans by debugging. We need to provide an easier and more efficient way for developers/users.

In this pr, we add EXPLAIN COST command to show stats in the optimized logical plan.
E.g.
```
spark-sql> EXPLAIN COST select count(1) from store_returns;

...
== Optimized Logical Plan ==
Aggregate [count(1) AS count(1)#24L], Statistics(sizeInBytes=16.0 B, rowCount=1, isBroadcastable=false)
+- Project, Statistics(sizeInBytes=4.3 GB, rowCount=5.76E+8, isBroadcastable=false)
   +- Relation[sr_returned_date_sk#3,sr_return_time_sk#4,sr_item_sk#5,sr_customer_sk#6,sr_cdemo_sk#7,sr_hdemo_sk#8,sr_addr_sk#9,sr_store_sk#10,sr_reason_sk#11,sr_ticket_number#12,sr_return_quantity#13,sr_return_amt#14,sr_return_tax#15,sr_return_amt_inc_tax#16,sr_fee#17,sr_return_ship_cost#18,sr_refunded_cash#19,sr_reversed_charge#20,sr_store_credit#21,sr_net_loss#22] parquet, Statistics(sizeInBytes=28.6 GB, rowCount=5.76E+8, isBroadcastable=false)
...
```

## How was this patch tested?

Add test cases.

Author: wangzhenhua <wangzhenhua@huawei.com>
Author: Zhenhua Wang <wzh_zju@163.com>

Closes #16594 from wzhfy/showStats.
2017-02-24 10:24:59 -08:00
jerryshao b0a8c16fec [SPARK-19707][CORE] Improve the invalid path check for sc.addJar
## What changes were proposed in this pull request?

Currently in Spark there're two issues when we add jars with invalid path:

* If the jar path is a empty string {--jar ",dummy.jar"}, then Spark will resolve it to the current directory path and add to classpath / file server, which is unwanted. This is happened in our programatic way to submit Spark application. From my understanding Spark should defensively filter out such empty path.
* If the jar path is a invalid path (file doesn't exist), `addJar` doesn't check it and will still add to file server, the exception will be delayed until job running. Actually this local path could be checked beforehand, no need to wait until task running. We have similar check in `addFile`, but lacks similar similar mechanism in `addJar`.

## How was this patch tested?

Add unit test and local manual verification.

Author: jerryshao <sshao@hortonworks.com>

Closes #17038 from jerryshao/SPARK-19707.
2017-02-24 09:28:59 -08:00
Marcelo Vanzin 17d83e1ee5 [SPARK-19652][UI] Do auth checks for REST API access.
The REST API has a security filter that performs auth checks
based on the UI root's security manager. That works fine when
the UI root is the app's UI, but not when it's the history server.

In the SHS case, all users would be allowed to see all applications
through the REST API, even if the UI itself wouldn't be available
to them.

This change adds auth checks for each app access through the API
too, so that only authorized users can see the app's data.

The change also modifies the existing security filter to use
`HttpServletRequest.getRemoteUser()`, which is used in other
places. That is not necessarily the same as the principal's
name; for example, when using Hadoop's SPNEGO auth filter,
the remote user strips the realm information, which then matches
the user name registered as the owner of the application.

I also renamed the UIRootFromServletContext trait to a more generic
name since I'm using it to store more context information now.

Tested manually with an authentication filter enabled.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #16978 from vanzin/SPARK-19652.
2017-02-21 16:14:34 -08:00
hyukjinkwon 17b93b5feb
[SPARK-18922][TESTS] Fix new test failures on Windows due to path and resource not closed
## What changes were proposed in this pull request?

This PR proposes to fix new test failures on WIndows as below:

**Before**

```
KafkaRelationSuite:
 - test late binding start offsets *** FAILED *** (7 seconds, 679 milliseconds)
   Cause: java.nio.file.FileSystemException: C:\projects\spark\target\tmp\spark-4c4b0cd1-4cb7-4908-949d-1b0cc8addb50\topic-4-0\00000000000000000000.log -> C:\projects\spark\target\tmp\spark-4c4b0cd1-4cb7-4908-949d-1b0cc8addb50\topic-4-0\00000000000000000000.log.deleted: The process cannot access the file because it is being used by another process.

KafkaSourceSuite:
 - deserialization of initial offset with Spark 2.1.0 *** FAILED *** (3 seconds, 542 milliseconds)
   java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-97ef64fc-ae61-4ce3-ac59-287fd38bd824

 - deserialization of initial offset written by Spark 2.1.0 *** FAILED *** (60 milliseconds)
   java.nio.file.InvalidPathException: Illegal char <:> at index 2: /C:/projects/spark/external/kafka-0-10-sql/target/scala-2.11/test-classes/kafka-source-initial-offset-version-2.1.0.b

HiveDDLSuite:
 - partitioned table should always put partition columns at the end of table schema *** FAILED *** (657 milliseconds)
   org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-f1b83d09-850a-4bba-8e43-a2a28dfaa757;

DDLSuite:
 - create a data source table without schema *** FAILED *** (94 milliseconds)
   org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-a3f3c161-afae-4d6f-9182-e8642f77062b;

 - SET LOCATION for managed table *** FAILED *** (219 milliseconds)
   org.apache.spark.sql.catalyst.errors.package$TreeNodeException: execute, tree:
 Exchange SinglePartit
 +- *HashAggregate(keys=[], functions=[partial_count(1)], output=[count#99367L])
    +- *FileScan parquet default.tbl[] Batched: true, Format: Parquet, Location: InMemoryFileIndex[file:/C:projectsspark	arget	mpspark-15be2f2f-4ea9-4c47-bfee-1b7b49363033], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<>

 - insert data to a data source table which has a not existed location should succeed *** FAILED *** (16 milliseconds)
   org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-34987671-e8d1-4624-ba5b-db1012e1246b;

 - insert into a data source table with no existed partition location should succeed *** FAILED *** (16 milliseconds)
   org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-4c6ccfbf-4091-4032-9fbc-3d40c58267d5;

 - read data from a data source table which has a not existed location should succeed *** FAILED *** (0 milliseconds)

 - read data from a data source table with no existed partition location should succeed *** FAILED *** (0 milliseconds)
   org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-6af39e37-abd1-44e8-ac68-e2dfcf67a2f3;

InputOutputMetricsSuite:
 - output metrics on records written *** FAILED *** (0 milliseconds)
   java.lang.IllegalArgumentException: Wrong FS: file://C:\projects\spark\target\tmp\spark-cd69ee77-88f2-4202-bed6-19c0ee05ef55\InputOutputMetricsSuite, expected: file:///

 - output metrics on records written - new Hadoop API *** FAILED *** (16 milliseconds)
   java.lang.IllegalArgumentException: Wrong FS: file://C:\projects\spark\target\tmp\spark-b69e8fcb-047b-4de8-9cdf-5f026efb6762\InputOutputMetricsSuite, expected: file:///
```

**After**

```
KafkaRelationSuite:
 - test late binding start offsets !!! CANCELED !!! (62 milliseconds)

KafkaSourceSuite:
 - deserialization of initial offset with Spark 2.1.0 (5 seconds, 341 milliseconds)
 - deserialization of initial offset written by Spark 2.1.0 (910 milliseconds)

HiveDDLSuite:
 - partitioned table should always put partition columns at the end of table schema (2 seconds)

DDLSuite:
 - create a data source table without schema (828 milliseconds)
 - SET LOCATION for managed table (406 milliseconds)
 - insert data to a data source table which has a not existed location should succeed (406 milliseconds)
 - insert into a data source table with no existed partition location should succeed (453 milliseconds)
 - read data from a data source table which has a not existed location should succeed (94 milliseconds)
 - read data from a data source table with no existed partition location should succeed (265 milliseconds)

InputOutputMetricsSuite:
 - output metrics on records written (172 milliseconds)
 - output metrics on records written - new Hadoop API (297 milliseconds)
```

## How was this patch tested?

Fixed tests in `InputOutputMetricsSuite`, `KafkaRelationSuite`,  `KafkaSourceSuite`, `DDLSuite.scala` and `HiveDDLSuite`.

Manually tested via AppVeyor as below:

`InputOutputMetricsSuite`: https://ci.appveyor.com/project/spark-test/spark/build/633-20170219-windows-test/job/ex8nvwa6tsh7rmto
`KafkaRelationSuite`: https://ci.appveyor.com/project/spark-test/spark/build/633-20170219-windows-test/job/h8dlcowew52y8ncw
`KafkaSourceSuite`: https://ci.appveyor.com/project/spark-test/spark/build/634-20170219-windows-test/job/9ybgjl7yeubxcre4
`DDLSuite`: https://ci.appveyor.com/project/spark-test/spark/build/635-20170219-windows-test
`HiveDDLSuite`: https://ci.appveyor.com/project/spark-test/spark/build/633-20170219-windows-test/job/up6o9n47er087ltb

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16999 from HyukjinKwon/windows-fix.
2017-02-20 21:26:54 -08:00
Sean Owen d0ecca6075
[SPARK-19646][CORE][STREAMING] binaryRecords replicates records in scala API
## What changes were proposed in this pull request?

Use `BytesWritable.copyBytes`, not `getBytes`, because `getBytes` returns the underlying array, which may be reused when repeated reads don't need a different size, as is the case with binaryRecords APIs

## How was this patch tested?

Existing tests

Author: Sean Owen <sowen@cloudera.com>

Closes #16974 from srowen/SPARK-19646.
2017-02-20 09:02:09 -08:00
Sean Owen 1487c9af20
[SPARK-19534][TESTS] Convert Java tests to use lambdas, Java 8 features
## What changes were proposed in this pull request?

Convert tests to use Java 8 lambdas, and modest related fixes to surrounding code.

## How was this patch tested?

Jenkins tests

Author: Sean Owen <sowen@cloudera.com>

Closes #16964 from srowen/SPARK-19534.
2017-02-19 09:42:50 -08:00
jinxing ba8912e5f3
[SPARK-19450] Replace askWithRetry with askSync.
## What changes were proposed in this pull request?

`askSync` is already added in `RpcEndpointRef` (see SPARK-19347 and https://github.com/apache/spark/pull/16690#issuecomment-276850068) and `askWithRetry` is marked as deprecated.
As mentioned SPARK-18113(https://github.com/apache/spark/pull/16503#event-927953218):

>askWithRetry is basically an unneeded API, and a leftover from the akka days that doesn't make sense anymore. It's prone to cause deadlocks (exactly because it's blocking), it imposes restrictions on the caller (e.g. idempotency) and other things that people generally don't pay that much attention to when using it.

Since `askWithRetry` is just used inside spark and not in user logic. It might make sense to replace all of them with `askSync`.

## How was this patch tested?
This PR doesn't change code logic, existing unit test can cover.

Author: jinxing <jinxing@meituan.com>

Closes #16790 from jinxing64/SPARK-19450.
2017-02-19 04:34:07 -08:00
jinxing 729ce37032 [SPARK-19263] DAGScheduler should avoid sending conflicting task set.
In current `DAGScheduler handleTaskCompletion` code, when event.reason is `Success`, it will first do `stage.pendingPartitions -= task.partitionId`, which maybe a bug when `FetchFailed` happens.

**Think about below**

1.  Stage 0 runs and generates shuffle output data.
2. Stage 1 reads the output from stage 0 and generates more shuffle data. It has two tasks: ShuffleMapTask1 and ShuffleMapTask2, and these tasks are launched on executorA.
3. ShuffleMapTask1 fails to fetch blocks locally and sends a FetchFailed to the driver. The driver marks executorA as lost and updates failedEpoch;
4. The driver resubmits stage 0 so the missing output can be re-generated, and then once it completes, resubmits stage 1 with ShuffleMapTask1x and ShuffleMapTask2x.
5. ShuffleMapTask2 (from the original attempt of stage 1) successfully finishes on executorA and sends Success back to driver. This causes DAGScheduler::handleTaskCompletion to remove partition 2 from stage.pendingPartitions (line 1149), but it does not add the partition to the set of output locations (line 1192), because the task’s epoch is less than the failure epoch for the executor (because of the earlier failure on executor A)
6. ShuffleMapTask1x successfully finishes on executorB, causing the driver to remove partition 1 from stage.pendingPartitions. Combined with the previous step, this means that there are no more pending partitions for the stage, so the DAGScheduler marks the stage as finished (line 1196). However, the shuffle stage is not available (line 1215) because the completion for ShuffleMapTask2 was ignored because of its epoch, so the DAGScheduler resubmits the stage.
7. ShuffleMapTask2x is still running, so when TaskSchedulerImpl::submitTasks is called for the re-submitted stage, it throws an error, because there’s an existing active task set

**In this fix**

If a task completion is from a previous stage attempt and the epoch is too low
(i.e., it was from a failed executor), don't remove the corresponding partition
from pendingPartitions.

Author: jinxing <jinxing@meituan.com>
Author: jinxing <jinxing6042@126.com>

Closes #16620 from jinxing64/SPARK-19263.
2017-02-18 10:55:18 -04:00
Liang-Chi Hsieh 4cc06f4eb1 [SPARK-18986][CORE] ExternalAppendOnlyMap shouldn't fail when forced to spill before calling its iterator
## What changes were proposed in this pull request?

`ExternalAppendOnlyMap.forceSpill` now uses an assert to check if an iterator is not null in the map. However, the assertion is only true after the map is asked for iterator. Before it, if another memory consumer asks more memory than currently available, `ExternalAppendOnlyMap.forceSpill` is also be called too. In this case, we will see failure like this:

    [info]   java.lang.AssertionError: assertion failed
    [info]   at scala.Predef$.assert(Predef.scala:156)
    [info]   at org.apache.spark.util.collection.ExternalAppendOnlyMap.forceSpill(ExternalAppendOnlyMap.scala:196)
    [info]   at org.apache.spark.util.collection.Spillable.spill(Spillable.scala:111)
    [info]   at org.apache.spark.util.collection.ExternalAppendOnlyMapSuite$$anonfun$13.apply$mcV$sp(ExternalAppendOnlyMapSuite.scala:294)

This fixing is motivated by http://apache-spark-developers-list.1001551.n3.nabble.com/java-lang-AssertionError-assertion-failed-tc20277.html.

## How was this patch tested?

Jenkins tests.

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

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

Closes #16387 from viirya/fix-externalappendonlymap.
2017-02-17 11:28:16 -08:00
Sean Owen 0e2405490f
[SPARK-19550][BUILD][CORE][WIP] Remove Java 7 support
- Move external/java8-tests tests into core, streaming, sql and remove
- Remove MaxPermGen and related options
- Fix some reflection / TODOs around Java 8+ methods
- Update doc references to 1.7/1.8 differences
- Remove Java 7/8 related build profiles
- Update some plugins for better Java 8 compatibility
- Fix a few Java-related warnings

For the future:

- Update Java 8 examples to fully use Java 8
- Update Java tests to use lambdas for simplicity
- Update Java internal implementations to use lambdas

## How was this patch tested?

Existing tests

Author: Sean Owen <sowen@cloudera.com>

Closes #16871 from srowen/SPARK-19493.
2017-02-16 12:32:45 +00:00
Ala Luszczak d785217b79 [SPARK-19549] Allow providing reason for stage/job cancelling
## What changes were proposed in this pull request?

This change add an optional argument to `SparkContext.cancelStage()` and `SparkContext.cancelJob()` functions, which allows the caller to provide exact reason  for the cancellation.

## How was this patch tested?

Adds unit test.

Author: Ala Luszczak <ala@databricks.com>

Closes #16887 from ala/cancel.
2017-02-10 21:10:02 +01:00
jinxing fd6c3a0b10 [SPARK-19263] Fix race in SchedulerIntegrationSuite.
## What changes were proposed in this pull request?

All the process of offering resource and generating `TaskDescription` should be guarded by taskScheduler.synchronized in `reviveOffers`, otherwise there is race condition.

## How was this patch tested?

Existing unit tests.

Author: jinxing <jinxing@meituan.com>

Closes #16831 from jinxing64/SPARK-19263-FixRaceInTest.
2017-02-09 16:05:44 -08:00
José Hiram Soltren 6287c94f08 [SPARK-16554][CORE] Automatically Kill Executors and Nodes when they are Blacklisted
## What changes were proposed in this pull request?

In SPARK-8425, we introduced a mechanism for blacklisting executors and nodes (hosts). After a certain number of failures, these resources would be "blacklisted" and no further work would be assigned to them for some period of time.

In some scenarios, it is better to fail fast, and to simply kill these unreliable resources. This changes proposes to do so by having the BlacklistTracker kill unreliable resources when they would otherwise be "blacklisted".

In order to be thread safe, this code depends on the CoarseGrainedSchedulerBackend sending a message to the driver backend in order to do the actual killing. This also helps to prevent a race which would permit work to begin on a resource (executor or node), between the time the resource is marked for killing and the time at which it is finally killed.

## How was this patch tested?

./dev/run-tests
Ran https://github.com/jsoltren/jose-utils/blob/master/blacklist/test-blacklist.sh, and checked logs to see executors and nodes being killed.

Testing can likely be improved here; suggestions welcome.

Author: José Hiram Soltren <jose@cloudera.com>

Closes #16650 from jsoltren/SPARK-16554-submit.
2017-02-09 12:49:31 -06:00
Marcelo Vanzin 3fc8e8caf8 [SPARK-17874][CORE] Add SSL port configuration.
Make the SSL port configuration explicit, instead of deriving it
from the non-SSL port, but retain the existing functionality in
case anyone depends on it.

The change starts the HTTPS and HTTP connectors separately, so
that it's possible to use independent ports for each. For that to
work, the initialization of the server needs to be shuffled around
a bit. The change also makes it so the initialization of both
connectors is similar, and end up using the same Scheduler - previously
only the HTTP connector would use the correct one.

Also fixed some outdated documentation about a couple of services
that were removed long ago.

Tested with unit tests and by running spark-shell with SSL configs.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #16625 from vanzin/SPARK-17874.
2017-02-09 22:06:46 +09:00
Liwei Lin 9d9d67c795 [SPARK-19265][SQL][FOLLOW-UP] Configurable tableRelationCache maximum size
## What changes were proposed in this pull request?

SPARK-19265 had made table relation cache general; this follow-up aims to make `tableRelationCache`'s maximum size configurable.

In order to do sanity-check, this patch also adds a `checkValue()` method to `TypedConfigBuilder`.

## How was this patch tested?

new test case: `test("conf entry: checkValue()")`

Author: Liwei Lin <lwlin7@gmail.com>

Closes #16736 from lw-lin/conf.
2017-02-09 00:48:47 -05:00
Sean Owen e8d3fca450
[SPARK-19464][CORE][YARN][TEST-HADOOP2.6] Remove support for Hadoop 2.5 and earlier
## What changes were proposed in this pull request?

- Remove support for Hadoop 2.5 and earlier
- Remove reflection and code constructs only needed to support multiple versions at once
- Update docs to reflect newer versions
- Remove older versions' builds and profiles.

## How was this patch tested?

Existing tests

Author: Sean Owen <sowen@cloudera.com>

Closes #16810 from srowen/SPARK-19464.
2017-02-08 12:20:07 +00:00
zuotingbing 8fd178d215
[SPARK-19260] Spaces or "%20" in path parameter are not correctly handled with…
JIRA Issue: https://issues.apache.org/jira/browse/SPARK-19260

## What changes were proposed in this pull request?

1. “spark.history.fs.logDirectory” supports with space character and “%20” characters.
2. As usually, if the run classpath includes hdfs-site.xml and core-site.xml files, the supplied path eg."/test" which does not contain a scheme should be taken as a HDFS path rather than a local path since the path parameter is a Hadoop dir.

## How was this patch tested?
Update Unit Test and take some manual tests

local:
.sbin/start-history-server.sh "file:/a b"
.sbin/start-history-server.sh "/abc%20c" (without hdfs-site.xml,core-site.xml)
.sbin/start-history-server.sh "/a b" (without hdfs-site.xml,core-site.xml)
.sbin/start-history-server.sh "/a b/a bc%20c" (without hdfs-site.xml,core-site.xml)

hdfs:
.sbin/start-history-server.sh "hdfs:/namenode:9000/a b"
.sbin/start-history-server.sh "/a b" (with hdfs-site.xml,core-site.xml)
.sbin/start-history-server.sh "/a b/a bc%20c" (with hdfs-site.xml,core-site.xml)

Author: zuotingbing <zuo.tingbing9@zte.com.cn>

Closes #16614 from zuotingbing/SPARK-19260.
2017-02-07 12:21:36 +00:00
Imran Rashid d9043092ca [SPARK-18967][SCHEDULER] compute locality levels even if delay = 0
## What changes were proposed in this pull request?

Before this change, with delay scheduling off, spark would effectively
ignore locality preferences for bulk scheduling.  With this change,
locality preferences are used when multiple offers are made
simultaneously.

## How was this patch tested?

Test case added which fails without this change.  All unit tests run via jenkins.

Author: Imran Rashid <irashid@cloudera.com>

Closes #16376 from squito/locality_without_delay.
2017-02-06 22:37:37 -08:00
erenavsarogullari 7beb227cc8 [SPARK-17663][CORE] SchedulableBuilder should handle invalid data access via scheduler.allocation.file
## What changes were proposed in this pull request?

If `spark.scheduler.allocation.file` has invalid `minShare` or/and `weight` values, these cause :
- `NumberFormatException` due to `toInt` function
- `SparkContext` can not be initialized.
- It does not show meaningful error message to user.

In a nutshell, this functionality can be more robust by selecting one of the following flows :

**1-** Currently, if `schedulingMode` has an invalid value, a warning message is logged and default value is set as `FIFO`. Same pattern can be used for `minShare`(default: 0) and `weight`(default: 1) as well
**2-** Meaningful error message can be shown to the user for all invalid cases.

PR offers :
- `schedulingMode` handles just empty values. It also needs to be supported for **whitespace**, **non-uppercase**(fair, FaIr etc...) or `SchedulingMode.NONE` cases by setting default value(`FIFO`)
- `minShare` and `weight` handle just empty values. They also need to be supported for **non-integer** cases by setting default values.
- Some refactoring of `PoolSuite`.

**Code to Reproduce :**

```
val conf = new SparkConf().setAppName("spark-fairscheduler").setMaster("local")
conf.set("spark.scheduler.mode", "FAIR")
conf.set("spark.scheduler.allocation.file", "src/main/resources/fairscheduler-invalid-data.xml")
val sc = new SparkContext(conf)
```

**fairscheduler-invalid-data.xml :**

```
<allocations>
    <pool name="production">
        <schedulingMode>FIFO</schedulingMode>
        <weight>invalid_weight</weight>
        <minShare>2</minShare>
    </pool>
</allocations>
```

**Stacktrace :**

```
Exception in thread "main" java.lang.NumberFormatException: For input string: "invalid_weight"
    at java.lang.NumberFormatException.forInputString(NumberFormatException.java:65)
    at java.lang.Integer.parseInt(Integer.java:580)
    at java.lang.Integer.parseInt(Integer.java:615)
    at scala.collection.immutable.StringLike$class.toInt(StringLike.scala:272)
    at scala.collection.immutable.StringOps.toInt(StringOps.scala:29)
    at org.apache.spark.scheduler.FairSchedulableBuilder$$anonfun$org$apache$spark$scheduler$FairSchedulableBuilder$$buildFairSchedulerPool$1.apply(SchedulableBuilder.scala:127)
    at org.apache.spark.scheduler.FairSchedulableBuilder$$anonfun$org$apache$spark$scheduler$FairSchedulableBuilder$$buildFairSchedulerPool$1.apply(SchedulableBuilder.scala:102)
```
## How was this patch tested?

Added Unit Test Case.

Author: erenavsarogullari <erenavsarogullari@gmail.com>

Closes #15237 from erenavsarogullari/SPARK-17663.
2017-02-06 08:24:17 -06:00
Liang-Chi Hsieh 2f523fa0c9 [SPARK-19244][CORE] Sort MemoryConsumers according to their memory usage when spilling
## What changes were proposed in this pull request?

In `TaskMemoryManager `, when we acquire memory by calling `acquireExecutionMemory` and we can't acquire required memory, we will try to spill other memory consumers.

Currently, we simply iterates the memory consumers in a hash set. Normally each time the consumer will be iterated in the same order.

The first issue is that we might spill additional consumers. For example, if consumer 1 uses 10MB, consumer 2 uses 50MB, then consumer 3 acquires 100MB but we can only get 60MB and spilling is needed. We might spill both consumer 1 and consumer 2. But we actually just need to spill consumer 2 and get the required 100MB.

The second issue is that if we spill consumer 1 in first time spilling. After a while, consumer 1 now uses 5MB. Then consumer 4 may acquire some memory and spilling is needed again. Because we iterate the memory consumers in the same order, we will spill consumer 1 again. So for consumer 1, we will produce many small spilling files.

This patch modifies the way iterating the memory consumers. It sorts the memory consumers by their memory usage. So the consumer using more memory will spill first. Once it is spilled, even it acquires few memory again, in next time spilling happens it will not be the consumers to spill again if there are other consumers using more memory than it.

## How was this patch tested?

Jenkins tests.

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

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

Closes #16603 from viirya/sort-memoryconsumer-when-spill.
2017-02-03 06:14:10 -08:00
jinxing c86a57f4d1 [SPARK-19437] Rectify spark executor id in HeartbeatReceiverSuite.
## What changes were proposed in this pull request?

The current code in `HeartbeatReceiverSuite`, executorId is set as below:
```
  private val executorId1 = "executor-1"
  private val executorId2 = "executor-2"
```

The executorId is sent to driver when register as below:

```
test("expire dead hosts should kill executors with replacement (SPARK-8119)")  {
  ...
  fakeSchedulerBackend.driverEndpoint.askSync[Boolean](
      RegisterExecutor(executorId1, dummyExecutorEndpointRef1, "1.2.3.4", 0, Map.empty))
  ...
}
```

Receiving `RegisterExecutor` in `CoarseGrainedSchedulerBackend`, the executorId will be compared with `currentExecutorIdCounter` as below:
```
case RegisterExecutor(executorId, executorRef, hostname, cores, logUrls)  =>
  if (executorDataMap.contains(executorId)) {
    executorRef.send(RegisterExecutorFailed("Duplicate executor ID: " + executorId))
    context.reply(true)
  } else {
  ...
  executorDataMap.put(executorId, data)
  if (currentExecutorIdCounter < executorId.toInt) {
    currentExecutorIdCounter = executorId.toInt
  }
  ...
```

`executorId.toInt` will cause NumberformatException.

This unit test can pass currently because of `askWithRetry`, when catching exception, RPC will call again, thus it will go `if` branch and return true.

**To fix**
Rectify executorId and replace `askWithRetry` with `askSync`, refer to https://github.com/apache/spark/pull/16690
## How was this patch tested?
This fix is for unit test and no need to add another one.(If this patch involves UI changes, please attach a screenshot; otherwise, remove this)

Author: jinxing <jinxing@meituan.com>

Closes #16779 from jinxing64/SPARK-19437.
2017-02-02 23:18:16 -08:00
Shixiong Zhu 21aa8c32ba [SPARK-19365][CORE] Optimize RequestMessage serialization
## What changes were proposed in this pull request?

Right now Netty PRC serializes `RequestMessage` using Java serialization, and the size of a single message (e.g., RequestMessage(..., "hello")`) is almost 1KB.

This PR optimizes it by serializing `RequestMessage` manually (eliminate unnecessary information from most messages, e.g., class names of `RequestMessage`, `NettyRpcEndpointRef`, ...), and reduces the above message size to 100+ bytes.

## How was this patch tested?

Jenkins

I did a simple test to measure the improvement:

Before
```
$ bin/spark-shell --master local-cluster[1,4,1024]
...
scala> for (i <- 1 to 10) {
     |   val start = System.nanoTime
     |   val s = sc.parallelize(1 to 1000000, 10 * 1000).count()
     |   val end = System.nanoTime
     |   println(s"$i\t" + ((end - start)/1000/1000))
     | }
1       6830
2       4353
3       3322
4       3107
5       3235
6       3139
7       3156
8       3166
9       3091
10      3029
```
After:
```
$ bin/spark-shell --master local-cluster[1,4,1024]
...
scala> for (i <- 1 to 10) {
     |   val start = System.nanoTime
     |   val s = sc.parallelize(1 to 1000000, 10 * 1000).count()
     |   val end = System.nanoTime
     |   println(s"$i\t" + ((end - start)/1000/1000))
     | }
1       6431
2       3643
3       2913
4       2679
5       2760
6       2710
7       2747
8       2793
9       2679
10      2651
```

I also captured the TCP packets for this test. Before this patch, the total size of TCP packets is ~1.5GB. After it, it reduces to ~1.2GB.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #16706 from zsxwing/rpc-opt.
2017-01-27 15:07:57 -08:00
Marcelo Vanzin d3dcb63b97 [SPARK-19220][UI] Make redirection to HTTPS apply to all URIs.
The redirect handler was installed only for the root of the server;
any other context ended up being served directly through the HTTP
port. Since every sub page (e.g. application UIs in the history
server) is a separate servlet context, this meant that everything
but the root was accessible via HTTP still.

The change adds separate names to each connector, and binds contexts
to specific connectors so that content is only served through the
HTTPS connector when it's enabled. In that case, the only thing that
binds to the HTTP connector is the redirect handler.

Tested with new unit tests and by checking a live history server.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #16582 from vanzin/SPARK-19220.
2017-01-26 17:24:44 +09:00
Marcelo Vanzin 8f3f73abc1 [SPARK-19139][CORE] New auth mechanism for transport library.
This change introduces a new auth mechanism to the transport library,
to be used when users enable strong encryption. This auth mechanism
has better security than the currently used DIGEST-MD5.

The new protocol uses symmetric key encryption to mutually authenticate
the endpoints, and is very loosely based on ISO/IEC 9798.

The new protocol falls back to SASL when it thinks the remote end is old.
Because SASL does not support asking the server for multiple auth protocols,
which would mean we could re-use the existing SASL code by just adding a
new SASL provider, the protocol is implemented outside of the SASL API
to avoid the boilerplate of adding a new provider.

Details of the auth protocol are discussed in the included README.md
file.

This change partly undos the changes added in SPARK-13331; AES encryption
is now decoupled from SASL authentication. The encryption code itself,
though, has been re-used as part of this change.

## How was this patch tested?

- Unit tests
- Tested Spark 2.2 against Spark 1.6 shuffle service with SASL enabled
- Tested Spark 2.2 against Spark 2.2 shuffle service with SASL fallback disabled

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #16521 from vanzin/SPARK-19139.
2017-01-24 10:44:04 -08:00
Parag Chaudhari 0ff67a1cf9 [SPARK-14049][CORE] Add functionality in spark history sever API to query applications by end time
## What changes were proposed in this pull request?

Currently, spark history server REST API provides functionality to query applications by application start time range based on minDate and maxDate query parameters, but it  lacks support to query applications by their end time. In this pull request we are proposing optional minEndDate and maxEndDate query parameters and filtering capability based on these parameters to spark history server REST API. This functionality can be used for following queries,
1. Applications finished in last 'x' minutes
2. Applications finished before 'y' time
3. Applications finished between 'x' time to 'y' time
4. Applications started from 'x' time and finished before 'y' time.

For backward compatibility, we can keep existing minDate and maxDate query parameters as they are and they can continue support filtering based on start time range.
## How was this patch tested?

Existing unit tests and 4 new unit tests.

Author: Parag Chaudhari <paragpc@amazon.com>

Closes #11867 from paragpc/master-SHS-query-by-endtime_2.
2017-01-24 08:41:46 -06:00
Yuming Wang c99492141b
[SPARK-19146][CORE] Drop more elements when stageData.taskData.size > retainedTasks
## What changes were proposed in this pull request?

Drop more elements when `stageData.taskData.size > retainedTasks` to reduce the number of times on call drop function.

## How was this patch tested?

Jenkins

Author: Yuming Wang <wgyumg@gmail.com>

Closes #16527 from wangyum/SPARK-19146.
2017-01-23 11:02:22 +00:00
hyukjinkwon 6113fe78a5
[SPARK-19117][SPARK-18922][TESTS] Fix the rest of flaky, newly introduced and missed test failures on Windows
## What changes were proposed in this pull request?

**Failed tests**

```
org.apache.spark.sql.hive.execution.HiveQuerySuite:
 - transform with SerDe3 *** FAILED ***
 - transform with SerDe4 *** FAILED ***
```

```
org.apache.spark.sql.hive.execution.HiveDDLSuite:
 - create hive serde table with new syntax *** FAILED ***
 - add/drop partition with location - managed table *** FAILED ***
```

```
org.apache.spark.sql.hive.ParquetMetastoreSuite:
 - Explicitly added partitions should be readable after load *** FAILED ***
 - Non-partitioned table readable after load *** FAILED ***
```

**Aborted tests**

```
Exception encountered when attempting to run a suite with class name: org.apache.spark.sql.hive.execution.HiveSerDeSuite *** ABORTED *** (157 milliseconds)
   org.apache.spark.sql.AnalysisException: LOAD DATA input path does not exist: C:projectssparksqlhive   argetscala-2.11   est-classesdatafilessales.txt;
```

**Flaky tests(failed 9ish out of 10)**

```
org.apache.spark.scheduler.SparkListenerSuite:
 - local metrics *** FAILED ***
```

## How was this patch tested?

Manually tested via AppVeyor.

**Failed tests**

```
org.apache.spark.sql.hive.execution.HiveQuerySuite:
 - transform with SerDe3 !!! CANCELED !!! (0 milliseconds)
 - transform with SerDe4 !!! CANCELED !!! (0 milliseconds)
```

```
org.apache.spark.sql.hive.execution.HiveDDLSuite:
 - create hive serde table with new syntax (1 second, 672 milliseconds)
 - add/drop partition with location - managed table (2 seconds, 391 milliseconds)
```

```
org.apache.spark.sql.hive.ParquetMetastoreSuite:
 - Explicitly added partitions should be readable after load (609 milliseconds)
 - Non-partitioned table readable after load (344 milliseconds)
```

**Aborted tests**

```
spark.sql.hive.execution.HiveSerDeSuite:
 - Read with RegexSerDe (2 seconds, 142 milliseconds)
 - Read and write with LazySimpleSerDe (tab separated) (2 seconds)
 - Read with AvroSerDe (1 second, 47 milliseconds)
 - Read Partitioned with AvroSerDe (1 second, 422 milliseconds)
```

**Flaky tests (failed 9ish out of 10)**

```
org.apache.spark.scheduler.SparkListenerSuite:
 - local metrics (4 seconds, 562 milliseconds)
```

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16586 from HyukjinKwon/set-path-appveyor.
2017-01-21 14:08:01 +00:00
Parag Chaudhari e20d9b1565 [SPARK-19069][CORE] Expose task 'status' and 'duration' in spark history server REST API.
## What changes were proposed in this pull request?

Although Spark history server UI shows task ‘status’ and ‘duration’ fields, it does not expose these fields in the REST API response. For the Spark history server API users, it is not possible to determine task status and duration. Spark history server has access to task status and duration from event log, but it is not exposing these in API. This patch is proposed to expose task ‘status’ and ‘duration’ fields in Spark history server REST API.

## How was this patch tested?

Modified existing test cases in org.apache.spark.deploy.history.HistoryServerSuite.

Author: Parag Chaudhari <paragpc@amazon.com>

Closes #16473 from paragpc/expose_task_status.
2017-01-20 10:49:05 -06:00
José Hiram Soltren 640f942337 [SPARK-16654][CORE] Add UI coverage for Application Level Blacklisting
Builds on top of work in SPARK-8425 to update Application Level Blacklisting in the scheduler.

## What changes were proposed in this pull request?

Adds a UI to these patches by:
- defining new listener events for blacklisting and unblacklisting, nodes and executors;
- sending said events at the relevant points in BlacklistTracker;
- adding JSON (de)serialization code for these events;
- augmenting the Executors UI page to show which, and how many, executors are blacklisted;
- adding a unit test to make sure events are being fired;
- adding HistoryServerSuite coverage to verify that the SHS reads these events correctly.
- updates the Executor UI to show Blacklisted/Active/Dead as a tri-state in Executors Status

Updates .rat-excludes to pass tests.

username squito

## How was this patch tested?

./dev/run-tests
testOnly org.apache.spark.util.JsonProtocolSuite
testOnly org.apache.spark.scheduler.BlacklistTrackerSuite
testOnly org.apache.spark.deploy.history.HistoryServerSuite
https://github.com/jsoltren/jose-utils/blob/master/blacklist/test-blacklist.sh
![blacklist-20161219](https://cloud.githubusercontent.com/assets/1208477/21335321/9eda320a-c623-11e6-8b8c-9c912a73c276.jpg)

Author: José Hiram Soltren <jose@cloudera.com>

Closes #16346 from jsoltren/SPARK-16654-submit.
2017-01-19 09:08:18 -06:00
jinxing 33791a8ced [SPARK-18113] Use ask to replace askWithRetry in canCommit and make receiver idempotent.
## What changes were proposed in this pull request?

Method canCommit sends AskPermissionToCommitOutput using askWithRetry. If timeout, it will send again. Thus AskPermissionToCommitOutput can be received multi times. Method canCommit should return the same value when called by the same attempt multi times.

In implementation before this fix, method handleAskPermissionToCommit just check if there is committer already registered, which is not enough. When worker retries AskPermissionToCommitOutput it will get CommitDeniedException, then the task will fail with reason TaskCommitDenied, which is not regarded as a task failure(SPARK-11178), so TaskScheduler will schedule this task infinitely.

In this fix, use `ask` to replace `askWithRetry` in `canCommit` and make receiver idempotent.

## How was this patch tested?

Added a new unit test to OutputCommitCoordinatorSuite.

Author: jinxing <jinxing@meituan.com>

Closes #16503 from jinxing64/SPARK-18113.
2017-01-18 10:47:22 -08:00
uncleGen eefdf9f9dd
[SPARK-19227][SPARK-19251] remove unused imports and outdated comments
## What changes were proposed in this pull request?
remove ununsed imports and outdated comments, and fix some minor code style issue.

## How was this patch tested?
existing ut

Author: uncleGen <hustyugm@gmail.com>

Closes #16591 from uncleGen/SPARK-19227.
2017-01-18 09:44:32 +00:00
Bryan Cutler 3bc2eff888 [SPARK-17568][CORE][DEPLOY] Add spark-submit option to override ivy settings used to resolve packages/artifacts
## What changes were proposed in this pull request?

Adding option in spark-submit to allow overriding the default IvySettings used to resolve artifacts as part of the Spark Packages functionality.  This will allow all artifact resolution to go through a central managed repository, such as Nexus or Artifactory, where site admins can better approve and control what is used with Spark apps.

This change restructures the creation of the IvySettings object in two distinct ways.  First, if the `spark.ivy.settings` option is not defined then `buildIvySettings` will create a default settings instance, as before, with defined repositories (Maven Central) included.  Second, if the option is defined, the ivy settings file will be loaded from the given path and only repositories defined within will be used for artifact resolution.
## How was this patch tested?

Existing tests for default behaviour, Manual tests that load a ivysettings.xml file with local and Nexus repositories defined.  Added new test to load a simple Ivy settings file with a local filesystem resolver.

Author: Bryan Cutler <cutlerb@gmail.com>
Author: Ian Hummel <ian@themodernlife.net>

Closes #15119 from BryanCutler/spark-custom-IvySettings.
2017-01-11 11:57:38 -08:00
hyukjinkwon 2cfd41ac02
[SPARK-19117][TESTS] Skip the tests using script transformation on Windows
## What changes were proposed in this pull request?

This PR proposes to skip the tests for script transformation failed on Windows due to fixed bash location.

```
SQLQuerySuite:
 - script *** FAILED *** (553 milliseconds)
   org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 56.0 failed 1 times, most recent failure: Lost task 0.0 in stage 56.0 (TID 54, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified

 - Star Expansion - script transform *** FAILED *** (2 seconds, 375 milliseconds)
   org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 389.0 failed 1 times, most recent failure: Lost task 0.0 in stage 389.0 (TID 725, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified

 - test script transform for stdout *** FAILED *** (2 seconds, 813 milliseconds)
   org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 391.0 failed 1 times, most recent failure: Lost task 0.0 in stage 391.0 (TID 726, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified

 - test script transform for stderr *** FAILED *** (2 seconds, 407 milliseconds)
   org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 393.0 failed 1 times, most recent failure: Lost task 0.0 in stage 393.0 (TID 727, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified

 - test script transform data type *** FAILED *** (171 milliseconds)
   org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 395.0 failed 1 times, most recent failure: Lost task 0.0 in stage 395.0 (TID 728, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified
```

```
HiveQuerySuite:
 - transform *** FAILED *** (359 milliseconds)
   Failed to execute query using catalyst:
   Error: Job aborted due to stage failure: Task 0 in stage 1347.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1347.0 (TID 2395, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified

 - schema-less transform *** FAILED *** (344 milliseconds)
   Failed to execute query using catalyst:
   Error: Job aborted due to stage failure: Task 0 in stage 1348.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1348.0 (TID 2396, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified

 - transform with custom field delimiter *** FAILED *** (296 milliseconds)
   Failed to execute query using catalyst:
   Error: Job aborted due to stage failure: Task 0 in stage 1349.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1349.0 (TID 2397, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified

 - transform with custom field delimiter2 *** FAILED *** (297 milliseconds)
   Failed to execute query using catalyst:
   Error: Job aborted due to stage failure: Task 0 in stage 1350.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1350.0 (TID 2398, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified

 - transform with custom field delimiter3 *** FAILED *** (312 milliseconds)
   Failed to execute query using catalyst:
   Error: Job aborted due to stage failure: Task 0 in stage 1351.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1351.0 (TID 2399, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified

 - transform with SerDe2 *** FAILED *** (437 milliseconds)
   org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 1355.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1355.0 (TID 2403, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified
```

```
LogicalPlanToSQLSuite:
 - script transformation - schemaless *** FAILED *** (78 milliseconds)
   ...
   Cause: org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 1968.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1968.0 (TID 3932, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified
  - script transformation - alias list *** FAILED *** (94 milliseconds)
   ...
   Cause: org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 1969.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1969.0 (TID 3933, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified

 - script transformation - alias list with type *** FAILED *** (93 milliseconds)
   ...
   Cause: org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 1970.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1970.0 (TID 3934, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified

 - script transformation - row format delimited clause with only one format property *** FAILED *** (78 milliseconds)
   ...
   Cause: org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 1971.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1971.0 (TID 3935, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified

 - script transformation - row format delimited clause with multiple format properties *** FAILED *** (94 milliseconds)
   ...
   Cause: org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 1972.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1972.0 (TID 3936, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified

 - script transformation - row format serde clauses with SERDEPROPERTIES *** FAILED *** (78 milliseconds)
   ...
   Cause: org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 1973.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1973.0 (TID 3937, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified

 - script transformation - row format serde clauses without SERDEPROPERTIES *** FAILED *** (78 milliseconds)
   ...
   Cause: org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 1974.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1974.0 (TID 3938, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified
```

```
ScriptTransformationSuite:
 - cat without SerDe *** FAILED *** (156 milliseconds)
   ...
   Caused by: java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified

 - cat with LazySimpleSerDe *** FAILED *** (63 milliseconds)
    ...
    org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 2383.0 failed 1 times, most recent failure: Lost task 0.0 in stage 2383.0 (TID 4819, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified

 - script transformation should not swallow errors from upstream operators (no serde) *** FAILED *** (78 milliseconds)
    ...
    org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 2384.0 failed 1 times, most recent failure: Lost task 0.0 in stage 2384.0 (TID 4820, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified

 - script transformation should not swallow errors from upstream operators (with serde) *** FAILED *** (47 milliseconds)
    ...
    org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 2385.0 failed 1 times, most recent failure: Lost task 0.0 in stage 2385.0 (TID 4821, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified

 - SPARK-14400 script transformation should fail for bad script command *** FAILED *** (47 milliseconds)
   "Job aborted due to stage failure: Task 0 in stage 2386.0 failed 1 times, most recent failure: Lost task 0.0 in stage 2386.0 (TID 4822, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified
```

## How was this patch tested?

AppVeyor as below:

```
SQLQuerySuite:
  - script !!! CANCELED !!! (63 milliseconds)
  - Star Expansion - script transform !!! CANCELED !!! (0 milliseconds)
  - test script transform for stdout !!! CANCELED !!! (0 milliseconds)
  - test script transform for stderr !!! CANCELED !!! (0 milliseconds)
  - test script transform data type !!! CANCELED !!! (0 milliseconds)
```

```
HiveQuerySuite:
  - transform !!! CANCELED !!! (31 milliseconds)
  - schema-less transform !!! CANCELED !!! (0 milliseconds)
  - transform with custom field delimiter !!! CANCELED !!! (0 milliseconds)
  - transform with custom field delimiter2 !!! CANCELED !!! (0 milliseconds)
  - transform with custom field delimiter3 !!! CANCELED !!! (0 milliseconds)
  - transform with SerDe2 !!! CANCELED !!! (0 milliseconds)
```

```
LogicalPlanToSQLSuite:
  - script transformation - schemaless !!! CANCELED !!! (78 milliseconds)
  - script transformation - alias list !!! CANCELED !!! (0 milliseconds)
  - script transformation - alias list with type !!! CANCELED !!! (0 milliseconds)
  - script transformation - row format delimited clause with only one format property !!! CANCELED !!! (15 milliseconds)
  - script transformation - row format delimited clause with multiple format properties !!! CANCELED !!! (0 milliseconds)
  - script transformation - row format serde clauses with SERDEPROPERTIES !!! CANCELED !!! (0 milliseconds)
  - script transformation - row format serde clauses without SERDEPROPERTIES !!! CANCELED !!! (0 milliseconds)
```

```
ScriptTransformationSuite:
  - cat without SerDe !!! CANCELED !!! (62 milliseconds)
  - cat with LazySimpleSerDe !!! CANCELED !!! (0 milliseconds)
  - script transformation should not swallow errors from upstream operators (no serde) !!! CANCELED !!! (0 milliseconds)
  - script transformation should not swallow errors from upstream operators (with serde) !!! CANCELED !!! (0 milliseconds)
  - SPARK-14400 script transformation should fail for bad script command !!! CANCELED !!! (0 milliseconds)
```

Jenkins tests

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16501 from HyukjinKwon/windows-bash.
2017-01-10 13:22:35 +00:00
hyukjinkwon 4e27578faa
[SPARK-18922][SQL][CORE][STREAMING][TESTS] Fix all identified tests failed due to path and resource-not-closed problems on Windows
## What changes were proposed in this pull request?

This PR proposes to fix all the test failures identified by testing with AppVeyor.

**Scala - aborted tests**

```
WindowQuerySuite:
  Exception encountered when attempting to run a suite with class name: org.apache.spark.sql.hive.execution.WindowQuerySuite *** ABORTED *** (156 milliseconds)
   org.apache.spark.sql.AnalysisException: LOAD DATA input path does not exist: C:projectssparksqlhive   argetscala-2.11   est-classesdatafilespart_tiny.txt;

OrcSourceSuite:
 Exception encountered when attempting to run a suite with class name: org.apache.spark.sql.hive.orc.OrcSourceSuite *** ABORTED *** (62 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

ParquetMetastoreSuite:
 Exception encountered when attempting to run a suite with class name: org.apache.spark.sql.hive.ParquetMetastoreSuite *** ABORTED *** (4 seconds, 703 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

ParquetSourceSuite:
 Exception encountered when attempting to run a suite with class name: org.apache.spark.sql.hive.ParquetSourceSuite *** ABORTED *** (3 seconds, 907 milliseconds)
   org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark  arget mpspark-581a6575-454f-4f21-a516-a07f95266143;

KafkaRDDSuite:
 Exception encountered when attempting to run a suite with class name: org.apache.spark.streaming.kafka.KafkaRDDSuite *** ABORTED *** (5 seconds, 212 milliseconds)
   java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-4722304d-213e-4296-b556-951df1a46807

DirectKafkaStreamSuite:
 Exception encountered when attempting to run a suite with class name: org.apache.spark.streaming.kafka.DirectKafkaStreamSuite *** ABORTED *** (7 seconds, 127 milliseconds)
   java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-d0d3eba7-4215-4e10-b40e-bb797e89338e
   at org.apache.spark.util.Utils$.deleteRecursively(Utils.scala:1010)

ReliableKafkaStreamSuite
 Exception encountered when attempting to run a suite with class name: org.apache.spark.streaming.kafka.ReliableKafkaStreamSuite *** ABORTED *** (5 seconds, 498 milliseconds)
   java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-d33e45a0-287e-4bed-acae-ca809a89d888

KafkaStreamSuite:
 Exception encountered when attempting to run a suite with class name: org.apache.spark.streaming.kafka.KafkaStreamSuite *** ABORTED *** (2 seconds, 892 milliseconds)
   java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-59c9d169-5a56-4519-9ef0-cefdbd3f2e6c

KafkaClusterSuite:
 Exception encountered when attempting to run a suite with class name: org.apache.spark.streaming.kafka.KafkaClusterSuite *** ABORTED *** (1 second, 690 milliseconds)
   java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-3ef402b0-8689-4a60-85ae-e41e274f179d

DirectKafkaStreamSuite:
 Exception encountered when attempting to run a suite with class name: org.apache.spark.streaming.kafka010.DirectKafkaStreamSuite *** ABORTED *** (59 seconds, 626 milliseconds)
   java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-426107da-68cf-4d94-b0d6-1f428f1c53f6

KafkaRDDSuite:
Exception encountered when attempting to run a suite with class name: org.apache.spark.streaming.kafka010.KafkaRDDSuite *** ABORTED *** (2 minutes, 6 seconds)
   java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-b9ce7929-5dae-46ab-a0c4-9ef6f58fbc2
```

**Java - failed tests**

```
Test org.apache.spark.streaming.kafka.JavaKafkaRDDSuite.testKafkaRDD failed: java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-1cee32f4-4390-4321-82c9-e8616b3f0fb0, took 9.61 sec

Test org.apache.spark.streaming.kafka.JavaKafkaStreamSuite.testKafkaStream failed: java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-f42695dd-242e-4b07-847c-f299b8e4676e, took 11.797 sec

Test org.apache.spark.streaming.kafka.JavaDirectKafkaStreamSuite.testKafkaStream failed: java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-85c0d062-78cf-459c-a2dd-7973572101ce, took 1.581 sec

Test org.apache.spark.streaming.kafka010.JavaKafkaRDDSuite.testKafkaRDD failed: java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-49eb6b5c-8366-47a6-83f2-80c443c48280, took 17.895 sec

org.apache.spark.streaming.kafka010.JavaDirectKafkaStreamSuite.testKafkaStream failed: java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-898cf826-d636-4b1c-a61a-c12a364c02e7, took 8.858 sec
```

**Scala - failed tests**

```
PartitionProviderCompatibilitySuite:
 - insert overwrite partition of new datasource table overwrites just partition *** FAILED *** (828 milliseconds)
   java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-bb6337b9-4f99-45ab-ad2c-a787ab965c09

 - SPARK-18635 special chars in partition values - partition management true *** FAILED *** (5 seconds, 360 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

 - SPARK-18635 special chars in partition values - partition management false *** FAILED *** (141 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
```

```
UtilsSuite:
 - reading offset bytes of a file (compressed) *** FAILED *** (0 milliseconds)
   java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-ecb2b7d5-db8b-43a7-b268-1bf242b5a491

 - reading offset bytes across multiple files (compressed) *** FAILED *** (0 milliseconds)
   java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-25cc47a8-1faa-4da5-8862-cf174df63ce0
```

```
StatisticsSuite:
 - MetastoreRelations fallback to HDFS for size estimation *** FAILED *** (110 milliseconds)
   org.apache.spark.sql.catalyst.analysis.NoSuchTableException: Table or view 'csv_table' not found in database 'default';
```

```
SQLQuerySuite:
 - permanent UDTF *** FAILED *** (125 milliseconds)
   org.apache.spark.sql.AnalysisException: Undefined function: 'udtf_count_temp'. This function is neither a registered temporary function nor a permanent function registered in the database 'default'.; line 1 pos 24

 - describe functions - user defined functions *** FAILED *** (125 milliseconds)
   org.apache.spark.sql.AnalysisException: Undefined function: 'udtf_count'. This function is neither a registered temporary function nor a permanent function registered in the database 'default'.; line 1 pos 7

 - CTAS without serde with location *** FAILED *** (16 milliseconds)
   java.lang.IllegalArgumentException: java.net.URISyntaxException: Relative path in absolute URI: file:C:projectsspark%09arget%09mpspark-ed673d73-edfc-404e-829e-2e2b9725d94e/c1

 - derived from Hive query file: drop_database_removes_partition_dirs.q *** FAILED *** (47 milliseconds)
   java.lang.IllegalArgumentException: java.net.URISyntaxException: Relative path in absolute URI: file:C:projectsspark%09arget%09mpspark-d2ddf08e-699e-45be-9ebd-3dfe619680fe/drop_database_removes_partition_dirs_table

 - derived from Hive query file: drop_table_removes_partition_dirs.q *** FAILED *** (0 milliseconds)
   java.lang.IllegalArgumentException: java.net.URISyntaxException: Relative path in absolute URI: file:C:projectsspark%09arget%09mpspark-d2ddf08e-699e-45be-9ebd-3dfe619680fe/drop_table_removes_partition_dirs_table2

 - SPARK-17796 Support wildcard character in filename for LOAD DATA LOCAL INPATH *** FAILED *** (109 milliseconds)
   java.nio.file.InvalidPathException: Illegal char <:> at index 2: /C:/projects/spark/sql/hive/projectsspark	arget	mpspark-1a122f8c-dfb3-46c4-bab1-f30764baee0e/*part-r*
```

```
HiveDDLSuite:
 - drop external tables in default database *** FAILED *** (16 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

 - add/drop partitions - external table *** FAILED *** (16 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

 - create/drop database - location without pre-created directory *** FAILED *** (16 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

 - create/drop database - location with pre-created directory *** FAILED *** (32 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

 - drop database containing tables - CASCADE *** FAILED *** (94 milliseconds)
   CatalogDatabase(db1,,file:/C:/projects/spark/target/tmp/warehouse-d0665ee0-1e39-4805-b471-0b764f7838be/db1.db,Map()) did not equal CatalogDatabase(db1,,file:C:/projects/spark/target/tmp/warehouse-d0665ee0-1e39-4805-b471-0b764f7838be\db1.db,Map()) (HiveDDLSuite.scala:675)

 - drop an empty database - CASCADE *** FAILED *** (63 milliseconds)
   CatalogDatabase(db1,,file:/C:/projects/spark/target/tmp/warehouse-d0665ee0-1e39-4805-b471-0b764f7838be/db1.db,Map()) did not equal CatalogDatabase(db1,,file:C:/projects/spark/target/tmp/warehouse-d0665ee0-1e39-4805-b471-0b764f7838be\db1.db,Map()) (HiveDDLSuite.scala:675)

 - drop database containing tables - RESTRICT *** FAILED *** (47 milliseconds)
   CatalogDatabase(db1,,file:/C:/projects/spark/target/tmp/warehouse-d0665ee0-1e39-4805-b471-0b764f7838be/db1.db,Map()) did not equal CatalogDatabase(db1,,file:C:/projects/spark/target/tmp/warehouse-d0665ee0-1e39-4805-b471-0b764f7838be\db1.db,Map()) (HiveDDLSuite.scala:675)

 - drop an empty database - RESTRICT *** FAILED *** (47 milliseconds)
   CatalogDatabase(db1,,file:/C:/projects/spark/target/tmp/warehouse-d0665ee0-1e39-4805-b471-0b764f7838be/db1.db,Map()) did not equal CatalogDatabase(db1,,file:C:/projects/spark/target/tmp/warehouse-d0665ee0-1e39-4805-b471-0b764f7838be\db1.db,Map()) (HiveDDLSuite.scala:675)

 - CREATE TABLE LIKE an external data source table *** FAILED *** (140 milliseconds)
   org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-c5eba16d-07ae-4186-95bb-21c5811cf888;

 - CREATE TABLE LIKE an external Hive serde table *** FAILED *** (16 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

 - desc table for data source table - no user-defined schema *** FAILED *** (125 milliseconds)
   org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-e8bf5bf5-721a-4cbe-9d6	at scala.collection.immutable.List.foreach(List.scala:381)d-5543a8301c1d;
```

```
MetastoreDataSourcesSuite
 - CTAS: persisted bucketed data source table *** FAILED *** (16 milliseconds)
   java.lang.IllegalArgumentException: Can not create a Path from an empty string
```

```
ShowCreateTableSuite:
 - simple external hive table *** FAILED *** (0 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
```

```
PartitionedTablePerfStatsSuite:
 - hive table: partitioned pruned table reports only selected files *** FAILED *** (313 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

 - datasource table: partitioned pruned table reports only selected files *** FAILED *** (219 milliseconds)
   org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-311f45f8-d064-4023-a4bb-e28235bff64d;

 - hive table: lazy partition pruning reads only necessary partition data *** FAILED *** (203 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

 - datasource table: lazy partition pruning reads only necessary partition data *** FAILED *** (187 milliseconds)
   org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-fde874ca-66bd-4d0b-a40f-a043b65bf957;

 - hive table: lazy partition pruning with file status caching enabled *** FAILED *** (188 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

 - datasource table: lazy partition pruning with file status caching enabled *** FAILED *** (187 milliseconds)
   org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-e6d20183-dd68-4145-acbe-4a509849accd;

 - hive table: file status caching respects refresh table and refreshByPath *** FAILED *** (172 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

 - datasource table: file status caching respects refresh table and refreshByPath *** FAILED *** (203 milliseconds)
   org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-8b2c9651-2adf-4d58-874f-659007e21463;

 - hive table: file status cache respects size limit *** FAILED *** (219 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

 - datasource table: file status cache respects size limit *** FAILED *** (171 milliseconds)
   org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-7835ab57-cb48-4d2c-bb1d-b46d5a4c47e4;

 - datasource table: table setup does not scan filesystem *** FAILED *** (266 milliseconds)
   org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-20598d76-c004-42a7-8061-6c56f0eda5e2;

 - hive table: table setup does not scan filesystem *** FAILED *** (266 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

 - hive table: num hive client calls does not scale with partition count *** FAILED *** (2 seconds, 281 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

 - datasource table: num hive client calls does not scale with partition count *** FAILED *** (2 seconds, 422 milliseconds)
   org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-4cfed321-4d1d-4b48-8d34-5c169afff383;

 - hive table: files read and cached when filesource partition management is off *** FAILED *** (234 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

 - datasource table: all partition data cached in memory when partition management is off *** FAILED *** (203 milliseconds)
   org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-4bcc0398-15c9-4f6a-811e-12d40f3eec12;

 - SPARK-18700: table loaded only once even when resolved concurrently *** FAILED *** (1 second, 266 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
```

```
HiveSparkSubmitSuite:
 - temporary Hive UDF: define a UDF and use it *** FAILED *** (2 seconds, 94 milliseconds)
   java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified

 - permanent Hive UDF: define a UDF and use it *** FAILED *** (281 milliseconds)
   java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified

 - permanent Hive UDF: use a already defined permanent function *** FAILED *** (718 milliseconds)
   java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified

 - SPARK-8368: includes jars passed in through --jars *** FAILED *** (3 seconds, 521 milliseconds)
   java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified

 - SPARK-8020: set sql conf in spark conf *** FAILED *** (0 milliseconds)
   java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified

 - SPARK-8489: MissingRequirementError during reflection *** FAILED *** (94 milliseconds)
   java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified

 - SPARK-9757 Persist Parquet relation with decimal column *** FAILED *** (16 milliseconds)
   java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified

 - SPARK-11009 fix wrong result of Window function in cluster mode *** FAILED *** (16 milliseconds)
   java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified

 - SPARK-14244 fix window partition size attribute binding failure *** FAILED *** (78 milliseconds)
   java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified

 - set spark.sql.warehouse.dir *** FAILED *** (16 milliseconds)
   java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified

 - set hive.metastore.warehouse.dir *** FAILED *** (15 milliseconds)
   java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified

 - SPARK-16901: set javax.jdo.option.ConnectionURL *** FAILED *** (16 milliseconds)
   java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified

 - SPARK-18360: default table path of tables in default database should depend on the location of default database *** FAILED *** (15 milliseconds)
   java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified
```

```
UtilsSuite:
 - resolveURIs with multiple paths *** FAILED *** (0 milliseconds)
   ".../jar3,file:/C:/pi.py[%23]py.pi,file:/C:/path%..." did not equal ".../jar3,file:/C:/pi.py[#]py.pi,file:/C:/path%..." (UtilsSuite.scala:468)
```

```
CheckpointSuite:
 - recovery with file input stream *** FAILED *** (10 seconds, 205 milliseconds)
   The code passed to eventually never returned normally. Attempted 660 times over 10.014272499999999 seconds. Last failure message: Unexpected internal error near index 1
   \
    ^. (CheckpointSuite.scala:680)
```

## How was this patch tested?

Manually via AppVeyor as below:

**Scala - aborted tests**

```
WindowQuerySuite - all passed
OrcSourceSuite:
- SPARK-18220: read Hive orc table with varchar column *** FAILED *** (4 seconds, 417 milliseconds)
  org.apache.spark.sql.execution.QueryExecutionException: FAILED: Execution Error, return code -101 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask. org.apache.hadoop.io.nativeio.NativeIO$Windows.access0(Ljava/lang/String;I)Z
  at org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$runHive$1.apply(HiveClientImpl.scala:625)
ParquetMetastoreSuite - all passed
ParquetSourceSuite - all passed
KafkaRDDSuite - all passed
DirectKafkaStreamSuite - all passed
ReliableKafkaStreamSuite - all passed
KafkaStreamSuite - all passed
KafkaClusterSuite - all passed
DirectKafkaStreamSuite - all passed
KafkaRDDSuite - all passed
```

**Java - failed tests**

```
org.apache.spark.streaming.kafka.JavaKafkaRDDSuite - all passed
org.apache.spark.streaming.kafka.JavaDirectKafkaStreamSuite - all passed
org.apache.spark.streaming.kafka.JavaKafkaStreamSuite - all passed
org.apache.spark.streaming.kafka010.JavaDirectKafkaStreamSuite - all passed
org.apache.spark.streaming.kafka010.JavaKafkaRDDSuite - all passed
```

**Scala - failed tests**

```
PartitionProviderCompatibilitySuite:
- insert overwrite partition of new datasource table overwrites just partition (1 second, 953 milliseconds)
- SPARK-18635 special chars in partition values - partition management true (6 seconds, 31 milliseconds)
- SPARK-18635 special chars in partition values - partition management false (4 seconds, 578 milliseconds)
```

```
UtilsSuite:
- reading offset bytes of a file (compressed) (203 milliseconds)
- reading offset bytes across multiple files (compressed) (0 milliseconds)
```

```
StatisticsSuite:
- MetastoreRelations fallback to HDFS for size estimation (94 milliseconds)
```

```
SQLQuerySuite:
 - permanent UDTF (407 milliseconds)
 - describe functions - user defined functions (441 milliseconds)
 - CTAS without serde with location (2 seconds, 831 milliseconds)
 - derived from Hive query file: drop_database_removes_partition_dirs.q (734 milliseconds)
 - derived from Hive query file: drop_table_removes_partition_dirs.q (563 milliseconds)
 - SPARK-17796 Support wildcard character in filename for LOAD DATA LOCAL INPATH (453 milliseconds)
```

```
HiveDDLSuite:
 - drop external tables in default database (3 seconds, 5 milliseconds)
 - add/drop partitions - external table (2 seconds, 750 milliseconds)
 - create/drop database - location without pre-created directory (500 milliseconds)
 - create/drop database - location with pre-created directory (407 milliseconds)
 - drop database containing tables - CASCADE (453 milliseconds)
 - drop an empty database - CASCADE (375 milliseconds)
 - drop database containing tables - RESTRICT (328 milliseconds)
 - drop an empty database - RESTRICT (391 milliseconds)
 - CREATE TABLE LIKE an external data source table (953 milliseconds)
 - CREATE TABLE LIKE an external Hive serde table (3 seconds, 782 milliseconds)
 - desc table for data source table - no user-defined schema (1 second, 150 milliseconds)
```

```
MetastoreDataSourcesSuite
 - CTAS: persisted bucketed data source table (875 milliseconds)
```

```
ShowCreateTableSuite:
 - simple external hive table (78 milliseconds)
```

```
PartitionedTablePerfStatsSuite:
 - hive table: partitioned pruned table reports only selected files (1 second, 109 milliseconds)
- datasource table: partitioned pruned table reports only selected files (860 milliseconds)
 - hive table: lazy partition pruning reads only necessary partition data (859 milliseconds)
 - datasource table: lazy partition pruning reads only necessary partition data (1 second, 219 milliseconds)
 - hive table: lazy partition pruning with file status caching enabled (875 milliseconds)
 - datasource table: lazy partition pruning with file status caching enabled (890 milliseconds)
 - hive table: file status caching respects refresh table and refreshByPath (922 milliseconds)
 - datasource table: file status caching respects refresh table and refreshByPath (640 milliseconds)
 - hive table: file status cache respects size limit (469 milliseconds)
 - datasource table: file status cache respects size limit (453 milliseconds)
 - datasource table: table setup does not scan filesystem (328 milliseconds)
 - hive table: table setup does not scan filesystem (313 milliseconds)
 - hive table: num hive client calls does not scale with partition count (5 seconds, 431 milliseconds)
 - datasource table: num hive client calls does not scale with partition count (4 seconds, 79 milliseconds)
 - hive table: files read and cached when filesource partition management is off (656 milliseconds)
 - datasource table: all partition data cached in memory when partition management is off (484 milliseconds)
 - SPARK-18700: table loaded only once even when resolved concurrently (2 seconds, 578 milliseconds)
```

```
HiveSparkSubmitSuite:
 - temporary Hive UDF: define a UDF and use it (1 second, 745 milliseconds)
 - permanent Hive UDF: define a UDF and use it (406 milliseconds)
 - permanent Hive UDF: use a already defined permanent function (375 milliseconds)
 - SPARK-8368: includes jars passed in through --jars (391 milliseconds)
 - SPARK-8020: set sql conf in spark conf (156 milliseconds)
 - SPARK-8489: MissingRequirementError during reflection (187 milliseconds)
 - SPARK-9757 Persist Parquet relation with decimal column (157 milliseconds)
 - SPARK-11009 fix wrong result of Window function in cluster mode (156 milliseconds)
 - SPARK-14244 fix window partition size attribute binding failure (156 milliseconds)
 - set spark.sql.warehouse.dir (172 milliseconds)
 - set hive.metastore.warehouse.dir (156 milliseconds)
 - SPARK-16901: set javax.jdo.option.ConnectionURL (157 milliseconds)
 - SPARK-18360: default table path of tables in default database should depend on the location of default database (172 milliseconds)
```

```
UtilsSuite:
 - resolveURIs with multiple paths (0 milliseconds)
```

```
CheckpointSuite:
 - recovery with file input stream (4 seconds, 452 milliseconds)
```

Note: after resolving the aborted tests, there is a test failure identified as below:

```
OrcSourceSuite:
- SPARK-18220: read Hive orc table with varchar column *** FAILED *** (4 seconds, 417 milliseconds)
  org.apache.spark.sql.execution.QueryExecutionException: FAILED: Execution Error, return code -101 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask. org.apache.hadoop.io.nativeio.NativeIO$Windows.access0(Ljava/lang/String;I)Z
  at org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$runHive$1.apply(HiveClientImpl.scala:625)
```

This does not look due to this problem so this PR does not fix it here.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16451 from HyukjinKwon/all-path-resource-fixes.
2017-01-10 13:19:21 +00:00
Kay Ousterhout 2e139eed31 [SPARK-17931] Eliminate unnecessary task (de) serialization
In the existing code, there are three layers of serialization
    involved in sending a task from the scheduler to an executor:
        - A Task object is serialized
        - The Task object is copied to a byte buffer that also
          contains serialized information about any additional JARs,
          files, and Properties needed for the task to execute. This
          byte buffer is stored as the member variable serializedTask
          in the TaskDescription class.
        - The TaskDescription is serialized (in addition to the serialized
          task + JARs, the TaskDescription class contains the task ID and
          other metadata) and sent in a LaunchTask message.

While it *is* necessary to have two layers of serialization, so that
the JAR, file, and Property info can be deserialized prior to
deserializing the Task object, the third layer of deserialization is
unnecessary.  This commit eliminates a layer of serialization by moving
the JARs, files, and Properties into the TaskDescription class.

This commit also serializes the Properties manually (by traversing the map),
as is done with the JARs and files, which reduces the final serialized size.

Unit tests

This is a simpler alternative to the approach proposed in #15505.

shivaram and I did some benchmarking of this and #15505 on a 20-machine m2.4xlarge EC2 machines (160 cores). We ran ~30 trials of code [1] (a very simple job with 10K tasks per stage) and measured the average time per stage:

Before this change: 2490ms
With this change: 2345 ms (so ~6% improvement over the baseline)
With witgo's approach in #15505: 2046 ms (~18% improvement over baseline)

The reason that #15505 has a more significant improvement is that it also moves the serialization from the TaskSchedulerImpl thread to the CoarseGrainedSchedulerBackend thread. I added that functionality on top of this change, and got almost the same improvement [1] as #15505 (average of 2103ms). I think we should decouple these two changes, both so we have some record of the improvement form each individual improvement, and because this change is more about simplifying the code base (the improvement is negligible) while the other is about performance improvement.  The plan, currently, is to merge this PR and then merge the remaining part of #15505 that moves serialization.

[1] The reason the improvement wasn't quite as good as with #15505 when we ran the benchmarks is almost certainly because, at the point when we ran the benchmarks, I hadn't updated the code to manually serialize the Properties (instead the code was using Java's default serialization for the Properties object, whereas #15505 manually serialized the Properties).  This PR has since been updated to manually serialize the Properties, just like the other maps.

Author: Kay Ousterhout <kayousterhout@gmail.com>

Closes #16053 from kayousterhout/SPARK-17931.
2017-01-06 10:48:08 -06:00
jerryshao 4a4c3dc9ca [SPARK-19033][CORE] Add admin acls for history server
## What changes were proposed in this pull request?

Current HistoryServer's ACLs is derived from application event-log, which means the newly changed ACLs cannot be applied to the old data, this will become a problem where newly added admin cannot access the old application history UI, only the new application can be affected.

So here propose to add admin ACLs for history server, any configured user/group could have the view access to all the applications, while the view ACLs derived from application run-time still take effect.

## How was this patch tested?

Unit test added.

Author: jerryshao <sshao@hortonworks.com>

Closes #16470 from jerryshao/SPARK-19033.
2017-01-06 10:07:54 -06:00
Rui Li f5d18af6a8 [SPARK-14958][CORE] Failed task not handled when there's error deserializing failure reason
## What changes were proposed in this pull request?

TaskResultGetter tries to deserialize the TaskEndReason before handling the failed task. If an error is thrown during deserialization, the failed task won't be handled, which leaves the job hanging.
The PR proposes to handle the failed task in a finally block.
## How was this patch tested?

In my case I hit a NoClassDefFoundError and the job hangs. Manually verified the patch can fix it.

Author: Rui Li <rui.li@intel.com>
Author: Rui Li <lirui@apache.org>
Author: Rui Li <shlr@cn.ibm.com>

Closes #12775 from lirui-intel/SPARK-14958.
2017-01-05 14:51:13 -08:00
Kay Ousterhout 00074b5778 [SPARK-19062] Utils.writeByteBuffer bug fix
This commit changes Utils.writeByteBuffer so that it does not change
the position of the ByteBuffer that it writes out, and adds a unit test for
this functionality.

cc mridulm

Author: Kay Ousterhout <kayousterhout@gmail.com>

Closes #16462 from kayousterhout/SPARK-19062.
2017-01-04 11:21:09 -08:00
Niranjan Padmanabhan a1e40b1f5d
[MINOR][DOCS] Remove consecutive duplicated words/typo in Spark Repo
## What changes were proposed in this pull request?
There are many locations in the Spark repo where the same word occurs consecutively. Sometimes they are appropriately placed, but many times they are not. This PR removes the inappropriately duplicated words.

## How was this patch tested?
N/A since only docs or comments were updated.

Author: Niranjan Padmanabhan <niranjan.padmanabhan@gmail.com>

Closes #16455 from neurons/np.structure_streaming_doc.
2017-01-04 15:07:29 +00:00
Weiqing Yang e5c307c50a
[MINOR] Add missing sc.stop() to end of examples
## What changes were proposed in this pull request?

Add `finally` clause for `sc.stop()` in the `test("register and deregister Spark listener from SparkContext")`.

## How was this patch tested?
Pass the build and unit tests.

Author: Weiqing Yang <yangweiqing001@gmail.com>

Closes #16426 from weiqingy/testIssue.
2017-01-03 09:56:42 +00:00
Sergei Lebedev 67fb33e7e0
[SPARK-19010][CORE] Include Kryo exception in case of overflow
## What changes were proposed in this pull request?

This is to workaround an implicit result of #4947 which suppressed the
original Kryo exception if the overflow happened during serialization.

## How was this patch tested?

`KryoSerializerSuite` was augmented to reflect this change.

Author: Sergei Lebedev <superbobry@gmail.com>

Closes #16416 from superbobry/patch-1.
2016-12-28 10:30:38 +00:00
Shixiong Zhu 7026ee23e0 [SPARK-17755][CORE] Use workerRef to send RegisterWorkerResponse to avoid the race condition
## What changes were proposed in this pull request?

The root cause of this issue is that RegisterWorkerResponse and LaunchExecutor are sent via two different channels (TCP connections) and their order is not guaranteed.

This PR changes the master and worker codes to use `workerRef` to send RegisterWorkerResponse, so that RegisterWorkerResponse and LaunchExecutor are sent via the same connection. Hence `LaunchExecutor` will always be after `RegisterWorkerResponse` and never be ignored.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #16345 from zsxwing/SPARK-17755.
2016-12-25 23:48:14 -08:00
jerryshao 31da755c80 [SPARK-18975][CORE] Add an API to remove SparkListener
## What changes were proposed in this pull request?

In current Spark we could add customized SparkListener through `SparkContext#addListener` API, but there's no equivalent API to remove the registered one. In our scenario SparkListener will be added repeatedly accordingly to the changed environment. If lacks the ability to remove listeners, there might be many registered listeners finally, this is unnecessary and potentially affects the performance. So here propose to add an API to remove registered listener.

## How was this patch tested?

Add an unit test to verify it.

Author: jerryshao <sshao@hortonworks.com>

Closes #16382 from jerryshao/SPARK-18975.
2016-12-22 11:18:22 -08:00
hyukjinkwon 4186aba632
[SPARK-18922][TESTS] Fix more resource-closing-related and path-related test failures in identified ones on Windows
## What changes were proposed in this pull request?

There are several tests failing due to resource-closing-related and path-related  problems on Windows as below.

- `SQLQuerySuite`:

```
- specifying database name for a temporary table is not allowed *** FAILED *** (125 milliseconds)
  org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark  arget mpspark-1f4471ab-aac0-4239-ae35-833d54b37e52;
  at org.apache.spark.sql.execution.datasources.DataSource$$anonfun$14.apply(DataSource.scala:382)
  at org.apache.spark.sql.execution.datasources.DataSource$$anonfun$14.apply(DataSource.scala:370)
```

- `JsonSuite`:

```
- Loading a JSON dataset from a text file with SQL *** FAILED *** (94 milliseconds)
  org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark  arget mpspark-c918a8b7-fc09-433c-b9d0-36c0f78ae918;
  at org.apache.spark.sql.execution.datasources.DataSource$$anonfun$14.apply(DataSource.scala:382)
  at org.apache.spark.sql.execution.datasources.DataSource$$anonfun$14.apply(DataSource.scala:370)
```

- `StateStoreSuite`:

```
- SPARK-18342: commit fails when rename fails *** FAILED *** (16 milliseconds)
  java.lang.IllegalArgumentException: java.net.URISyntaxException: Relative path in absolute URI: StateStoreSuite29777261fs://C:%5Cprojects%5Cspark%5Ctarget%5Ctmp%5Cspark-ef349862-7281-4963-aaf3-add0d670a4ad%5C?????-2218c2f8-2cf6-4f80-9cdf-96354e8246a77685899733421033312/0
  at org.apache.hadoop.fs.Path.initialize(Path.java:206)
  at org.apache.hadoop.fs.Path.<init>(Path.java:116)
  at org.apache.hadoop.fs.Path.<init>(Path.java:89)
  ...
  Cause: java.net.URISyntaxException: Relative path in absolute URI: StateStoreSuite29777261fs://C:%5Cprojects%5Cspark%5Ctarget%5Ctmp%5Cspark-ef349862-7281-4963-aaf3-add0d670a4ad%5C?????-2218c2f8-2cf6-4f80-9cdf-96354e8246a77685899733421033312/0
  at java.net.URI.checkPath(URI.java:1823)
  at java.net.URI.<init>(URI.java:745)
  at org.apache.hadoop.fs.Path.initialize(Path.java:203)
```

- `HDFSMetadataLogSuite`:

```
- FileManager: FileContextManager *** FAILED *** (94 milliseconds)
  java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-415bb0bd-396b-444d-be82-04599e025f21
  at org.apache.spark.util.Utils$.deleteRecursively(Utils.scala:1010)
  at org.apache.spark.sql.test.SQLTestUtils$class.withTempDir(SQLTestUtils.scala:127)
  at org.apache.spark.sql.execution.streaming.HDFSMetadataLogSuite.withTempDir(HDFSMetadataLogSuite.scala:38)

- FileManager: FileSystemManager *** FAILED *** (78 milliseconds)
  java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-ef8222cd-85aa-47c0-a396-bc7979e15088
  at org.apache.spark.util.Utils$.deleteRecursively(Utils.scala:1010)
  at org.apache.spark.sql.test.SQLTestUtils$class.withTempDir(SQLTestUtils.scala:127)
  at org.apache.spark.sql.execution.streaming.HDFSMetadataLogSuite.withTempDir(HDFSMetadataLogSuite.scala:38)
```

And, there are some tests being failed due to the length limitation on cmd in Windows as below:

- `LauncherBackendSuite`:

```
- local: launcher handle *** FAILED *** (30 seconds, 120 milliseconds)
  The code passed to eventually never returned normally. Attempted 283 times over 30.0960053 seconds. Last failure message: The reference was null. (LauncherBackendSuite.scala:56)
  org.scalatest.exceptions.TestFailedDueToTimeoutException:
  at org.scalatest.concurrent.Eventually$class.tryTryAgain$1(Eventually.scala:420)
  at org.scalatest.concurrent.Eventually$class.eventually(Eventually.scala:438)

- standalone/client: launcher handle *** FAILED *** (30 seconds, 47 milliseconds)
  The code passed to eventually never returned normally. Attempted 282 times over 30.037987100000002 seconds. Last failure message: The reference was null. (LauncherBackendSuite.scala:56)
  org.scalatest.exceptions.TestFailedDueToTimeoutException:
  at org.scalatest.concurrent.Eventually$class.tryTryAgain$1(Eventually.scala:420)
  at org.scalatest.concurrent.Eventually$class.eventually(Eventually.scala:438)
```

The executed command is, https://gist.github.com/HyukjinKwon/d3fdd2e694e5c022992838a618a516bd, which is 16K length; however, the length limitation is 8K on Windows. So, it is being failed to launch.

This PR proposes to fix the test failures on Windows and skip the tests failed due to the length limitation

## How was this patch tested?

Manually tested via AppVeyor

**Before**

`SQLQuerySuite `: https://ci.appveyor.com/project/spark-test/spark/build/306-pr-references
`JsonSuite`: https://ci.appveyor.com/project/spark-test/spark/build/307-pr-references
`StateStoreSuite` : https://ci.appveyor.com/project/spark-test/spark/build/305-pr-references
`HDFSMetadataLogSuite`: https://ci.appveyor.com/project/spark-test/spark/build/304-pr-references
`LauncherBackendSuite`: https://ci.appveyor.com/project/spark-test/spark/build/303-pr-references

**After**

`SQLQuerySuite`: https://ci.appveyor.com/project/spark-test/spark/build/293-SQLQuerySuite
`JsonSuite`: https://ci.appveyor.com/project/spark-test/spark/build/294-JsonSuite
`StateStoreSuite`: https://ci.appveyor.com/project/spark-test/spark/build/297-StateStoreSuite
`HDFSMetadataLogSuite`: https://ci.appveyor.com/project/spark-test/spark/build/319-pr-references
`LauncherBackendSuite`: failed test skipped.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16335 from HyukjinKwon/more-fixes-on-windows.
2016-12-22 16:15:54 +00:00
Josh Rosen fa829ce21f [SPARK-18761][CORE] Introduce "task reaper" to oversee task killing in executors
## What changes were proposed in this pull request?

Spark's current task cancellation / task killing mechanism is "best effort" because some tasks may not be interruptible or may not respond to their "killed" flags being set. If a significant fraction of a cluster's task slots are occupied by tasks that have been marked as killed but remain running then this can lead to a situation where new jobs and tasks are starved of resources that are being used by these zombie tasks.

This patch aims to address this problem by adding a "task reaper" mechanism to executors. At a high-level, task killing now launches a new thread which attempts to kill the task and then watches the task and periodically checks whether it has been killed. The TaskReaper will periodically re-attempt to call `TaskRunner.kill()` and will log warnings if the task keeps running. I modified TaskRunner to rename its thread at the start of the task, allowing TaskReaper to take a thread dump and filter it in order to log stacktraces from the exact task thread that we are waiting to finish. If the task has not stopped after a configurable timeout then the TaskReaper will throw an exception to trigger executor JVM death, thereby forcibly freeing any resources consumed by the zombie tasks.

This feature is flagged off by default and is controlled by four new configurations under the `spark.task.reaper.*` namespace. See the updated `configuration.md` doc for details.

## How was this patch tested?

Tested via a new test case in `JobCancellationSuite`, plus manual testing.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #16189 from JoshRosen/cancellation.
2016-12-19 18:43:59 -08:00
Shivaram Venkataraman 4cb49412d1 [SPARK-18836][CORE] Serialize one copy of task metrics in DAGScheduler
## What changes were proposed in this pull request?

Right now we serialize the empty task metrics once per task – Since this is shared across all tasks we could use the same serialized task metrics across all tasks of a stage.

## How was this patch tested?

- [x] Run tests on EC2 to measure performance improvement

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

Closes #16261 from shivaram/task-metrics-one-copy.
2016-12-19 14:53:01 -08:00
Yuming Wang 1e5c51f336
[SPARK-18827][CORE] Fix cannot read broadcast on disk
## What changes were proposed in this pull request?
`NoSuchElementException` will throw since https://github.com/apache/spark/pull/15056 if a broadcast cannot cache in memory. The reason is that that change cannot cover `!unrolled.hasNext` in `next()` function.

This change is to cover the `!unrolled.hasNext` and check `hasNext` before calling `next` in `blockManager.getLocalValues` to make it  more robust.

We can cache and read broadcast even it cannot fit in memory from this pull request.

Exception log:
```
16/12/10 10:10:04 INFO UnifiedMemoryManager: Will not store broadcast_131 as the required space (1048576 bytes) exceeds our memory limit (122764 bytes)
16/12/10 10:10:04 WARN MemoryStore: Failed to reserve initial memory threshold of 1024.0 KB for computing block broadcast_131 in memory.
16/12/10 10:10:04 WARN MemoryStore: Not enough space to cache broadcast_131 in memory! (computed 384.0 B so far)
16/12/10 10:10:04 INFO MemoryStore: Memory use = 95.6 KB (blocks) + 0.0 B (scratch space shared across 0 tasks(s)) = 95.6 KB. Storage limit = 119.9 KB.
16/12/10 10:10:04 ERROR Utils: Exception encountered
java.util.NoSuchElementException
	at org.apache.spark.util.collection.PrimitiveVector$$anon$1.next(PrimitiveVector.scala:58)
	at org.apache.spark.storage.memory.PartiallyUnrolledIterator.next(MemoryStore.scala:700)
	at org.apache.spark.util.CompletionIterator.next(CompletionIterator.scala:30)
	at org.apache.spark.broadcast.TorrentBroadcast$$anonfun$readBroadcastBlock$1$$anonfun$2.apply(TorrentBroadcast.scala:210)
	at org.apache.spark.broadcast.TorrentBroadcast$$anonfun$readBroadcastBlock$1$$anonfun$2.apply(TorrentBroadcast.scala:210)
	at scala.Option.map(Option.scala:146)
	at org.apache.spark.broadcast.TorrentBroadcast$$anonfun$readBroadcastBlock$1.apply(TorrentBroadcast.scala:210)
	at org.apache.spark.util.Utils$.tryOrIOException(Utils.scala:1269)
	at org.apache.spark.broadcast.TorrentBroadcast.readBroadcastBlock(TorrentBroadcast.scala:206)
	at org.apache.spark.broadcast.TorrentBroadcast._value$lzycompute(TorrentBroadcast.scala:66)
	at org.apache.spark.broadcast.TorrentBroadcast._value(TorrentBroadcast.scala:66)
	at org.apache.spark.broadcast.TorrentBroadcast.getValue(TorrentBroadcast.scala:96)
	at org.apache.spark.broadcast.Broadcast.value(Broadcast.scala:70)
	at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:86)
	at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:53)
	at org.apache.spark.scheduler.Task.run(Task.scala:108)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
	at java.lang.Thread.run(Thread.java:745)
16/12/10 10:10:04 ERROR Executor: Exception in task 1.0 in stage 86.0 (TID 134423)
java.io.IOException: java.util.NoSuchElementException
	at org.apache.spark.util.Utils$.tryOrIOException(Utils.scala:1276)
	at org.apache.spark.broadcast.TorrentBroadcast.readBroadcastBlock(TorrentBroadcast.scala:206)
	at org.apache.spark.broadcast.TorrentBroadcast._value$lzycompute(TorrentBroadcast.scala:66)
	at org.apache.spark.broadcast.TorrentBroadcast._value(TorrentBroadcast.scala:66)
	at org.apache.spark.broadcast.TorrentBroadcast.getValue(TorrentBroadcast.scala:96)
	at org.apache.spark.broadcast.Broadcast.value(Broadcast.scala:70)
	at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:86)
	at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:53)
	at org.apache.spark.scheduler.Task.run(Task.scala:108)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
	at java.lang.Thread.run(Thread.java:745)
Caused by: java.util.NoSuchElementException
	at org.apache.spark.util.collection.PrimitiveVector$$anon$1.next(PrimitiveVector.scala:58)
	at org.apache.spark.storage.memory.PartiallyUnrolledIterator.next(MemoryStore.scala:700)
	at org.apache.spark.util.CompletionIterator.next(CompletionIterator.scala:30)
	at org.apache.spark.broadcast.TorrentBroadcast$$anonfun$readBroadcastBlock$1$$anonfun$2.apply(TorrentBroadcast.scala:210)
	at org.apache.spark.broadcast.TorrentBroadcast$$anonfun$readBroadcastBlock$1$$anonfun$2.apply(TorrentBroadcast.scala:210)
	at scala.Option.map(Option.scala:146)
	at org.apache.spark.broadcast.TorrentBroadcast$$anonfun$readBroadcastBlock$1.apply(TorrentBroadcast.scala:210)
	at org.apache.spark.util.Utils$.tryOrIOException(Utils.scala:1269)
	... 12 more
```

## How was this patch tested?

Add unit test

Author: Yuming Wang <wgyumg@gmail.com>

Closes #16252 from wangyum/SPARK-18827.
2016-12-18 09:08:02 +00:00
hyukjinkwon 2bc1c95154 [SPARK-18895][TESTS] Fix resource-closing-related and path-related test failures in identified ones on Windows
## What changes were proposed in this pull request?

There are several tests failing due to resource-closing-related and path-related  problems on Windows as below.

- `RPackageUtilsSuite`:

```
- build an R package from a jar end to end *** FAILED *** (1 second, 625 milliseconds)
  java.io.IOException: Unable to delete file: C:\projects\spark\target\tmp\1481729427517-0\a\dep2\d\dep2-d.jar
  at org.apache.commons.io.FileUtils.forceDelete(FileUtils.java:2279)
  at org.apache.commons.io.FileUtils.cleanDirectory(FileUtils.java:1653)
  at org.apache.commons.io.FileUtils.deleteDirectory(FileUtils.java:1535)

- faulty R package shows documentation *** FAILED *** (359 milliseconds)
  java.io.IOException: Unable to delete file: C:\projects\spark\target\tmp\1481729428970-0\dep1-c.jar
  at org.apache.commons.io.FileUtils.forceDelete(FileUtils.java:2279)
  at org.apache.commons.io.FileUtils.cleanDirectory(FileUtils.java:1653)
  at org.apache.commons.io.FileUtils.deleteDirectory(FileUtils.java:1535)

- SparkR zipping works properly *** FAILED *** (47 milliseconds)
  java.util.regex.PatternSyntaxException: Unknown character property name {r} near index 4

C:\projects\spark\target\tmp\1481729429282-0

    ^
  at java.util.regex.Pattern.error(Pattern.java:1955)
  at java.util.regex.Pattern.charPropertyNodeFor(Pattern.java:2781)
```

- `InputOutputMetricsSuite`:

```
- input metrics for old hadoop with coalesce *** FAILED *** (240 milliseconds)
  java.io.IOException: Not a file: file:/C:/projects/spark/core/ignored
  at org.apache.hadoop.mapred.FileInputFormat.getSplits(FileInputFormat.java:277)
  at org.apache.spark.rdd.HadoopRDD.getPartitions(HadoopRDD.scala:202)
  at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:252)

- input metrics with cache and coalesce *** FAILED *** (109 milliseconds)
  java.io.IOException: Not a file: file:/C:/projects/spark/core/ignored
  at org.apache.hadoop.mapred.FileInputFormat.getSplits(FileInputFormat.java:277)
  at org.apache.spark.rdd.HadoopRDD.getPartitions(HadoopRDD.scala:202)
  at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:252)

- input metrics for new Hadoop API with coalesce *** FAILED *** (0 milliseconds)
  java.lang.IllegalArgumentException: Wrong FS: file://C:\projects\spark\target\tmp\spark-9366ec94-dac7-4a5c-a74b-3e7594a692ab\test\InputOutputMetricsSuite.txt, expected: file:///
  at org.apache.hadoop.fs.FileSystem.checkPath(FileSystem.java:642)
  at org.apache.hadoop.fs.FileSystem.makeQualified(FileSystem.java:462)
  at org.apache.hadoop.fs.FilterFileSystem.makeQualified(FilterFileSystem.java:114)

- input metrics when reading text file *** FAILED *** (110 milliseconds)
  java.io.IOException: Not a file: file:/C:/projects/spark/core/ignored
  at org.apache.hadoop.mapred.FileInputFormat.getSplits(FileInputFormat.java:277)
  at org.apache.spark.rdd.HadoopRDD.getPartitions(HadoopRDD.scala:202)
  at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:252)

- input metrics on records read - simple *** FAILED *** (125 milliseconds)
  java.io.IOException: Not a file: file:/C:/projects/spark/core/ignored
  at org.apache.hadoop.mapred.FileInputFormat.getSplits(FileInputFormat.java:277)
  at org.apache.spark.rdd.HadoopRDD.getPartitions(HadoopRDD.scala:202)
  at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:252)

- input metrics on records read - more stages *** FAILED *** (110 milliseconds)
  java.io.IOException: Not a file: file:/C:/projects/spark/core/ignored
  at org.apache.hadoop.mapred.FileInputFormat.getSplits(FileInputFormat.java:277)
  at org.apache.spark.rdd.HadoopRDD.getPartitions(HadoopRDD.scala:202)
  at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:252)

- input metrics on records - New Hadoop API *** FAILED *** (16 milliseconds)
  java.lang.IllegalArgumentException: Wrong FS: file://C:\projects\spark\target\tmp\spark-3f10a1a4-7820-4772-b821-25fd7523bf6f\test\InputOutputMetricsSuite.txt, expected: file:///
  at org.apache.hadoop.fs.FileSystem.checkPath(FileSystem.java:642)
  at org.apache.hadoop.fs.FileSystem.makeQualified(FileSystem.java:462)
  at org.apache.hadoop.fs.FilterFileSystem.makeQualified(FilterFileSystem.java:114)

- input metrics on records read with cache *** FAILED *** (93 milliseconds)
  java.io.IOException: Not a file: file:/C:/projects/spark/core/ignored
  at org.apache.hadoop.mapred.FileInputFormat.getSplits(FileInputFormat.java:277)
  at org.apache.spark.rdd.HadoopRDD.getPartitions(HadoopRDD.scala:202)
  at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:252)

- input read/write and shuffle read/write metrics all line up *** FAILED *** (93 milliseconds)
  java.io.IOException: Not a file: file:/C:/projects/spark/core/ignored
  at org.apache.hadoop.mapred.FileInputFormat.getSplits(FileInputFormat.java:277)
  at org.apache.spark.rdd.HadoopRDD.getPartitions(HadoopRDD.scala:202)
  at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:252)

- input metrics with interleaved reads *** FAILED *** (0 milliseconds)
  java.lang.IllegalArgumentException: Wrong FS: file://C:\projects\spark\target\tmp\spark-2638d893-e89b-47ce-acd0-bbaeee78dd9b\InputOutputMetricsSuite_cart.txt, expected: file:///
  at org.apache.hadoop.fs.FileSystem.checkPath(FileSystem.java:642)
  at org.apache.hadoop.fs.FileSystem.makeQualified(FileSystem.java:462)
  at org.apache.hadoop.fs.FilterFileSystem.makeQualified(FilterFileSystem.java:114)

- input metrics with old CombineFileInputFormat *** FAILED *** (157 milliseconds)
  17947 was not greater than or equal to 300000 (InputOutputMetricsSuite.scala:324)
  org.scalatest.exceptions.TestFailedException:
  at org.scalatest.Assertions$class.newAssertionFailedException(Assertions.scala:500)
  at org.scalatest.FunSuite.newAssertionFailedException(FunSuite.scala:1555)
  at org.scalatest.Assertions$AssertionsHelper.macroAssert(Assertions.scala:466)

- input metrics with new CombineFileInputFormat *** FAILED *** (16 milliseconds)
  java.lang.IllegalArgumentException: Wrong FS: file://C:\projects\spark\target\tmp\spark-11920c08-19d8-4c7c-9fba-28ed72b79f80\test\InputOutputMetricsSuite.txt, expected: file:///
  at org.apache.hadoop.fs.FileSystem.checkPath(FileSystem.java:642)
  at org.apache.hadoop.fs.FileSystem.makeQualified(FileSystem.java:462)
  at org.apache.hadoop.fs.FilterFileSystem.makeQualified(FilterFileSystem.java:114)
```

- `ReplayListenerSuite`:

```
- End-to-end replay *** FAILED *** (121 milliseconds)
  java.io.IOException: No FileSystem for scheme: C
  at org.apache.hadoop.fs.FileSystem.getFileSystemClass(FileSystem.java:2421)
  at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:2428)

- End-to-end replay with compression *** FAILED *** (516 milliseconds)
  java.io.IOException: No FileSystem for scheme: C
  at org.apache.hadoop.fs.FileSystem.getFileSystemClass(FileSystem.java:2421)
  at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:2428)
  at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:88)
```

- `EventLoggingListenerSuite`:

```
- End-to-end event logging *** FAILED *** (7 seconds, 435 milliseconds)
  java.io.IOException: No FileSystem for scheme: C
  at org.apache.hadoop.fs.FileSystem.getFileSystemClass(FileSystem.java:2421)
  at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:2428)
  at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:88)

- End-to-end event logging with compression *** FAILED *** (1 second)
  java.io.IOException: No FileSystem for scheme: C
  at org.apache.hadoop.fs.FileSystem.getFileSystemClass(FileSystem.java:2421)
  at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:2428)
  at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:88)

- Event log name *** FAILED *** (16 milliseconds)
  "file:/[]base-dir/app1" did not equal "file:/[C:/]base-dir/app1" (EventLoggingListenerSuite.scala:123)
  org.scalatest.exceptions.TestFailedException:
  at org.scalatest.Assertions$class.newAssertionFailedException(Assertions.scala:500)
  at org.scalatest.FunSuite.newAssertionFailedException(FunSuite.scala:1555)
  at org.scalatest.Assertions$AssertionsHelper.macroAssert(Assertions.scala:466)
```

This PR proposes to fix the test failures on Windows

## How was this patch tested?

Manually tested via AppVeyor

**Before**

`RPackageUtilsSuite`: https://ci.appveyor.com/project/spark-test/spark/build/273-RPackageUtilsSuite-before
`InputOutputMetricsSuite`: https://ci.appveyor.com/project/spark-test/spark/build/272-InputOutputMetricsSuite-before
`ReplayListenerSuite`: https://ci.appveyor.com/project/spark-test/spark/build/274-ReplayListenerSuite-before
`EventLoggingListenerSuite`: https://ci.appveyor.com/project/spark-test/spark/build/275-EventLoggingListenerSuite-before

**After**

`RPackageUtilsSuite`: https://ci.appveyor.com/project/spark-test/spark/build/270-RPackageUtilsSuite
`InputOutputMetricsSuite`: https://ci.appveyor.com/project/spark-test/spark/build/271-InputOutputMetricsSuite
`ReplayListenerSuite`: https://ci.appveyor.com/project/spark-test/spark/build/277-ReplayListenerSuite-after
`EventLoggingListenerSuite`: https://ci.appveyor.com/project/spark-test/spark/build/278-EventLoggingListenerSuite-after

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16305 from HyukjinKwon/RPackageUtilsSuite-InputOutputMetricsSuite.
2016-12-16 21:32:24 -08:00
Imran Rashid 93cdb8a7d0 [SPARK-8425][CORE] Application Level Blacklisting
## What changes were proposed in this pull request?

This builds upon the blacklisting introduced in SPARK-17675 to add blacklisting of executors and nodes for an entire Spark application.  Resources are blacklisted based on tasks that fail, in tasksets that eventually complete successfully; they are automatically returned to the pool of active resources based on a timeout.  Full details are available in a design doc attached to the jira.
## How was this patch tested?

Added unit tests, ran them via Jenkins, also ran a handful of them in a loop to check for flakiness.

The added tests include:
- verifying BlacklistTracker works correctly
- verifying TaskSchedulerImpl interacts with BlacklistTracker correctly (via a mock BlacklistTracker)
- an integration test for the entire scheduler with blacklisting in a few different scenarios

Author: Imran Rashid <irashid@cloudera.com>
Author: mwws <wei.mao@intel.com>

Closes #14079 from squito/blacklist-SPARK-8425.
2016-12-15 08:29:56 -06:00
hyukjinkwon 169b9d73ee
[SPARK-18830][TESTS] Fix tests in PipedRDDSuite to pass on Windows
## What changes were proposed in this pull request?

This PR proposes to fix the tests failed on Windows as below:

```
[info] - pipe with empty partition *** FAILED *** (672 milliseconds)
[info]   Set(0, 4, 5) did not equal Set(0, 5, 6) (PipedRDDSuite.scala:145)
[info]   org.scalatest.exceptions.TestFailedException:
...
```

In this case, `wc -c` counts the characters on both Windows and Linux but the newlines characters on Windows are `\r\n` which are two. So, the counts ends up one more for each.

```
[info] - test pipe exports map_input_file *** FAILED *** (62 milliseconds)
[info]   java.lang.IllegalStateException: Subprocess exited with status 1. Command ran: printenv map_input_file
[info]   at org.apache.spark.rdd.PipedRDD$$anon$1.hasNext(PipedRDD.scala:178)
...
```

```
[info] - test pipe exports mapreduce_map_input_file *** FAILED *** (172 milliseconds)
[info]   java.lang.IllegalStateException: Subprocess exited with status 1. Command ran: printenv mapreduce_map_input_file
[info]   at org.apache.spark.rdd.PipedRDD$$anon$1.hasNext(PipedRDD.scala:178)
...
```

`printenv` command prints the environment variables; however, when environment variables are set to `ProcessBuilder` as lower-cased keys, `printenv` in Windows ignores and does not print this although it is actually set and accessible. (this was tested in [here](https://ci.appveyor.com/project/spark-test/spark/build/208-PipedRDDSuite) for upper-cases with this [diff](https://github.com/apache/spark/compare/master...spark-test:74d39da) and [here](https://ci.appveyor.com/project/spark-test/spark/build/203-PipedRDDSuite) for lower-cases with this [diff](https://github.com/apache/spark/compare/master...spark-test:fde5e37f28032c15a8d8693ba033a8a779a26317). It seems a bug in `printenv`.
(BTW, note that environment variables on Windows are case-insensitive).

This is (I believe) a thirdparty tool on Windows that resembles `printenv` on Linux (installed in AppVeyor environment or Windows Server 2012 R2). This command does not exist, at least, for Windows 7 and 10 (manually tested).

On Windows, we can use `cmd.exe /C set [varname]` officially for this purpose. We could fix the tests with this in order to test if the environment variable is set.

## How was this patch tested?

Manually tested via AppVeyor.

**Before**
https://ci.appveyor.com/project/spark-test/spark/build/194-PipedRDDSuite

**After**
https://ci.appveyor.com/project/spark-test/spark/build/226-PipedRDDSuite

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16254 from HyukjinKwon/pipe-errors.
2016-12-14 19:27:29 +00:00
hyukjinkwon c6b8eb71a9
[SPARK-18842][TESTS][LAUNCHER] De-duplicate paths in classpaths in commands for local-cluster mode to work around the path length limitation on Windows
## What changes were proposed in this pull request?

Currently, some tests are being failed and hanging on Windows due to this problem. For the reason in SPARK-18718, some tests using `local-cluster` mode were disabled on Windows due to the length limitation by paths given to classpaths.

The limitation seems roughly 32K (see the [blog in MS](https://blogs.msdn.microsoft.com/oldnewthing/20031210-00/?p=41553/) and [another reference](https://support.thoughtworks.com/hc/en-us/articles/213248526-Getting-around-maximum-command-line-length-is-32767-characters-on-Windows)) but in `local-cluster` mode, executors were being launched as processes with the command such as [here](https://gist.github.com/HyukjinKwon/5bc81061c250d4af5a180869b59d42ea) in (only) tests.

This length is roughly 40K due to the classpaths given to `java` command. However, it seems duplicates are almost half of them. So, if we deduplicate the paths, it seems reduced to roughly 20K with the command, [here](https://gist.github.com/HyukjinKwon/dad0c8db897e5e094684a2dc6a417790).

Maybe, we should consider as some more paths are added in the future but it seems better than disabling all the tests for now with minimised changes.

Therefore, this PR proposes to deduplicate the paths in classpaths in case of launching executors as processes in `local-cluster` mode.

## How was this patch tested?

Existing tests in `ShuffleSuite` and `BroadcastJoinSuite` manually via AppVeyor

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16266 from HyukjinKwon/disable-local-cluster-tests.
2016-12-14 19:24:24 +00:00
Imran Rashid ac013ea589 [SPARK-18846][SCHEDULER] Fix flakiness in SchedulerIntegrationSuite
There is a small race in SchedulerIntegrationSuite.
The test assumes that the taskscheduler thread
processing that last task will finish before the DAGScheduler processes
the task event and notifies the job waiter, but that is not 100%
guaranteed.

ran the test locally a bunch of times, never failed, though admittedly
it never failed locally for me before either.  However I am nearly 100%
certain this is what caused the failure of one jenkins build
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/68694/consoleFull
(which is long gone now, sorry -- I fixed it as part of
https://github.com/apache/spark/pull/14079 initially)

Author: Imran Rashid <irashid@cloudera.com>

Closes #16270 from squito/sched_integ_flakiness.
2016-12-14 12:27:01 -06:00
Shixiong Zhu fb3081d3b3 [SPARK-13747][CORE] Fix potential ThreadLocal leaks in RPC when using ForkJoinPool
## What changes were proposed in this pull request?

Some places in SQL may call `RpcEndpointRef.askWithRetry` (e.g., ParquetFileFormat.buildReader -> SparkContext.broadcast -> ... -> BlockManagerMaster.updateBlockInfo -> RpcEndpointRef.askWithRetry), which will finally call `Await.result`. It may cause `java.lang.IllegalArgumentException: spark.sql.execution.id is already set` when running in Scala ForkJoinPool.

This PR includes the following changes to fix this issue:

- Remove `ThreadUtils.awaitResult`
- Rename `ThreadUtils. awaitResultInForkJoinSafely` to `ThreadUtils.awaitResult`
- Replace `Await.result` in RpcTimeout with `ThreadUtils.awaitResult`.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #16230 from zsxwing/fix-SPARK-13747.
2016-12-13 09:53:22 -08:00
Marcelo Vanzin bc59951bab [SPARK-18773][CORE] Make commons-crypto config translation consistent.
This change moves the logic that translates Spark configuration to
commons-crypto configuration to the network-common module. It also
extends TransportConf and ConfigProvider to provide the necessary
interfaces for the translation to work.

As part of the change, I removed SystemPropertyConfigProvider, which
was mostly used as an "empty config" in unit tests, and adjusted the
very few tests that required a specific config.

I also changed the config keys for AES encryption to live under the
"spark.network." namespace, which is more correct than their previous
names under "spark.authenticate.".

Tested via existing unit test.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #16200 from vanzin/SPARK-18773.
2016-12-12 16:27:04 -08:00
Steve Loughran 586d198228 [SPARK-15844][CORE] HistoryServer doesn't come up if spark.authenticate = true
## What changes were proposed in this pull request?

During history server startup, the spark configuration is examined. If security.authentication is
set, log at debug and set the value to false, so that {{SecurityManager}} can be created.

## How was this patch tested?

A new test in `HistoryServerSuite` sets the `spark.authenticate` property to true, tries to create a security manager via a new package-private method `HistoryServer.createSecurityManager(SparkConf)`. This is the method used in `HistoryServer.main`. All other instantiations of a security manager in `HistoryServerSuite` have been switched to the new method, for consistency with the production code.

Author: Steve Loughran <stevel@apache.org>

Closes #13579 from steveloughran/history/SPARK-15844-security.
2016-12-12 12:24:53 -08:00
hyukjinkwon e094d01156
[SPARK-18803][TESTS] Fix JarEntry-related & path-related test failures and skip some tests by path length limitation on Windows
## What changes were proposed in this pull request?

This PR proposes to fix some tests being failed on Windows as below for several problems.

### Incorrect path handling

- FileSuite
  ```
  [info] - binary file input as byte array *** FAILED *** (500 milliseconds)
  [info]   "file:/C:/projects/spark/target/tmp/spark-e7c3a3b8-0a4b-4a7f-9ebe-7c4883e48624/record-bytestream-00000.bin" did not contain "C:\projects\spark\target\tmp\spark-e7c3a3b8-0a4b-4a7f-9ebe-7c4883e48624\record-bytestream-00000.bin" (FileSuite.scala:258)
  [info]   org.scalatest.exceptions.TestFailedException:
  [info]   at org.scalatest.Assertions$class.newAssertionFailedException(Assertions.scala:500)
  ...
  ```
  ```
  [info] - Get input files via old Hadoop API *** FAILED *** (1 second, 94 milliseconds)
  [info]   Set("/C:/projects/spark/target/tmp/spark-cf5b1f8b-c5ed-43e0-8d17-546ebbfa8200/output/part-00000", "/C:/projects/spark/target/tmp/spark-cf5b1f8b-c5ed-43e0-8d17-546ebbfa8200/output/part-00001") did not equal Set("C:\projects\spark\target\tmp\spark-cf5b1f8b-c5ed-43e0-8d17-546ebbfa8200\output/part-00000", "C:\projects\spark\target\tmp\spark-cf5b1f8b-c5ed-43e0-8d17-546ebbfa8200\output/part-00001") (FileSuite.scala:535)
  [info]   org.scalatest.exceptions.TestFailedException:
  [info]   at org.scalatest.Assertions$class.newAssertionFailedException(Assertions.scala:500)
  ...
  ```

  ```
  [info] - Get input files via new Hadoop API *** FAILED *** (313 milliseconds)
  [info]   Set("/C:/projects/spark/target/tmp/spark-12bc1540-1111-4df6-9c4d-79e0e614407c/output/part-00000", "/C:/projects/spark/target/tmp/spark-12bc1540-1111-4df6-9c4d-79e0e614407c/output/part-00001") did not equal Set("C:\projects\spark\target\tmp\spark-12bc1540-1111-4df6-9c4d-79e0e614407c\output/part-00000", "C:\projects\spark\target\tmp\spark-12bc1540-1111-4df6-9c4d-79e0e614407c\output/part-00001") (FileSuite.scala:549)
  [info]   org.scalatest.exceptions.TestFailedException:
  ...
  ```

- TaskResultGetterSuite

  ```
  [info] - handling results larger than max RPC message size *** FAILED *** (1 second, 579 milliseconds)
  [info]   1 did not equal 0 Expect result to be removed from the block manager. (TaskResultGetterSuite.scala:129)
  [info]   org.scalatest.exceptions.TestFailedException:
  [info]   ...
  [info]   Cause: java.net.URISyntaxException: Illegal character in path at index 12: string:///C:\projects\spark\target\tmp\spark-93c485af-68da-440f-a907-aac7acd5fc25\repro\MyException.java
  [info]   at java.net.URI$Parser.fail(URI.java:2848)
  [info]   at java.net.URI$Parser.checkChars(URI.java:3021)
  ...
  ```
  ```
  [info] - failed task deserialized with the correct classloader (SPARK-11195) *** FAILED *** (0 milliseconds)
  [info]   java.lang.IllegalArgumentException: Illegal character in path at index 12: string:///C:\projects\spark\target\tmp\spark-93c485af-68da-440f-a907-aac7acd5fc25\repro\MyException.java
  [info]   at java.net.URI.create(URI.java:852)
  ...
  ```

- SparkSubmitSuite

  ```
  [info]   java.lang.IllegalArgumentException: Illegal character in path at index 12: string:///C:\projects\spark\target\tmp\1481210831381-0\870903339\MyLib.java
  [info]   at java.net.URI.create(URI.java:852)
  [info]   at org.apache.spark.TestUtils$.org$apache$spark$TestUtils$$createURI(TestUtils.scala:112)
  ...
  ```

### Incorrect separate for JarEntry

After the path fix from above, then `TaskResultGetterSuite` throws another exception as below:

```
[info] - failed task deserialized with the correct classloader (SPARK-11195) *** FAILED *** (907 milliseconds)
[info]   java.lang.ClassNotFoundException: repro.MyException
[info]   at java.net.URLClassLoader.findClass(URLClassLoader.java:381)
...
```

This is because `Paths.get` concatenates the given paths to an OS-specific path (Windows `\` and Linux `/`). However, for `JarEntry` we should comply ZIP specification meaning it should be always `/` according to ZIP specification.

See `4.4.17 file name: (Variable)` in https://pkware.cachefly.net/webdocs/casestudies/APPNOTE.TXT

### Long path problem on Windows

Some tests in `ShuffleSuite` via `ShuffleNettySuite` were skipped due to the same reason with SPARK-18718

## How was this patch tested?

Manually via AppVeyor.

**Before**

- `FileSuite`, `TaskResultGetterSuite`,`SparkSubmitSuite`
  https://ci.appveyor.com/project/spark-test/spark/build/164-tmp-windows-base (please grep each to check each)
- `ShuffleSuite`
  https://ci.appveyor.com/project/spark-test/spark/build/157-tmp-windows-base

**After**

- `FileSuite`
  https://ci.appveyor.com/project/spark-test/spark/build/166-FileSuite
- `TaskResultGetterSuite`
  https://ci.appveyor.com/project/spark-test/spark/build/173-TaskResultGetterSuite
- `SparkSubmitSuite`
  https://ci.appveyor.com/project/spark-test/spark/build/167-SparkSubmitSuite
- `ShuffleSuite`
  https://ci.appveyor.com/project/spark-test/spark/build/176-ShuffleSuite

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16234 from HyukjinKwon/test-errors-windows.
2016-12-10 19:55:22 +00:00
Davies Liu cf33a86285 [SPARK-4105] retry the fetch or stage if shuffle block is corrupt
## What changes were proposed in this pull request?

There is an outstanding issue that existed for a long time: Sometimes the shuffle blocks are corrupt and can't be decompressed. We recently hit this in three different workloads, sometimes we can reproduce it by every try, sometimes can't. I also found that when the corruption happened, the beginning and end of the blocks are correct, the corruption happen in the middle. There was one case that the string of block id is corrupt by one character. It seems that it's very likely the corruption is introduced by some weird machine/hardware, also the checksum (16 bits) in TCP is not strong enough to identify all the corruption.

Unfortunately, Spark does not have checksum for shuffle blocks or broadcast, the job will fail if any corruption happen in the shuffle block from disk, or broadcast blocks during network. This PR try to detect the corruption after fetching shuffle blocks by decompressing them, because most of the compression already have checksum in them. It will retry the block, or failed with FetchFailure, so the previous stage could be retried on different (still random) machines.

Checksum for broadcast will be added by another PR.

## How was this patch tested?

Added unit tests

Author: Davies Liu <davies@databricks.com>

Closes #15923 from davies/detect_corrupt.
2016-12-09 15:44:22 -08:00
Xiangrui Meng fd48d80a61 [SPARK-17822][R] Make JVMObjectTracker a member variable of RBackend
## What changes were proposed in this pull request?

* This PR changes `JVMObjectTracker` from `object` to `class` and let its instance associated with each RBackend. So we can manage the lifecycle of JVM objects when there are multiple `RBackend` sessions. `RBackend.close` will clear the object tracker explicitly.
* I assume that `SQLUtils` and `RRunner` do not need to track JVM instances, which could be wrong.
* Small refactor of `SerDe.sqlSerDe` to increase readability.

## How was this patch tested?

* Added unit tests for `JVMObjectTracker`.
* Wait for Jenkins to run full tests.

Author: Xiangrui Meng <meng@databricks.com>

Closes #16154 from mengxr/SPARK-17822.
2016-12-09 07:51:46 -08:00
Shixiong Zhu 26432df9cc [SPARK-18751][CORE] Fix deadlock when SparkContext.stop is called in Utils.tryOrStopSparkContext
## What changes were proposed in this pull request?

When `SparkContext.stop` is called in `Utils.tryOrStopSparkContext` (the following three places), it will cause deadlock because the `stop` method needs to wait for the thread running `stop` to exit.

- ContextCleaner.keepCleaning
- LiveListenerBus.listenerThread.run
- TaskSchedulerImpl.start

This PR adds `SparkContext.stopInNewThread` and uses it to eliminate the potential deadlock. I also removed my changes in #15775 since they are not necessary now.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #16178 from zsxwing/fix-stop-deadlock.
2016-12-08 11:54:04 -08:00
Ergin Seyfe ed8869ebbf [SPARK-8617][WEBUI] HistoryServer: Include in-progress files during cleanup
## What changes were proposed in this pull request?
- Removed the`attempt.completed ` filter so cleaner would include the orphan inprogress files.
- Use loading time for inprogress files as lastUpdated. Keep using the modTime for completed files. First one will prevent deletion of inprogress job files. Second one will ensure that lastUpdated time won't change for completed jobs in an event of HistoryServer reboot.

## How was this patch tested?
Added new unittests and via existing tests.

Author: Ergin Seyfe <eseyfe@fb.com>

Closes #16165 from seyfe/clear_old_inprogress_files.
2016-12-08 10:21:09 -08:00
hyukjinkwon 7f3c778fd0
[SPARK-18718][TESTS] Skip some test failures due to path length limitation and fix tests to pass on Windows
## What changes were proposed in this pull request?

There are some tests failed on Windows due to the wrong format of path and the limitation of path length as below:

This PR proposes both to fix the failed tests by fixing the path for the tests below:

- `InsertSuite`
  ```
  Exception encountered when attempting to run a suite with class name: org.apache.spark.sql.sources.InsertSuite *** ABORTED *** (12 seconds, 547 milliseconds)
      org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-177945ef-9128-42b4-8c07-de31f78bbbd6;
      at org.apache.spark.sql.execution.datasources.DataSource$$anonfun$14.apply(DataSource.scala:382)
      at org.apache.spark.sql.execution.datasources.DataSource$$anonfun$14.apply(DataSource.scala:370)
      at scala.collection.TraversableLike$$anonfun$flatMap$1.apply(TraversableLike.scala:241)
  ```

- `PathOptionSuite`
  ```
  - path option also exist for write path *** FAILED *** (1 second, 93 milliseconds)
    "C:[projectsspark	arget	mp]spark-5ab34a58-df8d-..." did not equal "C:[\projects\spark\target\tmp\]spark-5ab34a58-df8d-..." (PathOptionSuite.scala:93)
    org.scalatest.exceptions.TestFailedException:
        at org.scalatest.Assertions$class.newAssertionFailedException(Assertions.scala:500)
        at org.scalatest.FunSuite.newAssertionFailedException(FunSuite.scala:1555)
    ...
  ```

- `UDFSuite`
  ```
  - SPARK-8005 input_file_name *** FAILED *** (2 seconds, 234 milliseconds)
    "file:///C:/projects/spark/target/tmp/spark-e4e5720a-2006-48f9-8b11-797bf59794bf/part-00001-26fb05e4-603d-471d-ae9d-b9549e0c7765.snappy.parquet" did not contain "C:\projects\spark\target\tmp\spark-e4e5720a-2006-48f9-8b11-797bf59794bf" (UDFSuite.scala:67)
    org.scalatest.exceptions.TestFailedException:
      at org.scalatest.Assertions$class.newAssertionFailedException(Assertions.scala:500)
      at org.scalatest.FunSuite.newAssertionFailedException(FunSuite.scala:1555)
    ...
  ```

and to skip the tests belows which are being failed on Windows due to path length limitation.

- `SparkLauncherSuite`
  ```
  Test org.apache.spark.launcher.SparkLauncherSuite.testChildProcLauncher failed: java.lang.AssertionError: expected:<0> but was:<1>, took 0.062 sec
    at org.apache.spark.launcher.SparkLauncherSuite.testChildProcLauncher(SparkLauncherSuite.java:177)
      ...
  ```

  The stderr from the process is `The filename or extension is too long` which is equivalent to the one below.

- `BroadcastJoinSuite`
  ```
  04:09:40.882 ERROR org.apache.spark.deploy.worker.ExecutorRunner: Error running executor
  java.io.IOException: Cannot run program "C:\Progra~1\Java\jdk1.8.0\bin\java" (in directory "C:\projects\spark\work\app-20161205040542-0000\51658"): CreateProcess error=206, The filename or extension is too long
      at java.lang.ProcessBuilder.start(ProcessBuilder.java:1048)
      at org.apache.spark.deploy.worker.ExecutorRunner.org$apache$spark$deploy$worker$ExecutorRunner$$fetchAndRunExecutor(ExecutorRunner.scala:167)
      at org.apache.spark.deploy.worker.ExecutorRunner$$anon$1.run(ExecutorRunner.scala:73)
  Caused by: java.io.IOException: CreateProcess error=206, The filename or extension is too long
      at java.lang.ProcessImpl.create(Native Method)
      at java.lang.ProcessImpl.<init>(ProcessImpl.java:386)
      at java.lang.ProcessImpl.start(ProcessImpl.java:137)
      at java.lang.ProcessBuilder.start(ProcessBuilder.java:1029)
      ... 2 more
  04:09:40.929 ERROR org.apache.spark.deploy.worker.ExecutorRunner: Error running executor

    (appearently infinite same error messages)

  ...
  ```

## How was this patch tested?

Manually tested via AppVeyor.

**Before**

`InsertSuite`: https://ci.appveyor.com/project/spark-test/spark/build/148-InsertSuite-pr
`PathOptionSuite`: https://ci.appveyor.com/project/spark-test/spark/build/139-PathOptionSuite-pr
`UDFSuite`: https://ci.appveyor.com/project/spark-test/spark/build/143-UDFSuite-pr
`SparkLauncherSuite`: https://ci.appveyor.com/project/spark-test/spark/build/141-SparkLauncherSuite-pr
`BroadcastJoinSuite`: https://ci.appveyor.com/project/spark-test/spark/build/145-BroadcastJoinSuite-pr

**After**

`PathOptionSuite`: https://ci.appveyor.com/project/spark-test/spark/build/140-PathOptionSuite-pr
`SparkLauncherSuite`: https://ci.appveyor.com/project/spark-test/spark/build/142-SparkLauncherSuite-pr
`UDFSuite`: https://ci.appveyor.com/project/spark-test/spark/build/144-UDFSuite-pr
`InsertSuite`: https://ci.appveyor.com/project/spark-test/spark/build/147-InsertSuite-pr
`BroadcastJoinSuite`: https://ci.appveyor.com/project/spark-test/spark/build/149-BroadcastJoinSuite-pr

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16147 from HyukjinKwon/fix-tests.
2016-12-08 23:02:05 +08:00
Sean Owen 79f5f281bb
[SPARK-18678][ML] Skewed reservoir sampling in SamplingUtils
## What changes were proposed in this pull request?

Fix reservoir sampling bias for small k. An off-by-one error meant that the probability of replacement was slightly too high -- k/(l-1) after l element instead of k/l, which matters for small k.

## How was this patch tested?

Existing test plus new test case.

Author: Sean Owen <sowen@cloudera.com>

Closes #16129 from srowen/SPARK-18678.
2016-12-07 17:34:45 +08:00
Liang-Chi Hsieh dbf842b7a8 [SPARK-18666][WEB UI] Remove the codes checking deprecated config spark.sql.unsafe.enabled
## What changes were proposed in this pull request?

`spark.sql.unsafe.enabled` is deprecated since 1.6. There still are codes in UI to check it. We should remove it and clean the codes.

## How was this patch tested?

Changes to related existing unit test.

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

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

Closes #16095 from viirya/remove-deprecated-config-code.
2016-12-01 01:57:58 -08:00
Marcelo Vanzin 93e9d880bf [SPARK-18546][CORE] Fix merging shuffle spills when using encryption.
The problem exists because it's not possible to just concatenate encrypted
partition data from different spill files; currently each partition would
have its own initial vector to set up encryption, and the final merged file
should contain a single initial vector for each merged partiton, otherwise
iterating over each record becomes really hard.

To fix that, UnsafeShuffleWriter now decrypts the partitions when merging,
so that the merged file contains a single initial vector at the start of
the partition data.

Because it's not possible to do that using the fast transferTo path, when
encryption is enabled UnsafeShuffleWriter will revert back to using file
streams when merging. It may be possible to use a hybrid approach when
using encryption, using an intermediate direct buffer when reading from
files and encrypting the data, but that's better left for a separate patch.

As part of the change I made DiskBlockObjectWriter take a SerializerManager
instead of a "wrap stream" closure, since that makes it easier to test the
code without having to mock SerializerManager functionality.

Tested with newly added unit tests (UnsafeShuffleWriterSuite for the write
side and ExternalAppendOnlyMapSuite for integration), and by running some
apps that failed without the fix.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #15982 from vanzin/SPARK-18546.
2016-11-30 14:10:32 -08:00
Josh Rosen c51c772594 [SPARK-18640] Add synchronization to TaskScheduler.runningTasksByExecutors
## What changes were proposed in this pull request?

The method `TaskSchedulerImpl.runningTasksByExecutors()` accesses the mutable `executorIdToRunningTaskIds` map without proper synchronization. In addition, as markhamstra pointed out in #15986, the signature's use of parentheses is a little odd given that this is a pure getter method.

This patch fixes both issues.

## How was this patch tested?

Covered by existing tests.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #16073 from JoshRosen/runningTasksByExecutors-thread-safety.
2016-11-30 14:47:41 -05:00
uncleGen 56c82edabd [SPARK-18617][CORE][STREAMING] Close "kryo auto pick" feature for Spark Streaming
## What changes were proposed in this pull request?

#15992 provided a solution to fix the bug, i.e. **receiver data can not be deserialized properly**. As zsxwing said, it is a critical bug, but we should not break APIs between maintenance releases. It may be a rational choice to close auto pick kryo serializer for Spark Streaming in the first step. I will continue #15992 to optimize the solution.

## How was this patch tested?

existing ut

Author: uncleGen <hustyugm@gmail.com>

Closes #16052 from uncleGen/SPARK-18617.
2016-11-29 23:45:06 -08:00
Josh Rosen 9a02f68212 [SPARK-18553][CORE] Fix leak of TaskSetManager following executor loss
## What changes were proposed in this pull request?

_This is the master branch version of #15986; the original description follows:_

This patch fixes a critical resource leak in the TaskScheduler which could cause RDDs and ShuffleDependencies to be kept alive indefinitely if an executor with running tasks is permanently lost and the associated stage fails.

This problem was originally identified by analyzing the heap dump of a driver belonging to a cluster that had run out of shuffle space. This dump contained several `ShuffleDependency` instances that were retained by `TaskSetManager`s inside the scheduler but were not otherwise referenced. Each of these `TaskSetManager`s was considered a "zombie" but had no running tasks and therefore should have been cleaned up. However, these zombie task sets were still referenced by the `TaskSchedulerImpl.taskIdToTaskSetManager` map.

Entries are added to the `taskIdToTaskSetManager` map when tasks are launched and are removed inside of `TaskScheduler.statusUpdate()`, which is invoked by the scheduler backend while processing `StatusUpdate` messages from executors. The problem with this design is that a completely dead executor will never send a `StatusUpdate`. There is [some code](072f4c518c/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala (L338)) in `statusUpdate` which handles tasks that exit with the `TaskState.LOST` state (which is supposed to correspond to a task failure triggered by total executor loss), but this state only seems to be used in Mesos fine-grained mode. There doesn't seem to be any code which performs per-task state cleanup for tasks that were running on an executor that completely disappears without sending any sort of final death message. The `executorLost` and [`removeExecutor`](072f4c518c/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala (L527)) methods don't appear to perform any cleanup of the `taskId -> *` mappings, causing the leaks observed here.

This patch's fix is to maintain a `executorId -> running task id` mapping so that these `taskId -> *` maps can be properly cleaned up following an executor loss.

There are some potential corner-case interactions that I'm concerned about here, especially some details in [the comment](072f4c518c/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala (L523)) in `removeExecutor`, so I'd appreciate a very careful review of these changes.

## How was this patch tested?

I added a new unit test to `TaskSchedulerImplSuite`.

/cc kayousterhout and markhamstra, who reviewed #15986.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #16045 from JoshRosen/fix-leak-following-total-executor-loss-master.
2016-11-29 16:27:25 -08:00
Marcelo Vanzin 8b325b17ec [SPARK-18547][CORE] Propagate I/O encryption key when executors register.
This change modifies the method used to propagate encryption keys used during
shuffle. Instead of relying on YARN's UserGroupInformation credential propagation,
this change explicitly distributes the key using the messages exchanged between
driver and executor during registration. When RPC encryption is enabled, this means
key propagation is also secure.

This allows shuffle encryption to work in non-YARN mode, which means that it's
easier to write unit tests for areas of the code that are affected by the feature.

The key is stored in the SecurityManager; because there are many instances of
that class used in the code, the key is only guaranteed to exist in the instance
managed by the SparkEnv. This path was chosen to avoid storing the key in the
SparkConf, which would risk having the key being written to disk as part of the
configuration (as, for example, is done when starting YARN applications).

Tested by new and existing unit tests (which were moved from the YARN module to
core), and by running apps with shuffle encryption enabled.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #15981 from vanzin/SPARK-18547.
2016-11-28 21:10:57 -08:00
Imran Rashid 8b1609bebe [SPARK-18117][CORE] Add test for TaskSetBlacklist
## What changes were proposed in this pull request?

This adds tests to verify the interaction between TaskSetBlacklist and
TaskSchedulerImpl.  TaskSetBlacklist was introduced by SPARK-17675 but
it neglected to add these tests.

This change does not fix any bugs -- it is just for increasing test
coverage.
## How was this patch tested?

Jenkins

Author: Imran Rashid <irashid@cloudera.com>

Closes #15644 from squito/taskset_blacklist_test_update.
2016-11-28 13:47:09 -06:00
Mark Grover 237c3b9642 [SPARK-18535][UI][YARN] Redact sensitive information from Spark logs and UI
## What changes were proposed in this pull request?

This patch adds a new property called `spark.secret.redactionPattern` that
allows users to specify a scala regex to decide which Spark configuration
properties and environment variables in driver and executor environments
contain sensitive information. When this regex matches the property or
environment variable name, its value is redacted from the environment UI and
various logs like YARN and event logs.

This change uses this property to redact information from event logs and YARN
logs. It also, updates the UI code to adhere to this property instead of
hardcoding the logic to decipher which properties are sensitive.

Here's an image of the UI post-redaction:
![image](https://cloud.githubusercontent.com/assets/1709451/20506215/4cc30654-b007-11e6-8aee-4cde253fba2f.png)

Here's the text in the YARN logs, post-redaction:
``HADOOP_CREDSTORE_PASSWORD -> *********(redacted)``

Here's the text in the event logs, post-redaction:
``...,"spark.executorEnv.HADOOP_CREDSTORE_PASSWORD":"*********(redacted)","spark.yarn.appMasterEnv.HADOOP_CREDSTORE_PASSWORD":"*********(redacted)",...``

## How was this patch tested?
1. Unit tests are added to ensure that redaction works.
2. A YARN job reading data off of S3 with confidential information
(hadoop credential provider password) being provided in the environment
variables of driver and executor. And, afterwards, logs were grepped to make
sure that no mention of secret password was present. It was also ensure that
the job was able to read the data off of S3 correctly, thereby ensuring that
the sensitive information was being trickled down to the right places to read
the data.
3. The event logs were checked to make sure no mention of secret password was
present.
4. UI environment tab was checked to make sure there was no secret information
being displayed.

Author: Mark Grover <mark@apache.org>

Closes #15971 from markgrover/master_redaction.
2016-11-28 08:59:47 -08:00
Kazuaki Ishizaki d93b655247 [SPARK-18458][CORE] Fix signed integer overflow problem at an expression in RadixSort.java
## What changes were proposed in this pull request?

This PR avoids that a result of an expression is negative due to signed integer overflow (e.g. 0x10?????? * 8 < 0). This PR casts each operand to `long` before executing a calculation. Since the result is interpreted as long, the result of the expression is positive.

## How was this patch tested?

Manually executed query82 of TPC-DS with 100TB

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

Closes #15907 from kiszk/SPARK-18458.
2016-11-19 21:50:20 -08:00
Stavros Kontopoulos ea77c81ec0 [SPARK-17062][MESOS] add conf option to mesos dispatcher
Adds --conf option to set spark configuration properties in mesos dispacther.
Properties provided with --conf take precedence over properties within the properties file.
The reason for this PR is that for simple configuration or testing purposes we need to provide a property file (ideally a shared one for a cluster) even if we just provide a single property.

Manually tested.

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

Closes #14650 from skonto/dipatcher_conf.
2016-11-19 16:04:49 -08:00
hyukjinkwon d5b1d5fc80
[SPARK-18445][BUILD][DOCS] Fix the markdown for Note:/NOTE:/Note that/'''Note:''' across Scala/Java API documentation
## What changes were proposed in this pull request?

It seems in Scala/Java,

- `Note:`
- `NOTE:`
- `Note that`
- `'''Note:'''`
- `note`

This PR proposes to fix those to `note` to be consistent.

**Before**

- Scala
  ![2016-11-17 6 16 39](https://cloud.githubusercontent.com/assets/6477701/20383180/1a7aed8c-acf2-11e6-9611-5eaf6d52c2e0.png)

- Java
  ![2016-11-17 6 14 41](https://cloud.githubusercontent.com/assets/6477701/20383096/c8ffc680-acf1-11e6-914a-33460bf1401d.png)

**After**

- Scala
  ![2016-11-17 6 16 44](https://cloud.githubusercontent.com/assets/6477701/20383167/09940490-acf2-11e6-937a-0d5e1dc2cadf.png)

- Java
  ![2016-11-17 6 13 39](https://cloud.githubusercontent.com/assets/6477701/20383132/e7c2a57e-acf1-11e6-9c47-b849674d4d88.png)

## How was this patch tested?

The notes were found via

```bash
grep -r "NOTE: " . | \ # Note:|NOTE:|Note that|'''Note:'''
grep -v "// NOTE: " | \  # starting with // does not appear in API documentation.
grep -E '.scala|.java' | \ # java/scala files
grep -v Suite | \ # exclude tests
grep -v Test | \ # exclude tests
grep -e 'org.apache.spark.api.java' \ # packages appear in API documenation
-e 'org.apache.spark.api.java.function' \ # note that this is a regular expression. So actual matches were mostly `org/apache/spark/api/java/functions ...`
-e 'org.apache.spark.api.r' \
...
```

```bash
grep -r "Note that " . | \ # Note:|NOTE:|Note that|'''Note:'''
grep -v "// Note that " | \  # starting with // does not appear in API documentation.
grep -E '.scala|.java' | \ # java/scala files
grep -v Suite | \ # exclude tests
grep -v Test | \ # exclude tests
grep -e 'org.apache.spark.api.java' \ # packages appear in API documenation
-e 'org.apache.spark.api.java.function' \
-e 'org.apache.spark.api.r' \
...
```

```bash
grep -r "Note: " . | \ # Note:|NOTE:|Note that|'''Note:'''
grep -v "// Note: " | \  # starting with // does not appear in API documentation.
grep -E '.scala|.java' | \ # java/scala files
grep -v Suite | \ # exclude tests
grep -v Test | \ # exclude tests
grep -e 'org.apache.spark.api.java' \ # packages appear in API documenation
-e 'org.apache.spark.api.java.function' \
-e 'org.apache.spark.api.r' \
...
```

```bash
grep -r "'''Note:'''" . | \ # Note:|NOTE:|Note that|'''Note:'''
grep -v "// '''Note:''' " | \  # starting with // does not appear in API documentation.
grep -E '.scala|.java' | \ # java/scala files
grep -v Suite | \ # exclude tests
grep -v Test | \ # exclude tests
grep -e 'org.apache.spark.api.java' \ # packages appear in API documenation
-e 'org.apache.spark.api.java.function' \
-e 'org.apache.spark.api.r' \
...
```

And then fixed one by one comparing with API documentation/access modifiers.

After that, manually tested via `jekyll build`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #15889 from HyukjinKwon/SPARK-18437.
2016-11-19 11:24:15 +00:00
hyukjinkwon 40d59ff5ea
[SPARK-18422][CORE] Fix wholeTextFiles test to pass on Windows in JavaAPISuite
## What changes were proposed in this pull request?

This PR fixes the test `wholeTextFiles` in `JavaAPISuite.java`. This is failed due to the different path format on Windows.

For example, the path in `container` was

```
C:\projects\spark\target\tmp\1478967560189-0/part-00000
```

whereas `new URI(res._1()).getPath()` was as below:

```
/C:/projects/spark/target/tmp/1478967560189-0/part-00000
```

## How was this patch tested?

Tests in `JavaAPISuite.java`.

Tested via AppVeyor.

**Before**
Build: https://ci.appveyor.com/project/spark-test/spark/build/63-JavaAPISuite-1
Diff: https://github.com/apache/spark/compare/master...spark-test:JavaAPISuite-1

```
[info] Test org.apache.spark.JavaAPISuite.wholeTextFiles started
[error] Test org.apache.spark.JavaAPISuite.wholeTextFiles failed: java.lang.AssertionError: expected:<spark is easy to use.
[error] > but was:<null>, took 0.578 sec
[error]     at org.apache.spark.JavaAPISuite.wholeTextFiles(JavaAPISuite.java:1089)
...
```

**After**
Build started: [CORE] `org.apache.spark.JavaAPISuite` [![PR-15866](https://ci.appveyor.com/api/projects/status/github/spark-test/spark?branch=198DDA52-F201-4D2B-BE2F-244E0C1725B2&svg=true)](https://ci.appveyor.com/project/spark-test/spark/branch/198DDA52-F201-4D2B-BE2F-244E0C1725B2)
Diff: https://github.com/apache/spark/compare/master...spark-test:198DDA52-F201-4D2B-BE2F-244E0C1725B2

```
[info] Test org.apache.spark.JavaAPISuite.wholeTextFiles started
...
```

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #15866 from HyukjinKwon/SPARK-18422.
2016-11-18 21:45:18 +00:00
wm624@hotmail.com 22a9d064e9
[SPARK-14914][CORE] Fix Resource not closed after using, for unit tests and example
## What changes were proposed in this pull request?

This is a follow-up work of #15618.

Close file source;
For any newly created streaming context outside the withContext, explicitly close the context.

## How was this patch tested?

Existing unit tests.

Author: wm624@hotmail.com <wm624@hotmail.com>

Closes #15818 from wangmiao1981/rtest.
2016-11-10 10:54:36 +00:00
jiangxingbo 64fbdf1aa9 [SPARK-18191][CORE][FOLLOWUP] Call setConf if OutputFormat is Configurable.
## What changes were proposed in this pull request?

We should call `setConf` if `OutputFormat` is `Configurable`, this should be done before we create `OutputCommitter` and `RecordWriter`.
This is follow up of #15769, see discussion [here](https://github.com/apache/spark/pull/15769/files#r87064229)

## How was this patch tested?

Add test of this case in `PairRDDFunctionsSuite`.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #15823 from jiangxb1987/config-format.
2016-11-09 13:14:26 -08:00
Vinayak 06a13ecca7 [SPARK-16808][CORE] History Server main page does not honor APPLICATION_WEB_PROXY_BASE
## What changes were proposed in this pull request?

Application links generated on the history server UI no longer (regression from 1.6) contain the configured spark.ui.proxyBase in the links. To address this, made the uiRoot available globally to all javascripts for Web UI. Updated the mustache template (historypage-template.html) to include the uiroot for rendering links to the applications.

The existing test was not sufficient to verify the scenario where ajax call is used to populate the application listing template, so added a new selenium test case to cover this scenario.

## How was this patch tested?

Existing tests and a new unit test.
No visual changes to the UI.

Author: Vinayak <vijoshi5@in.ibm.com>

Closes #15742 from vijoshi/SPARK-16808_master.
2016-11-09 10:40:14 -08:00
Shixiong Zhu b6de0c98c7 [SPARK-18280][CORE] Fix potential deadlock in StandaloneSchedulerBackend.dead
## What changes were proposed in this pull request?

"StandaloneSchedulerBackend.dead" is called in a RPC thread, so it should not call "SparkContext.stop" in the same thread. "SparkContext.stop" will block until all RPC threads exit, if it's called inside a RPC thread, it will be dead-lock.

This PR add a thread local flag inside RPC threads. `SparkContext.stop` uses it to decide if launching a new thread to stop the SparkContext.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #15775 from zsxwing/SPARK-18280.
2016-11-08 13:14:56 -08:00
jiangxingbo 9c419698fe [SPARK-18191][CORE] Port RDD API to use commit protocol
## What changes were proposed in this pull request?

This PR port RDD API to use commit protocol, the changes made here:
1. Add new internal helper class that saves an RDD using a Hadoop OutputFormat named `SparkNewHadoopWriter`, it's similar with `SparkHadoopWriter` but uses commit protocol. This class supports the newer `mapreduce` API, instead of the old `mapred` API which is supported by `SparkHadoopWriter`;
2. Rewrite `PairRDDFunctions.saveAsNewAPIHadoopDataset` function, so it uses commit protocol now.

## How was this patch tested?
Exsiting test cases.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #15769 from jiangxb1987/rdd-commit.
2016-11-08 09:41:01 -08:00
Josh Rosen 3a710b94b0 [SPARK-18236] Reduce duplicate objects in Spark UI and HistoryServer
## What changes were proposed in this pull request?

When profiling heap dumps from the HistoryServer and live Spark web UIs, I found a large amount of memory being wasted on duplicated objects and strings. This patch's changes remove most of this duplication, resulting in over 40% memory savings for some benchmarks.

- **Task metrics** (6441f0624dfcda9c7193a64bfb416a145b5aabdf): previously, every `TaskUIData` object would have its own instances of `InputMetricsUIData`, `OutputMetricsUIData`, `ShuffleReadMetrics`, and `ShuffleWriteMetrics`, but for many tasks these metrics are irrelevant because they're all zero. This patch changes how we construct these metrics in order to re-use a single immutable "empty" value for the cases where these metrics are empty.
- **TaskInfo.accumulables** (ade86db901127bf13c0e0bdc3f09c933a093bb76): Previously, every `TaskInfo` object had its own empty `ListBuffer` for holding updates from named accumulators. Tasks which didn't use named accumulators still paid for the cost of allocating and storing this empty buffer. To avoid this overhead, I changed the `val` with a mutable buffer into a `var` which holds an immutable Scala list, allowing tasks which do not have named accumulator updates to share the same singleton `Nil` object.
- **String.intern() in JSONProtocol** (7e05630e9a78c455db8c8c499f0590c864624e05): in the HistoryServer, executor hostnames and ids are deserialized from JSON, leading to massive duplication of these string objects. By calling `String.intern()` on the deserialized values we can remove all of this duplication. Since Spark now requires Java 7+ we don't have to worry about string interning exhausting the permgen (see http://java-performance.info/string-intern-in-java-6-7-8/).

## How was this patch tested?

I ran

```
sc.parallelize(1 to 100000, 100000).count()
```

in `spark-shell` with event logging enabled, then loaded that event log in the HistoryServer, performed a full GC, and took a heap dump. According to YourKit, the changes in this patch reduced memory consumption by roughly 28 megabytes (or 770k Java objects):

![image](https://cloud.githubusercontent.com/assets/50748/19953276/4f3a28aa-a129-11e6-93df-d7fa91396f66.png)

Here's a table illustrating the drop in objects due to deduplication (the drop is <100k for some objects because some events were dropped from the listener bus; this is a separate, existing bug that I'll address separately after CPU-profiling):

![image](https://cloud.githubusercontent.com/assets/50748/19953290/6a271290-a129-11e6-93ad-b825f1448886.png)

Author: Josh Rosen <joshrosen@databricks.com>

Closes #15743 from JoshRosen/spark-ui-memory-usage.
2016-11-07 16:14:19 -08:00
Hyukjin Kwon 8f0ea011a7 [SPARK-14914][CORE] Fix Resource not closed after using, mostly for unit tests
## What changes were proposed in this pull request?

Close `FileStreams`, `ZipFiles` etc to release the resources after using. Not closing the resources will cause IO Exception to be raised while deleting temp files.
## How was this patch tested?

Existing tests

Author: U-FAREAST\tl <tl@microsoft.com>
Author: hyukjinkwon <gurwls223@gmail.com>
Author: Tao LI <tl@microsoft.com>

Closes #15618 from HyukjinKwon/SPARK-14914-1.
2016-11-07 12:47:39 -08:00
Dongjoon Hyun d24e736471 [SPARK-18200][GRAPHX] Support zero as an initial capacity in OpenHashSet
## What changes were proposed in this pull request?

[SPARK-18200](https://issues.apache.org/jira/browse/SPARK-18200) reports Apache Spark 2.x raises `java.lang.IllegalArgumentException: requirement failed: Invalid initial capacity` while running `triangleCount`. The root cause is that `VertexSet`, a type alias of `OpenHashSet`, does not allow zero as a initial size. This PR loosens the restriction to allow zero.

## How was this patch tested?

Pass the Jenkins test with a new test case in `OpenHashSetSuite`.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #15741 from dongjoon-hyun/SPARK-18200.
2016-11-02 23:50:50 -07:00
Sean Owen 9c8deef64e
[SPARK-18076][CORE][SQL] Fix default Locale used in DateFormat, NumberFormat to Locale.US
## What changes were proposed in this pull request?

Fix `Locale.US` for all usages of `DateFormat`, `NumberFormat`
## How was this patch tested?

Existing tests.

Author: Sean Owen <sowen@cloudera.com>

Closes #15610 from srowen/SPARK-18076.
2016-11-02 09:39:15 +00:00
Jacek Laskowski 70a5db7bbd
[SPARK-18204][WEBUI] Remove SparkUI.appUIAddress
## What changes were proposed in this pull request?

Removing `appUIAddress` attribute since it is no longer in use.
## How was this patch tested?

Local build

Author: Jacek Laskowski <jacek@japila.pl>

Closes #15603 from jaceklaskowski/sparkui-fixes.
2016-11-02 09:21:26 +00:00
wm624@hotmail.com 701a9d361b
[SPARK-CORE][TEST][MINOR] Fix the wrong comment in test
## What changes were proposed in this pull request?

While learning core scheduler code, I found two lines of wrong comments. This PR simply corrects the comments.

## How was this patch tested?

Author: wm624@hotmail.com <wm624@hotmail.com>

Closes #15631 from wangmiao1981/Rbug.
2016-10-27 10:00:37 +02:00
Miao Wang a76846cfb1 [SPARK-18126][SPARK-CORE] getIteratorZipWithIndex accepts negative value as index
## What changes were proposed in this pull request?

(Please fill in changes proposed in this fix)

`Utils.getIteratorZipWithIndex` was added to deal with number of records > 2147483647 in one partition.

method `getIteratorZipWithIndex` accepts `startIndex` < 0, which leads to negative index.

This PR just adds a defensive check on `startIndex` to make sure it is >= 0.

## How was this patch tested?

Add a new unit test.

Author: Miao Wang <miaowang@Miaos-MacBook-Pro.local>

Closes #15639 from wangmiao1981/zip.
2016-10-27 01:17:32 +02:00
Shuai Lin 402205ddf7
[SPARK-17802] Improved caller context logging.
## What changes were proposed in this pull request?

[SPARK-16757](https://issues.apache.org/jira/browse/SPARK-16757) sets the hadoop `CallerContext` when calling hadoop/hdfs apis to make spark applications more diagnosable in hadoop/hdfs logs. However, the `org.apache.hadoop.ipc.CallerContext` class is only added since [hadoop 2.8](https://issues.apache.org/jira/browse/HDFS-9184), which is not officially releaed yet. So each time `utils.CallerContext.setCurrentContext()` is called (e.g [when a task is created](https://github.com/apache/spark/blob/b678e46/core/src/main/scala/org/apache/spark/scheduler/Task.scala#L95-L96)), a "java.lang.ClassNotFoundException: org.apache.hadoop.ipc.CallerContext"
error is logged, which pollutes the spark logs when there are lots of tasks.

This patch improves this behaviour by only logging the `ClassNotFoundException` once.

## How was this patch tested?

Existing tests.

Author: Shuai Lin <linshuai2012@gmail.com>

Closes #15377 from lins05/spark-17802-improve-callercontext-logging.
2016-10-26 14:31:47 +02:00
Alex Bozarth 5d0f81da49
[SPARK-4411][WEB UI] Add "kill" link for jobs in the UI
## What changes were proposed in this pull request?

Currently users can kill stages via the web ui but not jobs directly (jobs are killed if one of their stages is). I've added the ability to kill jobs via the web ui. This code change is based on #4823 by lianhuiwang and updated to work with the latest code matching how stages are currently killed. In general I've copied the kill stage code warning and note comments and all. I also updated applicable tests and documentation.

## How was this patch tested?

Manually tested and dev/run-tests

![screen shot 2016-10-11 at 4 49 43 pm](https://cloud.githubusercontent.com/assets/13952758/19292857/12f1b7c0-8fd4-11e6-8982-210249f7b697.png)

Author: Alex Bozarth <ajbozart@us.ibm.com>
Author: Lianhui Wang <lianhuiwang09@gmail.com>

Closes #15441 from ajbozarth/spark4411.
2016-10-26 14:26:54 +02:00
hayashidac c329a568b5 [SPARK-16988][SPARK SHELL] spark history server log needs to be fixed to show https url when ssl is enabled
spark history server log needs to be fixed to show https url when ssl is enabled

Author: chie8842 <chie@chie-no-Mac-mini.local>

Closes #15611 from hayashidac/SPARK-16988.
2016-10-26 07:13:48 +09:00
Kay Ousterhout 483c37c581 [SPARK-17894][HOTFIX] Fix broken build from
The named parameter in an overridden class isn't supported in Scala 2.10 so was breaking the build.

cc zsxwing

Author: Kay Ousterhout <kayousterhout@gmail.com>

Closes #15617 from kayousterhout/hotfix.
2016-10-24 20:16:00 -07:00
Eren Avsarogullari 81d6933e75 [SPARK-17894][CORE] Ensure uniqueness of TaskSetManager name.
`TaskSetManager` should have unique name to avoid adding duplicate ones to parent `Pool` via `SchedulableBuilder`. This problem has been surfaced with following discussion: [[PR: Avoid adding duplicate schedulables]](https://github.com/apache/spark/pull/15326)

**Proposal** :
There is 1x1 relationship between `stageAttemptId` and `TaskSetManager` so `taskSet.Id` covering both `stageId` and `stageAttemptId` looks to be used for uniqueness of `TaskSetManager` name instead of just `stageId`.

**Current TaskSetManager Name** :
`var name = "TaskSet_" + taskSet.stageId.toString`
**Sample**: TaskSet_0

**Proposed TaskSetManager Name** :
`val name = "TaskSet_" + taskSet.Id ` `// taskSet.Id = (stageId + "." + stageAttemptId)`
**Sample** : TaskSet_0.0

Added new Unit Test.

Author: erenavsarogullari <erenavsarogullari@gmail.com>

Closes #15463 from erenavsarogullari/SPARK-17894.
2016-10-24 15:33:54 -07:00
Sandeep Singh bc167a2a53 [SPARK-928][CORE] Add support for Unsafe-based serializer in Kryo
## What changes were proposed in this pull request?
Now since we have migrated to Kryo-3.0.0 in https://issues.apache.org/jira/browse/SPARK-11416, we can gives users option to use unsafe SerDer. It can turned by setting `spark.kryo.useUnsafe` to `true`

## How was this patch tested?
Ran existing tests

```
     Benchmark Kryo Unsafe vs safe Serialization: Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
      ------------------------------------------------------------------------------------------------
      basicTypes: Int unsafe:true                    160 /  178         98.5          10.1       1.0X
      basicTypes: Long unsafe:true                   210 /  218         74.9          13.4       0.8X
      basicTypes: Float unsafe:true                  203 /  213         77.5          12.9       0.8X
      basicTypes: Double unsafe:true                 226 /  235         69.5          14.4       0.7X
      Array: Int unsafe:true                        1087 / 1101         14.5          69.1       0.1X
      Array: Long unsafe:true                       2758 / 2844          5.7         175.4       0.1X
      Array: Float unsafe:true                      1511 / 1552         10.4          96.1       0.1X
      Array: Double unsafe:true                     2942 / 2972          5.3         187.0       0.1X
      Map of string->Double unsafe:true             2645 / 2739          5.9         168.2       0.1X
      basicTypes: Int unsafe:false                   211 /  218         74.7          13.4       0.8X
      basicTypes: Long unsafe:false                  247 /  253         63.6          15.7       0.6X
      basicTypes: Float unsafe:false                 211 /  216         74.5          13.4       0.8X
      basicTypes: Double unsafe:false                227 /  233         69.2          14.4       0.7X
      Array: Int unsafe:false                       3012 / 3032          5.2         191.5       0.1X
      Array: Long unsafe:false                      4463 / 4515          3.5         283.8       0.0X
      Array: Float unsafe:false                     2788 / 2868          5.6         177.2       0.1X
      Array: Double unsafe:false                    3558 / 3752          4.4         226.2       0.0X
      Map of string->Double unsafe:false            2806 / 2933          5.6         178.4       0.1X
```

Author: Sandeep Singh <sandeep@techaddict.me>
Author: Sandeep Singh <sandeep@origamilogic.com>

Closes #12913 from techaddict/SPARK-928.
2016-10-22 12:03:37 -07:00
Zheng RuiFeng a8ea4da8d0
[SPARK-17331][FOLLOWUP][ML][CORE] Avoid allocating 0-length arrays
## What changes were proposed in this pull request?

`Array[T]()` -> `Array.empty[T]` to avoid allocating 0-length arrays.
Use regex `find . -name '*.scala' | xargs -i bash -c 'egrep "Array\[[A-Za-z]+\]\(\)" -n {} && echo {}'` to find modification candidates.

cc srowen

## How was this patch tested?
existing tests

Author: Zheng RuiFeng <ruifengz@foxmail.com>

Closes #15564 from zhengruifeng/avoid_0_length_array.
2016-10-21 09:49:37 +01:00
WeichenXu 39755169fb [SPARK-18003][SPARK CORE] Fix bug of RDD zipWithIndex & zipWithUniqueId index value overflowing
## What changes were proposed in this pull request?

- Fix bug of RDD `zipWithIndex` generating wrong result when one partition contains more than 2147483647 records.

- Fix bug of RDD `zipWithUniqueId` generating wrong result when one partition contains more than 2147483647 records.

## How was this patch tested?

test added.

Author: WeichenXu <WeichenXu123@outlook.com>

Closes #15550 from WeichenXu123/fix_rdd_zipWithIndex_overflow.
2016-10-19 23:41:38 -07:00
Yu Peng 231f39e3f6 [SPARK-17711] Compress rolled executor log
## What changes were proposed in this pull request?

This PR adds support for executor log compression.

## How was this patch tested?

Unit tests

cc: yhuai tdas mengxr

Author: Yu Peng <loneknightpy@gmail.com>

Closes #15285 from loneknightpy/compress-executor-log.
2016-10-18 13:23:31 -07:00
Sital Kedia c7ac027d5f [SPARK-17839][CORE] Use Nio's directbuffer instead of BufferedInputStream in order to avoid additional copy from os buffer cache to user buffer
## What changes were proposed in this pull request?

Currently we use BufferedInputStream to read the shuffle file which copies the file content from os buffer cache to the user buffer. This adds additional latency in reading the spill files. We made a change to use java nio's direct buffer to read the spill files and for certain pipelines spilling significant amount of data, we see up to 7% speedup for the entire pipeline.

## How was this patch tested?
Tested by running the job in the cluster and observed up to 7% speedup.

Author: Sital Kedia <skedia@fb.com>

Closes #15408 from sitalkedia/skedia/nio_spill_read.
2016-10-17 11:03:04 -07:00
Reynold Xin 72a6e7a57a Revert "[SPARK-17637][SCHEDULER] Packed scheduling for Spark tasks across executors"
This reverts commit ed14633414.

The patch merged had obvious quality and documentation issue. The idea is useful, and we should work towards improving its quality and merging it in again.
2016-10-15 22:31:37 -07:00
Zhan Zhang ed14633414 [SPARK-17637][SCHEDULER] Packed scheduling for Spark tasks across executors
## What changes were proposed in this pull request?

Restructure the code and implement two new task assigner.
PackedAssigner: try to allocate tasks to the executors with least available cores, so that spark can release reserved executors when dynamic allocation is enabled.

BalancedAssigner: try to allocate tasks to the executors with more available cores in order to balance the workload across all executors.

By default, the original round robin assigner is used.

We test a pipeline, and new PackedAssigner  save around 45% regarding the reserved cpu and memory with dynamic allocation enabled.

## How was this patch tested?

(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)
Both unit test in TaskSchedulerImplSuite and manual tests in production pipeline.

Author: Zhan Zhang <zhanzhang@fb.com>

Closes #15218 from zhzhan/packed-scheduler.
2016-10-15 18:45:04 -07:00
Imran Rashid 9ce7d3e542 [SPARK-17675][CORE] Expand Blacklist for TaskSets
## What changes were proposed in this pull request?

This is a step along the way to SPARK-8425.

To enable incremental review, the first step proposed here is to expand the blacklisting within tasksets. In particular, this will enable blacklisting for
* (task, executor) pairs (this already exists via an undocumented config)
* (task, node)
* (taskset, executor)
* (taskset, node)

Adding (task, node) is critical to making spark fault-tolerant of one-bad disk in a cluster, without requiring careful tuning of "spark.task.maxFailures". The other additions are also important to avoid many misleading task failures and long scheduling delays when there is one bad node on a large cluster.

Note that some of the code changes here aren't really required for just this -- they put pieces in place for SPARK-8425 even though they are not used yet (eg. the `BlacklistTracker` helper is a little out of place, `TaskSetBlacklist` holds onto a little more info than it needs to for just this change, and `ExecutorFailuresInTaskSet` is more complex than it needs to be).

## How was this patch tested?

Added unit tests, run tests via jenkins.

Author: Imran Rashid <irashid@cloudera.com>
Author: mwws <wei.mao@intel.com>

Closes #15249 from squito/taskset_blacklist_only.
2016-10-12 16:43:03 -05:00
Shixiong Zhu 47776e7c0c [SPARK-17850][CORE] Add a flag to ignore corrupt files
## What changes were proposed in this pull request?

Add a flag to ignore corrupt files. For Spark core, the configuration is `spark.files.ignoreCorruptFiles`. For Spark SQL, it's `spark.sql.files.ignoreCorruptFiles`.

## How was this patch tested?

The added unit tests

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #15422 from zsxwing/SPARK-17850.
2016-10-12 13:51:53 -07:00
Weiqing Yang 8a6bbe095b
[MINOR][SQL] Use resource path for test_script.sh
## What changes were proposed in this pull request?
This PR modified the test case `test("script")` to use resource path for `test_script.sh`. Make the test case portable (even in IntelliJ).

## How was this patch tested?
Passed the test case.
Before:
Run `test("script")` in IntelliJ:
```
Caused by: org.apache.spark.SparkException: Subprocess exited with status 127. Error: bash: src/test/resources/test_script.sh: No such file or directory
```
After:
Test passed.

Author: Weiqing Yang <yangweiqing001@gmail.com>

Closes #15246 from weiqingy/hivetest.
2016-10-08 12:12:35 +01:00
Sean Owen 4201ddcc07
[SPARK-17768][CORE] Small (Sum,Count,Mean)Evaluator problems and suboptimalities
## What changes were proposed in this pull request?

Fix:

- GroupedMeanEvaluator and GroupedSumEvaluator are unused, as is the StudentTCacher support class
- CountEvaluator can return a lower bound < 0, when counts can't be negative
- MeanEvaluator will actually fail on exactly 1 datum (yields t-test with 0 DOF)
- CountEvaluator uses a normal distribution, which may be an inappropriate approximation (leading to above)
- Test for SumEvaluator asserts incorrect expected sums – e.g. after observing 10% of data has sum of 2, expectation should be 20, not 38
- CountEvaluator, MeanEvaluator have no unit tests to catch these
- Duplication of distribution code across CountEvaluator, GroupedCountEvaluator
- The stats in each could use a bit of documentation as I had to guess at them
- (Code could use a few cleanups and optimizations too)

## How was this patch tested?

Existing and new tests

Author: Sean Owen <sowen@cloudera.com>

Closes #15341 from srowen/SPARK-17768.
2016-10-08 11:31:12 +01:00
Shixiong Zhu 221b418b1c [SPARK-17778][TESTS] Mock SparkContext to reduce memory usage of BlockManagerSuite
## What changes were proposed in this pull request?

Mock SparkContext to reduce memory usage of BlockManagerSuite

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #15350 from zsxwing/SPARK-17778.
2016-10-05 14:54:55 -07:00
Sean Owen 8e8de0073d
[SPARK-17671][WEBUI] Spark 2.0 history server summary page is slow even set spark.history.ui.maxApplications
## What changes were proposed in this pull request?

Return Iterator of applications internally in history server, for consistency and performance. See https://github.com/apache/spark/pull/15248 for some back-story.

The code called by and calling HistoryServer.getApplicationList wants an Iterator, but this method materializes an Iterable, which potentially causes a performance problem. It's simpler too to make this internal method also pass through an Iterator.

## How was this patch tested?

Existing tests.

Author: Sean Owen <sowen@cloudera.com>

Closes #15321 from srowen/SPARK-17671.
2016-10-04 10:29:22 +01:00
Tao LI 76dc2d9073 [SPARK-14914][CORE][SQL] Skip/fix some test cases on Windows due to limitation of Windows
## What changes were proposed in this pull request?

This PR proposes to fix/skip some tests failed on Windows. This PR takes over https://github.com/apache/spark/pull/12696.

**Before**

- **SparkSubmitSuite**

  ```
[info] - launch simple application with spark-submit *** FAILED *** (202 milliseconds)
[info]   java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specifie

[info] - includes jars passed in through --jars *** FAILED *** (1 second, 625 milliseconds)
[info]   java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified
```

- **DiskStoreSuite**

  ```
[info] - reads of memory-mapped and non memory-mapped files are equivalent *** FAILED *** (1 second, 78 milliseconds)
[info]   diskStoreMapped.remove(blockId) was false (DiskStoreSuite.scala:41)
```

**After**

- **SparkSubmitSuite**

  ```
[info] - launch simple application with spark-submit (578 milliseconds)
[info] - includes jars passed in through --jars (1 second, 875 milliseconds)
```

- **DiskStoreSuite**

  ```
[info] DiskStoreSuite:
[info] - reads of memory-mapped and non memory-mapped files are equivalent !!! CANCELED !!! (766 milliseconds
```

For `CreateTableAsSelectSuite` and `FsHistoryProviderSuite`, I could not reproduce as the Java version seems higher than the one that has the bugs about `setReadable(..)` and `setWritable(...)` but as they are bugs reported clearly, it'd be sensible to skip those. We should revert the changes for both back as soon as we drop the support of Java 7.

## How was this patch tested?

Manually tested via AppVeyor.

Closes #12696

Author: Tao LI <tl@microsoft.com>
Author: U-FAREAST\tl <tl@microsoft.com>
Author: hyukjinkwon <gurwls223@gmail.com>

Closes #15320 from HyukjinKwon/SPARK-14914.
2016-10-02 16:01:02 -07:00
Eric Liang 4bcd9b728b [SPARK-17740] Spark tests should mock / interpose HDFS to ensure that streams are closed
## What changes were proposed in this pull request?

As a followup to SPARK-17666, ensure filesystem connections are not leaked at least in unit tests. This is done here by intercepting filesystem calls as suggested by JoshRosen . At the end of each test, we assert no filesystem streams are left open.

This applies to all tests using SharedSQLContext or SharedSparkContext.

## How was this patch tested?

I verified that tests in sql and core are indeed using the filesystem backend, and fixed the detected leaks. I also checked that reverting https://github.com/apache/spark/pull/15245 causes many actual test failures due to connection leaks.

Author: Eric Liang <ekl@databricks.com>
Author: Eric Liang <ekhliang@gmail.com>

Closes #15306 from ericl/sc-4672.
2016-09-30 23:51:36 -07:00
Shubham Chopra a26afd5219 [SPARK-15353][CORE] Making peer selection for block replication pluggable
## What changes were proposed in this pull request?

This PR makes block replication strategies pluggable. It provides two trait that can be implemented, one that maps a host to its topology and is used in the master, and the second that helps prioritize a list of peers for block replication and would run in the executors.

This patch contains default implementations of these traits that make sure current Spark behavior is unchanged.

## How was this patch tested?

This patch should not change Spark behavior in any way, and was tested with unit tests for storage.

Author: Shubham Chopra <schopra31@bloomberg.net>

Closes #13152 from shubhamchopra/RackAwareBlockReplication.
2016-09-30 18:24:39 -07:00
Imran Rashid 3993ebca23 [SPARK-17676][CORE] FsHistoryProvider should ignore hidden files
## What changes were proposed in this pull request?

FsHistoryProvider was writing a hidden file (to check the fs's clock).
Even though it deleted the file immediately, sometimes another thread
would try to scan the files on the fs in-between, and then there would
be an error msg logged which was very misleading for the end-user.
(The logged error was harmless, though.)

## How was this patch tested?

I added one unit test, but to be clear, that test was passing before.  The actual change in behavior in that test is just logging (after the change, there is no more logged error), which I just manually verified.

Author: Imran Rashid <irashid@cloudera.com>

Closes #15250 from squito/SPARK-17676.
2016-09-29 15:40:35 -07:00
Imran Rashid 7f779e7439 [SPARK-17648][CORE] TaskScheduler really needs offers to be an IndexedSeq
## What changes were proposed in this pull request?

The Seq[WorkerOffer] is accessed by index, so it really should be an
IndexedSeq, otherwise an O(n) operation becomes O(n^2).  In practice
this hasn't been an issue b/c where these offers are generated, the call
to `.toSeq` just happens to create an IndexedSeq anyway.I got bitten by
this in performance tests I was doing, and its better for the types to be
more precise so eg. a change in Scala doesn't destroy performance.

## How was this patch tested?

Unit tests via jenkins.

Author: Imran Rashid <irashid@cloudera.com>

Closes #15221 from squito/SPARK-17648.
2016-09-29 15:36:40 -04:00
w00228970 46d1203bf2 [SPARK-17644][CORE] Do not add failedStages when abortStage for fetch failure
## What changes were proposed in this pull request?
| Time        |Thread 1 ,  Job1          | Thread 2 ,  Job2  |
|:-------------:|:-------------:|:-----:|
| 1 | abort stage due to FetchFailed |  |
| 2 | failedStages += failedStage |    |
| 3 |      |  task failed due to  FetchFailed |
| 4 |      |  can not post ResubmitFailedStages because failedStages is not empty |

Then job2 of thread2 never resubmit the failed stage and hang.

We should not add the failedStages when abortStage for fetch failure

## How was this patch tested?

added unit test

Author: w00228970 <wangfei1@huawei.com>
Author: wangfei <wangfei_hello@126.com>

Closes #15213 from scwf/dag-resubmit.
2016-09-28 12:02:59 -07:00
Weiqing Yang 6a68c5d7b4 [SPARK-16757] Set up Spark caller context to HDFS and YARN
## What changes were proposed in this pull request?

1. Pass `jobId` to Task.
2. Invoke Hadoop APIs.
    * A new function `setCallerContext` is added in `Utils`. `setCallerContext` function invokes APIs of   `org.apache.hadoop.ipc.CallerContext` to set up spark caller contexts, which will be written into `hdfs-audit.log` and Yarn RM audit log.
    * For HDFS: Spark sets up its caller context by invoking`org.apache.hadoop.ipc.CallerContext` in `Task` and Yarn `Client` and `ApplicationMaster`.
    * For Yarn: Spark sets up its caller context by invoking `org.apache.hadoop.ipc.CallerContext` in Yarn `Client`.

## How was this patch tested?
Manual Tests against some Spark applications in Yarn client mode and Yarn cluster mode. Need to check if spark caller contexts are written into HDFS hdfs-audit.log and Yarn RM audit log successfully.

For example, run SparkKmeans in Yarn client mode:
```
./bin/spark-submit --verbose --executor-cores 3 --num-executors 1 --master yarn --deploy-mode client --class org.apache.spark.examples.SparkKMeans examples/target/original-spark-examples_2.11-2.1.0-SNAPSHOT.jar hdfs://localhost:9000/lr_big.txt 2 5
```

**Before**:
There will be no Spark caller context in records of `hdfs-audit.log` and Yarn RM audit log.

**After**:
Spark caller contexts will be written in records of `hdfs-audit.log` and Yarn RM audit log.

These are records in `hdfs-audit.log`:
```
2016-09-20 11:54:24,116 INFO FSNamesystem.audit: allowed=true	ugi=wyang (auth:SIMPLE)	ip=/127.0.0.1	cmd=open	src=/lr_big.txt	dst=null	perm=null	proto=rpc	callerContext=SPARK_CLIENT_AppId_application_1474394339641_0005
2016-09-20 11:54:28,164 INFO FSNamesystem.audit: allowed=true	ugi=wyang (auth:SIMPLE)	ip=/127.0.0.1	cmd=open	src=/lr_big.txt	dst=null	perm=null	proto=rpc	callerContext=SPARK_TASK_AppId_application_1474394339641_0005_JobId_0_StageId_0_AttemptId_0_TaskId_2_AttemptNum_0
2016-09-20 11:54:28,164 INFO FSNamesystem.audit: allowed=true	ugi=wyang (auth:SIMPLE)	ip=/127.0.0.1	cmd=open	src=/lr_big.txt	dst=null	perm=null	proto=rpc	callerContext=SPARK_TASK_AppId_application_1474394339641_0005_JobId_0_StageId_0_AttemptId_0_TaskId_1_AttemptNum_0
2016-09-20 11:54:28,164 INFO FSNamesystem.audit: allowed=true	ugi=wyang (auth:SIMPLE)	ip=/127.0.0.1	cmd=open	src=/lr_big.txt	dst=null	perm=null	proto=rpc	callerContext=SPARK_TASK_AppId_application_1474394339641_0005_JobId_0_StageId_0_AttemptId_0_TaskId_0_AttemptNum_0
```
```
2016-09-20 11:59:33,868 INFO FSNamesystem.audit: allowed=true	ugi=wyang (auth:SIMPLE)	ip=/127.0.0.1	cmd=mkdirs	src=/private/tmp/hadoop-wyang/nm-local-dir/usercache/wyang/appcache/application_1474394339641_0006/container_1474394339641_0006_01_000001/spark-warehouse	dst=null	perm=wyang:supergroup:rwxr-xr-x	proto=rpc	callerContext=SPARK_APPLICATION_MASTER_AppId_application_1474394339641_0006_AttemptId_1
2016-09-20 11:59:37,214 INFO FSNamesystem.audit: allowed=true	ugi=wyang (auth:SIMPLE)	ip=/127.0.0.1	cmd=open	src=/lr_big.txt	dst=null	perm=null	proto=rpc	callerContext=SPARK_TASK_AppId_application_1474394339641_0006_AttemptId_1_JobId_0_StageId_0_AttemptId_0_TaskId_1_AttemptNum_0
2016-09-20 11:59:37,215 INFO FSNamesystem.audit: allowed=true	ugi=wyang (auth:SIMPLE)	ip=/127.0.0.1	cmd=open	src=/lr_big.txt	dst=null	perm=null	proto=rpc	callerContext=SPARK_TASK_AppId_application_1474394339641_0006_AttemptId_1_JobId_0_StageId_0_AttemptId_0_TaskId_2_AttemptNum_0
2016-09-20 11:59:37,215 INFO FSNamesystem.audit: allowed=true	ugi=wyang (auth:SIMPLE)	ip=/127.0.0.1	cmd=open	src=/lr_big.txt	dst=null	perm=null	proto=rpc	callerContext=SPARK_TASK_AppId_application_1474394339641_0006_AttemptId_1_JobId_0_StageId_0_AttemptId_0_TaskId_0_AttemptNum_0
2016-09-20 11:59:42,391 INFO FSNamesystem.audit: allowed=true	ugi=wyang (auth:SIMPLE)	ip=/127.0.0.1	cmd=open	src=/lr_big.txt	dst=null	perm=null	proto=rpc	callerContext=SPARK_TASK_AppId_application_1474394339641_0006_AttemptId_1_JobId_0_StageId_0_AttemptId_0_TaskId_3_AttemptNum_0
```
This is a record in Yarn RM log:
```
2016-09-20 11:59:24,050 INFO org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger: USER=wyang	IP=127.0.0.1	OPERATION=Submit Application Request	TARGET=ClientRMService	RESULT=SUCCESS	APPID=application_1474394339641_0006	CALLERCONTEXT=SPARK_CLIENT_AppId_application_1474394339641_0006
```

Author: Weiqing Yang <yangweiqing001@gmail.com>

Closes #14659 from Sherry302/callercontextSubmit.
2016-09-27 08:10:38 -05:00
Ding Fei 6ee28423ad Fix two comments since Actor is not used anymore.
## What changes were proposed in this pull request?

Fix two comments since Actor is not used anymore.

Author: Ding Fei <danis@danix>

Closes #15251 from danix800/comment-fixing.
2016-09-26 23:09:51 -07:00
Burak Yavuz 59d87d2407 [SPARK-17650] malformed url's throw exceptions before bricking Executors
## What changes were proposed in this pull request?

When a malformed URL was sent to Executors through `sc.addJar` and `sc.addFile`, the executors become unusable, because they constantly throw `MalformedURLException`s and can never acknowledge that the file or jar is just bad input.

This PR tries to fix that problem by making sure MalformedURLs can never be submitted through `sc.addJar` and `sc.addFile`. Another solution would be to blacklist bad files and jars on Executors. Maybe fail the first time, and then ignore the second time (but print a warning message).

## How was this patch tested?

Unit tests in SparkContextSuite

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #15224 from brkyvz/SPARK-17650.
2016-09-25 22:57:31 -07:00
jisookim 90a30f4634 [SPARK-12221] add cpu time to metrics
Currently task metrics don't support executor CPU time, so there's no way to calculate how much CPU time a stage/task took from History Server metrics. This PR enables reporting CPU time.

Author: jisookim <jisookim0513@gmail.com>

Closes #10212 from jisookim0513/add-cpu-time-metric.
2016-09-23 13:43:47 -07:00
Dhruve Ashar 17b72d31e0 [SPARK-17365][CORE] Remove/Kill multiple executors together to reduce RPC call time.
## What changes were proposed in this pull request?
We are killing multiple executors together instead of iterating over expensive RPC calls to kill single executor.

## How was this patch tested?
Executed sample spark job to observe executors being killed/removed with dynamic allocation enabled.

Author: Dhruve Ashar <dashar@yahoo-inc.com>
Author: Dhruve Ashar <dhruveashar@gmail.com>

Closes #15152 from dhruve/impr/SPARK-17365.
2016-09-22 10:10:37 -07:00
jerryshao 8c3ee2bc42 [SPARK-17512][CORE] Avoid formatting to python path for yarn and mesos cluster mode
## What changes were proposed in this pull request?

Yarn and mesos cluster mode support remote python path (HDFS/S3 scheme) by their own mechanism, it is not necessary to check and format the python when running on these modes. This is a potential regression compared to 1.6, so here propose to fix it.

## How was this patch tested?

Unit test to verify SparkSubmit arguments, also with local cluster verification. Because of lack of `MiniDFSCluster` support in Spark unit test, there's no integration test added.

Author: jerryshao <sshao@hortonworks.com>

Closes #15137 from jerryshao/SPARK-17512.
2016-09-21 17:57:21 -04:00
Imran Rashid 9fcf1c51d5 [SPARK-17623][CORE] Clarify type of TaskEndReason with a failed task.
## What changes were proposed in this pull request?

In TaskResultGetter, enqueueFailedTask currently deserializes the result
as a TaskEndReason. But the type is actually more specific, its a
TaskFailedReason. This just leads to more blind casting later on – it
would be more clear if the msg was cast to the right type immediately,
so method parameter types could be tightened.

## How was this patch tested?

Existing unit tests via jenkins.  Note that the code was already performing a blind-cast to a TaskFailedReason before in any case, just in a different spot, so there shouldn't be any behavior change.

Author: Imran Rashid <irashid@cloudera.com>

Closes #15181 from squito/SPARK-17623.
2016-09-21 17:49:36 -04:00
Marcelo Vanzin 2cd1bfa4f0 [SPARK-4563][CORE] Allow driver to advertise a different network address.
The goal of this feature is to allow the Spark driver to run in an
isolated environment, such as a docker container, and be able to use
the host's port forwarding mechanism to be able to accept connections
from the outside world.

The change is restricted to the driver: there is no support for achieving
the same thing on executors (or the YARN AM for that matter). Those still
need full access to the outside world so that, for example, connections
can be made to an executor's block manager.

The core of the change is simple: add a new configuration that tells what's
the address the driver should bind to, which can be different than the address
it advertises to executors (spark.driver.host). Everything else is plumbing
the new configuration where it's needed.

To use the feature, the host starting the container needs to set up the
driver's port range to fall into a range that is being forwarded; this
required the block manager port to need a special configuration just for
the driver, which falls back to the existing spark.blockManager.port when
not set. This way, users can modify the driver settings without affecting
the executors; it would theoretically be nice to also have different
retry counts for driver and executors, but given that docker (at least)
allows forwarding port ranges, we can probably live without that for now.

Because of the nature of the feature it's kinda hard to add unit tests;
I just added a simple one to make sure the configuration works.

This was tested with a docker image running spark-shell with the following
command:

 docker blah blah blah \
   -p 38000-38100:38000-38100 \
   [image] \
   spark-shell \
     --num-executors 3 \
     --conf spark.shuffle.service.enabled=false \
     --conf spark.dynamicAllocation.enabled=false \
     --conf spark.driver.host=[host's address] \
     --conf spark.driver.port=38000 \
     --conf spark.driver.blockManager.port=38020 \
     --conf spark.ui.port=38040

Running on YARN; verified the driver works, executors start up and listen
on ephemeral ports (instead of using the driver's config), and that caching
and shuffling (without the shuffle service) works. Clicked through the UI
to make sure all pages (including executor thread dumps) worked. Also tested
apps without docker, and ran unit tests.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #15120 from vanzin/SPARK-4563.
2016-09-21 14:42:41 -07:00
Josh Rosen 8faa5217b4 [SPARK-17491] Close serialization stream to fix wrong answer bug in putIteratorAsBytes()
## What changes were proposed in this pull request?

`MemoryStore.putIteratorAsBytes()` may silently lose values when used with `KryoSerializer` because it does not properly close the serialization stream before attempting to deserialize the already-serialized values, which may cause values buffered in Kryo's internal buffers to not be read.

This is the root cause behind a user-reported "wrong answer" bug in PySpark caching reported by bennoleslie on the Spark user mailing list in a thread titled "pyspark persist MEMORY_ONLY vs MEMORY_AND_DISK". Due to Spark 2.0's automatic use of KryoSerializer for "safe" types (such as byte arrays, primitives, etc.) this misuse of serializers manifested itself as silent data corruption rather than a StreamCorrupted error (which you might get from JavaSerializer).

The minimal fix, implemented here, is to close the serialization stream before attempting to deserialize written values. In addition, this patch adds several additional assertions / precondition checks to prevent misuse of `PartiallySerializedBlock` and `ChunkedByteBufferOutputStream`.

## How was this patch tested?

The original bug was masked by an invalid assert in the memory store test cases: the old assert compared two results record-by-record with `zip` but didn't first check that the lengths of the two collections were equal, causing missing records to go unnoticed. The updated test case reproduced this bug.

In addition, I added a new `PartiallySerializedBlockSuite` to unit test that component.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #15043 from JoshRosen/partially-serialized-block-values-iterator-bugfix.
2016-09-17 11:46:15 -07:00
David Navas 9dbd4b864e
[SPARK-17529][CORE] Implement BitSet.clearUntil and use it during merge joins
## What changes were proposed in this pull request?

Add a clearUntil() method on BitSet (adapted from the pre-existing setUntil() method).
Use this method to clear the subset of the BitSet which needs to be used during merge joins.

## How was this patch tested?

dev/run-tests, as well as performance tests on skewed data as described in jira.

I expect there to be a small local performance hit using BitSet.clearUntil rather than BitSet.clear for normally shaped (unskewed) joins (additional read on the last long).  This is expected to be de-minimis and was not specifically tested.

Author: David Navas <davidn@clearstorydata.com>

Closes #15084 from davidnavas/bitSet.
2016-09-17 16:22:23 +01:00
Josh Rosen 1202075c95 [SPARK-17484] Prevent invalid block locations from being reported after put() exceptions
## What changes were proposed in this pull request?

If a BlockManager `put()` call failed after the BlockManagerMaster was notified of a block's availability then incomplete cleanup logic in a `finally` block would never send a second block status method to inform the master of the block's unavailability. This, in turn, leads to fetch failures and used to be capable of causing complete job failures before #15037 was fixed.

This patch addresses this issue via multiple small changes:

- The `finally` block now calls `removeBlockInternal` when cleaning up from a failed `put()`; in addition to removing the `BlockInfo` entry (which was _all_ that the old cleanup logic did), this code (redundantly) tries to remove the block from the memory and disk stores (as an added layer of defense against bugs lower down in the stack) and optionally notifies the master of block removal (which now happens during exception-triggered cleanup).
- When a BlockManager receives a request for a block that it does not have it will now notify the master to update its block locations. This ensures that bad metadata pointing to non-existent blocks will eventually be fixed. Note that I could have implemented this logic in the block manager client (rather than in the remote server), but that would introduce the problem of distinguishing between transient and permanent failures; on the server, however, we know definitively that the block isn't present.
- Catch `NonFatal` instead of `Exception` to avoid swallowing `InterruptedException`s thrown from synchronous block replication calls.

This patch depends upon the refactorings in #15036, so that other patch will also have to be backported when backporting this fix.

For more background on this issue, including example logs from a real production failure, see [SPARK-17484](https://issues.apache.org/jira/browse/SPARK-17484).

## How was this patch tested?

Two new regression tests in BlockManagerSuite.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #15085 from JoshRosen/SPARK-17484.
2016-09-15 11:54:17 -07:00
Shixiong Zhu e33bfaed3b [SPARK-17463][CORE] Make CollectionAccumulator and SetAccumulator's value can be read thread-safely
## What changes were proposed in this pull request?

Make CollectionAccumulator and SetAccumulator's value can be read thread-safely to fix the ConcurrentModificationException reported in [JIRA](https://issues.apache.org/jira/browse/SPARK-17463).

## How was this patch tested?

Existing tests.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #15063 from zsxwing/SPARK-17463.
2016-09-14 13:33:51 -07:00
Xin Wu 040e46979d [SPARK-10747][SQL] Support NULLS FIRST|LAST clause in ORDER BY
## What changes were proposed in this pull request?
Currently, ORDER BY clause returns nulls value according to sorting order (ASC|DESC), considering null value is always smaller than non-null values.
However, SQL2003 standard support NULLS FIRST or NULLS LAST to allow users to specify whether null values should be returned first or last, regardless of sorting order (ASC|DESC).

This PR is to support this new feature.

## How was this patch tested?
New test cases are added to test NULLS FIRST|LAST for regular select queries and windowing queries.

(If this patch involves UI changes, please attach a screenshot; otherwise, remove this)

Author: Xin Wu <xinwu@us.ibm.com>

Closes #14842 from xwu0226/SPARK-10747.
2016-09-14 21:14:29 +02:00
Josh Rosen f9c580f110 [SPARK-17485] Prevent failed remote reads of cached blocks from failing entire job
## What changes were proposed in this pull request?

In Spark's `RDD.getOrCompute` we first try to read a local copy of a cached RDD block, then a remote copy, and only fall back to recomputing the block if no cached copy (local or remote) can be read. This logic works correctly in the case where no remote copies of the block exist, but if there _are_ remote copies and reads of those copies fail (due to network issues or internal Spark bugs) then the BlockManager will throw a `BlockFetchException` that will fail the task (and which could possibly fail the whole job if the read failures keep occurring).

In the cases of TorrentBroadcast and task result fetching we really do want to fail the entire job in case no remote blocks can be fetched, but this logic is inappropriate for reads of cached RDD blocks because those can/should be recomputed in case cached blocks are unavailable.

Therefore, I think that the `BlockManager.getRemoteBytes()` method should never throw on remote fetch errors and, instead, should handle failures by returning `None`.

## How was this patch tested?

Block manager changes should be covered by modified tests in `BlockManagerSuite`: the old tests expected exceptions to be thrown on failed remote reads, while the modified tests now expect `None` to be returned from the `getRemote*` method.

I also manually inspected all usages of `BlockManager.getRemoteValues()`, `getRemoteBytes()`, and `get()` to verify that they correctly pattern-match on the result and handle `None`. Note that these `None` branches are already exercised because the old `getRemoteBytes` returned `None` when no remote locations for the block could be found (which could occur if an executor died and its block manager de-registered with the master).

Author: Josh Rosen <joshrosen@databricks.com>

Closes #15037 from JoshRosen/SPARK-17485.
2016-09-12 15:43:57 -07:00
Sean Zhong 1742c3ab86 [SPARK-17503][CORE] Fix memory leak in Memory store when unable to cache the whole RDD in memory
## What changes were proposed in this pull request?

   MemoryStore may throw OutOfMemoryError when trying to cache a super big RDD that cannot fit in memory.
   ```
   scala> sc.parallelize(1 to 1000000000, 100).map(x => new Array[Long](1000)).cache().count()

   java.lang.OutOfMemoryError: Java heap space
	at $line14.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(<console>:24)
	at $line14.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(<console>:23)
	at scala.collection.Iterator$$anon$11.next(Iterator.scala:409)
	at scala.collection.Iterator$JoinIterator.next(Iterator.scala:232)
	at org.apache.spark.storage.memory.PartiallyUnrolledIterator.next(MemoryStore.scala:683)
	at org.apache.spark.InterruptibleIterator.next(InterruptibleIterator.scala:43)
	at org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1684)
	at org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1134)
	at org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1134)
	at org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1915)
	at org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1915)
	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:70)
	at org.apache.spark.scheduler.Task.run(Task.scala:86)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:274)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
	at java.lang.Thread.run(Thread.java:745)
   ```

Spark MemoryStore uses SizeTrackingVector as a temporary unrolling buffer to store all input values that it has read so far before transferring the values to storage memory cache. The problem is that when the input RDD is too big for caching in memory, the temporary unrolling memory SizeTrackingVector is not garbage collected in time. As SizeTrackingVector can occupy all available storage memory, it may cause the executor JVM to run out of memory quickly.

More info can be found at https://issues.apache.org/jira/browse/SPARK-17503

## How was this patch tested?

Unit test and manual test.

### Before change

Heap memory consumption
<img width="702" alt="screen shot 2016-09-12 at 4 16 15 pm" src="https://cloud.githubusercontent.com/assets/2595532/18429524/60d73a26-7906-11e6-9768-6f286f5c58c8.png">

Heap dump
<img width="1402" alt="screen shot 2016-09-12 at 4 34 19 pm" src="https://cloud.githubusercontent.com/assets/2595532/18429577/cbc1ef20-7906-11e6-847b-b5903f450b3b.png">

### After change

Heap memory consumption
<img width="706" alt="screen shot 2016-09-12 at 4 29 10 pm" src="https://cloud.githubusercontent.com/assets/2595532/18429503/4abe9342-7906-11e6-844a-b2f815072624.png">

Author: Sean Zhong <seanzhong@databricks.com>

Closes #15056 from clockfly/memory_store_leak.
2016-09-12 11:30:06 -07:00
Joseph K. Bradley 65b814bf50 [SPARK-17456][CORE] Utility for parsing Spark versions
## What changes were proposed in this pull request?

This patch adds methods for extracting major and minor versions as Int types in Scala from a Spark version string.

Motivation: There are many hacks within Spark's codebase to identify and compare Spark versions. We should add a simple utility to standardize these code paths, especially since there have been mistakes made in the past. This will let us add unit tests as well.  Currently, I want this functionality to check Spark versions to provide backwards compatibility for ML model persistence.

## How was this patch tested?

Unit tests

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

Closes #15017 from jkbradley/version-parsing.
2016-09-09 05:35:10 -07:00
Gurvinder Singh 92ce8d4849 [SPARK-15487][WEB UI] Spark Master UI to reverse proxy Application and Workers UI
## What changes were proposed in this pull request?

This pull request adds the functionality to enable accessing worker and application UI through master UI itself. Thus helps in accessing SparkUI when running spark cluster in closed networks e.g. Kubernetes. Cluster admin needs to expose only spark master UI and rest of the UIs can be in the private network, master UI will reverse proxy the connection request to corresponding resource. It adds the path for workers/application UIs as

WorkerUI: <http/https>://master-publicIP:<port>/target/workerID/
ApplicationUI: <http/https>://master-publicIP:<port>/target/appID/

This makes it easy for users to easily protect the Spark master cluster access by putting some reverse proxy e.g. https://github.com/bitly/oauth2_proxy

## How was this patch tested?

The functionality has been tested manually and there is a unit test too for testing access to worker UI with reverse proxy address.

pwendell bomeng BryanCutler can you please review it, thanks.

Author: Gurvinder Singh <gurvinder.singh@uninett.no>

Closes #13950 from gurvindersingh/rproxy.
2016-09-08 17:20:20 -07:00
Eric Liang 649fa4bf1d [SPARK-17370] Shuffle service files not invalidated when a slave is lost
## What changes were proposed in this pull request?

DAGScheduler invalidates shuffle files when an executor loss event occurs, but not when the external shuffle service is enabled. This is because when shuffle service is on, the shuffle file lifetime can exceed the executor lifetime.

However, it also doesn't invalidate shuffle files when the shuffle service itself is lost (due to whole slave loss). This can cause long hangs when slaves are lost since the file loss is not detected until a subsequent stage attempts to read the shuffle files.

The proposed fix is to also invalidate shuffle files when an executor is lost due to a `SlaveLost` event.

## How was this patch tested?

Unit tests, also verified on an actual cluster that slave loss invalidates shuffle files immediately as expected.

cc mateiz

Author: Eric Liang <ekl@databricks.com>

Closes #14931 from ericl/sc-4439.
2016-09-07 12:33:50 -07:00
Liwei Lin 3ce3a282c8 [SPARK-17359][SQL][MLLIB] Use ArrayBuffer.+=(A) instead of ArrayBuffer.append(A) in performance critical paths
## What changes were proposed in this pull request?

We should generally use `ArrayBuffer.+=(A)` rather than `ArrayBuffer.append(A)`, because `append(A)` would involve extra boxing / unboxing.

## How was this patch tested?

N/A

Author: Liwei Lin <lwlin7@gmail.com>

Closes #14914 from lw-lin/append_to_plus_eq_v2.
2016-09-07 10:04:00 +01:00
Josh Rosen 29cfab3f15 [SPARK-17110] Fix StreamCorruptionException in BlockManager.getRemoteValues()
## What changes were proposed in this pull request?

This patch fixes a `java.io.StreamCorruptedException` error affecting remote reads of cached values when certain data types are used. The problem stems from #11801 / SPARK-13990, a patch to have Spark automatically pick the "best" serializer when caching RDDs. If PySpark cached a PythonRDD, then this would be cached as an `RDD[Array[Byte]]` and the automatic serializer selection would pick KryoSerializer for replication and block transfer. However, the `getRemoteValues()` / `getRemoteBytes()` code path did not pass proper class tags in order to enable the same serializer to be used during deserialization, causing Java to be inappropriately used instead of Kryo, leading to the StreamCorruptedException.

We already fixed a similar bug in #14311, which dealt with similar issues in block replication. Prior to that patch, it seems that we had no tests to ensure that block replication actually succeeded. Similarly, prior to this bug fix patch it looks like we had no tests to perform remote reads of cached data, which is why this bug was able to remain latent for so long.

This patch addresses the bug by modifying `BlockManager`'s `get()` and  `getRemoteValues()` methods to accept ClassTags, allowing the proper class tag to be threaded in the `getOrElseUpdate` code path (which is used by `rdd.iterator`)

## How was this patch tested?

Extended the caching tests in `DistributedSuite` to exercise the `getRemoteValues` path, plus manual testing to verify that the PySpark bug reproduction in SPARK-17110 is fixed.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #14952 from JoshRosen/SPARK-17110.
2016-09-06 15:07:28 -07:00
Angus Gerry a0aac4b775 [SPARK-16533][CORE] resolve deadlocking in driver when executors die
## What changes were proposed in this pull request?
This pull request reverts the changes made as a part of #14605, which simply side-steps the deadlock issue. Instead, I propose the following approach:
* Use `scheduleWithFixedDelay` when calling `ExecutorAllocationManager.schedule` for scheduling executor requests. The intent of this is that if invocations are delayed beyond the default schedule interval on account of lock contention, then we avoid a situation where calls to `schedule` are made back-to-back, potentially releasing and then immediately reacquiring these locks - further exacerbating contention.
* Replace a number of calls to `askWithRetry` with `ask` inside of message handling code in `CoarseGrainedSchedulerBackend` and its ilk. This allows us queue messages with the relevant endpoints, release whatever locks we might be holding, and then block whilst awaiting the response. This change is made at the cost of being able to retry should sending the message fail, as retrying outside of the lock could easily cause race conditions if other conflicting messages have been sent whilst awaiting a response. I believe this to be the lesser of two evils, as in many cases these RPC calls are to process local components, and so failures are more likely to be deterministic, and timeouts are more likely to be caused by lock contention.

## How was this patch tested?
Existing tests, and manual tests under yarn-client mode.

Author: Angus Gerry <angolon@gmail.com>

Closes #14710 from angolon/SPARK-16533.
2016-09-01 10:35:31 -07:00
Alex Bozarth f7beae6da0 [SPARK-17243][WEB UI] Spark 2.0 History Server won't load with very large application history
## What changes were proposed in this pull request?

With the new History Server the summary page loads the application list via the the REST API, this makes it very slow to impossible to load with large (10K+) application history. This pr fixes this by adding the `spark.history.ui.maxApplications` conf to limit the number of applications the History Server displays. This is accomplished using a new optional `limit` param for the `applications` api. (Note this only applies to what the summary page displays, all the Application UI's are still accessible if the user knows the App ID and goes to the Application UI directly.)

I've also added a new test for the `limit` param in `HistoryServerSuite.scala`

## How was this patch tested?

Manual testing and dev/run-tests

Author: Alex Bozarth <ajbozart@us.ibm.com>

Closes #14835 from ajbozarth/spark17243.
2016-08-30 16:33:54 -05:00
Ferdinand Xu 4b4e329e49 [SPARK-5682][CORE] Add encrypted shuffle in spark
This patch is using Apache Commons Crypto library to enable shuffle encryption support.

Author: Ferdinand Xu <cheng.a.xu@intel.com>
Author: kellyzly <kellyzly@126.com>

Closes #8880 from winningsix/SPARK-10771.
2016-08-30 09:15:31 -07:00
Xin Ren 27209252f0 [MINOR][MLLIB][SQL] Clean up unused variables and unused import
## What changes were proposed in this pull request?

Clean up unused variables and unused import statements, unnecessary `return` and `toArray`, and some more style improvement,  when I walk through the code examples.

## How was this patch tested?

Testet manually on local laptop.

Author: Xin Ren <iamshrek@126.com>

Closes #14836 from keypointt/codeWalkThroughML.
2016-08-30 11:24:55 +01:00
Xin Ren 2d76cb11f5 [SPARK-17276][CORE][TEST] Stop env params output on Jenkins job page
https://issues.apache.org/jira/browse/SPARK-17276

## What changes were proposed in this pull request?

When trying to find error msg in a failed Jenkins build job, I'm annoyed by the huge env output.
The env parameter output should be muted.

![screen shot 2016-08-26 at 10 52 07 pm](https://cloud.githubusercontent.com/assets/3925641/18025581/b8d567ba-6be2-11e6-9eeb-6aec223f1730.png)

## How was this patch tested?

Tested manually on local laptop.

Author: Xin Ren <iamshrek@126.com>

Closes #14848 from keypointt/SPARK-17276.
2016-08-30 11:18:29 +01:00
Yin Huai a6bca3ad02 [SPARK-17266][TEST] Add empty strings to the regressionTests of PrefixComparatorsSuite
## What changes were proposed in this pull request?
This PR adds a regression test to PrefixComparatorsSuite's "String prefix comparator" because this test failed on jenkins once (https://amplab.cs.berkeley.edu/jenkins/job/spark-master-test-sbt-hadoop-2.4/1620/testReport/junit/org.apache.spark.util.collection.unsafe.sort/PrefixComparatorsSuite/String_prefix_comparator/).

I could not reproduce it locally. But, let's this test case in the regressionTests.

Author: Yin Huai <yhuai@databricks.com>

Closes #14837 from yhuai/SPARK-17266.
2016-08-26 19:38:52 -07:00
Michael Gummelt 8e5475be3c [SPARK-16967] move mesos to module
## What changes were proposed in this pull request?

Move Mesos code into a mvn module

## How was this patch tested?

unit tests
manually submitting a client mode and cluster mode job
spark/mesos integration test suite

Author: Michael Gummelt <mgummelt@mesosphere.io>

Closes #14637 from mgummelt/mesos-module.
2016-08-26 12:25:22 -07:00
Marcelo Vanzin 9b5a1d1d53 [SPARK-17240][CORE] Make SparkConf serializable again.
Make the config reader transient, and initialize it lazily so that
serialization works with both java and kryo (and hopefully any other
custom serializer).

Added unit test to make sure SparkConf remains serializable and the
reader works with both built-in serializers.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #14813 from vanzin/SPARK-17240.
2016-08-25 16:11:42 -07:00
Alex Bozarth 891ac2b914 [SPARK-15083][WEB UI] History Server can OOM due to unlimited TaskUIData
## What changes were proposed in this pull request?

Based on #12990 by tankkyo

Since the History Server currently loads all application's data it can OOM if too many applications have a significant task count. `spark.ui.trimTasks` (default: false) can be set to true to trim tasks by `spark.ui.retainedTasks` (default: 10000)

(This is a "quick fix" to help those running into the problem until a update of how the history server loads app data can be done)

## How was this patch tested?

Manual testing and dev/run-tests

![spark-15083](https://cloud.githubusercontent.com/assets/13952758/17713694/fe82d246-63b0-11e6-9697-b87ea75ff4ef.png)

Author: Alex Bozarth <ajbozart@us.ibm.com>

Closes #14673 from ajbozarth/spark15083.
2016-08-24 14:39:41 -05:00
Eric Liang 8e223ea67a [SPARK-16550][SPARK-17042][CORE] Certain classes fail to deserialize in block manager replication
## What changes were proposed in this pull request?

This is a straightforward clone of JoshRosen 's original patch. I have follow-up changes to fix block replication for repl-defined classes as well, but those appear to be flaking tests so I'm going to leave that for SPARK-17042

## How was this patch tested?

End-to-end test in ReplSuite (also more tests in DistributedSuite from the original patch).

Author: Eric Liang <ekl@databricks.com>

Closes #14311 from ericl/spark-16550.
2016-08-22 16:32:14 -07:00
Nick Lavers 5377fc6236 [SPARK-16961][CORE] Fixed off-by-one error that biased randomizeInPlace
JIRA issue link:
https://issues.apache.org/jira/browse/SPARK-16961

Changed one line of Utils.randomizeInPlace to allow elements to stay in place.

Created a unit test that runs a Pearson's chi squared test to determine whether the output diverges significantly from a uniform distribution.

Author: Nick Lavers <nick.lavers@videoamp.com>

Closes #14551 from nicklavers/SPARK-16961-randomizeInPlace.
2016-08-19 10:11:59 +01:00
Marcelo Vanzin 5da6c4b24f [SPARK-16671][CORE][SQL] Consolidate code to do variable substitution.
Both core and sql have slightly different code that does variable substitution
of config values. This change refactors that code and encapsulates the logic
of reading config values and expading variables in a new helper class, which
can be configured so that both core and sql can use it without losing existing
functionality, and allows for easier testing and makes it easier to add more
features in the future.

Tested with existing and new unit tests, and by running spark-shell with
some configs referencing variables and making sure it behaved as expected.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #14468 from vanzin/SPARK-16671.
2016-08-15 11:09:54 -07:00
Stavros Kontopoulos 1a028bdefa [SPARK-11714][MESOS] Make Spark on Mesos honor port restrictions on coarse grain mode
- Make mesos coarse grained scheduler accept port offers and pre-assign ports

Previous attempt was for fine grained: https://github.com/apache/spark/pull/10808

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

Closes #11157 from skonto/honour_ports_coarse.
2016-08-15 09:55:32 +01:00
Xin Ren 7f7133bdcc [MINOR][CORE] fix warnings on depreciated methods in MesosClusterSchedulerSuite and DiskBlockObjectWriterSuite
## What changes were proposed in this pull request?

Fixed warnings below after scanning through warnings during build:

```
[warn] /home/jenkins/workspace/SparkPullRequestBuilder/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSuite.scala:34: imported `Utils' is permanently hidden by definition of object Utils in package mesos
[warn] import org.apache.spark.scheduler.cluster.mesos.Utils
[warn]                                                 ^
```

and
```
[warn] /home/jenkins/workspace/SparkPullRequestBuilder/core/src/test/scala/org/apache/spark/storage/DiskBlockObjectWriterSuite.scala:113: method shuffleBytesWritten in class ShuffleWriteMetrics is deprecated: use bytesWritten instead
[warn]     assert(writeMetrics.shuffleBytesWritten === file.length())
[warn]                         ^
[warn] /home/jenkins/workspace/SparkPullRequestBuilder/core/src/test/scala/org/apache/spark/storage/DiskBlockObjectWriterSuite.scala:119: method shuffleBytesWritten in class ShuffleWriteMetrics is deprecated: use bytesWritten instead
[warn]     assert(writeMetrics.shuffleBytesWritten === file.length())
[warn]                         ^
[warn] /home/jenkins/workspace/SparkPullRequestBuilder/core/src/test/scala/org/apache/spark/storage/DiskBlockObjectWriterSuite.scala:131: method shuffleBytesWritten in class ShuffleWriteMetrics is deprecated: use bytesWritten instead
[warn]     assert(writeMetrics.shuffleBytesWritten === file.length())
[warn]                         ^
[warn] /home/jenkins/workspace/SparkPullRequestBuilder/core/src/test/scala/org/apache/spark/storage/DiskBlockObjectWriterSuite.scala:135: method shuffleBytesWritten in class ShuffleWriteMetrics is deprecated: use bytesWritten instead
[warn]     assert(writeMetrics.shuffleBytesWritten === file.length())
[warn]                         ^
```

## How was this patch tested?

Tested manually on local laptop.

Author: Xin Ren <iamshrek@126.com>

Closes #14609 from keypointt/suiteWarnings.
2016-08-13 11:29:42 +01:00
Jeff Zhang 7a9e25c383 [SPARK-13081][PYSPARK][SPARK_SUBMIT] Allow set pythonExec of driver and executor through conf…
Before this PR, user have to export environment variable to specify the python of driver & executor which is not so convenient for users. This PR is trying to allow user to specify python through configuration "--pyspark-driver-python" & "--pyspark-executor-python"

Manually test in local & yarn mode for pyspark-shell and pyspark batch mode.

Author: Jeff Zhang <zjffdu@apache.org>

Closes #13146 from zjffdu/SPARK-13081.
2016-08-11 20:08:39 -07:00
Bryan Cutler 1c9a386c6b [SPARK-13602][CORE] Add shutdown hook to DriverRunner to prevent driver process leak
## What changes were proposed in this pull request?

Added shutdown hook to DriverRunner to kill the driver process in case the Worker JVM exits suddenly and the `WorkerWatcher` was unable to properly catch this.  Did some cleanup to consolidate driver state management and setting of finalized vars within the running thread.

## How was this patch tested?

Added unit tests to verify that final state and exception variables are set accordingly for successfull, failed, and errors in the driver process.  Retrofitted existing test to verify killing of mocked process ends with the correct state and stops properly

Manually tested (with deploy-mode=cluster) that the shutdown hook is called by forcibly exiting the `Worker` and various points in the code with the `WorkerWatcher` both disabled and enabled.  Also, manually killed the driver through the ui and verified that the `DriverRunner` interrupted, killed the process and exited properly.

Author: Bryan Cutler <cutlerb@gmail.com>

Closes #11746 from BryanCutler/DriverRunner-shutdown-hook-SPARK-13602.
2016-08-11 14:49:11 -07:00
Michael Gummelt 4d496802f5 [SPARK-16952] don't lookup spark home directory when executor uri is set
## What changes were proposed in this pull request?

remove requirement to set spark.mesos.executor.home when spark.executor.uri is used

## How was this patch tested?

unit tests

Author: Michael Gummelt <mgummelt@mesosphere.io>

Closes #14552 from mgummelt/fix-spark-home.
2016-08-11 11:36:20 +01:00
Rajesh Balamohan bd2c12fb49 [SPARK-12920][CORE] Honor "spark.ui.retainedStages" to reduce mem-pressure
When large number of jobs are run concurrently with Spark thrift server, thrift server starts running at high CPU due to GC pressure. Job UI retention causes memory pressure with large jobs. https://issues.apache.org/jira/secure/attachment/12783302/SPARK-12920.profiler_job_progress_listner.png has the profiler snapshot. This PR honors `spark.ui.retainedStages` strictly to reduce memory pressure.

Manual and unit tests

Author: Rajesh Balamohan <rbalamohan@apache.org>

Closes #10846 from rajeshbalamohan/SPARK-12920.
2016-08-10 15:30:52 -07:00
Sun Rui af710e5bdd [SPARK-16522][MESOS] Spark application throws exception on exit.
## What changes were proposed in this pull request?
Spark applications running on Mesos throw exception upon exit. For details, refer to https://issues.apache.org/jira/browse/SPARK-16522.

I am not sure if there is any better fix, so wait for review comments.

## How was this patch tested?
Manual test. Observed that the exception is gone upon application exit.

Author: Sun Rui <sunrui2016@gmail.com>

Closes #14175 from sun-rui/SPARK-16522.
2016-08-09 09:39:45 +01:00
Holden Karau 9216901d52 [SPARK-16779][TRIVIAL] Avoid using postfix operators where they do not add much and remove whitelisting
## What changes were proposed in this pull request?

Avoid using postfix operation for command execution in SQLQuerySuite where it wasn't whitelisted and audit existing whitelistings removing postfix operators from most places. Some notable places where postfix operation remains is in the XML parsing & time units (seconds, millis, etc.) where it arguably can improve readability.

## How was this patch tested?

Existing tests.

Author: Holden Karau <holden@us.ibm.com>

Closes #14407 from holdenk/SPARK-16779.
2016-08-08 15:54:03 -07:00
Josh Rosen 4f5f9b670e [SPARK-16925] Master should call schedule() after all executor exit events, not only failures
## What changes were proposed in this pull request?

This patch fixes a bug in Spark's standalone Master which could cause applications to hang if tasks cause executors to exit with zero exit codes.

As an example of the bug, run

```
sc.parallelize(1 to 1, 1).foreachPartition { _ => System.exit(0) }
```

on a standalone cluster which has a single Spark application. This will cause all executors to die but those executors won't be replaced unless another Spark application or worker joins or leaves the cluster (or if an executor exits with a non-zero exit code). This behavior is caused by a bug in how the Master handles the `ExecutorStateChanged` event: the current implementation calls `schedule()` only if the executor exited with a non-zero exit code, so a task which causes a JVM to unexpectedly exit "cleanly" will skip the `schedule()` call.

This patch addresses this by modifying the `ExecutorStateChanged` to always unconditionally call `schedule()`. This should be safe because it should always be safe to call `schedule()`; adding extra `schedule()` calls can only affect performance and should not introduce correctness bugs.

## How was this patch tested?

I added a regression test in `DistributedSuite`.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #14510 from JoshRosen/SPARK-16925.
2016-08-06 19:29:19 -07:00
Josh Rosen e9fc0b6a8b [SPARK-16787] SparkContext.addFile() should not throw if called twice with the same file
## What changes were proposed in this pull request?

The behavior of `SparkContext.addFile()` changed slightly with the introduction of the Netty-RPC-based file server, which was introduced in Spark 1.6 (where it was disabled by default) and became the default / only file server in Spark 2.0.0.

Prior to 2.0, calling `SparkContext.addFile()` with files that have the same name and identical contents would succeed. This behavior was never explicitly documented but Spark has behaved this way since very early 1.x versions.

In 2.0 (or 1.6 with the Netty file server enabled), the second `addFile()` call will fail with a requirement error because NettyStreamManager tries to guard against duplicate file registration.

This problem also affects `addJar()` in a more subtle way: the `fileServer.addJar()` call will also fail with an exception but that exception is logged and ignored; I believe that the problematic exception-catching path was mistakenly copied from some old code which was only relevant to very old versions of Spark and YARN mode.

I believe that this change of behavior was unintentional, so this patch weakens the `require` check so that adding the same filename at the same path will succeed.

At file download time, Spark tasks will fail with exceptions if an executor already has a local copy of a file and that file's contents do not match the contents of the file being downloaded / added. As a result, it's important that we prevent files with the same name and different contents from being served because allowing that can effectively brick an executor by preventing it from successfully launching any new tasks. Before this patch's change, this was prevented by forbidding `addFile()` from being called twice on files with the same name. Because Spark does not defensively copy local files that are passed to `addFile` it is vulnerable to files' contents changing, so I think it's okay to rely on an implicit assumption that these files are intended to be immutable (since if they _are_ mutable then this can lead to either explicit task failures or implicit incorrectness (in case new executors silently get newer copies of the file while old executors continue to use an older version)). To guard against this, I have decided to only update the file addition timestamps on the first call to `addFile()`; duplicate calls will succeed but will not update the timestamp. This behavior is fine as long as we assume files are immutable, which seems reasonable given the behaviors described above.

As part of this change, I also improved the thread-safety of the `addedJars` and `addedFiles` maps; this is important because these maps may be concurrently read by a task launching thread and written by a driver thread in case the user's driver code is multi-threaded.

## How was this patch tested?

I added regression tests in `SparkContextSuite`.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #14396 from JoshRosen/SPARK-16787.
2016-08-02 12:02:11 -07:00
Sean Owen 0dc4310b47 [SPARK-16694][CORE] Use for/foreach rather than map for Unit expressions whose side effects are required
## What changes were proposed in this pull request?

Use foreach/for instead of map where operation requires execution of body, not actually defining a transformation

## How was this patch tested?

Jenkins

Author: Sean Owen <sowen@cloudera.com>

Closes #14332 from srowen/SPARK-16694.
2016-07-30 04:42:38 -07:00
Michael Gummelt 266b92faff [SPARK-16637] Unified containerizer
## What changes were proposed in this pull request?

New config var: spark.mesos.docker.containerizer={"mesos","docker" (default)}

This adds support for running docker containers via the Mesos unified containerizer: http://mesos.apache.org/documentation/latest/container-image/

The benefit is losing the dependency on `dockerd`, and all the costs which it incurs.

I've also updated the supported Mesos version to 0.28.2 for support of the required protobufs.

This is blocked on: https://github.com/apache/spark/pull/14167

## How was this patch tested?

- manually testing jobs submitted with both "mesos" and "docker" settings for the new config var.
- spark/mesos integration test suite

Author: Michael Gummelt <mgummelt@mesosphere.io>

Closes #14275 from mgummelt/unified-containerizer.
2016-07-29 05:50:47 -07:00
Mark Grover 70f846a313 [SPARK-5847][CORE] Allow for configuring MetricsSystem's use of app ID to namespace all metrics
## What changes were proposed in this pull request?
Adding a new property to SparkConf called spark.metrics.namespace that allows users to
set a custom namespace for executor and driver metrics in the metrics systems.

By default, the root namespace used for driver or executor metrics is
the value of `spark.app.id`. However, often times, users want to be able to track the metrics
across apps for driver and executor metrics, which is hard to do with application ID
(i.e. `spark.app.id`) since it changes with every invocation of the app. For such use cases,
users can set the `spark.metrics.namespace` property to another spark configuration key like
`spark.app.name` which is then used to populate the root namespace of the metrics system
(with the app name in our example). `spark.metrics.namespace` property can be set to any
arbitrary spark property key, whose value would be used to set the root namespace of the
metrics system. Non driver and executor metrics are never prefixed with `spark.app.id`, nor
does the `spark.metrics.namespace` property have any such affect on such metrics.

## How was this patch tested?
Added new unit tests, modified existing unit tests.

Author: Mark Grover <mark@apache.org>

Closes #14270 from markgrover/spark-5847.
2016-07-27 10:13:15 -07:00
Dhruve Ashar 0b71d9ae08 [SPARK-15703][SCHEDULER][CORE][WEBUI] Make ListenerBus event queue size configurable
## What changes were proposed in this pull request?
This change adds a new configuration entry to specify the size of the spark listener bus event queue. The value for this config ("spark.scheduler.listenerbus.eventqueue.size") is set to a default to 10000.

Note:
I haven't currently documented the configuration entry. We can decide whether it would be appropriate to make it a public configuration or keep it as an undocumented one. Refer JIRA for more details.

## How was this patch tested?
Ran existing jobs and verified the event queue size with debug logs and from the Spark WebUI Environment tab.

Author: Dhruve Ashar <dhruveashar@gmail.com>

Closes #14269 from dhruve/bug/SPARK-15703.
2016-07-26 13:23:33 -05:00
Philipp Hoffmann 0869b3a5f0 [SPARK-15271][MESOS] Allow force pulling executor docker images
## What changes were proposed in this pull request?

(Please fill in changes proposed in this fix)

## How was this patch tested?

(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)

(If this patch involves UI changes, please attach a screenshot; otherwise, remove this)

Mesos agents by default will not pull docker images which are cached
locally already. In order to run Spark executors from mutable tags like
`:latest` this commit introduces a Spark setting
(`spark.mesos.executor.docker.forcePullImage`). Setting this flag to
true will tell the Mesos agent to force pull the docker image (default is `false` which is consistent with the previous
implementation and Mesos' default
behaviour).

Author: Philipp Hoffmann <mail@philipphoffmann.de>

Closes #14348 from philipphoffmann/force-pull-image.
2016-07-26 16:09:10 +01:00
Tao Lin db36e1e75d [SPARK-15590][WEBUI] Paginate Job Table in Jobs tab
## What changes were proposed in this pull request?

This patch adds pagination support for the Job Tables in the Jobs tab. Pagination is provided for all of the three Job Tables (active, completed, and failed). Interactions (jumping, sorting, and setting page size) for paged tables are also included.

The diff didn't keep track of some lines based on the original ones. The function `makeRow`of the original `AllJobsPage.scala` is reused. They are separated at the beginning of the function `jobRow` (L427-439) and the function `row`(L594-618) in the new `AllJobsPage.scala`.

## How was this patch tested?

Tested manually by using checking the Web UI after completing and failing hundreds of jobs.
Generate completed jobs by:
```scala
val d = sc.parallelize(Array(1,2,3,4,5))
for(i <- 1 to 255){ var b = d.collect() }
```
Generate failed jobs by calling the following code multiple times:
```scala
var b = d.map(_/0).collect()
```
Interactions like jumping, sorting, and setting page size are all tested.

This shows the pagination for completed jobs:
![paginate success jobs](https://cloud.githubusercontent.com/assets/5558370/15986498/efa12ef6-303b-11e6-8b1d-c3382aeb9ad0.png)

This shows the sorting works in job tables:
![sorting](https://cloud.githubusercontent.com/assets/5558370/15986539/98c8a81a-303c-11e6-86f2-8d2bc7924ee9.png)

This shows the pagination for failed jobs and the effect of jumping and setting page size:
![paginate failed jobs](https://cloud.githubusercontent.com/assets/5558370/15986556/d8c1323e-303c-11e6-8e4b-7bdb030ea42b.png)

Author: Tao Lin <nblintao@gmail.com>

Closes #13620 from nblintao/dev.
2016-07-25 17:35:50 -07:00
jerryshao f5ea7fe539 [SPARK-16166][CORE] Also take off-heap memory usage into consideration in log and webui display
## What changes were proposed in this pull request?

Currently in the log and UI display, only on-heap storage memory is calculated and displayed,

```
16/06/27 13:41:52 INFO MemoryStore: Block rdd_5_0 stored as values in memory (estimated size 17.8 KB, free 665.9 MB)
```
<img width="1232" alt="untitled" src="https://cloud.githubusercontent.com/assets/850797/16369960/53fb614e-3c6e-11e6-8fa3-7ffe65abcb49.png">

With [SPARK-13992](https://issues.apache.org/jira/browse/SPARK-13992) off-heap memory is supported for data persistence, so here change to also take off-heap storage memory into consideration.

## How was this patch tested?

Unit test and local verification.

Author: jerryshao <sshao@hortonworks.com>

Closes #13920 from jerryshao/SPARK-16166.
2016-07-25 15:17:06 -07:00
Josh Rosen fc17121d59 Revert "[SPARK-15271][MESOS] Allow force pulling executor docker images"
This reverts commit 978cd5f125.
2016-07-25 12:43:44 -07:00
Philipp Hoffmann 978cd5f125 [SPARK-15271][MESOS] Allow force pulling executor docker images
## What changes were proposed in this pull request?

Mesos agents by default will not pull docker images which are cached
locally already. In order to run Spark executors from mutable tags like
`:latest` this commit introduces a Spark setting
`spark.mesos.executor.docker.forcePullImage`. Setting this flag to
true will tell the Mesos agent to force pull the docker image (default is `false` which is consistent with the previous
implementation and Mesos' default
behaviour).

## How was this patch tested?

I ran a sample application including this change on a Mesos cluster and verified the correct behaviour for both, with and without, force pulling the executor image. As expected the image is being force pulled if the flag is set.

Author: Philipp Hoffmann <mail@philipphoffmann.de>

Closes #13051 from philipphoffmann/force-pull-image.
2016-07-25 20:14:47 +01:00
Brian Cho daace60142 [SPARK-5581][CORE] When writing sorted map output file, avoid open / …
…close between each partition

## What changes were proposed in this pull request?

Replace commitAndClose with separate commit and close to avoid opening and closing
the file between partitions.

## How was this patch tested?

Run existing unit tests, add a few unit tests regarding reverts.

Observed a ~20% reduction in total time in tasks on stages with shuffle
writes to many partitions.

JoshRosen

Author: Brian Cho <bcho@fb.com>

Closes #13382 from dafrista/separatecommit-master.
2016-07-24 19:36:58 -07:00
Michael Gummelt 235cb256d0 [SPARK-16194] Mesos Driver env vars
## What changes were proposed in this pull request?

Added new configuration namespace: spark.mesos.env.*

This allows a user submitting a job in cluster mode to set arbitrary environment variables on the driver.
spark.mesos.driverEnv.KEY=VAL will result in the env var "KEY" being set to "VAL"

I've also refactored the tests a bit so we can re-use code in MesosClusterScheduler.

And I've refactored the command building logic in `buildDriverCommand`.  Command builder values were very intertwined before, and now it's easier to determine exactly how each variable is set.

## How was this patch tested?

unit tests

Author: Michael Gummelt <mgummelt@mesosphere.io>

Closes #14167 from mgummelt/driver-env-vars.
2016-07-21 18:29:00 +01:00
Marcelo Vanzin 75a06aa256 [SPARK-16272][CORE] Allow config values to reference conf, env, system props.
This allows configuration to be more flexible, for example, when the cluster does
not have a homogeneous configuration (e.g. packages are installed on different
paths in different nodes). By allowing one to reference the environment from
the conf, it becomes possible to work around those in certain cases.

As part of the implementation, ConfigEntry now keeps track of all "known" configs
(i.e. those created through the use of ConfigBuilder), since that list is used
by the resolution code. This duplicates some code in SQLConf, which could potentially
be merged with this now. It will also make it simpler to implement some missing
features such as filtering which configs show up in the UI or in event logs - which
are not part of this change.

Another change is in the way ConfigEntry reads config data; it now takes a string
map and a function that reads env variables, so that it can be called both from
SparkConf and SQLConf. This makes it so both places follow the same read path,
instead of having to replicate certain logic in SQLConf. There are still a
couple of methods in SQLConf that peek into fields of ConfigEntry directly,
though.

Tested via unit tests, and by using the new variable expansion functionality
in a shell session with a custom spark.sql.hive.metastore.jars value.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #14022 from vanzin/SPARK-16272.
2016-07-20 18:24:35 -07:00
Sean Owen 4b079dc396 [SPARK-16613][CORE] RDD.pipe returns values for empty partitions
## What changes were proposed in this pull request?

Document RDD.pipe semantics; don't execute process for empty input partitions.

Note this includes the fix in https://github.com/apache/spark/pull/14256 because it's necessary to even test this. One or the other will merge the fix.

## How was this patch tested?

Jenkins tests including new test.

Author: Sean Owen <sowen@cloudera.com>

Closes #14260 from srowen/SPARK-16613.
2016-07-20 09:48:52 -07:00
Shivaram Venkataraman fc23263623 [SPARK-10683][SPARK-16510][SPARKR] Move SparkR include jar test to SparkSubmitSuite
## What changes were proposed in this pull request?

This change moves the include jar test from R to SparkSubmitSuite and uses a dynamically compiled jar. This helps us remove the binary jar from the R package and solves both the CRAN warnings and the lack of source being available for this jar.

## How was this patch tested?
SparkR unit tests, SparkSubmitSuite, check-cran.sh

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

Closes #14243 from shivaram/sparkr-jar-move.
2016-07-19 19:28:08 -07:00
Andrew Duffy 004e29cba5 [SPARK-14702] Make environment of SparkLauncher launched process more configurable
## What changes were proposed in this pull request?

Adds a few public methods to `SparkLauncher` to allow configuring some extra features of the `ProcessBuilder`, including the working directory, output and error stream redirection.

## How was this patch tested?

Unit testing + simple Spark driver programs

Author: Andrew Duffy <root@aduffy.org>

Closes #14201 from andreweduffy/feature/launcher.
2016-07-19 17:08:38 -07:00
Liwei Lin 0bd76e872b [SPARK-16620][CORE] Add back the tokenization process in RDD.pipe(command: String)
## What changes were proposed in this pull request?

Currently `RDD.pipe(command: String)`:
- works only when the command is specified without any options, such as `RDD.pipe("wc")`
- does NOT work when the command is specified with some options, such as `RDD.pipe("wc -l")`

This is a regression from Spark 1.6.

This patch adds back the tokenization process in `RDD.pipe(command: String)` to fix this regression.

## How was this patch tested?
Added a test which:
- would pass in `1.6`
- _[prior to this patch]_ would fail in `master`
- _[after this patch]_ would pass in `master`

Author: Liwei Lin <lwlin7@gmail.com>

Closes #14256 from lw-lin/rdd-pipe.
2016-07-19 10:24:48 -07:00
jerryshao d8220c1e5e [SPARK-16435][YARN][MINOR] Add warning log if initialExecutors is less than minExecutors
## What changes were proposed in this pull request?

Currently if `spark.dynamicAllocation.initialExecutors` is less than `spark.dynamicAllocation.minExecutors`, Spark will automatically pick the minExecutors without any warning. While in 1.6 Spark will throw exception if configured like this. So here propose to add warning log if these parameters are configured invalidly.

## How was this patch tested?

Unit test added to verify the scenario.

Author: jerryshao <sshao@hortonworks.com>

Closes #14149 from jerryshao/SPARK-16435.
2016-07-13 13:24:47 -05:00
Alex Bozarth f156136dae [SPARK-16375][WEB UI] Fixed misassigned var: numCompletedTasks was assigned to numSkippedTasks
## What changes were proposed in this pull request?

I fixed a misassigned var,  numCompletedTasks was assigned to numSkippedTasks in the convertJobData method

## How was this patch tested?

dev/run-tests

Author: Alex Bozarth <ajbozart@us.ibm.com>

Closes #14141 from ajbozarth/spark16375.
2016-07-13 10:45:06 +01:00
Eric Liang d8b06f18dc [SPARK-16432] Empty blocks fail to serialize due to assert in ChunkedByteBuffer
## What changes were proposed in this pull request?

It's possible to also change the callers to not pass in empty chunks, but it seems cleaner to just allow `ChunkedByteBuffer` to handle empty arrays. cc JoshRosen

## How was this patch tested?

Unit tests, also checked that the original reproduction case in https://github.com/apache/spark/pull/11748#issuecomment-230760283 is resolved.

Author: Eric Liang <ekl@databricks.com>

Closes #14099 from ericl/spark-16432.
2016-07-08 20:18:49 -07:00
Tom Magrino ce3ea96980 [SPARK-15885][WEB UI] Provide links to executor logs from stage details page in UI
## What changes were proposed in this pull request?

This moves over old PR https://github.com/apache/spark/pull/13664 to target master rather than branch-1.6.

Added links to logs (or an indication that there are no logs) for entries which list an executor in the stage details page of the UI.

This helps streamline the workflow where a user views a stage details page and determines that they would like to see the associated executor log for further examination.  Previously, a user would have to cross reference the executor id listed on the stage details page with the corresponding entry on the executors tab.

Link to the JIRA: https://issues.apache.org/jira/browse/SPARK-15885

## How was this patch tested?

Ran existing unit tests.
Ran test queries on a platform which did not record executor logs and again on a platform which did record executor logs and verified that the new table column was empty and links to the logs (which were verified as linking to the appropriate files), respectively.

Attached is a screenshot of the UI page with no links, with the new columns highlighted.  Additional screenshot of these columns with the populated links.

Without links:
![updated without logs](https://cloud.githubusercontent.com/assets/1450821/16059721/2b69dbaa-3239-11e6-9eed-e539764ca159.png)

With links:
![updated with logs](https://cloud.githubusercontent.com/assets/1450821/16059725/32c6e316-3239-11e6-90bd-2553f43f7779.png)

This contribution is my original work and I license the work to the project under the Apache Spark project's open source license.

Author: Tom Magrino <tmagrino@fb.com>

Closes #13861 from tmagrino/uilogstweak.
2016-07-07 00:02:39 -07:00
petermaxlee 480357cc6d [SPARK-16304] LinkageError should not crash Spark executor
## What changes were proposed in this pull request?
This patch updates the failure handling logic so Spark executor does not crash when seeing LinkageError.

## How was this patch tested?
Added an end-to-end test in FailureSuite.

Author: petermaxlee <petermaxlee@gmail.com>

Closes #13982 from petermaxlee/SPARK-16304.
2016-07-06 10:46:22 -07:00
Sean Owen 2075bf8ef6 [SPARK-16182][CORE] Utils.scala -- terminateProcess() should call Process.destroyForcibly() if and only if Process.destroy() fails
## What changes were proposed in this pull request?

Utils.terminateProcess should `destroy()` first and only fall back to `destroyForcibly()` if it fails. It's kind of bad that we're force-killing executors -- and only in Java 8. See JIRA for an example of the impact: no shutdown

While here: `Utils.waitForProcess` should use the Java 8 method if available instead of a custom implementation.

## How was this patch tested?

Existing tests, which cover the force-kill case, and Amplab tests, which will cover both Java 7 and Java 8 eventually. However I tested locally on Java 8 and the PR builder will try Java 7 here.

Author: Sean Owen <sowen@cloudera.com>

Closes #13973 from srowen/SPARK-16182.
2016-07-01 09:22:27 +01:00
Imran Rashid fdf9f94f8c [SPARK-15865][CORE] Blacklist should not result in job hanging with less than 4 executors
## What changes were proposed in this pull request?

Before this change, when you turn on blacklisting with `spark.scheduler.executorTaskBlacklistTime`, but you have fewer than `spark.task.maxFailures` executors, you can end with a job "hung" after some task failures.

Whenever a taskset is unable to schedule anything on resourceOfferSingleTaskSet, we check whether the last pending task can be scheduled on *any* known executor.  If not, the taskset (and any corresponding jobs) are failed.
* Worst case, this is O(maxTaskFailures + numTasks).  But unless many executors are bad, this should be small
* This does not fail as fast as possible -- when a task becomes unschedulable, we keep scheduling other tasks.  This is to avoid an O(numPendingTasks * numExecutors) operation
* Also, it is conceivable this fails too quickly.  You may be 1 millisecond away from unblacklisting a place for a task to run, or acquiring a new executor.

## How was this patch tested?

Added unit test which failed before the change, ran new test 5k times manually, ran all scheduler tests manually, and the full suite via jenkins.

Author: Imran Rashid <irashid@cloudera.com>

Closes #13603 from squito/progress_w_few_execs_and_blacklist.
2016-06-30 13:36:06 -05:00
Sital Kedia 07f46afc73 [SPARK-13850] Force the sorter to Spill when number of elements in th…
## What changes were proposed in this pull request?

Force the sorter to Spill when number of elements in the pointer array reach a certain size. This is to workaround the issue of timSort failing on large buffer size.

## How was this patch tested?

Tested by running a job which was failing without this change due to TimSort bug.

Author: Sital Kedia <skedia@fb.com>

Closes #13107 from sitalkedia/fix_TimSort.
2016-06-30 10:53:18 -07:00
Tom Magrino ae14f36235 [SPARK-16148][SCHEDULER] Allow for underscores in TaskLocation in the Executor ID
## What changes were proposed in this pull request?

Previously, the TaskLocation implementation would not allow for executor ids which include underscores.  This tweaks the string split used to get the hostname and executor id, allowing for underscores in the executor id.

This addresses the JIRA found here: https://issues.apache.org/jira/browse/SPARK-16148

This is moved over from a previous PR against branch-1.6: https://github.com/apache/spark/pull/13857

## How was this patch tested?

Ran existing unit tests for core and streaming.  Manually ran a simple streaming job with an executor whose id contained underscores and confirmed that the job ran successfully.

This is my original work and I license the work to the project under the project's open source license.

Author: Tom Magrino <tmagrino@fb.com>

Closes #13858 from tmagrino/fixtasklocation.
2016-06-28 13:36:41 -07:00
Imran Rashid c15b552dd5 [SPARK-16106][CORE] TaskSchedulerImpl should properly track executors added to existing hosts
## What changes were proposed in this pull request?

TaskSchedulerImpl used to only set `newExecAvailable` when a new *host* was added, not when a new executor was added to an existing host.  It also didn't update some internal state tracking live executors until a task was scheduled on the executor.  This patch changes it to properly update as soon as it knows about a new executor.

## How was this patch tested?

added a unit test, ran everything via jenkins.

Author: Imran Rashid <irashid@cloudera.com>

Closes #13826 from squito/SPARK-16106_executorByHosts.
2016-06-27 16:38:03 -05:00
Imran Rashid 282158914d [SPARK-16136][CORE] Fix flaky TaskManagerSuite
## What changes were proposed in this pull request?

TaskManagerSuite "Kill other task attempts when one attempt belonging to the same task succeeds" was flaky.  When checking whether a task is speculatable, at least one millisecond must pass since the task was submitted.  Use a manual clock to avoid the problem.

I noticed these tests were leaving lots of threads lying around as well (which prevented me from running the test repeatedly), so I fixed that too.

## How was this patch tested?

Ran the test 1k times on my laptop, passed every time (it failed about 20% of the time before this).

Author: Imran Rashid <irashid@cloudera.com>

Closes #13848 from squito/fix_flaky_taskmanagersuite.
2016-06-27 16:28:59 -05:00
Sital Kedia bf665a9586 [SPARK-15958] Make initial buffer size for the Sorter configurable
## What changes were proposed in this pull request?

Currently the initial buffer size in the sorter is hard coded inside the code and is too small for large workload. As a result, the sorter spends significant time expanding the buffer size and copying the data. It would be useful to have it configurable.

## How was this patch tested?

Tested by running a job on the cluster.

Author: Sital Kedia <skedia@fb.com>

Closes #13699 from sitalkedia/config_sort_buffer_upstream.
2016-06-25 09:13:39 +01:00
Liwei Lin a4851ed050 [SPARK-15963][CORE] Catch TaskKilledException correctly in Executor.TaskRunner
## The problem

Before this change, if either of the following cases happened to a task , the task would be marked as `FAILED` instead of `KILLED`:
- the task was killed before it was deserialized
- `executor.kill()` marked `taskRunner.killed`, but before calling `task.killed()` the worker thread threw the `TaskKilledException`

The reason is, in the `catch` block of the current [Executor.TaskRunner](https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/executor/Executor.scala#L362)'s implementation, we are mistakenly catching:
```scala
case _: TaskKilledException | _: InterruptedException if task.killed => ...
```
the semantics of which is:
- **(**`TaskKilledException` **OR** `InterruptedException`**)** **AND** `task.killed`

Then when `TaskKilledException` is thrown but `task.killed` is not marked, we would mark the task as `FAILED` (which should really be `KILLED`).

## What changes were proposed in this pull request?

This patch alters the catch condition's semantics from:
- **(**`TaskKilledException` **OR** `InterruptedException`**)** **AND** `task.killed`

to

- `TaskKilledException` **OR** **(**`InterruptedException` **AND** `task.killed`**)**

so that we can catch `TaskKilledException` correctly and mark the task as `KILLED` correctly.

## How was this patch tested?

Added unit test which failed before the change, ran new test 1000 times manually

Author: Liwei Lin <lwlin7@gmail.com>

Closes #13685 from lw-lin/fix-task-killed.
2016-06-24 10:09:04 -05:00
peng.zhang f4fd7432fb [SPARK-16125][YARN] Fix not test yarn cluster mode correctly in YarnClusterSuite
## What changes were proposed in this pull request?

Since SPARK-13220(Deprecate "yarn-client" and "yarn-cluster"), YarnClusterSuite doesn't test "yarn cluster" mode correctly.
This pull request fixes it.

## How was this patch tested?
Unit test

(If this patch involves UI changes, please attach a screenshot; otherwise, remove this)

Author: peng.zhang <peng.zhang@xiaomi.com>

Closes #13836 from renozhang/SPARK-16125-test-yarn-cluster-mode.
2016-06-24 08:28:32 +01:00
Ryan Blue 738f134bf4 [SPARK-13723][YARN] Change behavior of --num-executors with dynamic allocation.
## What changes were proposed in this pull request?

This changes the behavior of --num-executors and spark.executor.instances when using dynamic allocation. Instead of turning dynamic allocation off, it uses the value for the initial number of executors.

This changes was discussed on [SPARK-13723](https://issues.apache.org/jira/browse/SPARK-13723). I highly recommend using it while we can change the behavior for 2.0.0. In practice, the 1.x behavior causes unexpected behavior for users (it is not clear that it disables dynamic allocation) and wastes cluster resources because users rarely notice the log message.

## How was this patch tested?

This patch updates tests and adds a test for Utils.getDynamicAllocationInitialExecutors.

Author: Ryan Blue <blue@apache.org>

Closes #13338 from rdblue/SPARK-13723-num-executors-with-dynamic-allocation.
2016-06-23 14:03:46 -05:00
Dongjoon Hyun 5eef1e6c6a [SPARK-15660][CORE] Update RDD variance/stdev description and add popVariance/popStdev
## What changes were proposed in this pull request?

In Spark-11490, `variance/stdev` are redefined as the **sample** `variance/stdev` instead of population ones. This PR updates the other old documentations to prevent users from misunderstanding. This will update the following Scala/Java API docs.

- http://spark.apache.org/docs/2.0.0-preview/api/scala/index.html#org.apache.spark.api.java.JavaDoubleRDD
- http://spark.apache.org/docs/2.0.0-preview/api/scala/index.html#org.apache.spark.rdd.DoubleRDDFunctions
- http://spark.apache.org/docs/2.0.0-preview/api/scala/index.html#org.apache.spark.util.StatCounter
- http://spark.apache.org/docs/2.0.0-preview/api/java/org/apache/spark/api/java/JavaDoubleRDD.html
- http://spark.apache.org/docs/2.0.0-preview/api/java/org/apache/spark/rdd/DoubleRDDFunctions.html
- http://spark.apache.org/docs/2.0.0-preview/api/java/org/apache/spark/util/StatCounter.html

Also, this PR adds them `popVariance` and `popStdev` functions clearly.

## How was this patch tested?

Pass the updated Jenkins tests.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #13403 from dongjoon-hyun/SPARK-15660.
2016-06-23 11:07:34 +01:00
Eric Liang 6f915c9ec2 [SPARK-16003] SerializationDebugger runs into infinite loop
## What changes were proposed in this pull request?

This fixes SerializationDebugger to not recurse forever when `writeReplace` returns an object of the same class, which is the case for at least the `SQLMetrics` class.

See also the OpenJDK unit tests on the behavior of recursive `writeReplace()`:
f4d80957e8/test/java/io/Serializable/nestedReplace/NestedReplace.java

cc davies cloud-fan

## How was this patch tested?

Unit tests for SerializationDebugger.

Author: Eric Liang <ekl@databricks.com>

Closes #13814 from ericl/spark-16003.
2016-06-22 12:12:34 -07:00
Imran Rashid cf1995a976 [SPARK-15783][CORE] Fix Flakiness in BlacklistIntegrationSuite
## What changes were proposed in this pull request?

Three changes here -- first two were causing failures w/ BlacklistIntegrationSuite

1. The testing framework didn't include the reviveOffers thread, so the test which involved delay scheduling might never submit offers late enough for the delay scheduling to kick in.  So added in the periodic revive offers, just like the real scheduler.

2. `assertEmptyDataStructures` would occasionally fail, because it appeared there was still an active job.  This is because in DAGScheduler, the jobWaiter is notified of the job completion before the data structures are cleaned up.  Most of the time the test code that is waiting on the jobWaiter won't become active until after the data structures are cleared, but occasionally the race goes the other way, and the assertions fail.

3. `DAGSchedulerSuite` was not stopping all the inner parts it was setting up, so each test was leaking a number of threads.  So we stop those parts too.

4. Turns out that `assertMapOutputAvailable` is not terribly useful in this framework -- most of the places I was trying to use it suffer from some race.

5. When there is an exception in the backend, try to improve the error msg a little bit.  Before the exception was printed to the console, but the test would fail w/ a timeout, and the logs wouldn't show anything.

## How was this patch tested?

I ran all the tests in `BlacklistIntegrationSuite` 5k times and everything in `DAGSchedulerSuite` 1k times on my laptop.  Also I ran a full jenkins build with `BlacklistIntegrationSuite` 500 times and `DAGSchedulerSuite` 50 times, see https://github.com/apache/spark/pull/13548.  (I tried more times but jenkins timed out.)

To check for more leaked threads, I added some code to dump the list of all threads at the end of each test in DAGSchedulerSuite, which is how I discovered the mapOutputTracker and eventLoop were leaking threads.  (I removed that code from the final pr, just part of the testing.)

And I'll run Jenkins on this a couple of times to do one more check.

Author: Imran Rashid <irashid@cloudera.com>

Closes #13565 from squito/blacklist_extra_tests.
2016-06-22 08:35:41 -05:00
Shixiong Zhu 62d8fe2089 [SPARK-16017][CORE] Send hostname from CoarseGrainedExecutorBackend to driver
## What changes were proposed in this pull request?

[SPARK-15395](https://issues.apache.org/jira/browse/SPARK-15395) changes the behavior that how the driver gets the executor host and the driver will get the executor IP address instead of the host name. This PR just sends the hostname from executors to driver so that driver can pass it to TaskScheduler.

## How was this patch tested?

Existing unit tests.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #13741 from zsxwing/SPARK-16017.
2016-06-17 15:48:17 -07:00
Kay Ousterhout c8809db5a5 [SPARK-15926] Improve readability of DAGScheduler stage creation methods
## What changes were proposed in this pull request?

This pull request refactors parts of the DAGScheduler to improve readability, focusing on the code around stage creation.  One goal of this change it to make it clearer which functions may create new stages (as opposed to looking up stages that already exist).  There are no functionality changes in this pull request.  In more detail:

* shuffleToMapStage was renamed to shuffleIdToMapStage (when reading the existing code I have sometimes struggled to remember what the key is -- is it a stage? A stage id? This change is intended to avoid that confusion)
* Cleaned up the code to create shuffle map stages.  Previously, creating a shuffle map stage involved 3 different functions (newOrUsedShuffleStage, newShuffleMapStage, and getShuffleMapStage), and it wasn't clear what the purpose of each function was.  With the new code, a single function (getOrCreateShuffleMapStage) is responsible for getting a stage (if it already exists) or creating new shuffle map stages and any missing ancestor stages, and it delegates to createShuffleMapStage when new stages need to be created.  There's some remaining confusion here because the getOrCreateParentStages call in createShuffleMapStage may recursively create ancestor stages; this is an issue I plan to fix in a future pull request, because it's trickier to fix and involves a slight functionality change.
* newResultStage was renamed to createResultStage, for consistency with naming around shuffle map stages.
* getParentStages has been renamed to getOrCreateParentStages, to make it clear that this function will sometimes create missing ancestor stages.
* The only *slight* functionality change is that on line 478, updateJobIdStageIdMaps now uses a stage's parents instance variable rather than re-calculating them (I couldn't see any reason why they'd need to be re-calculated, and suspect this is just leftover from older code).
* getAncestorShuffleDependencies was renamed to getMissingAncestorShuffleDependencies, to make it clear that this only returns dependencies that have not yet been run.

cc squito markhamstra JoshRosen (who requested more DAG scheduler commenting long ago -- an issue this pull request tries, in part, to address)

FYI rxin

Author: Kay Ousterhout <kayousterhout@gmail.com>

Closes #13677 from kayousterhout/SPARK-15926.
2016-06-17 12:12:46 -07:00
Nezih Yigitbasi 63470afc99 [SPARK-15782][YARN] Fix spark.jars and spark.yarn.dist.jars handling
When `--packages` is specified with spark-shell the classes from those packages cannot be found, which I think is due to some of the changes in SPARK-12343.

Tested manually with both scala 2.10 and 2.11 repls.

vanzin davies can you guys please review?

Author: Marcelo Vanzin <vanzin@cloudera.com>
Author: Nezih Yigitbasi <nyigitbasi@netflix.com>

Closes #13709 from nezihyigitbasi/SPARK-15782.
2016-06-16 18:20:16 -07:00
Sean Owen 457126e420 [SPARK-15796][CORE] Reduce spark.memory.fraction default to avoid overrunning old gen in JVM default config
## What changes were proposed in this pull request?

Reduce `spark.memory.fraction` default to 0.6 in order to make it fit within default JVM old generation size (2/3 heap). See JIRA discussion. This means a full cache doesn't spill into the new gen. CC andrewor14

## How was this patch tested?

Jenkins tests.

Author: Sean Owen <sowen@cloudera.com>

Closes #13618 from srowen/SPARK-15796.
2016-06-16 23:04:10 +02:00
Davies Liu a153e41c08 Revert "[SPARK-15782][YARN] Set spark.jars system property in client mode"
This reverts commit 4df8df5c2e.
2016-06-15 15:55:07 -07:00
Imran Rashid cafc696d09 [HOTFIX][CORE] fix flaky BasicSchedulerIntegrationTest
## What changes were proposed in this pull request?

SPARK-15927 exacerbated a race in BasicSchedulerIntegrationTest, so it went from very unlikely to fairly frequent.  The issue is that stage numbering is not completely deterministic, but these tests treated it like it was.  So turn off the tests.

## How was this patch tested?

on my laptop the test failed abotu 10% of the time before this change, and didn't fail in 500 runs after the change.

Author: Imran Rashid <irashid@cloudera.com>

Closes #13688 from squito/hotfix_basic_scheduler.
2016-06-15 16:44:18 -05:00
Nezih Yigitbasi 4df8df5c2e [SPARK-15782][YARN] Set spark.jars system property in client mode
## What changes were proposed in this pull request?

When `--packages` is specified with `spark-shell` the classes from those packages cannot be found, which I think is due to some of the changes in `SPARK-12343`. In particular `SPARK-12343` removes a line that sets the `spark.jars` system property in client mode, which is used by the repl main class to set the classpath.

## How was this patch tested?

Tested manually.

This system property is used by the repl to populate its classpath. If
this is not set properly the classes for external packages cannot be
found.

tgravescs vanzin as you may be familiar with this part of the code.

Author: Nezih Yigitbasi <nyigitbasi@netflix.com>

Closes #13527 from nezihyigitbasi/repl-fix.
2016-06-15 14:07:36 -07:00
Tejas Patil 279bd4aa5f [SPARK-15826][CORE] PipedRDD to allow configurable char encoding
## What changes were proposed in this pull request?

Link to jira which describes the problem: https://issues.apache.org/jira/browse/SPARK-15826

The fix in this PR is to allow users specify encoding in the pipe() operation. For backward compatibility,
keeping the default value to be system default.

## How was this patch tested?

Ran existing unit tests

Author: Tejas Patil <tejasp@fb.com>

Closes #13563 from tejasapatil/pipedrdd_utf8.
2016-06-15 12:03:00 -07:00
Liwei Lin 9b234b55d1 [SPARK-15518][CORE][FOLLOW-UP] Rename LocalSchedulerBackendEndpoint -> LocalSchedulerBackend
## What changes were proposed in this pull request?

This patch is a follow-up to https://github.com/apache/spark/pull/13288 completing the renaming:
 - LocalScheduler -> LocalSchedulerBackend~~Endpoint~~

## How was this patch tested?

Updated test cases to reflect the name change.

Author: Liwei Lin <lwlin7@gmail.com>

Closes #13683 from lw-lin/rename-backend.
2016-06-15 11:52:36 -07:00
Kay Ousterhout 5d50d4f0f9 [SPARK-15927] Eliminate redundant DAGScheduler code.
To try to eliminate redundant code to traverse the RDD dependency graph,
this PR creates a new function getShuffleDependencies that returns
shuffle dependencies that are immediate parents of a given RDD.  This
new function is used by getParentStages and
getAncestorShuffleDependencies.

Author: Kay Ousterhout <kayousterhout@gmail.com>

Closes #13646 from kayousterhout/SPARK-15927.
2016-06-14 17:27:01 -07:00
Sean Owen 6151d2641f [MINOR] Clean up several build warnings, mostly due to internal use of old accumulators
## What changes were proposed in this pull request?

Another PR to clean up recent build warnings. This particularly cleans up several instances of the old accumulator API usage in tests that are straightforward to update. I think this qualifies as "minor".

## How was this patch tested?

Jenkins

Author: Sean Owen <sowen@cloudera.com>

Closes #13642 from srowen/BuildWarnings.
2016-06-14 09:40:07 -07:00
Sean Owen 0a6f090837 [SPARK-15876][CORE] Remove support for "zk://" master URL
## What changes were proposed in this pull request?

Remove deprecated support for `zk://` master (`mesos://zk//` remains supported)

## How was this patch tested?

Jenkins

Author: Sean Owen <sowen@cloudera.com>

Closes #13625 from srowen/SPARK-15876.
2016-06-12 11:46:33 -07:00
Sean Owen f51dfe616b [SPARK-15086][CORE][STREAMING] Deprecate old Java accumulator API
## What changes were proposed in this pull request?

- Deprecate old Java accumulator API; should use Scala now
- Update Java tests and examples
- Don't bother testing old accumulator API in Java 8 (too)
- (fix a misspelling too)

## How was this patch tested?

Jenkins tests

Author: Sean Owen <sowen@cloudera.com>

Closes #13606 from srowen/SPARK-15086.
2016-06-12 11:44:33 -07:00
Imran Rashid 8cc22b0085 [SPARK-15878][CORE][TEST] fix cleanup in EventLoggingListenerSuite and ReplayListenerSuite
## What changes were proposed in this pull request?

These tests weren't properly using `LocalSparkContext` so weren't cleaning up correctly when tests failed.

## How was this patch tested?

Jenkins.

Author: Imran Rashid <irashid@cloudera.com>

Closes #13602 from squito/SPARK-15878_cleanup_replaylistener.
2016-06-12 12:54:57 +01:00
Eric Liang e1f986c7a3 [SPARK-15860] Metrics for codegen size and perf
## What changes were proposed in this pull request?

Adds codahale metrics for the codegen source text size and how long it takes to compile. The size is particularly interesting, since the JVM does have hard limits on how large methods can get.

To simplify, I added the metrics under a statically-initialized source that is always registered with SparkEnv.

## How was this patch tested?

Unit tests

Author: Eric Liang <ekl@databricks.com>

Closes #13586 from ericl/spark-15860.
2016-06-11 23:16:21 -07:00
Eric Liang c06c58bbbb [SPARK-14851][CORE] Support radix sort with nullable longs
## What changes were proposed in this pull request?

This adds support for radix sort of nullable long fields. When a sort field is null and radix sort is enabled, we keep nulls in a separate region of the sort buffer so that radix sort does not need to deal with them. This also has performance benefits when sorting smaller integer types, since the current representation of nulls in two's complement (Long.MIN_VALUE) otherwise forces a full-width radix sort.

This strategy for nulls does mean the sort is no longer stable. cc davies

## How was this patch tested?

Existing randomized sort tests for correctness. I also tested some TPCDS queries and there does not seem to be any significant regression for non-null sorts.

Some test queries (best of 5 runs each).
Before change:
scala> val start = System.nanoTime; spark.range(5000000).selectExpr("if(id > 5, cast(hash(id) as long), NULL) as h").coalesce(1).orderBy("h").collect(); (System.nanoTime - start) / 1e6
start: Long = 3190437233227987
res3: Double = 4716.471091

After change:
scala> val start = System.nanoTime; spark.range(5000000).selectExpr("if(id > 5, cast(hash(id) as long), NULL) as h").coalesce(1).orderBy("h").collect(); (System.nanoTime - start) / 1e6
start: Long = 3190367870952791
res4: Double = 2981.143045

Author: Eric Liang <ekl@databricks.com>

Closes #13161 from ericl/sc-2998.
2016-06-11 15:42:58 -07:00
Reynold Xin 254bc8c34e [SPARK-15866] Rename listAccumulator collectionAccumulator
## What changes were proposed in this pull request?
SparkContext.listAccumulator, by Spark's convention, makes it sound like "list" is a verb and the method should return a list of accumulators. This patch renames the method and the class collection accumulator.

## How was this patch tested?
Updated test case to reflect the names.

Author: Reynold Xin <rxin@databricks.com>

Closes #13594 from rxin/SPARK-15866.
2016-06-10 11:08:39 -07:00
Eric Liang b914e1930f [SPARK-15794] Should truncate toString() of very wide plans
## What changes were proposed in this pull request?

With very wide tables, e.g. thousands of fields, the plan output is unreadable and often causes OOMs due to inefficient string processing. This truncates all struct and operator field lists to a user configurable threshold to limit performance impact.

It would also be nice to optimize string generation to avoid these sort of O(n^2) slowdowns entirely (i.e. use StringBuilder everywhere including expressions), but this is probably too large of a change for 2.0 at this point, and truncation has other benefits for usability.

## How was this patch tested?

Added a microbenchmark that covers this case particularly well. I also ran the microbenchmark while varying the truncation threshold.

```
numFields = 5
wide shallowly nested struct field r/w:  Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
2000 wide x 50 rows (write in-mem)            2336 / 2558          0.0       23364.4       0.1X

numFields = 25
wide shallowly nested struct field r/w:  Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
2000 wide x 50 rows (write in-mem)            4237 / 4465          0.0       42367.9       0.1X

numFields = 100
wide shallowly nested struct field r/w:  Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
2000 wide x 50 rows (write in-mem)          10458 / 11223          0.0      104582.0       0.0X

numFields = Infinity
wide shallowly nested struct field r/w:  Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
[info]   java.lang.OutOfMemoryError: Java heap space
```

Author: Eric Liang <ekl@databricks.com>
Author: Eric Liang <ekhliang@gmail.com>

Closes #13537 from ericl/truncated-string.
2016-06-09 18:05:16 -07:00
Imran Rashid 36d3dfa59a [SPARK-15783][CORE] still some flakiness in these blacklist tests so ignore for now
## What changes were proposed in this pull request?

There is still some flakiness in BlacklistIntegrationSuite, so turning it off for the moment to avoid breaking more builds -- will turn it back with more fixes.

## How was this patch tested?

jenkins.

Author: Imran Rashid <irashid@cloudera.com>

Closes #13528 from squito/ignore_blacklist.
2016-06-06 12:53:11 -07:00
Zheng RuiFeng fd8af39713 [MINOR] Fix Typos 'an -> a'
## What changes were proposed in this pull request?

`an -> a`

Use cmds like `find . -name '*.R' | xargs -i sh -c "grep -in ' an [^aeiou]' {} && echo {}"` to generate candidates, and review them one by one.

## How was this patch tested?
manual tests

Author: Zheng RuiFeng <ruifengz@foxmail.com>

Closes #13515 from zhengruifeng/an_a.
2016-06-06 09:35:47 +01:00
Brett Randall 4e767d0f90 [SPARK-15723] Fixed local-timezone-brittle test where short-timezone form "EST" is …
## What changes were proposed in this pull request?

Stop using the abbreviated and ambiguous timezone "EST" in a test, since it is machine-local default timezone dependent, and fails in different timezones.  Fixed [SPARK-15723](https://issues.apache.org/jira/browse/SPARK-15723).

## How was this patch tested?

Note that to reproduce this problem in any locale/timezone, you can modify the scalatest-maven-plugin argLine to add a timezone:

    <argLine>-ea -Xmx3g -XX:MaxPermSize=${MaxPermGen} -XX:ReservedCodeCacheSize=${CodeCacheSize} -Duser.timezone="Australia/Sydney"</argLine>

and run

    $ mvn test -DwildcardSuites=org.apache.spark.status.api.v1.SimpleDateParamSuite -Dtest=none. Equally this will fix it in an effected timezone:

    <argLine>-ea -Xmx3g -XX:MaxPermSize=${MaxPermGen} -XX:ReservedCodeCacheSize=${CodeCacheSize} -Duser.timezone="America/New_York"</argLine>

To test the fix, apply the above change to `pom.xml` to set test TZ to `Australia/Sydney`, and confirm the test now passes.

Author: Brett Randall <javabrett@gmail.com>

Closes #13462 from javabrett/SPARK-15723-SimpleDateParamSuite.
2016-06-05 15:31:56 +01:00
Davies Liu 3074f575a3 [SPARK-15391] [SQL] manage the temporary memory of timsort
## What changes were proposed in this pull request?

Currently, the memory for temporary buffer used by TimSort is always allocated as on-heap without bookkeeping, it could cause OOM both in on-heap and off-heap mode.

This PR will try to manage that by preallocate it together with the pointer array, same with RadixSort. It both works for on-heap and off-heap mode.

This PR also change the loadFactor of BytesToBytesMap to 0.5 (it was 0.70), it enables use to radix sort also makes sure that we have enough memory for timsort.

## How was this patch tested?

Existing tests.

Author: Davies Liu <davies@databricks.com>

Closes #13318 from davies/fix_timsort.
2016-06-03 16:45:09 -07:00
Xin Wu 28ad0f7b0d [SPARK-15681][CORE] allow lowercase or mixed case log level string when calling sc.setLogLevel
## What changes were proposed in this pull request?
Currently `SparkContext API setLogLevel(level: String) `can not handle lower case or mixed case input string. But `org.apache.log4j.Level.toLevel` can take lowercase or mixed case.

This PR is to allow case-insensitive user input for the log level.

## How was this patch tested?
A unit testcase is added.

Author: Xin Wu <xinwu@us.ibm.com>

Closes #13422 from xwu0226/reset_loglevel.
2016-06-03 14:26:48 -07:00
Imran Rashid c2f0cb4f63 [SPARK-15714][CORE] Fix flaky o.a.s.scheduler.BlacklistIntegrationSuite
## What changes were proposed in this pull request?

BlacklistIntegrationSuite (introduced by SPARK-10372) is a bit flaky because of some race conditions:
1. Failed jobs might have non-empty results, because the resultHandler will be invoked for successful tasks (if there are task successes before failures)
2. taskScheduler.taskIdToTaskSetManager must be protected by a lock on taskScheduler

(1) has failed a handful of jenkins builds recently.  I don't think I've seen (2) in jenkins, but I've run into with some uncommitted tests I'm working on where there are lots more tasks.

While I was in there, I also made an unrelated fix to `runningTasks`in the test framework -- there was a pointless `O(n)` operation to remove completed tasks, could be `O(1)`.

## How was this patch tested?

I modified the o.a.s.scheduler.BlacklistIntegrationSuite to have it run the tests 1k times on my laptop.  It failed 11 times before this change, and none with it.  (Pretty sure all the failures were problem (1), though I didn't check all of them).

Also the full suite of tests via jenkins.

Author: Imran Rashid <irashid@cloudera.com>

Closes #13454 from squito/SPARK-15714.
2016-06-03 11:49:33 -05:00
Josh Rosen 229f902257 [SPARK-15736][CORE] Gracefully handle loss of DiskStore files
If an RDD partition is cached on disk and the DiskStore file is lost, then reads of that cached partition will fail and the missing partition is supposed to be recomputed by a new task attempt. In the current BlockManager implementation, however, the missing file does not trigger any metadata updates / does not invalidate the cache, so subsequent task attempts will be scheduled on the same executor and the doomed read will be repeatedly retried, leading to repeated task failures and eventually a total job failure.

In order to fix this problem, the executor with the missing file needs to properly mark the corresponding block as missing so that it stops advertising itself as a cache location for that block.

This patch fixes this bug and adds an end-to-end regression test (in `FailureSuite`) and a set of unit tests (`in BlockManagerSuite`).

Author: Josh Rosen <joshrosen@databricks.com>

Closes #13473 from JoshRosen/handle-missing-cache-files.
2016-06-02 17:36:31 -07:00
hyukjinkwon 252417fa21 [SPARK-15322][SQL][FOLLOWUP] Use the new long accumulator for old int accumulators.
## What changes were proposed in this pull request?

This PR corrects the remaining cases for using old accumulators.

This does not change some old accumulator usages below:

- `ImplicitSuite.scala` - Tests dedicated to old accumulator, for implicits with `AccumulatorParam`

- `AccumulatorSuite.scala` -  Tests dedicated to old accumulator

- `JavaSparkContext.scala` - For supporting old accumulators for Java API.

- `debug.package.scala` - Usage with `HashSet[String]`. Currently, it seems no implementation for this. I might be able to write an anonymous class for this but I didn't because I think it is not worth writing a lot of codes only for this.

- `SQLMetricsSuite.scala` - This uses the old accumulator for checking type boxing. It seems new accumulator does not require type boxing for this case whereas the old one requires (due to the use of generic).

## How was this patch tested?

Existing tests cover this.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #13434 from HyukjinKwon/accum.
2016-06-02 11:16:24 -05:00
Tejas Patil ac38bdc756 [SPARK-15601][CORE] CircularBuffer's toString() to print only the contents written if buffer isn't full
## What changes were proposed in this pull request?

1. The class allocated 4x space than needed as it was using `Int` to store the `Byte` values

2. If CircularBuffer isn't full, currently toString() will print some garbage chars along with the content written as is tries to print the entire array allocated for the buffer. The fix is to keep track of buffer getting full and don't print the tail of the buffer if it isn't full (suggestion by sameeragarwal over https://github.com/apache/spark/pull/12194#discussion_r64495331)

3. Simplified `toString()`

## How was this patch tested?

Added new test case

Author: Tejas Patil <tejasp@fb.com>

Closes #13351 from tejasapatil/circular_buffer.
2016-05-31 19:52:22 -05:00
Devaraj K 5b21139dbf [SPARK-10530][CORE] Kill other task attempts when one taskattempt belonging the same task is succeeded in speculation
## What changes were proposed in this pull request?

With this patch, TaskSetManager kills other running attempts when any one of the attempt succeeds for the same task. Also killed tasks will not be considered as failed tasks and they get listed separately in the UI and also shows the task state as KILLED instead of FAILED.

## How was this patch tested?

core\src\test\scala\org\apache\spark\ui\jobs\JobProgressListenerSuite.scala
core\src\test\scala\org\apache\spark\util\JsonProtocolSuite.scala

I have verified this patch manually by enabling spark.speculation as true, when any attempt gets succeeded then other running attempts are getting killed for the same task and other pending tasks are getting assigned in those. And also when any attempt gets killed then they are considered as KILLED tasks and not considered as FAILED tasks. Please find the attached screen shots for the reference.

![stage-tasks-table](https://cloud.githubusercontent.com/assets/3174804/14075132/394c6a12-f4f4-11e5-8638-20ff7b8cc9bc.png)
![stages-table](https://cloud.githubusercontent.com/assets/3174804/14075134/3b60f412-f4f4-11e5-9ea6-dd0dcc86eb03.png)

Ref : https://github.com/apache/spark/pull/11916

Author: Devaraj K <devaraj@apache.org>

Closes #11996 from devaraj-kavali/SPARK-10530.
2016-05-30 14:29:27 -07:00
Reynold Xin 73178c7556 [SPARK-15633][MINOR] Make package name for Java tests consistent
## What changes were proposed in this pull request?
This is a simple patch that makes package names for Java 8 test suites consistent. I moved everything to test.org.apache.spark to we can test package private APIs properly. Also added "java8" as the package name so we can easily run all the tests related to Java 8.

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

Author: Reynold Xin <rxin@databricks.com>

Closes #13364 from rxin/SPARK-15633.
2016-05-27 21:20:02 -07:00
dding3 88c9c467a3 [SPARK-15562][ML] Delete temp directory after program exit in DataFrameExample
## What changes were proposed in this pull request?
Temp directory used to save records is not deleted after program exit in DataFrameExample. Although it called deleteOnExit, it doesn't work as the directory is not empty. Similar things happend in ContextCleanerSuite. Update the code to make sure temp directory is deleted after program exit.

## How was this patch tested?

unit tests and local build.

Author: dding3 <ding.ding@intel.com>

Closes #13328 from dding3/master.
2016-05-27 21:01:50 -05:00
Sital Kedia ce756daa4f [SPARK-15569] Reduce frequency of updateBytesWritten function in Disk…
## What changes were proposed in this pull request?

Profiling a Spark job spilling large amount of intermediate data we found that significant portion of time is being spent in DiskObjectWriter.updateBytesWritten function. Looking at the code, we see that the function is being called too frequently to update the number of bytes written to disk. We should reduce the frequency to avoid this.

## How was this patch tested?

Tested by running the job on cluster and saw 20% CPU gain  by this change.

Author: Sital Kedia <skedia@fb.com>

Closes #13332 from sitalkedia/DiskObjectWriter.
2016-05-27 11:22:39 -07:00
Sameer Agarwal fe6de16f78 [SPARK-8428][SPARK-13850] Fix integer overflows in TimSort
## What changes were proposed in this pull request?

This patch fixes a few integer overflows in `UnsafeSortDataFormat.copyRange()` and `ShuffleSortDataFormat copyRange()` that seems to be the most likely cause behind a number of `TimSort` contract violation errors seen in Spark 2.0 and Spark 1.6 while sorting large datasets.

## How was this patch tested?

Added a test in `ExternalSorterSuite` that instantiates a large array of the form of [150000000, 150000001, 150000002, ...., 300000000, 0, 1, 2, ..., 149999999] that triggers a `copyRange` in `TimSort.mergeLo` or `TimSort.mergeHi`. Note that the input dataset should contain at least 268.43 million rows with a certain data distribution for an overflow to occur.

Author: Sameer Agarwal <sameer@databricks.com>

Closes #13336 from sameeragarwal/timsort-bug.
2016-05-26 15:49:16 -07:00
Imran Rashid dfc9fc02cc [SPARK-10372] [CORE] basic test framework for entire spark scheduler
This is a basic framework for testing the entire scheduler.  The tests this adds aren't very interesting -- the point of this PR is just to setup the framework, to keep the initial change small, but it can be built upon to test more features (eg., speculation, killing tasks, blacklisting, etc.).

Author: Imran Rashid <irashid@cloudera.com>

Closes #8559 from squito/SPARK-10372-scheduler-integs.
2016-05-26 00:29:09 -05:00
Takuya UESHIN 698ef762f8 [SPARK-14269][SCHEDULER] Eliminate unnecessary submitStage() call.
## What changes were proposed in this pull request?

Currently a method `submitStage()` for waiting stages is called on every iteration of the event loop in `DAGScheduler` to submit all waiting stages, but most of them are not necessary because they are not related to Stage status.
The case we should try to submit waiting stages is only when their parent stages are successfully completed.

This elimination can improve `DAGScheduler` performance.

## How was this patch tested?

Added some checks and other existing tests, and our projects.

We have a project bottle-necked by `DAGScheduler`, having about 2000 stages.

Before this patch the almost all execution time in `Driver` process was spent to process `submitStage()` of `dag-scheduler-event-loop` thread but after this patch the performance was improved as follows:

|        | total execution time | `dag-scheduler-event-loop` thread time | `submitStage()` |
|--------|---------------------:|---------------------------------------:|----------------:|
| Before |              760 sec |                                710 sec |         667 sec |
| After  |              440 sec |                                 14 sec |          10 sec |

Author: Takuya UESHIN <ueshin@happy-camper.st>

Closes #12060 from ueshin/issues/SPARK-14269.
2016-05-25 13:57:25 -07:00
Lukasz b120fba6ae [SPARK-9044] Fix "Storage" tab in UI so that it reflects RDD name change.
## What changes were proposed in this pull request?

1. Making 'name' field of RDDInfo mutable.
2. In StorageListener: catching the fact that RDD's name was changed and updating it in RDDInfo.

## How was this patch tested?

1. Manual verification - the 'Storage' tab now behaves as expected.
2. The commit also contains a new unit test which verifies this.

Author: Lukasz <lgieron@gmail.com>

Closes #13264 from lgieron/SPARK-9044.
2016-05-25 10:24:21 -07:00
Reynold Xin 14494da87b [SPARK-15518] Rename various scheduler backend for consistency
## What changes were proposed in this pull request?
This patch renames various scheduler backends to make them consistent:

- LocalScheduler -> LocalSchedulerBackend
- AppClient -> StandaloneAppClient
- AppClientListener -> StandaloneAppClientListener
- SparkDeploySchedulerBackend -> StandaloneSchedulerBackend
- CoarseMesosSchedulerBackend -> MesosCoarseGrainedSchedulerBackend
- MesosSchedulerBackend -> MesosFineGrainedSchedulerBackend

## How was this patch tested?
Updated test cases to reflect the name change.

Author: Reynold Xin <rxin@databricks.com>

Closes #13288 from rxin/SPARK-15518.
2016-05-24 20:55:47 -07:00
Dongjoon Hyun f08bf587b1 [SPARK-15512][CORE] repartition(0) should raise IllegalArgumentException
## What changes were proposed in this pull request?

Previously, SPARK-8893 added the constraints on positive number of partitions for repartition/coalesce operations in general. This PR adds one missing part for that and adds explicit two testcases.

**Before**
```scala
scala> sc.parallelize(1 to 5).coalesce(0)
java.lang.IllegalArgumentException: requirement failed: Number of partitions (0) must be positive.
...
scala> sc.parallelize(1 to 5).repartition(0).collect()
res1: Array[Int] = Array()   // empty
scala> spark.sql("select 1").coalesce(0)
res2: org.apache.spark.sql.Dataset[org.apache.spark.sql.Row] = [1: int]
scala> spark.sql("select 1").coalesce(0).collect()
java.lang.IllegalArgumentException: requirement failed: Number of partitions (0) must be positive.
scala> spark.sql("select 1").repartition(0)
res3: org.apache.spark.sql.Dataset[org.apache.spark.sql.Row] = [1: int]
scala> spark.sql("select 1").repartition(0).collect()
res4: Array[org.apache.spark.sql.Row] = Array()  // empty
```

**After**
```scala
scala> sc.parallelize(1 to 5).coalesce(0)
java.lang.IllegalArgumentException: requirement failed: Number of partitions (0) must be positive.
...
scala> sc.parallelize(1 to 5).repartition(0)
java.lang.IllegalArgumentException: requirement failed: Number of partitions (0) must be positive.
...
scala> spark.sql("select 1").coalesce(0)
java.lang.IllegalArgumentException: requirement failed: Number of partitions (0) must be positive.
...
scala> spark.sql("select 1").repartition(0)
java.lang.IllegalArgumentException: requirement failed: Number of partitions (0) must be positive.
...
```

## How was this patch tested?

Pass the Jenkins tests with new testcases.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #13282 from dongjoon-hyun/SPARK-15512.
2016-05-24 18:55:23 -07:00
Dongjoon Hyun be99a99fe7 [MINOR][CORE][TEST] Update obsolete takeSample test case.
## What changes were proposed in this pull request?

This PR fixes some obsolete comments and assertion in `takeSample` testcase of `RDDSuite.scala`.

## How was this patch tested?

This fixes the testcase only.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #13260 from dongjoon-hyun/SPARK-15481.
2016-05-24 11:09:54 -07:00
Xin Wu 01659bc50c [SPARK-15431][SQL] Support LIST FILE(s)|JAR(s) command natively
## What changes were proposed in this pull request?
Currently command `ADD FILE|JAR <filepath | jarpath>` is supported natively in SparkSQL. However, when this command is run, the file/jar is added to the resources that can not be looked up by `LIST FILE(s)|JAR(s)` command because the `LIST` command is passed to Hive command processor in Spark-SQL or simply not supported in Spark-shell. There is no way users can find out what files/jars are added to the spark context.
Refer to [Hive commands](https://cwiki.apache.org/confluence/display/Hive/LanguageManual+Cli)

This PR is to support following commands:
`LIST (FILE[s] [filepath ...] | JAR[s] [jarfile ...])`

### For example:
##### LIST FILE(s)
```
scala> spark.sql("add file hdfs://bdavm009.svl.ibm.com:8020/tmp/test.txt")
res1: org.apache.spark.sql.DataFrame = []
scala> spark.sql("add file hdfs://bdavm009.svl.ibm.com:8020/tmp/test1.txt")
res2: org.apache.spark.sql.DataFrame = []

scala> spark.sql("list file hdfs://bdavm009.svl.ibm.com:8020/tmp/test1.txt").show(false)
+----------------------------------------------+
|result                                        |
+----------------------------------------------+
|hdfs://bdavm009.svl.ibm.com:8020/tmp/test1.txt|
+----------------------------------------------+

scala> spark.sql("list files").show(false)
+----------------------------------------------+
|result                                        |
+----------------------------------------------+
|hdfs://bdavm009.svl.ibm.com:8020/tmp/test1.txt|
|hdfs://bdavm009.svl.ibm.com:8020/tmp/test.txt |
+----------------------------------------------+
```

##### LIST JAR(s)
```
scala> spark.sql("add jar /Users/xinwu/spark/core/src/test/resources/TestUDTF.jar")
res9: org.apache.spark.sql.DataFrame = [result: int]

scala> spark.sql("list jar TestUDTF.jar").show(false)
+---------------------------------------------+
|result                                       |
+---------------------------------------------+
|spark://192.168.1.234:50131/jars/TestUDTF.jar|
+---------------------------------------------+

scala> spark.sql("list jars").show(false)
+---------------------------------------------+
|result                                       |
+---------------------------------------------+
|spark://192.168.1.234:50131/jars/TestUDTF.jar|
+---------------------------------------------+
```
## How was this patch tested?
New test cases are added for Spark-SQL, Spark-Shell and SparkContext API code path.

Author: Xin Wu <xinwu@us.ibm.com>
Author: xin Wu <xinwu@us.ibm.com>

Closes #13212 from xwu0226/list_command.
2016-05-23 17:32:01 -07:00
Shixiong Zhu 305263954a Fix the compiler error introduced by #13153 for Scala 2.10 2016-05-19 12:36:44 -07:00
Shixiong Zhu 4e3cb7a5d9 [SPARK-15317][CORE] Don't store accumulators for every task in listeners
## What changes were proposed in this pull request?

In general, the Web UI doesn't need to store the Accumulator/AccumulableInfo for every task. It only needs the Accumulator values.

In this PR, it creates new UIData classes to store the necessary fields and make `JobProgressListener` store only these new classes, so that `JobProgressListener` won't store Accumulator/AccumulableInfo and the size of `JobProgressListener` becomes pretty small. I also eliminates `AccumulableInfo` from `SQLListener` so that we don't keep any references for those unused `AccumulableInfo`s.

## How was this patch tested?

I ran two tests reported in JIRA locally:

The first one is:
```
val data = spark.range(0, 10000, 1, 10000)
data.cache().count()
```
The retained size of JobProgressListener decreases from 60.7M to 6.9M.

The second one is:
```
import org.apache.spark.ml.CC
import org.apache.spark.sql.SQLContext
val sqlContext = SQLContext.getOrCreate(sc)
CC.runTest(sqlContext)
```

This test won't cause OOM after applying this patch.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #13153 from zsxwing/memory.
2016-05-19 12:05:17 -07:00