Commit graph

2262 commits

Author SHA1 Message Date
erenavsarogullari 73e64f7d50 [SPARK-19662][SCHEDULER][TEST] Add Fair Scheduler Unit Test coverage for different build cases
## What changes were proposed in this pull request?
Fair Scheduler can be built via one of the following options:
- By setting a `spark.scheduler.allocation.file` property,
- By setting `fairscheduler.xml` into classpath.

These options are checked **in order** and fair-scheduler is built via first found option. If invalid path is found, `FileNotFoundException` will be expected.

This PR aims unit test coverage of these use cases and a minor documentation change has been added for second option(`fairscheduler.xml` into classpath) to inform the users.

Also, this PR was related with #16813 and has been created separately to keep patch content as isolated and to help the reviewers.

## How was this patch tested?
Added new Unit Tests.

Author: erenavsarogullari <erenavsarogullari@gmail.com>

Closes #16992 from erenavsarogullari/SPARK-19662.
2017-08-28 14:54:00 -05:00
jerryshao 1813c4a8dd [SPARK-21714][CORE][YARN] Avoiding re-uploading remote resources in yarn client mode
## What changes were proposed in this pull request?

With SPARK-10643, Spark supports download resources from remote in client deploy mode. But the implementation overrides variables which representing added resources (like `args.jars`, `args.pyFiles`) to local path, And yarn client leverage this local path to re-upload resources to distributed cache. This is unnecessary to break the semantics of putting resources in a shared FS. So here proposed to fix it.

## How was this patch tested?

This is manually verified with jars, pyFiles in local and remote storage, both in client and cluster mode.

Author: jerryshao <sshao@hortonworks.com>

Closes #18962 from jerryshao/SPARK-21714.
2017-08-25 09:57:53 -07:00
Sean Owen de7af295c2 [MINOR][BUILD] Fix build warnings and Java lint errors
## What changes were proposed in this pull request?

Fix build warnings and Java lint errors. This just helps a bit in evaluating (new) warnings in another PR I have open.

## How was this patch tested?

Existing tests

Author: Sean Owen <sowen@cloudera.com>

Closes #19051 from srowen/JavaWarnings.
2017-08-25 16:07:13 +01:00
Jane Wang d58a3507ed [SPARK-19326] Speculated task attempts do not get launched in few scenarios
## What changes were proposed in this pull request?

Add a new listener event when a speculative task is created and notify it to ExecutorAllocationManager for requesting more executor.

## How was this patch tested?

- Added Unittests.
- For the test snippet in the jira:
val n = 100
val someRDD = sc.parallelize(1 to n, n)
someRDD.mapPartitionsWithIndex( (index: Int, it: Iterator[Int]) => {
if (index == 1) {
Thread.sleep(Long.MaxValue) // fake long running task(s)
}
it.toList.map(x => index + ", " + x).iterator
}).collect
With this code change, spark indicates 101 jobs are running (99 succeeded, 2 running and 1 is speculative job)

Author: Jane Wang <janewang@fb.com>

Closes #18492 from janewangfb/speculated_task_not_launched.
2017-08-23 11:31:54 +08:00
Sergey Serebryakov 77d046ec47 [SPARK-21782][CORE] Repartition creates skews when numPartitions is a power of 2
## Problem
When an RDD (particularly with a low item-per-partition ratio) is repartitioned to numPartitions = power of 2, the resulting partitions are very uneven-sized, due to using fixed seed to initialize PRNG, and using the PRNG only once. See details in https://issues.apache.org/jira/browse/SPARK-21782

## What changes were proposed in this pull request?
Instead of directly using `0, 1, 2,...` seeds to initialize `Random`, hash them with `scala.util.hashing.byteswap32()`.

## How was this patch tested?
`build/mvn -Dtest=none -DwildcardSuites=org.apache.spark.rdd.RDDSuite test`

Author: Sergey Serebryakov <sserebryakov@tesla.com>

