mustafaiman commented on a change in pull request #1280:
URL: https://github.com/apache/hive/pull/1280#discussion_r468723475
##########
File path:
ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorGroupByOperator.java
##########
@@ -252,6 +258,13 @@ protected VectorAggregationBufferRow
allocateAggregationBuffer() throws HiveExce
return bufferSet;
}
+ protected void finishAggregators(boolean aborted) {
Review comment:
Instead of `finishAggregators`, can you make this method default `close`
method for `ProcessingModeBase` and call `super.close(boolean)` from close
methods of appropriate subclasses. That way common finalization code would be
in `close` of common super class and specific finalization code would be in
`close` method of each subclass.
##########
File path:
ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorGroupByOperator.java
##########
@@ -517,6 +532,10 @@ public void close(boolean aborted) throws HiveException {
}
+ //TODO: implement finishAggregators
+ protected void finishAggregators(boolean aborted) {
Review comment:
What about this mode? Seems not complete.
##########
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:
Why is `withConf` a seperate method? Conf should be a parameter to
VectorAggregateExpression's constructor.
##########
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);
+ }
+
+ public void add(ElementWrapper wrapper) {
+ queue.add(wrapper);
+ }
+
+ @Override
+ public void run() {
+ while (true) {
+ ElementWrapper currentBf = null;
+ try {
+ currentBf = queue.take();
+ // 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
+ merge(currentBf);
+ } catch (InterruptedException e) {// Executor.shutdownNow() is called
+ if (!queue.isEmpty()){
+ LOG.debug(
+ "bloom filter merge was interrupted while processing and queue
is still not empty"
+ + ", this is fine in case of shutdownNow");
+ }
+ while (!queue.isEmpty()) { // time to finish work if any
Review comment:
What if the operator was aborted? Do we still want to continue
processing in that case? I am not sure how heavy an operation this is. If it is
a short operation in all cases, it is okay to not have an abort path.
Otherwise, I think there should be an abort path where we do not bother
completing operations.
##########
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:
Furthermore, the conf object is used for only a single config option:
TEZ_BLOOM_FILTER_MERGE_THREADS . Instead of passing the config around, we
should extract the value here and just pass a single int.
##########
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
Review comment:
The same check appears in `mergeBloomFilterBytesFromInputColumn`. I feel
like these checks should have happened when we initialized
VectorUDAFBloomFilterMerge. We should not be checking if parallel processing
was enabled every time aggregation is called.
##########
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) {
Review comment:
`batchSize` is unused
----------------------------------------------------------------
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:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]