Commit graph

2419 commits

Author SHA1 Message Date
Edwina Lu 9241e1e7e6 [SPARK-23429][CORE] Add executor memory metrics to heartbeat and expose in executors REST API
Add new executor level memory metrics (JVM used memory, on/off heap execution memory, on/off heap storage memory, on/off heap unified memory, direct memory, and mapped memory), and expose via the executors REST API. This information will help provide insight into how executor and driver JVM memory is used, and for the different memory regions. It can be used to help determine good values for spark.executor.memory, spark.driver.memory, spark.memory.fraction, and spark.memory.storageFraction.

## What changes were proposed in this pull request?

An ExecutorMetrics class is added, with jvmUsedHeapMemory, jvmUsedNonHeapMemory, onHeapExecutionMemory, offHeapExecutionMemory, onHeapStorageMemory, and offHeapStorageMemory, onHeapUnifiedMemory, offHeapUnifiedMemory, directMemory and mappedMemory. The new ExecutorMetrics is sent by executors to the driver as part of the Heartbeat. A heartbeat is added for the driver as well, to collect these metrics for the driver.

The EventLoggingListener store information about the peak values for each metric, per active stage and executor. When a StageCompleted event is seen, a StageExecutorsMetrics event will be logged for each executor, with peak values for the stage.

The AppStatusListener records the peak values for each memory metric.

The new memory metrics are added to the executors REST API.

## How was this patch tested?

New unit tests have been added. This was also tested on our cluster.

Author: Edwina Lu <edlu@linkedin.com>
Author: Imran Rashid <irashid@cloudera.com>
Author: edwinalu <edwina.lu@gmail.com>

Closes #21221 from edwinalu/SPARK-23429.2.
2018-09-07 10:42:46 -07:00
Sean Owen 4e3365b577 [SPARK-22357][CORE][FOLLOWUP] SparkContext.binaryFiles ignore minPartitions parameter
## What changes were proposed in this pull request?

This adds a test following https://github.com/apache/spark/pull/21638

## How was this patch tested?

Existing tests and new test.

Closes #22356 from srowen/SPARK-22357.2.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-09-06 21:43:14 -07:00
xuejianbest f5817d8bb3 [SPARK-25108][SQL] Fix the show method to display the wide character alignment problem
This is not a perfect solution. It is designed to minimize complexity on the basis of solving problems.

It is effective for English, Chinese characters, Japanese, Korean and so on.

```scala
before:
+---+---------------------------+-------------+
|id |中国                         |s2           |
+---+---------------------------+-------------+
|1  |ab                         |[a]          |
|2  |null                       |[中国, abc]    |
|3  |ab1                        |[hello world]|
|4  |か行 きゃ(kya) きゅ(kyu) きょ(kyo) |[“中国]        |
|5  |中国(你好)a                    |[“中(国), 312] |
|6  |中国山(东)服务区                  |[“中(国)]      |
|7  |中国山东服务区                    |[中(国)]       |
|8  |                           |[中国]         |
+---+---------------------------+-------------+

after:
+---+-----------------------------------+----------------+
|id |中国                               |s2              |
+---+-----------------------------------+----------------+
|1  |ab                                 |[a]             |
|2  |null                               |[中国, abc]     |
|3  |ab1                                |[hello world]   |
|4  |か行 きゃ(kya) きゅ(kyu) きょ(kyo) |[“中国]         |
|5  |中国(你好)a                      |[“中(国), 312]|
|6  |中国山(东)服务区                   |[“中(国)]      |
|7  |中国山东服务区                     |[中(国)]        |
|8  |                                   |[中国]          |
+---+-----------------------------------+----------------+
```

## What changes were proposed in this pull request?

When there are wide characters such as Chinese characters or Japanese characters in the data, the show method has a alignment problem.
Try to fix this problem.

## How was this patch tested?

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

