aboutsummaryrefslogtreecommitdiff
Commit message (Collapse)AuthorAgeFilesLines
* Removed StreamingContext.registerInputStream and registerOutputStream - they ↵Tathagata Das2014-01-1321-107/+115
| | | | were useless as InputDStream has been made to register itself. Also made DStream.register() private[streaming] - not useful to expose the confusing function. Updated a lot of documentation.
* Merge remote-tracking branch 'apache/master' into filestream-fixTathagata Das2014-01-131-1/+1
|\
| * Merge pull request #410 from rxin/scaladoc1Reynold Xin2014-01-131-1/+1
| |\ | | | | | | | | | | | | | | | Updated JavaStreamingContext to make scaladoc compile. `sbt/sbt doc` used to fail. This fixed it.
| | * Updated JavaStreamingContext to make scaladoc compile.Reynold Xin2014-01-131-1/+1
| |/ | | | | | | `sbt/sbt doc` used to fail. This fixed it.
* / Improved file input stream further.Tathagata Das2014-01-132-62/+69
|/
* Merge pull request #400 from tdas/dstream-movePatrick Wendell2014-01-1340-78/+121
|\ | | | | | | | | | | | | | | Moved DStream and PairDSream to org.apache.spark.streaming.dstream Similar to the package location of `org.apache.spark.rdd.RDD`, `DStream` has been moved from `org.apache.spark.streaming.DStream` to `org.apache.spark.streaming.dstream.DStream`. I know that the package name is a little long, but I think its better to keep it consistent with Spark's structure. Also fixed persistence of windowed DStream. The RDDs generated generated by windowed DStream are essentially unions of underlying RDDs, and persistent these union RDDs would store numerous copies of the underlying data. Instead setting the persistence level on the windowed DStream is made to set the persistence level of the underlying DStream.
| * Fixed import formatting.Tathagata Das2014-01-126-6/+6
| |
| * Merge remote-tracking branch 'apache/master' into dstream-moveTathagata Das2014-01-1212-24/+61
| |\ | | | | | | | | | | | | Conflicts: streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala
| * | Fixed persistence logic of WindowedDStream, and fixed default persistence ↵Tathagata Das2014-01-129-10/+41
| | | | | | | | | | | | level of input streams.
| * | Merge remote-tracking branch 'apache/master' into dstream-moveTathagata Das2014-01-124-6/+18
| |\ \
| * \ \ Merge branch 'error-handling' into dstream-moveTathagata Das2014-01-1211-70/+110
| |\ \ \
| * | | | Moved DStream, DStreamCheckpointData and PairDStream from ↵Tathagata Das2014-01-1237-70/+82
| | | | | | | | | | | | | | | | | | | | org.apache.spark.streaming to org.apache.spark.streaming.dstream.
* | | | | Merge pull request #397 from pwendell/host-portReynold Xin2014-01-1216-52/+7
|\ \ \ \ \ | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | 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).
| * | | | | Removing mentions in testsPatrick Wendell2014-01-1212-18/+2
| | | | | |
| * | | | | Remove now un-needed hostPort optionPatrick Wendell2014-01-124-34/+5
| | | | | |
* | | | | | Merge pull request #399 from pwendell/consolidate-offPatrick Wendell2014-01-122-2/+2
|\ \ \ \ \ \ | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | 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.
| * | | | | | Disable shuffle file consolidation by defaultPatrick Wendell2014-01-122-2/+2
| |/ / / / /
* | | | | | Merge pull request #395 from hsaputra/remove_simpleredundantreturn_scalaPatrick Wendell2014-01-1263-194/+194
|\ \ \ \ \ \ | |_|_|_|_|/ |/| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | 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.
| * | | | | Address code review concerns and comments.Henry Saputra2014-01-128-19/+18
| | | | | |
| * | | | | Fix accidental comment modification.Henry Saputra2014-01-121-1/+1
| | | | | |
| * | | | | Merge branch 'master' into remove_simpleredundantreturn_scalaHenry Saputra2014-01-12114-637/+3809
| |\| | | |
| * | | | | Remove simple redundant return statement for Scala methods/functions:Henry Saputra2014-01-1263-186/+187
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | -) Only change simple return statements at the end of method -) Ignore the complex if-else check -) Ignore the ones inside synchronized
* | | | | | Merge pull request #394 from tdas/error-handlingPatrick Wendell2014-01-1223-231/+591
|\ \ \ \ \ \ | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | Better error handling in Spark Streaming and more API cleanup Earlier errors in jobs generated by Spark Streaming (or in the generation of jobs) could not be caught from the main driver thread (i.e. the thread that called StreamingContext.start()) as it would be thrown in different threads. With this change, after `ssc.start`, one can call `ssc.awaitTermination()` which will be block until the ssc is closed, or there is an exception. This makes it easier to debug. This change also adds ssc.stop(<stop-spark-context>) where you can stop StreamingContext without stopping the SparkContext. Also fixes the bug that came up with PRs #393 and #381. MetadataCleaner default value has been changed from 3500 to -1 for normal SparkContext and 3600 when creating a StreamingContext. Also, updated StreamingListenerBus with changes similar to SparkListenerBus in #392. And changed a lot of protected[streaming] to private[streaming].
| * \ \ \ \ \ Merge remote-tracking branch 'apache/master' into error-handlingTathagata Das2014-01-124-6/+18
| |\ \ \ \ \ \ | | |_|_|_|/ / | |/| | | | / | | | |_|_|/ | | |/| | |
| * | | | | Changed StreamingContext.stopForWait to awaitTermination.Tathagata Das2014-01-124-15/+15
| | | | | |
| * | | | | Fixed bugs to ensure better cleanup of JobScheduler, JobGenerator and ↵Tathagata Das2014-01-129-56/+96
| | |_|_|/ | |/| | | | | | | | | | | | | NetworkInputTracker upon close.
| * | | | Fixed bugs.Tathagata Das2014-01-123-3/+3
| | | | |
| * | | | Merge remote-tracking branch 'apache/master' into error-handlingTathagata Das2014-01-1130-174/+1347
| |\ \ \ \ | | | |_|/ | | |/| |
| * | | | Added waitForStop and stop to JavaStreamingContext.Tathagata Das2014-01-112-3/+23
| | | | |
| * | | | Converted JobScheduler to use actors for event handling. Changed ↵Tathagata Das2014-01-1117-185/+485
| | | | | | | | | | | | | | | | | | | | protected[streaming] to private[streaming] in StreamingContext and DStream. Added waitForStop to StreamingContext, and StreamingContextSuite.
* | | | | Merge pull request #398 from pwendell/streaming-apiPatrick Wendell2014-01-1210-28/+65
|\ \ \ \ \ | |_|_|/ / |/| | | | | | | | | | | | | | | | | | | | | | | | | | | | | Rename DStream.foreach to DStream.foreachRDD `foreachRDD` makes it clear that the granularity of this operator is per-RDD. As it stands, `foreach` is inconsistent with with `map`, `filter`, and the other DStream operators which get pushed down to individual records within each RDD.
| * | | | Adding deprecated versions of old codePatrick Wendell2014-01-122-8/+45
| | | | |
| * | | | Rename DStream.foreach to DStream.foreachRDDPatrick Wendell2014-01-1210-22/+22
| | |/ / | |/| | | | | | | | | | | | | | | | | | `foreachRDD` makes it clear that the granularity of this operator is per-RDD. As it stands, `foreach` is inconsistent with with `map`, `filter`, and the other DStream operators which get pushed down to individual records within each RDD.
* | | | Merge pull request #396 from pwendell/executor-envPatrick Wendell2014-01-121-1/+1
|\ \ \ \ | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | 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.
| * | | | Setting load defaults to true in executorPatrick Wendell2014-01-121-1/+1
| |/ / /
* | | | Merge pull request #392 from rxin/listenerbusReynold Xin2014-01-123-5/+17
|\ \ \ \ | |/ / / |/| | | | | | | | | | | | | | | 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).
| * | | Stop SparkListenerBus daemon thread when DAGScheduler is stopped.Reynold Xin2014-01-113-5/+17
| | | |
* | | | Merge pull request #389 from rxin/clone-writablesReynold Xin2014-01-113-41/+71
|\ \ \ \ | | | | | | | | | | | | | | | Minor update for clone writables and more documentation.
| * | | | Renamed cloneKeyValues to cloneRecords; updated docs.Reynold Xin2014-01-113-44/+45
| | | | |
| * | | | Minor update for clone writables and more documentation.Reynold Xin2014-01-113-12/+41
| |/ / /
* | | | Merge pull request #388 from pwendell/masterReynold Xin2014-01-111-1/+1
|\ \ \ \ | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | Fix UI bug introduced in #244. The 'duration' field was incorrectly renamed to 'task time' in the table that lists stages.
| * | | | Fix UI bug introduced in #244.Patrick Wendell2014-01-111-1/+1
| | | | | | | | | | | | | | | | | | | | | | | | | The 'duration' field was incorrectly renamed to 'task time' in the table that lists stages.
* | | | | Merge pull request #393 from pwendell/revert-381Patrick Wendell2014-01-112-2/+2
|\ \ \ \ \ | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | Revert PR 381 This PR missed a bunch of test cases that require "spark.cleaner.ttl". I think it is what is causing test failures on Jenkins right now (though it's a bit hard to tell because the DNS for cs.berkeley.edu is down). I'm submitting this to see if it fixes jeknins. I did try just patching various tests but it was taking a really long time because there are a bunch of them, so for now I'm just seeing if a revert works.
| * | | | | Revert "Fix default TTL for metadata cleaner"Patrick Wendell2014-01-111-1/+1
| | | | | | | | | | | | | | | | | | | | | | | | This reverts commit 669ba4caa95014f4511f842206c3e506f1a41a7a.
| * | | | | Revert "Fix one unit test that was not setting spark.cleaner.ttl"Patrick Wendell2014-01-111-1/+1
|/ / / / / | | | | | | | | | | | | | | | This reverts commit 942c80b34c4642de3b0237761bc1aaeb8cbdd546.
* | | | | Merge pull request #387 from jerryshao/conf-fixReynold Xin2014-01-111-7/+8
|\ \ \ \ \ | |_|/ / / |/| | | | | | | | | Fix configure didn't work small problem in ALS
| * | | | Fix configure didn't work small problem in ALSjerryshao2014-01-111-7/+8
| | | | |
* | | | | Merge pull request #359 from ScrapCodes/clone-writablesReynold Xin2014-01-114-49/+106
|\ \ \ \ \ | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | We clone hadoop key and values by default and reuse objects if asked to. We try to clone for most common types of writables and we call WritableUtils.clone otherwise intention is to optimize, for example for NullWritable there is no need and for Long, int and String creating a new object with value set would be faster than doing copy on object hopefully. There is another way to do this PR where we ask for both key and values whether to clone them or not, but could not think of a use case for it except either of them is actually a NullWritable for which I have already worked around. So thought that would be unnecessary.
| * | | | | Fixes corresponding to Reynolds feedback commentsPrashant Sharma2014-01-094-32/+43
| | | | | |
| * | | | | we clone hadoop key and values by default and reuse if specified.Prashant Sharma2014-01-084-41/+87
| | | | | |