Closes #18990 from megaserg/repartition-skew.
2017-08-21 08:21:25 +01:00
Eyal Farago b8ffb51055 [SPARK-3151][BLOCK MANAGER] DiskStore.getBytes fails for files larger than 2GB
## What changes were proposed in this pull request?
introduced `DiskBlockData`, a new implementation of `BlockData` representing a whole file.
this is somehow related to [SPARK-6236](https://issues.apache.org/jira/browse/SPARK-6236) as well

This class follows the implementation of `EncryptedBlockData` just without the encryption. hence:
* `toInputStream` is implemented using a `FileInputStream` (todo: encrypted version actually uses `Channels.newInputStream`, not sure if it's the right choice for this)
* `toNetty` is implemented in terms of `io.netty.channel.DefaultFileRegion`
* `toByteBuffer` fails for files larger than 2GB (same behavior of the original code, just postponed a bit), it also respects the same configuration keys defined by the original code to choose between memory mapping and simple file read.

## How was this patch tested?
added test to DiskStoreSuite and MemoryManagerSuite

Author: Eyal Farago <eyal@nrgene.com>

Closes #18855 from eyalfa/SPARK-3151.
2017-08-17 09:21:50 +08:00
John Lee adf005dabe [SPARK-21656][CORE] spark dynamic allocation should not idle timeout executors when tasks still to run
## What changes were proposed in this pull request?

Right now spark lets go of executors when they are idle for the 60s (or configurable time). I have seen spark let them go when they are idle but they were really needed. I have seen this issue when the scheduler was waiting to get node locality but that takes longer than the default idle timeout. In these jobs the number of executors goes down really small (less than 10) but there are still like 80,000 tasks to run.
We should consider not allowing executors to idle timeout if they are still needed according to the number of tasks to be run.

## How was this patch tested?

Tested by manually adding executors to `executorsIdsToBeRemoved` list and seeing if those executors were removed when there are a lot of tasks and a high `numExecutorsTarget` value.

Code used

In  `ExecutorAllocationManager.start()`

```
    start_time = clock.getTimeMillis()
```

In `ExecutorAllocationManager.schedule()`
```
    val executorIdsToBeRemoved = ArrayBuffer[String]()
    if ( now > start_time + 1000 * 60 * 2) {
      logInfo("--- REMOVING 1/2 of the EXECUTORS ---")
      start_time +=  1000 * 60 * 100
      var counter = 0
      for (x <- executorIds) {
        counter += 1
        if (counter == 2) {
          counter = 0
          executorIdsToBeRemoved += x
        }
      }
    }

Author: John Lee <jlee2@yahoo-inc.com>

Closes #18874 from yoonlee95/SPARK-21656.
2017-08-16 09:44:09 -05:00
Marcelo Vanzin 3f958a9992 [SPARK-21731][BUILD] Upgrade scalastyle to 0.9.
This version fixes a few issues in the import order checker; it provides
better error messages, and detects more improper ordering (thus the need
to change a lot of files in this patch). The main fix is that it correctly
complains about the order of packages vs. classes.

As part of the above, I moved some "SparkSession" import in ML examples
inside the "$example on$" blocks; that didn't seem consistent across
different source files to start with, and avoids having to add more on/off blocks
around specific imports.

The new scalastyle also seems to have a better header detector, so a few
license headers had to be updated to match the expected indentation.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #18943 from vanzin/SPARK-21731.
2017-08-15 13:59:00 -07:00
Marcelo Vanzin cba826d001 [SPARK-17742][CORE] Handle child process exit in SparkLauncher.
Currently the launcher handle does not monitor the child spark-submit
process it launches; this means that if the child exits with an error,
the handle's state will never change, and an application will not know
that the application has failed.

This change adds code to monitor the child process, and changes the
handle state appropriately when the child process exits.

Tested with added unit tests.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #18877 from vanzin/SPARK-17742.
2017-08-15 11:26:29 -07:00
Andrew Ash 6847e93cf4 [SPARK-21563][CORE] Fix race condition when serializing TaskDescriptions and adding jars
## What changes were proposed in this pull request?

Fix the race condition when serializing TaskDescriptions and adding jars by keeping the set of jars and files for a TaskSet constant across the lifetime of the TaskSet.  Otherwise TaskDescription serialization can produce an invalid serialization when new file/jars are added concurrently as the TaskDescription is serialized.

## How was this patch tested?

Additional unit test ensures jars/files contained in the TaskDescription remain constant throughout the lifetime of the TaskSet.

Author: Andrew Ash <andrew@andrewash.com>

Closes #18913 from ash211/SPARK-21563.
2017-08-14 22:48:08 +08:00
Anderson Osagie 34d2134a9f [SPARK-21176][WEB UI] Format worker page links to work with proxy
## What changes were proposed in this pull request?

Several links on the worker page do not work correctly with the proxy because:
1) They don't acknowledge the proxy
2) They use relative paths (unlike the Application Page which uses full paths)

This patch fixes that. It also fixes a mistake in the proxy's Location header parsing which caused it to incorrectly handle redirects.

## How was this patch tested?

I checked the validity of every link with the proxy on and off.

Author: Anderson Osagie <osagie@gmail.com>

Closes #18915 from aosagie/fix/proxy-links.
2017-08-14 10:00:59 +01:00
10087686 6426adffaf [SPARK-21663][TESTS] test("remote fetch below max RPC message size") should call masterTracker.stop() in MapOutputTrackerSuite
Signed-off-by: 10087686 <wang.jiaochunzte.com.cn>

## What changes were proposed in this pull request?
After Unit tests end,there should be call masterTracker.stop() to free resource;
(Please fill in changes proposed in this fix)

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

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

Author: 10087686 <wang.jiaochun@zte.com.cn>

Closes #18867 from wangjiaochun/mapout.
2017-08-09 18:45:38 +08:00
Anderson Osagie ae8a2b1496 [SPARK-21176][WEB UI] Use a single ProxyServlet to proxy all workers and applications
## What changes were proposed in this pull request?

Currently, each application and each worker creates their own proxy servlet. Each proxy servlet is backed by its own HTTP client and a relatively large number of selector threads. This is excessive but was fixed (to an extent) by https://github.com/apache/spark/pull/18437.

However, a single HTTP client (backed by a single selector thread) should be enough to handle all proxy requests. This PR creates a single proxy servlet no matter how many applications and workers there are.

## How was this patch tested?
.
The unit tests for rewriting proxied locations and headers were updated. I then spun up a 100 node cluster to ensure that proxy'ing worked correctly

jiangxb1987 Please let me know if there's anything else I can do to help push this thru. Thanks!

Author: Anderson Osagie <osagie@gmail.com>

Closes #18499 from aosagie/fix/minimize-proxy-threads.
2017-08-09 14:35:27 +08:00
Xianyang Liu 534a063f7c [SPARK-21621][CORE] Reset numRecordsWritten after DiskBlockObjectWriter.commitAndGet called
## What changes were proposed in this pull request?

We should reset numRecordsWritten to zero after DiskBlockObjectWriter.commitAndGet called.
Because when `revertPartialWritesAndClose` be called, we decrease the written records in `ShuffleWriteMetrics` . However, we decreased the written records to zero, this should be wrong, we should only decreased the number reords after the last `commitAndGet` called.

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

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

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

Closes #18830 from ConeyLiu/DiskBlockObjectWriter.
2017-08-07 17:04:53 +08:00
Marcelo Vanzin 9456176da0 [SPARK-21490][CORE] Make sure SparkLauncher redirects needed streams.
The code was failing to account for some cases when setting up log
redirection. For example, if a user redirected only stdout to a file,
the launcher code would leave stderr without redirection, which could
lead to child processes getting stuck because stderr wasn't being
read.

So detect cases where only one of the streams is redirected, and
redirect the other stream to the log as appropriate.

For the old "launch()" API, redirection of the unconfigured stream
only happens if the user has explicitly requested for log redirection.
Log redirection is on by default with "startApplication()".

Most of the change is actually adding new unit tests to make sure the
different cases work as expected. As part of that, I moved some tests
that were in the core/ module to the launcher/ module instead, since
they don't depend on spark-submit.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #18696 from vanzin/SPARK-21490.
2017-08-02 12:05:55 -07:00
Dongjoon Hyun 14e75758ab [SPARK-21578][CORE] Add JavaSparkContextSuite
## What changes were proposed in this pull request?

Due to SI-8479, [SPARK-1093](https://issues.apache.org/jira/browse/SPARK-21578) introduced redundant [SparkContext constructors](https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/SparkContext.scala#L148-L181). However, [SI-8479](https://issues.scala-lang.org/browse/SI-8479) is already fixed in Scala 2.10.5 and Scala 2.11.1.

The real reason to provide this constructor is that Java code can access `SparkContext` directly. It's Scala behavior, SI-4278. So, this PR adds an explicit testsuite, `JavaSparkContextSuite`  to prevent future regression, and fixes the outdate comment, too.

## How was this patch tested?

Pass the Jenkins with a new test suite.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #18778 from dongjoon-hyun/SPARK-21578.
2017-08-01 21:59:02 -07:00
Wenchen Fan 9f5647d62e [SPARK-21319][SQL] Fix memory leak in sorter
## What changes were proposed in this pull request?

`UnsafeExternalSorter.recordComparator` can be either `KVComparator` or `RowComparator`, and both of them will keep the reference to the input rows they compared last time.

After sorting, we return the sorted iterator to upstream operators. However, the upstream operators may take a while to consume up the sorted iterator, and `UnsafeExternalSorter` is registered to `TaskContext` at [here](https://github.com/apache/spark/blob/v2.2.0/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java#L159-L161), which means we will keep the `UnsafeExternalSorter` instance and keep the last compared input rows in memory until the sorted iterator is consumed up.

Things get worse if we sort within partitions of a dataset and coalesce all partitions into one, as we will keep a lot of input rows in memory and the time to consume up all the sorted iterators is long.

This PR takes over https://github.com/apache/spark/pull/18543 , the idea is that, we do not keep the record comparator instance in `UnsafeExternalSorter`, but a generator of record comparator.

close #18543

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #18679 from cloud-fan/memory-leak.
2017-07-27 22:56:26 +08:00
Eric Vandenberg 06a9793793 [SPARK-21447][WEB UI] Spark history server fails to render compressed
inprogress history file in some cases.

Add failure handling for EOFException that can be thrown during
decompression of an inprogress spark history file, treat same as case
where can't parse the last line.

## What changes were proposed in this pull request?

Failure handling for case of EOFException thrown within the ReplayListenerBus.replay method to handle the case analogous to json parse fail case.  This path can arise in compressed inprogress history files since an incomplete compression block could be read (not flushed by writer on a block boundary).  See the stack trace of this occurrence in the jira ticket (https://issues.apache.org/jira/browse/SPARK-21447)

## How was this patch tested?

Added a unit test that specifically targets validating the failure handling path appropriately when maybeTruncated is true and false.

Author: Eric Vandenberg <ericvandenberg@fb.com>

Closes #18673 from ericvandenbergfb/fix_inprogress_compr_history_file.
2017-07-25 11:45:35 -07:00
Wenchen Fan 3ac6093086 [SPARK-10063] Follow-up: remove dead code related to an old output committer
## What changes were proposed in this pull request?

DirectParquetOutputCommitter was removed from Spark as it was deemed unsafe to use. We however still have some code to generate warning. This patch removes those code as well.

This is kind of a follow-up of https://github.com/apache/spark/pull/16796

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #18689 from cloud-fan/minor.
2017-07-20 12:08:20 -07:00
Dhruve Ashar ef61775586 [SPARK-21243][Core] Limit no. of map outputs in a shuffle fetch
## What changes were proposed in this pull request?
For configurations with external shuffle enabled, we have observed that if a very large no. of blocks are being fetched from a remote host, it puts the NM under extra pressure and can crash it. This change introduces a configuration `spark.reducer.maxBlocksInFlightPerAddress` , to limit the no. of map outputs being fetched from a given remote address. The changes applied here are applicable for both the scenarios - when external shuffle is enabled as well as disabled.

## How was this patch tested?
Ran the job with the default configuration which does not change the existing behavior and ran it with few configurations of lower values -10,20,50,100. The job ran fine and there is no change in the output. (I will update the metrics related to NM in some time.)

Author: Dhruve Ashar <dhruveashar@gmail.com>

Closes #18487 from dhruve/impr/SPARK-21243.
2017-07-19 15:53:28 -05:00
Peng 46307b2cd3 [SPARK-21401][ML][MLLIB] add poll function for BoundedPriorityQueue
## What changes were proposed in this pull request?
The most of BoundedPriorityQueue usages in ML/MLLIB are:
Get the value of BoundedPriorityQueue, then sort it.
For example, in Word2Vec: pq.toSeq.sortBy(-_._2)
in ALS, pq.toArray.sorted()

The test results show using pq.poll is much faster than sort the value.
It is good to add the poll function for BoundedPriorityQueue.

## How was this patch tested?
The existing UT

Author: Peng <peng.meng@intel.com>
Author: Peng Meng <peng.meng@intel.com>

Closes #18620 from mpjlu/add-poll.
2017-07-19 09:56:48 +01:00
Marcelo Vanzin 264b0f36ce [SPARK-21408][CORE] Better default number of RPC dispatch threads.
Instead of using the host's cpu count, use the number of cores allocated
for the Spark process when sizing the RPC dispatch thread pool. This avoids
creating large thread pools on large machines when the number of allocated
cores is small.

Tested by verifying number of threads with spark.executor.cores set
to 1 and 4; same thing for YARN AM.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #18639 from vanzin/SPARK-21408.
2017-07-18 13:36:10 -07:00
jerryshao cde64add18 [SPARK-21411][YARN] Lazily create FS within kerberized UGI to avoid token acquiring failure
## What changes were proposed in this pull request?

In the current `YARNHadoopDelegationTokenManager`, `FileSystem` to which to get tokens are created out of KDC logged UGI, using these `FileSystem` to get new tokens will lead to exception. The main thing is that Spark code trying to get new tokens from the FS created with token auth-ed UGI, but Hadoop can only grant new tokens in kerberized UGI. To fix this issue, we should lazily create these FileSystem within KDC logged UGI.

## How was this patch tested?

Manual verification in secure cluster.

CC vanzin mgummelt please help to review, thanks!

Author: jerryshao <sshao@hortonworks.com>

Closes #18633 from jerryshao/SPARK-21411.
2017-07-18 11:44:01 -07:00
Zhang A Peng 7aac755ba0 [SPARK-21410][CORE] Create less partitions for RangePartitioner if RDD.count() is less than partitions
## What changes were proposed in this pull request?

Fix a bug in RangePartitioner:
In RangePartitioner(partitions: Int, rdd: RDD[]), RangePartitioner.numPartitions is wrong if the number of elements in RDD (rdd.count()) is less than number of partitions (partitions in constructor).

## How was this patch tested?

test as described in [SPARK-SPARK-21410](https://issues.apache.org/jira/browse/SPARK-21410
)

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

Author: Zhang A Peng <zhangap@cn.ibm.com>

Closes #18631 from apapi/fixRangePartitioner.numPartitions.
2017-07-18 09:57:53 +08:00
Kazuaki Ishizaki ac5d5d7959 [SPARK-21344][SQL] BinaryType comparison does signed byte array comparison
## What changes were proposed in this pull request?

This PR fixes a wrong comparison for `BinaryType`. This PR enables unsigned comparison and unsigned prefix generation for an array for `BinaryType`. Previous implementations uses signed operations.

## How was this patch tested?

Added a test suite in `OrderingSuite`.

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

Closes #18571 from kiszk/SPARK-21344.
2017-07-14 20:16:04 -07:00
Sean Owen 425c4ada4c [SPARK-19810][BUILD][CORE] Remove support for Scala 2.10
## What changes were proposed in this pull request?

- Remove Scala 2.10 build profiles and support
- Replace some 2.10 support in scripts with commented placeholders for 2.12 later
- Remove deprecated API calls from 2.10 support
- Remove usages of deprecated context bounds where possible
- Remove Scala 2.10 workarounds like ScalaReflectionLock
- Other minor Scala warning fixes

## How was this patch tested?

Existing tests

Author: Sean Owen <sowen@cloudera.com>

Closes #17150 from srowen/SPARK-19810.
2017-07-13 17:06:24 +08:00
jinxing 97a1aa2c70 [SPARK-21315][SQL] Skip some spill files when generateIterator(startIndex) in ExternalAppendOnlyUnsafeRowArray.
## What changes were proposed in this pull request?

In current code, it is expensive to use `UnboundedFollowingWindowFunctionFrame`, because it is iterating from the start to lower bound every time calling `write` method. When traverse the iterator, it's possible to skip some spilled files thus to save some time.

## How was this patch tested?

Added unit test

Did a small test for benchmark:

Put 2000200 rows into `UnsafeExternalSorter`-- 2 spill files(each contains 1000000 rows) and inMemSorter contains 200 rows.
Move the iterator forward to index=2000001.

*With this change*:
`getIterator(2000001)`, it will cost almost 0ms~1ms;
*Without this change*:
`for(int i=0; i<2000001; i++)geIterator().loadNext()`, it will cost 300ms.

Author: jinxing <jinxing6042@126.com>

Closes #18541 from jinxing64/SPARK-21315.
2017-07-11 11:47:47 +08:00
jinxing 6a06c4b03c [SPARK-21342] Fix DownloadCallback to work well with RetryingBlockFetcher.
## What changes were proposed in this pull request?

When `RetryingBlockFetcher` retries fetching blocks. There could be two `DownloadCallback`s download the same content to the same target file. It could cause `ShuffleBlockFetcherIterator` reading a partial result.

This pr proposes to create and delete the tmp files in `OneForOneBlockFetcher`

Author: jinxing <jinxing6042@126.com>
Author: Shixiong Zhu <zsxwing@gmail.com>

Closes #18565 from jinxing64/SPARK-21342.
2017-07-10 21:06:58 +08:00
Eric Vandenberg 96d58f285b [SPARK-21219][CORE] Task retry occurs on same executor due to race condition with blacklisting
## What changes were proposed in this pull request?

There's a race condition in the current TaskSetManager where a failed task is added for retry (addPendingTask), and can asynchronously be assigned to an executor *prior* to the blacklist state (updateBlacklistForFailedTask), the result is the task might re-execute on the same executor.  This is particularly problematic if the executor is shutting down since the retry task immediately becomes a lost task (ExecutorLostFailure).  Another side effect is that the actual failure reason gets obscured by the retry task which never actually executed.  There are sample logs showing the issue in the https://issues.apache.org/jira/browse/SPARK-21219

The fix is to change the ordering of the addPendingTask and updatingBlackListForFailedTask calls in TaskSetManager.handleFailedTask

## How was this patch tested?

Implemented a unit test that verifies the task is black listed before it is added to the pending task.  Ran the unit test without the fix and it fails.  Ran the unit test with the fix and it passes.

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

Author: Eric Vandenberg <ericvandenberg@fb.com>

Closes #18427 from ericvandenbergfb/blacklistFix.
2017-07-10 14:40:20 +08:00
Marcelo Vanzin 9131bdb7e1 [SPARK-20342][CORE] Update task accumulators before sending task end event.
This makes sures that listeners get updated task information; otherwise it's
possible to write incomplete task information into event logs, for example,
making the information in a replayed UI inconsistent with the original
application.

Added a new unit test to try to detect the problem, but it's not guaranteed
to fail since it's a race; but it fails pretty reliably for me without the
scheduler changes.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #18393 from vanzin/SPARK-20342.try2.
2017-07-09 00:24:54 +08:00
Marcelo Vanzin 9760c15acb [SPARK-20379][CORE] Allow SSL config to reference env variables.
This change exposes the internal code path in SparkConf that allows
configs to be read with variable substitution applied, and uses that
new method in SSLOptions so that SSL configs can reference other
variables, and more importantly, environment variables, providing
a secure way to provide passwords to Spark when using SSL.

The approach is a little bit hacky, but is the smallest change possible.
Otherwise, the concept of "namespaced configs" would have to be added
to the config system, which would create a lot of noise for not much
gain at this point.

Tested with added unit tests, and on a real cluster with SSL enabled.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #18394 from vanzin/SPARK-20379.try2.
2017-07-08 14:20:09 +08:00
jerryshao 5800144a54 [SPARK-21012][SUBMIT] Add glob support for resources adding to Spark
Current "--jars (spark.jars)", "--files (spark.files)", "--py-files (spark.submit.pyFiles)" and "--archives (spark.yarn.dist.archives)" only support non-glob path. This is OK for most of the cases, but when user requires to add more jars, files into Spark, it is too verbose to list one by one. So here propose to add glob path support for resources.

Also improving the code of downloading resources.

## How was this patch tested?

UT added, also verified manually in local cluster.

Author: jerryshao <sshao@hortonworks.com>

Closes #18235 from jerryshao/SPARK-21012.
2017-07-06 15:32:49 +08:00
Shixiong Zhu ab866f1173 [SPARK-21248][SS] The clean up codes in StreamExecution should not be interrupted
## What changes were proposed in this pull request?

This PR uses `runUninterruptibly` to avoid that the clean up codes in StreamExecution is interrupted. It also removes an optimization in `runUninterruptibly` to make sure this method never throw `InterruptedException`.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #18461 from zsxwing/SPARK-21248.
2017-07-05 18:26:28 -07:00
he.qiao e3e2b5da36 [SPARK-21286][TEST] Modified StorageTabSuite unit test
## What changes were proposed in this pull request?
The old unit test not effect

## How was this patch tested?
unit test

Author: he.qiao <he.qiao17@zte.com.cn>

Closes #18511 from Geek-He/dev_0703.
2017-07-05 21:13:25 +08:00
liuxian 6657e00de3 [SPARK-21283][CORE] FileOutputStream should be created as append mode
## What changes were proposed in this pull request?

`FileAppender` is used to write `stderr` and `stdout` files  in `ExecutorRunner`, But before writing `ErrorStream` into the the `stderr` file, the header information has been written into ,if  FileOutputStream is  not created as append mode, the  header information will be lost

## How was this patch tested?
unit test case

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

Closes #18507 from 10110346/wip-lx-0703.
2017-07-04 09:16:40 +08:00
Devaraj K 6beca9ce94 [SPARK-21170][CORE] Utils.tryWithSafeFinallyAndFailureCallbacks throws IllegalArgumentException: Self-suppression not permitted
## What changes were proposed in this pull request?

Not adding the exception to the suppressed if it is the same instance as originalThrowable.

## How was this patch tested?

Added new tests to verify this, these tests fail without source code changes and passes with the change.

Author: Devaraj K <devaraj@apache.org>

Closes #18384 from devaraj-kavali/SPARK-21170.
2017-07-01 15:53:49 +01:00
Xingbo Jiang 3c2fc19d47 [SPARK-18294][CORE] Implement commit protocol to support mapred package's committer
## What changes were proposed in this pull request?

This PR makes the following changes:

- Implement a new commit protocol `HadoopMapRedCommitProtocol` which support the old `mapred` package's committer;
- Refactor SparkHadoopWriter and SparkHadoopMapReduceWriter, now they are combined together, thus we can support write through both mapred and mapreduce API by the new SparkHadoopWriter, a lot of duplicated codes are removed.

After this change, it should be pretty easy for us to support the committer from both the new and the old hadoop API at high level.

## How was this patch tested?
No major behavior change, passed the existing test cases.

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

Closes #18438 from jiangxb1987/SparkHadoopWriter.
2017-06-30 20:30:26 +08:00
jinxing d106a74c53 [SPARK-21240] Fix code style for constructing and stopping a SparkContext in UT.
## What changes were proposed in this pull request?

Same with SPARK-20985.
Fix code style for constructing and stopping a `SparkContext`. Assure the context is stopped to avoid other tests complain that there's only one `SparkContext` can exist.

Author: jinxing <jinxing6042@126.com>

Closes #18454 from jinxing64/SPARK-21240.
2017-06-29 09:59:36 +01:00
Sital Kedia a946be35ac [SPARK-3577] Report Spill size on disk for UnsafeExternalSorter
## What changes were proposed in this pull request?

Report Spill size on disk for UnsafeExternalSorter

## How was this patch tested?

Tested by running a job on cluster and verify the spill size on disk.

Author: Sital Kedia <skedia@fb.com>

Closes #17471 from sitalkedia/fix_disk_spill_size.
2017-06-29 14:25:51 +08:00
jerryshao 9e50a1d37a [SPARK-13669][SPARK-20898][CORE] Improve the blacklist mechanism to handle external shuffle service unavailable situation
## What changes were proposed in this pull request?

Currently we are running into an issue with Yarn work preserving enabled + external shuffle service.
In the work preserving enabled scenario, the failure of NM will not lead to the exit of executors, so executors can still accept and run the tasks. The problem here is when NM is failed, external shuffle service is actually inaccessible, so reduce tasks will always complain about the “Fetch failure”, and the failure of reduce stage will make the parent stage (map stage) rerun. The tricky thing here is Spark scheduler is not aware of the unavailability of external shuffle service, and will reschedule the map tasks on the executor where NM is failed, and again reduce stage will be failed with “Fetch failure”, and after 4 retries, the job is failed. This could also apply to other cluster manager with external shuffle service.

So here the main problem is that we should avoid assigning tasks to those bad executors (where shuffle service is unavailable). Current Spark's blacklist mechanism could blacklist executors/nodes by failure tasks, but it doesn't handle this specific fetch failure scenario. So here propose to improve the current application blacklist mechanism to handle fetch failure issue (especially with external shuffle service unavailable issue), to blacklist the executors/nodes where shuffle fetch is unavailable.

## How was this patch tested?

Unit test and small cluster verification.

Author: jerryshao <sshao@hortonworks.com>

Closes #17113 from jerryshao/SPARK-13669.
2017-06-26 11:14:03 -05:00
Thomas Graves 5b5a69bea9 [SPARK-20923] turn tracking of TaskMetrics._updatedBlockStatuses off
## What changes were proposed in this pull request?
Turn tracking of TaskMetrics._updatedBlockStatuses off by default. As far as I can see its not used by anything and it uses a lot of memory when caching and processing a lot of blocks.  In my case it was taking 5GB of a 10GB heap and I even went up to 50GB heap and the job still ran out of memory.  With this change in place the same job easily runs in less then 10GB of heap.

We leave the api there as well as a config to turn it back on just in case anyone is using it.  TaskMetrics is exposed via SparkListenerTaskEnd so if users are relying on it they can turn it back on.

## How was this patch tested?

Ran unit tests that were modified and manually tested on a couple of jobs (with and without caching).  Clicked through the UI and didn't see anything missing.
Ran my very large hive query job with 200,000 small tasks, 1000 executors, cached 6+TB of data this runs fine now whereas without this change it would go into full gcs and eventually die.

Author: Thomas Graves <tgraves@thirteenroutine.corp.gq1.yahoo.com>
Author: Tom Graves <tgraves@yahoo-inc.com>

Closes #18162 from tgravescs/SPARK-20923.
2017-06-23 09:19:02 +08:00
jinxing 58434acdd8 [SPARK-19937] Collect metrics for remote bytes read to disk during shuffle.
In current code(https://github.com/apache/spark/pull/16989), big blocks are shuffled to disk.
This pr proposes to collect metrics for remote bytes fetched to disk.

Author: jinxing <jinxing6042@126.com>

Closes #18249 from jinxing64/SPARK-19937.
2017-06-22 14:10:51 -07:00
Xingbo Jiang 2dadea95c8 [SPARK-20832][CORE] Standalone master should explicitly inform drivers of worker deaths and invalidate external shuffle service outputs
## What changes were proposed in this pull request?

In standalone mode, master should explicitly inform each active driver of any worker deaths, so the invalid external shuffle service outputs on the lost host would be removed from the shuffle mapStatus, thus we can avoid future `FetchFailure`s.

## How was this patch tested?
Manually tested by the following steps:
1. Start a standalone Spark cluster with one driver node and two worker nodes;
2. Run a Job with ShuffleMapStage, ensure the outputs distribute on each worker;
3. Run another Job to make all executors exit, but the workers are all alive;
4. Kill one of the workers;
5. Run rdd.collect(), before this change, we should see `FetchFailure`s and failed Stages, while after the change, the job should complete without failure.

Before the change:
![image](https://user-images.githubusercontent.com/4784782/27335366-c251c3d6-55fe-11e7-99dd-d1fdcb429210.png)

After the change:
![image](https://user-images.githubusercontent.com/4784782/27335393-d1c71640-55fe-11e7-89ed-bd760f1f39af.png)

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

Closes #18362 from jiangxb1987/removeWorker.
2017-06-22 20:48:12 +08:00
Li Yichao d107b3b910 [SPARK-20640][CORE] Make rpc timeout and retry for shuffle registration configurable.
## What changes were proposed in this pull request?

Currently the shuffle service registration timeout and retry has been hardcoded. This works well for small workloads but under heavy workload when the shuffle service is busy transferring large amount of data we see significant delay in responding to the registration request, as a result we often see the executors fail to register with the shuffle service, eventually failing the job. We need to make these two parameters configurable.

## How was this patch tested?

* Updated `BlockManagerSuite` to test registration timeout and max attempts configuration actually works.

cc sitalkedia

Author: Li Yichao <lyc@zhihu.com>

Closes #18092 from liyichao/SPARK-20640.
2017-06-21 21:54:29 +08:00
Yuming Wang 9b57cd8d5c [SPARK-21133][CORE] Fix HighlyCompressedMapStatus#writeExternal throws NPE
## What changes were proposed in this pull request?

Fix HighlyCompressedMapStatus#writeExternal NPE:
```
17/06/18 15:00:27 ERROR Utils: Exception encountered
java.lang.NullPointerException
        at org.apache.spark.scheduler.HighlyCompressedMapStatus$$anonfun$writeExternal$2.apply$mcV$sp(MapStatus.scala:171)
        at org.apache.spark.scheduler.HighlyCompressedMapStatus$$anonfun$writeExternal$2.apply(MapStatus.scala:167)
        at org.apache.spark.scheduler.HighlyCompressedMapStatus$$anonfun$writeExternal$2.apply(MapStatus.scala:167)
        at org.apache.spark.util.Utils$.tryOrIOException(Utils.scala:1303)
        at org.apache.spark.scheduler.HighlyCompressedMapStatus.writeExternal(MapStatus.scala:167)
        at java.io.ObjectOutputStream.writeExternalData(ObjectOutputStream.java:1459)
        at java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1430)
        at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1178)
        at java.io.ObjectOutputStream.writeArray(ObjectOutputStream.java:1378)
        at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1174)
        at java.io.ObjectOutputStream.writeObject(ObjectOutputStream.java:348)
        at org.apache.spark.MapOutputTracker$$anonfun$serializeMapStatuses$1.apply$mcV$sp(MapOutputTracker.scala:617)
        at org.apache.spark.MapOutputTracker$$anonfun$serializeMapStatuses$1.apply(MapOutputTracker.scala:616)
        at org.apache.spark.MapOutputTracker$$anonfun$serializeMapStatuses$1.apply(MapOutputTracker.scala:616)
        at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1337)
        at org.apache.spark.MapOutputTracker$.serializeMapStatuses(MapOutputTracker.scala:619)
        at org.apache.spark.MapOutputTrackerMaster.getSerializedMapOutputStatuses(MapOutputTracker.scala:562)
        at org.apache.spark.MapOutputTrackerMaster$MessageLoop.run(MapOutputTracker.scala:351)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
        at java.lang.Thread.run(Thread.java:745)
17/06/18 15:00:27 ERROR MapOutputTrackerMaster: java.lang.NullPointerException
java.io.IOException: java.lang.NullPointerException
        at org.apache.spark.util.Utils$.tryOrIOException(Utils.scala:1310)
        at org.apache.spark.scheduler.HighlyCompressedMapStatus.writeExternal(MapStatus.scala:167)
        at java.io.ObjectOutputStream.writeExternalData(ObjectOutputStream.java:1459)
        at java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1430)
        at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1178)
        at java.io.ObjectOutputStream.writeArray(ObjectOutputStream.java:1378)
        at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1174)
        at java.io.ObjectOutputStream.writeObject(ObjectOutputStream.java:348)
        at org.apache.spark.MapOutputTracker$$anonfun$serializeMapStatuses$1.apply$mcV$sp(MapOutputTracker.scala:617)
        at org.apache.spark.MapOutputTracker$$anonfun$serializeMapStatuses$1.apply(MapOutputTracker.scala:616)
        at org.apache.spark.MapOutputTracker$$anonfun$serializeMapStatuses$1.apply(MapOutputTracker.scala:616)
        at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1337)
        at org.apache.spark.MapOutputTracker$.serializeMapStatuses(MapOutputTracker.scala:619)
        at org.apache.spark.MapOutputTrackerMaster.getSerializedMapOutputStatuses(MapOutputTracker.scala:562)
        at org.apache.spark.MapOutputTrackerMaster$MessageLoop.run(MapOutputTracker.scala:351)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
        at java.lang.Thread.run(Thread.java:745)
Caused by: java.lang.NullPointerException
        at org.apache.spark.scheduler.HighlyCompressedMapStatus$$anonfun$writeExternal$2.apply$mcV$sp(MapStatus.scala:171)
        at org.apache.spark.scheduler.HighlyCompressedMapStatus$$anonfun$writeExternal$2.apply(MapStatus.scala:167)
        at org.apache.spark.scheduler.HighlyCompressedMapStatus$$anonfun$writeExternal$2.apply(MapStatus.scala:167)
        at org.apache.spark.util.Utils$.tryOrIOException(Utils.scala:1303)
        ... 17 more
17/06/18 15:00:27 INFO MapOutputTrackerMasterEndpoint: Asked to send map output locations for shuffle 0 to 10.17.47.20:50188
17/06/18 15:00:27 ERROR Utils: Exception encountered
java.lang.NullPointerException
        at org.apache.spark.scheduler.HighlyCompressedMapStatus$$anonfun$writeExternal$2.apply$mcV$sp(MapStatus.scala:171)
        at org.apache.spark.scheduler.HighlyCompressedMapStatus$$anonfun$writeExternal$2.apply(MapStatus.scala:167)
        at org.apache.spark.scheduler.HighlyCompressedMapStatus$$anonfun$writeExternal$2.apply(MapStatus.scala:167)
        at org.apache.spark.util.Utils$.tryOrIOException(Utils.scala:1303)
        at org.apache.spark.scheduler.HighlyCompressedMapStatus.writeExternal(MapStatus.scala:167)
        at java.io.ObjectOutputStream.writeExternalData(ObjectOutputStream.java:1459)
        at java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1430)
        at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1178)
        at java.io.ObjectOutputStream.writeArray(ObjectOutputStream.java:1378)
        at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1174)
        at java.io.ObjectOutputStream.writeObject(ObjectOutputStream.java:348)
        at org.apache.spark.MapOutputTracker$$anonfun$serializeMapStatuses$1.apply$mcV$sp(MapOutputTracker.scala:617)
        at org.apache.spark.MapOutputTracker$$anonfun$serializeMapStatuses$1.apply(MapOutputTracker.scala:616)
        at org.apache.spark.MapOutputTracker$$anonfun$serializeMapStatuses$1.apply(MapOutputTracker.scala:616)
        at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1337)
        at org.apache.spark.MapOutputTracker$.serializeMapStatuses(MapOutputTracker.scala:619)
        at org.apache.spark.MapOutputTrackerMaster.getSerializedMapOutputStatuses(MapOutputTracker.scala:562)
        at org.apache.spark.MapOutputTrackerMaster$MessageLoop.run(MapOutputTracker.scala:351)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
        at java.lang.Thread.run(Thread.java:745)
```

## How was this patch tested?

manual tests

Author: Yuming Wang <wgyumg@gmail.com>

Closes #18343 from wangyum/SPARK-21133.
2017-06-20 09:22:30 +08:00
Xingbo Jiang ea542d29b2 [SPARK-19824][CORE] Update JsonProtocol to keep consistent with the UI
## What changes were proposed in this pull request?

Fix any inconsistent part in JsonProtocol with the UI.
This PR also contains the modifications in #17181

## How was this patch tested?

Updated JsonProtocolSuite.

Before this change, localhost:8080/json shows:
```
{
  "url" : "spark://xingbos-MBP.local:7077",
  "workers" : [ {
    "id" : "worker-20170615172946-192.168.0.101-49450",
    "host" : "192.168.0.101",
    "port" : 49450,
    "webuiaddress" : "http://192.168.0.101:8081",
    "cores" : 8,
    "coresused" : 8,
    "coresfree" : 0,
    "memory" : 15360,
    "memoryused" : 1024,
    "memoryfree" : 14336,
    "state" : "ALIVE",
    "lastheartbeat" : 1497519481722
  }, {
    "id" : "worker-20170615172948-192.168.0.101-49452",
    "host" : "192.168.0.101",
    "port" : 49452,
    "webuiaddress" : "http://192.168.0.101:8082",
    "cores" : 8,
    "coresused" : 8,
    "coresfree" : 0,
    "memory" : 15360,
    "memoryused" : 1024,
    "memoryfree" : 14336,
    "state" : "ALIVE",
    "lastheartbeat" : 1497519484160
  }, {
    "id" : "worker-20170615172951-192.168.0.101-49469",
    "host" : "192.168.0.101",
    "port" : 49469,
    "webuiaddress" : "http://192.168.0.101:8083",
    "cores" : 8,
    "coresused" : 8,
    "coresfree" : 0,
    "memory" : 15360,
    "memoryused" : 1024,
    "memoryfree" : 14336,
    "state" : "ALIVE",
    "lastheartbeat" : 1497519486905
  } ],
  "cores" : 24,
  "coresused" : 24,
  "memory" : 46080,
  "memoryused" : 3072,
  "activeapps" : [ {
    "starttime" : 1497519426990,
    "id" : "app-20170615173706-0001",
    "name" : "Spark shell",
    "user" : "xingbojiang",
    "memoryperslave" : 1024,
    "submitdate" : "Thu Jun 15 17:37:06 CST 2017",
    "state" : "RUNNING",
    "duration" : 65362
  } ],
  "completedapps" : [ {
    "starttime" : 1497519250893,
    "id" : "app-20170615173410-0000",
    "name" : "Spark shell",
    "user" : "xingbojiang",
    "memoryperslave" : 1024,
    "submitdate" : "Thu Jun 15 17:34:10 CST 2017",
    "state" : "FINISHED",
    "duration" : 116895
  } ],
  "activedrivers" : [ ],
  "status" : "ALIVE"
}
```

After the change:
```
{
  "url" : "spark://xingbos-MBP.local:7077",
  "workers" : [ {
    "id" : "worker-20170615175032-192.168.0.101-49951",
    "host" : "192.168.0.101",
    "port" : 49951,
    "webuiaddress" : "http://192.168.0.101:8081",
    "cores" : 8,
    "coresused" : 8,
    "coresfree" : 0,
    "memory" : 15360,
    "memoryused" : 1024,
    "memoryfree" : 14336,
    "state" : "ALIVE",
    "lastheartbeat" : 1497520292900
  }, {
    "id" : "worker-20170615175034-192.168.0.101-49953",
    "host" : "192.168.0.101",
    "port" : 49953,
    "webuiaddress" : "http://192.168.0.101:8082",
    "cores" : 8,
    "coresused" : 8,
    "coresfree" : 0,
    "memory" : 15360,
    "memoryused" : 1024,
    "memoryfree" : 14336,
    "state" : "ALIVE",
    "lastheartbeat" : 1497520280301
  }, {
    "id" : "worker-20170615175037-192.168.0.101-49955",
    "host" : "192.168.0.101",
    "port" : 49955,
    "webuiaddress" : "http://192.168.0.101:8083",
    "cores" : 8,
    "coresused" : 8,
    "coresfree" : 0,
    "memory" : 15360,
    "memoryused" : 1024,
    "memoryfree" : 14336,
    "state" : "ALIVE",
    "lastheartbeat" : 1497520282884
  } ],
  "aliveworkers" : 3,
  "cores" : 24,
  "coresused" : 24,
  "memory" : 46080,
  "memoryused" : 3072,
  "activeapps" : [ {
    "id" : "app-20170615175122-0001",
    "starttime" : 1497520282115,
    "name" : "Spark shell",
    "cores" : 24,
    "user" : "xingbojiang",
    "memoryperslave" : 1024,
    "submitdate" : "Thu Jun 15 17:51:22 CST 2017",
    "state" : "RUNNING",
    "duration" : 10805
  } ],
  "completedapps" : [ {
    "id" : "app-20170615175058-0000",
    "starttime" : 1497520258766,
    "name" : "Spark shell",
    "cores" : 24,
    "user" : "xingbojiang",
    "memoryperslave" : 1024,
    "submitdate" : "Thu Jun 15 17:50:58 CST 2017",
    "state" : "FINISHED",
    "duration" : 9876
  } ],
  "activedrivers" : [ ],
  "completeddrivers" : [ ],
  "status" : "ALIVE"
}
```

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

Closes #18303 from jiangxb1987/json-protocol.
2017-06-18 22:05:06 -07:00
liuxian 112bd9bfc5 [SPARK-21090][CORE] Optimize the unified memory manager code
## What changes were proposed in this pull request?
1.In `acquireStorageMemory`, when the Memory Mode is OFF_HEAP ,the `maxOffHeapMemory` should be modified to `maxOffHeapStorageMemory`. after this PR,it will same as ON_HEAP Memory Mode.
Because when acquire memory is between `maxOffHeapStorageMemory` and `maxOffHeapMemory`,it will fail surely, so if acquire memory is greater than  `maxOffHeapStorageMemory`(not greater than `maxOffHeapMemory`),we should fail fast.
2. Borrow memory from execution, `numBytes` modified to `numBytes - storagePool.memoryFree` will be more reasonable.
Because we just acquire `(numBytes - storagePool.memoryFree)`, unnecessary borrowed `numBytes` from execution

## How was this patch tested?
added unit test case

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

Closes #18296 from 10110346/wip-lx-0614.
2017-06-19 11:46:58 +08:00
zuotingbing edcb878e2f [SPARK-20338][CORE] Spaces in spark.eventLog.dir are not correctly handled
## What changes were proposed in this pull request?

“spark.eventLog.dir” supports with space characters.

1. Update EventLoggingListenerSuite like `testDir = Utils.createTempDir(namePrefix = s"history log")`
2. Fix EventLoggingListenerSuite tests

## How was this patch tested?

update unit tests

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

Closes #18285 from zuotingbing/spark-resolveURI.
2017-06-16 10:34:52 -07:00
jerryshao 2837b14cdc [SPARK-12552][FOLLOWUP] Fix flaky test for "o.a.s.deploy.master.MasterSuite.master correctly recover the application"
## What changes were proposed in this pull request?

Due to the RPC asynchronous event processing, The test "correctly recover the application" could potentially be failed. The issue could be found in here: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/78126/testReport/org.apache.spark.deploy.master/MasterSuite/master_correctly_recover_the_application/.

So here fixing this flaky test.

## How was this patch tested?

Existing UT.

CC cloud-fan jiangxb1987 , please help to review, thanks!

Author: jerryshao <sshao@hortonworks.com>

Closes #18321 from jerryshao/SPARK-12552-followup.
2017-06-16 14:24:15 +08:00
Michael Gummelt a18d637112 [SPARK-20434][YARN][CORE] Move Hadoop delegation token code from yarn to core
## What changes were proposed in this pull request?

Move Hadoop delegation token code from `spark-yarn` to `spark-core`, so that other schedulers (such as Mesos), may use it.  In order to avoid exposing Hadoop interfaces in spark-core, the new Hadoop delegation token classes are kept private.  In order to provider backward compatiblity, and to allow YARN users to continue to load their own delegation token providers via Java service loading, the old YARN interfaces, as well as the client code that uses them, have been retained.

Summary:
- Move registered `yarn.security.ServiceCredentialProvider` classes from `spark-yarn` to `spark-core`.  Moved them into a new, private hierarchy under `HadoopDelegationTokenProvider`.  Client code in `HadoopDelegationTokenManager` now loads credentials from a whitelist of three providers (`HadoopFSDelegationTokenProvider`, `HiveDelegationTokenProvider`, `HBaseDelegationTokenProvider`), instead of service loading, which means that users are not able to implement their own delegation token providers, as they are in the `spark-yarn` module.

- The `yarn.security.ServiceCredentialProvider` interface has been kept for backwards compatibility, and to continue to allow YARN users to implement their own delegation token provider implementations.  Client code in YARN now fetches tokens via the new `YARNHadoopDelegationTokenManager` class, which fetches tokens from the core providers through `HadoopDelegationTokenManager`, as well as service loads them from `yarn.security.ServiceCredentialProvider`.

Old Hierarchy:

```
yarn.security.ServiceCredentialProvider (service loaded)
  HadoopFSCredentialProvider
  HiveCredentialProvider
  HBaseCredentialProvider
yarn.security.ConfigurableCredentialManager
```

New Hierarchy:

```
HadoopDelegationTokenManager
HadoopDelegationTokenProvider (not service loaded)
  HadoopFSDelegationTokenProvider
  HiveDelegationTokenProvider
  HBaseDelegationTokenProvider

yarn.security.ServiceCredentialProvider (service loaded)
yarn.security.YARNHadoopDelegationTokenManager
```
## How was this patch tested?

unit tests

Author: Michael Gummelt <mgummelt@mesosphere.io>
Author: Dr. Stefan Schimanski <sttts@mesosphere.io>

Closes #17723 from mgummelt/SPARK-20434-refactor-kerberos.
2017-06-15 11:46:00 -07:00
Xingbo Jiang 7dc3e697c7 [SPARK-16251][SPARK-20200][CORE][TEST] Flaky test: org.apache.spark.rdd.LocalCheckpointSuite.missing checkpoint block fails with informative message
## What changes were proposed in this pull request?

Currently we don't wait to confirm the removal of the block from the slave's BlockManager, if the removal takes too much time, we will fail the assertion in this test case.
The failure can be easily reproduced if we sleep for a while before we remove the block in BlockManagerSlaveEndpoint.receiveAndReply().

## How was this patch tested?
N/A

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

Closes #18314 from jiangxb1987/LocalCheckpointSuite.
2017-06-16 00:06:54 +08:00
Li Yichao 2924674406 [SPARK-19900][CORE] Remove driver when relaunching.
This is https://github.com/apache/spark/pull/17888 .

Below are some spark ui snapshots.

Master, after worker disconnects:

<img width="1433" alt="master_disconnect" src="https://cloud.githubusercontent.com/assets/2576762/26398687/d0ee228e-40ac-11e7-986d-d3b57b87029f.png">

Master, after worker reconnects, notice the `running drivers` part:

<img width="1412" alt="master_reconnects" src="https://cloud.githubusercontent.com/assets/2576762/26398697/d50735a4-40ac-11e7-80d8-6e9e1cf0b62f.png">

This patch, after worker disconnects:
<img width="1412" alt="patch_disconnect" src="https://cloud.githubusercontent.com/assets/2576762/26398009/c015d3dc-40aa-11e7-8bb4-df11a1f66645.png">

This patch, after worker reconnects:
![image](https://cloud.githubusercontent.com/assets/2576762/26398037/d313769c-40aa-11e7-8613-5f157d193150.png)

cc cloud-fan jiangxb1987

Author: Li Yichao <lyc@zhihu.com>

Closes #18084 from liyichao/SPARK-19900-1.
2017-06-15 08:08:26 +08:00
Sean Owen d6f76eb346 [SPARK-21057][ML] Do not use a PascalDistribution in countApprox
## What changes were proposed in this pull request?

Use Poisson analysis for approx count in all cases.

## How was this patch tested?

Existing tests.

Author: Sean Owen <sowen@cloudera.com>

Closes #18276 from srowen/SPARK-21057.
2017-06-14 09:01:20 +01:00
Sital Kedia dccc0aa3cf [SPARK-19753][CORE] Un-register all shuffle output on a host in case of slave lost or fetch failure
## What changes were proposed in this pull request?

Currently, when we detect fetch failure, we only remove the shuffle files produced by the executor, while the host itself might be down and all the shuffle files are not accessible. In case we are running multiple executors on a host, any host going down currently results in multiple fetch failures and multiple retries of the stage, which is very inefficient. If we remove all the shuffle files on that host, on first fetch failure, we can rerun all the tasks on that host in a single stage retry.

## How was this patch tested?

Unit testing and also ran a job on the cluster and made sure multiple retries are gone.

Author: Sital Kedia <skedia@fb.com>
Author: Imran Rashid <irashid@cloudera.com>

Closes #18150 from sitalkedia/cleanup_shuffle.
2017-06-14 11:34:09 +08:00
jerryshao 9eb095243b [SPARK-12552][CORE] Correctly count the driver resource when recovering from failure for Master
Currently in Standalone HA mode, the resource usage of driver is not correctly counted in Master when recovering from failure, this will lead to some unexpected behaviors like negative value in UI.

So here fix this to also count the driver's resource usage.

Also changing the recovered app's state to `RUNNING` when fully recovered. Previously it will always be WAITING even fully recovered.

andrewor14 please help to review, thanks a lot.

Author: jerryshao <sshao@hortonworks.com>

Closes #10506 from jerryshao/SPARK-12552.
2017-06-14 08:12:15 +08:00
DjvuLee b36ce2a246 [SPARK-21064][CORE][TEST] Fix the default value bug in NettyBlockTransferServiceSuite
## What changes were proposed in this pull request?

The default value for `spark.port.maxRetries` is 100,
but we use 10 in the suite file.
So we change it to 100 to avoid test failure.

## How was this patch tested?
No test

Author: DjvuLee <lihu@bytedance.com>

Closes #18280 from djvulee/NettyTestBug.
2017-06-13 15:56:03 +01:00
liuxian 2aaed0a4db [SPARK-21006][TESTS][FOLLOW-UP] Some Worker's RpcEnv is leaked in WorkerSuite
## What changes were proposed in this pull request?

Create rpcEnv and run later needs shutdown. as #18226

## How was this patch tested?
unit test

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

Closes #18259 from 10110346/wip-lx-0610.
2017-06-13 12:29:50 +01:00
Felix Cheung 278ba7a2c6 [TEST][SPARKR][CORE] Fix broken SparkSubmitSuite
## What changes were proposed in this pull request?

Fix test file path. This is broken in #18264 and undetected since R-only changes don't build core and subsequent post-commit with the change built fine (again because it wasn't building core)

actually appveyor builds everything but it's not running scala suites ...

## How was this patch tested?

jenkins
srowen gatorsmile

Author: Felix Cheung <felixcheung_m@hotmail.com>

Closes #18283 from felixcheung/rsubmitsuite.
2017-06-12 22:08:49 -07:00
Josh Rosen 3476390c6e [SPARK-20715] Store MapStatuses only in MapOutputTracker, not ShuffleMapStage
## What changes were proposed in this pull request?

This PR refactors `ShuffleMapStage` and `MapOutputTracker` in order to simplify the management of `MapStatuses`, reduce driver memory consumption, and remove a potential source of scheduler correctness bugs.

### Background

In Spark there are currently two places where MapStatuses are tracked:

- The `MapOutputTracker` maintains an `Array[MapStatus]` storing a single location for each map output. This mapping is used by the `DAGScheduler` for determining reduce-task locality preferences (when locality-aware reduce task scheduling is enabled) and is also used to serve map output locations to executors / tasks.
- Each `ShuffleMapStage` also contains a mapping of `Array[List[MapStatus]]` which holds the complete set of locations where each map output could be available. This mapping is used to determine which map tasks need to be run when constructing `TaskSets` for the stage.

This duplication adds complexity and creates the potential for certain types of correctness bugs.  Bad things can happen if these two copies of the map output locations get out of sync. For instance, if the `MapOutputTracker` is missing locations for a map output but `ShuffleMapStage` believes that locations are available then tasks will fail with `MetadataFetchFailedException` but `ShuffleMapStage` will not be updated to reflect the missing map outputs, leading to situations where the stage will be reattempted (because downstream stages experienced fetch failures) but no task sets will be launched (because `ShuffleMapStage` thinks all maps are available).

I observed this behavior in a real-world deployment. I'm still not quite sure how the state got out of sync in the first place, but we can completely avoid this class of bug if we eliminate the duplicate state.

### Why we only need to track a single location for each map output

I think that storing an `Array[List[MapStatus]]` in `ShuffleMapStage` is unnecessary.

First, note that this adds memory/object bloat to the driver we need one extra `List` per task. If you have millions of tasks across all stages then this can add up to be a significant amount of resources.

Secondly, I believe that it's extremely uncommon that these lists will ever contain more than one entry. It's not impossible, but is very unlikely given the conditions which must occur for that to happen:

- In normal operation (no task failures) we'll only run each task once and thus will have at most one output.
- If speculation is enabled then it's possible that we'll have multiple attempts of a task. The TaskSetManager will [kill duplicate attempts of a task](04901dd03a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala (L717)) after a task finishes successfully, reducing the likelihood that both the original and speculated task will successfully register map outputs.
- There is a [comment in `TaskSetManager`](04901dd03a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala (L113)) which suggests that running tasks are not killed if a task set becomes a zombie. However:
  - If the task set becomes a zombie due to the job being cancelled then it doesn't matter whether we record map outputs.
  - If the task set became a zombie because of a stage failure (e.g. the map stage itself had a fetch failure from an upstream match stage) then I believe that the "failedEpoch" will be updated which may cause map outputs from still-running tasks to [be ignored](04901dd03a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala (L1213)). (I'm not 100% sure on this point, though).
- Even if you _do_ manage to record multiple map outputs for a stage, only a single map output is reported to / tracked by the MapOutputTracker. The only situation where the additional output locations could actually be read or used would be if a task experienced a `FetchFailure` exception. The most likely cause of a `FetchFailure` exception is an executor lost, which will have most likely caused the loss of several map tasks' output, so saving on potential re-execution of a single map task isn't a huge win if we're going to have to recompute several other lost map outputs from other tasks which ran on that lost executor. Also note that the re-population of MapOutputTracker state from state in the ShuffleMapTask only happens after the reduce stage has failed; the additional location doesn't help to prevent FetchFailures but, instead, can only reduce the amount of work when recomputing missing parent stages.

Given this, this patch chooses to do away with tracking multiple locations for map outputs and instead stores only a single location. This change removes the main distinction between the `ShuffleMapTask` and `MapOutputTracker`'s copies of this state, paving the way for storing it only in the `MapOutputTracker`.

### Overview of other changes

- Significantly simplified the cache / lock management inside of the `MapOutputTrackerMaster`:
  - The old code had several parallel `HashMap`s which had to be guarded by maps of `Object`s which were used as locks. This code was somewhat complicated to follow.
  - The new code uses a new `ShuffleStatus` class to group together all of the state associated with a particular shuffle, including cached serialized map statuses, significantly simplifying the logic.
- Moved more code out of the shared `MapOutputTracker` abstract base class and into the `MapOutputTrackerMaster` and `MapOutputTrackerWorker` subclasses. This makes it easier to reason about which functionality needs to be supported only on the driver or executor.
- Removed a bunch of code from the `DAGScheduler` which was used to synchronize information from the `MapOutputTracker` to `ShuffleMapStage`.
- Added comments to clarify the role of `MapOutputTrackerMaster`'s `epoch` in invalidating executor-side shuffle map output caches.

I will comment on these changes via inline GitHub review comments.

/cc hvanhovell and rxin (whom I discussed this with offline), tgravescs (who recently worked on caching of serialized MapOutputStatuses), and kayousterhout and markhamstra (for scheduler changes).

## How was this patch tested?

Existing tests. I purposely avoided making interface / API which would require significant updates or modifications to test code.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #17955 from JoshRosen/map-output-tracker-rewrite.
2017-06-11 18:34:12 -07:00
Joseph K. Bradley 5a3371883a [SPARK-14408][CORE] Changed RDD.treeAggregate to use fold instead of reduce
## What changes were proposed in this pull request?

Previously, `RDD.treeAggregate` used `reduceByKey` and `reduce` in its implementation, neither of which technically allows the `seq`/`combOps` to modify and return their first arguments.

This PR uses `foldByKey` and `fold` instead and notes that `aggregate` and `treeAggregate` are semantically identical in the Scala doc.

Note that this had some test failures by unknown reasons. This was actually fixed in e3554605b3.

The root cause was, the `zeroValue` now becomes `AFTAggregator` and it compares `totalCnt` (where the value is actually 0). It starts merging one by one and it keeps returning `this` where `totalCnt` is 0. So, this looks not the bug in the current change.

This is now fixed in the commit. So, this should pass the tests.

## How was this patch tested?

Test case added in `RDDSuite`.

Closes #12217

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

Closes #18198 from HyukjinKwon/SPARK-14408.
2017-06-09 08:53:18 +01:00
Josh Rosen 2a23cdd078 [SPARK-20863] Add metrics/instrumentation to LiveListenerBus
## What changes were proposed in this pull request?

This patch adds Coda Hale metrics for instrumenting the `LiveListenerBus` in order to track the number of events received, dropped, and processed. In addition, it adds per-SparkListener-subclass timers to track message processing time. This is useful for identifying when slow third-party SparkListeners cause performance bottlenecks.

See the new `LiveListenerBusMetrics` for a complete description of the new metrics.

## How was this patch tested?

New tests in SparkListenerSuite, including a test to ensure proper counting of dropped listener events.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #18083 from JoshRosen/listener-bus-metrics.
2017-06-08 18:08:25 -07:00
10087686 9be7945861 [SPARK-21006][TESTS] Create rpcEnv and run later needs shutdown and awaitTermination
Signed-off-by: 10087686 <wang.jiaochunzte.com.cn>

## What changes were proposed in this pull request?
When  run test("port conflict") case, we need run anotherEnv.shutdown() and anotherEnv.awaitTermination() for free resource.
(Please fill in changes proposed in this fix)

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

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

Author: 10087686 <wang.jiaochun@zte.com.cn>

Closes #18226 from wangjiaochun/master.
2017-06-08 10:58:09 +01:00
jinxing 44de108d74 [SPARK-20985] Stop SparkContext using LocalSparkContext.withSpark
## What changes were proposed in this pull request?
SparkContext should always be stopped after using, thus other tests won't complain that there's only one `SparkContext` can exist.

Author: jinxing <jinxing6042@126.com>

Closes #18204 from jinxing64/SPARK-20985.
2017-06-06 11:14:39 +01:00
jerryshao 06c0544113 [SPARK-20981][SPARKSUBMIT] Add new configuration spark.jars.repositories as equivalence of --repositories
## What changes were proposed in this pull request?

In our use case of launching Spark applications via REST APIs (Livy), there's no way for user to specify command line arguments, all Spark configurations are set through configurations map. For "--repositories" because there's no equivalent Spark configuration, so we cannot specify the custom repository through configuration.

So here propose to add "--repositories" equivalent configuration in Spark.

## How was this patch tested?

New UT added.

Author: jerryshao <sshao@hortonworks.com>

Closes #18201 from jerryshao/SPARK-20981.
2017-06-05 11:06:50 -07:00
zuotingbing 887cf0ec33 [SPARK-20936][CORE] Lack of an important case about the test of resolveURI in UtilsSuite, and add it as needed.
## What changes were proposed in this pull request?
1.  add `assert(resolve(before) === after)` to check before and after in test of resolveURI.
the function `assertResolves(before: String, after: String)` have two params, it means we should check the before value whether equals the after value which we want.
e.g. the after value of Utils.resolveURI("hdfs:///root/spark.jar#app.jar").toString should be "hdfs:///root/spark.jar#app.jar" rather than "hdfs:/root/spark.jar#app.jar". we need `assert(resolve(before) === after)` to make it more safe.
2. identify the cases between resolveURI and resolveURIs.
3. delete duplicate cases and some small fix make this suit more clear.

## How was this patch tested?

unit tests

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

Closes #18158 from zuotingbing/spark-UtilsSuite.
2017-06-03 09:56:35 +01:00
Dongjoon Hyun 34661d8a5a [SPARK-20708][CORE] Make addExclusionRules up-to-date
## What changes were proposed in this pull request?

Since [SPARK-9263](https://issues.apache.org/jira/browse/SPARK-9263), `resolveMavenCoordinates` ignores Spark and Spark's dependencies by using `addExclusionRules`. This PR aims to make [addExclusionRules](https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala#L956-L974) up-to-date to neglect correctly because it fails to neglect some components like the following.

**mllib (correct)**
```
$ bin/spark-shell --packages org.apache.spark:spark-mllib_2.11:2.1.1
...
---------------------------------------------------------------------
|                  |            modules            ||   artifacts   |
|       conf       | number| search|dwnlded|evicted|| number|dwnlded|
---------------------------------------------------------------------
|      default     |   0   |   0   |   0   |   0   ||   0   |   0   |
---------------------------------------------------------------------
```

**mllib-local (wrong)**
```
$ bin/spark-shell --packages org.apache.spark:spark-mllib-local_2.11:2.1.1
...
---------------------------------------------------------------------
|                  |            modules            ||   artifacts   |
|       conf       | number| search|dwnlded|evicted|| number|dwnlded|
---------------------------------------------------------------------
|      default     |   15  |   2   |   2   |   0   ||   15  |   2   |
---------------------------------------------------------------------
```

## How was this patch tested?

Pass the Jenkins with a updated test case.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #17947 from dongjoon-hyun/SPARK-20708.
2017-05-31 22:39:25 -07:00
jerryshao 5854f77ce1 [SPARK-20244][CORE] Handle incorrect bytesRead metrics when using PySpark
## What changes were proposed in this pull request?

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

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

## How was this patch tested?

Unit test and local cluster verification.

Author: jerryshao <sshao@hortonworks.com>

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

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

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

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

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

Author: jinxing <jinxing6042@126.com>

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

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

Author: jinxing <jinxing6042@126.com>

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

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

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

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

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

## How was this patch tested?

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

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

Author: Yu Peng <loneknightpy@gmail.com>

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

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

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

## How was this patch tested?

a new test

Author: Wenchen Fan <wenchen@databricks.com>

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

This PR proposes two things:

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

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

- Fix path vs URI related test failures on Windows.

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

## How was this patch tested?

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

Author: hyukjinkwon <gurwls223@gmail.com>

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

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

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

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

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

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

Author: jinxing <jinxing6042@126.com>

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

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

## How was this patch tested?

Add new failing regression test case in `RDDSuite`.

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

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

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

## How was this patch tested?

- Unit tests and manual tests.

Author: James Shuster <jshuster@palantir.com>

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

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

## How was this patch tested?

Added test in MapStatusSuite.

Author: jinxing <jinxing6042@126.com>

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

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

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

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

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

## How was this patch tested?

manual tests

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

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

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

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

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

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

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

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

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

In

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

I ran

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

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

**Before:**

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

**After:**

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

## How was this patch tested?

Benchmarks described above.

Author: Josh Rosen <joshrosen@databricks.com>

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

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

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

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

## How was this patch tested?

The new added unit test.

Author: Shixiong Zhu <shixiong@databricks.com>

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

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

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

## How was this patch tested?

New unit tests.

Author: Shixiong Zhu <shixiong@databricks.com>

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

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

## How was this patch tested?

Existing tests.

Author: Sean Owen <sowen@cloudera.com>

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

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

## How was this patch tested?

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

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

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

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

## How was this patch tested?

Only changes code comments, no testing required

Author: Michael Mior <mmior@uwaterloo.ca>

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

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

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

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

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

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

Author: Sanket <schintap@untilservice-lm>

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

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

## How was this patch tested?

New test case.

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

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

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

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

## How was this patch tested?

The new unit test.

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

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

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

## How was this patch tested?

Existing tests

Author: Sean Owen <sowen@cloudera.com>

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

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

## How was this patch tested?

Existing tests.

Author: Ryan Blue <blue@apache.org>

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

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

Closes #17024

## How was this patch tested?

The new unit test.

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

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

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

Running the commands below:

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

produces ...

**Before**

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

**After**

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

## How was this patch tested?

Unit tests in `LocalDirsSuite.scala`.

Author: hyukjinkwon <gurwls223@gmail.com>

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

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

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

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

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

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

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

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

## How was this patch tested?

The new added test.

Author: Shixiong Zhu <shixiong@databricks.com>

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

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

## How was this patch tested?

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

Author: Mark Grover <mark@apache.org>

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

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

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

## How was this patch tested?

Existing UT and local verification.

CC squito and tgravescs .

Author: jerryshao <sshao@hortonworks.com>

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

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

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

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

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

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

Closes #15125 from dding3/cp2_pregel.
2017-04-25 11:20:32 -07:00