[SPARK-7282] [STREAMING] Fix the race conditions in StreamingListenerSuite

Fixed the following flaky test
```Scala
[info] StreamingListenerSuite:
[info] - batch info reporting (782 milliseconds)
[info] - receiver info reporting *** FAILED *** (3 seconds, 911 milliseconds)
[info]   The code passed to eventually never returned normally. Attempted 10 times over 3.4735783689999997 seconds. Last failure message: 0 did not equal 1. (StreamingListenerSuite.scala:104)
[info]   org.scalatest.exceptions.TestFailedDueToTimeoutException:
[info]   at org.scalatest.concurrent.Eventually$class.tryTryAgain$1(Eventually.scala:420)
[info]   at org.scalatest.concurrent.Eventually$class.eventually(Eventually.scala:438)
[info]   at org.scalatest.concurrent.Eventually$.eventually(Eventually.scala:478)
[info]   at org.scalatest.concurrent.Eventually$class.eventually(Eventually.scala:307)
[info]   at org.scalatest.concurrent.Eventually$.eventually(Eventually.scala:478)
[info]   at org.apache.spark.streaming.StreamingListenerSuite$$anonfun$2.apply$mcV$sp(StreamingListenerSuite.scala:104)
[info]   at org.apache.spark.streaming.StreamingListenerSuite$$anonfun$2.apply(StreamingListenerSuite.scala:94)
[info]   at org.apache.spark.streaming.StreamingListenerSuite$$anonfun$2.apply(StreamingListenerSuite.scala:94)
[info]   at org.scalatest.Transformer$$anonfun$apply$1.apply$mcV$sp(Transformer.scala:22)
[info]   at org.scalatest.OutcomeOf$class.outcomeOf(OutcomeOf.scala:85)
[info]   at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104)
[info]   at org.scalatest.Transformer.apply(Transformer.scala:22)
[info]   at org.scalatest.Transformer.apply(Transformer.scala:20)
[info]   at org.scalatest.FunSuiteLike$$anon$1.apply(FunSuiteLike.scala:166)
[info]   at org.scalatest.Suite$class.withFixture(Suite.scala:1122)
[info]   at org.scalatest.FunSuite.withFixture(FunSuite.scala:1555)
[info]   at org.scalatest.FunSuiteLike$class.invokeWithFixture$1(FunSuiteLike.scala:163)
[info]   at org.scalatest.FunSuiteLike$$anonfun$runTest$1.apply(FunSuiteLike.scala:175)
[info]   at org.scalatest.FunSuiteLike$$anonfun$runTest$1.apply(FunSuiteLike.scala:175)
[info]   at org.scalatest.SuperEngine.runTestImpl(Engine.scala:306)
[info]   at org.scalatest.FunSuiteLike$class.runTest(FunSuiteLike.scala:175)
[info]   at org.apache.spark.streaming.StreamingListenerSuite.org$scalatest$BeforeAndAfter$$super$runTest(StreamingListenerSuite.scala:34)
[info]   at org.scalatest.BeforeAndAfter$class.runTest(BeforeAndAfter.scala:200)
[info]   at org.apache.spark.streaming.StreamingListenerSuite.runTest(StreamingListenerSuite.scala:34)
[info]   at org.scalatest.FunSuiteLike$$anonfun$runTests$1.apply(FunSuiteLike.scala:208)
[info]   at org.scalatest.FunSuiteLike$$anonfun$runTests$1.apply(FunSuiteLike.scala:208)
[info]   at org.scalatest.SuperEngine$$anonfun$traverseSubNodes$1$1.apply(Engine.scala:413)
[info]   at org.scalatest.SuperEngine$$anonfun$traverseSubNodes$1$1.apply(Engine.scala:401)
[info]   at scala.collection.immutable.List.foreach(List.scala:318)
[info]   at org.scalatest.SuperEngine.traverseSubNodes$1(Engine.scala:401)
[info]   at org.scalatest.SuperEngine.org$scalatest$SuperEngine$$runTestsInBranch(Engine.scala:396)
[info]   at org.scalatest.SuperEngine.runTestsImpl(Engine.scala:483)
[info]   at org.scalatest.FunSuiteLike$class.runTests(FunSuiteLike.scala:208)
[info]   at org.scalatest.FunSuite.runTests(FunSuite.scala:1555)
[info]   at org.scalatest.Suite$class.run(Suite.scala:1424)
[info]   at org.scalatest.FunSuite.org$scalatest$FunSuiteLike$$super$run(FunSuite.scala:1555)
[info]   at org.scalatest.FunSuiteLike$$anonfun$run$1.apply(FunSuiteLike.scala:212)
[info]   at org.scalatest.FunSuiteLike$$anonfun$run$1.apply(FunSuiteLike.scala:212)
[info]   at org.scalatest.SuperEngine.runImpl(Engine.scala:545)
[info]   at org.scalatest.FunSuiteLike$class.run(FunSuiteLike.scala:212)
[info]   at org.apache.spark.streaming.StreamingListenerSuite.org$scalatest$BeforeAndAfter$$super$run(StreamingListenerSuite.scala:34)
[info]   at org.scalatest.BeforeAndAfter$class.run(BeforeAndAfter.scala:241)
[info]   at org.apache.spark.streaming.StreamingListenerSuite.run(StreamingListenerSuite.scala:34)
[info]   at org.scalatest.tools.Framework.org$scalatest$tools$Framework$$runSuite(Framework.scala:462)
[info]   at org.scalatest.tools.Framework$ScalaTestTask.execute(Framework.scala:671)
[info]   at sbt.ForkMain$Run$2.call(ForkMain.java:294)
[info]   at sbt.ForkMain$Run$2.call(ForkMain.java:284)
[info]   at java.util.concurrent.FutureTask.run(FutureTask.java:262)
[info]   at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
[info]   at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
[info]   at java.lang.Thread.run(Thread.java:745)
[info]   Cause: org.scalatest.exceptions.TestFailedException: 0 did not equal 1
[info]   at org.scalatest.MatchersHelper$.newTestFailedException(MatchersHelper.scala:160)
[info]   at org.scalatest.Matchers$ShouldMethodHelper$.shouldMatcher(Matchers.scala:6231)
[info]   at org.scalatest.Matchers$AnyShouldWrapper.should(Matchers.scala:6277)
[info]   at org.apache.spark.streaming.StreamingListenerSuite$$anonfun$2$$anonfun$apply$mcV$sp$1.apply$mcV$sp(StreamingListenerSuite.scala:105)
[info]   at org.apache.spark.streaming.StreamingListenerSuite$$anonfun$2$$anonfun$apply$mcV$sp$1.apply(StreamingListenerSuite.scala:104)
[info]   at org.apache.spark.streaming.StreamingListenerSuite$$anonfun$2$$anonfun$apply$mcV$sp$1.apply(StreamingListenerSuite.scala:104)
[info]   at org.scalatest.concurrent.Eventually$class.makeAValiantAttempt$1(Eventually.scala:394)
[info]   at org.scalatest.concurrent.Eventually$class.tryTryAgain$1(Eventually.scala:408)
[info]   at org.scalatest.concurrent.Eventually$class.eventually(Eventually.scala:438)
[info]   at org.scalatest.concurrent.Eventually$.eventually(Eventually.scala:478)
[info]   at org.scalatest.concurrent.Eventually$class.eventually(Eventually.scala:307)
[info]   at org.scalatest.concurrent.Eventually$.eventually(Eventually.scala:478)
[info]   at org.apache.spark.streaming.StreamingListenerSuite$$anonfun$2.apply$mcV$sp(StreamingListenerSuite.scala:104)
[info]   at org.apache.spark.streaming.StreamingListenerSuite$$anonfun$2.apply(StreamingListenerSuite.scala:94)
[info]   at org.apache.spark.streaming.StreamingListenerSuite$$anonfun$2.apply(StreamingListenerSuite.scala:94)
[info]   at org.scalatest.Transformer$$anonfun$apply$1.apply$mcV$sp(Transformer.scala:22)
[info]   at org.scalatest.OutcomeOf$class.outcomeOf(OutcomeOf.scala:85)
[info]   at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104)
[info]   at org.scalatest.Transformer.apply(Transformer.scala:22)
[info]   at org.scalatest.Transformer.apply(Transformer.scala:20)
[info]   at org.scalatest.FunSuiteLike$$anon$1.apply(FunSuiteLike.scala:166)
[info]   at org.scalatest.Suite$class.withFixture(Suite.scala:1122)
[info]   at org.scalatest.FunSuite.withFixture(FunSuite.scala:1555)
[info]   at org.scalatest.FunSuiteLike$class.invokeWithFixture$1(FunSuiteLike.scala:163)
[info]   at org.scalatest.FunSuiteLike$$anonfun$runTest$1.apply(FunSuiteLike.scala:175)
[info]   at org.scalatest.FunSuiteLike$$anonfun$runTest$1.apply(FunSuiteLike.scala:175)
[info]   at org.scalatest.SuperEngine.runTestImpl(Engine.scala:306)
[info]   at org.scalatest.FunSuiteLike$class.runTest(FunSuiteLike.scala:175)
[info]   at org.apache.spark.streaming.StreamingListenerSuite.org$scalatest$BeforeAndAfter$$super$runTest(StreamingListenerSuite.scala:34)
[info]   at org.scalatest.BeforeAndAfter$class.runTest(BeforeAndAfter.scala:200)
[info]   at org.apache.spark.streaming.StreamingListenerSuite.runTest(StreamingListenerSuite.scala:34)
[info]   at org.scalatest.FunSuiteLike$$anonfun$runTests$1.apply(FunSuiteLike.scala:208)
[info]   at org.scalatest.FunSuiteLike$$anonfun$runTests$1.apply(FunSuiteLike.scala:208)
[info]   at org.scalatest.SuperEngine$$anonfun$traverseSubNodes$1$1.apply(Engine.scala:413)
[info]   at org.scalatest.SuperEngine$$anonfun$traverseSubNodes$1$1.apply(Engine.scala:401)
[info]   at scala.collection.immutable.List.foreach(List.scala:318)
[info]   at org.scalatest.SuperEngine.traverseSubNodes$1(Engine.scala:401)
[info]   at org.scalatest.SuperEngine.org$scalatest$SuperEngine$$runTestsInBranch(Engine.scala:396)
[info]   at org.scalatest.SuperEngine.runTestsImpl(Engine.scala:483)
[info]   at org.scalatest.FunSuiteLike$class.runTests(FunSuiteLike.scala:208)
[info]   at org.scalatest.FunSuite.runTests(FunSuite.scala:1555)
[info]   at org.scalatest.Suite$class.run(Suite.scala:1424)
[info]   at org.scalatest.FunSuite.org$scalatest$FunSuiteLike$$super$run(FunSuite.scala:1555)
[info]   at org.scalatest.FunSuiteLike$$anonfun$run$1.apply(FunSuiteLike.scala:212)
[info]   at org.scalatest.FunSuiteLike$$anonfun$run$1.apply(FunSuiteLike.scala:212)
[info]   at org.scalatest.SuperEngine.runImpl(Engine.scala:545)
[info]   at org.scalatest.FunSuiteLike$class.run(FunSuiteLike.scala:212)
[info]   at org.apache.spark.streaming.StreamingListenerSuite.org$scalatest$BeforeAndAfter$$super$run(StreamingListenerSuite.scala:34)
[info]   at org.scalatest.BeforeAndAfter$class.run(BeforeAndAfter.scala:241)
[info]   at org.apache.spark.streaming.StreamingListenerSuite.run(StreamingListenerSuite.scala:34)
[info]   at org.scalatest.tools.Framework.org$scalatest$tools$Framework$$runSuite(Framework.scala:462)
[info]   at org.scalatest.tools.Framework$ScalaTestTask.execute(Framework.scala:671)
[info]   at sbt.ForkMain$Run$2.call(ForkMain.java:294)
[info]   at sbt.ForkMain$Run$2.call(ForkMain.java:284)
[info]   at java.util.concurrent.FutureTask.run(FutureTask.java:262)
[info]   at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
[info]   at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
[info]   at java.lang.Thread.run(Thread.java:745)
```

