| Commit message (Collapse) | Author | Age | Files | Lines |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
| |
pre-batch-start checkpoint
This is another alternative approach to https://github.com/apache/spark/pull/4964/
I think this is a simpler fix that can be backported easily to other branches (1.2 and 1.3).
All it does it introduce a flag so that the pre-batch-start checkpoint does not call clear checkpoint.
There is not unit test yet. I will add it when this approach is commented upon. Not sure if this is testable easily.
Author: Tathagata Das <tathagata.das1565@gmail.com>
Closes #5008 from tdas/SPARK-6222 and squashes the following commits:
7315bc2 [Tathagata Das] Removed empty line.
c438de4 [Tathagata Das] Revert unnecessary change.
5e98374 [Tathagata Das] Added unit test
50cb60b [Tathagata Das] Fixed style issue
295ca5c [Tathagata Das] Fixing SPARK-6222
(cherry picked from commit 645cf3fcc21987417b2946bdeeeb60af3edf667e)
Signed-off-by: Tathagata Das <tathagata.das1565@gmail.com>
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
| |
Java APIs
This PR addresses a Scala compiler bug ([SI-8905](https://issues.scala-lang.org/browse/SI-8905)) that was breaking some of the Spark Java APIs. In a nutshell, it seems that methods whose implementations are inherited from generic traits sometimes have their type parameters erased to Object. This was causing methods like `DoubleRDD.min()` to throw confusing NoSuchMethodErrors at runtime.
The fix implemented here is to introduce an intermediate layer of abstract classes and inherit from those instead of directly extends the `Java*Like` traits. This should not break binary compatibility.
I also improved the test coverage of the Java API, adding several new tests for methods that failed at runtime due to this bug.
Author: Josh Rosen <joshrosen@databricks.com>
Closes #5050 from JoshRosen/javardd-si-8905-fix and squashes the following commits:
2feb068 [Josh Rosen] Use intermediate abstract classes to work around SPARK-3266
d5f3e5d [Josh Rosen] Add failing regression tests for SPARK-3266
(cherry picked from commit 0f673c21f68ee3d5df3c01ae405709d3c1f4909b)
Signed-off-by: Josh Rosen <joshrosen@databricks.com>
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
| |
context from checkpoint
In streaming driver recovery, when the SparkConf is reconstructed based on the checkpointed configuration, it recovers the old master URL. This okay if the cluster on which the streaming application is relaunched is the same cluster as it was running before. But if that cluster changes, there is no way to inject the new master URL of the new cluster. As a result, the restarted app tries to connect to the non-existent old cluster and fails.
The solution is to check whether a master URL is set in the System properties (by Spark submit) before recreating the SparkConf. If a new master url is set in the properties, then use it as that is obviously the most relevant one. Otherwise load the old one (to maintain existing behavior).
Author: Tathagata Das <tathagata.das1565@gmail.com>
Closes #5024 from tdas/SPARK-6331 and squashes the following commits:
392fd44 [Tathagata Das] Fixed naming issue.
c7c0b99 [Tathagata Das] Addressed comments.
6a0857c [Tathagata Das] Updated testsuites.
222485d [Tathagata Das] Load new master URL if present when recovering streaming context from checkpoint
(cherry picked from commit c928796ade54f68e26bc55734a9867a046d2e3fe)
Signed-off-by: Tathagata Das <tathagata.das1565@gmail.com>
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
| |
Currently we would create a new streaming tab for each streamingContext even if there's already one on the same sparkContext which would cause duplicate StreamingTab created and none of them is taking effect.
snapshot: https://www.dropbox.com/s/t4gd6hqyqo0nivz/bad%20multiple%20streamings.png?dl=0
How to reproduce:
1)
import org.apache.spark.SparkConf
import org.apache.spark.streaming.
{Seconds, StreamingContext}
import org.apache.spark.storage.StorageLevel
val ssc = new StreamingContext(sc, Seconds(1))
val lines = ssc.socketTextStream("localhost", 9999, StorageLevel.MEMORY_AND_DISK_SER)
val words = lines.flatMap(_.split(" "))
val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _)
wordCounts.print()
ssc.start()
.....
2)
ssc.stop(false)
val ssc = new StreamingContext(sc, Seconds(1))
val lines = ssc.socketTextStream("localhost", 9999, StorageLevel.MEMORY_AND_DISK_SER)
val words = lines.flatMap(_.split(" "))
val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _)
wordCounts.print()
ssc.start()
Author: lisurprise <zhichao.li@intel.com>
Closes #4828 from zhichao-li/master and squashes the following commits:
c329806 [lisurprise] add test for attaching/detaching streaming tab
51e6c7f [lisurprise] move detach method into StreamingTab
31a44fa [lisurprise] add unit test for attaching and detaching new tab
db25ed2 [lisurprise] clean code
8281bcb [lisurprise] clean code
193c542 [lisurprise] remove streaming tab while closing streaming context
(cherry picked from commit f149b8b5e542af44650923d0156f037121b45a20)
Signed-off-by: Tathagata Das <tathagata.das1565@gmail.com>
|
|
|
|
|
|
|
|
|
|
|
| |
Author: Saisai Shao <saisai.shao@intel.com>
Closes #4722 from jerryshao/SPARK-5943 and squashes the following commits:
1b01233 [Saisai Shao] Update the test to use new API to reduce the warning
(cherry picked from commit 757b14b862a1d39c1bad7b321dae1a3ea8338fbb)
Signed-off-by: Sean Owen <sowen@cloudera.com>
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
| |
Another one from JoshRosen 's wish list. The first commit is much smaller and removes 2 of the 4 Clock classes. The second is much larger, necessary for consolidating the streaming one. I put together implementations in the way that seemed simplest. Almost all the change is standardizing class and method names.
Author: Sean Owen <sowen@cloudera.com>
Closes #4514 from srowen/SPARK-4682 and squashes the following commits:
5ed3a03 [Sean Owen] Javadoc Clock classes; make ManualClock private[spark]
169dd13 [Sean Owen] Add support for legacy org.apache.spark.streaming clock class names
277785a [Sean Owen] Reduce the net change in this patch by reversing some unnecessary syntax changes along the way
b5e53df [Sean Owen] FakeClock -> ManualClock; getTime() -> getTimeMillis()
160863a [Sean Owen] Consolidate Streaming Clock class into common util Clock
7c956b2 [Sean Owen] Consolidate Clocks except for Streaming Clock
(cherry picked from commit 34b7c35380c88569a1396fb4ed991a0bed4288e7)
Signed-off-by: Andrew Or <andrew@databricks.com>
|
|
|
|
|
|
|
|
|
|
|
|
| |
Author: jerryshao <saisai.shao@intel.com>
Closes #4612 from jerryshao/SPARK-5826 and squashes the following commits:
7ec71db [jerryshao] Remove transient for conf statement
88d84e6 [jerryshao] Fix Configuration not serializable problem
(cherry picked from commit a65766bf0244a41b793b9dc5fbdd2882664ad00e)
Signed-off-by: Sean Owen <sowen@cloudera.com>
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
| |
not friendly to java
Revise JavaPairDStream API declaration on saveAs Hadoop methods, to allow it to be called directly as intended.
CC tdas for review
Author: Sean Owen <sowen@cloudera.com>
Closes #4608 from srowen/SPARK-5795 and squashes the following commits:
36f1ead [Sean Owen] Add code that shows compile problem and fix
036bd27 [Sean Owen] Revise JavaPairDStream API declaration on saveAs Hadoop methods, to allow it to be called directly as intended.
(cherry picked from commit 8e25373ce72061d3b6a353259ec627606afa4a5f)
Signed-off-by: Sean Owen <sowen@cloudera.com>
|
|
|
|
|
|
|
|
|
|
|
|
|
| |
The warning log looks incorrect. Just fix it.
Author: Liang-Chi Hsieh <viirya@gmail.com>
Closes #4360 from viirya/fixing_typo and squashes the following commits:
48fbe4f [Liang-Chi Hsieh] Fix incorrect warning log.
(cherry picked from commit a74cbbf12fa59df37eb7172652138c78707d33d8)
Signed-off-by: Tathagata Das <tdas@databricks.com>
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
| |
Added `awaitTerminationOrTimeout` to return if the waiting time elapsed:
* `true` if it's stopped.
* `false` if the waiting time elapsed before returning from the method.
* throw the reported error if it's thrown during the execution.
Also deprecated `awaitTermination(timeout: Long)`.
Author: zsxwing <zsxwing@gmail.com>
Closes #4171 from zsxwing/SPARK-5379 and squashes the following commits:
c9e660b [zsxwing] Add a unit test for awaitTerminationOrTimeout
8a89f92 [zsxwing] Add awaitTerminationOrTimeout to python
cdc820b [zsxwing] Add awaitTerminationOrTimeout
(cherry picked from commit 4cf4cba08f1757ec0d9bffdfae6db719a4fb5a3f)
Signed-off-by: Tathagata Das <tdas@databricks.com>
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
| |
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
(cherry picked from commit 242b4f02df7f71ebcfa86a85c9ed39e40750a7fd)
Signed-off-by: Tathagata Das <tathagata.das1565@gmail.com>
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
| |
terminate
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
(cherry picked from commit 1e8b5394b44a0d3b36f64f10576c3ae3b977810c)
Signed-off-by: Tathagata Das <tathagata.das1565@gmail.com>
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
| |
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
|
|
|
|
|
|
|
| |
Whenever a directory is created by the utility method, immediately restrict
its permissions so that only the owner has access to its contents.
Signed-off-by: Josh Rosen <joshrosen@databricks.com>
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
| |
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
|
|
|
|
|
|
|
|
|
|
| |
Another trivial one. The RAT failure was due to temp files from `FailureSuite` not being cleaned up. This just makes the cleanup more reliable by using the standard temp dir mechanism.
Author: Sean Owen <sowen@cloudera.com>
Closes #4189 from srowen/SPARK-4430 and squashes the following commits:
9ea63ff [Sean Owen] Properly acquire a temp directory to ensure it is cleaned up at shutdown, which helps avoid a RAT check failure
|
|
|
|
|
|
|
|
|
|
|
|
|
|
| |
`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 PR:
- Reenables `surefire`, and copies config from `scalatest` (which is itself an old fork of `surefire`, so similar)
- Tells `surefire` to test only Java tests
- Enables `surefire` and `scalatest` for all children, and in turn eliminates some duplication.
For me this causes the Scala and Java tests to be run once each, it seems, as desired. It doesn't affect the SBT build but works for Maven. I still need to verify that all of the Scala tests and Java tests are being run.
Author: Sean Owen <sowen@cloudera.com>
Closes #3651 from srowen/SPARK-4159 and squashes the following commits:
2e8a0af [Sean Owen] Remove specialized SPARK_HOME setting for REPL, YARN tests as it appears to be obsolete
12e4558 [Sean Owen] Append to unit-test.log instead of overwriting, so that both surefire and scalatest output is preserved. Also standardize/correct comments a bit.
e6f8601 [Sean Owen] Reenable Java tests by reenabling surefire with config cloned from scalatest; centralize test config in the parent
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
| |
and SystemClock usage
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.
|
|
|
|
|
|
|
|
|
|
|
|
|
|
| |
Streaming UI
This is a follow-up work of [SPARK-4537](https://issues.apache.org/jira/browse/SPARK-4537). Adding total received records and processed records metrics back to UI.
![screenshot](https://dl.dropboxusercontent.com/u/19230832/screenshot.png)
Author: jerryshao <saisai.shao@intel.com>
Closes #3852 from jerryshao/SPARK-5028 and squashes the following commits:
c8c4877 [jerryshao] Add total received and processed metrics to Streaming UI
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
| |
...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.
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
| |
Several of our tests call System.setProperty (or test code which implicitly sets system properties) and don't always reset/clear the modified properties, which can create ordering dependencies between tests and cause hard-to-diagnose failures.
This patch removes most uses of System.setProperty from our tests, since in most cases we can use SparkConf to set these configurations (there are a few exceptions, including the tests of SparkConf itself).
For the cases where we continue to use System.setProperty, this patch introduces a `ResetSystemProperties` ScalaTest mixin class which snapshots the system properties before individual tests and to automatically restores them on test completion / failure. See the block comment at the top of the ResetSystemProperties class for more details.
Author: Josh Rosen <joshrosen@databricks.com>
Closes #3739 from JoshRosen/cleanup-system-properties-in-tests and squashes the following commits:
0236d66 [Josh Rosen] Replace setProperty uses in two example programs / tools
3888fe3 [Josh Rosen] Remove setProperty use in LocalJavaStreamingContext
4f4031d [Josh Rosen] Add note on why SparkSubmitSuite needs ResetSystemProperties
4742a5b [Josh Rosen] Clarify ResetSystemProperties trait inheritance ordering.
0eaf0b6 [Josh Rosen] Remove setProperty call in TaskResultGetterSuite.
7a3d224 [Josh Rosen] Fix trait ordering
3fdb554 [Josh Rosen] Remove setProperty call in TaskSchedulerImplSuite
bee20df [Josh Rosen] Remove setProperty calls in SparkContextSchedulerCreationSuite
655587c [Josh Rosen] Remove setProperty calls in JobCancellationSuite
3f2f955 [Josh Rosen] Remove System.setProperty calls in DistributedSuite
cfe9cce [Josh Rosen] Remove use of system properties in SparkContextSuite
8783ab0 [Josh Rosen] Remove TestUtils.setSystemProperty, since it is subsumed by the ResetSystemProperties trait.
633a84a [Josh Rosen] Remove use of system properties in FileServerSuite
25bfce2 [Josh Rosen] Use ResetSystemProperties in UtilsSuite
1d1aa5a [Josh Rosen] Use ResetSystemProperties in SizeEstimatorSuite
dd9492b [Josh Rosen] Use ResetSystemProperties in AkkaUtilsSuite
b0daff2 [Josh Rosen] Use ResetSystemProperties in BlockManagerSuite
e9ded62 [Josh Rosen] Use ResetSystemProperties in TaskSchedulerImplSuite
5b3cb54 [Josh Rosen] Use ResetSystemProperties in SparkListenerSuite
0995c4b [Josh Rosen] Use ResetSystemProperties in SparkContextSchedulerCreationSuite
c83ded8 [Josh Rosen] Use ResetSystemProperties in SparkConfSuite
51aa870 [Josh Rosen] Use withSystemProperty in ShuffleSuite
60a63a1 [Josh Rosen] Use ResetSystemProperties in JobCancellationSuite
14a92e4 [Josh Rosen] Use withSystemProperty in FileServerSuite
628f46c [Josh Rosen] Use ResetSystemProperties in DistributedSuite
9e3e0dd [Josh Rosen] Add ResetSystemProperties test fixture mixin; use it in SparkSubmitSuite.
4dcea38 [Josh Rosen] Move withSystemProperty to TestUtils class.
|
|
|
|
|
|
|
|
|
|
|
|
|
|
| |
'spurious wakeup'
Used `Condition` to rewrite `ContextWaiter` because it provides a convenient API `awaitNanos` for timeout.
Author: zsxwing <zsxwing@gmail.com>
Closes #3661 from zsxwing/SPARK-4813 and squashes the following commits:
52247f5 [zsxwing] Add explicit unit type
be42bcf [zsxwing] Update as per review suggestion
e06bd4f [zsxwing] Fix the issue that ContextWaiter didn't handle 'spurious wakeup'
|
|
|
|
|
|
|
|
| |
Author: CodingCat <zhunansjtu@gmail.com>
Closes #3807 from CodingCat/new_branch and squashes the following commits:
5167f01 [CodingCat] fix typo in the comment
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
| |
convenience
There is only one implicit function `toPairDStreamFunctions` in `StreamingContext`. This PR did similar reorganization like [SPARK-4397](https://issues.apache.org/jira/browse/SPARK-4397).
Compiled the following codes with Spark Streaming 1.1.0 and ran it with this PR. Everything is fine.
```Scala
import org.apache.spark._
import org.apache.spark.streaming._
import org.apache.spark.streaming.StreamingContext._
object StreamingApp {
def main(args: Array[String]) {
val conf = new SparkConf().setMaster("local[2]").setAppName("FileWordCount")
val ssc = new StreamingContext(conf, Seconds(10))
val lines = ssc.textFileStream("/some/path")
val words = lines.flatMap(_.split(" "))
val pairs = words.map(word => (word, 1))
val wordCounts = pairs.reduceByKey(_ + _)
wordCounts.print()
ssc.start()
ssc.awaitTermination()
}
}
```
Author: zsxwing <zsxwing@gmail.com>
Closes #3464 from zsxwing/SPARK-4608 and squashes the following commits:
aa6d44a [zsxwing] Fix a copy-paste error
f74c190 [zsxwing] Merge branch 'master' into SPARK-4608
e6f9cc9 [zsxwing] Update the docs
27833bb [zsxwing] Remove `import StreamingContext._`
c15162c [zsxwing] Reorganize StreamingContext implicit to improve API convenience
|
|
|
|
|
|
|
|
|
|
|
|
|
|
| |
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
|
|
|
|
|
|
|
|
|
|
|
|
| |
`Future.flatMap`(for-loop) in WriteAheadLogBasedBlockHandler
Use `Future.zip` instead of `Future.flatMap`(for-loop). `zip` implies these two Futures will run concurrently, while `flatMap` usually means one Future depends on the other one.
Author: zsxwing <zsxwing@gmail.com>
Closes #3721 from zsxwing/SPARK-4873 and squashes the following commits:
46a2cd9 [zsxwing] Use Future.zip instead of Future.flatMap(for-loop)
|
|
|
|
|
|
|
|
|
|
| |
There are a number of warnings generated in a normal, successful build right now. They're mostly Java unchecked cast warnings, which can be suppressed. But there's a grab bag of other Scala language warnings and so on that can all be easily fixed. The forthcoming PR fixes about 90% of the build warnings I see now.
Author: Sean Owen <sowen@cloudera.com>
Closes #3157 from srowen/SPARK-4297 and squashes the following commits:
8c9e469 [Sean Owen] Suppress unchecked cast warnings, and several other build warning fixes
|
|
|
|
|
|
|
|
|
|
|
|
| |
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
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
| |
Author: Ryan Williams <ryan.blake.williams@gmail.com>
Closes #3523 from ryan-williams/tweaks and squashes the following commits:
d2eddaa [Ryan Williams] code review feedback
ce27fc1 [Ryan Williams] CoGroupedRDD comment nit
c6cfad9 [Ryan Williams] remove unnecessary if statement
b74ea35 [Ryan Williams] comment fix
b0221f0 [Ryan Williams] fix a gendered pronoun
c71ffed [Ryan Williams] use names on a few boolean parameters
89954aa [Ryan Williams] clarify some comments in {Security,Shuffle}Manager
e465dac [Ryan Williams] Saved building-spark.md with Dillinger.io
83e8358 [Ryan Williams] fix pom.xml typo
dc4662b [Ryan Williams] typo fixes in tuning.md, configuration.md
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
| |
This PR should fix SPARK-4826, an issue where a bug in how we generate temp. file names was causing spurious test failures in the write ahead log suites.
Closes #3695.
Closes #3701.
Author: Josh Rosen <joshrosen@databricks.com>
Closes #3704 from JoshRosen/SPARK-4826 and squashes the following commits:
f2307f5 [Josh Rosen] Use Spark Utils class for directory creation/deletion
a693ddb [Josh Rosen] remove unused Random import
b275e41 [Josh Rosen] Move creation of temp. dir to beforeEach/afterEach.
9362919 [Josh Rosen] [SPARK-4826] Fix bug in generation of temp file names. in WAL suites.
86c1944 [Josh Rosen] Revert "HOTFIX: Disabling failing block manager test"
|
| |
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
| |
PairDStreamFunctions.saveAsNewAPIHadoopFiles
Solves two JIRAs in one shot
- Makes the ForechDStream created by saveAsNewAPIHadoopFiles serializable for checkpoints
- Makes the default configuration object used saveAsNewAPIHadoopFiles be the Spark's hadoop configuration
Author: Tathagata Das <tathagata.das1565@gmail.com>
Closes #3457 from tdas/savefiles-fix and squashes the following commits:
bb4729a [Tathagata Das] Same treatment for saveAsHadoopFiles
b382ea9 [Tathagata Das] Fix serialization issue in PairDStreamFunctions.saveAsNewAPIHadoopFiles.
|
|
|
|
|
|
|
|
|
|
|
|
| |
is displayed correctly on the Spark UI
When running the NetworkWordCount, the description of the word count jobs are set as "getCallsite at DStream:xxx" . This should be set to the line number of the streaming application that has the output operation that led to the job being created. This is because the callsite is incorrectly set in the thread launching the jobs. This PR fixes that.
Author: Tathagata Das <tathagata.das1565@gmail.com>
Closes #3455 from tdas/streaming-callsite-fix and squashes the following commits:
69fc26f [Tathagata Das] Set correct call site for streaming jobs so that it is displayed correctly on the Spark UI
|
|
|
|
|
|
|
|
|
|
|
|
| |
in Spark Streaming and there's no job executed
Author: jerryshao <saisai.shao@intel.com>
Closes #3244 from jerryshao/SPARK-4381 and squashes the following commits:
d2486c7 [jerryshao] Improve the warning log
d726e85 [jerryshao] Add local[1] to the filter condition
eca428b [jerryshao] Add warning log
|
|
|
|
|
|
|
|
|
|
|
| |
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'
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
| |
from being processed multiple times
Because of a corner case, a file already selected for batch t can get considered again for batch t+2. This refactoring fixes it by remembering all the files selected in the last 1 minute, so that this corner case does not arise. Also uses spark context's hadoop configuration to access the file system API for listing directories.
pwendell Please take look. I still have not run long-running integration tests, so I cannot say for sure whether this has indeed solved the issue. You could do a first pass on this in the meantime.
Author: Tathagata Das <tathagata.das1565@gmail.com>
Closes #3419 from tdas/filestream-fix2 and squashes the following commits:
c19dd8a [Tathagata Das] Addressed PR comments.
513b608 [Tathagata Das] Updated docs.
d364faf [Tathagata Das] Added the current time condition back
5526222 [Tathagata Das] Removed unnecessary imports.
38bb736 [Tathagata Das] Fix long line.
203bbc7 [Tathagata Das] Un-ignore tests.
eaef4e1 [Tathagata Das] Fixed SPARK-4519
9dbd40a [Tathagata Das] Refactored FileInputDStream to remember last few batches.
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
|
| |
in the same way as TransformedDStream
In class TransformedDStream:
```scala
require(parents.length > 0, "List of DStreams to transform is empty")
require(parents.map(.ssc).distinct.size == 1, "Some of the DStreams have different contexts")
require(parents.map(.slideDuration).distinct.size == 1,
"Some of the DStreams have different slide durations")
```
In class UnionDStream:
```scala
if (parents.length == 0)
{ throw new IllegalArgumentException("Empty array of parents") }
if (parents.map(.ssc).distinct.size > 1)
{ throw new IllegalArgumentException("Array of parents have different StreamingContexts") }
if (parents.map(.slideDuration).distinct.size > 1)
{ throw new IllegalArgumentException("Array of parents have different slide times") }
```
The function is the same, but the realization is not. I think they shoule be the same.
Author: Yadong Qi <qiyadong2010@gmail.com>
Closes #3152 from watermen/bug-fix1 and squashes the following commits:
ed66db6 [Yadong Qi] Change transform to union
b6b3b8b [Yadong Qi] The same function should have the same realization.
|
|
|
|
|
|
|
|
|
|
|
| |
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
|
|
|
|
|
|
|
|
|
|
|
|
| |
default
The write ahead log of ReceivedBlockTracker gets enabled as soon as checkpoint directory is set. This should not happen, as the WAL should be enabled only if the WAL is enabled in the Spark configuration.
Author: Tathagata Das <tathagata.das1565@gmail.com>
Closes #3358 from tdas/SPARK-4482 and squashes the following commits:
b740136 [Tathagata Das] Fixed bug in ReceivedBlockTracker
|