[ 
https://issues.apache.org/jira/browse/HIVE-23880?focusedWorklogId=469182&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-469182
 ]

ASF GitHub Bot logged work on HIVE-23880:
-----------------------------------------

                Author: ASF GitHub Bot
            Created on: 11/Aug/20 12:12
            Start Date: 11/Aug/20 12:12
    Worklog Time Spent: 10m 
      Work Description: pgaref commented on a change in pull request #1280:
URL: https://github.com/apache/hive/pull/1280#discussion_r468513649



##########
File path: common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
##########
@@ -4330,6 +4330,12 @@ private static void 
populateLlapDaemonVarsSet(Set<String> llapDaemonVarsSetLocal
             "Bloom filter should be of at max certain size to be effective"),
     TEZ_BLOOM_FILTER_FACTOR("hive.tez.bloom.filter.factor", (float) 1.0,
             "Bloom filter should be a multiple of this factor with nDV"),
+    TEZ_BLOOM_FILTER_MERGE_THREADS("hive.tez.bloom.filter.merge.threads", 1,
+        "How many threads are used for merging bloom filters?\n"

Review comment:
       The number of threads used variable is actually in **addition to tasks 
main threads** -- I would make this a bit clearer

##########
File path: 
ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorAggregateExpression.java
##########
@@ -20,24 +20,25 @@
 
 import java.io.Serializable;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.common.type.DataTypePhysicalVariation;
 import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
 import org.apache.hadoop.hive.ql.exec.vector.VectorAggregationBufferRow;
 import org.apache.hadoop.hive.ql.exec.vector.VectorAggregationDesc;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
 import org.apache.hadoop.hive.ql.exec.vector.expressions.VectorExpression;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
-import org.apache.hadoop.hive.ql.plan.AggregationDesc;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator;
-import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator.Mode;
 
 /**
  * Base class for aggregation expressions.
  */
 public abstract class VectorAggregateExpression  implements Serializable {
-
+  protected final Logger LOG = LoggerFactory.getLogger(getClass().getName());

Review comment:
       Should we make this static? Do we really want an instance per Expr?
   PS: it also seems that we dont use it all below..

##########
File path: common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
##########
@@ -4330,6 +4330,12 @@ private static void 
populateLlapDaemonVarsSet(Set<String> llapDaemonVarsSetLocal
             "Bloom filter should be of at max certain size to be effective"),
     TEZ_BLOOM_FILTER_FACTOR("hive.tez.bloom.filter.factor", (float) 1.0,
             "Bloom filter should be a multiple of this factor with nDV"),
+    TEZ_BLOOM_FILTER_MERGE_THREADS("hive.tez.bloom.filter.merge.threads", 1,
+        "How many threads are used for merging bloom filters?\n"
+            + "-1: sanity check, it will fail if execution hits bloom filter 
merge codepath\n"
+            + " 0: feature is disabled\n"

Review comment:
       feature disabled -- use only task main thread for BF merging

##########
File path: 
ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFBloomFilterMerge.java
##########
@@ -77,6 +75,211 @@ public void reset() {
       // Do not change the initial bytes which contain 
NumHashFunctions/NumBits!
       Arrays.fill(bfBytes, BloomKFilter.START_OF_SERIALIZED_LONGS, 
bfBytes.length, (byte) 0);
     }
+

Review comment:
       Could add a comment describing the return boolean value 

##########
File path: 
ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFBloomFilterMerge.java
##########
@@ -77,6 +75,211 @@ public void reset() {
       // Do not change the initial bytes which contain 
NumHashFunctions/NumBits!
       Arrays.fill(bfBytes, BloomKFilter.START_OF_SERIALIZED_LONGS, 
bfBytes.length, (byte) 0);
     }
+
+    public boolean mergeBloomFilterBytesFromInputColumn(BytesColumnVector 
inputColumn,
+        int batchSize, boolean selectedInUse, int[] selected, Configuration 
conf) {
+      // already set in previous iterations, no need to call initExecutor again
+      if (numThreads == 0) {
+        return false;
+      }
+      if (executor == null) {
+        initExecutor(conf, batchSize);
+        if (!isParallel) {
+          return false;
+        }
+      }
+
+      // 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]) {

Review comment:
       Could also follow an alternative approach -- where we have an external 
isSelected Condition and we check for isNull in internal ones (similar to what 
we do elsewhere in the code)

##########
File path: 
ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFBloomFilterMerge.java
##########
@@ -77,6 +75,211 @@ public void reset() {
       // Do not change the initial bytes which contain 
NumHashFunctions/NumBits!
       Arrays.fill(bfBytes, BloomKFilter.START_OF_SERIALIZED_LONGS, 
bfBytes.length, (byte) 0);
     }
+
+    public boolean mergeBloomFilterBytesFromInputColumn(BytesColumnVector 
inputColumn,
+        int batchSize, boolean selectedInUse, int[] selected, Configuration 
conf) {
+      // already set in previous iterations, no need to call initExecutor again
+      if (numThreads == 0) {
+        return false;
+      }
+      if (executor == null) {
+        initExecutor(conf, batchSize);
+        if (!isParallel) {
+          return false;
+        }
+      }
+
+      // 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]);
+          }
+        }
+      }
+
+      return true;
+    }
+
+    private void initExecutor(Configuration conf, int batchSize) {
+      numThreads = 
conf.getInt(HiveConf.ConfVars.TEZ_BLOOM_FILTER_MERGE_THREADS.varname,
+          HiveConf.ConfVars.TEZ_BLOOM_FILTER_MERGE_THREADS.defaultIntVal);
+      LOG.info("Number of threads used for bloom filter merge: {}", 
numThreads);
+
+      if (numThreads < 0) {
+        throw new RuntimeException(
+            "invalid number of threads for bloom filter merge: " + numThreads);
+      }
+      if (numThreads == 0) { // disable parallel feature
+        return; // this will leave isParallel=false
+      }
+      isParallel = true;
+      executor = Executors.newFixedThreadPool(numThreads);
+
+      workers = new BloomFilterMergeWorker[numThreads];
+      for (int f = 0; f < numThreads; f++) {
+        workers[f] = new BloomFilterMergeWorker(bfBytes, 0, bfBytes.length);
+      }
+
+      for (int f = 0; f < numThreads; f++) {
+        executor.submit(workers[f]);
+      }
+    }
+
+    public int getNumberOfWaitingMergeTasks(){
+      int size = 0;
+      for (BloomFilterMergeWorker w : workers){
+        size += w.queue.size();
+      }
+      return size;
+    }
+
+    public int getNumberOfMergingWorkers() {
+      int working = 0;
+      for (BloomFilterMergeWorker w : workers) {
+        if (w.isMerging.get()) {
+          working += 1;
+        }
+      }
+      return working;
+    }
+
+    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 void shutdownAndWaitForMergeTasks() {
+      /**
+       * Executor.shutdownNow() is supposed to send Thread.interrupt to worker 
threads, and they are
+       * supposed to finish their work.
+       */
+      executor.shutdownNow();
+      try {
+        executor.awaitTermination(180, TimeUnit.SECONDS);
+      } catch (InterruptedException e) {
+        LOG.warn("Bloom filter merge is interrupted while waiting to finish, 
this is unexpected",
+            e);
+      }
+    }
+  }
+
+  private static class BloomFilterMergeWorker implements Runnable {
+    private BlockingQueue<ElementWrapper> queue;
+    private byte[] bfAggregation;
+    private int bfAggregationStart;
+    private int bfAggregationLength;
+    AtomicBoolean isMerging = new AtomicBoolean(false);
+
+    public BloomFilterMergeWorker(byte[] bfAggregation, int bfAggregationStart,
+        int bfAggregationLength) {
+      this.bfAggregation = bfAggregation;
+      this.bfAggregationStart = bfAggregationStart;
+      this.bfAggregationLength = bfAggregationLength;
+      this.queue = new ArrayBlockingQueue<>(VectorizedRowBatch.DEFAULT_SIZE * 
2);

Review comment:
       Why VectorizedRowBatch.DEFAULT_SIZE * 2 aka 2048 ?
   maybe workers.len * 2? 

##########
File path: 
ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFBloomFilterMerge.java
##########
@@ -77,6 +75,211 @@ public void reset() {
       // Do not change the initial bytes which contain 
NumHashFunctions/NumBits!
       Arrays.fill(bfBytes, BloomKFilter.START_OF_SERIALIZED_LONGS, 
bfBytes.length, (byte) 0);
     }
+
+    public boolean mergeBloomFilterBytesFromInputColumn(BytesColumnVector 
inputColumn,
+        int batchSize, boolean selectedInUse, int[] selected, Configuration 
conf) {
+      // already set in previous iterations, no need to call initExecutor again
+      if (numThreads == 0) {
+        return false;
+      }
+      if (executor == null) {
+        initExecutor(conf, batchSize);
+        if (!isParallel) {
+          return false;
+        }
+      }
+
+      // 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]) {

Review comment:
       !inputColumn.isNull[j] can be merged with the above condition

##########
File path: 
ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFBloomFilterMerge.java
##########
@@ -77,6 +75,211 @@ public void reset() {
       // Do not change the initial bytes which contain 
NumHashFunctions/NumBits!
       Arrays.fill(bfBytes, BloomKFilter.START_OF_SERIALIZED_LONGS, 
bfBytes.length, (byte) 0);
     }
+
+    public boolean mergeBloomFilterBytesFromInputColumn(BytesColumnVector 
inputColumn,
+        int batchSize, boolean selectedInUse, int[] selected, Configuration 
conf) {
+      // already set in previous iterations, no need to call initExecutor again
+      if (numThreads == 0) {
+        return false;
+      }
+      if (executor == null) {
+        initExecutor(conf, batchSize);
+        if (!isParallel) {
+          return false;
+        }
+      }
+
+      // 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]);
+          }
+        }
+      }
+
+      return true;
+    }
+
+    private void initExecutor(Configuration conf, int batchSize) {
+      numThreads = 
conf.getInt(HiveConf.ConfVars.TEZ_BLOOM_FILTER_MERGE_THREADS.varname,
+          HiveConf.ConfVars.TEZ_BLOOM_FILTER_MERGE_THREADS.defaultIntVal);
+      LOG.info("Number of threads used for bloom filter merge: {}", 
numThreads);
+
+      if (numThreads < 0) {
+        throw new RuntimeException(
+            "invalid number of threads for bloom filter merge: " + numThreads);
+      }
+      if (numThreads == 0) { // disable parallel feature
+        return; // this will leave isParallel=false
+      }
+      isParallel = true;
+      executor = Executors.newFixedThreadPool(numThreads);
+
+      workers = new BloomFilterMergeWorker[numThreads];
+      for (int f = 0; f < numThreads; f++) {

Review comment:
       Combine in a single loop?




----------------------------------------------------------------
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: 469182)
    Time Spent: 3h 20m  (was: 3h 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: 3h 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)

Reply via email to