Commit graph

521 commits

Author SHA1 Message Date
Yadong Qi bd88b71853 [SPARK-3325][Streaming] Add a parameter to the method print in class DStream
This PR is a fixed version of the original PR #3237 by watermen and scwf.
This adds the ability to specify how many elements to print in `DStream.print`.

Author: Yadong Qi <qiyadong2010@gmail.com>
Author: q00251598 <qiyadong@huawei.com>
Author: Tathagata Das <tathagata.das1565@gmail.com>
Author: wangfei <wangfei1@huawei.com>

Closes #3865 from tdas/print-num and squashes the following commits:

cd34e9e [Tathagata Das] Fix bug
7c09f16 [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into HEAD
bb35d1a [Yadong Qi] Update MimaExcludes.scala
f8098ca [Yadong Qi] Update MimaExcludes.scala
f6ac3cb [Yadong Qi] Update MimaExcludes.scala
e4ed897 [Yadong Qi] Update MimaExcludes.scala
3b9d5cf [wangfei] fix conflicts
ec8a3af [q00251598] move to  Spark 1.3
26a70c0 [q00251598] extend the Python DStream's print
b589a4b [q00251598] add another print function
2015-01-02 15:09:41 -08:00
Josh Rosen fe6efacc0b [SPARK-5035] [Streaming] ReceiverMessage trait should extend Serializable
Spark Streaming's ReceiverMessage trait should extend Serializable in order to fix a subtle bug that only occurs when running on a real cluster:

If you attempt to send a fire-and-forget message to a remote Akka actor and that message cannot be serialized, then this seems to lead to more-or-less silent failures. As an optimization, Akka skips message serialization for messages sent within the same JVM. As a result, Spark's unit tests will never fail due to non-serializable Akka messages, but these will cause mostly-silent failures when running on a real cluster.

Before this patch, here was the code for ReceiverMessage:

```
/** Messages sent to the NetworkReceiver. */
private[streaming] sealed trait ReceiverMessage
private[streaming] object StopReceiver extends ReceiverMessage
```

Since ReceiverMessage does not extend Serializable and StopReceiver is a regular `object`, not a `case object`, StopReceiver will throw serialization errors. As a result, graceful receiver shutdown is broken on real clusters (and local-cluster mode) but works in local modes. If you want to reproduce this, try running the word count example from the Streaming Programming Guide in the Spark shell:

```
import org.apache.spark._
import org.apache.spark.streaming._
import org.apache.spark.streaming.StreamingContext._
val ssc = new StreamingContext(sc, Seconds(10))
// Create a DStream that will connect to hostname:port, like localhost:9999
val lines = ssc.socketTextStream("localhost", 9999)
// Split each line into words
val words = lines.flatMap(_.split(" "))
import org.apache.spark.streaming.StreamingContext._
// Count each word in each batch
val pairs = words.map(word => (word, 1))
val wordCounts = pairs.reduceByKey(_ + _)
// Print the first ten elements of each RDD generated in this DStream to the console
wordCounts.print()
ssc.start()
Thread.sleep(10000)
ssc.stop(true, true)
```

Prior to this patch, this would work correctly in local mode but fail when running against a real cluster (it would report that some receivers were not shut down).

Author: Josh Rosen <joshrosen@databricks.com>

Closes #3857 from JoshRosen/SPARK-5035 and squashes the following commits:

71d0eae [Josh Rosen] [SPARK-5035] ReceiverMessage trait should extend Serializable.
2014-12-31 16:02:47 -08:00
jerryshao fdc2aa4918 [SPARK-5028][Streaming]Add total received and processed records metrics to Streaming UI
This is a follow-up work of [SPARK-4537](https://issues.apache.org/jira/browse/SPARK-4537). Adding total received records and processed records metrics back to UI.

![screenshot](https://dl.dropboxusercontent.com/u/19230832/screenshot.png)

Author: jerryshao <saisai.shao@intel.com>

Closes #3852 from jerryshao/SPARK-5028 and squashes the following commits:

c8c4877 [jerryshao] Add total received and processed metrics to Streaming UI
2014-12-31 14:45:31 -08:00
Hari Shreedharan 3610d3c615 [SPARK-4790][STREAMING] Fix ReceivedBlockTrackerSuite waits for old file...
...s to get deleted before continuing.

Since the deletes are happening asynchronously, the getFileStatus call might throw an exception in older HDFS
versions, if the delete happens between the time listFiles is called on the directory and getFileStatus is called
on the file in the getFileStatus method.

This PR addresses this by adding an option to delete the files synchronously and then waiting for the deletion to
complete before proceeding.

Author: Hari Shreedharan <hshreedharan@apache.org>

Closes #3726 from harishreedharan/spark-4790 and squashes the following commits:

bbbacd1 [Hari Shreedharan] Call cleanUpOldLogs only once in the tests.
3255f17 [Hari Shreedharan] Add test for async deletion. Remove method from ReceiverTracker that does not take waitForCompletion.
e4c83ec [Hari Shreedharan] Making waitForCompletion a mandatory param. Remove eventually from WALSuite since the cleanup method returns only after all files are deleted.
af00fd1 [Hari Shreedharan] [SPARK-4790][STREAMING] Fix ReceivedBlockTrackerSuite waits for old files to get deleted before continuing.
2014-12-31 14:35:07 -08:00
zsxwing 6a89782944 [SPARK-4813][Streaming] Fix the issue that ContextWaiter didn't handle 'spurious wakeup'
Used `Condition` to rewrite `ContextWaiter` because it provides a convenient API `awaitNanos` for timeout.

Author: zsxwing <zsxwing@gmail.com>

Closes #3661 from zsxwing/SPARK-4813 and squashes the following commits:

52247f5 [zsxwing] Add explicit unit type
be42bcf [zsxwing] Update as per review suggestion
e06bd4f [zsxwing] Fix the issue that ContextWaiter didn't handle 'spurious wakeup'
2014-12-30 14:39:13 -08:00
CodingCat fda4331d58 SPARK-4971: Fix typo in BlockGenerator comment
Author: CodingCat <zhunansjtu@gmail.com>

Closes #3807 from CodingCat/new_branch and squashes the following commits:

5167f01 [CodingCat] fix typo in the comment
2014-12-26 12:04:46 -08:00
zsxwing f9ed2b6641 [SPARK-4608][Streaming] Reorganize StreamingContext implicit to improve API convenience
There is only one implicit function `toPairDStreamFunctions` in `StreamingContext`. This PR did similar reorganization like [SPARK-4397](https://issues.apache.org/jira/browse/SPARK-4397).

Compiled the following codes with Spark Streaming 1.1.0 and ran it with this PR. Everything is fine.
```Scala
import org.apache.spark._
import org.apache.spark.streaming._
import org.apache.spark.streaming.StreamingContext._

object StreamingApp {

  def main(args: Array[String]) {
    val conf = new SparkConf().setMaster("local[2]").setAppName("FileWordCount")
    val ssc = new StreamingContext(conf, Seconds(10))
    val lines = ssc.textFileStream("/some/path")
    val words = lines.flatMap(_.split(" "))
    val pairs = words.map(word => (word, 1))
    val wordCounts = pairs.reduceByKey(_ + _)
    wordCounts.print()

    ssc.start()
    ssc.awaitTermination()
  }
}
```

Author: zsxwing <zsxwing@gmail.com>

Closes #3464 from zsxwing/SPARK-4608 and squashes the following commits:

aa6d44a [zsxwing] Fix a copy-paste error
f74c190 [zsxwing] Merge branch 'master' into SPARK-4608
e6f9cc9 [zsxwing] Update the docs
27833bb [zsxwing] Remove `import StreamingContext._`
c15162c [zsxwing] Reorganize StreamingContext implicit to improve API convenience
2014-12-25 19:46:05 -08:00
jerryshao f205fe477c [SPARK-4537][Streaming] Expand StreamingSource to add more metrics
Add `processingDelay`, `schedulingDelay` and `totalDelay` for the last completed batch. Add `lastReceivedBatchRecords` and `totalReceivedBatchRecords` to the received records counting.

Author: jerryshao <saisai.shao@intel.com>

Closes #3466 from jerryshao/SPARK-4537 and squashes the following commits:

00f5f7f [jerryshao] Change the code style and add totalProcessedRecords
44721a6 [jerryshao] Further address the comments
c097ddc [jerryshao] Address the comments
02dd44f [jerryshao] Fix the addressed comments
c7a9376 [jerryshao] Expand StreamingSource to add more metrics
2014-12-25 19:39:49 -08:00
zsxwing b4d0db80a0 [SPARK-4873][Streaming] Use Future.zip instead of Future.flatMap(for-loop) in WriteAheadLogBasedBlockHandler
Use `Future.zip` instead of `Future.flatMap`(for-loop). `zip` implies these two Futures will run concurrently, while `flatMap` usually means one Future depends on the other one.

Author: zsxwing <zsxwing@gmail.com>

Closes #3721 from zsxwing/SPARK-4873 and squashes the following commits:

46a2cd9 [zsxwing] Use Future.zip instead of Future.flatMap(for-loop)
2014-12-24 19:49:41 -08:00
jerryshao 3f5f4cc4e7 [SPARK-4671][Streaming]Do not replicate streaming block when WAL is enabled
Currently streaming block will be replicated when specific storage level is set, since WAL is already fault tolerant, so replication is needless and will hurt the throughput of streaming application.

Hi tdas , as per discussed about this issue, I fixed with this implementation, I'm not is this the way you want, would you mind taking a look at it? Thanks a lot.

Author: jerryshao <saisai.shao@intel.com>

Closes #3534 from jerryshao/SPARK-4671 and squashes the following commits:

500b456 [jerryshao] Do not replicate streaming block when WAL is enabled
2014-12-23 15:45:53 -08:00
Ilayaperumal Gopinathan 10d69e9cbf [SPARK-4802] [streaming] Remove receiverInfo once receiver is de-registered
Once the streaming receiver is de-registered at executor, the `ReceiverTrackerActor` needs to
remove the corresponding reveiverInfo from the `receiverInfo` map at `ReceiverTracker`.

Author: Ilayaperumal Gopinathan <igopinathan@pivotal.io>

Closes #3647 from ilayaperumalg/receiverInfo-RTracker and squashes the following commits:

6eb97d5 [Ilayaperumal Gopinathan] Polishing based on the review
3640c86 [Ilayaperumal Gopinathan] Remove receiverInfo once receiver is de-registered
2014-12-23 15:14:54 -08:00
Ryan Williams 8176b7a02e [SPARK-4668] Fix some documentation typos.
Author: Ryan Williams <ryan.blake.williams@gmail.com>

Closes #3523 from ryan-williams/tweaks and squashes the following commits:

d2eddaa [Ryan Williams] code review feedback
ce27fc1 [Ryan Williams] CoGroupedRDD comment nit
c6cfad9 [Ryan Williams] remove unnecessary if statement
b74ea35 [Ryan Williams] comment fix
b0221f0 [Ryan Williams] fix a gendered pronoun
c71ffed [Ryan Williams] use names on a few boolean parameters
89954aa [Ryan Williams] clarify some comments in {Security,Shuffle}Manager
e465dac [Ryan Williams] Saved building-spark.md with Dillinger.io
83e8358 [Ryan Williams] fix pom.xml typo
dc4662b [Ryan Williams] typo fixes in tuning.md, configuration.md
2014-12-15 14:52:17 -08:00
Tathagata Das 8838ad7c13 [SPARK-4196][SPARK-4602][Streaming] Fix serialization issue in PairDStreamFunctions.saveAsNewAPIHadoopFiles
Solves two JIRAs in one shot
- Makes the ForechDStream created by saveAsNewAPIHadoopFiles serializable for checkpoints
- Makes the default configuration object used saveAsNewAPIHadoopFiles be the Spark's hadoop configuration

Author: Tathagata Das <tathagata.das1565@gmail.com>

Closes #3457 from tdas/savefiles-fix and squashes the following commits:

bb4729a [Tathagata Das] Same treatment for saveAsHadoopFiles
b382ea9 [Tathagata Das] Fix serialization issue in PairDStreamFunctions.saveAsNewAPIHadoopFiles.
2014-11-25 14:16:27 -08:00
Tathagata Das 69cd53eae2 [SPARK-4601][Streaming] Set correct call site for streaming jobs so that it is displayed correctly on the Spark UI
When running the NetworkWordCount, the description of the word count jobs are set as "getCallsite at DStream:xxx" . This should be set to the line number of the streaming application that has the output operation that led to the job being created. This is because the callsite is incorrectly set in the thread launching the jobs. This PR fixes that.

Author: Tathagata Das <tathagata.das1565@gmail.com>

Closes #3455 from tdas/streaming-callsite-fix and squashes the following commits:

69fc26f [Tathagata Das] Set correct call site for streaming jobs so that it is displayed correctly on the Spark UI
2014-11-25 06:50:36 -08:00
jerryshao fef27b2943 [SPARK-4381][Streaming]Add warning log when user set spark.master to local in Spark Streaming and there's no job executed
Author: jerryshao <saisai.shao@intel.com>

Closes #3244 from jerryshao/SPARK-4381 and squashes the following commits:

d2486c7 [jerryshao] Improve the warning log
d726e85 [jerryshao] Add local[1] to the filter condition
eca428b [jerryshao] Add warning log
2014-11-25 05:36:29 -08:00
q00251598 a51118a34a [SPARK-4535][Streaming] Fix the error in comments
change `NetworkInputDStream` to `ReceiverInputDStream`
change `ReceiverInputTracker` to `ReceiverTracker`

Author: q00251598 <qiyadong@huawei.com>

Closes #3400 from watermen/fix-comments and squashes the following commits:

75d795c [q00251598] change 'NetworkInputDStream' to 'ReceiverInputDStream' && change 'ReceiverInputTracker' to 'ReceiverTracker'
2014-11-25 04:01:56 -08:00
Tathagata Das cb0e9b0980 [SPARK-4518][SPARK-4519][Streaming] Refactored file stream to prevent files from being processed multiple times
Because of a corner case, a file already selected for batch t can get considered again for batch t+2. This refactoring fixes it by remembering all the files selected in the last 1 minute, so that this corner case does not arise. Also uses spark context's hadoop configuration to access the file system API for listing directories.

pwendell Please take look. I still have not run long-running integration tests, so I cannot say for sure whether this has indeed solved the issue. You could do a first pass on this in the meantime.

Author: Tathagata Das <tathagata.das1565@gmail.com>

Closes #3419 from tdas/filestream-fix2 and squashes the following commits:

c19dd8a [Tathagata Das] Addressed PR comments.
513b608 [Tathagata Das] Updated docs.
d364faf [Tathagata Das] Added the current time condition back
5526222 [Tathagata Das] Removed unnecessary imports.
38bb736 [Tathagata Das] Fix long line.
203bbc7 [Tathagata Das] Un-ignore tests.
eaef4e1 [Tathagata Das] Fixed SPARK-4519
9dbd40a [Tathagata Das] Refactored FileInputDStream to remember last few batches.
2014-11-24 13:50:20 -08:00
Yadong Qi c3002c4a61 [SPARK-4294][Streaming] UnionDStream stream should express the requirements in the same way as TransformedDStream
In class TransformedDStream:
```scala
require(parents.length > 0, "List of DStreams to transform is empty")
require(parents.map(.ssc).distinct.size == 1, "Some of the DStreams have different contexts")
require(parents.map(.slideDuration).distinct.size == 1,
"Some of the DStreams have different slide durations")
```

In class UnionDStream:
```scala
if (parents.length == 0)
{ throw new IllegalArgumentException("Empty array of parents") }
if (parents.map(.ssc).distinct.size > 1)
{ throw new IllegalArgumentException("Array of parents have different StreamingContexts") }
if (parents.map(.slideDuration).distinct.size > 1)
{ throw new IllegalArgumentException("Array of parents have different slide times") }
```

The function is the same, but the realization is not. I think they shoule be the same.

Author: Yadong Qi <qiyadong2010@gmail.com>

Closes #3152 from watermen/bug-fix1 and squashes the following commits:

ed66db6 [Yadong Qi] Change transform to union
b6b3b8b [Yadong Qi] The same function should have the same realization.
2014-11-19 15:53:06 -08:00
zsxwing 3bf7ceebb1 [SPARK-4481][Streaming][Doc] Fix the wrong description of updateFunc
Removed `If `this` function returns None, then corresponding state key-value pair will be eliminated.` for the description of `updateFunc: (Iterator[(K, Seq[V], Option[S])]) => Iterator[(K, S)]`

Author: zsxwing <zsxwing@gmail.com>

Closes #3356 from zsxwing/SPARK-4481 and squashes the following commits:

76a9891 [zsxwing] Add a note that keys may be added or removed
0ebc42a [zsxwing] Fix the wrong description of updateFunc
2014-11-19 13:17:15 -08:00
Tathagata Das 22fc4e751c [SPARK-4482][Streaming] Disable ReceivedBlockTracker's write ahead log by default
The write ahead log of ReceivedBlockTracker gets enabled as soon as checkpoint directory is set. This should not happen, as the WAL should be enabled only if the WAL is enabled in the Spark configuration.

Author: Tathagata Das <tathagata.das1565@gmail.com>

Closes #3358 from tdas/SPARK-4482 and squashes the following commits:

b740136 [Tathagata Das] Fixed bug in ReceivedBlockTracker
2014-11-19 13:06:48 -08:00
jerryshao 5930f64bf0 [SPARK-4062][Streaming]Add ReliableKafkaReceiver in Spark Streaming Kafka connector
Add ReliableKafkaReceiver in Kafka connector to prevent data loss if WAL in Spark Streaming is enabled. Details and design doc can be seen in [SPARK-4062](https://issues.apache.org/jira/browse/SPARK-4062).

Author: jerryshao <saisai.shao@intel.com>
Author: Tathagata Das <tathagata.das1565@gmail.com>
Author: Saisai Shao <saisai.shao@intel.com>

Closes #2991 from jerryshao/kafka-refactor and squashes the following commits:

5461f1c [Saisai Shao] Merge pull request #8 from tdas/kafka-refactor3
eae4ad6 [Tathagata Das] Refectored KafkaStreamSuiteBased to eliminate KafkaTestUtils and made Java more robust.
fab14c7 [Tathagata Das] minor update.
149948b [Tathagata Das] Fixed mistake
14630aa [Tathagata Das] Minor updates.
d9a452c [Tathagata Das] Minor updates.
ec2e95e [Tathagata Das] Removed the receiver's locks and essentially reverted to Saisai's original design.
2a20a01 [jerryshao] Address some comments
9f636b3 [Saisai Shao] Merge pull request #5 from tdas/kafka-refactor
b2b2f84 [Tathagata Das] Refactored Kafka receiver logic and Kafka testsuites
e501b3c [jerryshao] Add Mima excludes
b798535 [jerryshao] Fix the missed issue
e5e21c1 [jerryshao] Change to while loop
ea873e4 [jerryshao] Further address the comments
98f3d07 [jerryshao] Fix comment style
4854ee9 [jerryshao] Address all the comments
96c7a1d [jerryshao] Update the ReliableKafkaReceiver unit test
8135d31 [jerryshao] Fix flaky test
a949741 [jerryshao] Address the comments
16bfe78 [jerryshao] Change the ordering of imports
0894aef [jerryshao] Add some comments
77c3e50 [jerryshao] Code refactor and add some unit tests
dd9aeeb [jerryshao] Initial commit for reliable Kafka receiver
2014-11-14 14:33:37 -08:00
Soumitra Kumar 36ddeb7bf8 [SPARK-3660][STREAMING] Initial RDD for updateStateByKey transformation
SPARK-3660 : Initial RDD for updateStateByKey transformation

I have added a sample StatefulNetworkWordCountWithInitial inspired by StatefulNetworkWordCount.

Please let me know if any changes are required.

Author: Soumitra Kumar <kumar.soumitra@gmail.com>

Closes #2665 from soumitrak/master and squashes the following commits:

ee8980b [Soumitra Kumar] Fixed copy/paste issue.
304f636 [Soumitra Kumar] Added simpler version of updateStateByKey API with initialRDD and test.
9781135 [Soumitra Kumar] Fixed test, and renamed variable.
3da51a2 [Soumitra Kumar] Adding updateStateByKey with initialRDD API to JavaPairDStream.
2f78f7e [Soumitra Kumar] Merge remote-tracking branch 'upstream/master'
d4fdd18 [Soumitra Kumar] Renamed variable and moved method.
d0ce2cd [Soumitra Kumar] Merge remote-tracking branch 'upstream/master'
31399a4 [Soumitra Kumar] Merge remote-tracking branch 'upstream/master'
4efa58b [Soumitra Kumar] [SPARK-3660][STREAMING] Initial RDD for updateStateByKey transformation
8f40ca0 [Soumitra Kumar] Merge remote-tracking branch 'upstream/master'
dde4271 [Soumitra Kumar] Merge remote-tracking branch 'upstream/master'
fdd7db3 [Soumitra Kumar] Adding support of initial value for state update. SPARK-3660 : Initial RDD for updateStateByKey transformation
2014-11-12 12:25:31 -08:00
huangzhaowei 6e03de304e [Streaming][Minor]Replace some 'if-else' in Clock
Replace some 'if-else' statement by math.min and math.max in Clock.scala

Author: huangzhaowei <carlmartinmax@gmail.com>

Closes #3088 from SaintBacchus/StreamingClock and squashes the following commits:

7b7f8e7 [huangzhaowei] [Streaming][Minor]Replace some 'if-else' in Clock
2014-11-11 03:02:12 -08:00
surq ce6ed2abd1 [SPARK-3954][Streaming] Optimization to FileInputDStream
about convert files to RDDS there are 3 loops with files sequence in spark source.
loops files sequence:
1.files.map(...)
2.files.zip(fileRDDs)
3.files-size.foreach
It's will very time consuming when lots of files.So I do the following correction:
3 loops with files sequence => only one loop

Author: surq <surq@asiainfo.com>

Closes #2811 from surq/SPARK-3954 and squashes the following commits:

321bbe8 [surq]  updated the code style.The style from [for...yield]to [files.map(file=>{})]
88a2c20 [surq] Merge branch 'master' of https://github.com/apache/spark into SPARK-3954
178066f [surq] modify code's style. [Exceeds 100 columns]
626ef97 [surq] remove redundant import(ArrayBuffer)
739341f [surq] promote the speed of convert files to RDDS
2014-11-10 17:37:16 -08:00
Josh Rosen 7b41b17f32 [SPARK-4301] StreamingContext should not allow start() to be called after calling stop()
In Spark 1.0.0+, calling `stop()` on a StreamingContext that has not been started is a no-op which has no side-effects. This allows users to call `stop()` on a fresh StreamingContext followed by `start()`. I believe that this almost always indicates an error and is not behavior that we should support. Since we don't allow `start() stop() start()` then I don't think it makes sense to allow `stop() start()`.

The current behavior can lead to resource leaks when StreamingContext constructs its own SparkContext: if I call `stop(stopSparkContext=True)`, then I expect StreamingContext's underlying SparkContext to be stopped irrespective of whether the StreamingContext has been started. This is useful when writing unit test fixtures.

Prior discussions:
- https://github.com/apache/spark/pull/3053#discussion-diff-19710333R490
- https://github.com/apache/spark/pull/3121#issuecomment-61927353

Author: Josh Rosen <joshrosen@databricks.com>

Closes #3160 from JoshRosen/SPARK-4301 and squashes the following commits:

dbcc929 [Josh Rosen] Address more review comments
bdbe5da [Josh Rosen] Stop SparkContext after stopping scheduler, not before.
03e9c40 [Josh Rosen] Always stop SparkContext, even if stop(false) has already been called.
832a7f4 [Josh Rosen] Address review comment
5142517 [Josh Rosen] Add tests; improve Scaladoc.
813e471 [Josh Rosen] Revert workaround added in https://github.com/apache/spark/pull/3053/files#diff-e144dbee130ed84f9465853ddce65f8eR49
5558e70 [Josh Rosen] StreamingContext.stop() should stop SparkContext even if StreamingContext has not been started yet.
2014-11-08 18:10:23 -08:00
Tathagata Das 5f13759d36 [SPARK-4029][Streaming] Update streaming driver to reliably save and recover received block metadata on driver failures
As part of the initiative of preventing data loss on driver failure, this JIRA tracks the sub task of modifying the streaming driver to reliably save received block metadata, and recover them on driver restart.

This was solved by introducing a `ReceivedBlockTracker` that takes all the responsibility of managing the metadata of received blocks (i.e. `ReceivedBlockInfo`, and any actions on them (e.g, allocating blocks to batches, etc.). All actions to block info get written out to a write ahead log (using `WriteAheadLogManager`). On recovery, all the actions are replaying to recreate the pre-failure state of the `ReceivedBlockTracker`, which include the batch-to-block allocations and the unallocated blocks.

Furthermore, the `ReceiverInputDStream` was modified to create `WriteAheadLogBackedBlockRDD`s when file segment info is present in the `ReceivedBlockInfo`. After recovery of all the block info (through recovery `ReceivedBlockTracker`), the `WriteAheadLogBackedBlockRDD`s gets recreated with the recovered info, and jobs submitted. The data of the blocks gets pulled from the write ahead logs, thanks to the segment info present in the `ReceivedBlockInfo`.

This is still a WIP. Things that are missing here are.

- *End-to-end integration tests:* Unit tests that tests the driver recovery, by killing and restarting the streaming context, and verifying all the input data gets processed. This has been implemented but not included in this PR yet. A sneak peek of that DriverFailureSuite can be found in this PR (on my personal repo): https://github.com/tdas/spark/pull/25 I can either include it in this PR, or submit that as a separate PR after this gets in.

- *WAL cleanup:* Cleaning up the received data write ahead log, by calling `ReceivedBlockHandler.cleanupOldBlocks`. This is being worked on.

Author: Tathagata Das <tathagata.das1565@gmail.com>

Closes #3026 from tdas/driver-ha-rbt and squashes the following commits:

a8009ed [Tathagata Das] Added comment
1d704bb [Tathagata Das] Enabled storing recovered WAL-backed blocks to BM
2ee2484 [Tathagata Das] More minor changes based on PR
47fc1e3 [Tathagata Das] Addressed PR comments.
9a7e3e4 [Tathagata Das] Refactored ReceivedBlockTracker API a bit to make things a little cleaner for users of the tracker.
af63655 [Tathagata Das] Minor changes.
fce2b21 [Tathagata Das] Removed commented lines
59496d3 [Tathagata Das] Changed class names, made allocation more explicit and added cleanup
19aec7d [Tathagata Das] Fixed casting bug.
f66d277 [Tathagata Das] Fix line lengths.
cda62ee [Tathagata Das] Added license
25611d6 [Tathagata Das] Minor changes before submitting PR
7ae0a7fb [Tathagata Das] Transferred changes from driver-ha-working branch
2014-11-05 01:21:53 -08:00
Tathagata Das fb1fbca204 [SPARK-4027][Streaming] WriteAheadLogBackedBlockRDD to read received either from BlockManager or WAL in HDFS
As part of the initiative of preventing data loss on streaming driver failure, this sub-task implements a BlockRDD that is backed by HDFS. This BlockRDD can either read data from the Spark's BlockManager, or read the data from file-segments in write ahead log in HDFS.

Most of this code has been written by @harishreedharan

Author: Tathagata Das <tathagata.das1565@gmail.com>
Author: Hari Shreedharan <hshreedharan@apache.org>

Closes #2931 from tdas/driver-ha-rdd and squashes the following commits:

209e49c [Tathagata Das] Better fix to style issue.
4a5866f [Tathagata Das] Addressed one more comment.
ed5fbf0 [Tathagata Das] Minor updates.
b0a18b1 [Tathagata Das] Fixed import order.
20aa7c6 [Tathagata Das] Fixed more line length issues.
29aa099 [Tathagata Das] Fixed line length issues.
9e47b5b [Tathagata Das] Renamed class, simplified+added unit tests.
6e1bfb8 [Tathagata Das] Tweaks testuite to create spark contxt lazily to prevent contxt leaks.
9c86a61 [Tathagata Das] Merge pull request #22 from harishreedharan/driver-ha-rdd
2878c38 [Hari Shreedharan] Shutdown spark context after tests. Formatting/minor fixes
c709f2f [Tathagata Das] Merge pull request #21 from harishreedharan/driver-ha-rdd
5cce16f [Hari Shreedharan] Make sure getBlockLocations uses offset and length to find the blocks on HDFS
eadde56 [Tathagata Das] Transferred HDFSBackedBlockRDD for the driver-ha-working branch
2014-10-30 15:17:02 -07:00
Tathagata Das 234de9232b [SPARK-4028][Streaming] ReceivedBlockHandler interface to abstract the functionality of storage of received data
As part of the initiative to prevent data loss on streaming driver failure, this JIRA tracks the subtask of implementing a ReceivedBlockHandler, that abstracts the functionality of storage of received data blocks. The default implementation will maintain the current behavior of storing the data into BlockManager. The optional implementation will store the data to both BlockManager as well as a write ahead log.

Author: Tathagata Das <tathagata.das1565@gmail.com>

Closes #2940 from tdas/driver-ha-rbh and squashes the following commits:

78a4aaa [Tathagata Das] Fixed bug causing test failures.
f192f47 [Tathagata Das] Fixed import order.
df5f320 [Tathagata Das] Updated code to use ReceivedBlockStoreResult as the return type for handler's storeBlock
33c30c9 [Tathagata Das] Added license, and organized imports.
2f025b3 [Tathagata Das] Updates based on PR comments.
18aec1e [Tathagata Das] Moved ReceivedBlockInfo back into spark.streaming.scheduler package
95a4987 [Tathagata Das] Added ReceivedBlockHandler and its associated tests
2014-10-30 14:51:13 -07:00
Josh Rosen 6c98c29ae0 [SPARK-4080] Only throw IOException from [write|read][Object|External]
If classes implementing Serializable or Externalizable interfaces throw
exceptions other than IOException or ClassNotFoundException from their
(de)serialization methods, then this results in an unhelpful
"IOException: unexpected exception type" rather than the actual exception that
produced the (de)serialization error.

This patch fixes this by adding a utility method that re-wraps any uncaught
exceptions in IOException (unless they are already instances of IOException).

Author: Josh Rosen <joshrosen@databricks.com>

Closes #2932 from JoshRosen/SPARK-4080 and squashes the following commits:

cd3a9be [Josh Rosen] [SPARK-4080] Only throw IOException from [write|read][Object|External].
2014-10-24 15:06:15 -07:00
Hari Shreedharan 6a40a76848 [SPARK-4026][Streaming] Write ahead log management
As part of the effort to avoid data loss on Spark Streaming driver failure, we want to implement a write ahead log that can write received data to HDFS. This allows the received data to be persist across driver failures. So when the streaming driver is restarted, it can find and reprocess all the data that were received but not processed.

This was primarily implemented by @harishreedharan. This is still WIP, as he is going to improve the unitests by using HDFS mini cluster.

Author: Hari Shreedharan <hshreedharan@apache.org>
Author: Tathagata Das <tathagata.das1565@gmail.com>

Closes #2882 from tdas/driver-ha-wal and squashes the following commits:

e4bee20 [Tathagata Das] Removed synchronized, Path.getFileSystem is threadsafe
55514e2 [Tathagata Das] Minor changes based on PR comments.
d29fddd [Tathagata Das] Merge pull request #20 from harishreedharan/driver-ha-wal
a317a4d [Hari Shreedharan] Directory deletion should not fail tests
9514dc8 [Tathagata Das] Added unit tests to test reading of corrupted data and other minor edits
3881706 [Tathagata Das] Merge pull request #19 from harishreedharan/driver-ha-wal
4705fff [Hari Shreedharan] Sort listed files by name. Use local files for WAL tests.
eb356ca [Tathagata Das] Merge pull request #18 from harishreedharan/driver-ha-wal
82ce56e [Hari Shreedharan] Fix file ordering issue in WALManager tests
5ff90ee [Hari Shreedharan] Fix tests to not ignore ordering and also assert all data is present
ef8db09 [Tathagata Das] Merge pull request #17 from harishreedharan/driver-ha-wal
7e40e56 [Hari Shreedharan] Restore old build directory after tests
587b876 [Hari Shreedharan] Fix broken test. Call getFileSystem only from synchronized method.
b4be0c1 [Hari Shreedharan] Remove unused method
edcbee1 [Hari Shreedharan] Tests reading and writing data using writers now use Minicluster.
5c70d1f [Hari Shreedharan] Remove underlying stream from the WALWriter.
4ab602a [Tathagata Das] Refactored write ahead stuff from streaming.storage to streaming.util
b06be2b [Tathagata Das] Adding missing license.
5182ffb [Hari Shreedharan] Added documentation
172358d [Tathagata Das] Pulled WriteAheadLog-related stuff from tdas/spark/tree/driver-ha-working
2014-10-24 11:44:48 -07:00
Holden Karau 2aeb84bc79 replace awaitTransformation with awaitTermination in scaladoc/javadoc
Author: Holden Karau <holden@pigscanfly.ca>

Closes #2861 from holdenk/SPARK-4015-Documentation-in-the-streaming-context-references-non-existent-function and squashes the following commits:

081db8a [Holden Karau] fix pyspark streaming doc too
0e03863 [Holden Karau] replace awaitTransformation with awaitTermination
2014-10-21 09:37:17 -07:00
giwa 69c67abaa9 [SPARK-2377] Python API for Streaming
This patch brings Python API for Streaming.

This patch is based on work from @giwa

Author: giwa <ugw.gi.world@gmail.com>
Author: Ken Takagiwa <ken@Kens-MacBook-Pro.local>
Author: Davies Liu <davies.liu@gmail.com>
Author: Ken Takagiwa <ken@kens-mbp.gateway.sonic.net>
Author: Tathagata Das <tathagata.das1565@gmail.com>
Author: Ken <ugw.gi.world@gmail.com>
Author: Ken Takagiwa <ugw.gi.world@gmail.com>
Author: Matthew Farrellee <matt@redhat.com>

Closes #2538 from davies/streaming and squashes the following commits:

64561e4 [Davies Liu] fix tests
331ecce [Davies Liu] fix example
3e2492b [Davies Liu] change updateStateByKey() to easy API
182be73 [Davies Liu] Merge branch 'master' of github.com:apache/spark into streaming
02d0575 [Davies Liu] add wrapper for foreachRDD()
bebeb4a [Davies Liu] address all comments
6db00da [Davies Liu] Merge branch 'master' of github.com:apache/spark into streaming
8380064 [Davies Liu] Merge branch 'master' of github.com:apache/spark into streaming
52c535b [Davies Liu] remove fix for sum()
e108ec1 [Davies Liu]  address comments
37fe06f [Davies Liu] use random port for callback server
d05871e [Davies Liu] remove reuse of PythonRDD
be5e5ff [Davies Liu] merge branch of env, make tests stable.
8071541 [Davies Liu] Merge branch 'env' into streaming
c7bbbce [Davies Liu] fix sphinx docs
6bb9d91 [Davies Liu] Merge branch 'master' of github.com:apache/spark into streaming
4d0ea8b [Davies Liu] clear reference of SparkEnv after stop
54bd92b [Davies Liu] improve tests
c2b31cb [Davies Liu] Merge branch 'master' of github.com:apache/spark into streaming
7a88f9f [Davies Liu] rollback RDD.setContext(), use textFileStream() to test checkpointing
bd8a4c2 [Davies Liu] fix scala style
7797c70 [Davies Liu] refactor
ff88bec [Davies Liu] rename RDDFunction to TransformFunction
d328aca [Davies Liu] fix serializer in queueStream
6f0da2f [Davies Liu] recover from checkpoint
fa7261b [Davies Liu] refactor
a13ff34 [Davies Liu] address comments
8466916 [Davies Liu] support checkpoint
9a16bd1 [Davies Liu] change number of partitions during tests
b98d63f [Davies Liu] change private[spark] to private[python]
eed6e2a [Davies Liu] rollback not needed changes
e00136b [Davies Liu] address comments
069a94c [Davies Liu] fix the number of partitions during window()
338580a [Davies Liu] change _first(), _take(), _collect() as private API
19797f9 [Davies Liu] clean up
6ebceca [Davies Liu] add more tests
c40c52d [Davies Liu] change first(), take(n) to has the same behavior as RDD
98ac6c2 [Davies Liu] support ssc.transform()
b983f0f [Davies Liu] address comments
847f9b9 [Davies Liu] add more docs, add first(), take()
e059ca2 [Davies Liu] move check of window into Python
fce0ef5 [Davies Liu] rafactor of foreachRDD()
7001b51 [Davies Liu] refactor of queueStream()
26ea396 [Davies Liu] refactor
74df565 [Davies Liu] fix print and docs
b32774c [Davies Liu] move java_import into streaming
604323f [Davies Liu] enable streaming tests
c499ba0 [Davies Liu] remove Time and Duration
3f0fb4b [Davies Liu] refactor fix tests
c28f520 [Davies Liu] support updateStateByKey
d357b70 [Davies Liu] support windowed dstream
bd13026 [Davies Liu] fix examples
eec401e [Davies Liu] refactor, combine TransformedRDD, fix reuse PythonRDD, fix union
9a57685 [Davies Liu] fix python style
bd27874 [Davies Liu] fix scala style
7339be0 [Davies Liu] delete tests
7f53086 [Davies Liu] support transform(), refactor and cleanup
df098fc [Davies Liu] Merge branch 'master' into giwa
550dfd9 [giwa] WIP fixing 1.1 merge
5cdb6fa [giwa] changed for SCCallSiteSync
e685853 [giwa] meged with rebased 1.1 branch
2d32a74 [giwa] added some StreamingContextTestSuite
4a59e1e [giwa] WIP:added more test for StreamingContext
8ffdbf1 [giwa] added atexit to handle callback server
d5f5fcb [giwa] added comment for StreamingContext.sparkContext
63c881a [giwa] added StreamingContext.sparkContext
d39f102 [giwa] added StreamingContext.remember
d542743 [giwa] clean up code
2fdf0de [Matthew Farrellee] Fix scalastyle errors
c0a06bc [giwa] delete not implemented functions
f385976 [giwa] delete inproper comments
b0f2015 [giwa] added comment in dstream._test_output
bebb3f3 [giwa] remove the last brank line
fbed8da [giwa] revert pom.xml
8ed93af [giwa] fixed explanaiton
066ba90 [giwa] revert pom.xml
fa4af88 [giwa] remove duplicated import
6ae3caa [giwa] revert pom.xml
7dc7391 [giwa] fixed typo
62dc7a3 [giwa] clean up exmples
f04882c [giwa] clen up examples
b171ec3 [giwa] fixed pep8 violation
f198d14 [giwa] clean up code
3166d31 [giwa] clean up
c00e091 [giwa] change test case not to use awaitTermination
e80647e [giwa] adopted the latest compression way of python command
58e41ff [giwa] merge with master
455e5af [giwa] removed wasted print in DStream
af336b7 [giwa] add comments
ddd4ee1 [giwa] added TODO coments
99ce042 [giwa] added saveAsTextFiles and saveAsPickledFiles
2a06cdb [giwa] remove waste duplicated code
c5ecfc1 [giwa] basic function test cases are passed
8dcda84 [giwa] all tests are passed if numSlice is 2 and the numver of each input is over 4
795b2cd [giwa] broke something
1e126bf [giwa] WIP: solved partitioned and None is not recognized
f67cf57 [giwa] added mapValues and flatMapVaules WIP for glom and mapPartitions test
953deb0 [giwa] edited the comment to add more precise description
af610d3 [giwa] removed unnesessary changes
c1d546e [giwa] fixed PEP-008 violation
99410be [giwa] delete waste file
b3b0362 [giwa] added basic operation test cases
9cde7c9 [giwa] WIP added test case
bd3ba53 [giwa] WIP
5c04a5f [giwa] WIP: added PythonTestInputStream
019ef38 [giwa] WIP
1934726 [giwa] update comment
376e3ac [giwa] WIP
932372a [giwa] clean up dstream.py
0b09cff [giwa] added stop in StreamingContext
92e333e [giwa] implemented reduce and count function in Dstream
1b83354 [giwa] Removed the waste line
88f7506 [Ken Takagiwa] Kill py4j callback server properly
54b5358 [Ken Takagiwa] tried to restart callback server
4f07163 [Tathagata Das] Implemented DStream.foreachRDD in the Python API using Py4J callback server.
fe02547 [Ken Takagiwa] remove waste file
2ad7bd3 [Ken Takagiwa] clean up codes
6197a11 [Ken Takagiwa] clean up code
eb4bf48 [Ken Takagiwa] fix map function
98c2a00 [Ken Takagiwa] added count operation but this implementation need double check
58591d2 [Ken Takagiwa] reduceByKey is working
0df7111 [Ken Takagiwa] delete old file
f485b1d [Ken Takagiwa] fied input of socketTextDStream
dd6de81 [Ken Takagiwa] initial commit for socketTextStream
247fd74 [Ken Takagiwa] modified the code base on comment in https://github.com/tdas/spark/pull/10
4bcb318 [Ken Takagiwa] implementing transform function in Python
38adf95 [Ken Takagiwa] added reducedByKey not working yet
66fcfff [Ken Takagiwa] modify dstream.py to fix indent error
41886c2 [Ken Takagiwa] comment PythonDStream.PairwiseDStream
0b99bec [Ken] initial commit for pySparkStreaming
c214199 [giwa] added testcase for combineByKey
5625bdc [giwa] added gorupByKey testcase
10ab87b [giwa] added sparkContext as input parameter in StreamingContext
10b5b04 [giwa] removed wasted print in DStream
e54f986 [giwa] add comments
16aa64f [giwa] added TODO coments
74535d4 [giwa] added saveAsTextFiles and saveAsPickledFiles
f76c182 [giwa] remove waste duplicated code
18c8723 [giwa] modified streaming test case to add coment
13fb44c [giwa] basic function test cases are passed
3000b2b [giwa] all tests are passed if numSlice is 2 and the numver of each input is over 4
ff14070 [giwa] broke something
bcdec33 [giwa] WIP: solved partitioned and None is not recognized
270a9e1 [giwa] added mapValues and flatMapVaules WIP for glom and mapPartitions test
bb10956 [giwa] edited the comment to add more precise description
253a863 [giwa] removed unnesessary changes
3d37822 [giwa] fixed PEP-008 violation
f21cab3 [giwa] delete waste file
878bad7 [giwa] added basic operation test cases
ce2acd2 [giwa] WIP added test case
9ad6855 [giwa] WIP
1df77f5 [giwa] WIP: added PythonTestInputStream
1523b66 [giwa] WIP
8a0fbbc [giwa] update comment
fe648e3 [giwa] WIP
29c2bc5 [giwa] initial commit for testcase
4d40d63 [giwa] clean up dstream.py
c462bb3 [giwa] added stop in StreamingContext
d2c01ba [giwa] clean up examples
3c45cd2 [giwa] implemented reduce and count function in Dstream
b349649 [giwa] Removed the waste line
3b498e1 [Ken Takagiwa] Kill py4j callback server properly
84a9668 [Ken Takagiwa] tried to restart callback server
9ab8952 [Tathagata Das] Added extra line.
05e991b [Tathagata Das] Added missing file
b1d2a30 [Tathagata Das] Implemented DStream.foreachRDD in the Python API using Py4J callback server.
678e854 [Ken Takagiwa] remove waste file
0a8bbbb [Ken Takagiwa] clean up codes
bab31c1 [Ken Takagiwa] clean up code
72b9738 [Ken Takagiwa] fix map function
d3ee86a [Ken Takagiwa] added count operation but this implementation need double check
15feea9 [Ken Takagiwa] edit python sparkstreaming example
6f98e50 [Ken Takagiwa] reduceByKey is working
c455c8d [Ken Takagiwa] added reducedByKey not working yet
dc6995d [Ken Takagiwa] delete old file
b31446a [Ken Takagiwa] fixed typo of network_workdcount.py
ccfd214 [Ken Takagiwa] added doctest for pyspark.streaming.duration
0d1b954 [Ken Takagiwa] fied input of socketTextDStream
f746109 [Ken Takagiwa] initial commit for socketTextStream
bb7ccf3 [Ken Takagiwa] remove unused import in python
224fc5e [Ken Takagiwa] add empty line
d2099d8 [Ken Takagiwa] sorted the import following Spark coding convention
5bac7ec [Ken Takagiwa] revert streaming/pom.xml
e1df940 [Ken Takagiwa] revert pom.xml
494cae5 [Ken Takagiwa] remove not implemented DStream functions in python
17a74c6 [Ken Takagiwa] modified the code base on comment in https://github.com/tdas/spark/pull/10
1a0f065 [Ken Takagiwa] implementing transform function in Python
d7b4d6f [Ken Takagiwa] added reducedByKey not working yet
87438e2 [Ken Takagiwa] modify dstream.py to fix indent error
b406252 [Ken Takagiwa] comment PythonDStream.PairwiseDStream
454981d [Ken] initial commit for pySparkStreaming
150b94c [giwa] added some StreamingContextTestSuite
f7bc8f9 [giwa] WIP:added more test for StreamingContext
ee50c5a [giwa] added atexit to handle callback server
fdc9125 [giwa] added comment for StreamingContext.sparkContext
f5bfb70 [giwa] added StreamingContext.sparkContext
da09768 [giwa] added StreamingContext.remember
d68b568 [giwa] clean up code
4afa390 [giwa] clean up code
1fd6bc7 [Ken Takagiwa] Merge pull request #2 from mattf/giwa-master
d9d59fe [Matthew Farrellee] Fix scalastyle errors
67473a9 [giwa] delete not implemented functions
c97377c [giwa] delete inproper comments
2ea769e [giwa] added comment in dstream._test_output
3b27bd4 [giwa] remove the last brank line
acfcaeb [giwa] revert pom.xml
93f7637 [giwa] fixed explanaiton
50fd6f9 [giwa] revert pom.xml
4f82c89 [giwa] remove duplicated import
9d1de23 [giwa] revert pom.xml
7339df2 [giwa] fixed typo
9c85e48 [giwa] clean up exmples
24f95db [giwa] clen up examples
0d30109 [giwa] fixed pep8 violation
b7dab85 [giwa] improve test case
583e66d [giwa] move tests for streaming inside streaming directory
1d84142 [giwa] remove unimplement test
f0ea311 [giwa] clean up code
171edeb [giwa] clean up
4dedd2d [giwa] change test case not to use awaitTermination
268a6a5 [giwa] Changed awaitTermination not to call awaitTermincation in Scala. Just use time.sleep instread
09a28bf [giwa] improve testcases
58150f5 [giwa] Changed the test case to focus the test operation
199e37f [giwa] adopted the latest compression way of python command
185fdbf [giwa] merge with master
f1798c4 [giwa] merge with master
e70f706 [giwa] added testcase for combineByKey
e162822 [giwa] added gorupByKey testcase
97742fe [giwa] added sparkContext as input parameter in StreamingContext
14d4c0e [giwa] removed wasted print in DStream
6d8190a [giwa] add comments
4aa99e4 [giwa] added TODO coments
e9fab72 [giwa] added saveAsTextFiles and saveAsPickledFiles
94f2b65 [giwa] remove waste duplicated code
580fbc2 [giwa] modified streaming test case to add coment
99e4bb3 [giwa] basic function test cases are passed
7051a84 [giwa] all tests are passed if numSlice is 2 and the numver of each input is over 4
35933e1 [giwa] broke something
9767712 [giwa] WIP: solved partitioned and None is not recognized
4f2d7e6 [giwa] added mapValues and flatMapVaules WIP for glom and mapPartitions test
33c0f94d [giwa] edited the comment to add more precise description
774f18d [giwa] removed unnesessary changes
3a671cc [giwa] remove export PYSPARK_PYTHON in spark submit
8efa266 [giwa] fixed PEP-008 violation
fa75d71 [giwa] delete waste file
7f96294 [giwa] added basic operation test cases
3dda31a [giwa] WIP added test case
1f68b78 [giwa] WIP
c05922c [giwa] WIP: added PythonTestInputStream
1fd12ae [giwa] WIP
c880a33 [giwa] update comment
5d22c92 [giwa] WIP
ea4b06b [giwa] initial commit for testcase
5a9b525 [giwa] clean up dstream.py
79c5809 [giwa] added stop in StreamingContext
189dcea [giwa] clean up examples
b8d7d24 [giwa] implemented reduce and count function in Dstream
b6468e6 [giwa] Removed the waste line
b47b5fd [Ken Takagiwa] Kill py4j callback server properly
19ddcdd [Ken Takagiwa] tried to restart callback server
c9fc124 [Tathagata Das] Added extra line.
4caae3f [Tathagata Das] Added missing file
4eff053 [Tathagata Das] Implemented DStream.foreachRDD in the Python API using Py4J callback server.
5e822d4 [Ken Takagiwa] remove waste file
aeaf8a5 [Ken Takagiwa] clean up codes
9fa249b [Ken Takagiwa] clean up code
05459c6 [Ken Takagiwa] fix map function
a9f4ecb [Ken Takagiwa] added count operation but this implementation need double check
d1ee6ca [Ken Takagiwa] edit python sparkstreaming example
0b8b7d0 [Ken Takagiwa] reduceByKey is working
d25d5cf [Ken Takagiwa] added reducedByKey not working yet
7f7c5d1 [Ken Takagiwa] delete old file
967dc26 [Ken Takagiwa] fixed typo of network_workdcount.py
57fb740 [Ken Takagiwa] added doctest for pyspark.streaming.duration
4b69fb1 [Ken Takagiwa] fied input of socketTextDStream
02f618a [Ken Takagiwa] initial commit for socketTextStream
4ce4058 [Ken Takagiwa] remove unused import in python
856d98e [Ken Takagiwa] add empty line
490e338 [Ken Takagiwa] sorted the import following Spark coding convention
5594bd4 [Ken Takagiwa] revert pom.xml
2adca84 [Ken Takagiwa] remove not implemented DStream functions in python
e551e13 [Ken Takagiwa] add coment for hack why PYSPARK_PYTHON is needed in spark-submit
3758175 [Ken Takagiwa] add coment for hack why PYSPARK_PYTHON is needed in spark-submit
c5518b4 [Ken Takagiwa] modified the code base on comment in https://github.com/tdas/spark/pull/10
dcf243f [Ken Takagiwa] implementing transform function in Python
9af03f4 [Ken Takagiwa] added reducedByKey not working yet
6e0d9c7 [Ken Takagiwa] modify dstream.py to fix indent error
e497b9b [Ken Takagiwa] comment PythonDStream.PairwiseDStream
5c3a683 [Ken] initial commit for pySparkStreaming
665bfdb [giwa] added testcase for combineByKey
a3d2379 [giwa] added gorupByKey testcase
636090a [giwa] added sparkContext as input parameter in StreamingContext
e7ebb08 [giwa] removed wasted print in DStream
d8b593b [giwa] add comments
ea9c873 [giwa] added TODO coments
89ae38a [giwa] added saveAsTextFiles and saveAsPickledFiles
e3033fc [giwa] remove waste duplicated code
a14c7e1 [giwa] modified streaming test case to add coment
536def4 [giwa] basic function test cases are passed
2112638 [giwa] all tests are passed if numSlice is 2 and the numver of each input is over 4
080541a [giwa] broke something
0704b86 [giwa] WIP: solved partitioned and None is not recognized
90a6484 [giwa] added mapValues and flatMapVaules WIP for glom and mapPartitions test
a65f302 [giwa] edited the comment to add more precise description
bdde697 [giwa] removed unnesessary changes
e8c7bfc [giwa] remove export PYSPARK_PYTHON in spark submit
3334169 [giwa] fixed PEP-008 violation
db0a303 [giwa] delete waste file
2cfd3a0 [giwa] added basic operation test cases
90ae568 [giwa] WIP added test case
a120d07 [giwa] WIP
f671cdb [giwa] WIP: added PythonTestInputStream
56fae45 [giwa] WIP
e35e101 [giwa] Merge branch 'master' into testcase
ba5112d [giwa] update comment
28aa56d [giwa] WIP
fb08559 [giwa] initial commit for testcase
a613b85 [giwa] clean up dstream.py
c40c0ef [giwa] added stop in StreamingContext
31e4260 [giwa] clean up examples
d2127d6 [giwa] implemented reduce and count function in Dstream
48f7746 [giwa] Removed the waste line
0f83eaa [Ken Takagiwa] delete py4j 0.8.1
1679808 [Ken Takagiwa] Kill py4j callback server properly
f96cd4e [Ken Takagiwa] tried to restart callback server
fe86198 [Ken Takagiwa] add py4j 0.8.2.1 but server is not launched
1064fe0 [Ken Takagiwa] Merge branch 'master' of https://github.com/giwa/spark
28c6620 [Ken Takagiwa] Implemented DStream.foreachRDD in the Python API using Py4J callback server
85b0fe1 [Ken Takagiwa] Merge pull request #1 from tdas/python-foreach
54e2e8c [Tathagata Das] Added extra line.
e185338 [Tathagata Das] Added missing file
a778d4b [Tathagata Das] Implemented DStream.foreachRDD in the Python API using Py4J callback server.
cc2092b [Ken Takagiwa] remove waste file
d042ac6 [Ken Takagiwa] clean up codes
84a021f [Ken Takagiwa] clean up code
bd20e17 [Ken Takagiwa] fix map function
d01a125 [Ken Takagiwa] added count operation but this implementation need double check
7d05109 [Ken Takagiwa] merge with remote branch
ae464e0 [Ken Takagiwa] edit python sparkstreaming example
04af046 [Ken Takagiwa] reduceByKey is working
3b6d7b0 [Ken Takagiwa] implementing transform function in Python
571d52d [Ken Takagiwa] added reducedByKey not working yet
5720979 [Ken Takagiwa] delete old file
e604fcb [Ken Takagiwa] fixed typo of network_workdcount.py
4b7c08b [Ken Takagiwa] Merge branch 'master' of https://github.com/giwa/spark
ce7d426 [Ken Takagiwa] added doctest for pyspark.streaming.duration
a8c9fd5 [Ken Takagiwa] fixed for socketTextStream
a61fa9e [Ken Takagiwa] fied input of socketTextDStream
1e84f41 [Ken Takagiwa] initial commit for socketTextStream
6d012f7 [Ken Takagiwa] remove unused import in python
25d30d5 [Ken Takagiwa] add empty line
6e0a64a [Ken Takagiwa] sorted the import following Spark coding convention
fa4a7fc [Ken Takagiwa] revert streaming/pom.xml
8f8202b [Ken Takagiwa] revert streaming pom.xml
c9d79dd [Ken Takagiwa] revert pom.xml
57e3e52 [Ken Takagiwa] remove not implemented DStream functions in python
0a516f5 [Ken Takagiwa] add coment for hack why PYSPARK_PYTHON is needed in spark-submit
a7a0b5c [Ken Takagiwa] add coment for hack why PYSPARK_PYTHON is needed in spark-submit
72bfc66 [Ken Takagiwa] modified the code base on comment in https://github.com/tdas/spark/pull/10
69e9cd3 [Ken Takagiwa] implementing transform function in Python
94a0787 [Ken Takagiwa] added reducedByKey not working yet
88068cf [Ken Takagiwa] modify dstream.py to fix indent error
1367be5 [Ken Takagiwa] comment PythonDStream.PairwiseDStream
eb2b3ba [Ken] Merge remote-tracking branch 'upstream/master'
d8e51f9 [Ken] initial commit for pySparkStreaming
2014-10-12 02:46:56 -07:00
Davies Liu 655032965f [SPARK-3762] clear reference of SparkEnv after stop
SparkEnv is cached in ThreadLocal object, so after stop and create a new SparkContext, old SparkEnv is still used by some threads, it will trigger many problems, for example, pyspark will have problem after restart SparkContext, because py4j use thread pool for RPC.

This patch will clear all the references after stop a SparkEnv.

cc mateiz tdas pwendell

Author: Davies Liu <davies.liu@gmail.com>

Closes #2624 from davies/env and squashes the following commits:

a69f30c [Davies Liu] deprecate getThreadLocal
ba77ca4 [Davies Liu] remove getThreadLocal(), update docs
ee62bb7 [Davies Liu] cleanup ThreadLocal of SparnENV
4d0ea8b [Davies Liu] clear reference of SparkEnv after stop
2014-10-07 12:06:12 -07:00
Sean Owen c3f2a8588e SPARK-2932 [STREAMING] Move MasterFailureTest out of "main" source directory
(HT @vanzin) Whatever the reason was for having this test class in `main`, if there is one, appear to be moot. This may have been a result of earlier streaming test reorganization.

This simply puts `MasterFailureTest` back under `test/`, removes some redundant copied code, and touches up a few tiny inspection warnings along the way.

Author: Sean Owen <sowen@cloudera.com>

Closes #2399 from srowen/SPARK-2932 and squashes the following commits:

3909411 [Sean Owen] Move MasterFailureTest to src/test, and remove redundant TestOutputStream
2014-09-25 23:20:17 +05:30
Aaron Staple 8ca4ecb6a5 [SPARK-546] Add full outer join to RDD and DStream.
leftOuterJoin and rightOuterJoin are already implemented.  This patch adds fullOuterJoin.

Author: Aaron Staple <aaron.staple@gmail.com>

Closes #1395 from staple/SPARK-546 and squashes the following commits:

1f5595c [Aaron Staple] Fix python style
7ac0aa9 [Aaron Staple] [SPARK-546] Add full outer join to RDD and DStream.
3b5d137 [Aaron Staple] In JavaPairDStream, make class tag specification in rightOuterJoin consistent with other functions.
31f2956 [Aaron Staple] Fix left outer join documentation comments.
2014-09-24 20:39:09 -07:00
Mubarak Seyed 729952a5ef [SPARK-1853] Show Streaming application code context (file, line number) in Spark Stages UI
This is a refactored version of the original PR https://github.com/apache/spark/pull/1723 my mubarak

Please take a look andrewor14, mubarak

Author: Mubarak Seyed <mubarak.seyed@gmail.com>
Author: Tathagata Das <tathagata.das1565@gmail.com>

Closes #2464 from tdas/streaming-callsite and squashes the following commits:

dc54c71 [Tathagata Das] Made changes based on PR comments.
390b45d [Tathagata Das] Fixed minor bugs.
904cd92 [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into streaming-callsite
7baa427 [Tathagata Das] Refactored getCallSite and setCallSite to make it simpler. Also added unit test for DStream creation site.
b9ed945 [Mubarak Seyed] Adding streaming utils
c461cf4 [Mubarak Seyed] Merge remote-tracking branch 'upstream/master'
ceb43da [Mubarak Seyed] Changing default regex function name
8c5d443 [Mubarak Seyed] Merge remote-tracking branch 'upstream/master'
196121b [Mubarak Seyed] Merge remote-tracking branch 'upstream/master'
491a1eb [Mubarak Seyed] Removing streaming visibility from getRDDCreationCallSite in DStream
33a7295 [Mubarak Seyed] Fixing review comments: Merging both setCallSite methods
c26d933 [Mubarak Seyed] Merge remote-tracking branch 'upstream/master'
f51fd9f [Mubarak Seyed] Fixing scalastyle, Regex for Utils.getCallSite, and changing method names in DStream
5051c58 [Mubarak Seyed] Getting return value of compute() into variable and call setCallSite(prevCallSite) only once. Adding return for other code paths (for None)
a207eb7 [Mubarak Seyed] Fixing code review comments
ccde038 [Mubarak Seyed] Removing Utils import from MappedDStream
2a09ad6 [Mubarak Seyed] Changes in Utils.scala for SPARK-1853
1d90cc3 [Mubarak Seyed] Changes for SPARK-1853
5f3105a [Mubarak Seyed] Merge remote-tracking branch 'upstream/master'
70f494f [Mubarak Seyed] Changes for SPARK-1853
1500deb [Mubarak Seyed] Changes in Spark Streaming UI
9d38d3c [Mubarak Seyed] [SPARK-1853] Show Streaming application code context (file, line number) in Spark Stages UI
d466d75 [Mubarak Seyed] Changes for spark streaming UI
2014-09-23 15:09:12 -07:00
Sean Owen e73b48ace0 SPARK-2745 [STREAMING] Add Java friendly methods to Duration class
tdas is this what you had in mind for this JIRA? I saw this one and thought it would be easy to take care of, and helpful as I use streaming from Java.

I could do the same for `Time`? Happy to do so.

Author: Sean Owen <sowen@cloudera.com>

Closes #2403 from srowen/SPARK-2745 and squashes the following commits:

5a9e706 [Sean Owen] Change "Duration" to "Durations" to avoid changing Duration case class API
bda301c [Sean Owen] Just delegate to Scala binary operator syntax to avoid scalastyle warning
7dde949 [Sean Owen] Disable scalastyle for false positives. Add Java static factory methods seconds(), minutes() to Duration. Add Java-friendly methods to Time too, and unit tests. Remove unnecessary math.floor from Time.floor()
4dee32e [Sean Owen] Add named methods to Duration in parallel to symbolic methods for Java-friendliness. Also add unit tests for Duration, in Scala and Java.
2014-09-23 11:58:05 -07:00
Sean Owen feaa3706f1 SPARK-3470 [CORE] [STREAMING] Add Closeable / close() to Java context objects
...  that expose a stop() lifecycle method. This doesn't add `AutoCloseable`, which is Java 7+ only. But it should be possible to use try-with-resources on a `Closeable` in Java 7, as long as the `close()` does not throw a checked exception, and these don't. Q.E.D.

Author: Sean Owen <sowen@cloudera.com>

Closes #2346 from srowen/SPARK-3470 and squashes the following commits:

612c21d [Sean Owen] Add Closeable / close() to Java context objects that expose a stop() lifecycle method
2014-09-12 22:50:37 -07:00
Andrew Or 6324eb7b5b [Spark-3490] Disable SparkUI for tests
We currently open many ephemeral ports during the tests, and as a result we occasionally can't bind to new ones. This has caused the `DriverSuite` and the `SparkSubmitSuite` to fail intermittently.

By disabling the `SparkUI` when it's not needed, we already cut down on the number of ports opened significantly, on the order of the number of `SparkContexts` ever created. We must keep it enabled for a few tests for the UI itself, however.

Author: Andrew Or <andrewor14@gmail.com>

Closes #2363 from andrewor14/disable-ui-for-tests and squashes the following commits:

332a7d5 [Andrew Or] No need to set spark.ui.port to 0 anymore
30c93a2 [Andrew Or] Simplify streaming UISuite
a431b84 [Andrew Or] Fix streaming test failures
8f5ae53 [Andrew Or] Fix no new line at the end
29c9b5b [Andrew Or] Disable SparkUI for tests
2014-09-11 17:18:46 -07:00
Mario Pastorelli c419e4f1bd [Docs] actorStream storageLevel default is MEMORY_AND_DISK_SER_2
Comment of the storageLevel param of actorStream says that it defaults to memory-only while the default is MEMORY_AND_DISK_SER_2.

Author: Mario Pastorelli <pastorelli.mario@gmail.com>

Closes #2319 from melrief/master and squashes the following commits:

7b6ce68 [Mario Pastorelli] [Docs] actorStream storageLevel default is MEMORY_AND_DISK_SER_2
2014-09-09 00:51:28 -07:00
Andrew Or b21ae5bbb9 [SPARK-2886] Use more specific actor system name than "spark"
As of #1777 we log the name of the actor system when it binds to a port. The current name "spark" is super general and does not convey any meaning. For instance, the following line is taken from my driver log after setting `spark.driver.port` to 5001.
```
14/08/13 19:33:29 INFO Remoting: Remoting started; listening on addresses:
[akka.tcp://sparkandrews-mbp:5001]
14/08/13 19:33:29 INFO Remoting: Remoting now listens on addresses:
[akka.tcp://sparkandrews-mbp:5001]
14/08/06 13:40:05 INFO Utils: Successfully started service 'spark' on port 5001.
```
This commit renames this to "sparkDriver" and "sparkExecutor". The goal of this unambitious PR is simply to make the logged information more explicit without introducing any change in functionality.

Author: Andrew Or <andrewor14@gmail.com>

Closes #1810 from andrewor14/service-name and squashes the following commits:

8c459ed [Andrew Or] Use a common variable for driver/executor actor system names
3a92843 [Andrew Or] Change actor name to sparkDriver and sparkExecutor
921363e [Andrew Or] Merge branch 'master' of github.com:apache/spark into service-name
c8c6a62 [Andrew Or] Do not include hyphens in actor name
1c1b42e [Andrew Or] Avoid spaces in akka system name
f644b55 [Andrew Or] Use more specific service name
2014-08-25 23:36:09 -07:00
Marcelo Vanzin 66ade00f91 [SPARK-2169] Don't copy appName / basePath everywhere.
Instead of keeping copies in all pages, just reference the values
kept in the base SparkUI instance (by making them available via
getters).

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #1252 from vanzin/SPARK-2169 and squashes the following commits:

4412fc6 [Marcelo Vanzin] Simplify UIUtils.headerSparkPage signature.
4e5d35a [Marcelo Vanzin] [SPARK-2169] Don't copy appName / basePath everywhere.
2014-08-18 13:25:30 -07:00
Anand Avati 7589c39d39 [SPARK-2924] remove default args to overloaded methods
Not supported in Scala 2.11. Split them into separate methods instead.

Author: Anand Avati <avati@redhat.com>

Closes #1704 from avati/SPARK-1812-default-args and squashes the following commits:

3e3924a [Anand Avati] SPARK-1812: Add Mima excludes for the broken ABI
901dfc7 [Anand Avati] SPARK-1812: core - Fix overloaded methods with default arguments
07f00af [Anand Avati] SPARK-1812: streaming - Fix overloaded methods with default arguments
2014-08-15 08:53:52 -07:00
Reynold Xin eaeb0f76fa Minor cleanup of metrics.Source
- Added override.
- Marked some variables as private.

Author: Reynold Xin <rxin@apache.org>

Closes #1943 from rxin/metricsSource and squashes the following commits:

fbfa943 [Reynold Xin] Minor cleanup of metrics.Source. - Added override. - Marked some variables as private.
2014-08-14 11:22:41 -07:00
Andrew Or 148af6082c [SPARK-2454] Do not ship spark home to Workers
When standalone Workers launch executors, they inherit the Spark home set by the driver. This means if the worker machines do not share the same directory structure as the driver node, the Workers will attempt to run scripts (e.g. bin/compute-classpath.sh) that do not exist locally and fail. This is a common scenario if the driver is launched from outside of the cluster.

The solution is to simply not pass the driver's Spark home to the Workers. This PR further makes an attempt to avoid overloading the usages of `spark.home`, which is now only used for setting executor Spark home on Mesos and in python.

This is based on top of #1392 and originally reported by YanTangZhai. Tested on standalone cluster.

Author: Andrew Or <andrewor14@gmail.com>

Closes #1734 from andrewor14/spark-home-reprise and squashes the following commits:

f71f391 [Andrew Or] Revert changes in python
1c2532c [Andrew Or] Merge branch 'master' of github.com:apache/spark into spark-home-reprise
188fc5d [Andrew Or] Avoid using spark.home where possible
09272b7 [Andrew Or] Always use Worker's working directory as spark home
2014-08-02 00:45:38 -07:00
joyyoj b270309d76 [SPARK-2379] Fix the bug that streaming's receiver may fall into a dead loop
Author: joyyoj <sunshch@gmail.com>

Closes #1694 from joyyoj/SPARK-2379 and squashes the following commits:

d73790d [joyyoj] SPARK-2379 Fix the bug that streaming's receiver may fall into a dead loop
22e7821 [joyyoj] Merge remote-tracking branch 'apache/master'
3f4a602 [joyyoj] Merge remote-tracking branch 'remotes/apache/master'
f4660c5 [joyyoj] [SPARK-1998] SparkFlumeEvent with body bigger than 1020 bytes are not read properly
2014-08-01 13:41:55 -07:00
Andrew Or ecf30ee7e7 [SPARK-1777] Prevent OOMs from single partitions
**Problem.** When caching, we currently unroll the entire RDD partition before making sure we have enough free memory. This is a common cause for OOMs especially when (1) the BlockManager has little free space left in memory, and (2) the partition is large.

**Solution.** We maintain a global memory pool of `M` bytes shared across all threads, similar to the way we currently manage memory for shuffle aggregation. Then, while we unroll each partition, periodically check if there is enough space to continue. If not, drop enough RDD blocks to ensure we have at least `M` bytes to work with, then try again. If we still don't have enough space to unroll the partition, give up and drop the block to disk directly if applicable.

**New configurations.**
- `spark.storage.bufferFraction` - the value of `M` as a fraction of the storage memory. (default: 0.2)
- `spark.storage.safetyFraction` - a margin of safety in case size estimation is slightly off. This is the equivalent of the existing `spark.shuffle.safetyFraction`. (default 0.9)

For more detail, see the [design document](https://issues.apache.org/jira/secure/attachment/12651793/spark-1777-design-doc.pdf). Tests pending for performance and memory usage patterns.

Author: Andrew Or <andrewor14@gmail.com>

Closes #1165 from andrewor14/them-rdd-memories and squashes the following commits:

e77f451 [Andrew Or] Merge branch 'master' of github.com:apache/spark into them-rdd-memories
c7c8832 [Andrew Or] Simplify logic + update a few comments
269d07b [Andrew Or] Very minor changes to tests
6645a8a [Andrew Or] Merge branch 'master' of github.com:apache/spark into them-rdd-memories
b7e165c [Andrew Or] Add new tests for unrolling blocks
f12916d [Andrew Or] Slightly clean up tests
71672a7 [Andrew Or] Update unrollSafely tests
369ad07 [Andrew Or] Correct ensureFreeSpace and requestMemory behavior
f4d035c [Andrew Or] Allow one thread to unroll multiple blocks
a66fbd2 [Andrew Or] Rename a few things + update comments
68730b3 [Andrew Or] Fix weird scalatest behavior
e40c60d [Andrew Or] Fix MIMA excludes
ff77aa1 [Andrew Or] Fix tests
1a43c06 [Andrew Or] Merge branch 'master' of github.com:apache/spark into them-rdd-memories
b9a6eee [Andrew Or] Simplify locking behavior on unrollMemoryMap
ed6cda4 [Andrew Or] Formatting fix (super minor)
f9ff82e [Andrew Or] putValues -> putIterator + putArray
beb368f [Andrew Or] Merge branch 'master' of github.com:apache/spark into them-rdd-memories
8448c9b [Andrew Or] Fix tests
a49ba4d [Andrew Or] Do not expose unroll memory check period
69bc0a5 [Andrew Or] Always synchronize on putLock before unrollMemoryMap
3f5a083 [Andrew Or] Simplify signature of ensureFreeSpace
dce55c8 [Andrew Or] Merge branch 'master' of github.com:apache/spark into them-rdd-memories
8288228 [Andrew Or] Synchronize put and unroll properly
4f18a3d [Andrew Or] bufferFraction -> unrollFraction
28edfa3 [Andrew Or] Update a few comments / log messages
728323b [Andrew Or] Do not synchronize every 1000 elements
5ab2329 [Andrew Or] Merge branch 'master' of github.com:apache/spark into them-rdd-memories
129c441 [Andrew Or] Fix bug: Use toArray rather than array
9a65245 [Andrew Or] Update a few comments + minor control flow changes
57f8d85 [Andrew Or] Merge branch 'master' of github.com:apache/spark into them-rdd-memories
abeae4f [Andrew Or] Add comment clarifying the MEMORY_AND_DISK case
3dd96aa [Andrew Or] AppendOnlyBuffer -> Vector (+ a few small changes)
f920531 [Andrew Or] Merge branch 'master' of github.com:apache/spark into them-rdd-memories
0871835 [Andrew Or] Add an effective storage level interface to BlockManager
64e7d4c [Andrew Or] Add/modify a few comments (minor)
8af2f35 [Andrew Or] Merge branch 'master' of github.com:apache/spark into them-rdd-memories
4f4834e [Andrew Or] Use original storage level for blocks dropped to disk
ecc8c2d [Andrew Or] Fix binary incompatibility
24185ea [Andrew Or] Avoid dropping a block back to disk if reading from disk
2b7ee66 [Andrew Or] Fix bug in SizeTracking*
9b9a273 [Andrew Or] Fix tests
20eb3e5 [Andrew Or] Merge branch 'master' of github.com:apache/spark into them-rdd-memories
649bdb3 [Andrew Or] Document spark.storage.bufferFraction
a10b0e7 [Andrew Or] Add initial memory request threshold + rename a few things
e9c3cb0 [Andrew Or] cacheMemoryMap -> unrollMemoryMap
198e374 [Andrew Or] Unfold -> unroll
0d50155 [Andrew Or] Merge branch 'master' of github.com:apache/spark into them-rdd-memories
d9d02a8 [Andrew Or] Remove unused param in unfoldSafely
ec728d8 [Andrew Or] Add tests for safe unfolding of blocks
22b2209 [Andrew Or] Merge branch 'master' of github.com:apache/spark into them-rdd-memories
078eb83 [Andrew Or] Add check for hasNext in PrimitiveVector.iterator
0871535 [Andrew Or] Fix tests in BlockManagerSuite
d68f31e [Andrew Or] Safely unfold blocks for all memory puts
5961f50 [Andrew Or] Fix tests
195abd7 [Andrew Or] Refactor: move unfold logic to MemoryStore
1e82d00 [Andrew Or] Merge branch 'master' of github.com:apache/spark into them-rdd-memories
3ce413e [Andrew Or] Merge branch 'master' of github.com:apache/spark into them-rdd-memories
d5dd3b4 [Andrew Or] Free buffer memory in finally
ea02eec [Andrew Or] Fix tests
b8e1d9c [Andrew Or] Merge branch 'master' of github.com:apache/spark into them-rdd-memories
a8704c1 [Andrew Or] Merge branch 'master' of github.com:apache/spark into them-rdd-memories
e1b8b25 [Andrew Or] Merge branch 'master' of github.com:apache/spark into them-rdd-memories
87aa75c [Andrew Or] Fix mima excludes again (typo)
11eb921 [Andrew Or] Clarify comment (minor)
50cae44 [Andrew Or] Remove now duplicate mima exclude
7de5ef9 [Andrew Or] Merge branch 'master' of github.com:apache/spark into them-rdd-memories
df47265 [Andrew Or] Fix binary incompatibility
6d05a81 [Andrew Or] Merge branch 'master' of github.com:apache/spark into them-rdd-memories
f94f5af [Andrew Or] Update a few comments (minor)
776aec9 [Andrew Or] Prevent OOM if a single RDD partition is too large
bbd3eea [Andrew Or] Fix CacheManagerSuite to use Array
97ea499 [Andrew Or] Change BlockManager interface to use Arrays
c12f093 [Andrew Or] Add SizeTrackingAppendOnlyBuffer and tests
2014-07-27 16:08:16 -07:00
fireflyc a2715ccd94 replace println to log4j
Our program needs to receive a large amount of data and run for a long
time.
We set the log level to WARN but "Storing iterator" "received single"
as such message written to the log file. (over yarn)

Author: fireflyc <fireflyc@126.com>

Closes #1372 from fireflyc/fix-replace-stdout-log and squashes the following commits:

e684140 [fireflyc] 'info' modified into the 'debug'
fa22a38 [fireflyc] replace println to log4j
2014-07-25 10:47:52 -07:00
Matei Zaharia 8529ced35c SPARK-2657 Use more compact data structures than ArrayBuffer in groupBy & cogroup
JIRA: https://issues.apache.org/jira/browse/SPARK-2657

Our current code uses ArrayBuffers for each group of values in groupBy, as well as for the key's elements in CoGroupedRDD. ArrayBuffers have a lot of overhead if there are few values in them, which is likely to happen in cases such as join. In particular, they have a pointer to an Object[] of size 16 by default, which is 24 bytes for the array header + 128 for the pointers in there, plus at least 32 for the ArrayBuffer data structure. This patch replaces the per-group buffers with a CompactBuffer class that can store up to 2 elements more efficiently (in fields of itself) and acts like an ArrayBuffer beyond that. For a key's elements in CoGroupedRDD, we use an Array of CompactBuffers instead of an ArrayBuffer of ArrayBuffers.

There are some changes throughout the code to deal with CoGroupedRDD returning Array instead. We can also decide not to do that but CoGroupedRDD is a `DeveloperAPI` so I think it's okay to change it here.

Author: Matei Zaharia <matei@databricks.com>

Closes #1555 from mateiz/compact-groupby and squashes the following commits:

845a356 [Matei Zaharia] Lower initial size of CompactBuffer's vector to 8
07621a7 [Matei Zaharia] Review comments
0c1cd12 [Matei Zaharia] Don't use varargs in CompactBuffer.apply
bdc8a39 [Matei Zaharia] Small tweak to +=, and typos
f61f040 [Matei Zaharia] Fix line lengths
59da88b0 [Matei Zaharia] Fix line lengths
197cde8 [Matei Zaharia] Make CompactBuffer extend Seq to make its toSeq more efficient
775110f [Matei Zaharia] Change CoGroupedRDD to give (K, Array[Iterable[_]]) to avoid wrappers
9b4c6e8 [Matei Zaharia] Use CompactBuffer in CoGroupedRDD
ed577ab [Matei Zaharia] Use CompactBuffer in groupByKey
10f0de1 [Matei Zaharia] A CompactBuffer that's more memory-efficient than ArrayBuffer for small buffers
2014-07-25 00:32:32 -07:00
James Z.M. Gao caa163f086 fix compile error of streaming project
explicit return type for implicit function

Author: James Z.M. Gao <gaozhm@mediav.com>

Closes #153 from gzm55/work/streaming-compile and squashes the following commits:

11e9c8d [James Z.M. Gao] fix style error
fe88109 [James Z.M. Gao] fix compile error of streaming project
2014-07-16 11:35:21 -07:00