Skip to content
Snippets Groups Projects
Commit 61d1e87c authored by Hari Shreedharan's avatar Hari Shreedharan Committed by Andrew Or
Browse files

[SPARK-7356] [STREAMING] Fix flakey tests in FlumePollingStreamSuite using...

[SPARK-7356] [STREAMING] Fix flakey tests in FlumePollingStreamSuite using SparkSink's batch CountDownLatch.

This is meant to make the FlumePollingStreamSuite deterministic. Now we basically count the number of batches that have been completed - and then verify the results rather than sleeping for random periods of time.

Author: Hari Shreedharan <hshreedharan@apache.org>

Closes #5918 from harishreedharan/flume-test-fix and squashes the following commits:

93f24f3 [Hari Shreedharan] Add an eventually block to ensure that all received data is processed. Refactor the dstream creation and remove redundant code.
1108804 [Hari Shreedharan] [SPARK-7356][STREAMING] Fix flakey tests in FlumePollingStreamSuite using SparkSink's batch CountDownLatch.
parent bb6dec3b
No related branches found
No related tags found
No related merge requests found
...@@ -18,15 +18,18 @@ ...@@ -18,15 +18,18 @@
package org.apache.spark.streaming.flume package org.apache.spark.streaming.flume
import java.net.InetSocketAddress import java.net.InetSocketAddress
import java.util.concurrent.{Callable, ExecutorCompletionService, Executors} import java.util.concurrent._
import scala.collection.JavaConversions._ import scala.collection.JavaConversions._
import scala.collection.mutable.{SynchronizedBuffer, ArrayBuffer} import scala.collection.mutable.{SynchronizedBuffer, ArrayBuffer}
import scala.concurrent.duration._
import scala.language.postfixOps
import org.apache.flume.Context import org.apache.flume.Context
import org.apache.flume.channel.MemoryChannel import org.apache.flume.channel.MemoryChannel
import org.apache.flume.conf.Configurables import org.apache.flume.conf.Configurables
import org.apache.flume.event.EventBuilder import org.apache.flume.event.EventBuilder
import org.scalatest.concurrent.Eventually._
import org.scalatest.{BeforeAndAfter, FunSuite} import org.scalatest.{BeforeAndAfter, FunSuite}
...@@ -57,11 +60,11 @@ class FlumePollingStreamSuite extends FunSuite with BeforeAndAfter with Logging ...@@ -57,11 +60,11 @@ class FlumePollingStreamSuite extends FunSuite with BeforeAndAfter with Logging
before(beforeFunction()) before(beforeFunction())
ignore("flume polling test") { test("flume polling test") {
testMultipleTimes(testFlumePolling) testMultipleTimes(testFlumePolling)
} }
ignore("flume polling test multiple hosts") { test("flume polling test multiple hosts") {
testMultipleTimes(testFlumePollingMultipleHost) testMultipleTimes(testFlumePollingMultipleHost)
} }
...@@ -100,18 +103,8 @@ class FlumePollingStreamSuite extends FunSuite with BeforeAndAfter with Logging ...@@ -100,18 +103,8 @@ class FlumePollingStreamSuite extends FunSuite with BeforeAndAfter with Logging
Configurables.configure(sink, context) Configurables.configure(sink, context)
sink.setChannel(channel) sink.setChannel(channel)
sink.start() sink.start()
// Set up the streaming context and input streams
val ssc = new StreamingContext(conf, batchDuration)
val flumeStream: ReceiverInputDStream[SparkFlumeEvent] =
FlumeUtils.createPollingStream(ssc, Seq(new InetSocketAddress("localhost", sink.getPort())),
StorageLevel.MEMORY_AND_DISK, eventsPerBatch, 1)
val outputBuffer = new ArrayBuffer[Seq[SparkFlumeEvent]]
with SynchronizedBuffer[Seq[SparkFlumeEvent]]
val outputStream = new TestOutputStream(flumeStream, outputBuffer)
outputStream.register()
ssc.start()
writeAndVerify(Seq(channel), ssc, outputBuffer) writeAndVerify(Seq(sink), Seq(channel))
assertChannelIsEmpty(channel) assertChannelIsEmpty(channel)
sink.stop() sink.stop()
channel.stop() channel.stop()
...@@ -142,10 +135,22 @@ class FlumePollingStreamSuite extends FunSuite with BeforeAndAfter with Logging ...@@ -142,10 +135,22 @@ class FlumePollingStreamSuite extends FunSuite with BeforeAndAfter with Logging
Configurables.configure(sink2, context) Configurables.configure(sink2, context)
sink2.setChannel(channel2) sink2.setChannel(channel2)
sink2.start() sink2.start()
try {
writeAndVerify(Seq(sink, sink2), Seq(channel, channel2))
assertChannelIsEmpty(channel)
assertChannelIsEmpty(channel2)
} finally {
sink.stop()
sink2.stop()
channel.stop()
channel2.stop()
}
}
def writeAndVerify(sinks: Seq[SparkSink], channels: Seq[MemoryChannel]) {
// Set up the streaming context and input streams // Set up the streaming context and input streams
val ssc = new StreamingContext(conf, batchDuration) val ssc = new StreamingContext(conf, batchDuration)
val addresses = Seq(sink.getPort(), sink2.getPort()).map(new InetSocketAddress("localhost", _)) val addresses = sinks.map(sink => new InetSocketAddress("localhost", sink.getPort()))
val flumeStream: ReceiverInputDStream[SparkFlumeEvent] = val flumeStream: ReceiverInputDStream[SparkFlumeEvent] =
FlumeUtils.createPollingStream(ssc, addresses, StorageLevel.MEMORY_AND_DISK, FlumeUtils.createPollingStream(ssc, addresses, StorageLevel.MEMORY_AND_DISK,
eventsPerBatch, 5) eventsPerBatch, 5)
...@@ -155,61 +160,49 @@ class FlumePollingStreamSuite extends FunSuite with BeforeAndAfter with Logging ...@@ -155,61 +160,49 @@ class FlumePollingStreamSuite extends FunSuite with BeforeAndAfter with Logging
outputStream.register() outputStream.register()
ssc.start() ssc.start()
try {
writeAndVerify(Seq(channel, channel2), ssc, outputBuffer)
assertChannelIsEmpty(channel)
assertChannelIsEmpty(channel2)
} finally {
sink.stop()
sink2.stop()
channel.stop()
channel2.stop()
}
}
def writeAndVerify(channels: Seq[MemoryChannel], ssc: StreamingContext,
outputBuffer: ArrayBuffer[Seq[SparkFlumeEvent]]) {
val clock = ssc.scheduler.clock.asInstanceOf[ManualClock] val clock = ssc.scheduler.clock.asInstanceOf[ManualClock]
val executor = Executors.newCachedThreadPool() val executor = Executors.newCachedThreadPool()
val executorCompletion = new ExecutorCompletionService[Void](executor) val executorCompletion = new ExecutorCompletionService[Void](executor)
channels.map(channel => {
val latch = new CountDownLatch(batchCount * channels.size)
sinks.foreach(_.countdownWhenBatchReceived(latch))
channels.foreach(channel => {
executorCompletion.submit(new TxnSubmitter(channel, clock)) executorCompletion.submit(new TxnSubmitter(channel, clock))
}) })
for (i <- 0 until channels.size) { for (i <- 0 until channels.size) {
executorCompletion.take() executorCompletion.take()
} }
val startTime = System.currentTimeMillis()
while (outputBuffer.size < batchCount * channels.size &&
System.currentTimeMillis() - startTime < 15000) {
logInfo("output.size = " + outputBuffer.size)
Thread.sleep(100)
}
val timeTaken = System.currentTimeMillis() - startTime
assert(timeTaken < 15000, "Operation timed out after " + timeTaken + " ms")
logInfo("Stopping context")
ssc.stop()
val flattenedBuffer = outputBuffer.flatten latch.await(15, TimeUnit.SECONDS) // Ensure all data has been received.
assert(flattenedBuffer.size === totalEventsPerChannel * channels.size) clock.advance(batchDuration.milliseconds)
var counter = 0
for (k <- 0 until channels.size; i <- 0 until totalEventsPerChannel) { // The eventually is required to ensure that all data in the batch has been processed.
val eventToVerify = EventBuilder.withBody((channels(k).getName + " - " + eventually(timeout(10 seconds), interval(100 milliseconds)) {
String.valueOf(i)).getBytes("utf-8"), val flattenedBuffer = outputBuffer.flatten
Map[String, String]("test-" + i.toString -> "header")) assert(flattenedBuffer.size === totalEventsPerChannel * channels.size)
var found = false var counter = 0
var j = 0 for (k <- 0 until channels.size; i <- 0 until totalEventsPerChannel) {
while (j < flattenedBuffer.size && !found) { val eventToVerify = EventBuilder.withBody((channels(k).getName + " - " +
val strToCompare = new String(flattenedBuffer(j).event.getBody.array(), "utf-8") String.valueOf(i)).getBytes("utf-8"),
if (new String(eventToVerify.getBody, "utf-8") == strToCompare && Map[String, String]("test-" + i.toString -> "header"))
eventToVerify.getHeaders.get("test-" + i.toString) var found = false
.equals(flattenedBuffer(j).event.getHeaders.get("test-" + i.toString))) { var j = 0
found = true while (j < flattenedBuffer.size && !found) {
counter += 1 val strToCompare = new String(flattenedBuffer(j).event.getBody.array(), "utf-8")
if (new String(eventToVerify.getBody, "utf-8") == strToCompare &&
eventToVerify.getHeaders.get("test-" + i.toString)
.equals(flattenedBuffer(j).event.getHeaders.get("test-" + i.toString))) {
found = true
counter += 1
}
j += 1
} }
j += 1
} }
assert(counter === totalEventsPerChannel * channels.size)
} }
assert(counter === totalEventsPerChannel * channels.size) ssc.stop()
} }
def assertChannelIsEmpty(channel: MemoryChannel): Unit = { def assertChannelIsEmpty(channel: MemoryChannel): Unit = {
...@@ -234,7 +227,6 @@ class FlumePollingStreamSuite extends FunSuite with BeforeAndAfter with Logging ...@@ -234,7 +227,6 @@ class FlumePollingStreamSuite extends FunSuite with BeforeAndAfter with Logging
tx.commit() tx.commit()
tx.close() tx.close()
Thread.sleep(500) // Allow some time for the events to reach Thread.sleep(500) // Allow some time for the events to reach
clock.advance(batchDuration.milliseconds)
} }
null null
} }
......
0% Loading or .
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment