Skip to content

Commit

Permalink
HOTFIX: Fix concurrency issue in FlumePollingStreamSuite.
Browse files Browse the repository at this point in the history
This has been failing on master. One possible cause is that the port
gets contended if multiple test runs happen concurrently and they
hit this test at the same time. Since this test takes a long time
(60 seconds) that's very plausible. This patch randomizes the port
used in this test to avoid contention.
  • Loading branch information
pwendell committed Aug 2, 2014
1 parent 25cad6a commit 44460ba
Showing 1 changed file with 6 additions and 1 deletion.
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ package org.apache.spark.streaming.flume

import java.net.InetSocketAddress
import java.util.concurrent.{Callable, ExecutorCompletionService, Executors}
import java.util.Random

import scala.collection.JavaConversions._
import scala.collection.mutable.{SynchronizedBuffer, ArrayBuffer}
Expand All @@ -37,13 +38,16 @@ import org.apache.spark.streaming.flume.sink._

class FlumePollingStreamSuite extends TestSuiteBase {

val testPort = 9999
val random = new Random()
/** Return a port in the ephemeral range. */
def getTestPort = random.nextInt(16382) + 49152
val batchCount = 5
val eventsPerBatch = 100
val totalEventsPerChannel = batchCount * eventsPerBatch
val channelCapacity = 5000

test("flume polling test") {
val testPort = getTestPort
// Set up the streaming context and input streams
val ssc = new StreamingContext(conf, batchDuration)
val flumeStream: ReceiverInputDStream[SparkFlumeEvent] =
Expand Down Expand Up @@ -77,6 +81,7 @@ class FlumePollingStreamSuite extends TestSuiteBase {
}

test("flume polling test multiple hosts") {
val testPort = getTestPort
// Set up the streaming context and input streams
val ssc = new StreamingContext(conf, batchDuration)
val addresses = Seq(testPort, testPort + 1).map(new InetSocketAddress("localhost", _))
Expand Down

0 comments on commit 44460ba

Please sign in to comment.