-
Notifications
You must be signed in to change notification settings - Fork 1.3k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Configure final reduce phase threads for heavy aggreagtion functions #14662
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -19,24 +19,32 @@ | |
package org.apache.pinot.core.data.table; | ||
|
||
import com.google.common.base.Preconditions; | ||
import java.util.ArrayList; | ||
import java.util.Arrays; | ||
import java.util.Collection; | ||
import java.util.Iterator; | ||
import java.util.List; | ||
import java.util.Map; | ||
import java.util.concurrent.ExecutionException; | ||
import java.util.concurrent.ExecutorService; | ||
import java.util.concurrent.Future; | ||
import java.util.concurrent.TimeUnit; | ||
import org.apache.pinot.common.request.context.ExpressionContext; | ||
import org.apache.pinot.common.utils.DataSchema; | ||
import org.apache.pinot.common.utils.DataSchema.ColumnDataType; | ||
import org.apache.pinot.core.query.aggregation.function.AggregationFunction; | ||
import org.apache.pinot.core.query.request.context.QueryContext; | ||
import org.apache.pinot.core.query.scheduler.resources.ResourceManager; | ||
import org.apache.pinot.core.util.QueryMultiThreadingUtils; | ||
import org.apache.pinot.core.util.trace.TraceCallable; | ||
|
||
|
||
/** | ||
* Base implementation of Map-based Table for indexed lookup | ||
*/ | ||
@SuppressWarnings({"rawtypes", "unchecked"}) | ||
public abstract class IndexedTable extends BaseTable { | ||
xiangfu0 marked this conversation as resolved.
Show resolved
Hide resolved
|
||
private final ExecutorService _executorService; | ||
protected final Map<Key, Record> _lookupMap; | ||
protected final boolean _hasFinalInput; | ||
protected final int _resultSize; | ||
|
@@ -46,6 +54,8 @@ public abstract class IndexedTable extends BaseTable { | |
protected final TableResizer _tableResizer; | ||
protected final int _trimSize; | ||
protected final int _trimThreshold; | ||
protected final int _numThreadsExtractFinalResult; | ||
protected final int _chunkSizeExtractFinalResult; | ||
|
||
protected Collection<Record> _topRecords; | ||
private int _numResizes; | ||
|
@@ -63,13 +73,14 @@ public abstract class IndexedTable extends BaseTable { | |
* @param lookupMap Map from keys to records | ||
*/ | ||
protected IndexedTable(DataSchema dataSchema, boolean hasFinalInput, QueryContext queryContext, int resultSize, | ||
int trimSize, int trimThreshold, Map<Key, Record> lookupMap) { | ||
int trimSize, int trimThreshold, Map<Key, Record> lookupMap, ExecutorService executorService) { | ||
super(dataSchema); | ||
|
||
Preconditions.checkArgument(resultSize >= 0, "Result size can't be negative"); | ||
Preconditions.checkArgument(trimSize >= 0, "Trim size can't be negative"); | ||
Preconditions.checkArgument(trimThreshold >= 0, "Trim threshold can't be negative"); | ||
|
||
_executorService = executorService; | ||
_lookupMap = lookupMap; | ||
_hasFinalInput = hasFinalInput; | ||
_resultSize = resultSize; | ||
|
@@ -84,6 +95,10 @@ protected IndexedTable(DataSchema dataSchema, boolean hasFinalInput, QueryContex | |
assert _hasOrderBy || (trimSize == Integer.MAX_VALUE && trimThreshold == Integer.MAX_VALUE); | ||
_trimSize = trimSize; | ||
_trimThreshold = trimThreshold; | ||
// NOTE: The upper limit of threads number for final reduce is set to 2 * number of available processors by default | ||
_numThreadsExtractFinalResult = Math.min(queryContext.getNumThreadsExtractFinalResult(), | ||
Math.max(1, ResourceManager.DEFAULT_QUERY_RUNNER_THREADS)); | ||
_chunkSizeExtractFinalResult = queryContext.getChunkSizeExtractFinalResult(); | ||
} | ||
|
||
@Override | ||
|
@@ -157,14 +172,88 @@ public void finish(boolean sort, boolean storeFinalResult) { | |
for (int i = 0; i < numAggregationFunctions; i++) { | ||
columnDataTypes[i + _numKeyColumns] = _aggregationFunctions[i].getFinalResultColumnType(); | ||
} | ||
for (Record record : _topRecords) { | ||
Object[] values = record.getValues(); | ||
for (int i = 0; i < numAggregationFunctions; i++) { | ||
int colId = i + _numKeyColumns; | ||
values[colId] = _aggregationFunctions[i].extractFinalResult(values[colId]); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think it'd make sense to either :
If shared thread pool is overwhelmed by running tasks it might be good to use current thread not only to wait but also task processing, stealing tasks until there's nothing left and only then waiting for futures to finish. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Potentially, and this can be done transparently by configuring the executor's rejected execution handler to CallerRunsPolicy. However, beware if the executor, which does non-blocking work, is sized to the number of available processors, then if the thread pool is overwhelmed, it means the available CPUs are overwhelmed too. Performing reductions on the caller thread would only lead to excessive context switching and it might be better, from a global perspective, for the task to wait for capacity to be available. |
||
int numThreadsExtractFinalResult = inferNumThreadsExtractFinalResult(); | ||
// Submit task when the EXECUTOR_SERVICE is not overloaded | ||
if (numThreadsExtractFinalResult > 1) { | ||
// Multi-threaded final reduce | ||
List<Future<Void>> futures = new ArrayList<>(numThreadsExtractFinalResult); | ||
try { | ||
List<Record> topRecordsList = new ArrayList<>(_topRecords); | ||
int chunkSize = (topRecordsList.size() + numThreadsExtractFinalResult - 1) / numThreadsExtractFinalResult; | ||
for (int threadId = 0; threadId < numThreadsExtractFinalResult; threadId++) { | ||
int startIdx = threadId * chunkSize; | ||
int endIdx = Math.min(startIdx + chunkSize, topRecordsList.size()); | ||
if (startIdx < endIdx) { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Is this always true? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. not always the case in the test with very small segment. |
||
// Submit a task for processing a chunk of values | ||
futures.add(_executorService.submit(new TraceCallable<Void>() { | ||
@Override | ||
public Void callJob() { | ||
for (int recordIdx = startIdx; recordIdx < endIdx; recordIdx++) { | ||
Object[] values = topRecordsList.get(recordIdx).getValues(); | ||
for (int i = 0; i < numAggregationFunctions; i++) { | ||
int colId = i + _numKeyColumns; | ||
values[colId] = _aggregationFunctions[i].extractFinalResult(values[colId]); | ||
} | ||
} | ||
return null; | ||
} | ||
})); | ||
} | ||
} | ||
// Wait for all tasks to complete | ||
for (Future<Void> future : futures) { | ||
future.get(); | ||
} | ||
xiangfu0 marked this conversation as resolved.
Show resolved
Hide resolved
|
||
} catch (InterruptedException | ExecutionException e) { | ||
// Cancel all running tasks | ||
for (Future<Void> future : futures) { | ||
future.cancel(true); | ||
} | ||
throw new RuntimeException("Error during multi-threaded final reduce", e); | ||
} | ||
} else { | ||
for (Record record : _topRecords) { | ||
Object[] values = record.getValues(); | ||
for (int i = 0; i < numAggregationFunctions; i++) { | ||
int colId = i + _numKeyColumns; | ||
values[colId] = _aggregationFunctions[i].extractFinalResult(values[colId]); | ||
} | ||
} | ||
} | ||
} | ||
} | ||
|
||
private int inferNumThreadsExtractFinalResult() { | ||
if (_numThreadsExtractFinalResult > 1) { | ||
return _numThreadsExtractFinalResult; | ||
} | ||
if (containsExpensiveAggregationFunctions()) { | ||
int parallelChunkSize = _chunkSizeExtractFinalResult; | ||
if (_topRecords != null && _topRecords.size() > parallelChunkSize) { | ||
int estimatedThreads = (int) Math.ceil((double) _topRecords.size() / parallelChunkSize); | ||
if (estimatedThreads == 0) { | ||
return 1; | ||
} | ||
return Math.min(estimatedThreads, QueryMultiThreadingUtils.MAX_NUM_THREADS_PER_QUERY); | ||
} | ||
} | ||
// Default to 1 thread | ||
return 1; | ||
} | ||
|
||
xiangfu0 marked this conversation as resolved.
Show resolved
Hide resolved
|
||
private boolean containsExpensiveAggregationFunctions() { | ||
for (AggregationFunction aggregationFunction : _aggregationFunctions) { | ||
switch (aggregationFunction.getType()) { | ||
case FUNNELCOMPLETECOUNT: | ||
case FUNNELCOUNT: | ||
case FUNNELMATCHSTEP: | ||
case FUNNELMAXSTEP: | ||
return true; | ||
default: | ||
break; | ||
} | ||
} | ||
return false; | ||
} | ||
|
||
@Override | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Would it be possible to show that final reduce is parallelized in explain output ?