[ 
https://issues.apache.org/jira/browse/HIVE-20096?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16533871#comment-16533871
 ] 

Gopal V commented on HIVE-20096:
--------------------------------

bq. at 
org.apache.tez.mapreduce.common.MRInputAMSplitGenerator.handleInputInitializerEvent(MRInputAMSplitGenerator.java:170)

This is supposed to be using HiveSplitGenerator in Hive.

https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/HiveSplitGenerator.java#L377

Looks like a configuration issue with the split-generation, where the 
CombineInputFormat splits are generated in the CLI & sent over RPC (instead of 
being generated on the AM from HiveInputFormat).

Check {{hive.tez.input.format}} and {{hive.compute.splits.in.am}}.

> dynamic partition pruning results in java.lang.UnsupportedOperationException: 
> Not expecting to handle any events
> ----------------------------------------------------------------------------------------------------------------
>
>                 Key: HIVE-20096
>                 URL: https://issues.apache.org/jira/browse/HIVE-20096
>             Project: Hive
>          Issue Type: Bug
>    Affects Versions: 3.0.0
>         Environment: Hive 3.0.0 and Tez 0.91 . 
>            Reporter: Karthik
>            Priority: Major
>
> * Setting hive.tez.dynamic.partition.pruning=true partition results in error 
> "java.lang.UnsupportedOperationException: Not expecting to handle any events" 
> for some queries.
>  * Below is stack trace as shown in Hive CLI and JDBC.
> ----------------------------------------------------------------------------------------------
>  VERTICES MODE STATUS TOTAL COMPLETED RUNNING PENDING FAILED KILLED
> ----------------------------------------------------------------------------------------------
> Map 3 llap INITED 1 0 0 1 0 0
> Map 4 .......... llap SUCCEEDED 1 1 0 0 0 0
> Map 1 llap INITIALIZING -1 0 0 -1 0 0
> Reducer 2 llap INITED 3 0 0 3 0 0
> ----------------------------------------------------------------------------------------------
> VERTICES: 01/04 [=====>>---------------------] 20% ELAPSED TIME: 0.56 s
> ----------------------------------------------------------------------------------------------
> 2018-07-05 06:41:22,813 INFO [32a75ad3-f0de-4255-b1d5-0c919e3bc672 main] 
> SessionState (SessionState.java:logInfo(1163)) - Map 1: -/- Map 3: 0/1 Map 4: 
> 1/1 Reducer 2: 0/3
> Status: Failed
> 2018-07-05 06:41:22,814 ERROR [32a75ad3-f0de-4255-b1d5-0c919e3bc672 main] 
> SessionState (SessionState.java:printError(1237)) - Status: Failed
> Vertex failed, vertexName=Map 1, vertexId=vertex_1530776596475_0020_4_02, 
> diagnostics=[Vertex vertex_1530776596475_0020_4_02 [Map 1] killed/failed due 
> to:ROOT_INPUT_INIT_FAILURE, Vertex Input: my_fact_table initializer failed, 
> vertex=vertex_1530776596475_0020_4_02 [Map 1], 
> java.lang.UnsupportedOperationException: Not expecting to handle any events
>  at 
> org.apache.tez.mapreduce.common.MRInputAMSplitGenerator.handleInputInitializerEvent(MRInputAMSplitGenerator.java:170)
>  at 
> org.apache.tez.dag.app.dag.RootInputInitializerManager$InitializerWrapper.sendEvents(RootInputInitializerManager.java:518)
>  at 
> org.apache.tez.dag.app.dag.RootInputInitializerManager$InitializerWrapper.handleInputInitializerEvents(RootInputInitializerManager.java:511)
>  at 
> org.apache.tez.dag.app.dag.RootInputInitializerManager.handleInitializerEvents(RootInputInitializerManager.java:201)
>  at 
> org.apache.tez.dag.app.dag.impl.VertexImpl.handleRoutedTezEvents(VertexImpl.java:4009)
>  at 
> org.apache.tez.dag.app.dag.impl.VertexImpl.access$4000(VertexImpl.java:205)
>  at 
> org.apache.tez.dag.app.dag.impl.VertexImpl$RouteEventTransition.transition(VertexImpl.java:3738)
>  at 
> org.apache.tez.dag.app.dag.impl.VertexImpl$RouteEventTransition.transition(VertexImpl.java:3731)
>  at 
> org.apache.hadoop.yarn.state.StateMachineFactory$MultipleInternalArc.doTransition(StateMachineFactory.java:385)
>  at 
> org.apache.hadoop.yarn.state.StateMachineFactory.doTransition(StateMachineFactory.java:302)
>  at 
> org.apache.hadoop.yarn.state.StateMachineFactory.access$500(StateMachineFactory.java:46)
>  at 
> org.apache.hadoop.yarn.state.StateMachineFactory$InternalStateMachine.doTransition(StateMachineFactory.java:487)
>  at org.apache.tez.state.StateMachineTez.doTransition(StateMachineTez.java:59)
>  at org.apache.tez.dag.app.dag.impl.VertexImpl.handle(VertexImpl.java:1939)
>  at org.apache.tez.dag.app.dag.impl.VertexImpl.handle(VertexImpl.java:204)
>  at 
> org.apache.tez.dag.app.DAGAppMaster$VertexEventDispatcher.handle(DAGAppMaster.java:2317)
>  at 
> org.apache.tez.dag.app.DAGAppMaster$VertexEventDispatcher.handle(DAGAppMaster.java:2303)
>  at org.apache.tez.common.AsyncDispatcher.dispatch(AsyncDispatcher.java:180)
>  at org.apache.tez.common.AsyncDispatcher$1.run(AsyncDispatcher.java:115)
>  at java.lang.Thread.run(Thread.java:748)
> ]
> 2018-07-05 06:41:22,814 ERROR [32a75ad3-f0de-4255-b1d5-0c919e3bc672 main] 
> SessionState (SessionState.java:printError(1237)) - Vertex failed, 
> vertexName=Map 1, vertexId=vertex_1530776596475_0020_4_02, 
> diagnostics=[Vertex vertex_1530776596475_0020_4_02 [Map 1] killed/failed due 
> to:ROOT_INPUT_INIT_FAILURE, Vertex Input: my_fact_table initializer failed, 
> vertex=vertex_1530776596475_0020_4_02 [Map 1], 
> java.lang.UnsupportedOperationException: Not expecting to handle any events
>  at 
> org.apache.tez.mapreduce.common.MRInputAMSplitGenerator.handleInputInitializerEvent(MRInputAMSplitGenerator.java:170)
>  at 
> org.apache.tez.dag.app.dag.RootInputInitializerManager$InitializerWrapper.sendEvents(RootInputInitializerManager.java:518)
>  at 
> org.apache.tez.dag.app.dag.RootInputInitializerManager$InitializerWrapper.handleInputInitializerEvents(RootInputInitializerManager.java:511)
>  at 
> org.apache.tez.dag.app.dag.RootInputInitializerManager.handleInitializerEvents(RootInputInitializerManager.java:201)
>  at 
> org.apache.tez.dag.app.dag.impl.VertexImpl.handleRoutedTezEvents(VertexImpl.java:4009)
>  at 
> org.apache.tez.dag.app.dag.impl.VertexImpl.access$4000(VertexImpl.java:205)
>  at 
> org.apache.tez.dag.app.dag.impl.VertexImpl$RouteEventTransition.transition(VertexImpl.java:3738)
>  at 
> org.apache.tez.dag.app.dag.impl.VertexImpl$RouteEventTransition.transition(VertexImpl.java:3731)
>  at 
> org.apache.hadoop.yarn.state.StateMachineFactory$MultipleInternalArc.doTransition(StateMachineFactory.java:385)
>  at 
> org.apache.hadoop.yarn.state.StateMachineFactory.doTransition(StateMachineFactory.java:302)
>  at 
> org.apache.hadoop.yarn.state.StateMachineFactory.access$500(StateMachineFactory.java:46)
>  at 
> org.apache.hadoop.yarn.state.StateMachineFactory$InternalStateMachine.doTransition(StateMachineFactory.java:487)
>  at org.apache.tez.state.StateMachineTez.doTransition(StateMachineTez.java:59)
>  at org.apache.tez.dag.app.dag.impl.VertexImpl.handle(VertexImpl.java:1939)
>  at org.apache.tez.dag.app.dag.impl.VertexImpl.handle(VertexImpl.java:204)
>  at 
> org.apache.tez.dag.app.DAGAppMaster$VertexEventDispatcher.handle(DAGAppMaster.java:2317)
>  at 
> org.apache.tez.dag.app.DAGAppMaster$VertexEventDispatcher.handle(DAGAppMaster.java:2303)
>  at org.apache.tez.common.AsyncDispatcher.dispatch(AsyncDispatcher.java:180)
>  at org.apache.tez.common.AsyncDispatcher$1.run(AsyncDispatcher.java:115)
>  at java.lang.Thread.run(Thread.java:748)
> ]
> Vertex killed, vertexName=Reducer 2, vertexId=vertex_1530776596475_0020_4_03, 
> diagnostics=[Vertex received Kill in INITED state., Vertex 
> vertex_1530776596475_0020_4_03 [Reducer 2] killed/failed due 
> to:OTHER_VERTEX_FAILURE]
> 2018-07-05 06:41:22,814 ERROR [32a75ad3-f0de-4255-b1d5-0c919e3bc672 main] 
> SessionState (SessionState.java:printError(1237)) - Vertex killed, 
> vertexName=Reducer 2, vertexId=vertex_1530776596475_0020_4_03, 
> diagnostics=[Vertex received Kill in INITED state., Vertex 
> vertex_1530776596475_0020_4_03 [Reducer 2] killed/failed due 
> to:OTHER_VERTEX_FAILURE]
> Vertex killed, vertexName=Map 3, vertexId=vertex_1530776596475_0020_4_01, 
> diagnostics=[Vertex received Kill while in RUNNING state., Vertex did not 
> succeed due to OTHER_VERTEX_FAILURE, failedTasks:0 killedTasks:1, Vertex 
> vertex_1530776596475_0020_4_01 [Map 3] killed/failed due 
> to:OTHER_VERTEX_FAILURE]
> 2018-07-05 06:41:22,814 ERROR [32a75ad3-f0de-4255-b1d5-0c919e3bc672 main] 
> SessionState (SessionState.java:printError(1237)) - Vertex killed, 
> vertexName=Map 3, vertexId=vertex_1530776596475_0020_4_01, 
> diagnostics=[Vertex received Kill while in RUNNING state., Vertex did not 
> succeed due to OTHER_VERTEX_FAILURE, failedTasks:0 killedTasks:1, Vertex 
> vertex_1530776596475_0020_4_01 [Map 3] killed/failed due 
> to:OTHER_VERTEX_FAILURE]
> DAG did not succeed due to VERTEX_FAILURE. failedVertices:1 killedVertices:2
> 2018-07-05 06:41:22,814 ERROR [32a75ad3-f0de-4255-b1d5-0c919e3bc672 main] 
> SessionState (SessionState.java:printError(1237)) - DAG did not succeed due 
> to VERTEX_FAILURE. failedVertices:1 killedVertices:2
> 2018-07-05 06:41:22,833 INFO [32a75ad3-f0de-4255-b1d5-0c919e3bc672 main] 
> reexec.ReOptimizePlugin (ReOptimizePlugin.java:run(70)) - ReOptimization: 
> retryPossible: false
> FAILED: Execution Error, return code 2 from 
> org.apache.hadoop.hive.ql.exec.tez.TezTask. Vertex failed, vertexName=Map 1, 
> vertexId=vertex_1530776596475_0020_4_02, diagnostics=[Vertex 
> vertex_1530776596475_0020_4_02 [Map 1] killed/failed due 
> to:ROOT_INPUT_INIT_FAILURE, Vertex Input: my_fact_table initializer failed, 
> vertex=vertex_1530776596475_0020_4_02 [Map 1], 
> java.lang.UnsupportedOperationException: Not expecting to handle any events
>  at 
> org.apache.tez.mapreduce.common.MRInputAMSplitGenerator.handleInputInitializerEvent(MRInputAMSplitGenerator.java:170)
>  at 
> org.apache.tez.dag.app.dag.RootInputInitializerManager$InitializerWrapper.sendEvents(RootInputInitializerManager.java:518)
>  at 
> org.apache.tez.dag.app.dag.RootInputInitializerManager$InitializerWrapper.handleInputInitializerEvents(RootInputInitializerManager.java:511)
>  at 
> org.apache.tez.dag.app.dag.RootInputInitializerManager.handleInitializerEvents(RootInputInitializerManager.java:201)
>  at 
> org.apache.tez.dag.app.dag.impl.VertexImpl.handleRoutedTezEvents(VertexImpl.java:4009)
>  at 
> org.apache.tez.dag.app.dag.impl.VertexImpl.access$4000(VertexImpl.java:205)
>  at 
> org.apache.tez.dag.app.dag.impl.VertexImpl$RouteEventTransition.transition(VertexImpl.java:3738)
>  at 
> org.apache.tez.dag.app.dag.impl.VertexImpl$RouteEventTransition.transition(VertexImpl.java:3731)
>  at 
> org.apache.hadoop.yarn.state.StateMachineFactory$MultipleInternalArc.doTransition(StateMachineFactory.java:385)
>  at 
> org.apache.hadoop.yarn.state.StateMachineFactory.doTransition(StateMachineFactory.java:302)
>  at 
> org.apache.hadoop.yarn.state.StateMachineFactory.access$500(StateMachineFactory.java:46)
>  at 
> org.apache.hadoop.yarn.state.StateMachineFactory$InternalStateMachine.doTransition(StateMachineFactory.java:487)
>  at org.apache.tez.state.StateMachineTez.doTransition(StateMachineTez.java:59)
>  at org.apache.tez.dag.app.dag.impl.VertexImpl.handle(VertexImpl.java:1939)
>  at org.apache.tez.dag.app.dag.impl.VertexImpl.handle(VertexImpl.java:204)
>  at 
> org.apache.tez.dag.app.DAGAppMaster$VertexEventDispatcher.handle(DAGAppMaster.java:2317)
>  at 
> org.apache.tez.dag.app.DAGAppMaster$VertexEventDispatcher.handle(DAGAppMaster.java:2303)
>  at org.apache.tez.common.AsyncDispatcher.dispatch(AsyncDispatcher.java:180)
>  at org.apache.tez.common.AsyncDispatcher$1.run(AsyncDispatcher.java:115)
>  at java.lang.Thread.run(Thread.java:748)
>  
>  * This issue appears to be similar to 
> https://issues.apache.org/jira/browse/TEZ-3336
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to