JoshRosen opened a new pull request, #37282: URL: https://github.com/apache/spark/pull/37282
### What changes were proposed in this pull request? This PR modifies `ExecutionListenerManager` so that its `ExecutionListenerBus` SparkListener is lazily registered during the first `.register(QueryExceutionListener)` (instead of eagerly registering it in the constructor). ### Why are the changes needed? This addresses a ListenerBus performance problem in applications with large numbers of short-lived SparkSessions. The `ExecutionListenerBus` SparkListener is unregistered by the ContextCleaner after its associated ExecutionListenerManager/SparkSession is garbage-collected (see #31839). If many sessions are rapidly created and destroyed but the driver GC doesn't run then this can result in large number of unused ExecutionListenerBus listeners being registered on the shared ListenerBus queue. This can cause performance problems in the ListenerBus because each listener invocation has some overhead. In one real-world application with a very large driver heap and high rate of SparkSession creation (hundreds per minute), I saw 5000 idle ExecutionListenerBus listeners, resulting in ~50ms median event processing times on the shared listener queue. This patch avoids this problem by making the listener registration lazy: if a short-lived SparkSession never uses QueryExecutionListeners then we won't register the ExecutionListenerBus and won't incur these overheads. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Added a new unit test. -- 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: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
