Uploaded image for project: 'Hive'
  1. Hive
  2. HIVE-7624

Reduce operator initialization failed when running multiple MR query on spark

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • None
    • 1.1.0
    • Spark
    • None

    Description

      The following error occurs when I try to run a query with multiple reduce works (M->R->R):

      14/08/05 12:17:07 ERROR Executor: Exception in task 0.0 in stage 2.0 (TID 1)
      java.lang.RuntimeException: Reduce operator initialization failed
      at org.apache.hadoop.hive.ql.exec.mr.ExecReducer.configure(ExecReducer.java:170)
      at org.apache.hadoop.hive.ql.exec.spark.HiveReduceFunction.call(HiveReduceFunction.java:53)
      at org.apache.hadoop.hive.ql.exec.spark.HiveReduceFunction.call(HiveReduceFunction.java:31)
      at org.apache.spark.api.java.JavaRDDLike$$anonfun$fn$7$1.apply(JavaRDDLike.scala:164)
      at org.apache.spark.api.java.JavaRDDLike$$anonfun$fn$7$1.apply(JavaRDDLike.scala:164)
      at org.apache.spark.rdd.RDD$$anonfun$13.apply(RDD.scala:596)
      at org.apache.spark.rdd.RDD$$anonfun$13.apply(RDD.scala:596)
      at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:35)
      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:68)
      at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41)
      at org.apache.spark.scheduler.Task.run(Task.scala:54)
      at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:199)
      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)
      Caused by: java.lang.RuntimeException: cannot find field reducesinkkey0 from [0:_col0]
      at org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils.getStandardStructFieldRef(ObjectInspectorUtils.java:415)
      at org.apache.hadoop.hive.serde2.objectinspector.StandardStructObjectInspector.getStructFieldRef(StandardStructObjectInspector.java:147)

      I suspect we're applying the reduce function in wrong order.

      Attachments

        1. HIVE-7624.patch
          5 kB
          Rui Li
        2. HIVE-7624.2-spark.patch
          4 kB
          Rui Li
        3. HIVE-7624.3-spark.patch
          4 kB
          Rui Li
        4. HIVE-7624.4-spark.patch
          4 kB
          Rui Li
        5. HIVE-7624.5-spark.patch
          5 kB
          Rui Li
        6. HIVE-7624.6-spark.patch
          5 kB
          Rui Li
        7. HIVE-7624.7-spark.patch
          3 kB
          Rui Li

        Issue Links

          Activity

            People

              lirui Rui Li
              lirui Rui Li
              Votes:
              0 Vote for this issue
              Watchers:
              5 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: