[ https://issues.apache.org/jira/browse/SPARK-1407?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13963025#comment-13963025 ]
Thomas Graves commented on SPARK-1407: -------------------------------------- Note that the logging also doesn't work on secure HDFS. Exception in thread "Thread-3" java.lang.reflect.UndeclaredThrowableException: Unknown exception in doAs at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1275) at org.apache.spark.deploy.SparkHadoopUtil.runAsUser(SparkHadoopUtil.scala:42) at org.apache.spark.deploy.yarn.ApplicationMaster$$anon$2.run(ApplicationMaster.scala:192) Caused by: java.security.PrivilegedActionException: java.lang.reflect.InvocationTargetException at java.security.AccessController.doPrivileged(Native Method) at javax.security.auth.Subject.doAs(Subject.java:415) at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1262) ... 2 more Caused by: 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:601) at org.apache.spark.deploy.yarn.ApplicationMaster$$anon$2$$anonfun$run$1.apply$mcV$sp(ApplicationMaster.scala:198) at org.apache.spark.deploy.SparkHadoopUtil$$anon$1.run(SparkHadoopUtil.scala:43) at org.apache.spark.deploy.SparkHadoopUtil$$anon$1.run(SparkHadoopUtil.scala:42) ... 5 more Caused by: java.io.IOException: Can't replace _HOST pattern since client address is null at org.apache.hadoop.security.SecurityUtil.getServerPrincipal(SecurityUtil.java:255) at org.apache.hadoop.ipc.Client$ConnectionId.getRemotePrincipal(Client.java:1326) at org.apache.hadoop.ipc.Client$ConnectionId.getConnectionId(Client.java:1298) at org.apache.hadoop.ipc.WritableRpcEngine$Invoker.<init>(WritableRpcEngine.java:183) at org.apache.hadoop.ipc.WritableRpcEngine.getProxy(WritableRpcEngine.java:236) at org.apache.hadoop.ipc.RPC.getProtocolProxy(RPC.java:441) at org.apache.hadoop.ipc.RPC.getProtocolProxy(RPC.java:387) at org.apache.hadoop.ipc.RPC.getProxy(RPC.java:364) at org.apache.hadoop.hdfs.DFSUtil.createRPCNamenode(DFSUtil.java:642) at org.apache.hadoop.hdfs.DFSClient.<init>(DFSClient.java:346) at org.apache.hadoop.hdfs.DFSClient.<init>(DFSClient.java:319) at org.apache.hadoop.hdfs.DistributedFileSystem.initialize(DistributedFileSystem.java:110) at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:2160) at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:85) at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:2194) at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:2176) at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:306) at org.apache.spark.util.Utils$.getHadoopFileSystem(Utils.scala:1022) at org.apache.spark.util.FileLogger.<init>(FileLogger.scala:51) at org.apache.spark.scheduler.EventLoggingListener.<init>(EventLoggingListener.scala:49) at org.apache.spark.SparkContext.<init>(SparkContext.scala:172) at org.apache.spark.SparkContext.<init>(SparkContext.scala:96) at org.apache.spark.examples.SparkPi$.main(SparkPi.scala:31) at org.apache.spark.examples.SparkPi.main(SparkPi.scala) ... 12 more Exception in thread "main" java.lang.AssertionError: assertion failed at scala.Predef$.assert(Predef.scala:165) at org.apache.spark.deploy.yarn.ApplicationMaster.waitForSparkContextInitialized(ApplicationMaster.scala:234) at org.apache.spark.deploy.yarn.ApplicationMaster.run(ApplicationMaster.scala:107) at org.apache.spark.deploy.yarn.ApplicationMaster$.main(ApplicationMaster.scala:480) at org.apache.spark.deploy.yarn.ApplicationMaster.main(ApplicationMaster.scala) 14/04/08 14:23:03 INFO yarn.ApplicationMaster: AppMaster received a signal. 14/04/08 14:23:03 INFO yarn.ApplicationMaster: Deleting staging directory .sparkStaging/application_1396660776541_63976 > EventLogging to HDFS doesn't work properly on yarn > -------------------------------------------------- > > Key: SPARK-1407 > URL: https://issues.apache.org/jira/browse/SPARK-1407 > Project: Spark > Issue Type: Bug > Components: Spark Core > Affects Versions: 1.0.0 > Reporter: Thomas Graves > Priority: Blocker > > When running on spark on yarn and accessing an HDFS file (like in the > SparkHdfsLR example) while using the event logging configured to write logs > to HDFS, it throws an exception at the end of the application. > SPARK_JAVA_OPTS=-Dspark.eventLog.enabled=true > -Dspark.eventLog.dir=hdfs:///history/spark/ > 14/04/03 13:41:31 INFO yarn.ApplicationMaster$$anon$1: Invoking sc stop from > shutdown hook > Exception in thread "Thread-41" java.io.IOException: Filesystem closed > at org.apache.hadoop.hdfs.DFSClient.checkOpen(DFSClient.java:398) > at > org.apache.hadoop.hdfs.DFSOutputStream.hflush(DFSOutputStream.java:1465) > at > org.apache.hadoop.hdfs.DFSOutputStream.sync(DFSOutputStream.java:1450) > at > org.apache.hadoop.fs.FSDataOutputStream.sync(FSDataOutputStream.java:116) > at > org.apache.spark.util.FileLogger$$anonfun$flush$2.apply(FileLogger.scala:137) > at > org.apache.spark.util.FileLogger$$anonfun$flush$2.apply(FileLogger.scala:137) > at scala.Option.foreach(Option.scala:236) > at org.apache.spark.util.FileLogger.flush(FileLogger.scala:137) > at > org.apache.spark.scheduler.EventLoggingListener.logEvent(EventLoggingListener.scala:69) > at > org.apache.spark.scheduler.EventLoggingListener.onApplicationEnd(EventLoggingListener.scala:101) > at > org.apache.spark.scheduler.SparkListenerBus$$anonfun$postToAll$13.apply(SparkListenerBus.scala:67) > at > org.apache.spark.scheduler.SparkListenerBus$$anonfun$postToAll$13.apply(SparkListenerBus.scala:67) > at > scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59) > at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:47) > at > org.apache.spark.scheduler.SparkListenerBus$class.postToAll(SparkListenerBus.scala:67) > at > org.apache.spark.scheduler.LiveListenerBus.postToAll(LiveListenerBus.scala:31) > at > org.apache.spark.scheduler.LiveListenerBus.post(LiveListenerBus.scala:78) > at > org.apache.spark.SparkContext.postApplicationEnd(SparkContext.scala:1081) > at org.apache.spark.SparkContext.stop(SparkContext.scala:828) > at > org.apache.spark.deploy.yarn.ApplicationMaster$$anon$1.run(ApplicationMaster.scala:460) -- This message was sent by Atlassian JIRA (v6.2#6252)