You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "Bertrand Bossy (JIRA)" <ji...@apache.org> on 2014/11/13 09:24:34 UTC

[jira] [Commented] (SPARK-3039) Spark assembly for new hadoop API (hadoop 2) contains avro-mapred for hadoop 1 API

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

Bertrand Bossy commented on SPARK-3039:
---------------------------------------

Needs more fixes, since although I got the build to work for hadoop-2.4, yarn and hive with the sbt build. It doesn't work with the maven build, which AFAIK is required for pySpark. Dependency management seems to be quite different in sbt and maven

> Spark assembly for new hadoop API (hadoop 2) contains avro-mapred for hadoop 1 API
> ----------------------------------------------------------------------------------
>
>                 Key: SPARK-3039
>                 URL: https://issues.apache.org/jira/browse/SPARK-3039
>             Project: Spark
>          Issue Type: Bug
>          Components: Build, Input/Output, Spark Core
>    Affects Versions: 0.9.1, 1.0.0, 1.1.0
>         Environment: hadoop2, hadoop-2.4.0, HDP-2.1
>            Reporter: Bertrand Bossy
>            Assignee: Bertrand Bossy
>             Fix For: 1.2.0
>
>
> The spark assembly contains the artifact "org.apache.avro:avro-mapred" as a dependency of "org.spark-project.hive:hive-serde".
> The avro-mapred package provides a hadoop FileInputFormat to read and write avro files. There are two versions of this package, distinguished by a classifier. avro-mapred for the new Hadoop API uses the classifier "hadoop2". avro-mapred for the old Hadoop API uses no classifier.
> E.g. when reading avro files using 
> {code}
> sc.newAPIHadoopFile[AvroKey[SomeClass]],NullWritable,AvroKeyInputFormat[SomeClass]]("hdfs://path/to/file.avro")
> {code}
> The following error occurs:
> {code}
> java.lang.IncompatibleClassChangeError: Found interface org.apache.hadoop.mapreduce.TaskAttemptContext, but class was expected
>         at org.apache.avro.mapreduce.AvroKeyInputFormat.createRecordReader(AvroKeyInputFormat.java:47)
>         at org.apache.spark.rdd.NewHadoopRDD$$anon$1.<init>(NewHadoopRDD.scala:111)
>         at org.apache.spark.rdd.NewHadoopRDD.compute(NewHadoopRDD.scala:99)
>         at org.apache.spark.rdd.NewHadoopRDD.compute(NewHadoopRDD.scala:61)
>         at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:262)
>         at org.apache.spark.rdd.RDD.iterator(RDD.scala:229)
>         at org.apache.spark.rdd.MappedRDD.compute(MappedRDD.scala:31)
>         at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:262)
>         at org.apache.spark.rdd.RDD.iterator(RDD.scala:229)
>         at org.apache.spark.rdd.FilteredRDD.compute(FilteredRDD.scala:34)
>         at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:262)
>         at org.apache.spark.rdd.RDD.iterator(RDD.scala:229)
>         at org.apache.spark.rdd.MappedRDD.compute(MappedRDD.scala:31)
>         at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:262)
>         at org.apache.spark.rdd.RDD.iterator(RDD.scala:229)
>         at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:158)
>         at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:99)
>         at org.apache.spark.scheduler.Task.run(Task.scala:51)
>         at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:187)
>         at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
>         at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
>         at java.lang.Thread.run(Thread.java:744)
> {code}
> This error usually is a hint that there was a mix up of the old and the new Hadoop API. As a work-around, if avro-mapred for hadoop2 is "forced" to appear before the version that is bundled with Spark, reading avro files works fine. 
> Also, if Spark is built using avro-mapred for hadoop2, it works fine as well.



--
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