Xing Lin created HDFS-16852:
-------------------------------

             Summary: swallow IllegalStateException in KeyProviderCache
                 Key: HDFS-16852
                 URL: https://issues.apache.org/jira/browse/HDFS-16852
             Project: Hadoop HDFS
          Issue Type: Bug
          Components: hdfs
            Reporter: Xing Lin


When an HDFS client is created, it will register a shutdownhook to 
shutdownHookManager. ShutdownHookManager doesn't allow adding a new 
shutdownHook when the process is already in shutdown and throws an 
IllegalStateException.

This behavior is not ideal, when a spark program failed during pre-launch. In 
that case, during shutdown, spark would call cleanStagingDir() to clean the 
staging dir. In cleanStagingDir(), it will create a FileSystem object to talk 
to HDFS. However, since this would be the first time to use a filesystem object 
in that process, it will need to create an hdfs client and register the 
shutdownHook. Then, we will hit the IllegalStateException. This 
illegalStateException will mask the actual exception which causes the spark 
program to fail during pre-launch.

We propose to swallow IllegalStateException in KeyProviderCache and log a 
warning. The TCP connection between the client and NameNode should be closed by 
the OS when the process is shutdown. 

Example stacktrace
{code:java}
13-09-2022 14:39:42 PDT INFO - 22/09/13 21:39:41 ERROR util.Utils: Uncaught 
exception in thread shutdown-hook-0   
13-09-2022 14:39:42 PDT INFO - java.lang.IllegalStateException: Shutdown in 
progress, cannot add a shutdownHook    
13-09-2022 14:39:42 PDT INFO - at 
org.apache.hadoop.util.ShutdownHookManager.addShutdownHook(ShutdownHookManager.java:299)
          
13-09-2022 14:39:42 PDT INFO - at 
org.apache.hadoop.hdfs.KeyProviderCache.<init>(KeyProviderCache.java:71)        
  
13-09-2022 14:39:42 PDT INFO - at 
org.apache.hadoop.hdfs.ClientContext.<init>(ClientContext.java:130)          
13-09-2022 14:39:42 PDT INFO - at 
org.apache.hadoop.hdfs.ClientContext.get(ClientContext.java:167)          
13-09-2022 14:39:42 PDT INFO - at 
org.apache.hadoop.hdfs.DFSClient.<init>(DFSClient.java:383)          
13-09-2022 14:39:42 PDT INFO - at 
org.apache.hadoop.hdfs.DFSClient.<init>(DFSClient.java:287)          
13-09-2022 14:39:42 PDT INFO - at 
org.apache.hadoop.hdfs.DistributedFileSystem.initialize(DistributedFileSystem.java:159)
          
13-09-2022 14:39:42 PDT INFO - at 
org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:3261)          
13-09-2022 14:39:42 PDT INFO - at 
org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:121)          
13-09-2022 14:39:42 PDT INFO - at 
org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:3310)         
 
13-09-2022 14:39:42 PDT INFO - at 
org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:3278)          
13-09-2022 14:39:42 PDT INFO - at 
org.apache.hadoop.fs.FileSystem.get(FileSystem.java:475)          
13-09-2022 14:39:42 PDT INFO - at 
org.apache.hadoop.fs.Path.getFileSystem(Path.java:356)          
13-09-2022 14:39:42 PDT INFO - at 
org.apache.spark.deploy.yarn.ApplicationMaster.cleanupStagingDir(ApplicationMaster.scala:675)
          
13-09-2022 14:39:42 PDT INFO - at 
org.apache.spark.deploy.yarn.ApplicationMaster.$anonfun$run$2(ApplicationMaster.scala:259)
          
13-09-2022 14:39:42 PDT INFO - at 
org.apache.spark.util.SparkShutdownHook.run(ShutdownHookManager.scala:214)      
    
13-09-2022 14:39:42 PDT INFO - at 
org.apache.spark.util.SparkShutdownHookManager.$anonfun$runAll$2(ShutdownHookManager.scala:188)
          
13-09-2022 14:39:42 PDT INFO - at 
scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)          
13-09-2022 14:39:42 PDT INFO - at 
org.apache.spark.util.Utils$.logUncaughtExceptions(Utils.scala:2023)          
13-09-2022 14:39:42 PDT INFO - at 
org.apache.spark.util.SparkShutdownHookManager.$anonfun$runAll$1(ShutdownHookManager.scala:188)
          
13-09-2022 14:39:42 PDT INFO - at 
scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)          
13-09-2022 14:39:42 PDT INFO - at scala.util.Try$.apply(Try.scala:213)          
13-09-2022 14:39:42 PDT INFO - at 
org.apache.spark.util.SparkShutdownHookManager.runAll(ShutdownHookManager.scala:188)
          
13-09-2022 14:39:42 PDT INFO - at 
org.apache.spark.util.SparkShutdownHookManager$$anon$2.run(ShutdownHookManager.scala:178)
          
13-09-2022 14:39:42 PDT INFO - at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)         
 
13-09-2022 14:39:42 PDT INFO - at 
java.util.concurrent.FutureTask.run(FutureTask.java:266)          
13-09-2022 14:39:42 PDT INFO - at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) 
         
13-09-2022 14:39:42 PDT INFO - at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) 
         
13-09-2022 14:39:42 PDT INFO - at java.lang.Thread.run(Thread.java:748)         
 
13-09-2022 14:39:42 PDT INFO - 22/09/13 21:39:41 INFO util.ShutdownHookManager: 
Shutdown hook called     
 {code}
 



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

---------------------------------------------------------------------
To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org

Reply via email to