Yi Zhang created TEZ-3474: ----------------------------- Summary: CombineHiveInputFormat with Tez fails to initiate vertex if table is empty Key: TEZ-3474 URL: https://issues.apache.org/jira/browse/TEZ-3474 Project: Apache Tez Issue Type: Bug Affects Versions: 0.7.1 Reporter: Yi Zhang
Sometimes user have developed custom inputformat that extends from CombineHiveInputFormat due to difficulty of extending from HiveInputFormat directly, for example to filter out old data files. in this use case, vertex fails to get initialized: SELECT city.cid FROM (select city_id as cid, row_number() over(partition by timezone order by population) rnum from cities) city JOIN (select datestr, id from yizhang.emptyparts where datestr >= date_sub(current_date(),30)) emp on city.cid = emp.id ; -------------------------------------------------------------------------------- VERTICES STATUS TOTAL COMPLETED RUNNING PENDING FAILED KILLED -------------------------------------------------------------------------------- Map 1 KILLED -1 0 0 -1 0 0 Map 3 FAILED -1 0 0 -1 0 0 Reducer 2 KILLED 1 0 0 1 0 0 -------------------------------------------------------------------------------- VERTICES: 00/03 [>>--------------------------] 0% ELAPSED TIME: 0.34 s -------------------------------------------------------------------------------- Status: Failed Vertex failed, vertexName=Map 3, vertexId=vertex_1476217616538_398108_1_01, diagnostics=[Vertex vertex_1476217616538_398108_1_01 [Map 3] killed/failed due to:ROOT_INPUT_INIT_FAILURE, Vertex Input: emp initializer failed, vertex=vertex_1476217616538_398108_1_01 [Map 3], java.lang.IllegalArgumentException at java.util.concurrent.ThreadPoolExecutor.<init>(ThreadPoolExecutor.java:1307) at java.util.concurrent.ThreadPoolExecutor.<init>(ThreadPoolExecutor.java:1195) at java.util.concurrent.Executors.newFixedThreadPool(Executors.java:89) at org.apache.hadoop.hive.ql.io.CombineHiveInputFormat.getSplits(CombineHiveInputFormat.java:519) at org.apache.tez.mapreduce.hadoop.MRInputHelpers.generateOldSplits(MRInputHelpers.java:447) at org.apache.tez.mapreduce.hadoop.MRInputHelpers.generateInputSplitsToMem(MRInputHelpers.java:299) at org.apache.tez.mapreduce.common.MRInputAMSplitGenerator.initialize(MRInputAMSplitGenerator.java:121) at org.apache.tez.dag.app.dag.RootInputInitializerManager$InputInitializerCallable$1.run(RootInputInitializerManager.java:264) at org.apache.tez.dag.app.dag.RootInputInitializerManager$InputInitializerCallable$1.run(RootInputInitializerManager.java:258) at java.security.AccessController.doPrivileged(Native Method) at javax.security.auth.Subject.doAs(Subject.java:422) at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1671) at org.apache.tez.dag.app.dag.RootInputInitializerManager$InputInitializerCallable.call(RootInputInitializerManager.java:258) at org.apache.tez.dag.app.dag.RootInputInitializerManager$InputInitializerCallable.call(RootInputInitializerManager.java:245) at java.util.concurrent.FutureTask.run(FutureTask.java:266) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) at java.lang.Thread.run(Thread.java:745) ] -- This message was sent by Atlassian JIRA (v6.3.4#6332)