You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@hive.apache.org by "Gunther Hagleitner (JIRA)" <ji...@apache.org> on 2014/10/31 07:11:33 UTC

[jira] [Updated] (HIVE-8677) TPC-DS Q51 : fails with "init not supported" exception in GenericUDAFStreamingEvaluator.init

     [ https://issues.apache.org/jira/browse/HIVE-8677?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Gunther Hagleitner updated HIVE-8677:
-------------------------------------
    Attachment: HIVE-8677.1.patch

> TPC-DS Q51 : fails with "init not supported" exception in GenericUDAFStreamingEvaluator.init
> --------------------------------------------------------------------------------------------
>
>                 Key: HIVE-8677
>                 URL: https://issues.apache.org/jira/browse/HIVE-8677
>             Project: Hive
>          Issue Type: Bug
>    Affects Versions: 0.14.0
>            Reporter: Mostafa Mokhtar
>            Assignee: Gunther Hagleitner
>            Priority: Critical
>             Fix For: 0.14.0
>
>         Attachments: HIVE-8677.1.patch
>
>
> TPC-DS Q51 fails with the exception below 
> {code}
> , TaskAttempt 3 failed, info=[Error: Failure while running task:java.lang.RuntimeException: java.lang.RuntimeException: Reduce operator initialization failed
> 	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:186)
> 	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.run(TezProcessor.java:138)
> 	at org.apache.tez.runtime.LogicalIOProcessorRuntimeTask.run(LogicalIOProcessorRuntimeTask.java:324)
> 	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable$1.run(TezTaskRunner.java:176)
> 	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable$1.run(TezTaskRunner.java:168)
> 	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:1548)
> 	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable.call(TezTaskRunner.java:168)
> 	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable.call(TezTaskRunner.java:163)
> 	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:744)
> Caused by: java.lang.RuntimeException: Reduce operator initialization failed
> 	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordProcessor.init(ReduceRecordProcessor.java:146)
> 	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:162)
> 	... 13 more
> Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: : init not supported
> 	at org.apache.hadoop.hive.ql.udf.generic.GenericUDAFStreamingEvaluator.init(GenericUDAFStreamingEvaluator.java:70)
> 	at org.apache.hadoop.hive.ql.plan.PTFDeserializer.setupWdwFnEvaluator(PTFDeserializer.java:209)
> 	at org.apache.hadoop.hive.ql.plan.PTFDeserializer.initializeWindowing(PTFDeserializer.java:130)
> 	at org.apache.hadoop.hive.ql.plan.PTFDeserializer.initializePTFChain(PTFDeserializer.java:94)
> 	at org.apache.hadoop.hive.ql.exec.PTFOperator.reconstructQueryDef(PTFOperator.java:144)
> 	at org.apache.hadoop.hive.ql.exec.PTFOperator.initializeOp(PTFOperator.java:74)
> 	at org.apache.hadoop.hive.ql.exec.Operator.initialize(Operator.java:385)
> 	at org.apache.hadoop.hive.ql.exec.Operator.initialize(Operator.java:469)
> 	at org.apache.hadoop.hive.ql.exec.Operator.initializeChildren(Operator.java:425)
> 	at org.apache.hadoop.hive.ql.exec.ExtractOperator.initializeOp(ExtractOperator.java:40)
> 	at org.apache.hadoop.hive.ql.exec.Operator.initialize(Operator.java:385)
> 	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordProcessor.init(ReduceRecordProcessor.java:116)
> 	... 14 more
> {code}
> Query
> {code}
> set hive.cbo.enable=true;
> set hive.stats.fetch.column.stats=true;
> set hive.exec.dynamic.partition.mode=nonstrict;
> set hive.tez.auto.reducer.parallelism=true;
> set hive.tez.exec.print.summary=true;
> set hive.auto.convert.join.noconditionaltask.size=1280000000;
> set hive.exec.reducers.bytes.per.reducer=100000000;
> set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DummyTxnManager;
> set hive.support.concurrency=false;
>  
> WITH web_v1 as (
> select
>   ws_item_sk item_sk, d_date, sum(ws_sales_price),
>   sum(sum(ws_sales_price))
>       over (partition by ws_item_sk order by d_date rows between unbounded preceding and current row) cume_sales
> from web_sales
>     ,date_dim
> where ws_sold_date_sk=d_date_sk
>   and d_month_seq between 1193 and 1193+11
>   and ws_item_sk is not NULL
> group by ws_item_sk, d_date),
> store_v1 as (
> select
>   ss_item_sk item_sk, d_date, sum(ss_sales_price),
>   sum(sum(ss_sales_price))
>       over (partition by ss_item_sk order by d_date rows between unbounded preceding and current row) cume_sales
> from store_sales
>     ,date_dim
> where ss_sold_date_sk=d_date_sk
>   and d_month_seq between 1193 and 1193+11
>   and ss_item_sk is not NULL
> group by ss_item_sk, d_date)
>  select  *
> from (select item_sk
>      ,d_date
>      ,web_sales
>      ,store_sales
>      ,max(web_sales)
>          over (partition by item_sk order by d_date rows between unbounded preceding and current row) web_cumulative
>      ,max(store_sales)
>          over (partition by item_sk order by d_date rows between unbounded preceding and current row) store_cumulative
>      from (select case when web.item_sk is not null then web.item_sk else store.item_sk end item_sk
>                  ,case when web.d_date is not null then web.d_date else store.d_date end d_date
>                  ,web.cume_sales web_sales
>                  ,store.cume_sales store_sales
>            from web_v1 web full outer join store_v1 store on (web.item_sk = store.item_sk
>                                                           and web.d_date = store.d_date)
>           )x )y
> where web_cumulative > store_cumulative
> order by item_sk
>         ,d_date
> limit 100
> {code}
> Plan
> {code}
> OK
> STAGE DEPENDENCIES:
>   Stage-1 is a root stage
>   Stage-0 depends on stages: Stage-1
> STAGE PLANS:
>   Stage: Stage-1
>     Tez
>       Edges:
>         Map 2 <- Map 8 (BROADCAST_EDGE)
>         Map 9 <- Map 1 (BROADCAST_EDGE)
>         Reducer 10 <- Map 9 (SIMPLE_EDGE)
>         Reducer 11 <- Reducer 10 (SIMPLE_EDGE)
>         Reducer 3 <- Map 2 (SIMPLE_EDGE)
>         Reducer 4 <- Reducer 3 (SIMPLE_EDGE)
>         Reducer 5 <- Reducer 11 (SIMPLE_EDGE), Reducer 4 (SIMPLE_EDGE)
>         Reducer 6 <- Reducer 5 (SIMPLE_EDGE)
>         Reducer 7 <- Reducer 6 (SIMPLE_EDGE)
>       DagName: mmokhtar_20141030010808_11af3ba0-8b28-4a33-9f4d-73618503e272:1
>       Vertices:
>         Map 1 
>             Map Operator Tree:
>                 TableScan
>                   alias: date_dim
>                   filterExpr: (d_date_sk is not null and d_month_seq BETWEEN 1193 AND 1204) (type: boolean)
>                   Statistics: Num rows: 73049 Data size: 81741831 Basic stats: COMPLETE Column stats: COMPLETE
>                   Filter Operator
>                     predicate: (d_date_sk is not null and d_month_seq BETWEEN 1193 AND 1204) (type: boolean)
>                     Statistics: Num rows: 36524 Data size: 3725448 Basic stats: COMPLETE Column stats: COMPLETE
>                     Reduce Output Operator
>                       key expressions: d_date_sk (type: int)
>                       sort order: +
>                       Map-reduce partition columns: d_date_sk (type: int)
>                       Statistics: Num rows: 36524 Data size: 3725448 Basic stats: COMPLETE Column stats: COMPLETE
>                       value expressions: d_date (type: string), d_month_seq (type: int)
>                     Select Operator
>                       expressions: d_date_sk (type: int)
>                       outputColumnNames: _col0
>                       Statistics: Num rows: 36524 Data size: 3725448 Basic stats: COMPLETE Column stats: COMPLETE
>                       Group By Operator
>                         keys: _col0 (type: int)
>                         mode: hash
>                         outputColumnNames: _col0
>                         Statistics: Num rows: 36524 Data size: 3725448 Basic stats: COMPLETE Column stats: COMPLETE
>                         Dynamic Partitioning Event Operator
>                           Target Input: store_sales
>                           Partition key expr: ss_sold_date_sk
>                           Statistics: Num rows: 36524 Data size: 3725448 Basic stats: COMPLETE Column stats: COMPLETE
>                           Target column: ss_sold_date_sk
>                           Target Vertex: Map 9
>             Execution mode: vectorized
>         Map 2 
>             Map Operator Tree:
>                 TableScan
>                   alias: web_sales
>                   filterExpr: ws_item_sk is not null (type: boolean)
>                   Statistics: Num rows: 21594638446 Data size: 2850189889652 Basic stats: COMPLETE Column stats: COMPLETE
>                   Filter Operator
>                     predicate: ws_item_sk is not null (type: boolean)
>                     Statistics: Num rows: 21594638446 Data size: 259124859072 Basic stats: COMPLETE Column stats: COMPLETE
>                     Map Join Operator
>                       condition map:
>                            Inner Join 0 to 1
>                       condition expressions:
>                         0 {ws_item_sk} {ws_sales_price} {ws_sold_date_sk}
>                         1 {d_date_sk} {d_date} {d_month_seq}
>                       keys:
>                         0 ws_sold_date_sk (type: int)
>                         1 d_date_sk (type: int)
>                       outputColumnNames: _col2, _col20, _col33, _col37, _col39, _col40
>                       input vertices:
>                         1 Map 8
>                       Statistics: Num rows: 24145061366 Data size: 2752536995724 Basic stats: COMPLETE Column stats: COMPLETE
>                       Filter Operator
>                         predicate: (((_col33 = _col37) and _col40 BETWEEN 1193 AND 1204) and _col2 is not null) (type: boolean)
>                         Statistics: Num rows: 6036265341 Data size: 688134248874 Basic stats: COMPLETE Column stats: COMPLETE
>                         Select Operator
>                           expressions: _col2 (type: int), _col39 (type: string), _col20 (type: float)
>                           outputColumnNames: _col2, _col39, _col20
>                           Statistics: Num rows: 6036265341 Data size: 688134248874 Basic stats: COMPLETE Column stats: COMPLETE
>                           Group By Operator
>                             aggregations: sum(_col20)
>                             keys: _col2 (type: int), _col39 (type: string)
>                             mode: hash
>                             outputColumnNames: _col0, _col1, _col2
>                             Statistics: Num rows: 6036265341 Data size: 639844126146 Basic stats: COMPLETE Column stats: COMPLETE
>                             Reduce Output Operator
>                               key expressions: _col0 (type: int), _col1 (type: string)
>                               sort order: ++
>                               Map-reduce partition columns: _col0 (type: int), _col1 (type: string)
>                               Statistics: Num rows: 6036265341 Data size: 639844126146 Basic stats: COMPLETE Column stats: COMPLETE
>                               value expressions: _col2 (type: double)
>             Execution mode: vectorized
>         Map 8 
>             Map Operator Tree:
>                 TableScan
>                   alias: date_dim
>                   filterExpr: (d_date_sk is not null and d_month_seq BETWEEN 1193 AND 1204) (type: boolean)
>                   Statistics: Num rows: 73049 Data size: 81741831 Basic stats: COMPLETE Column stats: COMPLETE
>                   Filter Operator
>                     predicate: (d_date_sk is not null and d_month_seq BETWEEN 1193 AND 1204) (type: boolean)
>                     Statistics: Num rows: 36524 Data size: 3725448 Basic stats: COMPLETE Column stats: COMPLETE
>                     Reduce Output Operator
>                       key expressions: d_date_sk (type: int)
>                       sort order: +
>                       Map-reduce partition columns: d_date_sk (type: int)
>                       Statistics: Num rows: 36524 Data size: 3725448 Basic stats: COMPLETE Column stats: COMPLETE
>                       value expressions: d_date (type: string), d_month_seq (type: int)
>                     Select Operator
>                       expressions: d_date_sk (type: int)
>                       outputColumnNames: _col0
>                       Statistics: Num rows: 36524 Data size: 3725448 Basic stats: COMPLETE Column stats: COMPLETE
>                       Group By Operator
>                         keys: _col0 (type: int)
>                         mode: hash
>                         outputColumnNames: _col0
>                         Statistics: Num rows: 36524 Data size: 3725448 Basic stats: COMPLETE Column stats: COMPLETE
>                         Dynamic Partitioning Event Operator
>                           Target Input: web_sales
>                           Partition key expr: ws_sold_date_sk
>                           Statistics: Num rows: 36524 Data size: 3725448 Basic stats: COMPLETE Column stats: COMPLETE
>                           Target column: ws_sold_date_sk
>                           Target Vertex: Map 2
>             Execution mode: vectorized
>         Map 9 
>             Map Operator Tree:
>                 TableScan
>                   alias: store_sales
>                   filterExpr: ss_item_sk is not null (type: boolean)
>                   Statistics: Num rows: 82510879939 Data size: 6873789738208 Basic stats: COMPLETE Column stats: COMPLETE
>                   Filter Operator
>                     predicate: ss_item_sk is not null (type: boolean)
>                     Statistics: Num rows: 82510879939 Data size: 982359338028 Basic stats: COMPLETE Column stats: COMPLETE
>                     Map Join Operator
>                       condition map:
>                            Inner Join 0 to 1
>                       condition expressions:
>                         0 {ss_item_sk} {ss_sales_price} {ss_sold_date_sk}
>                         1 {d_date_sk} {d_date} {d_month_seq}
>                       keys:
>                         0 ss_sold_date_sk (type: int)
>                         1 d_date_sk (type: int)
>                       outputColumnNames: _col1, _col12, _col22, _col26, _col28, _col29
>                       input vertices:
>                         1 Map 1
>                       Statistics: Num rows: 92255782124 Data size: 10517159162136 Basic stats: COMPLETE Column stats: COMPLETE
>                       Filter Operator
>                         predicate: (((_col22 = _col26) and _col29 BETWEEN 1193 AND 1204) and _col1 is not null) (type: boolean)
>                         Statistics: Num rows: 23063945531 Data size: 2629289790534 Basic stats: COMPLETE Column stats: COMPLETE
>                         Select Operator
>                           expressions: _col1 (type: int), _col28 (type: string), _col12 (type: float)
>                           outputColumnNames: _col1, _col28, _col12
>                           Statistics: Num rows: 23063945531 Data size: 2629289790534 Basic stats: COMPLETE Column stats: COMPLETE
>                           Group By Operator
>                             aggregations: sum(_col12)
>                             keys: _col1 (type: int), _col28 (type: string)
>                             mode: hash
>                             outputColumnNames: _col0, _col1, _col2
>                             Statistics: Num rows: 23063945531 Data size: 2444778226286 Basic stats: COMPLETE Column stats: COMPLETE
>                             Reduce Output Operator
>                               key expressions: _col0 (type: int), _col1 (type: string)
>                               sort order: ++
>                               Map-reduce partition columns: _col0 (type: int), _col1 (type: string)
>                               Statistics: Num rows: 23063945531 Data size: 2444778226286 Basic stats: COMPLETE Column stats: COMPLETE
>                               value expressions: _col2 (type: double)
>             Execution mode: vectorized
>         Reducer 10 
>             Reduce Operator Tree:
>               Group By Operator
>                 aggregations: sum(VALUE._col0)
>                 keys: KEY._col0 (type: int), KEY._col1 (type: string)
>                 mode: mergepartial
>                 outputColumnNames: _col0, _col1, _col2
>                 Statistics: Num rows: 23063945531 Data size: 2537034008410 Basic stats: COMPLETE Column stats: COMPLETE
>                 Reduce Output Operator
>                   key expressions: _col0 (type: int), _col1 (type: string)
>                   sort order: ++
>                   Map-reduce partition columns: _col0 (type: int)
>                   Statistics: Num rows: 23063945531 Data size: 2537034008410 Basic stats: COMPLETE Column stats: COMPLETE
>                   value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: double)
>             Execution mode: vectorized
>         Reducer 11 
>             Reduce Operator Tree:
>               Extract
>                 Statistics: Num rows: 23063945531 Data size: 2537034008410 Basic stats: COMPLETE Column stats: COMPLETE
>                 PTF Operator
>                   Statistics: Num rows: 23063945531 Data size: 2537034008410 Basic stats: COMPLETE Column stats: COMPLETE
>                   Select Operator
>                     expressions: _col0 (type: int), _col1 (type: string), _wcol0 (type: double)
>                     outputColumnNames: _col0, _col1, _col3
>                     Statistics: Num rows: 23063945531 Data size: 184511564248 Basic stats: COMPLETE Column stats: COMPLETE
>                     Reduce Output Operator
>                       key expressions: _col0 (type: int), _col1 (type: string)
>                       sort order: ++
>                       Map-reduce partition columns: _col0 (type: int), _col1 (type: string)
>                       Statistics: Num rows: 23063945531 Data size: 184511564248 Basic stats: COMPLETE Column stats: COMPLETE
>                       value expressions: _col3 (type: double)
>         Reducer 3 
>             Reduce Operator Tree:
>               Group By Operator
>                 aggregations: sum(VALUE._col0)
>                 keys: KEY._col0 (type: int), KEY._col1 (type: string)
>                 mode: mergepartial
>                 outputColumnNames: _col0, _col1, _col2
>                 Statistics: Num rows: 6036265341 Data size: 663989187510 Basic stats: COMPLETE Column stats: COMPLETE
>                 Reduce Output Operator
>                   key expressions: _col0 (type: int), _col1 (type: string)
>                   sort order: ++
>                   Map-reduce partition columns: _col0 (type: int)
>                   Statistics: Num rows: 6036265341 Data size: 663989187510 Basic stats: COMPLETE Column stats: COMPLETE
>                   value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: double)
>             Execution mode: vectorized
>         Reducer 4 
>             Reduce Operator Tree:
>               Extract
>                 Statistics: Num rows: 6036265341 Data size: 663989187510 Basic stats: COMPLETE Column stats: COMPLETE
>                 PTF Operator
>                   Statistics: Num rows: 6036265341 Data size: 663989187510 Basic stats: COMPLETE Column stats: COMPLETE
>                   Select Operator
>                     expressions: _col0 (type: int), _col1 (type: string), _wcol0 (type: double)
>                     outputColumnNames: _col0, _col1, _col3
>                     Statistics: Num rows: 6036265341 Data size: 48290122728 Basic stats: COMPLETE Column stats: COMPLETE
>                     Reduce Output Operator
>                       key expressions: _col0 (type: int), _col1 (type: string)
>                       sort order: ++
>                       Map-reduce partition columns: _col0 (type: int), _col1 (type: string)
>                       Statistics: Num rows: 6036265341 Data size: 48290122728 Basic stats: COMPLETE Column stats: COMPLETE
>                       value expressions: _col3 (type: double)
>         Reducer 5 
>             Reduce Operator Tree:
>               Merge Join Operator
>                 condition map:
>                      Outer Join 0 to 1
>                 condition expressions:
>                   0 {KEY.reducesinkkey0} {KEY.reducesinkkey1} {VALUE._col1}
>                   1 {KEY.reducesinkkey0} {KEY.reducesinkkey1} {VALUE._col1}
>                 outputColumnNames: _col0, _col1, _col3, _col4, _col5, _col7
>                 Statistics: Num rows: 9223372036854775807 Data size: 9223372036854775807 Basic stats: COMPLETE Column stats: COMPLETE
>                 Select Operator
>                   expressions: CASE WHEN (_col0 is not null) THEN (_col0) ELSE (_col4) END (type: int), CASE WHEN (_col1 is not null) THEN (_col1) ELSE (_col5) END (type: string), _col3 (type: double), _col7 (type: double)
>                   outputColumnNames: _col0, _col1, _col2, _col3
>                   Statistics: Num rows: 9223372036854775807 Data size: 9223372036854775807 Basic stats: COMPLETE Column stats: COMPLETE
>                   Reduce Output Operator
>                     key expressions: _col0 (type: int), _col1 (type: string)
>                     sort order: ++
>                     Map-reduce partition columns: _col0 (type: int)
>                     Statistics: Num rows: 9223372036854775807 Data size: 9223372036854775807 Basic stats: COMPLETE Column stats: COMPLETE
>                     value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: double), _col3 (type: double)
>         Reducer 6 
>             Reduce Operator Tree:
>               Extract
>                 Statistics: Num rows: 9223372036854775807 Data size: 9223372036854775807 Basic stats: COMPLETE Column stats: COMPLETE
>                 PTF Operator
>                   Statistics: Num rows: 9223372036854775807 Data size: 9223372036854775807 Basic stats: COMPLETE Column stats: COMPLETE
>                   Filter Operator
>                     predicate: (_wcol0 > _wcol1) (type: boolean)
>                     Statistics: Num rows: 3074457345618258602 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
>                     Select Operator
>                       expressions: _col0 (type: int), _col1 (type: string), _col2 (type: double), _col3 (type: double), _wcol0 (type: double), _wcol1 (type: double)
>                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
>                       Statistics: Num rows: 3074457345618258602 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
>                       Reduce Output Operator
>                         key expressions: _col0 (type: int), _col1 (type: string)
>                         sort order: ++
>                         Statistics: Num rows: 3074457345618258602 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
>                         TopN Hash Memory Usage: 0.04
>                         value expressions: _col2 (type: double), _col3 (type: double), _col4 (type: double), _col5 (type: double)
>         Reducer 7 
>             Reduce Operator Tree:
>               Select Operator
>                 expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: string), VALUE._col0 (type: double), VALUE._col1 (type: double), VALUE._col2 (type: double), VALUE._col3 (type: double)
>                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
>                 Statistics: Num rows: 3074457345618258602 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
>                 Limit
>                   Number of rows: 100
>                   Statistics: Num rows: 100 Data size: 0 Basic stats: PARTIAL Column stats: COMPLETE
>                   File Output Operator
>                     compressed: false
>                     Statistics: Num rows: 100 Data size: 0 Basic stats: PARTIAL 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
>   Stage: Stage-0
>     Fetch Operator
>       limit: 100
>       Processor Tree:
>         ListSink
> {code}
> The full exception 
> {code}
> Status: Failed
> 14/10/30 01:19:19 [main]: ERROR tez.TezJobMonitor: Status: Failed
> Vertex failed, vertexName=Reducer 11, vertexId=vertex_1414029100044_0733_1_05, diagnostics=[Task failed, taskId=task_1414029100044_0733_1_05_000027, diagnostics=[TaskAttempt 0 failed, info=[Error: Failure while running task:java.lang.RuntimeException: java.lang.RuntimeException: java.lang.RuntimeException: problem advancing post rec#334499
> 	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:186)
> 	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.run(TezProcessor.java:138)
> 	at org.apache.tez.runtime.LogicalIOProcessorRuntimeTask.run(LogicalIOProcessorRuntimeTask.java:324)
> 	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable$1.run(TezTaskRunner.java:176)
> 	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable$1.run(TezTaskRunner.java:168)
> 	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:1548)
> 	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable.call(TezTaskRunner.java:168)
> 	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable.call(TezTaskRunner.java:163)
> 	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:744)
> Caused by: java.lang.RuntimeException: java.lang.RuntimeException: problem advancing post rec#334499
> 	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource.pushRecord(ReduceRecordSource.java:262)
> 	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordProcessor.run(ReduceRecordProcessor.java:168)
> 	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:163)
> 	... 13 more
> Caused by: java.lang.RuntimeException: problem advancing post rec#334499
> 	at org.apache.tez.runtime.library.common.ValuesIterator$1$1.next(ValuesIterator.java:142)
> 	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource.processKeyValues(ReduceRecordSource.java:288)
> 	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource.pushRecord(ReduceRecordSource.java:252)
> 	... 15 more
> Caused by: org.apache.hadoop.fs.ChecksumException: Checksum Error:  CurrentOffset=166741, offset=4, off=0, dataLength=167232, origLen=495, len=491, length=167236, checksumSize=4
> 	at org.apache.tez.runtime.library.common.sort.impl.IFileInputStream.doRead(IFileInputStream.java:254)
> 	at org.apache.tez.runtime.library.common.sort.impl.IFileInputStream.read(IFileInputStream.java:184)
> 	at org.apache.tez.runtime.library.common.sort.impl.IFileInputStream.close(IFileInputStream.java:131)
> 	at org.apache.hadoop.io.compress.DecompressorStream.close(DecompressorStream.java:205)
> 	at org.apache.tez.runtime.library.common.sort.impl.IFile$Reader.close(IFile.java:784)
> 	at org.apache.tez.runtime.library.common.sort.impl.TezMerger$Segment.closeReader(TezMerger.java:332)
> 	at org.apache.tez.runtime.library.common.sort.impl.TezMerger$Segment.close(TezMerger.java:338)
> 	at org.apache.tez.runtime.library.common.sort.impl.TezMerger$MergeQueue.adjustPriorityQueue(TezMerger.java:489)
> 	at org.apache.tez.runtime.library.common.sort.impl.TezMerger$MergeQueue.next(TezMerger.java:503)
> 	at org.apache.tez.runtime.library.common.shuffle.orderedgrouped.MergeManager$RawKVIteratorReader.readRawKey(MergeManager.java:765)
> 	at org.apache.tez.runtime.library.common.sort.impl.TezMerger$Segment.readRawKey(TezMerger.java:319)
> 	at org.apache.tez.runtime.library.common.sort.impl.TezMerger$MergeQueue.adjustPriorityQueue(TezMerger.java:481)
> 	at org.apache.tez.runtime.library.common.sort.impl.TezMerger$MergeQueue.next(TezMerger.java:503)
> 	at org.apache.tez.runtime.library.common.ValuesIterator.readNextKey(ValuesIterator.java:181)
> 	at org.apache.tez.runtime.library.common.ValuesIterator.access$300(ValuesIterator.java:47)
> 	at org.apache.tez.runtime.library.common.ValuesIterator$1$1.next(ValuesIterator.java:140)
> 	... 17 more
> ], TaskAttempt 1 failed, info=[Error: Failure while running task:java.lang.RuntimeException: java.lang.RuntimeException: Reduce operator initialization failed
> 	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:186)
> 	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.run(TezProcessor.java:138)
> 	at org.apache.tez.runtime.LogicalIOProcessorRuntimeTask.run(LogicalIOProcessorRuntimeTask.java:324)
> 	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable$1.run(TezTaskRunner.java:176)
> 	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable$1.run(TezTaskRunner.java:168)
> 	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:1548)
> 	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable.call(TezTaskRunner.java:168)
> 	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable.call(TezTaskRunner.java:163)
> 	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:744)
> Caused by: java.lang.RuntimeException: Reduce operator initialization failed
> 	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordProcessor.init(ReduceRecordProcessor.java:146)
> 	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:162)
> 	... 13 more
> Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: : init not supported
> 	at org.apache.hadoop.hive.ql.udf.generic.GenericUDAFStreamingEvaluator.init(GenericUDAFStreamingEvaluator.java:70)
> 	at org.apache.hadoop.hive.ql.plan.PTFDeserializer.setupWdwFnEvaluator(PTFDeserializer.java:209)
> 	at org.apache.hadoop.hive.ql.plan.PTFDeserializer.initializeWindowing(PTFDeserializer.java:130)
> 	at org.apache.hadoop.hive.ql.plan.PTFDeserializer.initializePTFChain(PTFDeserializer.java:94)
> 	at org.apache.hadoop.hive.ql.exec.PTFOperator.reconstructQueryDef(PTFOperator.java:144)
> 	at org.apache.hadoop.hive.ql.exec.PTFOperator.initializeOp(PTFOperator.java:74)
> 	at org.apache.hadoop.hive.ql.exec.Operator.initialize(Operator.java:385)
> 	at org.apache.hadoop.hive.ql.exec.Operator.initialize(Operator.java:469)
> 	at org.apache.hadoop.hive.ql.exec.Operator.initializeChildren(Operator.java:425)
> 	at org.apache.hadoop.hive.ql.exec.ExtractOperator.initializeOp(ExtractOperator.java:40)
> 	at org.apache.hadoop.hive.ql.exec.Operator.initialize(Operator.java:385)
> 	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordProcessor.init(ReduceRecordProcessor.java:116)
> 	... 14 more
> ], TaskAttempt 2 failed, info=[Error: Failure while running task:java.lang.RuntimeException: java.lang.RuntimeException: Reduce operator initialization failed
> 	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:186)
> 	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.run(TezProcessor.java:138)
> 	at org.apache.tez.runtime.LogicalIOProcessorRuntimeTask.run(LogicalIOProcessorRuntimeTask.java:324)
> 	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable$1.run(TezTaskRunner.java:176)
> 	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable$1.run(TezTaskRunner.java:168)
> 	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:1548)
> 	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable.call(TezTaskRunner.java:168)
> 	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable.call(TezTaskRunner.java:163)
> 	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:744)
> Caused by: java.lang.RuntimeException: Reduce operator initialization failed
> 	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordProcessor.init(ReduceRecordProcessor.java:146)
> 	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:162)
> 	... 13 more
> Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: : init not supported
> 	at org.apache.hadoop.hive.ql.udf.generic.GenericUDAFStreamingEvaluator.init(GenericUDAFStreamingEvaluator.java:70)
> 	at org.apache.hadoop.hive.ql.plan.PTFDeserializer.setupWdwFnEvaluator(PTFDeserializer.java:209)
> 	at org.apache.hadoop.hive.ql.plan.PTFDeserializer.initializeWindowing(PTFDeserializer.java:130)
> 	at org.apache.hadoop.hive.ql.plan.PTFDeserializer.initializePTFChain(PTFDeserializer.java:94)
> 	at org.apache.hadoop.hive.ql.exec.PTFOperator.reconstructQueryDef(PTFOperator.java:144)
> 	at org.apache.hadoop.hive.ql.exec.PTFOperator.initializeOp(PTFOperator.java:74)
> 	at org.apache.hadoop.hive.ql.exec.Operator.initialize(Operator.java:385)
> 	at org.apache.hadoop.hive.ql.exec.Operator.initialize(Operator.java:469)
> 	at org.apache.hadoop.hive.ql.exec.Operator.initializeChildren(Operator.java:425)
> 	at org.apache.hadoop.hive.ql.exec.ExtractOperator.initializeOp(ExtractOperator.java:40)
> 	at org.apache.hadoop.hive.ql.exec.Operator.initialize(Operator.java:385)
> 	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordProcessor.init(ReduceRecordProcessor.java:116)
> 	... 14 more
> ], TaskAttempt 3 failed, info=[Error: Failure while running task:java.lang.RuntimeException: java.lang.RuntimeException: Reduce operator initialization failed
> 	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:186)
> 	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.run(TezProcessor.java:138)
> 	at org.apache.tez.runtime.LogicalIOProcessorRuntimeTask.run(LogicalIOProcessorRuntimeTask.java:324)
> 	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable$1.run(TezTaskRunner.java:176)
> 	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable$1.run(TezTaskRunner.java:168)
> 	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:1548)
> 	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable.call(TezTaskRunner.java:168)
> 	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable.call(TezTaskRunner.java:163)
> 	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:744)
> Caused by: java.lang.RuntimeException: Reduce operator initialization failed
> 	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordProcessor.init(ReduceRecordProcessor.java:146)
> 	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:162)
> 	... 13 more
> Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: : init not supported
> 	at org.apache.hadoop.hive.ql.udf.generic.GenericUDAFStreamingEvaluator.init(GenericUDAFStreamingEvaluator.java:70)
> 	at org.apache.hadoop.hive.ql.plan.PTFDeserializer.setupWdwFnEvaluator(PTFDeserializer.java:209)
> 	at org.apache.hadoop.hive.ql.plan.PTFDeserializer.initializeWindowing(PTFDeserializer.java:130)
> 	at org.apache.hadoop.hive.ql.plan.PTFDeserializer.initializePTFChain(PTFDeserializer.java:94)
> 	at org.apache.hadoop.hive.ql.exec.PTFOperator.reconstructQueryDef(PTFOperator.java:144)
> 	at org.apache.hadoop.hive.ql.exec.PTFOperator.initializeOp(PTFOperator.java:74)
> 	at org.apache.hadoop.hive.ql.exec.Operator.initialize(Operator.java:385)
> 	at org.apache.hadoop.hive.ql.exec.Operator.initialize(Operator.java:469)
> 	at org.apache.hadoop.hive.ql.exec.Operator.initializeChildren(Operator.java:425)
> 	at org.apache.hadoop.hive.ql.exec.ExtractOperator.initializeOp(ExtractOperator.java:40)
> 	at org.apache.hadoop.hive.ql.exec.Operator.initialize(Operator.java:385)
> 	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordProcessor.init(ReduceRecordProcessor.java:116)
> 	... 14 more
> ]], Vertex failed as one or more tasks failed. failedTasks:1, Vertex vertex_1414029100044_0733_1_05 [Reducer 11] killed/failed due to:null]
> 14/10/30 01:19:19 [main]: ERROR tez.TezJobMonitor: Vertex failed, vertexName=Reducer 11, vertexId=vertex_1414029100044_0733_1_05, diagnostics=[Task failed, taskId=task_1414029100044_0733_1_05_000027, diagnostics=[TaskAttempt 0 failed, info=[Error: Failure while running task:java.lang.RuntimeException: java.lang.RuntimeException: java.lang.RuntimeException: problem advancing post rec#334499
> 	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:186)
> 	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.run(TezProcessor.java:138)
> 	at org.apache.tez.runtime.LogicalIOProcessorRuntimeTask.run(LogicalIOProcessorRuntimeTask.java:324)
> 	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable$1.run(TezTaskRunner.java:176)
> 	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable$1.run(TezTaskRunner.java:168)
> 	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:1548)
> 	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable.call(TezTaskRunner.java:168)
> 	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable.call(TezTaskRunner.java:163)
> 	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:744)
> Caused by: java.lang.RuntimeException: java.lang.RuntimeException: problem advancing post rec#334499
> 	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource.pushRecord(ReduceRecordSource.java:262)
> 	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordProcessor.run(ReduceRecordProcessor.java:168)
> 	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:163)
> 	... 13 more
> Caused by: java.lang.RuntimeException: problem advancing post rec#334499
> 	at org.apache.tez.runtime.library.common.ValuesIterator$1$1.next(ValuesIterator.java:142)
> 	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource.processKeyValues(ReduceRecordSource.java:288)
> 	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordSource.pushRecord(ReduceRecordSource.java:252)
> 	... 15 more
> Caused by: org.apache.hadoop.fs.ChecksumException: Checksum Error:  CurrentOffset=166741, offset=4, off=0, dataLength=167232, origLen=495, len=491, length=167236, checksumSize=4
> 	at org.apache.tez.runtime.library.common.sort.impl.IFileInputStream.doRead(IFileInputStream.java:254)
> 	at org.apache.tez.runtime.library.common.sort.impl.IFileInputStream.read(IFileInputStream.java:184)
> 	at org.apache.tez.runtime.library.common.sort.impl.IFileInputStream.close(IFileInputStream.java:131)
> 	at org.apache.hadoop.io.compress.DecompressorStream.close(DecompressorStream.java:205)
> 	at org.apache.tez.runtime.library.common.sort.impl.IFile$Reader.close(IFile.java:784)
> 	at org.apache.tez.runtime.library.common.sort.impl.TezMerger$Segment.closeReader(TezMerger.java:332)
> 	at org.apache.tez.runtime.library.common.sort.impl.TezMerger$Segment.close(TezMerger.java:338)
> 	at org.apache.tez.runtime.library.common.sort.impl.TezMerger$MergeQueue.adjustPriorityQueue(TezMerger.java:489)
> 	at org.apache.tez.runtime.library.common.sort.impl.TezMerger$MergeQueue.next(TezMerger.java:503)
> 	at org.apache.tez.runtime.library.common.shuffle.orderedgrouped.MergeManager$RawKVIteratorReader.readRawKey(MergeManager.java:765)
> 	at org.apache.tez.runtime.library.common.sort.impl.TezMerger$Segment.readRawKey(TezMerger.java:319)
> 	at org.apache.tez.runtime.library.common.sort.impl.TezMerger$MergeQueue.adjustPriorityQueue(TezMerger.java:481)
> 	at org.apache.tez.runtime.library.common.sort.impl.TezMerger$MergeQueue.next(TezMerger.java:503)
> 	at org.apache.tez.runtime.library.common.ValuesIterator.readNextKey(ValuesIterator.java:181)
> 	at org.apache.tez.runtime.library.common.ValuesIterator.access$300(ValuesIterator.java:47)
> 	at org.apache.tez.runtime.library.common.ValuesIterator$1$1.next(ValuesIterator.java:140)
> 	... 17 more
> ], TaskAttempt 1 failed, info=[Error: Failure while running task:java.lang.RuntimeException: java.lang.RuntimeException: Reduce operator initialization failed
> 	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:186)
> 	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.run(TezProcessor.java:138)
> 	at org.apache.tez.runtime.LogicalIOProcessorRuntimeTask.run(LogicalIOProcessorRuntimeTask.java:324)
> 	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable$1.run(TezTaskRunner.java:176)
> 	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable$1.run(TezTaskRunner.java:168)
> 	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:1548)
> 	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable.call(TezTaskRunner.java:168)
> 	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable.call(TezTaskRunner.java:163)
> 	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:744)
> Caused by: java.lang.RuntimeException: Reduce operator initialization failed
> 	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordProcessor.init(ReduceRecordProcessor.java:146)
> 	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:162)
> 	... 13 more
> Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: : init not supported
> 	at org.apache.hadoop.hive.ql.udf.generic.GenericUDAFStreamingEvaluator.init(GenericUDAFStreamingEvaluator.java:70)
> 	at org.apache.hadoop.hive.ql.plan.PTFDeserializer.setupWdwFnEvaluator(PTFDeserializer.java:209)
> 	at org.apache.hadoop.hive.ql.plan.PTFDeserializer.initializeWindowing(PTFDeserializer.java:130)
> 	at org.apache.hadoop.hive.ql.plan.PTFDeserializer.initializePTFChain(PTFDeserializer.java:94)
> 	at org.apache.hadoop.hive.ql.exec.PTFOperator.reconstructQueryDef(PTFOperator.java:144)
> 	at org.apache.hadoop.hive.ql.exec.PTFOperator.initializeOp(PTFOperator.java:74)
> 	at org.apache.hadoop.hive.ql.exec.Operator.initialize(Operator.java:385)
> 	at org.apache.hadoop.hive.ql.exec.Operator.initialize(Operator.java:469)
> 	at org.apache.hadoop.hive.ql.exec.Operator.initializeChildren(Operator.java:425)
> 	at org.apache.hadoop.hive.ql.exec.ExtractOperator.initializeOp(ExtractOperator.java:40)
> 	at org.apache.hadoop.hive.ql.exec.Operator.initialize(Operator.java:385)
> 	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordProcessor.init(ReduceRecordProcessor.java:116)
> 	... 14 more
> ], TaskAttempt 2 failed, info=[Error: Failure while running task:java.lang.RuntimeException: java.lang.RuntimeException: Reduce operator initialization failed
> 	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:186)
> 	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.run(TezProcessor.java:138)
> 	at org.apache.tez.runtime.LogicalIOProcessorRuntimeTask.run(LogicalIOProcessorRuntimeTask.java:324)
> 	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable$1.run(TezTaskRunner.java:176)
> 	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable$1.run(TezTaskRunner.java:168)
> 	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:1548)
> 	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable.call(TezTaskRunner.java:168)
> 	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable.call(TezTaskRunner.java:163)
> 	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:744)
> Caused by: java.lang.RuntimeException: Reduce operator initialization failed
> 	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordProcessor.init(ReduceRecordProcessor.java:146)
> 	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:162)
> 	... 13 more
> Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: : init not supported
> 	at org.apache.hadoop.hive.ql.udf.generic.GenericUDAFStreamingEvaluator.init(GenericUDAFStreamingEvaluator.java:70)
> 	at org.apache.hadoop.hive.ql.plan.PTFDeserializer.setupWdwFnEvaluator(PTFDeserializer.java:209)
> 	at org.apache.hadoop.hive.ql.plan.PTFDeserializer.initializeWindowing(PTFDeserializer.java:130)
> 	at org.apache.hadoop.hive.ql.plan.PTFDeserializer.initializePTFChain(PTFDeserializer.java:94)
> 	at org.apache.hadoop.hive.ql.exec.PTFOperator.reconstructQueryDef(PTFOperator.java:144)
> 	at org.apache.hadoop.hive.ql.exec.PTFOperator.initializeOp(PTFOperator.java:74)
> 	at org.apache.hadoop.hive.ql.exec.Operator.initialize(Operator.java:385)
> 	at org.apache.hadoop.hive.ql.exec.Operator.initialize(Operator.java:469)
> 	at org.apache.hadoop.hive.ql.exec.Operator.initializeChildren(Operator.java:425)
> 	at org.apache.hadoop.hive.ql.exec.ExtractOperator.initializeOp(ExtractOperator.java:40)
> 	at org.apache.hadoop.hive.ql.exec.Operator.initialize(Operator.java:385)
> 	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordProcessor.init(ReduceRecordProcessor.java:116)
> 	... 14 more
> ], TaskAttempt 3 failed, info=[Error: Failure while running task:java.lang.RuntimeException: java.lang.RuntimeException: Reduce operator initialization failed
> 	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:186)
> 	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.run(TezProcessor.java:138)
> 	at org.apache.tez.runtime.LogicalIOProcessorRuntimeTask.run(LogicalIOProcessorRuntimeTask.java:324)
> 	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable$1.run(TezTaskRunner.java:176)
> 	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable$1.run(TezTaskRunner.java:168)
> 	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:1548)
> 	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable.call(TezTaskRunner.java:168)
> 	at org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable.call(TezTaskRunner.java:163)
> 	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:744)
> Caused by: java.lang.RuntimeException: Reduce operator initialization failed
> 	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordProcessor.init(ReduceRecordProcessor.java:146)
> 	at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:162)
> 	... 13 more
> Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: : init not supported
> 	at org.apache.hadoop.hive.ql.udf.generic.GenericUDAFStreamingEvaluator.init(GenericUDAFStreamingEvaluator.java:70)
> 	at org.apache.hadoop.hive.ql.plan.PTFDeserializer.setupWdwFnEvaluator(PTFDeserializer.java:209)
> 	at org.apache.hadoop.hive.ql.plan.PTFDeserializer.initializeWindowing(PTFDeserializer.java:130)
> 	at org.apache.hadoop.hive.ql.plan.PTFDeserializer.initializePTFChain(PTFDeserializer.java:94)
> 	at org.apache.hadoop.hive.ql.exec.PTFOperator.reconstructQueryDef(PTFOperator.java:144)
> 	at org.apache.hadoop.hive.ql.exec.PTFOperator.initializeOp(PTFOperator.java:74)
> 	at org.apache.hadoop.hive.ql.exec.Operator.initialize(Operator.java:385)
> 	at org.apache.hadoop.hive.ql.exec.Operator.initialize(Operator.java:469)
> 	at org.apache.hadoop.hive.ql.exec.Operator.initializeChildren(Operator.java:425)
> 	at org.apache.hadoop.hive.ql.exec.ExtractOperator.initializeOp(ExtractOperator.java:40)
> 	at org.apache.hadoop.hive.ql.exec.Operator.initialize(Operator.java:385)
> 	at org.apache.hadoop.hive.ql.exec.tez.ReduceRecordProcessor.init(ReduceRecordProcessor.java:116)
> 	... 14 more
> ]], Vertex failed as one or more tasks failed. failedTasks:1, Vertex vertex_1414029100044_0733_1_05 [Reducer 11] killed/failed due to:null]
> Vertex killed, vertexName=Reducer 7, vertexId=vertex_1414029100044_0733_1_10, diagnostics=[Vertex received Kill while in RUNNING state., Vertex killed as other vertex failed. failedTasks:0, Vertex vertex_1414029100044_0733_1_10 [Reducer 7] killed/failed due to:null]
> 14/10/30 01:19:19 [main]: ERROR tez.TezJobMonitor: Vertex killed, vertexName=Reducer 7, vertexId=vertex_1414029100044_0733_1_10, diagnostics=[Vertex received Kill while in RUNNING state., Vertex killed as other vertex failed. failedTasks:0, Vertex vertex_1414029100044_0733_1_10 [Reducer 7] killed/failed due to:null]
> Vertex killed, vertexName=Reducer 6, vertexId=vertex_1414029100044_0733_1_09, diagnostics=[Vertex received Kill while in RUNNING state., Vertex killed as other vertex failed. failedTasks:0, Vertex vertex_1414029100044_0733_1_09 [Reducer 6] killed/failed due to:null]
> 14/10/30 01:19:19 [main]: ERROR tez.TezJobMonitor: Vertex killed, vertexName=Reducer 6, vertexId=vertex_1414029100044_0733_1_09, diagnostics=[Vertex received Kill while in RUNNING state., Vertex killed as other vertex failed. failedTasks:0, Vertex vertex_1414029100044_0733_1_09 [Reducer 6] killed/failed due to:null]
> Vertex killed, vertexName=Reducer 4, vertexId=vertex_1414029100044_0733_1_07, diagnostics=[Vertex received Kill while in RUNNING state., Vertex killed as other vertex failed. failedTasks:0, Vertex vertex_1414029100044_0733_1_07 [Reducer 4] killed/failed due to:null]
> 14/10/30 01:19:19 [main]: ERROR tez.TezJobMonitor: Vertex killed, vertexName=Reducer 4, vertexId=vertex_1414029100044_0733_1_07, diagnostics=[Vertex received Kill while in RUNNING state., Vertex killed as other vertex failed. failedTasks:0, Vertex vertex_1414029100044_0733_1_07 [Reducer 4] killed/failed due to:null]
> Vertex killed, vertexName=Reducer 5, vertexId=vertex_1414029100044_0733_1_08, diagnostics=[Vertex received Kill while in RUNNING state., Vertex killed as other vertex failed. failedTasks:0, Vertex vertex_1414029100044_0733_1_08 [Reducer 5] killed/failed due to:null]
> 14/10/30 01:19:19 [main]: ERROR tez.TezJobMonitor: Vertex killed, vertexName=Reducer 5, vertexId=vertex_1414029100044_0733_1_08, diagnostics=[Vertex received Kill while in RUNNING state., Vertex killed as other vertex failed. failedTasks:0, Vertex vertex_1414029100044_0733_1_08 [Reducer 5] killed/failed due to:null]
> DAG failed due to vertex failure. failedVertices:1 killedVertices:4
> 14/10/30 01:19:19 [main]: ERROR tez.TezJobMonitor: DAG failed due to vertex failure. failedVertices:1 killedVertices:4
> FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.tez.TezTask
> 14/10/30 01:19:19 [main]: ERROR ql.Driver: FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.tez.TezTask
> {code}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)