Description
This is a regression of SPARK-2261. In branch-1.3 and master, EventLoggingListener throws "java.io.IOException: Filesystem closed" when ctrl+c or ctrl+d the spark-sql shell.
The root cause is that DFSClient is already shut down before EventLoggingListener invokes the following HDFS methods, and thus, DFSClient.isClientRunning() check fails-
Line #135: hadoopDataStream.foreach(hadoopFlushMethod.invoke(_))
Line #187: if (fileSystem.exists(target)) {
The followings are full stack trace-
java.lang.reflect.InvocationTargetException at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:606) at org.apache.spark.scheduler.EventLoggingListener$$anonfun$logEvent$3.apply(EventLoggingListener.scala:135) at org.apache.spark.scheduler.EventLoggingListener$$anonfun$logEvent$3.apply(EventLoggingListener.scala:135) at scala.Option.foreach(Option.scala:236) at org.apache.spark.scheduler.EventLoggingListener.logEvent(EventLoggingListener.scala:135) at org.apache.spark.scheduler.EventLoggingListener.onApplicationEnd(EventLoggingListener.scala:170) at org.apache.spark.scheduler.SparkListenerBus$class.onPostEvent(SparkListenerBus.scala:54) at org.apache.spark.scheduler.LiveListenerBus.onPostEvent(LiveListenerBus.scala:31) at org.apache.spark.scheduler.LiveListenerBus.onPostEvent(LiveListenerBus.scala:31) at org.apache.spark.util.ListenerBus$class.postToAll(ListenerBus.scala:53) at org.apache.spark.util.AsynchronousListenerBus.postToAll(AsynchronousListenerBus.scala:36) at org.apache.spark.util.AsynchronousListenerBus$$anon$1$$anonfun$run$1.apply$mcV$sp(AsynchronousListenerBus.scala:76) at org.apache.spark.util.AsynchronousListenerBus$$anon$1$$anonfun$run$1.apply(AsynchronousListenerBus.scala:61) at org.apache.spark.util.AsynchronousListenerBus$$anon$1$$anonfun$run$1.apply(AsynchronousListenerBus.scala:61) at org.apache.spark.util.Utils$.logUncaughtExceptions(Utils.scala:1613) at org.apache.spark.util.AsynchronousListenerBus$$anon$1.run(AsynchronousListenerBus.scala:60) Caused by: java.io.IOException: Filesystem closed at org.apache.hadoop.hdfs.DFSClient.checkOpen(DFSClient.java:707) at org.apache.hadoop.hdfs.DFSOutputStream.flushOrSync(DFSOutputStream.java:1843) at org.apache.hadoop.hdfs.DFSOutputStream.hflush(DFSOutputStream.java:1804) at org.apache.hadoop.fs.FSDataOutputStream.hflush(FSDataOutputStream.java:127) ... 19 more
Exception in thread "Thread-3" java.io.IOException: Filesystem closed
at org.apache.hadoop.hdfs.DFSClient.checkOpen(DFSClient.java:707)
at org.apache.hadoop.hdfs.DFSClient.getFileInfo(DFSClient.java:1760)
at org.apache.hadoop.hdfs.DistributedFileSystem$17.doCall(DistributedFileSystem.java:1124)
at org.apache.hadoop.hdfs.DistributedFileSystem$17.doCall(DistributedFileSystem.java:1120)
at org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
at org.apache.hadoop.hdfs.DistributedFileSystem.getFileStatus(DistributedFileSystem.java:1120)
at org.apache.hadoop.fs.FileSystem.exists(FileSystem.java:1398)
at org.apache.spark.scheduler.EventLoggingListener.stop(EventLoggingListener.scala:187)
at org.apache.spark.SparkContext$$anonfun$stop$4.apply(SparkContext.scala:1379)
at org.apache.spark.SparkContext$$anonfun$stop$4.apply(SparkContext.scala:1379)
at scala.Option.foreach(Option.scala:236)
at org.apache.spark.SparkContext.stop(SparkContext.scala:1379)
at org.apache.spark.sql.hive.thriftserver.SparkSQLEnv$.stop(SparkSQLEnv.scala:66)
at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$$anon$1.run(SparkSQLCLIDriver.scala:107)
Attachments
Issue Links
- is duplicated by
-
SPARK-7865 Hadoop Filesystem for eventlog closed before sparkContext stopped
- Resolved
-
SPARK-1304 Job fails with spot instances (due to IllegalStateException: Shutdown in progress)
- Resolved
-
SPARK-6445 IOException: Filesystem closed is thrown while existing spark-sql console
- Resolved
-
SPARK-10358 Spark-sql throws IOException on exit when using HDFS to store event log.
- Resolved
- is related to
-
SPARK-25183 Spark HiveServer2 registers shutdown hook with JVM, not ShutdownHookManager; race conditions can arise
- Resolved
- links to