[ https://issues.apache.org/jira/browse/SPARK-8385?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Steve Loughran reopened SPARK-8385: ----------------------------------- I can reproduce this too, and have actually managed to do it in the spark tests, somehow {code} 2015-07-27 16:28:13,267 [IPC Server handler 9 on 61736] INFO resourcemanager.ClientRMService (ClientRMService.java:getNewApplicationId(286)) - Allocated new applicationId: 2 2015-07-27 16:28:13,341 [stderr] INFO util.Utils (Logging.scala:logInfo(59)) - Exception in thread "main" java.lang.UnsupportedOperationException: Not implemented by the TFS FileSystem implementation 2015-07-27 16:28:13,341 [stderr] INFO util.Utils (Logging.scala:logInfo(59)) - at org.apache.hadoop.fs.FileSystem.getScheme(FileSystem.java:214) 2015-07-27 16:28:13,342 [stderr] INFO util.Utils (Logging.scala:logInfo(59)) - at org.apache.hadoop.fs.FileSystem.loadFileSystems(FileSystem.java:2365) 2015-07-27 16:28:13,342 [stderr] INFO util.Utils (Logging.scala:logInfo(59)) - at org.apache.hadoop.fs.FileSystem.getFileSystemClass(FileSystem.java:2375) 2015-07-27 16:28:13,342 [stderr] INFO util.Utils (Logging.scala:logInfo(59)) - at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:2392) 2015-07-27 16:28:13,342 [stderr] INFO util.Utils (Logging.scala:logInfo(59)) - at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:89) 2015-07-27 16:28:13,342 [stderr] INFO util.Utils (Logging.scala:logInfo(59)) - at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:2431) 2015-07-27 16:28:13,342 [stderr] INFO util.Utils (Logging.scala:logInfo(59)) - at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:2413) 2015-07-27 16:28:13,342 [stderr] INFO util.Utils (Logging.scala:logInfo(59)) - at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:368) 2015-07-27 16:28:13,343 [stderr] INFO util.Utils (Logging.scala:logInfo(59)) - at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:167) 2015-07-27 16:28:13,343 [stderr] INFO util.Utils (Logging.scala:logInfo(59)) - at org.apache.spark.deploy.yarn.Client.prepareLocalResources(Client.scala:216) 2015-07-27 16:28:13,343 [stderr] INFO util.Utils (Logging.scala:logInfo(59)) - at org.apache.spark.deploy.yarn.Client.createContainerLaunchContext(Client.scala:384) 2015-07-27 16:28:13,343 [stderr] INFO util.Utils (Logging.scala:logInfo(59)) - at org.apache.spark.deploy.yarn.Client.submitApplication(Client.scala:102) 2015-07-27 16:28:13,343 [stderr] INFO util.Utils (Logging.scala:logInfo(59)) - at org.apache.spark.deploy.yarn.Client.run(Client.scala:619) 2015-07-27 16:28:13,343 [stderr] INFO util.Utils (Logging.scala:logInfo(59)) - at org.apache.spark.deploy.yarn.Client$.main(Client.scala:647) 2015-07-27 16:28:13,343 [stderr] INFO util.Utils (Logging.scala:logInfo(59)) - at org.apache.spark.deploy.yarn.Client.main(Client.scala) 2015-07-27 16:28:13,343 [stderr] INFO util.Utils (Logging.scala:logInfo(59)) - at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) {code} > java.lang.UnsupportedOperationException: Not implemented by the TFS > FileSystem implementation > --------------------------------------------------------------------------------------------- > > Key: SPARK-8385 > URL: https://issues.apache.org/jira/browse/SPARK-8385 > Project: Spark > Issue Type: Bug > Components: Input/Output > Affects Versions: 1.4.0 > Environment: RHEL 7.1 > Reporter: Peter Haumer > > I used to be able to debug my Spark apps in Eclipse. With Spark 1.3.1 I > created a launch and just set the vm var "-Dspark.master=local[4]". > With 1.4 this stopped working when reading files from the OS filesystem. > Running the same apps with spark-submit works fine. Loosing the ability to > debug that way has a major impact on the usability of Spark. > The following exception is thrown: > Exception in thread "main" java.lang.UnsupportedOperationException: Not > implemented by the TFS FileSystem implementation > at org.apache.hadoop.fs.FileSystem.getScheme(FileSystem.java:213) > at org.apache.hadoop.fs.FileSystem.loadFileSystems(FileSystem.java:2401) > at org.apache.hadoop.fs.FileSystem.getFileSystemClass(FileSystem.java:2411) > at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:2428) > at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:88) > at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:2467) > at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:2449) > at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:367) > at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:166) > at org.apache.hadoop.mapred.JobConf.getWorkingDirectory(JobConf.java:653) > at > org.apache.hadoop.mapred.FileInputFormat.setInputPaths(FileInputFormat.java:389) > at > org.apache.hadoop.mapred.FileInputFormat.setInputPaths(FileInputFormat.java:362) > at org.apache.spark.SparkContext$$anonfun$28.apply(SparkContext.scala:762) > at org.apache.spark.SparkContext$$anonfun$28.apply(SparkContext.scala:762) > at > org.apache.spark.rdd.HadoopRDD$$anonfun$getJobConf$6.apply(HadoopRDD.scala:172) > at > org.apache.spark.rdd.HadoopRDD$$anonfun$getJobConf$6.apply(HadoopRDD.scala:172) > at scala.Option.map(Option.scala:145) > at org.apache.spark.rdd.HadoopRDD.getJobConf(HadoopRDD.scala:172) > at org.apache.spark.rdd.HadoopRDD.getPartitions(HadoopRDD.scala:196) > at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:219) > at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:217) > at scala.Option.getOrElse(Option.scala:120) > at org.apache.spark.rdd.RDD.partitions(RDD.scala:217) > at > org.apache.spark.rdd.MapPartitionsRDD.getPartitions(MapPartitionsRDD.scala:32) > at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:219) > at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:217) > at scala.Option.getOrElse(Option.scala:120) > at org.apache.spark.rdd.RDD.partitions(RDD.scala:217) > at > org.apache.spark.rdd.MapPartitionsRDD.getPartitions(MapPartitionsRDD.scala:32) > at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:219) > at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:217) > at scala.Option.getOrElse(Option.scala:120) > at org.apache.spark.rdd.RDD.partitions(RDD.scala:217) > at > org.apache.spark.rdd.MapPartitionsRDD.getPartitions(MapPartitionsRDD.scala:32) > at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:219) > at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:217) > at scala.Option.getOrElse(Option.scala:120) > at org.apache.spark.rdd.RDD.partitions(RDD.scala:217) > at org.apache.spark.SparkContext.runJob(SparkContext.scala:1535) > at org.apache.spark.rdd.RDD.reduce(RDD.scala:900) > at org.apache.spark.api.java.JavaRDDLike$class.reduce(JavaRDDLike.scala:357) > at org.apache.spark.api.java.AbstractJavaRDDLike.reduce(JavaRDDLike.scala:46) > at com.databricks.apps.logs.LogAnalyzer.main(LogAnalyzer.java:60) -- This message was sent by Atlassian JIRA (v6.3.4#6332) --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org