Commit graph

2906 commits

Author SHA1 Message Date
Mridul Muralidharan 0aea33d39e Expose method and class - so that we can use it from user code (particularly since checkpoint directory is autogenerated now 2014-01-15 12:44:44 +05:30
Reynold Xin 74b46acdc5 Merge pull request #428 from pwendell/writeable-objects
Don't clone records for text files
2014-01-14 14:59:13 -08:00
Reynold Xin d601a76d1f Merge pull request #427 from pwendell/deprecate-aggregator
Deprecate rather than remove old combineValuesByKey function
2014-01-14 14:52:24 -08:00
Patrick Wendell b1b22b7a13 Style fix 2014-01-14 13:56:27 -08:00
Patrick Wendell 8ea2cd56e4 Adding fix covering combineCombinersByKey as well 2014-01-14 13:52:23 -08:00
Patrick Wendell b683608c9f Deprecate rather than remove old combineValuesByKey function 2014-01-14 12:15:10 -08:00
Patrick Wendell 6f965a46a9 Don't clone records for text files 2014-01-14 11:57:53 -08:00
Reynold Xin f12e506c9e Fixed a typo in JavaSparkContext's API doc. 2014-01-14 11:42:28 -08:00
Reynold Xin 1b5623fd0b Maintain Serializable API compatibility by reverting back to java.io.Serializable for Broadcast and Accumulator. 2014-01-14 11:30:59 -08:00
Reynold Xin 55db77416b Added license header for package.scala in the Java API package. 2014-01-14 11:20:12 -08:00
Reynold Xin f8c12e9457 Added package doc for the Java API. 2014-01-14 11:16:25 -08:00
Reynold Xin 6a12b9ebc5 Updated API doc for Accumulable and Accumulator. 2014-01-14 11:16:08 -08:00
Reynold Xin 71b3007dbd Broadcast variable visibility change & doc update.
Note that previously Broadcast class was accidentally marked as private[spark]. It needs to be public
for broadcast variables to work. Also exposing the broadcast varaible id.
2014-01-14 11:15:21 -08:00
Patrick Wendell 23034798d7 Add missing header files 2014-01-14 01:17:13 -08:00
Patrick Wendell 0984647aae Enable compression by default for spills 2014-01-13 23:25:25 -08:00
Patrick Wendell 4a805aff5e Merge pull request #367 from ankurdave/graphx
GraphX: Unifying Graphs and Tables

GraphX extends Spark's distributed fault-tolerant collections API and interactive console with a new graph API which leverages recent advances in graph systems (e.g., [GraphLab](http://graphlab.org)) to enable users to easily and interactively build, transform, and reason about graph structured data at scale. See http://amplab.github.io/graphx/.

Thanks to @jegonzal, @rxin, @ankurdave, @dcrankshaw, @jianpingjwang, @amatsukawa, @kellrott, and @adamnovak.

Tasks left:
- [x] Graph-level uncache
- [x] Uncache previous iterations in Pregel
- [x] ~~Uncache previous iterations in GraphLab~~ (postponed to post-release)
- [x] - Describe GC issue with GraphLab
- [ ] Write `docs/graphx-programming-guide.md`
- [x] - Mention future Bagel support in docs
- [ ] - Section on caching/uncaching in docs: As with Spark, cache something that is used more than once. In an iterative algorithm, try to cache and force (i.e., materialize) something every iteration, then uncache the cached things that depended on the newly materialized RDD but that won't be referenced again.
- [x] Undo modifications to core collections and instead copy them to org.apache.spark.graphx
- [x] Make Graph serializable to work around capture in Spark shell
- [x] Rename graph -> graphx in package name and subproject
- [x] Remove standalone PageRank
- [x] ~~Fix amplab/graphx#52 by checking `iter.hasNext`~~
2014-01-13 22:58:38 -08:00
Patrick Wendell 945fe7a37e Merge pull request #408 from pwendell/external-serializers
Improvements to external sorting

1. Adds the option of compressing outputs.
2. Adds batching to the serialization to prevent OOM on the read side.
3. Slight renaming of config options.
4. Use Spark's buffer size for reads in addition to writes.
2014-01-13 22:56:12 -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
Patrick Wendell 08b9fec93d Merge pull request #409 from tdas/unpersist
Automatically unpersisting RDDs that have been cleaned up from DStreams

Earlier RDDs generated by DStreams were forgotten but not unpersisted. The system relied on the natural BlockManager LRU to drop the data. The cleaner.ttl was a hammer to clean up RDDs but it is something that needs to be set separately and need to be set very conservatively (at best, few minutes). This automatic unpersisting allows the system to handle this automatically, which reduces memory usage. As a side effect it will also improve GC performance as there are less number of objects stored in memory. In fact, for some workloads, it may allow RDDs to be cached as deserialized, which speeds up processing without too much GC overheads.

This is disabled by default. To enable it set configuration spark.streaming.unpersist to true. In future release, this will be set to true by default.

Also, reduced sleep time in TaskSchedulerImpl.stop() from 5 second to 1 second. From my conversation with Matei, there does not seem to be any good reason for the sleep for letting messages be sent out be so long.
2014-01-13 22:29:03 -08:00
Andrew Or 839934140f Wording changes per Patrick 2014-01-13 20:51:38 -08:00
Reynold Xin 33022d6656 Adjusted visibility of various components. 2014-01-13 19:58:53 -08:00
Harvey 9e84e70509 Add default value for HadoopRDD's cloneRecords constructor arg, to maintain backwards compatibility. 2014-01-13 19:43:40 -08:00
Patrick Wendell d4cd5debf4 Fix for Kryo Serializer 2014-01-13 19:03:59 -08:00
Reynold Xin e2d25d2dfe Merge branch 'master' into graphx 2014-01-13 16:21:26 -08:00
Tathagata Das 27311b1332 Added unpersisting and modified testsuite to better test out metadata cleaning. 2014-01-13 14:57:07 -08:00
Patrick Wendell c3816de504 Changing option wording per discussion with Andrew 2014-01-13 13:25:06 -08:00
Patrick Wendell 5d61e051c2 Improvements to external sorting
1. Adds the option of compressing outputs.
2. Adds batching to the serialization to prevent OOM on the read side.
3. Slight renaming of config options.
4. Use Spark's buffer size for reads in addition to writes.
2014-01-13 12:21:39 -08:00
Andrew Or a1f0992fae Report bytes spilled for both memory and disk on Web UI 2014-01-12 23:42:57 -08:00
Andrew Or 69c9aebed0 Enable external sorting by default 2014-01-12 22:43:01 -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
Andrew Or 8d40e7222f Get rid of spill map in SparkEnv 2014-01-12 22:34:33 -08:00
Patrick Wendell 0b96d85c20 Merge pull request #399 from pwendell/consolidate-off
Disable shuffle file consolidation by default

After running various performance tests for the 0.9 release, this still seems to have performance issues even on XFS. So let's keep this off-by-default for 0.9 and users can experiment with it depending on their disk configurations.
2014-01-12 21:31:43 -08:00
Patrick Wendell 0ab505a29e Merge pull request #395 from hsaputra/remove_simpleredundantreturn_scala
Remove simple redundant return statements for Scala methods/functions

Remove simple redundant return statements 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
-) Add small changes to making var to val if possible and remove () for simple get

This hopefully makes the review simpler =)

Pass compile and tests.
2014-01-12 21:31:04 -08:00
Patrick Wendell 2802cc80bc Disable shuffle file consolidation by default 2014-01-12 19:16:43 -08:00
Henry Saputra 5a8abfb70e Address code review concerns and comments. 2014-01-12 19:15:09 -08:00
Tathagata Das aa2c993858 Merge remote-tracking branch 'apache/master' into error-handling 2014-01-12 17:37:46 -08:00
Patrick Wendell 074f50232f Merge pull request #396 from pwendell/executor-env
Setting load defaults to true in executor

This preserves the behavior in earlier releases. If properties are set for the executors via `spark-env.sh` on the slaves, then they should take precedence over spark defaults. This is useful for if system administrators are setting properties for a standalone cluster, such as shuffle locations.

/cc @andrewor14 who initially reported this issue.
2014-01-12 17:01:13 -08:00
Reynold Xin 82e2b92c6d Merge pull request #392 from rxin/listenerbus
Stop SparkListenerBus daemon thread when DAGScheduler is stopped.

Otherwise this leads to hundreds of SparkListenerBus daemon threads in our unit tests (and also problematic if user applications launches multiple SparkContext).
2014-01-12 16:55:11 -08:00
Patrick Wendell 0bb33076e2 Removing mentions in tests 2014-01-12 16:53:58 -08:00
Patrick Wendell 0d4886c000 Remove now un-needed hostPort option 2014-01-12 16:47:52 -08:00
Patrick Wendell cfb1e6c13c Setting load defaults to true in executor 2014-01-12 15:35:08 -08:00
Henry Saputra f1c5eca494 Fix accidental comment modification. 2014-01-12 10:40:21 -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
Tathagata Das 18f4889d96 Merge remote-tracking branch 'apache/master' into error-handling 2014-01-11 23:40:57 -08:00
Tathagata Das f5108ffc24 Converted JobScheduler to use actors for event handling. Changed protected[streaming] to private[streaming] in StreamingContext and DStream. Added waitForStop to StreamingContext, and StreamingContextSuite. 2014-01-11 23:15:09 -08:00
Reynold Xin 288a878999 Merge pull request #389 from rxin/clone-writables
Minor update for clone writables and more documentation.
2014-01-11 21:53:19 -08:00
Reynold Xin dbc11df411 Merge pull request #388 from pwendell/master
Fix UI bug introduced in #244.

The 'duration' field was incorrectly renamed to 'task time' in the table that
lists stages.
2014-01-11 18:07:13 -08:00
Reynold Xin 362cda18bc Renamed cloneKeyValues to cloneRecords; updated docs. 2014-01-11 18:01:29 -08:00