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

Csaba Ringhofer commented on IMPALA-11677:
------------------------------------------

Looked up the history a bit:
async insert event firing was added here:
https://gerrit.cloudera.org/#/c/15263/

Later we switched to the bulk api with Hive3 HMS and returned to synchronous 
API calls for Hive3, but not for Hive2 (in the same commit that added self 
event detection):
https://gerrit.cloudera.org/#/c/15648/

I am a bit concerned about possible side effects of going async, even if Impala 
used to work like this. My understanding is this happens on a completely 
different thread and we let the INSERT to be finished before the events are 
fired and also release the table level lock. This could mean that another 
operation that comes after the first insert may fire its events before the 
first insert's events reach HMS.

> Could you share some perf numbers that you saw are slow?
As [~stigahuang] wrote, it would be good to know some info, e.g. number of 
changed partitions, time needed to finish this RPC. Profiling HMS could be also 
useful. Even if going async helps with making the INSERT quicker, if the HMS is 
overloaded, the overall performance may remain bad, in the worst case this fire 
events RPCs could queue up inside HMS (I don't know how HMS handles this). 

> FireInsertEvents function can be very slow for tables with large number of 
> partitions.
> --------------------------------------------------------------------------------------
>
>                 Key: IMPALA-11677
>                 URL: https://issues.apache.org/jira/browse/IMPALA-11677
>             Project: IMPALA
>          Issue Type: Improvement
>          Components: Catalog
>    Affects Versions: Impala 4.1.0
>            Reporter: Qihong Jiang
>            Assignee: Qihong Jiang
>            Priority: Major
>
> In src/compat-apache-hive-3/java/org/apache/impala/compat/MetastoreShim.java. 
> fireInsertEvents function can be very slow for tables with large number of 
> partitions. So we should use asynchronous calls.Just like in impala-3.x



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to