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

Vectorization : TPC-DS Q80 fails with java.lang.ClassCastException when hive.vectorized.execution.reduce.enabled is enabled

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • 0.14.0
    • 1.2.1
    • Vectorization
    • None

    Description

      Query

      set hive.vectorized.execution.reduce.enabled=true;
      with ssr as
       (select  s_store_id as store_id,
                sum(ss_ext_sales_price) as sales,
                sum(coalesce(sr_return_amt, 0)) as returns,
                sum(ss_net_profit - coalesce(sr_net_loss, 0)) as profit
        from store_sales left outer join store_returns on
               (ss_item_sk = sr_item_sk and ss_ticket_number = sr_ticket_number),
           date_dim,
           store,
           item,
           promotion
       where ss_sold_date_sk = d_date_sk
             and d_date between cast('1998-08-04' as date) 
                        and (cast('1998-09-04' as date))
             and ss_store_sk = s_store_sk
             and ss_item_sk = i_item_sk
             and i_current_price > 50
             and ss_promo_sk = p_promo_sk
             and p_channel_tv = 'N'
       group by s_store_id)
       ,
       csr as
       (select  cp_catalog_page_id as catalog_page_id,
                sum(cs_ext_sales_price) as sales,
                sum(coalesce(cr_return_amount, 0)) as returns,
                sum(cs_net_profit - coalesce(cr_net_loss, 0)) as profit
        from catalog_sales left outer join catalog_returns on
               (cs_item_sk = cr_item_sk and cs_order_number = cr_order_number),
           date_dim,
           catalog_page,
           item,
           promotion
       where cs_sold_date_sk = d_date_sk
             and d_date between cast('1998-08-04' as date)
                        and (cast('1998-09-04' as date))
              and cs_catalog_page_sk = cp_catalog_page_sk
             and cs_item_sk = i_item_sk
             and i_current_price > 50
             and cs_promo_sk = p_promo_sk
             and p_channel_tv = 'N'
      group by cp_catalog_page_id)
       ,
       wsr as
       (select  web_site_id,
                sum(ws_ext_sales_price) as sales,
                sum(coalesce(wr_return_amt, 0)) as returns,
                sum(ws_net_profit - coalesce(wr_net_loss, 0)) as profit
        from web_sales left outer join web_returns on
               (ws_item_sk = wr_item_sk and ws_order_number = wr_order_number),
           date_dim,
           web_site,
           item,
           promotion
       where ws_sold_date_sk = d_date_sk
             and d_date between cast('1998-08-04' as date)
                        and (cast('1998-09-04' as date))
              and ws_web_site_sk = web_site_sk
             and ws_item_sk = i_item_sk
             and i_current_price > 50
             and ws_promo_sk = p_promo_sk
             and p_channel_tv = 'N'
      group by web_site_id)
        select  channel
              , id
              , sum(sales) as sales
              , sum(returns) as returns
              , sum(profit) as profit
       from 
       (select 'store channel' as channel
              , concat('store', store_id) as id
              , sales
              , returns
              , profit
       from   ssr
       union all
       select 'catalog channel' as channel
              , concat('catalog_page', catalog_page_id) as id
              , sales
              , returns
              , profit
       from  csr
       union all
       select 'web channel' as channel
              , concat('web_site', web_site_id) as id
              , sales
              , returns
              , profit
       from   wsr
       ) x
       group by channel, id with rollup
       order by channel
               ,id
       limit 100
      

      Exception

      Vertex failed, vertexName=Reducer 5, vertexId=vertex_1426707664723_1377_1_22, diagnostics=[Task failed, taskId=task_1426707664723_1377_1_22_000000, diagnostics=[TaskAttempt 0 failed, info=[Error: Failure while running task:java.lang.RuntimeException: java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing vector batch (tag=0) \N\N09.285817653506076E84.639990363237801E7-1.1814318134887291E8
      \N\N04.682909323885761E82.2415242712669864E7-5.966176123188091E7
      \N\N01.2847032699693155E96.300096113768728E7-5.94963316209578E8
      	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:171)
      	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.run(TezProcessor.java:137)
      	at org.apache.tez.runtime.LogicalIOProcessorRuntimeTask.run(LogicalIOProcessorRuntimeTask.java:330)
      	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable$1.run(TezTaskRunner.java:179)
      	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable$1.run(TezTaskRunner.java:171)
      	at java.security.AccessController.doPrivileged(Native Method)
      	at javax.security.auth.Subject.doAs(Subject.java:415)
      	at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1628)
      	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable.callInternal(TezTaskRunner.java:171)
      	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable.callInternal(TezTaskRunner.java:167)
      	at org.apache.tez.common.CallableWithNdc.call(CallableWithNdc.java:36)
      	at java.util.concurrent.FutureTask.run(FutureTask.java:262)
      	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:745)
      Caused by: java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing vector batch (tag=0) \N\N09.285817653506076E84.639990363237801E7-1.1814318134887291E8
      \N\N04.682909323885761E82.2415242712669864E7-5.966176123188091E7
      \N\N01.2847032699693155E96.300096113768728E7-5.94963316209578E8
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource.pushRecord(ReduceRecordSource.java:267)
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordProcessor.run(ReduceRecordProcessor.java:248)
      	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:148)
      	... 14 more
      Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing vector batch (tag=0) \N\N09.285817653506076E84.639990363237801E7-1.1814318134887291E8
      \N\N04.682909323885761E82.2415242712669864E7-5.966176123188091E7
      \N\N01.2847032699693155E96.300096113768728E7-5.94963316209578E8
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource.processVectors(ReduceRecordSource.java:394)
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource.pushRecord(ReduceRecordSource.java:252)
      	... 16 more
      Caused by: java.lang.ClassCastException: org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector cannot be cast to org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector
      	at org.apache.hadoop.hive.ql.exec.vector.VectorGroupKeyHelper.copyGroupKey(VectorGroupKeyHelper.java:94)
      	at org.apache.hadoop.hive.ql.exec.vector.VectorGroupByOperator$ProcessingModeGroupBatches.processBatch(VectorGroupByOperator.java:729)
      	at org.apache.hadoop.hive.ql.exec.vector.VectorGroupByOperator.process(VectorGroupByOperator.java:878)
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource.processVectors(ReduceRecordSource.java:378)
      	... 17 more
      ], TaskAttempt 1 failed, info=[Error: Failure while running task:java.lang.RuntimeException: java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing vector batch (tag=0) \N\N09.285817653506076E84.639990363237801E7-1.1814318134887291E8
      \N\N04.682909323885761E82.2415242712669864E7-5.966176123188091E7
      \N\N01.2847032699693155E96.300096113768728E7-5.94963316209578E8
      	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:171)
      	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.run(TezProcessor.java:137)
      	at org.apache.tez.runtime.LogicalIOProcessorRuntimeTask.run(LogicalIOProcessorRuntimeTask.java:330)
      	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable$1.run(TezTaskRunner.java:179)
      	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable$1.run(TezTaskRunner.java:171)
      	at java.security.AccessController.doPrivileged(Native Method)
      	at javax.security.auth.Subject.doAs(Subject.java:415)
      	at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1628)
      	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable.callInternal(TezTaskRunner.java:171)
      	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable.callInternal(TezTaskRunner.java:167)
      	at org.apache.tez.common.CallableWithNdc.call(CallableWithNdc.java:36)
      	at java.util.concurrent.FutureTask.run(FutureTask.java:262)
      	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:745)
      Caused by: java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing vector batch (tag=0) \N\N09.285817653506076E84.639990363237801E7-1.1814318134887291E8
      \N\N04.682909323885761E82.2415242712669864E7-5.966176123188091E7
      \N\N01.2847032699693155E96.300096113768728E7-5.94963316209578E8
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource.pushRecord(ReduceRecordSource.java:267)
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordProcessor.run(ReduceRecordProcessor.java:248)
      	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:148)
      	... 14 more
      Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing vector batch (tag=0) \N\N09.285817653506076E84.639990363237801E7-1.1814318134887291E8
      \N\N04.682909323885761E82.2415242712669864E7-5.966176123188091E7
      \N\N01.2847032699693155E96.300096113768728E7-5.94963316209578E8
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource.processVectors(ReduceRecordSource.java:394)
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource.pushRecord(ReduceRecordSource.java:252)
      	... 16 more
      Caused by: java.lang.ClassCastException: org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector cannot be cast to org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector
      	at org.apache.hadoop.hive.ql.exec.vector.VectorGroupKeyHelper.copyGroupKey(VectorGroupKeyHelper.java:94)
      	at org.apache.hadoop.hive.ql.exec.vector.VectorGroupByOperator$ProcessingModeGroupBatches.processBatch(VectorGroupByOperator.java:729)
      	at org.apache.hadoop.hive.ql.exec.vector.VectorGroupByOperator.process(VectorGroupByOperator.java:878)
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource.processVectors(ReduceRecordSource.java:378)
      	... 17 more
      ], TaskAttempt 2 failed, info=[Error: Failure while running task:java.lang.RuntimeException: java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing vector batch (tag=0) \N\N09.285817653506076E84.639990363237801E7-1.1814318134887291E8
      \N\N04.682909323885761E82.2415242712669864E7-5.966176123188091E7
      \N\N01.2847032699693155E96.300096113768728E7-5.94963316209578E8
      	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:171)
      	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.run(TezProcessor.java:137)
      	at org.apache.tez.runtime.LogicalIOProcessorRuntimeTask.run(LogicalIOProcessorRuntimeTask.java:330)
      	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable$1.run(TezTaskRunner.java:179)
      	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable$1.run(TezTaskRunner.java:171)
      	at java.security.AccessController.doPrivileged(Native Method)
      	at javax.security.auth.Subject.doAs(Subject.java:415)
      	at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1628)
      	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable.callInternal(TezTaskRunner.java:171)
      	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable.callInternal(TezTaskRunner.java:167)
      	at org.apache.tez.common.CallableWithNdc.call(CallableWithNdc.java:36)
      	at java.util.concurrent.FutureTask.run(FutureTask.java:262)
      	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:745)
      Caused by: java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing vector batch (tag=0) \N\N09.285817653506076E84.639990363237801E7-1.1814318134887291E8
      \N\N04.682909323885761E82.2415242712669864E7-5.966176123188091E7
      \N\N01.2847032699693155E96.300096113768728E7-5.94963316209578E8
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource.pushRecord(ReduceRecordSource.java:267)
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordProcessor.run(ReduceRecordProcessor.java:248)
      	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:148)
      	... 14 more
      Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing vector batch (tag=0) \N\N09.285817653506076E84.639990363237801E7-1.1814318134887291E8
      \N\N04.682909323885761E82.2415242712669864E7-5.966176123188091E7
      \N\N01.2847032699693155E96.300096113768728E7-5.94963316209578E8
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource.processVectors(ReduceRecordSource.java:394)
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource.pushRecord(ReduceRecordSource.java:252)
      	... 16 more
      Caused by: java.lang.ClassCastException: org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector cannot be cast to org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector
      	at org.apache.hadoop.hive.ql.exec.vector.VectorGroupKeyHelper.copyGroupKey(VectorGroupKeyHelper.java:94)
      	at org.apache.hadoop.hive.ql.exec.vector.VectorGroupByOperator$ProcessingModeGroupBatches.processBatch(VectorGroupByOperator.java:729)
      	at org.apache.hadoop.hive.ql.exec.vector.VectorGroupByOperator.process(VectorGroupByOperator.java:878)
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource.processVectors(ReduceRecordSource.java:378)
      	... 17 more
      ], TaskAttempt 3 failed, info=[Error: Failure while running task:java.lang.RuntimeException: java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing vector batch (tag=0) \N\N09.285817653506076E84.639990363237801E7-1.1814318134887291E8
      \N\N04.682909323885761E82.2415242712669864E7-5.966176123188091E7
      \N\N01.2847032699693155E96.300096113768728E7-5.94963316209578E8
      	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:171)
      	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.run(TezProcessor.java:137)
      	at org.apache.tez.runtime.LogicalIOProcessorRuntimeTask.run(LogicalIOProcessorRuntimeTask.java:330)
      	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable$1.run(TezTaskRunner.java:179)
      	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable$1.run(TezTaskRunner.java:171)
      	at java.security.AccessController.doPrivileged(Native Method)
      	at javax.security.auth.Subject.doAs(Subject.java:415)
      	at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1628)
      	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable.callInternal(TezTaskRunner.java:171)
      	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable.callInternal(TezTaskRunner.java:167)
      	at org.apache.tez.common.CallableWithNdc.call(CallableWithNdc.java:36)
      	at java.util.concurrent.FutureTask.run(FutureTask.java:262)
      	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:745)
      Caused by: java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing vector batch (tag=0) \N\N09.285817653506076E84.639990363237801E7-1.1814318134887291E8
      \N\N04.682909323885761E82.2415242712669864E7-5.966176123188091E7
      \N\N01.2847032699693155E96.300096113768728E7-5.94963316209578E8
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource.pushRecord(ReduceRecordSource.java:267)
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordProcessor.run(ReduceRecordProcessor.java:248)
      	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:148)
      	... 14 more
      Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing vector batch (tag=0) \N\N09.285817653506076E84.639990363237801E7-1.1814318134887291E8
      \N\N04.682909323885761E82.2415242712669864E7-5.966176123188091E7
      \N\N01.2847032699693155E96.300096113768728E7-5.94963316209578E8
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource.processVectors(ReduceRecordSource.java:394)
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource.pushRecord(ReduceRecordSource.java:252)
      	... 16 more
      Caused by: java.lang.ClassCastException: org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector cannot be cast to org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector
      	at org.apache.hadoop.hive.ql.exec.vector.VectorGroupKeyHelper.copyGroupKey(VectorGroupKeyHelper.java:94)
      	at org.apache.hadoop.hive.ql.exec.vector.VectorGroupByOperator$ProcessingModeGroupBatches.processBatch(VectorGroupByOperator.java:729)
      	at org.apache.hadoop.hive.ql.exec.vector.VectorGroupByOperator.process(VectorGroupByOperator.java:878)
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource.processVectors(ReduceRecordSource.java:378)
      	... 17 more
      ]], Vertex failed as one or more tasks failed. failedTasks:1, Vertex vertex_1426707664723_1377_1_22 [Reducer 5] killed/failed due to:null]
      15/04/07 05:14:52 [main]: ERROR SessionState: Vertex failed, vertexName=Reducer 5, vertexId=vertex_1426707664723_1377_1_22, diagnostics=[Task failed, taskId=task_1426707664723_1377_1_22_000000, diagnostics=[TaskAttempt 0 failed, info=[Error: Failure while running task:java.lang.RuntimeException: java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing vector batch (tag=0) \N\N09.285817653506076E84.639990363237801E7-1.1814318134887291E8
      \N\N04.682909323885761E82.2415242712669864E7-5.966176123188091E7
      \N\N01.2847032699693155E96.300096113768728E7-5.94963316209578E8
      	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:171)
      	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.run(TezProcessor.java:137)
      	at org.apache.tez.runtime.LogicalIOProcessorRuntimeTask.run(LogicalIOProcessorRuntimeTask.java:330)
      	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable$1.run(TezTaskRunner.java:179)
      	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable$1.run(TezTaskRunner.java:171)
      	at java.security.AccessController.doPrivileged(Native Method)
      	at javax.security.auth.Subject.doAs(Subject.java:415)
      	at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1628)
      	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable.callInternal(TezTaskRunner.java:171)
      	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable.callInternal(TezTaskRunner.java:167)
      	at org.apache.tez.common.CallableWithNdc.call(CallableWithNdc.java:36)
      	at java.util.concurrent.FutureTask.run(FutureTask.java:262)
      	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:745)
      Caused by: java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing vector batch (tag=0) \N\N09.285817653506076E84.639990363237801E7-1.1814318134887291E8
      \N\N04.682909323885761E82.2415242712669864E7-5.966176123188091E7
      \N\N01.2847032699693155E96.300096113768728E7-5.94963316209578E8
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource.pushRecord(ReduceRecordSource.java:267)
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordProcessor.run(ReduceRecordProcessor.java:248)
      	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:148)
      	... 14 more
      Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing vector batch (tag=0) \N\N09.285817653506076E84.639990363237801E7-1.1814318134887291E8
      \N\N04.682909323885761E82.2415242712669864E7-5.966176123188091E7
      \N\N01.2847032699693155E96.300096113768728E7-5.94963316209578E8
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource.processVectors(ReduceRecordSource.java:394)
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource.pushRecord(ReduceRecordSource.java:252)
      	... 16 more
      Caused by: java.lang.ClassCastException: org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector cannot be cast to org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector
      	at org.apache.hadoop.hive.ql.exec.vector.VectorGroupKeyHelper.copyGroupKey(VectorGroupKeyHelper.java:94)
      	at org.apache.hadoop.hive.ql.exec.vector.VectorGroupByOperator$ProcessingModeGroupBatches.processBatch(VectorGroupByOperator.java:729)
      	at org.apache.hadoop.hive.ql.exec.vector.VectorGroupByOperator.process(VectorGroupByOperator.java:878)
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource.processVectors(ReduceRecordSource.java:378)
      	... 17 more
      ], TaskAttempt 1 failed, info=[Error: Failure while running task:java.lang.RuntimeException: java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing vector batch (tag=0) \N\N09.285817653506076E84.639990363237801E7-1.1814318134887291E8
      \N\N04.682909323885761E82.2415242712669864E7-5.966176123188091E7
      \N\N01.2847032699693155E96.300096113768728E7-5.94963316209578E8
      	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:171)
      	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.run(TezProcessor.java:137)
      	at org.apache.tez.runtime.LogicalIOProcessorRuntimeTask.run(LogicalIOProcessorRuntimeTask.java:330)
      	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable$1.run(TezTaskRunner.java:179)
      	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable$1.run(TezTaskRunner.java:171)
      	at java.security.AccessController.doPrivileged(Native Method)
      	at javax.security.auth.Subject.doAs(Subject.java:415)
      	at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1628)
      	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable.callInternal(TezTaskRunner.java:171)
      	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable.callInternal(TezTaskRunner.java:167)
      	at org.apache.tez.common.CallableWithNdc.call(CallableWithNdc.java:36)
      	at java.util.concurrent.FutureTask.run(FutureTask.java:262)
      	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:745)
      Caused by: java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing vector batch (tag=0) \N\N09.285817653506076E84.639990363237801E7-1.1814318134887291E8
      \N\N04.682909323885761E82.2415242712669864E7-5.966176123188091E7
      \N\N01.2847032699693155E96.300096113768728E7-5.94963316209578E8
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource.pushRecord(ReduceRecordSource.java:267)
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordProcessor.run(ReduceRecordProcessor.java:248)
      	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:148)
      	... 14 more
      Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing vector batch (tag=0) \N\N09.285817653506076E84.639990363237801E7-1.1814318134887291E8
      \N\N04.682909323885761E82.2415242712669864E7-5.966176123188091E7
      \N\N01.2847032699693155E96.300096113768728E7-5.94963316209578E8
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource.processVectors(ReduceRecordSource.java:394)
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource.pushRecord(ReduceRecordSource.java:252)
      	... 16 more
      Caused by: java.lang.ClassCastException: org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector cannot be cast to org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector
      	at org.apache.hadoop.hive.ql.exec.vector.VectorGroupKeyHelper.copyGroupKey(VectorGroupKeyHelper.java:94)
      	at org.apache.hadoop.hive.ql.exec.vector.VectorGroupByOperator$ProcessingModeGroupBatches.processBatch(VectorGroupByOperator.java:729)
      	at org.apache.hadoop.hive.ql.exec.vector.VectorGroupByOperator.process(VectorGroupByOperator.java:878)
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource.processVectors(ReduceRecordSource.java:378)
      	... 17 more
      ], TaskAttempt 2 failed, info=[Error: Failure while running task:java.lang.RuntimeException: java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing vector batch (tag=0) \N\N09.285817653506076E84.639990363237801E7-1.1814318134887291E8
      \N\N04.682909323885761E82.2415242712669864E7-5.966176123188091E7
      \N\N01.2847032699693155E96.300096113768728E7-5.94963316209578E8
      	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:171)
      	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.run(TezProcessor.java:137)
      	at org.apache.tez.runtime.LogicalIOProcessorRuntimeTask.run(LogicalIOProcessorRuntimeTask.java:330)
      	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable$1.run(TezTaskRunner.java:179)
      	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable$1.run(TezTaskRunner.java:171)
      	at java.security.AccessController.doPrivileged(Native Method)
      	at javax.security.auth.Subject.doAs(Subject.java:415)
      	at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1628)
      	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable.callInternal(TezTaskRunner.java:171)
      	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable.callInternal(TezTaskRunner.java:167)
      	at org.apache.tez.common.CallableWithNdc.call(CallableWithNdc.java:36)
      	at java.util.concurrent.FutureTask.run(FutureTask.java:262)
      	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:745)
      Caused by: java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing vector batch (tag=0) \N\N09.285817653506076E84.639990363237801E7-1.1814318134887291E8
      \N\N04.682909323885761E82.2415242712669864E7-5.966176123188091E7
      \N\N01.2847032699693155E96.300096113768728E7-5.94963316209578E8
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource.pushRecord(ReduceRecordSource.java:267)
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordProcessor.run(ReduceRecordProcessor.java:248)
      	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:148)
      	... 14 more
      Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing vector batch (tag=0) \N\N09.285817653506076E84.639990363237801E7-1.1814318134887291E8
      \N\N04.682909323885761E82.2415242712669864E7-5.966176123188091E7
      \N\N01.2847032699693155E96.300096113768728E7-5.94963316209578E8
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource.processVectors(ReduceRecordSource.java:394)
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource.pushRecord(ReduceRecordSource.java:252)
      	... 16 more
      Caused by: java.lang.ClassCastException: org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector cannot be cast to org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector
      	at org.apache.hadoop.hive.ql.exec.vector.VectorGroupKeyHelper.copyGroupKey(VectorGroupKeyHelper.java:94)
      	at org.apache.hadoop.hive.ql.exec.vector.VectorGroupByOperator$ProcessingModeGroupBatches.processBatch(VectorGroupByOperator.java:729)
      	at org.apache.hadoop.hive.ql.exec.vector.VectorGroupByOperator.process(VectorGroupByOperator.java:878)
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource.processVectors(ReduceRecordSource.java:378)
      	... 17 more
      ], TaskAttempt 3 failed, info=[Error: Failure while running task:java.lang.RuntimeException: java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing vector batch (tag=0) \N\N09.285817653506076E84.639990363237801E7-1.1814318134887291E8
      \N\N04.682909323885761E82.2415242712669864E7-5.966176123188091E7
      \N\N01.2847032699693155E96.300096113768728E7-5.94963316209578E8
      	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:171)
      	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.run(TezProcessor.java:137)
      	at org.apache.tez.runtime.LogicalIOProcessorRuntimeTask.run(LogicalIOProcessorRuntimeTask.java:330)
      	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable$1.run(TezTaskRunner.java:179)
      	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable$1.run(TezTaskRunner.java:171)
      	at java.security.AccessController.doPrivileged(Native Method)
      	at javax.security.auth.Subject.doAs(Subject.java:415)
      	at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1628)
      	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable.callInternal(TezTaskRunner.java:171)
      	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable.callInternal(TezTaskRunner.java:167)
      	at org.apache.tez.common.CallableWithNdc.call(CallableWithNdc.java:36)
      	at java.util.concurrent.FutureTask.run(FutureTask.java:262)
      	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:745)
      Caused by: java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing vector batch (tag=0) \N\N09.285817653506076E84.639990363237801E7-1.1814318134887291E8
      \N\N04.682909323885761E82.2415242712669864E7-5.966176123188091E7
      \N\N01.2847032699693155E96.300096113768728E7-5.94963316209578E8
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource.pushRecord(ReduceRecordSource.java:267)
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordProcessor.run(ReduceRecordProcessor.java:248)
      	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:148)
      	... 14 more
      Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing vector batch (tag=0) \N\N09.285817653506076E84.639990363237801E7-1.1814318134887291E8
      \N\N04.682909323885761E82.2415242712669864E7-5.966176123188091E7
      \N\N01.2847032699693155E96.300096113768728E7-5.94963316209578E8
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource.processVectors(ReduceRecordSource.java:394)
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource.pushRecord(ReduceRecordSource.java:252)
      	... 16 more
      Caused by: java.lang.ClassCastException: org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector cannot be cast to org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector
      	at org.apache.hadoop.hive.ql.exec.vector.VectorGroupKeyHelper.copyGroupKey(VectorGroupKeyHelper.java:94)
      	at org.apache.hadoop.hive.ql.exec.vector.VectorGroupByOperator$ProcessingModeGroupBatches.processBatch(VectorGroupByOperator.java:729)
      	at org.apache.hadoop.hive.ql.exec.vector.VectorGroupByOperator.process(VectorGroupByOperator.java:878)
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource.processVectors(ReduceRecordSource.java:378)
      	... 17 more
      ]], Vertex failed as one or more tasks failed. failedTasks:1, Vertex vertex_1426707664723_1377_1_22 [Reducer 5] killed/failed due to:null]
      Vertex killed, vertexName=Reducer 6, vertexId=vertex_1426707664723_1377_1_23, diagnostics=[Vertex received Kill while in RUNNING state., Vertex killed as other vertex failed. failedTasks:0, Vertex vertex_1426707664723_1377_1_23 [Reducer 6] killed/failed due to:null]
      15/04/07 05:14:52 [main]: ERROR SessionState: Vertex killed, vertexName=Reducer 6, vertexId=vertex_1426707664723_1377_1_23, diagnostics=[Vertex received Kill while in RUNNING state., Vertex killed as other vertex failed. failedTasks:0, Vertex vertex_1426707664723_1377_1_23 [Reducer 6] killed/failed due to:null]
      DAG failed due to vertex failure. failedVertices:1 killedVertices:1
      15/04/07 05:14:52 [main]: ERROR SessionState: DAG failed due to vertex failure. failedVertices:1 killedVertices:1
      FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.tez.TezTask. Vertex failed, vertexName=Reducer 5, vertexId=vertex_1426707664723_1377_1_22, diagnostics=[Task failed, taskId=task_1426707664723_1377_1_22_000000, diagnostics=[TaskAttempt 0 failed, info=[Error: Failure while running task:java.lang.RuntimeException: java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing vector batch (tag=0) \N\N09.285817653506076E84.639990363237801E7-1.1814318134887291E8
      \N\N04.682909323885761E82.2415242712669864E7-5.966176123188091E7
      \N\N01.2847032699693155E96.300096113768728E7-5.94963316209578E8
      	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:171)
      	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.run(TezProcessor.java:137)
      	at org.apache.tez.runtime.LogicalIOProcessorRuntimeTask.run(LogicalIOProcessorRuntimeTask.java:330)
      	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable$1.run(TezTaskRunner.java:179)
      	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable$1.run(TezTaskRunner.java:171)
      	at java.security.AccessController.doPrivileged(Native Method)
      	at javax.security.auth.Subject.doAs(Subject.java:415)
      	at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1628)
      	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable.callInternal(TezTaskRunner.java:171)
      	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable.callInternal(TezTaskRunner.java:167)
      	at org.apache.tez.common.CallableWithNdc.call(CallableWithNdc.java:36)
      	at java.util.concurrent.FutureTask.run(FutureTask.java:262)
      	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:745)
      Caused by: java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing vector batch (tag=0) \N\N09.285817653506076E84.639990363237801E7-1.1814318134887291E8
      \N\N04.682909323885761E82.2415242712669864E7-5.966176123188091E7
      \N\N01.2847032699693155E96.300096113768728E7-5.94963316209578E8
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource.pushRecord(ReduceRecordSource.java:267)
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordProcessor.run(ReduceRecordProcessor.java:248)
      	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:148)
      	... 14 more
      Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing vector batch (tag=0) \N\N09.285817653506076E84.639990363237801E7-1.1814318134887291E8
      \N\N04.682909323885761E82.2415242712669864E7-5.966176123188091E7
      \N\N01.2847032699693155E96.300096113768728E7-5.94963316209578E8
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource.processVectors(ReduceRecordSource.java:394)
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource.pushRecord(ReduceRecordSource.java:252)
      	... 16 more
      Caused by: java.lang.ClassCastException: org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector cannot be cast to org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector
      	at org.apache.hadoop.hive.ql.exec.vector.VectorGroupKeyHelper.copyGroupKey(VectorGroupKeyHelper.java:94)
      	at org.apache.hadoop.hive.ql.exec.vector.VectorGroupByOperator$ProcessingModeGroupBatches.processBatch(VectorGroupByOperator.java:729)
      	at org.apache.hadoop.hive.ql.exec.vector.VectorGroupByOperator.process(VectorGroupByOperator.java:878)
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource.processVectors(ReduceRecordSource.java:378)
      	... 17 more
      ], TaskAttempt 1 failed, info=[Error: Failure while running task:java.lang.RuntimeException: java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing vector batch (tag=0) \N\N09.285817653506076E84.639990363237801E7-1.1814318134887291E8
      \N\N04.682909323885761E82.2415242712669864E7-5.966176123188091E7
      \N\N01.2847032699693155E96.300096113768728E7-5.94963316209578E8
      	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:171)
      	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.run(TezProcessor.java:137)
      	at org.apache.tez.runtime.LogicalIOProcessorRuntimeTask.run(LogicalIOProcessorRuntimeTask.java:330)
      	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable$1.run(TezTaskRunner.java:179)
      	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable$1.run(TezTaskRunner.java:171)
      	at java.security.AccessController.doPrivileged(Native Method)
      	at javax.security.auth.Subject.doAs(Subject.java:415)
      	at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1628)
      	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable.callInternal(TezTaskRunner.java:171)
      	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable.callInternal(TezTaskRunner.java:167)
      	at org.apache.tez.common.CallableWithNdc.call(CallableWithNdc.java:36)
      	at java.util.concurrent.FutureTask.run(FutureTask.java:262)
      	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:745)
      Caused by: java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing vector batch (tag=0) \N\N09.285817653506076E84.639990363237801E7-1.1814318134887291E8
      \N\N04.682909323885761E82.2415242712669864E7-5.966176123188091E7
      \N\N0☃00011.2847032699693155E96.300096113768728E7-5.94963316209578E8
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource.pushRecord(ReduceRecordSource.java:267)
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordProcessor.run(ReduceRecordProcessor.java:248)
      	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:148)
      	... 14 more
      Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing vector batch (tag=0) \N\N09.285817653506076E84.639990363237801E7-1.1814318134887291E8
      \N\N04.682909323885761E82.2415242712669864E7-5.966176123188091E7
      \N\N01.2847032699693155E96.300096113768728E7-5.94963316209578E8
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource.processVectors(ReduceRecordSource.java:394)
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource.pushRecord(ReduceRecordSource.java:252)
      	... 16 more
      Caused by: java.lang.ClassCastException: org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector cannot be cast to org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector
      	at org.apache.hadoop.hive.ql.exec.vector.VectorGroupKeyHelper.copyGroupKey(VectorGroupKeyHelper.java:94)
      	at org.apache.hadoop.hive.ql.exec.vector.VectorGroupByOperator$ProcessingModeGroupBatches.processBatch(VectorGroupByOperator.java:729)
      	at org.apache.hadoop.hive.ql.exec.vector.VectorGroupByOperator.process(VectorGroupByOperator.java:878)
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource.processVectors(ReduceRecordSource.java:378)
      	... 17 more
      ], TaskAttempt 2 failed, info=[Error: Failure while running task:java.lang.RuntimeException: java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing vector batch (tag=0) \N\N09.285817653506076E84.639990363237801E7-1.1814318134887291E8
      \N\N04.682909323885761E82.2415242712669864E7-5.966176123188091E7
      \N\N01.2847032699693155E96.300096113768728E7-5.94963316209578E8
      	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:171)
      	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.run(TezProcessor.java:137)
      	at org.apache.tez.runtime.LogicalIOProcessorRuntimeTask.run(LogicalIOProcessorRuntimeTask.java:330)
      	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable$1.run(TezTaskRunner.java:179)
      	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable$1.run(TezTaskRunner.java:171)
      	at java.security.AccessController.doPrivileged(Native Method)
      	at javax.security.auth.Subject.doAs(Subject.java:415)
      	at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1628)
      	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable.callInternal(TezTaskRunner.java:171)
      	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable.callInternal(TezTaskRunner.java:167)
      	at org.apache.tez.common.CallableWithNdc.call(CallableWithNdc.java:36)
      	at java.util.concurrent.FutureTask.run(FutureTask.java:262)
      	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:745)
      Caused by: java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing vector batch (tag=0) \N\N09.285817653506076E84.639990363237801E7-1.1814318134887291E8
      \N\N04.682909323885761E82.2415242712669864E7-5.966176123188091E7
      \N\N01.2847032699693155E96.300096113768728E7-5.94963316209578E8
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource.pushRecord(ReduceRecordSource.java:267)
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordProcessor.run(ReduceRecordProcessor.java:248)
      	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:148)
      	... 14 more
      Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing vector batch (tag=0) \N\N09.285817653506076E84.639990363237801E7-1.1814318134887291E8
      \N\N04.682909323885761E82.2415242712669864E7-5.966176123188091E7
      \N\N01.2847032699693155E96.300096113768728E7-5.94963316209578E8
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource.processVectors(ReduceRecordSource.java:394)
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource.pushRecord(ReduceRecordSource.java:252)
      	... 16 more
      Caused by: java.lang.ClassCastException: org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector cannot be cast to org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector
      	at org.apache.hadoop.hive.ql.exec.vector.VectorGroupKeyHelper.copyGroupKey(VectorGroupKeyHelper.java:94)
      	at org.apache.hadoop.hive.ql.exec.vector.VectorGroupByOperator$ProcessingModeGroupBatches.processBatch(VectorGroupByOperator.java:729)
      	at org.apache.hadoop.hive.ql.exec.vector.VectorGroupByOperator.process(VectorGroupByOperator.java:878)
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource.processVectors(ReduceRecordSource.java:378)
      	... 17 more
      ], TaskAttempt 3 failed, info=[Error: Failure while running task:java.lang.RuntimeException: java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing vector batch (tag=0) \N\N09.285817653506076E84.639990363237801E7-1.1814318134887291E8
      \N\N04.682909323885761E82.2415242712669864E7-5.966176123188091E7
      \N\N01.2847032699693155E96.300096113768728E7-5.94963316209578E8
      	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:171)
      	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.run(TezProcessor.java:137)
      	at org.apache.tez.runtime.LogicalIOProcessorRuntimeTask.run(LogicalIOProcessorRuntimeTask.java:330)
      	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable$1.run(TezTaskRunner.java:179)
      	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable$1.run(TezTaskRunner.java:171)
      	at java.security.AccessController.doPrivileged(Native Method)
      	at javax.security.auth.Subject.doAs(Subject.java:415)
      	at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1628)
      	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable.callInternal(TezTaskRunner.java:171)
      	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable.callInternal(TezTaskRunner.java:167)
      	at org.apache.tez.common.CallableWithNdc.call(CallableWithNdc.java:36)
      	at java.util.concurrent.FutureTask.run(FutureTask.java:262)
      	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:745)
      Caused by: java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing vector batch (tag=0) \N\N09.285817653506076E84.639990363237801E7-1.1814318134887291E8
      \N\N04.682909323885761E82.2415242712669864E7-5.966176123188091E7
      \N\N01.2847032699693155E96.300096113768728E7-5.94963316209578E8
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource.pushRecord(ReduceRecordSource.java:267)
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordProcessor.run(ReduceRecordProcessor.java:248)
      	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:148)
      	... 14 more
      Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing vector batch (tag=0) \N\N09.285817653506076E84.639990363237801E7-1.1814318134887291E8
      \N\N04.682909323885761E82.2415242712669864E7-5.966176123188091E7
      \N\N01.2847032699693155E96.300096113768728E7-5.94963316209578E8
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource.processVectors(ReduceRecordSource.java:394)
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource.pushRecord(ReduceRecordSource.java:252)
      	... 16 more
      Caused by: java.lang.ClassCastException: org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector cannot be cast to org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector
      	at org.apache.hadoop.hive.ql.exec.vector.VectorGroupKeyHelper.copyGroupKey(VectorGroupKeyHelper.java:94)
      	at org.apache.hadoop.hive.ql.exec.vector.VectorGroupByOperator$ProcessingModeGroupBatches.processBatch(VectorGroupByOperator.java:729)
      	at org.apache.hadoop.hive.ql.exec.vector.VectorGroupByOperator.process(VectorGroupByOperator.java:878)
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource.processVectors(ReduceRecordSource.java:378)
      	... 17 more
      ]], Vertex failed as one or more tasks failed. failedTasks:1, Vertex vertex_1426707664723_1377_1_22 [Reducer 5] killed/failed due to:null]Vertex killed, vertexName=Reducer 6, vertexId=vertex_1426707664723_1377_1_23, diagnostics=[Vertex received Kill while in RUNNING state., Vertex killed as other vertex failed. failedTasks:0, Vertex vertex_1426707664723_1377_1_23 [Reducer 6] killed/failed due to:null]DAG failed due to vertex failure. failedVertices:1 killedVertices:1
      15/04/07 05:14:52 [main]: ERROR ql.Driver: FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.tez.TezTask. Vertex failed, vertexName=Reducer 5, vertexId=vertex_1426707664723_1377_1_22, diagnostics=[Task failed, taskId=task_1426707664723_1377_1_22_000000, diagnostics=[TaskAttempt 0 failed, info=[Error: Failure while running task:java.lang.RuntimeException: java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing vector batch (tag=0) \N\N09.285817653506076E84.639990363237801E7-1.1814318134887291E8
      \N\N04.682909323885761E82.2415242712669864E7-5.966176123188091E7
      \N\N01.2847032699693155E96.300096113768728E7-5.94963316209578E8
      	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:171)
      	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.run(TezProcessor.java:137)
      	at org.apache.tez.runtime.LogicalIOProcessorRuntimeTask.run(LogicalIOProcessorRuntimeTask.java:330)
      	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable$1.run(TezTaskRunner.java:179)
      	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable$1.run(TezTaskRunner.java:171)
      	at java.security.AccessController.doPrivileged(Native Method)
      	at javax.security.auth.Subject.doAs(Subject.java:415)
      	at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1628)
      	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable.callInternal(TezTaskRunner.java:171)
      	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable.callInternal(TezTaskRunner.java:167)
      	at org.apache.tez.common.CallableWithNdc.call(CallableWithNdc.java:36)
      	at java.util.concurrent.FutureTask.run(FutureTask.java:262)
      	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:745)
      Caused by: java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing vector batch (tag=0) \N\N09.285817653506076E84.639990363237801E7-1.1814318134887291E8
      \N\N04.682909323885761E82.2415242712669864E7-5.966176123188091E7
      \N\N01.2847032699693155E96.300096113768728E7-5.94963316209578E8
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource.pushRecord(ReduceRecordSource.java:267)
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordProcessor.run(ReduceRecordProcessor.java:248)
      	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:148)
      	... 14 more
      Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing vector batch (tag=0) \N\N09.285817653506076E84.639990363237801E7-1.1814318134887291E8
      \N\N04.682909323885761E82.2415242712669864E7-5.966176123188091E7
      \N\N01.2847032699693155E96.300096113768728E7-5.94963316209578E8
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource.processVectors(ReduceRecordSource.java:394)
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource.pushRecord(ReduceRecordSource.java:252)
      	... 16 more
      Caused by: java.lang.ClassCastException: org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector cannot be cast to org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector
      	at org.apache.hadoop.hive.ql.exec.vector.VectorGroupKeyHelper.copyGroupKey(VectorGroupKeyHelper.java:94)
      	at org.apache.hadoop.hive.ql.exec.vector.VectorGroupByOperator$ProcessingModeGroupBatches.processBatch(VectorGroupByOperator.java:729)
      	at org.apache.hadoop.hive.ql.exec.vector.VectorGroupByOperator.process(VectorGroupByOperator.java:878)
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource.processVectors(ReduceRecordSource.java:378)
      	... 17 more
      ], TaskAttempt 1 failed, info=[Error: Failure while running task:java.lang.RuntimeException: java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing vector batch (tag=0) \N\N09.285817653506076E84.639990363237801E7-1.1814318134887291E8
      \N\N04.682909323885761E82.2415242712669864E7-5.966176123188091E7
      \N\N01.2847032699693155E96.300096113768728E7-5.94963316209578E8
      	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:171)
      	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.run(TezProcessor.java:137)
      	at org.apache.tez.runtime.LogicalIOProcessorRuntimeTask.run(LogicalIOProcessorRuntimeTask.java:330)
      	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable$1.run(TezTaskRunner.java:179)
      	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable$1.run(TezTaskRunner.java:171)
      	at java.security.AccessController.doPrivileged(Native Method)
      	at javax.security.auth.Subject.doAs(Subject.java:415)
      	at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1628)
      	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable.callInternal(TezTaskRunner.java:171)
      	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable.callInternal(TezTaskRunner.java:167)
      	at org.apache.tez.common.CallableWithNdc.call(CallableWithNdc.java:36)
      	at java.util.concurrent.FutureTask.run(FutureTask.java:262)
      	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:745)
      Caused by: java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing vector batch (tag=0) \N\N09.285817653506076E84.639990363237801E7-1.1814318134887291E8
      \N\N04.682909323885761E82.2415242712669864E7-5.966176123188091E7
      \N\N01.2847032699693155E96.300096113768728E7-5.94963316209578E8
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource.pushRecord(ReduceRecordSource.java:267)
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordProcessor.run(ReduceRecordProcessor.java:248)
      	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:148)
      	... 14 more
      Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing vector batch (tag=0) \N\N09.285817653506076E84.639990363237801E7-1.1814318134887291E8
      \N\N04.682909323885761E82.2415242712669864E7-5.966176123188091E7
      \N\N01.2847032699693155E96.300096113768728E7-5.94963316209578E8
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource.processVectors(ReduceRecordSource.java:394)
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource.pushRecord(ReduceRecordSource.java:252)
      	... 16 more
      Caused by: java.lang.ClassCastException: org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector cannot be cast to org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector
      	at org.apache.hadoop.hive.ql.exec.vector.VectorGroupKeyHelper.copyGroupKey(VectorGroupKeyHelper.java:94)
      	at org.apache.hadoop.hive.ql.exec.vector.VectorGroupByOperator$ProcessingModeGroupBatches.processBatch(VectorGroupByOperator.java:729)
      	at org.apache.hadoop.hive.ql.exec.vector.VectorGroupByOperator.process(VectorGroupByOperator.java:878)
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource.processVectors(ReduceRecordSource.java:378)
      	... 17 more
      ], TaskAttempt 2 failed, info=[Error: Failure while running task:java.lang.RuntimeException: java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing vector batch (tag=0) \N\N09.285817653506076E84.639990363237801E7-1.1814318134887291E8
      \N\N04.682909323885761E82.2415242712669864E7-5.966176123188091E7
      \N\N01.2847032699693155E96.300096113768728E7-5.94963316209578E8
      	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:171)
      	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.run(TezProcessor.java:137)
      	at org.apache.tez.runtime.LogicalIOProcessorRuntimeTask.run(LogicalIOProcessorRuntimeTask.java:330)
      	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable$1.run(TezTaskRunner.java:179)
      	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable$1.run(TezTaskRunner.java:171)
      	at java.security.AccessController.doPrivileged(Native Method)
      	at javax.security.auth.Subject.doAs(Subject.java:415)
      	at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1628)
      	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable.callInternal(TezTaskRunner.java:171)
      	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable.callInternal(TezTaskRunner.java:167)
      	at org.apache.tez.common.CallableWithNdc.call(CallableWithNdc.java:36)
      	at java.util.concurrent.FutureTask.run(FutureTask.java:262)
      	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:745)
      Caused by: java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing vector batch (tag=0) \N\N09.285817653506076E84.639990363237801E7-1.1814318134887291E8
      \N\N04.682909323885761E8☃00012.2415242712669864E7-5.966176123188091E7
      \N\N01.2847032699693155E96.300096113768728E7-5.94963316209578E8
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource.pushRecord(ReduceRecordSource.java:267)
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordProcessor.run(ReduceRecordProcessor.java:248)
      	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:148)
      	... 14 more
      Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing vector batch (tag=0) \N\N09.285817653506076E84.639990363237801E7-1.1814318134887291E8
      \N\N04.682909323885761E82.2415242712669864E7-5.966176123188091E7
      \N\N01.2847032699693155E96.300096113768728E7-5.94963316209578E8
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource.processVectors(ReduceRecordSource.java:394)
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource.pushRecord(ReduceRecordSource.java:252)
      	... 16 more
      Caused by: java.lang.ClassCastException: org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector cannot be cast to org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector
      	at org.apache.hadoop.hive.ql.exec.vector.VectorGroupKeyHelper.copyGroupKey(VectorGroupKeyHelper.java:94)
      	at org.apache.hadoop.hive.ql.exec.vector.VectorGroupByOperator$ProcessingModeGroupBatches.processBatch(VectorGroupByOperator.java:729)
      	at org.apache.hadoop.hive.ql.exec.vector.VectorGroupByOperator.process(VectorGroupByOperator.java:878)
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource.processVectors(ReduceRecordSource.java:378)
      	... 17 more
      ], TaskAttempt 3 failed, info=[Error: Failure while running task:java.lang.RuntimeException: java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing vector batch (tag=0) \N\N09.285817653506076E84.639990363237801E7-1.1814318134887291E8
      \N\N04.682909323885761E82.2415242712669864E7-5.966176123188091E7
      \N\N01.2847032699693155E96.300096113768728E7-5.94963316209578E8
      	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:171)
      	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.run(TezProcessor.java:137)
      	at org.apache.tez.runtime.LogicalIOProcessorRuntimeTask.run(LogicalIOProcessorRuntimeTask.java:330)
      	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable$1.run(TezTaskRunner.java:179)
      	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable$1.run(TezTaskRunner.java:171)
      	at java.security.AccessController.doPrivileged(Native Method)
      	at javax.security.auth.Subject.doAs(Subject.java:415)
      	at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1628)
      	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable.callInternal(TezTaskRunner.java:171)
      	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable.callInternal(TezTaskRunner.java:167)
      	at org.apache.tez.common.CallableWithNdc.call(CallableWithNdc.java:36)
      	at java.util.concurrent.FutureTask.run(FutureTask.java:262)
      	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:745)
      Caused by: java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing vector batch (tag=0) \N\N09.285817653506076E84.639990363237801E7-1.1814318134887291E8
      \N\N04.682909323885761E82.2415242712669864E7-5.966176123188091E7
      \N\N01.2847032699693155E96.300096113768728E7-5.94963316209578E8
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource.pushRecord(ReduceRecordSource.java:267)
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordProcessor.run(ReduceRecordProcessor.java:248)
      	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:148)
      	... 14 more
      Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: Hive Runtime Error while processing vector batch (tag=0) \N\N09.285817653506076E84.639990363237801E7-1.1814318134887291E8
      \N\N04.682909323885761E82.2415242712669864E7-5.966176123188091E7
      \N\N01.2847032699693155E96.300096113768728E7-5.94963316209578E8
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource.processVectors(ReduceRecordSource.java:394)
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource.pushRecord(ReduceRecordSource.java:252)
      	... 16 more
      Caused by: java.lang.ClassCastException: org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector cannot be cast to org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector
      	at org.apache.hadoop.hive.ql.exec.vector.VectorGroupKeyHelper.copyGroupKey(VectorGroupKeyHelper.java:94)
      	at org.apache.hadoop.hive.ql.exec.vector.VectorGroupByOperator$ProcessingModeGroupBatches.processBatch(VectorGroupByOperator.java:729)
      	at org.apache.hadoop.hive.ql.exec.vector.VectorGroupByOperator.process(VectorGroupByOperator.java:878)
      	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource.processVectors(ReduceRecordSource.java:378)
      	... 17 more
      ]], Vertex failed as one or more tasks failed. failedTasks:1, Vertex vertex_1426707664723_1377_1_22 [Reducer 5] killed/failed due to:null]Vertex killed, vertexName=Reducer 6, vertexId=vertex_1426707664723_1377_1_23, diagnostics=[Vertex received Kill while in RUNNING state., Vertex killed as other vertex failed. failedTasks:0, Vertex vertex_1426707664723_1377_1_23 [Reducer 6] killed/failed due to:null]DAG failed due to vertex failure. failedVertices:1 killedVertices:1
      

      Plan

      STAGE DEPENDENCIES:
        Stage-1 is a root stage
        Stage-0 depends on stages: Stage-1
      
      STAGE PLANS:
        Stage: Stage-1
          Tez
            Edges:
              Map 12 <- Map 14 (BROADCAST_EDGE), Map 15 (BROADCAST_EDGE), Map 16 (BROADCAST_EDGE), Map 17 (BROADCAST_EDGE), Map 18 (BROADCAST_EDGE)
              Map 19 <- Map 21 (BROADCAST_EDGE), Map 22 (BROADCAST_EDGE), Map 23 (BROADCAST_EDGE), Map 24 (BROADCAST_EDGE), Map 25 (BROADCAST_EDGE)
              Reducer 13 <- Map 12 (SIMPLE_EDGE), Union 4 (CONTAINS)
              Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 10 (BROADCAST_EDGE), Map 11 (BROADCAST_EDGE), Map 7 (SIMPLE_EDGE), Map 8 (BROADCAST_EDGE), Map 9 (BROADCAST_EDGE)
              Reducer 20 <- Map 19 (SIMPLE_EDGE), Union 4 (CONTAINS)
              Reducer 3 <- Reducer 2 (SIMPLE_EDGE), Union 4 (CONTAINS)
              Reducer 5 <- Union 4 (SIMPLE_EDGE)
              Reducer 6 <- Reducer 5 (SIMPLE_EDGE)
            DagName: mmokhtar_20150407051226_eb6d232e-cb00-4174-8b2f-d70aa2b3fb15:1
            Vertices:
              Map 1 
                  Map Operator Tree:
                      TableScan
                        alias: store_sales
                        filterExpr: ((ss_item_sk is not null and ss_promo_sk is not null) and ss_store_sk is not null) (type: boolean)
                        Statistics: Num rows: 550076554 Data size: 47370018896 Basic stats: COMPLETE Column stats: COMPLETE
                        Filter Operator
                          predicate: ((ss_item_sk is not null and ss_promo_sk is not null) and ss_store_sk is not null) (type: boolean)
                          Statistics: Num rows: 524469260 Data size: 14487496336 Basic stats: COMPLETE Column stats: COMPLETE
                          Select Operator
                            expressions: ss_item_sk (type: int), ss_store_sk (type: int), ss_promo_sk (type: int), ss_ticket_number (type: int), ss_ext_sales_price (type: float), ss_net_profit (type: float), ss_sold_date_sk (type: int)
                            outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
                            Statistics: Num rows: 524469260 Data size: 14487496336 Basic stats: COMPLETE Column stats: COMPLETE
                            Reduce Output Operator
                              key expressions: _col0 (type: int), _col3 (type: int)
                              sort order: ++
                              Map-reduce partition columns: _col0 (type: int), _col3 (type: int)
                              Statistics: Num rows: 524469260 Data size: 14487496336 Basic stats: COMPLETE Column stats: COMPLETE
                              value expressions: _col1 (type: int), _col2 (type: int), _col4 (type: float), _col5 (type: float), _col6 (type: int)
                  Execution mode: vectorized
              Map 10 
                  Map Operator Tree:
                      TableScan
                        alias: promotion
                        filterExpr: ((p_channel_tv = 'N') and p_promo_sk is not null) (type: boolean)
                        Statistics: Num rows: 450 Data size: 530848 Basic stats: COMPLETE Column stats: COMPLETE
                        Filter Operator
                          predicate: ((p_channel_tv = 'N') and p_promo_sk is not null) (type: boolean)
                          Statistics: Num rows: 225 Data size: 20025 Basic stats: COMPLETE Column stats: COMPLETE
                          Select Operator
                            expressions: p_promo_sk (type: int)
                            outputColumnNames: _col0
                            Statistics: Num rows: 225 Data size: 900 Basic stats: COMPLETE Column stats: COMPLETE
                            Reduce Output Operator
                              key expressions: _col0 (type: int)
                              sort order: +
                              Map-reduce partition columns: _col0 (type: int)
                              Statistics: Num rows: 225 Data size: 900 Basic stats: COMPLETE Column stats: COMPLETE
                  Execution mode: vectorized
              Map 11 
                  Map Operator Tree:
                      TableScan
                        alias: store
                        filterExpr: s_store_sk is not null (type: boolean)
                        Statistics: Num rows: 212 Data size: 405680 Basic stats: COMPLETE Column stats: COMPLETE
                        Filter Operator
                          predicate: s_store_sk is not null (type: boolean)
                          Statistics: Num rows: 212 Data size: 22048 Basic stats: COMPLETE Column stats: COMPLETE
                          Select Operator
                            expressions: s_store_sk (type: int), s_store_id (type: string)
                            outputColumnNames: _col0, _col1
                            Statistics: Num rows: 212 Data size: 22048 Basic stats: COMPLETE Column stats: COMPLETE
                            Reduce Output Operator
                              key expressions: _col0 (type: int)
                              sort order: +
                              Map-reduce partition columns: _col0 (type: int)
                              Statistics: Num rows: 212 Data size: 22048 Basic stats: COMPLETE Column stats: COMPLETE
                              value expressions: _col1 (type: string)
                  Execution mode: vectorized
              Map 12 
                  Map Operator Tree:
                      TableScan
                        alias: catalog_sales
                        filterExpr: ((cs_item_sk is not null and cs_promo_sk is not null) and cs_catalog_page_sk is not null) (type: boolean)
                        Statistics: Num rows: 286549727 Data size: 37743959324 Basic stats: COMPLETE Column stats: COMPLETE
                        Filter Operator
                          predicate: ((cs_item_sk is not null and cs_promo_sk is not null) and cs_catalog_page_sk is not null) (type: boolean)
                          Statistics: Num rows: 285112475 Data size: 7974560516 Basic stats: COMPLETE Column stats: COMPLETE
                          Select Operator
                            expressions: cs_catalog_page_sk (type: int), cs_item_sk (type: int), cs_promo_sk (type: int), cs_order_number (type: int), cs_ext_sales_price (type: float), cs_net_profit (type: float), cs_sold_date_sk (type: int)
                            outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
                            Statistics: Num rows: 285112475 Data size: 7974560516 Basic stats: COMPLETE Column stats: COMPLETE
                            Map Join Operator
                              condition map:
                                   Left Outer Join0 to 1
                              keys:
                                0 _col1 (type: int), _col3 (type: int)
                                1 _col0 (type: int), _col1 (type: int)
                              outputColumnNames: _col0, _col1, _col2, _col4, _col5, _col6, _col9, _col10
                              input vertices:
                                1 Map 14
                              Statistics: Num rows: 3412616 Data size: 109203712 Basic stats: COMPLETE Column stats: COMPLETE
                              Map Join Operator
                                condition map:
                                     Inner Join 0 to 1
                                keys:
                                  0 _col6 (type: int)
                                  1 _col0 (type: int)
                                outputColumnNames: _col0, _col1, _col2, _col4, _col5, _col9, _col10
                                input vertices:
                                  1 Map 15
                                Statistics: Num rows: 3815661 Data size: 106838508 Basic stats: COMPLETE Column stats: COMPLETE
                                Map Join Operator
                                  condition map:
                                       Inner Join 0 to 1
                                  keys:
                                    0 _col1 (type: int)
                                    1 _col0 (type: int)
                                  outputColumnNames: _col0, _col2, _col4, _col5, _col9, _col10
                                  input vertices:
                                    1 Map 16
                                  Statistics: Num rows: 1271887 Data size: 30525288 Basic stats: COMPLETE Column stats: COMPLETE
                                  Map Join Operator
                                    condition map:
                                         Inner Join 0 to 1
                                    keys:
                                      0 _col2 (type: int)
                                      1 _col0 (type: int)
                                    outputColumnNames: _col0, _col4, _col5, _col9, _col10
                                    input vertices:
                                      1 Map 17
                                    Statistics: Num rows: 635944 Data size: 12718880 Basic stats: COMPLETE Column stats: COMPLETE
                                    Map Join Operator
                                      condition map:
                                           Inner Join 0 to 1
                                      keys:
                                        0 _col0 (type: int)
                                        1 _col0 (type: int)
                                      outputColumnNames: _col4, _col5, _col9, _col10, _col18
                                      input vertices:
                                        1 Map 18
                                      Statistics: Num rows: 635944 Data size: 73769504 Basic stats: COMPLETE Column stats: COMPLETE
                                      Select Operator
                                        expressions: _col18 (type: string), _col4 (type: float), COALESCE(_col9,0) (type: float), (_col5 - COALESCE(_col10,0)) (type: float)
                                        outputColumnNames: _col0, _col1, _col2, _col3
                                        Statistics: Num rows: 635944 Data size: 73769504 Basic stats: COMPLETE Column stats: COMPLETE
                                        Group By Operator
                                          aggregations: sum(_col1), sum(_col2), sum(_col3)
                                          keys: _col0 (type: string)
                                          mode: hash
                                          outputColumnNames: _col0, _col1, _col2, _col3
                                          Statistics: Num rows: 10590 Data size: 1313160 Basic stats: COMPLETE Column stats: COMPLETE
                                          Reduce Output Operator
                                            key expressions: _col0 (type: string)
                                            sort order: +
                                            Map-reduce partition columns: _col0 (type: string)
                                            Statistics: Num rows: 10590 Data size: 1313160 Basic stats: COMPLETE Column stats: COMPLETE
                                            value expressions: _col1 (type: double), _col2 (type: double), _col3 (type: double)
                  Execution mode: vectorized
              Map 14 
                  Map Operator Tree:
                      TableScan
                        alias: catalog_returns
                        filterExpr: cr_item_sk is not null (type: boolean)
                        Statistics: Num rows: 28798881 Data size: 2942039156 Basic stats: COMPLETE Column stats: COMPLETE
                        Filter Operator
                          predicate: cr_item_sk is not null (type: boolean)
                          Statistics: Num rows: 28798881 Data size: 456171072 Basic stats: COMPLETE Column stats: COMPLETE
                          Select Operator
                            expressions: cr_item_sk (type: int), cr_order_number (type: int), cr_return_amount (type: float), cr_net_loss (type: float)
                            outputColumnNames: _col0, _col1, _col2, _col3
                            Statistics: Num rows: 28798881 Data size: 456171072 Basic stats: COMPLETE Column stats: COMPLETE
                            Reduce Output Operator
                              key expressions: _col0 (type: int), _col1 (type: int)
                              sort order: ++
                              Map-reduce partition columns: _col0 (type: int), _col1 (type: int)
                              Statistics: Num rows: 28798881 Data size: 456171072 Basic stats: COMPLETE Column stats: COMPLETE
                              value expressions: _col2 (type: float), _col3 (type: float)
                  Execution mode: vectorized
              Map 15 
                  Map Operator Tree:
                      TableScan
                        alias: date_dim
                        filterExpr: (d_date BETWEEN 1998-08-04 AND 1998-09-04 and d_date_sk is not null) (type: boolean)
                        Statistics: Num rows: 73049 Data size: 81741831 Basic stats: COMPLETE Column stats: COMPLETE
                        Filter Operator
                          predicate: (d_date BETWEEN 1998-08-04 AND 1998-09-04 and d_date_sk is not null) (type: boolean)
                          Statistics: Num rows: 36524 Data size: 3579352 Basic stats: COMPLETE Column stats: COMPLETE
                          Select Operator
                            expressions: d_date_sk (type: int)
                            outputColumnNames: _col0
                            Statistics: Num rows: 36524 Data size: 146096 Basic stats: COMPLETE Column stats: COMPLETE
                            Reduce Output Operator
                              key expressions: _col0 (type: int)
                              sort order: +
                              Map-reduce partition columns: _col0 (type: int)
                              Statistics: Num rows: 36524 Data size: 146096 Basic stats: COMPLETE Column stats: COMPLETE
                            Select Operator
                              expressions: _col0 (type: int)
                              outputColumnNames: _col0
                              Statistics: Num rows: 36524 Data size: 146096 Basic stats: COMPLETE Column stats: COMPLETE
                              Group By Operator
                                keys: _col0 (type: int)
                                mode: hash
                                outputColumnNames: _col0
                                Statistics: Num rows: 18262 Data size: 73048 Basic stats: COMPLETE Column stats: COMPLETE
                                Dynamic Partitioning Event Operator
                                  Target Input: catalog_sales
                                  Partition key expr: cs_sold_date_sk
                                  Statistics: Num rows: 18262 Data size: 73048 Basic stats: COMPLETE Column stats: COMPLETE
                                  Target column: cs_sold_date_sk
                                  Target Vertex: Map 12
                  Execution mode: vectorized
              Map 16 
                  Map Operator Tree:
                      TableScan
                        alias: item
                        filterExpr: ((i_current_price > 50.0) and i_item_sk is not null) (type: boolean)
                        Statistics: Num rows: 48000 Data size: 68732712 Basic stats: COMPLETE Column stats: COMPLETE
                        Filter Operator
                          predicate: ((i_current_price > 50.0) and i_item_sk is not null) (type: boolean)
                          Statistics: Num rows: 16000 Data size: 127832 Basic stats: COMPLETE Column stats: COMPLETE
                          Select Operator
                            expressions: i_item_sk (type: int)
                            outputColumnNames: _col0
                            Statistics: Num rows: 16000 Data size: 64000 Basic stats: COMPLETE Column stats: COMPLETE
                            Reduce Output Operator
                              key expressions: _col0 (type: int)
                              sort order: +
                              Map-reduce partition columns: _col0 (type: int)
                              Statistics: Num rows: 16000 Data size: 64000 Basic stats: COMPLETE Column stats: COMPLETE
                  Execution mode: vectorized
              Map 17 
                  Map Operator Tree:
                      TableScan
                        alias: promotion
                        filterExpr: ((p_channel_tv = 'N') and p_promo_sk is not null) (type: boolean)
                        Statistics: Num rows: 450 Data size: 530848 Basic stats: COMPLETE Column stats: COMPLETE
                        Filter Operator
                          predicate: ((p_channel_tv = 'N') and p_promo_sk is not null) (type: boolean)
                          Statistics: Num rows: 225 Data size: 20025 Basic stats: COMPLETE Column stats: COMPLETE
                          Select Operator
                            expressions: p_promo_sk (type: int)
                            outputColumnNames: _col0
                            Statistics: Num rows: 225 Data size: 900 Basic stats: COMPLETE Column stats: COMPLETE
                            Reduce Output Operator
                              key expressions: _col0 (type: int)
                              sort order: +
                              Map-reduce partition columns: _col0 (type: int)
                              Statistics: Num rows: 225 Data size: 900 Basic stats: COMPLETE Column stats: COMPLETE
                  Execution mode: vectorized
              Map 18 
                  Map Operator Tree:
                      TableScan
                        alias: catalog_page
                        filterExpr: cp_catalog_page_sk is not null (type: boolean)
                        Statistics: Num rows: 11718 Data size: 5400282 Basic stats: COMPLETE Column stats: COMPLETE
                        Filter Operator
                          predicate: cp_catalog_page_sk is not null (type: boolean)
                          Statistics: Num rows: 11718 Data size: 1218672 Basic stats: COMPLETE Column stats: COMPLETE
                          Select Operator
                            expressions: cp_catalog_page_sk (type: int), cp_catalog_page_id (type: string)
                            outputColumnNames: _col0, _col1
                            Statistics: Num rows: 11718 Data size: 1218672 Basic stats: COMPLETE Column stats: COMPLETE
                            Reduce Output Operator
                              key expressions: _col0 (type: int)
                              sort order: +
                              Map-reduce partition columns: _col0 (type: int)
                              Statistics: Num rows: 11718 Data size: 1218672 Basic stats: COMPLETE Column stats: COMPLETE
                              value expressions: _col1 (type: string)
                  Execution mode: vectorized
              Map 19 
                  Map Operator Tree:
                      TableScan
                        alias: web_sales
                        filterExpr: ((ws_item_sk is not null and ws_promo_sk is not null) and ws_web_site_sk is not null) (type: boolean)
                        Statistics: Num rows: 143966864 Data size: 19001610332 Basic stats: COMPLETE Column stats: COMPLETE
                        Filter Operator
                          predicate: ((ws_item_sk is not null and ws_promo_sk is not null) and ws_web_site_sk is not null) (type: boolean)
                          Statistics: Num rows: 143930635 Data size: 4029840544 Basic stats: COMPLETE Column stats: COMPLETE
                          Select Operator
                            expressions: ws_item_sk (type: int), ws_web_site_sk (type: int), ws_promo_sk (type: int), ws_order_number (type: int), ws_ext_sales_price (type: float), ws_net_profit (type: float), ws_sold_date_sk (type: int)
                            outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
                            Statistics: Num rows: 143930635 Data size: 4029840544 Basic stats: COMPLETE Column stats: COMPLETE
                            Map Join Operator
                              condition map:
                                   Left Outer Join0 to 1
                              keys:
                                0 _col0 (type: int), _col3 (type: int)
                                1 _col0 (type: int), _col1 (type: int)
                              outputColumnNames: _col0, _col1, _col2, _col4, _col5, _col6, _col9, _col10
                              input vertices:
                                1 Map 21
                              Statistics: Num rows: 2406359 Data size: 77003488 Basic stats: COMPLETE Column stats: COMPLETE
                              Map Join Operator
                                condition map:
                                     Inner Join 0 to 1
                                keys:
                                  0 _col6 (type: int)
                                  1 _col0 (type: int)
                                outputColumnNames: _col0, _col1, _col2, _col4, _col5, _col9, _col10
                                input vertices:
                                  1 Map 22
                                Statistics: Num rows: 2690560 Data size: 75335680 Basic stats: COMPLETE Column stats: COMPLETE
                                Map Join Operator
                                  condition map:
                                       Inner Join 0 to 1
                                  keys:
                                    0 _col0 (type: int)
                                    1 _col0 (type: int)
                                  outputColumnNames: _col1, _col2, _col4, _col5, _col9, _col10
                                  input vertices:
                                    1 Map 23
                                  Statistics: Num rows: 896854 Data size: 21524496 Basic stats: COMPLETE Column stats: COMPLETE
                                  Map Join Operator
                                    condition map:
                                         Inner Join 0 to 1
                                    keys:
                                      0 _col2 (type: int)
                                      1 _col0 (type: int)
                                    outputColumnNames: _col1, _col4, _col5, _col9, _col10
                                    input vertices:
                                      1 Map 24
                                    Statistics: Num rows: 448427 Data size: 8968540 Basic stats: COMPLETE Column stats: COMPLETE
                                    Map Join Operator
                                      condition map:
                                           Inner Join 0 to 1
                                      keys:
                                        0 _col1 (type: int)
                                        1 _col0 (type: int)
                                      outputColumnNames: _col4, _col5, _col9, _col10, _col18
                                      input vertices:
                                        1 Map 25
                                      Statistics: Num rows: 448427 Data size: 52017532 Basic stats: COMPLETE Column stats: COMPLETE
                                      Select Operator
                                        expressions: _col18 (type: string), _col4 (type: float), COALESCE(_col9,0) (type: float), (_col5 - COALESCE(_col10,0)) (type: float)
                                        outputColumnNames: _col0, _col1, _col2, _col3
                                        Statistics: Num rows: 448427 Data size: 52017532 Basic stats: COMPLETE Column stats: COMPLETE
                                        Group By Operator
                                          aggregations: sum(_col1), sum(_col2), sum(_col3)
                                          keys: _col0 (type: string)
                                          mode: hash
                                          outputColumnNames: _col0, _col1, _col2, _col3
                                          Statistics: Num rows: 17 Data size: 2108 Basic stats: COMPLETE Column stats: COMPLETE
                                          Reduce Output Operator
                                            key expressions: _col0 (type: string)
                                            sort order: +
                                            Map-reduce partition columns: _col0 (type: string)
                                            Statistics: Num rows: 17 Data size: 2108 Basic stats: COMPLETE Column stats: COMPLETE
                                            value expressions: _col1 (type: double), _col2 (type: double), _col3 (type: double)
                  Execution mode: vectorized
              Map 21 
                  Map Operator Tree:
                      TableScan
                        alias: web_returns
                        filterExpr: wr_item_sk is not null (type: boolean)
                        Statistics: Num rows: 13749816 Data size: 1237758344 Basic stats: COMPLETE Column stats: COMPLETE
                        Filter Operator
                          predicate: wr_item_sk is not null (type: boolean)
                          Statistics: Num rows: 13749816 Data size: 217404672 Basic stats: COMPLETE Column stats: COMPLETE
                          Select Operator
                            expressions: wr_item_sk (type: int), wr_order_number (type: int), wr_return_amt (type: float), wr_net_loss (type: float)
                            outputColumnNames: _col0, _col1, _col2, _col3
                            Statistics: Num rows: 13749816 Data size: 217404672 Basic stats: COMPLETE Column stats: COMPLETE
                            Reduce Output Operator
                              key expressions: _col0 (type: int), _col1 (type: int)
                              sort order: ++
                              Map-reduce partition columns: _col0 (type: int), _col1 (type: int)
                              Statistics: Num rows: 13749816 Data size: 217404672 Basic stats: COMPLETE Column stats: COMPLETE
                              value expressions: _col2 (type: float), _col3 (type: float)
                  Execution mode: vectorized
              Map 22 
                  Map Operator Tree:
                      TableScan
                        alias: date_dim
                        filterExpr: (d_date BETWEEN 1998-08-04 AND 1998-09-04 and d_date_sk is not null) (type: boolean)
                        Statistics: Num rows: 73049 Data size: 81741831 Basic stats: COMPLETE Column stats: COMPLETE
                        Filter Operator
                          predicate: (d_date BETWEEN 1998-08-04 AND 1998-09-04 and d_date_sk is not null) (type: boolean)
                          Statistics: Num rows: 36524 Data size: 3579352 Basic stats: COMPLETE Column stats: COMPLETE
                          Select Operator
                            expressions: d_date_sk (type: int)
                            outputColumnNames: _col0
                            Statistics: Num rows: 36524 Data size: 146096 Basic stats: COMPLETE Column stats: COMPLETE
                            Reduce Output Operator
                              key expressions: _col0 (type: int)
                              sort order: +
                              Map-reduce partition columns: _col0 (type: int)
                              Statistics: Num rows: 36524 Data size: 146096 Basic stats: COMPLETE Column stats: COMPLETE
                            Select Operator
                              expressions: _col0 (type: int)
                              outputColumnNames: _col0
                              Statistics: Num rows: 36524 Data size: 146096 Basic stats: COMPLETE Column stats: COMPLETE
                              Group By Operator
                                keys: _col0 (type: int)
                                mode: hash
                                outputColumnNames: _col0
                                Statistics: Num rows: 18262 Data size: 73048 Basic stats: COMPLETE Column stats: COMPLETE
                                Dynamic Partitioning Event Operator
                                  Target Input: web_sales
                                  Partition key expr: ws_sold_date_sk
                                  Statistics: Num rows: 18262 Data size: 73048 Basic stats: COMPLETE Column stats: COMPLETE
                                  Target column: ws_sold_date_sk
                                  Target Vertex: Map 19
                  Execution mode: vectorized
              Map 23 
                  Map Operator Tree:
                      TableScan
                        alias: item
                        filterExpr: ((i_current_price > 50.0) and i_item_sk is not null) (type: boolean)
                        Statistics: Num rows: 48000 Data size: 68732712 Basic stats: COMPLETE Column stats: COMPLETE
                        Filter Operator
                          predicate: ((i_current_price > 50.0) and i_item_sk is not null) (type: boolean)
                          Statistics: Num rows: 16000 Data size: 127832 Basic stats: COMPLETE Column stats: COMPLETE
                          Select Operator
                            expressions: i_item_sk (type: int)
                            outputColumnNames: _col0
                            Statistics: Num rows: 16000 Data size: 64000 Basic stats: COMPLETE Column stats: COMPLETE
                            Reduce Output Operator
                              key expressions: _col0 (type: int)
                              sort order: +
                              Map-reduce partition columns: _col0 (type: int)
                              Statistics: Num rows: 16000 Data size: 64000 Basic stats: COMPLETE Column stats: COMPLETE
                  Execution mode: vectorized
              Map 24 
                  Map Operator Tree:
                      TableScan
                        alias: promotion
                        filterExpr: ((p_channel_tv = 'N') and p_promo_sk is not null) (type: boolean)
                        Statistics: Num rows: 450 Data size: 530848 Basic stats: COMPLETE Column stats: COMPLETE
                        Filter Operator
                          predicate: ((p_channel_tv = 'N') and p_promo_sk is not null) (type: boolean)
                          Statistics: Num rows: 225 Data size: 20025 Basic stats: COMPLETE Column stats: COMPLETE
                          Select Operator
                            expressions: p_promo_sk (type: int)
                            outputColumnNames: _col0
                            Statistics: Num rows: 225 Data size: 900 Basic stats: COMPLETE Column stats: COMPLETE
                            Reduce Output Operator
                              key expressions: _col0 (type: int)
                              sort order: +
                              Map-reduce partition columns: _col0 (type: int)
                              Statistics: Num rows: 225 Data size: 900 Basic stats: COMPLETE Column stats: COMPLETE
                  Execution mode: vectorized
              Map 25 
                  Map Operator Tree:
                      TableScan
                        alias: web_site
                        filterExpr: web_site_sk is not null (type: boolean)
                        Statistics: Num rows: 38 Data size: 70614 Basic stats: COMPLETE Column stats: COMPLETE
                        Filter Operator
                          predicate: web_site_sk is not null (type: boolean)
                          Statistics: Num rows: 38 Data size: 3952 Basic stats: COMPLETE Column stats: COMPLETE
                          Select Operator
                            expressions: web_site_sk (type: int), web_site_id (type: string)
                            outputColumnNames: _col0, _col1
                            Statistics: Num rows: 38 Data size: 3952 Basic stats: COMPLETE Column stats: COMPLETE
                            Reduce Output Operator
                              key expressions: _col0 (type: int)
                              sort order: +
                              Map-reduce partition columns: _col0 (type: int)
                              Statistics: Num rows: 38 Data size: 3952 Basic stats: COMPLETE Column stats: COMPLETE
                              value expressions: _col1 (type: string)
                  Execution mode: vectorized
              Map 7 
                  Map Operator Tree:
                      TableScan
                        alias: store_returns
                        filterExpr: sr_item_sk is not null (type: boolean)
                        Statistics: Num rows: 55578005 Data size: 4155315616 Basic stats: COMPLETE Column stats: COMPLETE
                        Filter Operator
                          predicate: sr_item_sk is not null (type: boolean)
                          Statistics: Num rows: 55578005 Data size: 881176504 Basic stats: COMPLETE Column stats: COMPLETE
                          Select Operator
                            expressions: sr_item_sk (type: int), sr_ticket_number (type: int), sr_return_amt (type: float), sr_net_loss (type: float)
                            outputColumnNames: _col0, _col1, _col2, _col3
                            Statistics: Num rows: 55578005 Data size: 881176504 Basic stats: COMPLETE Column stats: COMPLETE
                            Reduce Output Operator
                              key expressions: _col0 (type: int), _col1 (type: int)
                              sort order: ++
                              Map-reduce partition columns: _col0 (type: int), _col1 (type: int)
                              Statistics: Num rows: 55578005 Data size: 881176504 Basic stats: COMPLETE Column stats: COMPLETE
                              value expressions: _col2 (type: float), _col3 (type: float)
                  Execution mode: vectorized
              Map 8 
                  Map Operator Tree:
                      TableScan
                        alias: date_dim
                        filterExpr: (d_date BETWEEN 1998-08-04 AND 1998-09-04 and d_date_sk is not null) (type: boolean)
                        Statistics: Num rows: 73049 Data size: 81741831 Basic stats: COMPLETE Column stats: COMPLETE
                        Filter Operator
                          predicate: (d_date BETWEEN 1998-08-04 AND 1998-09-04 and d_date_sk is not null) (type: boolean)
                          Statistics: Num rows: 36524 Data size: 3579352 Basic stats: COMPLETE Column stats: COMPLETE
                          Select Operator
                            expressions: d_date_sk (type: int)
                            outputColumnNames: _col0
                            Statistics: Num rows: 36524 Data size: 146096 Basic stats: COMPLETE Column stats: COMPLETE
                            Reduce Output Operator
                              key expressions: _col0 (type: int)
                              sort order: +
                              Map-reduce partition columns: _col0 (type: int)
                              Statistics: Num rows: 36524 Data size: 146096 Basic stats: COMPLETE Column stats: COMPLETE
                            Select Operator
                              expressions: _col0 (type: int)
                              outputColumnNames: _col0
                              Statistics: Num rows: 36524 Data size: 146096 Basic stats: COMPLETE Column stats: COMPLETE
                              Group By Operator
                                keys: _col0 (type: int)
                                mode: hash
                                outputColumnNames: _col0
                                Statistics: Num rows: 18262 Data size: 73048 Basic stats: COMPLETE Column stats: COMPLETE
                                Dynamic Partitioning Event Operator
                                  Target Input: store_sales
                                  Partition key expr: ss_sold_date_sk
                                  Statistics: Num rows: 18262 Data size: 73048 Basic stats: COMPLETE Column stats: COMPLETE
                                  Target column: ss_sold_date_sk
                                  Target Vertex: Map 1
                  Execution mode: vectorized
              Map 9 
                  Map Operator Tree:
                      TableScan
                        alias: item
                        filterExpr: ((i_current_price > 50.0) and i_item_sk is not null) (type: boolean)
                        Statistics: Num rows: 48000 Data size: 68732712 Basic stats: COMPLETE Column stats: COMPLETE
                        Filter Operator
                          predicate: ((i_current_price > 50.0) and i_item_sk is not null) (type: boolean)
                          Statistics: Num rows: 16000 Data size: 127832 Basic stats: COMPLETE Column stats: COMPLETE
                          Select Operator
                            expressions: i_item_sk (type: int)
                            outputColumnNames: _col0
                            Statistics: Num rows: 16000 Data size: 64000 Basic stats: COMPLETE Column stats: COMPLETE
                            Reduce Output Operator
                              key expressions: _col0 (type: int)
                              sort order: +
                              Map-reduce partition columns: _col0 (type: int)
                              Statistics: Num rows: 16000 Data size: 64000 Basic stats: COMPLETE Column stats: COMPLETE
                  Execution mode: vectorized
              Reducer 13 
                  Reduce Operator Tree:
                    Group By Operator
                      aggregations: sum(VALUE._col0), sum(VALUE._col1), sum(VALUE._col2)
                      keys: KEY._col0 (type: string)
                      mode: mergepartial
                      outputColumnNames: _col0, _col1, _col2, _col3
                      Select Operator
                        expressions: 'catalog channel' (type: string), concat('catalog_page', _col0) (type: string), _col1 (type: double), _col2 (type: double), _col3 (type: double)
                        outputColumnNames: _col0, _col1, _col2, _col3, _col4
                        Group By Operator
                          aggregations: sum(_col2), sum(_col3), sum(_col4)
                          keys: _col0 (type: string), _col1 (type: string), '0' (type: string)
                          mode: hash
                          outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
                          Reduce Output Operator
                            key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string)
                            sort order: +++
                            Map-reduce partition columns: _col0 (type: string), _col1 (type: string), _col2 (type: string)
                            value expressions: _col3 (type: double), _col4 (type: double), _col5 (type: double)
              Reducer 2 
                  Reduce Operator Tree:
                    Merge Join Operator
                      condition map:
                           Left Outer Join0 to 1
                      keys:
                        0 _col0 (type: int), _col3 (type: int)
                        1 _col0 (type: int), _col1 (type: int)
                      outputColumnNames: _col0, _col1, _col2, _col4, _col5, _col6, _col9, _col10
                      Statistics: Num rows: 7811006 Data size: 249952192 Basic stats: COMPLETE Column stats: COMPLETE
                      Map Join Operator
                        condition map:
                             Inner Join 0 to 1
                        keys:
                          0 _col6 (type: int)
                          1 _col0 (type: int)
                        outputColumnNames: _col0, _col1, _col2, _col4, _col5, _col9, _col10
                        input vertices:
                          1 Map 8
                        Statistics: Num rows: 8733520 Data size: 244538560 Basic stats: COMPLETE Column stats: COMPLETE
                        Map Join Operator
                          condition map:
                               Inner Join 0 to 1
                          keys:
                            0 _col0 (type: int)
                            1 _col0 (type: int)
                          outputColumnNames: _col1, _col2, _col4, _col5, _col9, _col10
                          input vertices:
                            1 Map 9
                          Statistics: Num rows: 2911174 Data size: 69868176 Basic stats: COMPLETE Column stats: COMPLETE
                          Map Join Operator
                            condition map:
                                 Inner Join 0 to 1
                            keys:
                              0 _col2 (type: int)
                              1 _col0 (type: int)
                            outputColumnNames: _col1, _col4, _col5, _col9, _col10
                            input vertices:
                              1 Map 10
                            Statistics: Num rows: 1455587 Data size: 29111740 Basic stats: COMPLETE Column stats: COMPLETE
                            Map Join Operator
                              condition map:
                                   Inner Join 0 to 1
                              keys:
                                0 _col1 (type: int)
                                1 _col0 (type: int)
                              outputColumnNames: _col4, _col5, _col9, _col10, _col18
                              input vertices:
                                1 Map 11
                              Statistics: Num rows: 1455587 Data size: 168848092 Basic stats: COMPLETE Column stats: COMPLETE
                              Select Operator
                                expressions: _col18 (type: string), _col4 (type: float), COALESCE(_col9,0) (type: float), (_col5 - COALESCE(_col10,0)) (type: float)
                                outputColumnNames: _col0, _col1, _col2, _col3
                                Statistics: Num rows: 1455587 Data size: 168848092 Basic stats: COMPLETE Column stats: COMPLETE
                                Group By Operator
                                  aggregations: sum(_col1), sum(_col2), sum(_col3)
                                  keys: _col0 (type: string)
                                  mode: hash
                                  outputColumnNames: _col0, _col1, _col2, _col3
                                  Statistics: Num rows: 234 Data size: 29016 Basic stats: COMPLETE Column stats: COMPLETE
                                  Reduce Output Operator
                                    key expressions: _col0 (type: string)
                                    sort order: +
                                    Map-reduce partition columns: _col0 (type: string)
                                    Statistics: Num rows: 234 Data size: 29016 Basic stats: COMPLETE Column stats: COMPLETE
                                    value expressions: _col1 (type: double), _col2 (type: double), _col3 (type: double)
              Reducer 20 
                  Reduce Operator Tree:
                    Group By Operator
                      aggregations: sum(VALUE._col0), sum(VALUE._col1), sum(VALUE._col2)
                      keys: KEY._col0 (type: string)
                      mode: mergepartial
                      outputColumnNames: _col0, _col1, _col2, _col3
                      Select Operator
                        expressions: 'web channel' (type: string), concat('web_site', _col0) (type: string), _col1 (type: double), _col2 (type: double), _col3 (type: double)
                        outputColumnNames: _col0, _col1, _col2, _col3, _col4
                        Group By Operator
                          aggregations: sum(_col2), sum(_col3), sum(_col4)
                          keys: _col0 (type: string), _col1 (type: string), '0' (type: string)
                          mode: hash
                          outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
                          Reduce Output Operator
                            key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string)
                            sort order: +++
                            Map-reduce partition columns: _col0 (type: string), _col1 (type: string), _col2 (type: string)
                            value expressions: _col3 (type: double), _col4 (type: double), _col5 (type: double)
              Reducer 3 
                  Reduce Operator Tree:
                    Group By Operator
                      aggregations: sum(VALUE._col0), sum(VALUE._col1), sum(VALUE._col2)
                      keys: KEY._col0 (type: string)
                      mode: mergepartial
                      outputColumnNames: _col0, _col1, _col2, _col3
                      Select Operator
                        expressions: 'store channel' (type: string), concat('store', _col0) (type: string), _col1 (type: double), _col2 (type: double), _col3 (type: double)
                        outputColumnNames: _col0, _col1, _col2, _col3, _col4
                        Group By Operator
                          aggregations: sum(_col2), sum(_col3), sum(_col4)
                          keys: _col0 (type: string), _col1 (type: string), '0' (type: string)
                          mode: hash
                          outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
                          Reduce Output Operator
                            key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string)
                            sort order: +++
                            Map-reduce partition columns: _col0 (type: string), _col1 (type: string), _col2 (type: string)
                            value expressions: _col3 (type: double), _col4 (type: double), _col5 (type: double)
              Reducer 5 
                  Reduce Operator Tree:
                    Group By Operator
                      aggregations: sum(VALUE._col0), sum(VALUE._col1), sum(VALUE._col2)
                      keys: KEY._col0 (type: string), KEY._col1 (type: string), KEY._col2 (type: string)
                      mode: mergepartial
                      outputColumnNames: _col0, _col1, _col3, _col4, _col5
                      Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
                      pruneGroupingSetId: true
                      Select Operator
                        expressions: _col0 (type: string), _col1 (type: string), _col3 (type: double), _col4 (type: double), _col5 (type: double)
                        outputColumnNames: _col0, _col1, _col2, _col3, _col4
                        Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
                        Reduce Output Operator
                          key expressions: _col0 (type: string), _col1 (type: string)
                          sort order: ++
                          Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
                          TopN Hash Memory Usage: 0.04
                          value expressions: _col2 (type: double), _col3 (type: double), _col4 (type: double)
                  Execution mode: vectorized
              Reducer 6 
                  Reduce Operator Tree:
                    Select Operator
                      expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string), VALUE._col0 (type: double), VALUE._col1 (type: double), VALUE._col2 (type: double)
                      outputColumnNames: _col0, _col1, _col2, _col3, _col4
                      Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
                      Limit
                        Number of rows: 100
                        Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
                        File Output Operator
                          compressed: false
                          Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
                          table:
                              input format: org.apache.hadoop.mapred.TextInputFormat
                              output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
                              serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
                  Execution mode: vectorized
              Union 4 
                  Vertex: Union 4
      
        Stage: Stage-0
          Fetch Operator
            limit: 100
            Processor Tree:
              ListSink
      

      Attachments

        1. explain_q80_vectorized_reduce_on.txt
          2.10 MB
          Mostafa Mokhtar
        2. HIVE-10244.01.patch
          26 kB
          Matt McCline

        Issue Links

          Activity

            People

              mmccline Matt McCline
              mmokhtar Mostafa Mokhtar
              Votes:
              0 Vote for this issue
              Watchers:
              6 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: