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

ClassCastException in vectorized order-by query over avro table with uniontype column

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Open
    • Minor
    • Resolution: Unresolved
    • 3.1.0, 3.1.2
    • None
    • Avro, Vectorization
    • None

    Description

      Reproducer
      create table avro_reproducer (key int, union_col uniontype <int, string>) stored as avro location '/tmp/avro_reproducer';
      INSERT INTO TABLE avro_reproducer values (0, create_union(0, 123, 'not me')),  (1, create_union(1, -1, 'me, me, me!'));
      
      --these queries are ok:
      select count(*) from avro_reproducer;  
      select * from avro_reproducer;  
      --these queries are not ok
      select * from avro_reproducer order by union_col; 
      select * from avro_reproducer sort by key; 
      select * from avro_reproducer order by 'does not have to be a column, really'; 
      

      I have verified this reproducer on CDH703, HDP301.
      It seems the issue is restricted to AVRO; this reproducer does not trigger failures against textfile tables, orc tables, and parquet tables.

      Also, the issue is restricted to vectorized execution; it goes away if I set hive.vectorized.execution.enabled=false

      Error message in CLI
      Driver stacktrace:
              at org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages(DAGScheduler.scala:1889)
              at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1877)
              at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1876)
              at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
              at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)
              at org.apache.spark.scheduler.DAGScheduler.abortStage(DAGScheduler.scala:1876)
              at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:926)
              at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:926)
              at scala.Option.foreach(Option.scala:257)
              at org.apache.spark.scheduler.DAGScheduler.handleTaskSetFailed(DAGScheduler.scala:926)
              at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.doOnReceive(DAGScheduler.scala:2110)
              at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:2059)
              at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:2048)
              at org.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:49)
      Caused by: java.lang.RuntimeException: Error processing row: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing row 
              at org.apache.hadoop.hive.ql.exec.spark.SparkMapRecordHandler.processRow(SparkMapRecordHandler.java:155)
              at org.apache.hadoop.hive.ql.exec.spark.HiveMapFunctionResultList.processNextRecord(HiveMapFunctionResultList.java:48)
              at org.apache.hadoop.hive.ql.exec.spark.HiveMapFunctionResultList.processNextRecord(HiveMapFunctionResultList.java:27)
              at org.apache.hadoop.hive.ql.exec.spark.HiveBaseFunctionResultList.hasNext(HiveBaseFunctionResultList.java:85)
              at scala.collection.convert.Wrappers$JIteratorWrapper.hasNext(Wrappers.scala:42)
              at org.apache.spark.shuffle.sort.BypassMergeSortShuffleWriter.write(BypassMergeSortShuffleWriter.java:125)
              at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:99)
              at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:55)
              at org.apache.spark.scheduler.Task.run(Task.scala:123)
              at org.apache.spark.executor.Executor$TaskRunner$$anonfun$10.apply(Executor.scala:408)
              at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1315)
              at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:414)
              at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
              at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
              at java.lang.Thread.run(Thread.java:748)
      Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing row 
              at org.apache.hadoop.hive.ql.exec.vector.VectorMapOperator.process(VectorMapOperator.java:970)
              at org.apache.hadoop.hive.ql.exec.spark.SparkMapRecordHandler.processRow(SparkMapRecordHandler.java:142)
              ... 14 more
      Caused by: java.lang.ClassCastException: org.apache.hadoop.io.IntWritable cannot be cast to org.apache.hadoop.hive.serde2.objectinspector.StandardUnionObjectInspector$StandardUnion
              at org.apache.hadoop.hive.ql.exec.vector.VectorAssignRow.assignRowColumn(VectorAssignRow.java:619)
              at org.apache.hadoop.hive.ql.exec.vector.VectorAssignRow.assignRowColumn(VectorAssignRow.java:351)
              at org.apache.hadoop.hive.ql.exec.vector.VectorAssignRow.assignRow(VectorAssignRow.java:1020)
              at org.apache.hadoop.hive.ql.exec.vector.VectorMapOperator.process(VectorMapOperator.java:959)
              ... 15 more
      

      Attachments

        Activity

          People

            Unassigned Unassigned
            gabriel.balan Gabriel C Balan
            Votes:
            0 Vote for this issue
            Watchers:
            3 Start watching this issue

            Dates

              Created:
              Updated: