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

Vectorization is not supported for datatype:VOID error while inserting data into specific columns

Details

    • Bug
    • Status: Closed
    • Major
    • Resolution: Fixed
    • 2.2.0
    • 2.1.1, 2.2.0
    • None
    • None

    Description

      Insert into specific columns fails due to following error:

      Vertex failed, vertexName=Reducer 2, vertexId=vertex_1465261180142_0160_1_01, diagnostics=[Task failed, taskId=task_1465261180142_0160_1_01_000000, diagnostics=[TaskAttempt 0 failed, info=[Error: Error while running task ( failure ) : attempt_1465261180142_0160_1_01_000000_0:java.lang.RuntimeException: java.lang.RuntimeException: Reduce operator initialization failed
      	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:198)
      	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.run(TezProcessor.java:160)
      	at org.apache.tez.runtime.LogicalIOProcessorRuntimeTask.run(LogicalIOProcessorRuntimeTask.java:370)
      	at org.apache.tez.runtime.task.TaskRunner2Callable$1.run(TaskRunner2Callable.java:73)
      	at org.apache.tez.runtime.task.TaskRunner2Callable$1.run(TaskRunner2Callable.java:61)
      	at java.security.AccessController.doPrivileged(Native Method)
      	at javax.security.auth.Subject.doAs(Subject.java:422)
      	at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1724)
      	at org.apache.tez.runtime.task.TaskRunner2Callable.callInternal(TaskRunner2Callable.java:61)
      	at org.apache.tez.runtime.task.TaskRunner2Callable.callInternal(TaskRunner2Callable.java:37)
      	at org.apache.tez.common.CallableWithNdc.call(CallableWithNdc.java:36)
      	at java.util.concurrent.FutureTask.run(FutureTask.java:266)
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
      	at java.lang.Thread.run(Thread.java:745)
      Caused by: java.lang.RuntimeException: Reduce operator initialization failed
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource.init(ReduceRecordSource.java:221)
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordProcessor.initializeSourceForTag(ReduceRecordProcessor.java:245)
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordProcessor.init(ReduceRecordProcessor.java:163)
      	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:171)
      	... 14 more
      Caused by: java.lang.RuntimeException: Vectorizaton is not supported for datatype:VOID
      	at org.apache.hadoop.hive.ql.exec.vector.VectorizedBatchUtil.createColumnVector(VectorizedBatchUtil.java:172)
      	at org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatchCtx.createVectorizedRowBatch(VectorizedRowBatchCtx.java:194)
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource.init(ReduceRecordSource.java:177)
      	... 17 more
      ], TaskAttempt 1 failed, info=[Error: Error while running task ( failure ) : attempt_1465261180142_0160_1_01_000000_1:java.lang.RuntimeException: java.lang.RuntimeException: Reduce operator initialization failed
      	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:198)
      	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.run(TezProcessor.java:160)
      	at org.apache.tez.runtime.LogicalIOProcessorRuntimeTask.run(LogicalIOProcessorRuntimeTask.java:370)
      	at org.apache.tez.runtime.task.TaskRunner2Callable$1.run(TaskRunner2Callable.java:73)
      	at org.apache.tez.runtime.task.TaskRunner2Callable$1.run(TaskRunner2Callable.java:61)
      	at java.security.AccessController.doPrivileged(Native Method)
      	at javax.security.auth.Subject.doAs(Subject.java:422)
      	at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1724)
      	at org.apache.tez.runtime.task.TaskRunner2Callable.callInternal(TaskRunner2Callable.java:61)
      	at org.apache.tez.runtime.task.TaskRunner2Callable.callInternal(TaskRunner2Callable.java:37)
      	at org.apache.tez.common.CallableWithNdc.call(CallableWithNdc.java:36)
      	at java.util.concurrent.FutureTask.run(FutureTask.java:266)
      	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
      	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
      	at java.lang.Thread.run(Thread.java:745)
      Caused by: java.lang.RuntimeException: Reduce operator initialization failed
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource.init(ReduceRecordSource.java:221)
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordProcessor.initializeSourceForTag(ReduceRecordProcessor.java:245)
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordProcessor.init(ReduceRecordProcessor.java:163)
      	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:171)
      	... 14 more
      Caused by: java.lang.RuntimeException: Vectorizaton is not supported for datatype:VOID
      	at org.apache.hadoop.hive.ql.exec.vector.VectorizedBatchUtil.createColumnVector(VectorizedBatchUtil.java:172)
      	at org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatchCtx.createVectorizedRowBatch(VectorizedRowBatchCtx.java:194)
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource.init(ReduceRecordSource.java:177)
      	... 17 more
      

      Steps to reproduce the issue:

      set hive.vectorized.execution.enabled=true;
      set hive.support.concurrency=true;
      set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
      
      drop table if exists newtable;
      create external table newtable(
                  a string,
                  b int,
                  c double)
      row format delimited
      fields terminated by '\t'
      stored as textfile;
      
      drop table if exists newtable_acid;
      create table newtable_acid (b int, a varchar(50),c decimal(3,2), d int)
      clustered by (b) into 2 buckets
      stored as orc
      tblproperties ('transactional'='true');
      
      insert into newtable_acid(a,b,c)
      select * from newtable;
      
      select a, b, c from newtable_acid;
      

      Attachments

        1. HIVE-14076.1.patch
          15 kB
          jcamachorodriguez
        2. HIVE-14076.patch
          7 kB
          jcamachorodriguez

        Activity

          jcamacho Jesús Camacho Rodríguez added a comment - - edited

          ekoifman, could you review it? Thanks

          jcamacho Jesús Camacho Rodríguez added a comment - - edited ekoifman , could you review it? Thanks
          hiveqa Hive QA added a comment -

          Here are the results of testing the latest attachment:
          https://issues.apache.org/jira/secure/attachment/12812591/HIVE-14076.patch

          SUCCESS: +1 due to 1 test(s) being added or modified.

          ERROR: -1 due to 10 failed/errored test(s), 10244 tests executed
          Failed tests:

          TestMiniTezCliDriver-cte_4.q-vector_groupby_reduce.q-empty_join.q-and-12-more - did not produce a TEST-*.xml file
          org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_cbo_rp_insert
          org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_constantPropagateForSubQuery
          org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_insert_into_with_schema
          org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_insert_into_with_schema2
          org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_stats_list_bucket
          org.apache.hadoop.hive.cli.TestMiniLlapCliDriver.testCliDriver_vector_complex_all
          org.apache.hadoop.hive.cli.TestMiniLlapCliDriver.testCliDriver_vector_complex_join
          org.apache.hadoop.hive.cli.TestMiniSparkOnYarnCliDriver.testCliDriver_index_bitmap3
          org.apache.hadoop.hive.llap.tezplugins.TestLlapTaskSchedulerService.testDelayedLocalityNodeCommErrorImmediateAllocation
          

          Test results: https://builds.apache.org/job/PreCommit-HIVE-MASTER-Build/230/testReport
          Console output: https://builds.apache.org/job/PreCommit-HIVE-MASTER-Build/230/console
          Test logs: http://ec2-50-18-27-0.us-west-1.compute.amazonaws.com/logs/PreCommit-HIVE-MASTER-Build-230/

          Messages:

          Executing org.apache.hive.ptest.execution.TestCheckPhase
          Executing org.apache.hive.ptest.execution.PrepPhase
          Executing org.apache.hive.ptest.execution.ExecutionPhase
          Executing org.apache.hive.ptest.execution.ReportingPhase
          Tests exited with: TestsFailedException: 10 tests failed
          

          This message is automatically generated.

          ATTACHMENT ID: 12812591 - PreCommit-HIVE-MASTER-Build

          hiveqa Hive QA added a comment - Here are the results of testing the latest attachment: https://issues.apache.org/jira/secure/attachment/12812591/HIVE-14076.patch SUCCESS: +1 due to 1 test(s) being added or modified. ERROR: -1 due to 10 failed/errored test(s), 10244 tests executed Failed tests: TestMiniTezCliDriver-cte_4.q-vector_groupby_reduce.q-empty_join.q-and-12-more - did not produce a TEST-*.xml file org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_cbo_rp_insert org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_constantPropagateForSubQuery org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_insert_into_with_schema org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_insert_into_with_schema2 org.apache.hadoop.hive.cli.TestCliDriver.testCliDriver_stats_list_bucket org.apache.hadoop.hive.cli.TestMiniLlapCliDriver.testCliDriver_vector_complex_all org.apache.hadoop.hive.cli.TestMiniLlapCliDriver.testCliDriver_vector_complex_join org.apache.hadoop.hive.cli.TestMiniSparkOnYarnCliDriver.testCliDriver_index_bitmap3 org.apache.hadoop.hive.llap.tezplugins.TestLlapTaskSchedulerService.testDelayedLocalityNodeCommErrorImmediateAllocation Test results: https://builds.apache.org/job/PreCommit-HIVE-MASTER-Build/230/testReport Console output: https://builds.apache.org/job/PreCommit-HIVE-MASTER-Build/230/console Test logs: http://ec2-50-18-27-0.us-west-1.compute.amazonaws.com/logs/PreCommit-HIVE-MASTER-Build-230/ Messages: Executing org.apache.hive.ptest.execution.TestCheckPhase Executing org.apache.hive.ptest.execution.PrepPhase Executing org.apache.hive.ptest.execution.ExecutionPhase Executing org.apache.hive.ptest.execution.ReportingPhase Tests exited with: TestsFailedException: 10 tests failed This message is automatically generated. ATTACHMENT ID: 12812591 - PreCommit-HIVE-MASTER-Build
          jcamacho Jesús Camacho Rodríguez added a comment - - edited

          Regenerated q file changes. ashutoshc, could you review it? Thanks

          jcamacho Jesús Camacho Rodríguez added a comment - - edited Regenerated q file changes. ashutoshc , could you review it? Thanks

          +1

          ashutoshc Ashutosh Chauhan added a comment - +1
          jcamacho Jesús Camacho Rodríguez added a comment - - edited

          Pushed to master, branch-2.1. Thanks for reviewing ashutoshc!

          jcamacho Jesús Camacho Rodríguez added a comment - - edited Pushed to master, branch-2.1. Thanks for reviewing ashutoshc !

          People

            jcamacho Jesús Camacho Rodríguez
            jvaria Jagruti Varia
            Votes:
            0 Vote for this issue
            Watchers:
            2 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: