From 347feb17beb2112f08744326e7cf776415371b14 Mon Sep 17 00:00:00 2001 From: Gal Lalouche Date: Thu, 8 May 2025 00:15:23 +0300 Subject: [PATCH 01/16] ESQL: Aggressive release of shard contexts Keep better track of shard contexts using `RefCounted`, so they can be released more aggressively during operator processing. For example, during TopN, we can potentially release some contexts if they don't pass the limit filter. This is done in preparation of TopN fetch optimization, which will delay the fetching of additional columns to the data node coordinator, instead of doing it in each individual worker, thereby reducing IO. Since the node coordinator would need to maintain the shard contexts for a potentially longer duration, it is important we try to release what we can eariler. An even more advanced optimization is to delay fetching to the main cluster coordinator, but that would be more involved, since we need to first figure out how to transport the shard contexts between nodes. --- .../org/elasticsearch/core/Releasables.java | 4 + .../search/internal/SearchContext.java | 4 + .../search/MockSearchService.java | 7 +- .../elasticsearch/compute/data/DocBlock.java | 12 +- .../elasticsearch/compute/data/DocVector.java | 58 ++++++- .../compute/lucene/LuceneCountOperator.java | 20 ++- .../compute/lucene/LuceneOperator.java | 11 +- .../compute/lucene/LuceneSourceOperator.java | 18 ++- .../lucene/LuceneTopNSourceOperator.java | 16 +- .../compute/lucene/ShardContext.java | 3 +- .../compute/lucene/ShardRefCounted.java | 36 +++++ .../lucene/TimeSeriesSourceOperator.java | 29 +++- .../TimeSeriesSourceOperatorFactory.java | 5 +- .../lucene/ValuesSourceReaderOperator.java | 2 +- .../compute/operator/Driver.java | 34 +++-- .../lookup/EnrichQuerySourceOperator.java | 18 ++- .../operator/topn/ResultBuilderForDoc.java | 31 +++- .../compute/operator/topn/TopNOperator.java | 43 +++++- .../operator/topn/ValueExtractorForDoc.java | 4 + ...ctLongGroupingAggregatorFunctionTests.java | 4 +- .../CountGroupingAggregatorFunctionTests.java | 4 +- ...axLongGroupingAggregatorFunctionTests.java | 4 +- ...onLongGroupingAggregatorFunctionTests.java | 4 +- ...inLongGroupingAggregatorFunctionTests.java | 4 +- ...leLongGroupingAggregatorFunctionTests.java | 4 +- ...umLongGroupingAggregatorFunctionTests.java | 4 +- ...esLongGroupingAggregatorFunctionTests.java | 4 +- .../compute/data/BasicBlockTests.java | 17 ++- .../compute/data/DocVectorTests.java | 47 +++++- .../lucene/LuceneQueryEvaluatorTests.java | 1 + .../lucene/LuceneSourceOperatorTests.java | 22 +++ .../compute/operator/DriverContextTests.java | 9 +- .../compute/operator/EvalOperatorTests.java | 2 +- .../compute/operator/FilterOperatorTests.java | 2 +- .../HashAggregationOperatorTests.java | 2 +- .../operator/ProjectOperatorTests.java | 2 +- .../RowInTableLookupOperatorTests.java | 2 +- .../compute/operator/ShuffleDocsOperator.java | 3 +- .../TupleAbstractBlockSourceOperator.java | 97 ++++++++++++ .../operator/TupleBlockSourceOperator.java | 72 --------- .../TupleDocLongBlockSourceOperator.java | 47 ++++++ .../TupleLongLongBlockSourceOperator.java | 51 +++++++ .../EnrichQuerySourceOperatorTests.java | 8 +- .../compute/operator/topn/ExtractorTests.java | 6 + .../operator/topn/TopNOperatorTests.java | 143 +++++++++++++++--- .../compute/operator/topn/TopNRowTests.java | 1 + .../compute/test/NoOpReleasable.java | 15 ++ .../xpack/esql/qa/rest/RestEsqlTestCase.java | 2 +- .../action/AbstractPausableIntegTestCase.java | 17 ++- .../action/EsqlTopNShardManagementIT.java | 121 +++++++++++++++ .../xpack/esql/action/LookupFromIndexIT.java | 1 + .../esql/enrich/AbstractLookupService.java | 3 +- .../planner/EsPhysicalOperationProviders.java | 49 +++++- .../xpack/esql/plugin/ComputeService.java | 9 +- .../xpack/esql/plugin/QueryPragmas.java | 2 +- .../enrich/LookupFromIndexOperatorTests.java | 7 +- .../planner/LocalExecutionPlannerTests.java | 9 +- .../TestPhysicalOperationProviders.java | 2 + 58 files changed, 947 insertions(+), 211 deletions(-) create mode 100644 x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/ShardRefCounted.java create mode 100644 x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/TupleAbstractBlockSourceOperator.java delete mode 100644 x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/TupleBlockSourceOperator.java create mode 100644 x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/TupleDocLongBlockSourceOperator.java create mode 100644 x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/TupleLongLongBlockSourceOperator.java create mode 100644 x-pack/plugin/esql/compute/test/src/main/java/org/elasticsearch/compute/test/NoOpReleasable.java create mode 100644 x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/EsqlTopNShardManagementIT.java diff --git a/libs/core/src/main/java/org/elasticsearch/core/Releasables.java b/libs/core/src/main/java/org/elasticsearch/core/Releasables.java index 12417e0971c08..14b9b284292ce 100644 --- a/libs/core/src/main/java/org/elasticsearch/core/Releasables.java +++ b/libs/core/src/main/java/org/elasticsearch/core/Releasables.java @@ -202,6 +202,10 @@ public boolean equals(Object obj) { } } + public static Releasable fromRefCounted(RefCounted refCounted) { + return () -> refCounted.decRef(); + } + private static class ReleaseOnce extends AtomicReference implements Releasable { ReleaseOnce(Releasable releasable) { super(releasable); diff --git a/server/src/main/java/org/elasticsearch/search/internal/SearchContext.java b/server/src/main/java/org/elasticsearch/search/internal/SearchContext.java index 580fb5efc7222..7d018a7ef4ba9 100644 --- a/server/src/main/java/org/elasticsearch/search/internal/SearchContext.java +++ b/server/src/main/java/org/elasticsearch/search/internal/SearchContext.java @@ -115,6 +115,10 @@ public final void close() { closeFuture.onResponse(null); } + public final boolean isClosed() { + return closeFuture.isDone(); + } + /** * Should be called before executing the main query and after all other parameters have been set. */ diff --git a/test/framework/src/main/java/org/elasticsearch/search/MockSearchService.java b/test/framework/src/main/java/org/elasticsearch/search/MockSearchService.java index 14c75a01e5b67..07ffb3ab9a4eb 100644 --- a/test/framework/src/main/java/org/elasticsearch/search/MockSearchService.java +++ b/test/framework/src/main/java/org/elasticsearch/search/MockSearchService.java @@ -152,7 +152,12 @@ protected SearchContext createContext( @Override public SearchContext createSearchContext(ShardSearchRequest request, TimeValue timeout) throws IOException { SearchContext searchContext = super.createSearchContext(request, timeout); - onPutContext.accept(searchContext.readerContext()); + try { + onCreateSearchContext.accept(searchContext); + } catch (Exception e) { + searchContext.close(); + throw e; + } searchContext.addReleasable(() -> onRemoveContext.accept(searchContext.readerContext())); return searchContext; } diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/DocBlock.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/DocBlock.java index 7d1360c5102d8..dcf91bb3db7ee 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/DocBlock.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/DocBlock.java @@ -9,6 +9,8 @@ import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.compute.lucene.ShardRefCounted; +import org.elasticsearch.core.RefCounted; import org.elasticsearch.core.ReleasableIterator; import org.elasticsearch.core.Releasables; @@ -17,7 +19,7 @@ /** * Wrapper around {@link DocVector} to make a valid {@link Block}. */ -public class DocBlock extends AbstractVectorBlock implements Block { +public class DocBlock extends AbstractVectorBlock implements Block, RefCounted { private final DocVector vector; @@ -96,6 +98,12 @@ public static class Builder implements Block.Builder { private final IntVector.Builder shards; private final IntVector.Builder segments; private final IntVector.Builder docs; + private ShardRefCounted shardRefCounters = ShardRefCounted.ALWAYS_REFERENCED; + + public Builder setShardRefCounted(ShardRefCounted shardRefCounters) { + this.shardRefCounters = shardRefCounters; + return this; + } private Builder(BlockFactory blockFactory, int estimatedSize) { IntVector.Builder shards = null; @@ -183,7 +191,7 @@ public DocBlock build() { shards = this.shards.build(); segments = this.segments.build(); docs = this.docs.build(); - result = new DocVector(shards, segments, docs, null); + result = new DocVector(shardRefCounters, shards, segments, docs, null); return result.asBlock(); } finally { if (result == null) { diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/DocVector.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/DocVector.java index 5c8d23c6a2963..6f1343f626709 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/DocVector.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/DocVector.java @@ -10,6 +10,7 @@ import org.apache.lucene.util.IntroSorter; import org.apache.lucene.util.RamUsageEstimator; import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.compute.lucene.ShardRefCounted; import org.elasticsearch.core.ReleasableIterator; import org.elasticsearch.core.Releasables; @@ -48,8 +49,21 @@ public final class DocVector extends AbstractVector implements Vector { */ private int[] shardSegmentDocMapBackwards; - public DocVector(IntVector shards, IntVector segments, IntVector docs, Boolean singleSegmentNonDecreasing) { + private final ShardRefCounted shardRefCounters; + + public ShardRefCounted shardRefCounters() { + return shardRefCounters; + } + + public DocVector( + ShardRefCounted shardRefCounters, + IntVector shards, + IntVector segments, + IntVector docs, + Boolean singleSegmentNonDecreasing + ) { super(shards.getPositionCount(), shards.blockFactory()); + this.shardRefCounters = shardRefCounters; this.shards = shards; this.segments = segments; this.docs = docs; @@ -65,10 +79,19 @@ public DocVector(IntVector shards, IntVector segments, IntVector docs, Boolean s ); } blockFactory().adjustBreaker(BASE_RAM_BYTES_USED); + + forEachShardRefCounter(DecOrInc.INC); } - public DocVector(IntVector shards, IntVector segments, IntVector docs, int[] docMapForwards, int[] docMapBackwards) { - this(shards, segments, docs, null); + public DocVector( + ShardRefCounted shardRefCounters, + IntVector shards, + IntVector segments, + IntVector docs, + int[] docMapForwards, + int[] docMapBackwards + ) { + this(shardRefCounters, shards, segments, docs, null); this.shardSegmentDocMapForwards = docMapForwards; this.shardSegmentDocMapBackwards = docMapBackwards; } @@ -238,7 +261,7 @@ public DocVector filter(int... positions) { filteredShards = shards.filter(positions); filteredSegments = segments.filter(positions); filteredDocs = docs.filter(positions); - result = new DocVector(filteredShards, filteredSegments, filteredDocs, null); + result = new DocVector(shardRefCounters, filteredShards, filteredSegments, filteredDocs, null); return result; } finally { if (result == null) { @@ -317,5 +340,32 @@ public void closeInternal() { segments, docs ); + forEachShardRefCounter(DecOrInc.DEC); + } + + private enum DecOrInc { + DEC, + INC; + + void apply(ShardRefCounted counters, int shardId) { + switch (this) { + case DEC -> counters.get(shardId).decRef(); + case INC -> counters.get(shardId).mustIncRef(); + } + } + } + + private void forEachShardRefCounter(DecOrInc mode) { + switch (shards) { + case ConstantIntVector constantIntVector -> mode.apply(shardRefCounters, constantIntVector.getInt(0)); + case ConstantNullVector ignored -> { + // Noop + } + default -> { + for (int i = 0; i < shards.getPositionCount(); i++) { + mode.apply(shardRefCounters, shards.getInt(i)); + } + } + } } } diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneCountOperator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneCountOperator.java index fb733e0cb0576..5fc62db0a4bd1 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneCountOperator.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneCountOperator.java @@ -18,6 +18,7 @@ import org.elasticsearch.compute.data.Page; import org.elasticsearch.compute.operator.DriverContext; import org.elasticsearch.compute.operator.SourceOperator; +import org.elasticsearch.core.RefCounted; import org.elasticsearch.core.Releasables; import java.io.IOException; @@ -34,12 +35,14 @@ public class LuceneCountOperator extends LuceneOperator { private static final int PAGE_SIZE = 1; + private final List shardRefCounters; private int totalHits = 0; private int remainingDocs; private final LeafCollector leafCollector; public static class Factory extends LuceneOperator.Factory { + private final List shardRefCounters; public Factory( List contexts, @@ -58,11 +61,12 @@ public Factory( false, ScoreMode.COMPLETE_NO_SCORES ); + this.shardRefCounters = contexts; } @Override public SourceOperator get(DriverContext driverContext) { - return new LuceneCountOperator(driverContext.blockFactory(), sliceQueue, limit); + return new LuceneCountOperator(shardRefCounters, driverContext.blockFactory(), sliceQueue, limit); } @Override @@ -71,8 +75,15 @@ public String describe() { } } - public LuceneCountOperator(BlockFactory blockFactory, LuceneSliceQueue sliceQueue, int limit) { + public LuceneCountOperator( + List shardRefCounters, + BlockFactory blockFactory, + LuceneSliceQueue sliceQueue, + int limit + ) { super(blockFactory, PAGE_SIZE, sliceQueue); + this.shardRefCounters = shardRefCounters; + shardRefCounters.forEach(RefCounted::mustIncRef); this.remainingDocs = limit; this.leafCollector = new LeafCollector() { @Override @@ -171,4 +182,9 @@ protected Page getCheckedOutput() throws IOException { protected void describe(StringBuilder sb) { sb.append(", remainingDocs=").append(remainingDocs); } + + @Override + public void close() { + shardRefCounters.forEach(RefCounted::decRef); + } } diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneOperator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneOperator.java index 0da3915c9ad0c..8ce7eaabf7c95 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneOperator.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneOperator.java @@ -138,7 +138,11 @@ public final Page getOutput() { protected abstract Page getCheckedOutput() throws IOException; @Override - public void close() {} + public void close() { + if (currentScorer != null) { + currentScorer.shardContext().decRef(); + } + } LuceneScorer getCurrentOrLoadNextScorer() { while (currentScorer == null || currentScorer.isDone()) { @@ -161,7 +165,11 @@ LuceneScorer getCurrentOrLoadNextScorer() { ) { final Weight weight = currentSlice.weight(); processedQueries.add(weight.getQuery()); + var previousScorer = currentScorer; currentScorer = new LuceneScorer(currentSlice.shardContext(), weight, currentSlice.tags(), leaf); + if (previousScorer != null) { + previousScorer.shardContext().decRef(); + } } assert currentScorer.maxPosition <= partialLeaf.maxDoc() : currentScorer.maxPosition + ">" + partialLeaf.maxDoc(); currentScorer.maxPosition = partialLeaf.maxDoc(); @@ -188,6 +196,7 @@ static final class LuceneScorer { private Thread executingThread; LuceneScorer(ShardContext shardContext, Weight weight, List tags, LeafReaderContext leafReaderContext) { + shardContext.incRef(); this.shardContext = shardContext; this.weight = weight; this.tags = tags; diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneSourceOperator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneSourceOperator.java index 26339d2bdb108..d8dd90f1d53f2 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneSourceOperator.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneSourceOperator.java @@ -28,6 +28,7 @@ import org.elasticsearch.compute.operator.DriverContext; import org.elasticsearch.compute.operator.Limiter; import org.elasticsearch.compute.operator.SourceOperator; +import org.elasticsearch.core.RefCounted; import org.elasticsearch.core.Releasables; import org.elasticsearch.logging.LogManager; import org.elasticsearch.logging.Logger; @@ -48,6 +49,7 @@ */ public class LuceneSourceOperator extends LuceneOperator { private static final Logger log = LogManager.getLogger(LuceneSourceOperator.class); + private final List shardContextCounters; private int currentPagePos = 0; private int remainingDocs; @@ -59,7 +61,7 @@ public class LuceneSourceOperator extends LuceneOperator { private final int minPageSize; public static class Factory extends LuceneOperator.Factory { - + private final List contexts; private final int maxPageSize; private final Limiter limiter; @@ -82,6 +84,7 @@ public Factory( needsScore, needsScore ? COMPLETE : COMPLETE_NO_SCORES ); + this.contexts = contexts; this.maxPageSize = maxPageSize; // TODO: use a single limiter for multiple stage execution this.limiter = limit == NO_LIMIT ? Limiter.NO_LIMIT : new Limiter(limit); @@ -89,7 +92,7 @@ public Factory( @Override public SourceOperator get(DriverContext driverContext) { - return new LuceneSourceOperator(driverContext.blockFactory(), maxPageSize, sliceQueue, limit, limiter, needsScore); + return new LuceneSourceOperator(contexts, driverContext.blockFactory(), maxPageSize, sliceQueue, limit, limiter, needsScore); } public int maxPageSize() { @@ -216,6 +219,7 @@ private static PartitioningStrategy highSpeedAutoStrategyForBoolean(BooleanQuery @SuppressWarnings("this-escape") public LuceneSourceOperator( + List shardContextCounters, BlockFactory blockFactory, int maxPageSize, LuceneSliceQueue sliceQueue, @@ -224,6 +228,8 @@ public LuceneSourceOperator( boolean needsScore ) { super(blockFactory, maxPageSize, sliceQueue); + this.shardContextCounters = shardContextCounters; + shardContextCounters.forEach(RefCounted::mustIncRef); this.minPageSize = Math.max(1, maxPageSize / 2); this.remainingDocs = limit; this.limiter = limiter; @@ -329,7 +335,7 @@ public Page getCheckedOutput() throws IOException { docs = buildDocsVector(currentPagePos); docsBuilder = blockFactory.newIntVectorBuilder(Math.min(remainingDocs, maxPageSize)); int b = 0; - blocks[b++] = new DocVector(shard, leaf, docs, true).asBlock(); + blocks[b++] = new DocVector(ShardRefCounted.fromList(shardContextCounters), shard, leaf, docs, true).asBlock(); shard = null; leaf = null; docs = null; @@ -388,7 +394,11 @@ private DoubleVector buildScoresVector(int upToPositions) { @Override public void close() { - Releasables.close(docsBuilder, scoreBuilder); + Releasables.close( + docsBuilder, + scoreBuilder, + Releasables.wrap(shardContextCounters.stream().map(Releasables::fromRefCounted).toList()) + ); } @Override diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneTopNSourceOperator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneTopNSourceOperator.java index 5457caa25e158..83d5efe62f6ce 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneTopNSourceOperator.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneTopNSourceOperator.java @@ -53,6 +53,7 @@ public final class LuceneTopNSourceOperator extends LuceneOperator { public static class Factory extends LuceneOperator.Factory { + private final List contexts; private final int maxPageSize; private final List> sorts; @@ -76,13 +77,14 @@ public Factory( needsScore, needsScore ? TOP_DOCS_WITH_SCORES : TOP_DOCS ); + this.contexts = contexts; this.maxPageSize = maxPageSize; this.sorts = sorts; } @Override public SourceOperator get(DriverContext driverContext) { - return new LuceneTopNSourceOperator(driverContext.blockFactory(), maxPageSize, sorts, limit, sliceQueue, needsScore); + return new LuceneTopNSourceOperator(contexts, driverContext.blockFactory(), maxPageSize, sorts, limit, sliceQueue, needsScore); } public int maxPageSize() { @@ -116,11 +118,13 @@ public String describe() { private int offset = 0; private PerShardCollector perShardCollector; + private final List contexts; private final List> sorts; private final int limit; private final boolean needsScore; public LuceneTopNSourceOperator( + List contexts, BlockFactory blockFactory, int maxPageSize, List> sorts, @@ -129,6 +133,8 @@ public LuceneTopNSourceOperator( boolean needsScore ) { super(blockFactory, maxPageSize, sliceQueue); + this.contexts = contexts; + contexts.forEach(ShardContext::mustIncRef); this.sorts = sorts; this.limit = limit; this.needsScore = needsScore; @@ -239,7 +245,7 @@ private Page emit(boolean startEmitting) { shard = blockFactory.newConstantIntBlockWith(perShardCollector.shardContext.index(), size); segments = currentSegmentBuilder.build(); docs = currentDocsBuilder.build(); - docBlock = new DocVector(shard.asVector(), segments, docs, null).asBlock(); + docBlock = new DocVector(ShardRefCounted.fromList(contexts), shard.asVector(), segments, docs, null).asBlock(); shard = null; segments = null; docs = null; @@ -364,4 +370,10 @@ private static PerShardCollector newPerShardCollector(ShardContext context, List sort = new Sort(l.toArray(SortField[]::new)); return new ScoringPerShardCollector(context, new TopFieldCollectorManager(sort, limit, null, 0).newCollector()); } + + @Override + public void close() { + super.close(); + contexts.forEach(ShardContext::decRef); + } } diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/ShardContext.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/ShardContext.java index 8d1656899617f..d20a002407be6 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/ShardContext.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/ShardContext.java @@ -9,6 +9,7 @@ import org.apache.lucene.search.IndexSearcher; import org.elasticsearch.compute.data.Block; +import org.elasticsearch.core.RefCounted; import org.elasticsearch.index.mapper.BlockLoader; import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.SourceLoader; @@ -22,7 +23,7 @@ /** * Context of each shard we're operating against. */ -public interface ShardContext { +public interface ShardContext extends RefCounted { /** * The index of this shard in the list of shards being processed. */ diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/ShardRefCounted.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/ShardRefCounted.java new file mode 100644 index 0000000000000..a1caa0dfae3d4 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/ShardRefCounted.java @@ -0,0 +1,36 @@ +/* + * 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.compute.lucene; + +import org.elasticsearch.core.RefCounted; + +import java.util.List; + +/** Manages reference counting for {@link ShardContext}. */ +public interface ShardRefCounted { + RefCounted get(int shardId); + + static ShardRefCounted fromList(List refCounters) { + return shardId -> refCounters.get(shardId); + } + + static ShardRefCounted fromShardContext(ShardContext shardContext) { + return single(shardContext.index(), shardContext); + } + + static ShardRefCounted single(int index, RefCounted refCounted) { + return shardId -> { + if (shardId != index) { + throw new IllegalArgumentException("Invalid shardId: " + shardId + ", expected: " + index); + } + return refCounted; + }; + } + + ShardRefCounted ALWAYS_REFERENCED = shardId -> RefCounted.ALWAYS_REFERENCED; +} diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/TimeSeriesSourceOperator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/TimeSeriesSourceOperator.java index d0f1a5ee5fcd0..0e2a65ca9a5ef 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/TimeSeriesSourceOperator.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/TimeSeriesSourceOperator.java @@ -30,18 +30,20 @@ import org.elasticsearch.compute.data.OrdinalBytesRefVector; import org.elasticsearch.compute.data.Page; import org.elasticsearch.compute.operator.Operator; +import org.elasticsearch.core.RefCounted; import org.elasticsearch.core.Releasable; import org.elasticsearch.core.Releasables; import org.elasticsearch.xcontent.XContentBuilder; import java.io.IOException; import java.io.UncheckedIOException; +import java.util.List; import java.util.Map; import java.util.Objects; import java.util.Set; public final class TimeSeriesSourceOperator extends LuceneOperator { - + private final List contexts; private final int maxPageSize; private final BlockFactory blockFactory; private final LuceneSliceQueue sliceQueue; @@ -55,8 +57,16 @@ public final class TimeSeriesSourceOperator extends LuceneOperator { private DocIdCollector docCollector; private long tsidsLoaded; - TimeSeriesSourceOperator(BlockFactory blockFactory, LuceneSliceQueue sliceQueue, int maxPageSize, int limit) { + TimeSeriesSourceOperator( + List contexts, + BlockFactory blockFactory, + LuceneSliceQueue sliceQueue, + int maxPageSize, + int limit + ) { super(blockFactory, maxPageSize, sliceQueue); + contexts.forEach(RefCounted::mustIncRef); + this.contexts = contexts; this.maxPageSize = maxPageSize; this.blockFactory = blockFactory; this.remainingDocs = limit; @@ -100,7 +110,7 @@ public Page getCheckedOutput() throws IOException { throw new UnsupportedOperationException("tags not supported by " + getClass()); } iterator = new SegmentsIterator(slice); - docCollector = new DocIdCollector(blockFactory, slice.shardContext()); + docCollector = new DocIdCollector(contexts, blockFactory, slice.shardContext()); } iterator.readDocsForNextPage(); if (currentPagePos > 0) { @@ -132,7 +142,12 @@ public Page getCheckedOutput() throws IOException { @Override public void close() { - Releasables.closeExpectNoException(timestampsBuilder, tsHashesBuilder, docCollector); + Releasables.closeExpectNoException( + timestampsBuilder, + tsHashesBuilder, + docCollector, + Releasables.wrap(contexts.stream().map(Releasables::fromRefCounted).toList()) + ); } class SegmentsIterator { @@ -350,12 +365,14 @@ OrdinalBytesRefVector build() throws IOException { } static final class DocIdCollector implements Releasable { + private final List contexts; private final BlockFactory blockFactory; private final ShardContext shardContext; private IntVector.Builder docsBuilder; private IntVector.Builder segmentsBuilder; - DocIdCollector(BlockFactory blockFactory, ShardContext shardContext) { + DocIdCollector(List contexts, BlockFactory blockFactory, ShardContext shardContext) { + this.contexts = contexts; this.blockFactory = blockFactory; this.shardContext = shardContext; } @@ -382,7 +399,7 @@ DocVector build() { segments = segmentsBuilder.build(); segmentsBuilder = null; shards = blockFactory.newConstantIntVector(shardContext.index(), docs.getPositionCount()); - docVector = new DocVector(shards, segments, docs, segments.isConstant()); + docVector = new DocVector(ShardRefCounted.fromShardContext(shardContext), shards, segments, docs, segments.isConstant()); return docVector; } finally { if (docVector == null) { diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/TimeSeriesSourceOperatorFactory.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/TimeSeriesSourceOperatorFactory.java index 7ee13b3e6e0f5..97286761b7bcf 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/TimeSeriesSourceOperatorFactory.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/TimeSeriesSourceOperatorFactory.java @@ -27,7 +27,7 @@ * in order to read tsdb indices in parallel. */ public class TimeSeriesSourceOperatorFactory extends LuceneOperator.Factory { - + private final List contexts; private final int maxPageSize; private TimeSeriesSourceOperatorFactory( @@ -47,12 +47,13 @@ private TimeSeriesSourceOperatorFactory( false, ScoreMode.COMPLETE_NO_SCORES ); + this.contexts = contexts; this.maxPageSize = maxPageSize; } @Override public SourceOperator get(DriverContext driverContext) { - return new TimeSeriesSourceOperator(driverContext.blockFactory(), sliceQueue, maxPageSize, limit); + return new TimeSeriesSourceOperator(contexts, driverContext.blockFactory(), sliceQueue, maxPageSize, limit); } @Override diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/ValuesSourceReaderOperator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/ValuesSourceReaderOperator.java index d9f56340b458b..0067b6a562e84 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/ValuesSourceReaderOperator.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/ValuesSourceReaderOperator.java @@ -529,7 +529,7 @@ public void close() { } private LeafReaderContext ctx(int shard, int segment) { - return shardContexts.get(shard).reader.leaves().get(segment); + return shardContexts.get(shard).reader().leaves().get(segment); } @Override diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/Driver.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/Driver.java index ef7eef4c111bf..d297322ee4ac7 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/Driver.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/Driver.java @@ -75,7 +75,7 @@ public class Driver implements Releasable, Describable { private final long startNanos; private final DriverContext driverContext; private final Supplier description; - private final List activeOperators; + private List activeOperators; private final List statusOfCompletedOperators = new ArrayList<>(); private final Releasable releasable; private final long statusNanos; @@ -184,7 +184,8 @@ SubscribableListener run(TimeValue maxTime, int maxIterations, LongSupplie assert driverContext.assertBeginRunLoop(); isBlocked = runSingleLoopIteration(); } catch (DriverEarlyTerminationException unused) { - closeEarlyFinishedOperators(); + int lastFinished = closeEarlyFinishedOperators(0, activeOperators.size() - 1); + activeOperators = new ArrayList<>(activeOperators.subList(lastFinished + 1, activeOperators.size())); assert isFinished() : "not finished after early termination"; } finally { assert driverContext.assertEndRunLoop(); @@ -251,6 +252,7 @@ private IsBlockedResult runSingleLoopIteration() { driverContext.checkForEarlyTermination(); boolean movedPage = false; + int lastClosed = -1; for (int i = 0; i < activeOperators.size() - 1; i++) { Operator op = activeOperators.get(i); Operator nextOp = activeOperators.get(i + 1); @@ -283,11 +285,15 @@ private IsBlockedResult runSingleLoopIteration() { if (op.isFinished()) { driverContext.checkForEarlyTermination(); - nextOp.finish(); + closeEarlyFinishedOperators(lastClosed + 1, i); + lastClosed = i; } } - closeEarlyFinishedOperators(); + lastClosed = closeEarlyFinishedOperators(lastClosed + 1, activeOperators.size() - 1); + if (lastClosed >= 0) { + activeOperators = new ArrayList<>(activeOperators.subList(lastClosed + 1, activeOperators.size())); + } if (movedPage == false) { return oneOf( @@ -300,32 +306,32 @@ private IsBlockedResult runSingleLoopIteration() { return Operator.NOT_BLOCKED; } - private void closeEarlyFinishedOperators() { - for (int index = activeOperators.size() - 1; index >= 0; index--) { + private int closeEarlyFinishedOperators(int minIndex, int maxIndex) { + for (int index = maxIndex; index >= minIndex; index--) { if (activeOperators.get(index).isFinished()) { /* - * Close and remove this operator and all source operators in the + * Remove this operator and all unclosed source operators in the * most paranoid possible way. Closing operators shouldn't throw, * but if it does, this will make sure we don't try to close any * that succeed twice. */ - List finishedOperators = this.activeOperators.subList(0, index + 1); - Iterator itr = finishedOperators.iterator(); + List operatorsToClose = this.activeOperators.subList(minIndex, index + 1); + Iterator itr = operatorsToClose.iterator(); while (itr.hasNext()) { Operator op = itr.next(); statusOfCompletedOperators.add(new OperatorStatus(op.toString(), op.status())); op.close(); - itr.remove(); } - // Finish the next operator, which is now the first operator. - if (activeOperators.isEmpty() == false) { - Operator newRootOperator = activeOperators.get(0); + // Finish the next operator. + if (index + 1 < activeOperators.size()) { + Operator newRootOperator = activeOperators.get(index + 1); newRootOperator.finish(); } - break; + return index; } } + return minIndex - 1; } public void cancel(String reason) { diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/lookup/EnrichQuerySourceOperator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/lookup/EnrichQuerySourceOperator.java index 0cd34d2ad4066..214e7197b2c84 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/lookup/EnrichQuerySourceOperator.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/lookup/EnrichQuerySourceOperator.java @@ -21,6 +21,8 @@ import org.elasticsearch.compute.data.IntBlock; import org.elasticsearch.compute.data.IntVector; import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.lucene.ShardContext; +import org.elasticsearch.compute.lucene.ShardRefCounted; import org.elasticsearch.compute.operator.SourceOperator; import org.elasticsearch.compute.operator.Warnings; import org.elasticsearch.core.Releasables; @@ -37,6 +39,7 @@ public final class EnrichQuerySourceOperator extends SourceOperator { private final BlockFactory blockFactory; private final QueryList queryList; private int queryPosition = -1; + private final ShardContext shardContext; private final IndexReader indexReader; private final IndexSearcher searcher; private final Warnings warnings; @@ -49,14 +52,16 @@ public EnrichQuerySourceOperator( BlockFactory blockFactory, int maxPageSize, QueryList queryList, - IndexReader indexReader, + ShardContext shardContext, Warnings warnings ) { this.blockFactory = blockFactory; this.maxPageSize = maxPageSize; this.queryList = queryList; - this.indexReader = indexReader; - this.searcher = new IndexSearcher(indexReader); + this.shardContext = shardContext; + this.shardContext.incRef(); + this.searcher = shardContext.searcher(); + this.indexReader = searcher.getIndexReader(); this.warnings = warnings; } @@ -142,7 +147,10 @@ Page buildPage(int positions, IntVector.Builder positionsBuilder, IntVector.Buil segmentsVector = segmentsBuilder.build(); } docsVector = docsBuilder.build(); - page = new Page(new DocVector(shardsVector, segmentsVector, docsVector, null).asBlock(), positionsVector.asBlock()); + page = new Page( + new DocVector(ShardRefCounted.fromShardContext(shardContext), shardsVector, segmentsVector, docsVector, null).asBlock(), + positionsVector.asBlock() + ); } finally { if (page == null) { Releasables.close(positionsBuilder, segmentsVector, docsBuilder, positionsVector, shardsVector, docsVector); @@ -185,6 +193,6 @@ public void collect(int doc) { @Override public void close() { - + this.shardContext.decRef(); } } diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/topn/ResultBuilderForDoc.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/topn/ResultBuilderForDoc.java index 779e1dece2b33..c388692810593 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/topn/ResultBuilderForDoc.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/topn/ResultBuilderForDoc.java @@ -12,14 +12,22 @@ import org.elasticsearch.compute.data.BlockFactory; import org.elasticsearch.compute.data.DocVector; import org.elasticsearch.compute.data.IntVector; +import org.elasticsearch.compute.lucene.ShardRefCounted; +import org.elasticsearch.core.Nullable; +import org.elasticsearch.core.RefCounted; import org.elasticsearch.core.Releasables; +import java.util.HashMap; +import java.util.Map; + class ResultBuilderForDoc implements ResultBuilder { private final BlockFactory blockFactory; private final int[] shards; private final int[] segments; private final int[] docs; private int position; + private @Nullable RefCounted nextRefCounted; + private final Map refCounted = new HashMap<>(); ResultBuilderForDoc(BlockFactory blockFactory, int positions) { // TODO use fixed length builders @@ -34,12 +42,23 @@ public void decodeKey(BytesRef keys) { throw new AssertionError("_doc can't be a key"); } + void setNextRefCounted(RefCounted nextRefCounted) { + this.nextRefCounted = nextRefCounted; + // Since rows can be closed before build is called, we need to increment the ref count to ensure the shard context isn't closed. + this.nextRefCounted.mustIncRef(); + } + @Override public void decodeValue(BytesRef values) { + if (nextRefCounted == null) { + throw new IllegalStateException("setNextRefCounted must be set before each decodeValue call"); + } shards[position] = TopNEncoder.DEFAULT_UNSORTABLE.decodeInt(values); segments[position] = TopNEncoder.DEFAULT_UNSORTABLE.decodeInt(values); docs[position] = TopNEncoder.DEFAULT_UNSORTABLE.decodeInt(values); + refCounted.putIfAbsent(shards[position], nextRefCounted); position++; + nextRefCounted = null; } @Override @@ -51,16 +70,26 @@ public Block build() { shardsVector = blockFactory.newIntArrayVector(shards, position); segmentsVector = blockFactory.newIntArrayVector(segments, position); var docsVector = blockFactory.newIntArrayVector(docs, position); - var docsBlock = new DocVector(shardsVector, segmentsVector, docsVector, null).asBlock(); + var docsBlock = new DocVector(new ShardRefCountedMap(refCounted), shardsVector, segmentsVector, docsVector, null).asBlock(); success = true; return docsBlock; } finally { + // The DocVector constructor already incremented the relevant RefCounted, so we can now decrement them since we incremented them + // in setNextRefCounted. + refCounted.values().forEach(RefCounted::decRef); if (success == false) { Releasables.closeExpectNoException(shardsVector, segmentsVector); } } } + private record ShardRefCountedMap(Map refCounters) implements ShardRefCounted { + @Override + public RefCounted get(int shardId) { + return refCounters.get(shardId); + } + } + @Override public String toString() { return "ValueExtractorForDoc"; diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/topn/TopNOperator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/topn/TopNOperator.java index 0489be58fade7..412500f6d6857 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/topn/TopNOperator.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/topn/TopNOperator.java @@ -15,11 +15,14 @@ import org.elasticsearch.common.collect.Iterators; import org.elasticsearch.compute.data.Block; import org.elasticsearch.compute.data.BlockFactory; +import org.elasticsearch.compute.data.DocVector; import org.elasticsearch.compute.data.ElementType; import org.elasticsearch.compute.data.Page; import org.elasticsearch.compute.operator.BreakingBytesRefBuilder; import org.elasticsearch.compute.operator.DriverContext; import org.elasticsearch.compute.operator.Operator; +import org.elasticsearch.core.Nullable; +import org.elasticsearch.core.RefCounted; import org.elasticsearch.core.Releasable; import org.elasticsearch.core.Releasables; @@ -71,6 +74,21 @@ static final class Row implements Accountable, Releasable { */ final BreakingBytesRefBuilder values; + /** + * Reference counter for the shard this row belongs to, used for rows containing a {@link DocVector} to ensure that the shard + * context before we build the final result. + */ + @Nullable + RefCounted shardRefCounter; + + void setShardRefCountersAndShard(RefCounted shardRefCounter) { + if (this.shardRefCounter != null) { + this.shardRefCounter.decRef(); + } + this.shardRefCounter = shardRefCounter; + this.shardRefCounter.mustIncRef(); + } + Row(CircuitBreaker breaker, List sortOrders, int preAllocatedKeysSize, int preAllocatedValueSize) { boolean success = false; try { @@ -92,8 +110,16 @@ public long ramBytesUsed() { @Override public void close() { + clearRefCounters(); Releasables.closeExpectNoException(keys, values, bytesOrder); } + + public void clearRefCounters() { + if (shardRefCounter != null) { + shardRefCounter.decRef(); + } + shardRefCounter = null; + } } static final class BytesOrder implements Releasable, Accountable { @@ -174,7 +200,7 @@ static final class RowFiller { */ void row(int position, Row destination) { writeKey(position, destination); - writeValues(position, destination.values); + writeValues(position, destination); } private void writeKey(int position, Row row) { @@ -187,9 +213,12 @@ private void writeKey(int position, Row row) { } } - private void writeValues(int position, BreakingBytesRefBuilder values) { + private void writeValues(int position, Row destination) { for (ValueExtractor e : valueExtractors) { - e.writeValue(values, position); + if (e instanceof ValueExtractorForDoc fd) { + destination.setShardRefCountersAndShard(fd.vector().shardRefCounters().get(fd.vector().shards().getInt(position))); + } + e.writeValue(destination.values, position); } } } @@ -376,6 +405,7 @@ public void addInput(Page page) { } else { spare.keys.clear(); spare.values.clear(); + spare.clearRefCounters(); } rowFiller.row(i, spare); @@ -456,6 +486,10 @@ private Iterator toPages() { BytesRef values = row.values.bytesRefView(); for (ResultBuilder builder : builders) { + if (builder instanceof ResultBuilderForDoc fd) { + assert row.shardRefCounter != null : "shardRefCounter must be set for ResultBuilderForDoc"; + fd.setNextRefCounted(row.shardRefCounter); + } builder.decodeValue(values); } if (values.length != 0) { @@ -463,7 +497,6 @@ private Iterator toPages() { } list.set(i, null); - row.close(); p++; if (p == size) { @@ -481,6 +514,8 @@ private Iterator toPages() { Releasables.closeExpectNoException(builders); builders = null; } + // It's important to close the row only after we build the new block, so we don't pre-release any shard counter. + row.close(); } assert builders == null; success = true; diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/topn/ValueExtractorForDoc.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/topn/ValueExtractorForDoc.java index b6fc30e221cd7..db0985275080d 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/topn/ValueExtractorForDoc.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/topn/ValueExtractorForDoc.java @@ -18,6 +18,10 @@ class ValueExtractorForDoc implements ValueExtractor { this.vector = vector; } + DocVector vector() { + return vector; + } + @Override public void writeValue(BreakingBytesRefBuilder values, int position) { TopNEncoder.DEFAULT_UNSORTABLE.encodeInt(vector.shards().getInt(position), values); diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/CountDistinctLongGroupingAggregatorFunctionTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/CountDistinctLongGroupingAggregatorFunctionTests.java index db08fd0428e7b..d3b374a4d487f 100644 --- a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/CountDistinctLongGroupingAggregatorFunctionTests.java +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/CountDistinctLongGroupingAggregatorFunctionTests.java @@ -14,7 +14,7 @@ import org.elasticsearch.compute.data.LongVector; import org.elasticsearch.compute.data.Page; import org.elasticsearch.compute.operator.SourceOperator; -import org.elasticsearch.compute.operator.TupleBlockSourceOperator; +import org.elasticsearch.compute.operator.TupleLongLongBlockSourceOperator; import org.elasticsearch.core.Tuple; import java.util.List; @@ -36,7 +36,7 @@ protected String expectedDescriptionOfAggregator() { @Override protected SourceOperator simpleInput(BlockFactory blockFactory, int size) { - return new TupleBlockSourceOperator( + return new TupleLongLongBlockSourceOperator( blockFactory, LongStream.range(0, size).mapToObj(l -> Tuple.tuple(randomGroupId(size), randomLongBetween(0, 100_000))) ); diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/CountGroupingAggregatorFunctionTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/CountGroupingAggregatorFunctionTests.java index 06a0666586290..d0dcf39029d83 100644 --- a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/CountGroupingAggregatorFunctionTests.java +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/CountGroupingAggregatorFunctionTests.java @@ -15,7 +15,7 @@ import org.elasticsearch.compute.data.Page; import org.elasticsearch.compute.operator.LongDoubleTupleBlockSourceOperator; import org.elasticsearch.compute.operator.SourceOperator; -import org.elasticsearch.compute.operator.TupleBlockSourceOperator; +import org.elasticsearch.compute.operator.TupleLongLongBlockSourceOperator; import org.elasticsearch.core.Tuple; import java.util.List; @@ -37,7 +37,7 @@ protected String expectedDescriptionOfAggregator() { @Override protected SourceOperator simpleInput(BlockFactory blockFactory, int size) { if (randomBoolean()) { - return new TupleBlockSourceOperator( + return new TupleLongLongBlockSourceOperator( blockFactory, LongStream.range(0, size).mapToObj(l -> Tuple.tuple(randomLongBetween(0, 4), randomLong())) ); diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/MaxLongGroupingAggregatorFunctionTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/MaxLongGroupingAggregatorFunctionTests.java index 6d6c37fb306a0..b6223e36597d5 100644 --- a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/MaxLongGroupingAggregatorFunctionTests.java +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/MaxLongGroupingAggregatorFunctionTests.java @@ -12,7 +12,7 @@ import org.elasticsearch.compute.data.LongBlock; import org.elasticsearch.compute.data.Page; import org.elasticsearch.compute.operator.SourceOperator; -import org.elasticsearch.compute.operator.TupleBlockSourceOperator; +import org.elasticsearch.compute.operator.TupleLongLongBlockSourceOperator; import org.elasticsearch.core.Tuple; import java.util.List; @@ -34,7 +34,7 @@ protected String expectedDescriptionOfAggregator() { @Override protected SourceOperator simpleInput(BlockFactory blockFactory, int size) { - return new TupleBlockSourceOperator( + return new TupleLongLongBlockSourceOperator( blockFactory, LongStream.range(0, size).mapToObj(l -> Tuple.tuple(randomLongBetween(0, 4), randomLong())) ); diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/MedianAbsoluteDeviationLongGroupingAggregatorFunctionTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/MedianAbsoluteDeviationLongGroupingAggregatorFunctionTests.java index 55895ceadd52c..fbd41d8ab06be 100644 --- a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/MedianAbsoluteDeviationLongGroupingAggregatorFunctionTests.java +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/MedianAbsoluteDeviationLongGroupingAggregatorFunctionTests.java @@ -13,7 +13,7 @@ import org.elasticsearch.compute.data.DoubleBlock; import org.elasticsearch.compute.data.Page; import org.elasticsearch.compute.operator.SourceOperator; -import org.elasticsearch.compute.operator.TupleBlockSourceOperator; +import org.elasticsearch.compute.operator.TupleLongLongBlockSourceOperator; import org.elasticsearch.core.Tuple; import java.util.ArrayList; @@ -42,7 +42,7 @@ protected SourceOperator simpleInput(BlockFactory blockFactory, int end) { values.add(Tuple.tuple((long) i, v)); } } - return new TupleBlockSourceOperator(blockFactory, values.subList(0, Math.min(values.size(), end))); + return new TupleLongLongBlockSourceOperator(blockFactory, values.subList(0, Math.min(values.size(), end))); } @Override diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/MinLongGroupingAggregatorFunctionTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/MinLongGroupingAggregatorFunctionTests.java index da8a63a429200..82095553fdd58 100644 --- a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/MinLongGroupingAggregatorFunctionTests.java +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/MinLongGroupingAggregatorFunctionTests.java @@ -12,7 +12,7 @@ import org.elasticsearch.compute.data.LongBlock; import org.elasticsearch.compute.data.Page; import org.elasticsearch.compute.operator.SourceOperator; -import org.elasticsearch.compute.operator.TupleBlockSourceOperator; +import org.elasticsearch.compute.operator.TupleLongLongBlockSourceOperator; import org.elasticsearch.core.Tuple; import java.util.List; @@ -34,7 +34,7 @@ protected String expectedDescriptionOfAggregator() { @Override protected SourceOperator simpleInput(BlockFactory blockFactory, int size) { - return new TupleBlockSourceOperator( + return new TupleLongLongBlockSourceOperator( blockFactory, LongStream.range(0, size).mapToObj(l -> Tuple.tuple(randomLongBetween(0, 4), randomLong())) ); diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/PercentileLongGroupingAggregatorFunctionTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/PercentileLongGroupingAggregatorFunctionTests.java index 55065129df0ce..74f6b20a9f9fb 100644 --- a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/PercentileLongGroupingAggregatorFunctionTests.java +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/PercentileLongGroupingAggregatorFunctionTests.java @@ -13,7 +13,7 @@ import org.elasticsearch.compute.data.DoubleBlock; import org.elasticsearch.compute.data.Page; import org.elasticsearch.compute.operator.SourceOperator; -import org.elasticsearch.compute.operator.TupleBlockSourceOperator; +import org.elasticsearch.compute.operator.TupleLongLongBlockSourceOperator; import org.elasticsearch.core.Tuple; import org.elasticsearch.search.aggregations.metrics.TDigestState; import org.junit.Before; @@ -45,7 +45,7 @@ protected String expectedDescriptionOfAggregator() { @Override protected SourceOperator simpleInput(BlockFactory blockFactory, int size) { long max = randomLongBetween(1, Long.MAX_VALUE / size / 5); - return new TupleBlockSourceOperator( + return new TupleLongLongBlockSourceOperator( blockFactory, LongStream.range(0, size).mapToObj(l -> Tuple.tuple(randomLongBetween(0, 4), randomLongBetween(-0, max))) ); diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/SumLongGroupingAggregatorFunctionTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/SumLongGroupingAggregatorFunctionTests.java index f289686f8e844..f39df0071aab2 100644 --- a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/SumLongGroupingAggregatorFunctionTests.java +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/SumLongGroupingAggregatorFunctionTests.java @@ -12,7 +12,7 @@ import org.elasticsearch.compute.data.LongBlock; import org.elasticsearch.compute.data.Page; import org.elasticsearch.compute.operator.SourceOperator; -import org.elasticsearch.compute.operator.TupleBlockSourceOperator; +import org.elasticsearch.compute.operator.TupleLongLongBlockSourceOperator; import org.elasticsearch.core.Tuple; import java.util.List; @@ -34,7 +34,7 @@ protected String expectedDescriptionOfAggregator() { @Override protected SourceOperator simpleInput(BlockFactory blockFactory, int size) { long max = randomLongBetween(1, Long.MAX_VALUE / size / 5); - return new TupleBlockSourceOperator( + return new TupleLongLongBlockSourceOperator( blockFactory, LongStream.range(0, size).mapToObj(l -> Tuple.tuple(randomLongBetween(0, 4), randomLongBetween(-max, max))) ); diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/ValuesLongGroupingAggregatorFunctionTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/ValuesLongGroupingAggregatorFunctionTests.java index 3180ac53f6efc..bb00541f24fe5 100644 --- a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/ValuesLongGroupingAggregatorFunctionTests.java +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/aggregation/ValuesLongGroupingAggregatorFunctionTests.java @@ -12,7 +12,7 @@ import org.elasticsearch.compute.data.BlockUtils; import org.elasticsearch.compute.data.Page; import org.elasticsearch.compute.operator.SourceOperator; -import org.elasticsearch.compute.operator.TupleBlockSourceOperator; +import org.elasticsearch.compute.operator.TupleLongLongBlockSourceOperator; import org.elasticsearch.core.Tuple; import java.util.Arrays; @@ -38,7 +38,7 @@ protected String expectedDescriptionOfAggregator() { @Override protected SourceOperator simpleInput(BlockFactory blockFactory, int size) { - return new TupleBlockSourceOperator( + return new TupleLongLongBlockSourceOperator( blockFactory, LongStream.range(0, size).mapToObj(l -> Tuple.tuple(randomLongBetween(0, 4), randomLong())) ); diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/BasicBlockTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/BasicBlockTests.java index 7192146939ec5..d077d8d2160b1 100644 --- a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/BasicBlockTests.java +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/BasicBlockTests.java @@ -18,6 +18,7 @@ import org.elasticsearch.common.util.LongArray; import org.elasticsearch.common.util.MockBigArrays; import org.elasticsearch.common.util.PageCacheRecycler; +import org.elasticsearch.compute.lucene.ShardRefCounted; import org.elasticsearch.compute.test.BlockTestUtils; import org.elasticsearch.compute.test.TestBlockFactory; import org.elasticsearch.core.RefCounted; @@ -1394,7 +1395,13 @@ public void testRefCountingConstantNullBlock() { public void testRefCountingDocBlock() { int positionCount = randomIntBetween(0, 100); - DocBlock block = new DocVector(intVector(positionCount), intVector(positionCount), intVector(positionCount), true).asBlock(); + DocBlock block = new DocVector( + ShardRefCounted.ALWAYS_REFERENCED, + intVector(positionCount), + intVector(positionCount), + intVector(positionCount), + true + ).asBlock(); assertThat(breaker.getUsed(), greaterThan(0L)); assertRefCountingBehavior(block); assertThat(breaker.getUsed(), is(0L)); @@ -1430,7 +1437,13 @@ public void testRefCountingConstantVector() { public void testRefCountingDocVector() { int positionCount = randomIntBetween(0, 100); - DocVector vector = new DocVector(intVector(positionCount), intVector(positionCount), intVector(positionCount), true); + DocVector vector = new DocVector( + ShardRefCounted.ALWAYS_REFERENCED, + intVector(positionCount), + intVector(positionCount), + intVector(positionCount), + true + ); assertThat(breaker.getUsed(), greaterThan(0L)); assertRefCountingBehavior(vector); assertThat(breaker.getUsed(), is(0L)); diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/DocVectorTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/DocVectorTests.java index 78192d6363d48..59520a25c5235 100644 --- a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/DocVectorTests.java +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/data/DocVectorTests.java @@ -10,6 +10,7 @@ import org.elasticsearch.common.Randomness; import org.elasticsearch.common.breaker.CircuitBreakingException; import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.compute.lucene.ShardRefCounted; import org.elasticsearch.compute.test.ComputeTestCase; import org.elasticsearch.compute.test.TestBlockFactory; import org.elasticsearch.core.Releasables; @@ -28,27 +29,51 @@ public class DocVectorTests extends ComputeTestCase { public void testNonDecreasingSetTrue() { int length = between(1, 100); - DocVector docs = new DocVector(intRange(0, length), intRange(0, length), intRange(0, length), true); + DocVector docs = new DocVector( + ShardRefCounted.ALWAYS_REFERENCED, + intRange(0, length), + intRange(0, length), + intRange(0, length), + true + ); assertTrue(docs.singleSegmentNonDecreasing()); } public void testNonDecreasingSetFalse() { BlockFactory blockFactory = blockFactory(); - DocVector docs = new DocVector(intRange(0, 2), intRange(0, 2), blockFactory.newIntArrayVector(new int[] { 1, 0 }, 2), false); + DocVector docs = new DocVector( + ShardRefCounted.ALWAYS_REFERENCED, + intRange(0, 2), + intRange(0, 2), + blockFactory.newIntArrayVector(new int[] { 1, 0 }, 2), + false + ); assertFalse(docs.singleSegmentNonDecreasing()); docs.close(); } public void testNonDecreasingNonConstantShard() { BlockFactory blockFactory = blockFactory(); - DocVector docs = new DocVector(intRange(0, 2), blockFactory.newConstantIntVector(0, 2), intRange(0, 2), null); + DocVector docs = new DocVector( + ShardRefCounted.ALWAYS_REFERENCED, + intRange(0, 2), + blockFactory.newConstantIntVector(0, 2), + intRange(0, 2), + null + ); assertFalse(docs.singleSegmentNonDecreasing()); docs.close(); } public void testNonDecreasingNonConstantSegment() { BlockFactory blockFactory = blockFactory(); - DocVector docs = new DocVector(blockFactory.newConstantIntVector(0, 2), intRange(0, 2), intRange(0, 2), null); + DocVector docs = new DocVector( + ShardRefCounted.ALWAYS_REFERENCED, + blockFactory.newConstantIntVector(0, 2), + intRange(0, 2), + intRange(0, 2), + null + ); assertFalse(docs.singleSegmentNonDecreasing()); docs.close(); } @@ -56,6 +81,7 @@ public void testNonDecreasingNonConstantSegment() { public void testNonDecreasingDescendingDocs() { BlockFactory blockFactory = blockFactory(); DocVector docs = new DocVector( + ShardRefCounted.ALWAYS_REFERENCED, blockFactory.newConstantIntVector(0, 2), blockFactory.newConstantIntVector(0, 2), blockFactory.newIntArrayVector(new int[] { 1, 0 }, 2), @@ -209,7 +235,13 @@ private void testShardSegmentDocMapBreaks(BlockFactory blockFactory) { public void testCannotDoubleRelease() { BlockFactory blockFactory = blockFactory(); - var block = new DocVector(intRange(0, 2), blockFactory.newConstantIntBlockWith(0, 2).asVector(), intRange(0, 2), null).asBlock(); + var block = new DocVector( + ShardRefCounted.ALWAYS_REFERENCED, + intRange(0, 2), + blockFactory.newConstantIntBlockWith(0, 2).asVector(), + intRange(0, 2), + null + ).asBlock(); assertThat(block.isReleased(), is(false)); Page page = new Page(block); @@ -229,6 +261,7 @@ public void testCannotDoubleRelease() { public void testRamBytesUsedWithout() { BlockFactory blockFactory = blockFactory(); DocVector docs = new DocVector( + ShardRefCounted.ALWAYS_REFERENCED, blockFactory.newConstantIntBlockWith(0, 1).asVector(), blockFactory.newConstantIntBlockWith(0, 1).asVector(), blockFactory.newConstantIntBlockWith(0, 1).asVector(), @@ -243,6 +276,7 @@ public void testFilter() { BlockFactory factory = blockFactory(); try ( DocVector docs = new DocVector( + ShardRefCounted.ALWAYS_REFERENCED, factory.newConstantIntVector(0, 10), factory.newConstantIntVector(0, 10), factory.newIntArrayVector(new int[] { 0, 1, 2, 3, 4, 5, 6, 7, 8, 9 }, 10), @@ -250,6 +284,7 @@ public void testFilter() { ); DocVector filtered = docs.filter(1, 2, 3); DocVector expected = new DocVector( + ShardRefCounted.ALWAYS_REFERENCED, factory.newConstantIntVector(0, 3), factory.newConstantIntVector(0, 3), factory.newIntArrayVector(new int[] { 1, 2, 3 }, 3), @@ -270,7 +305,7 @@ public void testFilterBreaks() { shards = factory.newConstantIntVector(0, 10); segments = factory.newConstantIntVector(0, 10); docs = factory.newConstantIntVector(0, 10); - result = new DocVector(shards, segments, docs, false); + result = new DocVector(ShardRefCounted.ALWAYS_REFERENCED, shards, segments, docs, false); return result; } finally { if (result == null) { diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/LuceneQueryEvaluatorTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/LuceneQueryEvaluatorTests.java index eb7cb32fd0e74..4828f70e51dcd 100644 --- a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/LuceneQueryEvaluatorTests.java +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/LuceneQueryEvaluatorTests.java @@ -191,6 +191,7 @@ private List runQuery(Set values, Query query, boolean shuffleDocs BlockFactory blockFactory = driverContext.blockFactory(); return withReader(values, reader -> { IndexSearcher searcher = new IndexSearcher(reader); + var shardContext = new LuceneSourceOperatorTests.MockShardContext(reader, 0); LuceneQueryEvaluator.ShardConfig shard = new LuceneQueryEvaluator.ShardConfig(searcher.rewrite(query), searcher); List operators = new ArrayList<>(); if (shuffleDocs) { diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/LuceneSourceOperatorTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/LuceneSourceOperatorTests.java index 4c5c860d244a0..a8cb202f2be2c 100644 --- a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/LuceneSourceOperatorTests.java +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/lucene/LuceneSourceOperatorTests.java @@ -405,6 +405,11 @@ public static class MockShardContext implements ShardContext { private final int index; private final ContextIndexSearcher searcher; + // TODO Reuse this overload in the places that pass 0. + public MockShardContext(IndexReader reader) { + this(reader, 0); + } + public MockShardContext(IndexReader reader, int index) { this.index = index; try { @@ -458,5 +463,22 @@ public String shardIdentifier() { public MappedFieldType fieldType(String name) { throw new UnsupportedOperationException(); } + + public void incRef() {} + + @Override + public boolean tryIncRef() { + return true; + } + + @Override + public boolean decRef() { + return false; + } + + @Override + public boolean hasReferences() { + return true; + } } } diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/DriverContextTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/DriverContextTests.java index 563e88ab4eeb1..29ec46bc34403 100644 --- a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/DriverContextTests.java +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/DriverContextTests.java @@ -13,6 +13,7 @@ import org.elasticsearch.common.util.MockBigArrays; import org.elasticsearch.common.util.PageCacheRecycler; import org.elasticsearch.common.util.concurrent.EsExecutors; +import org.elasticsearch.compute.test.NoOpReleasable; import org.elasticsearch.compute.test.TestBlockFactory; import org.elasticsearch.core.Releasable; import org.elasticsearch.core.Releasables; @@ -267,14 +268,6 @@ record ReleasablePoint(int x, int y) implements Releasable { public void close() {} } - static class NoOpReleasable implements Releasable { - - @Override - public void close() { - // no-op - } - } - static class CheckableReleasable implements Releasable { boolean closed; diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/EvalOperatorTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/EvalOperatorTests.java index 544541ef49d29..189ccdb402f94 100644 --- a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/EvalOperatorTests.java +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/EvalOperatorTests.java @@ -30,7 +30,7 @@ public class EvalOperatorTests extends OperatorTestCase { @Override protected SourceOperator simpleInput(BlockFactory blockFactory, int end) { - return new TupleBlockSourceOperator(blockFactory, LongStream.range(0, end).mapToObj(l -> Tuple.tuple(l, end - l))); + return new TupleLongLongBlockSourceOperator(blockFactory, LongStream.range(0, end).mapToObj(l -> Tuple.tuple(l, end - l))); } record Addition(DriverContext driverContext, int lhs, int rhs) implements EvalOperator.ExpressionEvaluator { diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/FilterOperatorTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/FilterOperatorTests.java index a0de030bf4c97..fb1f7b5422308 100644 --- a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/FilterOperatorTests.java +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/FilterOperatorTests.java @@ -29,7 +29,7 @@ public class FilterOperatorTests extends OperatorTestCase { @Override protected SourceOperator simpleInput(BlockFactory blockFactory, int end) { - return new TupleBlockSourceOperator(blockFactory, LongStream.range(0, end).mapToObj(l -> Tuple.tuple(l, end - l))); + return new TupleLongLongBlockSourceOperator(blockFactory, LongStream.range(0, end).mapToObj(l -> Tuple.tuple(l, end - l))); } record SameLastDigit(DriverContext context, int lhs, int rhs) implements EvalOperator.ExpressionEvaluator { diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/HashAggregationOperatorTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/HashAggregationOperatorTests.java index ec84d17045af4..106b9613d7bb2 100644 --- a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/HashAggregationOperatorTests.java +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/HashAggregationOperatorTests.java @@ -38,7 +38,7 @@ public class HashAggregationOperatorTests extends ForkingOperatorTestCase { @Override protected SourceOperator simpleInput(BlockFactory blockFactory, int size) { long max = randomLongBetween(1, Long.MAX_VALUE / size); - return new TupleBlockSourceOperator( + return new TupleLongLongBlockSourceOperator( blockFactory, LongStream.range(0, size).mapToObj(l -> Tuple.tuple(l % 5, randomLongBetween(-max, max))) ); diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/ProjectOperatorTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/ProjectOperatorTests.java index de32b51f93edb..88b664533dbbb 100644 --- a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/ProjectOperatorTests.java +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/ProjectOperatorTests.java @@ -62,7 +62,7 @@ private List randomProjection(int size) { @Override protected SourceOperator simpleInput(BlockFactory blockFactory, int end) { - return new TupleBlockSourceOperator(blockFactory, LongStream.range(0, end).mapToObj(l -> Tuple.tuple(l, end - l))); + return new TupleLongLongBlockSourceOperator(blockFactory, LongStream.range(0, end).mapToObj(l -> Tuple.tuple(l, end - l))); } @Override diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/RowInTableLookupOperatorTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/RowInTableLookupOperatorTests.java index 63f8239073c2a..441d125c5608a 100644 --- a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/RowInTableLookupOperatorTests.java +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/RowInTableLookupOperatorTests.java @@ -105,7 +105,7 @@ protected Matcher expectedToStringOfSimple() { public void testSelectBlocks() { DriverContext context = driverContext(); List input = CannedSourceOperator.collectPages( - new TupleBlockSourceOperator( + new TupleLongLongBlockSourceOperator( context.blockFactory(), LongStream.range(0, 1000).mapToObj(l -> Tuple.tuple(randomLong(), randomFrom(1L, 7L, 14L, 20L))) ) diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/ShuffleDocsOperator.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/ShuffleDocsOperator.java index 955d0237c65f7..2f0f86ee19ad9 100644 --- a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/ShuffleDocsOperator.java +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/ShuffleDocsOperator.java @@ -12,6 +12,7 @@ import org.elasticsearch.compute.data.DocVector; import org.elasticsearch.compute.data.IntVector; import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.lucene.ShardRefCounted; import org.elasticsearch.core.Releasables; import java.util.ArrayList; @@ -60,7 +61,7 @@ protected Page process(Page page) { } } Block[] blocks = new Block[page.getBlockCount()]; - blocks[0] = new DocVector(shards, segments, docs, false).asBlock(); + blocks[0] = new DocVector(ShardRefCounted.ALWAYS_REFERENCED, shards, segments, docs, false).asBlock(); for (int i = 1; i < blocks.length; i++) { blocks[i] = page.getBlock(i); } diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/TupleAbstractBlockSourceOperator.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/TupleAbstractBlockSourceOperator.java new file mode 100644 index 0000000000000..739c54e6e8eec --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/TupleAbstractBlockSourceOperator.java @@ -0,0 +1,97 @@ +/* + * 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.compute.operator; + +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BlockFactory; +import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.test.AbstractBlockSourceOperator; +import org.elasticsearch.core.Tuple; + +import java.util.List; + +/** + * A source operator whose output is the given tuple values. This operator produces pages + * with two Blocks. The returned pages preserve the order of values as given in the in initial list. + */ +public abstract class TupleAbstractBlockSourceOperator extends AbstractBlockSourceOperator { + private static final int DEFAULT_MAX_PAGE_POSITIONS = 8 * 1024; + + private final List> values; + private final ElementType firstElementType; + private final ElementType secondElementType; + + public TupleAbstractBlockSourceOperator( + BlockFactory blockFactory, + List> values, + ElementType firstElementType, + ElementType secondElementType + ) { + this(blockFactory, values, DEFAULT_MAX_PAGE_POSITIONS, firstElementType, secondElementType); + } + + public TupleAbstractBlockSourceOperator( + BlockFactory blockFactory, + List> values, + int maxPagePositions, + ElementType firstElementType, + ElementType secondElementType + ) { + super(blockFactory, maxPagePositions); + this.values = values; + this.firstElementType = firstElementType; + this.secondElementType = secondElementType; + } + + @Override + protected Page createPage(int positionOffset, int length) { + try (var blockBuilder1 = firstElementBlockBuilder(length); var blockBuilder2 = secondElementBlockBuilder(length)) { + for (int i = 0; i < length; i++) { + Tuple item = values.get(positionOffset + i); + if (item.v1() == null) { + blockBuilder1.appendNull(); + } else { + consumeFirstElement(item.v1(), blockBuilder1); + } + if (item.v2() == null) { + blockBuilder2.appendNull(); + } else { + consumeSecondElement(item.v2(), blockBuilder2); + } + } + currentPosition += length; + return new Page(Block.Builder.buildAll(blockBuilder1, blockBuilder2)); + } + } + + protected abstract void consumeFirstElement(T t, Block.Builder blockBuilder1); + + protected Block.Builder firstElementBlockBuilder(int length) { + return firstElementType.newBlockBuilder(length, blockFactory); + } + + protected Block.Builder secondElementBlockBuilder(int length) { + return secondElementType.newBlockBuilder(length, blockFactory); + } + + protected abstract void consumeSecondElement(S t, Block.Builder blockBuilder1); + + @Override + protected int remaining() { + return values.size() - currentPosition; + } + + public List elementTypes() { + return List.of(firstElementType, secondElementType); + } + + public List> values() { + return values; + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/TupleBlockSourceOperator.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/TupleBlockSourceOperator.java deleted file mode 100644 index b905de17608cb..0000000000000 --- a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/TupleBlockSourceOperator.java +++ /dev/null @@ -1,72 +0,0 @@ -/* - * 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.compute.operator; - -import org.elasticsearch.compute.data.Block; -import org.elasticsearch.compute.data.BlockFactory; -import org.elasticsearch.compute.data.Page; -import org.elasticsearch.compute.test.AbstractBlockSourceOperator; -import org.elasticsearch.core.Tuple; - -import java.util.List; -import java.util.stream.Stream; - -/** - * A source operator whose output is the given tuple values. This operator produces pages - * with two Blocks. The returned pages preserve the order of values as given in the in initial list. - */ -public class TupleBlockSourceOperator extends AbstractBlockSourceOperator { - - private static final int DEFAULT_MAX_PAGE_POSITIONS = 8 * 1024; - - private final List> values; - - public TupleBlockSourceOperator(BlockFactory blockFactory, Stream> values) { - this(blockFactory, values, DEFAULT_MAX_PAGE_POSITIONS); - } - - public TupleBlockSourceOperator(BlockFactory blockFactory, Stream> values, int maxPagePositions) { - super(blockFactory, maxPagePositions); - this.values = values.toList(); - } - - public TupleBlockSourceOperator(BlockFactory blockFactory, List> values) { - this(blockFactory, values, DEFAULT_MAX_PAGE_POSITIONS); - } - - public TupleBlockSourceOperator(BlockFactory blockFactory, List> values, int maxPagePositions) { - super(blockFactory, maxPagePositions); - this.values = values; - } - - @Override - protected Page createPage(int positionOffset, int length) { - try (var blockBuilder1 = blockFactory.newLongBlockBuilder(length); var blockBuilder2 = blockFactory.newLongBlockBuilder(length)) { - for (int i = 0; i < length; i++) { - Tuple item = values.get(positionOffset + i); - if (item.v1() == null) { - blockBuilder1.appendNull(); - } else { - blockBuilder1.appendLong(item.v1()); - } - if (item.v2() == null) { - blockBuilder2.appendNull(); - } else { - blockBuilder2.appendLong(item.v2()); - } - } - currentPosition += length; - return new Page(Block.Builder.buildAll(blockBuilder1, blockBuilder2)); - } - } - - @Override - protected int remaining() { - return values.size() - currentPosition; - } -} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/TupleDocLongBlockSourceOperator.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/TupleDocLongBlockSourceOperator.java new file mode 100644 index 0000000000000..26e84fe46d012 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/TupleDocLongBlockSourceOperator.java @@ -0,0 +1,47 @@ +/* + * 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.compute.operator; + +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BlockFactory; +import org.elasticsearch.compute.data.BlockUtils; +import org.elasticsearch.compute.data.DocBlock; +import org.elasticsearch.core.Tuple; +import org.elasticsearch.index.mapper.BlockLoader; + +import java.util.List; + +import static org.elasticsearch.compute.data.ElementType.DOC; +import static org.elasticsearch.compute.data.ElementType.LONG; + +/** + * A source operator whose output is the given tuple values. This operator produces pages + * with two Blocks. The returned pages preserve the order of values as given in the in initial list. + */ +public class TupleDocLongBlockSourceOperator extends TupleAbstractBlockSourceOperator { + public TupleDocLongBlockSourceOperator(BlockFactory blockFactory, List> values) { + super(blockFactory, values, DOC, LONG); + } + + public TupleDocLongBlockSourceOperator(BlockFactory blockFactory, List> values, int maxPagePositions) { + super(blockFactory, values, maxPagePositions, DOC, LONG); + } + + @Override + protected void consumeFirstElement(BlockUtils.Doc doc, Block.Builder builder) { + var docBuilder = (DocBlock.Builder) builder; + docBuilder.appendShard(doc.shard()); + docBuilder.appendSegment(doc.segment()); + docBuilder.appendDoc(doc.doc()); + } + + @Override + protected void consumeSecondElement(Long l, Block.Builder blockBuilder) { + ((BlockLoader.LongBuilder) blockBuilder).appendLong(l); + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/TupleLongLongBlockSourceOperator.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/TupleLongLongBlockSourceOperator.java new file mode 100644 index 0000000000000..ae5045f04c9b9 --- /dev/null +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/TupleLongLongBlockSourceOperator.java @@ -0,0 +1,51 @@ +/* + * 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.compute.operator; + +import org.elasticsearch.compute.data.Block; +import org.elasticsearch.compute.data.BlockFactory; +import org.elasticsearch.core.Tuple; +import org.elasticsearch.index.mapper.BlockLoader; + +import java.util.List; +import java.util.stream.Stream; + +import static org.elasticsearch.compute.data.ElementType.LONG; + +/** + * A source operator whose output is the given tuple values. This operator produces pages + * with two Blocks. The returned pages preserve the order of values as given in the in initial list. + */ +public class TupleLongLongBlockSourceOperator extends TupleAbstractBlockSourceOperator { + + public TupleLongLongBlockSourceOperator(BlockFactory blockFactory, Stream> values) { + super(blockFactory, values.toList(), LONG, LONG); + } + + public TupleLongLongBlockSourceOperator(BlockFactory blockFactory, Stream> values, int maxPagePositions) { + super(blockFactory, values.toList(), maxPagePositions, LONG, LONG); + } + + public TupleLongLongBlockSourceOperator(BlockFactory blockFactory, List> values) { + super(blockFactory, values, LONG, LONG); + } + + public TupleLongLongBlockSourceOperator(BlockFactory blockFactory, List> values, int maxPagePositions) { + super(blockFactory, values, maxPagePositions, LONG, LONG); + } + + @Override + protected void consumeFirstElement(Long l, Block.Builder blockBuilder) { + ((BlockLoader.LongBuilder) blockBuilder).appendLong(l); + } + + @Override + protected void consumeSecondElement(Long l, Block.Builder blockBuilder) { + consumeFirstElement(l, blockBuilder); + } +} diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/lookup/EnrichQuerySourceOperatorTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/lookup/EnrichQuerySourceOperatorTests.java index 2aadb81a8b086..d1a3b408c41af 100644 --- a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/lookup/EnrichQuerySourceOperatorTests.java +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/lookup/EnrichQuerySourceOperatorTests.java @@ -32,6 +32,7 @@ import org.elasticsearch.compute.data.IntBlock; import org.elasticsearch.compute.data.IntVector; import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.lucene.LuceneSourceOperatorTests; import org.elasticsearch.compute.operator.DriverContext; import org.elasticsearch.compute.operator.Warnings; import org.elasticsearch.core.IOUtils; @@ -104,7 +105,8 @@ public void testQueries() throws Exception { blockFactory, 128, queryList, - directoryData.reader, + + new LuceneSourceOperatorTests.MockShardContext(directoryData.reader), warnings() ); Page page = queryOperator.getOutput(); @@ -165,7 +167,7 @@ public void testRandomMatchQueries() throws Exception { blockFactory, maxPageSize, queryList, - directoryData.reader, + new LuceneSourceOperatorTests.MockShardContext(directoryData.reader), warnings() ); Map> actualPositions = new HashMap<>(); @@ -214,7 +216,7 @@ public void testQueries_OnlySingleValues() throws Exception { blockFactory, 128, queryList, - directoryData.reader, + new LuceneSourceOperatorTests.MockShardContext(directoryData.reader), warnings() ); Page page = queryOperator.getOutput(); diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/topn/ExtractorTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/topn/ExtractorTests.java index 101c129e77201..0db2d82a09ac2 100644 --- a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/topn/ExtractorTests.java +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/topn/ExtractorTests.java @@ -18,9 +18,11 @@ import org.elasticsearch.compute.data.BlockUtils; import org.elasticsearch.compute.data.DocVector; import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.compute.lucene.ShardRefCounted; import org.elasticsearch.compute.operator.BreakingBytesRefBuilder; import org.elasticsearch.compute.test.BlockTestUtils; import org.elasticsearch.compute.test.TestBlockFactory; +import org.elasticsearch.core.RefCounted; import org.elasticsearch.test.ESTestCase; import java.util.ArrayList; @@ -94,6 +96,7 @@ public static Iterable parameters() { e, TopNEncoder.DEFAULT_UNSORTABLE, () -> new DocVector( + ShardRefCounted.ALWAYS_REFERENCED, blockFactory.newConstantIntBlockWith(randomInt(), 1).asVector(), blockFactory.newConstantIntBlockWith(randomInt(), 1).asVector(), blockFactory.newConstantIntBlockWith(randomInt(), 1).asVector(), @@ -172,6 +175,9 @@ public void testNotInKey() { 1 ); BytesRef values = valuesBuilder.bytesRefView(); + if (result instanceof ResultBuilderForDoc fd) { + fd.setNextRefCounted(RefCounted.ALWAYS_REFERENCED); + } result.decodeValue(values); assertThat(values.length, equalTo(0)); diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/topn/TopNOperatorTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/topn/TopNOperatorTests.java index 8561ce84744aa..1180cdca64569 100644 --- a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/topn/TopNOperatorTests.java +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/topn/TopNOperatorTests.java @@ -17,23 +17,30 @@ import org.elasticsearch.common.util.MockBigArrays; import org.elasticsearch.compute.data.Block; import org.elasticsearch.compute.data.BlockFactory; +import org.elasticsearch.compute.data.BlockUtils; +import org.elasticsearch.compute.data.DocBlock; import org.elasticsearch.compute.data.ElementType; import org.elasticsearch.compute.data.IntBlock; import org.elasticsearch.compute.data.LongBlock; import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.lucene.ShardRefCounted; import org.elasticsearch.compute.operator.CountingCircuitBreaker; import org.elasticsearch.compute.operator.Driver; import org.elasticsearch.compute.operator.DriverContext; import org.elasticsearch.compute.operator.Operator; import org.elasticsearch.compute.operator.PageConsumerOperator; import org.elasticsearch.compute.operator.SourceOperator; -import org.elasticsearch.compute.operator.TupleBlockSourceOperator; +import org.elasticsearch.compute.operator.TupleAbstractBlockSourceOperator; +import org.elasticsearch.compute.operator.TupleDocLongBlockSourceOperator; +import org.elasticsearch.compute.operator.TupleLongLongBlockSourceOperator; import org.elasticsearch.compute.test.CannedSourceOperator; import org.elasticsearch.compute.test.OperatorTestCase; import org.elasticsearch.compute.test.SequenceLongBlockSourceOperator; import org.elasticsearch.compute.test.TestBlockBuilder; import org.elasticsearch.compute.test.TestBlockFactory; import org.elasticsearch.compute.test.TestDriverFactory; +import org.elasticsearch.core.RefCounted; +import org.elasticsearch.core.SimpleRefCounted; import org.elasticsearch.core.Tuple; import org.elasticsearch.indices.CrankyCircuitBreakerService; import org.elasticsearch.test.ESTestCase; @@ -53,10 +60,12 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.function.BiFunction; import java.util.function.Function; import java.util.stream.Collectors; import java.util.stream.IntStream; import java.util.stream.LongStream; +import java.util.stream.Stream; import static java.util.Comparator.naturalOrder; import static java.util.Comparator.reverseOrder; @@ -289,16 +298,19 @@ private List topNLong( boolean ascendingOrder, boolean nullsFirst ) { - return topNTwoColumns( + return topNTwoLongColumns( driverContext, inputValues.stream().map(v -> tuple(v, 0L)).toList(), limit, - List.of(LONG, LONG), List.of(DEFAULT_UNSORTABLE, DEFAULT_UNSORTABLE), List.of(new TopNOperator.SortOrder(0, ascendingOrder, nullsFirst)) ).stream().map(Tuple::v1).toList(); } + private static TupleLongLongBlockSourceOperator longLongSourceOperator(DriverContext driverContext, List> values) { + return new TupleLongLongBlockSourceOperator(driverContext.blockFactory(), values, randomIntBetween(1, 1000)); + } + private List topNLong(List inputValues, int limit, boolean ascendingOrder, boolean nullsFirst) { return topNLong(driverContext(), inputValues, limit, ascendingOrder, nullsFirst); } @@ -465,33 +477,30 @@ private TopNOperator.Row row( public void testTopNTwoColumns() { List> values = Arrays.asList(tuple(1L, 1L), tuple(1L, 2L), tuple(null, null), tuple(null, 1L), tuple(1L, null)); assertThat( - topNTwoColumns( + topNTwoLongColumns( driverContext(), values, 5, - List.of(LONG, LONG), List.of(TopNEncoder.DEFAULT_SORTABLE, TopNEncoder.DEFAULT_SORTABLE), List.of(new TopNOperator.SortOrder(0, true, false), new TopNOperator.SortOrder(1, true, false)) ), equalTo(List.of(tuple(1L, 1L), tuple(1L, 2L), tuple(1L, null), tuple(null, 1L), tuple(null, null))) ); assertThat( - topNTwoColumns( + topNTwoLongColumns( driverContext(), values, 5, - List.of(LONG, LONG), List.of(TopNEncoder.DEFAULT_SORTABLE, TopNEncoder.DEFAULT_SORTABLE), List.of(new TopNOperator.SortOrder(0, true, true), new TopNOperator.SortOrder(1, true, false)) ), equalTo(List.of(tuple(null, 1L), tuple(null, null), tuple(1L, 1L), tuple(1L, 2L), tuple(1L, null))) ); assertThat( - topNTwoColumns( + topNTwoLongColumns( driverContext(), values, 5, - List.of(LONG, LONG), List.of(TopNEncoder.DEFAULT_SORTABLE, TopNEncoder.DEFAULT_SORTABLE), List.of(new TopNOperator.SortOrder(0, true, false), new TopNOperator.SortOrder(1, true, true)) ), @@ -657,45 +666,82 @@ public void testCollectAllValues_RandomMultiValues() { assertDriverContext(driverContext); } - private List> topNTwoColumns( + private List> topNTwoLongColumns( DriverContext driverContext, - List> inputValues, + List> values, int limit, - List elementTypes, List encoder, List sortOrders ) { - List> outputValues = new ArrayList<>(); + var page = topNTwoColumns( + driverContext, + new TupleLongLongBlockSourceOperator(driverContext.blockFactory(), values, randomIntBetween(1, 1000)), + limit, + encoder, + sortOrders + ); + var result = pageToTuples( + (block, i) -> block.isNull(i) ? null : ((LongBlock) block).getLong(i), + (block, i) -> block.isNull(i) ? null : ((LongBlock) block).getLong(i), + page + ); + assertThat(result, hasSize(Math.min(limit, values.size()))); + return result; + } + + private List topNTwoColumns( + DriverContext driverContext, + TupleAbstractBlockSourceOperator sourceOperator, + int limit, + List encoder, + List sortOrders + ) { + var pages = new ArrayList(); try ( Driver driver = TestDriverFactory.create( driverContext, - new TupleBlockSourceOperator(driverContext.blockFactory(), inputValues, randomIntBetween(1, 1000)), + sourceOperator, List.of( new TopNOperator( driverContext.blockFactory(), nonBreakingBigArrays().breakerService().getBreaker("request"), limit, - elementTypes, + sourceOperator.elementTypes(), encoder, sortOrders, randomPageSize() ) ), - new PageConsumerOperator(page -> { - LongBlock block1 = page.getBlock(0); - LongBlock block2 = page.getBlock(1); - for (int i = 0; i < block1.getPositionCount(); i++) { - outputValues.add(tuple(block1.isNull(i) ? null : block1.getLong(i), block2.isNull(i) ? null : block2.getLong(i))); - } - page.releaseBlocks(); - }) + new PageConsumerOperator(pages::add) ) ) { runDriver(driver); } - assertThat(outputValues, hasSize(Math.min(limit, inputValues.size()))); assertDriverContext(driverContext); - return outputValues; + return pages; + } + + private static List> pageToTuples( + BiFunction getFirstBlockValue, + BiFunction getSecondBlockValue, + List pages + ) { + var result = new ArrayList>(); + for (Page page : pages) { + var block1 = page.getBlock(0); + var block2 = page.getBlock(1); + for (int i = 0; i < block1.getPositionCount(); i++) { + result.add( + tuple( + block1.isNull(i) ? null : getFirstBlockValue.apply(block1, i), + block2.isNull(i) ? null : getSecondBlockValue.apply(block2, i) + ) + ); + } + page.releaseBlocks(); + } + + return result; } public void testTopNManyDescriptionAndToString() { @@ -1447,6 +1493,53 @@ public void testRowResizes() { } } + public void testShardContextManagement_limitEqualToCount_noShardContextIsReleased() { + topNShardContextManagementAux(4, Stream.generate(() -> true).limit(4).toList()); + } + + public void testShardContextManagement_notAllShardsPassTopN_shardsAreReleased() { + topNShardContextManagementAux(2, List.of(true, false, false, true)); + } + + private void topNShardContextManagementAux(int limit, List expectedOpenAfterTopN) { + List> values = Arrays.asList( + tuple(new BlockUtils.Doc(0, 10, 100), 1L), + tuple(new BlockUtils.Doc(1, 20, 200), 2L), + tuple(new BlockUtils.Doc(2, 30, 300), null), + tuple(new BlockUtils.Doc(3, 40, 400), -3L) + ); + List refCountedList = Stream.generate(() -> new SimpleRefCounted()).limit(4).toList(); + var shardRefCounted = ShardRefCounted.fromList(refCountedList); + + var pages = topNTwoColumns(driverContext(), new TupleDocLongBlockSourceOperator(driverContext().blockFactory(), values) { + @Override + protected Block.Builder firstElementBlockBuilder(int length) { + return DocBlock.newBlockBuilder(blockFactory, length).setShardRefCounted(shardRefCounted); + } + }, + limit, + List.of(TopNEncoder.DEFAULT_UNSORTABLE, TopNEncoder.DEFAULT_SORTABLE), + List.of(new TopNOperator.SortOrder(1, true, false)) + + ); + refCountedList.forEach(RefCounted::decRef); + + assertThat(refCountedList.stream().map(RefCounted::hasReferences).toList(), equalTo(expectedOpenAfterTopN)); + + var expectedValues = values.stream() + .sorted(Comparator.comparingLong(t -> t.v2() == null ? Long.MAX_VALUE : t.v2())) + .limit(limit) + .toList(); + assertThat( + pageToTuples((b, i) -> (BlockUtils.Doc) BlockUtils.toJavaObject(b, i), (b, i) -> ((LongBlock) b).getLong(i), pages), + equalTo(expectedValues) + ); + + for (var rc : refCountedList) { + assertFalse(rc.hasReferences()); + } + } + @SuppressWarnings({ "unchecked", "rawtypes" }) private static void readAsRows(List>> values, Page page) { if (page.getBlockCount() == 0) { diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/topn/TopNRowTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/topn/TopNRowTests.java index fdf62706e210a..8171299c4618c 100644 --- a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/topn/TopNRowTests.java +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/topn/TopNRowTests.java @@ -63,6 +63,7 @@ private long expectedRamBytesUsed(TopNOperator.Row row) { expected -= RamUsageTester.ramUsed("topn"); // the sort orders are shared expected -= RamUsageTester.ramUsed(sortOrders()); + // expected -= RamUsageTester.ramUsed(row.docVector); return expected; } } diff --git a/x-pack/plugin/esql/compute/test/src/main/java/org/elasticsearch/compute/test/NoOpReleasable.java b/x-pack/plugin/esql/compute/test/src/main/java/org/elasticsearch/compute/test/NoOpReleasable.java new file mode 100644 index 0000000000000..8053685a2fd9d --- /dev/null +++ b/x-pack/plugin/esql/compute/test/src/main/java/org/elasticsearch/compute/test/NoOpReleasable.java @@ -0,0 +1,15 @@ +/* + * 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.compute.test; + +import org.elasticsearch.core.Releasable; + +public class NoOpReleasable implements Releasable { + @Override + public void close() {/* no-op */} +} diff --git a/x-pack/plugin/esql/qa/server/src/main/java/org/elasticsearch/xpack/esql/qa/rest/RestEsqlTestCase.java b/x-pack/plugin/esql/qa/server/src/main/java/org/elasticsearch/xpack/esql/qa/rest/RestEsqlTestCase.java index 293200b227916..505a63a79a664 100644 --- a/x-pack/plugin/esql/qa/server/src/main/java/org/elasticsearch/xpack/esql/qa/rest/RestEsqlTestCase.java +++ b/x-pack/plugin/esql/qa/server/src/main/java/org/elasticsearch/xpack/esql/qa/rest/RestEsqlTestCase.java @@ -1650,7 +1650,7 @@ private static Request prepareRequest(Mode mode) { } private static Request prepareAsyncGetRequest(String id) { - return finishRequest(new Request("GET", "/_query/async/" + id + "?wait_for_completion_timeout=60s")); + return finishRequest(new Request("GET", "/_query/async/" + id + "?wait_for_completion_timeout=6000s")); } private static Request prepareAsyncDeleteRequest(String id) { diff --git a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/AbstractPausableIntegTestCase.java b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/AbstractPausableIntegTestCase.java index 86277b1c1cd24..0131e5b81b66b 100644 --- a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/AbstractPausableIntegTestCase.java +++ b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/AbstractPausableIntegTestCase.java @@ -39,7 +39,11 @@ public abstract class AbstractPausableIntegTestCase extends AbstractEsqlIntegTes @Override protected Collection> nodePlugins() { - return CollectionUtils.appendToCopy(super.nodePlugins(), PausableFieldPlugin.class); + return CollectionUtils.appendToCopy(super.nodePlugins(), pausableFieldPluginClass()); + } + + protected Class pausableFieldPluginClass() { + return PausableFieldPlugin.class; } protected int pageSize() { @@ -56,6 +60,10 @@ protected int numberOfDocs() { return numberOfDocs; } + protected int shardCount() { + return 1; + } + @Before public void setupIndex() throws IOException { assumeTrue("requires query pragmas", canUseQueryPragmas()); @@ -71,7 +79,7 @@ public void setupIndex() throws IOException { mapping.endObject(); } mapping.endObject(); - client().admin().indices().prepareCreate("test").setSettings(indexSettings(1, 0)).setMapping(mapping.endObject()).get(); + client().admin().indices().prepareCreate("test").setSettings(indexSettings(shardCount(), 0)).setMapping(mapping.endObject()).get(); BulkRequestBuilder bulk = client().prepareBulk().setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE); for (int i = 0; i < numberOfDocs(); i++) { @@ -89,10 +97,11 @@ public void setupIndex() throws IOException { * failed to reduce the index to a single segment and caused this test * to fail in very difficult to debug ways. If it fails again, it'll * trip here. Or maybe it won't! And we'll learn something. Maybe - * it's ghosts. + * it's ghosts. Extending classes can override the shardCount method if + * more than a single segment is expected. */ SegmentsStats stats = client().admin().indices().prepareStats("test").get().getPrimaries().getSegments(); - if (stats.getCount() != 1L) { + if (stats.getCount() != shardCount()) { fail(Strings.toString(stats)); } } diff --git a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/EsqlTopNShardManagementIT.java b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/EsqlTopNShardManagementIT.java new file mode 100644 index 0000000000000..4f15d7c65b52f --- /dev/null +++ b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/EsqlTopNShardManagementIT.java @@ -0,0 +1,121 @@ +/* + * 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.action; + +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.CollectionUtils; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.search.MockSearchService; +import org.elasticsearch.search.SearchService; +import org.elasticsearch.search.internal.SearchContext; +import org.elasticsearch.test.ESIntegTestCase; +import org.elasticsearch.xpack.core.async.GetAsyncResultRequest; +import org.elasticsearch.xpack.esql.plugin.QueryPragmas; +import org.junit.Before; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.concurrent.TimeUnit; + +import static org.elasticsearch.core.TimeValue.timeValueSeconds; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; + +// Verifies that the TopNOperator can release shard contexts as it processes its input. +@ESIntegTestCase.ClusterScope(numDataNodes = 1) +public class EsqlTopNShardManagementIT extends AbstractPausableIntegTestCase { + private static List searchContexts = new ArrayList<>(); + private static final int SHARD_COUNT = 10; + + @Override + protected Class pausableFieldPluginClass() { + return TopNPausableFieldPlugin.class; + } + + @Override + protected int shardCount() { + return SHARD_COUNT; + } + + @Override + protected Collection> nodePlugins() { + return CollectionUtils.appendToCopy(super.nodePlugins(), MockSearchService.TestPlugin.class); + } + + @Before + public void setupMockService() { + searchContexts.clear(); + for (SearchService service : internalCluster().getInstances(SearchService.class)) { + var mockSearchService = (MockSearchService) service; + mockSearchService.setOnPutContext(ctx -> System.out.println("Putting search context: " + ctx.id())); + mockSearchService.setOnCreateSearchContext(ctx -> { + System.out.println("Creating search context: " + ctx.id()); + searchContexts.add(ctx); + scriptPermits.release(); + }); + mockSearchService.setOnRemoveContext(ctx -> System.out.println("Removing search context: " + ctx.id())); + } + } + + public void testTopNOperatorReleasesContexts() throws Exception { + try (var initialResponse = sendAsyncQuery()) { + var getResultsRequest = new GetAsyncResultRequest(initialResponse.asyncExecutionId().get()); + scriptPermits.release(numberOfDocs()); + getResultsRequest.setWaitForCompletionTimeout(timeValueSeconds(10)); + var result = client().execute(EsqlAsyncGetResultAction.INSTANCE, getResultsRequest).get(); + assertThat(result.isRunning(), equalTo(false)); + assertThat(result.isPartial(), equalTo(false)); + result.close(); + } + } + + private static EsqlQueryResponse sendAsyncQuery() { + scriptPermits.drainPermits(); + return EsqlQueryRequestBuilder.newAsyncEsqlQueryRequestBuilder(client()) + // Ensures there is no TopN pushdown to lucene, and that the pause happens after the TopN operator has been applied. + .query("from test | sort foo + 1 | limit 1 | where pause_me + 1 > 42 | stats sum(pause_me)") + .pragmas( + new QueryPragmas( + Settings.builder() + // Configured to ensure that there is only one worker handling all the shards, so that we can assert the correct + // expected behavior. + .put(QueryPragmas.MAX_CONCURRENT_NODES_PER_CLUSTER.getKey(), 1) + .put(QueryPragmas.MAX_CONCURRENT_SHARDS_PER_NODE.getKey(), SHARD_COUNT) + .put(QueryPragmas.TASK_CONCURRENCY.getKey(), 1) + .build() + ) + ) + .execute() + .actionGet(1, TimeUnit.MINUTES); + } + + public static class TopNPausableFieldPlugin extends AbstractPauseFieldPlugin { + @Override + protected boolean onWait() throws InterruptedException { + var acquired = scriptPermits.tryAcquire(SHARD_COUNT, 1, TimeUnit.MINUTES); + assertTrue("Failed to acquire permits", acquired); + int closed = 0; + int open = 0; + for (SearchContext searchContext : searchContexts) { + if (searchContext.isClosed()) { + closed++; + } else { + open++; + } + } + assertThat( + Strings.format("most contexts to be closed, but %d were closed and %d were open", closed, open), + closed, + greaterThanOrEqualTo(open) + ); + return scriptPermits.tryAcquire(1, 1, TimeUnit.MINUTES); + } + } +} diff --git a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/LookupFromIndexIT.java b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/LookupFromIndexIT.java index e6b96e2e1267d..7a64edeed27f0 100644 --- a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/LookupFromIndexIT.java +++ b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/LookupFromIndexIT.java @@ -183,6 +183,7 @@ private void runLookup(DataType keyType, PopulateIndices populateIndices) throws ) { ShardContext esqlContext = new EsPhysicalOperationProviders.DefaultShardContext( 0, + searchContext, searchContext.getSearchExecutionContext(), AliasFilter.EMPTY ); diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/enrich/AbstractLookupService.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/enrich/AbstractLookupService.java index a963d6d738918..649b7dfbc8e6a 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/enrich/AbstractLookupService.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/enrich/AbstractLookupService.java @@ -342,7 +342,7 @@ private void doLookup(T request, CancellableTask task, ActionListener driverContext.blockFactory(), EnrichQuerySourceOperator.DEFAULT_MAX_PAGE_SIZE, queryList, - shardContext.context.searcher().getIndexReader(), + shardContext.context, warnings ); releasables.add(queryOperator); @@ -686,6 +686,7 @@ public static LookupShardContext fromSearchContext(SearchContext context) { return new LookupShardContext( new EsPhysicalOperationProviders.DefaultShardContext( 0, + context, context.getSearchExecutionContext(), context.request().getAliasFilter() ), diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/EsPhysicalOperationProviders.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/EsPhysicalOperationProviders.java index 0b2fafcf2df2e..3cb9c3302bb8c 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/EsPhysicalOperationProviders.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/EsPhysicalOperationProviders.java @@ -31,7 +31,9 @@ import org.elasticsearch.compute.operator.OrdinalsGroupingOperator; import org.elasticsearch.compute.operator.SourceOperator; import org.elasticsearch.compute.operator.TimeSeriesAggregationOperator; +import org.elasticsearch.core.AbstractRefCounted; import org.elasticsearch.core.Nullable; +import org.elasticsearch.core.Releasable; import org.elasticsearch.index.IndexMode; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.analysis.AnalysisRegistry; @@ -92,12 +94,38 @@ public class EsPhysicalOperationProviders extends AbstractPhysicalOperationProvi /** * Context of each shard we're operating against. */ - public interface ShardContext extends org.elasticsearch.compute.lucene.ShardContext { + public abstract static class ShardContext implements org.elasticsearch.compute.lucene.ShardContext, Releasable { + private final AbstractRefCounted refCounted = new AbstractRefCounted() { + @Override + protected void closeInternal() { + ShardContext.this.close(); + } + }; + + @Override + public void incRef() { + refCounted.incRef(); + } + + @Override + public boolean tryIncRef() { + return refCounted.tryIncRef(); + } + + @Override + public boolean decRef() { + return refCounted.decRef(); + } + + @Override + public boolean hasReferences() { + return refCounted.hasReferences(); + } /** * Convert a {@link QueryBuilder} into a real {@link Query lucene query}. */ - Query toQuery(QueryBuilder queryBuilder); + public abstract Query toQuery(QueryBuilder queryBuilder); /** * Tuning parameter for deciding when to use the "merge" stored field loader. @@ -107,7 +135,7 @@ public interface ShardContext extends org.elasticsearch.compute.lucene.ShardCont * A value of {@code .2} means we'll use the sequential reader even if we only * need one in ten documents. */ - double storedFieldsSequentialProportion(); + public abstract double storedFieldsSequentialProportion(); } private final List shardContexts; @@ -180,7 +208,7 @@ private static class DefaultShardContextForUnmappedField extends DefaultShardCon private final KeywordEsField unmappedEsField; DefaultShardContextForUnmappedField(DefaultShardContext ctx, PotentiallyUnmappedKeywordEsField unmappedEsField) { - super(ctx.index, ctx.ctx, ctx.aliasFilter); + super(ctx.index, ctx.releasable, ctx.ctx, ctx.aliasFilter); this.unmappedEsField = unmappedEsField; } @@ -349,18 +377,20 @@ public Operator.OperatorFactory timeSeriesAggregatorOperatorFactory( ); } - public static class DefaultShardContext implements ShardContext { + public static class DefaultShardContext extends ShardContext { private final int index; + private final Releasable releasable; private final SearchExecutionContext ctx; private final AliasFilter aliasFilter; private final String shardIdentifier; - public DefaultShardContext(int index, SearchExecutionContext ctx, AliasFilter aliasFilter) { + public DefaultShardContext(int index, Releasable releasable, SearchExecutionContext ctx, AliasFilter aliasFilter) { this.index = index; + this.releasable = releasable; this.ctx = ctx; this.aliasFilter = aliasFilter; // Build the shardIdentifier once up front so we can reuse references to it in many places. - this.shardIdentifier = ctx.getFullyQualifiedIndex().getName() + ":" + ctx.getShardId(); + this.shardIdentifier = this.ctx.getFullyQualifiedIndex().getName() + ":" + this.ctx.getShardId(); } @Override @@ -473,6 +503,11 @@ public FieldNamesFieldMapper.FieldNamesFieldType fieldNames() { public double storedFieldsSequentialProportion() { return EsqlPlugin.STORED_FIELDS_SEQUENTIAL_PROPORTION.get(ctx.getIndexSettings().getSettings()); } + + @Override + public void close() { + releasable.close(); + } } private static class TypeConvertingBlockLoader implements BlockLoader { diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/ComputeService.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/ComputeService.java index a16e5f8bd78e0..a454cdfddce9b 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/ComputeService.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/ComputeService.java @@ -27,6 +27,7 @@ import org.elasticsearch.compute.operator.exchange.ExchangeSink; import org.elasticsearch.compute.operator.exchange.ExchangeSinkHandler; import org.elasticsearch.compute.operator.exchange.ExchangeSourceHandler; +import org.elasticsearch.core.RefCounted; import org.elasticsearch.core.Releasable; import org.elasticsearch.core.Releasables; import org.elasticsearch.core.Tuple; @@ -545,7 +546,12 @@ public SourceProvider createSourceProvider() { } }; contexts.add( - new EsPhysicalOperationProviders.DefaultShardContext(i, searchExecutionContext, searchContext.request().getAliasFilter()) + new EsPhysicalOperationProviders.DefaultShardContext( + i, + searchContext, + searchExecutionContext, + searchContext.request().getAliasFilter() + ) ); } EsPhysicalOperationProviders physicalOperationProviders = new EsPhysicalOperationProviders( @@ -584,6 +590,7 @@ public SourceProvider createSourceProvider() { LOGGER.debug("Local execution plan:\n{}", localExecutionPlan.describe()); } drivers = localExecutionPlan.createDrivers(context.sessionId()); + contexts.forEach(RefCounted::decRef); if (drivers.isEmpty()) { throw new IllegalStateException("no drivers created"); } diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/QueryPragmas.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/QueryPragmas.java index 29951070a96cb..345bf3b8767ef 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/QueryPragmas.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/QueryPragmas.java @@ -35,7 +35,7 @@ public final class QueryPragmas implements Writeable { public static final Setting EXCHANGE_CONCURRENT_CLIENTS = Setting.intSetting("exchange_concurrent_clients", 2); public static final Setting ENRICH_MAX_WORKERS = Setting.intSetting("enrich_max_workers", 1); - private static final Setting TASK_CONCURRENCY = Setting.intSetting( + public static final Setting TASK_CONCURRENCY = Setting.intSetting( "task_concurrency", ThreadPool.searchOrGetThreadPoolSize(EsExecutors.allocatedProcessors(Settings.EMPTY)) ); diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/enrich/LookupFromIndexOperatorTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/enrich/LookupFromIndexOperatorTests.java index badc164f24722..410970ab82c6a 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/enrich/LookupFromIndexOperatorTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/enrich/LookupFromIndexOperatorTests.java @@ -35,6 +35,7 @@ import org.elasticsearch.compute.operator.DriverContext; import org.elasticsearch.compute.operator.Operator; import org.elasticsearch.compute.operator.SourceOperator; +import org.elasticsearch.compute.test.NoOpReleasable; import org.elasticsearch.compute.test.OperatorTestCase; import org.elasticsearch.compute.test.SequenceLongBlockSourceOperator; import org.elasticsearch.core.IOUtils; @@ -245,11 +246,7 @@ private AbstractLookupService.LookupShardContextFactory lookupShardContextFactor }"""); DirectoryReader reader = DirectoryReader.open(lookupIndexDirectory); SearchExecutionContext executionCtx = mapperHelper.createSearchExecutionContext(mapperService, newSearcher(reader)); - EsPhysicalOperationProviders.DefaultShardContext ctx = new EsPhysicalOperationProviders.DefaultShardContext( - 0, - executionCtx, - AliasFilter.EMPTY - ); + var ctx = new EsPhysicalOperationProviders.DefaultShardContext(0, new NoOpReleasable(), executionCtx, AliasFilter.EMPTY); return new AbstractLookupService.LookupShardContext(ctx, executionCtx, () -> { try { IOUtils.close(reader, mapperService); diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/LocalExecutionPlannerTests.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/LocalExecutionPlannerTests.java index 0fee0c13178d4..08632a70e99e3 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/LocalExecutionPlannerTests.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/LocalExecutionPlannerTests.java @@ -24,6 +24,7 @@ import org.elasticsearch.compute.lucene.LuceneSourceOperator; import org.elasticsearch.compute.lucene.LuceneTopNSourceOperator; import org.elasticsearch.compute.operator.SourceOperator; +import org.elasticsearch.compute.test.NoOpReleasable; import org.elasticsearch.compute.test.TestBlockFactory; import org.elasticsearch.core.IOUtils; import org.elasticsearch.core.Releasable; @@ -31,6 +32,7 @@ import org.elasticsearch.index.IndexMode; import org.elasticsearch.index.cache.query.TrivialQueryCachingPolicy; import org.elasticsearch.index.mapper.MapperServiceTestCase; +import org.elasticsearch.index.query.SearchExecutionContext; import org.elasticsearch.node.Node; import org.elasticsearch.plugins.ExtensiblePlugin; import org.elasticsearch.plugins.Plugin; @@ -296,10 +298,11 @@ private List createShardContexts() th true ); for (int i = 0; i < numShards; i++) { + SearchExecutionContext searchExecutionContext = createSearchExecutionContext(createMapperService(mapping(b -> { + b.startObject("point").field("type", "geo_point").endObject(); + })), searcher); shardContexts.add( - new EsPhysicalOperationProviders.DefaultShardContext(i, createSearchExecutionContext(createMapperService(mapping(b -> { - b.startObject("point").field("type", "geo_point").endObject(); - })), searcher), AliasFilter.EMPTY) + new EsPhysicalOperationProviders.DefaultShardContext(i, new NoOpReleasable(), searchExecutionContext, AliasFilter.EMPTY) ); } releasables.add(searcher); diff --git a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/TestPhysicalOperationProviders.java b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/TestPhysicalOperationProviders.java index d5aa1af7feec2..a8916f140ea1f 100644 --- a/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/TestPhysicalOperationProviders.java +++ b/x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/planner/TestPhysicalOperationProviders.java @@ -26,6 +26,7 @@ import org.elasticsearch.compute.data.IntVector; import org.elasticsearch.compute.data.LongBlock; import org.elasticsearch.compute.data.Page; +import org.elasticsearch.compute.lucene.ShardRefCounted; import org.elasticsearch.compute.operator.DriverContext; import org.elasticsearch.compute.operator.HashAggregationOperator; import org.elasticsearch.compute.operator.Operator; @@ -188,6 +189,7 @@ public Page getOutput() { var page = pageIndex.page; BlockFactory blockFactory = driverContext.blockFactory(); DocVector docVector = new DocVector( + ShardRefCounted.ALWAYS_REFERENCED, // The shard ID is used to encode the index ID. blockFactory.newConstantIntVector(index, page.getPositionCount()), blockFactory.newConstantIntVector(0, page.getPositionCount()), From 06de0d76355f96e2fc04188d95dc67fec88b8d5a Mon Sep 17 00:00:00 2001 From: Gal Lalouche Date: Sun, 15 Jun 2025 17:13:23 +0300 Subject: [PATCH 02/16] Update docs/changelog/129454.yaml --- docs/changelog/129454.yaml | 5 +++++ 1 file changed, 5 insertions(+) create mode 100644 docs/changelog/129454.yaml diff --git a/docs/changelog/129454.yaml b/docs/changelog/129454.yaml new file mode 100644 index 0000000000000..538c5266c6162 --- /dev/null +++ b/docs/changelog/129454.yaml @@ -0,0 +1,5 @@ +pr: 129454 +summary: Aggressive release of shard contexts +area: ES|QL +type: enhancement +issues: [] From 25fd5141bc49fdf722cdee5cc35c545d692dd4d9 Mon Sep 17 00:00:00 2001 From: Gal Lalouche Date: Sun, 15 Jun 2025 18:28:01 +0300 Subject: [PATCH 03/16] Fix compilation errors --- .../operator/ValuesSourceReaderBenchmark.java | 15 ++++++++++++++- .../elasticsearch/compute/operator/Driver.java | 1 + 2 files changed, 15 insertions(+), 1 deletion(-) diff --git a/benchmarks/src/main/java/org/elasticsearch/benchmark/compute/operator/ValuesSourceReaderBenchmark.java b/benchmarks/src/main/java/org/elasticsearch/benchmark/compute/operator/ValuesSourceReaderBenchmark.java index 2796bb5c6de1f..a4504bedb3644 100644 --- a/benchmarks/src/main/java/org/elasticsearch/benchmark/compute/operator/ValuesSourceReaderBenchmark.java +++ b/benchmarks/src/main/java/org/elasticsearch/benchmark/compute/operator/ValuesSourceReaderBenchmark.java @@ -40,6 +40,7 @@ import org.elasticsearch.compute.data.LongVector; import org.elasticsearch.compute.data.Page; import org.elasticsearch.compute.lucene.LuceneSourceOperator; +import org.elasticsearch.compute.lucene.ShardRefCounted; import org.elasticsearch.compute.lucene.ValuesSourceReaderOperator; import org.elasticsearch.compute.operator.topn.TopNOperator; import org.elasticsearch.core.IOUtils; @@ -477,6 +478,7 @@ private void setupPages() { pages.add( new Page( new DocVector( + ShardRefCounted.ALWAYS_REFERENCED, blockFactory.newConstantIntBlockWith(0, end - begin).asVector(), blockFactory.newConstantIntBlockWith(ctx.ord, end - begin).asVector(), docs.build(), @@ -512,7 +514,14 @@ record ItrAndOrd(PrimitiveIterator.OfInt itr, int ord) {} if (size >= BLOCK_LENGTH) { pages.add( new Page( - new DocVector(blockFactory.newConstantIntVector(0, size), leafs.build(), docs.build(), null).asBlock() + new DocVector( + + ShardRefCounted.ALWAYS_REFERENCED, + blockFactory.newConstantIntVector(0, size), + leafs.build(), + docs.build(), + null + ).asBlock() ) ); docs = blockFactory.newIntVectorBuilder(BLOCK_LENGTH); @@ -525,6 +534,8 @@ record ItrAndOrd(PrimitiveIterator.OfInt itr, int ord) {} pages.add( new Page( new DocVector( + + ShardRefCounted.ALWAYS_REFERENCED, blockFactory.newConstantIntBlockWith(0, size).asVector(), leafs.build().asBlock().asVector(), docs.build(), @@ -551,6 +562,8 @@ record ItrAndOrd(PrimitiveIterator.OfInt itr, int ord) {} pages.add( new Page( new DocVector( + + ShardRefCounted.ALWAYS_REFERENCED, blockFactory.newConstantIntVector(0, 1), blockFactory.newConstantIntVector(next.ord, 1), blockFactory.newConstantIntVector(next.itr.nextInt(), 1), diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/Driver.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/Driver.java index d297322ee4ac7..061391b4666d5 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/Driver.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/Driver.java @@ -306,6 +306,7 @@ private IsBlockedResult runSingleLoopIteration() { return Operator.NOT_BLOCKED; } + // Returns the index of the last operator that was closed, or minIndex - 1 if no operators were closed. private int closeEarlyFinishedOperators(int minIndex, int maxIndex) { for (int index = maxIndex; index >= minIndex; index--) { if (activeOperators.get(index).isFinished()) { From 0210097d69dfcb93c81152efb0e6c4456ec74d25 Mon Sep 17 00:00:00 2001 From: Gal Lalouche Date: Sun, 15 Jun 2025 22:31:28 +0300 Subject: [PATCH 04/16] Fix bug caused by mishandling of errors during driver iteration --- .../compute/operator/Driver.java | 64 ++++++++++--------- 1 file changed, 34 insertions(+), 30 deletions(-) diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/Driver.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/Driver.java index 061391b4666d5..d38b53453cef8 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/Driver.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/Driver.java @@ -24,6 +24,7 @@ import java.util.ArrayList; import java.util.Iterator; import java.util.List; +import java.util.ListIterator; import java.util.concurrent.Executor; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; @@ -184,8 +185,7 @@ SubscribableListener run(TimeValue maxTime, int maxIterations, LongSupplie assert driverContext.assertBeginRunLoop(); isBlocked = runSingleLoopIteration(); } catch (DriverEarlyTerminationException unused) { - int lastFinished = closeEarlyFinishedOperators(0, activeOperators.size() - 1); - activeOperators = new ArrayList<>(activeOperators.subList(lastFinished + 1, activeOperators.size())); + closeEarlyFinishedOperators(activeOperators.listIterator(activeOperators.size())); assert isFinished() : "not finished after early termination"; } finally { assert driverContext.assertEndRunLoop(); @@ -252,10 +252,13 @@ private IsBlockedResult runSingleLoopIteration() { driverContext.checkForEarlyTermination(); boolean movedPage = false; - int lastClosed = -1; - for (int i = 0; i < activeOperators.size() - 1; i++) { - Operator op = activeOperators.get(i); - Operator nextOp = activeOperators.get(i + 1); + ListIterator iterator = activeOperators.listIterator(); + while (iterator.hasNext()) { + Operator op = iterator.next(); + if (iterator.hasNext() == false) { + break; + } + Operator nextOp = activeOperators.get(iterator.nextIndex()); // skip blocked operator if (op.isBlocked().listener().isDone() == false) { @@ -264,6 +267,7 @@ private IsBlockedResult runSingleLoopIteration() { if (op.isFinished() == false && nextOp.needsInput()) { driverContext.checkForEarlyTermination(); + assert nextOp.isFinished() == false : "next operator should not be finished yet: " + nextOp; Page page = op.getOutput(); if (page == null) { // No result, just move to the next iteration @@ -285,15 +289,15 @@ private IsBlockedResult runSingleLoopIteration() { if (op.isFinished()) { driverContext.checkForEarlyTermination(); - closeEarlyFinishedOperators(lastClosed + 1, i); - lastClosed = i; + var originalIndex = iterator.previousIndex(); + var index = closeEarlyFinishedOperators(iterator); + if (index >= 0) { + iterator = new ArrayList<>(activeOperators).listIterator(originalIndex - index); + } } } - lastClosed = closeEarlyFinishedOperators(lastClosed + 1, activeOperators.size() - 1); - if (lastClosed >= 0) { - activeOperators = new ArrayList<>(activeOperators.subList(lastClosed + 1, activeOperators.size())); - } + closeEarlyFinishedOperators(activeOperators.listIterator(activeOperators.size())); if (movedPage == false) { return oneOf( @@ -306,33 +310,33 @@ private IsBlockedResult runSingleLoopIteration() { return Operator.NOT_BLOCKED; } - // Returns the index of the last operator that was closed, or minIndex - 1 if no operators were closed. - private int closeEarlyFinishedOperators(int minIndex, int maxIndex) { - for (int index = maxIndex; index >= minIndex; index--) { - if (activeOperators.get(index).isFinished()) { + // Returns the index of the last operator that was closed, -1 if no operator was closed. + private int closeEarlyFinishedOperators(ListIterator operators) { + var iterator = activeOperators.listIterator(operators.nextIndex()); + while (iterator.hasPrevious()) { + if (iterator.previous().isFinished()) { + var index = iterator.nextIndex(); + iterator.next(); /* - * Remove this operator and all unclosed source operators in the - * most paranoid possible way. Closing operators shouldn't throw, - * but if it does, this will make sure we don't try to close any - * that succeed twice. - */ - List operatorsToClose = this.activeOperators.subList(minIndex, index + 1); - Iterator itr = operatorsToClose.iterator(); - while (itr.hasNext()) { - Operator op = itr.next(); + * Remove this operator and all unclosed source operators in the + * most paranoid possible way. Closing operators shouldn't throw, + * but if it does, this will make sure we don't try to close any + * that succeed twice. + */ + while (iterator.hasPrevious()) { + Operator op = iterator.previous(); statusOfCompletedOperators.add(new OperatorStatus(op.toString(), op.status())); op.close(); + iterator.remove(); } - // Finish the next operator. - if (index + 1 < activeOperators.size()) { - Operator newRootOperator = activeOperators.get(index + 1); - newRootOperator.finish(); + if (iterator.hasNext()) { + iterator.next().finish(); } return index; } } - return minIndex - 1; + return -1; } public void cancel(String reason) { From f51a7c439279857da06ad34bd3c1da51f96a1e08 Mon Sep 17 00:00:00 2001 From: Gal Lalouche Date: Mon, 16 Jun 2025 15:59:39 +0300 Subject: [PATCH 05/16] Change order of removal from first to last --- .../compute/operator/Driver.java | 26 ++++++++++--------- 1 file changed, 14 insertions(+), 12 deletions(-) diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/Driver.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/Driver.java index d38b53453cef8..7942dfd4b1f12 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/Driver.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/Driver.java @@ -316,22 +316,24 @@ private int closeEarlyFinishedOperators(ListIterator operators) { while (iterator.hasPrevious()) { if (iterator.previous().isFinished()) { var index = iterator.nextIndex(); - iterator.next(); /* - * Remove this operator and all unclosed source operators in the - * most paranoid possible way. Closing operators shouldn't throw, - * but if it does, this will make sure we don't try to close any - * that succeed twice. - */ - while (iterator.hasPrevious()) { - Operator op = iterator.previous(); + * Remove this operator and all source operators in the + * most paranoid possible way. Closing operators shouldn't throw, + * but if it does, this will make sure we don't try to close any + * that succeed twice. + */ + Iterator finishedOperators = this.activeOperators.subList(0, index + 1).iterator(); + while (finishedOperators.hasNext()) { + Operator op = finishedOperators.next(); statusOfCompletedOperators.add(new OperatorStatus(op.toString(), op.status())); op.close(); - iterator.remove(); + finishedOperators.remove(); } - // Finish the next operator. - if (iterator.hasNext()) { - iterator.next().finish(); + + // Finish the next operator, which is now the first operator. + if (activeOperators.isEmpty() == false) { + Operator newRootOperator = activeOperators.get(0); + newRootOperator.finish(); } return index; } From d6ebed2cd3a48301f75f37d5eae2eba7e716816b Mon Sep 17 00:00:00 2001 From: Gal Lalouche Date: Mon, 16 Jun 2025 16:13:15 +0300 Subject: [PATCH 06/16] Remove printlns --- .../xpack/esql/action/EsqlTopNShardManagementIT.java | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/EsqlTopNShardManagementIT.java b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/EsqlTopNShardManagementIT.java index 4f15d7c65b52f..b74b300af68a8 100644 --- a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/EsqlTopNShardManagementIT.java +++ b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/EsqlTopNShardManagementIT.java @@ -53,14 +53,10 @@ protected Collection> nodePlugins() { public void setupMockService() { searchContexts.clear(); for (SearchService service : internalCluster().getInstances(SearchService.class)) { - var mockSearchService = (MockSearchService) service; - mockSearchService.setOnPutContext(ctx -> System.out.println("Putting search context: " + ctx.id())); - mockSearchService.setOnCreateSearchContext(ctx -> { - System.out.println("Creating search context: " + ctx.id()); + ((MockSearchService) service).setOnCreateSearchContext(ctx -> { searchContexts.add(ctx); scriptPermits.release(); }); - mockSearchService.setOnRemoveContext(ctx -> System.out.println("Removing search context: " + ctx.id())); } } From 8fb564a97578101d04142ea3e912c4b27acc2db2 Mon Sep 17 00:00:00 2001 From: Gal Lalouche Date: Mon, 16 Jun 2025 21:14:14 +0300 Subject: [PATCH 07/16] Add ref counter to OrdinalsGroupingOperator --- .../org/elasticsearch/compute/data/DocVector.java | 2 +- .../operator/OrdinalsGroupingOperator.java | 15 ++++++++++++--- .../compute/operator/topn/TopNOperator.java | 2 +- 3 files changed, 14 insertions(+), 5 deletions(-) diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/DocVector.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/DocVector.java index 6f1343f626709..c29cd77938d42 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/DocVector.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/DocVector.java @@ -51,7 +51,7 @@ public final class DocVector extends AbstractVector implements Vector { private final ShardRefCounted shardRefCounters; - public ShardRefCounted shardRefCounters() { + public ShardRefCounted shardRefCounted() { return shardRefCounters; } diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/OrdinalsGroupingOperator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/OrdinalsGroupingOperator.java index c030b329dd2d8..40f1b44dbf0e3 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/OrdinalsGroupingOperator.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/OrdinalsGroupingOperator.java @@ -33,6 +33,7 @@ import org.elasticsearch.compute.data.IntVector; import org.elasticsearch.compute.data.Page; import org.elasticsearch.compute.lucene.ValuesSourceReaderOperator; +import org.elasticsearch.core.RefCounted; import org.elasticsearch.core.Releasable; import org.elasticsearch.core.Releasables; import org.elasticsearch.index.mapper.BlockLoader; @@ -136,6 +137,7 @@ public void addInput(Page page) { requireNonNull(page, "page is null"); DocVector docVector = page.getBlock(docChannel).asVector(); final int shardIndex = docVector.shards().getInt(0); + RefCounted shardRefCounter = docVector.shardRefCounted().get(shardIndex); final var blockLoader = blockLoaders.apply(shardIndex); boolean pagePassed = false; try { @@ -150,7 +152,8 @@ public void addInput(Page page) { driverContext.blockFactory(), this::createGroupingAggregators, () -> blockLoader.ordinals(shardContexts.get(k.shardIndex).reader().leaves().get(k.segmentIndex)), - driverContext.bigArrays() + driverContext.bigArrays(), + shardRefCounter ); } catch (IOException e) { throw new UncheckedIOException(e); @@ -343,15 +346,19 @@ static final class OrdinalSegmentAggregator implements Releasable, SeenGroupIds private final List aggregators; private final CheckedSupplier docValuesSupplier; private final BitArray visitedOrds; + private final RefCounted shardRefCounted; private BlockOrdinalsReader currentReader; OrdinalSegmentAggregator( BlockFactory blockFactory, Supplier> aggregatorsSupplier, CheckedSupplier docValuesSupplier, - BigArrays bigArrays + BigArrays bigArrays, + RefCounted shardRefCounted ) throws IOException { boolean success = false; + shardRefCounted.mustIncRef(); + this.shardRefCounted = shardRefCounted; List groupingAggregators = null; BitArray bitArray = null; try { @@ -363,11 +370,13 @@ static final class OrdinalSegmentAggregator implements Releasable, SeenGroupIds this.docValuesSupplier = docValuesSupplier; this.aggregators = groupingAggregators; this.visitedOrds = bitArray; + this.shardRefCounted.mustIncRef(); success = true; } finally { if (success == false) { if (bitArray != null) Releasables.close(bitArray); if (groupingAggregators != null) Releasables.close(groupingAggregators); + shardRefCounted.decRef(); } } } @@ -447,7 +456,7 @@ public BitArray seenGroupIds(BigArrays bigArrays) { @Override public void close() { - Releasables.close(visitedOrds, () -> Releasables.close(aggregators)); + Releasables.close(visitedOrds, () -> Releasables.close(aggregators), Releasables.fromRefCounted(shardRefCounted)); } } diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/topn/TopNOperator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/topn/TopNOperator.java index 412500f6d6857..bb5c42807e89f 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/topn/TopNOperator.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/topn/TopNOperator.java @@ -216,7 +216,7 @@ private void writeKey(int position, Row row) { private void writeValues(int position, Row destination) { for (ValueExtractor e : valueExtractors) { if (e instanceof ValueExtractorForDoc fd) { - destination.setShardRefCountersAndShard(fd.vector().shardRefCounters().get(fd.vector().shards().getInt(position))); + destination.setShardRefCountersAndShard(fd.vector().shardRefCounted().get(fd.vector().shards().getInt(position))); } e.writeValue(destination.values, position); } From 2843e20c9ccb3abe9ae424d4b34b668543f7732a Mon Sep 17 00:00:00 2001 From: Gal Lalouche Date: Tue, 17 Jun 2025 13:09:05 +0300 Subject: [PATCH 08/16] Fix failing test (ManyShardsIT) --- .../org/elasticsearch/xpack/esql/action/ManyShardsIT.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/ManyShardsIT.java b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/ManyShardsIT.java index 08d59eade600e..40c7f5b88c76f 100644 --- a/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/ManyShardsIT.java +++ b/x-pack/plugin/esql/src/internalClusterTest/java/org/elasticsearch/xpack/esql/action/ManyShardsIT.java @@ -247,7 +247,7 @@ public void testLimitConcurrentShards() { for (SearchService searchService : searchServices) { SearchContextCounter counter = new SearchContextCounter(pragmas.maxConcurrentShardsPerNode()); var mockSearchService = (MockSearchService) searchService; - mockSearchService.setOnPutContext(r -> counter.onNewContext()); + mockSearchService.setOnCreateSearchContext(r -> counter.onNewContext()); mockSearchService.setOnRemoveContext(r -> counter.onContextReleased()); } run(q, pragmas).close(); @@ -255,7 +255,7 @@ public void testLimitConcurrentShards() { } finally { for (SearchService searchService : searchServices) { var mockSearchService = (MockSearchService) searchService; - mockSearchService.setOnPutContext(r -> {}); + mockSearchService.setOnCreateSearchContext(r -> {}); mockSearchService.setOnRemoveContext(r -> {}); } } From cfbf4b29c8723037b9c7076d3d559b6a2b2a1320 Mon Sep 17 00:00:00 2001 From: Gal Lalouche Date: Tue, 17 Jun 2025 13:09:35 +0300 Subject: [PATCH 09/16] CR fixes --- .../org/elasticsearch/core/Releasables.java | 1 + .../elasticsearch/compute/data/DocVector.java | 62 ++++++++++++------- .../compute/lucene/ShardRefCounted.java | 4 ++ .../planner/EsPhysicalOperationProviders.java | 7 ++- .../xpack/esql/plugin/ComputeService.java | 2 + 5 files changed, 54 insertions(+), 22 deletions(-) diff --git a/libs/core/src/main/java/org/elasticsearch/core/Releasables.java b/libs/core/src/main/java/org/elasticsearch/core/Releasables.java index 14b9b284292ce..8eee84050ca39 100644 --- a/libs/core/src/main/java/org/elasticsearch/core/Releasables.java +++ b/libs/core/src/main/java/org/elasticsearch/core/Releasables.java @@ -202,6 +202,7 @@ public boolean equals(Object obj) { } } + /** Creates a {@link Releasable} that calls {@link RefCounted#decRef()} when closed. */ public static Releasable fromRefCounted(RefCounted refCounted) { return () -> refCounted.decRef(); } diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/DocVector.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/DocVector.java index c29cd77938d42..126783bbb9d5b 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/DocVector.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/DocVector.java @@ -11,10 +11,13 @@ import org.apache.lucene.util.RamUsageEstimator; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.compute.lucene.ShardRefCounted; +import org.elasticsearch.core.RefCounted; import org.elasticsearch.core.ReleasableIterator; import org.elasticsearch.core.Releasables; +import java.util.BitSet; import java.util.Objects; +import java.util.function.Consumer; /** * {@link Vector} where each entry references a lucene document. @@ -30,6 +33,7 @@ public final class DocVector extends AbstractVector implements Vector { public static final int SHARD_SEGMENT_DOC_MAP_PER_ROW_OVERHEAD = Integer.BYTES * 2; private final IntVector shards; + private final IntVector uniqueShards; private final IntVector segments; private final IntVector docs; @@ -78,9 +82,34 @@ public DocVector( "invalid position count [" + shards.getPositionCount() + " != " + docs.getPositionCount() + "]" ); } + + this.uniqueShards = computeUniqueShards(shards); + forEachShardRefCounter(RefCounted::mustIncRef); blockFactory().adjustBreaker(BASE_RAM_BYTES_USED); + } - forEachShardRefCounter(DecOrInc.INC); + private static IntVector computeUniqueShards(IntVector shards) { + switch (shards) { + case ConstantIntVector constantIntVector -> { + return shards.blockFactory().newConstantIntVector(constantIntVector.getInt(0), 1); + } + case ConstantNullVector unused -> { + return shards.blockFactory().newConstantIntVector(0, 0); + } + default -> { + var seen = new BitSet(128); + try (IntVector.Builder uniqueShardsBuilder = shards.blockFactory().newIntVectorBuilder(shards.getPositionCount())) { + for (int p = 0; p < shards.getPositionCount(); p++) { + int shardId = shards.getInt(p); + if (seen.get(shardId) == false) { + seen.set(shardId); + uniqueShardsBuilder.appendInt(shardId); + } + } + return uniqueShardsBuilder.build(); + } + } + } } public DocVector( @@ -310,18 +339,20 @@ private static long ramBytesOrZero(int[] array) { public static long ramBytesEstimated( IntVector shards, + IntVector uniqueShards, IntVector segments, IntVector docs, int[] shardSegmentDocMapForwards, int[] shardSegmentDocMapBackwards ) { - return BASE_RAM_BYTES_USED + RamUsageEstimator.sizeOf(shards) + RamUsageEstimator.sizeOf(segments) + RamUsageEstimator.sizeOf(docs) - + ramBytesOrZero(shardSegmentDocMapForwards) + ramBytesOrZero(shardSegmentDocMapBackwards); + return BASE_RAM_BYTES_USED + RamUsageEstimator.sizeOf(shards) + RamUsageEstimator.sizeOf(uniqueShards) + RamUsageEstimator.sizeOf( + segments + ) + RamUsageEstimator.sizeOf(docs) + ramBytesOrZero(shardSegmentDocMapForwards) + ramBytesOrZero(shardSegmentDocMapBackwards); } @Override public long ramBytesUsed() { - return ramBytesEstimated(shards, segments, docs, shardSegmentDocMapForwards, shardSegmentDocMapBackwards); + return ramBytesEstimated(shards, uniqueShards, segments, docs, shardSegmentDocMapForwards, shardSegmentDocMapBackwards); } @Override @@ -337,33 +368,22 @@ public void closeInternal() { Releasables.closeExpectNoException( () -> blockFactory().adjustBreaker(-BASE_RAM_BYTES_USED - (shardSegmentDocMapForwards == null ? 0 : sizeOfSegmentDocMap())), shards, + uniqueShards, segments, docs ); - forEachShardRefCounter(DecOrInc.DEC); - } - - private enum DecOrInc { - DEC, - INC; - - void apply(ShardRefCounted counters, int shardId) { - switch (this) { - case DEC -> counters.get(shardId).decRef(); - case INC -> counters.get(shardId).mustIncRef(); - } - } + forEachShardRefCounter(RefCounted::decRef); } - private void forEachShardRefCounter(DecOrInc mode) { + private void forEachShardRefCounter(Consumer consumer) { switch (shards) { - case ConstantIntVector constantIntVector -> mode.apply(shardRefCounters, constantIntVector.getInt(0)); + case ConstantIntVector constantIntVector -> consumer.accept(shardRefCounters.get(constantIntVector.getInt(0))); case ConstantNullVector ignored -> { // Noop } default -> { - for (int i = 0; i < shards.getPositionCount(); i++) { - mode.apply(shardRefCounters, shards.getInt(i)); + for (int i = 0; i < uniqueShards.getPositionCount(); i++) { + consumer.accept(shardRefCounters.get(uniqueShards.getInt(i))); } } } diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/ShardRefCounted.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/ShardRefCounted.java index a1caa0dfae3d4..e63d4ab0641f3 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/ShardRefCounted.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/ShardRefCounted.java @@ -13,6 +13,10 @@ /** Manages reference counting for {@link ShardContext}. */ public interface ShardRefCounted { + /** + * @param shardId The shard index used by {@link org.elasticsearch.compute.data.DocVector}. + * @return the {@link RefCounted} for the given shard. In production, this will almost always be a {@link ShardContext}. + */ RefCounted get(int shardId); static ShardRefCounted fromList(List refCounters) { diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/EsPhysicalOperationProviders.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/EsPhysicalOperationProviders.java index 3cb9c3302bb8c..58493f07c0ee1 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/EsPhysicalOperationProviders.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/planner/EsPhysicalOperationProviders.java @@ -92,7 +92,8 @@ public class EsPhysicalOperationProviders extends AbstractPhysicalOperationProvi private static final Logger logger = LogManager.getLogger(EsPhysicalOperationProviders.class); /** - * Context of each shard we're operating against. + * Context of each shard we're operating against. Note these objects are shared across multiple operators as + * {@link org.elasticsearch.core.RefCounted}. */ public abstract static class ShardContext implements org.elasticsearch.compute.lucene.ShardContext, Releasable { private final AbstractRefCounted refCounted = new AbstractRefCounted() { @@ -379,6 +380,10 @@ public Operator.OperatorFactory timeSeriesAggregatorOperatorFactory( public static class DefaultShardContext extends ShardContext { private final int index; + /** + * In production, this will be a {@link org.elasticsearch.search.internal.SearchContext}, but we don't want to drag that huge + * dependency here. + */ private final Releasable releasable; private final SearchExecutionContext ctx; private final AliasFilter aliasFilter; diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/ComputeService.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/ComputeService.java index a454cdfddce9b..234336a53d1c2 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/ComputeService.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/ComputeService.java @@ -590,6 +590,8 @@ public SourceProvider createSourceProvider() { LOGGER.debug("Local execution plan:\n{}", localExecutionPlan.describe()); } drivers = localExecutionPlan.createDrivers(context.sessionId()); + // After creating the drivers (and therefore, the operators), we can safely decrement the reference count since the operators + // will hold a reference to the contexts where relevant. contexts.forEach(RefCounted::decRef); if (drivers.isEmpty()) { throw new IllegalStateException("no drivers created"); From 1ad8eb4e3786217b870d3e8de6e5b7365b0d11f0 Mon Sep 17 00:00:00 2001 From: Gal Lalouche Date: Tue, 17 Jun 2025 15:48:50 +0300 Subject: [PATCH 10/16] More test fixes --- .../elasticsearch/compute/data/DocVector.java | 16 +++++++++++----- .../elasticsearch/compute/operator/Driver.java | 2 +- .../operator/OrdinalsGroupingOperator.java | 3 +-- 3 files changed, 13 insertions(+), 8 deletions(-) diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/DocVector.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/DocVector.java index 126783bbb9d5b..f900ce4896115 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/DocVector.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/DocVector.java @@ -82,10 +82,15 @@ public DocVector( "invalid position count [" + shards.getPositionCount() + " != " + docs.getPositionCount() + "]" ); } - - this.uniqueShards = computeUniqueShards(shards); - forEachShardRefCounter(RefCounted::mustIncRef); - blockFactory().adjustBreaker(BASE_RAM_BYTES_USED); + var uniqueShards = computeUniqueShards(shards); + try { + blockFactory().adjustBreaker(BASE_RAM_BYTES_USED); + this.uniqueShards = uniqueShards; + forEachShardRefCounter(RefCounted::mustIncRef); + } catch (Exception e) { + Releasables.close(uniqueShards); + throw e; + } } private static IntVector computeUniqueShards(IntVector shards) { @@ -359,12 +364,14 @@ public long ramBytesUsed() { public void allowPassingToDifferentDriver() { super.allowPassingToDifferentDriver(); shards.allowPassingToDifferentDriver(); + uniqueShards.allowPassingToDifferentDriver(); segments.allowPassingToDifferentDriver(); docs.allowPassingToDifferentDriver(); } @Override public void closeInternal() { + forEachShardRefCounter(RefCounted::decRef); Releasables.closeExpectNoException( () -> blockFactory().adjustBreaker(-BASE_RAM_BYTES_USED - (shardSegmentDocMapForwards == null ? 0 : sizeOfSegmentDocMap())), shards, @@ -372,7 +379,6 @@ public void closeInternal() { segments, docs ); - forEachShardRefCounter(RefCounted::decRef); } private void forEachShardRefCounter(Consumer consumer) { diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/Driver.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/Driver.java index 7942dfd4b1f12..775ac401cd916 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/Driver.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/Driver.java @@ -317,7 +317,7 @@ private int closeEarlyFinishedOperators(ListIterator operators) { if (iterator.previous().isFinished()) { var index = iterator.nextIndex(); /* - * Remove this operator and all source operators in the + * Close and remove this operator and all source operators in the * most paranoid possible way. Closing operators shouldn't throw, * but if it does, this will make sure we don't try to close any * that succeed twice. diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/OrdinalsGroupingOperator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/OrdinalsGroupingOperator.java index 40f1b44dbf0e3..586e37ed0a13b 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/OrdinalsGroupingOperator.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/OrdinalsGroupingOperator.java @@ -357,8 +357,8 @@ static final class OrdinalSegmentAggregator implements Releasable, SeenGroupIds RefCounted shardRefCounted ) throws IOException { boolean success = false; - shardRefCounted.mustIncRef(); this.shardRefCounted = shardRefCounted; + this.shardRefCounted.mustIncRef(); List groupingAggregators = null; BitArray bitArray = null; try { @@ -370,7 +370,6 @@ static final class OrdinalSegmentAggregator implements Releasable, SeenGroupIds this.docValuesSupplier = docValuesSupplier; this.aggregators = groupingAggregators; this.visitedOrds = bitArray; - this.shardRefCounted.mustIncRef(); success = true; } finally { if (success == false) { From b0a35b11955d929d1177fa5e877adad85919260c Mon Sep 17 00:00:00 2001 From: Gal Lalouche Date: Tue, 17 Jun 2025 17:38:03 +0300 Subject: [PATCH 11/16] Fix random DocVector generation (shard cannot be negative) --- .../java/org/elasticsearch/compute/test/BlockTestUtils.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/x-pack/plugin/esql/compute/test/src/main/java/org/elasticsearch/compute/test/BlockTestUtils.java b/x-pack/plugin/esql/compute/test/src/main/java/org/elasticsearch/compute/test/BlockTestUtils.java index 80f6cbdb81e8b..442d32d6f64b8 100644 --- a/x-pack/plugin/esql/compute/test/src/main/java/org/elasticsearch/compute/test/BlockTestUtils.java +++ b/x-pack/plugin/esql/compute/test/src/main/java/org/elasticsearch/compute/test/BlockTestUtils.java @@ -38,6 +38,7 @@ import static org.elasticsearch.test.ESTestCase.randomFloat; import static org.elasticsearch.test.ESTestCase.randomInt; import static org.elasticsearch.test.ESTestCase.randomLong; +import static org.elasticsearch.test.ESTestCase.randomNonNegativeInt; import static org.elasticsearch.test.ESTestCase.randomRealisticUnicodeOfCodepointLengthBetween; import static org.hamcrest.Matchers.equalTo; import static org.junit.Assert.assertThat; @@ -54,7 +55,7 @@ public static Object randomValue(ElementType e) { case DOUBLE -> randomDouble(); case BYTES_REF -> new BytesRef(randomRealisticUnicodeOfCodepointLengthBetween(0, 5)); // TODO: also test spatial WKB case BOOLEAN -> randomBoolean(); - case DOC -> new BlockUtils.Doc(randomInt(), randomInt(), between(0, Integer.MAX_VALUE)); + case DOC -> new BlockUtils.Doc(randomNonNegativeInt(), randomInt(), between(0, Integer.MAX_VALUE)); case NULL -> null; case COMPOSITE -> throw new IllegalArgumentException("can't make random values for composite"); case AGGREGATE_METRIC_DOUBLE -> throw new IllegalArgumentException("can't make random values for aggregate_metric_double"); From 73ddfe719d9e26b8ad1dca4dbca533df6f27940f Mon Sep 17 00:00:00 2001 From: Gal Lalouche Date: Tue, 17 Jun 2025 20:00:26 +0300 Subject: [PATCH 12/16] More edge cases for shard IDs in tests --- .../compute/operator/topn/ExtractorTests.java | 3 ++- .../org/elasticsearch/compute/test/BlockTestUtils.java | 8 ++++++-- 2 files changed, 8 insertions(+), 3 deletions(-) diff --git a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/topn/ExtractorTests.java b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/topn/ExtractorTests.java index 0db2d82a09ac2..b345d8c0b196a 100644 --- a/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/topn/ExtractorTests.java +++ b/x-pack/plugin/esql/compute/src/test/java/org/elasticsearch/compute/operator/topn/ExtractorTests.java @@ -97,7 +97,8 @@ public static Iterable parameters() { TopNEncoder.DEFAULT_UNSORTABLE, () -> new DocVector( ShardRefCounted.ALWAYS_REFERENCED, - blockFactory.newConstantIntBlockWith(randomInt(), 1).asVector(), + // Shard ID should be small and non-negative. + blockFactory.newConstantIntBlockWith(randomIntBetween(0, 255), 1).asVector(), blockFactory.newConstantIntBlockWith(randomInt(), 1).asVector(), blockFactory.newConstantIntBlockWith(randomInt(), 1).asVector(), randomBoolean() ? null : randomBoolean() diff --git a/x-pack/plugin/esql/compute/test/src/main/java/org/elasticsearch/compute/test/BlockTestUtils.java b/x-pack/plugin/esql/compute/test/src/main/java/org/elasticsearch/compute/test/BlockTestUtils.java index 442d32d6f64b8..dcfec4b268aa9 100644 --- a/x-pack/plugin/esql/compute/test/src/main/java/org/elasticsearch/compute/test/BlockTestUtils.java +++ b/x-pack/plugin/esql/compute/test/src/main/java/org/elasticsearch/compute/test/BlockTestUtils.java @@ -37,8 +37,8 @@ import static org.elasticsearch.test.ESTestCase.randomDouble; import static org.elasticsearch.test.ESTestCase.randomFloat; import static org.elasticsearch.test.ESTestCase.randomInt; +import static org.elasticsearch.test.ESTestCase.randomIntBetween; import static org.elasticsearch.test.ESTestCase.randomLong; -import static org.elasticsearch.test.ESTestCase.randomNonNegativeInt; import static org.elasticsearch.test.ESTestCase.randomRealisticUnicodeOfCodepointLengthBetween; import static org.hamcrest.Matchers.equalTo; import static org.junit.Assert.assertThat; @@ -55,7 +55,11 @@ public static Object randomValue(ElementType e) { case DOUBLE -> randomDouble(); case BYTES_REF -> new BytesRef(randomRealisticUnicodeOfCodepointLengthBetween(0, 5)); // TODO: also test spatial WKB case BOOLEAN -> randomBoolean(); - case DOC -> new BlockUtils.Doc(randomNonNegativeInt(), randomInt(), between(0, Integer.MAX_VALUE)); + case DOC -> new BlockUtils.Doc( + randomIntBetween(0, 255), // Shard ID should be small and non-negative. + randomInt(), + between(0, Integer.MAX_VALUE) + ); case NULL -> null; case COMPOSITE -> throw new IllegalArgumentException("can't make random values for composite"); case AGGREGATE_METRIC_DOUBLE -> throw new IllegalArgumentException("can't make random values for aggregate_metric_double"); From e621964454cbf4288daa07e9fa1a9ca405afead4 Mon Sep 17 00:00:00 2001 From: Gal Lalouche Date: Mon, 23 Jun 2025 22:07:49 +0300 Subject: [PATCH 13/16] CR comments --- .../elasticsearch/compute/lucene/LuceneOperator.java | 11 +---------- .../compute/operator/topn/ResultBuilder.java | 8 ++++++++ .../compute/operator/topn/ResultBuilderForDoc.java | 3 ++- .../compute/operator/topn/TopNOperator.java | 10 ++++------ .../compute/operator/topn/ValueExtractor.java | 11 +++++++++++ .../compute/operator/topn/ValueExtractorForDoc.java | 6 ++++++ 6 files changed, 32 insertions(+), 17 deletions(-) diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneOperator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneOperator.java index 8ce7eaabf7c95..0da3915c9ad0c 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneOperator.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneOperator.java @@ -138,11 +138,7 @@ public final Page getOutput() { protected abstract Page getCheckedOutput() throws IOException; @Override - public void close() { - if (currentScorer != null) { - currentScorer.shardContext().decRef(); - } - } + public void close() {} LuceneScorer getCurrentOrLoadNextScorer() { while (currentScorer == null || currentScorer.isDone()) { @@ -165,11 +161,7 @@ LuceneScorer getCurrentOrLoadNextScorer() { ) { final Weight weight = currentSlice.weight(); processedQueries.add(weight.getQuery()); - var previousScorer = currentScorer; currentScorer = new LuceneScorer(currentSlice.shardContext(), weight, currentSlice.tags(), leaf); - if (previousScorer != null) { - previousScorer.shardContext().decRef(); - } } assert currentScorer.maxPosition <= partialLeaf.maxDoc() : currentScorer.maxPosition + ">" + partialLeaf.maxDoc(); currentScorer.maxPosition = partialLeaf.maxDoc(); @@ -196,7 +188,6 @@ static final class LuceneScorer { private Thread executingThread; LuceneScorer(ShardContext shardContext, Weight weight, List tags, LeafReaderContext leafReaderContext) { - shardContext.incRef(); this.shardContext = shardContext; this.weight = weight; this.tags = tags; diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/topn/ResultBuilder.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/topn/ResultBuilder.java index 6ad550c439ecf..c3da40254c09a 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/topn/ResultBuilder.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/topn/ResultBuilder.java @@ -11,6 +11,8 @@ import org.elasticsearch.compute.data.Block; import org.elasticsearch.compute.data.BlockFactory; import org.elasticsearch.compute.data.ElementType; +import org.elasticsearch.core.Nullable; +import org.elasticsearch.core.RefCounted; import org.elasticsearch.core.Releasable; /** @@ -33,6 +35,12 @@ interface ResultBuilder extends Releasable { */ void decodeValue(BytesRef values); + /** + * Sets the RefCounted value, which was extracted by {@link ValueExtractor#getRefCountedForShard(int)}. By default, this is a no-op, + * since most builders do not the shard ref counter. + */ + default void setNextRefCounted(@Nullable RefCounted nextRefCounted) { /* no-op */ } + /** * Build the result block. */ diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/topn/ResultBuilderForDoc.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/topn/ResultBuilderForDoc.java index c388692810593..cb659e8921aa2 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/topn/ResultBuilderForDoc.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/topn/ResultBuilderForDoc.java @@ -42,7 +42,8 @@ public void decodeKey(BytesRef keys) { throw new AssertionError("_doc can't be a key"); } - void setNextRefCounted(RefCounted nextRefCounted) { + @Override + public void setNextRefCounted(RefCounted nextRefCounted) { this.nextRefCounted = nextRefCounted; // Since rows can be closed before build is called, we need to increment the ref count to ensure the shard context isn't closed. this.nextRefCounted.mustIncRef(); diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/topn/TopNOperator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/topn/TopNOperator.java index bb5c42807e89f..fdf88cf8f55b4 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/topn/TopNOperator.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/topn/TopNOperator.java @@ -215,8 +215,9 @@ private void writeKey(int position, Row row) { private void writeValues(int position, Row destination) { for (ValueExtractor e : valueExtractors) { - if (e instanceof ValueExtractorForDoc fd) { - destination.setShardRefCountersAndShard(fd.vector().shardRefCounted().get(fd.vector().shards().getInt(position))); + var refCounted = e.getRefCountedForShard(position); + if (refCounted != null) { + destination.setShardRefCountersAndShard(refCounted); } e.writeValue(destination.values, position); } @@ -486,10 +487,7 @@ private Iterator toPages() { BytesRef values = row.values.bytesRefView(); for (ResultBuilder builder : builders) { - if (builder instanceof ResultBuilderForDoc fd) { - assert row.shardRefCounter != null : "shardRefCounter must be set for ResultBuilderForDoc"; - fd.setNextRefCounted(row.shardRefCounter); - } + builder.setNextRefCounted(row.shardRefCounter); builder.decodeValue(values); } if (values.length != 0) { diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/topn/ValueExtractor.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/topn/ValueExtractor.java index ccf36a08c280b..b6f3a1198d1ff 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/topn/ValueExtractor.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/topn/ValueExtractor.java @@ -18,6 +18,8 @@ import org.elasticsearch.compute.data.IntBlock; import org.elasticsearch.compute.data.LongBlock; import org.elasticsearch.compute.operator.BreakingBytesRefBuilder; +import org.elasticsearch.core.Nullable; +import org.elasticsearch.core.RefCounted; /** * Extracts values into a {@link BreakingBytesRefBuilder}. @@ -25,6 +27,15 @@ interface ValueExtractor { void writeValue(BreakingBytesRefBuilder values, int position); + /** + * This should return a non-null value if the row is supposed to hold a temporary reference to a shard (including incrementing and + * decrementing it) in between encoding and decoding the row values. + */ + @Nullable + default RefCounted getRefCountedForShard(int position) { + return null; + } + static ValueExtractor extractorFor(ElementType elementType, TopNEncoder encoder, boolean inKey, Block block) { if (false == (elementType == block.elementType() || ElementType.NULL == block.elementType())) { // While this maybe should be an IllegalArgumentException, it's important to throw an exception that causes a 500 response. diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/topn/ValueExtractorForDoc.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/topn/ValueExtractorForDoc.java index db0985275080d..e0d7cffabdfbb 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/topn/ValueExtractorForDoc.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/topn/ValueExtractorForDoc.java @@ -9,10 +9,16 @@ import org.elasticsearch.compute.data.DocVector; import org.elasticsearch.compute.operator.BreakingBytesRefBuilder; +import org.elasticsearch.core.RefCounted; class ValueExtractorForDoc implements ValueExtractor { private final DocVector vector; + @Override + public RefCounted getRefCountedForShard(int position) { + return vector().shardRefCounted().get(vector().shards().getInt(position)); + } + ValueExtractorForDoc(TopNEncoder encoder, DocVector vector) { assert encoder == TopNEncoder.DEFAULT_UNSORTABLE; this.vector = vector; From 5246f3a7f1f133a352d816ee3d889730c6c0ef8d Mon Sep 17 00:00:00 2001 From: Gal Lalouche Date: Tue, 24 Jun 2025 13:26:09 +0300 Subject: [PATCH 14/16] CR fixes --- .../elasticsearch/compute/data/DocVector.java | 52 +++---------------- .../operator/OrdinalsGroupingOperator.java | 2 + 2 files changed, 10 insertions(+), 44 deletions(-) diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/DocVector.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/DocVector.java index f900ce4896115..20ca4ed70e3f8 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/DocVector.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/data/DocVector.java @@ -15,7 +15,6 @@ import org.elasticsearch.core.ReleasableIterator; import org.elasticsearch.core.Releasables; -import java.util.BitSet; import java.util.Objects; import java.util.function.Consumer; @@ -33,7 +32,6 @@ public final class DocVector extends AbstractVector implements Vector { public static final int SHARD_SEGMENT_DOC_MAP_PER_ROW_OVERHEAD = Integer.BYTES * 2; private final IntVector shards; - private final IntVector uniqueShards; private final IntVector segments; private final IntVector docs; @@ -82,39 +80,9 @@ public DocVector( "invalid position count [" + shards.getPositionCount() + " != " + docs.getPositionCount() + "]" ); } - var uniqueShards = computeUniqueShards(shards); - try { - blockFactory().adjustBreaker(BASE_RAM_BYTES_USED); - this.uniqueShards = uniqueShards; - forEachShardRefCounter(RefCounted::mustIncRef); - } catch (Exception e) { - Releasables.close(uniqueShards); - throw e; - } - } + blockFactory().adjustBreaker(BASE_RAM_BYTES_USED); - private static IntVector computeUniqueShards(IntVector shards) { - switch (shards) { - case ConstantIntVector constantIntVector -> { - return shards.blockFactory().newConstantIntVector(constantIntVector.getInt(0), 1); - } - case ConstantNullVector unused -> { - return shards.blockFactory().newConstantIntVector(0, 0); - } - default -> { - var seen = new BitSet(128); - try (IntVector.Builder uniqueShardsBuilder = shards.blockFactory().newIntVectorBuilder(shards.getPositionCount())) { - for (int p = 0; p < shards.getPositionCount(); p++) { - int shardId = shards.getInt(p); - if (seen.get(shardId) == false) { - seen.set(shardId); - uniqueShardsBuilder.appendInt(shardId); - } - } - return uniqueShardsBuilder.build(); - } - } - } + forEachShardRefCounter(RefCounted::mustIncRef); } public DocVector( @@ -344,41 +312,37 @@ private static long ramBytesOrZero(int[] array) { public static long ramBytesEstimated( IntVector shards, - IntVector uniqueShards, IntVector segments, IntVector docs, int[] shardSegmentDocMapForwards, int[] shardSegmentDocMapBackwards ) { - return BASE_RAM_BYTES_USED + RamUsageEstimator.sizeOf(shards) + RamUsageEstimator.sizeOf(uniqueShards) + RamUsageEstimator.sizeOf( - segments - ) + RamUsageEstimator.sizeOf(docs) + ramBytesOrZero(shardSegmentDocMapForwards) + ramBytesOrZero(shardSegmentDocMapBackwards); + return BASE_RAM_BYTES_USED + RamUsageEstimator.sizeOf(shards) + RamUsageEstimator.sizeOf(segments) + RamUsageEstimator.sizeOf(docs) + + ramBytesOrZero(shardSegmentDocMapForwards) + ramBytesOrZero(shardSegmentDocMapBackwards); } @Override public long ramBytesUsed() { - return ramBytesEstimated(shards, uniqueShards, segments, docs, shardSegmentDocMapForwards, shardSegmentDocMapBackwards); + return ramBytesEstimated(shards, segments, docs, shardSegmentDocMapForwards, shardSegmentDocMapBackwards); } @Override public void allowPassingToDifferentDriver() { super.allowPassingToDifferentDriver(); shards.allowPassingToDifferentDriver(); - uniqueShards.allowPassingToDifferentDriver(); segments.allowPassingToDifferentDriver(); docs.allowPassingToDifferentDriver(); } @Override public void closeInternal() { - forEachShardRefCounter(RefCounted::decRef); Releasables.closeExpectNoException( () -> blockFactory().adjustBreaker(-BASE_RAM_BYTES_USED - (shardSegmentDocMapForwards == null ? 0 : sizeOfSegmentDocMap())), shards, - uniqueShards, segments, docs ); + forEachShardRefCounter(RefCounted::decRef); } private void forEachShardRefCounter(Consumer consumer) { @@ -388,8 +352,8 @@ private void forEachShardRefCounter(Consumer consumer) { // Noop } default -> { - for (int i = 0; i < uniqueShards.getPositionCount(); i++) { - consumer.accept(shardRefCounters.get(uniqueShards.getInt(i))); + for (int i = 0; i < shards.getPositionCount(); i++) { + consumer.accept(shardRefCounters.get(shards.getInt(i))); } } } diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/OrdinalsGroupingOperator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/OrdinalsGroupingOperator.java index 586e37ed0a13b..9c15b0f3fc7d5 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/OrdinalsGroupingOperator.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/operator/OrdinalsGroupingOperator.java @@ -375,6 +375,8 @@ static final class OrdinalSegmentAggregator implements Releasable, SeenGroupIds if (success == false) { if (bitArray != null) Releasables.close(bitArray); if (groupingAggregators != null) Releasables.close(groupingAggregators); + // There is no danger of double decRef here, since this decRef is called only if the constructor throws, so it would be + // impossible to call close on the instance. shardRefCounted.decRef(); } } From 627083349effddd3376e6efa24f49a6a5f5e7643 Mon Sep 17 00:00:00 2001 From: Gal Lalouche Date: Tue, 24 Jun 2025 18:27:18 +0300 Subject: [PATCH 15/16] Move shardRefCounter logic to the super class --- .../compute/lucene/LuceneCountOperator.java | 10 +--------- .../compute/lucene/LuceneMaxFactory.java | 4 +++- .../compute/lucene/LuceneMinFactory.java | 13 ++++++++++++- .../compute/lucene/LuceneMinMaxOperator.java | 5 ++++- .../compute/lucene/LuceneOperator.java | 18 ++++++++++++++++-- .../compute/lucene/LuceneSourceOperator.java | 18 +++++++----------- .../lucene/LuceneTopNSourceOperator.java | 9 +-------- .../lucene/TimeSeriesSourceOperator.java | 19 +++++-------------- 8 files changed, 49 insertions(+), 47 deletions(-) diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneCountOperator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneCountOperator.java index 5fc62db0a4bd1..626f0b00f0e2c 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneCountOperator.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneCountOperator.java @@ -35,7 +35,6 @@ public class LuceneCountOperator extends LuceneOperator { private static final int PAGE_SIZE = 1; - private final List shardRefCounters; private int totalHits = 0; private int remainingDocs; @@ -81,9 +80,7 @@ public LuceneCountOperator( LuceneSliceQueue sliceQueue, int limit ) { - super(blockFactory, PAGE_SIZE, sliceQueue); - this.shardRefCounters = shardRefCounters; - shardRefCounters.forEach(RefCounted::mustIncRef); + super(shardRefCounters, blockFactory, PAGE_SIZE, sliceQueue); this.remainingDocs = limit; this.leafCollector = new LeafCollector() { @Override @@ -182,9 +179,4 @@ protected Page getCheckedOutput() throws IOException { protected void describe(StringBuilder sb) { sb.append(", remainingDocs=").append(remainingDocs); } - - @Override - public void close() { - shardRefCounters.forEach(RefCounted::decRef); - } } diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneMaxFactory.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneMaxFactory.java index 49a6471b3e708..82d766349ce9e 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneMaxFactory.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneMaxFactory.java @@ -108,6 +108,7 @@ public final long evaluate(long value1, long value2) { abstract long bytesToLong(byte[] bytes); } + private final List contexts; private final String fieldName; private final NumberType numberType; @@ -130,13 +131,14 @@ public LuceneMaxFactory( false, ScoreMode.COMPLETE_NO_SCORES ); + this.contexts = contexts; this.fieldName = fieldName; this.numberType = numberType; } @Override public SourceOperator get(DriverContext driverContext) { - return new LuceneMinMaxOperator(driverContext.blockFactory(), sliceQueue, fieldName, numberType, limit, Long.MIN_VALUE); + return new LuceneMinMaxOperator(contexts, driverContext.blockFactory(), sliceQueue, fieldName, numberType, limit, Long.MIN_VALUE); } @Override diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneMinFactory.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneMinFactory.java index 1abb2e7f8085a..505e5cd3f0d75 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneMinFactory.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneMinFactory.java @@ -16,6 +16,7 @@ import org.elasticsearch.compute.data.BlockFactory; import org.elasticsearch.compute.operator.DriverContext; import org.elasticsearch.compute.operator.SourceOperator; +import org.elasticsearch.core.RefCounted; import org.elasticsearch.search.MultiValueMode; import java.io.IOException; @@ -108,6 +109,7 @@ public final long evaluate(long value1, long value2) { abstract long bytesToLong(byte[] bytes); } + private final List shardRefCounters; private final String fieldName; private final NumberType numberType; @@ -130,13 +132,22 @@ public LuceneMinFactory( false, ScoreMode.COMPLETE_NO_SCORES ); + this.shardRefCounters = contexts; this.fieldName = fieldName; this.numberType = numberType; } @Override public SourceOperator get(DriverContext driverContext) { - return new LuceneMinMaxOperator(driverContext.blockFactory(), sliceQueue, fieldName, numberType, limit, Long.MAX_VALUE); + return new LuceneMinMaxOperator( + shardRefCounters, + driverContext.blockFactory(), + sliceQueue, + fieldName, + numberType, + limit, + Long.MAX_VALUE + ); } @Override diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneMinMaxOperator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneMinMaxOperator.java index d0b508f14025e..b9e05567411fc 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneMinMaxOperator.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneMinMaxOperator.java @@ -20,10 +20,12 @@ import org.elasticsearch.compute.data.BlockFactory; import org.elasticsearch.compute.data.BooleanBlock; import org.elasticsearch.compute.data.Page; +import org.elasticsearch.core.RefCounted; import org.elasticsearch.core.Releasables; import org.elasticsearch.search.MultiValueMode; import java.io.IOException; +import java.util.List; /** * Operator that finds the min or max value of a field using Lucene searches @@ -65,6 +67,7 @@ sealed interface NumberType permits LuceneMinFactory.NumberType, LuceneMaxFactor private final String fieldName; LuceneMinMaxOperator( + List shardRefCounters, BlockFactory blockFactory, LuceneSliceQueue sliceQueue, String fieldName, @@ -72,7 +75,7 @@ sealed interface NumberType permits LuceneMinFactory.NumberType, LuceneMaxFactor int limit, long initialResult ) { - super(blockFactory, PAGE_SIZE, sliceQueue); + super(shardRefCounters, blockFactory, PAGE_SIZE, sliceQueue); this.remainingDocs = limit; this.numberType = numberType; this.fieldName = fieldName; diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneOperator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneOperator.java index 0da3915c9ad0c..5033476fbe584 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneOperator.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneOperator.java @@ -25,6 +25,7 @@ import org.elasticsearch.compute.data.Page; import org.elasticsearch.compute.operator.Operator; import org.elasticsearch.compute.operator.SourceOperator; +import org.elasticsearch.core.RefCounted; import org.elasticsearch.core.TimeValue; import org.elasticsearch.logging.LogManager; import org.elasticsearch.logging.Logger; @@ -52,6 +53,7 @@ public abstract class LuceneOperator extends SourceOperator { public static final int NO_LIMIT = Integer.MAX_VALUE; + protected final List shardContextCounters; protected final BlockFactory blockFactory; /** @@ -77,7 +79,14 @@ public abstract class LuceneOperator extends SourceOperator { */ long rowsEmitted; - protected LuceneOperator(BlockFactory blockFactory, int maxPageSize, LuceneSliceQueue sliceQueue) { + protected LuceneOperator( + List shardContextCounters, + BlockFactory blockFactory, + int maxPageSize, + LuceneSliceQueue sliceQueue + ) { + this.shardContextCounters = shardContextCounters; + shardContextCounters.forEach(RefCounted::incRef); this.blockFactory = blockFactory; this.maxPageSize = maxPageSize; this.sliceQueue = sliceQueue; @@ -138,7 +147,12 @@ public final Page getOutput() { protected abstract Page getCheckedOutput() throws IOException; @Override - public void close() {} + public final void close() { + shardContextCounters.forEach(RefCounted::decRef); + additionalClose(); + } + + protected void additionalClose() { /* Override this method to add any additional cleanup logic if needed */ } LuceneScorer getCurrentOrLoadNextScorer() { while (currentScorer == null || currentScorer.isDone()) { diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneSourceOperator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneSourceOperator.java index d8dd90f1d53f2..862f11007c233 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneSourceOperator.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneSourceOperator.java @@ -49,7 +49,6 @@ */ public class LuceneSourceOperator extends LuceneOperator { private static final Logger log = LogManager.getLogger(LuceneSourceOperator.class); - private final List shardContextCounters; private int currentPagePos = 0; private int remainingDocs; @@ -227,8 +226,7 @@ public LuceneSourceOperator( Limiter limiter, boolean needsScore ) { - super(blockFactory, maxPageSize, sliceQueue); - this.shardContextCounters = shardContextCounters; + super(shardContextCounters, blockFactory, maxPageSize, sliceQueue); shardContextCounters.forEach(RefCounted::mustIncRef); this.minPageSize = Math.max(1, maxPageSize / 2); this.remainingDocs = limit; @@ -330,12 +328,14 @@ public Page getCheckedOutput() throws IOException { Block[] blocks = new Block[1 + (scoreBuilder == null ? 0 : 1) + scorer.tags().size()]; currentPagePos -= discardedDocs; try { - shard = blockFactory.newConstantIntVector(scorer.shardContext().index(), currentPagePos); + int shardId = scorer.shardContext().index(); + shard = blockFactory.newConstantIntVector(shardId, currentPagePos); leaf = blockFactory.newConstantIntVector(scorer.leafReaderContext().ord, currentPagePos); docs = buildDocsVector(currentPagePos); docsBuilder = blockFactory.newIntVectorBuilder(Math.min(remainingDocs, maxPageSize)); int b = 0; - blocks[b++] = new DocVector(ShardRefCounted.fromList(shardContextCounters), shard, leaf, docs, true).asBlock(); + ShardRefCounted refCounted = ShardRefCounted.single(shardId, shardContextCounters.get(shardId)); + blocks[b++] = new DocVector(refCounted, shard, leaf, docs, true).asBlock(); shard = null; leaf = null; docs = null; @@ -393,12 +393,8 @@ private DoubleVector buildScoresVector(int upToPositions) { } @Override - public void close() { - Releasables.close( - docsBuilder, - scoreBuilder, - Releasables.wrap(shardContextCounters.stream().map(Releasables::fromRefCounted).toList()) - ); + public void additionalClose() { + Releasables.close(docsBuilder, scoreBuilder); } @Override diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneTopNSourceOperator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneTopNSourceOperator.java index 83d5efe62f6ce..eef018add5504 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneTopNSourceOperator.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneTopNSourceOperator.java @@ -132,9 +132,8 @@ public LuceneTopNSourceOperator( LuceneSliceQueue sliceQueue, boolean needsScore ) { - super(blockFactory, maxPageSize, sliceQueue); + super(contexts, blockFactory, maxPageSize, sliceQueue); this.contexts = contexts; - contexts.forEach(ShardContext::mustIncRef); this.sorts = sorts; this.limit = limit; this.needsScore = needsScore; @@ -370,10 +369,4 @@ private static PerShardCollector newPerShardCollector(ShardContext context, List sort = new Sort(l.toArray(SortField[]::new)); return new ScoringPerShardCollector(context, new TopFieldCollectorManager(sort, limit, null, 0).newCollector()); } - - @Override - public void close() { - super.close(); - contexts.forEach(ShardContext::decRef); - } } diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/TimeSeriesSourceOperator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/TimeSeriesSourceOperator.java index 0e2a65ca9a5ef..3800ffb101bf3 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/TimeSeriesSourceOperator.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/TimeSeriesSourceOperator.java @@ -43,7 +43,6 @@ import java.util.Set; public final class TimeSeriesSourceOperator extends LuceneOperator { - private final List contexts; private final int maxPageSize; private final BlockFactory blockFactory; private final LuceneSliceQueue sliceQueue; @@ -64,9 +63,8 @@ public final class TimeSeriesSourceOperator extends LuceneOperator { int maxPageSize, int limit ) { - super(blockFactory, maxPageSize, sliceQueue); + super(contexts, blockFactory, maxPageSize, sliceQueue); contexts.forEach(RefCounted::mustIncRef); - this.contexts = contexts; this.maxPageSize = maxPageSize; this.blockFactory = blockFactory; this.remainingDocs = limit; @@ -110,7 +108,7 @@ public Page getCheckedOutput() throws IOException { throw new UnsupportedOperationException("tags not supported by " + getClass()); } iterator = new SegmentsIterator(slice); - docCollector = new DocIdCollector(contexts, blockFactory, slice.shardContext()); + docCollector = new DocIdCollector(blockFactory, slice.shardContext()); } iterator.readDocsForNextPage(); if (currentPagePos > 0) { @@ -141,13 +139,8 @@ public Page getCheckedOutput() throws IOException { } @Override - public void close() { - Releasables.closeExpectNoException( - timestampsBuilder, - tsHashesBuilder, - docCollector, - Releasables.wrap(contexts.stream().map(Releasables::fromRefCounted).toList()) - ); + public void additionalClose() { + Releasables.closeExpectNoException(timestampsBuilder, tsHashesBuilder, docCollector); } class SegmentsIterator { @@ -365,14 +358,12 @@ OrdinalBytesRefVector build() throws IOException { } static final class DocIdCollector implements Releasable { - private final List contexts; private final BlockFactory blockFactory; private final ShardContext shardContext; private IntVector.Builder docsBuilder; private IntVector.Builder segmentsBuilder; - DocIdCollector(List contexts, BlockFactory blockFactory, ShardContext shardContext) { - this.contexts = contexts; + DocIdCollector(BlockFactory blockFactory, ShardContext shardContext) { this.blockFactory = blockFactory; this.shardContext = shardContext; } From de0d4bdaf67cd28951af3061be6e86d2238a4bc0 Mon Sep 17 00:00:00 2001 From: Gal Lalouche Date: Tue, 24 Jun 2025 20:12:04 +0300 Subject: [PATCH 16/16] Fix double ref counting from previous PR --- .../org/elasticsearch/compute/lucene/LuceneOperator.java | 2 +- .../elasticsearch/compute/lucene/LuceneSourceOperator.java | 1 - .../compute/lucene/LuceneTopNSourceOperator.java | 6 ++++-- .../compute/lucene/TimeSeriesSourceOperator.java | 2 -- 4 files changed, 5 insertions(+), 6 deletions(-) diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneOperator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneOperator.java index 5033476fbe584..366715530f665 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneOperator.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneOperator.java @@ -86,7 +86,7 @@ protected LuceneOperator( LuceneSliceQueue sliceQueue ) { this.shardContextCounters = shardContextCounters; - shardContextCounters.forEach(RefCounted::incRef); + shardContextCounters.forEach(RefCounted::mustIncRef); this.blockFactory = blockFactory; this.maxPageSize = maxPageSize; this.sliceQueue = sliceQueue; diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneSourceOperator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneSourceOperator.java index 862f11007c233..9fedc595641b4 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneSourceOperator.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneSourceOperator.java @@ -227,7 +227,6 @@ public LuceneSourceOperator( boolean needsScore ) { super(shardContextCounters, blockFactory, maxPageSize, sliceQueue); - shardContextCounters.forEach(RefCounted::mustIncRef); this.minPageSize = Math.max(1, maxPageSize / 2); this.remainingDocs = limit; this.limiter = limiter; diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneTopNSourceOperator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneTopNSourceOperator.java index eef018add5504..d93a5493a3aba 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneTopNSourceOperator.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/LuceneTopNSourceOperator.java @@ -241,10 +241,12 @@ private Page emit(boolean startEmitting) { } } - shard = blockFactory.newConstantIntBlockWith(perShardCollector.shardContext.index(), size); + int shardId = perShardCollector.shardContext.index(); + shard = blockFactory.newConstantIntBlockWith(shardId, size); segments = currentSegmentBuilder.build(); docs = currentDocsBuilder.build(); - docBlock = new DocVector(ShardRefCounted.fromList(contexts), shard.asVector(), segments, docs, null).asBlock(); + ShardRefCounted shardRefCounted = ShardRefCounted.single(shardId, contexts.get(shardId)); + docBlock = new DocVector(shardRefCounted, shard.asVector(), segments, docs, null).asBlock(); shard = null; segments = null; docs = null; diff --git a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/TimeSeriesSourceOperator.java b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/TimeSeriesSourceOperator.java index 3800ffb101bf3..089846f9939ae 100644 --- a/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/TimeSeriesSourceOperator.java +++ b/x-pack/plugin/esql/compute/src/main/java/org/elasticsearch/compute/lucene/TimeSeriesSourceOperator.java @@ -30,7 +30,6 @@ import org.elasticsearch.compute.data.OrdinalBytesRefVector; import org.elasticsearch.compute.data.Page; import org.elasticsearch.compute.operator.Operator; -import org.elasticsearch.core.RefCounted; import org.elasticsearch.core.Releasable; import org.elasticsearch.core.Releasables; import org.elasticsearch.xcontent.XContentBuilder; @@ -64,7 +63,6 @@ public final class TimeSeriesSourceOperator extends LuceneOperator { int limit ) { super(contexts, blockFactory, maxPageSize, sliceQueue); - contexts.forEach(RefCounted::mustIncRef); this.maxPageSize = maxPageSize; this.blockFactory = blockFactory; this.remainingDocs = limit;