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

Mostafa Mokhtar updated HIVE-8291:
----------------------------------
    Description: 
Reading from bucketed partitioned tables has significantly higher overhead 
compared to non-bucketed non-partitioned files.


50% of the time is spent in these two lines of code in 
OrcInputFormate.getReader()
{code}
    String txnString = conf.get(ValidTxnList.VALID_TXNS_KEY,
                                Long.MAX_VALUE + ":");
    ValidTxnList validTxnList = new ValidTxnListImpl(txnString);
{code}

{code}
Stack Trace     Sample Count    Percentage(%)
                  hive.ql.exec.tez.MapRecordSource.pushRecord() 2,981   87.215
                     org.apache.tez.mapreduce.lib.MRReaderMapred.next() 2,002   
58.572
                        
mapred.split.TezGroupedSplitsInputFormat$TezGroupedSplitsRecordReader.next(Object,
 Object)      2,002   58.572
                           
mapred.split.TezGroupedSplitsInputFormat$TezGroupedSplitsRecordReader.initNextRecordReader()
 1,984   58.046
                              
hive.ql.io.HiveInputFormat.getRecordReader(InputSplit, JobConf, Reporter) 1,983 
  58.016
                                 
hive.ql.io.orc.OrcInputFormat.getRecordReader(InputSplit, JobConf, Reporter)   
1,891   55.325
                                    
hive.ql.io.orc.OrcInputFormat.getReader(InputSplit, AcidInputFormat$Options)    
    1,723   50.41
                                       
hive.common.ValidTxnListImpl.<init>(String)      934     27.326
                                       conf.Configuration.get(String, String)   
621     18.169
 {code}

  was:
When loading into a partitioned bucketed sorted table the query fails with 
{code}
Caused by: 
org.apache.hadoop.ipc.RemoteException(org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException):
 Failed to create file 
[/tmp/hive/mmokhtar/621d7923-90d1-4d9d-a4c6-b3bb075c7a8c/hive_2014-09-22_23-25-11_678_1598300430132235708-1/_task_tmp.-ext-10000/ss_sold_date=1998-01-02/_tmp.000000_3/delta_0123305_0123305/bucket_00000]
 for [DFSClient_attempt_1406566393272_6085_r_000144_3_-1677753045_12] for 
client [172.21.128.111], because this file is already being created by 
[DFSClient_attempt_1406566393272_6085_r_000031_3_-1506661042_12] on 
[172.21.128.122]
        at 
org.apache.hadoop.hdfs.server.namenode.FSNamesystem.recoverLeaseInternal(FSNamesystem.java:2543)
        at 
org.apache.hadoop.hdfs.server.namenode.FSNamesystem.startFileInternal(FSNamesystem.java:2308)
        at 
org.apache.hadoop.hdfs.server.namenode.FSNamesystem.startFileInt(FSNamesystem.java:2237)
        at 
org.apache.hadoop.hdfs.server.namenode.FSNamesystem.startFile(FSNamesystem.java:2190)
        at 
org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.create(NameNodeRpcServer.java:520)
        at 
org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolServerSideTranslatorPB.create(ClientNamenodeProtocolServerSideTranslatorPB.java:354)
        at 
org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos$ClientNamenodeProtocol$2.callBlockingMethod(ClientNamenodeProtocolProtos.java)
        at 
org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:585)
        at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:928)
        at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2013)
        at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2009)
        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:1557)
        at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2007)

        at org.apache.hadoop.ipc.Client.call(Client.java:1410)
        at org.apache.hadoop.ipc.Client.call(Client.java:1363)
        at 
org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:206)
        at com.sun.proxy.$Proxy15.create(Unknown Source)
        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:190)
        at 
org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:103)
        at com.sun.proxy.$Proxy15.create(Unknown Source)
        at 
org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.create(ClientNamenodeProtocolTranslatorPB.java:258)
        at 
org.apache.hadoop.hdfs.DFSOutputStream.newStreamForCreate(DFSOutputStream.java:1600)
        at org.apache.hadoop.hdfs.DFSClient.create(DFSClient.java:1465)
        at org.apache.hadoop.hdfs.DFSClient.create(DFSClient.java:1390)
        at 
org.apache.hadoop.hdfs.DistributedFileSystem$6.doCall(DistributedFileSystem.java:394)
        at 
org.apache.hadoop.hdfs.DistributedFileSystem$6.doCall(DistributedFileSystem.java:390)
        at 
org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
        at 
org.apache.hadoop.hdfs.DistributedFileSystem.create(DistributedFileSystem.java:390)
        at 
org.apache.hadoop.hdfs.DistributedFileSystem.create(DistributedFileSystem.java:334)
        at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:906)
        at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:887)
        at 
org.apache.hadoop.hive.ql.io.orc.WriterImpl.getStream(WriterImpl.java:1966)
        at 
org.apache.hadoop.hive.ql.io.orc.WriterImpl.flushStripe(WriterImpl.java:1983)
        at 
org.apache.hadoop.hive.ql.io.orc.WriterImpl.close(WriterImpl.java:2287)
        at 
org.apache.hadoop.hive.ql.io.orc.OrcRecordUpdater.close(OrcRecordUpdater.java:356)
        at org.apache.hadoop.hive.ql.exec.FileSinkOperator$FSPaths.closeWriter
{code}

DDL  
{code}
CREATE TABLE store_sales_sb ( 
        ss_sold_date_sk           int,
    ss_sold_time_sk           int,
    ss_item_sk                int,
    ss_customer_sk            int,
    ss_cdemo_sk               int,
    ss_hdemo_sk               int,
    ss_addr_sk                int,
    ss_store_sk               int,
    ss_promo_sk               int,
    ss_ticket_number          int,
    ss_quantity               int,
    ss_wholesale_cost         float,
    ss_list_price             float,
    ss_sales_price            float,
    ss_ext_discount_amt       float,
    ss_ext_sales_price        float,
    ss_ext_wholesale_cost     float,
    ss_ext_list_price         float,
    ss_ext_tax                float,
    ss_coupon_amt             float,
    ss_net_paid               float,
    ss_net_paid_inc_tax       float,
    ss_net_profit             float)
 PARTITIONED BY(ss_sold_date string)
 CLUSTERED BY(ss_item_sk,ss_ticket_number) SORTED 
BY(ss_item_sk,ss_ticket_number) INTO 32 BUCKETS
 STORED AS ORC;
{code}

DML 
{code}
insert overwrite table store_sales_sb partition (ss_sold_date) 
select
        ss.ss_sold_date_sk,
        ss.ss_sold_time_sk,
        ss.ss_item_sk,
        ss.ss_customer_sk,
        ss.ss_cdemo_sk,
        ss.ss_hdemo_sk,
        ss.ss_addr_sk,
        ss.ss_store_sk,
        ss.ss_promo_sk,
        ss.ss_ticket_number,
        ss.ss_quantity,
        ss.ss_wholesale_cost,
        ss.ss_list_price,
        ss.ss_sales_price,
        ss.ss_ext_discount_amt,
        ss.ss_ext_sales_price,
        ss.ss_ext_wholesale_cost,
        ss.ss_ext_list_price,
        ss.ss_ext_tax,
        ss.ss_coupon_amt,
        ss.ss_net_paid,
        ss.ss_net_paid_inc_tax,
        ss.ss_net_profit,
        ss.ss_sold_date
      from store_sales ss;
{code}


> Reading from partitioned bucketed tables has high overhead, 50% of time is 
> spent in OrcInputFormat.getReader
> ------------------------------------------------------------------------------------------------------------
>
>                 Key: HIVE-8291
>                 URL: https://issues.apache.org/jira/browse/HIVE-8291
>             Project: Hive
>          Issue Type: Bug
>          Components: Query Processor
>    Affects Versions: 0.14.0
>         Environment: cn105
>            Reporter: Mostafa Mokhtar
>            Assignee: Prasanth J
>             Fix For: 0.14.0
>
>
> Reading from bucketed partitioned tables has significantly higher overhead 
> compared to non-bucketed non-partitioned files.
> 50% of the time is spent in these two lines of code in 
> OrcInputFormate.getReader()
> {code}
>     String txnString = conf.get(ValidTxnList.VALID_TXNS_KEY,
>                                 Long.MAX_VALUE + ":");
>     ValidTxnList validTxnList = new ValidTxnListImpl(txnString);
> {code}
> {code}
> Stack Trace   Sample Count    Percentage(%)
>                   hive.ql.exec.tez.MapRecordSource.pushRecord()       2,981   
> 87.215
>                      org.apache.tez.mapreduce.lib.MRReaderMapred.next()       
> 2,002   58.572
>                         
> mapred.split.TezGroupedSplitsInputFormat$TezGroupedSplitsRecordReader.next(Object,
>  Object)    2,002   58.572
>                            
> mapred.split.TezGroupedSplitsInputFormat$TezGroupedSplitsRecordReader.initNextRecordReader()
>        1,984   58.046
>                               
> hive.ql.io.HiveInputFormat.getRecordReader(InputSplit, JobConf, Reporter)     
>   1,983   58.016
>                                  
> hive.ql.io.orc.OrcInputFormat.getRecordReader(InputSplit, JobConf, Reporter) 
> 1,891   55.325
>                                     
> hive.ql.io.orc.OrcInputFormat.getReader(InputSplit, AcidInputFormat$Options)  
>     1,723   50.41
>                                        
> hive.common.ValidTxnListImpl.<init>(String)    934     27.326
>                                        conf.Configuration.get(String, String) 
> 621     18.169
>  {code}



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

Reply via email to