Fix for spark.task.maxFailures not enforced correctly.
Docs at http://spark.incubator.apache.org/docs/latest/configuration.html say:
```
spark.task.maxFailures
Number of individual task failures before giving up on the job. Should be greater than or equal to 1. Number of allowed retries = this value - 1.
```
Previous implementation worked incorrectly. When for example `spark.task.maxFailures` was set to 1, the job was aborted only after the second task failure, not after the first one.
- Made file stream more robust to transient failures.
- Changed Spark.setCheckpointDir API to not have the second
'useExisting' parameter. Spark will always create a unique directory
for checkpointing underneath the directory provide to the funtion.
- Fixed bug wrt local relative paths as checkpoint directory.
- Made DStream and RDD checkpointing use
SparkContext.hadoopConfiguration, so that more HDFS compatible
filesystems are supported for checkpointing.
stageId <--> jobId mapping in DAGScheduler
Okay, I think this one is ready to go -- or at least it's ready for review and discussion. It's a carry-over of https://github.com/mesos/spark/pull/842 with updates for the newer job cancellation functionality. The prior discussion still applies. I've actually changed the job cancellation flow a bit: Instead of ``cancelTasks`` going to the TaskScheduler and then ``taskSetFailed`` coming back to the DAGScheduler (resulting in ``abortStage`` there), the DAGScheduler now takes care of figuring out which stages should be cancelled, tells the TaskScheduler to cancel tasks for those stages, then does the cleanup within the DAGScheduler directly without the need for any further prompting by the TaskScheduler.
I know of three outstanding issues, each of which can and should, I believe, be handled in follow-up pull requests:
1) https://spark-project.atlassian.net/browse/SPARK-960
2) JobLogger should be re-factored to eliminate duplication
3) Related to 2), the WebUI should also become a consumer of the DAGScheduler's new understanding of the relationship between jobs and stages so that it can display progress indication and the like grouped by job. Right now, some of this information is just being sent out as part of ``SparkListenerJobStart`` messages, but more or different job <--> stage information may need to be exported from the DAGScheduler to meet listeners needs.
Except for the eventQueue -> Actor commit, the rest can be cherry-picked almost cleanly into branch-0.8. A little merging is needed in MapOutputTracker and the DAGScheduler. Merged versions of those files are in aba2b40ce0
Note that between the recent Actor change in the DAGScheduler and the cleaning up of DAGScheduler data structures on job completion in this PR, some races have been introduced into the DAGSchedulerSuite. Those tests usually pass, and I don't think that better-behaved code that doesn't directly inspect DAGScheduler data structures should be seeing any problems, but I'll work on fixing DAGSchedulerSuite as either an addition to this PR or as a separate request.
UPDATE: Fixed the race that I introduced. Created a JIRA issue (SPARK-965) for the one that was introduced with the switch to eventProcessorActor in the DAGScheduler.
...and make sure that DAGScheduler data structures are cleaned up on job completion.
Initial effort and discussion at https://github.com/mesos/spark/pull/842
Re-enable zk:// urls for Mesos SparkContexts
This was broken in PR #71 when we explicitly disallow anything that didn't fit a mesos:// url.
Although it is not really clear that a zk:// url should match Mesos, it is what the docs say and it is necessary for backwards compatibility.
Additionally added a unit test for the creation of all types of TaskSchedulers. Since YARN and Mesos are not necessarily available in the system, they are allowed to pass as long as the YARN/Mesos code paths are exercised.
OpenHashSet fixes
Incorporated ideas from pull request #200.
- Use Murmur Hash 3 finalization step to scramble the bits of HashCode
instead of the simpler version in java.util.HashMap; the latter one
had trouble with ranges of consecutive integers. Murmur Hash 3 is used
by fastutil.
- Don't check keys for equality when re-inserting due to growing the
table; the keys will already be unique.
- Remember the grow threshold instead of recomputing it on each insert
Also added unit tests for size estimation for specialized hash sets and maps.
Use the proper partition index in mapPartitionsWIthIndex
mapPartitionsWithIndex uses TaskContext.partitionId as the partition index. TaskContext.partitionId used to be identical to the partition index in a RDD. However, pull request #186 introduced a scenario (with partition pruning) that the two can be different. This pull request uses the right partition index in all mapPartitionsWithIndex related calls.
Also removed the extra MapPartitionsWIthContextRDD and put all the mapPartitions related functionality in MapPartitionsRDD.
XORShift RNG with unit tests and benchmark
This patch was introduced to address SPARK-950 - the discussion below the ticket explains not only the rationale, but also the design and testing decisions: https://spark-project.atlassian.net/browse/SPARK-950
To run unit test, start SBT console and type:
compile
test-only org.apache.spark.util.XORShiftRandomSuite
To run benchmark, type:
project core
console
Once the Scala console starts, type:
org.apache.spark.util.XORShiftRandom.benchmark(100000000)
XORShiftRandom is also an object with a main method taking the
number of iterations as an argument, so you can also run it
from the command line.
Also changed the semantics of the index parameter in mapPartitionsWithIndex from the partition index of the output partition to the partition index in the current RDD.
PartitionPruningRDD is using index from parent
I was getting a ArrayIndexOutOfBoundsException exception after doing union on pruned RDD. The index it was using on the partition was the index in the original RDD not the new pruned RDD.
To run unit test, start SBT console and type:
compile
test-only org.apache.spark.util.XORShiftRandomSuite
To run benchmark, type:
project core
console
Once the Scala console starts, type:
org.apache.spark.util.XORShiftRandom.benchmark(100000000)
Migrate the daemon thread started by DAGScheduler to Akka actor
`DAGScheduler` adopts an event queue and a daemon thread polling the it to process events sent to a `DAGScheduler`. This is a classical actor use case. By migrating this thread to Akka actor, we may benefit from both cleaner code and better performance (context switching cost of Akka actor is much less than that of a native thread).
But things become a little complicated when taking existing test code into consideration.
Code in `DAGSchedulerSuite` is somewhat tightly coupled with `DAGScheduler`, and directly calls `DAGScheduler.processEvent` instead of posting event messages to `DAGScheduler`. To minimize code change, I chose to let the actor to delegate messages to `processEvent`. Maybe this doesn't follow conventional actor usage, but I tried to make it apparently correct.
Another tricky part is that, since `DAGScheduler` depends on the `ActorSystem` provided by its field `env`, `env` cannot be null. But the `dagScheduler` field created in `DAGSchedulerSuite.before` was given a null `env`. What's more, `BlockManager.blockIdsToBlockManagers` checks whether `env` is null to determine whether to run the production code or the test code (bad smell here, huh?). I went through all callers of `BlockManager.blockIdsToBlockManagers`, and made sure that if `env != null` holds, then `blockManagerMaster == null` must also hold. That's the logic behind `BlockManager.scala` [line 896](https://github.com/liancheng/incubator-spark/compare/dagscheduler-actor-refine?expand=1#diff-2b643ea78c1add0381754b1f47eec132L896).
At last, since `DAGScheduler` instances are always `start()`ed after creation, I removed the `start()` method, and starts the `eventProcessActor` within the constructor.
Include appId in executor cmd line args
add the appId back into the executor cmd line args.
I also made a pretty lame regression test, just to make sure it doesn't get dropped in the future. not sure it will run on the build server, though, b/c `ExecutorRunner.buildCommandSeq()` expects to be abel to run the scripts in `bin`.
add javadoc to JobLogger, and some small fix
against Spark-941
add javadoc to JobLogger, output more info for RDD, modify recordStageDepGraph to avoid output duplicate stage dependency information
(cherry picked from commit 518cf22eb2)
Signed-off-by: Reynold Xin <rxin@apache.org>
For some reason, even calling
java.nio.Files.createTempDirectory().getFile.deleteOnExit()
does not delete the directory on exit. Guava's analagous function
seems to work, however.
Overhead of each shuffle block for consolidation has been reduced from >300 bytes
to 8 bytes (1 primitive Long). Verified via profiler testing with 1 mil shuffle blocks,
net overhead was ~8,400,000 bytes.
Despite the memory-optimized implementation incurring extra CPU overhead, the runtime
of the shuffle phase in this test was only around 2% slower, while the reduce phase
was 40% faster, when compared to not using any shuffle file consolidation.
The code in LocalScheduler/LocalTaskSetManager was nearly identical
to the code in ClusterScheduler/ClusterTaskSetManager. The redundancy
made making updating the schedulers unnecessarily painful and error-
prone. This commit combines the two into a single TaskScheduler/
TaskSetManager.
This patch adds an operator called repartition with more straightforward
semantics than the current `coalesce` operator. There are a few use cases
where this operator is useful:
1. If a user wants to increase the number of partitions in the RDD. This
is more common now with streaming. E.g. a user is ingesting data on one
node but they want to add more partitions to ensure parallelism of
subsequent operations across threads or the cluster.
Right now they have to call rdd.coalesce(numSplits, shuffle=true) - that's
super confusing.
2. If a user has input data where the number of partitions is not known. E.g.
> sc.textFile("some file").coalesce(50)....
This is both vague semantically (am I growing or shrinking this RDD) but also,
may not work correctly if the base RDD has fewer than 50 partitions.
The new operator forces shuffles every time, so it will always produce exactly
the number of new partitions. It also throws an exception rather than silently
not-working if a bad input is passed.
I am currently adding streaming tests (requires refactoring some of the test
suite to allow testing at partition granularity), so this is not ready for
merge yet. But feedback is welcome.
This patch fixes a bug where the Spark UI didn't display the correct number of total
tasks if the number of tasks in a Stage doesn't equal the number of RDD partitions.
It also cleans up the listener API a bit by embedding this information in the
StageInfo class rather than passing it seperately.
This commit adds a set of calls using the SparkListener interface
that indicate when a task is remotely fetching results, so that
we can display this (potentially time-consuming) phase of execution
to users through the UI.
Previously, MapOutputTracker contained fields and methods that
were only applicable to the master or worker instances. This
commit introduces a MasterMapOutputTracker class to prevent
the master-specific methods from being accessed on workers.
I also renamed a few methods and made others protected/private.
Job cancellation via job group id.
This PR adds a simple API to group together a set of jobs belonging to a thread and threads spawned from it. It also allows the cancellation of all jobs in this group.
An example:
sc.setJobDescription("this_is_the_group_id", "some job description")
sc.parallelize(1 to 10000, 2).map { i => Thread.sleep(10); i }.count()
In a separate thread:
sc.cancelJobGroup("this_is_the_group_id")