Commit graph

6297 commits

Author SHA1 Message Date
曾林西 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
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
liuxian 7ba8bf288d [SPARK-21016][CORE] Improve code fault tolerance for converting string to number
## What changes were proposed in this pull request?
When converting `string` to `number`(int, long or double),  if the string has a space before or after,will lead to unnecessary mistakes.

## How was this patch tested?
unit test

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

Closes #18238 from 10110346/lx-wip-0608.
2017-06-13 10:12:28 -07: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
guoxiaolong b7304f2559 [SPARK-21060][WEB-UI] Css style about paging function is error in the executor page. Css style about paging function is error in the executor page. It is different of history server ui paging function css style.
## What changes were proposed in this pull request?

Css style about paging function is error in the executor page. It is different of history server ui paging function css style.

**But their style should be consistent**. There are three reasons.

1. The first reason: 'Previous', 'Next' and number should be the button format.

2. The second reason: when you are on the first page, 'Previous' and '1' should be gray and can not be clicked.
![1](https://user-images.githubusercontent.com/26266482/27026667-1fe745ee-4f91-11e7-8b34-150819d22bd3.png)

3. The third reason: when you are on the last page, 'Previous' and 'Max number' should be gray and can not be clicked.
![2](https://user-images.githubusercontent.com/26266482/27026811-9d8d6fa0-4f91-11e7-8b51-7816c3feb381.png)

before fix:
![fix_before](https://user-images.githubusercontent.com/26266482/27026428-47ec5c56-4f90-11e7-9dd5-d52c22d7bd36.png)

after fix:
![fix_after](https://user-images.githubusercontent.com/26266482/27026439-50d17072-4f90-11e7-8405-6f81da5ab32c.png)

The style of history server ui:
![history](https://user-images.githubusercontent.com/26266482/27026528-9c90f780-4f90-11e7-91e6-90d32651fe03.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>
Author: 郭小龙 10207633 <guo.xiaolong1@zte.com.cn>
Author: guoxiaolongzte <guo.xiaolong1@zte.com.cn>

Closes #18275 from guoxiaolongzte/SPARK-21060.
2017-06-13 15:38:11 +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
guoxiaolong 82faacd791 [SPARK-20997][CORE] driver-cores' standalone or Mesos or YARN in Cluster deploy mode only.
## What changes were proposed in this pull request?

'--driver-cores'  standalone or Mesos or YARN in Cluster deploy mode only.So  The description of spark-submit about it is not very accurate.

## 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>
Author: 郭小龙 10207633 <guo.xiaolong1@zte.com.cn>
Author: guoxiaolongzte <guo.xiaolong1@zte.com.cn>

Closes #18241 from guoxiaolongzte/SPARK-20997.
2017-06-09 14:26:54 +01: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
liupengcheng 2d39711b05 [SPARK-20945] Fix TID key not found in TaskSchedulerImpl
## What changes were proposed in this pull request?

This pull request fix the TaskScheulerImpl bug in some condition.
Detail see:
https://issues.apache.org/jira/browse/SPARK-20945

(Please fill in changes proposed in this fix)

## How was this patch tested?
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)

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

Author: liupengcheng <liupengcheng@xiaomi.com>
Author: PengchengLiu <pengchengliu_bupt@163.com>

Closes #18171 from liupc/Fix-tid-key-not-found-in-TaskSchedulerImpl.
2017-06-05 10:23:04 +01: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
Shixiong Zhu 16186cdcbc [SPARK-20955][CORE] Intern "executorId" to reduce the memory usage
## What changes were proposed in this pull request?

In [this line](f7cf2096fd/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala (L128)), it uses the `executorId` string received from executors and finally it will go into `TaskUIData`. As deserializing the `executorId` string will always create a new instance, we have a lot of duplicated string instances.

This PR does a String interning for TaskUIData to reduce the memory usage.

## How was this patch tested?

Manually test using `bin/spark-shell --master local-cluster[6,1,1024]`. Test codes:
```
for (_ <- 1 to 10) { sc.makeRDD(1 to 1000, 1000).count() }
Thread.sleep(2000)
val l = sc.getClass.getMethod("jobProgressListener").invoke(sc).asInstanceOf[org.apache.spark.ui.jobs.JobProgressListener]
org.apache.spark.util.SizeEstimator.estimate(l.stageIdToData)
```
This PR reduces the size of `stageIdToData` from 3487280 to 3009744 (86.3%) in the above case.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #18177 from zsxwing/SPARK-20955.
2017-06-02 10:33:21 -07:00
guoxiaolong 625cebfde6 [SPARK-20942][WEB-UI] The title style about field is error in the history server web ui.
## What changes were proposed in this pull request?

1.The title style about field is error.
fix before:
![before](https://cloud.githubusercontent.com/assets/26266482/26661987/a7bed018-46b3-11e7-8a54-a5152d2df0f4.png)

fix after:
![fix](https://cloud.githubusercontent.com/assets/26266482/26662000/ba6cc814-46b3-11e7-8f33-cfd4cc2c60fe.png)

![fix1](https://cloud.githubusercontent.com/assets/26266482/26662080/3c732e3e-46b4-11e7-8768-20b5a6aeadcb.png)

executor-page style:
![executor_page](https://cloud.githubusercontent.com/assets/26266482/26662384/167cbd10-46b6-11e7-9e07-bf391dbc6e08.png)

2.Title text description, 'the application' should be changed to 'this application'.

3.Analysis of code:
 $('#history-summary [data-toggle="tooltip"]').tooltip();
The id of 'history-summary' is not there. We only contain id of 'history-summary-table'.

## 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>
Author: 郭小龙 10207633 <guo.xiaolong1@zte.com.cn>
Author: guoxiaolongzte <guo.xiaolong1@zte.com.cn>

Closes #18170 from guoxiaolongzte/SPARK-20942.
2017-06-02 14:38:00 +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
jerryshao 52ed9b289d [SPARK-20275][UI] Do not display "Completed" column for in-progress applications
## What changes were proposed in this pull request?

Current HistoryServer will display completed date of in-progress application as `1969-12-31 23:59:59`, which is not so meaningful. Instead of unnecessarily showing this incorrect completed date, here propose to make this column invisible for in-progress applications.

The purpose of only making this column invisible rather than deleting this field is that: this data is fetched through REST API, and in the REST API  the format is like below shows, in which `endTime` matches `endTimeEpoch`. So instead of changing REST API to break backward compatibility, here choosing a simple solution to only make this column invisible.

```
[ {
  "id" : "local-1491805439678",
  "name" : "Spark shell",
  "attempts" : [ {
    "startTime" : "2017-04-10T06:23:57.574GMT",
    "endTime" : "1969-12-31T23:59:59.999GMT",
    "lastUpdated" : "2017-04-10T06:23:57.574GMT",
    "duration" : 0,
    "sparkUser" : "",
    "completed" : false,
    "startTimeEpoch" : 1491805437574,
    "endTimeEpoch" : -1,
    "lastUpdatedEpoch" : 1491805437574
  } ]
} ]%
```

Here is UI before changed:

<img width="1317" alt="screen shot 2017-04-10 at 3 45 57 pm" src="https://cloud.githubusercontent.com/assets/850797/24851938/17d46cc0-1e08-11e7-84c7-90120e171b41.png">

And after:

<img width="1281" alt="screen shot 2017-04-10 at 4 02 35 pm" src="https://cloud.githubusercontent.com/assets/850797/24851945/1fe9da58-1e08-11e7-8d0d-9262324f9074.png">

## How was this patch tested?

Manual verification.

Author: jerryshao <sshao@hortonworks.com>

Closes #17588 from jerryshao/SPARK-20275.
2017-05-30 20:24:43 -07: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
liuzhaokun 8faffc4167 [SPARK-20875] Spark should print the log when the directory has been deleted
[https://issues.apache.org/jira/browse/SPARK-20875](https://issues.apache.org/jira/browse/SPARK-20875)
When the "deleteRecursively" method is invoked,spark doesn't print any log if the path was deleted.For example,spark only print "Removing directory" when the worker began cleaning spark.work.dir,but didn't print any log about "the path has been delete".So, I can't judge whether the path was deleted form the worker's logfile,If there is any accidents about Linux.

Author: liuzhaokun <liu.zhaokun@zte.com.cn>

Closes #18102 from liu-zhaokun/master_log.
2017-05-27 13:26:01 +01:00
Shixiong Zhu 6c1dbd6fc8 [SPARK-20843][CORE] Add a config to set driver terminate timeout
## What changes were proposed in this pull request?

Add a `worker` configuration to set how long to wait before forcibly killing driver.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #18126 from zsxwing/SPARK-20843.
2017-05-26 22:25:38 -07: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
Sital Kedia 473d7552ac [SPARK-20014] Optimize mergeSpillsWithFileStream method
## What changes were proposed in this pull request?

When the individual partition size in a spill is small, mergeSpillsWithTransferTo method does many small disk ios which is really inefficient. One way to improve the performance will be to use mergeSpillsWithFileStream method by turning off transfer to and using buffered file read/write to improve the io throughput.
However, the current implementation of mergeSpillsWithFileStream does not do a buffer read/write of the files and in addition to that it unnecessarily flushes the output files for each partitions.

## How was this patch tested?

Tested this change by running a job on the cluster and the map stage run time was reduced by around 20%.

Author: Sital Kedia <skedia@fb.com>

Closes #17343 from sitalkedia/upstream_mergeSpillsWithFileStream.
2017-05-26 13:41:13 -07:00
10129659 0fd84b05dc [SPARK-20835][CORE] It should exit directly when the --total-executor-cores parameter is setted less than 0 when submit a application
## What changes were proposed in this pull request?
In my test, the submitted app running with out an error when the --total-executor-cores less than 0
and given the warnings:
"2017-05-22 17:19:36,319 WARN org.apache.spark.scheduler.TaskSchedulerImpl: Initial job has not accepted any resources; check your cluster UI to ensure that workers are registered and have sufficient resources";

It should exit directly when the --total-executor-cores parameter is setted less than 0 when submit a application
(Please fill in changes proposed in this fix)

## How was this patch tested?
Run the ut 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: 10129659 <chen.yanshan@zte.com.cn>

Closes #18060 from eatoncys/totalcores.
2017-05-26 18:03:23 +01: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
Wenchen Fan d9ad78908f [SPARK-20868][CORE] UnsafeShuffleWriter should verify the position after FileChannel.transferTo
## What changes were proposed in this pull request?

Long time ago we fixed a [bug](https://issues.apache.org/jira/browse/SPARK-3948) in shuffle writer about `FileChannel.transferTo`. We were not very confident about that fix, so we added a position check after the writing, try to discover the bug earlier.

 However this checking is missing in the new `UnsafeShuffleWriter`, this PR adds it.

https://issues.apache.org/jira/browse/SPARK-18105 maybe related to that `FileChannel.transferTo` bug, hopefully we can find out the root cause after adding this position check.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #18091 from cloud-fan/shuffle.
2017-05-26 15:01:28 +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
Xianyang Liu 731462a04f [SPARK-20250][CORE] Improper OOM error when a task been killed while spilling data
## What changes were proposed in this pull request?

Currently, when a task is calling spill() but it receives a killing request from driver (e.g., speculative task), the `TaskMemoryManager` will throw an `OOM` exception.  And we don't catch `Fatal` exception when a error caused by `Thread.interrupt`. So for `ClosedByInterruptException`, we should throw `RuntimeException` instead of `OutOfMemoryError`.

https://issues.apache.org/jira/browse/SPARK-20250?jql=project%20%3D%20SPARK

## How was this patch tested?

Existing unit tests.

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

Closes #18090 from ConeyLiu/SPARK-20250.
2017-05-25 15:47:59 +08:00
Marcelo Vanzin 95aef660b7 [SPARK-20205][CORE] Make sure StageInfo is updated before sending event.
The DAGScheduler was sending a "stage submitted" event before it properly
updated the event's information. This meant that a listener (e.g. the
even logging listener) could record wrong information about the event.

This change sets the stage's submission time before the event is submitted,
when there are tasks to be executed in the stage.

Tested with existing unit tests.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #17925 from vanzin/SPARK-20205.
2017-05-24 16:57:17 -07: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
John Lee aea73be1b4 [SPARK-20813][WEB UI] Fixed Web UI executor page tab search by status not working
## What changes were proposed in this pull request?
On status column of the table, I removed the condition  that forced only the display value to take on values Active, Blacklisted and Dead.

Before the removal, values used for sort and filter for that particular column was True and False.
## How was this patch tested?

Tested with Active, Blacklisted and Dead present as current status.

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

Closes #18036 from yoonlee95/SPARK-20813.
2017-05-22 14:24:49 +01: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
Shixiong Zhu f8e0f0f47c [SPARK-20788][CORE] Fix the Executor task reaper's false alarm warning logs
## What changes were proposed in this pull request?

Executor task reaper may fail to detect if a task is finished or not when a task is finishing but being killed at the same time.

The fix is pretty easy, just flip the "finished" flag when a task is successful.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #18021 from zsxwing/SPARK-20788.
2017-05-17 14:13:49 -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
Wenchen Fan e1aaab1e27 [SPARK-12837][SPARK-20666][CORE][FOLLOWUP] getting name should not fail if accumulator is garbage collected
## What changes were proposed in this pull request?

After https://github.com/apache/spark/pull/17596 , we do not send internal accumulator name to executor side anymore, and always look up the accumulator name in `AccumulatorContext`.

This cause a regression if the accumulator is already garbage collected, this PR fixes this by still sending accumulator name for `SQLMetrics`.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #17931 from cloud-fan/bug.
2017-05-15 09:22:06 -07:00
guoxiaolong 8da6e8b1f3 [SPARK-20720][WEB-UI] Executor Summary' should show the exact number, 'Removed Executors' should display the specific number, in the Application Page
## What changes were proposed in this pull request?

When the number of spark worker executors is large, if the specific number is displayed, will better help us to analyze and observe by spark ui.

Although this is a small improvement, but it is indeed very valuable.

After fix:
![executor1](https://cloud.githubusercontent.com/assets/26266482/25986597/2d8e4386-3723-11e7-9c24-e5bff17c26e2.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>
Author: 郭小龙 10207633 <guo.xiaolong1@zte.com.cn>
Author: guoxiaolongzte <guo.xiaolong1@zte.com.cn>

Closes #17961 from guoxiaolongzte/SPARK-20720.
2017-05-15 07:53:57 +01:00
guoxiaolong 99d5799927 [SPARK-20705][WEB-UI] The sort function can not be used in the master page when you use Firefox or Google Chrome.
## What changes were proposed in this pull request?
When you open the master page, when you use Firefox or Google Chrom, the console of Firefox or Google Chrome is wrong. But The IE  is no problem.
e.g.
![error](https://cloud.githubusercontent.com/assets/26266482/25946143/74467a5c-367c-11e7-8f9f-d3585b1aea88.png)

My Firefox version is 48.0.2.
My Google Chrome version  is 49.0.2623.75 m.

## 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>
Author: 郭小龙 10207633 <guo.xiaolong1@zte.com.cn>
Author: guoxiaolongzte <guo.xiaolong1@zte.com.cn>

Closes #17952 from guoxiaolongzte/SPARK-20705.
2017-05-15 07:51:50 +01: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
Xianyang Liu fcb88f9211 [MINOR][BUILD] Fix lint-java breaks.
## What changes were proposed in this pull request?

This PR proposes to fix the lint-breaks as below:
```
[ERROR] src/main/java/org/apache/spark/unsafe/Platform.java:[51] (regexp) RegexpSingleline: No trailing whitespace allowed.
[ERROR] src/main/scala/org/apache/spark/sql/streaming/Trigger.java:[45,25] (naming) MethodName: Method name 'ProcessingTime' must match pattern '^[a-z][a-z0-9][a-zA-Z0-9_]*$'.
[ERROR] src/main/scala/org/apache/spark/sql/streaming/Trigger.java:[62,25] (naming) MethodName: Method name 'ProcessingTime' must match pattern '^[a-z][a-z0-9][a-zA-Z0-9_]*$'.
[ERROR] src/main/scala/org/apache/spark/sql/streaming/Trigger.java:[78,25] (naming) MethodName: Method name 'ProcessingTime' must match pattern '^[a-z][a-z0-9][a-zA-Z0-9_]*$'.
[ERROR] src/main/scala/org/apache/spark/sql/streaming/Trigger.java:[92,25] (naming) MethodName: Method name 'ProcessingTime' must match pattern '^[a-z][a-z0-9][a-zA-Z0-9_]*$'.
[ERROR] src/main/scala/org/apache/spark/sql/streaming/Trigger.java:[102,25] (naming) MethodName: Method name 'Once' must match pattern '^[a-z][a-z0-9][a-zA-Z0-9_]*$'.
[ERROR] src/test/java/org/apache/spark/streaming/kinesis/JavaKinesisInputDStreamBuilderSuite.java:[28,8] (imports) UnusedImports: Unused import - org.apache.spark.streaming.api.java.JavaDStream.
```

after:
```
dev/lint-java
Checkstyle checks passed.
```
[Test Result](https://travis-ci.org/ConeyLiu/spark/jobs/229666169)

## How was this patch tested?

Travis CI

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

Closes #17890 from ConeyLiu/codestyle.
2017-05-10 13:56:34 +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
Alex Bozarth ca4625e0e5 [SPARK-20630][WEB UI] Fixed column visibility in Executor Tab
## What changes were proposed in this pull request?

#14617 added new columns to the executor table causing the visibility checks for the logs and threadDump columns to toggle the wrong columns since they used hard-coded column numbers.

I've updated the checks to use column names instead of numbers so future updates don't accidentally break this again.

Note: This will also need to be back ported into 2.2 since #14617 was merged there

## How was this patch tested?

Manually tested

Author: Alex Bozarth <ajbozart@us.ibm.com>

Closes #17904 from ajbozarth/spark20630.
2017-05-10 10:20:10 +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
jerryshao 829cd7b8b7 [SPARK-20605][CORE][YARN][MESOS] Deprecate not used AM and executor port configuration
## What changes were proposed in this pull request?

After SPARK-10997, client mode Netty RpcEnv doesn't require to start server, so port configurations are not used any more, here propose to remove these two configurations: "spark.executor.port" and "spark.am.port".

## How was this patch tested?

Existing UTs.

Author: jerryshao <sshao@hortonworks.com>

Closes #17866 from jerryshao/SPARK-20605.
2017-05-08 14:27:56 -07: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
liuxian 0f820e2b6c [SPARK-20519][SQL][CORE] Modify to prevent some possible runtime exceptions
Signed-off-by: liuxian <liu.xian3zte.com.cn>

## What changes were proposed in this pull request?

When the input parameter is null, may be a runtime exception occurs

## How was this patch tested?
Existing unit tests

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

Closes #17796 from 10110346/wip_lx_0428.
2017-05-08 10:00:58 +01: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
Wenchen Fan b946f3160e [SPARK-20558][CORE] clear InheritableThreadLocal variables in SparkContext when stopping it
## What changes were proposed in this pull request?

To better understand this problem, let's take a look at an example first:
```
object Main {
  def main(args: Array[String]): Unit = {
    var t = new Test
    new Thread(new Runnable {
      override def run() = {}
    }).start()
    println("first thread finished")

    t.a = null
    t = new Test
    new Thread(new Runnable {
      override def run() = {}
    }).start()
  }

}

class Test {
  var a = new InheritableThreadLocal[String] {
    override protected def childValue(parent: String): String = {
      println("parent value is: " + parent)
      parent
    }
  }
  a.set("hello")
}
```
The result is:
```
parent value is: hello
first thread finished
parent value is: hello
parent value is: hello
```

Once an `InheritableThreadLocal` has been set value, child threads will inherit its value as long as it has not been GCed, so setting the variable which holds the `InheritableThreadLocal` to `null` doesn't work as we expected.

In `SparkContext`, we have an `InheritableThreadLocal` for local properties, we should clear it when stopping `SparkContext`, or all the future child threads will still inherit it and copy the properties and waste memory.

This is the root cause of https://issues.apache.org/jira/browse/SPARK-20548 , which creates/stops `SparkContext` many times and finally have a lot of `InheritableThreadLocal` alive, and cause OOM when starting new threads in the internal thread pools.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #17833 from cloud-fan/core.
2017-05-03 10:08:46 +08:00
Marcelo Vanzin ef3df9125a [SPARK-20421][CORE] Add a missing deprecation tag.
In the previous patch I deprecated StorageStatus, but not the
method in SparkContext that exposes that class publicly. So deprecate
the method too.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #17824 from vanzin/SPARK-20421.
2017-05-02 14:30:06 -07: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
Kunal Khamar 6fc6cf88d8 [SPARK-20464][SS] Add a job group and description for streaming queries and fix cancellation of running jobs using the job group
## What changes were proposed in this pull request?

Job group: adding a job group is required to properly cancel running jobs related to a query.
Description: the new description makes it easier to group the batches of a query by sorting by name in the Spark Jobs UI.

## How was this patch tested?

- Unit tests
- UI screenshot

  - Order by job id:
![screen shot 2017-04-27 at 5 10 09 pm](https://cloud.githubusercontent.com/assets/7865120/25509468/15452274-2b6e-11e7-87ba-d929816688cf.png)

  - Order by description:
![screen shot 2017-04-27 at 5 10 22 pm](https://cloud.githubusercontent.com/assets/7865120/25509474/1c298512-2b6e-11e7-99b8-fef1ef7665c1.png)

  - Order by job id (no query name):
![screen shot 2017-04-27 at 5 21 33 pm](https://cloud.githubusercontent.com/assets/7865120/25509482/28c96dc8-2b6e-11e7-8df0-9d3cdbb05e36.png)

  - Order by description (no query name):
![screen shot 2017-04-27 at 5 21 44 pm](https://cloud.githubusercontent.com/assets/7865120/25509489/37674742-2b6e-11e7-9357-b5c38ec16ac4.png)

Author: Kunal Khamar <kkhamar@outlook.com>

Closes #17765 from kunalkhamar/sc-6696.
2017-05-01 11:37:30 -07:00
jerryshao ab30590f44 [SPARK-20517][UI] Fix broken history UI download link
The download link in history server UI is concatenated with:

```
 <td><a href="{{uiroot}}/api/v1/applications/{{id}}/{{num}}/logs" class="btn btn-info btn-mini">Download</a></td>
```

Here `num` field represents number of attempts, this is not equal to REST APIs. In the REST API, if attempt id is not existed the URL should be `api/v1/applications/<id>/logs`, otherwise the URL should be `api/v1/applications/<id>/<attemptId>/logs`. Using `<num>` to represent `<attemptId>` will lead to the issue of "no such app".

Manual verification.

CC ajbozarth can you please review this change, since you add this feature before? Thanks!

Author: jerryshao <sshao@hortonworks.com>

Closes #17795 from jerryshao/SPARK-20517.
2017-05-01 10:26:08 -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
Mark Grover 5d71f3db83 [SPARK-20514][CORE] Upgrade Jetty to 9.3.11.v20160721
Upgrade Jetty so it can work with Hadoop 3 (alpha 2 release, in particular).
Without this change, because of incompatibily between Jetty versions,
Spark fails to compile when built against Hadoop 3

## How was this patch tested?
Unit tests being run.

Author: Mark Grover <mark@apache.org>

Closes #17790 from markgrover/spark-20514.
2017-04-28 14:06:57 -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