Uploaded image for project: 'Spark'
  1. Spark
  2. SPARK-3039

Spark assembly for new hadoop API (hadoop 2) contains avro-mapred for hadoop 1 API

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Critical
    • Resolution: Fixed
    • 0.9.1, 1.0.0, 1.1.0, 1.2.0
    • 1.3.0
    • Build, Input/Output, Spark Core
    • None
    • hadoop2, hadoop-2.4.0, HDP-2.1

    Description

      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

      sc.newAPIHadoopFile[AvroKey[SomeClass]],NullWritable,AvroKeyInputFormat[SomeClass]]("hdfs://path/to/file.avro")
      

      The following error occurs:

      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)
      

      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.

      Attachments

        Issue Links

          Activity

            People

              bbossy Bertrand Bossy
              bbossy Bertrand Bossy
              Votes:
              4 Vote for this issue
              Watchers:
              7 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: