-
Notifications
You must be signed in to change notification settings - Fork 25.6k
ESQL: Account for the inlinestats LocalRelations #134455
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
Changes from 5 commits
72dd7ed
dd69956
e61528c
1927f1d
ac57e8a
55e6a14
d2a1577
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 |
|---|---|---|
|
|
@@ -16,6 +16,7 @@ | |
| import org.elasticsearch.action.support.SubscribableListener; | ||
| import org.elasticsearch.common.collect.Iterators; | ||
| import org.elasticsearch.compute.data.Block; | ||
| import org.elasticsearch.compute.data.BlockFactory; | ||
| import org.elasticsearch.compute.data.BlockUtils; | ||
| import org.elasticsearch.compute.data.Page; | ||
| import org.elasticsearch.compute.operator.DriverCompletionInfo; | ||
|
|
@@ -83,6 +84,7 @@ | |
| import java.util.List; | ||
| import java.util.Map; | ||
| import java.util.Set; | ||
| import java.util.concurrent.atomic.AtomicReference; | ||
| import java.util.stream.Collectors; | ||
| import java.util.stream.Stream; | ||
|
|
||
|
|
@@ -124,6 +126,7 @@ public interface PlanRunner { | |
| private final IndicesExpressionGrouper indicesExpressionGrouper; | ||
| private final InferenceService inferenceService; | ||
| private final RemoteClusterService remoteClusterService; | ||
| private final BlockFactory blockFactory; | ||
|
|
||
| private boolean explainMode; | ||
| private String parsedPlanString; | ||
|
|
@@ -160,6 +163,7 @@ public EsqlSession( | |
| this.inferenceService = services.inferenceService(); | ||
| this.preMapper = new PreMapper(services); | ||
| this.remoteClusterService = services.transportService().getRemoteClusterService(); | ||
| this.blockFactory = services.blockFactoryProvider().blockFactory(); | ||
| } | ||
|
|
||
| public String sessionId() { | ||
|
|
@@ -269,10 +273,13 @@ private void executeSubPlan( | |
| var physicalSubPlan = logicalPlanToPhysicalPlan(subPlans.stubReplacedSubPlan(), request); | ||
|
|
||
| runner.run(physicalSubPlan, listener.delegateFailureAndWrap((next, result) -> { | ||
| AtomicReference<Block[]> localRelationBlocks = new AtomicReference<>(); | ||
| try { | ||
| // Translate the subquery into a separate, coordinator based plan and the results 'broadcasted' as a local relation | ||
| completionInfoAccumulator.accumulate(result.completionInfo()); | ||
| LocalRelation resultWrapper = resultToPlan(subPlans.stubReplacedSubPlan(), result); | ||
| localRelationBlocks.set(resultWrapper.supplier().get()); | ||
| var releasingNext = ActionListener.runAfter(next, () -> releaseLocalRelationBlocks(localRelationBlocks)); | ||
|
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. Why is this needed? So that LocalRelation generated blocks are not kept in memory unnecessarily? For example, in case there are multiple 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.
Yes. Not kept in memory, we're not referencing them from the circuite breaker, but unaccounted within the circuite breaker.
Right. Ideally, we'd just pass the blocks from the produced |
||
|
|
||
| // replace the original logical plan with the backing result | ||
| LogicalPlan newLogicalPlan = optimizedPlan.transformUp( | ||
|
|
@@ -290,29 +297,40 @@ private void executeSubPlan( | |
| if (newSubPlan == null) {// run the final "main" plan | ||
| LOGGER.debug("Executing final plan:\n{}", newLogicalPlan); | ||
| var newPhysicalPlan = logicalPlanToPhysicalPlan(newLogicalPlan, request); | ||
| runner.run(newPhysicalPlan, next.delegateFailureAndWrap((finalListener, finalResult) -> { | ||
| runner.run(newPhysicalPlan, releasingNext.delegateFailureAndWrap((finalListener, finalResult) -> { | ||
| completionInfoAccumulator.accumulate(finalResult.completionInfo()); | ||
| finalListener.onResponse( | ||
| new Result(finalResult.schema(), finalResult.pages(), completionInfoAccumulator.finish(), executionInfo) | ||
| ); | ||
| })); | ||
| } else {// continue executing the subplans | ||
| executeSubPlan(completionInfoAccumulator, newLogicalPlan, newSubPlan, executionInfo, runner, request, listener); | ||
| executeSubPlan(completionInfoAccumulator, newLogicalPlan, newSubPlan, executionInfo, runner, request, releasingNext); | ||
| } | ||
| } catch (Exception e) { | ||
| // safely release the blocks in case an exception occurs either before, but also after the "final" runner.run() forks off | ||
| // the current thread, but with the blocks still referenced | ||
| releaseLocalRelationBlocks(localRelationBlocks); | ||
| throw e; | ||
| } finally { | ||
| Releasables.closeExpectNoException(Releasables.wrap(Iterators.map(result.pages().iterator(), p -> p::releaseBlocks))); | ||
| } | ||
| })); | ||
| } | ||
|
|
||
| private static LocalRelation resultToPlan(LogicalPlan plan, Result result) { | ||
| private LocalRelation resultToPlan(LogicalPlan plan, Result result) { | ||
| List<Page> pages = result.pages(); | ||
| List<Attribute> schema = result.schema(); | ||
| // if (pages.size() > 1) { | ||
| Block[] blocks = SessionUtils.fromPages(schema, pages); | ||
| Block[] blocks = SessionUtils.fromPages(schema, pages, blockFactory); | ||
| return new LocalRelation(plan.source(), schema, LocalSupplier.of(blocks)); | ||
| } | ||
|
|
||
| private static void releaseLocalRelationBlocks(AtomicReference<Block[]> localRelationBlocks) { | ||
| Block[] relationBlocks = localRelationBlocks.getAndSet(null); | ||
| if (relationBlocks != null) { | ||
| Releasables.closeExpectNoException(relationBlocks); | ||
| } | ||
| } | ||
|
|
||
| private EsqlStatement parse(String query, QueryParams params) { | ||
| var parsed = new EsqlParser().createQuery(query, params, planTelemetry, configuration); | ||
| if (LOGGER.isDebugEnabled()) { | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,117 @@ | ||
| /* | ||
| * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one | ||
| * or more contributor license agreements. Licensed under the Elastic License | ||
| * 2.0; you may not use this file except in compliance with the Elastic License | ||
| * 2.0. | ||
| */ | ||
|
|
||
| package org.elasticsearch.xpack.esql.session; | ||
|
|
||
| import org.apache.lucene.util.BytesRef; | ||
| import org.elasticsearch.common.breaker.CircuitBreaker; | ||
| import org.elasticsearch.common.breaker.CircuitBreakingException; | ||
| import org.elasticsearch.common.unit.ByteSizeValue; | ||
| import org.elasticsearch.common.util.BigArrays; | ||
| import org.elasticsearch.common.util.MockBigArrays; | ||
| import org.elasticsearch.common.util.PageCacheRecycler; | ||
| import org.elasticsearch.compute.data.Block; | ||
| import org.elasticsearch.compute.data.BlockFactory; | ||
| import org.elasticsearch.compute.data.BytesRefBlock; | ||
| import org.elasticsearch.compute.data.Page; | ||
| import org.elasticsearch.core.Releasables; | ||
| import org.elasticsearch.indices.breaker.CircuitBreakerService; | ||
| import org.elasticsearch.test.ESTestCase; | ||
| import org.elasticsearch.xpack.esql.core.expression.Attribute; | ||
| import org.elasticsearch.xpack.esql.core.expression.ReferenceAttribute; | ||
| import org.elasticsearch.xpack.esql.core.type.DataType; | ||
|
|
||
| import java.util.ArrayList; | ||
| import java.util.List; | ||
|
|
||
| import static org.elasticsearch.common.unit.ByteSizeUnit.GB; | ||
| import static org.elasticsearch.xpack.esql.plan.AbstractNodeSerializationTests.randomSource; | ||
| import static org.elasticsearch.xpack.esql.session.SessionUtils.fromPages; | ||
| import static org.hamcrest.Matchers.is; | ||
| import static org.mockito.Mockito.mock; | ||
| import static org.mockito.Mockito.when; | ||
|
|
||
| public class SessionUtilsTests extends ESTestCase { | ||
|
|
||
| /* | ||
| * 1. Generate a list of Pages with one BytesRef block, each of different positions, filled with random bytes. | ||
| * 2. Convert the list of Pages into a single BytesRefBlock Page using `fromPages()`. | ||
| * 3. Verify that the resulting BytesRefBlock contains the same bytes from the input Pages. | ||
| * 4. Verify that a CircuitBreakingException is thrown when the memory limit is too low. | ||
| */ | ||
| public void testFromPages() { | ||
| final int minBytes = 500; | ||
| final int maxBytes = randomIntBetween(minBytes, minBytes * 1_000); | ||
| byte[] inBuffer = new byte[maxBytes]; | ||
| BlockFactory blockFactory = blockFactory((int) GB.toBytes(1)); | ||
|
|
||
| BytesRefBlock.Builder builder = blockFactory.newBytesRefBlockBuilder(maxBytes); | ||
| List<Page> pages = new ArrayList<>(); | ||
| int producedBytes = 0; | ||
| int producedRows = 0; | ||
| int rowsPerPage = randomIntBetween(1, 100); | ||
| int rows = 0; | ||
| while (producedBytes < maxBytes) { | ||
| int rowBytes = Math.min(randomIntBetween(1, maxBytes / minBytes), maxBytes - producedBytes); | ||
| byte[] rowValue = randomByteArrayOfLength(rowBytes); | ||
|
|
||
| builder.appendBytesRef(new BytesRef(rowValue)); | ||
| System.arraycopy(rowValue, 0, inBuffer, producedBytes, rowBytes); | ||
|
|
||
| producedBytes += rowBytes; | ||
| rows++; | ||
|
|
||
| if (rows > rowsPerPage) { | ||
| producedRows += rows; | ||
| rows = 0; | ||
| enqueueBlock(builder, pages); | ||
| builder = blockFactory.newBytesRefBlockBuilder(maxBytes); | ||
| rowsPerPage = randomIntBetween(1, 100); | ||
| } | ||
| } | ||
| if (rows > 0) { | ||
| producedRows += rows; | ||
| enqueueBlock(builder, pages); | ||
| } | ||
|
|
||
| Attribute attr = new ReferenceAttribute(randomSource(), randomAlphaOfLengthOrNull(10), randomAlphaOfLength(10), DataType.KEYWORD); | ||
|
|
||
| Block[] outBlocks = fromPages(List.of(attr), pages, blockFactory); | ||
| assertThat(outBlocks.length, is(1)); | ||
| BytesRefBlock bytesBlock = (BytesRefBlock) outBlocks[0]; | ||
| assertThat(bytesBlock.getPositionCount(), is(producedRows)); | ||
|
|
||
| byte[] outBuffer = new byte[producedBytes]; | ||
| for (int i = 0, posCount = bytesBlock.getPositionCount(), outOffset = 0; i < posCount; i++) { | ||
| BytesRef ref = bytesBlock.getBytesRef(i, new BytesRef()); | ||
| System.arraycopy(ref.bytes, ref.offset, outBuffer, outOffset, ref.length); | ||
| outOffset += ref.length; | ||
| } | ||
| assertThat(outBuffer, is(inBuffer)); | ||
|
|
||
| Releasables.close(outBlocks); | ||
|
|
||
| BlockFactory convertBlockFactory = blockFactory(minBytes); | ||
| assertThrows(CircuitBreakingException.class, () -> fromPages(List.of(attr), pages, convertBlockFactory)); | ||
|
|
||
| Releasables.close(pages); | ||
| } | ||
|
|
||
| private BlockFactory blockFactory(long maxBytes) { | ||
| CircuitBreaker breaker = new MockBigArrays.LimitedBreaker(this.getClass().getName(), ByteSizeValue.ofBytes(maxBytes)); | ||
| CircuitBreakerService breakerService = mock(CircuitBreakerService.class); | ||
| when(breakerService.getBreaker(CircuitBreaker.REQUEST)).thenReturn(breaker); | ||
| BigArrays bigArrays = new MockBigArrays(PageCacheRecycler.NON_RECYCLING_INSTANCE, breakerService); | ||
| return new BlockFactory(bigArrays.breakerService().getBreaker(CircuitBreaker.REQUEST), bigArrays); | ||
| } | ||
|
|
||
| private void enqueueBlock(BytesRefBlock.Builder builder, List<Page> pages) { | ||
| Block block = builder.build(); | ||
| pages.add(new Page(block)); | ||
| Releasables.close(builder); | ||
| } | ||
| } |
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.
There is the
rowwhich creates a special CopyingLocalSupplier (see https://github.com/elastic/elasticsearch/pull/128917/files#diff-23897d0bd181d50370709de01c3ab3acc2f91be91d15db03f5dcdf5f27cf7866R32). I don't think it has anything to do with this PR, but I am mentioning in case you notice something that might have something to do with it. Before I added that supplier, the blocks created as part ofrowwere being double released withinlinestatsqueries resulting in exceptions.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.
I think that's to do with how inlinestats is executed: the "source" (ES or a row supplier) is accessed twice when executing
INLINE STATS: once for doing the agg, then for doing the join. The operator for the agg will get the blocks from the local relation and when done release them. And then the join operator will try to do the same, but those blocks are already released. These blocks are concerned with the left-hand side of the join.The blocks tracked with this PR are concerned with the right-hand side. The plan looks something like: