aboutsummaryrefslogtreecommitdiff
diff options
context:
space:
mode:
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLog.scala5
1 files changed, 3 insertions, 2 deletions
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLog.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLog.scala
index 9985fedc35..87ba4f84a9 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLog.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLog.scala
@@ -26,7 +26,7 @@ import scala.language.postfixOps
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.Path
-import org.apache.spark.util.ThreadUtils
+import org.apache.spark.util.{CompletionIterator, ThreadUtils}
import org.apache.spark.{Logging, SparkConf}
/**
@@ -124,7 +124,8 @@ private[streaming] class FileBasedWriteAheadLog(
logFilesToRead.iterator.map { file =>
logDebug(s"Creating log reader with $file")
- new FileBasedWriteAheadLogReader(file, hadoopConf)
+ val reader = new FileBasedWriteAheadLogReader(file, hadoopConf)
+ CompletionIterator[ByteBuffer, Iterator[ByteBuffer]](reader, reader.close _)
} flatMap { x => x }
}