Skip to content

Commit cbd9f05

Browse files
authored
ESQL: Account for the inlinestats LocalRelations (#134455)
This adds memory tracking for the blocks used in the `LocalRelation`s generated at the intermediary phase of executing an INLINE STATS. Closes #124744
1 parent f1bcb07 commit cbd9f05

File tree

6 files changed

+246
-19
lines changed

6 files changed

+246
-19
lines changed

x-pack/plugin/esql/qa/testFixtures/src/main/java/org/elasticsearch/xpack/esql/EsqlTestUtils.java

Lines changed: 4 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -29,6 +29,7 @@
2929
import org.elasticsearch.common.xcontent.XContentHelper;
3030
import org.elasticsearch.compute.data.AggregateMetricDoubleBlockBuilder;
3131
import org.elasticsearch.compute.data.BlockFactory;
32+
import org.elasticsearch.compute.data.BlockFactoryProvider;
3233
import org.elasticsearch.compute.data.BlockUtils;
3334
import org.elasticsearch.compute.data.BytesRefBlock;
3435
import org.elasticsearch.compute.data.DoubleBlock;
@@ -94,6 +95,7 @@
9495
import org.elasticsearch.xpack.esql.plan.logical.local.EmptyLocalSupplier;
9596
import org.elasticsearch.xpack.esql.plan.logical.local.LocalRelation;
9697
import org.elasticsearch.xpack.esql.plan.logical.local.LocalSupplier;
98+
import org.elasticsearch.xpack.esql.planner.PlannerUtils;
9799
import org.elasticsearch.xpack.esql.plugin.EsqlPlugin;
98100
import org.elasticsearch.xpack.esql.plugin.QueryPragmas;
99101
import org.elasticsearch.xpack.esql.plugin.TransportActionServices;
@@ -430,7 +432,8 @@ public static LogicalOptimizerContext unboundLogicalOptimizerContext() {
430432
mock(ProjectResolver.class),
431433
mock(IndexNameExpressionResolver.class),
432434
null,
433-
new InferenceService(mock(Client.class))
435+
new InferenceService(mock(Client.class)),
436+
new BlockFactoryProvider(PlannerUtils.NON_BREAKING_BLOCK_FACTORY)
434437
);
435438

436439
private static TransportService createMockTransportService() {

x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/TransportActionServices.java

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -10,6 +10,7 @@
1010
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
1111
import org.elasticsearch.cluster.project.ProjectResolver;
1212
import org.elasticsearch.cluster.service.ClusterService;
13+
import org.elasticsearch.compute.data.BlockFactoryProvider;
1314
import org.elasticsearch.compute.operator.exchange.ExchangeService;
1415
import org.elasticsearch.search.SearchService;
1516
import org.elasticsearch.transport.TransportService;
@@ -24,5 +25,6 @@ public record TransportActionServices(
2425
ProjectResolver projectResolver,
2526
IndexNameExpressionResolver indexNameExpressionResolver,
2627
UsageService usageService,
27-
InferenceService inferenceService
28+
InferenceService inferenceService,
29+
BlockFactoryProvider blockFactoryProvider
2830
) {}

x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/TransportEsqlQueryAction.java

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -170,7 +170,8 @@ public TransportEsqlQueryAction(
170170
projectResolver,
171171
indexNameExpressionResolver,
172172
usageService,
173-
new InferenceService(client)
173+
new InferenceService(client),
174+
blockFactoryProvider
174175
);
175176

176177
this.computeService = new ComputeService(

x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/EsqlSession.java

Lines changed: 48 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -15,7 +15,9 @@
1515
import org.elasticsearch.action.search.ShardSearchFailure;
1616
import org.elasticsearch.action.support.SubscribableListener;
1717
import org.elasticsearch.common.collect.Iterators;
18+
import org.elasticsearch.common.unit.ByteSizeValue;
1819
import org.elasticsearch.compute.data.Block;
20+
import org.elasticsearch.compute.data.BlockFactory;
1921
import org.elasticsearch.compute.data.BlockUtils;
2022
import org.elasticsearch.compute.data.Page;
2123
import org.elasticsearch.compute.operator.DriverCompletionInfo;
@@ -84,12 +86,14 @@
8486
import java.util.List;
8587
import java.util.Map;
8688
import java.util.Set;
89+
import java.util.concurrent.atomic.AtomicReference;
8790

8891
import static java.util.stream.Collectors.joining;
8992
import static java.util.stream.Collectors.toSet;
9093
import static org.elasticsearch.index.query.QueryBuilders.boolQuery;
9194
import static org.elasticsearch.xpack.esql.core.tree.Source.EMPTY;
9295
import static org.elasticsearch.xpack.esql.plan.logical.join.InlineJoin.firstSubPlan;
96+
import static org.elasticsearch.xpack.esql.session.SessionUtils.checkPagesBelowSize;
9397

9498
public class EsqlSession {
9599

@@ -104,6 +108,9 @@ public interface PlanRunner {
104108
}
105109

106110
private static final TransportVersion LOOKUP_JOIN_CCS = TransportVersion.fromName("lookup_join_ccs");
111+
private static final double INTERMEDIATE_LOCAL_RELATION_CB_PERCETAGE = .1;
112+
private static final long INTERMEDIATE_LOCAL_RELATION_MIN_SIZE = ByteSizeValue.ofMb(1).getBytes();
113+
private static final long INTERMEDIATE_LOCAL_RELATION_MAX_SIZE = ByteSizeValue.ofMb(30).getBytes();
107114

108115
private final String sessionId;
109116
private final Configuration configuration;
@@ -123,6 +130,8 @@ public interface PlanRunner {
123130
private final IndicesExpressionGrouper indicesExpressionGrouper;
124131
private final InferenceService inferenceService;
125132
private final RemoteClusterService remoteClusterService;
133+
private final BlockFactory blockFactory;
134+
private final long maxIntermediateLocalRelationSize;
126135

127136
private boolean explainMode;
128137
private String parsedPlanString;
@@ -159,6 +168,8 @@ public EsqlSession(
159168
this.inferenceService = services.inferenceService();
160169
this.preMapper = new PreMapper(services);
161170
this.remoteClusterService = services.transportService().getRemoteClusterService();
171+
this.blockFactory = services.blockFactoryProvider().blockFactory();
172+
maxIntermediateLocalRelationSize = maxIntermediateLocalRelationSize(blockFactory);
162173
}
163174

164175
public String sessionId() {
@@ -279,10 +290,13 @@ private void executeSubPlan(
279290
executionInfo.startSubPlans();
280291

281292
runner.run(physicalSubPlan, listener.delegateFailureAndWrap((next, result) -> {
293+
AtomicReference<Block[]> localRelationBlocks = new AtomicReference<>();
282294
try {
283295
// Translate the subquery into a separate, coordinator based plan and the results 'broadcasted' as a local relation
284296
completionInfoAccumulator.accumulate(result.completionInfo());
285-
LocalRelation resultWrapper = resultToPlan(subPlans.stubReplacedSubPlan(), result);
297+
LocalRelation resultWrapper = resultToPlan(subPlans.stubReplacedSubPlan().source(), result);
298+
localRelationBlocks.set(resultWrapper.supplier().get());
299+
var releasingNext = ActionListener.runAfter(next, () -> releaseLocalRelationBlocks(localRelationBlocks));
286300

287301
// replace the original logical plan with the backing result
288302
LogicalPlan newLogicalPlan = optimizedPlan.transformUp(
@@ -301,27 +315,54 @@ private void executeSubPlan(
301315
executionInfo.finishSubPlans();
302316
LOGGER.debug("Executing final plan:\n{}", newLogicalPlan);
303317
var newPhysicalPlan = logicalPlanToPhysicalPlan(newLogicalPlan, request);
304-
runner.run(newPhysicalPlan, next.delegateFailureAndWrap((finalListener, finalResult) -> {
318+
runner.run(newPhysicalPlan, releasingNext.delegateFailureAndWrap((finalListener, finalResult) -> {
305319
completionInfoAccumulator.accumulate(finalResult.completionInfo());
306320
finalListener.onResponse(
307321
new Result(finalResult.schema(), finalResult.pages(), completionInfoAccumulator.finish(), executionInfo)
308322
);
309323
}));
310324
} else {// continue executing the subplans
311-
executeSubPlan(completionInfoAccumulator, newLogicalPlan, newSubPlan, executionInfo, runner, request, listener);
325+
executeSubPlan(completionInfoAccumulator, newLogicalPlan, newSubPlan, executionInfo, runner, request, releasingNext);
312326
}
327+
} catch (Exception e) {
328+
// safely release the blocks in case an exception occurs either before, but also after the "final" runner.run() forks off
329+
// the current thread, but with the blocks still referenced
330+
releaseLocalRelationBlocks(localRelationBlocks);
331+
throw e;
313332
} finally {
314333
Releasables.closeExpectNoException(Releasables.wrap(Iterators.map(result.pages().iterator(), p -> p::releaseBlocks)));
315334
}
316335
}));
317336
}
318337

319-
private static LocalRelation resultToPlan(LogicalPlan plan, Result result) {
338+
private LocalRelation resultToPlan(Source planSource, Result result) {
320339
List<Page> pages = result.pages();
340+
checkPagesBelowSize(
341+
pages,
342+
maxIntermediateLocalRelationSize,
343+
(actual) -> "sub-plan execution results too large ["
344+
+ ByteSizeValue.ofBytes(actual)
345+
+ "] > "
346+
+ ByteSizeValue.ofBytes(maxIntermediateLocalRelationSize)
347+
);
321348
List<Attribute> schema = result.schema();
322-
// if (pages.size() > 1) {
323-
Block[] blocks = SessionUtils.fromPages(schema, pages);
324-
return new LocalRelation(plan.source(), schema, LocalSupplier.of(blocks));
349+
Block[] blocks = SessionUtils.fromPages(schema, pages, blockFactory);
350+
return new LocalRelation(planSource, schema, LocalSupplier.of(blocks));
351+
}
352+
353+
private static void releaseLocalRelationBlocks(AtomicReference<Block[]> localRelationBlocks) {
354+
Block[] relationBlocks = localRelationBlocks.getAndSet(null);
355+
if (relationBlocks != null) {
356+
Releasables.closeExpectNoException(relationBlocks);
357+
}
358+
}
359+
360+
// returns INTERMEDIATE_LOCAL_RELATION_CB_PERCETAGE percent of the circuit breaker limit, but at least
361+
// INTERMEDIATE_LOCAL_RELATION_MIN_SIZE and at most INTERMEDIATE_LOCAL_RELATION_MAX_SIZE
362+
static long maxIntermediateLocalRelationSize(BlockFactory blockFactory) {
363+
long breakerLimit = blockFactory.breaker().getLimit();
364+
long percentageLimit = (long) (breakerLimit * INTERMEDIATE_LOCAL_RELATION_CB_PERCETAGE / 100.d);
365+
return Math.min(Math.max(percentageLimit, INTERMEDIATE_LOCAL_RELATION_MIN_SIZE), INTERMEDIATE_LOCAL_RELATION_MAX_SIZE);
325366
}
326367

327368
private EsqlStatement parse(String query, QueryParams params) {

x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/session/SessionUtils.java

Lines changed: 12 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -7,8 +7,8 @@
77

88
package org.elasticsearch.xpack.esql.session;
99

10-
import org.elasticsearch.common.unit.ByteSizeValue;
1110
import org.elasticsearch.compute.data.Block;
11+
import org.elasticsearch.compute.data.BlockFactory;
1212
import org.elasticsearch.compute.data.BlockUtils;
1313
import org.elasticsearch.compute.data.Page;
1414
import org.elasticsearch.core.Releasables;
@@ -17,24 +17,19 @@
1717

1818
import java.util.ArrayList;
1919
import java.util.List;
20+
import java.util.function.LongFunction;
2021

2122
public class SessionUtils {
2223

2324
private SessionUtils() {}
2425

25-
public static Block[] fromPages(List<Attribute> schema, List<Page> pages) {
26-
// Limit ourselves to 1mb of results similar to LOOKUP for now.
27-
long bytesUsed = pages.stream().mapToLong(Page::ramBytesUsedByBlocks).sum();
28-
if (bytesUsed > ByteSizeValue.ofMb(1).getBytes()) {
29-
throw new IllegalArgumentException("sub-plan execution results too large [" + ByteSizeValue.ofBytes(bytesUsed) + "] > 1mb");
30-
}
26+
public static Block[] fromPages(List<Attribute> schema, List<Page> pages, BlockFactory blockFactory) {
3127
int positionCount = pages.stream().mapToInt(Page::getPositionCount).sum();
3228
Block.Builder[] builders = new Block.Builder[schema.size()];
3329
Block[] blocks;
3430
try {
3531
for (int b = 0; b < builders.length; b++) {
36-
builders[b] = PlannerUtils.toElementType(schema.get(b).dataType())
37-
.newBlockBuilder(positionCount, PlannerUtils.NON_BREAKING_BLOCK_FACTORY);
32+
builders[b] = PlannerUtils.toElementType(schema.get(b).dataType()).newBlockBuilder(positionCount, blockFactory);
3833
}
3934
for (Page p : pages) {
4035
for (int b = 0; b < builders.length; b++) {
@@ -48,6 +43,14 @@ public static Block[] fromPages(List<Attribute> schema, List<Page> pages) {
4843
return blocks;
4944
}
5045

46+
public static long checkPagesBelowSize(List<Page> pages, long maxSize, LongFunction<String> exceptionMessage) {
47+
long currentSize = pages.stream().mapToLong(Page::ramBytesUsedByBlocks).sum();
48+
if (currentSize > maxSize) {
49+
throw new IllegalArgumentException(exceptionMessage.apply(currentSize));
50+
}
51+
return currentSize;
52+
}
53+
5154
public static List<Object> fromPage(List<Attribute> schema, Page page) {
5255
if (page.getPositionCount() != 1) {
5356
throw new IllegalArgumentException("expected single row");

0 commit comments

Comments
 (0)