Skip to content

Commit

Permalink
Attempt to fix streaming test failures after yarn branch merge
Browse files Browse the repository at this point in the history
  • Loading branch information
mridulm committed Apr 28, 2013
1 parent 1f20ef2 commit afee902
Show file tree
Hide file tree
Showing 10 changed files with 42 additions and 10 deletions.
1 change: 1 addition & 0 deletions bagel/src/test/scala/bagel/BagelSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ class BagelSuite extends FunSuite with Assertions with BeforeAndAfter with Timeo
}
// To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
System.clearProperty("spark.driver.port")
System.clearProperty("spark.hostPort")
}

test("halting by voting") {
Expand Down
3 changes: 2 additions & 1 deletion core/src/test/scala/spark/LocalSparkContext.scala
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@ object LocalSparkContext {
sc.stop()
// To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
System.clearProperty("spark.driver.port")
System.clearProperty("spark.hostPort")
}

/** Runs `f` by passing in `sc` and ensures that `sc` is stopped. */
Expand All @@ -38,4 +39,4 @@ object LocalSparkContext {
}
}

}
}
1 change: 1 addition & 0 deletions repl/src/test/scala/spark/repl/ReplSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@ class ReplSuite extends FunSuite {
interp.sparkContext.stop()
// To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
System.clearProperty("spark.driver.port")
System.clearProperty("spark.hostPort")
return out.toString
}

