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

Xiangrui Meng commented on SPARK-1556:
--------------------------------------

[~srowen] I saw you set jets3t's scope to runtime. Any particular reason for 
that setting? Now sbt reads deps info from pom. The assembly jar won't include 
jets3t if its scope is runtime only. 

> jets3t dep doesn't update properly with newer Hadoop versions
> -------------------------------------------------------------
>
>                 Key: SPARK-1556
>                 URL: https://issues.apache.org/jira/browse/SPARK-1556
>             Project: Spark
>          Issue Type: Bug
>          Components: Spark Core
>    Affects Versions: 0.8.1, 0.9.0, 1.0.0
>            Reporter: Nan Zhu
>            Assignee: Sean Owen
>            Priority: Blocker
>             Fix For: 1.0.0
>
>
> In Hadoop 2.2.x or newer, Jet3st 0.9.0 which defines 
> S3ServiceException/ServiceException is introduced, however, Spark still 
> relies on Jet3st 0.7.x which has no definition of these classes
> What I met is that 
> [code]
> 14/04/21 19:30:53 INFO deprecation: mapred.job.id is deprecated. Instead, use 
> mapreduce.job.id
> 14/04/21 19:30:53 INFO deprecation: mapred.tip.id is deprecated. Instead, use 
> mapreduce.task.id
> 14/04/21 19:30:53 INFO deprecation: mapred.task.id is deprecated. Instead, 
> use mapreduce.task.attempt.id
> 14/04/21 19:30:53 INFO deprecation: mapred.task.is.map is deprecated. 
> Instead, use mapreduce.task.ismap
> 14/04/21 19:30:53 INFO deprecation: mapred.task.partition is deprecated. 
> Instead, use mapreduce.task.partition
> java.lang.NoClassDefFoundError: org/jets3t/service/S3ServiceException
>       at 
> org.apache.hadoop.fs.s3native.NativeS3FileSystem.createDefaultStore(NativeS3FileSystem.java:280)
>       at 
> org.apache.hadoop.fs.s3native.NativeS3FileSystem.initialize(NativeS3FileSystem.java:270)
>       at 
> org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:2316)
>       at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:90)
>       at 
> org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:2350)
>       at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:2332)
>       at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:369)
>       at org.apache.hadoop.fs.Path.getFileSystem(Path.java:296)
>       at 
> org.apache.hadoop.mapred.FileInputFormat.listStatus(FileInputFormat.java:221)
>       at 
> org.apache.hadoop.mapred.FileInputFormat.getSplits(FileInputFormat.java:270)
>       at org.apache.spark.rdd.HadoopRDD.getPartitions(HadoopRDD.scala:140)
>       at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:207)
>       at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:205)
>       at scala.Option.getOrElse(Option.scala:120)
>       at org.apache.spark.rdd.RDD.partitions(RDD.scala:205)
>       at org.apache.spark.rdd.MappedRDD.getPartitions(MappedRDD.scala:28)
>       at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:207)
>       at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:205)
>       at scala.Option.getOrElse(Option.scala:120)
>       at org.apache.spark.rdd.RDD.partitions(RDD.scala:205)
>       at org.apache.spark.rdd.MappedRDD.getPartitions(MappedRDD.scala:28)
>       at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:207)
>       at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:205)
>       at scala.Option.getOrElse(Option.scala:120)
>       at org.apache.spark.rdd.RDD.partitions(RDD.scala:205)
>       at org.apache.spark.SparkContext.runJob(SparkContext.scala:891)
>       at 
> org.apache.spark.rdd.PairRDDFunctions.saveAsHadoopDataset(PairRDDFunctions.scala:741)
>       at 
> org.apache.spark.rdd.PairRDDFunctions.saveAsHadoopFile(PairRDDFunctions.scala:692)
>       at 
> org.apache.spark.rdd.PairRDDFunctions.saveAsHadoopFile(PairRDDFunctions.scala:574)
>       at org.apache.spark.rdd.RDD.saveAsTextFile(RDD.scala:900)
>       at $iwC$$iwC$$iwC$$iwC.<init>(<console>:15)
>       at $iwC$$iwC$$iwC.<init>(<console>:20)
>       at $iwC$$iwC.<init>(<console>:22)
>       at $iwC.<init>(<console>:24)
>       at <init>(<console>:26)
>       at .<init>(<console>:30)
>       at .<clinit>(<console>)
>       at .<init>(<console>:7)
>       at .<clinit>(<console>)
>       at $print(<console>)
>       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.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:772)
>       at 
> org.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1040)
>       at 
> org.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:609)
>       at org.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:640)
>       at org.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:604)
>       at 
> org.apache.spark.repl.SparkILoop.reallyInterpret$1(SparkILoop.scala:793)
>       at 
> org.apache.spark.repl.SparkILoop.interpretStartingWith(SparkILoop.scala:838)
>       at org.apache.spark.repl.SparkILoop.command(SparkILoop.scala:750)
>       at org.apache.spark.repl.SparkILoop.processLine$1(SparkILoop.scala:598)
>       at org.apache.spark.repl.SparkILoop.innerLoop$1(SparkILoop.scala:605)
>       at org.apache.spark.repl.SparkILoop.loop(SparkILoop.scala:608)
>       at 
> org.apache.spark.repl.SparkILoop$$anonfun$process$1.apply$mcZ$sp(SparkILoop.scala:931)
>       at 
> org.apache.spark.repl.SparkILoop$$anonfun$process$1.apply(SparkILoop.scala:881)
>       at 
> org.apache.spark.repl.SparkILoop$$anonfun$process$1.apply(SparkILoop.scala:881)
>       at 
> scala.tools.nsc.util.ScalaClassLoader$.savingContextLoader(ScalaClassLoader.scala:135)
>       at org.apache.spark.repl.SparkILoop.process(SparkILoop.scala:881)
>       at org.apache.spark.repl.SparkILoop.process(SparkILoop.scala:973)
>       at org.apache.spark.repl.Main$.main(Main.scala:31)
>       at org.apache.spark.repl.Main.main(Main.scala)
> Caused by: java.lang.ClassNotFoundException: 
> org.jets3t.service.S3ServiceException
>       at java.net.URLClassLoader$1.run(URLClassLoader.java:366)
>       at java.net.URLClassLoader$1.run(URLClassLoader.java:355)
>       at java.security.AccessController.doPrivileged(Native Method)
>       at java.net.URLClassLoader.findClass(URLClassLoader.java:354)
>       at java.lang.ClassLoader.loadClass(ClassLoader.java:425)
>       at sun.misc.Launcher$AppClassLoader.loadClass(Launcher.java:308)
>       at java.lang.ClassLoader.loadClass(ClassLoader.java:358)
>       ... 63 more
> [/code]



--
This message was sent by Atlassian JIRA
(v6.2#6252)

Reply via email to