-
Notifications
You must be signed in to change notification settings - Fork 382
Fine grain control of specifying ttl #914
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -46,7 +46,7 @@ private[spark] class BlockManager( | |
| maxMemory: Long) | ||
| extends Logging { | ||
|
|
||
| private class BlockInfo(val level: StorageLevel, val tellMaster: Boolean) { | ||
| private class BlockInfo(val level: StorageLevel, val tellMaster: Boolean, val broadcastBlock: Boolean) { | ||
| @volatile var pending: Boolean = true | ||
| @volatile var size: Long = -1L | ||
| @volatile var initThread: Thread = null | ||
|
|
@@ -154,7 +154,8 @@ private[spark] class BlockManager( | |
|
|
||
| var heartBeatTask: Cancellable = null | ||
|
|
||
| val metadataCleaner = new MetadataCleaner("BlockManager", this.dropOldBlocks) | ||
| private val metadataCleaner = new MetadataCleaner(MetadataCleanerType.BLOCK_MANAGER, "BlockManager", this.dropOldNonBroadcastBlocks) | ||
| private val broadcastCleaner = new MetadataCleaner(MetadataCleanerType.BROADCAST_VARS, "BroadcastVars", this.dropOldBroadcastBlocks) | ||
| initialize() | ||
|
|
||
| // The compression codec to use. Note that the "lazy" val is necessary because we want to delay | ||
|
|
@@ -522,11 +523,14 @@ private[spark] class BlockManager( | |
| iter | ||
| } | ||
|
|
||
| def put(blockId: String, values: Iterator[Any], level: StorageLevel, tellMaster: Boolean) | ||
| : Long = { | ||
| def put(blockId: String, values: Iterator[Any], level: StorageLevel, tellMaster: Boolean): Long = { | ||
| put(blockId, values, level, tellMaster, false) | ||
| } | ||
|
|
||
| def put(blockId: String, values: Iterator[Any], level: StorageLevel, tellMaster: Boolean, broadcastBlock: Boolean): Long = { | ||
| val elements = new ArrayBuffer[Any] | ||
| elements ++= values | ||
| put(blockId, elements, level, tellMaster) | ||
| put(blockId, elements, level, tellMaster, broadcastBlock) | ||
| } | ||
|
|
||
| /** | ||
|
|
@@ -538,7 +542,7 @@ private[spark] class BlockManager( | |
| : BlockObjectWriter = { | ||
| val writer = diskStore.getBlockWriter(blockId, serializer, bufferSize) | ||
| writer.registerCloseEventHandler(() => { | ||
| val myInfo = new BlockInfo(StorageLevel.DISK_ONLY, false) | ||
| val myInfo = new BlockInfo(StorageLevel.DISK_ONLY, false, false) | ||
| blockInfo.put(blockId, myInfo) | ||
| myInfo.markReady(writer.size()) | ||
| }) | ||
|
|
@@ -548,9 +552,12 @@ private[spark] class BlockManager( | |
| /** | ||
| * Put a new block of values to the block manager. Returns its (estimated) size in bytes. | ||
| */ | ||
| def put(blockId: String, values: ArrayBuffer[Any], level: StorageLevel, | ||
| tellMaster: Boolean = true) : Long = { | ||
| def put(blockId: String, values: ArrayBuffer[Any], level: StorageLevel, tellMaster: Boolean = true) : Long = { | ||
|
|
||
| put(blockId, values, level, tellMaster, false) | ||
| } | ||
|
|
||
| def put(blockId: String, values: ArrayBuffer[Any], level: StorageLevel, tellMaster: Boolean, broadcastBlock: Boolean) : Long = { | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. So far we've been using the blockID instead of a flag to determine whether a block is a broadcast, an RDD, or something else. Please do it that way here instead of adding a new boolean flag just for broadcasts. In the cleanup part you can just check if the block ID starts with "broadcast_". |
||
| if (blockId == null) { | ||
| throw new IllegalArgumentException("Block Id is null") | ||
| } | ||
|
|
@@ -565,7 +572,7 @@ private[spark] class BlockManager( | |
| // to be dropped right after it got put into memory. Note, however, that other threads will | ||
| // not be able to get() this block until we call markReady on its BlockInfo. | ||
| val myInfo = { | ||
| val tinfo = new BlockInfo(level, tellMaster) | ||
| val tinfo = new BlockInfo(level, tellMaster, broadcastBlock) | ||
| // Do atomically ! | ||
| val oldBlockOpt = blockInfo.putIfAbsent(blockId, tinfo) | ||
|
|
||
|
|
@@ -667,9 +674,12 @@ private[spark] class BlockManager( | |
| /** | ||
| * Put a new block of serialized bytes to the block manager. | ||
| */ | ||
| def putBytes( | ||
| blockId: String, bytes: ByteBuffer, level: StorageLevel, tellMaster: Boolean = true) { | ||
| def putBytes(blockId: String, bytes: ByteBuffer, level: StorageLevel, tellMaster: Boolean = true) { | ||
|
|
||
| putBytes(blockId, bytes, level, tellMaster, false) | ||
| } | ||
|
|
||
| def putBytes(blockId: String, bytes: ByteBuffer, level: StorageLevel, tellMaster: Boolean, broadcastBlock: Boolean) { | ||
| if (blockId == null) { | ||
| throw new IllegalArgumentException("Block Id is null") | ||
| } | ||
|
|
@@ -684,7 +694,7 @@ private[spark] class BlockManager( | |
| // to be dropped right after it got put into memory. Note, however, that other threads will | ||
| // not be able to get() this block until we call markReady on its BlockInfo. | ||
| val myInfo = { | ||
| val tinfo = new BlockInfo(level, tellMaster) | ||
| val tinfo = new BlockInfo(level, tellMaster, broadcastBlock) | ||
| // Do atomically ! | ||
| val oldBlockOpt = blockInfo.putIfAbsent(blockId, tinfo) | ||
|
|
||
|
|
@@ -800,7 +810,11 @@ private[spark] class BlockManager( | |
| * Write a block consisting of a single object. | ||
| */ | ||
| def putSingle(blockId: String, value: Any, level: StorageLevel, tellMaster: Boolean = true) { | ||
| put(blockId, Iterator(value), level, tellMaster) | ||
| putSingle(blockId, value, level, tellMaster, false) | ||
| } | ||
|
|
||
| def putSingle(blockId: String, value: Any, level: StorageLevel, tellMaster: Boolean, broadcastVar: Boolean) { | ||
| put(blockId, Iterator(value), level, tellMaster, broadcastVar) | ||
| } | ||
|
|
||
| /** | ||
|
|
@@ -886,13 +900,36 @@ private[spark] class BlockManager( | |
| } | ||
| } | ||
|
|
||
| def dropOldBlocks(cleanupTime: Long) { | ||
| logInfo("Dropping blocks older than " + cleanupTime) | ||
| def dropOldNonBroadcastBlocks(cleanupTime: Long) { | ||
| logInfo("Dropping non broadcast blocks older than " + cleanupTime) | ||
| val iterator = blockInfo.internalMap.entrySet().iterator() | ||
| while (iterator.hasNext) { | ||
| val entry = iterator.next() | ||
| val (id, info, time) = (entry.getKey, entry.getValue._1, entry.getValue._2) | ||
| if (time < cleanupTime && ! info.broadcastBlock) { | ||
| info.synchronized { | ||
| val level = info.level | ||
| if (level.useMemory) { | ||
| memoryStore.remove(id) | ||
| } | ||
| if (level.useDisk) { | ||
| diskStore.remove(id) | ||
| } | ||
| iterator.remove() | ||
| logInfo("Dropped block " + id) | ||
| } | ||
| reportBlockStatus(id, info) | ||
| } | ||
| } | ||
| } | ||
|
|
||
| def dropOldBroadcastBlocks(cleanupTime: Long) { | ||
| logInfo("Dropping broaddcast blocks older than " + cleanupTime) | ||
| val iterator = blockInfo.internalMap.entrySet().iterator() | ||
| while (iterator.hasNext) { | ||
| val entry = iterator.next() | ||
| val (id, info, time) = (entry.getKey, entry.getValue._1, entry.getValue._2) | ||
| if (time < cleanupTime) { | ||
| if (time < cleanupTime && info.broadcastBlock) { | ||
| info.synchronized { | ||
| val level = info.level | ||
| if (level.useMemory) { | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -20,13 +20,14 @@ package org.apache.spark.util | |
| import java.util.concurrent.{TimeUnit, ScheduledFuture, Executors} | ||
| import java.util.{TimerTask, Timer} | ||
| import org.apache.spark.Logging | ||
| import scala.collection.mutable.HashMap | ||
|
|
||
|
|
||
| /** | ||
| * Runs a timer task to periodically clean up metadata (e.g. old files or hashtable entries) | ||
| */ | ||
| class MetadataCleaner(name: String, cleanupFunc: (Long) => Unit) extends Logging { | ||
| private val delaySeconds = MetadataCleaner.getDelaySeconds | ||
| class MetadataCleaner(cleanerType: MetadataCleanerType.MetadataCleanerType, name: String, cleanupFunc: (Long) => Unit) extends Logging { | ||
| private val delaySeconds = MetadataCleaner.getDelaySeconds(cleanerType) | ||
| private val periodSeconds = math.max(10, delaySeconds / 10) | ||
| private val timer = new Timer(name + " cleanup timer", true) | ||
|
|
||
|
|
@@ -53,9 +54,39 @@ class MetadataCleaner(name: String, cleanupFunc: (Long) => Unit) extends Logging | |
| } | ||
| } | ||
|
|
||
| object MetadataCleanerType extends Enumeration("map_output_tracker", "spark_context", "http_broadcast", "dag_scheduler", "result_task", | ||
| "shuffle_map_task", "block_manager", "broadcast_vars") { | ||
|
|
||
| val MAP_OUTPUT_TRACKER, SPARK_CONTEXT, HTTP_BROADCAST, DAG_SCHEDULER, RESULT_TASK, SHUFFLE_MAP_TASK, BLOCK_MANAGER, BROADCAST_VARS = Value | ||
|
|
||
| type MetadataCleanerType = Value | ||
|
|
||
| def systemProperty(which: MetadataCleanerType.MetadataCleanerType) = "spark.cleaner.ttl." + which.toString | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Avoid underscores in the system properties -- use camelCase to match our other ones. I'd also be okay if these were full class names (MapOutputTracker, SparkContext, etc). |
||
| } | ||
|
|
||
|
|
||
| object MetadataCleaner { | ||
|
|
||
|
|
||
| // using only sys props for now : so that workers can also get to it while preserving earlier behavior. | ||
| def getDelaySeconds = System.getProperty("spark.cleaner.ttl", "-1").toInt | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Let's call this getDefaultDelaySeconds or something like that; I'm not sure it gets called directly anymore after this change |
||
| def setDelaySeconds(delay: Int) { System.setProperty("spark.cleaner.ttl", delay.toString) } | ||
|
|
||
| def getDelaySeconds(cleanerType: MetadataCleanerType.MetadataCleanerType): Int = { | ||
| System.getProperty(MetadataCleanerType.systemProperty(cleanerType), getDelaySeconds.toString).toInt | ||
| } | ||
|
|
||
| def setDelaySeconds(cleanerType: MetadataCleanerType.MetadataCleanerType, delay: Int) { | ||
| System.setProperty(MetadataCleanerType.systemProperty(cleanerType), delay.toString) | ||
| } | ||
|
|
||
| def setDelaySeconds(delay: Int, resetAll: Boolean = true) { | ||
| // override for all ? | ||
| System.setProperty("spark.cleaner.ttl", delay.toString) | ||
| if (resetAll) { | ||
| for (cleanerType <- MetadataCleanerType.values) { | ||
| System.clearProperty(MetadataCleanerType.systemProperty(cleanerType)) | ||
| } | ||
| } | ||
| } | ||
| } | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I don't like that these take both an enum type and a string name as arguments -- let's have them take just the enum.