joonaspessi opened a new issue, #2465: URL: https://github.com/apache/sedona/issues/2465
# Log4j Class Initialization Deadlock in KryoSedonaRegistrator Class ## Problem Description I'm experiencing occasional deadlocks in AWS EMR 7.8.0 Spark clusters that cause executors to hang indefinitely during JVM initialization. After investigation with Java thread dumps, I have identified that the static logger initialization in `SedonaKryoRegistrator` is one of the contributing factors to this deadlock. This deadlock seems to happen extremely rarely but still there seems to be possibility for it as I have seen it now with multiple Spark clusters. ## Environment - **Platform:** AWS EMR 7.8.0 - **Spark Version:** 3.5.4-amzn-0 - **Sedona Version:** 1.7.0 - **Log4j Version:** log4j-1.2-api-2.20.0 (bundled and delivered with spark 3.5.x) - **Cluster Configuration:** Spark cluster with 500 Executors where each executor have 4 cores. (So there are quite many JVM instances where this problem can occur) - **Java version:** Amazon Corretto 17 - **Architecture:** ARM64/Graviton ## What Happens The deadlock occurs during Spark executor JVM startup due to a race condition between multiple threads concurrently initializing Log4j classes (log4j-1.2-api-2.20.0). The circular class initialization dependency chain is: ``` Priority → Level (requires Level class) Level extends Priority (requires Priority class) LogManager → Level.DEBUG (requires both Level and Priority) ``` ### Thread Dump Analysis When the deadlock occurs, we observe two threads stuck: **Thread # 59: "Executor task launch worker for task 381.0 in stage 3.0 (TID 383)"** ``` java.lang.Thread.State: RUNNABLE at org.apache.log4j.LogManager.<clinit>(LogManager.java:72) - waiting on the Class initialization monitor for org.apache.log4j.Priority at org.apache.log4j.Logger.getLogger(Logger.java:35) at org.apache.sedona.core.serde.SedonaKryoRegistrator.<clinit>(SedonaKryoRegistrator.java:40) at java.lang.Class.forName0([email protected]/Native Method) at java.lang.Class.forName([email protected]/Class.java:467) at org.apache.spark.util.SparkClassUtils.classForName(SparkClassUtils.scala:43) at org.apache.spark.util.Utils$.classForName(Utils.scala:96) at org.apache.spark.serializer.KryoSerializer.$anonfun$newKryo$7(KryoSerializer.scala:182) at scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:286) at scala.collection.mutable.ResizableArray.foreach(ResizableArray.scala:62) at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:49) at scala.collection.TraversableLike.map(TraversableLike.scala:286) at org.apache.spark.serializer.KryoSerializer.$anonfun$newKryo$5(KryoSerializer.scala:182) at org.apache.spark.util.Utils$.withContextClassLoader(Utils.scala:180) at org.apache.spark.serializer.KryoSerializer.newKryo(KryoSerializer.scala:175) at org.apache.spark.serializer.KryoSerializer$$anon$1.create(KryoSerializer.scala:106) at com.esotericsoftware.kryo.pool.KryoPoolQueueImpl.borrow(KryoPoolQueueImpl.java:48) at org.apache.spark.serializer.KryoSerializerInstance.borrowKryo(KryoSerializer.scala:416) at org.apache.spark.serializer.KryoDeserializationStream.<init>(KryoSerializer.scala:312) at org.apache.spark.serializer.KryoSerializerInstance.deserializeStream(KryoSerializer.scala:506) at org.apache.spark.broadcast.TorrentBroadcast$.unBlockifyObject(TorrentBroadcast.scala:420) at org.apache.spark.broadcast.TorrentBroadcast.readBroadcastBlock(TorrentBroadcast.scala:279) at org.apache.spark.broadcast.TorrentBroadcast.getValue(TorrentBroadcast.scala:125) at org.apache.spark.broadcast.Broadcast.value(Broadcast.scala:77) at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:91) at org.apache.spark.scheduler.Task.run(Task.scala:152) at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:635) at java.util.concurrent.ThreadPoolExecutor.runWorker([email protected]/ThreadPoolExecutor.java:1136) at java.util.concurrent.ThreadPoolExecutor$Worker.run([email protected]/ThreadPoolExecutor.java:635) at java.lang.Thread.run([email protected]/Thread.java:840) ``` **Thread # 51: "pool-10-thread-4"** ``` java.lang.Thread.State: RUNNABLE at org.apache.log4j.Priority.<clinit>(Priority.java:67) - waiting on the Class initialization monitor for org.apache.log4j.Level at java.lang.Class.forName0([email protected]/Native Method) at java.lang.Class.forName([email protected]/Class.java:375) at software.amazon.awssdk.thirdparty.org.apache.commons.logging.impl.Log4JLogger.class$(Log4JLogger.java:53) at software.amazon.awssdk.thirdparty.org.apache.commons.logging.impl.Log4JLogger.<clinit>(Log4JLogger.java:78) at java.lang.Class.forName0([email protected]/Native Method) at java.lang.Class.forName([email protected]/Class.java:467) at software.amazon.awssdk.thirdparty.org.apache.commons.logging.impl.LogFactoryImpl.createLogFromClass(LogFactoryImpl.java:998) at software.amazon.awssdk.thirdparty.org.apache.commons.logging.impl.LogFactoryImpl.discoverLogImplementation(LogFactoryImpl.java:844) at software.amazon.awssdk.thirdparty.org.apache.commons.logging.impl.LogFactoryImpl.newInstance(LogFactoryImpl.java:541) at software.amazon.awssdk.thirdparty.org.apache.commons.logging.impl.LogFactoryImpl.getInstance(LogFactoryImpl.java:292) at software.amazon.awssdk.thirdparty.org.apache.commons.logging.LogFactory.getLog(LogFactory.java:655) at software.amazon.awssdk.thirdparty.org.apache.http.conn.ssl.AbstractVerifier.<init>(AbstractVerifier.java:61) at software.amazon.awssdk.thirdparty.org.apache.http.conn.ssl.AllowAllHostnameVerifier.<init>(AllowAllHostnameVerifier.java:44) at software.amazon.awssdk.thirdparty.org.apache.http.conn.ssl.SSLConnectionSocketFactory.<clinit>(SSLConnectionSocketFactory.java:151) at org.apache.hadoop.fs.s3a.impl.ConfigureShadedAWSSocketFactory.configureSocketFactory(ConfigureShadedAWSSocketFactory.java:42) at org.apache.hadoop.fs.s3a.impl.NetworkBinding.bindSSLChannelMode(NetworkBinding.java:91) at org.apache.hadoop.fs.s3a.impl.AWSClientConfig.createHttpClientBuilder(AWSClientConfig.java:165) at org.apache.hadoop.fs.s3a.DefaultS3ClientFactory.createS3Client(DefaultS3ClientFactory.java:140) at org.apache.hadoop.fs.s3a.impl.ClientManagerImpl.lambda$createS3Client$0(ClientManagerImpl.java:133) at org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.invokeTrackingDuration(IOStatisticsBinding.java:547) at org.apache.hadoop.util.functional.LazyAtomicReference.eval(LazyAtomicReference.java:94) at org.apache.hadoop.util.functional.LazyAutoCloseableReference.eval(LazyAutoCloseableReference.java:54) at org.apache.hadoop.fs.s3a.impl.ClientManagerImpl.getOrCreateS3Client(ClientManagerImpl.java:174) at org.apache.hadoop.fs.s3a.impl.S3AStoreImpl.getOrCreateS3Client(S3AStoreImpl.java:244) at org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:868) at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:3670) at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:3771) at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:3722) at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:564) at org.apache.hadoop.fs.Path.getFileSystem(Path.java:374) at org.apache.spark.executor.CoarseGrainedExecutorBackend.$anonfun$eagerFSInit$1(CoarseGrainedExecutorBackend.scala:359) at scala.collection.parallel.mutable.ParArray$ParArrayIterator.flatmap2combiner(ParArray.scala:419) at scala.collection.parallel.ParIterableLike$FlatMap.leaf(ParIterableLike.scala:1082) at scala.collection.parallel.Task.$anonfun$tryLeaf$1(Tasks.scala:53) at scala.util.control.Breaks$$anon$1.catchBreak(Breaks.scala:67) at scala.collection.parallel.Task.tryLeaf(Tasks.scala:56) at scala.collection.parallel.FutureTasks.$anonfun$exec$5(Tasks.scala:499) at scala.concurrent.Future$.$anonfun$apply$1(Future.scala:659) at scala.util.Success.$anonfun$map$1(Try.scala:255) at scala.util.Success.map(Try.scala:213) at scala.concurrent.Future.$anonfun$map$1(Future.scala:292) at scala.concurrent.impl.Promise.liftedTree1$1(Promise.scala:33) at scala.concurrent.impl.Promise.$anonfun$transform$1(Promise.scala:33) at scala.concurrent.impl.CallbackRunnable.run(Promise.scala:64) at java.util.concurrent.ThreadPoolExecutor.runWorker([email protected]/ThreadPoolExecutor.java:1136) at java.util.concurrent.ThreadPoolExecutor$Worker.run([email protected]/ThreadPoolExecutor.java:635) at java.lang.Thread.run([email protected]/Thread.java:840) ``` **Result:** Both threads hold locks on different Log4j classes and wait for each other, creating a circular deadlock. ### Why SedonaKryoRegistrator is Involved The Spark framework instantiates `SedonaKryoRegistrator` implementation during initialization setup. During class loading, the static field initialization in `SedonaKryoRegistrator`: ```java static final Logger log = Logger.getLogger(SedonaKryoRegistrator.class); ``` This line triggers Log4j class loading **before** any actual logging occurs. If another thread (Spark eagerFSInit in this case) is simultaneously initializing Log4j classes, the JVM class loading locks can apparently deadlock. ### Affected Classes Both Kryo registrator classes in Sedona have this pattern: 1. `org.apache.sedona.core.serde.SedonaKryoRegistrator` (line 40) - Used by: Main Sedona geometry serialization - Logs: "Registering custom serializers for geometry types" 2. `org.apache.sedona.viz.core.Serde.SedonaVizKryoRegistrator` (line 29) - Used by: Sedona visualization features - Logs: "Registering custom serializers for visualization related types" ## Thinking about the mitigation I propose **removing the static logger initialization and logging all together** from both KryoRegistrator classes, as the logging provides minimal operational value compared to the deadlock risk with rather old log4j2 version that is bundled with spark 3.5.x versions. This is of course not apache/sedona fault/bug itself but still it woul help to reduce risk to end up in this deadlock. Probability that this deadlock materializes seems to be really small. But definitely there's possibility for that, especially when large spark clusters are used. I think that the underlying circular class initialization issue in Log4j has been fixed in newer versions (log4j-1.2-api-2.24.2 and later) through LoggerRegistry improvements refactoring. However, Spark 3.5.x is tested and delivered with log4j-1.2-api-2.20.0 and updating this version would require additional testing and manual maintenance. -- 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]
