Skip to content
Closed
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
catch execution context race condition
  • Loading branch information
brkyvz committed Nov 25, 2015
commit 890e6dc33b35866971e9fc896b06c8ed43b0933a
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@
package org.apache.spark.streaming.util

import java.nio.ByteBuffer
import java.util.concurrent.ThreadPoolExecutor
import java.util.concurrent.{RejectedExecutionException, ThreadPoolExecutor}
import java.util.{Iterator => JIterator}

import scala.collection.JavaConverters._
Expand Down Expand Up @@ -176,10 +176,16 @@ private[streaming] class FileBasedWriteAheadLog(
}
oldLogFiles.foreach { logInfo =>
if (!executionContext.isShutdown) {
val f = Future { deleteFile(logInfo) }(executionContext)
if (waitForCompletion) {
import scala.concurrent.duration._
Await.ready(f, 1 second)
try {
val f = Future { deleteFile(logInfo) }(executionContext)
if (waitForCompletion) {
import scala.concurrent.duration._
Await.ready(f, 1 second)
}
} catch {
case e: RejectedExecutionException =>
logWarning("Execution context shutdown before deleting old WriteAheadLogs. " +
"This would not affect recovery correctness.", e)
}
}
}
Expand Down