aboutsummaryrefslogtreecommitdiff
path: root/streaming/src/main
diff options
context:
space:
mode:
authorTathagata Das <tathagata.das1565@gmail.com>2014-04-24 18:18:22 -0700
committerTathagata Das <tathagata.das1565@gmail.com>2014-04-24 18:18:22 -0700
commit526a518bf32ad55b926a26f16086f445fd0ae29f (patch)
treedc4bcf8aa155aae8fa5e5bdeb40b47c423745b9d /streaming/src/main
parent35e3d199f04fba3230625002a458d43b9578b2e8 (diff)
downloadspark-526a518bf32ad55b926a26f16086f445fd0ae29f.tar.gz
spark-526a518bf32ad55b926a26f16086f445fd0ae29f.tar.bz2
spark-526a518bf32ad55b926a26f16086f445fd0ae29f.zip
[SPARK-1592][streaming] Automatically remove streaming input blocks
The raw input data is stored as blocks in BlockManagers. Earlier they were cleared by cleaner ttl. Now since streaming does not require cleaner TTL to be set, the block would not get cleared. This increases up the Spark's memory usage, which is not even accounted and shown in the Spark storage UI. It may cause the data blocks to spill over to disk, which eventually slows down the receiving of data (persisting to memory become bottlenecked by writing to disk). The solution in this PR is to automatically remove those blocks. The mechanism to keep track of which BlockRDDs (which has presents the raw data blocks as a RDD) can be safely cleared already exists. Just use it to explicitly remove blocks from BlockRDDs. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #512 from tdas/block-rdd-unpersist and squashes the following commits: d25e610 [Tathagata Das] Merge remote-tracking branch 'apache/master' into block-rdd-unpersist 5f46d69 [Tathagata Das] Merge remote-tracking branch 'apache/master' into block-rdd-unpersist 2c320cd [Tathagata Das] Updated configuration with spark.streaming.unpersist setting. 2d4b2fd [Tathagata Das] Automatically removed input blocks
Diffstat (limited to 'streaming/src/main')
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/Time.scala2
-rw-r--r--streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala16
2 files changed, 14 insertions, 4 deletions
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Time.scala b/streaming/src/main/scala/org/apache/spark/streaming/Time.scala
index 6a6b00a778..37b3b28fa0 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/Time.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/Time.scala
@@ -68,5 +68,5 @@ case class Time(private val millis: Long) {
}
object Time {
- val ordering = Ordering.by((time: Time) => time.millis)
+ implicit val ordering = Ordering.by((time: Time) => time.millis)
}
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala
index d393cc03cb..f69f69e0c4 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala
@@ -25,7 +25,7 @@ import scala.reflect.ClassTag
import java.io.{IOException, ObjectInputStream, ObjectOutputStream}
import org.apache.spark.Logging
-import org.apache.spark.rdd.RDD
+import org.apache.spark.rdd.{BlockRDD, RDD}
import org.apache.spark.storage.StorageLevel
import org.apache.spark.util.MetadataCleaner
import org.apache.spark.streaming._
@@ -340,13 +340,23 @@ abstract class DStream[T: ClassTag] (
* this to clear their own metadata along with the generated RDDs.
*/
private[streaming] def clearMetadata(time: Time) {
+ val unpersistData = ssc.conf.getBoolean("spark.streaming.unpersist", true)
val oldRDDs = generatedRDDs.filter(_._1 <= (time - rememberDuration))
logDebug("Clearing references to old RDDs: [" +
oldRDDs.map(x => s"${x._1} -> ${x._2.id}").mkString(", ") + "]")
generatedRDDs --= oldRDDs.keys
- if (ssc.conf.getBoolean("spark.streaming.unpersist", false)) {
+ if (unpersistData) {
logDebug("Unpersisting old RDDs: " + oldRDDs.values.map(_.id).mkString(", "))
- oldRDDs.values.foreach(_.unpersist(false))
+ oldRDDs.values.foreach { rdd =>
+ rdd.unpersist(false)
+ // Explicitly remove blocks of BlockRDD
+ rdd match {
+ case b: BlockRDD[_] =>
+ logInfo("Removing blocks of RDD " + b + " of time " + time)
+ b.removeBlocks()
+ case _ =>
+ }
+ }
}
logDebug("Cleared " + oldRDDs.size + " RDDs that were older than " +
(time - rememberDuration) + ": " + oldRDDs.keys.mkString(", "))