Skip to content

Commit d008709

Browse files
authored
Add OOM Protection Support for Multi-Stage Queries (apache#13598)
track cpu and memory usage in multi-stage queries if query resource usage tracking is enabled
1 parent 3f324a4 commit d008709

File tree

29 files changed

+399
-25
lines changed

29 files changed

+399
-25
lines changed

pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/MultiStageBrokerRequestHandler.java

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -64,10 +64,12 @@
6464
import org.apache.pinot.query.runtime.MultiStageStatsTreeBuilder;
6565
import org.apache.pinot.query.runtime.plan.MultiStageQueryStats;
6666
import org.apache.pinot.query.service.dispatch.QueryDispatcher;
67+
import org.apache.pinot.spi.accounting.ThreadExecutionContext;
6768
import org.apache.pinot.spi.auth.TableAuthorizationResult;
6869
import org.apache.pinot.spi.env.PinotConfiguration;
6970
import org.apache.pinot.spi.exception.DatabaseConflictException;
7071
import org.apache.pinot.spi.trace.RequestContext;
72+
import org.apache.pinot.spi.trace.Tracing;
7173
import org.apache.pinot.spi.utils.CommonConstants;
7274
import org.apache.pinot.spi.utils.builder.TableNameBuilder;
7375
import org.apache.pinot.sql.parsers.SqlNodeAndOptions;
@@ -210,6 +212,8 @@ protected BrokerResponse handleRequest(long requestId, String query, @Nullable S
210212
return new BrokerResponseNative(QueryException.getException(QueryException.QUOTA_EXCEEDED_ERROR, errorMessage));
211213
}
212214

215+
Tracing.ThreadAccountantOps.setupRunner(String.valueOf(requestId), ThreadExecutionContext.TaskType.MSE);
216+
213217
long executionStartTimeNs = System.nanoTime();
214218
QueryDispatcher.QueryResult queryResults;
215219
try {
@@ -228,6 +232,8 @@ protected BrokerResponse handleRequest(long requestId, String query, @Nullable S
228232
requestContext.setErrorCode(QueryException.QUERY_EXECUTION_ERROR_CODE);
229233
return new BrokerResponseNative(
230234
QueryException.getException(QueryException.QUERY_EXECUTION_ERROR, consolidatedMessage));
235+
} finally {
236+
Tracing.getThreadAccountant().clear();
231237
}
232238
long executionEndTimeNs = System.nanoTime();
233239
updatePhaseTimingForTables(tableNames, BrokerQueryPhase.QUERY_EXECUTION, executionEndTimeNs - executionStartTimeNs);

pinot-core/src/main/java/org/apache/pinot/core/accounting/CPUMemThreadLevelAccountingObjects.java

Lines changed: 17 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -103,8 +103,15 @@ public int getTaskId() {
103103
return taskEntry == null ? -1 : taskEntry.getTaskId();
104104
}
105105

106-
public void setThreadTaskStatus(@Nonnull String queryId, int taskId, @Nonnull Thread anchorThread) {
107-
_currentThreadTaskStatus.set(new TaskEntry(queryId, taskId, anchorThread));
106+
@Override
107+
public ThreadExecutionContext.TaskType getTaskType() {
108+
TaskEntry taskEntry = _currentThreadTaskStatus.get();
109+
return taskEntry == null ? ThreadExecutionContext.TaskType.UNKNOWN : taskEntry.getTaskType();
110+
}
111+
112+
public void setThreadTaskStatus(@Nonnull String queryId, int taskId, ThreadExecutionContext.TaskType taskType,
113+
@Nonnull Thread anchorThread) {
114+
_currentThreadTaskStatus.set(new TaskEntry(queryId, taskId, taskType, anchorThread));
108115
}
109116
}
110117

@@ -117,15 +124,17 @@ public static class TaskEntry implements ThreadExecutionContext {
117124
private final String _queryId;
118125
private final int _taskId;
119126
private final Thread _anchorThread;
127+
private final TaskType _taskType;
120128

121129
public boolean isAnchorThread() {
122130
return _taskId == CommonConstants.Accounting.ANCHOR_TASK_ID;
123131
}
124132

125-
public TaskEntry(String queryId, int taskId, Thread anchorThread) {
133+
public TaskEntry(String queryId, int taskId, TaskType taskType, Thread anchorThread) {
126134
_queryId = queryId;
127135
_taskId = taskId;
128136
_anchorThread = anchorThread;
137+
_taskType = taskType;
129138
}
130139

131140
public String getQueryId() {
@@ -140,6 +149,11 @@ public Thread getAnchorThread() {
140149
return _anchorThread;
141150
}
142151

152+
@Override
153+
public TaskType getTaskType() {
154+
return _taskType;
155+
}
156+
143157
@Override
144158
public String toString() {
145159
return "TaskEntry{" + "_queryId='" + _queryId + '\'' + ", _taskId=" + _taskId + ", _rootThread=" + _anchorThread

pinot-core/src/main/java/org/apache/pinot/core/accounting/PerQueryCPUMemAccountantFactory.java

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -293,10 +293,10 @@ public void createExecutionContextInner(@Nullable String queryId, int taskId, @N
293293
// is anchor thread
294294
assert queryId != null;
295295
_threadLocalEntry.get().setThreadTaskStatus(queryId, CommonConstants.Accounting.ANCHOR_TASK_ID,
296-
Thread.currentThread());
296+
ThreadExecutionContext.TaskType.UNKNOWN, Thread.currentThread());
297297
} else {
298298
// not anchor thread
299-
_threadLocalEntry.get().setThreadTaskStatus(parentContext.getQueryId(), taskId,
299+
_threadLocalEntry.get().setThreadTaskStatus(parentContext.getQueryId(), taskId, parentContext.getTaskType(),
300300
parentContext.getAnchorThread());
301301
}
302302
}

pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/QueryRunner.java

Lines changed: 5 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -50,6 +50,7 @@
5050
import org.apache.pinot.query.runtime.plan.pipeline.PipelineBreakerExecutor;
5151
import org.apache.pinot.query.runtime.plan.pipeline.PipelineBreakerResult;
5252
import org.apache.pinot.query.runtime.plan.server.ServerPlanRequestUtils;
53+
import org.apache.pinot.spi.accounting.ThreadExecutionContext;
5354
import org.apache.pinot.spi.env.PinotConfiguration;
5455
import org.apache.pinot.spi.utils.CommonConstants;
5556
import org.apache.pinot.spi.utils.CommonConstants.Broker.Request.QueryOptionKey;
@@ -152,7 +153,8 @@ public void shutDown() {
152153
* <p>This execution entry point should be asynchronously called by the request handler and caller should not wait
153154
* for results/exceptions.</p>
154155
*/
155-
public void processQuery(WorkerMetadata workerMetadata, StagePlan stagePlan, Map<String, String> requestMetadata) {
156+
public void processQuery(WorkerMetadata workerMetadata, StagePlan stagePlan, Map<String, String> requestMetadata,
157+
@Nullable ThreadExecutionContext parentContext) {
156158
long requestId = Long.parseLong(requestMetadata.get(CommonConstants.Query.Request.MetadataKeys.REQUEST_ID));
157159
long timeoutMs = Long.parseLong(requestMetadata.get(CommonConstants.Broker.Request.QueryOptionKey.TIMEOUT_MS));
158160
long deadlineMs = System.currentTimeMillis() + timeoutMs;
@@ -163,7 +165,7 @@ public void processQuery(WorkerMetadata workerMetadata, StagePlan stagePlan, Map
163165
// run pre-stage execution for all pipeline breakers
164166
PipelineBreakerResult pipelineBreakerResult =
165167
PipelineBreakerExecutor.executePipelineBreakers(_opChainScheduler, _mailboxService, workerMetadata, stagePlan,
166-
opChainMetadata, requestId, deadlineMs);
168+
opChainMetadata, requestId, deadlineMs, parentContext);
167169

168170
// Send error block to all the receivers if pipeline breaker fails
169171
if (pipelineBreakerResult != null && pipelineBreakerResult.getErrorBlock() != null) {
@@ -196,7 +198,7 @@ public void processQuery(WorkerMetadata workerMetadata, StagePlan stagePlan, Map
196198
// run OpChain
197199
OpChainExecutionContext executionContext =
198200
new OpChainExecutionContext(_mailboxService, requestId, deadlineMs, opChainMetadata, stageMetadata,
199-
workerMetadata, pipelineBreakerResult);
201+
workerMetadata, pipelineBreakerResult, parentContext);
200202
OpChain opChain;
201203
if (workerMetadata.isLeafStageWorker()) {
202204
opChain = ServerPlanRequestUtils.compileLeafStage(executionContext, stagePlan, _helixManager, _serverMetrics,

pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/executor/OpChainSchedulerService.java

Lines changed: 8 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -27,6 +27,9 @@
2727
import org.apache.pinot.query.runtime.blocks.TransferableBlock;
2828
import org.apache.pinot.query.runtime.operator.OpChain;
2929
import org.apache.pinot.query.runtime.operator.OpChainId;
30+
import org.apache.pinot.spi.accounting.ThreadExecutionContext;
31+
import org.apache.pinot.spi.accounting.ThreadResourceUsageProvider;
32+
import org.apache.pinot.spi.trace.Tracing;
3033
import org.slf4j.Logger;
3134
import org.slf4j.LoggerFactory;
3235

@@ -50,6 +53,10 @@ public void runJob() {
5053
TransferableBlock returnedErrorBlock = null;
5154
Throwable thrown = null;
5255
try {
56+
ThreadResourceUsageProvider threadResourceUsageProvider = new ThreadResourceUsageProvider();
57+
Tracing.ThreadAccountantOps.setupWorker(operatorChain.getId().getStageId(),
58+
ThreadExecutionContext.TaskType.MSE, threadResourceUsageProvider,
59+
operatorChain.getParentContext());
5360
LOGGER.trace("({}): Executing", operatorChain);
5461
TransferableBlock result = operatorChain.getRoot().nextBlock();
5562
while (!result.isEndOfStreamBlock()) {
@@ -76,6 +83,7 @@ public void runJob() {
7683
} else if (isFinished) {
7784
operatorChain.close();
7885
}
86+
Tracing.ThreadAccountantOps.clear();
7987
}
8088
}
8189
});

pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/AggregateOperator.java

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -173,6 +173,7 @@ private TransferableBlock consumeGroupBy() {
173173
TransferableBlock block = _input.nextBlock();
174174
while (block.isDataBlock()) {
175175
_groupByExecutor.processBlock(block);
176+
sampleAndCheckInterruption();
176177
block = _input.nextBlock();
177178
}
178179
return block;
@@ -187,6 +188,7 @@ private TransferableBlock consumeAggregation() {
187188
TransferableBlock block = _input.nextBlock();
188189
while (block.isDataBlock()) {
189190
_aggregationExecutor.processBlock(block);
191+
sampleAndCheckInterruption();
190192
block = _input.nextBlock();
191193
}
192194
return block;

pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/HashJoinOperator.java

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -253,6 +253,7 @@ private void buildBroadcastHashTable()
253253
hashCollection.add(row);
254254
}
255255
_currentRowsInHashTable += container.size();
256+
sampleAndCheckInterruption();
256257
rightBlock = _rightInput.nextBlock();
257258
}
258259
if (rightBlock.isErrorBlock()) {
@@ -297,6 +298,7 @@ private TransferableBlock buildJoinedDataBlock() throws ProcessingException {
297298
}
298299
assert leftBlock.isDataBlock();
299300
List<Object[]> rows = buildJoinedRows(leftBlock);
301+
sampleAndCheckInterruption();
300302
if (!rows.isEmpty()) {
301303
return new TransferableBlock(rows, _resultSchema, DataBlock.Type.ROW);
302304
}

pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/MailboxReceiveOperator.java

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -60,6 +60,8 @@ protected TransferableBlock getNextBlock() {
6060
}
6161
if (block.isSuccessfulEndOfStreamBlock()) {
6262
updateEosBlock(block, _statMap);
63+
} else if (block.isDataBlock()) {
64+
sampleAndCheckInterruption();
6365
}
6466
return block;
6567
}

pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/MailboxSendOperator.java

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -143,6 +143,7 @@ protected TransferableBlock getNextBlock() {
143143
earlyTerminate();
144144
}
145145
}
146+
sampleAndCheckInterruption();
146147
return block;
147148
} catch (QueryCancelledException e) {
148149
LOGGER.debug("Query was cancelled! for opChain: {}", _context.getId());

pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/MultiStageOperator.java

Lines changed: 9 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -66,6 +66,15 @@ public MultiStageOperator(OpChainExecutionContext context) {
6666

6767
public abstract void registerExecution(long time, int numRows);
6868

69+
// Samples resource usage of the operator. The operator should call this function for every block of data or
70+
// assuming the block holds 10000 rows or more.
71+
protected void sampleAndCheckInterruption() {
72+
Tracing.ThreadAccountantOps.sample();
73+
if (Tracing.ThreadAccountantOps.isInterrupted()) {
74+
earlyTerminate();
75+
}
76+
}
77+
6978
/**
7079
* Returns the next block from the operator. It should return non-empty data blocks followed by an end-of-stream (EOS)
7180
* block when all the data is processed, or an error block if an error occurred. After it returns EOS or error block,

0 commit comments

Comments
 (0)