[ https://issues.apache.org/jira/browse/HIVE-23880?focusedWorklogId=469682&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-469682 ]
ASF GitHub Bot logged work on HIVE-23880: ----------------------------------------- Author: ASF GitHub Bot Created on: 12/Aug/20 12:41 Start Date: 12/Aug/20 12:41 Worklog Time Spent: 10m Work Description: abstractdog commented on a change in pull request #1280: URL: https://github.com/apache/hive/pull/1280#discussion_r469099535 ########## File path: ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorGroupByOperator.java ########## @@ -1126,6 +1137,7 @@ protected void initializeOp(Configuration hconf) throws HiveException { VectorAggregateExpression vecAggrExpr = null; try { vecAggrExpr = ctor.newInstance(vecAggrDesc); + vecAggrExpr.withConf(hconf); Review comment: 1. constructor: first I tried to pass it to constructor, but that breaks compatibility with every other subclasses of VectorAggregateExpression, if I use ctor.newInstance(vecAggrDesc, hconf), I need to add that constructor to all subclasses, because they don't inherit this ctor from VectorAggregateExpression...withConf can solve this, let me know about better ways 2. single int: this config is specific to VectorUDAFBloomFilterMerge, so I believe I should not pass it through a constructor to every VectorAggregateExpression, and I didn't want to go for an instanceof hack for a cast + specific call ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org Issue Time Tracking ------------------- Worklog Id: (was: 469682) Time Spent: 6.5h (was: 6h 20m) > Bloom filters can be merged in a parallel way in VectorUDAFBloomFilterMerge > --------------------------------------------------------------------------- > > Key: HIVE-23880 > URL: https://issues.apache.org/jira/browse/HIVE-23880 > Project: Hive > Issue Type: Improvement > Reporter: László Bodor > Assignee: László Bodor > Priority: Major > Labels: pull-request-available > Attachments: lipwig-output3605036885489193068.svg > > Time Spent: 6.5h > Remaining Estimate: 0h > > Merging bloom filters in semijoin reduction can become the main bottleneck in > case of large number of source mapper tasks (~1000, Map 1 in below example) > and a large amount of expected entries (50M) in bloom filters. > For example in TPCDS Q93: > {code} > select /*+ semi(store_returns, sr_item_sk, store_sales, 70000000)*/ > ss_customer_sk > ,sum(act_sales) sumsales > from (select ss_item_sk > ,ss_ticket_number > ,ss_customer_sk > ,case when sr_return_quantity is not null then > (ss_quantity-sr_return_quantity)*ss_sales_price > else > (ss_quantity*ss_sales_price) end act_sales > from store_sales left outer join store_returns on (sr_item_sk = > ss_item_sk > and > sr_ticket_number = ss_ticket_number) > ,reason > where sr_reason_sk = r_reason_sk > and r_reason_desc = 'reason 66') t > group by ss_customer_sk > order by sumsales, ss_customer_sk > limit 100; > {code} > On 10TB-30TB scale there is a chance that from 3-4 mins of query runtime 1-2 > mins are spent with merging bloom filters (Reducer 2), as in: > [^lipwig-output3605036885489193068.svg] > {code} > ---------------------------------------------------------------------------------------------- > VERTICES MODE STATUS TOTAL COMPLETED RUNNING PENDING > FAILED KILLED > ---------------------------------------------------------------------------------------------- > Map 3 .......... llap SUCCEEDED 1 1 0 0 > 0 0 > Map 1 .......... llap SUCCEEDED 1263 1263 0 0 > 0 0 > Reducer 2 llap RUNNING 1 0 1 0 > 0 0 > Map 4 llap RUNNING 6154 0 207 5947 > 0 0 > Reducer 5 llap INITED 43 0 0 43 > 0 0 > Reducer 6 llap INITED 1 0 0 1 > 0 0 > ---------------------------------------------------------------------------------------------- > VERTICES: 02/06 [====>>----------------------] 16% ELAPSED TIME: 149.98 s > ---------------------------------------------------------------------------------------------- > {code} > For example, 70M entries in bloom filter leads to a 436 465 696 bits, so > merging 1263 bloom filters means running ~ 1263 * 436 465 696 bitwise OR > operation, which is very hot codepath, but can be parallelized. -- This message was sent by Atlassian Jira (v8.3.4#803005)