You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "Steve Loughran (JIRA)" <ji...@apache.org> on 2015/07/28 03:01:05 UTC

[jira] [Comment Edited] (SPARK-8385) java.lang.UnsupportedOperationException: Not implemented by the TFS FileSystem implementation

    [ https://issues.apache.org/jira/browse/SPARK-8385?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14643652#comment-14643652 ] 

Steve Loughran edited comment on SPARK-8385 at 7/28/15 1:01 AM:
----------------------------------------------------------------

What's happening? Hadoop is trying to enum all the filesystems via the service loader mechanism (HADOOP-7549), auto-registering all filsystems listed in any JAR's resource file {{META-INF/services/org.apache.hadoop.fs.FileSystem}} —in a map indexed by the filesystem scheme as returned by {{FileSystem.getScheme()}}

The default value for that raises an exception, so the FS init fails, and the user gets to see a stack trace

# every filesystem needs to implement this method
# hadoop's FS contract tests need to explicitly call the method and verify it is non null, non empty, so at anyone who implements those tests gets to find the problem. (there's an implicit probe already)
# maybe, hadoop should be more forgiving of filesystems which don't know their own name, yet have metadata entries. That's a tough call: it'd be more forgiving at startup time, but less intuitive downstream when things simply don't work if a filesystem is named but not found (i.e. there's no fallback to fs.*.impl=classname entry in the cluster configs.

Spark does ship with Tachyon 0.6.4, which has the method, but Tachyon 0.5.0 does not. Except Tachyon 0.5.0 does not have a resource file {{META-INF/services/org.apache.hadoop.fs.FileSystem}} -that is new with 0.6.x.

Which leads to the following hypothesis about what is going wrong:
# There are two versions of tachyon on the classpath
# tachyon 0.6.4+ explicitly declares the FS in the metadata file, triggering an auto instantiate/load
# tachyon 0.5.0's version of the FS class is the one being loaded by Hadoop (i.e. that JAR comes first in the classpath)

It's OK to have 0.50 on the classpath; or 0.6.4: it's the combination which is triggering the problem.

This isn't something Spark can fix, nor can Hadoop: duplicate, inconsistent JAR versions is always a disaster. This stack trace is how the specific case of >1 tachyon JAR on the classpath surfaces if v 0.5.0 comes first.

Closing as a WONTFIX as its an installation side problem, not anything that is fixable in source.

h2. For anyone seeing this:

# Check your SPARK_HOME environment variable and make sure its not pointing to an older one than the rest of your code is trying to use. 
# Check your build to make sure you aren't explicitly  pulling in a tachyon JAR —theres one packaged up in spark-assembly
# Make sure that you aren't pulling in another assembly module with its own tachyon version
# Make sure no tachyon JAR has been copied into any of your hadoop directories (i.e. {{HADOOP_HOME/lib}}


was (Author: stevel@apache.org):
What's happening? Hadoop is trying to enum all the filesystems via the service loader mechanism (HADOOP-7549), auto-registering all filsystems listed in any JAR's resource file {{META-INF/services/org.apache.hadoop.fs.FileSystem}} —in a map indexed by the filesystem scheme as returned by {{FileSystem.getScheme()}}

The default value for that raises an exception, so the FS init fails, and the user gets to see a stack trace

# every filesystem needs to implement this method
# hadoop's FS contract tests need to explicitly call the method and verify it is non null, non empty, so at anyone who implements those tests gets to find the problem. (there's an implicit probe already)
# maybe, hadoop should be more forgiving of filesystems which don't know their own name, yet have metadata entries. That's a tough call: it'd be more forgiving at startup time, but less intuitive downstream when things simply don't work if a filesystem is named but not found (i.e. there's no fallback to fs.*.impl=classname entry in the cluster configs.

Spark does ship with Tachyon 0.6.4, which has the method, but Tachyon 0.5.0 does not. Except Tachyon 0.5.0 does no have a resource file {{META-INF/services/org.apache.hadoop.fs.FileSystem}} -that is new with 0.60.

Which leads to the following hypothesis about what is going wrong:
# There are two versions of tachyon on the classpath
# tachyon 0.6.4+ explicitly declares the FS in the metadata file, triggering an auto instantiate/load
# tachyon 0.5.0's version of the FS class is the one being loaded by Hadoop (i.e. that JAR comes first in the classpath)

It's OK to have 0.50 on the classpath; or 0.6.4: it's the combination which is triggering the problem.

This isn't something Spark can fix, nor can Hadoop: duplicate, inconsistent JAR versions is always a disaster. This stack trace is how the specific case of >1 tachyon JAR on the classpath surfaces if v 0.5.0 comes first.

Closing as a WONTFIX as its an installation side problem, not anything that is fixable in source.

h2. For anyone seeing this:

# Check your SPARK_HOME environment variable and make sure its not pointing to an older one than the rest of your code is trying to use. 
# Check your build to make sure you aren't explicitly  pulling in a tachyon JAR —theres one packaged up in spark-assembly
# Make sure that you aren't pulling in another assembly module with its own tachyon version
# Make sure no tachyon JAR has been copied into any of your hadoop directories (i.e. {{HADOOP_HOME/lib}}

> 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-unsubscribe@spark.apache.org
For additional commands, e-mail: issues-help@spark.apache.org