Commit graph

6132 commits

Author SHA1 Message Date
Josh Rosen 5952ad2b40 [SPARK-21444] Be more defensive when removing broadcasts in MapOutputTracker
## What changes were proposed in this pull request?

In SPARK-21444, sitalkedia reported an issue where the `Broadcast.destroy()` call in `MapOutputTracker`'s `ShuffleStatus.invalidateSerializedMapOutputStatusCache()` was failing with an `IOException`, causing the DAGScheduler to crash and bring down the entire driver.

This is a bug introduced by #17955. In the old code, we removed a broadcast variable by calling `BroadcastManager.unbroadcast` with `blocking=false`, but the new code simply calls `Broadcast.destroy()` which is capable of failing with an IOException in case certain blocking RPCs time out.

The fix implemented here is to replace this with a call to `destroy(blocking = false)` and to wrap the entire operation in `Utils.tryLogNonFatalError`.

## How was this patch tested?

I haven't written regression tests for this because it's really hard to inject mocks to simulate RPC failures here. Instead, this class of issue is probably best uncovered with more generalized error injection / network unreliability / fuzz testing tools.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #18662 from JoshRosen/SPARK-21444.
2017-07-17 20:40:32 -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
John Lee 0e07a29cf4 [SPARK-21321][SPARK CORE] Spark very verbose on shutdown
## What changes were proposed in this pull request?

The current code is very verbose on shutdown.

The changes I propose is to change the log level when the driver is shutting down and the RPC connections are closed (RpcEnvStoppedException).

## How was this patch tested?

Tested with word count(deploy-mode = cluster, master = yarn, num-executors = 4) with 300GB of data.

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

Closes #18547 from yoonlee95/SPARK-21321.
2017-07-17 13:13:35 -05: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
Stavros Kontopoulos d8257b99dd [SPARK-21403][MESOS] fix --packages for mesos
## What changes were proposed in this pull request?
Fixes --packages flag for mesos in cluster mode. Probably I will handle standalone and Yarn in another commit, I need to investigate those cases as they are different.

## How was this patch tested?
Tested with a community 1.9 dc/os cluster. packages were successfully resolved in cluster mode within a container.

andrewor14  susanxhuynh ArtRand srowen  pls review.

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

Closes #18587 from skonto/fix_packages_mesos_cluster.
2017-07-13 10:37:15 -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
Devaraj K e16e8c7ad3 [SPARK-21146][CORE] Master/Worker should handle and shutdown when any thread gets UncaughtException
## What changes were proposed in this pull request?

Adding the default UncaughtExceptionHandler to the Worker.

## How was this patch tested?

I verified it manually, when any of the worker thread gets uncaught exceptions then the default UncaughtExceptionHandler will handle those exceptions.

Author: Devaraj K <devaraj@apache.org>

Closes #18357 from devaraj-kavali/SPARK-21146.
2017-07-12 00:14:58 -07: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
jinxing 062c336d06 [SPARK-21343] Refine the document for spark.reducer.maxReqSizeShuffleToMem.
## What changes were proposed in this pull request?

In current code, reducer can break the old shuffle service when `spark.reducer.maxReqSizeShuffleToMem` is enabled. Let's refine document.

Author: jinxing <jinxing6042@126.com>

Closes #18566 from jinxing64/SPARK-21343.
2017-07-09 00:27:58 +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
CodingCat fbbe37ed41 [SPARK-19358][CORE] LiveListenerBus shall log the event name when dropping them due to a fully filled queue
## What changes were proposed in this pull request?

Some dropped event will make the whole application behaves unexpectedly, e.g. some UI problem...we shall log the dropped event name to facilitate the debugging

## How was this patch tested?

Existing tests

Author: CodingCat <zhunansjtu@gmail.com>

Closes #16697 from CodingCat/SPARK-19358.
2017-07-07 20:10:24 +08:00
Takuya UESHIN 53c2eb59b2 [SPARK-21327][SQL][PYSPARK] ArrayConstructor should handle an array of typecode 'l' as long rather than int in Python 2.
## What changes were proposed in this pull request?

Currently `ArrayConstructor` handles an array of typecode `'l'` as `int` when converting Python object in Python 2 into Java object, so if the value is larger than `Integer.MAX_VALUE` or smaller than `Integer.MIN_VALUE` then the overflow occurs.

```python
import array
data = [Row(longarray=array.array('l', [-9223372036854775808, 0, 9223372036854775807]))]
df = spark.createDataFrame(data)
df.show(truncate=False)
```

```
+----------+
|longarray |
+----------+
|[0, 0, -1]|
+----------+
```

This should be:

```
+----------------------------------------------+
|longarray                                     |
+----------------------------------------------+
|[-9223372036854775808, 0, 9223372036854775807]|
+----------------------------------------------+
```

## How was this patch tested?

Added a test and existing tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #18553 from ueshin/issues/SPARK-21327.
2017-07-07 14:05:22 +09:00
caoxuewen 565e7a8d4a [SPARK-20950][CORE] add a new config to diskWriteBufferSize which is hard coded before
## What changes were proposed in this pull request?

This PR Improvement in two:
1.With spark.shuffle.spill.diskWriteBufferSize configure diskWriteBufferSize of ShuffleExternalSorter.
    when change the size of the diskWriteBufferSize to test `forceSorterToSpill`
    The average performance of running 10 times is as follows:(their unit is MS).
```
diskWriteBufferSize:       1M    512K    256K    128K    64K    32K    16K    8K    4K
---------------------------------------------------------------------------------------
RecordSize = 2.5M          742   722     694     686     667    668    671    669   683
RecordSize = 1M            294   293     292     287     283    285    281    279   285
```

2.Remove outputBufferSizeInBytes and inputBufferSizeInBytes to initialize in mergeSpillsWithFileStream function.

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

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

Closes #18174 from heary-cao/buffersize.
2017-07-06 19:49:34 +08:00
Liang-Chi Hsieh 6ff05a66fe [SPARK-20703][SQL] Associate metrics with data writes onto DataFrameWriter operations
## What changes were proposed in this pull request?

Right now in the UI, after SPARK-20213, we can show the operations to write data out. However, there is no way to associate metrics with data writes. We should show relative metrics on the operations.

#### Supported commands

This change supports updating metrics for file-based data writing operations, including `InsertIntoHadoopFsRelationCommand`, `InsertIntoHiveTable`.

Supported metrics:

* number of written files
* number of dynamic partitions
* total bytes of written data
* total number of output rows
* average writing data out time (ms)
* (TODO) min/med/max number of output rows per file/partition
* (TODO) min/med/max bytes of written data per file/partition

####  Commands not supported

`InsertIntoDataSourceCommand`, `SaveIntoDataSourceCommand`:

The two commands uses DataSource APIs to write data out, i.e., the logic of writing data out is delegated to the DataSource implementations, such as  `InsertableRelation.insert` and `CreatableRelationProvider.createRelation`. So we can't obtain metrics from delegated methods for now.

`CreateHiveTableAsSelectCommand`, `CreateDataSourceTableAsSelectCommand` :

The two commands invokes other commands to write data out. The invoked commands can even write to non file-based data source. We leave them as future TODO.

#### How to update metrics of writing files out

A `RunnableCommand` which wants to update metrics, needs to override its `metrics` and provide the metrics data structure to `ExecutedCommandExec`.

The metrics are prepared during the execution of `FileFormatWriter`. The callback function passed to `FileFormatWriter` will accept the metrics and update accordingly.

There is a metrics updating function in `RunnableCommand`. In runtime, the function will be bound to the spark context and `metrics` of `ExecutedCommandExec` and pass to `FileFormatWriter`.

## How was this patch tested?

Updated unit tests.

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

Closes #18159 from viirya/SPARK-20703-2.
2017-07-06 15:47: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
Dongjoon Hyun c8d0aba198 [SPARK-21278][PYSPARK] Upgrade to Py4J 0.10.6
## What changes were proposed in this pull request?

This PR aims to bump Py4J in order to fix the following float/double bug.
Py4J 0.10.5 fixes this (https://github.com/bartdag/py4j/issues/272) and the latest Py4J is 0.10.6.

**BEFORE**
```
>>> df = spark.range(1)
>>> df.select(df['id'] + 17.133574204226083).show()
+--------------------+
|(id + 17.1335742042)|
+--------------------+
|       17.1335742042|
+--------------------+
```

**AFTER**
```
>>> df = spark.range(1)
>>> df.select(df['id'] + 17.133574204226083).show()
+-------------------------+
|(id + 17.133574204226083)|
+-------------------------+
|       17.133574204226083|
+-------------------------+
```

## How was this patch tested?

Manual.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #18546 from dongjoon-hyun/SPARK-21278.
2017-07-05 16:33:23 -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
hyukjinkwon 2b1e94b9ad [MINOR][SPARK SUBMIT] Print out R file usage in spark-submit
## What changes were proposed in this pull request?

Currently, running the shell below:

```bash
$ ./bin/spark-submit tmp.R a b c
```

with R file, `tmp.R` as below:

```r
#!/usr/bin/env Rscript

library(SparkR)
sparkRSQL.init(sparkR.init(master = "local"))
collect(createDataFrame(list(list(1))))
print(commandArgs(trailingOnly = TRUE))
```

working fine as below:

```bash
  _1
1  1
[1] "a" "b" "c"
```

However, it looks not printed in usage documentation as below:

```bash
$ ./bin/spark-submit
```

```
Usage: spark-submit [options] <app jar | python file> [app arguments]
...
```

For `./bin/sparkR`, it looks fine as below:

```bash
$ ./bin/sparkR tmp.R
```

```
Running R applications through 'sparkR' is not supported as of Spark 2.0.
Use ./bin/spark-submit <R file>
```

Running the script below:

```bash
$ ./bin/spark-submit
```

**Before**

```
Usage: spark-submit [options] <app jar | python file> [app arguments]
...
```

**After**

```
Usage: spark-submit [options] <app jar | python file | R file> [app arguments]
...
```

## How was this patch tested?

Manually tested.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #18505 from HyukjinKwon/minor-doc-summit.
2017-07-04 12:18:42 +01: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
Sean Owen a9339db99f [SPARK-21137][CORE] Spark reads many small files slowly
## What changes were proposed in this pull request?

Parallelize FileInputFormat.listStatus in Hadoop API via LIST_STATUS_NUM_THREADS to speed up examination of file sizes for wholeTextFiles et al

## How was this patch tested?

Existing tests, which will exercise the key path here: using a local file system.

Author: Sean Owen <sowen@cloudera.com>

Closes #18441 from srowen/SPARK-21137.
2017-07-03 19:52:39 +08:00
guoxiaolong d913db16a0 [SPARK-21250][WEB-UI] Add a url in the table of 'Running Executors' in worker page to visit job page.
## What changes were proposed in this pull request?

Add a url in the table of 'Running Executors' in worker page to visit job page.

When I click URL of 'Name', the current page jumps to the job page. Of course this is only in the table of 'Running Executors'.

This URL of 'Name' is in the table of 'Finished Executors' does not exist, the click will not jump to any page.

fix before:
![1](https://user-images.githubusercontent.com/26266482/27679397-30ddc262-5ceb-11e7-839b-0889d1f42480.png)

fix after:
![2](https://user-images.githubusercontent.com/26266482/27679405-3588ef12-5ceb-11e7-9756-0a93815cd698.png)

## How was this patch tested?
manual tests

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

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

Closes #18464 from guoxiaolongzte/SPARK-21250.
2017-07-03 13:31:01 +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
Liang-Chi Hsieh fd13255225 [SPARK-21052][SQL][FOLLOW-UP] Add hash map metrics to join
## What changes were proposed in this pull request?

Remove `numHashCollisions` in `BytesToBytesMap`. And change `getAverageProbesPerLookup()` to `getAverageProbesPerLookup` as suggested.

## How was this patch tested?

Existing tests.

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

Closes #18480 from viirya/SPARK-21052-followup.
2017-06-30 15:11:27 -07:00
曾林西 1fe08d62f0 [SPARK-21223] Change fileToAppInfo in FsHistoryProvider to fix concurrent issue.
# What issue does this PR address ?
Jira:https://issues.apache.org/jira/browse/SPARK-21223
fix the Thread-safety issue in FsHistoryProvider
Currently, Spark HistoryServer use a HashMap named fileToAppInfo in class FsHistoryProvider to store the map of eventlog path and attemptInfo.
When use ThreadPool to Replay the log files in the list and merge the list of old applications with new ones, multi thread may update fileToAppInfo at the same time, which may cause Thread-safety issues, such as  falling into an infinite loop because of calling resize func of the hashtable.

Author: 曾林西 <zenglinxi@meituan.com>

Closes #18430 from zenglinxi0615/master.
2017-06-30 19:28:43 +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
IngoSchuster 88a536babf [SPARK-21176][WEB UI] Limit number of selector threads for admin ui proxy servlets to 8
## What changes were proposed in this pull request?
Please see also https://issues.apache.org/jira/browse/SPARK-21176

This change limits the number of selector threads that jetty creates to maximum 8 per proxy servlet (Jetty default is number of processors / 2).
The newHttpClient for Jettys ProxyServlet class is overwritten to avoid the Jetty defaults (which are designed for high-performance http servers).
Once https://github.com/eclipse/jetty.project/issues/1643 is available, the code could be cleaned up to avoid the method override.

I really need this on v2.1.1 - what is the best way for a backport automatic merge works fine)? Shall I create another PR?

## How was this patch tested?
(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)
The patch was tested manually on a Spark cluster with a head node that has 88 processors using JMX to verify that the number of selector threads is now limited to 8 per proxy.

gurvindersingh zsxwing can you please review the change?

Author: IngoSchuster <ingo.schuster@de.ibm.com>
Author: Ingo Schuster <ingo.schuster@de.ibm.com>

Closes #18437 from IngoSchuster/master.
2017-06-30 11:16:09 +08:00
Shixiong Zhu 80f7ac3a60 [SPARK-21253][CORE] Disable spark.reducer.maxReqSizeShuffleToMem
## What changes were proposed in this pull request?

Disable spark.reducer.maxReqSizeShuffleToMem because it breaks the old shuffle service.

Credits to wangyum

Closes #18466

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>
Author: Yuming Wang <wgyumg@gmail.com>

Closes #18467 from zsxwing/SPARK-21253.
2017-06-30 11:02:22 +08:00
Feng Liu f9151bebca [SPARK-21188][CORE] releaseAllLocksForTask should synchronize the whole method
## What changes were proposed in this pull request?

Since the objects `readLocksByTask`, `writeLocksByTask` and `info`s are coupled and supposed to be modified by other threads concurrently, all the read and writes of them in the method `releaseAllLocksForTask` should be protected by a single synchronized block like other similar methods.

## How was this patch tested?

existing tests

Author: Feng Liu <fengliu@databricks.com>

Closes #18400 from liufengdb/synchronize.
2017-06-29 16:03:15 -07:00
杨治国10192065 29bd251dd5 [SPARK-21225][CORE] Considering CPUS_PER_TASK when allocating task slots for each WorkerOffer
JIRA Issue:https://issues.apache.org/jira/browse/SPARK-21225
    In the function "resourceOffers", It declare a variable "tasks" for storage the tasks which have allocated a executor. It declared like this:
`val tasks = shuffledOffers.map(o => new ArrayBuffer[TaskDescription](o.cores))`
    But, I think this code only conside a situation for that one task per core. If the user set "spark.task.cpus" as 2 or 3, It really don't need so much Mem. I think It can motify as follow:
val tasks = shuffledOffers.map(o => new ArrayBuffer[TaskDescription](o.cores / CPUS_PER_TASK))
 to instead.
    Motify like this the other earning is that it's more easy to understand the way how the tasks allocate offers.

Author: 杨治国10192065 <yang.zhiguo@zte.com.cn>

Closes #18435 from JackYangzg/motifyTaskCoreDisp.
2017-06-29 20:53:48 +08:00
fjh100456 d7da2b94d6 [SPARK-21135][WEB UI] On history server page,duration of incompleted applications should be hidden instead of showing up as 0
## What changes were proposed in this pull request?

Hide duration of incompleted applications.

## How was this patch tested?

manual tests

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

Closes #18351 from fjh100456/master.
2017-06-29 10:01:12 +01: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
Eric Vandenberg 2d686a19e3 [SPARK-21155][WEBUI] Add (? running tasks) into Spark UI progress
## What changes were proposed in this pull request?

Add metric on number of running tasks to status bar on Jobs / Active Jobs.

## How was this patch tested?

Run a long running (1 minute) query in spark-shell and use localhost:4040 web UI to observe progress.  See jira for screen snapshot.

Author: Eric Vandenberg <ericvandenberg@fb.com>

Closes #18369 from ericvandenbergfb/runningTasks.
2017-06-28 09:26:33 +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
Marcelo Vanzin bfd73a7c48 [SPARK-21159][CORE] Don't try to connect to launcher in standalone cluster mode.
Monitoring for standalone cluster mode is not implemented (see SPARK-11033), but
the same scheduler implementation is used, and if it tries to connect to the
launcher it will fail. So fix the scheduler so it only tries that in client mode;
cluster mode applications will be correctly launched and will work, but monitoring
through the launcher handle will not be available.

Tested by running a cluster mode app with "SparkLauncher.startApplication".

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #18397 from vanzin/SPARK-21159.
2017-06-24 13:23:43 +08:00
10129659 acd208ee50 [SPARK-21115][CORE] If the cores left is less than the coresPerExecutor,the cores left will not be allocated, so it should not to check in every schedule
## What changes were proposed in this pull request?
If we start an app with the param --total-executor-cores=4 and spark.executor.cores=3, the cores left is always 1, so it will try to allocate executors in the function org.apache.spark.deploy.master.startExecutorsOnWorkers in every schedule.
Another question is, is it will be better to allocate another executor with 1 core for the cores left.

## How was this patch tested?
unit test

Author: 10129659 <chen.yanshan@zte.com.cn>

Closes #18322 from eatoncys/leftcores.
2017-06-23 20:53:26 +08: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
sjarvie ba78514da7 [SPARK-21125][PYTHON] Extend setJobDescription to PySpark and JavaSpark APIs
## What changes were proposed in this pull request?

Extend setJobDescription to PySpark and JavaSpark APIs

SPARK-21125

## How was this patch tested?

Testing was done by running a local Spark shell on the built UI. I originally had added a unit test but the PySpark context cannot easily access the Scala Spark Context's private variable with the Job Description key so I omitted the test, due to the simplicity of this addition.

Also ran the existing tests.

# Misc

This contribution is my original work and that I license the work to the project under the project's open source license.

Author: sjarvie <sjarvie@uber.com>

Closes #18332 from sjarvie/add_python_set_job_description.
2017-06-21 10:51:45 -07: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
Xingbo Jiang ef1622899f [SPARK-20989][CORE] Fail to start multiple workers on one host if external shuffle service is enabled in standalone mode
## What changes were proposed in this pull request?

In standalone mode, if we enable external shuffle service by setting `spark.shuffle.service.enabled` to true, and then we try to start multiple workers on one host(by setting `SPARK_WORKER_INSTANCES=3` in spark-env.sh, and then run `sbin/start-slaves.sh`), we can only launch one worker on each host successfully and the rest of the workers fail to launch.
The reason is the port of external shuffle service if configed by `spark.shuffle.service.port`, so currently we could start no more than one external shuffle service on each host. In our case, each worker tries to start a external shuffle service, and only one of them succeeded doing this.

We should give explicit reason of failure instead of fail silently.

## How was this patch tested?
Manually test by the following steps:
1. SET `SPARK_WORKER_INSTANCES=1` in `conf/spark-env.sh`;
2. SET `spark.shuffle.service.enabled` to `true` in `conf/spark-defaults.conf`;
3. Run `sbin/start-all.sh`.

Before the change, you will see no error in the command line, as the following:
```
starting org.apache.spark.deploy.master.Master, logging to /Users/xxx/workspace/spark/logs/spark-xxx-org.apache.spark.deploy.master.Master-1-xxx.local.out
localhost: starting org.apache.spark.deploy.worker.Worker, logging to /Users/xxx/workspace/spark/logs/spark-xxx-org.apache.spark.deploy.worker.Worker-1-xxx.local.out
localhost: starting org.apache.spark.deploy.worker.Worker, logging to /Users/xxx/workspace/spark/logs/spark-xxx-org.apache.spark.deploy.worker.Worker-2-xxx.local.out
localhost: starting org.apache.spark.deploy.worker.Worker, logging to /Users/xxx/workspace/spark/logs/spark-xxx-org.apache.spark.deploy.worker.Worker-3-xxx.local.out
```
And you can see in the webUI that only one worker is running.

After the change, you get explicit error messages in the command line:
```
starting org.apache.spark.deploy.master.Master, logging to /Users/xxx/workspace/spark/logs/spark-xxx-org.apache.spark.deploy.master.Master-1-xxx.local.out
localhost: starting org.apache.spark.deploy.worker.Worker, logging to /Users/xxx/workspace/spark/logs/spark-xxx-org.apache.spark.deploy.worker.Worker-1-xxx.local.out
localhost: failed to launch: nice -n 0 /Users/xxx/workspace/spark/bin/spark-class org.apache.spark.deploy.worker.Worker --webui-port 8081 spark://xxx.local:7077
localhost:   17/06/13 23:24:53 INFO SecurityManager: Changing view acls to: xxx
localhost:   17/06/13 23:24:53 INFO SecurityManager: Changing modify acls to: xxx
localhost:   17/06/13 23:24:53 INFO SecurityManager: Changing view acls groups to:
localhost:   17/06/13 23:24:53 INFO SecurityManager: Changing modify acls groups to:
localhost:   17/06/13 23:24:53 INFO SecurityManager: SecurityManager: authentication disabled; ui acls disabled; users  with view permissions: Set(xxx); groups with view permissions: Set(); users  with modify permissions: Set(xxx); groups with modify permissions: Set()
localhost:   17/06/13 23:24:54 INFO Utils: Successfully started service 'sparkWorker' on port 63354.
localhost:   Exception in thread "main" java.lang.IllegalArgumentException: requirement failed: Start multiple worker on one host failed because we may launch no more than one external shuffle service on each host, please set spark.shuffle.service.enabled to false or set SPARK_WORKER_INSTANCES to 1 to resolve the conflict.
localhost:   	at scala.Predef$.require(Predef.scala:224)
localhost:   	at org.apache.spark.deploy.worker.Worker$.main(Worker.scala:752)
localhost:   	at org.apache.spark.deploy.worker.Worker.main(Worker.scala)
localhost: full log in /Users/xxx/workspace/spark/logs/spark-xxx-org.apache.spark.deploy.worker.Worker-1-xxx.local.out
localhost: starting org.apache.spark.deploy.worker.Worker, logging to /Users/xxx/workspace/spark/logs/spark-xxx-org.apache.spark.deploy.worker.Worker-2-xxx.local.out
localhost: failed to launch: nice -n 0 /Users/xxx/workspace/spark/bin/spark-class org.apache.spark.deploy.worker.Worker --webui-port 8082 spark://xxx.local:7077
localhost:   17/06/13 23:24:56 INFO SecurityManager: Changing view acls to: xxx
localhost:   17/06/13 23:24:56 INFO SecurityManager: Changing modify acls to: xxx
localhost:   17/06/13 23:24:56 INFO SecurityManager: Changing view acls groups to:
localhost:   17/06/13 23:24:56 INFO SecurityManager: Changing modify acls groups to:
localhost:   17/06/13 23:24:56 INFO SecurityManager: SecurityManager: authentication disabled; ui acls disabled; users  with view permissions: Set(xxx); groups with view permissions: Set(); users  with modify permissions: Set(xxx); groups with modify permissions: Set()
localhost:   17/06/13 23:24:56 INFO Utils: Successfully started service 'sparkWorker' on port 63359.
localhost:   Exception in thread "main" java.lang.IllegalArgumentException: requirement failed: Start multiple worker on one host failed because we may launch no more than one external shuffle service on each host, please set spark.shuffle.service.enabled to false or set SPARK_WORKER_INSTANCES to 1 to resolve the conflict.
localhost:   	at scala.Predef$.require(Predef.scala:224)
localhost:   	at org.apache.spark.deploy.worker.Worker$.main(Worker.scala:752)
localhost:   	at org.apache.spark.deploy.worker.Worker.main(Worker.scala)
localhost: full log in /Users/xxx/workspace/spark/logs/spark-xxx-org.apache.spark.deploy.worker.Worker-2-xxx.local.out
localhost: starting org.apache.spark.deploy.worker.Worker, logging to /Users/xxx/workspace/spark/logs/spark-xxx-org.apache.spark.deploy.worker.Worker-3-xxx.local.out
localhost: failed to launch: nice -n 0 /Users/xxx/workspace/spark/bin/spark-class org.apache.spark.deploy.worker.Worker --webui-port 8083 spark://xxx.local:7077
localhost:   17/06/13 23:24:59 INFO SecurityManager: Changing view acls to: xxx
localhost:   17/06/13 23:24:59 INFO SecurityManager: Changing modify acls to: xxx
localhost:   17/06/13 23:24:59 INFO SecurityManager: Changing view acls groups to:
localhost:   17/06/13 23:24:59 INFO SecurityManager: Changing modify acls groups to:
localhost:   17/06/13 23:24:59 INFO SecurityManager: SecurityManager: authentication disabled; ui acls disabled; users  with view permissions: Set(xxx); groups with view permissions: Set(); users  with modify permissions: Set(xxx); groups with modify permissions: Set()
localhost:   17/06/13 23:24:59 INFO Utils: Successfully started service 'sparkWorker' on port 63360.
localhost:   Exception in thread "main" java.lang.IllegalArgumentException: requirement failed: Start multiple worker on one host failed because we may launch no more than one external shuffle service on each host, please set spark.shuffle.service.enabled to false or set SPARK_WORKER_INSTANCES to 1 to resolve the conflict.
localhost:   	at scala.Predef$.require(Predef.scala:224)
localhost:   	at org.apache.spark.deploy.worker.Worker$.main(Worker.scala:752)
localhost:   	at org.apache.spark.deploy.worker.Worker.main(Worker.scala)
localhost: full log in /Users/xxx/workspace/spark/logs/spark-xxx-org.apache.spark.deploy.worker.Worker-3-xxx.local.out
```

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

Closes #18290 from jiangxb1987/start-slave.
2017-06-20 17:17:21 +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
Marcelo Vanzin 581565dd87 [SPARK-21124][UI] Show correct application user in UI.
The jobs page currently shows the application user, but it assumes
the OS user is the same as the user running the application, which
may not be true in all scenarios (e.g., kerberos). While it might be
useful to show both in the UI, this change just chooses the application
user over the OS user, since the latter can be found in the environment
page if needed.

Tested in live application and in history server.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #18331 from vanzin/SPARK-21124.
2017-06-19 14:41:58 -07:00
Dongjoon Hyun ecc5631351 [MINOR][BUILD] Fix Java linter errors
## What changes were proposed in this pull request?

This PR cleans up a few Java linter errors for Apache Spark 2.2 release.

## How was this patch tested?

```bash
$ dev/lint-java
Using `mvn` from path: /usr/local/bin/mvn
Checkstyle checks passed.
```

We can check the result at Travis CI, [here](https://travis-ci.org/dongjoon-hyun/spark/builds/244297894).

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #18345 from dongjoon-hyun/fix_lint_java_2.
2017-06-19 20:17:54 +01: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