You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hive.apache.org by "Sahil Takiar (JIRA)" <ji...@apache.org> on 2016/06/30 01:14:11 UTC

[jira] [Commented] (HIVE-14137) Hive on Spark throws FileAlreadyExistsException for jobs with multiple empty tables

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

Sahil Takiar commented on HIVE-14137:
-------------------------------------

The root cause is the {{Utilities.getInputPaths}} method. If it is ever invoked for two different aliases, both of which are empty tables, it will try to create the same temp file for the two tables. The first file creation will succeed, and then the second one will fail with a {{FileAlreadyExistsException}}.

The solution is to ensure that each alias used a unique file path, this is done by including the alias name in the path of the file to be created.

> Hive on Spark throws FileAlreadyExistsException for jobs with multiple empty tables
> -----------------------------------------------------------------------------------
>
>                 Key: HIVE-14137
>                 URL: https://issues.apache.org/jira/browse/HIVE-14137
>             Project: Hive
>          Issue Type: Bug
>            Reporter: Sahil Takiar
>            Assignee: Sahil Takiar
>         Attachments: HIVE-14137.patch
>
>
> The following queries:
> {code}
> -- Setup
> drop table if exists empty1;
> create table empty1 (col1 bigint) stored as parquet tblproperties ('parquet.compress'='snappy');
> drop table if exists empty2;
> create table empty2 (col1 bigint, col2 bigint) stored as parquet tblproperties ('parquet.compress'='snappy');
> drop table if exists empty3;
> create table empty3 (col1 bigint) stored as parquet tblproperties ('parquet.compress'='snappy');
> -- All empty HDFS directories.
> -- Fails with [08S01]: Error while processing statement: FAILED: Execution Error, return code 3 from org.apache.hadoop.hive.ql.exec.spark.SparkTask.
> select empty1.col1
> from empty1
> inner join empty2
> on empty2.col1 = empty1.col1
> inner join empty3
> on empty3.col1 = empty2.col2;
> -- Two empty HDFS directories.
> -- Create an empty file in HDFS.
> insert into empty1 select * from empty1 where false;
> -- Same query fails with [08S01]: Error while processing statement: FAILED: Execution Error, return code 3 from org.apache.hadoop.hive.ql.exec.spark.SparkTask.
> select empty1.col1
> from empty1
> inner join empty2
> on empty2.col1 = empty1.col1
> inner join empty3
> on empty3.col1 = empty2.col2;
> -- One empty HDFS directory.
> -- Create an empty file in HDFS.
> insert into empty2 select * from empty2 where false;
> -- Same query succeeds.
> select empty1.col1
> from empty1
> inner join empty2
> on empty2.col1 = empty1.col1
> inner join empty3
> on empty3.col1 = empty2.col2;
> {code}
> Will result in the following exception:
> {code}
> org.apache.hadoop.fs.FileAlreadyExistsException: /tmp/hive/hive/1f3837aa-9407-4780-92b1-42a66d205139/hive_2016-06-24_15-45-23_206_79177714958655528-2/-mr-10004/0/emptyFile for client 172.26.14.151 already exists
> 	at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.startFileInternal(FSNamesystem.java:2784)
> 	at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.startFileInt(FSNamesystem.java:2676)
> 	at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.startFile(FSNamesystem.java:2561)
> 	at org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.create(NameNodeRpcServer.java:593)
> 	at org.apache.hadoop.hdfs.server.namenode.AuthorizationProviderProxyClientProtocol.create(AuthorizationProviderProxyClientProtocol.java:111)
> 	at org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolServerSideTranslatorPB.create(ClientNamenodeProtocolServerSideTranslatorPB.java:393)
> 	at org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos$ClientNamenodeProtocol$2.callBlockingMethod(ClientNamenodeProtocolProtos.java)
> 	at org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:617)
> 	at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:1073)
> 	at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2086)
> 	at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2082)
> 	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:1693)
> 	at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2080)
> 	at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
> 	at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:57)
> 	at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
> 	at java.lang.reflect.Constructor.newInstance(Constructor.java:526)
> 	at org.apache.hadoop.ipc.RemoteException.instantiateException(RemoteException.java:106)
> 	at org.apache.hadoop.ipc.RemoteException.unwrapRemoteException(RemoteException.java:73)
> 	at org.apache.hadoop.hdfs.DFSOutputStream.newStreamForCreate(DFSOutputStream.java:1902)
> 	at org.apache.hadoop.hdfs.DFSClient.create(DFSClient.java:1738)
> 	at org.apache.hadoop.hdfs.DFSClient.create(DFSClient.java:1663)
> 	at org.apache.hadoop.hdfs.DistributedFileSystem$6.doCall(DistributedFileSystem.java:405)
> 	at org.apache.hadoop.hdfs.DistributedFileSystem$6.doCall(DistributedFileSystem.java:401)
> 	at org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
> 	at org.apache.hadoop.hdfs.DistributedFileSystem.create(DistributedFileSystem.java:401)
> 	at org.apache.hadoop.hdfs.DistributedFileSystem.create(DistributedFileSystem.java:344)
> 	at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:920)
> 	at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:901)
> 	at parquet.hadoop.ParquetFileWriter.<init>(ParquetFileWriter.java:220)
> 	at parquet.hadoop.ParquetOutputFormat.getRecordWriter(ParquetOutputFormat.java:311)
> 	at parquet.hadoop.ParquetOutputFormat.getRecordWriter(ParquetOutputFormat.java:287)
> 	at org.apache.hadoop.hive.ql.io.parquet.write.ParquetRecordWriterWrapper.<init>(ParquetRecordWriterWrapper.java:65)
> 	at org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat.getParquerRecordWriterWrapper(MapredParquetOutputFormat.java:125)
> 	at org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat.getHiveRecordWriter(MapredParquetOutputFormat.java:114)
> 	at org.apache.hadoop.hive.ql.exec.Utilities.createEmptyFile(Utilities.java:3435)
> 	at org.apache.hadoop.hive.ql.exec.Utilities.createDummyFileForEmptyPartition(Utilities.java:3468)
> 	at org.apache.hadoop.hive.ql.exec.Utilities.getInputPaths(Utilities.java:3392)
> 	at org.apache.hadoop.hive.ql.exec.spark.SparkPlanGenerator.cloneJobConf(SparkPlanGenerator.java:273)
> 	at org.apache.hadoop.hive.ql.exec.spark.SparkPlanGenerator.generate(SparkPlanGenerator.java:213)
> 	at org.apache.hadoop.hive.ql.exec.spark.SparkPlanGenerator.generate(SparkPlanGenerator.java:107)
> 	at org.apache.hadoop.hive.ql.exec.spark.RemoteHiveSparkClient$JobStatusJob.call(RemoteHiveSparkClient.java:325)
> 	at org.apache.hive.spark.client.RemoteDriver$JobWrapper.call(RemoteDriver.java:366)
> 	at org.apache.hive.spark.client.RemoteDriver$JobWrapper.call(RemoteDriver.java:335)
> 	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:745)
> Caused by: org.apache.hadoop.ipc.RemoteException(org.apache.hadoop.fs.FileAlreadyExistsException): /tmp/hive/hive/1f3837aa-9407-4780-92b1-42a66d205139/hive_2016-06-24_15-45-23_206_79177714958655528-2/-mr-10004/0/emptyFile for client 172.26.14.151 already exists
> 	at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.startFileInternal(FSNamesystem.java:2784)
> 	at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.startFileInt(FSNamesystem.java:2676)
> 	at org.apache.hadoop.hdfs.server.namenode.FSNamesystem.startFile(FSNamesystem.java:2561)
> 	at org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.create(NameNodeRpcServer.java:593)
> 	at org.apache.hadoop.hdfs.server.namenode.AuthorizationProviderProxyClientProtocol.create(AuthorizationProviderProxyClientProtocol.java:111)
> 	at org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolServerSideTranslatorPB.create(ClientNamenodeProtocolServerSideTranslatorPB.java:393)
> 	at org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos$ClientNamenodeProtocol$2.callBlockingMethod(ClientNamenodeProtocolProtos.java)
> 	at org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:617)
> 	at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:1073)
> 	at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2086)
> 	at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2082)
> 	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:1693)
> 	at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2080)
> 	at org.apache.hadoop.ipc.Client.call(Client.java:1471)
> 	at org.apache.hadoop.ipc.Client.call(Client.java:1408)
> 	at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:230)
> 	at com.sun.proxy.$Proxy14.create(Unknown Source)
> 	at org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.create(ClientNamenodeProtocolTranslatorPB.java:296)
> 	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
> 	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
> 	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> 	at java.lang.reflect.Method.invoke(Method.java:606)
> 	at org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:256)
> 	at org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:104)
> 	at com.sun.proxy.$Proxy15.create(Unknown Source)
> 	at org.apache.hadoop.hdfs.DFSOutputStream.newStreamForCreate(DFSOutputStream.java:1897)
> 	... 28 more
> {code}
> This bug is also happening in a subset of the TPC-DS queries.
> This exception does not happen with Hive-on-MR



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