[ https://issues.apache.org/jira/browse/HIVE-23880?focusedWorklogId=460968&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-460968 ]
ASF GitHub Bot logged work on HIVE-23880: ----------------------------------------- Author: ASF GitHub Bot Created on: 20/Jul/20 08:43 Start Date: 20/Jul/20 08:43 Worklog Time Spent: 10m Work Description: rbalamohan commented on a change in pull request #1280: URL: https://github.com/apache/hive/pull/1280#discussion_r457184037 ########## File path: storage-api/src/java/org/apache/hive/common/util/BloomKFilter.java ########## @@ -362,16 +379,178 @@ public static void mergeBloomFilterBytes( // Just bitwise-OR the bits together - size/# functions should be the same, // rest of the data is serialized long values for the bitset which are supposed to be bitwise-ORed. - for (int idx = START_OF_SERIALIZED_LONGS; idx < bf1Length; ++idx) { + for (int idx = mergeStart; idx < mergeEnd; ++idx) { bf1Bytes[bf1Start + idx] |= bf2Bytes[bf2Start + idx]; } } + public static void mergeBloomFilterBytesFromInputColumn( + byte[] bf1Bytes, int bf1Start, int bf1Length, long bf1ExpectedEntries, + BytesColumnVector inputColumn, int batchSize, boolean selectedInUse, int[] selected, int numThreads) { + if (numThreads == 0) { + numThreads = Runtime.getRuntime().availableProcessors(); + } + if (numThreads < 0) { + throw new RuntimeException("invalid number of threads: " + numThreads); + } + + ExecutorService executor = Executors.newFixedThreadPool(numThreads); + + BloomFilterMergeWorker[] workers = new BloomFilterMergeWorker[numThreads]; + for (int f = 0; f < numThreads; f++) { + workers[f] = new BloomFilterMergeWorker(executor, bf1Bytes, bf1Start, bf1Length); + } + + // split every bloom filter (represented by a part of a byte[]) across workers + for (int j = 0; j < batchSize; j++) { + if (!selectedInUse && inputColumn.noNulls) { + splitVectorAcrossWorkers(workers, inputColumn.vector[j], inputColumn.start[j], + inputColumn.length[j]); + } else if (!selectedInUse) { + if (!inputColumn.isNull[j]) { + splitVectorAcrossWorkers(workers, inputColumn.vector[j], inputColumn.start[j], + inputColumn.length[j]); + } + } else if (inputColumn.noNulls) { + int i = selected[j]; + splitVectorAcrossWorkers(workers, inputColumn.vector[i], inputColumn.start[i], + inputColumn.length[i]); + } else { + int i = selected[j]; + if (!inputColumn.isNull[i]) { + splitVectorAcrossWorkers(workers, inputColumn.vector[i], inputColumn.start[i], + inputColumn.length[i]); + } + } + } + + for (int f = 0; f < numThreads; f++) { + executor.submit(workers[f]); + } + + executor.shutdown(); + try { + executor.awaitTermination(3600, TimeUnit.SECONDS); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + + private static void splitVectorAcrossWorkers(BloomFilterMergeWorker[] workers, byte[] bytes, + int start, int length) { + if (bytes == null || length == 0) { + return; + } + /* + * This will split a byte[] across workers as below: + * let's say there are 10 workers for 7813 bytes, in this case + * length: 7813, elementPerBatch: 781 + * bytes assigned to workers: inclusive lower bound, exclusive upper bound + * 1. worker: 5 -> 786 + * 2. worker: 786 -> 1567 + * 3. worker: 1567 -> 2348 + * 4. worker: 2348 -> 3129 + * 5. worker: 3129 -> 3910 + * 6. worker: 3910 -> 4691 + * 7. worker: 4691 -> 5472 + * 8. worker: 5472 -> 6253 + * 9. worker: 6253 -> 7034 + * 10. worker: 7034 -> 7813 (last element per batch is: 779) + * + * This way, a particular worker will be given with the same part + * of all bloom filters along with the shared base bloom filter, + * so the bitwise OR function will not be a subject of threading/sync issues. + */ + int elementPerBatch = + (int) Math.ceil((double) (length - START_OF_SERIALIZED_LONGS) / workers.length); + + for (int w = 0; w < workers.length; w++) { + int modifiedStart = START_OF_SERIALIZED_LONGS + w * elementPerBatch; + int modifiedLength = (w == workers.length - 1) + ? length - (START_OF_SERIALIZED_LONGS + w * elementPerBatch) : elementPerBatch; + + ElementWrapper wrapper = + new ElementWrapper(bytes, start, length, modifiedStart, modifiedLength); + workers[w].add(wrapper); + } + } + + public static byte[] getInitialBytes(long expectedEntries) { + ByteArrayOutputStream bytesOut = null; + try { + bytesOut = new ByteArrayOutputStream(); + BloomKFilter bf = new BloomKFilter(expectedEntries); + BloomKFilter.serialize(bytesOut, bf); + return bytesOut.toByteArray(); + } catch (Exception err) { + throw new IllegalArgumentException("Error creating aggregation buffer", err); + } finally { + IOUtils.closeStream(bytesOut); + } + } + + public static class ElementWrapper { + public byte[] bytes; + public int start; + public int length; + public int modifiedStart; + public int modifiedLength; + + public ElementWrapper(byte[] bytes, int start, int length, int modifiedStart, int modifiedLength) { + this.bytes = bytes; + this.start = start; + this.length = length; + this.modifiedStart = modifiedStart; + this.modifiedLength = modifiedLength; + } + } + + private static class BloomFilterMergeWorker implements Runnable { + Queue<ElementWrapper> queue = new LinkedBlockingDeque<>(); + private ExecutorService executor; + + private byte[] bfAggregation; + private int bfAggregationStart; + private int bfAggregationLength; + + public BloomFilterMergeWorker(ExecutorService executor, byte[] bfAggregation, int bfAggregationStart, int bfAggregationLength) { + this.executor = executor; + this.bfAggregation = bfAggregation; + this.bfAggregationStart = bfAggregationStart; + this.bfAggregationLength = bfAggregationLength; + } + + public void add(ElementWrapper wrapper) { + queue.add(wrapper); + } + + @Override + public void run() { + while (!executor.isTerminated() && !queue.isEmpty()) { + ElementWrapper currentBf = queue.poll(); + if (currentBf != null) { + // at this point we have a currentBf wrapper which contains the whole byte[] of the + // serialized bloomfilter, but we only want to merge a modified "start -> start+length" + // part of it, which is pointed by modifiedStart/modifiedLength fields by ElementWrapper + BloomKFilter.mergeBloomFilterBytes(bfAggregation, bfAggregationStart, bfAggregationLength, + currentBf.bytes, currentBf.start, currentBf.length, currentBf.modifiedStart, + currentBf.modifiedStart + currentBf.modifiedLength); + } else { + try { + Thread.sleep(10); // relax this thread while the queue is empty Review comment: Why is this needed? "splitVectorAcrossWorkers" seems to be adding all data before submitting to threadPool. IAC, it would be good to try with "queue::take()" (with additional change) which is a blocking call, to avoid "Thread.sleep". ---------------------------------------------------------------- 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: 460968) Time Spent: 20m (was: 10m) > 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: 20m > 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)