Uploaded image for project: 'Hive'
  1. Hive
  2. HIVE-7292 Hive on Spark
  3. HIVE-7773

Union all query finished with errors [Spark Branch]

    XMLWordPrintableJSON

Details

    • Sub-task
    • Status: Resolved
    • Critical
    • Resolution: Fixed
    • None
    • 1.1.0
    • Spark
    • None

    Description

      When I run a union all query, I found the following error in spark log (the query finished with correct results though):

      java.lang.RuntimeException: Map operator initialization failed
              at org.apache.hadoop.hive.ql.exec.spark.SparkMapRecordHandler.init(SparkMapRecordHandler.java:127)
              at org.apache.hadoop.hive.ql.exec.spark.HiveMapFunction.call(HiveMapFunction.java:52)
              at org.apache.hadoop.hive.ql.exec.spark.HiveMapFunction.call(HiveMapFunction.java:30)
              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: org.apache.hadoop.hive.ql.metadata.HiveException: org.apache.hadoop.hive.ql.metadata.HiveException: Configuration and input path are inconsistent
              at org.apache.hadoop.hive.ql.exec.MapOperator.setChildren(MapOperator.java:404)
              at org.apache.hadoop.hive.ql.exec.spark.SparkMapRecordHandler.init(SparkMapRecordHandler.java:93)
              ... 16 more
      

      Judging from the log, I think we don't properly handle the input paths when cloning the job conf, so it may also affect other queries with multiple maps or reduces.

      Attachments

        1. HIVE-7773.2-spark.patch
          8 kB
          Brock Noland
        2. HIVE-7773.spark.patch
          8 kB
          Rui Li

        Issue Links

          Activity

            People

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

              Dates

                Created:
                Updated:
                Resolved: