Author: Prashant Sharma <prashant.s@imaginea.com>
Author: Patrick Wendell <pwendell@gmail.com>
Closes#17 from ScrapCodes/java8-lambdas and squashes the following commits:
95850e6 [Patrick Wendell] Some doc improvements and build changes to the Java 8 patch.
85a954e [Prashant Sharma] Nit. import orderings.
673f7ac [Prashant Sharma] Added support for -java-home as well
80a13e8 [Prashant Sharma] Used fake class tag syntax
26eb3f6 [Prashant Sharma] Patrick's comments on PR.
35d8d79 [Prashant Sharma] Specified java 8 building in the docs
31d4cd6 [Prashant Sharma] Maven build to support -Pjava8-tests flag.
4ab87d3 [Prashant Sharma] Review feedback on the pr
c33dc2c [Prashant Sharma] SPARK-964, Java 8 API Support.
Lookup didn't have a unit test. Added two tests, one for with a partitioner, and one for without.
Author: Bryn Keller <bryn.keller@intel.com>
Closes#36 from xoltar/lookup and squashes the following commits:
3bc0d44 [Bryn Keller] Added a unit test for PairRDDFunctions.lookup
This metric is confusing: it adds up all of the time to fetch
shuffle inputs, but fetches often happen in parallel, so
remoteFetchTime can be much longer than the task execution time.
@squito it looks like you added this metric -- do you have a use case for it?
cc @shivaram -- I know you've looked at the shuffle performance a lot so chime in here if this metric has turned out to be useful for you!
Author: Kay Ousterhout <kayousterhout@gmail.com>
Closes#62 from kayousterhout/remove_fetch_variable and squashes the following commits:
43341eb [Kay Ousterhout] Remote the remoteFetchTime metric.
Thanks for Diana Carroll to report this issue (https://spark-project.atlassian.net/browse/SPARK-1100)
the current saveAsTextFile/SequenceFile will overwrite the output directory silently if the directory already exists, this behaviour is not desirable because
overwriting the data silently is not user-friendly
if the partition number of two writing operation changed, then the output directory will contain the results generated by two runnings
My fix includes:
add some new APIs with a flag for users to define whether he/she wants to overwrite the directory:
if the flag is set to true, then the output directory is deleted first and then written into the new data to prevent the output directory contains results from multiple rounds of running;
if the flag is set to false, Spark will throw an exception if the output directory already exists
changed JavaAPI part
default behaviour is overwriting
Two questions
should we deprecate the old APIs without such a flag?
I noticed that Spark Streaming also called these APIs, I thought we don't need to change the related part in streaming? @tdas
Author: CodingCat <zhunansjtu@gmail.com>
Closes#11 from CodingCat/SPARK-1100 and squashes the following commits:
6a4e3a3 [CodingCat] code clean
ef2d43f [CodingCat] add new test cases and code clean
ac63136 [CodingCat] checkOutputSpecs not applicable to FSOutputFormat
ec490e8 [CodingCat] prevent Spark from overwriting directory silently and leaving dirty directory
This commit randomizes the order of resource offers to avoid scheduling
all tasks on the same small set of machines.
This is a much simpler solution to SPARK-979 than #7.
Author: Kay Ousterhout <kayousterhout@gmail.com>
Closes#27 from kayousterhout/randomize and squashes the following commits:
435d817 [Kay Ousterhout] [SPARK-979] Randomize order of offers.
(Ported from https://github.com/apache/incubator-spark/pull/637 )
Author: Sean Owen <sowen@cloudera.com>
Closes#31 from srowen/SPARK-1084.1 and squashes the following commits:
6c4a32c [Sean Owen] Suppress warnings about legitimate unchecked array creations, or change code to avoid it
f35b833 [Sean Owen] Fix two misc javadoc problems
254e8ef [Sean Owen] Fix one new style error introduced in scaladoc warning commit
5b2fce2 [Sean Owen] Fix scaladoc invocation warning, and enable javac warnings properly, with plugin config updates
007762b [Sean Owen] Remove dead scaladoc links
b8ff8cb [Sean Owen] Replace deprecated Ant <tasks> with <target>
If the seed is zero, XORShift generates all zeros, which would create unexpected result.
JIRA: https://spark-project.atlassian.net/browse/SPARK-1129
Author: Xiangrui Meng <meng@databricks.com>
Closes#645 from mengxr/xor and squashes the following commits:
1b086ab [Xiangrui Meng] use MurmurHash3 to set seed in XORShiftRandom
45c6f16 [Xiangrui Meng] minor style change
51f4050 [Xiangrui Meng] use a predefined seed when seed is zero in XORShiftRandom
ClusterScheduler was renamed to TaskSchedulerImpl; this commit
updates comments and tests accordingly.
Author: Kay Ousterhout <kayousterhout@gmail.com>
Closes#9 from kayousterhout/cluster_scheduler_death and squashes the following commits:
d6fd119 [Kay Ousterhout] Remove references to ClusterScheduler.
The aim of the Json4s project is to provide a common API for
Scala JSON libraries. It is Apache-licensed, easier for
downstream distributions to package, and mostly API-compatible
with lift-json. Furthermore, the Jackson-backed implementation
parses faster than lift-json on all but the smallest inputs.
Author: William Benton <willb@redhat.com>
Closes#582 from willb/json4s and squashes the following commits:
7ca62c4 [William Benton] Replace lift-json with json4s-jackson.
[SPARK-1108] This allows us to use, e.g. HBase's TableOutputFormat with PairRDDFunctions.saveAsNewAPIHadoopFile, which otherwise would throw NullPointerException because the output table name hasn't been configured.
Note this bug also affects branch-0.9
Author: Bryn Keller <bryn.keller@intel.com>
Closes#638 from xoltar/SPARK-1108 and squashes the following commits:
7e94e7d [Bryn Keller] Import, comment, and format cleanup per code review
7cbcaa1 [Bryn Keller] For outputformats that are Configurable, call setConf before sending data to them. This allows us to use, e.g. HBase TableOutputFormat, which otherwise would throw NullPointerException because the output table name hasn't been configured
In the previous code, if you had a failing map stage and then tried to
run reduce stages on it repeatedly, the first reduce stage would fail
correctly, but the later ones would mistakenly believe that all map
outputs are available and start failing infinitely with fetch failures
from "null".
These classes can't be migrated:
StorageLevels: impossible to create static fields in Scala
JavaSparkContextVarargsWorkaround: incompatible varargs
JavaAPISuite: should test Java APIs in pure Java (for sanity)
Author: Punya Biswal <pbiswal@palantir.com>
Closes#605 from punya/move-java-sources and squashes the following commits:
25b00b2 [Punya Biswal] Remove redundant type param; reformat
853da46 [Punya Biswal] Use factory method rather than constructor
e5d53d9 [Punya Biswal] Migrate Java code to Scala or move it to src/main/java
The original poster of this bug is @guojc, who opened a PR that preceded this one at https://github.com/apache/incubator-spark/pull/612.
ExternalAppendOnlyMap uses key hash code to order the buffer streams from which spilled files are read back into memory. When a buffer stream is empty, the default hash code for that stream is equal to Int.MaxValue. This is, however, a perfectly legitimate candidate for a key hash code. When reading from a spilled map containing such a key, a hash collision may occur, in which case we attempt to read from an empty stream and throw NoSuchElementException.
The fix is to maintain the invariant that empty buffer streams are never added back to the merge queue to be considered. This guarantees that we never read from an empty buffer stream, ever again.
This PR also includes two new tests for hash collisions.
Author: Andrew Or <andrewor14@gmail.com>
Closes#624 from andrewor14/spilling-bug and squashes the following commits:
9e7263d [Andrew Or] Slightly optimize next()
2037ae2 [Andrew Or] Move a few comments around...
cf95942 [Andrew Or] Remove default value of Int.MaxValue for minKeyHash
c11f03b [Andrew Or] Fix Int.MaxValue hash collision bug in ExternalAppendOnlyMap
21c1a39 [Andrew Or] Add hash collision tests to ExternalAppendOnlyMapSuite
Fixes an error where HDFS URL's cause an exception. Should be merged into master and 0.9.
Author: Patrick Wendell <pwendell@gmail.com>
Closes#625 from pwendell/url-validation and squashes the following commits:
d14bfe3 [Patrick Wendell] SPARK-1111: URL Validation Throws Error for HDFS URL's
Optimized imports and arranged according to scala style guide @
https://cwiki.apache.org/confluence/display/SPARK/Spark+Code+Style+Guide#SparkCodeStyleGuide-Imports
Author: NirmalReddy <nirmal.reddy@imaginea.com>
Author: NirmalReddy <nirmal_reddy2000@yahoo.com>
Closes#613 from NirmalReddy/opt-imports and squashes the following commits:
578b4f5 [NirmalReddy] imported java.lang.Double as JDouble
a2cbcc5 [NirmalReddy] addressed the comments
776d664 [NirmalReddy] Optimized imports in core
SPARK-1076: zipWithIndex and zipWithUniqueId to RDD
Assign ranks to an ordered or unordered data set is a common operation. This could be done by first counting records in each partition and then assign ranks in parallel.
The purpose of assigning ranks to an unordered set is usually to get a unique id for each item, e.g., to map feature names to feature indices. In such cases, the assignment could be done without counting records, saving one spark job.
https://spark-project.atlassian.net/browse/SPARK-1076
== update ==
Because assigning ranks is very similar to Scala's zipWithIndex, I changed the method name to zipWithIndex and put the index in the value field.
Author: Xiangrui Meng <meng@databricks.com>
Closes#578 and squashes the following commits:
52a05e1 [Xiangrui Meng] changed assignRanks to zipWithIndex changed assignUniqueIds to zipWithUniqueId minor updates
756881c [Xiangrui Meng] simplified RankedRDD by implementing assignUniqueIds separately moved couting iterator size to Utils do not count items in the last partition and skip counting if there is only one partition
630868c [Xiangrui Meng] newline
21b434b [Xiangrui Meng] add assignRanks and assignUniqueIds to RDD
SPARK-1072 Use binary search when needed in RangePartioner
Author: Holden Karau <holden@pigscanfly.ca>
Closes#571 and squashes the following commits:
f31a2e1 [Holden Karau] Swith to using CollectionsUtils in Partitioner
4c7a0c3 [Holden Karau] Add CollectionsUtil as suggested by aarondav
7099962 [Holden Karau] Add the binary search to only init once
1bef01d [Holden Karau] CR feedback
a21e097 [Holden Karau] Use binary search if we have more than 1000 elements inside of RangePartitioner
[SPARK-1038] Add more fields in JsonProtocol and add tests that verify the JSON itself
This is a PR for SPARK-1038. Two major changes:
1 add some fields to JsonProtocol which is new and important to standalone-related data structures
2 Use Diff in liftweb.json to verity the stringified Json output for detecting someone mod type T to Option[T]
Author: qqsun8819 <jin.oyj@alibaba-inc.com>
Closes#551 and squashes the following commits:
fdf0b4e [qqsun8819] [SPARK-1038] 1. Change code style for more readable according to rxin review 2. change submitdate hard-coded string to a date object toString for more complexiblity
095a26f [qqsun8819] [SPARK-1038] mod according to review of pwendel, use hard-coded json string for json data validation. Each test use its own json string
0524e41 [qqsun8819] Merge remote-tracking branch 'upstream/master' into json-protocol
d203d5c [qqsun8819] [SPARK-1038] Add more fields in JsonProtocol and add tests that verify the JSON itself
[SPARK-1060] startJettyServer should explicitly use IP information
https://spark-project.atlassian.net/browse/SPARK-1060
In the current implementation, the webserver in Master/Worker is started with
val (srv, bPort) = JettyUtils.startJettyServer("0.0.0.0", port, handlers)
inside startJettyServer:
val server = new Server(currentPort) //here, the Server will take "0.0.0.0" as the hostname, i.e. will always bind to the IP address of the first NIC
this can cause wrong IP binding, e.g. if the host has two NICs, N1 and N2, the user specify the SPARK_LOCAL_IP as the N2's IP address, however, when starting the web server, for the reason stated above, it will always bind to the N1's address
Author: CodingCat <zhunansjtu@gmail.com>
== Merge branch commits ==
commit 6c6d9a8ccc9ec4590678a3b34cb03df19092029d
Author: CodingCat <zhunansjtu@gmail.com>
Date: Thu Feb 6 14:53:34 2014 -0500
startJettyServer should explicitly use IP information
Inform DAG scheduler about all started/finished tasks.
Previously, the DAG scheduler was not always informed
when tasks started and finished. The simplest example here
is for speculated tasks: the DAGScheduler was only told about
the first attempt of a task, meaning that SparkListeners were
also not told about multiple task attempts, so users can't see
what's going on with speculation in the UI. The DAGScheduler
also wasn't always told about finished tasks, so in the UI, some
tasks will never be shown as finished (this occurs, for example,
if a task set gets killed).
The other problem is that the fairness accounting was wrong
-- the number of running tasks in a pool was decreased when a
task set was considered done, even if all of its tasks hadn't
yet finished.
Author: Kay Ousterhout <kayousterhout@gmail.com>
== Merge branch commits ==
commit c8d547d0f7a17f5a193bef05f5872b9f475675c5
Author: Kay Ousterhout <kayousterhout@gmail.com>
Date: Wed Jan 15 16:47:33 2014 -0800
Addressed Reynold's review comments.
Always use a TaskEndReason (remove the option), and explicitly
signal when we don't know the reason. Also, always tell
DAGScheduler (and associated listeners) about started tasks, even
when they're speculated.
commit 3fee1e2e3c06b975ff7f95d595448f38cce97a04
Author: Kay Ousterhout <kayousterhout@gmail.com>
Date: Wed Jan 8 22:58:13 2014 -0800
Fixed broken test and improved logging
commit ff12fcaa2567c5d02b75a1d5db35687225bcd46f
Author: Kay Ousterhout <kayousterhout@gmail.com>
Date: Sun Dec 29 21:08:20 2013 -0800
Inform DAG scheduler about all finished tasks.
Previously, the DAG scheduler was not always informed
when tasks finished. For example, when a task set was
aborted, the DAG scheduler was never told when the tasks
in that task set finished. The DAG scheduler was also
never told about the completion of speculated tasks.
This led to confusion with SparkListeners because information
about the completion of those tasks was never passed on to
the listeners (so in the UI, for example, some tasks will never
be shown as finished).
The other problem is that the fairness accounting was wrong
-- the number of running tasks in a pool was decreased when a
task set was considered done, even if all of its tasks hadn't
yet finished.
Fixed warnings in test compilation.
This commit fixes two problems: a redundant import, and a
deprecated function.
Author: Kay Ousterhout <kayousterhout@gmail.com>
== Merge branch commits ==
commit da9d2e13ee4102bc58888df0559c65cb26232a82
Author: Kay Ousterhout <kayousterhout@gmail.com>
Date: Wed Feb 5 11:41:51 2014 -0800
Fixed warnings in test compilation.
This commit fixes two problems: a redundant import, and a
deprecated function.
Refactor RDD sampling and add randomSplit to RDD (update)
Replace SampledRDD by PartitionwiseSampledRDD, which accepts a RandomSampler instance as input. The current sample with/without replacement can be easily integrated via BernoulliSampler and PoissonSampler. The benefits are:
1) RDD.randomSplit is implemented in the same way, related to https://github.com/apache/incubator-spark/pull/513
2) Stratified sampling and importance sampling can be implemented in the same manner as well.
Unit tests are included for samplers and RDD.randomSplit.
This should performance better than my previous request where the BernoulliSampler creates many Iterator instances:
https://github.com/apache/incubator-spark/pull/513
Author: Xiangrui Meng <meng@databricks.com>
== Merge branch commits ==
commit e8ce957e5f0a600f2dec057924f4a2ca6adba373
Author: Xiangrui Meng <meng@databricks.com>
Date: Mon Feb 3 12:21:08 2014 -0800
more docs to PartitionwiseSampledRDD
commit fbb4586d0478ff638b24bce95f75ff06f713d43b
Author: Xiangrui Meng <meng@databricks.com>
Date: Mon Feb 3 00:44:23 2014 -0800
move XORShiftRandom to util.random and use it in BernoulliSampler
commit 987456b0ee8612fd4f73cb8c40967112dc3c4c2d
Author: Xiangrui Meng <meng@databricks.com>
Date: Sat Feb 1 11:06:59 2014 -0800
relax assertions in SortingSuite because the RangePartitioner has large variance in this case
commit 3690aae416b2dc9b2f9ba32efa465ba7948477f4
Author: Xiangrui Meng <meng@databricks.com>
Date: Sat Feb 1 09:56:28 2014 -0800
test split ratio of RDD.randomSplit
commit 8a410bc933a60c4d63852606f8bbc812e416d6ae
Author: Xiangrui Meng <meng@databricks.com>
Date: Sat Feb 1 09:25:22 2014 -0800
add a test to ensure seed distribution and minor style update
commit ce7e866f674c30ab48a9ceb09da846d5362ab4b6
Author: Xiangrui Meng <meng@databricks.com>
Date: Fri Jan 31 18:06:22 2014 -0800
minor style change
commit 750912b4d77596ed807d361347bd2b7e3b9b7a74
Author: Xiangrui Meng <meng@databricks.com>
Date: Fri Jan 31 18:04:54 2014 -0800
fix some long lines
commit c446a25c38d81db02821f7f194b0ce5ab4ed7ff5
Author: Xiangrui Meng <meng@databricks.com>
Date: Fri Jan 31 17:59:59 2014 -0800
add complement to BernoulliSampler and minor style changes
commit dbe2bc2bd888a7bdccb127ee6595840274499403
Author: Xiangrui Meng <meng@databricks.com>
Date: Fri Jan 31 17:45:08 2014 -0800
switch to partition-wise sampling for better performance
commit a1fca5232308feb369339eac67864c787455bb23
Merge: ac712e4 cf6128f
Author: Xiangrui Meng <meng@databricks.com>
Date: Fri Jan 31 16:33:09 2014 -0800
Merge branch 'sample' of github.com:mengxr/incubator-spark into sample
commit cf6128fb672e8c589615adbd3eaa3cbdb72bd461
Author: Xiangrui Meng <meng@databricks.com>
Date: Sun Jan 26 14:40:07 2014 -0800
set SampledRDD deprecated in 1.0
commit f430f847c3df91a3894687c513f23f823f77c255
Author: Xiangrui Meng <meng@databricks.com>
Date: Sun Jan 26 14:38:59 2014 -0800
update code style
commit a8b5e2021a9204e318c80a44d00c5c495f1befb6
Author: Xiangrui Meng <meng@databricks.com>
Date: Sun Jan 26 12:56:27 2014 -0800
move package random to util.random
commit ab0fa2c4965033737a9e3a9bf0a59cbb0df6a6f5
Author: Xiangrui Meng <meng@databricks.com>
Date: Sun Jan 26 12:50:35 2014 -0800
add Apache headers and update code style
commit 985609fe1a55655ad11966e05a93c18c138a403d
Author: Xiangrui Meng <meng@databricks.com>
Date: Sun Jan 26 11:49:25 2014 -0800
add new lines
commit b21bddf29850a2c006a868869b8f91960a029322
Author: Xiangrui Meng <meng@databricks.com>
Date: Sun Jan 26 11:46:35 2014 -0800
move samplers to random.IndependentRandomSampler and add tests
commit c02dacb4a941618e434cefc129c002915db08be6
Author: Xiangrui Meng <meng@databricks.com>
Date: Sat Jan 25 15:20:24 2014 -0800
add RandomSampler
commit 8ff7ba3c5cf1fc338c29ae8b5fa06c222640e89c
Author: Xiangrui Meng <meng@databricks.com>
Date: Fri Jan 24 13:23:22 2014 -0800
init impl of IndependentlySampledRDD
This fixes SPARK-1043, a bug introduced in 0.9.0
where PySpark couldn't serialize strings > 64kB.
This fix was written by @tyro89 and @bouk in #512.
This commit squashes and rebases their pull request
in order to fix some merge conflicts.
Replace the check for None Option with isDefined and isEmpty in Scala code
Propose to replace the Scala check for Option "!= None" with Option.isDefined and "=== None" with Option.isEmpty.
I think this, using method call if possible then operator function plus argument, will make the Scala code easier to read and understand.
Pass compile and tests.
This fixes an issue where collectAsMap() could
fail when called on a JavaPairRDD that was derived
by transforming a non-JavaPairRDD.
The root problem was that we were creating the
JavaPairRDD's ClassTag by casting a
ClassTag[AnyRef] to a ClassTag[Tuple2[K2, V2]].
To fix this, I cast a ClassTag[Tuple2[_, _]]
instead, since this actually produces a ClassTag
of the appropriate type because ClassTags don't
capture type parameters:
scala> implicitly[ClassTag[Tuple2[_, _]]] == implicitly[ClassTag[Tuple2[Int, Int]]]
res8: Boolean = true
scala> implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[Tuple2[Int, Int]]] == implicitly[ClassTag[Tuple2[Int, Int]]]
res9: Boolean = false
Remove Typesafe Config usage and conf files to fix nested property names
With Typesafe Config we had the subtle problem of no longer allowing
nested property names, which are used for a few of our properties:
http://apache-spark-developers-list.1001551.n3.nabble.com/Config-properties-broken-in-master-td208.html
This PR is for branch 0.9 but should be added into master too.
(cherry picked from commit 34e911ce9a)
Signed-off-by: Patrick Wendell <pwendell@gmail.com>
Prior to this commit, if a task crashes the JVM, the task (and
all other tasks running on that executor) is marked at KILLED rather
than FAILED. As a result, the TaskSetManager will retry the task
indefiniteily rather than failing the job after maxFailures. This
commit fixes that problem by marking tasks as FAILED rather than
killed when an executor is lost.
The downside of this commit is that if task A fails because another
task running on the same executor caused the VM to crash, the failure
will incorrectly be counted as a failure of task A. This should not
be an issue because we typically set maxFailures to 3, and it is
unlikely that a task will be co-located with a JVM-crashing task
multiple times.
Remove now un-needed hostPort option
I noticed this was logging some scary error messages in various places. After I looked into it, this is no longer really used. I removed the option and re-wrote the one remaining use case (it was unnecessary there anyways).
External Sorting for Aggregator and CoGroupedRDDs (Revisited)
(This pull request is re-opened from https://github.com/apache/incubator-spark/pull/303, which was closed because Jenkins / github was misbehaving)
The target issue for this patch is the out-of-memory exceptions triggered by aggregate operations such as reduce, groupBy, join, and cogroup. The existing AppendOnlyMap used by these operations resides purely in memory, and grows with the size of the input data until the amount of allocated memory is exceeded. Under large workloads, this problem is aggravated by the fact that OOM frequently occurs only after a very long (> 1 hour) map phase, in which case the entire job must be restarted.
The solution is to spill the contents of this map to disk once a certain memory threshold is exceeded. This functionality is provided by ExternalAppendOnlyMap, which additionally sorts this buffer before writing it out to disk, and later merges these buffers back in sorted order.
Under normal circumstances in which OOM is not triggered, ExternalAppendOnlyMap is simply a wrapper around AppendOnlyMap and incurs little overhead. Only when the memory usage is expected to exceed the given threshold does ExternalAppendOnlyMap spill to disk.
SPARK-998: Support Launching Driver Inside of Standalone Mode
[NOTE: I need to bring the tests up to date with new changes, so for now they will fail]
This patch provides support for launching driver programs inside of a standalone cluster manager. It also supports monitoring and re-launching of driver programs which is useful for long running, recoverable applications such as Spark Streaming jobs. For those jobs, this patch allows a deployment mode which is resilient to the failure of any worker node, failure of a master node (provided a multi-master setup), and even failures of the applicaiton itself, provided they are recoverable on a restart. Driver information, such as the status and logs from a driver, is displayed in the UI
There are a few small TODO's here, but the code is generally feature-complete. They are:
- Bring tests up to date and add test coverage
- Restarting on failure should be optional and maybe off by default.
- See if we can re-use akka connections to facilitate clients behind a firewall
A sensible place to start for review would be to look at the `DriverClient` class which presents users the ability to launch their driver program. I've also added an example program (`DriverSubmissionTest`) that allows you to test this locally and play around with killing workers, etc. Most of the code is devoted to persisting driver state in the cluster manger, exposing it in the UI, and dealing correctly with various types of failures.
Instructions to test locally:
- `sbt/sbt assembly/assembly examples/assembly`
- start a local version of the standalone cluster manager
```
./spark-class org.apache.spark.deploy.client.DriverClient \
-j -Dspark.test.property=something \
-e SPARK_TEST_KEY=SOMEVALUE \
launch spark://10.99.1.14:7077 \
../path-to-examples-assembly-jar \
org.apache.spark.examples.DriverSubmissionTest 1000 some extra options --some-option-here -X 13
```
- Go in the UI and make sure it started correctly, look at the output etc
- Kill workers, the driver program, masters, etc.
Get rid of `Either[ActorRef, ActorSelection]'
In this pull request, instead of returning an `Either[ActorRef, ActorSelection]`, `registerOrLookup` identifies the remote actor blockingly to obtain an `ActorRef`, or throws an exception if the remote actor doesn't exist or the lookup times out (configured by `spark.akka.lookupTimeout`). This function is only called when an `SparkEnv` is constructed (instantiating driver or executor), so the blocking call is considered acceptable. Executor side `ActorSelection`s/`ActorRef`s to driver side `MapOutputTrackerMasterActor` and `BlockManagerMasterActor` are affected by this pull request.
`ActorSelection` is dangerous and should be used with care. It's only absolutely safe to send messages via an `ActorSelection` when the remote actor is stateless, so that actor incarnation is irrelevant. But as pointed by @ScrapCodes in the comments below, executor exits immediately once the connection to the driver lost, `ActorSelection`s are not harmful in this scenario. So this pull request is mostly a code style patch.
Add way to limit default # of cores used by apps in standalone mode
Also documents the spark.deploy.spreadOut option, and fixes a config option that had a dash in its name.
Although we can send messages via an ActorSelection, it would be better to identify the actor and obtain an ActorRef first, so that we can get informed earlier if the remote actor doesn't exist, and get rid of the annoying Either wrapper.
Further, divide this threshold by the number of tasks running concurrently.
Note that this does not guard against the following scenario: a new task
quickly fills up its share of the memory before old tasks finish spilling
their contents, in which case the total memory used by such maps may exceed
what was specified. Currently, spark.shuffle.safetyFraction mitigates the
effect of this.
Improvements to DStream window ops and refactoring of Spark's CheckpointSuite
- Added a new RDD - PartitionerAwareUnionRDD. Using this RDD, one can take multiple RDDs partitioned by the same partitioner and unify them into a single RDD while preserving the partitioner. So m RDDs with p partitions each will be unified to a single RDD with p partitions and the same partitioner. The preferred location for each partition of the unified RDD will be the most common preferred location of the corresponding partitions of the parent RDDs. For example, location of partition 0 of the unified RDD will be where most of partition 0 of the parent RDDs are located.
- Improved the performance of DStream's reduceByKeyAndWindow and groupByKeyAndWindow. Both these operations work by doing per-batch reduceByKey/groupByKey and then using PartitionerAwareUnionRDD to union the RDDs across the window. This eliminates a shuffle related to the window operation, which can reduce batch processing time by 30-40% for simple workloads.
- Fixed bugs and simplified Spark's CheckpointSuite. Some of the tests were incorrect and unreliable. Added missing tests for ZippedRDD. I can go into greater detail if necessary.
- Added mapSideCombine option to combineByKeyAndWindow.
Also replaced SparkConf.getOrElse with just a "get" that takes a default
value, and added getInt, getLong, etc to make code that uses this
simpler later on.
Approximate distinct count
Added countApproxDistinct() to RDD and countApproxDistinctByKey() to PairRDDFunctions to approximately count distinct number of elements and distinct number of values per key, respectively. Both functions use HyperLogLog from stream-lib for counting. Both functions take a parameter that controls the trade-off between accuracy and memory consumption. Also added Scala docs and test suites for both methods.
Bug fixes for file input stream and checkpointing
- Fixed bugs in the file input stream that led the stream to fail due to transient HDFS errors (listing files when a background thread it deleting fails caused errors, etc.)
- Updated Spark's CheckpointRDD and Streaming's CheckpointWriter to use SparkContext.hadoopConfiguration, to allow checkpoints to be written to any HDFS compatible store requiring special configuration.
- Changed the API of SparkContext.setCheckpointDir() - eliminated the unnecessary 'useExisting' parameter. Now SparkContext will always create a unique subdirectory within the user specified checkpoint directory. This is to ensure that previous checkpoint files are not accidentally overwritten.
- Fixed bug where setting checkpoint directory as a relative local path caused the checkpointing to fail.
- Got rid of global SparkContext.globalConf
- Pass SparkConf to serializers and compression codecs
- Made SparkConf public instead of private[spark]
- Improved API of SparkContext and SparkConf
- Switched executor environment vars to be passed through SparkConf
- Fixed some places that were still using system properties
- Fixed some tests, though others are still failing
This still fails several tests in core, repl and streaming, likely due
to properties not being set or cleared correctly (some of the tests run
fine in isolation).
The rest of the SparkListener events are named with "SparkListener"
as the prefix of the name; this commit renames the StageCompleted
event to SparkListenerStageCompleted for consistency.