In Spark 1.2 we added a `binaryRecords` input method for loading flat binary data. This format is useful for numerical array data, e.g. in scientific computing applications. This PR adds support for the same format in Streaming applications, where it is similarly useful, especially for streaming time series or sensor data.
Summary of additions
- adding `binaryRecordsStream` to Spark Streaming
- exposing `binaryRecordsStream` in the new PySpark Streaming
- new unit tests in Scala and Python
This required adding an optional Hadoop configuration param to `fileStream` and `FileInputStream`, but was otherwise straightforward.
tdas davies
Author: freeman <the.freeman.lab@gmail.com>
Closes#3803 from freeman-lab/streaming-binary-records and squashes the following commits:
b676534 [freeman] Clarify note
5ff1b75 [freeman] Add note to java streaming context
eba925c [freeman] Simplify notes
c4237b8 [freeman] Add experimental tag
30eba67 [freeman] Add filter and newFilesOnly alongside conf
c2cfa6d [freeman] Expose new version of fileStream with conf in java
34d20ef [freeman] Add experimental tag
14bca9a [freeman] Add experimental tag
b85bffc [freeman] Formatting
47560f4 [freeman] Space formatting
9a3715a [freeman] Refactor to reflect changes to FileInputSuite
7373f73 [freeman] Add note and defensive assertion for byte length
3ceb684 [freeman] Merge remote-tracking branch 'upstream/master' into streaming-binary-records
317b6d1 [freeman] Make test inline
fcb915c [freeman] Formatting
becb344 [freeman] Formatting
d3e75b2 [freeman] Add tests in python
a4324a3 [freeman] Line length
029d49c [freeman] Formatting
1c739aa [freeman] Simpler default arg handling
94d90d0 [freeman] Spelling
2843e9d [freeman] Add params to docstring
8b70fbc [freeman] Reorganization
28bff9b [freeman] Fix missing arg
9398bcb [freeman] Expose optional hadoop configuration
23dd69f [freeman] Tests for binaryRecordsStream
36cb0fd [freeman] Add binaryRecordsStream to scala
fe4e803 [freeman] Add binaryRecordStream to Java API
ecef0eb [freeman] Add binaryRecordsStream to python
8550c26 [freeman] Expose additional argument combination
A slow receiver might not have enough time to shutdown cleanly even when graceful shutdown is used. This PR extends graceful waiting to make sure all receivers have deregistered and that the receiver job has terminated.
Author: Jesper Lundgren <jesper.lundgren@vpon.com>
Closes#4338 from cleaton/stopreceivers and squashes the following commits:
a9cf223 [Jesper Lundgren] remove cleaner.ttl config
f969b6e [Jesper Lundgren] fix inversed logic in unit test
3d0bd35 [Jesper Lundgren] switch boleans to match running status instead of terminated
9a9ff88 [Jesper Lundgren] wait for receivers to shutdown and receiver job to terminate
d179372 [Jesper Lundgren] Add graceful shutdown unit test covering slow receiver onStop
SPARK-3883: SSL support for Akka connections and Jetty based file servers.
This story introduced the following changes:
- Introduced SSLOptions object which holds the SSL configuration and can build the appropriate configuration for Akka or Jetty. SSLOptions can be created by parsing SparkConf entries at a specified namespace.
- SSLOptions is created and kept by SecurityManager
- All Akka actor address creation snippets based on interpolated strings were replaced by a dedicated methods from AkkaUtils. Those methods select the proper Akka protocol - whether akka.tcp or akka.ssl.tcp
- Added tests cases for AkkaUtils, FileServer, SSLOptions and SecurityManager
- Added a way to use node local SSL configuration by executors and driver in standalone mode. It can be done by specifying spark.ssl.useNodeLocalConf in SparkConf.
- Made CoarseGrainedExecutorBackend not overwrite the settings which are executor startup configuration - they are passed anyway from Worker
Refer to https://github.com/apache/spark/pull/3571 for discussion and details
Author: Jacek Lewandowski <lewandowski.jacek@gmail.com>
Author: Jacek Lewandowski <jacek.lewandowski@datastax.com>
Closes#3571 from jacek-lewandowski/SPARK-3883-master and squashes the following commits:
9ef4ed1 [Jacek Lewandowski] Merge pull request #2 from jacek-lewandowski/SPARK-3883-docs2
fb31b49 [Jacek Lewandowski] SPARK-3883: Added SSL setup documentation
2532668 [Jacek Lewandowski] SPARK-3883: Refactored AkkaUtils.protocol method to not use Try
90a8762 [Jacek Lewandowski] SPARK-3883: Refactored methods to resolve Akka address and made it possible to easily configure multiple communication layers for SSL
72b2541 [Jacek Lewandowski] SPARK-3883: A reference to the fallback SSLOptions can be provided when constructing SSLOptions
93050f4 [Jacek Lewandowski] SPARK-3883: SSL support for HttpServer and Akka
This PR refactors LiveListenerBus and StreamingListenerBus and extracts the common codes to a parent class `ListenerBus`.
It also includes bug fixes in #3710:
1. Fix the race condition of queueFullErrorMessageLogged in LiveListenerBus and StreamingListenerBus to avoid outputing `queue-full-error` logs multiple times.
2. Make sure the SHUTDOWN message will be delivered to listenerThread, so that we can make sure listenerThread will always be able to exit.
3. Log the error from listener rather than crashing listenerThread in StreamingListenerBus.
During fixing the above bugs, we find it's better to make LiveListenerBus and StreamingListenerBus have the same bahaviors. Then there will be many duplicated codes in LiveListenerBus and StreamingListenerBus.
Therefore, I extracted their common codes to `ListenerBus` as a parent class: LiveListenerBus and StreamingListenerBus only need to extend `ListenerBus` and implement `onPostEvent` (how to process an event) and `onDropEvent` (do something when droppping an event).
Author: zsxwing <zsxwing@gmail.com>
Closes#4006 from zsxwing/SPARK-4859-refactor and squashes the following commits:
c8dade2 [zsxwing] Fix the code style after renaming
5715061 [zsxwing] Rename ListenerHelper to ListenerBus and the original ListenerBus to AsynchronousListenerBus
f0ef647 [zsxwing] Fix the code style
4e85ffc [zsxwing] Merge branch 'master' into SPARK-4859-refactor
d2ef990 [zsxwing] Add private[spark]
4539f91 [zsxwing] Remove final to pass MiMa tests
a9dccd3 [zsxwing] Remove SparkListenerShutdown
7cc04c3 [zsxwing] Refactor LiveListenerBus and StreamingListenerBus and make them share same code base
`reduceByWindow` for Java API is actually not Java compatible, change to make it Java compatible.
Current solution is to deprecate the old one and add a new API, but since old API actually is not correct, so is keeping the old one meaningful? just to keep the binary compatible? Also even adding new API still need to add to Mima exclusion, I'm not sure to change the API, or deprecate the old API and add a new one, which is the best solution?
Author: jerryshao <saisai.shao@intel.com>
Closes#4104 from jerryshao/SPARK-5315 and squashes the following commits:
5bc8987 [jerryshao] Address the comment
c7aa1b4 [jerryshao] Deprecate the old one to keep binary compatible
8e9dc67 [jerryshao] Fix JavaDStream reduceByWindow signature error
Because of lacking of `BlockAllocationEvent` in WAL recovery, the dangled event will mix into the new batch, which will lead to the wrong result. Details can be seen in [SPARK-5233](https://issues.apache.org/jira/browse/SPARK-5233).
Author: jerryshao <saisai.shao@intel.com>
Closes#4032 from jerryshao/SPARK-5233 and squashes the following commits:
f0b0c0b [jerryshao] Further address the comments
a237c75 [jerryshao] Address the comments
e356258 [jerryshao] Fix bug in unit test
558bdc3 [jerryshao] Correctly replay the WAL log when recovering from failure
This is a refactored fix based on jerryshao 's PR #4037
This enabled deletion of old WAL files containing the received block data.
Improvements over #4037
- Respecting the rememberDuration of all receiver streams. In #4037, if there were two receiver streams with multiple remember durations, the deletion would have delete based on the shortest remember duration, thus deleting data prematurely for the receiver stream with longer remember duration.
- Added unit test to test creation of receiver WAL, automatic deletion, and respecting of remember duration.
jerryshao I am going to merge this ASAP to make it 1.2.1 Thanks for the initial draft of this PR. Made my job much easier.
Author: Tathagata Das <tathagata.das1565@gmail.com>
Author: jerryshao <saisai.shao@intel.com>
Closes#4149 from tdas/SPARK-5147 and squashes the following commits:
730798b [Tathagata Das] Added comments.
c4cf067 [Tathagata Das] Minor fixes
2579b27 [Tathagata Das] Refactored the fix to make sure that the cleanup respects the remember duration of all the receiver streams
2736fd1 [jerryshao] Delete the old WAL log periodically
Current Java file stream doesn't support custom key/value type because of loss of type information, details can be seen in [SPARK-5297](https://issues.apache.org/jira/browse/SPARK-5297). Fix this problem by getting correct `ClassTag` from `Class[_]`.
Author: jerryshao <saisai.shao@intel.com>
Closes#4101 from jerryshao/SPARK-5297 and squashes the following commits:
e022ca3 [jerryshao] Add Mima exclusion
ecd61b8 [jerryshao] Fix Java fileInputStream type erasure problem
- The ReceiverTracker receivers `RegisterReceiver` messages two times
1) When the actor at `ReceiverSupervisorImpl`'s preStart is invoked
2) After the receiver is started at the executor `onReceiverStart()` at `ReceiverSupervisorImpl`
Though, RegisterReceiver message uses the same streamId and the receiverInfo gets updated everytime
the message is processed at the `ReceiverTracker`, it makes sense to call register receiver only after the
receiver is started.
Author: Ilayaperumal Gopinathan <igopinathan@pivotal.io>
Closes#3648 from ilayaperumalg/RTActor-remove-prestart and squashes the following commits:
868efab [Ilayaperumal Gopinathan] Increase receiverInfo collector timeout to 2 secs
3118e5e [Ilayaperumal Gopinathan] Fix StreamingListenerSuite's startedReceiverStreamIds size
634abde [Ilayaperumal Gopinathan] Remove duplicate RegisterReceiver message
Currently WAL-backed block is read out from HDFS and put into BlockManger with storage level MEMORY_ONLY_SER by default, since WAL-backed block is already materialized in HDFS with fault-tolerance, no need to put into BlockManger again by default.
Author: jerryshao <saisai.shao@intel.com>
Closes#3906 from jerryshao/SPARK-4999 and squashes the following commits:
b95f95e [jerryshao] Change storeInBlockManager to false by default
https://issues.apache.org/jira/browse/SPARK-5181
Currently, even the logManager is not created, we still see the log entry
s"Writing to log $record"
a simple fix to make log more accurate
Author: CodingCat <zhunansjtu@gmail.com>
Closes#3985 from CodingCat/SPARK-5181 and squashes the following commits:
0e27dc5 [CodingCat] do not print writing WAL log when WAL is disabled
This patch refactors Spark Streaming's FileInputStream tests to remove uses of Thread.sleep() and SystemClock, which should hopefully resolve some longstanding flakiness in these tests (see SPARK-1600).
Key changes:
- Modify FileInputDStream to use the scheduler's Clock instead of System.currentTimeMillis(); this allows it to be tested using ManualClock.
- Fix a synchronization issue in ManualClock's `currentTime` method.
- Add a StreamingTestWaiter class which allows callers to block until a certain number of batches have finished.
- Change the FileInputStream tests so that files' modification times are manually set based off of ManualClock; this eliminates many Thread.sleep calls.
- Update these tests to use the withStreamingContext fixture.
Author: Josh Rosen <joshrosen@databricks.com>
Closes#3801 from JoshRosen/SPARK-1600 and squashes the following commits:
e4494f4 [Josh Rosen] Address a potential race when setting file modification times
8340bd0 [Josh Rosen] Use set comparisons for output.
0b9c252 [Josh Rosen] Fix some ManualClock usage problems.
1cc689f [Josh Rosen] ConcurrentHashMap -> SynchronizedMap
db26c3a [Josh Rosen] Use standard timeout in ScalaTest `eventually` blocks.
3939432 [Josh Rosen] Rename StreamingTestWaiter to BatchCounter
0b9c3a1 [Josh Rosen] Wait for checkpoint to complete
863d71a [Josh Rosen] Remove Thread.sleep that was used to make task run slowly
b4442c3 [Josh Rosen] batchTimeToSelectedFiles should be thread-safe
15b48ee [Josh Rosen] Replace several TestWaiter methods w/ ScalaTest eventually.
fffc51c [Josh Rosen] Revert "Remove last remaining sleep() call"
dbb8247 [Josh Rosen] Remove last remaining sleep() call
566a63f [Josh Rosen] Fix log message and comment typos
da32f3f [Josh Rosen] Fix log message and comment typos
3689214 [Josh Rosen] Merge remote-tracking branch 'origin/master' into SPARK-1600
c8f06b1 [Josh Rosen] Remove Thread.sleep calls in FileInputStream CheckpointSuite test.
d4f2d87 [Josh Rosen] Refactor file input stream tests to not rely on SystemClock.
dda1403 [Josh Rosen] Add StreamingTestWaiter class.
3c3efc3 [Josh Rosen] Synchronize `currentTime` in ManualClock
a95ddc4 [Josh Rosen] Modify FileInputDStream to use Clock class.
This patch disables output spec. validation for jobs launched through Spark Streaming, since this interferes with checkpoint recovery.
Hadoop OutputFormats have a `checkOutputSpecs` method which performs certain checks prior to writing output, such as checking whether the output directory already exists. SPARK-1100 added checks for FileOutputFormat, SPARK-1677 (#947) added a SparkConf configuration to disable these checks, and SPARK-2309 (#1088) extended these checks to run for all OutputFormats, not just FileOutputFormat.
In Spark Streaming, we might have to re-process a batch during checkpoint recovery, so `save` actions may be called multiple times. In addition to `DStream`'s own save actions, users might use `transform` or `foreachRDD` and call the `RDD` and `PairRDD` save actions. When output spec. validation is enabled, the second calls to these actions will fail due to existing output.
This patch automatically disables output spec. validation for jobs submitted by the Spark Streaming scheduler. This is done by using Scala's `DynamicVariable` to propagate the bypass setting without having to mutate SparkConf or introduce a global variable.
Author: Josh Rosen <joshrosen@databricks.com>
Closes#3832 from JoshRosen/SPARK-4835 and squashes the following commits:
36eaf35 [Josh Rosen] Add comment explaining use of transform() in test.
6485cf8 [Josh Rosen] Add test case in Streaming; fix bug for transform()
7b3e06a [Josh Rosen] Remove Streaming-specific setting to undo this change; update conf. guide
bf9094d [Josh Rosen] Revise disableOutputSpecValidation() comment to not refer to Spark Streaming.
e581d17 [Josh Rosen] Deduplicate isOutputSpecValidationEnabled logic.
762e473 [Josh Rosen] [SPARK-4835] Disable validateOutputSpecs for Spark Streaming jobs.
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
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.
...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.
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'
Author: CodingCat <zhunansjtu@gmail.com>
Closes#3807 from CodingCat/new_branch and squashes the following commits:
5167f01 [CodingCat] fix typo in the comment
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
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
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)
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
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
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.
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
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
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'
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.
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.
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
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
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
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
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
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
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.
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
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
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
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].
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
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
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
(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
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.
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
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.
... 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
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
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
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
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.
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
- 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.
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
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
**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
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
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
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
Author: Issac Buenrostro <buenrostro@ooyala.com>
Closes#945 from ibuenros/SPARK-1341-throttle and squashes the following commits:
5514916 [Issac Buenrostro] Formatting changes, added documentation for streaming throttling, stricter unit tests for throttling.
62f395f [Issac Buenrostro] Add comments and license to streaming RateLimiter.scala
7066438 [Issac Buenrostro] Moved throttle code to RateLimiter class, smoother pushing when throttling active
ccafe09 [Issac Buenrostro] Throttle BlockGenerator to limit rate of data consumption.
Fix QueueInputDStream which was not removing dequeued items when used with the oneAtATime flag disabled.
Author: Manuel Laflamme <manuel.laflamme@gmail.com>
Closes#1285 from mlaflamm/spark-2343 and squashes the following commits:
61c9e38 [Manuel Laflamme] Unit tests for queue input stream
c51d029 [Manuel Laflamme] Fix QueueInputDStream with oneAtATime false
The newFilesOnly logic should be inverted: the logic should be that if the flag newFilesOnly==true then only start reading files older than current time. As the code is now if newFilesOnly==true then it will start to read files that are older than 0L (that is: every file in the directory).
Author: Gabriele Nizzoli <mail@nizzoli.net>
Closes#1077 from gabrielenizzoli/master and squashes the following commits:
4f1d261 [Gabriele Nizzoli] Fix for newFilesOnly logic in file DStream
These commits cause `ClosureCleaner.clean` to attempt to serialize the cleaned closure with the default closure serializer and throw a `SparkException` if doing so fails. This behavior is enabled by default but can be disabled at individual callsites of `SparkContext.clean`.
Commit 98e01ae8 fixes some no-op assertions in `GraphSuite` that this work exposed; I'm happy to put that in a separate PR if that would be more appropriate.
Author: William Benton <willb@redhat.com>
Closes#143 from willb/spark-897 and squashes the following commits:
bceab8a [William Benton] Commented DStream corner cases for serializability checking.
64d04d2 [William Benton] FailureSuite now checks both messages and causes.
3b3f74a [William Benton] Stylistic and doc cleanups.
b215dea [William Benton] Fixed spurious failures in ImplicitOrderingSuite
be1ecd6 [William Benton] Don't check serializability of DStream transforms.
abe816b [William Benton] Make proactive serializability checking optional.
5bfff24 [William Benton] Adds proactive closure-serializablilty checking
ed2ccf0 [William Benton] Test cases for SPARK-897.
I got "java.util.NoSuchElementException: key not found: 1401756085000 ms" exception when using kafka stream and 1 sec batchPeriod.
Investigation showed that the reason is that ReceiverLauncher.startReceivers is asynchronous (started in a thread).
https://github.com/vchekan/spark/blob/master/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala#L206
In case of slow starting receiver, such as Kafka, it easily takes more than 2sec to start. In result, no single "compute" will be called on ReceiverInputDStream before first batch job is executed and receivedBlockInfo remains empty (obviously). Batch job will cause ReceiverInputDStream.getReceivedBlockInfo call and "key not found" exception.
The patch makes getReceivedBlockInfo more robust by tolerating missing values.
Author: Vadim Chekan <kot.begemot@gmail.com>
Closes#961 from vchekan/branch-1.0 and squashes the following commits:
e86f82b [Vadim Chekan] Fixed indentation
4609563 [Vadim Chekan] Key not found exception: if receiver is slow to start, it is possible that getReceivedBlockInfo will be called before compute has been called
(cherry picked from commit 26f6b98931)
Signed-off-by: Patrick Wendell <pwendell@gmail.com>
Author: Lars Albertsson <lalle@spotify.com>
Closes#1001 from lallea/contextwaiter_stopped and squashes the following commits:
93cd314 [Lars Albertsson] Mend StreamingContext stop() followed by awaitTermination().
stop resetting spark.driver.port in unit tests (scala, java and python).
Author: Syed Hashmi <shashmi@cloudera.com>
Author: CodingCat <zhunansjtu@gmail.com>
Closes#943 from syedhashmi/master and squashes the following commits:
885f210 [Syed Hashmi] Removing unnecessary file (created by mergetool)
b8bd4b5 [Syed Hashmi] Merge remote-tracking branch 'upstream/master'
b895e59 [Syed Hashmi] Revert "[SPARK-1784] Add a new partitioner"
57b6587 [Syed Hashmi] Revert "[SPARK-1784] Add a balanced partitioner"
1574769 [Syed Hashmi] [SPARK-1942] Stop clearing spark.driver.port in unit tests
4354836 [Syed Hashmi] Revert "SPARK-1686: keep schedule() calling in the main thread"
fd36542 [Syed Hashmi] [SPARK-1784] Add a balanced partitioner
6668015 [CodingCat] SPARK-1686: keep schedule() calling in the main thread
4ca94cc [Syed Hashmi] [SPARK-1784] Add a new partitioner
JIRA: https://issues.apache.org/jira/browse/SPARK-1878
Author: zsxwing <zsxwing@gmail.com>
Closes#822 from zsxwing/SPARK-1878 and squashes the following commits:
4a47e27 [zsxwing] SPARK-1878: Fix the incorrect initialization order
This is a few changes based on the original patch by @scrapcodes.
Author: Prashant Sharma <prashant.s@imaginea.com>
Author: Patrick Wendell <pwendell@gmail.com>
Closes#785 from pwendell/package-docs and squashes the following commits:
c32b731 [Patrick Wendell] Changes based on Prashant's patch
c0463d3 [Prashant Sharma] added eof new line
ce8bf73 [Prashant Sharma] Added eof new line to all files.
4c35f2e [Prashant Sharma] SPARK-1563 Add package-info.java and package.scala files for all packages that appear in docs
Three issues related to temp files that tests generate – these should be touched up for hygiene but are not urgent.
Modules have a log4j.properties which directs the unit-test.log output file to a directory like `[module]/target/unit-test.log`. But this ends up creating `[module]/[module]/target/unit-test.log` instead of former.
The `work/` directory is not deleted by "mvn clean", in the parent and in modules. Neither is the `checkpoint/` directory created under the various external modules.
Many tests create a temp directory, which is not usually deleted. This can be largely resolved by calling `deleteOnExit()` at creation and trying to call `Utils.deleteRecursively` consistently to clean up, sometimes in an `@After` method.
_If anyone seconds the motion, I can create a more significant change that introduces a new test trait along the lines of `LocalSparkContext`, which provides management of temp directories for subclasses to take advantage of._
Author: Sean Owen <sowen@cloudera.com>
Closes#732 from srowen/SPARK-1798 and squashes the following commits:
5af578e [Sean Owen] Try to consistently delete test temp dirs and files, and set deleteOnExit() for each
b21b356 [Sean Owen] Remove work/ and checkpoint/ dirs with mvn clean
bdd0f41 [Sean Owen] Remove duplicate module dir in log4j.properties output path for tests
- [x] Move all of them into subpackages of org.apache.spark.examples (right now some are in org.apache.spark.streaming.examples, for instance, and others are in org.apache.spark.examples.mllib)
- [x] Move Python examples into examples/src/main/python
- [x] Update docs to reflect these changes
Author: Sandeep <sandeep@techaddict.me>
This patch had conflicts when merged, resolved by
Committer: Matei Zaharia <matei@databricks.com>
Closes#571 from techaddict/SPARK-1637 and squashes the following commits:
47ef86c [Sandeep] Changes based on Discussions on PR, removing use of RawTextHelper from examples
8ed2d3f [Sandeep] Docs Updated for changes, Change for java examples
5f96121 [Sandeep] Move Python examples into examples/src/main/python
0a8dd77 [Sandeep] Move all Scala Examples to org.apache.spark.examples (some are in org.apache.spark.streaming.examples, for instance, and others are in org.apache.spark.examples.mllib)
- SPARK-1558: Updated custom receiver guide to match it with the new API
- SPARK-1504: Added deployment and monitoring subsection to streaming
- SPARK-1505: Added migration guide for migrating from 0.9.x and below to Spark 1.0
- Updated various Java streaming examples to use JavaReceiverInputDStream to highlight the API change.
- Removed the requirement for cleaner ttl from streaming guide
Author: Tathagata Das <tathagata.das1565@gmail.com>
Closes#652 from tdas/doc-fix and squashes the following commits:
cb4f4b7 [Tathagata Das] Possible fix for flaky graceful shutdown test.
ab71f7f [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into doc-fix
8d6ff9b [Tathagata Das] Addded migration guide to Spark Streaming.
7d171df [Tathagata Das] Added reference to JavaReceiverInputStream in examples and streaming guide.
49edd7c [Tathagata Das] Change java doc links to use Java docs.
11528d7 [Tathagata Das] Updated links on index page.
ff80970 [Tathagata Das] More updates to streaming guide.
4dc42e9 [Tathagata Das] Added monitoring and other documentation in the streaming guide.
14c6564 [Tathagata Das] Updated custom receiver guide.
It appears that one of these methods doesn't use `org.apache.spark.api.java.function.Function2` like all the others, but uses Scala's `Function2`.
Author: Sean Owen <sowen@cloudera.com>
Closes#633 from srowen/SPARK-1663.2 and squashes the following commits:
1e0232d [Sean Owen] Fix signature of one version of reduceByKeyAndWindow to use Java API Function2, as apparently intended
Author: Tathagata Das <tathagata.das1565@gmail.com>
Closes#558 from tdas/more-fixes and squashes the following commits:
c0c84e6 [Tathagata Das] Removing extra println()
d8a8cf4 [Tathagata Das] More tweaks to make unit test work in Jenkins.
b7caa98 [Tathagata Das] More tweaks.
d337367 [Tathagata Das] More tweaks
22d6f2d [Tathagata Das] Merge remote-tracking branch 'apache/master' into more-fixes
40a961b [Tathagata Das] Modified java test to reduce flakiness.
9410ca6 [Tathagata Das] Merge remote-tracking branch 'apache/master' into more-fixes
86d9147 [Tathagata Das] scala style fix
2f3d7b1 [Tathagata Das] Added Scala custom receiver example.
d677611 [Tathagata Das] Merge remote-tracking branch 'apache/master' into more-fixes
bec3fc2 [Tathagata Das] Added license.
51d6514 [Tathagata Das] Fixed docs on receiver.
81aafa0 [Tathagata Das] Added Java test for Receiver API, and added JavaCustomReceiver example.
@zsxwing I cherry-picked your changes and merged the master. #365 had some conflicts once again!
Author: zsxwing <zsxwing@gmail.com>
Author: Tathagata Das <tathagata.das1565@gmail.com>
Closes#562 from tdas/SPARK-1382 and squashes the following commits:
e2962c1 [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into SPARK-1382
20968d9 [zsxwing] Replace Exception with SparkException in DStream
e476651 [zsxwing] Merge remote-tracking branch 'origin/master' into SPARK-1382
35ba56a [zsxwing] SPARK-1382: Fix NPE in DStream.slice
1617: These changes expose the receiver state (active or inactive) and last error in the UI
1618: If the socket receiver cannot connect in the first attempt, it should try to restart after a delay. That was broken, as the thread that restarts (hence, stops) the receiver waited on Thread.join on itself!
Author: Tathagata Das <tathagata.das1565@gmail.com>
Closes#540 from tdas/streaming-ui-fix and squashes the following commits:
e469434 [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into streaming-ui-fix
dbddf75 [Tathagata Das] Style fix.
66df1a5 [Tathagata Das] Merge remote-tracking branch 'apache/master' into streaming-ui-fix
ad98bc9 [Tathagata Das] Refactored streaming listener to use ReceiverInfo.
d7f849c [Tathagata Das] Revert "Moved BatchInfo from streaming.scheduler to streaming.ui"
5c80919 [Tathagata Das] Moved BatchInfo from streaming.scheduler to streaming.ui
da244f6 [Tathagata Das] Fixed socket receiver as well as made receiver state and error visible in the streamign UI.
This pulls in changes made by @jerryshao in https://github.com/apache/spark/pull/424 and merges with the master.
Author: jerryshao <saisai.shao@intel.com>
Author: Tathagata Das <tathagata.das1565@gmail.com>
Closes#545 from tdas/streaming-metrics and squashes the following commits:
034b443 [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into streaming-metrics
fb3b0a5 [jerryshao] Modify according master update
21939f5 [jerryshao] Style changes according to style check error
976116b [jerryshao] Add StreamSource in StreamingContext for better monitoring through metrics system
The raw input data is stored as blocks in BlockManagers. Earlier they were cleared by cleaner ttl. Now since streaming does not require cleaner TTL to be set, the block would not get cleared. This increases up the Spark's memory usage, which is not even accounted and shown in the Spark storage UI. It may cause the data blocks to spill over to disk, which eventually slows down the receiving of data (persisting to memory become bottlenecked by writing to disk).
The solution in this PR is to automatically remove those blocks. The mechanism to keep track of which BlockRDDs (which has presents the raw data blocks as a RDD) can be safely cleared already exists. Just use it to explicitly remove blocks from BlockRDDs.
Author: Tathagata Das <tathagata.das1565@gmail.com>
Closes#512 from tdas/block-rdd-unpersist and squashes the following commits:
d25e610 [Tathagata Das] Merge remote-tracking branch 'apache/master' into block-rdd-unpersist
5f46d69 [Tathagata Das] Merge remote-tracking branch 'apache/master' into block-rdd-unpersist
2c320cd [Tathagata Das] Updated configuration with spark.streaming.unpersist setting.
2d4b2fd [Tathagata Das] Automatically removed input blocks
A recent commit reintroduced some of the same warnings that SPARK-1488 resolved. These are just a few more of the same changes to remove these warnings.
Author: Sean Owen <sowen@cloudera.com>
Closes#528 from srowen/SPARK-1488.2 and squashes the following commits:
62d592c [Sean Owen] More feature warnings in tests
4e2e94b [Sean Owen] Squash more language feature warnings in new commits by importing implicitConversion
In https://issues.apache.org/jira/browse/SPARK-1540 we'd like to look at Spark's API to see if we can take advantage of Comparable keys in more places, which will make external spilling more efficient. This PR is a first step towards that that shows how to pass an Ordering when available and still continue functioning otherwise. It does this using a new implicit parameter with a default value of null.
The API is currently only in Scala -- in Java we'd have to add new versions of mapToPair and such that take a Comparator, or a new method to add a "type hint" to an RDD. We can address those later though.
Unfortunately requiring all keys to be Comparable would not work without requiring RDDs in general to contain only Comparable types. The reason is that methods such as distinct() and intersection() do a shuffle, but should be usable on RDDs of any type. So ordering will have to remain an optimization for the types that can be ordered. I think this isn't a horrible outcome though because one of the nice things about Spark's API is that it works on objects of *any* type, without requiring you to specify a schema or implement Writable or stuff like that.
Author: Matei Zaharia <matei@databricks.com>
This patch had conflicts when merged, resolved by
Committer: Reynold Xin <rxin@apache.org>
Closes#487 from mateiz/ordered-keys and squashes the following commits:
bd565f6 [Matei Zaharia] Pass an Ordering to only one version of groupBy because the Scala language spec doesn't allow having an optional parameter on all of them (this was only compiling in Scala 2.10 due to a bug).
4629965 [Matei Zaharia] Add tests for other versions of groupBy
3beae85 [Matei Zaharia] Added a test for implicit orderings
80b7a3b [Matei Zaharia] Add an optional Ordering parameter to PairRDDFunctions.
Since shuffles and RDDs that are out of context are automatically cleaned by Spark core (using ContextCleaner) there is no need for setting the cleaner TTL while creating a StreamingContext.
Author: Tathagata Das <tathagata.das1565@gmail.com>
Closes#491 from tdas/ttl-fix and squashes the following commits:
cf01dc7 [Tathagata Das] Removed requirement for TTL in StreamingContext.
A simple change, mostly had to change a bunch of example code.
Author: Patrick Wendell <pwendell@gmail.com>
Closes#438 from pwendell/jar-of-class and squashes the following commits:
aa010ff [Patrick Wendell] SPARK-1496: Have jarOfClass return Option[String]
The current Network Receiver API makes it slightly complicated to right a new receiver as one needs to create an instance of BlockGenerator as shown in SocketReceiver
https://github.com/apache/spark/blob/master/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala#L51
Exposing the BlockGenerator interface has made it harder to improve the receiving process. The API of NetworkReceiver (which was not a very stable API anyways) needs to be change if we are to ensure future stability.
Additionally, the functions like streamingContext.socketStream that create input streams, return DStream objects. That makes it hard to expose functionality (say, rate limits) unique to input dstreams. They should return InputDStream or NetworkInputDStream. This is still not yet implemented.
This PR is blocked on the graceful shutdown PR #247
Author: Tathagata Das <tathagata.das1565@gmail.com>
Closes#300 from tdas/network-receiver-api and squashes the following commits:
ea27b38 [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into network-receiver-api
3a4777c [Tathagata Das] Renamed NetworkInputDStream to ReceiverInputDStream, and ActorReceiver related stuff.
838dd39 [Tathagata Das] Added more events to the StreamingListener to report errors and stopped receivers.
a75c7a6 [Tathagata Das] Address some PR comments and fixed other issues.
91bfa72 [Tathagata Das] Fixed bugs.
8533094 [Tathagata Das] Scala style fixes.
028bde6 [Tathagata Das] Further refactored receiver to allow restarting of a receiver.
43f5290 [Tathagata Das] Made functions that create input streams return InputDStream and NetworkInputDStream, for both Scala and Java.
2c94579 [Tathagata Das] Fixed graceful shutdown by removing interrupts on receiving thread.
9e37a0b [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into network-receiver-api
3223e95 [Tathagata Das] Refactored the code that runs the NetworkReceiver into further classes and traits to make them more testable.
a36cc48 [Tathagata Das] Refactored the NetworkReceiver API for future stability.
change _slideDuration to _windowDuration
Author: baishuo(白硕) <vc_java@hotmail.com>
Closes#425 from baishuo/master and squashes the following commits:
6f09ea1 [baishuo(白硕)] Update ReducedWindowedDStream.scala
For your consideration: scalac currently notes a number of feature warnings during compilation:
```
[warn] there were 65 feature warning(s); re-run with -feature for details
```
Warnings are like:
```
[warn] /Users/srowen/Documents/spark/core/src/main/scala/org/apache/spark/SparkContext.scala:1261: implicit conversion method rddToPairRDDFunctions should be enabled
[warn] by making the implicit value scala.language.implicitConversions visible.
[warn] This can be achieved by adding the import clause 'import scala.language.implicitConversions'
[warn] or by setting the compiler option -language:implicitConversions.
[warn] See the Scala docs for value scala.language.implicitConversions for a discussion
[warn] why the feature should be explicitly enabled.
[warn] implicit def rddToPairRDDFunctions[K: ClassTag, V: ClassTag](rdd: RDD[(K, V)]) =
[warn] ^
```
scalac is suggesting that it's just best practice to explicitly enable certain language features by importing them where used.
This PR simply adds the imports it suggests (and squashes one other Java warning along the way). This leaves just deprecation warnings in the build.
Author: Sean Owen <sowen@cloudera.com>
Closes#404 from srowen/SPARK-1488 and squashes the following commits:
8598980 [Sean Owen] Quiet scalac warnings about language features by explicitly importing language features.
39bc831 [Sean Owen] Enable -feature in scalac to emit language feature warnings
When debugging Spark Streaming applications it is necessary to monitor certain metrics that are not shown in the Spark application UI. For example, what is average processing time of batches? What is the scheduling delay? Is the system able to process as fast as it is receiving data? How many records I am receiving through my receivers?
While the StreamingListener interface introduced in the 0.9 provided some of this information, it could only be accessed programmatically. A UI that shows information specific to the streaming applications is necessary for easier debugging. This PR introduces such a UI. It shows various statistics related to the streaming application. Here is a screenshot of the UI running on my local machine.
http://i.imgur.com/1ooDGhm.png
This UI is integrated into the Spark UI running at 4040.
Author: Tathagata Das <tathagata.das1565@gmail.com>
Author: Andrew Or <andrewor14@gmail.com>
Closes#290 from tdas/streaming-web-ui and squashes the following commits:
fc73ca5 [Tathagata Das] Merge pull request #9 from andrewor14/ui-refactor
642dd88 [Andrew Or] Merge SparkUISuite.scala into UISuite.scala
eb30517 [Andrew Or] Merge github.com:apache/spark into ui-refactor
f4f4cbe [Tathagata Das] More minor fixes.
34bb364 [Tathagata Das] Merge branch 'streaming-web-ui' of github.com:tdas/spark into streaming-web-ui
252c566 [Tathagata Das] Merge pull request #8 from andrewor14/ui-refactor
e038b4b [Tathagata Das] Addressed Patrick's comments.
125a054 [Andrew Or] Disable serving static resources with gzip
90feb8d [Andrew Or] Address Patrick's comments
89dae36 [Tathagata Das] Merge branch 'streaming-web-ui' of github.com:tdas/spark into streaming-web-ui
72fe256 [Tathagata Das] Merge pull request #6 from andrewor14/ui-refactor
2fc09c8 [Tathagata Das] Added binary check exclusions
aa396d4 [Andrew Or] Rename tabs and pages (No more IndexPage.scala)
f8e1053 [Tathagata Das] Added Spark and Streaming UI unit tests.
caa5e05 [Tathagata Das] Merge branch 'streaming-web-ui' of github.com:tdas/spark into streaming-web-ui
585cd65 [Tathagata Das] Merge pull request #5 from andrewor14/ui-refactor
914b8ff [Tathagata Das] Moved utils functions to UIUtils.
548c98c [Andrew Or] Wide refactoring of WebUI, UITab, and UIPage (see commit message)
6de06b0 [Tathagata Das] Merge remote-tracking branch 'apache/master' into streaming-web-ui
ee6543f [Tathagata Das] Minor changes based on Andrew's comments.
fa760fe [Tathagata Das] Fixed long line.
1c0bcef [Tathagata Das] Refactored streaming UI into two files.
1af239b [Tathagata Das] Changed streaming UI to attach itself as a tab with the Spark UI.
827e81a [Tathagata Das] Merge branch 'streaming-web-ui' of github.com:tdas/spark into streaming-web-ui
168fe86 [Tathagata Das] Merge pull request #2 from andrewor14/ui-refactor
3e986f8 [Tathagata Das] Merge remote-tracking branch 'apache/master' into streaming-web-ui
c78c92d [Andrew Or] Remove outdated comment
8f7323b [Andrew Or] End of file new lines, indentation, and imports (minor)
0d61ee8 [Andrew Or] Merge branch 'streaming-web-ui' of github.com:tdas/spark into ui-refactor
9a48fa1 [Andrew Or] Allow adding tabs to SparkUI dynamically + add example
61358e3 [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into streaming-web-ui
53be2c5 [Tathagata Das] Minor style updates.
ed25dfc [Andrew Or] Generalize SparkUI header to display tabs dynamically
a37ad4f [Andrew Or] Comments, imports and formatting (minor)
cd000b0 [Andrew Or] Merge github.com:apache/spark into ui-refactor
7d57444 [Andrew Or] Refactoring the UI interface to add flexibility
aef4dd5 [Tathagata Das] Added Apache licenses.
db27bad [Tathagata Das] Added last batch processing time to StreamingUI.
4d86e98 [Tathagata Das] Added basic stats to the StreamingUI and refactored the UI to a Page to make it easier to transition to using SparkUI later.
93f1c69 [Tathagata Das] Added network receiver information to the Streaming UI.
56cc7fb [Tathagata Das] First cut implementation of Streaming UI.
(This is for discussion at this point -- I'm not suggesting this should be committed.)
This is what removing fastutil looks like. Much of it is straightforward, like using `java.io` buffered stream classes, and Guava for murmurhash3.
Uses of the `FastByteArrayOutputStream` were a little trickier. In only one case though do I think the change to use `java.io` actually entails an extra array copy.
The rest is using `OpenHashMap` and `OpenHashSet`. These are now written in terms of more scala-like operations.
`OpenHashMap` is where I made three non-trivial changes to make it work, and they need review:
- It is no longer private
- The key must be a `ClassTag`
- Unless a lot of other code changes, the key type can't enforce being a supertype of `Null`
It all works and tests pass, and I think there is reason to believe it's OK from a speed perspective.
But what about those last changes?
Author: Sean Owen <sowen@cloudera.com>
Closes#266 from srowen/SPARK-1057-alternate and squashes the following commits:
2601129 [Sean Owen] Fix Map return type error not previously caught
ec65502 [Sean Owen] Updates from matei's review
00bc81e [Sean Owen] Remove use of fastutil and replace with use of java.io, spark.util and Guava classes
update the content of Exception when windowDuration is not multiple of parent.slideDuration
Author: baishuo(白硕) <vc_java@hotmail.com>
Closes#390 from baishuo/windowdstream and squashes the following commits:
533c968 [baishuo(白硕)] Update WindowedDStream.scala
stack these together in a commit else they show up chunk by chunk in different commits.
Author: Sandeep <sandeep@techaddict.me>
Closes#380 from techaddict/white_space and squashes the following commits:
b58f294 [Sandeep] Remove Unnecessary Whitespace's
[SPARK-729](https://spark-project.atlassian.net/browse/SPARK-729) concerns when free variables in closure arguments to transformations are captured. Currently, it is possible for closures to get the environment in which they are serialized (not the environment in which they are created). There are a few possible approaches to solving this problem and this PR will discuss some of them. The approach I took has the advantage of being simple, obviously correct, and minimally-invasive, but it preserves something that has been bothering me about Spark's closure handling, so I'd like to discuss an alternative and get some feedback on whether or not it is worth pursuing.
## What I did
The basic approach I took depends on the work I did for #143, and so this PR is based atop that. Specifically: #143 modifies `ClosureCleaner.clean` to preemptively determine whether or not closures are serializable immediately upon closure cleaning (rather than waiting for an job involving that closure to be scheduled). Thus non-serializable closure exceptions will be triggered by the line defining the closure rather than triggered where the closure is used.
Since the easiest way to determine whether or not a closure is serializable is to attempt to serialize it, the code in #143 is creating a serialized closure as part of `ClosureCleaner.clean`. `clean` currently modifies its argument, but the method in `SparkContext` that wraps it to return a value (a reference to the modified-in-place argument). This branch modifies `ClosureCleaner.clean` so that it returns a value: if it is cleaning a serializable closure, it returns the result of deserializing its serialized argument; therefore it is returning a closure with an environment captured at cleaning time. `SparkContext.clean` then returns the result of `ClosureCleaner.clean`, rather than a reference to its modified-in-place argument.
I've added tests for this behavior (777a1bc). The pull request as it stands, given the changes in #143, is nearly trivial. There is some overhead from deserializing the closure, but it is minimal and the benefit of obvious operational correctness (vs. a more sophisticated but harder-to-validate transformation in `ClosureCleaner`) seems pretty important. I think this is a fine way to solve this problem, but it's not perfect.
## What we might want to do
The thing that has been bothering me about Spark's handling of closures is that it seems like we should be able to statically ensure that cleaning and serialization happen exactly once for a given closure. If we serialize a closure in order to determine whether or not it is serializable, we should be able to hang on to the generated byte buffer and use it instead of re-serializing the closure later. By replacing closures with instances of a sum type that encodes whether or not a closure has been cleaned or serialized, we could handle clean, to-be-cleaned, and serialized closures separately with case matches. Here's a somewhat-concrete sketch (taken from my git stash) of what this might look like:
```scala
package org.apache.spark.util
import java.nio.ByteBuffer
import scala.reflect.ClassManifest
sealed abstract class ClosureBox[T] { def func: T }
final case class RawClosure[T](func: T) extends ClosureBox[T] {}
final case class CleanedClosure[T](func: T) extends ClosureBox[T] {}
final case class SerializedClosure[T](func: T, bytebuf: ByteBuffer) extends ClosureBox[T] {}
object ClosureBoxImplicits {
implicit def closureBoxFromFunc[T <: AnyRef](fun: T) = new RawClosure[T](fun)
}
```
With these types declared, we'd be able to change `ClosureCleaner.clean` to take a `ClosureBox[T=>U]` (possibly generated by implicit conversion) and return a `ClosureBox[T=>U]` (either a `CleanedClosure[T=>U]` or a `SerializedClosure[T=>U]`, depending on whether or not serializability-checking was enabled) instead of a `T=>U`. A case match could thus short-circuit cleaning or serializing closures that had already been cleaned or serialized (both in `ClosureCleaner` and in the closure serializer). Cleaned-and-serialized closures would be represented by a boxed tuple of the original closure and a serialized copy (complete with an environment quiesced at transformation time). Additional implicit conversions could convert from `ClosureBox` instances to the underlying function type where appropriate. Tracking this sort of state in the type system seems like the right thing to do to me.
### Why we might not want to do that
_It's pretty invasive._ Every function type used by every `RDD` subclass would have to change to reflect that they expected a `ClosureBox[T=>U]` instead of a `T=>U`. This obscures what's going on and is not a little ugly. Although I really like the idea of using the type system to enforce the clean-or-serialize once discipline, it might not be worth adding another layer of types (even if we could hide some of the extra boilerplate with judicious application of implicit conversions).
_It statically guarantees a property whose absence is unlikely to cause any serious problems as it stands._ It appears that all closures are currently dynamically cleaned once and it's not obvious that repeated closure-cleaning is likely to be a problem in the future. Furthermore, serializing closures is relatively cheap, so doing it once to check for serialization and once again to actually ship them across the wire doesn't seem like a big deal.
Taken together, these seem like a high price to pay for statically guaranteeing that closures are operated upon only once.
## Other possibilities
I felt like the serialize-and-deserialize approach was best due to its obvious simplicity. But it would be possible to do a more sophisticated transformation within `ClosureCleaner.clean`. It might also be possible for `clean` to modify its argument in a way so that whether or not a given closure had been cleaned would be apparent upon inspection; this would buy us some of the operational benefits of the `ClosureBox` approach but not the static cleanliness.
I'm interested in any feedback or discussion on whether or not the problems with the type-based approach indeed outweigh the advantage, as well as of approaches to this issue and to closure handling in general.
Author: William Benton <willb@redhat.com>
Closes#189 from willb/spark-729 and squashes the following commits:
f4cafa0 [William Benton] Stylistic changes and cleanups
b3d9c86 [William Benton] Fixed style issues in tests
9b56ce0 [William Benton] Added array-element capture test
97e9d91 [William Benton] Split closure-serializability failure tests
12ef6e3 [William Benton] Skip proactive closure capture for runJob
8ee3ee7 [William Benton] Predictable closure environment capture
12c63a7 [William Benton] Added tests for variable capture in closures
d6e8dd6 [William Benton] Don't check serializability of DStream transforms.
4ecf841 [William Benton] Make proactive serializability checking optional.
d8df3db [William Benton] Adds proactive closure-serializablilty checking
21b4b06 [William Benton] Test cases for SPARK-897.
d5947b3 [William Benton] Ensure assertions in Graph.apply are asserted.
Author: Holden Karau <holden@pigscanfly.ca>
Closes#242 from holdenk/spark-1320-cogroupandgroupshouldpassiterator and squashes the following commits:
f289536 [Holden Karau] Fix bad merge, should have been Iterable rather than Iterator
77048f8 [Holden Karau] Fix merge up to master
d3fe909 [Holden Karau] use toSeq instead
7a092a3 [Holden Karau] switch resultitr to resultiterable
eb06216 [Holden Karau] maybe I should have had a coffee first. use correct import for guava iterables
c5075aa [Holden Karau] If guava 14 had iterables
2d06e10 [Holden Karau] Fix Java 8 cogroup tests for the new API
11e730c [Holden Karau] Fix streaming tests
66b583d [Holden Karau] Fix the core test suite to compile
4ed579b [Holden Karau] Refactor from iterator to iterable
d052c07 [Holden Karau] Python tests now pass with iterator pandas
3bcd81d [Holden Karau] Revert "Try and make pickling list iterators work"
cd1e81c [Holden Karau] Try and make pickling list iterators work
c60233a [Holden Karau] Start investigating moving to iterators for python API like the Java/Scala one. tl;dr: We will have to write our own iterator since the default one doesn't pickle well
88a5cef [Holden Karau] Fix cogroup test in JavaAPISuite for streaming
a5ee714 [Holden Karau] oops, was checking wrong iterator
e687f21 [Holden Karau] Fix groupbykey test in JavaAPISuite of streaming
ec8cc3e [Holden Karau] Fix test issues\!
4b0eeb9 [Holden Karau] Switch cast in PairDStreamFunctions
fa395c9 [Holden Karau] Revert "Add a join based on the problem in SVD"
ec99e32 [Holden Karau] Revert "Revert this but for now put things in list pandas"
b692868 [Holden Karau] Revert
7e533f7 [Holden Karau] Fix the bug
8a5153a [Holden Karau] Revert me, but we have some stuff to debug
b4e86a9 [Holden Karau] Add a join based on the problem in SVD
c4510e2 [Holden Karau] Revert this but for now put things in list pandas
b4e0b1d [Holden Karau] Fix style issues
71e8b9f [Holden Karau] I really need to stop calling size on iterators, it is the path of sadness.
b1ae51a [Holden Karau] Fix some of the types in the streaming JavaAPI suite. Probably still needs more work
37888ec [Holden Karau] core/tests now pass
249abde [Holden Karau] org.apache.spark.rdd.PairRDDFunctionsSuite passes
6698186 [Holden Karau] Revert "I think this might be a bad rabbit hole. Started work to make CoGroupedRDD use iterator and then went crazy"
fe992fe [Holden Karau] hmmm try and fix up basic operation suite
172705c [Holden Karau] Fix Java API suite
caafa63 [Holden Karau] I think this might be a bad rabbit hole. Started work to make CoGroupedRDD use iterator and then went crazy
88b3329 [Holden Karau] Fix groupbykey to actually give back an iterator
4991af6 [Holden Karau] Fix some tests
be50246 [Holden Karau] Calling size on an iterator is not so good if we want to use it after
687ffbc [Holden Karau] This is the it compiles point of replacing Seq with Iterator and JList with JIterator in the groupby and cogroup signatures
Current version of StreamingContext.stop() directly kills all the data receivers (NetworkReceiver) without waiting for the data already received to be persisted and processed. This PR provides the fix. Now, when the StreamingContext.stop() is called, the following sequence of steps will happen.
1. The driver will send a stop signal to all the active receivers.
2. Each receiver, when it gets a stop signal from the driver, first stop receiving more data, then waits for the thread that persists data blocks to BlockManager to finish persisting all receive data, and finally quits.
3. After all the receivers have stopped, the driver will wait for the Job Generator and Job Scheduler to finish processing all the received data.
It also fixes the semantics of StreamingContext.start and stop. It will throw appropriate errors and warnings if stop() is called before start(), stop() is called twice, etc.
Author: Tathagata Das <tathagata.das1565@gmail.com>
Closes#247 from tdas/graceful-shutdown and squashes the following commits:
61c0016 [Tathagata Das] Updated MIMA binary check excludes.
ae1d39b [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into graceful-shutdown
6b59cfc [Tathagata Das] Minor changes based on Andrew's comment on PR.
d0b8d65 [Tathagata Das] Reduced time taken by graceful shutdown unit test.
f55bc67 [Tathagata Das] Fix scalastyle
c69b3a7 [Tathagata Das] Updates based on Patrick's comments.
c43b8ae [Tathagata Das] Added graceful shutdown to Spark Streaming.
This PR allows Spark to automatically cleanup metadata and data related to persisted RDDs, shuffles and broadcast variables when the corresponding RDDs, shuffles and broadcast variables fall out of scope from the driver program. This is still a work in progress as broadcast cleanup has not been implemented.
**Implementation Details**
A new class `ContextCleaner` is responsible cleaning all the state. It is instantiated as part of a `SparkContext`. RDD and ShuffleDependency classes have overridden `finalize()` function that gets called whenever their instances go out of scope. The `finalize()` function enqueues the object’s identifier (i.e. RDD ID, shuffle ID, etc.) with the `ContextCleaner`, which is a very short and cheap operation and should not significantly affect the garbage collection mechanism. The `ContextCleaner`, on a different thread, performs the cleanup, whose details are given below.
*RDD cleanup:*
`ContextCleaner` calls `RDD.unpersist()` is used to cleanup persisted RDDs. Regarding metadata, the DAGScheduler automatically cleans up all metadata related to a RDD after all jobs have completed. Only the `SparkContext.persistentRDDs` keeps strong references to persisted RDDs. The `TimeStampedHashMap` used for that has been replaced by `TimeStampedWeakValueHashMap` that keeps only weak references to the RDDs, allowing them to be garbage collected.
*Shuffle cleanup:*
New BlockManager message `RemoveShuffle(<shuffle ID>)` asks the `BlockManagerMaster` and currently active `BlockManager`s to delete all the disk blocks related to the shuffle ID. `ContextCleaner` cleans up shuffle data using this message and also cleans up the metadata in the `MapOutputTracker` of the driver. The `MapOutputTracker` at the workers, that caches the shuffle metadata, maintains a `BoundedHashMap` to limit the shuffle information it caches. Refetching the shuffle information from the driver is not too costly.
*Broadcast cleanup:*
To be done. [This PR](https://github.com/apache/incubator-spark/pull/543/) adds mechanism for explicit cleanup of broadcast variables. `Broadcast.finalize()` will enqueue its own ID with ContextCleaner and the PRs mechanism will be used to unpersist the Broadcast data.
*Other cleanup:*
`ShuffleMapTask` and `ResultTask` caches tasks and used TTL based cleanup (using `TimeStampedHashMap`), so nothing got cleaned up if TTL was not set. Instead, they now use `BoundedHashMap` to keep a limited number of map output information. Cost of repopulating the cache if necessary is very small.
**Current state of implementation**
Implemented RDD and shuffle cleanup. Things left to be done are.
- Cleaning up for broadcast variable still to be done.
- Automatic cleaning up keys with empty weak refs as values in `TimeStampedWeakValueHashMap`
Author: Tathagata Das <tathagata.das1565@gmail.com>
Author: Andrew Or <andrewor14@gmail.com>
Author: Roman Pastukhov <ignatich@mail.ru>
Closes#126 from tdas/state-cleanup and squashes the following commits:
61b8d6e [Tathagata Das] Fixed issue with Tachyon + new BlockManager methods.
f489fdc [Tathagata Das] Merge remote-tracking branch 'apache/master' into state-cleanup
d25a86e [Tathagata Das] Fixed stupid typo.
cff023c [Tathagata Das] Fixed issues based on Andrew's comments.
4d05314 [Tathagata Das] Scala style fix.
2b95b5e [Tathagata Das] Added more documentation on Broadcast implementations, specially which blocks are told about to the driver. Also, fixed Broadcast API to hide destroy functionality.
41c9ece [Tathagata Das] Added more unit tests for BlockManager, DiskBlockManager, and ContextCleaner.
6222697 [Tathagata Das] Fixed bug and adding unit test for removeBroadcast in BlockManagerSuite.
104a89a [Tathagata Das] Fixed failing BroadcastSuite unit tests by introducing blocking for removeShuffle and removeBroadcast in BlockManager*
a430f06 [Tathagata Das] Fixed compilation errors.
b27f8e8 [Tathagata Das] Merge pull request #3 from andrewor14/cleanup
cd72d19 [Andrew Or] Make automatic cleanup configurable (not documented)
ada45f0 [Andrew Or] Merge branch 'state-cleanup' of github.com:tdas/spark into cleanup
a2cc8bc [Tathagata Das] Merge remote-tracking branch 'apache/master' into state-cleanup
c5b1d98 [Andrew Or] Address Patrick's comments
a6460d4 [Andrew Or] Merge github.com:apache/spark into cleanup
762a4d8 [Tathagata Das] Merge pull request #1 from andrewor14/cleanup
f0aabb1 [Andrew Or] Correct semantics for TimeStampedWeakValueHashMap + add tests
5016375 [Andrew Or] Address TD's comments
7ed72fb [Andrew Or] Fix style test fail + remove verbose test message regarding broadcast
634a097 [Andrew Or] Merge branch 'state-cleanup' of github.com:tdas/spark into cleanup
7edbc98 [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into state-cleanup
8557c12 [Andrew Or] Merge github.com:apache/spark into cleanup
e442246 [Andrew Or] Merge github.com:apache/spark into cleanup
88904a3 [Andrew Or] Make TimeStampedWeakValueHashMap a wrapper of TimeStampedHashMap
fbfeec8 [Andrew Or] Add functionality to query executors for their local BlockStatuses
34f436f [Andrew Or] Generalize BroadcastBlockId to remove BroadcastHelperBlockId
0d17060 [Andrew Or] Import, comments, and style fixes (minor)
c92e4d9 [Andrew Or] Merge github.com:apache/spark into cleanup
f201a8d [Andrew Or] Test broadcast cleanup in ContextCleanerSuite + remove BoundedHashMap
e95479c [Andrew Or] Add tests for unpersisting broadcast
544ac86 [Andrew Or] Clean up broadcast blocks through BlockManager*
d0edef3 [Andrew Or] Add framework for broadcast cleanup
ba52e00 [Andrew Or] Refactor broadcast classes
c7ccef1 [Andrew Or] Merge branch 'bc-unpersist-merge' of github.com:ignatich/incubator-spark into cleanup
6c9dcf6 [Tathagata Das] Added missing Apache license
d2f8b97 [Tathagata Das] Removed duplicate unpersistRDD.
a007307 [Tathagata Das] Merge remote-tracking branch 'apache/master' into state-cleanup
620eca3 [Tathagata Das] Changes based on PR comments.
f2881fd [Tathagata Das] Changed ContextCleaner to use ReferenceQueue instead of finalizer
e1fba5f [Tathagata Das] Style fix
892b952 [Tathagata Das] Removed use of BoundedHashMap, and made BlockManagerSlaveActor cleanup shuffle metadata in MapOutputTrackerWorker.
a7260d3 [Tathagata Das] Added try-catch in context cleaner and null value cleaning in TimeStampedWeakValueHashMap.
e61daa0 [Tathagata Das] Modifications based on the comments on PR 126.
ae9da88 [Tathagata Das] Removed unncessary TimeStampedHashMap from DAGScheduler, added try-catches in finalize() methods, and replaced ArrayBlockingQueue to LinkedBlockingQueue to avoid blocking in Java's finalizing thread.
cb0a5a6 [Tathagata Das] Fixed docs and styles.
a24fefc [Tathagata Das] Merge remote-tracking branch 'apache/master' into state-cleanup
8512612 [Tathagata Das] Changed TimeStampedHashMap to use WrappedJavaHashMap.
e427a9e [Tathagata Das] Added ContextCleaner to automatically clean RDDs and shuffles when they fall out of scope. Also replaced TimeStampedHashMap to BoundedHashMaps and TimeStampedWeakValueHashMap for the necessary hashmap behavior.
80dd977 [Roman Pastukhov] Fix for Broadcast unpersist patch.
1e752f1 [Roman Pastukhov] Added unpersist method to Broadcast.
Author: Prashant Sharma <prashant.s@imaginea.com>
Closes#124 from ScrapCodes/SPARK-1096/scalastyle-comment-check and squashes the following commits:
214135a [Prashant Sharma] Review feedback.
5eba88c [Prashant Sharma] Fixed style checks for ///+ comments.
e54b2f8 [Prashant Sharma] improved message, work around.
83e7144 [Prashant Sharma] removed dependency on scalastyle in plugin, since scalastyle sbt plugin already depends on the right version. Incase we update the plugin we will have to adjust our spark-style project to depend on right scalastyle version.
810a1d6 [Prashant Sharma] SPARK-1096, a space after comment style checker.
ba33193 [Prashant Sharma] scala style as a project
Excluded those that are self-evident and the cases that are discussed in the mailing list.
Author: NirmalReddy <nirmal_reddy2000@yahoo.com>
Author: NirmalReddy <nirmal.reddy@imaginea.com>
Closes#168 from NirmalReddy/Spark-1095 and squashes the following commits:
ac54b29 [NirmalReddy] import misplaced
8c5ff3e [NirmalReddy] Changed syntax of unit returning methods
02d0778 [NirmalReddy] fixed explicit types in all the other packages
1c17773 [NirmalReddy] fixed explicit types in core package