Commit graph

983 commits

Author SHA1 Message Date
Patrick Wendell 796977acdb SPARK-1244: Throw exception if map output status exceeds frame size
This is a very small change on top of @andrewor14's patch in #147.

Author: Patrick Wendell <pwendell@gmail.com>
Author: Andrew Or <andrewor14@gmail.com>

Closes #152 from pwendell/akka-frame and squashes the following commits:

e5fb3ff [Patrick Wendell] Reversing test order
393af4c [Patrick Wendell] Small improvement suggested by Andrew Or
8045103 [Patrick Wendell] Breaking out into two tests
2b4e085 [Patrick Wendell] Consolidate Executor use of akka frame size
c9b6109 [Andrew Or] Simplify test + make access to akka frame size more modular
281d7c9 [Andrew Or] Throw exception on spark.akka.frameSize exceeded + Unit tests
2014-03-17 14:03:32 -07:00
CodingCat dc9654638f SPARK-1240: handle the case of empty RDD when takeSample
https://spark-project.atlassian.net/browse/SPARK-1240

It seems that the current implementation does not handle the empty RDD case when run takeSample

In this patch, before calling sample() inside takeSample API, I add a checker for this case and returns an empty Array when it's a empty RDD; also in sample(), I add a checker for the invalid fraction value

In the test case, I also add several lines for this case

Author: CodingCat <zhunansjtu@gmail.com>

Closes #135 from CodingCat/SPARK-1240 and squashes the following commits:

fef57d4 [CodingCat] fix the same problem in PySpark
36db06b [CodingCat] create new test cases for takeSample from an empty red
810948d [CodingCat] further fix
a40e8fb [CodingCat] replace if with require
ad483fd [CodingCat] handle the case with empty RDD when take sample
2014-03-16 22:14:59 -07:00
Reynold Xin f5486e9f75 SPARK-1255: Allow user to pass Serializer object instead of class name for shuffle.
This is more general than simply passing a string name and leaves more room for performance optimizations.

Note that this is technically an API breaking change in the following two ways:
1. The shuffle serializer specification in ShuffleDependency now require an object instead of a String (of the class name), but I suspect nobody else in this world has used this API other than me in GraphX and Shark.
2. Serializer's in Spark from now on are required to be serializable.

Author: Reynold Xin <rxin@apache.org>

Closes #149 from rxin/serializer and squashes the following commits:

5acaccd [Reynold Xin] Properly call serializer's constructors.
2a8d75a [Reynold Xin] Added more documentation for the serializer option in ShuffleDependency.
7420185 [Reynold Xin] Allow user to pass Serializer object instead of class name for shuffle.
2014-03-16 09:57:21 -07:00
liguoqiang 5d1ec64e79 Fix #SPARK-1149 Bad partitioners can cause Spark to hang
Author: liguoqiang <liguoqiang@rd.tuan800.com>

Closes #44 from witgo/SPARK-1149 and squashes the following commits:

3dcdcaf [liguoqiang] Merge branch 'master' into SPARK-1149
8425395 [liguoqiang] Merge remote-tracking branch 'upstream/master' into SPARK-1149
3dad595 [liguoqiang] review comment
e3e56aa [liguoqiang] Merge branch 'master' into SPARK-1149
b0d5c07 [liguoqiang] review comment
d0a6005 [liguoqiang] review comment
3395ee7 [liguoqiang] Merge remote-tracking branch 'upstream/master' into SPARK-1149
ac006a3 [liguoqiang] code Formatting
3feb3a8 [liguoqiang] Merge branch 'master' into SPARK-1149
adc443e [liguoqiang] partitions check  bugfix
928e1e3 [liguoqiang] Added a unit test for PairRDDFunctions.lookup with bad partitioner
db6ecc5 [liguoqiang] Merge branch 'master' into SPARK-1149
1e3331e [liguoqiang] Merge branch 'master' into SPARK-1149
3348619 [liguoqiang] Optimize performance for partitions check
61e5a87 [liguoqiang] Merge branch 'master' into SPARK-1149
e68210a [liguoqiang] add partition index check to submitJob
3a65903 [liguoqiang] make the code more readable
6bb725e [liguoqiang] fix #SPARK-1149 Bad partitioners can cause Spark to hang
2014-03-12 13:00:04 -07:00
Cheng Lian 0b7b7fd45c [SPARK-1194] Fix the same-RDD rule for cache replacement
SPARK-1194: https://spark-project.atlassian.net/browse/SPARK-1194

In the current implementation, when selecting candidate blocks to be swapped out, once we find a block from the same RDD that the block to be stored belongs to, cache eviction fails  and aborts.

In this PR, we keep selecting blocks *not* from the RDD that the block to be stored belongs to until either enough free space can be ensured (cache eviction succeeds) or all such blocks are checked (cache eviction fails).

Author: Cheng Lian <lian.cs.zju@gmail.com>

Closes #96 from liancheng/fix-spark-1194 and squashes the following commits:

2524ab9 [Cheng Lian] Added regression test case for SPARK-1194
6e40c22 [Cheng Lian] Remove redundant comments
40cdcb2 [Cheng Lian] Bug fix, and addressed PR comments from @mridulm
62c92ac [Cheng Lian] Fixed SPARK-1194 https://spark-project.atlassian.net/browse/SPARK-1194
2014-03-07 23:26:46 -08:00
Prashant Sharma 6e730edcde Spark 1165 rdd.intersection in python and java
Author: Prashant Sharma <prashant.s@imaginea.com>
Author: Prashant Sharma <scrapcodes@gmail.com>

Closes #80 from ScrapCodes/SPARK-1165/RDD.intersection and squashes the following commits:

