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

Mithun Radhakrishnan reassigned HIVE-17803:
-------------------------------------------


> With Pig multi-query, 2 HCatStorers writing to the same table will trample 
> each other's outputs
> -----------------------------------------------------------------------------------------------
>
>                 Key: HIVE-17803
>                 URL: https://issues.apache.org/jira/browse/HIVE-17803
>             Project: Hive
>          Issue Type: Bug
>          Components: HCatalog
>    Affects Versions: 2.2.0, 3.0.0
>            Reporter: Mithun Radhakrishnan
>            Assignee: Chris Drome
>
> When Pig scripts use multi-query and {{HCatStorer}} with 
> dynamic-partitioning, and use more than one {{HCatStorer}} instance to write 
> to the same table, they might trample on each other's outputs. The failure 
> looks as follows:
> {noformat}
> Caused by: org.apache.hive.hcatalog.common.HCatException : 2006 : Error 
> adding partition to metastore. Cause : 
> org.apache.hadoop.ipc.RemoteException(org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException):
>  No lease on /projects/foo/bar/activity_date=2016022306/_placeholder (inode 
> 2878224200): File does not exist. [Lease.  Holder: 
> DFSClient_NONMAPREDUCE_-1281544466_4952, pendingcreates: 1]
>       at 
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.checkLease(FSNamesystem.java:3429)
>       at 
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.completeFileInternal(FSNamesystem.java:3517)
>       at 
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.completeFile(FSNamesystem.java:3484)
>       at 
> org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.complete(NameNodeRpcServer.java:791)
>       at 
> org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolServerSideTranslatorPB.complete(ClientNamenodeProtocolServerSideTranslatorPB.java:537)
>       at 
> org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos$ClientNamenodeProtocol$2.callBlockingMethod(ClientNamenodeProtocolProtos.java)
>       at 
> org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:608)
>       at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:969)
>       at org.apache.hadoop.ipc.Server.call(Server.java:2267)
>       at org.apache.hadoop.ipc.Server$RpcCall.run(Server.java:648)
>       at org.apache.hadoop.ipc.Server$RpcCall.run(Server.java:615)
>       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:1679)
>       at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2217)
>       at 
> org.apache.hive.hcatalog.mapreduce.FileOutputCommitterContainer.registerPartitions(FileOutputCommitterContainer.java:1022)
>       at 
> org.apache.hive.hcatalog.mapreduce.FileOutputCommitterContainer.commitJob(FileOutputCommitterContainer.java:269)
>       ... 20 more
> Caused by: 
> org.apache.hadoop.ipc.RemoteException(org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException):
>  No lease on /projects/foo/bar/activity_date=2016022306/_placeholder (inode 
> 2878224200): File does not exist. [Lease.  Holder: 
> DFSClient_NONMAPREDUCE_-1281544466_4952, pendingcreates: 1]
>       at 
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.checkLease(FSNamesystem.java:3429)
>       at 
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.completeFileInternal(FSNamesystem.java:3517)
>       at 
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.completeFile(FSNamesystem.java:3484)
>       at 
> org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.complete(NameNodeRpcServer.java:791)
>       at 
> org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolServerSideTranslatorPB.complete(ClientNamenodeProtocolServerSideTranslatorPB.java:537)
>       at 
> org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos$ClientNamenodeProtocol$2.callBlockingMethod(ClientNamenodeProtocolProtos.java)
>       at 
> org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:608)
>       at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:969)
>       at org.apache.hadoop.ipc.Server.call(Server.java:2267)
>       at org.apache.hadoop.ipc.Server$RpcCall.run(Server.java:648)
>       at org.apache.hadoop.ipc.Server$RpcCall.run(Server.java:615)
>       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:1679)
>       at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2217)
>       at org.apache.hadoop.ipc.Client.call(Client.java:1457)
>       at org.apache.hadoop.ipc.Client.call(Client.java:1394)
>       at 
> org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:230)
>       at com.sun.proxy.$Proxy11.complete(Unknown Source)
>       at 
> org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.complete(ClientNamenodeProtocolTranslatorPB.java:462)
>       at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>       at 
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
>       at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>       at java.lang.reflect.Method.invoke(Method.java:483)
>       at 
> org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:191)
>       at 
> org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:102)
>       at com.sun.proxy.$Proxy12.complete(Unknown Source)
>       at 
> org.apache.hadoop.hdfs.DFSOutputStream.completeFile(DFSOutputStream.java:2300)
>       at 
> org.apache.hadoop.hdfs.DFSOutputStream.closeImpl(DFSOutputStream.java:2281)
>       at 
> org.apache.hadoop.hdfs.DFSOutputStream.close(DFSOutputStream.java:2245)
>       at 
> org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:72)
>       at 
> org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:106)
>       at 
> org.apache.hive.hcatalog.mapreduce.FileOutputCommitterContainer.moveTaskOutputs(FileOutputCommitterContainer.java:624)
>       at 
> org.apache.hive.hcatalog.mapreduce.FileOutputCommitterContainer.moveTaskOutputs(FileOutputCommitterContainer.java:606)
>       at 
> org.apache.hive.hcatalog.mapreduce.FileOutputCommitterContainer.moveTaskOutputs(FileOutputCommitterContainer.java:606)
>       at 
> org.apache.hive.hcatalog.mapreduce.FileOutputCommitterContainer.registerPartitions(FileOutputCommitterContainer.java:988)
> {noformat}
> This is the result of the following code in the 
> {{FileOutputCommitterContainer}}:
> {code:java}
>   private void moveTaskOutputs(FileSystem fs, Path file, Path srcDir,
>                  Path destDir, final boolean dryRun, boolean immutable
>       ) throws IOException {
>   // ...
>             if (dynamicPartitioningUsed) {
>             // Optimization: if the first child is file, we have reached the 
> leaf directory, move the parent directory itself
>             // instead of moving each file under the directory. See 
> HCATALOG-538
>             // Note for future Append implementation : This optimization is 
> another reason dynamic
>             // partitioning is currently incompatible with append on mutable 
> tables.
>             final Path parentDir = finalOutputPath.getParent();
>             // Create the directory
>             Path placeholder = new Path(parentDir, "_placeholder"); // <--- 
> HERE!
>             if (fs.mkdirs(parentDir)) {
>               // It is weired but we need a placeholder, 
>               // otherwise rename cannot move file to the right place
>               fs.create(placeholder).close();
>             }
>   // ...
>   }
> {code}
> Two storers will use the same placeholder location during commit. We should 
> have better isolation.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Reply via email to