You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hive.apache.org by "wanguangping (Jira)" <ji...@apache.org> on 2020/06/30 03:01:00 UTC

[jira] [Reopened] (HIVE-23748) tez task with File Merge operator generate tmp file with wrong suffix

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

wanguangping reopened HIVE-23748:
---------------------------------

> tez task with File Merge operator generate tmp file with wrong suffix
> ---------------------------------------------------------------------
>
>                 Key: HIVE-23748
>                 URL: https://issues.apache.org/jira/browse/HIVE-23748
>             Project: Hive
>          Issue Type: Bug
>          Components: Tez
>    Affects Versions: 3.1.0
>            Reporter: wanguangping
>            Priority: Major
>
> h1. background
>  * SQL on TEZ 
>  * it's a Occasional problem
> h1. hiveserver2 log
> SLF4J: Class path contains multiple SLF4J bindings.
>  SLF4J: Found binding in [jar:file:/usr/hdp/3.1.4.0-315/hive/lib/log4j-slf4j-impl-2.10.0.jar!/org/slf4j/impl/StaticLoggerBinder.class]
>  SLF4J: Found binding in [jar:file:/usr/hdp/3.1.4.0-315/hadoop/lib/slf4j-log4j12-1.7.25.jar!/org/slf4j/impl/StaticLoggerBinder.class]
>  SLF4J: See [http://www.slf4j.org/codes.html#multiple_bindings] for an explanation.
>  SLF4J: Actual binding is of type [org.apache.logging.slf4j.Log4jLoggerFactory]
>  Connecting to jdbc:hive2://xxx:10000/prod
>  Connected to: Apache Hive (version 3.1.0.3.1.4.0-315)
>  Driver: Hive JDBC (version 3.1.0.3.1.4.0-315)
>  Transaction isolation: TRANSACTION_REPEATABLE_READ
>  INFO : Compiling command(queryId=hive_20200609033313_ed882b48-7ab4-42a2-84e4-c9ef764271e2): use prod
>  INFO : Concurrency mode is disabled, not creating a lock manager
>  INFO : Semantic Analysis Completed (retrial = false)
>  INFO : Returning Hive schema: Schema(fieldSchemas:null, properties:null)
>  INFO : Completed compiling command(queryId=hive_20200609033313_ed882b48-7ab4-42a2-84e4-c9ef764271e2); Time taken: 0.887 seconds
>  INFO : Concurrency mode is disabled, not creating a lock manager
>  INFO : Executing command(queryId=hive_20200609033313_ed882b48-7ab4-42a2-84e4-c9ef764271e2): use prod
>  INFO : Starting task [Stage-0:DDL] in serial mode
>  INFO : Completed executing command(queryId=hive_20200609033313_ed882b48-7ab4-42a2-84e4-c9ef764271e2); Time taken: 0.197 seconds
>  INFO : OK
>  INFO : Concurrency mode is disabled, not creating a lock manager
>  No rows affected (1.096 seconds)
>  No rows affected (0.004 seconds)
>  INFO : Compiling command(queryId=hive_20200609033314_cba66b08-ad42-4b94-ad61-d15fe48efe23): drop table if exists temp.shawnlee_newbase_devicebase
>  INFO : Concurrency mode is disabled, not creating a lock manager
>  INFO : Semantic Analysis Completed (retrial = false)
>  INFO : Returning Hive schema: Schema(fieldSchemas:null, properties:null)
>  INFO : Completed compiling command(queryId=hive_20200609033314_cba66b08-ad42-4b94-ad61-d15fe48efe23); Time taken: 1.324 seconds
>  INFO : Concurrency mode is disabled, not creating a lock manager
>  INFO : Executing command(queryId=hive_20200609033314_cba66b08-ad42-4b94-ad61-d15fe48efe23): drop table if exists temp.shawnlee_newbase_devicebase
>  INFO : Starting task [Stage-0:DDL] in serial mode
>  INFO : Completed executing command(queryId=hive_20200609033314_cba66b08-ad42-4b94-ad61-d15fe48efe23); Time taken: 12.895 seconds
>  INFO : OK
>  INFO : Concurrency mode is disabled, not creating a lock manager
>  No rows affected (14.229 seconds)
>  INFO : Compiling command(queryId=hive_20200609033329_3fbf0a38-e5b0-4e3a-ae8b-ef95f400b50f): xxxxx
>  INFO : Concurrency mode is disabled, not creating a lock manager
>  INFO : No Stats for user_profile@dw_uba_event_daily, Columns: attribute, event
>  INFO : Semantic Analysis Completed (retrial = false)
>  INFO : Returning Hive schema: Schema(fieldSchemas:[FieldSchema(name:day, type:string, comment:null), FieldSchema(name:device_id, type:string, comment:null), FieldSchema(name:is_new, type:int, comment:null), FieldSchema(name:first_attribute, type:map<string,string>, comment:null), FieldSchema(name:first_app_version, type:string, comment:null), FieldSchema(name:first_platform_type, type:string, comment:null), FieldSchema(name:first_manufacturer, type:string, comment:null), FieldSchema(name:first_model, type:string, comment:null), FieldSchema(name:first_ipprovince, type:string, comment:null), FieldSchema(name:first_ipcity, type:string, comment:null), FieldSchema(name:last_attribute, type:map<string,string>, comment:null), FieldSchema(name:last_app_version, type:string, comment:null), FieldSchema(name:last_platform_type, type:string, comment:null), FieldSchema(name:last_manufacturer, type:string, comment:null), FieldSchema(name:last_model, type:string, comment:null), FieldSchema(name:last_ipprovince, type:string, comment:null), FieldSchema(name:last_ipcity, type:string, comment:null)], properties:null)
>  INFO : Completed compiling command(queryId=hive_20200609033329_3fbf0a38-e5b0-4e3a-ae8b-ef95f400b50f); Time taken: 78.517 seconds
>  INFO : Concurrency mode is disabled, not creating a lock manager
>  INFO : Executing command(queryId=hive_20200609033329_3fbf0a38-e5b0-4e3a-ae8b-ef95f400b50f): xxxx
>  INFO : Query ID = hive_20200609033329_3fbf0a38-e5b0-4e3a-ae8b-ef95f400b50f
>  INFO : Total jobs = 3
>  INFO : Launching Job 1 out of 3
>  INFO : Starting task [Stage-1:MAPRED] in serial mode
>  INFO : Subscribed to counters: [] for queryId: hive_20200609033329_3fbf0a38-e5b0-4e3a-ae8b-ef95f400b50f
>  INFO : Tez session hasn't been created yet. Opening session
>  INFO : Dag name: xxxx
>  INFO : Status: Running (Executing on YARN cluster with App id application_1590738759875_100602)
>  。。。。
>  INFO : Starting task [Stage-8:CONDITIONAL] in serial mode
>  INFO : Stage-5 is filtered out by condition resolver.
>  INFO : Stage-4 is selected by condition resolver.
>  INFO : Stage-6 is filtered out by condition resolver.
>  INFO : Launching Job 3 out of 3
>  INFO : Starting task [Stage-4:MAPRED] in serial mode
>  INFO : Subscribed to counters: [] for queryId: hive_20200609033329_3fbf0a38-e5b0-4e3a-ae8b-ef95f400b50f
>  INFO : Session is already open
>  INFO : Dag name:xxxx
>  INFO : Status: Running (Executing on YARN cluster with App id application_1590738759875_100602)
>  ERROR : Status: Failed
>  ERROR : Vertex failed, vertexName=File Merge, vertexId=vertex_1590738759875_100602_2_00, diagnostics=[Task failed, taskId=task_1590738759875_100602_2_00_000007, diagnostics=[TaskAttempt 0 failed, info=[Error: Error while running task ( failure ) : attempt_1590738759875_100602_2_00_000007_0:java.lang.RuntimeException: java.lang.RuntimeException: Hive Runtime Error while closing operators
>  at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:296)
>  at org.apache.hadoop.hive.ql.exec.tez.MergeFileTezProcessor.run(MergeFileTezProcessor.java:42)
>  at org.apache.tez.runtime.LogicalIOProcessorRuntimeTask.run(LogicalIOProcessorRuntimeTask.java:374)
>  at org.apache.tez.runtime.task.TaskRunner2Callable$1.run(TaskRunner2Callable.java:73)
>  at org.apache.tez.runtime.task.TaskRunner2Callable$1.run(TaskRunner2Callable.java:61)
>  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:1730)
>  at org.apache.tez.runtime.task.TaskRunner2Callable.callInternal(TaskRunner2Callable.java:61)
>  at org.apache.tez.runtime.task.TaskRunner2Callable.callInternal(TaskRunner2Callable.java:37)
>  at org.apache.tez.common.CallableWithNdc.call(CallableWithNdc.java:36)
>  at com.google.common.util.concurrent.TrustedListenableFutureTask$TrustedFutureInterruptibleTask.runInterruptibly(TrustedListenableFutureTask.java:125)
>  at com.google.common.util.concurrent.InterruptibleTask.run(InterruptibleTask.java:69)
>  at com.google.common.util.concurrent.TrustedListenableFutureTask.run(TrustedListenableFutureTask.java:78)
>  at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>  at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>  at java.lang.Thread.run(Thread.java:748)
>  Caused by: java.lang.RuntimeException: Hive Runtime Error while closing operators
>  at org.apache.hadoop.hive.ql.exec.tez.MergeFileRecordProcessor.close(MergeFileRecordProcessor.java:188)
>  at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:284)
>  ... 16 more
>  Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: Failed to close AbstractFileMergeOperator
>  at org.apache.hadoop.hive.ql.exec.AbstractFileMergeOperator.closeOp(AbstractFileMergeOperator.java:315)
>  at org.apache.hadoop.hive.ql.exec.OrcFileMergeOperator.closeOp(OrcFileMergeOperator.java:265)
>  at org.apache.hadoop.hive.ql.exec.Operator.close(Operator.java:732)
>  at org.apache.hadoop.hive.ql.exec.tez.MergeFileRecordProcessor.close(MergeFileRecordProcessor.java:180)
>  ... 17 more
>  Caused by: java.io.IOException: Unable to rename xxx/warehouse/tablespace/hive/temp.db/.hive-staging_hive_2020-06-09_03-33-29_060_8741275828320666826-9431/_task_tmp.-ext-10002/_tmp.000002_0 to xxx/warehouse/tablespace/hive/temp.db/.hive-staging_hive_2020-06-09_03-33-29_060_8741275828320666826-9431/_tmp.-ext-10002/000002_0
>  at org.apache.hadoop.hive.ql.exec.AbstractFileMergeOperator.closeOp(AbstractFileMergeOperator.java:254)
>  ... 20 more
>  ], TaskAttempt 1 failed, info=[Error: Error while running task ( failure ) : attempt_1590738759875_100602_2_00_000007_1:java.lang.RuntimeException: java.lang.RuntimeException: Hive Runtime Error while closing operators
>  at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:296)
>  at org.apache.hadoop.hive.ql.exec.tez.MergeFileTezProcessor.run(MergeFileTezProcessor.java:42)
>  at org.apache.tez.runtime.LogicalIOProcessorRuntimeTask.run(LogicalIOProcessorRuntimeTask.java:374)
>  at org.apache.tez.runtime.task.TaskRunner2Callable$1.run(TaskRunner2Callable.java:73)
>  at org.apache.tez.runtime.task.TaskRunner2Callable$1.run(TaskRunner2Callable.java:61)
>  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:1730)
>  at org.apache.tez.runtime.task.TaskRunner2Callable.callInternal(TaskRunner2Callable.java:61)
>  at org.apache.tez.runtime.task.TaskRunner2Callable.callInternal(TaskRunner2Callable.java:37)
>  at org.apache.tez.common.CallableWithNdc.call(CallableWithNdc.java:36)
>  at com.google.common.util.concurrent.TrustedListenableFutureTask$TrustedFutureInterruptibleTask.runInterruptibly(TrustedListenableFutureTask.java:125)
>  at com.google.common.util.concurrent.InterruptibleTask.run(InterruptibleTask.java:69)
>  at com.google.common.util.concurrent.TrustedListenableFutureTask.run(TrustedListenableFutureTask.java:78)
>  at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>  at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>  at java.lang.Thread.run(Thread.java:748)
>  Caused by: java.lang.RuntimeException: Hive Runtime Error while closing operators
>  at org.apache.hadoop.hive.ql.exec.tez.MergeFileRecordProcessor.close(MergeFileRecordProcessor.java:188)
>  at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:284)
>  ... 16 more
>  Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: Failed to close AbstractFileMergeOperator
>  at org.apache.hadoop.hive.ql.exec.AbstractFileMergeOperator.closeOp(AbstractFileMergeOperator.java:315)
>  at org.apache.hadoop.hive.ql.exec.OrcFileMergeOperator.closeOp(OrcFileMergeOperator.java:265)
>  at org.apache.hadoop.hive.ql.exec.Operator.close(Operator.java:732)
>  at org.apache.hadoop.hive.ql.exec.tez.MergeFileRecordProcessor.close(MergeFileRecordProcessor.java:180)
>  ... 17 more
>  Caused by: java.io.IOException: Unable to rename xxx/warehouse/tablespace/hive/temp.db/.hive-staging_hive_2020-06-09_03-33-29_060_8741275828320666826-9431/_task_tmp.-ext-10002/_tmp.000009_0 to xxx/warehouse/tablespace/hive/temp.db/.hive-staging_hive_2020-06-09_03-33-29_060_8741275828320666826-9431/_tmp.-ext-10002/000009_0
>  at org.apache.hadoop.hive.ql.exec.AbstractFileMergeOperator.closeOp(AbstractFileMergeOperator.java:254)
>  ... 20 more
>  ], TaskAttempt 2 failed, info=[Error: Error while running task ( failure ) : attempt_1590738759875_100602_2_00_000007_2:java.lang.RuntimeException: java.lang.RuntimeException: Hive Runtime Error while closing operators
>  at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:296)
>  at org.apache.hadoop.hive.ql.exec.tez.MergeFileTezProcessor.run(MergeFileTezProcessor.java:42)
>  at org.apache.tez.runtime.LogicalIOProcessorRuntimeTask.run(LogicalIOProcessorRuntimeTask.java:374)
>  at org.apache.tez.runtime.task.TaskRunner2Callable$1.run(TaskRunner2Callable.java:73)
>  at org.apache.tez.runtime.task.TaskRunner2Callable$1.run(TaskRunner2Callable.java:61)
>  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:1730)
>  at org.apache.tez.runtime.task.TaskRunner2Callable.callInternal(TaskRunner2Callable.java:61)
>  at org.apache.tez.runtime.task.TaskRunner2Callable.callInternal(TaskRunner2Callable.java:37)
>  at org.apache.tez.common.CallableWithNdc.call(CallableWithNdc.java:36)
>  at com.google.common.util.concurrent.TrustedListenableFutureTask$TrustedFutureInterruptibleTask.runInterruptibly(TrustedListenableFutureTask.java:125)
>  at com.google.common.util.concurrent.InterruptibleTask.run(InterruptibleTask.java:69)
>  at com.google.common.util.concurrent.TrustedListenableFutureTask.run(TrustedListenableFutureTask.java:78)
>  at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>  at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>  at java.lang.Thread.run(Thread.java:748)
>  Caused by: java.lang.RuntimeException: Hive Runtime Error while closing operators
>  at org.apache.hadoop.hive.ql.exec.tez.MergeFileRecordProcessor.close(MergeFileRecordProcessor.java:188)
>  at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:284)
>  ... 16 more
>  Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: Failed to close AbstractFileMergeOperator
>  at org.apache.hadoop.hive.ql.exec.AbstractFileMergeOperator.closeOp(AbstractFileMergeOperator.java:315)
>  at org.apache.hadoop.hive.ql.exec.OrcFileMergeOperator.closeOp(OrcFileMergeOperator.java:265)
>  at org.apache.hadoop.hive.ql.exec.Operator.close(Operator.java:732)
>  at org.apache.hadoop.hive.ql.exec.tez.MergeFileRecordProcessor.close(MergeFileRecordProcessor.java:180)
>  ... 17 more
>  Caused by: java.io.IOException: Unable to rename xxx/warehouse/tablespace/hive/temp.db/.hive-staging_hive_2020-06-09_03-33-29_060_8741275828320666826-9431/_task_tmp.-ext-10002/_tmp.000004_1 to xxx/warehouse/tablespace/hive/temp.db/.hive-staging_hive_2020-06-09_03-33-29_060_8741275828320666826-9431/_tmp.-ext-10002/000004_1
>  at org.apache.hadoop.hive.ql.exec.AbstractFileMergeOperator.closeOp(AbstractFileMergeOperator.java:254)
>  ... 20 more
>  ], TaskAttempt 3 failed, info=[Error: Error while running task ( failure ) : attempt_1590738759875_100602_2_00_000007_3:java.lang.RuntimeException: java.lang.RuntimeException: Hive Runtime Error while closing operators
>  at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:296)
>  at org.apache.hadoop.hive.ql.exec.tez.MergeFileTezProcessor.run(MergeFileTezProcessor.java:42)
>  at org.apache.tez.runtime.LogicalIOProcessorRuntimeTask.run(LogicalIOProcessorRuntimeTask.java:374)
>  at org.apache.tez.runtime.task.TaskRunner2Callable$1.run(TaskRunner2Callable.java:73)
>  at org.apache.tez.runtime.task.TaskRunner2Callable$1.run(TaskRunner2Callable.java:61)
>  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:1730)
>  at org.apache.tez.runtime.task.TaskRunner2Callable.callInternal(TaskRunner2Callable.java:61)
>  at org.apache.tez.runtime.task.TaskRunner2Callable.callInternal(TaskRunner2Callable.java:37)
>  at org.apache.tez.common.CallableWithNdc.call(CallableWithNdc.java:36)
>  at com.google.common.util.concurrent.TrustedListenableFutureTask$TrustedFutureInterruptibleTask.runInterruptibly(TrustedListenableFutureTask.java:125)
>  at com.google.common.util.concurrent.InterruptibleTask.run(InterruptibleTask.java:69)
>  at com.google.common.util.concurrent.TrustedListenableFutureTask.run(TrustedListenableFutureTask.java:78)
>  at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>  at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>  at java.lang.Thread.run(Thread.java:748)
>  Caused by: java.lang.RuntimeException: Hive Runtime Error while closing operators
>  at org.apache.hadoop.hive.ql.exec.tez.MergeFileRecordProcessor.close(MergeFileRecordProcessor.java:188)
>  at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:284)
>  ... 16 more
>  Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: Failed to close AbstractFileMergeOperator
>  at org.apache.hadoop.hive.ql.exec.AbstractFileMergeOperator.closeOp(AbstractFileMergeOperator.java:315)
>  at org.apache.hadoop.hive.ql.exec.OrcFileMergeOperator.closeOp(OrcFileMergeOperator.java:265)
>  at org.apache.hadoop.hive.ql.exec.Operator.close(Operator.java:732)
>  at org.apache.hadoop.hive.ql.exec.tez.MergeFileRecordProcessor.close(MergeFileRecordProcessor.java:180)
>  ... 17 more
>  Caused by: java.io.IOException: Unable to rename xxx/warehouse/tablespace/hive/temp.db/.hive-staging_hive_2020-06-09_03-33-29_060_8741275828320666826-9431/_task_tmp.-ext-10002/_tmp.000006_2 to xxx/warehouse/tablespace/hive/temp.db/.hive-staging_hive_2020-06-09_03-33-29_060_8741275828320666826-9431/_tmp.-ext-10002/000006_2
>  at org.apache.hadoop.hive.ql.exec.AbstractFileMergeOperator.closeOp(AbstractFileMergeOperator.java:254)
>  ... 20 more
>  ]], Vertex did not succeed due to OWN_TASK_FAILURE, failedTasks:1 killedTasks:0, Vertex vertex_1590738759875_100602_2_00 [File Merge] killed/failed due to:OWN_TASK_FAILURE]
>  ERROR : DAG did not succeed due to VERTEX_FAILURE. failedVertices:1 killedVertices:0
>  INFO : org.apache.tez.common.counters.DAGCounter:
>  INFO : NUM_FAILED_TASKS: 7
>  INFO : NUM_SUCCEEDED_TASKS: 9
>  INFO : TOTAL_LAUNCHED_TASKS: 16
>  INFO : OTHER_LOCAL_TASKS: 1
>  INFO : RACK_LOCAL_TASKS: 8
>  INFO : AM_CPU_MILLISECONDS: 21600
>  INFO : AM_GC_TIME_MILLIS: 40
>  INFO : File System Counters:
>  INFO : FILE_BYTES_READ: 19368
>  INFO : OBS_BYTES_READ: 4338648131
>  INFO : OBS_BYTES_WRITTEN: 4377004106
>  INFO : OBS_READ_OPS: 4503
>  INFO : OBS_WRITE_OPS: 59
>  INFO : org.apache.tez.common.counters.TaskCounter:
>  INFO : GC_TIME_MILLIS: 1334
>  INFO : TASK_DURATION_MILLIS: 1030360
>  INFO : CPU_MILLISECONDS: 162060
>  INFO : PHYSICAL_MEMORY_BYTES: 61851303936
>  INFO : VIRTUAL_MEMORY_BYTES: 82899070976
>  INFO : COMMITTED_HEAP_BYTES: 61851303936
>  INFO : INPUT_RECORDS_PROCESSED: 897
>  INFO : INPUT_SPLIT_LENGTH_BYTES: 4310433482
>  INFO : OUTPUT_RECORDS: 0
>  INFO : HIVE:
>  INFO : RECORDS_OUT_INTERMEDIATE_File_Merge: 0
>  INFO : RECORDS_OUT_OPERATOR_OFM_57: 0
>  INFO : INPUT_RECORDS_PROCESSED: 897
>  INFO : INPUT_SPLIT_LENGTH_BYTES: 4310433482
>  INFO : TaskCounter_File_Merge_OUTPUT_out_File_Merge:
>  INFO : OUTPUT_RECORDS: 0
>  ERROR : FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.tez.TezTask. Vertex failed, vertexName=File Merge, vertexId=vertex_1590738759875_100602_2_00, diagnostics=[Task failed, taskId=task_1590738759875_100602_2_00_000007, diagnostics=[TaskAttempt 0 failed, info=[Error: Error while running task ( failure ) : attempt_1590738759875_100602_2_00_000007_0:java.lang.RuntimeException: java.lang.RuntimeException: Hive Runtime Error while closing operators
>  at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:296)
>  at org.apache.hadoop.hive.ql.exec.tez.MergeFileTezProcessor.run(MergeFileTezProcessor.java:42)
>  at org.apache.tez.runtime.LogicalIOProcessorRuntimeTask.run(LogicalIOProcessorRuntimeTask.java:374)
>  at org.apache.tez.runtime.task.TaskRunner2Callable$1.run(TaskRunner2Callable.java:73)
>  at org.apache.tez.runtime.task.TaskRunner2Callable$1.run(TaskRunner2Callable.java:61)
>  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:1730)
>  at org.apache.tez.runtime.task.TaskRunner2Callable.callInternal(TaskRunner2Callable.java:61)
>  at org.apache.tez.runtime.task.TaskRunner2Callable.callInternal(TaskRunner2Callable.java:37)
>  at org.apache.tez.common.CallableWithNdc.call(CallableWithNdc.java:36)
>  at com.google.common.util.concurrent.TrustedListenableFutureTask$TrustedFutureInterruptibleTask.runInterruptibly(TrustedListenableFutureTask.java:125)
>  at com.google.common.util.concurrent.InterruptibleTask.run(InterruptibleTask.java:69)
>  at com.google.common.util.concurrent.TrustedListenableFutureTask.run(TrustedListenableFutureTask.java:78)
>  at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>  at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>  at java.lang.Thread.run(Thread.java:748)
>  Caused by: java.lang.RuntimeException: Hive Runtime Error while closing operators
>  at org.apache.hadoop.hive.ql.exec.tez.MergeFileRecordProcessor.close(MergeFileRecordProcessor.java:188)
>  at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:284)
>  ... 16 more
>  Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: Failed to close AbstractFileMergeOperator
>  at org.apache.hadoop.hive.ql.exec.AbstractFileMergeOperator.closeOp(AbstractFileMergeOperator.java:315)
>  at org.apache.hadoop.hive.ql.exec.OrcFileMergeOperator.closeOp(OrcFileMergeOperator.java:265)
>  at org.apache.hadoop.hive.ql.exec.Operator.close(Operator.java:732)
>  at org.apache.hadoop.hive.ql.exec.tez.MergeFileRecordProcessor.close(MergeFileRecordProcessor.java:180)
>  ... 17 more
>  Caused by: java.io.IOException: Unable to rename xxx/warehouse/tablespace/hive/temp.db/.hive-staging_hive_2020-06-09_03-33-29_060_8741275828320666826-9431/_task_tmp.-ext-10002/_tmp.000002_0 to xxx/warehouse/tablespace/hive/temp.db/.hive-staging_hive_2020-06-09_03-33-29_060_8741275828320666826-9431/_tmp.-ext-10002/000002_0
>  at org.apache.hadoop.hive.ql.exec.AbstractFileMergeOperator.closeOp(AbstractFileMergeOperator.java:254)
>  ... 20 more
>  ], TaskAttempt 1 failed, info=[Error: Error while running task ( failure ) : attempt_1590738759875_100602_2_00_000007_1:java.lang.RuntimeException: java.lang.RuntimeException: Hive Runtime Error while closing operators
>  at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:296)
>  at org.apache.hadoop.hive.ql.exec.tez.MergeFileTezProcessor.run(MergeFileTezProcessor.java:42)
>  at org.apache.tez.runtime.LogicalIOProcessorRuntimeTask.run(LogicalIOProcessorRuntimeTask.java:374)
>  at org.apache.tez.runtime.task.TaskRunner2Callable$1.run(TaskRunner2Callable.java:73)
>  at org.apache.tez.runtime.task.TaskRunner2Callable$1.run(TaskRunner2Callable.java:61)
>  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:1730)
>  at org.apache.tez.runtime.task.TaskRunner2Callable.callInternal(TaskRunner2Callable.java:61)
>  at org.apache.tez.runtime.task.TaskRunner2Callable.callInternal(TaskRunner2Callable.java:37)
>  at org.apache.tez.common.CallableWithNdc.call(CallableWithNdc.java:36)
>  at com.google.common.util.concurrent.TrustedListenableFutureTask$TrustedFutureInterruptibleTask.runInterruptibly(TrustedListenableFutureTask.java:125)
>  at com.google.common.util.concurrent.InterruptibleTask.run(InterruptibleTask.java:69)
>  at com.google.common.util.concurrent.TrustedListenableFutureTask.run(TrustedListenableFutureTask.java:78)
>  at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>  at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>  at java.lang.Thread.run(Thread.java:748)
>  Caused by: java.lang.RuntimeException: Hive Runtime Error while closing operators
>  at org.apache.hadoop.hive.ql.exec.tez.MergeFileRecordProcessor.close(MergeFileRecordProcessor.java:188)
>  at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:284)
>  ... 16 more
>  Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: Failed to close AbstractFileMergeOperator
>  at org.apache.hadoop.hive.ql.exec.AbstractFileMergeOperator.closeOp(AbstractFileMergeOperator.java:315)
>  at org.apache.hadoop.hive.ql.exec.OrcFileMergeOperator.closeOp(OrcFileMergeOperator.java:265)
>  at org.apache.hadoop.hive.ql.exec.Operator.close(Operator.java:732)
>  at org.apache.hadoop.hive.ql.exec.tez.MergeFileRecordProcessor.close(MergeFileRecordProcessor.java:180)
>  ... 17 more
>  Caused by: java.io.IOException: Unable to rename xxx/warehouse/tablespace/hive/temp.db/.hive-staging_hive_2020-06-09_03-33-29_060_8741275828320666826-9431/_task_tmp.-ext-10002/_tmp.000009_0 to xxx/warehouse/tablespace/hive/temp.db/.hive-staging_hive_2020-06-09_03-33-29_060_8741275828320666826-9431/_tmp.-ext-10002/000009_0
>  at org.apache.hadoop.hive.ql.exec.AbstractFileMergeOperator.closeOp(AbstractFileMergeOperator.java:254)
>  ... 20 more
>  ], TaskAttempt 2 failed, info=[Error: Error while running task ( failure ) : attempt_1590738759875_100602_2_00_000007_2:java.lang.RuntimeException: java.lang.RuntimeException: Hive Runtime Error while closing operators
>  at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:296)
>  at org.apache.hadoop.hive.ql.exec.tez.MergeFileTezProcessor.run(MergeFileTezProcessor.java:42)
>  at org.apache.tez.runtime.LogicalIOProcessorRuntimeTask.run(LogicalIOProcessorRuntimeTask.java:374)
>  at org.apache.tez.runtime.task.TaskRunner2Callable$1.run(TaskRunner2Callable.java:73)
>  at org.apache.tez.runtime.task.TaskRunner2Callable$1.run(TaskRunner2Callable.java:61)
>  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:1730)
>  at org.apache.tez.runtime.task.TaskRunner2Callable.callInternal(TaskRunner2Callable.java:61)
>  at org.apache.tez.runtime.task.TaskRunner2Callable.callInternal(TaskRunner2Callable.java:37)
>  at org.apache.tez.common.CallableWithNdc.call(CallableWithNdc.java:36)
>  at com.google.common.util.concurrent.TrustedListenableFutureTask$TrustedFutureInterruptibleTask.runInterruptibly(TrustedListenableFutureTask.java:125)
>  at com.google.common.util.concurrent.InterruptibleTask.run(InterruptibleTask.java:69)
>  at com.google.common.util.concurrent.TrustedListenableFutureTask.run(TrustedListenableFutureTask.java:78)
>  at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>  at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>  at java.lang.Thread.run(Thread.java:748)
>  Caused by: java.lang.RuntimeException: Hive Runtime Error while closing operators
>  at org.apache.hadoop.hive.ql.exec.tez.MergeFileRecordProcessor.close(MergeFileRecordProcessor.java:188)
>  at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:284)
>  ... 16 more
>  Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: Failed to close AbstractFileMergeOperator
>  at org.apache.hadoop.hive.ql.exec.AbstractFileMergeOperator.closeOp(AbstractFileMergeOperator.java:315)
>  at org.apache.hadoop.hive.ql.exec.OrcFileMergeOperator.closeOp(OrcFileMergeOperator.java:265)
>  at org.apache.hadoop.hive.ql.exec.Operator.close(Operator.java:732)
>  at org.apache.hadoop.hive.ql.exec.tez.MergeFileRecordProcessor.close(MergeFileRecordProcessor.java:180)
>  ... 17 more
>  Caused by: java.io.IOException: Unable to rename xxx/warehouse/tablespace/hive/temp.db/.hive-staging_hive_2020-06-09_03-33-29_060_8741275828320666826-9431/_task_tmp.-ext-10002/_tmp.000004_1 to xxxl/warehouse/tablespace/hive/temp.db/.hive-staging_hive_2020-06-09_03-33-29_060_8741275828320666826-9431/_tmp.-ext-10002/000004_1
>  at org.apache.hadoop.hive.ql.exec.AbstractFileMergeOperator.closeOp(AbstractFileMergeOperator.java:254)
>  ... 20 more
>  ], TaskAttempt 3 failed, info=[Error: Error while running task ( failure ) : attempt_1590738759875_100602_2_00_000007_3:java.lang.RuntimeException: java.lang.RuntimeException: Hive Runtime Error while closing operators
>  at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:296)
>  at org.apache.hadoop.hive.ql.exec.tez.MergeFileTezProcessor.run(MergeFileTezProcessor.java:42)
>  at org.apache.tez.runtime.LogicalIOProcessorRuntimeTask.run(LogicalIOProcessorRuntimeTask.java:374)
>  at org.apache.tez.runtime.task.TaskRunner2Callable$1.run(TaskRunner2Callable.java:73)
>  at org.apache.tez.runtime.task.TaskRunner2Callable$1.run(TaskRunner2Callable.java:61)
>  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:1730)
>  at org.apache.tez.runtime.task.TaskRunner2Callable.callInternal(TaskRunner2Callable.java:61)
>  at org.apache.tez.runtime.task.TaskRunner2Callable.callInternal(TaskRunner2Callable.java:37)
>  at org.apache.tez.common.CallableWithNdc.call(CallableWithNdc.java:36)
>  at com.google.common.util.concurrent.TrustedListenableFutureTask$TrustedFutureInterruptibleTask.runInterruptibly(TrustedListenableFutureTask.java:125)
>  at com.google.common.util.concurrent.InterruptibleTask.run(InterruptibleTask.java:69)
>  at com.google.common.util.concurrent.TrustedListenableFutureTask.run(TrustedListenableFutureTask.java:78)
>  at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>  at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>  at java.lang.Thread.run(Thread.java:748)
>  Caused by: java.lang.RuntimeException: Hive Runtime Error while closing operators
>  at org.apache.hadoop.hive.ql.exec.tez.MergeFileRecordProcessor.close(MergeFileRecordProcessor.java:188)
>  at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:284)
>  ... 16 more
>  Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: Failed to close AbstractFileMergeOperator
>  at org.apache.hadoop.hive.ql.exec.AbstractFileMergeOperator.closeOp(AbstractFileMergeOperator.java:315)
>  at org.apache.hadoop.hive.ql.exec.OrcFileMergeOperator.closeOp(OrcFileMergeOperator.java:265)
>  at org.apache.hadoop.hive.ql.exec.Operator.close(Operator.java:732)
>  at org.apache.hadoop.hive.ql.exec.tez.MergeFileRecordProcessor.close(MergeFileRecordProcessor.java:180)
>  ... 17 more
>  Caused by: java.io.IOException: Unable to rename xxx/warehouse/tablespace/hive/temp.db/.hive-staging_hive_2020-06-09_03-33-29_060_8741275828320666826-9431/_task_tmp.-ext-10002/_tmp.000006_2 to xxx/warehouse/tablespace/hive/temp.db/.hive-staging_hive_2020-06-09_03-33-29_060_8741275828320666826-9431/_tmp.-ext-10002/000006_2
>  at org.apache.hadoop.hive.ql.exec.AbstractFileMergeOperator.closeOp(AbstractFileMergeOperator.java:254)
>  ... 20 more
>  ]], Vertex did not succeed due to OWN_TASK_FAILURE, failedTasks:1 killedTasks:0, Vertex vertex_1590738759875_100602_2_00 [File Merge] killed/failed due to:OWN_TASK_FAILURE]DAG did not succeed due to VERTEX_FAILURE. failedVertices:1 killedVertices:0
>  INFO : Completed executing command(queryId=hive_20200609033329_3fbf0a38-e5b0-4e3a-ae8b-ef95f400b50f); Time taken: 4243.011 seconds
>  
> INFO : Concurrency mode is disabled, not creating a lock manager
>  Error: Error while processing statement: FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.tez.TezTask. Vertex failed, vertexName=File Merge, vertexId=vertex_1590738759875_100602_2_00, diagnostics=[Task failed, taskId=task_1590738759875_100602_2_00_000007, diagnostics=[TaskAttempt 0 failed, info=[Error: Error while running task ( failure ) : attempt_1590738759875_100602_2_00_000007_0:java.lang.RuntimeException: java.lang.RuntimeException: Hive Runtime Error while closing operators
>  at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:296)
>  at org.apache.hadoop.hive.ql.exec.tez.MergeFileTezProcessor.run(MergeFileTezProcessor.java:42)
>  at org.apache.tez.runtime.LogicalIOProcessorRuntimeTask.run(LogicalIOProcessorRuntimeTask.java:374)
>  at org.apache.tez.runtime.task.TaskRunner2Callable$1.run(TaskRunner2Callable.java:73)
>  at org.apache.tez.runtime.task.TaskRunner2Callable$1.run(TaskRunner2Callable.java:61)
>  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:1730)
>  at org.apache.tez.runtime.task.TaskRunner2Callable.callInternal(TaskRunner2Callable.java:61)
>  at org.apache.tez.runtime.task.TaskRunner2Callable.callInternal(TaskRunner2Callable.java:37)
>  at org.apache.tez.common.CallableWithNdc.call(CallableWithNdc.java:36)
>  at com.google.common.util.concurrent.TrustedListenableFutureTask$TrustedFutureInterruptibleTask.runInterruptibly(TrustedListenableFutureTask.java:125)
>  at com.google.common.util.concurrent.InterruptibleTask.run(InterruptibleTask.java:69)
>  at com.google.common.util.concurrent.TrustedListenableFutureTask.run(TrustedListenableFutureTask.java:78)
>  at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>  at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>  at java.lang.Thread.run(Thread.java:748)
>  Caused by: java.lang.RuntimeException: Hive Runtime Error while closing operators
>  at org.apache.hadoop.hive.ql.exec.tez.MergeFileRecordProcessor.close(MergeFileRecordProcessor.java:188)
>  at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:284)
>  ... 16 more
>  Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: Failed to close AbstractFileMergeOperator
>  at org.apache.hadoop.hive.ql.exec.AbstractFileMergeOperator.closeOp(AbstractFileMergeOperator.java:315)
>  at org.apache.hadoop.hive.ql.exec.OrcFileMergeOperator.closeOp(OrcFileMergeOperator.java:265)
>  at org.apache.hadoop.hive.ql.exec.Operator.close(Operator.java:732)
>  at org.apache.hadoop.hive.ql.exec.tez.MergeFileRecordProcessor.close(MergeFileRecordProcessor.java:180)
>  ... 17 more
>  Caused by: java.io.IOException: Unable to rename xxx/warehouse/tablespace/hive/temp.db/.hive-staging_hive_2020-06-09_03-33-29_060_8741275828320666826-9431/_task_tmp.-ext-10002/_tmp.000002_0 to xxx/warehouse/tablespace/hive/temp.db/.hive-staging_hive_2020-06-09_03-33-29_060_8741275828320666826-9431/_tmp.-ext-10002/000002_0
>  at org.apache.hadoop.hive.ql.exec.AbstractFileMergeOperator.closeOp(AbstractFileMergeOperator.java:254)
>  ... 20 more
>  ], TaskAttempt 1 failed, info=[Error: Error while running task ( failure ) : attempt_1590738759875_100602_2_00_000007_1:java.lang.RuntimeException: java.lang.RuntimeException: Hive Runtime Error while closing operators
>  at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:296)
>  at org.apache.hadoop.hive.ql.exec.tez.MergeFileTezProcessor.run(MergeFileTezProcessor.java:42)
>  at org.apache.tez.runtime.LogicalIOProcessorRuntimeTask.run(LogicalIOProcessorRuntimeTask.java:374)
>  at org.apache.tez.runtime.task.TaskRunner2Callable$1.run(TaskRunner2Callable.java:73)
>  at org.apache.tez.runtime.task.TaskRunner2Callable$1.run(TaskRunner2Callable.java:61)
>  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:1730)
>  at org.apache.tez.runtime.task.TaskRunner2Callable.callInternal(TaskRunner2Callable.java:61)
>  at org.apache.tez.runtime.task.TaskRunner2Callable.callInternal(TaskRunner2Callable.java:37)
>  at org.apache.tez.common.CallableWithNdc.call(CallableWithNdc.java:36)
>  at com.google.common.util.concurrent.TrustedListenableFutureTask$TrustedFutureInterruptibleTask.runInterruptibly(TrustedListenableFutureTask.java:125)
>  at com.google.common.util.concurrent.InterruptibleTask.run(InterruptibleTask.java:69)
>  at com.google.common.util.concurrent.TrustedListenableFutureTask.run(TrustedListenableFutureTask.java:78)
>  at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>  at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>  at java.lang.Thread.run(Thread.java:748)
>  Caused by: java.lang.RuntimeException: Hive Runtime Error while closing operators
>  at org.apache.hadoop.hive.ql.exec.tez.MergeFileRecordProcessor.close(MergeFileRecordProcessor.java:188)
>  at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:284)
>  ... 16 more
>  Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: Failed to close AbstractFileMergeOperator
>  at org.apache.hadoop.hive.ql.exec.AbstractFileMergeOperator.closeOp(AbstractFileMergeOperator.java:315)
>  at org.apache.hadoop.hive.ql.exec.OrcFileMergeOperator.closeOp(OrcFileMergeOperator.java:265)
>  at org.apache.hadoop.hive.ql.exec.Operator.close(Operator.java:732)
>  at org.apache.hadoop.hive.ql.exec.tez.MergeFileRecordProcessor.close(MergeFileRecordProcessor.java:180)
>  ... 17 more
>  Caused by: java.io.IOException: Unable to rename xxx/warehouse/tablespace/hive/temp.db/.hive-staging_hive_2020-06-09_03-33-29_060_8741275828320666826-9431/_task_tmp.-ext-10002/_tmp.000009_0 to xxx/warehouse/tablespace/hive/temp.db/.hive-staging_hive_2020-06-09_03-33-29_060_8741275828320666826-9431/_tmp.-ext-10002/000009_0
>  at org.apache.hadoop.hive.ql.exec.AbstractFileMergeOperator.closeOp(AbstractFileMergeOperator.java:254)
>  ... 20 more
>  ], TaskAttempt 2 failed, info=[Error: Error while running task ( failure ) : attempt_1590738759875_100602_2_00_000007_2:java.lang.RuntimeException: java.lang.RuntimeException: Hive Runtime Error while closing operators
>  at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:296)
>  at org.apache.hadoop.hive.ql.exec.tez.MergeFileTezProcessor.run(MergeFileTezProcessor.java:42)
>  at org.apache.tez.runtime.LogicalIOProcessorRuntimeTask.run(LogicalIOProcessorRuntimeTask.java:374)
>  at org.apache.tez.runtime.task.TaskRunner2Callable$1.run(TaskRunner2Callable.java:73)
>  at org.apache.tez.runtime.task.TaskRunner2Callable$1.run(TaskRunner2Callable.java:61)
>  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:1730)
>  at org.apache.tez.runtime.task.TaskRunner2Callable.callInternal(TaskRunner2Callable.java:61)
>  at org.apache.tez.runtime.task.TaskRunner2Callable.callInternal(TaskRunner2Callable.java:37)
>  at org.apache.tez.common.CallableWithNdc.call(CallableWithNdc.java:36)
>  at com.google.common.util.concurrent.TrustedListenableFutureTask$TrustedFutureInterruptibleTask.runInterruptibly(TrustedListenableFutureTask.java:125)
>  at com.google.common.util.concurrent.InterruptibleTask.run(InterruptibleTask.java:69)
>  at com.google.common.util.concurrent.TrustedListenableFutureTask.run(TrustedListenableFutureTask.java:78)
>  at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>  at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>  at java.lang.Thread.run(Thread.java:748)
>  Caused by: java.lang.RuntimeException: Hive Runtime Error while closing operators
>  at org.apache.hadoop.hive.ql.exec.tez.MergeFileRecordProcessor.close(MergeFileRecordProcessor.java:188)
>  at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:284)
>  ... 16 more
>  Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: Failed to close AbstractFileMergeOperator
>  at org.apache.hadoop.hive.ql.exec.AbstractFileMergeOperator.closeOp(AbstractFileMergeOperator.java:315)
>  at org.apache.hadoop.hive.ql.exec.OrcFileMergeOperator.closeOp(OrcFileMergeOperator.java:265)
>  at org.apache.hadoop.hive.ql.exec.Operator.close(Operator.java:732)
>  at org.apache.hadoop.hive.ql.exec.tez.MergeFileRecordProcessor.close(MergeFileRecordProcessor.java:180)
>  ... 17 more
>  Caused by: java.io.IOException: Unable to rename xxx/warehouse/tablespace/hive/temp.db/.hive-staging_hive_2020-06-09_03-33-29_060_8741275828320666826-9431/_task_tmp.-ext-10002/_tmp.000004_1 to xxx/warehouse/tablespace/hive/temp.db/.hive-staging_hive_2020-06-09_03-33-29_060_8741275828320666826-9431/_tmp.-ext-10002/000004_1
>  at org.apache.hadoop.hive.ql.exec.AbstractFileMergeOperator.closeOp(AbstractFileMergeOperator.java:254)
>  ... 20 more
>  ], TaskAttempt 3 failed, info=[Error: Error while running task ( failure ) : attempt_1590738759875_100602_2_00_000007_3:java.lang.RuntimeException: java.lang.RuntimeException: Hive Runtime Error while closing operators
>  at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:296)
>  at org.apache.hadoop.hive.ql.exec.tez.MergeFileTezProcessor.run(MergeFileTezProcessor.java:42)
>  at org.apache.tez.runtime.LogicalIOProcessorRuntimeTask.run(LogicalIOProcessorRuntimeTask.java:374)
>  at org.apache.tez.runtime.task.TaskRunner2Callable$1.run(TaskRunner2Callable.java:73)
>  at org.apache.tez.runtime.task.TaskRunner2Callable$1.run(TaskRunner2Callable.java:61)
>  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:1730)
>  at org.apache.tez.runtime.task.TaskRunner2Callable.callInternal(TaskRunner2Callable.java:61)
>  at org.apache.tez.runtime.task.TaskRunner2Callable.callInternal(TaskRunner2Callable.java:37)
>  at org.apache.tez.common.CallableWithNdc.call(CallableWithNdc.java:36)
>  at com.google.common.util.concurrent.TrustedListenableFutureTask$TrustedFutureInterruptibleTask.runInterruptibly(TrustedListenableFutureTask.java:125)
>  at com.google.common.util.concurrent.InterruptibleTask.run(InterruptibleTask.java:69)
>  at com.google.common.util.concurrent.TrustedListenableFutureTask.run(TrustedListenableFutureTask.java:78)
>  at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>  at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>  at java.lang.Thread.run(Thread.java:748)
>  Caused by: java.lang.RuntimeException: Hive Runtime Error while closing operators
>  at org.apache.hadoop.hive.ql.exec.tez.MergeFileRecordProcessor.close(MergeFileRecordProcessor.java:188)
>  at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:284)
>  ... 16 more
>  Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: Failed to close AbstractFileMergeOperator
>  at org.apache.hadoop.hive.ql.exec.AbstractFileMergeOperator.closeOp(AbstractFileMergeOperator.java:315)
>  at org.apache.hadoop.hive.ql.exec.OrcFileMergeOperator.closeOp(OrcFileMergeOperator.java:265)
>  at org.apache.hadoop.hive.ql.exec.Operator.close(Operator.java:732)
>  at org.apache.hadoop.hive.ql.exec.tez.MergeFileRecordProcessor.close(MergeFileRecordProcessor.java:180)
>  ... 17 more
>  Caused by: java.io.IOException: Unable to rename xxx/warehouse/tablespace/hive/temp.db/.hive-staging_hive_2020-06-09_03-33-29_060_8741275828320666826-9431/_task_tmp.-ext-10002/_tmp.000006_2 to xxxl/warehouse/tablespace/hive/temp.db/.hive-staging_hive_2020-06-09_03-33-29_060_8741275828320666826-9431/_tmp.-ext-10002/000006_2
>  at org.apache.hadoop.hive.ql.exec.AbstractFileMergeOperator.closeOp(AbstractFileMergeOperator.java:254)
>  ... 20 more
>  ]], Vertex did not succeed due to OWN_TASK_FAILURE, failedTasks:1 killedTasks:0, Vertex vertex_1590738759875_100602_2_00 [File Merge] killed/failed due to:OWN_TASK_FAILURE]DAG did not succeed due to VERTEX_FAILURE. failedVertices:1 killedVertices:0 (state=08S01,code=2)
>  Closing: 0: jdbc:hive2://xxxxx:10000/prod
>  
>  
>  
>  
>  
>  
>  



--
This message was sent by Atlassian Jira
(v8.3.4#803005)