9b015e9 [Prashant Sharma] Added a note, shuffle is required for intersection.
1fea813 [Prashant Sharma] correct the lines wrapping
d0c71f3 [Prashant Sharma] SPARK-1165 RDD.intersection in java
d6effee [Prashant Sharma] SPARK-1165 Implemented RDD.intersection in python.
2014-03-07 18:48:07 -08:00
Thomas Graves b7cd9e992c SPARK-1195: set map_input_file environment variable in PipedRDD
Hadoop uses the config mapreduce.map.input.file to indicate the input filename to the map when the input split is of type FileSplit. Some of the hadoop input and output formats set or use this config. This config can also be used by user code.
PipedRDD runs an external process and the configs aren't available to that process. Hadoop Streaming does something very similar and the way they make configs available is exporting them into the environment replacing '.' with '_'. Spark should also export this variable when launching the pipe command so the user code has access to that config.
Note that the config mapreduce.map.input.file is the new one, the old one which is deprecated but not yet removed is map.input.file. So we should handle both.

Perhaps it would be better to abstract this out somehow so it goes into the HadoopParition code?

Author: Thomas Graves <tgraves@apache.org>

Closes #94 from tgravescs/map_input_file and squashes the following commits:

cc97a6a [Thomas Graves] Update test to check for existence of command, add a getPipeEnvVars function to HadoopRDD
e3401dc [Thomas Graves] Merge remote-tracking branch 'upstream/master' into map_input_file
2ba805e [Thomas Graves] set map_input_file environment variable in PipedRDD
2014-03-07 10:36:55 -08:00
Patrick Wendell 33baf14b04 Small clean-up to flatmap tests 2014-03-06 17:57:31 -08:00
Sandy Ryza 328c73d037 SPARK-1197. Change yarn-standalone to yarn-cluster and fix up running on YARN docs
This patch changes "yarn-standalone" to "yarn-cluster" (but still supports the former).  It also cleans up the Running on YARN docs and adds a section on how to view logs.

Author: Sandy Ryza <sandy@cloudera.com>

Closes #95 from sryza/sandy-spark-1197 and squashes the following commits:

563ef3a [Sandy Ryza] Review feedback
6ad06d4 [Sandy Ryza] Change yarn-standalone to yarn-cluster and fix up running on YARN docs
2014-03-06 17:12:58 -08:00
Thomas Graves 7edbea41b4 SPARK-1189: Add Security to Spark - Akka, Http, ConnectionManager, UI use servlets
resubmit pull request.  was https://github.com/apache/incubator-spark/pull/332.

Author: Thomas Graves <tgraves@apache.org>

Closes #33 from tgravescs/security-branch-0.9-with-client-rebase and squashes the following commits:

dfe3918 [Thomas Graves] Fix merge conflict since startUserClass now using runAsUser
05eebed [Thomas Graves] Fix dependency lost in upmerge
d1040ec [Thomas Graves] Fix up various imports
05ff5e0 [Thomas Graves] Fix up imports after upmerging to master
ac046b3 [Thomas Graves] Merge remote-tracking branch 'upstream/master' into security-branch-0.9-with-client-rebase
13733e1 [Thomas Graves] Pass securityManager and SparkConf around where we can. Switch to use sparkConf for reading config whereever possible. Added ConnectionManagerSuite unit tests.
4a57acc [Thomas Graves] Change UI createHandler routines to createServlet since they now return servlets
2f77147 [Thomas Graves] Rework from comments
50dd9f2 [Thomas Graves] fix header in SecurityManager
ecbfb65 [Thomas Graves] Fix spacing and formatting
b514bec [Thomas Graves] Fix reference to config
ed3d1c1 [Thomas Graves] Add security.md
6f7ddf3 [Thomas Graves] Convert SaslClient and SaslServer to scala, change spark.authenticate.ui to spark.ui.acls.enable, and fix up various other things from review comments
2d9e23e [Thomas Graves] Merge remote-tracking branch 'upstream/master' into security-branch-0.9-with-client-rebase_rework
5721c5a [Thomas Graves] update AkkaUtilsSuite test for the actorSelection changes, fix typos based on comments, and remove extra lines I missed in rebase from AkkaUtils
f351763 [Thomas Graves] Add Security to Spark - Akka, Http, ConnectionManager, UI to use servlets
2014-03-06 18:27:50 -06:00
Kyle Ellrott 40566e10aa SPARK-942: Do not materialize partitions when DISK_ONLY storage level is used
This is a port of a pull request original targeted at incubator-spark: https://github.com/apache/incubator-spark/pull/180

Essentially if a user returns a generative iterator (from a flatMap operation), when trying to persist the data, Spark would first unroll the iterator into an ArrayBuffer, and then try to figure out if it could store the data. In cases where the user provided an iterator that generated more data then available memory, this would case a crash. With this patch, if the user requests a persist with a 'StorageLevel.DISK_ONLY', the iterator will be unrolled as it is inputed into the serializer.

To do this, two changes where made:
1) The type of the 'values' argument in the putValues method of the BlockStore interface was changed from ArrayBuffer to Iterator (and all code interfacing with this method was modified to connect correctly.
2) The JavaSerializer now calls the ObjectOutputStream 'reset' method every 1000 objects. This was done because the ObjectOutputStream caches objects (thus preventing them from being GC'd) to write more compact serialization. If reset is never called, eventually the memory fills up, if it is called too often then the serialization streams become much larger because of redundant class descriptions.

Author: Kyle Ellrott <kellrott@gmail.com>

Closes #50 from kellrott/iterator-to-disk and squashes the following commits:

9ef7cb8 [Kyle Ellrott] Fixing formatting issues.
60e0c57 [Kyle Ellrott] Fixing issues (formatting, variable names, etc.) from review comments
8aa31cd [Kyle Ellrott] Merge ../incubator-spark into iterator-to-disk
33ac390 [Kyle Ellrott] Merge branch 'iterator-to-disk' of github.com:kellrott/incubator-spark into iterator-to-disk
2f684ea [Kyle Ellrott] Refactoring the BlockManager to replace the Either[Either[A,B]] usage. Now using trait 'Values'. Also modified BlockStore.putBytes call to return PutResult, so that it behaves like putValues.
f70d069 [Kyle Ellrott] Adding docs for spark.serializer.objectStreamReset configuration
7ccc74b [Kyle Ellrott] Moving the 'LargeIteratorSuite' to simply test persistance of iterators. It doesn't try to invoke a OOM error any more
16a4cea [Kyle Ellrott] Streamlined the LargeIteratorSuite unit test. It should now run in ~25 seconds. Confirmed that it still crashes an unpatched copy of Spark.
c2fb430 [Kyle Ellrott] Removing more un-needed array-buffer to iterator conversions
627a8b7 [Kyle Ellrott] Wrapping a few long lines
0f28ec7 [Kyle Ellrott] Adding second putValues to BlockStore interface that accepts an ArrayBuffer (rather then an Iterator). This will allow BlockStores to have slightly different behaviors dependent on whether they get an Iterator or ArrayBuffer. In the case of the MemoryStore, it needs to duplicate and cache an Iterator into an ArrayBuffer, but if handed a ArrayBuffer, it can skip the duplication.
656c33e [Kyle Ellrott] Fixing the JavaSerializer to read from the SparkConf rather then the System property.
8644ee8 [Kyle Ellrott] Merge branch 'master' into iterator-to-disk
00c98e0 [Kyle Ellrott] Making the Java ObjectStreamSerializer reset rate configurable by the system variable 'spark.serializer.objectStreamReset', default is not 10000.
40fe1d7 [Kyle Ellrott] Removing rouge space
31fe08e [Kyle Ellrott] Removing un-needed semi-colons
9df0276 [Kyle Ellrott] Added check to make sure that streamed-to-dist RDD actually returns good data in the LargeIteratorSuite
a6424ba [Kyle Ellrott] Wrapping long line
2eeda75 [Kyle Ellrott] Fixing dumb mistake ("||" instead of "&&")
0e6f808 [Kyle Ellrott] Deleting temp output directory when done
95c7f67 [Kyle Ellrott] Simplifying StorageLevel checks
56f71cd [Kyle Ellrott] Merge branch 'master' into iterator-to-disk
44ec35a [Kyle Ellrott] Adding some comments.
5eb2b7e [Kyle Ellrott] Changing the JavaSerializer reset to occur every 1000 objects.
f403826 [Kyle Ellrott] Merge branch 'master' into iterator-to-disk
81d670c [Kyle Ellrott] Adding unit test for straight to disk iterator methods.
d32992f [Kyle Ellrott] Merge remote-tracking branch 'origin/master' into iterator-to-disk
cac1fad [Kyle Ellrott] Fixing MemoryStore, so that it converts incoming iterators to ArrayBuffer objects. This was previously done higher up the stack.
efe1102 [Kyle Ellrott] Changing CacheManager and BlockManager to pass iterators directly to the serializer when a 'DISK_ONLY' persist is called. This is in response to SPARK-942.
2014-03-06 14:51:19 -08:00
Prashant Sharma 181ec50307 [java8API] SPARK-964 Investigate the potential for using JDK 8 lambda expressions for the Java/Scala APIs
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.
2014-03-03 22:31:30 -08:00
Bryn Keller 923dba5096 Added a unit test for PairRDDFunctions.lookup
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
2014-03-03 16:38:57 -08:00
Kay Ousterhout b55cade853 Remove the remoteFetchTime metric.
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.
2014-03-03 16:12:00 -08:00
CodingCat 3a8b698e96 [SPARK-1100] prevent Spark from overwriting directory silently
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
2014-03-01 17:27:54 -08:00
Kay Ousterhout 556c56689b [SPARK-979] Randomize order of offers.
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.
2014-03-01 11:24:22 -08:00
Sean Owen 12bbca2065 SPARK 1084.1 (resubmitted)
(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>
2014-02-27 11:12:21 -08:00
Xiangrui Meng 5a3ad107c0 SPARK-1129: use a predefined seed when seed is zero in XORShiftRandom
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
2014-02-26 23:22:30 -08:00
Kay Ousterhout 71f69d66ce Remove references to ClusterScheduler (SPARK-1140)
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.
2014-02-26 22:52:42 -08:00
William Benton fbedc8eff2 SPARK-1078: Replace lift-json with json4s-jackson.
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.
2014-02-26 10:09:50 -08:00
Bryn Keller 4d88030486 For outputformats that are Configurable, call setConf before sending data to them.
[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
2014-02-24 17:35:22 -08:00
Matei Zaharia cd32d5e4de SPARK-1124: Fix infinite retries of reduce stage when a map stage failed
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".
2014-02-23 23:48:32 -08:00
Punya Biswal 29ac7ea52f Migrate Java code to Scala or move it to src/main/java
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
2014-02-22 17:53:48 -08:00
Andrew Or fefd22f4c3 [SPARK-1113] External spilling - fix Int.MaxValue hash code collision bug
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
2014-02-21 20:05:39 -08:00
Patrick Wendell 45b15e27a8 SPARK-1111: URL Validation Throws Error for HDFS URL's
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
2014-02-21 11:11:55 -08:00
NirmalReddy ccb327a49a Optimized imports
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
2014-02-18 14:44:36 -08:00
Xiangrui Meng e733d655df Merge pull request #578 from mengxr/rank.
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
2014-02-12 00:42:42 -08:00
Holden Karau b0dab1bb9f Merge pull request #571 from holdenk/switchtobinarysearch.
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
2014-02-11 14:48:59 -08:00
qqsun8819 afc8f3cb9a Merge pull request #551 from qqsun8819/json-protocol.
[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
2014-02-09 13:57:29 -08:00
CodingCat b6dba10ae5 Merge pull request #556 from CodingCat/JettyUtil. Closes #556.
[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
2014-02-08 23:39:17 -08:00
Andrew Ash 3a9d82cc9e Merge pull request #506 from ash211/intersection. Closes #506.
SPARK-1062 Add rdd.intersection(otherRdd) method

Author: Andrew Ash <andrew@andrewash.com>

== Merge branch commits ==

commit 5d9982b171b9572649e9828f37ef0b43f0242912
Author: Andrew Ash <andrew@andrewash.com>
Date:   Thu Feb 6 18:11:45 2014 -0800

    Minor fixes

    - style: (v,null) => (v, null)
    - mention the shuffle in Javadoc

commit b86d02f14e810902719cef893cf6bfa18ff9acb0
Author: Andrew Ash <andrew@andrewash.com>
Date:   Sun Feb 2 13:17:40 2014 -0800

    Overload .intersection() for numPartitions and custom Partitioner

commit bcaa34911fcc6bb5bc5e4f9fe46d1df73cb71c09
Author: Andrew Ash <andrew@andrewash.com>
Date:   Sun Feb 2 13:05:40 2014 -0800

    Better naming of parameters in intersection's filter

commit b10a6af2d793ec6e9a06c798007fac3f6b860d89
Author: Andrew Ash <andrew@andrewash.com>
Date:   Sat Jan 25 23:06:26 2014 -0800

    Follow spark code format conventions of tab => 2 spaces

commit 965256e4304cca514bb36a1a36087711dec535ec
Author: Andrew Ash <andrew@andrewash.com>
Date:   Fri Jan 24 00:28:01 2014 -0800

    Add rdd.intersection(otherRdd) method
2014-02-06 22:39:08 -08:00
Kay Ousterhout 18ad59e2c6 Merge pull request #321 from kayousterhout/ui_kill_fix. Closes #321.
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.
2014-02-06 16:10:48 -08:00
Kay Ousterhout cc14ba974c Merge pull request #544 from kayousterhout/fix_test_warnings. Closes #544.
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.
2014-02-05 12:44:24 -08:00
Xiangrui Meng 23af00f9e0 Merge pull request #528 from mengxr/sample. Closes #528.
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
2014-02-03 13:02:09 -08:00
Josh Rosen 1381fc72f7 Switch from MUTF8 to UTF8 in PySpark serializers.
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.
2014-01-28 20:20:08 -08:00
Reynold Xin f16c21e22f Merge pull request #490 from hsaputra/modify_checkoption_with_isdefined
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.
2014-01-27 14:24:06 -08:00
Josh Rosen 740e865f40 Fix ClassCastException in JavaPairRDD.collectAsMap() (SPARK-1040)
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
2014-01-25 16:41:12 -08:00
Henry Saputra 90ea9d5a8f Replace the code to check for Option != None with Option.isDefined call in Scala code.
This hopefully will make the code cleaner.
2014-01-21 23:22:10 -08:00
CodingCat 29f4b6a2d9 fix for SPARK-1027
change TestClient & Worker to Some("xxx")

kill manager if it is started

remove unnecessary .get when fetch "SPARK_HOME" values
2014-01-20 02:50:30 -05:00
Patrick Wendell bf5699543b Merge pull request #462 from mateiz/conf-file-fix
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>
2014-01-18 16:20:00 -08:00
Kay Ousterhout 718a13c179 Updated unit test comment 2014-01-15 23:46:14 -08:00
Kay Ousterhout a268d63411 Fail rather than hanging if a task crashes the JVM.
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.
2014-01-15 16:03:40 -08:00
Patrick Wendell 23034798d7 Add missing header files 2014-01-14 01:17:13 -08:00
Patrick Wendell 68641bce61 Merge pull request #413 from rxin/scaladoc
Adjusted visibility of various components and documentation for 0.9.0 release.
2014-01-13 22:54:13 -08:00
Patrick Wendell 0ca0d4d657 Merge pull request #401 from andrewor14/master
External sorting - Add number of bytes spilled to Web UI

Additionally, update test suite for external sorting to induce spilling.
2014-01-13 22:32:21 -08:00
Reynold Xin 33022d6656 Adjusted visibility of various components. 2014-01-13 19:58:53 -08:00
Reynold Xin e6ed13f255 Merge pull request #397 from pwendell/host-port
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).
2014-01-12 22:35:14 -08:00
Henry Saputra 5a8abfb70e Address code review concerns and comments. 2014-01-12 19:15:09 -08:00
Patrick Wendell 0bb33076e2 Removing mentions in tests 2014-01-12 16:53:58 -08:00
Henry Saputra 91a563608e Merge branch 'master' into remove_simpleredundantreturn_scala 2014-01-12 10:34:13 -08:00
Henry Saputra 93a65e5fde Remove simple redundant return statement for Scala methods/functions:
-) Only change simple return statements at the end of method
-) Ignore the complex if-else check
-) Ignore the ones inside synchronized
2014-01-12 10:30:04 -08:00
Andrew Or e6447152b3 Induce spilling in ExternalAppendOnlyMapSuite 2014-01-10 18:33:48 -08:00
Patrick Wendell d37408f39c Merge pull request #377 from andrewor14/master
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.
2014-01-10 16:25:01 -08:00
Reynold Xin 0eaf01c5ed Merge pull request #369 from pillis/master
SPARK-961 Add a Vector.random() method

Added method and testcases
2014-01-10 15:32:19 -08:00
Pillis 8d021b42bc SPARK-961. Add a Vector.random() method - update 1 2014-01-10 00:07:36 -08:00
Andrew Or d76e1f90a8 Merge github.com:apache/incubator-spark
Conflicts:
	core/src/main/scala/org/apache/spark/SparkEnv.scala
	streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java
2014-01-09 21:38:48 -08:00
Patrick Wendell d86a85e9ca Merge pull request #293 from pwendell/standalone-driver
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.
2014-01-09 18:37:52 -08:00
Pillis 181471906e SPARK-961 Add a Vector.random() method 2014-01-09 10:16:19 +01:00
Matei Zaharia a01f3401e3 Use typed getters for configuration settings 2014-01-09 00:07:29 -08:00
Patrick Wendell bc81ce040d Merge remote-tracking branch 'apache-github/master' into standalone-driver
Conflicts:
	core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala
	pom.xml
2014-01-08 00:38:31 -08:00
Patrick Wendell f5f12dc282 Merge pull request #336 from liancheng/akka-remote-lookup
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.
2014-01-07 21:56:35 -08:00
Matei Zaharia d75dc428da Merge pull request #350 from mateiz/standalone-limit
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.
2014-01-08 00:30:03 -05:00
Mark Hamstra 86ed1ad252 Fix BlockManagerSuite#after 2014-01-07 16:39:37 -08:00
Matei Zaharia 2c421749ea Address review comments 2014-01-07 19:30:23 -05:00
Patrick Wendell e21a707a13 Adding unit tests and some refactoring to promote testability. 2014-01-07 15:39:47 -08:00
Matei Zaharia 044c8ad3a4 Fix unit test compilation 2014-01-07 16:12:20 -05:00
Patrick Wendell c0498f9265 Merge remote-tracking branch 'apache-github/master' into standalone-driver
Conflicts:
	core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala
	core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala
	core/src/main/scala/org/apache/spark/deploy/master/Master.scala
	core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
	core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala
2014-01-06 17:29:21 -08:00
Patrick Wendell 9272a004af Fix test breaking downstream builds 2014-01-06 13:03:19 -08:00
Lian, Cheng eb24684748 Fixed test suite compilation errors 2014-01-06 11:26:59 +08:00
Lian, Cheng 5c152e3e21 Fixed several compilation errors in test suites 2014-01-06 10:39:05 +08:00
Lian, Cheng a4048ff31e Get rid of `Either[ActorRef, ActorSelection]'
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.
2014-01-06 09:18:17 +08:00
Andrew Or 2db7884f6f Address Mark's comments 2014-01-04 01:20:09 -08:00
Andrew Or 4296d96c82 Assign spill threshold as a fraction of maximum memory
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.
2014-01-04 00:00:57 -08:00
Patrick Wendell 604fad9c39 Merge remote-tracking branch 'apache-github/master' into remove-binaries
Conflicts:
	core/src/test/scala/org/apache/spark/DriverSuite.scala
	docs/python-programming-guide.md
2014-01-03 21:29:33 -08:00
Patrick Wendell 9e6f3bdcda Changes on top of Prashant's patch.
Closes #316
2014-01-03 18:30:17 -08:00
Andrew Or 838b0e7d15 Refactor using SparkConf 2014-01-03 16:13:40 -08:00
Patrick Wendell 4ae101ff38 Merge pull request #317 from ScrapCodes/spark-915-segregate-scripts
Spark-915 segregate scripts
2014-01-03 11:24:35 -08:00
Prashant Sharma 74ba97fcf7 sbin/spark-class* -> bin/spark-class* 2014-01-03 15:08:01 +05:30
Prashant Sharma bc311bb826 Restored the previously removed test 2014-01-03 14:52:37 +05:30
Prashant Sharma 94f2fffa23 fixed review comments 2014-01-03 14:43:37 +05:30
Prashant Sharma b4bb80002b Merge branch 'master' into spark-1002-remove-jars 2014-01-03 12:12:04 +05:30
Andrew Or df413e996f Merge remote-tracking branch 'spark/master'
Conflicts:
	core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala
2014-01-02 20:51:23 -08:00
Patrick Wendell 588a1695f4 Merge pull request #297 from tdas/window-improvement
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.
2014-01-02 13:20:54 -08:00
Prashant Sharma 980afd280a Merge branch 'scripts-reorg' of github.com:shane-huang/incubator-spark into spark-915-segregate-scripts
Conflicts:
	bin/spark-shell
	core/pom.xml
	core/src/main/scala/org/apache/spark/SparkContext.scala
	core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
	core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala
	core/src/test/scala/org/apache/spark/DriverSuite.scala
	python/run-tests
	sbin/compute-classpath.sh
	sbin/spark-class
	sbin/stop-slaves.sh
2014-01-02 17:55:21 +05:30
Prashant Sharma 08ec10de17 Removed a repeated test and changed tests to not use uncommons jar 2014-01-02 17:32:11 +05:30
Prashant Sharma 436f3d2856 ignoring tests for now, contrary to what I assumed these tests make sense given what they are testing. 2014-01-02 16:08:35 +05:30
Matei Zaharia e2c68642c6 Miscellaneous fixes from code review.
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.
2014-01-01 22:03:39 -05:00
Matei Zaharia 45ff8f413d Merge remote-tracking branch 'apache/master' into conf2
Conflicts:
	core/src/main/scala/org/apache/spark/SparkContext.scala
	core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala
	core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala
2014-01-01 21:25:00 -05:00
Patrick Wendell f8d245bdfc Merge remote-tracking branch 'apache-github/master' into log4j-fix-2
Conflicts:
	streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala
2014-01-01 16:10:51 -08:00
Andrew Or 92c304fd03 Simplify ExternalAppendOnlyMap on the assumption that the mergeCombiners function is specified 2014-01-01 11:42:33 -08:00
Matei Zaharia 0e5b2adb5c Merge remote-tracking branch 'apache/master' into conf2
Conflicts:
	project/SparkBuild.scala
2014-01-01 13:28:54 -05:00
Reynold Xin 8b8e70ebde Merge pull request #73 from falaki/ApproximateDistinctCount
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.
2013-12-31 17:48:24 -08:00
Andrew Or 8bbe08b21e Merge branch 'master' of github.com:andrewor14/incubator-spark 2013-12-31 17:26:26 -08:00
Andrew Or 53d8d36684 Add support and test for null keys in ExternalAppendOnlyMap
Also add safeguard against use of destructively sorted AppendOnlyMap
2013-12-31 17:19:02 -08:00
Matei Zaharia ba9338f104 Merge remote-tracking branch 'apache/master' into conf2
Conflicts:
	core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala
	streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala
	streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala
2013-12-31 18:23:14 -05:00
Patrick Wendell 55b7e2fdff Merge pull request #289 from tdas/filestream-fix
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.
2013-12-31 10:12:51 -08:00
Patrick Wendell 4d009dcac6 Removing use in test 2013-12-31 00:01:44 -08:00
Aaron Davidson daa7792ad6 Refactor SamplingSizeTracker into SizeTrackingAppendOnlyMap 2013-12-30 23:39:02 -08:00
Hossein Falaki d6cded7155 Added Java unit tests for countApproxDistinct and countApproxDistinctByKey 2013-12-30 19:32:05 -08:00
Hossein Falaki a7de8e9b1c Renamed countDistinct and countDistinctByKey methods to include Approx 2013-12-30 19:28:03 -08:00
Matei Zaharia 0fa5809768 Updated docs for SparkConf and handled review comments 2013-12-30 22:17:28 -05:00
Hossein Falaki d50ccc5ca9 Using origin version 2013-12-30 15:08:34 -08:00
Andrew Or 2a48d71528 Add test suite for ExternalAppendOnlyMap 2013-12-29 21:56:13 -08:00
Matei Zaharia 994f080f8a Properly show Spark properties on web UI, and change app name property 2013-12-29 22:19:33 -05:00
Matei Zaharia 11540b798d Added tests for SparkConf and fixed a bug
Typesafe Config caches system properties the first time it's invoked
by default, ignoring later changes unless you do something special
2013-12-29 18:44:06 -05:00
Matei Zaharia b4ceed40d6 Merge remote-tracking branch 'origin/master' into conf2
Conflicts:
	core/src/main/scala/org/apache/spark/SparkContext.scala
	core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
	core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
	core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManager.scala
	core/src/main/scala/org/apache/spark/scheduler/local/LocalScheduler.scala
	core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala
	core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala
	core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala
	new-yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
	streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala
	streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala
	streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala
	streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala
	streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala
	streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala
	streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala
	streaming/src/test/scala/org/apache/spark/streaming/WindowOperationsSuite.scala
2013-12-29 15:08:08 -05:00
Patrick Wendell 35f6dc252a Changes to allow fate sharing of drivers/executors and workers. 2013-12-29 11:14:36 -08:00
Matei Zaharia 20631348d1 Fix other failing tests 2013-12-28 23:17:58 -05:00
Matei Zaharia 642029e7f4 Various fixes to configuration code
- 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).
2013-12-28 17:13:15 -05:00
Kay Ousterhout b4619e509b Changed naming of StageCompleted event to be consistent
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.
2013-12-27 17:45:20 -08:00
Aaron Davidson 2a7b3511f4 Add Apache headers 2013-12-27 10:55:16 -08:00
Andrew Or 64b2d54a02 Move maps to util, and refactor more 2013-12-26 23:40:07 -08:00
Aaron Davidson 804beb43be SamplingSizeTracker + Map + test suite 2013-12-26 23:40:07 -08:00
Patrick Wendell 55c8bb741c Intermediate clean-up of tests to appease jenkins 2013-12-26 15:43:15 -08:00
Tathagata Das 3579647cdc Merge branch 'apache-master' into window-improvement 2013-12-26 12:12:10 -08:00
Tathagata Das c4a54f51b5 Merge branch 'master' into window-improvement 2013-12-26 12:03:11 -08:00
Patrick Wendell 85a344b4f0 Merge pull request #127 from kayousterhout/consolidate_schedulers
Deduplicate Local and Cluster schedulers.

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.

Unfortunately the diff makes this change look much more invasive than it is -- TaskScheduler.scala is only superficially changed (names updated, overrides removed) from the old ClusterScheduler.scala, and the same with
TaskSetManager.scala.

Thanks @rxin for suggesting this change!
2013-12-24 16:35:06 -08:00
Tathagata Das 9f79fd89dc Merge branch 'apache-master' into filestream-fix 2013-12-24 11:38:17 -08:00
Prashant Sharma 2573add94c spark-544, introducing SparkConf and related configuration overhaul. 2013-12-25 00:09:36 +05:30
Reynold Xin 11107c9de5 Merge pull request #244 from leftnoteasy/master
Added SPARK-968 implementation for review

Added SPARK-968 implementation for review
2013-12-23 10:38:20 -08:00
Kay Ousterhout b7bfae1afe Correctly merged in maxTaskFailures fix 2013-12-22 07:34:44 -08:00
Kay Ousterhout b8ae096a40 Fix build error in test 2013-12-21 23:28:48 -08:00
Kay Ousterhout 30186aa264 Renamed ClusterScheduler to TaskSchedulerImpl 2013-12-20 14:58:04 -08:00
Kay Ousterhout c06945cfe0 Merge remote branch 'upstream/master' into consolidate_schedulers
Conflicts:
	core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManager.scala
	core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
	core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala
2013-12-20 14:39:30 -08:00
Tathagata Das 61f4bbda0d Added tests for PartitionerAwareUnionRDD in the CheckpointSuite. Refactored CheckpointSuite to make the tests simpler and more reliable. Added missing test for ZippedRDD. 2013-12-20 00:41:47 -08:00
Patrick Wendell eca68d4425 Merge pull request #272 from tmyklebu/master
Track and report task result serialisation time.

 - DirectTaskResult now has a ByteBuffer valueBytes instead of a T value.
 - DirectTaskResult now has a member function T value() that deserialises valueBytes.
 - Executor serialises value into a ByteBuffer and passes it to DTR's ctor.
 - Executor tracks the time taken to do so and puts it in a new field in TaskMetrics.
 - StagePage now reports serialisation time from TaskMetrics along with the other things it reported.
2013-12-19 18:12:22 -08:00
Tathagata Das de41c436a0 Merge branch 'scheduler-update' into window-improvement
Conflicts:
	streaming/src/main/scala/org/apache/spark/streaming/dstream/WindowedDStream.scala
2013-12-19 12:05:08 -08:00
Shivaram Venkataraman d3234f9726 Make collectPartitions take an array of partitions
Change the implementation to use runJob instead of PartitionPruningRDD.
Also update the unit tests and the python take implementation
to use the new interface.
2013-12-19 11:40:34 -08:00
Tathagata Das 984c582487 Merge branch 'scheduler-update' into filestream-fix
Conflicts:
	core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala
	streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala
	streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala
	streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala
	streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala
2013-12-19 11:20:48 -08:00
Shivaram Venkataraman af0cd6bd27 Add collectPartition to JavaRDD interface.
Also remove takePartition from PythonRDD and use collectPartition in rdd.py.
2013-12-18 11:40:07 -08:00
Tor Myklebust 717c7fddb2 objectSer -> valueSer in a test. 2013-12-17 23:02:21 -05:00
wangda.tan 36060f4f50 spark-898, changes according to review comments 2013-12-17 17:55:38 +08:00
Tor Myklebust 25fa976580 Merge branch 'master' of git://github.com/apache/incubator-spark 2013-12-16 23:48:37 -05:00
Tor Myklebust 963d6f065a Incorporate pwendell's code review suggestions. 2013-12-16 23:14:52 -05:00
Reynold Xin 883e034aeb Merge pull request #245 from gregakespret/task-maxfailures-fix
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.
2013-12-16 14:16:02 -08:00
Tor Myklebust 8a397a959b Track task value serialisation time in TaskMetrics. 2013-12-16 12:07:39 -05:00
wangda.tan 8ab8c6a526 Merge branch 'master' of git://github.com/apache/incubator-spark 2013-12-16 21:45:43 +08:00
Tathagata Das 5e9ce83d68 Fixed multiple file stream and checkpointing bugs.
- 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.
2013-12-11 14:01:36 -08:00
Prashant Sharma 603af51bb5 Merge branch 'master' into akka-bug-fix
Conflicts:
	core/pom.xml
	core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
	pom.xml
	project/SparkBuild.scala
	streaming/pom.xml
	yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala
2013-12-11 10:21:53 +05:30
Hossein Falaki 49bf47e1b7 Removed superfluous abs call from test cases. 2013-12-10 19:50:50 -08:00
Grega Kespret 558af87334 Fix tests. 2013-12-10 11:43:42 +01:00
Patrick Wendell 5b74609d97 License headers 2013-12-09 16:41:01 -08:00
wangda.tan ee68a85cff SPARK-968, added sc finalize code to avoid akka rebinding to the same port 2013-12-09 09:38:58 +08:00
Aaron Davidson 40f63eb034 Merge master into 127 2013-12-08 11:16:52 -08:00
wangda.tan 850c4b709a Merge branch 'master' of https://github.com/leftnoteasy/incubator-spark-1 2013-12-09 00:12:46 +08:00
wangda.tan 48e4f2ad14 SPARK-968, In stage UI, add an overview section that shows task stats grouped by executor id 2013-12-09 00:02:59 +08:00
Matei Zaharia e0392343a0 Merge pull request #190 from markhamstra/Stages4Jobs
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.
2013-12-06 11:49:59 -08:00
Mark Hamstra ee888f6b25 FutureAction result tests 2013-12-05 23:01:18 -08:00
Patrick Wendell 75d161b357 Forcing shuffle consolidation in DiskBlockManagerSuite 2013-12-05 11:36:41 -08:00
Mark Hamstra f55d0b935d Synchronous, inline cleanup after runLocally 2013-12-03 09:57:32 -08:00
Mark Hamstra 51458ab4a1 Added stageId <--> jobId mapping in 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
2013-12-03 09:57:31 -08:00
Reynold Xin 58d9bbcfec Merge pull request #217 from aarondav/mesos-urls
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.
2013-12-02 21:58:53 -08:00
Aaron Davidson 96df26be47 Add spaces between tests 2013-11-29 13:20:43 -08:00
Aaron Davidson 081a0b6861 Add unit test for SparkContext scheduler creation
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.
2013-11-28 20:40:57 -08:00
Prashant Sharma 17987778da Merge branch 'master' into wip-scala-2.10
Conflicts:
	core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
	core/src/main/scala/org/apache/spark/rdd/MapPartitionsRDD.scala
	core/src/main/scala/org/apache/spark/rdd/MapPartitionsWithContextRDD.scala
	core/src/main/scala/org/apache/spark/rdd/RDD.scala
	python/pyspark/rdd.py
2013-11-27 14:44:12 +05:30
Mark Hamstra ed7ecb93ce [SPARK-963] Wait for SparkListenerBus eventQueue to be empty before checking jobLogger state 2013-11-26 13:30:17 -08:00
Prashant Sharma d092a8cc6a Fixed compile time warnings and formatting post merge. 2013-11-26 15:21:50 +05:30
Matei Zaharia 18d6df0e17 Merge pull request #86 from holdenk/master
Add histogram functionality to DoubleRDDFunctions

This pull request add histogram functionality to the DoubleRDDFunctions.
2013-11-26 00:00:07 -08:00
Holden Karau 7222ee2977 Fix the test 2013-11-25 21:06:42 -08:00
Matei Zaharia 0e2109ddb2 Merge pull request #204 from rxin/hash
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.
2013-11-25 20:48:37 -08:00
Matei Zaharia 14bb465bb3 Merge pull request #201 from rxin/mappartitions
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.
2013-11-25 18:50:18 -08:00
Prashant Sharma 44fd30d3fb Merge branch 'master' into scala-2.10-wip
Conflicts:
	core/src/main/scala/org/apache/spark/rdd/RDD.scala
	project/SparkBuild.scala
2013-11-25 18:10:54 +05:30
Reynold Xin 95c55df1c2 Added unit tests for size estimation for specialized hash sets and maps. 2013-11-25 18:27:06 +08:00
Matei Zaharia 65de73c7f8 Merge pull request #185 from mkolod/random-number-generator
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.
2013-11-24 15:52:33 -08:00
Reynold Xin e9ff13ec72 Consolidated both mapPartitions related RDDs into a single MapPartitionsRDD.
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.
2013-11-24 17:56:43 +08:00
Tathagata Das fd031679df Added partitioner aware union, modified DStream.window. 2013-11-21 11:28:37 -08:00
Prashant Sharma 95d8dbce91 Merge branch 'master' of github.com:apache/incubator-spark into scala-2.10-temp
Conflicts:
	core/src/main/scala/org/apache/spark/util/collection/PrimitiveVector.scala
	streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala
2013-11-21 12:34:46 +05:30
Prashant Sharma 199e9cf02d Merge branch 'scala210-master' of github.com:colorant/incubator-spark into scala-2.10
Conflicts:
	core/src/main/scala/org/apache/spark/deploy/client/Client.scala
	core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
	core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
	core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala
2013-11-21 11:55:48 +05:30
Reynold Xin 2fead510f7 Merge branch 'master' of github.com:tbfenet/incubator-spark
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.
2013-11-21 07:15:55 +08:00
Henry Saputra 9c934b640f Remove the semicolons at the end of Scala code to make it more pure Scala code.
Also remove unused imports as I found them along the way.
Remove return statements when returning value in the Scala code.

Passing compile and tests.
2013-11-19 10:19:03 -08:00
Matthew Taylor f639b65eab PartitionPruningRDD is using index from parent(review changes) 2013-11-19 10:48:48 +00:00
Matthew Taylor 13b9bf494b PartitionPruningRDD is using index from parent 2013-11-19 06:27:33 +00:00
Holden Karau e163e31c20 Add spaces 2013-11-18 20:13:25 -08:00
Marek Kolodziej 99cfe89c68 Updates to reflect pull request code review 2013-11-18 22:00:36 -05:00
Marek Kolodziej 09bdfe3b16 XORShift RNG with unit tests and benchmark
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)
2013-11-18 15:21:43 -05:00
Aaron Davidson 85763f4942 Add PrimitiveVectorSuite and fix bug in resize() 2013-11-17 18:16:51 -08:00
Kay Ousterhout 0913c22971 Merge remote-tracking branch 'upstream/master' into consolidate_schedulers
Conflicts:
	core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManager.scala
2013-11-15 10:59:33 -08:00
Raymond Liu a60620b76a Merge branch 'master' into scala-2.10 2013-11-14 12:44:19 +08:00
Kay Ousterhout 46f9c6b858 Fixed naming issues and added back ability to specify max task failures. 2013-11-13 17:12:14 -08:00
Matei Zaharia 2054c61a18 Merge pull request #159 from liancheng/dagscheduler-actor-refine
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.
2013-11-13 16:49:55 -08:00
Kay Ousterhout 150615a31e Merge remote-tracking branch 'upstream/master' into consolidate_schedulers
Conflicts:
	core/src/main/scala/org/apache/spark/scheduler/ClusterScheduler.scala
2013-11-13 14:38:44 -08:00
Kay Ousterhout 68e5ad58b7 Extracted TaskScheduler interface.
Also changed the default maximum number of task failures to be
0 when running in local mode.
2013-11-13 14:32:50 -08:00
Raymond Liu 0f2e3c6e31 Merge branch 'master' into scala-2.10 2013-11-13 16:55:11 +08:00
Prashant Sharma 6860b79f6e Remove deprecated actorFor and use actorSelection everywhere. 2013-11-12 12:43:53 +05:30
Lian, Cheng 2539c06745 Replaced the daemon thread started by DAGScheduler with an actor 2013-11-09 19:05:18 +08:00
Aaron Davidson dd63c548c2 Use SPARK_HOME instead of user.dir in ExecutorRunnerTest 2013-11-08 12:51:05 -08:00
Reynold Xin 3d4ad84b63 Merge pull request #148 from squito/include_appId
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`.
2013-11-07 11:08:27 -08:00
Imran Rashid ca66f5d5a2 fix formatting 2013-11-07 07:23:59 -06:00
Imran Rashid 8d3cdda9a2 very basic regression test to make sure appId doesnt get dropped in future 2013-11-07 01:35:48 -06:00
jerryshao 12dc385a49 Add Spark multi-user support for standalone mode and Mesos 2013-11-07 11:18:09 +08:00
Aaron Davidson 80e98d2bd7 Attempt to fix SparkListenerSuite breakage
Could not reproduce locally, but this test could've been flaky if the
build machine was too fast.
2013-11-06 08:03:35 -08:00
Reynold Xin 0b26a392df Merge pull request #128 from shimingfei/joblogger-doc
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>
2013-11-04 18:22:06 -08:00
Aaron Davidson 39d93ed4b9 Clean up test files properly
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.
2013-11-03 21:52:59 -08:00
Aaron Davidson 3ca52309f2 Fix test breakage 2013-11-03 21:34:44 -08:00
Aaron Davidson 84991a1b91 Memory-optimized shuffle file consolidation
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.
2013-11-03 21:34:13 -08:00
Reynold Xin eb5f8a3f97 Code review feedback. 2013-11-03 18:11:44 -08:00
Reynold Xin 1e9543b567 Fixed a bug that uses twice amount of memory for the primitive arrays due to a scala compiler bug.
Also addressed Matei's code review comment.
2013-11-02 23:19:01 -07:00
Reynold Xin da6bb0aedd Merge branch 'master' into hash1 2013-11-02 22:45:15 -07:00
Kay Ousterhout fb64828b0b Cleaned up imports and fixed test bug 2013-10-31 23:42:56 -07:00
Kay Ousterhout a124658e53 Fixed most issues with unit tests 2013-10-30 19:29:38 -07:00