![image](https://user-images.githubusercontent.com/13044869/44250564-69f6b400-a227-11e8-88b2-6cf6960377ff.png)

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

Closes #22048 from xuejianbest/master.

Authored-by: xuejianbest <384329882@qq.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-09-06 07:17:37 -07:00
Yuming Wang 3e033035a3 [SPARK-25258][SPARK-23131][SPARK-25176][BUILD] Upgrade Kryo to 4.0.2
## What changes were proposed in this pull request?

Upgrade chill to 0.9.3, Kryo to 4.0.2, to get bug fixes and improvements.

The resolved tickets includes:
- SPARK-25258 Upgrade kryo package to version 4.0.2
- SPARK-23131 Kryo raises StackOverflow during serializing GLR model
- SPARK-25176 Kryo fails to serialize a parametrised type hierarchy

More details:
https://github.com/twitter/chill/releases/tag/v0.9.3
cc3910d501

## How was this patch tested?

Existing tests.

Closes #22179 from wangyum/SPARK-23131.

Lead-authored-by: Yuming Wang <yumwang@ebay.com>
Co-authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-09-05 15:48:41 -07:00
Wenchen Fan 71bd796517 [SPARK-23243][CORE] Fix RDD.repartition() data correctness issue
## What changes were proposed in this pull request?

An alternative fix for https://github.com/apache/spark/pull/21698

When Spark rerun tasks for an RDD, there are 3 different behaviors:
1. determinate. Always return the same result with same order when rerun.
2. unordered. Returns same data set in random order when rerun.
3. indeterminate. Returns different result when rerun.

Normally Spark doesn't need to care about it. Spark runs stages one by one, when a task is failed, just rerun it. Although the rerun task may return a different result, users will not be surprised.

However, Spark may rerun a finished stage when seeing fetch failures. When this happens, Spark needs to rerun all the tasks of all the succeeding stages if the RDD output is indeterminate, because the input of the succeeding stages has been changed.

If the RDD output is determinate, we only need to rerun the failed tasks of the succeeding stages, because the input doesn't change.

If the RDD output is unordered, it's same as determinate, because shuffle partitioner is always deterministic(round-robin partitioner is not a shuffle partitioner that extends `org.apache.spark.Partitioner`), so the reducers will still get the same input data set.

This PR fixed the failure handling for `repartition`, to avoid correctness issues.

For `repartition`, it applies a stateful map function to generate a round-robin id, which is order sensitive and makes the RDD's output indeterminate. When the stage contains `repartition` reruns, we must also rerun all the tasks of all the succeeding stages.

**future improvement:**
1. Currently we can't rollback and rerun a shuffle map stage, and just fail. We should fix it later. https://issues.apache.org/jira/browse/SPARK-25341
2. Currently we can't rollback and rerun a result stage, and just fail. We should fix it later. https://issues.apache.org/jira/browse/SPARK-25342
3. We should provide public API to allow users to tag the random level of the RDD's computing function.

## How is this pull request tested?
a new test case

Closes #22112 from cloud-fan/repartition.

Lead-authored-by: Wenchen Fan <wenchen@databricks.com>
Co-authored-by: Xingbo Jiang <xingbo.jiang@databricks.com>
Signed-off-by: Xiao Li <gatorsmile@gmail.com>
2018-09-05 15:36:34 -07:00
pgandhi 559b899ace [SPARK-25231] Fix synchronization of executor heartbeat receiver in TaskSchedulerImpl
Running a large Spark job with speculation turned on was causing executor heartbeats to time out on the driver end after sometime and eventually, after hitting the max number of executor failures, the job would fail.

## What changes were proposed in this pull request?

The main reason for the heartbeat timeouts was that the heartbeat-receiver-event-loop-thread was blocked waiting on the TaskSchedulerImpl object which was being held by one of the dispatcher-event-loop threads executing the method dequeueSpeculativeTasks() in TaskSetManager.scala. On further analysis of the heartbeat receiver method executorHeartbeatReceived() in TaskSchedulerImpl class, we found out that instead of waiting to acquire the lock on the TaskSchedulerImpl object, we can remove that lock and make the operations to the global variables inside the code block to be atomic. The block of code in that method only uses  one global HashMap taskIdToTaskSetManager. Making that map a ConcurrentHashMap, we are ensuring atomicity of operations and speeding up the heartbeat receiver thread operation.

## How was this patch tested?

Screenshots of the thread dump have been attached below:
**heartbeat-receiver-event-loop-thread:**

<img width="1409" alt="screen shot 2018-08-24 at 9 19 57 am" src="https://user-images.githubusercontent.com/22228190/44593413-e25df780-a788-11e8-9520-176a18401a59.png">

**dispatcher-event-loop-thread:**

<img width="1409" alt="screen shot 2018-08-24 at 9 21 56 am" src="https://user-images.githubusercontent.com/22228190/44593484-13d6c300-a789-11e8-8d88-34b1d51d4541.png">

Closes #22221 from pgandhi999/SPARK-25231.

Authored-by: pgandhi <pgandhi@oath.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2018-09-05 16:10:49 -05:00
ankurgupta 39a02d8f75 [SPARK-24415][CORE] Fixed the aggregated stage metrics by retaining stage objects in liveStages until all tasks are complete
The problem occurs because stage object is removed from liveStages in
AppStatusListener onStageCompletion. Because of this any onTaskEnd event
received after onStageCompletion event do not update stage metrics.

The fix is to retain stage objects in liveStages until all tasks are complete.

1. Fixed the reproducible example posted in the JIRA
2. Added unit test

Closes #22209 from ankuriitg/ankurgupta/SPARK-24415.

Authored-by: ankurgupta <ankur.gupta@cloudera.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-09-05 09:52:04 -07:00
liuxian ca861fea21 [SPARK-25300][CORE] Unified the configuration parameter spark.shuffle.service.enabled
## What changes were proposed in this pull request?

The configuration parameter "spark.shuffle.service.enabled"  has defined in `package.scala`,  and it  is also used in many place,  so we can replace it with `SHUFFLE_SERVICE_ENABLED`.
and unified  this configuration parameter "spark.shuffle.service.port"  together.

## How was this patch tested?
N/A

Closes #22306 from 10110346/unifiedserviceenable.

Authored-by: liuxian <liu.xian3@zte.com.cn>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-09-05 10:43:46 +08:00
Liang-Chi Hsieh 7c36ee46d9 [SPARK-25290][CORE][TEST] Reduce the size of acquired arrays to avoid OOM error
## What changes were proposed in this pull request?

`BytesToBytesMapOnHeapSuite`.`randomizedStressTest` caused `OutOfMemoryError` on several test runs. Seems better to reduce memory usage in this test.

## How was this patch tested?

Unit tests.

Closes #22297 from viirya/SPARK-25290.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-09-01 16:25:29 +08:00
Thomas Graves ec3e998638 [SPARK-24909][CORE] Always unregister pending partition on task completion.
Spark scheduler can hang when fetch failures, executor lost, task running on lost executor, and multiple stage attempts. To fix this we change to always unregister the pending partition on task completion.

## What changes were proposed in this pull request?
this PR is actually reverting the change in SPARK-19263, so that it always does shuffleStage.pendingPartitions -= task.partitionId.   The change in SPARK-23433, should fix the issue originally from SPARK-19263.

## How was this patch tested?

Unit tests.  The condition happens on a race which I haven't reproduced on a real customer, just see it sometimes on customers jobs in a real cluster.
I am also working on adding spark scheduler integration tests.

Closes #21976 from tgravescs/SPARK-24909.

Authored-by: Thomas Graves <tgraves@unharmedunarmed.corp.ne1.yahoo.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-08-29 16:32:02 -07:00
liuxian 381a967a76 [SPARK-25249][CORE][TEST] add a unit test for OpenHashMap
## What changes were proposed in this pull request?

This PR adds a unit test for OpenHashMap , this can help developers  to distinguish between the 0/0.0/0L and null

## How was this patch tested?

Closes #22241 from 10110346/openhashmap.

Authored-by: liuxian <liu.xian3@zte.com.cn>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-08-27 12:05:33 -05:00
Imran Rashid 99d2e4e007 [SPARK-24296][CORE] Replicate large blocks as a stream.
When replicating large cached RDD blocks, it can be helpful to replicate
them as a stream, to avoid using large amounts of memory during the
transfer.  This also allows blocks larger than 2GB to be replicated.

Added unit tests in DistributedSuite.  Also ran tests on a cluster for
blocks > 2gb.

Closes #21451 from squito/clean_replication.

Authored-by: Imran Rashid <irashid@cloudera.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-08-21 11:26:41 -07:00
Xingbo Jiang 5059255d91 [SPARK-25161][CORE] Fix several bugs in failure handling of barrier execution mode
## What changes were proposed in this pull request?

Fix several bugs in failure handling of barrier execution mode:
* Mark TaskSet for a barrier stage as zombie when a task attempt fails;
* Multiple barrier task failures from a single barrier stage should not trigger multiple stage retries;
* Barrier task failure from a previous failed stage attempt should not trigger stage retry;
* Fail the job when a task from a barrier ResultStage failed;
* RDD.isBarrier() should not rely on `ShuffleDependency`s.

## How was this patch tested?

Added corresponding test cases in `DAGSchedulerSuite` and `TaskSchedulerImplSuite`.

Closes #22158 from jiangxb1987/failure.

Authored-by: Xingbo Jiang <xingbo.jiang@databricks.com>
Signed-off-by: Xiangrui Meng <meng@databricks.com>
2018-08-21 08:25:02 -07:00
Xingbo Jiang bfb74394a5 [SPARK-24819][CORE] Fail fast when no enough slots to launch the barrier stage on job submitted
## What changes were proposed in this pull request?

We shall check whether the barrier stage requires more slots (to be able to launch all tasks in the barrier stage together) than the total number of active slots currently, and fail fast if trying to submit a barrier stage that requires more slots than current total number.

This PR proposes to add a new method `getNumSlots()` to try to get the total number of currently active slots in `SchedulerBackend`, support of this new method has been added to all the first-class scheduler backends except `MesosFineGrainedSchedulerBackend`.

## How was this patch tested?

Added new test cases in `BarrierStageOnSubmittedSuite`.

Closes #22001 from jiangxb1987/SPARK-24819.

Lead-authored-by: Xingbo Jiang <xingbo.jiang@databricks.com>
Co-authored-by: Xiangrui Meng <meng@databricks.com>
Signed-off-by: Xiangrui Meng <meng@databricks.com>
2018-08-15 13:31:28 -07:00
Eyal Farago 2e3abdff23 [SPARK-22713][CORE] ExternalAppendOnlyMap leaks when spilled during iteration
## What changes were proposed in this pull request?
This PR solves [SPARK-22713](https://issues.apache.org/jira/browse/SPARK-22713) which describes a memory leak that occurs when and ExternalAppendOnlyMap is spilled during iteration (opposed to  insertion).

(Please fill in changes proposed in this fix)
ExternalAppendOnlyMap's iterator supports spilling but it kept a reference to the internal map (via an internal iterator) after spilling, it seems that the original code was actually supposed to 'get rid' of this reference on the next iteration but according to the elaborate investigation described in the JIRA this didn't happen.
the fix was simply replacing the internal iterator immediately after spilling.

## How was this patch tested?
I've introduced a new test to test suite ExternalAppendOnlyMapSuite, this test asserts that neither the external map itself nor its iterator hold any reference to the internal map after a spill.
These approach required some access relaxation of some members variables and nested classes of ExternalAppendOnlyMap, this members are now package provate and annotated with VisibleForTesting.

Closes #21369 from eyalfa/SPARK-22713__ExternalAppendOnlyMap_effective_spill.

Authored-by: Eyal Farago <eyal@nrgene.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-08-13 20:55:46 +08:00
Kazuhiro Sera 8ec25cd67e Fix typos detected by github.com/client9/misspell
## What changes were proposed in this pull request?

Fixing typos is sometimes very hard. It's not so easy to visually review them. Recently, I discovered a very useful tool for it, [misspell](https://github.com/client9/misspell).

This pull request fixes minor typos detected by [misspell](https://github.com/client9/misspell) except for the false positives. If you would like me to work on other files as well, let me know.

## How was this patch tested?

### before

```
$ misspell . | grep -v '.js'
R/pkg/R/SQLContext.R:354:43: "definiton" is a misspelling of "definition"
R/pkg/R/SQLContext.R:424:43: "definiton" is a misspelling of "definition"
R/pkg/R/SQLContext.R:445:43: "definiton" is a misspelling of "definition"
R/pkg/R/SQLContext.R:495:43: "definiton" is a misspelling of "definition"
NOTICE-binary:454:16: "containd" is a misspelling of "contained"
R/pkg/R/context.R:46:43: "definiton" is a misspelling of "definition"
R/pkg/R/context.R:74:43: "definiton" is a misspelling of "definition"
R/pkg/R/DataFrame.R:591:48: "persistance" is a misspelling of "persistence"
R/pkg/R/streaming.R:166:44: "occured" is a misspelling of "occurred"
R/pkg/inst/worker/worker.R:65:22: "ouput" is a misspelling of "output"
R/pkg/tests/fulltests/test_utils.R:106:25: "environemnt" is a misspelling of "environment"
common/kvstore/src/test/java/org/apache/spark/util/kvstore/InMemoryStoreSuite.java:38:39: "existant" is a misspelling of "existent"
common/kvstore/src/test/java/org/apache/spark/util/kvstore/LevelDBSuite.java:83:39: "existant" is a misspelling of "existent"
common/network-common/src/main/java/org/apache/spark/network/crypto/TransportCipher.java:243:46: "transfered" is a misspelling of "transferred"
common/network-common/src/main/java/org/apache/spark/network/sasl/SaslEncryption.java:234:19: "transfered" is a misspelling of "transferred"
common/network-common/src/main/java/org/apache/spark/network/sasl/SaslEncryption.java:238:63: "transfered" is a misspelling of "transferred"
common/network-common/src/main/java/org/apache/spark/network/sasl/SaslEncryption.java:244:46: "transfered" is a misspelling of "transferred"
common/network-common/src/main/java/org/apache/spark/network/sasl/SaslEncryption.java:276:39: "transfered" is a misspelling of "transferred"
common/network-common/src/main/java/org/apache/spark/network/util/AbstractFileRegion.java:27:20: "transfered" is a misspelling of "transferred"
common/unsafe/src/test/scala/org/apache/spark/unsafe/types/UTF8StringPropertyCheckSuite.scala:195:15: "orgin" is a misspelling of "origin"
core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala:621:39: "gauranteed" is a misspelling of "guaranteed"
core/src/main/scala/org/apache/spark/status/storeTypes.scala:113:29: "ect" is a misspelling of "etc"
core/src/main/scala/org/apache/spark/storage/DiskStore.scala:282:18: "transfered" is a misspelling of "transferred"
core/src/main/scala/org/apache/spark/util/ListenerBus.scala:64:17: "overriden" is a misspelling of "overridden"
core/src/test/scala/org/apache/spark/ShuffleSuite.scala:211:7: "substracted" is a misspelling of "subtracted"
core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala:1922:49: "agriculteur" is a misspelling of "agriculture"
core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala:2468:84: "truely" is a misspelling of "truly"
core/src/test/scala/org/apache/spark/storage/FlatmapIteratorSuite.scala:25:18: "persistance" is a misspelling of "persistence"
core/src/test/scala/org/apache/spark/storage/FlatmapIteratorSuite.scala:26:69: "persistance" is a misspelling of "persistence"
data/streaming/AFINN-111.txt:1219:0: "humerous" is a misspelling of "humorous"
dev/run-pip-tests:55:28: "enviroments" is a misspelling of "environments"
dev/run-pip-tests:91:37: "virutal" is a misspelling of "virtual"
dev/merge_spark_pr.py:377:72: "accross" is a misspelling of "across"
dev/merge_spark_pr.py:378:66: "accross" is a misspelling of "across"
dev/run-pip-tests:126:25: "enviroments" is a misspelling of "environments"
docs/configuration.md:1830:82: "overriden" is a misspelling of "overridden"
docs/structured-streaming-programming-guide.md:525:45: "processs" is a misspelling of "processes"
docs/structured-streaming-programming-guide.md:1165:61: "BETWEN" is a misspelling of "BETWEEN"
docs/sql-programming-guide.md:1891:810: "behaivor" is a misspelling of "behavior"
examples/src/main/python/sql/arrow.py:98:8: "substract" is a misspelling of "subtract"
examples/src/main/python/sql/arrow.py:103:27: "substract" is a misspelling of "subtract"
licenses/LICENSE-heapq.txt:5:63: "Stichting" is a misspelling of "Stitching"
licenses/LICENSE-heapq.txt:6:2: "Mathematisch" is a misspelling of "Mathematics"
licenses/LICENSE-heapq.txt:262:29: "Stichting" is a misspelling of "Stitching"
licenses/LICENSE-heapq.txt:262:39: "Mathematisch" is a misspelling of "Mathematics"
licenses/LICENSE-heapq.txt:269:49: "Stichting" is a misspelling of "Stitching"
licenses/LICENSE-heapq.txt:269:59: "Mathematisch" is a misspelling of "Mathematics"
licenses/LICENSE-heapq.txt:274:2: "STICHTING" is a misspelling of "STITCHING"
licenses/LICENSE-heapq.txt:274:12: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
licenses/LICENSE-heapq.txt:276:29: "STICHTING" is a misspelling of "STITCHING"
licenses/LICENSE-heapq.txt:276:39: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
licenses-binary/LICENSE-heapq.txt:5:63: "Stichting" is a misspelling of "Stitching"
licenses-binary/LICENSE-heapq.txt:6:2: "Mathematisch" is a misspelling of "Mathematics"
licenses-binary/LICENSE-heapq.txt:262:29: "Stichting" is a misspelling of "Stitching"
licenses-binary/LICENSE-heapq.txt:262:39: "Mathematisch" is a misspelling of "Mathematics"
licenses-binary/LICENSE-heapq.txt:269:49: "Stichting" is a misspelling of "Stitching"
licenses-binary/LICENSE-heapq.txt:269:59: "Mathematisch" is a misspelling of "Mathematics"
licenses-binary/LICENSE-heapq.txt:274:2: "STICHTING" is a misspelling of "STITCHING"
licenses-binary/LICENSE-heapq.txt:274:12: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
licenses-binary/LICENSE-heapq.txt:276:29: "STICHTING" is a misspelling of "STITCHING"
licenses-binary/LICENSE-heapq.txt:276:39: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/hungarian.txt:170:0: "teh" is a misspelling of "the"
mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/portuguese.txt:53:0: "eles" is a misspelling of "eels"
mllib/src/main/scala/org/apache/spark/ml/stat/Summarizer.scala:99:20: "Euclidian" is a misspelling of "Euclidean"
mllib/src/main/scala/org/apache/spark/ml/stat/Summarizer.scala:539:11: "Euclidian" is a misspelling of "Euclidean"
mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAOptimizer.scala:77:36: "Teh" is a misspelling of "The"
mllib/src/main/scala/org/apache/spark/mllib/clustering/StreamingKMeans.scala:230:24: "inital" is a misspelling of "initial"
mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala:276:9: "Euclidian" is a misspelling of "Euclidean"
mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala:237:26: "descripiton" is a misspelling of "descriptions"
python/pyspark/find_spark_home.py:30:13: "enviroment" is a misspelling of "environment"
python/pyspark/context.py:937:12: "supress" is a misspelling of "suppress"
python/pyspark/context.py:938:12: "supress" is a misspelling of "suppress"
python/pyspark/context.py:939:12: "supress" is a misspelling of "suppress"
python/pyspark/context.py:940:12: "supress" is a misspelling of "suppress"
python/pyspark/heapq3.py:6:63: "Stichting" is a misspelling of "Stitching"
python/pyspark/heapq3.py:7:2: "Mathematisch" is a misspelling of "Mathematics"
python/pyspark/heapq3.py:263:29: "Stichting" is a misspelling of "Stitching"
python/pyspark/heapq3.py:263:39: "Mathematisch" is a misspelling of "Mathematics"
python/pyspark/heapq3.py:270:49: "Stichting" is a misspelling of "Stitching"
python/pyspark/heapq3.py:270:59: "Mathematisch" is a misspelling of "Mathematics"
python/pyspark/heapq3.py:275:2: "STICHTING" is a misspelling of "STITCHING"
python/pyspark/heapq3.py:275:12: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
python/pyspark/heapq3.py:277:29: "STICHTING" is a misspelling of "STITCHING"
python/pyspark/heapq3.py:277:39: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
python/pyspark/heapq3.py:713:8: "probabilty" is a misspelling of "probability"
python/pyspark/ml/clustering.py:1038:8: "Currenlty" is a misspelling of "Currently"
python/pyspark/ml/stat.py:339:23: "Euclidian" is a misspelling of "Euclidean"
python/pyspark/ml/regression.py:1378:20: "paramter" is a misspelling of "parameter"
python/pyspark/mllib/stat/_statistics.py:262:8: "probabilty" is a misspelling of "probability"
python/pyspark/rdd.py:1363:32: "paramter" is a misspelling of "parameter"
python/pyspark/streaming/tests.py:825:42: "retuns" is a misspelling of "returns"
python/pyspark/sql/tests.py:768:29: "initalization" is a misspelling of "initialization"
python/pyspark/sql/tests.py:3616:31: "initalize" is a misspelling of "initialize"
resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala:120:39: "arbitary" is a misspelling of "arbitrary"
resource-managers/mesos/src/test/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherArgumentsSuite.scala:26:45: "sucessfully" is a misspelling of "successfully"
resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala:358:27: "constaints" is a misspelling of "constraints"
resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala:111:24: "senstive" is a misspelling of "sensitive"
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala:1063:5: "overwirte" is a misspelling of "overwrite"
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala:1348:17: "compatability" is a misspelling of "compatibility"
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala:77:36: "paramter" is a misspelling of "parameter"
sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala:1374:22: "precendence" is a misspelling of "precedence"
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala:238:27: "unnecassary" is a misspelling of "unnecessary"
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ConditionalExpressionSuite.scala:212:17: "whn" is a misspelling of "when"
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinHelper.scala:147:60: "timestmap" is a misspelling of "timestamp"
sql/core/src/test/scala/org/apache/spark/sql/TPCDSQuerySuite.scala:150:45: "precentage" is a misspelling of "percentage"
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchemaSuite.scala:135:29: "infered" is a misspelling of "inferred"
sql/hive/src/test/resources/golden/udf_instr-1-2e76f819563dbaba4beb51e3a130b922:1:52: "occurance" is a misspelling of "occurrence"
sql/hive/src/test/resources/golden/udf_instr-2-32da357fc754badd6e3898dcc8989182:1:52: "occurance" is a misspelling of "occurrence"
sql/hive/src/test/resources/golden/udf_locate-1-6e41693c9c6dceea4d7fab4c02884e4e:1:63: "occurance" is a misspelling of "occurrence"
sql/hive/src/test/resources/golden/udf_locate-2-d9b5934457931447874d6bb7c13de478:1:63: "occurance" is a misspelling of "occurrence"
sql/hive/src/test/resources/golden/udf_translate-2-f7aa38a33ca0df73b7a1e6b6da4b7fe8:9:79: "occurence" is a misspelling of "occurrence"
sql/hive/src/test/resources/golden/udf_translate-2-f7aa38a33ca0df73b7a1e6b6da4b7fe8:13:110: "occurence" is a misspelling of "occurrence"
sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_join.q:46:105: "distint" is a misspelling of "distinct"
sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_11.q:29:3: "Currenly" is a misspelling of "Currently"
sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_partitioned.q:72:15: "existant" is a misspelling of "existent"
sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_udf.q:25:3: "substraction" is a misspelling of "subtraction"
sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_map_multi_distinct.q:16:51: "funtion" is a misspelling of "function"
sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_8.q:15:30: "issueing" is a misspelling of "issuing"
sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala:669:52: "wiht" is a misspelling of "with"
sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionImpl.java:474:9: "Refering" is a misspelling of "Referring"
```

### after

```
$ misspell . | grep -v '.js'
common/network-common/src/main/java/org/apache/spark/network/util/AbstractFileRegion.java:27:20: "transfered" is a misspelling of "transferred"
core/src/main/scala/org/apache/spark/status/storeTypes.scala:113:29: "ect" is a misspelling of "etc"
core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala:1922:49: "agriculteur" is a misspelling of "agriculture"
data/streaming/AFINN-111.txt:1219:0: "humerous" is a misspelling of "humorous"
licenses/LICENSE-heapq.txt:5:63: "Stichting" is a misspelling of "Stitching"
licenses/LICENSE-heapq.txt:6:2: "Mathematisch" is a misspelling of "Mathematics"
licenses/LICENSE-heapq.txt:262:29: "Stichting" is a misspelling of "Stitching"
licenses/LICENSE-heapq.txt:262:39: "Mathematisch" is a misspelling of "Mathematics"
licenses/LICENSE-heapq.txt:269:49: "Stichting" is a misspelling of "Stitching"
licenses/LICENSE-heapq.txt:269:59: "Mathematisch" is a misspelling of "Mathematics"
licenses/LICENSE-heapq.txt:274:2: "STICHTING" is a misspelling of "STITCHING"
licenses/LICENSE-heapq.txt:274:12: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
licenses/LICENSE-heapq.txt:276:29: "STICHTING" is a misspelling of "STITCHING"
licenses/LICENSE-heapq.txt:276:39: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
licenses-binary/LICENSE-heapq.txt:5:63: "Stichting" is a misspelling of "Stitching"
licenses-binary/LICENSE-heapq.txt:6:2: "Mathematisch" is a misspelling of "Mathematics"
licenses-binary/LICENSE-heapq.txt:262:29: "Stichting" is a misspelling of "Stitching"
licenses-binary/LICENSE-heapq.txt:262:39: "Mathematisch" is a misspelling of "Mathematics"
licenses-binary/LICENSE-heapq.txt:269:49: "Stichting" is a misspelling of "Stitching"
licenses-binary/LICENSE-heapq.txt:269:59: "Mathematisch" is a misspelling of "Mathematics"
licenses-binary/LICENSE-heapq.txt:274:2: "STICHTING" is a misspelling of "STITCHING"
licenses-binary/LICENSE-heapq.txt:274:12: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
licenses-binary/LICENSE-heapq.txt:276:29: "STICHTING" is a misspelling of "STITCHING"
licenses-binary/LICENSE-heapq.txt:276:39: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/hungarian.txt:170:0: "teh" is a misspelling of "the"
mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/portuguese.txt:53:0: "eles" is a misspelling of "eels"
mllib/src/main/scala/org/apache/spark/ml/stat/Summarizer.scala:99:20: "Euclidian" is a misspelling of "Euclidean"
mllib/src/main/scala/org/apache/spark/ml/stat/Summarizer.scala:539:11: "Euclidian" is a misspelling of "Euclidean"
mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAOptimizer.scala:77:36: "Teh" is a misspelling of "The"
mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala:276:9: "Euclidian" is a misspelling of "Euclidean"
python/pyspark/heapq3.py:6:63: "Stichting" is a misspelling of "Stitching"
python/pyspark/heapq3.py:7:2: "Mathematisch" is a misspelling of "Mathematics"
python/pyspark/heapq3.py:263:29: "Stichting" is a misspelling of "Stitching"
python/pyspark/heapq3.py:263:39: "Mathematisch" is a misspelling of "Mathematics"
python/pyspark/heapq3.py:270:49: "Stichting" is a misspelling of "Stitching"
python/pyspark/heapq3.py:270:59: "Mathematisch" is a misspelling of "Mathematics"
python/pyspark/heapq3.py:275:2: "STICHTING" is a misspelling of "STITCHING"
python/pyspark/heapq3.py:275:12: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
python/pyspark/heapq3.py:277:29: "STICHTING" is a misspelling of "STITCHING"
python/pyspark/heapq3.py:277:39: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
python/pyspark/ml/stat.py:339:23: "Euclidian" is a misspelling of "Euclidean"
```

Closes #22070 from seratch/fix-typo.

Authored-by: Kazuhiro Sera <seratch@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2018-08-11 21:23:36 -05:00
Shixiong Zhu f5aba65739
[SPARK-25081][CORE] Nested spill in ShuffleExternalSorter should not access released memory page
## What changes were proposed in this pull request?

This issue is pretty similar to [SPARK-21907](https://issues.apache.org/jira/browse/SPARK-21907).

"allocateArray" in [ShuffleInMemorySorter.reset](9b8521e53e/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorter.java (L99)) may trigger a spill and cause ShuffleInMemorySorter access the released `array`. Another task may get the same memory page from the pool. This will cause two tasks access the same memory page. When a task reads memory written by another task, many types of failures may happen. Here are some examples I  have seen:

- JVM crash. (This is easy to reproduce in a unit test as we fill newly allocated and deallocated memory with 0xa5 and 0x5a bytes which usually points to an invalid memory address)
- java.lang.IllegalArgumentException: Comparison method violates its general contract!
- java.lang.NullPointerException at org.apache.spark.memory.TaskMemoryManager.getPage(TaskMemoryManager.java:384)
- java.lang.UnsupportedOperationException: Cannot grow BufferHolder by size -536870912 because the size after growing exceeds size limitation 2147483632

This PR resets states in `ShuffleInMemorySorter.reset` before calling `allocateArray` to fix the issue.

## How was this patch tested?

The new unit test will make JVM crash without the fix.

Closes #22062 from zsxwing/SPARK-25081.

Authored-by: Shixiong Zhu <zsxwing@gmail.com>
Signed-off-by: Shixiong Zhu <zsxwing@gmail.com>
2018-08-10 10:53:44 -07:00
Liang-Chi Hsieh 4f17585098 [SPARK-19355][SQL] Use map output statistics to improve global limit's parallelism
## What changes were proposed in this pull request?

A logical `Limit` is performed physically by two operations `LocalLimit` and `GlobalLimit`.

Most of time, we gather all data into a single partition in order to run `GlobalLimit`. If we use a very big limit number, shuffling data causes performance issue also reduces parallelism.

We can avoid shuffling into single partition if we don't care data ordering. This patch implements this idea by doing a map stage during global limit. It collects the info of row numbers at each partition. For each partition, we locally retrieves limited data without any shuffling to finish this global limit.

For example, we have three partitions with rows (100, 100, 50) respectively. In global limit of 100 rows, we may take (34, 33, 33) rows for each partition locally. After global limit we still have three partitions.

If the data partition has certain ordering, we can't distribute required rows evenly to each partitions because it could change data ordering. But we still can avoid shuffling.

## How was this patch tested?

Jenkins tests.

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

Closes #16677 from viirya/improve-global-limit-parallelism.
2018-08-10 11:32:15 +02:00
Kazuaki Ishizaki 56e9e97073 [MINOR][DOC] Fix typo
## What changes were proposed in this pull request?

This PR fixes typo regarding `auxiliary verb + verb[s]`. This is a follow-on of #21956.

## How was this patch tested?

N/A

Closes #22040 from kiszk/spellcheck1.

Authored-by: Kazuaki Ishizaki <ishizaki@jp.ibm.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-08-09 20:10:17 +08:00
Xingbo Jiang d90f1336d8 [SPARK-25045][CORE] Make RDDBarrier.mapParititions similar to RDD.mapPartitions
## What changes were proposed in this pull request?

Signature of the function passed to `RDDBarrier.mapPartitions()` is different from that of `RDD.mapPartitions`. The later doesn’t take a `TaskContext`. We shall make the function signature the same to avoid confusion and misusage.

This PR proposes the following API changes:
- In `RDDBarrier`, migrate `mapPartitions` from
   ```
        def mapPartitions[S: ClassTag](
            f: (Iterator[T], BarrierTaskContext) => Iterator[S],
            preservesPartitioning: Boolean = false): RDD[S]
        }
   ```
    to
   ```
        def mapPartitions[S: ClassTag](
            f: Iterator[T] => Iterator[S],
            preservesPartitioning: Boolean = false): RDD[S]
        }
   ```
- Add new static method to get a `BarrierTaskContext`:
   ```
        object BarrierTaskContext {
           def get(): BarrierTaskContext
        }
   ```

## How was this patch tested?

Existing test cases.

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

Closes #22026 from jiangxb1987/mapPartitions.
2018-08-07 17:32:41 -07:00
Sean Owen 66699c5c30 [SPARK-25029][TESTS] Scala 2.12 issues: TaskNotSerializable and Janino "Two non-abstract methods ..." errors
## What changes were proposed in this pull request?

Fixes for test issues that arose after Scala 2.12 support was added -- ones that only affect the 2.12 build.

## How was this patch tested?

Existing tests.

Closes #22004 from srowen/SPARK-25029.

Authored-by: Sean Owen <srowen@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2018-08-07 17:30:37 -05:00
Maxim Gekk 131ca146ed [SPARK-24005][CORE] Remove usage of Scala’s parallel collection
## What changes were proposed in this pull request?

In the PR, I propose to replace Scala parallel collections by new methods `parmap()`. The methods use futures to transform a sequential collection by applying a lambda function to each element in parallel. The result of `parmap` is another regular (sequential) collection.

The proposed `parmap` method aims to solve the problem of impossibility to interrupt parallel Scala collection. This possibility is needed for reliable task preemption.

## How was this patch tested?

A test was added to `ThreadUtilsSuite`

Closes #21913 from MaxGekk/par-map.

Authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-08-07 17:14:30 +08:00
Xingbo Jiang 388f5a0635 [SPARK-24817][CORE] Implement BarrierTaskContext.barrier()
## What changes were proposed in this pull request?

Implement BarrierTaskContext.barrier(), to support global sync between all the tasks in a barrier stage.
The function set a global barrier and waits until all tasks in this stage hit this barrier. Similar to MPI_Barrier function in MPI, the barrier() function call blocks until all tasks in the same stage have reached this routine. The global sync shall finish immediately once all tasks in the same barrier stage reaches the same barrier.

This PR implements BarrierTaskContext.barrier() based on netty-based RPC client, introduces new `BarrierCoordinator` and new `BarrierCoordinatorMessage`, and new config to handle timeout issue.

## How was this patch tested?
Add `BarrierTaskContextSuite` to test `BarrierTaskContext.barrier()`

Closes #21898 from jiangxb1987/taskcontext.barrier.

Authored-by: Xingbo Jiang <xingbo.jiang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-08-07 15:06:32 +08:00
Marco Gaido 3c96937c7b [SPARK-24948][SHS] Delegate check access permissions to the file system
## What changes were proposed in this pull request?

In `SparkHadoopUtil. checkAccessPermission`,  we consider only basic permissions in order to check wether a user can access a file or not. This is not a complete check, as it ignores ACLs and other policies a file system may apply in its internal. So this can result in returning wrongly that a user cannot access a file (despite he actually can).

The PR proposes to delegate to the filesystem the check whether a file is accessible or not, in order to return the right result. A caching layer is added for performance reasons.

## How was this patch tested?

modified UTs

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21895 from mgaido91/SPARK-24948.
2018-08-06 14:29:05 -07:00
hyukjinkwon 55e3ae6930 [SPARK-25001][BUILD] Fix miscellaneous build warnings
## What changes were proposed in this pull request?

There are many warnings in the current build (for instance see https://amplab.cs.berkeley.edu/jenkins/job/spark-master-test-sbt-hadoop-2.7/4734/console).

**common**:

```
[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDB.java:237: warning: [rawtypes] found raw type: LevelDBIterator
[warn]   void closeIterator(LevelDBIterator it) throws IOException {
[warn]                      ^

[warn]   missing type arguments for generic class LevelDBIterator<T>
[warn]   where T is a type-variable:
[warn]     T extends Object declared in class LevelDBIterator
[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/main/java/org/apache/spark/network/server/TransportServer.java:151: warning: [deprecation] group() in AbstractBootstrap has been deprecated
[warn]     if (bootstrap != null && bootstrap.group() != null) {
[warn]                                       ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/main/java/org/apache/spark/network/server/TransportServer.java:152: warning: [deprecation] group() in AbstractBootstrap has been deprecated
[warn]       bootstrap.group().shutdownGracefully();
[warn]                ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/main/java/org/apache/spark/network/server/TransportServer.java:154: warning: [deprecation] childGroup() in ServerBootstrap has been deprecated
[warn]     if (bootstrap != null && bootstrap.childGroup() != null) {
[warn]                                       ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/main/java/org/apache/spark/network/server/TransportServer.java:155: warning: [deprecation] childGroup() in ServerBootstrap has been deprecated
[warn]       bootstrap.childGroup().shutdownGracefully();
[warn]                ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/main/java/org/apache/spark/network/util/NettyUtils.java:112: warning: [deprecation] PooledByteBufAllocator(boolean,int,int,int,int,int,int,int) in PooledByteBufAllocator has been deprecated
[warn]     return new PooledByteBufAllocator(
[warn]            ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/main/java/org/apache/spark/network/client/TransportClient.java:321: warning: [rawtypes] found raw type: Future
[warn]     public void operationComplete(Future future) throws Exception {
[warn]                                   ^

[warn]   missing type arguments for generic class Future<V>
[warn]   where V is a type-variable:
[warn]     V extends Object declared in interface Future
[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java:215: warning: [rawtypes] found raw type: StreamInterceptor
[warn]           StreamInterceptor interceptor = new StreamInterceptor(this, resp.streamId, resp.byteCount,
[warn]           ^

[warn]   missing type arguments for generic class StreamInterceptor<T>
[warn]   where T is a type-variable:
[warn]     T extends Message declared in class StreamInterceptor
[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java:215: warning: [rawtypes] found raw type: StreamInterceptor
[warn]           StreamInterceptor interceptor = new StreamInterceptor(this, resp.streamId, resp.byteCount,
[warn]                                               ^

[warn]   missing type arguments for generic class StreamInterceptor<T>
[warn]   where T is a type-variable:
[warn]     T extends Message declared in class StreamInterceptor
[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java:215: warning: [unchecked] unchecked call to StreamInterceptor(MessageHandler<T>,String,long,StreamCallback) as a member of the raw type StreamInterceptor
[warn]           StreamInterceptor interceptor = new StreamInterceptor(this, resp.streamId, resp.byteCount,
[warn]                                           ^

[warn]   where T is a type-variable:
[warn]     T extends Message declared in class StreamInterceptor
[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java:255: warning: [rawtypes] found raw type: StreamInterceptor
[warn]         StreamInterceptor interceptor = new StreamInterceptor(this, wrappedCallback.getID(),
[warn]         ^

[warn]   missing type arguments for generic class StreamInterceptor<T>
[warn]   where T is a type-variable:
[warn]     T extends Message declared in class StreamInterceptor
[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java:255: warning: [rawtypes] found raw type: StreamInterceptor
[warn]         StreamInterceptor interceptor = new StreamInterceptor(this, wrappedCallback.getID(),
[warn]                                             ^

[warn]   missing type arguments for generic class StreamInterceptor<T>
[warn]   where T is a type-variable:
[warn]     T extends Message declared in class StreamInterceptor
[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java:255: warning: [unchecked] unchecked call to StreamInterceptor(MessageHandler<T>,String,long,StreamCallback) as a member of the raw type StreamInterceptor
[warn]         StreamInterceptor interceptor = new StreamInterceptor(this, wrappedCallback.getID(),
[warn]                                         ^

[warn]   where T is a type-variable:
[warn]     T extends Message declared in class StreamInterceptor
[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/main/java/org/apache/spark/network/crypto/TransportCipher.java:270: warning: [deprecation] transfered() in FileRegion has been deprecated
[warn]         region.transferTo(byteRawChannel, region.transfered());
[warn]                                                 ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/main/java/org/apache/spark/network/sasl/SaslEncryption.java:304: warning: [deprecation] transfered() in FileRegion has been deprecated
[warn]         region.transferTo(byteChannel, region.transfered());
[warn]                                              ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/test/java/org/apache/spark/network/ProtocolSuite.java:119: warning: [deprecation] transfered() in FileRegion has been deprecated
[warn]       while (in.transfered() < in.count()) {
[warn]                ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/test/java/org/apache/spark/network/ProtocolSuite.java:120: warning: [deprecation] transfered() in FileRegion has been deprecated
[warn]         in.transferTo(channel, in.transfered());
[warn]                                  ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/unsafe/src/test/java/org/apache/spark/unsafe/hash/Murmur3_x86_32Suite.java:80: warning: [static] static method should be qualified by type name, Murmur3_x86_32, instead of by an expression
[warn]     Assert.assertEquals(-300363099, hasher.hashUnsafeWords(bytes, offset, 16, 42));
[warn]                                           ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/unsafe/src/test/java/org/apache/spark/unsafe/hash/Murmur3_x86_32Suite.java:84: warning: [static] static method should be qualified by type name, Murmur3_x86_32, instead of by an expression
[warn]     Assert.assertEquals(-1210324667, hasher.hashUnsafeWords(bytes, offset, 16, 42));
[warn]                                            ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/unsafe/src/test/java/org/apache/spark/unsafe/hash/Murmur3_x86_32Suite.java:88: warning: [static] static method should be qualified by type name, Murmur3_x86_32, instead of by an expression
[warn]     Assert.assertEquals(-634919701, hasher.hashUnsafeWords(bytes, offset, 16, 42));
[warn]                                           ^
```

**launcher**:

```
[warn] Pruning sources from previous analysis, due to incompatible CompileSetup.
[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/launcher/src/main/java/org/apache/spark/launcher/AbstractLauncher.java:31: warning: [rawtypes] found raw type: AbstractLauncher
[warn] public abstract class AbstractLauncher<T extends AbstractLauncher> {
[warn]                                                  ^
[warn]   missing type arguments for generic class AbstractLauncher<T>
[warn]   where T is a type-variable:
[warn]     T extends AbstractLauncher declared in class AbstractLauncher
```

**core**:

```
[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/main/scala/org/apache/spark/api/r/RBackend.scala:99: method group in class AbstractBootstrap is deprecated: see corresponding Javadoc for more information.
[warn]     if (bootstrap != null && bootstrap.group() != null) {
[warn]                                        ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/main/scala/org/apache/spark/api/r/RBackend.scala💯 method group in class AbstractBootstrap is deprecated: see corresponding Javadoc for more information.
[warn]       bootstrap.group().shutdownGracefully()
[warn]                 ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/main/scala/org/apache/spark/api/r/RBackend.scala:102: method childGroup in class ServerBootstrap is deprecated: see corresponding Javadoc for more information.
[warn]     if (bootstrap != null && bootstrap.childGroup() != null) {
[warn]                                        ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/main/scala/org/apache/spark/api/r/RBackend.scala:103: method childGroup in class ServerBootstrap is deprecated: see corresponding Javadoc for more information.
[warn]       bootstrap.childGroup().shutdownGracefully()
[warn]                 ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala:151: reflective access of structural type member method getData should be enabled
[warn] by making the implicit value scala.language.reflectiveCalls visible.
[warn] This can be achieved by adding the import clause 'import scala.language.reflectiveCalls'
[warn] or by setting the compiler option -language:reflectiveCalls.
[warn] See the Scaladoc for value scala.language.reflectiveCalls for a discussion
[warn] why the feature should be explicitly enabled.
[warn]       val rdd = sc.parallelize(1 to 1).map(concreteObject.getData)
[warn]                                                           ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala:175: reflective access of structural type member value innerObject2 should be enabled
[warn] by making the implicit value scala.language.reflectiveCalls visible.
[warn]       val rdd = sc.parallelize(1 to 1).map(concreteObject.innerObject2.getData)
[warn]                                                           ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala:175: reflective access of structural type member method getData should be enabled
[warn] by making the implicit value scala.language.reflectiveCalls visible.
[warn]       val rdd = sc.parallelize(1 to 1).map(concreteObject.innerObject2.getData)
[warn]                                                                        ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/LocalSparkContext.scala:32: constructor Slf4JLoggerFactory in class Slf4JLoggerFactory is deprecated: see corresponding Javadoc for more information.
[warn]     InternalLoggerFactory.setDefaultFactory(new Slf4JLoggerFactory())
[warn]                                             ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala:218: value attemptId in class StageInfo is deprecated: Use attemptNumber instead
[warn]         assert(wrapper.stageAttemptId === stages.head.attemptId)
[warn]                                                       ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala:261: value attemptId in class StageInfo is deprecated: Use attemptNumber instead
[warn]       stageAttemptId = stages.head.attemptId))
[warn]                                    ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala:287: value attemptId in class StageInfo is deprecated: Use attemptNumber instead
[warn]       stageAttemptId = stages.head.attemptId))
[warn]                                    ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala:471: value attemptId in class StageInfo is deprecated: Use attemptNumber instead
[warn]       stageAttemptId = stages.last.attemptId))
[warn]                                    ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala:966: value attemptId in class StageInfo is deprecated: Use attemptNumber instead
[warn]     listener.onTaskStart(SparkListenerTaskStart(dropped.stageId, dropped.attemptId, task))
[warn]                                                                          ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala:972: value attemptId in class StageInfo is deprecated: Use attemptNumber instead
[warn]     listener.onTaskEnd(SparkListenerTaskEnd(dropped.stageId, dropped.attemptId,
[warn]                                                                      ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala:976: value attemptId in class StageInfo is deprecated: Use attemptNumber instead
[warn]       .taskSummary(dropped.stageId, dropped.attemptId, Array(0.25d, 0.50d, 0.75d))
[warn]                                             ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala:1146: value attemptId in class StageInfo is deprecated: Use attemptNumber instead
[warn]       SparkListenerTaskEnd(stage1.stageId, stage1.attemptId, "taskType", Success, tasks(1), null))
[warn]                                                   ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala:1150: value attemptId in class StageInfo is deprecated: Use attemptNumber instead
[warn]       SparkListenerTaskEnd(stage1.stageId, stage1.attemptId, "taskType", Success, tasks(0), null))
[warn]                                                   ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/storage/DiskStoreSuite.scala:197: method transfered in trait FileRegion is deprecated: see corresponding Javadoc for more information.
[warn]     while (region.transfered() < region.count()) {
[warn]                   ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/storage/DiskStoreSuite.scala:198: method transfered in trait FileRegion is deprecated: see corresponding Javadoc for more information.
[warn]       region.transferTo(byteChannel, region.transfered())
[warn]                                             ^
```

**sql**:

```
[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala:534: abstract type T is unchecked since it is eliminated by erasure
[warn]       assert(partitioning.isInstanceOf[T])
[warn]                                       ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala:534: abstract type T is unchecked since it is eliminated by erasure
[warn]       assert(partitioning.isInstanceOf[T])
[warn]             ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala:323: inferred existential type Option[Class[_$1]]( forSome { type _$1 }), which cannot be expressed by wildcards,  should be enabled
[warn] by making the implicit value scala.language.existentials visible.
[warn] This can be achieved by adding the import clause 'import scala.language.existentials'
[warn] or by setting the compiler option -language:existentials.
[warn] See the Scaladoc for value scala.language.existentials for a discussion
[warn] why the feature should be explicitly enabled.
[warn]       val optClass = Option(collectionCls)
[warn]                            ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java:226: warning: [deprecation] ParquetFileReader(Configuration,FileMetaData,Path,List<BlockMetaData>,List<ColumnDescriptor>) in ParquetFileReader has been deprecated
[warn]     this.reader = new ParquetFileReader(
[warn]                   ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java:178: warning: [deprecation] getType() in ColumnDescriptor has been deprecated
[warn]             (descriptor.getType() == PrimitiveType.PrimitiveTypeName.INT32 ||
[warn]                        ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java:179: warning: [deprecation] getType() in ColumnDescriptor has been deprecated
[warn]             (descriptor.getType() == PrimitiveType.PrimitiveTypeName.INT64  &&
[warn]                        ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java:181: warning: [deprecation] getType() in ColumnDescriptor has been deprecated
[warn]             descriptor.getType() == PrimitiveType.PrimitiveTypeName.FLOAT ||
[warn]                       ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java:182: warning: [deprecation] getType() in ColumnDescriptor has been deprecated
[warn]             descriptor.getType() == PrimitiveType.PrimitiveTypeName.DOUBLE ||
[warn]                       ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java:183: warning: [deprecation] getType() in ColumnDescriptor has been deprecated
[warn]             descriptor.getType() == PrimitiveType.PrimitiveTypeName.BINARY))) {
[warn]                       ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java:198: warning: [deprecation] getType() in ColumnDescriptor has been deprecated
[warn]         switch (descriptor.getType()) {
[warn]                           ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java:221: warning: [deprecation] getTypeLength() in ColumnDescriptor has been deprecated
[warn]             readFixedLenByteArrayBatch(rowId, num, column, descriptor.getTypeLength());
[warn]                                                                      ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java:224: warning: [deprecation] getType() in ColumnDescriptor has been deprecated
[warn]             throw new IOException("Unsupported type: " + descriptor.getType());
[warn]                                                                    ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java:246: warning: [deprecation] getType() in ColumnDescriptor has been deprecated
[warn]       descriptor.getType().toString(),
[warn]                 ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java:258: warning: [deprecation] getType() in ColumnDescriptor has been deprecated
[warn]     switch (descriptor.getType()) {
[warn]                       ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java:384: warning: [deprecation] getType() in ColumnDescriptor has been deprecated
[warn]         throw new UnsupportedOperationException("Unsupported type: " + descriptor.getType());
[warn]                                                                                  ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/main/java/org/apache/spark/sql/vectorized/ArrowColumnVector.java:458: warning: [static] static variable should be qualified by type name, BaseRepeatedValueVector, instead of by an expression
[warn]       int index = rowId * accessor.OFFSET_WIDTH;
[warn]                                   ^
[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/main/java/org/apache/spark/sql/vectorized/ArrowColumnVector.java:460: warning: [static] static variable should be qualified by type name, BaseRepeatedValueVector, instead of by an expression
[warn]       int end = offsets.getInt(index + accessor.OFFSET_WIDTH);
[warn]                                                ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/test/scala/org/apache/spark/sql/BenchmarkQueryTest.scala:57: a pure expression does nothing in statement position; you may be omitting necessary parentheses
[warn]       case s => s
[warn]                 ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala:182: inferred existential type org.apache.parquet.column.statistics.Statistics[?0]( forSome { type ?0 <: Comparable[?0] }), which cannot be expressed by wildcards,  should be enabled
[warn] by making the implicit value scala.language.existentials visible.
[warn] This can be achieved by adding the import clause 'import scala.language.existentials'
[warn] or by setting the compiler option -language:existentials.
[warn] See the Scaladoc for value scala.language.existentials for a discussion
[warn] why the feature should be explicitly enabled.
[warn]                 val columnStats = oneBlockColumnMeta.getStatistics
[warn]                                                      ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ForeachBatchSinkSuite.scala:146: implicit conversion method conv should be enabled
[warn] by making the implicit value scala.language.implicitConversions visible.
[warn] This can be achieved by adding the import clause 'import scala.language.implicitConversions'
[warn] or by setting the compiler option -language:implicitConversions.
[warn] See the Scaladoc for value scala.language.implicitConversions for a discussion
[warn] why the feature should be explicitly enabled.
[warn]     implicit def conv(x: (Int, Long)): KV = KV(x._1, x._2)
[warn]                  ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/shuffle/ContinuousShuffleSuite.scala:48: implicit conversion method unsafeRow should be enabled
[warn] by making the implicit value scala.language.implicitConversions visible.
[warn]   private implicit def unsafeRow(value: Int) = {
[warn]                        ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala:178: method getType in class ColumnDescriptor is deprecated: see corresponding Javadoc for more information.
[warn]                 assert(oneFooter.getFileMetaData.getSchema.getColumns.get(0).getType() ===
[warn]                                                                              ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetTest.scala:154: method readAllFootersInParallel in object ParquetFileReader is deprecated: see corresponding Javadoc for more information.
[warn]     ParquetFileReader.readAllFootersInParallel(configuration, fs.getFileStatus(path)).asScala.toSeq
[warn]                       ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/hive/src/test/java/org/apache/spark/sql/hive/test/Complex.java:679: warning: [cast] redundant cast to Complex
[warn]     Complex typedOther = (Complex)other;
[warn]                          ^
```

**mllib**:

```
[warn] Pruning sources from previous analysis, due to incompatible CompileSetup.
[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala:597: match may not be exhaustive.
[warn] It would fail on the following inputs: None, Some((x: Tuple2[?, ?] forSome x not in (?, ?)))
[warn]     val df = dfs.find {
[warn]                       ^
```

This PR does not target fix all of them since some look pretty tricky to fix and there look too many warnings including false positive (like deprecated API but it's used in its test, etc.)

## How was this patch tested?

Existing tests should cover this.

Author: hyukjinkwon <gurwls223@apache.org>

Closes #21975 from HyukjinKwon/remove-build-warnings.
2018-08-04 11:52:49 -05:00
Xingbo Jiang 92b48842b9 [SPARK-24954][CORE] Fail fast on job submit if run a barrier stage with dynamic resource allocation enabled
## What changes were proposed in this pull request?

We don't support run a barrier stage with dynamic resource allocation enabled, it shall lead to some confusing behaviors (eg. with dynamic resource allocation enabled, it may happen that we acquire some executors (but not enough to launch all the tasks in a barrier stage) and later release them due to executor idle time expire, and then acquire again).

We perform the check on job submit and fail fast if running a barrier stage with dynamic resource allocation enabled.

## How was this patch tested?

Added new test suite `BarrierStageOnSubmittedSuite` to cover all the fail fast cases that submitted a job containing one or more barrier stages.

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

Closes #21915 from jiangxb1987/SPARK-24954.
2018-08-03 09:36:56 -07:00
Xingbo Jiang 38e4699c97 [SPARK-24820][SPARK-24821][CORE] Fail fast when submitted job contains a barrier stage with unsupported RDD chain pattern
## What changes were proposed in this pull request?

Check on job submit to make sure we don't launch a barrier stage with unsupported RDD chain pattern. The following patterns are not supported:
- Ancestor RDDs that have different number of partitions from the resulting RDD (eg. union()/coalesce()/first()/PartitionPruningRDD);
- An RDD that depends on multiple barrier RDDs (eg. barrierRdd1.zip(barrierRdd2)).

## How was this patch tested?

Add test cases in `BarrierStageOnSubmittedSuite`.

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

Closes #21927 from jiangxb1987/SPARK-24820.
2018-08-02 09:36:26 -07:00
Stavros Kontopoulos a65736996b [SPARK-14540][CORE] Fix remaining major issues for Scala 2.12 Support
## What changes were proposed in this pull request?
This PR addresses issues 2,3 in this [document](https://docs.google.com/document/d/1fbkjEL878witxVQpOCbjlvOvadHtVjYXeB-2mgzDTvk).

* We modified the closure cleaner to identify closures that are implemented via the LambdaMetaFactory mechanism (serializedLambdas) (issue2).

* We also fix the issue due to scala/bug#11016. There are two options for solving the Unit issue, either add () at the end of the closure or use the trick described in the doc. Otherwise overloading resolution does not work (we are not going to eliminate either of the methods) here. Compiler tries to adapt to Unit and makes these two methods candidates for overloading, when there is polymorphic overloading there is no ambiguity (that is the workaround implemented). This does not look that good but it serves its purpose as we need to support two different uses for method: `addTaskCompletionListener`. One that passes a TaskCompletionListener and one that passes a closure that is wrapped with a TaskCompletionListener later on (issue3).

Note: regarding issue 1 in the doc the plan is:

> Do Nothing. Don’t try to fix this as this is only a problem for Java users who would want to use 2.11 binaries. In that case they can cast to MapFunction to be able to utilize lambdas. In Spark 3.0.0 the API should be simplified so that this issue is removed.

## How was this patch tested?
This was manually tested:
```./dev/change-scala-version.sh 2.12
./build/mvn -DskipTests -Pscala-2.12 clean package
./build/mvn -Pscala-2.12 clean package -DwildcardSuites=org.apache.spark.serializer.ProactiveClosureSerializationSuite -Dtest=None
./build/mvn -Pscala-2.12 clean package -DwildcardSuites=org.apache.spark.util.ClosureCleanerSuite -Dtest=None
./build/mvn -Pscala-2.12 clean package -DwildcardSuites=org.apache.spark.streaming.DStreamClosureSuite -Dtest=None```

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

Closes #21930 from skonto/scala2.12-sup.
2018-08-02 09:17:09 -05:00
Xingbo Jiang 275415777b [SPARK-24795][CORE][FOLLOWUP] Kill all running tasks when a task in a barrier stage fail
## What changes were proposed in this pull request?

Kill all running tasks when a task in a barrier stage fail in the middle. `TaskScheduler`.`cancelTasks()` will also fail the job, so we implemented a new method `killAllTaskAttempts()` to just kill all running tasks of a stage without cancel the stage/job.

## How was this patch tested?

Add new test cases in `TaskSchedulerImplSuite`.

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

Closes #21943 from jiangxb1987/killAllTasks.
2018-08-02 20:54:36 +08:00
Kazuaki Ishizaki 95a9d5e3a5 [SPARK-23915][SQL] Add array_except function
## What changes were proposed in this pull request?

The PR adds the SQL function `array_except`. The behavior of the function is based on Presto's one.

This function returns returns an array of the elements in array1 but not in array2.

Note: The order of elements in the result is not defined.

## How was this patch tested?

Added UTs.

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

Closes #21103 from kiszk/SPARK-23915.
2018-08-02 02:52:30 +08:00
Xingbo Jiang 3695ba5773 [MINOR][CORE][TEST] Fix afterEach() in TastSetManagerSuite and TaskSchedulerImplSuite
## What changes were proposed in this pull request?

In the `afterEach()` method of both `TastSetManagerSuite` and `TaskSchedulerImplSuite`, `super.afterEach()` shall be called at the end, because it shall stop the SparkContext.

https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/93706/testReport/org.apache.spark.scheduler/TaskSchedulerImplSuite/_It_is_not_a_test_it_is_a_sbt_testing_SuiteSelector_/
The test failure is caused by the above reason, the newly added `barrierCoordinator` required `rpcEnv` which has been stopped before `TaskSchedulerImpl` doing cleanup.

## How was this patch tested?
Existing tests.

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

Closes #21908 from jiangxb1987/afterEach.
2018-07-30 09:58:28 +08:00
Hieu Huynh 5828f41a52 [SPARK-13343] speculative tasks that didn't commit shouldn't be marked as success
**Description**
Currently Speculative tasks that didn't commit can show up as success (depending on timing of commit). This is a bit confusing because that task didn't really succeed in the sense it didn't write anything.
I think these tasks should be marked as KILLED or something that is more obvious to the user exactly what happened. it is happened to hit the timing where it got a commit denied exception then it shows up as failed and counts against your task failures. It shouldn't count against task failures since that failure really doesn't matter.
MapReduce handles these situation so perhaps we can look there for a model.

<img width="1420" alt="unknown" src="https://user-images.githubusercontent.com/15680678/42013170-99db48c2-7a61-11e8-8c7b-ef94c84e36ea.png">

**How can this issue happen?**
When both attempts of a task finish before the driver sends command to kill one of them, both of them send the status update FINISHED to the driver. The driver calls TaskSchedulerImpl to handle one successful task at a time. When it handles the first successful task, it sends the command to kill the other copy of the task, however, because that task is already finished, the executor will ignore the command. After finishing handling the first attempt, it processes the second one, although all actions on the result of this task are skipped, this copy of the task is still marked as SUCCESS. As a result, even though this issue does not affect the result of the job, it might cause confusing to user because both of them appear to be successful.

**How does this PR fix the issue?**
The simple way to fix this issue is that when taskSetManager handles successful task, it checks if any other attempt succeeded. If this is the case, it will call handleFailedTask with state==KILLED and reason==TaskKilled(“another attempt succeeded”) to handle this task as begin killed.

**How was this patch tested?**
I tested this manually by running applications, that caused the issue before, a few times, and observed that the issue does not happen again. Also, I added a unit test in TaskSetManagerSuite to test that if we call handleSuccessfulTask to handle status update for 2 copies of a task, only the one that is handled first will be mark as SUCCESS

Author: Hieu Huynh <“Hieu.huynh@oath.com”>
Author: hthuynh2 <hthieu96@gmail.com>

Closes #21653 from hthuynh2/SPARK_13343.
2018-07-27 12:34:14 -05:00
Xingbo Jiang e3486e1b95 [SPARK-24795][CORE] Implement barrier execution mode
## What changes were proposed in this pull request?

Propose new APIs and modify job/task scheduling to support barrier execution mode, which requires all tasks in a same barrier stage start at the same time, and retry all tasks in case some tasks fail in the middle. The barrier execution mode is useful for some ML/DL workloads.

The proposed API changes include:

- `RDDBarrier` that marks an RDD as barrier (Spark must launch all the tasks together for the current stage).
- `BarrierTaskContext` that support global sync of all tasks in a barrier stage, and provide extra `BarrierTaskInfo`s.

In DAGScheduler, we retry all tasks of a barrier stage in case some tasks fail in the middle, this is achieved by unregistering map outputs for a shuffleId (for ShuffleMapStage) or clear the finished partitions in an active job (for ResultStage).

## How was this patch tested?

Add `RDDBarrierSuite` to ensure we convert RDDs correctly;
Add new test cases in `DAGSchedulerSuite` to ensure we do task scheduling correctly;
Add new test cases in `SparkContextSuite` to ensure the barrier execution mode actually works (both under local mode and local cluster mode).
Add new test cases in `TaskSchedulerImplSuite` to ensure we schedule tasks for barrier taskSet together.

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

Closes #21758 from jiangxb1987/barrier-execution-mode.
2018-07-26 12:09:01 -07:00
Imran Rashid 7e847646d1 [SPARK-24307][CORE] Support reading remote cached partitions > 2gb
(1) Netty's ByteBuf cannot support data > 2gb.  So to transfer data from a
ChunkedByteBuffer over the network, we use a custom version of
FileRegion which is backed by the ChunkedByteBuffer.

(2) On the receiving end, we need to expose all the data in a
FileSegmentManagedBuffer as a ChunkedByteBuffer.  We do that by memory
mapping the entire file in chunks.

Added unit tests.  Ran the randomized test a couple of hundred times on my laptop.  Tests cover the equivalent of SPARK-24107 for the ChunkedByteBufferFileRegion.  Also tested on a cluster with remote cache reads >2gb (in memory and on disk).

Author: Imran Rashid <irashid@cloudera.com>

Closes #21440 from squito/chunked_bb_file_region.
2018-07-20 11:16:53 +08:00
Hieu Huynh 8d707b0600 [SPARK-24755][CORE] Executor loss can cause task to not be resubmitted
**Description**
As described in [SPARK-24755](https://issues.apache.org/jira/browse/SPARK-24755), when speculation is enabled, there is scenario that executor loss can cause task to not be resubmitted.
This patch changes the variable killedByOtherAttempt to keeps track of the taskId of tasks that are killed by other attempt. By doing this, we can still prevent resubmitting task killed by other attempt while resubmit successful attempt when executor lost.

**How was this patch tested?**
A UT is added based on the UT written by xuanyuanking with modification to simulate the scenario described in SPARK-24755.

Author: Hieu Huynh <“Hieu.huynh@oath.com”>

Closes #21729 from hthuynh2/SPARK_24755.
2018-07-19 09:52:07 -05:00
sychen c8bee932cb [SPARK-24677][CORE] Avoid NoSuchElementException from MedianHeap
## What changes were proposed in this pull request?
When speculation is enabled,
TaskSetManager#markPartitionCompleted should write successful task duration to MedianHeap,
not just increase tasksSuccessful.

Otherwise when TaskSetManager#checkSpeculatableTasks,tasksSuccessful non-zero, but MedianHeap is empty.
Then throw an exception successfulTaskDurations.median java.util.NoSuchElementException: MedianHeap is empty.
Finally led to stopping SparkContext.
## How was this patch tested?
TaskSetManagerSuite.scala
unit test:[SPARK-24677] MedianHeap should not be empty when speculation is enabled

Author: sychen <sychen@ctrip.com>

Closes #21656 from cxzl25/fix_MedianHeap_empty.
2018-07-18 13:24:41 -05:00
Nihar Sheth 2694dd2bf0 [MINOR][CORE] Add test cases for RDD.cartesian
## What changes were proposed in this pull request?

While looking through the codebase, it appeared that the scala code for RDD.cartesian does not have any tests for correctness. This adds a couple basic tests to verify cartesian yields correct values. While the implementation for RDD.cartesian is pretty simple, it always helps to have a few tests!

## How was this patch tested?

The new test cases pass, and the scala style tests from running dev/run-tests all pass.

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

Author: Nihar Sheth <niharrsheth@gmail.com>

Closes #21765 from NiharS/cartesianTests.
2018-07-18 09:14:36 -05:00
Dhruve Ashar 1055c94cdf [SPARK-24610] fix reading small files via wholeTextFiles
## What changes were proposed in this pull request?
The `WholeTextFileInputFormat` determines the `maxSplitSize` for the file/s being read using the `wholeTextFiles` method. While this works well for large files, for smaller files where the maxSplitSize is smaller than the defaults being used with configs like hive-site.xml or explicitly passed in the form of `mapreduce.input.fileinputformat.split.minsize.per.node` or `mapreduce.input.fileinputformat.split.minsize.per.rack` , it just throws up an exception.

```java
java.io.IOException: Minimum split size pernode 123456 cannot be larger than maximum split size 9962
at org.apache.hadoop.mapreduce.lib.input.CombineFileInputFormat.getSplits(CombineFileInputFormat.java:200)
at org.apache.spark.rdd.WholeTextFileRDD.getPartitions(WholeTextFileRDD.scala:50)
at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:252)
at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:250)
at scala.Option.getOrElse(Option.scala:121)
at org.apache.spark.rdd.RDD.partitions(RDD.scala:250)
at org.apache.spark.rdd.MapPartitionsRDD.getPartitions(MapPartitionsRDD.scala:35)
at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:252)
at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:250)
at scala.Option.getOrElse(Option.scala:121)
at org.apache.spark.rdd.RDD.partitions(RDD.scala:250)
at org.apache.spark.SparkContext.runJob(SparkContext.scala:2096)
at org.apache.spark.rdd.RDD.count(RDD.scala:1158)
... 48 elided
`

This change checks the maxSplitSize against the minSplitSizePerNode and minSplitSizePerRack and set them if `maxSplitSize < minSplitSizePerNode/Rack`

## How was this patch tested?
Test manually setting the conf while launching the job and added unit test.

Author: Dhruve Ashar <dhruveashar@gmail.com>

Closes #21601 from dhruve/bug/SPARK-24610.
2018-07-12 15:36:02 -05:00
sharkdtu 6fe32869cc [SPARK-24678][SPARK-STREAMING] Give priority in use of 'PROCESS_LOCAL' for spark-streaming
## What changes were proposed in this pull request?

Currently, `BlockRDD.getPreferredLocations`  only get hosts info of blocks, which results in subsequent schedule level is not better than 'NODE_LOCAL'. We can just make a small changes, the schedule level can be improved to 'PROCESS_LOCAL'

## How was this patch tested?

manual test

Author: sharkdtu <sharkdtu@tencent.com>

Closes #21658 from sharkdtu/master.
2018-07-10 20:18:34 +08:00
jerryshao e2c7e09f74 [SPARK-24646][CORE] Minor change to spark.yarn.dist.forceDownloadSchemes to support wildcard '*'
## What changes were proposed in this pull request?

In the case of getting tokens via customized `ServiceCredentialProvider`, it is required that `ServiceCredentialProvider` be available in local spark-submit process classpath. In this case, all the configured remote sources should be forced to download to local.

For the ease of using this configuration, here propose to add wildcard '*' support to `spark.yarn.dist.forceDownloadSchemes`, also clarify the usage of this configuration.

## How was this patch tested?

New UT added.

Author: jerryshao <sshao@hortonworks.com>

Closes #21633 from jerryshao/SPARK-21917-followup.
2018-07-09 10:21:40 +08:00
Xingbo Jiang 5b05966488 [SPARK-24564][TEST] Add test suite for RecordBinaryComparator
## What changes were proposed in this pull request?

Add a new test suite to test RecordBinaryComparator.

## How was this patch tested?

New test suite.

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

Closes #21570 from jiangxb1987/rbc-test.
2018-06-28 14:19:50 +08:00
jerryshao 33e77fa89b [SPARK-24518][CORE] Using Hadoop credential provider API to store password
## What changes were proposed in this pull request?

In our distribution,  because we don't do such fine-grained access control of config file, also configuration file is world readable shared between different components, so password may leak to different users.

Hadoop credential provider API support storing password in a secure way, in which Spark could read it in a secure way, so here propose to add support of using credential provider API to get password.

## How was this patch tested?

Adding tests and verified locally.

Author: jerryshao <sshao@hortonworks.com>

Closes #21548 from jerryshao/SPARK-24518.
2018-06-22 10:14:12 -07:00
Hieu Huynh 39dfaf2fd1 [SPARK-24519] Make the threshold for highly compressed map status configurable
**Problem**
MapStatus uses hardcoded value of 2000 partitions to determine if it should use highly compressed map status. We should make it configurable to allow users to more easily tune their jobs with respect to this without having for them to modify their code to change the number of partitions.  Note we can leave this as an internal/undocumented config for now until we have more advise for the users on how to set this config.
Some of my reasoning:
The config gives you a way to easily change something without the user having to change code, redeploy jar, and then run again. You can simply change the config and rerun. It also allows for easier experimentation. Changing the # of partitions has other side affects, whether good or bad is situation dependent. It can be worse are you could be increasing # of output files when you don't want to be, affects the # of tasks needs and thus executors to run in parallel, etc.
There have been various talks about this number at spark summits where people have told customers to increase it to be 2001 partitions. Note if you just do a search for spark 2000 partitions you will fine various things all talking about this number.  This shows that people are modifying their code to take this into account so it seems to me having this configurable would be better.
Once we have more advice for users we could expose this and document information on it.

**What changes were proposed in this pull request?**
I make the hardcoded value mentioned above to be configurable under the name _SHUFFLE_MIN_NUM_PARTS_TO_HIGHLY_COMPRESS_, which has default value to be 2000. Users can set it to the value they want by setting the property name _spark.shuffle.minNumPartitionsToHighlyCompress_

**How was this patch tested?**
I wrote a unit test to make sure that the default value is 2000, and  _IllegalArgumentException_ will be thrown if user set it to a non-positive value. The unit test also checks that highly compressed map status is correctly used when the number of partition is greater than _SHUFFLE_MIN_NUM_PARTS_TO_HIGHLY_COMPRESS_.

Author: Hieu Huynh <“Hieu.huynh@oath.com”>

Closes #21527 from hthuynh2/spark_branch_1.
2018-06-22 09:16:14 -05:00
Marcelo Vanzin c8e909cd49 [SPARK-24589][CORE] Correctly identify tasks in output commit coordinator.
When an output stage is retried, it's possible that tasks from the previous
attempt are still running. In that case, there would be a new task for the
same partition in the new attempt, and the coordinator would allow both
tasks to commit their output since it did not keep track of stage attempts.

The change adds more information to the stage state tracked by the coordinator,
so that only one task is allowed to commit the output in the above case.
The stage state in the coordinator is also maintained across stage retries,
so that a stray speculative task from a previous stage attempt is not allowed
to commit.

This also removes some code added in SPARK-18113 that allowed for duplicate
commit requests; with the RPC code used in Spark 2, that situation cannot
happen, so there is no need to handle it.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #21577 from vanzin/SPARK-24552.
2018-06-21 13:25:15 -05:00
“attilapiros” b56e9c613f [SPARK-16630][YARN] Blacklist a node if executors won't launch on it
## What changes were proposed in this pull request?

This change extends YARN resource allocation handling with blacklisting functionality.
This handles cases when node is messed up or misconfigured such that a container won't launch on it. Before this change backlisting only focused on task execution but this change introduces YarnAllocatorBlacklistTracker which tracks allocation failures per host (when enabled via "spark.yarn.blacklist.executor.launch.blacklisting.enabled").

## How was this patch tested?

### With unit tests

Including a new suite: YarnAllocatorBlacklistTrackerSuite.

#### Manually

It was tested on a cluster by deleting the Spark jars on one of the node.

#### Behaviour before these changes

Starting Spark as:
```
spark2-shell --master yarn --deploy-mode client --num-executors 4  --conf spark.executor.memory=4g --conf "spark.yarn.max.executor.failures=6"
```

Log is:
```
18/04/12 06:49:36 INFO yarn.ApplicationMaster: Final app status: FAILED, exitCode: 11, (reason: Max number of executor failures (6) reached)
18/04/12 06:49:39 INFO yarn.ApplicationMaster: Unregistering ApplicationMaster with FAILED (diag message: Max number of executor failures (6) reached)
18/04/12 06:49:39 INFO impl.AMRMClientImpl: Waiting for application to be successfully unregistered.
18/04/12 06:49:39 INFO yarn.ApplicationMaster: Deleting staging directory hdfs://apiros-1.gce.test.com:8020/user/systest/.sparkStaging/application_1523459048274_0016
18/04/12 06:49:39 INFO util.ShutdownHookManager: Shutdown hook called
```

#### Behaviour after these changes

Starting Spark as:
```
spark2-shell --master yarn --deploy-mode client --num-executors 4  --conf spark.executor.memory=4g --conf "spark.yarn.max.executor.failures=6" --conf "spark.yarn.blacklist.executor.launch.blacklisting.enabled=true"
```

And the log is:
```
18/04/13 05:37:43 INFO yarn.YarnAllocator: Will request 1 executor container(s), each with 1 core(s) and 4505 MB memory (including 409 MB of overhead)
18/04/13 05:37:43 INFO yarn.YarnAllocator: Submitted 1 unlocalized container requests.
18/04/13 05:37:43 INFO yarn.YarnAllocator: Launching container container_1523459048274_0025_01_000008 on host apiros-4.gce.test.com for executor with ID 6
18/04/13 05:37:43 INFO yarn.YarnAllocator: Received 1 containers from YARN, launching executors on 1 of them.
18/04/13 05:37:43 INFO yarn.YarnAllocator: Completed container container_1523459048274_0025_01_000007 on host: apiros-4.gce.test.com (state: COMPLETE, exit status: 1)
18/04/13 05:37:43 INFO yarn.YarnAllocatorBlacklistTracker: blacklisting host as YARN allocation failed: apiros-4.gce.test.com
18/04/13 05:37:43 INFO yarn.YarnAllocatorBlacklistTracker: adding nodes to YARN application master's blacklist: List(apiros-4.gce.test.com)
18/04/13 05:37:43 WARN yarn.YarnAllocator: Container marked as failed: container_1523459048274_0025_01_000007 on host: apiros-4.gce.test.com. Exit status: 1. Diagnostics: Exception from container-launch.
Container id: container_1523459048274_0025_01_000007
Exit code: 1
Stack trace: ExitCodeException exitCode=1:
        at org.apache.hadoop.util.Shell.runCommand(Shell.java:604)
        at org.apache.hadoop.util.Shell.run(Shell.java:507)
        at org.apache.hadoop.util.Shell$ShellCommandExecutor.execute(Shell.java:789)
        at org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor.launchContainer(DefaultContainerExecutor.java:213)
        at org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:302)
        at org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:82)
        at java.util.concurrent.FutureTask.run(FutureTask.java:266)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at java.lang.Thread.run(Thread.java:748)
```

Where the most important part is:

```
18/04/13 05:37:43 INFO yarn.YarnAllocatorBlacklistTracker: blacklisting host as YARN allocation failed: apiros-4.gce.test.com
18/04/13 05:37:43 INFO yarn.YarnAllocatorBlacklistTracker: adding nodes to YARN application master's blacklist: List(apiros-4.gce.test.com)
```

And execution was continued (no shutdown called).

### Testing the backlisting of the whole cluster

Starting Spark with YARN blacklisting enabled then removing a the Spark core jar one by one from all the cluster nodes. Then executing a simple spark job which fails checking the yarn log the expected exit status is contained:

```
18/06/15 01:07:10 INFO yarn.ApplicationMaster: Final app status: FAILED, exitCode: 11, (reason: Due to executor failures all available nodes are blacklisted)
18/06/15 01:07:13 INFO util.ShutdownHookManager: Shutdown hook called
```

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

Closes #21068 from attilapiros/SPARK-16630.
2018-06-21 09:17:18 -05:00
Marco Gaido bc111463a7 [SPARK-23778][CORE] Avoid unneeded shuffle when union gets an empty RDD
## What changes were proposed in this pull request?

When a `union` is invoked on several RDDs of which one is an empty RDD, the result of the operation is a `UnionRDD`. This causes an unneeded extra-shuffle when all the other RDDs have the same partitioning.

The PR ignores incoming empty RDDs in the union method.

## How was this patch tested?

added UT

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21333 from mgaido91/SPARK-23778.
2018-06-19 22:29:00 -07:00
Xingbo Jiang 534065efeb [MINOR][CORE][TEST] Remove unnecessary sort in UnsafeInMemorySorterSuite
## What changes were proposed in this pull request?

We don't require specific ordering of the input data, the sort action is not necessary and misleading.

## How was this patch tested?

Existing test suite.

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

Closes #21536 from jiangxb1987/sorterSuite.
2018-06-14 14:20:48 +08:00
Fangshi Li cc88d7fad1 [SPARK-24216][SQL] Spark TypedAggregateExpression uses getSimpleName that is not safe in scala
## What changes were proposed in this pull request?

When user create a aggregator object in scala and pass the aggregator to Spark Dataset's agg() method, Spark's will initialize TypedAggregateExpression with the nodeName field as aggregator.getClass.getSimpleName. However, getSimpleName is not safe in scala environment, depending on how user creates the aggregator object. For example, if the aggregator class full qualified name is "com.my.company.MyUtils$myAgg$2$", the getSimpleName will throw java.lang.InternalError "Malformed class name". This has been reported in scalatest https://github.com/scalatest/scalatest/pull/1044 and discussed in many scala upstream jiras such as SI-8110, SI-5425.

To fix this issue, we follow the solution in https://github.com/scalatest/scalatest/pull/1044 to add safer version of getSimpleName as a util method, and TypedAggregateExpression will invoke this util method rather than getClass.getSimpleName.

## How was this patch tested?
added unit test

Author: Fangshi Li <fli@linkedin.com>

Closes #21276 from fangshil/SPARK-24216.
2018-06-12 12:10:08 -07:00
Xingbo Jiang 8ef167a5f9 [SPARK-24340][CORE] Clean up non-shuffle disk block manager files following executor exits on a Standalone cluster
## What changes were proposed in this pull request?

Currently we only clean up the local directories on application removed. However, when executors die and restart repeatedly, many temp files are left untouched in the local directories, which is undesired behavior and could cause disk space used up gradually.

We can detect executor death in the Worker, and clean up the non-shuffle files (files not ended with ".index" or ".data") in the local directories, we should not touch the shuffle files since they are expected to be used by the external shuffle service.

Scope of this PR is limited to only implement the cleanup logic on a Standalone cluster, we defer to experts familiar with other cluster managers(YARN/Mesos/K8s) to determine whether it's worth to add similar support.

## How was this patch tested?

Add new test suite to cover.

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

Closes #21390 from jiangxb1987/cleanupNonshuffleFiles.
2018-06-01 13:46:05 -07:00
William Sheu 0053e153fa [SPARK-24337][CORE] Improve error messages for Spark conf values
## What changes were proposed in this pull request?

Improve the exception messages when retrieving Spark conf values to include the key name when the value is invalid.

## How was this patch tested?

Unit tests for all get* operations in SparkConf that require a specific value format

Author: William Sheu <william.sheu@databricks.com>

Closes #21454 from PenguinToast/SPARK-24337-spark-config-errors.
2018-05-30 22:37:27 -07:00