Commit graph

1158 commits

Author SHA1 Message Date
Tathagata Das 11eabbe125 [SPARK-1103] Automatic garbage collection of RDD, shuffle and broadcast data
This PR allows Spark to automatically cleanup metadata and data related to persisted RDDs, shuffles and broadcast variables when the corresponding RDDs, shuffles and broadcast variables fall out of scope from the driver program. This is still a work in progress as broadcast cleanup has not been implemented.

**Implementation Details**
A new class `ContextCleaner` is responsible cleaning all the state. It is instantiated as part of a `SparkContext`. RDD and ShuffleDependency classes have overridden `finalize()` function that gets called whenever their instances go out of scope. The `finalize()` function enqueues the object’s identifier (i.e. RDD ID, shuffle ID, etc.) with the `ContextCleaner`, which is a very short and cheap operation and should not significantly affect the garbage collection mechanism. The `ContextCleaner`, on a different thread, performs the cleanup, whose details are given below.

*RDD cleanup:*
`ContextCleaner` calls `RDD.unpersist()` is used to cleanup persisted RDDs. Regarding metadata, the DAGScheduler automatically cleans up all metadata related to a RDD after all jobs have completed. Only the `SparkContext.persistentRDDs` keeps strong references to persisted RDDs. The `TimeStampedHashMap` used for that has been replaced by `TimeStampedWeakValueHashMap` that keeps only weak references to the RDDs, allowing them to be garbage collected.

*Shuffle cleanup:*
New BlockManager message `RemoveShuffle(<shuffle ID>)` asks the `BlockManagerMaster` and currently active `BlockManager`s to delete all the disk blocks related to the shuffle ID. `ContextCleaner` cleans up shuffle data using this message and also cleans up the metadata in the `MapOutputTracker` of the driver. The `MapOutputTracker` at the workers, that caches the shuffle metadata, maintains a `BoundedHashMap` to limit the shuffle information it caches. Refetching the shuffle information from the driver is not too costly.

*Broadcast cleanup:*
To be done. [This PR](https://github.com/apache/incubator-spark/pull/543/) adds mechanism for explicit cleanup of broadcast variables. `Broadcast.finalize()` will enqueue its own ID with ContextCleaner and the PRs mechanism will be used to unpersist the Broadcast data.

*Other cleanup:*
`ShuffleMapTask` and `ResultTask` caches tasks and used TTL based cleanup (using `TimeStampedHashMap`), so nothing got cleaned up if TTL was not set. Instead, they now use `BoundedHashMap` to keep a limited number of map output information. Cost of repopulating the cache if necessary is very small.

**Current state of implementation**
Implemented RDD and shuffle cleanup. Things left to be done are.
- Cleaning up for broadcast variable still to be done.
- Automatic cleaning up keys with empty weak refs as values in `TimeStampedWeakValueHashMap`

Author: Tathagata Das <tathagata.das1565@gmail.com>
Author: Andrew Or <andrewor14@gmail.com>
Author: Roman Pastukhov <ignatich@mail.ru>

Closes #126 from tdas/state-cleanup and squashes the following commits:

61b8d6e [Tathagata Das] Fixed issue with Tachyon + new BlockManager methods.
f489fdc [Tathagata Das] Merge remote-tracking branch 'apache/master' into state-cleanup
d25a86e [Tathagata Das] Fixed stupid typo.
cff023c [Tathagata Das] Fixed issues based on Andrew's comments.
4d05314 [Tathagata Das] Scala style fix.
2b95b5e [Tathagata Das] Added more documentation on Broadcast implementations, specially which blocks are told about to the driver. Also, fixed Broadcast API to hide destroy functionality.
41c9ece [Tathagata Das] Added more unit tests for BlockManager, DiskBlockManager, and ContextCleaner.
6222697 [Tathagata Das] Fixed bug and adding unit test for removeBroadcast in BlockManagerSuite.
104a89a [Tathagata Das] Fixed failing BroadcastSuite unit tests by introducing blocking for removeShuffle and removeBroadcast in BlockManager*
a430f06 [Tathagata Das] Fixed compilation errors.
b27f8e8 [Tathagata Das] Merge pull request #3 from andrewor14/cleanup
cd72d19 [Andrew Or] Make automatic cleanup configurable (not documented)
ada45f0 [Andrew Or] Merge branch 'state-cleanup' of github.com:tdas/spark into cleanup
a2cc8bc [Tathagata Das] Merge remote-tracking branch 'apache/master' into state-cleanup
c5b1d98 [Andrew Or] Address Patrick's comments
a6460d4 [Andrew Or] Merge github.com:apache/spark into cleanup
762a4d8 [Tathagata Das] Merge pull request #1 from andrewor14/cleanup
f0aabb1 [Andrew Or] Correct semantics for TimeStampedWeakValueHashMap + add tests
5016375 [Andrew Or] Address TD's comments
7ed72fb [Andrew Or] Fix style test fail + remove verbose test message regarding broadcast
634a097 [Andrew Or] Merge branch 'state-cleanup' of github.com:tdas/spark into cleanup
7edbc98 [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into state-cleanup
8557c12 [Andrew Or] Merge github.com:apache/spark into cleanup
e442246 [Andrew Or] Merge github.com:apache/spark into cleanup
88904a3 [Andrew Or] Make TimeStampedWeakValueHashMap a wrapper of TimeStampedHashMap
fbfeec8 [Andrew Or] Add functionality to query executors for their local BlockStatuses
34f436f [Andrew Or] Generalize BroadcastBlockId to remove BroadcastHelperBlockId
0d17060 [Andrew Or] Import, comments, and style fixes (minor)
c92e4d9 [Andrew Or] Merge github.com:apache/spark into cleanup
f201a8d [Andrew Or] Test broadcast cleanup in ContextCleanerSuite + remove BoundedHashMap
e95479c [Andrew Or] Add tests for unpersisting broadcast
544ac86 [Andrew Or] Clean up broadcast blocks through BlockManager*
d0edef3 [Andrew Or] Add framework for broadcast cleanup
ba52e00 [Andrew Or] Refactor broadcast classes
c7ccef1 [Andrew Or] Merge branch 'bc-unpersist-merge' of github.com:ignatich/incubator-spark into cleanup
6c9dcf6 [Tathagata Das] Added missing Apache license
d2f8b97 [Tathagata Das] Removed duplicate unpersistRDD.
a007307 [Tathagata Das] Merge remote-tracking branch 'apache/master' into state-cleanup
620eca3 [Tathagata Das] Changes based on PR comments.
f2881fd [Tathagata Das] Changed ContextCleaner to use ReferenceQueue instead of finalizer
e1fba5f [Tathagata Das] Style fix
892b952 [Tathagata Das] Removed use of BoundedHashMap, and made BlockManagerSlaveActor cleanup shuffle metadata in MapOutputTrackerWorker.
a7260d3 [Tathagata Das] Added try-catch in context cleaner and null value cleaning in TimeStampedWeakValueHashMap.
e61daa0 [Tathagata Das] Modifications based on the comments on PR 126.
ae9da88 [Tathagata Das] Removed unncessary TimeStampedHashMap from DAGScheduler, added try-catches in finalize() methods, and replaced ArrayBlockingQueue to LinkedBlockingQueue to avoid blocking in Java's finalizing thread.
cb0a5a6 [Tathagata Das] Fixed docs and styles.
a24fefc [Tathagata Das] Merge remote-tracking branch 'apache/master' into state-cleanup
8512612 [Tathagata Das] Changed TimeStampedHashMap to use WrappedJavaHashMap.
e427a9e [Tathagata Das] Added ContextCleaner to automatically clean RDDs and shuffles when they fall out of scope. Also replaced TimeStampedHashMap to BoundedHashMaps and TimeStampedWeakValueHashMap for the necessary hashmap behavior.
80dd977 [Roman Pastukhov] Fix for Broadcast unpersist patch.
1e752f1 [Roman Pastukhov] Added unpersist method to Broadcast.
2014-04-07 23:40:36 -07:00
Aaron Davidson 0307db0f55 SPARK-1099: Introduce local[*] mode to infer number of cores
This is the default mode for running spark-shell and pyspark, intended to allow users running spark for the first time to see the performance benefits of using multiple cores, while not breaking backwards compatibility for users who use "local" mode and expect exactly 1 core.

Author: Aaron Davidson <aaron@databricks.com>

Closes #182 from aarondav/110 and squashes the following commits:

a88294c [Aaron Davidson] Rebased changes for new spark-shell
a9f393e [Aaron Davidson] SPARK-1099: Introduce local[*] mode to infer number of cores
2014-04-07 13:06:30 -07:00
Evan Chan 1440154c27 SPARK-1154: Clean up app folders in worker nodes
This is a fix for [SPARK-1154](https://issues.apache.org/jira/browse/SPARK-1154).   The issue is that worker nodes fill up with a huge number of app-* folders after some time.  This change adds a periodic cleanup task which asynchronously deletes app directories older than a configurable TTL.

Two new configuration parameters have been introduced:
  spark.worker.cleanup_interval
  spark.worker.app_data_ttl

This change does not include moving the downloads of application jars to a location outside of the work directory.  We will address that if we have time, but that potentially involves caching so it will come either as part of this PR or a separate PR.

Author: Evan Chan <ev@ooyala.com>
Author: Kelvin Chu <kelvinkwchu@yahoo.com>

Closes #288 from velvia/SPARK-1154-cleanup-app-folders and squashes the following commits:

0689995 [Evan Chan] CR from @aarondav - move config, clarify for standalone mode
9f10d96 [Evan Chan] CR from @pwendell - rename configs and add cleanup.enabled
f2f6027 [Evan Chan] CR from @andrewor14
553d8c2 [Kelvin Chu] change the variable name to currentTimeMillis since it actually tracks in seconds
8dc9cb5 [Kelvin Chu] Fixed a bug in Utils.findOldFiles() after merge.
cb52f2b [Kelvin Chu] Change the name of findOldestFiles() to findOldFiles()
72f7d2d [Kelvin Chu] Fix a bug of Utils.findOldestFiles(). file.lastModified is returned in milliseconds.
ad99955 [Kelvin Chu] Add unit test for Utils.findOldestFiles()
dc1a311 [Evan Chan] Don't recompute current time with every new file
e3c408e [Evan Chan] Document the two new settings
b92752b [Evan Chan] SPARK-1154: Add a periodic task to clean up app directories
2014-04-06 19:21:40 -07:00
Egor Pakhomov e258e5040f [SPARK-1259] Make RDD locally iterable
Author: Egor Pakhomov <pahomov.egor@gmail.com>

Closes #156 from epahomov/SPARK-1259 and squashes the following commits:

8ec8f24 [Egor Pakhomov] Make to local iterator shorter
34aa300 [Egor Pakhomov] Fix toLocalIterator docs
08363ef [Egor Pakhomov] SPARK-1259 from toLocallyIterable to toLocalIterator
6a994eb [Egor Pakhomov] SPARK-1259 Make RDD locally iterable
8be3dcf [Egor Pakhomov] SPARK-1259 Make RDD locally iterable
33ecb17 [Egor Pakhomov] SPARK-1259 Make RDD locally iterable
2014-04-06 16:43:01 -07:00
Haoyuan Li b50ddfde03 SPARK-1305: Support persisting RDD's directly to Tachyon
Move the PR#468 of apache-incubator-spark to the apache-spark
"Adding an option to persist Spark RDD blocks into Tachyon."

Author: Haoyuan Li <haoyuan@cs.berkeley.edu>
Author: RongGu <gurongwalker@gmail.com>

Closes #158 from RongGu/master and squashes the following commits:

72b7768 [Haoyuan Li] merge master
9f7fa1b [Haoyuan Li] fix code style
ae7834b [Haoyuan Li] minor cleanup
a8b3ec6 [Haoyuan Li] merge master branch
e0f4891 [Haoyuan Li] better check offheap.
55b5918 [RongGu] address matei's comment on the replication of offHeap storagelevel
7cd4600 [RongGu] remove some logic code for tachyonstore's replication
51149e7 [RongGu] address aaron's comment on returning value of the remove() function in tachyonstore
8adfcfa [RongGu] address arron's comment on inTachyonSize
120e48a [RongGu] changed the root-level dir name in Tachyon
5cc041c [Haoyuan Li] address aaron's comments
9b97935 [Haoyuan Li] address aaron's comments
d9a6438 [Haoyuan Li] fix for pspark
77d2703 [Haoyuan Li] change python api.git status
3dcace4 [Haoyuan Li] address matei's comments
91fa09d [Haoyuan Li] address patrick's comments
589eafe [Haoyuan Li] use TRY_CACHE instead of MUST_CACHE
64348b2 [Haoyuan Li] update conf docs.
ed73e19 [Haoyuan Li] Merge branch 'master' of github.com:RongGu/spark-1
619a9a8 [RongGu] set number of directories in TachyonStore back to 64; added a TODO tag for duplicated code from the DiskStore
be79d77 [RongGu] find a way to clean up some unnecessay metods and classed to make the code simpler
49cc724 [Haoyuan Li] update docs with off_headp option
4572f9f [RongGu] reserving the old apply function API of StorageLevel
04301d3 [RongGu] rename StorageLevel.TACHYON to Storage.OFF_HEAP
c9aeabf [RongGu] rename the StorgeLevel.TACHYON as StorageLevel.OFF_HEAP
76805aa [RongGu] unifies the config properties name prefix; add the configs into docs/configuration.md
e700d9c [RongGu] add the SparkTachyonHdfsLR example and some comments
fd84156 [RongGu] use randomUUID to generate sparkapp directory name on tachyon;minor code style fix
939e467 [Haoyuan Li] 0.4.1-thrift from maven central
86a2eab [Haoyuan Li] tachyon 0.4.1-thrift is in the staging repo. but jenkins failed to download it. temporarily revert it back to 0.4.1
16c5798 [RongGu] make the dependency on tachyon as tachyon-0.4.1-thrift
eacb2e8 [RongGu] Merge branch 'master' of https://github.com/RongGu/spark-1
bbeb4de [RongGu] fix the JsonProtocolSuite test failure problem
6adb58f [RongGu] Merge branch 'master' of https://github.com/RongGu/spark-1
d827250 [RongGu] fix JsonProtocolSuie test failure
716e93b [Haoyuan Li] revert the version
ca14469 [Haoyuan Li] bump tachyon version to 0.4.1-thrift
2825a13 [RongGu] up-merging to the current master branch of the apache spark
6a22c1a [Haoyuan Li] fix scalastyle
8968b67 [Haoyuan Li] exclude more libraries from tachyon dependency to be the same as referencing tachyon-client.
77be7e8 [RongGu] address mateiz's comment about the temp folder name problem. The implementation followed mateiz's advice.
1dcadf9 [Haoyuan Li] typo
bf278fa [Haoyuan Li] fix python tests
e82909c [Haoyuan Li] minor cleanup
776a56c [Haoyuan Li] address patrick's and ali's comments from the previous PR
8859371 [Haoyuan Li] various minor fixes and clean up
e3ddbba [Haoyuan Li] add doc to use Tachyon cache mode.
fcaeab2 [Haoyuan Li] address Aaron's comment
e554b1e [Haoyuan Li] add python code
47304b3 [Haoyuan Li] make tachyonStore in BlockMananger lazy val; add more comments StorageLevels.
dc8ef24 [Haoyuan Li] add old storelevel constructor
e01a271 [Haoyuan Li] update tachyon 0.4.1
8011a96 [RongGu] fix a brought-in mistake in StorageLevel
70ca182 [RongGu] a bit change in comment
556978b [RongGu] fix the scalastyle errors
791189b [RongGu] "Adding an option to persist Spark RDD blocks into Tachyon." move the PR#468 of apache-incubator-spark to the apache-spark
2014-04-04 20:38:20 -07:00
Patrick Wendell 5f3c1bb513 Add test utility for generating Jar files with compiled classes.
This was requested by a few different people and may be generally
useful, so I'd like to contribute this and not block on a different
PR for it to get in.

Author: Patrick Wendell <pwendell@gmail.com>

Closes #326 from pwendell/class-loader-test-utils and squashes the following commits:

ff3e88e [Patrick Wendell] Add test utility for generating Jar files with compiled classes.
2014-04-04 19:15:15 -07:00
Thomas Graves 198892fe8d [SPARK-1198] Allow pipes tasks to run in different sub-directories
This works as is on Linux/Mac/etc but doesn't cover working on Windows.  In here I use ln -sf for symlinks. Putting this up for comments on that. Do we want to create perhaps some classes for doing shell commands - Linux vs Windows.  Is there some other way we want to do this?   I assume we are still supporting jdk1.6?

Also should I update the Java API for pipes to allow this parameter?

Author: Thomas Graves <tgraves@apache.org>

Closes #128 from tgravescs/SPARK1198 and squashes the following commits:

abc1289 [Thomas Graves] remove extra tag in pom file
ba23fc0 [Thomas Graves] Add support for symlink on windows, remove commons-io usage
da4b221 [Thomas Graves] Merge branch 'master' of https://github.com/tgravescs/spark into SPARK1198
61be271 [Thomas Graves] Fix file name filter
6b783bd [Thomas Graves] style fixes
1ab49ca [Thomas Graves] Add support for running pipe tasks is separate directories
2014-04-04 17:16:31 -07:00
Patrick Wendell a02b535d5e Don't create SparkContext in JobProgressListenerSuite.
This reduces the time of the test from 11 seconds to 20 milliseconds.

Author: Patrick Wendell <pwendell@gmail.com>

Closes #324 from pwendell/job-test and squashes the following commits:

868d9eb [Patrick Wendell] Don't create SparkContext in JobProgressListenerSuite.
2014-04-04 14:46:32 -07:00
Xusen Yin f1fa617023 [SPARK-1133] Add whole text files reader in MLlib
Here is a pointer to the former [PR164](https://github.com/apache/spark/pull/164).

I add the pull request for the JIRA issue [SPARK-1133](https://spark-project.atlassian.net/browse/SPARK-1133), which brings a new files reader API in MLlib.

Author: Xusen Yin <yinxusen@gmail.com>

Closes #252 from yinxusen/whole-files-input and squashes the following commits:

7191be6 [Xusen Yin] refine comments
0af3faf [Xusen Yin] add JavaAPI test
01745ee [Xusen Yin] fix deletion error
cc97dca [Xusen Yin] move whole text file API to Spark core
d792cee [Xusen Yin] remove the typo character "+"
6bdf2c2 [Xusen Yin] test for small local file system block size
a1f1e7e [Xusen Yin] add two extra spaces
28cb0fe [Xusen Yin] add whole text files reader
2014-04-04 11:12:47 -07:00
Patrick Wendell ee6e9e7d86 SPARK-1337: Application web UI garbage collects newest stages
Simple fix...

Author: Patrick Wendell <pwendell@gmail.com>

Closes #320 from pwendell/stage-clean-up and squashes the following commits:

29be62e [Patrick Wendell] SPARK-1337: Application web UI garbage collects newest stages instead old ones
2014-04-03 22:13:56 -07:00
Andrew Or de8eefa804 [SPARK-1385] Use existing code for JSON de/serialization of BlockId
`BlockId.scala` offers a way to reconstruct a BlockId from a string through regex matching. `util/JsonProtocol.scala` duplicates this functionality by explicitly matching on the BlockId type.
With this PR, the de/serialization of BlockIds will go through the first (older) code path.

(Most of the line changes in this PR involve changing `==` to `===` in `JsonProtocolSuite.scala`)

Author: Andrew Or <andrewor14@gmail.com>

Closes #289 from andrewor14/blockid-json and squashes the following commits:

409d226 [Andrew Or] Simplify JSON de/serialization for BlockId
2014-04-02 10:43:09 -07:00
Kay Ousterhout 11973a7bda Renamed stageIdToActiveJob to jobIdToActiveJob.
This data structure was misused and, as a result, later renamed to an incorrect name.

This data structure seems to have gotten into this tangled state as a result of @henrydavidge using the stageID instead of the job Id to index into it and later @andrewor14 renaming the data structure to reflect this misunderstanding.

This patch renames it and removes an incorrect indexing into it.  The incorrect indexing into it meant that the code added by @henrydavidge to warn when a task size is too large (added here 57579934f0) was not always executed; this commit fixes that.

Author: Kay Ousterhout <kayousterhout@gmail.com>

Closes #301 from kayousterhout/fixCancellation and squashes the following commits:

bd3d3a4 [Kay Ousterhout] Renamed stageIdToActiveJob to jobIdToActiveJob.
2014-04-02 10:35:52 -07:00
Patrick Wendell 841721e03c SPARK-1352: Improve robustness of spark-submit script
1. Better error messages when required arguments are missing.
2. Support for unit testing cases where presented arguments are invalid.
3. Bug fix: Only use environment varaibles when they are set (otherwise will cause NPE).
4. A verbose mode to aid debugging.
5. Visibility of several variables is set to private.
6. Deprecation warning for existing scripts.

Author: Patrick Wendell <pwendell@gmail.com>

Closes #271 from pwendell/spark-submit and squashes the following commits:

9146def [Patrick Wendell] SPARK-1352: Improve robustness of spark-submit script
2014-03-31 12:07:14 -07:00
Sandy Ryza 1617816090 SPARK-1126. spark-app preliminary
This is a starting version of the spark-app script for running compiled binaries against Spark.  It still needs tests and some polish.  The only testing I've done so far has been using it to launch jobs in yarn-standalone mode against a pseudo-distributed cluster.

This leaves out the changes required for launching python scripts.  I think it might be best to save those for another JIRA/PR (while keeping to the design so that they won't require backwards-incompatible changes).

Author: Sandy Ryza <sandy@cloudera.com>

Closes #86 from sryza/sandy-spark-1126 and squashes the following commits:

d428d85 [Sandy Ryza] Commenting, doc, and import fixes from Patrick's comments
e7315c6 [Sandy Ryza] Fix failing tests
34de899 [Sandy Ryza] Change --more-jars to --jars and fix docs
299ddca [Sandy Ryza] Fix scalastyle
a94c627 [Sandy Ryza] Add newline at end of SparkSubmit
04bc4e2 [Sandy Ryza] SPARK-1126. spark-submit script
2014-03-29 14:41:36 -07:00
Prashant Sharma 60abc25254 SPARK-1096, a space after comment start style checker.
Author: Prashant Sharma <prashant.s@imaginea.com>

Closes #124 from ScrapCodes/SPARK-1096/scalastyle-comment-check and squashes the following commits:

214135a [Prashant Sharma] Review feedback.
5eba88c [Prashant Sharma] Fixed style checks for ///+ comments.
e54b2f8 [Prashant Sharma] improved message, work around.
83e7144 [Prashant Sharma] removed dependency on scalastyle in plugin, since scalastyle sbt plugin already depends on the right version. Incase we update the plugin we will have to adjust our spark-style project to depend on right scalastyle version.
810a1d6 [Prashant Sharma] SPARK-1096, a space after comment style checker.
ba33193 [Prashant Sharma] scala style as a project
2014-03-28 00:21:49 -07:00
Petko Nikolov 6f986f0b87 [SPARK-1268] Adding XOR and AND-NOT operations to spark.util.collection.BitSet
Symmetric difference (xor) in particular is useful for computing some distance metrics (e.g. Hamming). Unit tests added.

Author: Petko Nikolov <nikolov@soundcloud.com>

Closes #172 from petko-nikolov/bitset-imprv and squashes the following commits:

451f28b [Petko Nikolov] fixed style mistakes
5beba18 [Petko Nikolov] rm outer loop in andNot test
0e61035 [Petko Nikolov] conform to spark style; rm redundant asserts; more unit tests added; use arraycopy instead of loop
d53cdb9 [Petko Nikolov] rm incidentally added space
4e1df43 [Petko Nikolov] adding xor and and-not to BitSet; unit tests added
2014-03-27 15:49:07 -07:00
witgo 8237df8060 Avoid Option while generating call site
This is an update on https://github.com/apache/spark/pull/180, which changes the solution from blacklisting "Option.scala" to avoiding the Option code path while generating the call path.

Also includes a unit test to prevent this issue in the future, and some minor refactoring.

Thanks @witgo for reporting this issue and working on the initial solution!

Author: witgo <witgo@qq.com>
Author: Aaron Davidson <aaron@databricks.com>

Closes #222 from aarondav/180 and squashes the following commits:

f74aad1 [Aaron Davidson] Avoid Option while generating call site & add unit tests
d2b4980 [witgo] Modify the position of the filter
1bc22d7 [witgo] Fix Stage.name return "apply at Option.scala:120"
2014-03-25 13:28:13 -07:00
Shivaram Venkataraman f8111eaeb0 SPARK-1319: Fix scheduler to account for tasks using > 1 CPUs.
Move CPUS_PER_TASK to TaskSchedulerImpl as the value is a constant and use it in both Mesos and CoarseGrained scheduler backends.

Thanks @kayousterhout for the design discussion

Author: Shivaram Venkataraman <shivaram@eecs.berkeley.edu>

Closes #219 from shivaram/multi-cpus and squashes the following commits:

5c7d685 [Shivaram Venkataraman] Don't pass availableCpus to TaskSetManager
260e4d5 [Shivaram Venkataraman] Add a check for non-zero CPUs in TaskSetManager
73fcf6f [Shivaram Venkataraman] Add documentation for spark.task.cpus
647bc45 [Shivaram Venkataraman] Fix scheduler to account for tasks using > 1 CPUs. Move CPUS_PER_TASK to TaskSchedulerImpl as the value is a constant and use it in both Mesos and CoarseGrained scheduler backends.
2014-03-25 13:05:30 -07:00
Sean Owen 71d4ed271b SPARK-1316. Remove use of Commons IO
(This follows from a side point on SPARK-1133, in discussion of the PR: https://github.com/apache/spark/pull/164 )

Commons IO is barely used in the project, and can easily be replaced with equivalent calls to Guava or the existing Spark `Utils.scala` class.

Removing a dependency feels good, and this one in particular can get a little problematic since Hadoop uses it too.

Author: Sean Owen <sowen@cloudera.com>

Closes #226 from srowen/SPARK-1316 and squashes the following commits:

21efef3 [Sean Owen] Remove use of Commons IO
2014-03-25 10:21:25 -07:00
Aaron Davidson ffe272d97c Revert "SPARK-1099:Spark's local mode should probably respect spark.cores.max by default"
This reverts commit 16789317a3. Jenkins was not run for this PR.
2014-03-19 17:56:48 -07:00
qqsun8819 16789317a3 SPARK-1099:Spark's local mode should probably respect spark.cores.max by default
This is for JIRA:https://spark-project.atlassian.net/browse/SPARK-1099
And this is what I do in this patch (also commented in the JIRA) @aarondav

 This is really a behavioral change, so I do this with great caution, and welcome any review advice:

1 I change the "MASTER=local" pattern of create LocalBackEnd . In the past, we passed 1 core to it . now it use a default cores
The reason here is that when someone use spark-shell to start local mode , Repl will use this "MASTER=local" pattern as default.
So if one also specify cores in the spark-shell command line, it will all go in here. So here pass 1 core is not suitalbe reponding to our change here.
2 In the LocalBackEnd , the "totalCores" variable are fetched following a different rule(in the past it just take in a userd passed cores, like 1 in "MASTER=local" pattern, 2 in "MASTER=local[2]" pattern"
rules:
a The second argument of LocalBackEnd 's constructor indicating cores have a default value which is Int.MaxValue. If user didn't pass it , its first default value is Int.MaxValue
b In getMaxCores, we first compare the former value to Int.MaxValue. if it's not equal, we think that user has passed their desired value, so just use it
c. If b is not satified, we then get cores from spark.cores.max, and we get real logical cores from Runtime. And if cores specified by spark.cores.max is bigger than logical cores, we use logical cores, otherwise we use spark.cores.max
3 In SparkContextSchedulerCreationSuite 's test("local") case, assertion is modified from 1 to logical cores, because "MASTER=local" pattern use default vaules.

Author: qqsun8819 <jin.oyj@alibaba-inc.com>

Closes #110 from qqsun8819/local-cores and squashes the following commits:

731aefa [qqsun8819] 1 LocalBackend not change 2 In SparkContext do some process to the cores and pass it to original LocalBackend constructor
78b9c60 [qqsun8819] 1 SparkContext MASTER=local pattern use default cores instead of 1 to construct LocalBackEnd , for use of spark-shell and cores specified in cmd line 2 some test case change from local to local[1]. 3 SparkContextSchedulerCreationSuite test spark.cores.max config in local pattern
6ae1ee8 [qqsun8819] Add a static function in LocalBackEnd to let it use spark.cores.max specified cores when no cores are passed to it
2014-03-19 16:33:54 -07:00
Andrew Or 79d07d6604 [SPARK-1132] Persisting Web UI through refactoring the SparkListener interface
The fleeting nature of the Spark Web UI has long been a problem reported by many users: The existing Web UI disappears as soon as the associated application terminates. This is because SparkUI is tightly coupled with SparkContext, and cannot be instantiated independently from it. To solve this, some state must be saved to persistent storage while the application is still running.

The approach taken by this PR involves persisting the UI state through SparkListenerEvents. This requires a major refactor of the SparkListener interface because existing events (1) maintain deep references, making de/serialization is difficult, and (2) do not encode all the information displayed on the UI. In this design, each existing listener for the UI (e.g. ExecutorsListener) maintains state that can be fully constructed from SparkListenerEvents. This state is then supplied to the parent UI (e.g. ExecutorsUI), which renders the associated page(s) on demand.

This PR introduces two important classes: the **EventLoggingListener**, and the **ReplayListenerBus**. In a live application, SparkUI registers an EventLoggingListener with the SparkContext in addition to the existing listeners. Over the course of the application, this listener serializes and logs all events to persisted storage. Then, after the application has finished, the SparkUI can be revived by replaying all the logged events to the existing UI listeners through the ReplayListenerBus.

This feature is currently integrated with the Master Web UI, which optionally rebuilds a SparkUI from event logs as soon as the corresponding application finishes.

More details can be found in the commit messages, comments within the code, and the [design doc](https://spark-project.atlassian.net/secure/attachment/12900/PersistingSparkWebUI.pdf). Comments and feedback are most welcome.

Author: Andrew Or <andrewor14@gmail.com>
Author: andrewor14 <andrewor14@gmail.com>

Closes #42 from andrewor14/master and squashes the following commits:

e5f14fa [Andrew Or] Merge github.com:apache/spark
a1c5cd9 [Andrew Or] Merge github.com:apache/spark
b8ba817 [Andrew Or] Remove UI from map when removing application in Master
83af656 [Andrew Or] Scraps and pieces (no functionality change)
222adcd [Andrew Or] Merge github.com:apache/spark
124429f [Andrew Or] Clarify LiveListenerBus behavior + Add tests for new behavior
f80bd31 [Andrew Or] Simplify static handler and BlockManager status update logic
9e14f97 [Andrew Or] Moved around functionality + renamed classes per Patrick
6740e49 [Andrew Or] Fix comment nits
650eb12 [Andrew Or] Add unit tests + Fix bugs found through tests
45fd84c [Andrew Or] Remove now deprecated test
c5c2c8f [Andrew Or] Remove list of (TaskInfo, TaskMetrics) from StageInfo
3456090 [Andrew Or] Address Patrick's comments
bf80e3d [Andrew Or] Imports, comments, and code formatting, once again (minor)
ac69ec8 [Andrew Or] Fix test fail
d801d11 [Andrew Or] Merge github.com:apache/spark (major)
dc93915 [Andrew Or] Imports, comments, and code formatting (minor)
77ba283 [Andrew Or] Address Kay's and Patrick's comments
b6eaea7 [Andrew Or] Treating SparkUI as a handler of MasterUI
d59da5f [Andrew Or] Avoid logging all the blocks on each executor
d6e3b4a [Andrew Or] Merge github.com:apache/spark
ca258a4 [Andrew Or] Master UI - add support for reading compressed event logs
176e68e [Andrew Or] Fix deprecated message for JavaSparkContext (minor)
4f69c4a [Andrew Or] Master UI - Rebuild SparkUI on application finish
291b2be [Andrew Or] Correct directory in log message "INFO: Logging events to <dir>"
1ba3407 [Andrew Or] Add a few configurable options to event logging
e375431 [Andrew Or] Add new constructors for SparkUI
18b256d [Andrew Or] Refactor out event logging and replaying logic from UI
bb4c503 [Andrew Or] Use a more mnemonic path for logging
aef411c [Andrew Or] Fix bug: storage status was not reflected on UI in the local case
03eda0b [Andrew Or] Fix HDFS flush behavior
36b3e5d [Andrew Or] Add HDFS support for event logging
cceff2b [andrewor14] Fix 100 char format fail
2fee310 [Andrew Or] Address Patrick's comments
2981d61 [Andrew Or] Move SparkListenerBus out of DAGScheduler + Clean up
5d2cec1 [Andrew Or] JobLogger: ID -> Id
0503e4b [Andrew Or] Fix PySpark tests + remove sc.clearFiles/clearJars
4d2fb0c [Andrew Or] Fix format fail
faa113e [Andrew Or] General clean up
d47585f [Andrew Or] Clean up FileLogger
472fd8a [Andrew Or] Fix a couple of tests
996d7a2 [Andrew Or] Reflect RDD unpersist on UI
7b2f811 [Andrew Or] Guard against TaskMetrics NPE + Fix tests
d1f4285 [Andrew Or] Migrate from lift-json to json4s-jackson
28019ca [Andrew Or] Merge github.com:apache/spark
bbe3501 [Andrew Or] Embed storage status and RDD info in Task events
6631c02 [Andrew Or] More formatting changes, this time mainly for Json DSL
70e7e7a [Andrew Or] Formatting changes
e9e1c6d [Andrew Or] Move all JSON de/serialization logic to JsonProtocol
d646df6 [Andrew Or] Completely decouple SparkUI from SparkContext
6814da0 [Andrew Or] Explicitly register each UI listener rather than through some magic
64d2ce1 [Andrew Or] Fix BlockManagerUI bug by introducing new event
4273013 [Andrew Or] Add a gateway SparkListener to simplify event logging
904c729 [Andrew Or] Fix another major bug
5ac906d [Andrew Or] Mostly naming, formatting, and code style changes
3fd584e [Andrew Or] Fix two major bugs
f3fc13b [Andrew Or] General refactor
4dfcd22 [Andrew Or] Merge git://git.apache.org/incubator-spark into persist-ui
b3976b0 [Andrew Or] Add functionality of reconstructing a persisted UI from SparkContext
8add36b [Andrew Or] JobProgressUI: Add JSON functionality
d859efc [Andrew Or] BlockManagerUI: Add JSON functionality
c4cd480 [Andrew Or] Also deserialize new events
8a2ebe6 [Andrew Or] Fix bugs for EnvironmentUI and ExecutorsUI
de8a1cd [Andrew Or] Serialize events both to and from JSON (rather than just to)
bf0b2e9 [Andrew Or] ExecutorUI: Serialize events rather than arbitary executor information
bb222b9 [Andrew Or] ExecutorUI: render completely from JSON
dcbd312 [Andrew Or] Add JSON Serializability for all SparkListenerEvent's
10ed49d [Andrew Or] Merge github.com:apache/incubator-spark into persist-ui
8e09306 [Andrew Or] Use JSON for ExecutorsUI
e3ae35f [Andrew Or] Merge github.com:apache/incubator-spark
3ddeb7e [Andrew Or] Also privatize fields
090544a [Andrew Or] Privatize methods
13920c9 [Andrew Or] Update docs
bd5a1d7 [Andrew Or] Typo: phyiscal -> physical
287ef44 [Andrew Or] Avoid reading the entire batch into memory; also simplify streaming logic
3df7005 [Andrew Or] Merge branch 'master' of github.com:andrewor14/incubator-spark
a531d2e [Andrew Or] Relax assumptions on compressors and serializers when batching
164489d [Andrew Or] Relax assumptions on compressors and serializers when batching
2014-03-19 13:17:01 -07:00
Mridul Muralidharan ab747d39dd Bugfixes/improvements to scheduler
Move the PR#517 of apache-incubator-spark to the apache-spark

Author: Mridul Muralidharan <mridul@gmail.com>

Closes #159 from mridulm/master and squashes the following commits:

5ff59c2 [Mridul Muralidharan] Change property in suite also
167fad8 [Mridul Muralidharan] Address review comments
9bda70e [Mridul Muralidharan] Address review comments, akwats add to failedExecutors
270d841 [Mridul Muralidharan] Address review comments
fa5d9f1 [Mridul Muralidharan] Bugfixes/improvements to scheduler : PR #517
2014-03-19 12:46:55 -07:00
CodingCat 2fa26ec02f SPARK-1102: Create a saveAsNewAPIHadoopDataset method
https://spark-project.atlassian.net/browse/SPARK-1102

Create a saveAsNewAPIHadoopDataset method

By @mateiz: "Right now RDDs can only be saved as files using the new Hadoop API, not as "datasets" with no filename and just a JobConf. See http://codeforhire.com/2014/02/18/using-spark-with-mongodb/ for an example of how you have to give a bogus filename. For the old Hadoop API, we have saveAsHadoopDataset."

Author: CodingCat <zhunansjtu@gmail.com>

Closes #12 from CodingCat/SPARK-1102 and squashes the following commits:

6ba0c83 [CodingCat] add test cases for saveAsHadoopDataSet (new&old API)
a8d11ba [CodingCat] style fix.........
95a6929 [CodingCat] code clean
7643c88 [CodingCat] change the parameter type back to Configuration
a8583ee [CodingCat] Create a saveAsNewAPIHadoopDataset method
2014-03-18 11:06:18 -07:00
Dan McClary e3681f26fa Spark 1246 add min max to stat counter
Here's the addition of min and max to statscounter.py and min and max methods to rdd.py.

Author: Dan McClary <dan.mcclary@gmail.com>

Closes #144 from dwmclary/SPARK-1246-add-min-max-to-stat-counter and squashes the following commits:

fd3fd4b [Dan McClary] fixed  error, updated test
82cde0e [Dan McClary] flipped incorrectly assigned inf values in StatCounter
5d96799 [Dan McClary] added max and min to StatCounter repr for pyspark
21dd366 [Dan McClary] added max and min to StatCounter output, updated doc
1a97558 [Dan McClary] added max and min to StatCounter output, updated doc
a5c13b0 [Dan McClary] Added min and max to Scala and Java RDD, added min and max to StatCounter
ed67136 [Dan McClary] broke min/max out into separate transaction, added to rdd.py
1e7056d [Dan McClary] added underscore to getBucket
37a7dea [Dan McClary] cleaned up boundaries for histogram -- uses real min/max when buckets are derived
29981f2 [Dan McClary] fixed indentation on doctest comment
eaf89d9 [Dan McClary] added correct doctest for histogram
4916016 [Dan McClary] added histogram method, added max and min to statscounter
2014-03-18 00:45:47 -07:00
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
Kay Ousterhout 5e91495f5c 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.
2013-10-30 18:48:34 -07:00
Evan Chan de0285556a Add support for local:// URI scheme for addJars()
This indicates that a jar is available locally on each worker node.
2013-10-30 09:41:35 -07:00
Patrick Wendell e5f6d5697b Spacing fix 2013-10-24 22:08:06 -07:00
Patrick Wendell 31e92b72e3 Adding Java versions and associated tests 2013-10-24 21:14:56 -07:00
Patrick Wendell 05ac9940ee Adding tests 2013-10-24 14:31:34 -07:00
Patrick Wendell 08c1a42d7d Add a repartition operator.
This patch adds an operator called repartition with more straightforward
semantics than the current `coalesce` operator. There are a few use cases
where this operator is useful:

1. If a user wants to increase the number of partitions in the RDD. This
is more common now with streaming. E.g. a user is ingesting data on one
node but they want to add more partitions to ensure parallelism of
subsequent operations across threads or the cluster.

Right now they have to call rdd.coalesce(numSplits, shuffle=true) - that's
super confusing.

2. If a user has input data where the number of partitions is not known. E.g.

> sc.textFile("some file").coalesce(50)....

This is both vague semantically (am I growing or shrinking this RDD) but also,
may not work correctly if the base RDD has fewer than 50 partitions.

The new operator forces shuffles every time, so it will always produce exactly
the number of new partitions. It also throws an exception rather than silently
not-working if a bad input is passed.

I am currently adding streaming tests (requires refactoring some of the test
suite to allow testing at partition granularity), so this is not ready for
merge yet. But feedback is welcome.
2013-10-24 14:31:33 -07:00
Kay Ousterhout b45352e373 Clear akka frame size property in tests 2013-10-23 18:23:28 -07:00
Kay Ousterhout c42f5d1787 Fixed broken tests 2013-10-23 17:35:01 -07:00
Kay Ousterhout a5f8f54ecd Merge remote-tracking branch 'upstream/master' into ui_new_state
Conflicts:
	core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala
2013-10-23 16:06:28 -07:00
Patrick Wendell c22046b3cc Minor clean-up in review 2013-10-22 11:00:50 -07:00
Patrick Wendell 7de0ea4d42 Response to code review and adding some more tests 2013-10-22 11:00:50 -07:00
Patrick Wendell 2fa3c4c49c Fix for Spark-870.
This patch fixes a bug where the Spark UI didn't display the correct number of total
tasks if the number of tasks in a Stage doesn't equal the number of RDD partitions.

It also cleans up the listener API a bit by embedding this information in the
StageInfo class rather than passing it seperately.
2013-10-22 11:00:25 -07:00
Patrick Wendell a854f5bfcf SPARK-940: Do not directly pass Stage objects to SparkListener. 2013-10-22 11:00:06 -07:00
Kay Ousterhout 916270f5f3 Show "GETTING_RESULTS" state in UI.
This commit adds a set of calls using the SparkListener interface
that indicate when a task is remotely fetching results, so that
we can display this (potentially time-consuming) phase of execution
to users through the UI.
2013-10-21 12:46:57 -07:00
Holden Karau 092b87e7c8 Remove extranious type definitions from inside of tests 2013-10-21 00:20:15 -07:00
Holden Karau 699f7d28c0 CR feedback 2013-10-21 00:10:03 -07:00
Holden Karau e58c69d955 Add tests for the Java implementation. 2013-10-20 01:17:13 -07:00
Josh Rosen 9159d2d09d Split MapOutputTracker into Master/Worker classes.
Previously, MapOutputTracker contained fields and methods that
were only applicable to the master or worker instances.  This
commit introduces a MasterMapOutputTracker class to prevent
the master-specific methods from being accessed on workers.

I also renamed a few methods and made others protected/private.
2013-10-19 20:01:22 -07:00
Holden Karau 2a37235825 Initial commit of adding histogram functionality to the DoubleRDDFunctions. 2013-10-19 00:57:25 -07:00
Matei Zaharia 599dcb0ddf Merge pull request #74 from rxin/kill
Job cancellation via job group id.

This PR adds a simple API to group together a set of jobs belonging to a thread and threads spawned from it. It also allows the cancellation of all jobs in this group.

An example:

    sc.setJobDescription("this_is_the_group_id", "some job description")
    sc.parallelize(1 to 10000, 2).map { i => Thread.sleep(10); i }.count()

In a separate thread:

    sc.cancelJobGroup("this_is_the_group_id")
2013-10-18 22:49:00 -07:00
Reynold Xin 806f3a3adb Job cancellation via job group id. 2013-10-18 21:46:08 -07:00
Hossein Falaki 2d511ab320 Made SerializableHyperLogLog Externalizable and added Kryo tests 2013-10-18 15:30:45 -07:00
Hossein Falaki 79868fe724 Improved code style. 2013-10-17 23:39:20 -07:00
Mosharaf Chowdhury 08391dbcb8 Should compile now. 2013-10-17 23:06:17 -07:00
Mosharaf Chowdhury 8612641362 Added an after block to reset spark.broadcast.factory 2013-10-17 22:44:04 -07:00
Hossein Falaki ec5df800fd Added countDistinctByKey to PairRDDFunctions that counts the approximate number of unique values for each key in the RDD. 2013-10-17 22:26:00 -07:00
Hossein Falaki 1a701358c0 Added a countDistinct method to RDD that takes takes an accuracy parameter and returns the (approximate) number of distinct elements in the RDD. 2013-10-17 22:24:48 -07:00
Mosharaf Chowdhury e178ae4e9b BroadcastSuite updated to test both HttpBroadcast and TorrentBroadcast in local, local[N], local-cluster settings. 2013-10-17 16:38:43 -07:00
Kay Ousterhout 809f547633 Fixed unit tests 2013-10-16 23:16:12 -07:00
Reynold Xin 3e7df8f6c6 Added a number of very fast, memory-efficient data structures: BitSet, OpenHashSet, OpenHashMap, PrimitiveKeyOpenHashMap. 2013-10-16 22:58:52 -07:00
Reynold Xin 9cd8786e4a Merge branch 'master' of github.com:apache/incubator-spark into kill
Conflicts:
	core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala
2013-10-14 21:51:30 -07:00
Aaron Davidson 4a45019fb0 Address Matei's comments 2013-10-14 00:24:17 -07:00
Aaron Davidson da896115ec Change BlockId filename to name + rest of Patrick's comments 2013-10-13 11:15:02 -07:00
Aaron Davidson d60352283c Add unit test and address rest of Reynold's comments 2013-10-12 22:45:15 -07:00
Aaron Davidson a395911138 Refactor BlockId into an actual type
This is an unfortunately invasive change which converts all of our BlockId
strings into actual BlockId types. Here are some advantages of doing this now:

+ Type safety

+ Code clarity - it's now obvious what the key of a shuffle or rdd block is,
  for instance. Additionally, appearing in tuple/map type signatures is a big
  readability bonus. A Seq[(String, BlockStatus)] is not very clear.
  Further, we can now use more Scala features, like matching on BlockId types.

+ Explicit usage - we can now formally tell where various BlockIds are being used
  (without doing string searches); this makes updating current BlockIds a much
  clearer process, and compiler-supported.
  (I'm looking at you, shuffle file consolidation.)

+ It will only get harder to make this change as time goes on.

Since this touches a lot of files, it'd be best to either get this patch
in quickly or throw it on the ground to avoid too many secondary merge conflicts.
2013-10-12 22:44:57 -07:00
Reynold Xin 88866ea9c9 Fixed PairRDDFunctionsSuite after removing InterruptibleRDD. 2013-10-12 20:05:23 -07:00
Reynold Xin 6b288b75d4 Job cancellation: address Matei's code review feedback. 2013-10-12 15:53:31 -07:00
Reynold Xin 97ffebbe87 Fixed dagscheduler suite because of a logging message change. 2013-10-11 16:18:22 -07:00
Reynold Xin a61cf40ab9 Job cancellation: addressed code review feedback from Kay. 2013-10-11 15:58:14 -07:00
Reynold Xin e2047d3927 Making takeAsync and collectAsync deterministic. 2013-10-11 13:04:45 -07:00
Reynold Xin 42fb1df694 Merge branch 'master' of github.com:apache/incubator-spark into kill
Conflicts:
	core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala
2013-10-10 23:48:05 -07:00
Reynold Xin d9e724e756 Fixed the broken local scheduler test. 2013-10-10 23:08:13 -07:00
Reynold Xin 37397b73ba Added comprehensive tests for job cancellation in a variety of environments (local vs cluster, fifo vs fair). 2013-10-10 22:57:43 -07:00
Reynold Xin 80cdbf4f49 Switched to use daemon thread in executor and fixed a bug in job cancellation for fair scheduler. 2013-10-10 22:40:48 -07:00
Reynold Xin ec2e2ed1e1 Use the same Executor in LocalScheduler as in ClusterScheduler. 2013-10-10 18:55:25 -07:00
Matei Zaharia c71499b779 Merge pull request #19 from aarondav/master-zk
Standalone Scheduler fault tolerance using ZooKeeper

This patch implements full distributed fault tolerance for standalone scheduler Masters.
There is only one master Leader at a time, which is actively serving scheduling
requests. If this Leader crashes, another master will eventually be elected, reconstruct
the state from the first Master, and continue serving scheduling requests.

Leader election is performed using the ZooKeeper leader election pattern. We try to minimize
the use of ZooKeeper and the assumptions about ZooKeeper's behavior, so there is a layer of
retries and session monitoring on top of the ZooKeeper client.

Master failover follows directly from the single-node Master recovery via the file
system (patch d5a96fe), save that the Master state is stored in ZooKeeper instead.

Configuration:
By default, no recovery mechanism is enabled (spark.deploy.recoveryMode = NONE).
By setting spark.deploy.recoveryMode to ZOOKEEPER and setting spark.deploy.zookeeper.url
to an appropriate ZooKeeper URL, ZooKeeper recovery mode is enabled.
By setting spark.deploy.recoveryMode to FILESYSTEM and setting spark.deploy.recoveryDirectory
to an appropriate directory accessible by the Master, we will keep the behavior of from d5a96fe.

Additionally, places where a Master could be specificied by a spark:// url can now take
comma-delimited lists to specify backup masters. Note that this is only used for registration
of NEW Workers and application Clients. Once a Worker or Client has registered with the
Master Leader, it is "in the system" and will never need to register again.
2013-10-10 17:16:42 -07:00
Matei Zaharia 001d13f7b9 Merge branch 'master' into fast-map
Conflicts:
	core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala
2013-10-10 13:26:43 -07:00
Reynold Xin 3bd2890d2b Fixed the deadlock situation in multi-job actions and added more unit tests. 2013-10-10 12:07:09 -07:00
Prashant Sharma 34da58ae50 Changed message-frame-size to maximum-frame-size as property.
Removed a test accidentally added during merge.
2013-10-10 15:13:44 +05:30
Reynold Xin 0353f74a9a Put the job cancellation handling into the dagscheduler's main event loop. 2013-10-10 00:28:00 -07:00
Reynold Xin dbae7795ba Merge branch 'master' of github.com:apache/incubator-spark into kill
Conflicts:
	core/src/main/scala/org/apache/spark/CacheManager.scala
	core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala
	core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerSource.scala
2013-10-09 22:57:35 -07:00
Reynold Xin 53895f9cde Implemented FutureAction, FutureJob, CancellablePromise.
Implemented more unit tests for async actions.
2013-10-09 22:43:06 -07:00
Prashant Sharma 026ab75661 Merge branch 'master' of github.com:apache/incubator-spark into scala-2.10 2013-10-10 09:42:55 +05:30
Prashant Sharma 26860639c5 Merge branch 'scala-2.10' of github.com:ScrapCodes/spark into scala-2.10
Conflicts:
	core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManager.scala
	project/SparkBuild.scala
2013-10-10 09:42:23 +05:30
Reynold Xin 320418f7c8 Merge pull request #49 from mateiz/kryo-fix-2
Fix Chill serialization of Range objects

It used to write out each element one by one, creating very large objects.
2013-10-09 16:55:30 -07:00
Matei Zaharia c84c205289 Fix Chill serialization of Range objects, which used to write out each
element, and register user and Spark classes before Chill's serializers
to let them override Chill's behavior in general.
2013-10-09 16:23:40 -07:00
Kay Ousterhout 36966f65df Style fixes 2013-10-09 15:36:34 -07:00
Kay Ousterhout a34a4e8174 Fix race condition in SparkListenerSuite (fixes SPARK-908). 2013-10-09 15:07:53 -07:00
Matei Zaharia 4acbc5afdd Moved files that were in the wrong directory after package rename 2013-10-08 23:16:17 -07:00
Matei Zaharia 0e40cfabf8 Fix some review comments 2013-10-08 23:16:16 -07:00
Matei Zaharia b535db7d89 Added a fast and low-memory append-only map implementation for cogroup
and parallel reduce operations
2013-10-08 23:14:38 -07:00
Prashant Sharma 7be75682b9 Merge branch 'master' into wip-merge-master
Conflicts:
	bagel/pom.xml
	core/pom.xml
	core/src/test/scala/org/apache/spark/ui/UISuite.scala
	examples/pom.xml
	mllib/pom.xml
	pom.xml
	project/SparkBuild.scala
	repl/pom.xml
	streaming/pom.xml
	tools/pom.xml

In scala 2.10, a shorter representation is used for naming artifacts
 so changed to shorter scala version for artifacts and made it a property in pom.
2013-10-08 11:29:40 +05:30
Reynold Xin 213b70a2db Merge pull request #31 from sundeepn/branch-0.8
Resolving package conflicts with hadoop 0.23.9

Hadoop 0.23.9 is having a package conflict with easymock's dependencies.

(cherry picked from commit 023e3fdf00)
Signed-off-by: Reynold Xin <rxin@apache.org>
2013-10-07 10:54:22 -07:00
Patrick Wendell aa9fb84994 Merging build changes in from 0.8 2013-10-05 22:07:00 -07:00
Aaron Davidson 0f070279e7 Address Matei's comments 2013-10-05 15:15:29 -07:00
Martin Weindel e09f4a9601 fixed some warnings 2013-10-05 23:08:23 +02:00
Prashant Sharma 3e41495288 Fixed tests, changed property akka.remote.netty.x to akka.remote.netty.tcp.x 2013-10-05 16:39:25 +05:30
Prashant Sharma c810ee0690 Merge branch 'master' into scala-2.10
Conflicts:
	core/src/test/scala/org/apache/spark/DistributedSuite.scala
	project/SparkBuild.scala
2013-10-05 15:52:57 +05:30
Aaron Davidson db6f154940 Fix race conditions during recovery
One major change was the use of messages instead of raw functions as the
parameter of Akka scheduled timers. Since messages are serialized, unlike
raw functions, the behavior is easier to think about and doesn't cause
race conditions when exceptions are thrown.

Another change is to avoid using global pointers that might change without
a lock.
2013-10-04 19:54:33 -07:00
Andre Schumacher c84946fe21 Fixing SPARK-602: PythonPartitioner
Currently PythonPartitioner determines partition ID by hashing a
byte-array representation of PySpark's key. This PR lets
PythonPartitioner use the actual partition ID, which is required e.g.
for sorting via PySpark.
2013-10-04 11:56:47 -07:00
Reynold Xin d29e8035a0 Added countAsync and various unit tests for async actions. 2013-10-03 15:13:44 -07:00
Reynold Xin e8e917f209 Merge branch 'master' into kill
Conflicts:
	core/src/main/scala/org/apache/spark/TaskEndReason.scala
	core/src/main/scala/org/apache/spark/executor/Executor.scala
	core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManager.scala
	core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala
2013-10-02 23:01:34 -07:00
Reynold Xin 1c48ba0d9f Merge remote-tracking branch 'origin' into kill
Conflicts:
	core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala
	core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManager.scala
	core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala
2013-10-02 16:40:44 -07:00
Prashant Sharma 5829692885 Merge branch 'master' into scala-2.10
Conflicts:
	core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala
	docs/_config.yml
	project/SparkBuild.scala
	repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala
2013-10-01 11:57:24 +05:30
Kay Ousterhout 0dcad2edcb Added additional unit test for repeated task failures 2013-09-30 23:26:15 -07:00
Kay Ousterhout dea4677c88 Fixed compilation errors and broken test. 2013-09-30 22:07:01 -07:00
Kay Ousterhout 8deda427bc Merge remote-tracking branch 'upstream/master' into results_through-bm
Conflicts:
	core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterScheduler.scala
	core/src/main/scala/org/apache/spark/scheduler/cluster/ClusterTaskSetManager.scala
	core/src/main/scala/org/apache/spark/scheduler/local/LocalTaskSetManager.scala
2013-09-30 10:16:58 -07:00
Kay Ousterhout 58b764b7c6 Addressed Matei's code review comments 2013-09-30 10:11:59 -07:00
shane-huang 84849baf88 Merge branch 'reorgscripts' into scripts-reorg 2013-09-27 09:28:33 +08:00
Aaron Davidson 42d72308fb Add license notices 2013-09-26 15:45:20 -07:00
Reynold Xin 70a0b993d4 Merge pull request #14 from kayousterhout/untangle_scheduler
Improved organization of scheduling packages.

This commit does not change any code -- only file organization.
Please let me know if there was some masterminded strategy behind
the existing organization that I failed to understand!

There are two components of this change:
(1) Moving files out of the cluster package, and down
a level to the scheduling package. These files are all used by
the local scheduler in addition to the cluster scheduler(s), so
should not be in the cluster package. As a result of this change,
none of the files in the local package reference files in the
cluster package.

(2) Moving the mesos package to within the cluster package.
The mesos scheduling code is for a cluster, and represents a
specific case of cluster scheduling (the Mesos-related classes
often subclass cluster scheduling classes). Thus, the most logical
place for it seems to be within the cluster package.

The one thing about the scheduling code that seems a little funny to me
is the naming of the SchedulerBackends.  The StandaloneSchedulerBackend
is not just for Standalone mode, but instead is used by Mesos coarse grained
mode and Yarn, and the backend that *is* just for Standalone mode is instead called SparkDeploySchedulerBackend. I didn't change this because I wasn't sure if there
was a reason for this naming that I'm just not aware of.
2013-09-26 14:11:54 -07:00
Patrick Wendell 6566a19b38 Merge pull request #9 from rxin/limit
Smarter take/limit implementation.
2013-09-26 08:01:04 -07:00
Prashant Sharma 42f30b5590 Fixed UISuite, for case when port 4040 is already bound on machine running the test. 2013-09-26 14:38:42 +05:30
Prashant Sharma 604dc40996 Sync with master and some build fixes 2013-09-26 11:40:02 +05:30
Kay Ousterhout d85fe41b2b Improved organization of scheduling packages.
This commit does not change any code -- only file organization.

There are two components of this change:
(1) Moving files out of the cluster package, and down
a level to the scheduling package. These files are all used by
the local scheduler in addition to the cluster scheduler(s), so
should not be in the cluster package. As a result of this change,
none of the files in the local package reference files in the
cluster package.

(2) Moving the mesos package to within the cluster package.
The mesos scheduling code is for a cluster, and represents a
specific case of cluster scheduling (the Mesos-related classes
often subclass cluster scheduling classes). Thus, the most logical
place for it is within the cluster package.
2013-09-25 12:45:46 -07:00
Reynold Xin ff540a015b Merge branch 'master' of github.com:markhamstra/incubator-spark 2013-09-23 11:55:02 -07:00
Kay Ousterhout c75eb14fe5 Send Task results through the block manager when larger than Akka frame size.
This change requires adding an extra failure mode: tasks can complete
successfully, but the result gets lost or flushed from the block manager
before it's been fetched.
2013-09-22 21:20:48 -07:00
shane-huang dfbdc9ddb7 added spark-class and spark-executor to sbin
Signed-off-by: shane-huang <shengsheng.huang@intel.com>
2013-09-23 11:28:58 +08:00
Reynold Xin a2ea069a5f Merge pull request #937 from jerryshao/localProperties-fix
Fix PR926 local properties issues in Spark Streaming like scenarios
2013-09-21 23:04:42 -07:00
Prashant Sharma 276c37a51c Akka 2.2 migration 2013-09-22 08:20:12 +05:30
jerryshao aa0c29f747 Add barrier for local properties unit test and fix some styles 2013-09-22 09:53:11 +08:00
Reynold Xin 42571d30d0 Smarter take/limit implementation. 2013-09-20 17:09:53 -07:00
Ankur Dave 026dba6aba After unit tests, clear port properties unconditionally
In MapOutputTrackerSuite, the "remote fetch" test sets spark.driver.port
and spark.hostPort, assuming that they will be cleared by
LocalSparkContext. However, the test never sets sc, so it remains null,
causing LocalSparkContext to skip clearing these properties. Subsequent
tests therefore fail with java.net.BindException: "Address already in
use".

This commit makes LocalSparkContext clear the properties even if sc is
null.
2013-09-19 22:05:23 -07:00
jerryshao ffa5f8e11d Fix issue when local properties pass from parent to child thread 2013-09-18 17:33:24 +08:00
Reynold Xin 37d8f37a8e Added a submitJob interface that returns a Future of the result. 2013-09-17 21:13:59 -07:00
Reynold Xin cbc48be13b Initial commit for job killing. 2013-09-16 18:54:06 -07:00
Prashant Sharma 383e151fd7 Merge branch 'master' of git://github.com/mesos/spark into scala-2.10
Conflicts:
	core/src/main/scala/org/apache/spark/SparkContext.scala
	project/SparkBuild.scala
2013-09-15 10:55:12 +05:30
Patrick Wendell bddf135670 Change port from 3030 to 4040 2013-09-11 10:01:38 -07:00
Matei Zaharia a85758c200 Merge pull request #907 from stephenh/document_coalesce_shuffle
Add better docs for coalesce.
2013-09-09 13:45:40 -07:00
Stephen Haberman 59003d387d Use a set since shuffle could change order. 2013-09-09 11:45:03 -05:00
Matei Zaharia 7d3204b056 Merge pull request #905 from mateiz/docs2
Job scheduling and cluster mode docs
2013-09-08 21:39:12 -07:00
Patrick Wendell f68848d95d Merge pull request #906 from pwendell/ganglia-sink
Clean-up of Metrics Code/Docs and Add Ganglia Sink
2013-09-08 18:32:16 -07:00
Matei Zaharia 170b3869ee Fix unit test failure due to changed default 2013-09-08 17:51:27 -07:00
Patrick Wendell c190b48bf5 Adding more docs and some code cleanup 2013-09-08 13:46:28 -07:00
Stephen Haberman df5fd35273 Add better docs for coalesce.
Include the useful tip that if shuffle=true, coalesce can actually
increase the number of partitions.

This makes coalesce more like a generic `RDD.repartition` operation.

(Ideally this `RDD.repartition` could automatically choose either a coalesce or
a shuffle if numPartitions was either less than or greater than, respectively,
the current number of partitions.)
2013-09-08 15:39:04 -05:00
Matei Zaharia 651a96adf7 More fair scheduler docs and property names.
Also changed uses of "job" terminology to "application" when they
referred to an entire Spark program, to avoid confusion.
2013-09-08 00:29:11 -07:00
Matei Zaharia 98fb69822c Work in progress:
- Add job scheduling docs
- Rename some fair scheduler properties
- Organize intro page better
- Link to Apache wiki for "contributing to Spark"
2013-09-08 00:29:11 -07:00
Reynold Xin 1e15feb5a3 Hot fix to resolve the compilation error caused by SPARK-821. 2013-09-06 22:44:05 +08:00
Prashant Sharma 4106ae9fbf Merged with master 2013-09-06 17:53:01 +05:30
Aaron Davidson 3a04e76c89 Reynold's second round of comments 2013-09-05 21:43:26 -07:00
Aaron Davidson 4f2236a1c5 Add unit test and address comments 2013-09-05 18:06:30 -07:00
Aaron Davidson 1418d18af4 SPARK-821: Don't cache results when action run locally on driver
Caching the results of local actions (e.g., rdd.first()) causes the driver to
store entire partitions in its own memory, which may be highly constrained.
This patch simply makes the CacheManager avoid caching the result of all locally-run computations.
2013-09-05 15:34:42 -07:00
Aaron Davidson 714e7f9e32 Fix line over 100 chars 2013-09-04 22:40:08 -07:00
Aaron Davidson 37db141aef Address Patrick's comments 2013-09-04 21:34:20 -07:00
Aaron Davidson 9e6f2b6822 SPARK-884: Add unit test to validate Spark JSON output
This unit test simply validates that the outputs of
the JsonProtocol methods are syntactically valid JSON.
2013-09-04 15:26:46 -07:00
Mark Hamstra c9bc8af3d1 Removed repetative import; fixes hidden definition compiler warning. 2013-09-03 15:25:20 -07:00
Matei Zaharia 12b2f1f9c9 Add missing license headers found with RAT 2013-09-02 12:23:03 -07:00
Matei Zaharia 246bf67f58 Fix test 2013-09-02 10:57:34 -07:00
Matei Zaharia 0a8cc30921 Move some classes to more appropriate packages:
* RDD, *RDDFunctions -> org.apache.spark.rdd
* Utils, ClosureCleaner, SizeEstimator -> org.apache.spark.util
* JavaSerializer, KryoSerializer -> org.apache.spark.serializer
2013-09-01 14:13:16 -07:00
Matei Zaharia 46eecd110a Initial work to rename package to org.apache.spark 2013-09-01 14:13:13 -07:00
Matei Zaharia 53cd50c069 Change build and run instructions to use assemblies
This commit makes Spark invocation saner by using an assembly JAR to
find all of Spark's dependencies instead of adding all the JARs in
lib_managed. It also packages the examples into an assembly and uses
that as SPARK_EXAMPLES_JAR. Finally, it replaces the old "run" script
with two better-named scripts: "run-examples" for examples, and
"spark-class" for Spark internal classes (e.g. REPL, master, etc). This
is also designed to minimize the confusion people have in trying to use
"run" to run their own classes; it's not meant to do that, but now at
least if they look at it, they can modify run-examples to do a decent
job for them.

As part of this, Bagel's examples are also now properly moved to the
examples package instead of bagel.
2013-08-29 21:19:04 -07:00
Ali Ghodsi c0942a710f Bug in test fixed 2013-08-20 16:16:05 -07:00
Ali Ghodsi 5db41919b5 Added a test to make sure no locality preferences are ignored 2013-08-20 16:16:05 -07:00
Ali Ghodsi 7b123b3126 Simpler code 2013-08-20 16:16:05 -07:00
Ali Ghodsi a75a64eade Fixed almost all of Matei's feedback 2013-08-20 16:16:05 -07:00