Return-Path: X-Original-To: apmail-hive-dev-archive@www.apache.org Delivered-To: apmail-hive-dev-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 650AA177A8 for ; Fri, 31 Oct 2014 06:11:34 +0000 (UTC) Received: (qmail 54130 invoked by uid 500); 31 Oct 2014 06:11:34 -0000 Delivered-To: apmail-hive-dev-archive@hive.apache.org Received: (qmail 54063 invoked by uid 500); 31 Oct 2014 06:11:33 -0000 Mailing-List: contact dev-help@hive.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@hive.apache.org Delivered-To: mailing list dev@hive.apache.org Received: (qmail 54052 invoked by uid 500); 31 Oct 2014 06:11:33 -0000 Delivered-To: apmail-hadoop-hive-dev@hadoop.apache.org Received: (qmail 54049 invoked by uid 99); 31 Oct 2014 06:11:33 -0000 Received: from arcas.apache.org (HELO arcas.apache.org) (140.211.11.28) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 31 Oct 2014 06:11:33 +0000 Date: Fri, 31 Oct 2014 06:11:33 +0000 (UTC) From: "Gunther Hagleitner (JIRA)" To: hive-dev@hadoop.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Updated] (HIVE-8677) TPC-DS Q51 : fails with "init not supported" exception in GenericUDAFStreamingEvaluator.init MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 [ 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)