ericm-db commented on code in PR #50595: URL: https://github.com/apache/spark/pull/50595#discussion_r2064587027
########## sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala: ########## @@ -1111,60 +1149,52 @@ object StateStore extends Logging { } } + // Block until we can process this partition + private def awaitProcessThisPartition( + id: StateStoreProviderId, + storeConf: StateStoreConf): Boolean = { + maintenanceThreadPoolLock.synchronized { + val timeoutMs = storeConf.stateStoreMaintenanceProcessingTimeout * 1000 + val endTime = System.currentTimeMillis() + timeoutMs + + // Try to process immediately first + if (processThisPartition(id)) return true + + // Wait with timeout and process after notification + def timeRemaining: Long = endTime - System.currentTimeMillis() + + while (timeRemaining > 0) { + maintenanceThreadPoolLock.wait(Math.min(timeRemaining, 10000)) + if (processThisPartition(id)) return true + } + + // Timeout reached without successfully processing the partition + return false + } + } + + private def doMaintenance(): Unit = doMaintenance(StateStoreConf.empty) + /** * Execute background maintenance task in all the loaded store providers if they are still * the active instances according to the coordinator. */ - private def doMaintenance(): Unit = { + private def doMaintenance( + storeConf: StateStoreConf + ): Unit = { logDebug("Doing maintenance") if (SparkEnv.get == null) { throw new IllegalStateException("SparkEnv not active, cannot do maintenance on StateStores") } + + // Process queued providers first + processQueuedProviders(storeConf) + loadedProviders.synchronized { loadedProviders.toSeq }.foreach { case (id, provider) => if (processThisPartition(id)) { - maintenanceThreadPool.execute(() => { - val startTime = System.currentTimeMillis() - try { - provider.doMaintenance() - if (!verifyIfStoreInstanceActive(id)) { - unload(id) - logInfo(log"Unloaded ${MDC(LogKeys.STATE_STORE_PROVIDER, provider)}") - } - } catch { - case NonFatal(e) => - logWarning(log"Error managing ${MDC(LogKeys.STATE_STORE_PROVIDER, provider)}, " + - log"unloading state store provider", e) - // When we get a non-fatal exception, we just unload the provider. - // - // By not bubbling the exception to the maintenance task thread or the query execution - // thread, it's possible for a maintenance thread pool task to continue failing on - // the same partition. Additionally, if there is some global issue that will cause - // all maintenance thread pool tasks to fail, then bubbling the exception and - // stopping the pool is faster than waiting for all tasks to see the same exception. - // - // However, we assume that repeated failures on the same partition and global issues - // are rare. The benefit to unloading just the partition with an exception is that - // transient issues on a given provider do not affect any other providers; so, in - // most cases, this should be a more performant solution. - unload(id) - } finally { - val duration = System.currentTimeMillis() - startTime - val logMsg = - log"Finished maintenance task for " + - log"provider=${MDC(LogKeys.STATE_STORE_PROVIDER_ID, id)}" + - log" in elapsed_time=${MDC(LogKeys.TIME_UNITS, duration)}\n" - if (duration > 5000) { - logInfo(logMsg) - } else { - logDebug(logMsg) - } - maintenanceThreadPoolLock.synchronized { - maintenancePartitions.remove(id) - } - } - }) + submitMaintenanceWorkForProvider(id, provider, storeConf) Review Comment: submit multiple providers -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org