-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-11639][STREAMING][FLAKY-TEST] Implement BlockingWriteAheadLog for testing the BatchedWriteAheadLog #9605
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Closed
Closed
Changes from 1 commit
Commits
Show all changes
5 commits
Select commit
Hold shift + click to select a range
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
address 1
- Loading branch information
commit 6d5e50e1c972ea660f13a3e44e8524eeb7ef5a5d
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -18,15 +18,14 @@ package org.apache.spark.streaming.util | |
|
|
||
| import java.io._ | ||
| import java.nio.ByteBuffer | ||
| import java.util.concurrent.{ExecutionException, ThreadPoolExecutor} | ||
| import java.util.concurrent.atomic.AtomicInteger | ||
| import java.util.{Iterator => JIterator} | ||
| import java.util.concurrent.ThreadPoolExecutor | ||
|
|
||
| import scala.collection.JavaConverters._ | ||
| import scala.collection.mutable.ArrayBuffer | ||
| import scala.concurrent._ | ||
| import scala.concurrent.duration._ | ||
| import scala.language.{implicitConversions, postfixOps} | ||
| import scala.util.{Failure, Success} | ||
|
|
||
| import org.apache.hadoop.conf.Configuration | ||
| import org.apache.hadoop.fs.Path | ||
|
|
@@ -37,13 +36,12 @@ import org.mockito.invocation.InvocationOnMock | |
| import org.mockito.stubbing.Answer | ||
| import org.scalatest.concurrent.Eventually | ||
| import org.scalatest.concurrent.Eventually._ | ||
| import org.scalatest.concurrent.AsyncAssertions.Waiter | ||
| import org.scalatest.{BeforeAndAfterEach, BeforeAndAfter} | ||
| import org.scalatest.mock.MockitoSugar | ||
|
|
||
| import org.apache.spark.streaming.scheduler._ | ||
| import org.apache.spark.util.{ThreadUtils, ManualClock, Utils} | ||
| import org.apache.spark.{SparkException, SparkConf, SparkFunSuite} | ||
| import org.apache.spark.{SparkConf, SparkFunSuite} | ||
|
|
||
| /** Common tests for WriteAheadLogs that we would like to test with different configurations. */ | ||
| abstract class CommonWriteAheadLogTests( | ||
|
|
@@ -367,7 +365,7 @@ class BatchedWriteAheadLogSuite extends CommonWriteAheadLogTests( | |
| } | ||
|
|
||
| // we make the write requests in separate threads so that we don't block the test thread | ||
| private def promiseWriteEvent(wal: WriteAheadLog, event: String, time: Long): Promise[Unit] = { | ||
| private def writeAsync(wal: WriteAheadLog, event: String, time: Long): Promise[Unit] = { | ||
| val p = Promise[Unit]() | ||
| p.completeWith(Future { | ||
| val v = wal.write(event, time) | ||
|
|
@@ -376,30 +374,9 @@ class BatchedWriteAheadLogSuite extends CommonWriteAheadLogTests( | |
| p | ||
| } | ||
|
|
||
| /** | ||
| * In order to block the writes on the writer thread, we mock the write method, and block it | ||
| * for some time with a promise. | ||
| */ | ||
| private def writeBlockingPromise(wal: WriteAheadLog): (Promise[Any], Waiter) = { | ||
| // we would like to block the write so that we can queue requests | ||
| val promise = Promise[Any]() | ||
| val waiter = new Waiter | ||
| when(wal.write(any[ByteBuffer], any[Long])).thenAnswer( | ||
| new Answer[WriteAheadLogRecordHandle] { | ||
| override def answer(invocation: InvocationOnMock): WriteAheadLogRecordHandle = { | ||
| waiter.dismiss() | ||
| Await.ready(promise.future, 4.seconds) | ||
| walHandle | ||
| } | ||
| } | ||
| ) | ||
| (promise, waiter) | ||
| } | ||
|
|
||
| test("BatchedWriteAheadLog - name log with aggregated entries with the timestamp of last entry") { | ||
| val batchedWal = new BatchedWriteAheadLog(wal, sparkConf) | ||
| // block the write so that we can batch some records | ||
| val (promise, waiter) = writeBlockingPromise(wal) | ||
| val blockingWal = new BlockingWriteAheadLog(wal, walHandle) | ||
| val batchedWal = new BatchedWriteAheadLog(blockingWal, sparkConf) | ||
|
|
||
| val event1 = "hello" | ||
| val event2 = "world" | ||
|
|
@@ -409,17 +386,19 @@ class BatchedWriteAheadLogSuite extends CommonWriteAheadLogTests( | |
|
|
||
| // The queue.take() immediately takes the 3, and there is nothing left in the queue at that | ||
| // moment. Then the promise blocks the writing of 3. The rest get queued. | ||
| promiseWriteEvent(batchedWal, event1, 3L) | ||
| waiter.await() | ||
| writeAsync(batchedWal, event1, 3L) | ||
| eventually(timeout(1 second)) { | ||
| assert(blockingWal.isBlocked) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Add |
||
| } | ||
| // rest of the records will be batched while it takes 3 to get written | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. grammar:it takes time for 3 to get written |
||
| promiseWriteEvent(batchedWal, event2, 5L) | ||
| promiseWriteEvent(batchedWal, event3, 8L) | ||
| promiseWriteEvent(batchedWal, event4, 12L) | ||
| promiseWriteEvent(batchedWal, event5, 10L) | ||
| writeAsync(batchedWal, event2, 5L) | ||
| writeAsync(batchedWal, event3, 8L) | ||
| writeAsync(batchedWal, event4, 12L) | ||
| writeAsync(batchedWal, event5, 10L) | ||
| eventually(timeout(1 second)) { | ||
| assert(walBatchingThreadPool.getActiveCount === 5) | ||
| } | ||
| promise.success(true) | ||
| blockingWal.allowWrite() | ||
|
|
||
| val buffer1 = wrapArrayArrayByte(Array(event1)) | ||
| val buffer2 = wrapArrayArrayByte(Array(event2, event3, event4, event5)) | ||
|
|
@@ -441,10 +420,8 @@ class BatchedWriteAheadLogSuite extends CommonWriteAheadLogTests( | |
| } | ||
|
|
||
| test("BatchedWriteAheadLog - fail everything in queue during shutdown") { | ||
| val batchedWal = new BatchedWriteAheadLog(wal, sparkConf) | ||
|
|
||
| // block the write so that we can batch some records | ||
| writeBlockingPromise(wal) | ||
| val blockingWal = new BlockingWriteAheadLog(wal, walHandle) | ||
| val batchedWal = new BatchedWriteAheadLog(blockingWal, sparkConf) | ||
|
|
||
| val event1 = ("hello", 3L) | ||
| val event2 = ("world", 5L) | ||
|
|
@@ -455,7 +432,7 @@ class BatchedWriteAheadLogSuite extends CommonWriteAheadLogTests( | |
| // The queue.take() immediately takes the 3, and there is nothing left in the queue at that | ||
| // moment. Then the promise blocks the writing of 3. The rest get queued. | ||
| val writePromises = Seq(event1, event2, event3, event4, event5).map { event => | ||
| promiseWriteEvent(batchedWal, event._1, event._2) | ||
| writeAsync(batchedWal, event._1, event._2) | ||
| } | ||
|
|
||
| eventually(timeout(1 second)) { | ||
|
|
@@ -645,4 +622,35 @@ object WriteAheadLogSuite { | |
| def wrapArrayArrayByte[T](records: Array[T]): ByteBuffer = { | ||
| ByteBuffer.wrap(Utils.serialize[Array[Array[Byte]]](records.map(Utils.serialize[T]))) | ||
| } | ||
|
|
||
| /** | ||
| * A wrapper WriteAheadLog that blocks the write function to allow batching with the | ||
| * BatchedWriteAheadLog. | ||
| */ | ||
| class BlockingWriteAheadLog( | ||
| wal: WriteAheadLog, | ||
| handle: WriteAheadLogRecordHandle) extends WriteAheadLog { | ||
| @volatile private var isWriteCalled: Boolean = false | ||
| @volatile private var blockWrite: Boolean = true | ||
|
|
||
| override def write(record: ByteBuffer, time: Long): WriteAheadLogRecordHandle = { | ||
| isWriteCalled = true | ||
| eventually(Eventually.timeout(2 second)) { | ||
| assert(!blockWrite) | ||
| } | ||
| wal.write(record, time) | ||
| isWriteCalled = false | ||
| handle | ||
| } | ||
| override def read(segment: WriteAheadLogRecordHandle): ByteBuffer = ??? | ||
| override def readAll(): JIterator[ByteBuffer] = ??? | ||
| override def clean(threshTime: Long, waitForCompletion: Boolean): Unit = ??? | ||
| override def close(): Unit = wal.close() | ||
|
|
||
| def allowWrite(): Unit = { | ||
| blockWrite = false | ||
| } | ||
|
|
||
| def isBlocked: Boolean = isWriteCalled | ||
| } | ||
| } | ||
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Can you also add checks that the batchedWALs internal queue lengths? Here is should be 0, and in line 399 it will be 5.