@@ -19,28 +19,27 @@ package org.apache.spark.storage
1919
2020import java .io .DataInputStream
2121import java .nio .ByteBuffer
22- import java .util .concurrent .{ BlockingQueue , LinkedBlockingQueue , TimeUnit }
22+ import java .util .concurrent .TimeUnit
2323import java .util .concurrent .atomic .AtomicBoolean
2424
25- import scala .concurrent .Future
25+ import scala .concurrent .{ ExecutionContext , Future }
2626import scala .reflect .ClassTag
2727
2828import org .apache .hadoop .conf .Configuration
2929import org .apache .hadoop .fs .{FileSystem , Path }
3030
31- import org .apache .spark .{SparkConf , SparkException }
31+ import org .apache .spark .{SparkConf , SparkEnv , SparkException }
3232import org .apache .spark .deploy .SparkHadoopUtil
3333import org .apache .spark .internal .Logging
3434import org .apache .spark .internal .LogKeys ._
35- import org .apache .spark .internal .config .{STORAGE_DECOMMISSION_FALLBACK_STORAGE_CLEANUP , STORAGE_DECOMMISSION_FALLBACK_STORAGE_CLEANUP_WAIT_ON_SHUTDOWN , STORAGE_DECOMMISSION_FALLBACK_STORAGE_PATH }
35+ import org .apache .spark .internal .config .{STORAGE_DECOMMISSION_FALLBACK_STORAGE_CLEANUP , STORAGE_DECOMMISSION_FALLBACK_STORAGE_CLEANUP_THREADS , STORAGE_DECOMMISSION_FALLBACK_STORAGE_CLEANUP_WAIT_ON_SHUTDOWN , STORAGE_DECOMMISSION_FALLBACK_STORAGE_PATH }
3636import org .apache .spark .network .buffer .{ManagedBuffer , NioManagedBuffer }
3737import org .apache .spark .network .util .JavaUtils
3838import org .apache .spark .rpc .{RpcAddress , RpcEndpointRef , RpcTimeout }
3939import org .apache .spark .shuffle .{IndexShuffleBlockResolver , ShuffleBlockInfo }
4040import org .apache .spark .shuffle .IndexShuffleBlockResolver .NOOP_REDUCE_ID
4141import org .apache .spark .storage .BlockManagerMessages .RemoveShuffle
42- import org .apache .spark .storage .FallbackStorage .{cleanupShufflesThread , stopped }
43- import org .apache .spark .util .{ShutdownHookManager , Utils }
42+ import org .apache .spark .util .{ShutdownHookManager , ThreadUtils , Utils }
4443
4544/**
4645 * A fallback storage used by storage decommissioners.
@@ -54,18 +53,6 @@ private[storage] class FallbackStorage(conf: SparkConf) extends Logging {
5453 private val fallbackFileSystem = FileSystem .get(fallbackPath.toUri, hadoopConf)
5554 private val appId = conf.getAppId
5655
57- // Ensure cleanup work only blocks Spark shutdown when configured so
58- ShutdownHookManager .addShutdownHook { () =>
59- // indicate cleanup thread to shut down once queue is drained
60- stopped.set(true )
61-
62- // only wait for cleanup thread to finish when configured so
63- // thread is set daemon so JVM can shut down while it is running
64- if (conf.get(STORAGE_DECOMMISSION_FALLBACK_STORAGE_CLEANUP_WAIT_ON_SHUTDOWN )) {
65- cleanupShufflesThread.join()
66- }
67- }
68-
6956 // Visible for testing
7057 def copy (
7158 shuffleBlockInfo : ShuffleBlockInfo ,
@@ -139,25 +126,30 @@ private[spark] object FallbackStorage extends Logging {
139126
140127 private val stopped = new AtomicBoolean (false )
141128
142- // a queue of shuffle cleanup requests and a daemon thread processing them
143- private val cleanupShufflesQueue : BlockingQueue [CleanUp ] = new LinkedBlockingQueue [CleanUp ]()
144- private val cleanupShufflesThread : Thread = new Thread (new Runnable {
145- override def run (): Unit = {
146- // if stopped and queue is empty, this thread terminates
147- while (! stopped.get() || ! cleanupShufflesQueue.isEmpty) {
148- Utils .tryLogNonFatalError {
149- // wait a second for another cleanup request, then check while condition
150- val cleanup = Option (cleanupShufflesQueue.poll(1L , TimeUnit .SECONDS ))
151- cleanup.foreach { c => cleanUp(c.conf, c.hadoopConf, c.shuffleId) }
152- }
153- }
129+ // a daemon thread pool for shuffle cleanups
130+ private val cleanupShufflesNumThreads =
131+ SparkEnv .get.conf.get(STORAGE_DECOMMISSION_FALLBACK_STORAGE_CLEANUP_THREADS )
132+ private val cleanupShufflesThreadPool =
133+ ThreadUtils .newDaemonFixedThreadPool(cleanupShufflesNumThreads, " fallback-storage-cleanup" )
134+ private val cleanupShufflesExecutionContext =
135+ ExecutionContext .fromExecutor(cleanupShufflesThreadPool)
136+
137+ // Ensure cleanup work only blocks Spark shutdown when configured so
138+ private val cleanupShufflesWaitOnShutdown =
139+ SparkEnv .get.conf.get(STORAGE_DECOMMISSION_FALLBACK_STORAGE_CLEANUP_WAIT_ON_SHUTDOWN )
140+
141+ ShutdownHookManager .addShutdownHook { () =>
142+ // indicate the cleanup thread to terminate once the queue is drained
143+ stopped.set(true )
144+
145+ // only wait for cleanups to finish when configured so
146+ if (cleanupShufflesWaitOnShutdown) {
147+ cleanupShufflesThreadPool.shutdown()
148+ while (! cleanupShufflesThreadPool.awaitTermination(1L , TimeUnit .SECONDS )) {}
149+ } else {
150+ cleanupShufflesThreadPool.shutdownNow()
154151 }
155- }, " fallback-storage-cleanup" )
156- // this is a daemon thread so JVM shutdown is not blocked by this thread running
157- // we block ShutdownHook above if STORAGE_DECOMMISSION_FALLBACK_STORAGE_CLEANUP_WAIT_ON_SHUTDOWN
158- // is true
159- cleanupShufflesThread.setDaemon(true )
160- cleanupShufflesThread.start()
152+ }
161153
162154 def getFallbackStorage (conf : SparkConf ): Option [FallbackStorage ] = {
163155 if (conf.get(STORAGE_DECOMMISSION_FALLBACK_STORAGE_PATH ).isDefined) {
@@ -187,7 +179,7 @@ private[spark] object FallbackStorage extends Logging {
187179 if (stopped.get()) {
188180 logInfo(" Not queueing cleanup due to shutdown" )
189181 } else {
190- cleanupShufflesQueue.put( CleanUp ( conf, hadoopConf, shuffleId))
182+ Future { cleanUp( conf, hadoopConf, shuffleId) }(cleanupShufflesExecutionContext )
191183 }
192184 }
193185
0 commit comments