The original codes didn't have a memory barrier in the `eventually` closure, which might fail the test, because JVM doesn't guarantee the memory consistency between different threads without  a memory barrier.

This PR used `ConcurrentLinkedQueue` to set up the memory barrier.

Author: zsxwing <zsxwing@gmail.com>

Closes #5812 from zsxwing/SPARK-7282 and squashes the following commits:

59115ef [zsxwing] Use SynchronizedBuffer
014dd2b [zsxwing] Fix the race conditions in StreamingListenerSuite
This commit is contained in:
zsxwing 2015-04-30 21:32:11 -07:00 committed by Tathagata Das
parent beeafcfd6e
commit 69a739c7f5

View file

@ -17,7 +17,7 @@
package org.apache.spark.streaming
import scala.collection.mutable.ArrayBuffer
import scala.collection.mutable.{ArrayBuffer, SynchronizedBuffer}
import scala.concurrent.Future
import scala.concurrent.ExecutionContext.Implicits.global
@ -127,9 +127,9 @@ class StreamingListenerSuite extends TestSuiteBase with Matchers {
/** Listener that collects information on processed batches */
class BatchInfoCollector extends StreamingListener {
val batchInfosCompleted = new ArrayBuffer[BatchInfo]
val batchInfosStarted = new ArrayBuffer[BatchInfo]
val batchInfosSubmitted = new ArrayBuffer[BatchInfo]
val batchInfosCompleted = new ArrayBuffer[BatchInfo] with SynchronizedBuffer[BatchInfo]
val batchInfosStarted = new ArrayBuffer[BatchInfo] with SynchronizedBuffer[BatchInfo]
val batchInfosSubmitted = new ArrayBuffer[BatchInfo] with SynchronizedBuffer[BatchInfo]
override def onBatchSubmitted(batchSubmitted: StreamingListenerBatchSubmitted) {
batchInfosSubmitted += batchSubmitted.batchInfo
@ -146,9 +146,10 @@ class BatchInfoCollector extends StreamingListener {
/** Listener that collects information on processed batches */
class ReceiverInfoCollector extends StreamingListener {
val startedReceiverStreamIds = new ArrayBuffer[Int]
val stoppedReceiverStreamIds = new ArrayBuffer[Int]()
val receiverErrors = new ArrayBuffer[(Int, String, String)]()
val startedReceiverStreamIds = new ArrayBuffer[Int] with SynchronizedBuffer[Int]
val stoppedReceiverStreamIds = new ArrayBuffer[Int] with SynchronizedBuffer[Int]
val receiverErrors =
new ArrayBuffer[(Int, String, String)] with SynchronizedBuffer[(Int, String, String)]
override def onReceiverStarted(receiverStarted: StreamingListenerReceiverStarted) {
startedReceiverStreamIds += receiverStarted.receiverInfo.streamId