Expand Down
30 changes: 23 additions & 7 deletions streaming/src/main/scala/spark/streaming/Checkpoint.scala
Original file line number Diff line number Diff line change
Expand Up @@ -38,28 +38,43 @@ class Checkpoint(@transient ssc: StreamingContext, val checkpointTime: Time)
private[streaming]
class CheckpointWriter(checkpointDir: String) extends Logging {
val file = new Path(checkpointDir, "graph")
// The file to which we actually write - and then "move" to file.
private val writeFile = new Path(file.getParent, file.getName + ".next")
private val bakFile = new Path(file.getParent, file.getName + ".bk")

@volatile private var stopped = false

val conf = new Configuration()
var fs = file.getFileSystem(conf)
val maxAttempts = 3
val executor = Executors.newFixedThreadPool(1)

// Removed code which validates whether there is only one CheckpointWriter per path 'file' since
// I did not notice any errors - reintroduce it ?

class CheckpointWriteHandler(checkpointTime: Time, bytes: Array[Byte]) extends Runnable {
def run() {
var attempts = 0
val startTime = System.currentTimeMillis()
while (attempts < maxAttempts) {
if (stopped) {
logInfo("Already stopped, ignore checkpoint attempt for " + file)
return
}
attempts += 1
try {
logDebug("Saving checkpoint for time " + checkpointTime + " to file '" + file + "'")
if (fs.exists(file)) {
val bkFile = new Path(file.getParent, file.getName + ".bk")
FileUtil.copy(fs, file, fs, bkFile, true, true, conf)
logDebug("Moved existing checkpoint file to " + bkFile)
}
val fos = fs.create(file)
// This is inherently thread unsafe .. so alleviating it by writing to '.new' and then doing moves : which should be pretty fast.
val fos = fs.create(writeFile)
fos.write(bytes)
fos.close()
fos.close()
if (fs.exists(file) && fs.rename(file, bakFile)) {
logDebug("Moved existing checkpoint file to " + bakFile)
}
// paranoia
fs.delete(file, false)
fs.rename(writeFile, file)

val finishTime = System.currentTimeMillis();
logInfo("Checkpoint for time " + checkpointTime + " saved to file '" + file +
"', took " + bytes.length + " bytes and " + (finishTime - startTime) + " milliseconds")
Expand All @@ -84,6 +99,7 @@ class CheckpointWriter(checkpointDir: String) extends Logging {
}

def stop() {
stopped = true
executor.shutdown()
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -74,6 +74,7 @@ object MasterFailureTest extends Logging {

val operation = (st: DStream[String]) => {
val updateFunc = (values: Seq[Long], state: Option[Long]) => {
logInfo("UpdateFunc .. state = " + state.getOrElse(0L) + ", values = " + values)
Some(values.foldLeft(0L)(_ + _) + state.getOrElse(0L))
}
st.flatMap(_.split(" "))
Expand Down Expand Up @@ -159,6 +160,7 @@ object MasterFailureTest extends Logging {

// Setup the streaming computation with the given operation
System.clearProperty("spark.driver.port")
System.clearProperty("spark.hostPort")
var ssc = new StreamingContext("local[4]", "MasterFailureTest", batchDuration, null, Nil, Map())
ssc.checkpoint(checkpointDir.toString)
val inputStream = ssc.textFileStream(testDir.toString)
Expand Down Expand Up @@ -205,6 +207,7 @@ object MasterFailureTest extends Logging {
// (iii) Its not timed out yet
System.clearProperty("spark.streaming.clock")
System.clearProperty("spark.driver.port")
System.clearProperty("spark.hostPort")
ssc.start()
val startTime = System.currentTimeMillis()
while (!killed && !isLastOutputGenerated && !isTimedOut) {
Expand Down Expand Up @@ -357,13 +360,16 @@ class FileGeneratingThread(input: Seq[String], testDir: Path, interval: Long)
// Write the data to a local file and then move it to the target test directory
val localFile = new File(localTestDir, (i+1).toString)
val hadoopFile = new Path(testDir, (i+1).toString)
val tempHadoopFile = new Path(testDir, ".tmp_" + (i+1).toString)
FileUtils.writeStringToFile(localFile, input(i).toString + "\n")
var tries = 0
var done = false
while (!done && tries < maxTries) {
tries += 1
try {
fs.copyFromLocalFile(new Path(localFile.toString), hadoopFile)
// fs.copyFromLocalFile(new Path(localFile.toString), hadoopFile)
fs.copyFromLocalFile(new Path(localFile.toString), tempHadoopFile)
fs.rename(tempHadoopFile, hadoopFile)
done = true
} catch {
case ioe: IOException => {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@ class BasicOperationsSuite extends TestSuiteBase {
after {
// To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
System.clearProperty("spark.driver.port")
System.clearProperty("spark.hostPort")
}

test("map") {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter {

// To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
System.clearProperty("spark.driver.port")
System.clearProperty("spark.hostPort")
}

var ssc: StreamingContext = null
Expand Down Expand Up @@ -325,6 +326,7 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter {
)
ssc = new StreamingContext(checkpointDir)
System.clearProperty("spark.driver.port")
System.clearProperty("spark.hostPort")
ssc.start()
val outputNew = advanceTimeWithRealDelay[V](ssc, nextNumBatches)
// the first element will be re-processed data of the last batch before restart
Expand All @@ -350,4 +352,4 @@ class CheckpointSuite extends TestSuiteBase with BeforeAndAfter {
val outputStream = ssc.graph.getOutputStreams.head.asInstanceOf[TestOutputStream[V]]
outputStream.output
}
}
}
2 changes: 2 additions & 0 deletions streaming/src/test/scala/spark/streaming/FailureSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -22,10 +22,12 @@ class FailureSuite extends FunSuite with BeforeAndAfter with Logging {
val batchDuration = Milliseconds(1000)

before {
logInfo("BEFORE ...")
FileUtils.deleteDirectory(new File(directory))
}

after {
logInfo("AFTER ...")
FileUtils.deleteDirectory(new File(directory))
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -41,6 +41,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter {
after {
// To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
System.clearProperty("spark.driver.port")
System.clearProperty("spark.hostPort")
}


Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ class WindowOperationsSuite extends TestSuiteBase {
after {
// To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown
System.clearProperty("spark.driver.port")
System.clearProperty("spark.hostPort")
}

val largerSlideInput = Seq(
Expand Down

0 comments on commit afee902

Please sign in to comment.