From 8a37ca9560a5a0599ec13c13266310115b12b470 Mon Sep 17 00:00:00 2001 From: Andrei Dan Date: Thu, 6 Feb 2025 15:54:51 +0000 Subject: [PATCH 01/56] Account for the SearchHit source in circuit breaker This adds bytes accounting for the source loading of search hits in the request circuit breaker. --- .../org/elasticsearch/search/SearchHit.java | 59 +++++++++++---- .../elasticsearch/search/SearchService.java | 14 ++-- .../metrics/TopHitsAggregator.java | 7 +- .../search/fetch/FetchPhase.java | 73 ++++++++++++++----- .../search/fetch/subphase/InnerHitsPhase.java | 8 +- .../action/search/ExpandSearchPhaseTests.java | 20 +++-- .../search/FetchLookupFieldsPhaseTests.java | 8 +- .../action/search/FetchSearchPhaseTests.java | 7 +- .../search/SearchResponseMergerTests.java | 4 +- .../action/search/SearchResponseTests.java | 4 +- .../elasticsearch/search/SearchHitTests.java | 19 +++-- .../fetch/FetchPhaseDocsIteratorTests.java | 6 +- .../search/SearchResponseUtils.java | 3 +- 13 files changed, 163 insertions(+), 69 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/search/SearchHit.java b/server/src/main/java/org/elasticsearch/search/SearchHit.java index 8a70f8a7f41a6..00697022ee057 100644 --- a/server/src/main/java/org/elasticsearch/search/SearchHit.java +++ b/server/src/main/java/org/elasticsearch/search/SearchHit.java @@ -13,6 +13,8 @@ import org.elasticsearch.ElasticsearchParseException; import org.elasticsearch.TransportVersions; import org.elasticsearch.common.Strings; +import org.elasticsearch.common.breaker.CircuitBreaker; +import org.elasticsearch.common.breaker.NoopCircuitBreaker; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.compress.CompressorFactory; @@ -83,6 +85,8 @@ public final class SearchHit implements Writeable, ToXContentObject, RefCounted private long primaryTerm; private BytesReference source; + @Nullable + private BytesReference unfilteredSourceRef; private final Map documentFields; private final Map metaFields; @@ -110,21 +114,27 @@ public final class SearchHit implements Writeable, ToXContentObject, RefCounted private Map innerHits; private final RefCounted refCounted; + private final CircuitBreaker circuitBreaker; - // used only in tests - public SearchHit(int docId) { - this(docId, null); + public SearchHit(int docId, CircuitBreaker circuitBreaker) { + this(docId, null, circuitBreaker); } - public SearchHit(int docId, String id) { - this(docId, id, null); + public SearchHit(int docId, String id, CircuitBreaker circuitBreaker) { + this(docId, id, null, circuitBreaker); } - public SearchHit(int nestedTopDocId, String id, NestedIdentity nestedIdentity) { - this(nestedTopDocId, id, nestedIdentity, null); + public SearchHit(int nestedTopDocId, String id, NestedIdentity nestedIdentity, CircuitBreaker circuitBreaker) { + this(nestedTopDocId, id, nestedIdentity, null, circuitBreaker); } - private SearchHit(int nestedTopDocId, String id, NestedIdentity nestedIdentity, @Nullable RefCounted refCounted) { + private SearchHit( + int nestedTopDocId, + String id, + NestedIdentity nestedIdentity, + @Nullable RefCounted refCounted, + @Nullable CircuitBreaker circuitBreaker + ) { this( nestedTopDocId, DEFAULT_SCORE, @@ -145,7 +155,8 @@ private SearchHit(int nestedTopDocId, String id, NestedIdentity nestedIdentity, null, new HashMap<>(), new HashMap<>(), - refCounted + refCounted, + circuitBreaker ); } @@ -169,7 +180,8 @@ public SearchHit( Map innerHits, Map documentFields, Map metaFields, - @Nullable RefCounted refCounted + @Nullable RefCounted refCounted, + @Nullable CircuitBreaker circuitBreaker ) { this.docId = docId; this.score = score; @@ -191,6 +203,7 @@ public SearchHit( this.documentFields = documentFields; this.metaFields = metaFields; this.refCounted = refCounted == null ? LeakTracker.wrap(new SimpleRefCounted()) : refCounted; + this.circuitBreaker = circuitBreaker; } public static SearchHit readFrom(StreamInput in, boolean pooled) throws IOException { @@ -280,7 +293,8 @@ public static SearchHit readFrom(StreamInput in, boolean pooled) throws IOExcept innerHits, documentFields, metaFields, - isPooled ? null : ALWAYS_REFERENCED + isPooled ? null : ALWAYS_REFERENCED, + null ); } @@ -293,7 +307,8 @@ public static SearchHit unpooled(int docId, String id) { } public static SearchHit unpooled(int nestedTopDocId, String id, NestedIdentity nestedIdentity) { - return new SearchHit(nestedTopDocId, id, nestedIdentity, ALWAYS_REFERENCED); + // always referenced search hits do NOT call #deallocate + return new SearchHit(nestedTopDocId, id, nestedIdentity, ALWAYS_REFERENCED, null); } private static final Text SINGLE_MAPPING_TYPE = new Text(MapperService.SINGLE_MAPPING_NAME); @@ -447,6 +462,17 @@ public SearchHit sourceRef(BytesReference source) { return this; } + /** + * We track the unfiltered, entire, source so we can release the entire size from the + * circuit breakers when the hit is released. + * The regular source might be a subset of the unfiltered source due to either + * source filtering, field collapsing or inner hits. + */ + public SearchHit unfilteredSourceRef(BytesReference source) { + this.unfilteredSourceRef = source; + return this; + } + /** * Is the source available or not. A source with no fields will return true. This will return false if {@code fields} doesn't contain * {@code _source} or if source is disabled in the mapping. @@ -724,6 +750,12 @@ private void deallocate() { r.decRef(); } SearchHit.this.source = null; + + if (unfilteredSourceRef != null && circuitBreaker != null) { + System.out.println(" removing source loaded by inner hit " + unfilteredSourceRef.length()); + circuitBreaker.addWithoutBreaking(-unfilteredSourceRef.length()); + } + this.unfilteredSourceRef = null; } @Override @@ -758,7 +790,8 @@ public SearchHit asUnpooled() { : innerHits.entrySet().stream().collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().asUnpooled())), documentFields, metaFields, - ALWAYS_REFERENCED + ALWAYS_REFERENCED, + null ); } diff --git a/server/src/main/java/org/elasticsearch/search/SearchService.java b/server/src/main/java/org/elasticsearch/search/SearchService.java index efa27b2f3448c..e56d6c1d4c47a 100644 --- a/server/src/main/java/org/elasticsearch/search/SearchService.java +++ b/server/src/main/java/org/elasticsearch/search/SearchService.java @@ -290,6 +290,7 @@ public class SearchService extends AbstractLifecycleComponent implements IndexEv private final BigArrays bigArrays; private final FetchPhase fetchPhase; + private final CircuitBreaker circuitBreaker; private volatile Executor searchExecutor; private volatile boolean enableQueryPhaseParallelCollection; @@ -340,11 +341,8 @@ public SearchService( this.scriptService = scriptService; this.bigArrays = bigArrays; this.fetchPhase = fetchPhase; - this.multiBucketConsumerService = new MultiBucketConsumerService( - clusterService, - settings, - circuitBreakerService.getBreaker(CircuitBreaker.REQUEST) - ); + circuitBreaker = circuitBreakerService.getBreaker(CircuitBreaker.REQUEST); + this.multiBucketConsumerService = new MultiBucketConsumerService(clusterService, settings, circuitBreaker); this.executorSelector = executorSelector; this.tracer = tracer; @@ -788,7 +786,7 @@ public void executeRankFeaturePhase(RankFeatureShardRequest request, SearchShard return searchContext.rankFeatureResult(); } RankFeatureShardPhase.prepareForFetch(searchContext, request); - fetchPhase.execute(searchContext, docIds, null); + fetchPhase.execute(searchContext, docIds, null, circuitBreaker); RankFeatureShardPhase.processFetch(searchContext); var rankFeatureResult = searchContext.rankFeatureResult(); rankFeatureResult.incRef(); @@ -806,7 +804,7 @@ private QueryFetchSearchResult executeFetchPhase(ReaderContext reader, SearchCon Releasable scope = tracer.withScope(context.getTask()); SearchOperationListenerExecutor executor = new SearchOperationListenerExecutor(context, true, afterQueryTime) ) { - fetchPhase.execute(context, shortcutDocIdsToLoad(context), null); + fetchPhase.execute(context, shortcutDocIdsToLoad(context), null, circuitBreaker); if (reader.singleSession()) { freeReaderContext(reader.id()); } @@ -972,7 +970,7 @@ public void executeFetchPhase(ShardFetchRequest request, SearchShardTask task, A System.nanoTime() ) ) { - fetchPhase.execute(searchContext, request.docIds(), request.getRankDocks()); + fetchPhase.execute(searchContext, request.docIds(), request.getRankDocks(), circuitBreaker); if (readerContext.singleSession()) { freeReaderContext(request.contextId()); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/TopHitsAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/TopHitsAggregator.java index 87d8f839dfca1..bc7043b652d4a 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/TopHitsAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/TopHitsAggregator.java @@ -25,6 +25,7 @@ import org.apache.lucene.search.TopScoreDocCollectorManager; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.action.search.MaxScoreCollector; +import org.elasticsearch.common.breaker.CircuitBreaker; import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.lucene.search.TopDocsAndMaxScore; import org.elasticsearch.common.util.BigArrays; @@ -196,7 +197,7 @@ public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOE for (int i = 0; i < topDocs.scoreDocs.length; i++) { docIdsToLoad[i] = topDocs.scoreDocs[i].doc; } - FetchSearchResult fetchResult = runFetchPhase(subSearchContext, docIdsToLoad); + FetchSearchResult fetchResult = runFetchPhase(subSearchContext, docIdsToLoad, context.breaker()); if (fetchProfiles != null) { fetchProfiles.add(fetchResult.profileResult()); } @@ -220,7 +221,7 @@ public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOE ); } - private static FetchSearchResult runFetchPhase(SubSearchContext subSearchContext, int[] docIdsToLoad) { + private static FetchSearchResult runFetchPhase(SubSearchContext subSearchContext, int[] docIdsToLoad, CircuitBreaker breaker) { // Fork the search execution context for each slice, because the fetch phase does not support concurrent execution yet. SearchExecutionContext searchExecutionContext = new SearchExecutionContext(subSearchContext.getSearchExecutionContext()); SubSearchContext fetchSubSearchContext = new SubSearchContext(subSearchContext) { @@ -229,7 +230,7 @@ public SearchExecutionContext getSearchExecutionContext() { return searchExecutionContext; } }; - fetchSubSearchContext.fetchPhase().execute(fetchSubSearchContext, docIdsToLoad, null); + fetchSubSearchContext.fetchPhase().execute(fetchSubSearchContext, docIdsToLoad, null, breaker); return fetchSubSearchContext.fetchResult(); } diff --git a/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java b/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java index 2fbe3c1fc1532..46bf391faad0f 100644 --- a/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java +++ b/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java @@ -13,6 +13,9 @@ import org.apache.logging.log4j.Logger; import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.search.TotalHits; +import org.elasticsearch.common.breaker.CircuitBreaker; +import org.elasticsearch.common.breaker.NoopCircuitBreaker; +import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.index.fieldvisitor.LeafStoredFieldLoader; import org.elasticsearch.index.fieldvisitor.StoredFieldLoader; import org.elasticsearch.index.mapper.IdLoader; @@ -51,6 +54,7 @@ */ public final class FetchPhase { private static final Logger LOGGER = LogManager.getLogger(FetchPhase.class); + public static final NoopCircuitBreaker NOOP_CIRCUIT_BREAKER = new NoopCircuitBreaker("fetch_phase_nested_hits"); private final FetchSubPhase[] fetchSubPhases; @@ -59,7 +63,7 @@ public FetchPhase(List fetchSubPhases) { this.fetchSubPhases[fetchSubPhases.size()] = new InnerHitsPhase(this); } - public void execute(SearchContext context, int[] docIdsToLoad, RankDocShardInfo rankDocs) { + public void execute(SearchContext context, int[] docIdsToLoad, RankDocShardInfo rankDocs, CircuitBreaker circuitBreaker) { if (LOGGER.isTraceEnabled()) { LOGGER.trace("{}", new SearchContextSourcePrinter(context)); } @@ -81,7 +85,7 @@ public void execute(SearchContext context, int[] docIdsToLoad, RankDocShardInfo : Profilers.startProfilingFetchPhase(); SearchHits hits = null; try { - hits = buildSearchHits(context, docIdsToLoad, profiler, rankDocs); + hits = buildSearchHits(context, docIdsToLoad, profiler, rankDocs, circuitBreaker); } finally { // Always finish profiling ProfileResult profileResult = profiler.finish(); @@ -103,7 +107,13 @@ public Source getSource(LeafReaderContext ctx, int doc) { } } - private SearchHits buildSearchHits(SearchContext context, int[] docIdsToLoad, Profiler profiler, RankDocShardInfo rankDocs) { + private SearchHits buildSearchHits( + SearchContext context, + int[] docIdsToLoad, + Profiler profiler, + RankDocShardInfo rankDocs, + CircuitBreaker circuitBreaker + ) { FetchContext fetchContext = new FetchContext(context); SourceLoader sourceLoader = context.newSourceLoader(); @@ -172,7 +182,8 @@ protected SearchHit nextDoc(int doc) throws IOException { ctx, leafSourceLoader, leafIdLoader, - rankDocs == null ? null : rankDocs.get(doc) + rankDocs == null ? null : rankDocs.get(doc), + circuitBreaker ); boolean success = false; try { @@ -226,7 +237,7 @@ List getProcessors(SearchShardTarget target, FetchContex } } - private static HitContext prepareHitContext( + private HitContext prepareHitContext( SearchContext context, boolean requiresSource, Profiler profiler, @@ -236,7 +247,8 @@ private static HitContext prepareHitContext( LeafReaderContext subReaderContext, SourceLoader.Leaf sourceLoader, IdLoader.Leaf idLoader, - RankDoc rankDoc + RankDoc rankDoc, + CircuitBreaker circuitBreaker ) throws IOException { if (nestedDocuments.advance(docId - subReaderContext.docBase) == null) { return prepareNonNestedHitContext( @@ -247,7 +259,8 @@ private static HitContext prepareHitContext( subReaderContext, sourceLoader, idLoader, - rankDoc + rankDoc, + circuitBreaker ); } else { return prepareNestedHitContext( @@ -278,7 +291,8 @@ private static HitContext prepareNonNestedHitContext( LeafReaderContext subReaderContext, SourceLoader.Leaf sourceLoader, IdLoader.Leaf idLoader, - RankDoc rankDoc + RankDoc rankDoc, + CircuitBreaker circuitBreaker ) throws IOException { int subDocId = docId - subReaderContext.docBase; @@ -286,36 +300,55 @@ private static HitContext prepareNonNestedHitContext( String id = idLoader.getId(subDocId); if (id == null) { - SearchHit hit = new SearchHit(docId); + SearchHit hit = new SearchHit(docId, circuitBreaker); // TODO: can we use real pooled buffers here as well? - Source source = Source.lazy(lazyStoredSourceLoader(profiler, subReaderContext, subDocId)); + Source source = Source.lazy(lazyStoredSourceLoader(profiler, subReaderContext, subDocId, hit, circuitBreaker)); return new HitContext(hit, subReaderContext, subDocId, Map.of(), source, rankDoc); } else { - SearchHit hit = new SearchHit(docId, id); + SearchHit hit = new SearchHit(docId, id, circuitBreaker); Source source; if (requiresSource) { Timer timer = profiler.startLoadingSource(); try { source = sourceLoader.source(leafStoredFieldLoader, subDocId); + BytesReference sourceRef = source.internalSourceRef(); + circuitBreaker.addEstimateBytesAndMaybeBreak(sourceRef.length(), "fetch phase source loader"); + hit.unfilteredSourceRef(sourceRef); } finally { if (timer != null) { timer.stop(); } } } else { - source = Source.lazy(lazyStoredSourceLoader(profiler, subReaderContext, subDocId)); + source = Source.lazy(lazyStoredSourceLoader(profiler, subReaderContext, subDocId, hit, circuitBreaker)); } return new HitContext(hit, subReaderContext, subDocId, leafStoredFieldLoader.storedFields(), source, rankDoc); } } - private static Supplier lazyStoredSourceLoader(Profiler profiler, LeafReaderContext ctx, int doc) { + private static Supplier lazyStoredSourceLoader( + Profiler profiler, + LeafReaderContext ctx, + int doc, + SearchHit hit, + CircuitBreaker circuitBreaker + ) { return () -> { StoredFieldLoader rootLoader = profiler.storedFields(StoredFieldLoader.create(true, Collections.emptySet())); try { LeafStoredFieldLoader leafRootLoader = rootLoader.getLoader(ctx, null); leafRootLoader.advanceTo(doc); - return Source.fromBytes(leafRootLoader.source()); + BytesReference source = leafRootLoader.source(); + circuitBreaker.addEstimateBytesAndMaybeBreak(source.length(), "fetch phase source loader"); + // Saving the entire source we loaded in the hit, so that we can release it entirely when the hit is released + // This is important for the circuit breaker accounting - note that this lazy loader can be triggered in the case of + // inner hits even though the top hit source is not requested (see {@link FetchPhase#prepareNestedHitContext} when + // the `nestedSource` is created), so we need to save the entire source on the hit - we account + // for the top level source via the {@link SearchHit#unfilteredSourceRef(BytesReference)} method because the + // {@link SearchHit#source()} method can be null when the top level source is not requested. + // NB all of the above also applies for source filtering and field collapsing + hit.unfilteredSourceRef(source); + return Source.fromBytes(source); } catch (IOException e) { throw new UncheckedIOException(e); } @@ -357,8 +390,10 @@ private static HitContext prepareNestedHitContext( rootId = leafRootLoader.id(); if (requiresSource) { - if (leafRootLoader.source() != null) { - rootSource = Source.fromBytes(leafRootLoader.source()); + BytesReference source = leafRootLoader.source(); + if (source != null) { + NOOP_CIRCUIT_BREAKER.addEstimateBytesAndMaybeBreak(source.length(), "fetch phase nested hit source loader"); + rootSource = Source.fromBytes(source); } } } @@ -369,8 +404,10 @@ private static HitContext prepareNestedHitContext( assert nestedIdentity != null; Source nestedSource = nestedIdentity.extractSource(rootSource); - SearchHit hit = new SearchHit(topDocId, rootId, nestedIdentity); - return new HitContext(hit, subReaderContext, nestedInfo.doc(), childFieldLoader.storedFields(), nestedSource, rankDoc); + // nested hits do not record their source size, as the top level hit will do so (nested hits will only reference part of the top + // level source) + SearchHit nestedHit = new SearchHit(topDocId, rootId, nestedIdentity, FetchPhase.NOOP_CIRCUIT_BREAKER); + return new HitContext(nestedHit, subReaderContext, nestedInfo.doc(), childFieldLoader.storedFields(), nestedSource, rankDoc); } interface Profiler { diff --git a/server/src/main/java/org/elasticsearch/search/fetch/subphase/InnerHitsPhase.java b/server/src/main/java/org/elasticsearch/search/fetch/subphase/InnerHitsPhase.java index 374c96fdefe86..ec9911a5b5a23 100644 --- a/server/src/main/java/org/elasticsearch/search/fetch/subphase/InnerHitsPhase.java +++ b/server/src/main/java/org/elasticsearch/search/fetch/subphase/InnerHitsPhase.java @@ -12,6 +12,8 @@ import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.search.FieldDoc; import org.apache.lucene.search.ScoreDoc; +import org.elasticsearch.common.breaker.CircuitBreaker; +import org.elasticsearch.common.breaker.NoopCircuitBreaker; import org.elasticsearch.common.lucene.search.TopDocsAndMaxScore; import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.SearchHits; @@ -32,6 +34,10 @@ public final class InnerHitsPhase implements FetchSubPhase { + // inner hits will potentially load deeper inner hits, so we need to make sure we don't count their source towards the breaker + // because the sub-top-hit-source they reference is already accounted for in the outer hits (i.e. the top level hit will account for + // the entire source which all inner hits will sub reference) + private static final NoopCircuitBreaker NOOP_CIRCUIT_BREAKER = new NoopCircuitBreaker("inner_hits_phase"); private final FetchPhase fetchPhase; public InnerHitsPhase(FetchPhase fetchPhase) { @@ -93,7 +99,7 @@ private void hitExecute(Map innerHi innerHitsContext.setRootId(hit.getId()); innerHitsContext.setRootLookup(rootSource); - fetchPhase.execute(innerHitsContext, docIdsToLoad, null); + fetchPhase.execute(innerHitsContext, docIdsToLoad, null, NOOP_CIRCUIT_BREAKER); FetchSearchResult fetchResult = innerHitsContext.fetchResult(); SearchHit[] internalHits = fetchResult.fetchResult().hits().getHits(); for (int j = 0; j < internalHits.length; j++) { diff --git a/server/src/test/java/org/elasticsearch/action/search/ExpandSearchPhaseTests.java b/server/src/test/java/org/elasticsearch/action/search/ExpandSearchPhaseTests.java index 5f84739599514..374dbcf6cb519 100644 --- a/server/src/test/java/org/elasticsearch/action/search/ExpandSearchPhaseTests.java +++ b/server/src/test/java/org/elasticsearch/action/search/ExpandSearchPhaseTests.java @@ -12,6 +12,8 @@ import org.apache.lucene.search.TotalHits; import org.elasticsearch.action.ActionListener; import org.elasticsearch.common.Strings; +import org.elasticsearch.common.breaker.CircuitBreaker; +import org.elasticsearch.common.breaker.NoopCircuitBreaker; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.document.DocumentField; import org.elasticsearch.common.util.concurrent.AtomicArray; @@ -113,7 +115,7 @@ void sendExecuteMultiSearch(MultiSearchRequest request, SearchTask task, ActionL } }; - SearchHit hit = new SearchHit(1, "ID"); + SearchHit hit = new SearchHit(1, "ID", new NoopCircuitBreaker(CircuitBreaker.REQUEST)); hit.setDocumentField("someField", new DocumentField("someField", Collections.singletonList(collapseValue))); SearchHits hits = new SearchHits(new SearchHit[] { hit }, new TotalHits(1, TotalHits.Relation.EQUAL_TO), 1.0F); try { @@ -156,8 +158,9 @@ protected void run() { public void testFailOneItemFailsEntirePhase() throws IOException { AtomicBoolean executedMultiSearch = new AtomicBoolean(false); + NoopCircuitBreaker breaker = new NoopCircuitBreaker(CircuitBreaker.REQUEST); SearchHits collapsedHits = new SearchHits( - new SearchHit[] { new SearchHit(2, "ID"), new SearchHit(3, "ID") }, + new SearchHit[] { new SearchHit(2, "ID", breaker), new SearchHit(3, "ID", breaker) }, new TotalHits(1, TotalHits.Relation.EQUAL_TO), 1.0F ); @@ -201,9 +204,9 @@ void sendExecuteMultiSearch(MultiSearchRequest request, SearchTask task, ActionL } }; - SearchHit hit1 = new SearchHit(1, "ID"); + SearchHit hit1 = new SearchHit(1, "ID", breaker); hit1.setDocumentField("someField", new DocumentField("someField", Collections.singletonList(collapseValue))); - SearchHit hit2 = new SearchHit(2, "ID2"); + SearchHit hit2 = new SearchHit(2, "ID2", breaker); hit2.setDocumentField("someField", new DocumentField("someField", Collections.singletonList(collapseValue))); SearchHits hits = new SearchHits(new SearchHit[] { hit1, hit2 }, new TotalHits(1, TotalHits.Relation.EQUAL_TO), 1.0F); try { @@ -237,9 +240,10 @@ void sendExecuteMultiSearch(MultiSearchRequest request, SearchTask task, ActionL } }; - SearchHit hit1 = new SearchHit(1, "ID"); + CircuitBreaker breaker = new NoopCircuitBreaker(CircuitBreaker.REQUEST); + SearchHit hit1 = new SearchHit(1, "ID", breaker); hit1.setDocumentField("someField", new DocumentField("someField", Collections.singletonList(null))); - SearchHit hit2 = new SearchHit(2, "ID2"); + SearchHit hit2 = new SearchHit(2, "ID2", breaker); hit2.setDocumentField("someField", new DocumentField("someField", Collections.singletonList(null))); SearchHits hits = new SearchHits(new SearchHit[] { hit1, hit2 }, new TotalHits(1, TotalHits.Relation.EQUAL_TO), 1.0F); try { @@ -329,7 +333,7 @@ void sendExecuteMultiSearch(MultiSearchRequest request, SearchTask task, ActionL .preference("foobar") .routing("baz"); - SearchHit hit = new SearchHit(1, "ID"); + SearchHit hit = new SearchHit(1, "ID", new NoopCircuitBreaker(CircuitBreaker.REQUEST)); hit.setDocumentField("someField", new DocumentField("someField", Collections.singletonList("foo"))); SearchHits hits = new SearchHits(new SearchHit[] { hit }, new TotalHits(1, TotalHits.Relation.EQUAL_TO), 1.0F); try { @@ -378,7 +382,7 @@ void sendExecuteMultiSearch(MultiSearchRequest request, SearchTask task, ActionL ) .routing("baz"); - SearchHit hit = new SearchHit(1, "ID"); + SearchHit hit = new SearchHit(1, "ID", new NoopCircuitBreaker(CircuitBreaker.REQUEST)); hit.setDocumentField("someField", new DocumentField("someField", Collections.singletonList("foo"))); SearchHits hits = new SearchHits(new SearchHit[] { hit }, new TotalHits(1, TotalHits.Relation.EQUAL_TO), 1.0F); try { diff --git a/server/src/test/java/org/elasticsearch/action/search/FetchLookupFieldsPhaseTests.java b/server/src/test/java/org/elasticsearch/action/search/FetchLookupFieldsPhaseTests.java index 5c508dce61fc3..4cd16d7a19537 100644 --- a/server/src/test/java/org/elasticsearch/action/search/FetchLookupFieldsPhaseTests.java +++ b/server/src/test/java/org/elasticsearch/action/search/FetchLookupFieldsPhaseTests.java @@ -11,6 +11,8 @@ import org.apache.lucene.search.TotalHits; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.common.breaker.CircuitBreaker; +import org.elasticsearch.common.breaker.NoopCircuitBreaker; import org.elasticsearch.common.document.DocumentField; import org.elasticsearch.index.query.TermQueryBuilder; import org.elasticsearch.search.SearchHit; @@ -100,7 +102,7 @@ void sendExecuteMultiSearch( }; final SearchHits searchHits; if (fields != null) { - final SearchHit hit = new SearchHit(randomInt(1000)); + final SearchHit hit = new SearchHit(randomInt(1000), new NoopCircuitBreaker(CircuitBreaker.REQUEST)); fields.forEach((f, values) -> hit.setDocumentField(f, new DocumentField(f, values, List.of()))); searchHits = new SearchHits(new SearchHit[] { hit }, new TotalHits(1, TotalHits.Relation.EQUAL_TO), 1.0f); } else { @@ -132,7 +134,7 @@ void sendExecuteMultiSearch( } }; - SearchHit leftHit0 = new SearchHit(randomInt(100)); + SearchHit leftHit0 = new SearchHit(randomInt(100), new NoopCircuitBreaker(CircuitBreaker.REQUEST)); final List fetchFields = List.of(new FieldAndFormat(randomAlphaOfLength(10), null)); { leftHit0.setDocumentField( @@ -158,7 +160,7 @@ void sendExecuteMultiSearch( ); } - SearchHit leftHit1 = new SearchHit(randomInt(100)); + SearchHit leftHit1 = new SearchHit(randomInt(100), new NoopCircuitBreaker(CircuitBreaker.REQUEST)); { leftHit1.setDocumentField( "lookup_field_2", diff --git a/server/src/test/java/org/elasticsearch/action/search/FetchSearchPhaseTests.java b/server/src/test/java/org/elasticsearch/action/search/FetchSearchPhaseTests.java index fd60621c7e400..3283cc9c063ef 100644 --- a/server/src/test/java/org/elasticsearch/action/search/FetchSearchPhaseTests.java +++ b/server/src/test/java/org/elasticsearch/action/search/FetchSearchPhaseTests.java @@ -791,7 +791,7 @@ public void testFetchTimeoutWithPartialResults() throws IOException { ContextIndexSearcher contextIndexSearcher = createSearcher(r); try (SearchContext searchContext = createSearchContext(contextIndexSearcher, true)) { FetchPhase fetchPhase = createFetchPhase(contextIndexSearcher); - fetchPhase.execute(searchContext, new int[] { 0, 1, 2 }, null); + fetchPhase.execute(searchContext, new int[] { 0, 1, 2 }, null, new NoopCircuitBreaker(CircuitBreaker.REQUEST)); assertTrue(searchContext.queryResult().searchTimedOut()); assertEquals(1, searchContext.fetchResult().hits().getHits().length); } finally { @@ -812,7 +812,10 @@ public void testFetchTimeoutNoPartialResults() throws IOException { try (SearchContext searchContext = createSearchContext(contextIndexSearcher, false)) { FetchPhase fetchPhase = createFetchPhase(contextIndexSearcher); - expectThrows(SearchTimeoutException.class, () -> fetchPhase.execute(searchContext, new int[] { 0, 1, 2 }, null)); + expectThrows( + SearchTimeoutException.class, + () -> fetchPhase.execute(searchContext, new int[] { 0, 1, 2 }, null, new NoopCircuitBreaker(CircuitBreaker.REQUEST)) + ); assertNull(searchContext.fetchResult().hits()); } finally { r.close(); diff --git a/server/src/test/java/org/elasticsearch/action/search/SearchResponseMergerTests.java b/server/src/test/java/org/elasticsearch/action/search/SearchResponseMergerTests.java index d54ac9c66d9a5..197ece084e7f5 100644 --- a/server/src/test/java/org/elasticsearch/action/search/SearchResponseMergerTests.java +++ b/server/src/test/java/org/elasticsearch/action/search/SearchResponseMergerTests.java @@ -13,6 +13,8 @@ import org.apache.lucene.search.TotalHits; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.action.search.TransportSearchAction.SearchTimeProvider; +import org.elasticsearch.common.breaker.CircuitBreaker; +import org.elasticsearch.common.breaker.NoopCircuitBreaker; import org.elasticsearch.common.text.Text; import org.elasticsearch.common.time.DateFormatter; import org.elasticsearch.core.Tuple; @@ -1094,7 +1096,7 @@ private static SearchHit[] randomSearchHitArray( for (int j = 0; j < numDocs; j++) { ShardId shardId = new ShardId(randomFrom(indices), randomIntBetween(0, 10)); SearchShardTarget shardTarget = new SearchShardTarget(randomAlphaOfLengthBetween(3, 8), shardId, clusterAlias); - SearchHit hit = new SearchHit(randomIntBetween(0, Integer.MAX_VALUE)); + SearchHit hit = new SearchHit(randomIntBetween(0, Integer.MAX_VALUE), new NoopCircuitBreaker(CircuitBreaker.REQUEST)); float score = Float.NaN; if (Float.isNaN(maxScore) == false) { diff --git a/server/src/test/java/org/elasticsearch/action/search/SearchResponseTests.java b/server/src/test/java/org/elasticsearch/action/search/SearchResponseTests.java index bbeae6b19b8ac..7d3912e923b59 100644 --- a/server/src/test/java/org/elasticsearch/action/search/SearchResponseTests.java +++ b/server/src/test/java/org/elasticsearch/action/search/SearchResponseTests.java @@ -15,6 +15,8 @@ import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.common.Strings; +import org.elasticsearch.common.breaker.CircuitBreaker; +import org.elasticsearch.common.breaker.NoopCircuitBreaker; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.settings.Settings; @@ -365,7 +367,7 @@ public void testFromXContentWithFailures() throws IOException { } public void testToXContent() throws IOException { - SearchHit hit = new SearchHit(1, "id1"); + SearchHit hit = new SearchHit(1, "id1", new NoopCircuitBreaker(CircuitBreaker.REQUEST)); hit.score(2.0f); SearchHit[] hits = new SearchHit[] { hit }; var sHits = new SearchHits(hits, new TotalHits(100, TotalHits.Relation.EQUAL_TO), 1.5f); diff --git a/server/src/test/java/org/elasticsearch/search/SearchHitTests.java b/server/src/test/java/org/elasticsearch/search/SearchHitTests.java index 25a71d04b321d..cd8002f62c6b5 100644 --- a/server/src/test/java/org/elasticsearch/search/SearchHitTests.java +++ b/server/src/test/java/org/elasticsearch/search/SearchHitTests.java @@ -13,6 +13,8 @@ import org.apache.lucene.search.TotalHits; import org.elasticsearch.TransportVersion; import org.elasticsearch.common.Strings; +import org.elasticsearch.common.breaker.CircuitBreaker; +import org.elasticsearch.common.breaker.NoopCircuitBreaker; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.document.DocumentField; @@ -74,7 +76,7 @@ public static SearchHit createTestItem(XContentType xContentType, boolean withOp documentFields = GetResultTests.randomDocumentFields(xContentType, false).v2(); } } - SearchHit hit = new SearchHit(internalId, uid, nestedIdentity); + SearchHit hit = new SearchHit(internalId, uid, nestedIdentity, new NoopCircuitBreaker(CircuitBreaker.REQUEST)); hit.addDocumentFields(documentFields, metaFields); if (frequently()) { if (rarely()) { @@ -230,7 +232,7 @@ public void testFromXContentWithoutTypeAndId() throws IOException { } public void testToXContent() throws IOException { - SearchHit searchHit = new SearchHit(1, "id1"); + SearchHit searchHit = new SearchHit(1, "id1", new NoopCircuitBreaker(CircuitBreaker.REQUEST)); try { searchHit.score(1.5f); XContentBuilder builder = JsonXContent.contentBuilder(); @@ -254,27 +256,28 @@ public void testRankToXContent() throws IOException { public void testSerializeShardTarget() throws Exception { String clusterAlias = randomBoolean() ? null : "cluster_alias"; SearchShardTarget target = new SearchShardTarget("_node_id", new ShardId(new Index("_index", "_na_"), 0), clusterAlias); + NoopCircuitBreaker noopCircuitBreaker = new NoopCircuitBreaker(CircuitBreaker.REQUEST); Map innerHits = new HashMap<>(); - SearchHit innerHit1 = new SearchHit(0, "_id"); + SearchHit innerHit1 = new SearchHit(0, "_id", noopCircuitBreaker); innerHit1.shard(target); - SearchHit innerInnerHit2 = new SearchHit(0, "_id"); + SearchHit innerInnerHit2 = new SearchHit(0, "_id", noopCircuitBreaker); innerInnerHit2.shard(target); innerHits.put("1", new SearchHits(new SearchHit[] { innerInnerHit2 }, new TotalHits(1, TotalHits.Relation.EQUAL_TO), 1f)); innerHit1.setInnerHits(innerHits); - SearchHit innerHit2 = new SearchHit(0, "_id"); + SearchHit innerHit2 = new SearchHit(0, "_id", noopCircuitBreaker); innerHit2.shard(target); - SearchHit innerHit3 = new SearchHit(0, "_id"); + SearchHit innerHit3 = new SearchHit(0, "_id", noopCircuitBreaker); innerHit3.shard(target); innerHits = new HashMap<>(); - SearchHit hit1 = new SearchHit(0, "_id"); + SearchHit hit1 = new SearchHit(0, "_id", noopCircuitBreaker); innerHits.put("1", new SearchHits(new SearchHit[] { innerHit1, innerHit2 }, new TotalHits(1, TotalHits.Relation.EQUAL_TO), 1f)); innerHits.put("2", new SearchHits(new SearchHit[] { innerHit3 }, new TotalHits(1, TotalHits.Relation.EQUAL_TO), 1f)); hit1.shard(target); hit1.setInnerHits(innerHits); - SearchHit hit2 = new SearchHit(0, "_id"); + SearchHit hit2 = new SearchHit(0, "_id", noopCircuitBreaker); hit2.shard(target); SearchHits hits = new SearchHits(new SearchHit[] { hit1, hit2 }, new TotalHits(2, TotalHits.Relation.EQUAL_TO), 1f); diff --git a/server/src/test/java/org/elasticsearch/search/fetch/FetchPhaseDocsIteratorTests.java b/server/src/test/java/org/elasticsearch/search/fetch/FetchPhaseDocsIteratorTests.java index c8d1b6721c64b..cae4ffe2ee2c1 100644 --- a/server/src/test/java/org/elasticsearch/search/fetch/FetchPhaseDocsIteratorTests.java +++ b/server/src/test/java/org/elasticsearch/search/fetch/FetchPhaseDocsIteratorTests.java @@ -16,6 +16,8 @@ import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.store.Directory; import org.apache.lucene.tests.index.RandomIndexWriter; +import org.elasticsearch.common.breaker.CircuitBreaker; +import org.elasticsearch.common.breaker.NoopCircuitBreaker; import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.query.QuerySearchResult; import org.elasticsearch.test.ESTestCase; @@ -74,7 +76,7 @@ protected void setNextReader(LeafReaderContext ctx, int[] docsInLeaf) { protected SearchHit nextDoc(int doc) { assertThat(doc, equalTo(this.docsInLeaf[this.index] + this.ctx.docBase)); index++; - return new SearchHit(doc); + return new SearchHit(doc, new NoopCircuitBreaker(CircuitBreaker.REQUEST)); } }; @@ -122,7 +124,7 @@ protected SearchHit nextDoc(int doc) { if (doc == badDoc) { throw new IllegalArgumentException("Error processing doc"); } - return new SearchHit(doc); + return new SearchHit(doc, new NoopCircuitBreaker(CircuitBreaker.REQUEST)); } }; diff --git a/test/framework/src/main/java/org/elasticsearch/search/SearchResponseUtils.java b/test/framework/src/main/java/org/elasticsearch/search/SearchResponseUtils.java index 21b43636222f9..c197bceb7ce8c 100644 --- a/test/framework/src/main/java/org/elasticsearch/search/SearchResponseUtils.java +++ b/test/framework/src/main/java/org/elasticsearch/search/SearchResponseUtils.java @@ -892,7 +892,8 @@ public static SearchHit searchHitFromMap(Map values) { get(SearchHit.Fields.INNER_HITS, values, null), get(SearchHit.DOCUMENT_FIELDS, values, Collections.emptyMap()), get(SearchHit.METADATA_FIELDS, values, Collections.emptyMap()), - RefCounted.ALWAYS_REFERENCED // TODO: do we ever want pooling here? + RefCounted.ALWAYS_REFERENCED, // TODO: do we ever want pooling here? + null ); } From 9b524a3f3e5c8eb3a4a2c7243c9e07968e1bea12 Mon Sep 17 00:00:00 2001 From: Andrei Dan Date: Thu, 6 Feb 2025 15:55:07 +0000 Subject: [PATCH 02/56] Add source memory accounting for enrich source --- .../enrich/action/EnrichShardMultiSearchAction.java | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/x-pack/plugin/enrich/src/main/java/org/elasticsearch/xpack/enrich/action/EnrichShardMultiSearchAction.java b/x-pack/plugin/enrich/src/main/java/org/elasticsearch/xpack/enrich/action/EnrichShardMultiSearchAction.java index a7167e176cbef..c0249063625b1 100644 --- a/x-pack/plugin/enrich/src/main/java/org/elasticsearch/xpack/enrich/action/EnrichShardMultiSearchAction.java +++ b/x-pack/plugin/enrich/src/main/java/org/elasticsearch/xpack/enrich/action/EnrichShardMultiSearchAction.java @@ -30,6 +30,7 @@ import org.elasticsearch.cluster.routing.ShardsIterator; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.Strings; +import org.elasticsearch.common.breaker.CircuitBreaker; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.NamedWriteableAwareStreamInput; @@ -221,6 +222,7 @@ protected ShardsIterator shards(ClusterState state, InternalRequest request) { protected MultiSearchResponse shardOperation(Request request, ShardId shardId) throws IOException { final IndexService indexService = indicesService.indexService(shardId.getIndex()); final IndexShard indexShard = indicesService.getShardOrNull(shardId); + final CircuitBreaker breaker = indicesService.getCircuitBreakerService().getBreaker(CircuitBreaker.REQUEST); try (Engine.Searcher searcher = indexShard.acquireSearcher("enrich_msearch")) { final FieldsVisitor visitor = new FieldsVisitor(true); /* @@ -259,8 +261,11 @@ protected MultiSearchResponse shardOperation(Request request, ShardId shardId) t } return context.getFieldType(field); }); - final SearchHit hit = new SearchHit(scoreDoc.doc, visitor.id()); - hit.sourceRef(filterSource(fetchSourceContext, visitor.source())); + final SearchHit hit = new SearchHit(scoreDoc.doc, visitor.id(), breaker); + BytesReference source = visitor.source(); + hit.unfilteredSourceRef(source); + breaker.addEstimateBytesAndMaybeBreak(source.length(), "enrich_msearch"); + hit.sourceRef(filterSource(fetchSourceContext, source)); hits[j] = hit; } items[i] = new MultiSearchResponse.Item(createSearchResponse(topDocs, hits), null); From 64a696af2412332a4fea4e3def6715c1e1d2602a Mon Sep 17 00:00:00 2001 From: Andrei Dan Date: Thu, 6 Feb 2025 16:00:24 +0000 Subject: [PATCH 03/56] spotless --- server/src/main/java/org/elasticsearch/search/SearchHit.java | 1 - .../org/elasticsearch/search/fetch/subphase/InnerHitsPhase.java | 1 - 2 files changed, 2 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/search/SearchHit.java b/server/src/main/java/org/elasticsearch/search/SearchHit.java index 00697022ee057..34f3505c4c786 100644 --- a/server/src/main/java/org/elasticsearch/search/SearchHit.java +++ b/server/src/main/java/org/elasticsearch/search/SearchHit.java @@ -14,7 +14,6 @@ import org.elasticsearch.TransportVersions; import org.elasticsearch.common.Strings; import org.elasticsearch.common.breaker.CircuitBreaker; -import org.elasticsearch.common.breaker.NoopCircuitBreaker; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.compress.CompressorFactory; diff --git a/server/src/main/java/org/elasticsearch/search/fetch/subphase/InnerHitsPhase.java b/server/src/main/java/org/elasticsearch/search/fetch/subphase/InnerHitsPhase.java index ec9911a5b5a23..556f5f439d296 100644 --- a/server/src/main/java/org/elasticsearch/search/fetch/subphase/InnerHitsPhase.java +++ b/server/src/main/java/org/elasticsearch/search/fetch/subphase/InnerHitsPhase.java @@ -12,7 +12,6 @@ import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.search.FieldDoc; import org.apache.lucene.search.ScoreDoc; -import org.elasticsearch.common.breaker.CircuitBreaker; import org.elasticsearch.common.breaker.NoopCircuitBreaker; import org.elasticsearch.common.lucene.search.TopDocsAndMaxScore; import org.elasticsearch.search.SearchHit; From 19592efbf0b26e58f8d28d4962093d93e618a863 Mon Sep 17 00:00:00 2001 From: Andrei Dan Date: Thu, 6 Feb 2025 16:21:19 +0000 Subject: [PATCH 04/56] test compile --- .../saml/TransportSamlInvalidateSessionActionTests.java | 4 +++- .../security/authz/store/NativePrivilegeStoreTests.java | 8 +++++++- 2 files changed, 10 insertions(+), 2 deletions(-) diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/action/saml/TransportSamlInvalidateSessionActionTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/action/saml/TransportSamlInvalidateSessionActionTests.java index 269f8cb0471fc..31d1c85735677 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/action/saml/TransportSamlInvalidateSessionActionTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/action/saml/TransportSamlInvalidateSessionActionTests.java @@ -38,6 +38,8 @@ import org.elasticsearch.action.update.UpdateResponse; import org.elasticsearch.client.internal.Client; import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.breaker.CircuitBreaker; +import org.elasticsearch.common.breaker.NoopCircuitBreaker; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.ThreadContext; @@ -333,7 +335,7 @@ private SearchHit tokenHit(int idx, BytesReference source) { final Map accessToken = (Map) sourceMap.get("access_token"); @SuppressWarnings("unchecked") final Map userToken = (Map) accessToken.get("user_token"); - final SearchHit hit = new SearchHit(idx, "token_" + userToken.get("id")); + final SearchHit hit = new SearchHit(idx, "token_" + userToken.get("id"), new NoopCircuitBreaker(CircuitBreaker.REQUEST)); hit.sourceRef(source); return hit; } catch (IOException e) { diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/store/NativePrivilegeStoreTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/store/NativePrivilegeStoreTests.java index ed3949450cb9f..60c7e2e6bc72c 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/store/NativePrivilegeStoreTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/store/NativePrivilegeStoreTests.java @@ -33,6 +33,8 @@ import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.Strings; import org.elasticsearch.common.UUIDs; +import org.elasticsearch.common.breaker.CircuitBreaker; +import org.elasticsearch.common.breaker.NoopCircuitBreaker; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.settings.ClusterSettings; @@ -921,7 +923,11 @@ private SearchHit[] buildHits(List sourcePrivile final SearchHit[] hits = new SearchHit[sourcePrivileges.size()]; for (int i = 0; i < hits.length; i++) { final ApplicationPrivilegeDescriptor p = sourcePrivileges.get(i); - hits[i] = new SearchHit(i, "application-privilege_" + p.getApplication() + ":" + p.getName()); + hits[i] = new SearchHit( + i, + "application-privilege_" + p.getApplication() + ":" + p.getName(), + new NoopCircuitBreaker(CircuitBreaker.REQUEST) + ); hits[i].sourceRef(new BytesArray(Strings.toString(p))); } return hits; From 40fc1780da388b6215688336ddbd04074cc8bcc7 Mon Sep 17 00:00:00 2001 From: Andrei Dan Date: Thu, 6 Feb 2025 16:26:22 +0000 Subject: [PATCH 05/56] Test compilation --- .../xpack/ml/job/persistence/JobResultsProviderTests.java | 4 +++- .../xpack/ml/job/persistence/MockClientBuilder.java | 4 +++- .../xpack/ml/job/persistence/StateStreamerTests.java | 6 +++++- .../job/retention/AbstractExpiredJobDataRemoverTests.java | 4 +++- 4 files changed, 14 insertions(+), 4 deletions(-) diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsProviderTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsProviderTests.java index 3dcbbeb3fcce5..8aaa0452a643f 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsProviderTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsProviderTests.java @@ -20,6 +20,8 @@ import org.elasticsearch.client.internal.Client; import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.cluster.metadata.MappingMetadata; +import org.elasticsearch.common.breaker.CircuitBreaker; +import org.elasticsearch.common.breaker.NoopCircuitBreaker; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.document.DocumentField; import org.elasticsearch.common.settings.Settings; @@ -921,7 +923,7 @@ private static SearchResponse createSearchResponse(List> sou fields.put("field_1", new DocumentField("field_1", Collections.singletonList("foo"))); fields.put("field_2", new DocumentField("field_2", Collections.singletonList("foo"))); - SearchHit hit = new SearchHit(123, String.valueOf(map.hashCode())); + SearchHit hit = new SearchHit(123, String.valueOf(map.hashCode()), new NoopCircuitBreaker(CircuitBreaker.REQUEST)); hit.addDocumentFields(fields, Collections.emptyMap()); hit.sourceRef(BytesReference.bytes(XContentFactory.jsonBuilder().map(_source))); diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/MockClientBuilder.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/MockClientBuilder.java index ebc92eee61ea2..5ee00058dedb8 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/MockClientBuilder.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/MockClientBuilder.java @@ -25,6 +25,8 @@ import org.elasticsearch.client.internal.Client; import org.elasticsearch.client.internal.ClusterAdminClient; import org.elasticsearch.client.internal.IndicesAdminClient; +import org.elasticsearch.common.breaker.CircuitBreaker; +import org.elasticsearch.common.breaker.NoopCircuitBreaker; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.document.DocumentField; import org.elasticsearch.common.settings.Settings; @@ -130,7 +132,7 @@ public MockClientBuilder prepareSearch(String indexName, List do SearchHit hits[] = new SearchHit[docs.size()]; for (int i = 0; i < docs.size(); i++) { - SearchHit hit = new SearchHit(10); + SearchHit hit = new SearchHit(10, new NoopCircuitBreaker(CircuitBreaker.REQUEST)); hit.sourceRef(docs.get(i)); hits[i] = hit; } diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/StateStreamerTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/StateStreamerTests.java index 47f7d8c65a27a..9b87edd5f564e 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/StateStreamerTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/StateStreamerTests.java @@ -9,6 +9,8 @@ import org.elasticsearch.action.search.SearchRequestBuilder; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.client.internal.Client; +import org.elasticsearch.common.breaker.CircuitBreaker; +import org.elasticsearch.common.breaker.NoopCircuitBreaker; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.index.query.QueryBuilder; import org.elasticsearch.index.query.QueryBuilders; @@ -103,7 +105,9 @@ private static SearchResponse createSearchResponse(List> sou SearchHit[] hits = new SearchHit[source.size()]; int i = 0; for (Map s : source) { - SearchHit hit = new SearchHit(1).sourceRef(BytesReference.bytes(XContentFactory.jsonBuilder().map(s))); + SearchHit hit = new SearchHit(1, new NoopCircuitBreaker(CircuitBreaker.REQUEST)).sourceRef( + BytesReference.bytes(XContentFactory.jsonBuilder().map(s)) + ); hits[i++] = hit; } SearchHits searchHits = new SearchHits(hits, null, (float) 0.0); diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/retention/AbstractExpiredJobDataRemoverTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/retention/AbstractExpiredJobDataRemoverTests.java index 6ec43ca2a3201..ac7bffc0dc16d 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/retention/AbstractExpiredJobDataRemoverTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/retention/AbstractExpiredJobDataRemoverTests.java @@ -11,6 +11,8 @@ import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.client.internal.Client; import org.elasticsearch.client.internal.OriginSettingClient; +import org.elasticsearch.common.breaker.CircuitBreaker; +import org.elasticsearch.common.breaker.NoopCircuitBreaker; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.core.TimeValue; import org.elasticsearch.search.SearchHit; @@ -105,7 +107,7 @@ static SearchResponse createSearchResponseFromHits(List hits) { private static SearchResponse createSearchResponse(List toXContents, int totalHits) throws IOException { SearchHit[] hitsArray = new SearchHit[toXContents.size()]; for (int i = 0; i < toXContents.size(); i++) { - hitsArray[i] = new SearchHit(randomInt()); + hitsArray[i] = new SearchHit(randomInt(), new NoopCircuitBreaker(CircuitBreaker.REQUEST)); XContentBuilder jsonBuilder = JsonXContent.contentBuilder(); toXContents.get(i).toXContent(jsonBuilder, ToXContent.EMPTY_PARAMS); hitsArray[i].sourceRef(BytesReference.bytes(jsonBuilder)); From 72d2c0e15e67205787421ad51e92374d0cca3e37 Mon Sep 17 00:00:00 2001 From: Andrei Dan Date: Thu, 6 Feb 2025 16:29:46 +0000 Subject: [PATCH 06/56] test compile --- .../inference/highlight/SemanticTextHighlighterTests.java | 4 +++- .../xpack/transform/transforms/TransformIndexerTests.java | 7 ++++++- 2 files changed, 9 insertions(+), 2 deletions(-) diff --git a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/highlight/SemanticTextHighlighterTests.java b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/highlight/SemanticTextHighlighterTests.java index b808dce4f9750..9ec98e2fca546 100644 --- a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/highlight/SemanticTextHighlighterTests.java +++ b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/highlight/SemanticTextHighlighterTests.java @@ -21,6 +21,8 @@ import org.apache.lucene.util.BytesRef; import org.elasticsearch.action.OriginalIndices; import org.elasticsearch.action.search.SearchRequest; +import org.elasticsearch.common.breaker.CircuitBreaker; +import org.elasticsearch.common.breaker.NoopCircuitBreaker; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.io.Streams; @@ -240,7 +242,7 @@ private void assertHighlightOneDoc( Mockito.when(fetchContext.getSearchExecutionContext()).thenReturn(execContext); FetchSubPhase.HitContext hitContext = new FetchSubPhase.HitContext( - new SearchHit(docID), + new SearchHit(docID, new NoopCircuitBreaker(CircuitBreaker.REQUEST)), getOnlyLeafReader(reader).getContext(), docID, Map.of(), diff --git a/x-pack/plugin/transform/src/test/java/org/elasticsearch/xpack/transform/transforms/TransformIndexerTests.java b/x-pack/plugin/transform/src/test/java/org/elasticsearch/xpack/transform/transforms/TransformIndexerTests.java index 644518538638d..01a6c7a0b228e 100644 --- a/x-pack/plugin/transform/src/test/java/org/elasticsearch/xpack/transform/transforms/TransformIndexerTests.java +++ b/x-pack/plugin/transform/src/test/java/org/elasticsearch/xpack/transform/transforms/TransformIndexerTests.java @@ -21,6 +21,7 @@ import org.elasticsearch.client.internal.Client; import org.elasticsearch.common.breaker.CircuitBreaker; import org.elasticsearch.common.breaker.CircuitBreakingException; +import org.elasticsearch.common.breaker.NoopCircuitBreaker; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.core.TimeValue; import org.elasticsearch.index.reindex.BulkByScrollResponse; @@ -82,7 +83,11 @@ public class TransformIndexerTests extends ESTestCase { private static final SearchResponse ONE_HIT_SEARCH_RESPONSE = new SearchResponse( - new SearchHits(new SearchHit[] { new SearchHit(1) }, new TotalHits(1L, TotalHits.Relation.EQUAL_TO), 1.0f), + new SearchHits( + new SearchHit[] { new SearchHit(1, new NoopCircuitBreaker(CircuitBreaker.REQUEST)) }, + new TotalHits(1L, TotalHits.Relation.EQUAL_TO), + 1.0f + ), // Simulate completely null aggs null, new Suggest(Collections.emptyList()), From 55f40ef9dce4c345a0e9725b12631ea84851879a Mon Sep 17 00:00:00 2001 From: Andrei Dan Date: Thu, 6 Feb 2025 16:54:38 +0000 Subject: [PATCH 07/56] no sysout --- server/src/main/java/org/elasticsearch/search/SearchHit.java | 1 - 1 file changed, 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/search/SearchHit.java b/server/src/main/java/org/elasticsearch/search/SearchHit.java index 34f3505c4c786..1f414368604a0 100644 --- a/server/src/main/java/org/elasticsearch/search/SearchHit.java +++ b/server/src/main/java/org/elasticsearch/search/SearchHit.java @@ -751,7 +751,6 @@ private void deallocate() { SearchHit.this.source = null; if (unfilteredSourceRef != null && circuitBreaker != null) { - System.out.println(" removing source loaded by inner hit " + unfilteredSourceRef.length()); circuitBreaker.addWithoutBreaking(-unfilteredSourceRef.length()); } this.unfilteredSourceRef = null; From 8b1f8c1e523fe3506bba8a8aef6466d5b8f6557e Mon Sep 17 00:00:00 2001 From: Andrei Dan Date: Fri, 7 Feb 2025 09:42:37 +0000 Subject: [PATCH 08/56] DecRef for newly created hit on circuit breaking exception --- .../main/java/org/elasticsearch/search/fetch/FetchPhase.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java b/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java index 46bf391faad0f..8baf1f73719e2 100644 --- a/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java +++ b/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java @@ -14,6 +14,7 @@ import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.search.TotalHits; import org.elasticsearch.common.breaker.CircuitBreaker; +import org.elasticsearch.common.breaker.CircuitBreakingException; import org.elasticsearch.common.breaker.NoopCircuitBreaker; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.index.fieldvisitor.LeafStoredFieldLoader; @@ -314,6 +315,9 @@ private static HitContext prepareNonNestedHitContext( BytesReference sourceRef = source.internalSourceRef(); circuitBreaker.addEstimateBytesAndMaybeBreak(sourceRef.length(), "fetch phase source loader"); hit.unfilteredSourceRef(sourceRef); + } catch (CircuitBreakingException e) { + hit.decRef(); + throw e; } finally { if (timer != null) { timer.stop(); From edaeebeede46fc1165a08eacbc7d7279ec34ea17 Mon Sep 17 00:00:00 2001 From: Andrei Dan Date: Fri, 7 Feb 2025 11:39:25 +0000 Subject: [PATCH 09/56] FetchPhase doc iterator lets CircuitBreakingException bubble up --- .../org/elasticsearch/search/fetch/FetchPhaseDocsIterator.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/search/fetch/FetchPhaseDocsIterator.java b/server/src/main/java/org/elasticsearch/search/fetch/FetchPhaseDocsIterator.java index 4a242f70e8d02..b12f31493b3d5 100644 --- a/server/src/main/java/org/elasticsearch/search/fetch/FetchPhaseDocsIterator.java +++ b/server/src/main/java/org/elasticsearch/search/fetch/FetchPhaseDocsIterator.java @@ -12,6 +12,7 @@ import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.ReaderUtil; +import org.elasticsearch.common.breaker.CircuitBreakingException; import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.SearchHits; import org.elasticsearch.search.SearchShardTarget; @@ -98,7 +99,7 @@ public final SearchHit[] iterate( return partialSearchHits; } } - } catch (SearchTimeoutException e) { + } catch (SearchTimeoutException | CircuitBreakingException e) { throw e; } catch (Exception e) { purgeSearchHits(searchHits); From ff6ab2452a97e7a62af73b41e4f1afb00f17d798 Mon Sep 17 00:00:00 2001 From: Andrei Dan Date: Fri, 7 Feb 2025 14:03:14 +0000 Subject: [PATCH 10/56] DecRef after hit creation --- .../action/EnrichShardMultiSearchAction.java | 16 +++++++++++----- 1 file changed, 11 insertions(+), 5 deletions(-) diff --git a/x-pack/plugin/enrich/src/main/java/org/elasticsearch/xpack/enrich/action/EnrichShardMultiSearchAction.java b/x-pack/plugin/enrich/src/main/java/org/elasticsearch/xpack/enrich/action/EnrichShardMultiSearchAction.java index c0249063625b1..2f72144dd3d63 100644 --- a/x-pack/plugin/enrich/src/main/java/org/elasticsearch/xpack/enrich/action/EnrichShardMultiSearchAction.java +++ b/x-pack/plugin/enrich/src/main/java/org/elasticsearch/xpack/enrich/action/EnrichShardMultiSearchAction.java @@ -31,6 +31,7 @@ import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.Strings; import org.elasticsearch.common.breaker.CircuitBreaker; +import org.elasticsearch.common.breaker.CircuitBreakingException; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.NamedWriteableAwareStreamInput; @@ -262,11 +263,16 @@ protected MultiSearchResponse shardOperation(Request request, ShardId shardId) t return context.getFieldType(field); }); final SearchHit hit = new SearchHit(scoreDoc.doc, visitor.id(), breaker); - BytesReference source = visitor.source(); - hit.unfilteredSourceRef(source); - breaker.addEstimateBytesAndMaybeBreak(source.length(), "enrich_msearch"); - hit.sourceRef(filterSource(fetchSourceContext, source)); - hits[j] = hit; + try { + BytesReference source = visitor.source(); + hit.unfilteredSourceRef(source); + breaker.addEstimateBytesAndMaybeBreak(source.length(), "enrich_msearch"); + hit.sourceRef(filterSource(fetchSourceContext, source)); + hits[j] = hit; + } catch (CircuitBreakingException e) { + hit.decRef(); + throw e; + } } items[i] = new MultiSearchResponse.Item(createSearchResponse(topDocs, hits), null); } From c1034519916456a07ac8a20b345ea6758394334c Mon Sep 17 00:00:00 2001 From: Andrei Dan Date: Sat, 8 Feb 2025 10:19:35 +0000 Subject: [PATCH 11/56] Purge hits on CBE --- .../elasticsearch/search/fetch/FetchPhaseDocsIterator.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/search/fetch/FetchPhaseDocsIterator.java b/server/src/main/java/org/elasticsearch/search/fetch/FetchPhaseDocsIterator.java index b12f31493b3d5..552f9e339bd7c 100644 --- a/server/src/main/java/org/elasticsearch/search/fetch/FetchPhaseDocsIterator.java +++ b/server/src/main/java/org/elasticsearch/search/fetch/FetchPhaseDocsIterator.java @@ -99,7 +99,10 @@ public final SearchHit[] iterate( return partialSearchHits; } } - } catch (SearchTimeoutException | CircuitBreakingException e) { + } catch (SearchTimeoutException e) { + throw e; + } catch (CircuitBreakingException e) { + purgeSearchHits(searchHits); throw e; } catch (Exception e) { purgeSearchHits(searchHits); From d4b22e177bf0c2c9fcf0f1d89cc49e7c90a97ecf Mon Sep 17 00:00:00 2001 From: Andrei Dan Date: Sat, 8 Feb 2025 10:19:52 +0000 Subject: [PATCH 12/56] Enrich purge hits on CBE --- .../xpack/enrich/action/EnrichShardMultiSearchAction.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/x-pack/plugin/enrich/src/main/java/org/elasticsearch/xpack/enrich/action/EnrichShardMultiSearchAction.java b/x-pack/plugin/enrich/src/main/java/org/elasticsearch/xpack/enrich/action/EnrichShardMultiSearchAction.java index 2f72144dd3d63..ec4d1a74adb59 100644 --- a/x-pack/plugin/enrich/src/main/java/org/elasticsearch/xpack/enrich/action/EnrichShardMultiSearchAction.java +++ b/x-pack/plugin/enrich/src/main/java/org/elasticsearch/xpack/enrich/action/EnrichShardMultiSearchAction.java @@ -271,6 +271,11 @@ protected MultiSearchResponse shardOperation(Request request, ShardId shardId) t hits[j] = hit; } catch (CircuitBreakingException e) { hit.decRef(); + for (SearchHit searchHit : hits) { + if (searchHit != null) { + searchHit.decRef(); + } + } throw e; } } From ffe9053ccac98a00f76755b99d39e3e44f55089d Mon Sep 17 00:00:00 2001 From: Andrei Dan Date: Tue, 11 Feb 2025 12:46:46 +0000 Subject: [PATCH 13/56] Use a mem accountign ref counted for the parent, unfiltered, source --- .../MemoryAccountingBytesRefCounted.java | 39 +++++++++++++++ .../org/elasticsearch/search/SearchHit.java | 49 +++++++------------ .../search/fetch/FetchPhase.java | 24 ++++++--- .../action/search/ExpandSearchPhaseTests.java | 17 +++---- .../search/FetchLookupFieldsPhaseTests.java | 8 ++- .../search/SearchResponseMergerTests.java | 4 +- .../action/search/SearchResponseTests.java | 4 +- .../elasticsearch/search/SearchHitTests.java | 19 +++---- .../fetch/FetchPhaseDocsIteratorTests.java | 6 +-- .../search/SearchResponseUtils.java | 3 +- .../SemanticTextHighlighterTests.java | 4 +- .../persistence/JobResultsProviderTests.java | 4 +- .../AbstractExpiredJobDataRemoverTests.java | 4 +- ...sportSamlInvalidateSessionActionTests.java | 4 +- .../store/NativePrivilegeStoreTests.java | 8 +-- .../transforms/TransformIndexerTests.java | 7 +-- 16 files changed, 103 insertions(+), 101 deletions(-) create mode 100644 server/src/main/java/org/elasticsearch/common/MemoryAccountingBytesRefCounted.java diff --git a/server/src/main/java/org/elasticsearch/common/MemoryAccountingBytesRefCounted.java b/server/src/main/java/org/elasticsearch/common/MemoryAccountingBytesRefCounted.java new file mode 100644 index 0000000000000..2307e4639f06b --- /dev/null +++ b/server/src/main/java/org/elasticsearch/common/MemoryAccountingBytesRefCounted.java @@ -0,0 +1,39 @@ +/* + * 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", the "GNU Affero General Public License v3.0 only", and the "Server Side + * Public License v 1"; you may not use this file except in compliance with, at + * your election, the "Elastic License 2.0", the "GNU Affero General Public + * License v3.0 only", or the "Server Side Public License, v 1". + */ + +package org.elasticsearch.common; + +import org.elasticsearch.common.breaker.CircuitBreaker; +import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.core.AbstractRefCounted; + +public final class MemoryAccountingBytesRefCounted extends AbstractRefCounted { + + private BytesReference bytes; + private CircuitBreaker breaker; + + private MemoryAccountingBytesRefCounted(BytesReference bytes, CircuitBreaker breaker) { + this.bytes = bytes; + this.breaker = breaker; + } + + public static MemoryAccountingBytesRefCounted createAndAccountForBytes( + BytesReference bytes, + CircuitBreaker breaker, + String memAccountingLabel + ) { + breaker.addEstimateBytesAndMaybeBreak(bytes.length(), memAccountingLabel); + return new MemoryAccountingBytesRefCounted(bytes, breaker); + } + + @Override + protected void closeInternal() { + breaker.addWithoutBreaking(-bytes.length()); + } +} diff --git a/server/src/main/java/org/elasticsearch/search/SearchHit.java b/server/src/main/java/org/elasticsearch/search/SearchHit.java index 1f414368604a0..4d29ce112ae94 100644 --- a/server/src/main/java/org/elasticsearch/search/SearchHit.java +++ b/server/src/main/java/org/elasticsearch/search/SearchHit.java @@ -13,7 +13,6 @@ import org.elasticsearch.ElasticsearchParseException; import org.elasticsearch.TransportVersions; import org.elasticsearch.common.Strings; -import org.elasticsearch.common.breaker.CircuitBreaker; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.compress.CompressorFactory; @@ -85,7 +84,7 @@ public final class SearchHit implements Writeable, ToXContentObject, RefCounted private BytesReference source; @Nullable - private BytesReference unfilteredSourceRef; + private RefCounted unfilteredSource; private final Map documentFields; private final Map metaFields; @@ -113,27 +112,20 @@ public final class SearchHit implements Writeable, ToXContentObject, RefCounted private Map innerHits; private final RefCounted refCounted; - private final CircuitBreaker circuitBreaker; - public SearchHit(int docId, CircuitBreaker circuitBreaker) { - this(docId, null, circuitBreaker); + public SearchHit(int docId) { + this(docId, null); } - public SearchHit(int docId, String id, CircuitBreaker circuitBreaker) { - this(docId, id, null, circuitBreaker); + public SearchHit(int docId, String id) { + this(docId, id, null); } - public SearchHit(int nestedTopDocId, String id, NestedIdentity nestedIdentity, CircuitBreaker circuitBreaker) { - this(nestedTopDocId, id, nestedIdentity, null, circuitBreaker); + public SearchHit(int nestedTopDocId, String id, NestedIdentity nestedIdentity) { + this(nestedTopDocId, id, nestedIdentity, null); } - private SearchHit( - int nestedTopDocId, - String id, - NestedIdentity nestedIdentity, - @Nullable RefCounted refCounted, - @Nullable CircuitBreaker circuitBreaker - ) { + private SearchHit(int nestedTopDocId, String id, NestedIdentity nestedIdentity, @Nullable RefCounted refCounted) { this( nestedTopDocId, DEFAULT_SCORE, @@ -154,8 +146,7 @@ private SearchHit( null, new HashMap<>(), new HashMap<>(), - refCounted, - circuitBreaker + refCounted ); } @@ -179,8 +170,7 @@ public SearchHit( Map innerHits, Map documentFields, Map metaFields, - @Nullable RefCounted refCounted, - @Nullable CircuitBreaker circuitBreaker + @Nullable RefCounted refCounted ) { this.docId = docId; this.score = score; @@ -202,7 +192,6 @@ public SearchHit( this.documentFields = documentFields; this.metaFields = metaFields; this.refCounted = refCounted == null ? LeakTracker.wrap(new SimpleRefCounted()) : refCounted; - this.circuitBreaker = circuitBreaker; } public static SearchHit readFrom(StreamInput in, boolean pooled) throws IOException { @@ -292,8 +281,7 @@ public static SearchHit readFrom(StreamInput in, boolean pooled) throws IOExcept innerHits, documentFields, metaFields, - isPooled ? null : ALWAYS_REFERENCED, - null + isPooled ? null : ALWAYS_REFERENCED ); } @@ -307,7 +295,7 @@ public static SearchHit unpooled(int docId, String id) { public static SearchHit unpooled(int nestedTopDocId, String id, NestedIdentity nestedIdentity) { // always referenced search hits do NOT call #deallocate - return new SearchHit(nestedTopDocId, id, nestedIdentity, ALWAYS_REFERENCED, null); + return new SearchHit(nestedTopDocId, id, nestedIdentity, ALWAYS_REFERENCED); } private static final Text SINGLE_MAPPING_TYPE = new Text(MapperService.SINGLE_MAPPING_NAME); @@ -467,8 +455,8 @@ public SearchHit sourceRef(BytesReference source) { * The regular source might be a subset of the unfiltered source due to either * source filtering, field collapsing or inner hits. */ - public SearchHit unfilteredSourceRef(BytesReference source) { - this.unfilteredSourceRef = source; + public SearchHit unfilteredSource(RefCounted source) { + this.unfilteredSource = source; return this; } @@ -750,10 +738,10 @@ private void deallocate() { } SearchHit.this.source = null; - if (unfilteredSourceRef != null && circuitBreaker != null) { - circuitBreaker.addWithoutBreaking(-unfilteredSourceRef.length()); + if (unfilteredSource != null) { + unfilteredSource.decRef(); + unfilteredSource = null; } - this.unfilteredSourceRef = null; } @Override @@ -788,8 +776,7 @@ public SearchHit asUnpooled() { : innerHits.entrySet().stream().collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().asUnpooled())), documentFields, metaFields, - ALWAYS_REFERENCED, - null + ALWAYS_REFERENCED ); } diff --git a/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java b/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java index 8baf1f73719e2..27834a848fb30 100644 --- a/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java +++ b/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java @@ -13,6 +13,7 @@ import org.apache.logging.log4j.Logger; import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.search.TotalHits; +import org.elasticsearch.common.MemoryAccountingBytesRefCounted; import org.elasticsearch.common.breaker.CircuitBreaker; import org.elasticsearch.common.breaker.CircuitBreakingException; import org.elasticsearch.common.breaker.NoopCircuitBreaker; @@ -301,20 +302,23 @@ private static HitContext prepareNonNestedHitContext( String id = idLoader.getId(subDocId); if (id == null) { - SearchHit hit = new SearchHit(docId, circuitBreaker); + SearchHit hit = new SearchHit(docId); // TODO: can we use real pooled buffers here as well? Source source = Source.lazy(lazyStoredSourceLoader(profiler, subReaderContext, subDocId, hit, circuitBreaker)); return new HitContext(hit, subReaderContext, subDocId, Map.of(), source, rankDoc); } else { - SearchHit hit = new SearchHit(docId, id, circuitBreaker); + SearchHit hit = new SearchHit(docId, id); Source source; if (requiresSource) { Timer timer = profiler.startLoadingSource(); try { source = sourceLoader.source(leafStoredFieldLoader, subDocId); - BytesReference sourceRef = source.internalSourceRef(); - circuitBreaker.addEstimateBytesAndMaybeBreak(sourceRef.length(), "fetch phase source loader"); - hit.unfilteredSourceRef(sourceRef); + MemoryAccountingBytesRefCounted sourceRef = MemoryAccountingBytesRefCounted.createAndAccountForBytes( + source.internalSourceRef(), + circuitBreaker, + "fetch phase source loader" + ); + hit.unfilteredSource(sourceRef); } catch (CircuitBreakingException e) { hit.decRef(); throw e; @@ -343,7 +347,11 @@ private static Supplier lazyStoredSourceLoader( LeafStoredFieldLoader leafRootLoader = rootLoader.getLoader(ctx, null); leafRootLoader.advanceTo(doc); BytesReference source = leafRootLoader.source(); - circuitBreaker.addEstimateBytesAndMaybeBreak(source.length(), "fetch phase source loader"); + MemoryAccountingBytesRefCounted memAccountingSourceRef = MemoryAccountingBytesRefCounted.createAndAccountForBytes( + source, + circuitBreaker, + "fetch phase source loader" + ); // Saving the entire source we loaded in the hit, so that we can release it entirely when the hit is released // This is important for the circuit breaker accounting - note that this lazy loader can be triggered in the case of // inner hits even though the top hit source is not requested (see {@link FetchPhase#prepareNestedHitContext} when @@ -351,7 +359,7 @@ private static Supplier lazyStoredSourceLoader( // for the top level source via the {@link SearchHit#unfilteredSourceRef(BytesReference)} method because the // {@link SearchHit#source()} method can be null when the top level source is not requested. // NB all of the above also applies for source filtering and field collapsing - hit.unfilteredSourceRef(source); + hit.unfilteredSource(memAccountingSourceRef); return Source.fromBytes(source); } catch (IOException e) { throw new UncheckedIOException(e); @@ -410,7 +418,7 @@ private static HitContext prepareNestedHitContext( // nested hits do not record their source size, as the top level hit will do so (nested hits will only reference part of the top // level source) - SearchHit nestedHit = new SearchHit(topDocId, rootId, nestedIdentity, FetchPhase.NOOP_CIRCUIT_BREAKER); + SearchHit nestedHit = new SearchHit(topDocId, rootId, nestedIdentity); return new HitContext(nestedHit, subReaderContext, nestedInfo.doc(), childFieldLoader.storedFields(), nestedSource, rankDoc); } diff --git a/server/src/test/java/org/elasticsearch/action/search/ExpandSearchPhaseTests.java b/server/src/test/java/org/elasticsearch/action/search/ExpandSearchPhaseTests.java index 374dbcf6cb519..3f8a12f9a8ef6 100644 --- a/server/src/test/java/org/elasticsearch/action/search/ExpandSearchPhaseTests.java +++ b/server/src/test/java/org/elasticsearch/action/search/ExpandSearchPhaseTests.java @@ -115,7 +115,7 @@ void sendExecuteMultiSearch(MultiSearchRequest request, SearchTask task, ActionL } }; - SearchHit hit = new SearchHit(1, "ID", new NoopCircuitBreaker(CircuitBreaker.REQUEST)); + SearchHit hit = new SearchHit(1, "ID"); hit.setDocumentField("someField", new DocumentField("someField", Collections.singletonList(collapseValue))); SearchHits hits = new SearchHits(new SearchHit[] { hit }, new TotalHits(1, TotalHits.Relation.EQUAL_TO), 1.0F); try { @@ -158,9 +158,8 @@ protected void run() { public void testFailOneItemFailsEntirePhase() throws IOException { AtomicBoolean executedMultiSearch = new AtomicBoolean(false); - NoopCircuitBreaker breaker = new NoopCircuitBreaker(CircuitBreaker.REQUEST); SearchHits collapsedHits = new SearchHits( - new SearchHit[] { new SearchHit(2, "ID", breaker), new SearchHit(3, "ID", breaker) }, + new SearchHit[] { new SearchHit(2, "ID"), new SearchHit(3, "ID") }, new TotalHits(1, TotalHits.Relation.EQUAL_TO), 1.0F ); @@ -204,9 +203,9 @@ void sendExecuteMultiSearch(MultiSearchRequest request, SearchTask task, ActionL } }; - SearchHit hit1 = new SearchHit(1, "ID", breaker); + SearchHit hit1 = new SearchHit(1, "ID"); hit1.setDocumentField("someField", new DocumentField("someField", Collections.singletonList(collapseValue))); - SearchHit hit2 = new SearchHit(2, "ID2", breaker); + SearchHit hit2 = new SearchHit(2, "ID2"); hit2.setDocumentField("someField", new DocumentField("someField", Collections.singletonList(collapseValue))); SearchHits hits = new SearchHits(new SearchHit[] { hit1, hit2 }, new TotalHits(1, TotalHits.Relation.EQUAL_TO), 1.0F); try { @@ -241,9 +240,9 @@ void sendExecuteMultiSearch(MultiSearchRequest request, SearchTask task, ActionL }; CircuitBreaker breaker = new NoopCircuitBreaker(CircuitBreaker.REQUEST); - SearchHit hit1 = new SearchHit(1, "ID", breaker); + SearchHit hit1 = new SearchHit(1, "ID"); hit1.setDocumentField("someField", new DocumentField("someField", Collections.singletonList(null))); - SearchHit hit2 = new SearchHit(2, "ID2", breaker); + SearchHit hit2 = new SearchHit(2, "ID2"); hit2.setDocumentField("someField", new DocumentField("someField", Collections.singletonList(null))); SearchHits hits = new SearchHits(new SearchHit[] { hit1, hit2 }, new TotalHits(1, TotalHits.Relation.EQUAL_TO), 1.0F); try { @@ -333,7 +332,7 @@ void sendExecuteMultiSearch(MultiSearchRequest request, SearchTask task, ActionL .preference("foobar") .routing("baz"); - SearchHit hit = new SearchHit(1, "ID", new NoopCircuitBreaker(CircuitBreaker.REQUEST)); + SearchHit hit = new SearchHit(1, "ID"); hit.setDocumentField("someField", new DocumentField("someField", Collections.singletonList("foo"))); SearchHits hits = new SearchHits(new SearchHit[] { hit }, new TotalHits(1, TotalHits.Relation.EQUAL_TO), 1.0F); try { @@ -382,7 +381,7 @@ void sendExecuteMultiSearch(MultiSearchRequest request, SearchTask task, ActionL ) .routing("baz"); - SearchHit hit = new SearchHit(1, "ID", new NoopCircuitBreaker(CircuitBreaker.REQUEST)); + SearchHit hit = new SearchHit(1, "ID"); hit.setDocumentField("someField", new DocumentField("someField", Collections.singletonList("foo"))); SearchHits hits = new SearchHits(new SearchHit[] { hit }, new TotalHits(1, TotalHits.Relation.EQUAL_TO), 1.0F); try { diff --git a/server/src/test/java/org/elasticsearch/action/search/FetchLookupFieldsPhaseTests.java b/server/src/test/java/org/elasticsearch/action/search/FetchLookupFieldsPhaseTests.java index 4cd16d7a19537..5c508dce61fc3 100644 --- a/server/src/test/java/org/elasticsearch/action/search/FetchLookupFieldsPhaseTests.java +++ b/server/src/test/java/org/elasticsearch/action/search/FetchLookupFieldsPhaseTests.java @@ -11,8 +11,6 @@ import org.apache.lucene.search.TotalHits; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.common.breaker.CircuitBreaker; -import org.elasticsearch.common.breaker.NoopCircuitBreaker; import org.elasticsearch.common.document.DocumentField; import org.elasticsearch.index.query.TermQueryBuilder; import org.elasticsearch.search.SearchHit; @@ -102,7 +100,7 @@ void sendExecuteMultiSearch( }; final SearchHits searchHits; if (fields != null) { - final SearchHit hit = new SearchHit(randomInt(1000), new NoopCircuitBreaker(CircuitBreaker.REQUEST)); + final SearchHit hit = new SearchHit(randomInt(1000)); fields.forEach((f, values) -> hit.setDocumentField(f, new DocumentField(f, values, List.of()))); searchHits = new SearchHits(new SearchHit[] { hit }, new TotalHits(1, TotalHits.Relation.EQUAL_TO), 1.0f); } else { @@ -134,7 +132,7 @@ void sendExecuteMultiSearch( } }; - SearchHit leftHit0 = new SearchHit(randomInt(100), new NoopCircuitBreaker(CircuitBreaker.REQUEST)); + SearchHit leftHit0 = new SearchHit(randomInt(100)); final List fetchFields = List.of(new FieldAndFormat(randomAlphaOfLength(10), null)); { leftHit0.setDocumentField( @@ -160,7 +158,7 @@ void sendExecuteMultiSearch( ); } - SearchHit leftHit1 = new SearchHit(randomInt(100), new NoopCircuitBreaker(CircuitBreaker.REQUEST)); + SearchHit leftHit1 = new SearchHit(randomInt(100)); { leftHit1.setDocumentField( "lookup_field_2", diff --git a/server/src/test/java/org/elasticsearch/action/search/SearchResponseMergerTests.java b/server/src/test/java/org/elasticsearch/action/search/SearchResponseMergerTests.java index 197ece084e7f5..d54ac9c66d9a5 100644 --- a/server/src/test/java/org/elasticsearch/action/search/SearchResponseMergerTests.java +++ b/server/src/test/java/org/elasticsearch/action/search/SearchResponseMergerTests.java @@ -13,8 +13,6 @@ import org.apache.lucene.search.TotalHits; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.action.search.TransportSearchAction.SearchTimeProvider; -import org.elasticsearch.common.breaker.CircuitBreaker; -import org.elasticsearch.common.breaker.NoopCircuitBreaker; import org.elasticsearch.common.text.Text; import org.elasticsearch.common.time.DateFormatter; import org.elasticsearch.core.Tuple; @@ -1096,7 +1094,7 @@ private static SearchHit[] randomSearchHitArray( for (int j = 0; j < numDocs; j++) { ShardId shardId = new ShardId(randomFrom(indices), randomIntBetween(0, 10)); SearchShardTarget shardTarget = new SearchShardTarget(randomAlphaOfLengthBetween(3, 8), shardId, clusterAlias); - SearchHit hit = new SearchHit(randomIntBetween(0, Integer.MAX_VALUE), new NoopCircuitBreaker(CircuitBreaker.REQUEST)); + SearchHit hit = new SearchHit(randomIntBetween(0, Integer.MAX_VALUE)); float score = Float.NaN; if (Float.isNaN(maxScore) == false) { diff --git a/server/src/test/java/org/elasticsearch/action/search/SearchResponseTests.java b/server/src/test/java/org/elasticsearch/action/search/SearchResponseTests.java index 7d3912e923b59..bbeae6b19b8ac 100644 --- a/server/src/test/java/org/elasticsearch/action/search/SearchResponseTests.java +++ b/server/src/test/java/org/elasticsearch/action/search/SearchResponseTests.java @@ -15,8 +15,6 @@ import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.common.Strings; -import org.elasticsearch.common.breaker.CircuitBreaker; -import org.elasticsearch.common.breaker.NoopCircuitBreaker; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.settings.Settings; @@ -367,7 +365,7 @@ public void testFromXContentWithFailures() throws IOException { } public void testToXContent() throws IOException { - SearchHit hit = new SearchHit(1, "id1", new NoopCircuitBreaker(CircuitBreaker.REQUEST)); + SearchHit hit = new SearchHit(1, "id1"); hit.score(2.0f); SearchHit[] hits = new SearchHit[] { hit }; var sHits = new SearchHits(hits, new TotalHits(100, TotalHits.Relation.EQUAL_TO), 1.5f); diff --git a/server/src/test/java/org/elasticsearch/search/SearchHitTests.java b/server/src/test/java/org/elasticsearch/search/SearchHitTests.java index cd8002f62c6b5..25a71d04b321d 100644 --- a/server/src/test/java/org/elasticsearch/search/SearchHitTests.java +++ b/server/src/test/java/org/elasticsearch/search/SearchHitTests.java @@ -13,8 +13,6 @@ import org.apache.lucene.search.TotalHits; import org.elasticsearch.TransportVersion; import org.elasticsearch.common.Strings; -import org.elasticsearch.common.breaker.CircuitBreaker; -import org.elasticsearch.common.breaker.NoopCircuitBreaker; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.document.DocumentField; @@ -76,7 +74,7 @@ public static SearchHit createTestItem(XContentType xContentType, boolean withOp documentFields = GetResultTests.randomDocumentFields(xContentType, false).v2(); } } - SearchHit hit = new SearchHit(internalId, uid, nestedIdentity, new NoopCircuitBreaker(CircuitBreaker.REQUEST)); + SearchHit hit = new SearchHit(internalId, uid, nestedIdentity); hit.addDocumentFields(documentFields, metaFields); if (frequently()) { if (rarely()) { @@ -232,7 +230,7 @@ public void testFromXContentWithoutTypeAndId() throws IOException { } public void testToXContent() throws IOException { - SearchHit searchHit = new SearchHit(1, "id1", new NoopCircuitBreaker(CircuitBreaker.REQUEST)); + SearchHit searchHit = new SearchHit(1, "id1"); try { searchHit.score(1.5f); XContentBuilder builder = JsonXContent.contentBuilder(); @@ -256,28 +254,27 @@ public void testRankToXContent() throws IOException { public void testSerializeShardTarget() throws Exception { String clusterAlias = randomBoolean() ? null : "cluster_alias"; SearchShardTarget target = new SearchShardTarget("_node_id", new ShardId(new Index("_index", "_na_"), 0), clusterAlias); - NoopCircuitBreaker noopCircuitBreaker = new NoopCircuitBreaker(CircuitBreaker.REQUEST); Map innerHits = new HashMap<>(); - SearchHit innerHit1 = new SearchHit(0, "_id", noopCircuitBreaker); + SearchHit innerHit1 = new SearchHit(0, "_id"); innerHit1.shard(target); - SearchHit innerInnerHit2 = new SearchHit(0, "_id", noopCircuitBreaker); + SearchHit innerInnerHit2 = new SearchHit(0, "_id"); innerInnerHit2.shard(target); innerHits.put("1", new SearchHits(new SearchHit[] { innerInnerHit2 }, new TotalHits(1, TotalHits.Relation.EQUAL_TO), 1f)); innerHit1.setInnerHits(innerHits); - SearchHit innerHit2 = new SearchHit(0, "_id", noopCircuitBreaker); + SearchHit innerHit2 = new SearchHit(0, "_id"); innerHit2.shard(target); - SearchHit innerHit3 = new SearchHit(0, "_id", noopCircuitBreaker); + SearchHit innerHit3 = new SearchHit(0, "_id"); innerHit3.shard(target); innerHits = new HashMap<>(); - SearchHit hit1 = new SearchHit(0, "_id", noopCircuitBreaker); + SearchHit hit1 = new SearchHit(0, "_id"); innerHits.put("1", new SearchHits(new SearchHit[] { innerHit1, innerHit2 }, new TotalHits(1, TotalHits.Relation.EQUAL_TO), 1f)); innerHits.put("2", new SearchHits(new SearchHit[] { innerHit3 }, new TotalHits(1, TotalHits.Relation.EQUAL_TO), 1f)); hit1.shard(target); hit1.setInnerHits(innerHits); - SearchHit hit2 = new SearchHit(0, "_id", noopCircuitBreaker); + SearchHit hit2 = new SearchHit(0, "_id"); hit2.shard(target); SearchHits hits = new SearchHits(new SearchHit[] { hit1, hit2 }, new TotalHits(2, TotalHits.Relation.EQUAL_TO), 1f); diff --git a/server/src/test/java/org/elasticsearch/search/fetch/FetchPhaseDocsIteratorTests.java b/server/src/test/java/org/elasticsearch/search/fetch/FetchPhaseDocsIteratorTests.java index cae4ffe2ee2c1..c8d1b6721c64b 100644 --- a/server/src/test/java/org/elasticsearch/search/fetch/FetchPhaseDocsIteratorTests.java +++ b/server/src/test/java/org/elasticsearch/search/fetch/FetchPhaseDocsIteratorTests.java @@ -16,8 +16,6 @@ import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.store.Directory; import org.apache.lucene.tests.index.RandomIndexWriter; -import org.elasticsearch.common.breaker.CircuitBreaker; -import org.elasticsearch.common.breaker.NoopCircuitBreaker; import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.query.QuerySearchResult; import org.elasticsearch.test.ESTestCase; @@ -76,7 +74,7 @@ protected void setNextReader(LeafReaderContext ctx, int[] docsInLeaf) { protected SearchHit nextDoc(int doc) { assertThat(doc, equalTo(this.docsInLeaf[this.index] + this.ctx.docBase)); index++; - return new SearchHit(doc, new NoopCircuitBreaker(CircuitBreaker.REQUEST)); + return new SearchHit(doc); } }; @@ -124,7 +122,7 @@ protected SearchHit nextDoc(int doc) { if (doc == badDoc) { throw new IllegalArgumentException("Error processing doc"); } - return new SearchHit(doc, new NoopCircuitBreaker(CircuitBreaker.REQUEST)); + return new SearchHit(doc); } }; diff --git a/test/framework/src/main/java/org/elasticsearch/search/SearchResponseUtils.java b/test/framework/src/main/java/org/elasticsearch/search/SearchResponseUtils.java index c197bceb7ce8c..21b43636222f9 100644 --- a/test/framework/src/main/java/org/elasticsearch/search/SearchResponseUtils.java +++ b/test/framework/src/main/java/org/elasticsearch/search/SearchResponseUtils.java @@ -892,8 +892,7 @@ public static SearchHit searchHitFromMap(Map values) { get(SearchHit.Fields.INNER_HITS, values, null), get(SearchHit.DOCUMENT_FIELDS, values, Collections.emptyMap()), get(SearchHit.METADATA_FIELDS, values, Collections.emptyMap()), - RefCounted.ALWAYS_REFERENCED, // TODO: do we ever want pooling here? - null + RefCounted.ALWAYS_REFERENCED // TODO: do we ever want pooling here? ); } diff --git a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/highlight/SemanticTextHighlighterTests.java b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/highlight/SemanticTextHighlighterTests.java index 9ec98e2fca546..b808dce4f9750 100644 --- a/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/highlight/SemanticTextHighlighterTests.java +++ b/x-pack/plugin/inference/src/test/java/org/elasticsearch/xpack/inference/highlight/SemanticTextHighlighterTests.java @@ -21,8 +21,6 @@ import org.apache.lucene.util.BytesRef; import org.elasticsearch.action.OriginalIndices; import org.elasticsearch.action.search.SearchRequest; -import org.elasticsearch.common.breaker.CircuitBreaker; -import org.elasticsearch.common.breaker.NoopCircuitBreaker; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.io.Streams; @@ -242,7 +240,7 @@ private void assertHighlightOneDoc( Mockito.when(fetchContext.getSearchExecutionContext()).thenReturn(execContext); FetchSubPhase.HitContext hitContext = new FetchSubPhase.HitContext( - new SearchHit(docID, new NoopCircuitBreaker(CircuitBreaker.REQUEST)), + new SearchHit(docID), getOnlyLeafReader(reader).getContext(), docID, Map.of(), diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsProviderTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsProviderTests.java index 8aaa0452a643f..3dcbbeb3fcce5 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsProviderTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsProviderTests.java @@ -20,8 +20,6 @@ import org.elasticsearch.client.internal.Client; import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.cluster.metadata.MappingMetadata; -import org.elasticsearch.common.breaker.CircuitBreaker; -import org.elasticsearch.common.breaker.NoopCircuitBreaker; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.document.DocumentField; import org.elasticsearch.common.settings.Settings; @@ -923,7 +921,7 @@ private static SearchResponse createSearchResponse(List> sou fields.put("field_1", new DocumentField("field_1", Collections.singletonList("foo"))); fields.put("field_2", new DocumentField("field_2", Collections.singletonList("foo"))); - SearchHit hit = new SearchHit(123, String.valueOf(map.hashCode()), new NoopCircuitBreaker(CircuitBreaker.REQUEST)); + SearchHit hit = new SearchHit(123, String.valueOf(map.hashCode())); hit.addDocumentFields(fields, Collections.emptyMap()); hit.sourceRef(BytesReference.bytes(XContentFactory.jsonBuilder().map(_source))); diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/retention/AbstractExpiredJobDataRemoverTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/retention/AbstractExpiredJobDataRemoverTests.java index ac7bffc0dc16d..6ec43ca2a3201 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/retention/AbstractExpiredJobDataRemoverTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/retention/AbstractExpiredJobDataRemoverTests.java @@ -11,8 +11,6 @@ import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.client.internal.Client; import org.elasticsearch.client.internal.OriginSettingClient; -import org.elasticsearch.common.breaker.CircuitBreaker; -import org.elasticsearch.common.breaker.NoopCircuitBreaker; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.core.TimeValue; import org.elasticsearch.search.SearchHit; @@ -107,7 +105,7 @@ static SearchResponse createSearchResponseFromHits(List hits) { private static SearchResponse createSearchResponse(List toXContents, int totalHits) throws IOException { SearchHit[] hitsArray = new SearchHit[toXContents.size()]; for (int i = 0; i < toXContents.size(); i++) { - hitsArray[i] = new SearchHit(randomInt(), new NoopCircuitBreaker(CircuitBreaker.REQUEST)); + hitsArray[i] = new SearchHit(randomInt()); XContentBuilder jsonBuilder = JsonXContent.contentBuilder(); toXContents.get(i).toXContent(jsonBuilder, ToXContent.EMPTY_PARAMS); hitsArray[i].sourceRef(BytesReference.bytes(jsonBuilder)); diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/action/saml/TransportSamlInvalidateSessionActionTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/action/saml/TransportSamlInvalidateSessionActionTests.java index 31d1c85735677..269f8cb0471fc 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/action/saml/TransportSamlInvalidateSessionActionTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/action/saml/TransportSamlInvalidateSessionActionTests.java @@ -38,8 +38,6 @@ import org.elasticsearch.action.update.UpdateResponse; import org.elasticsearch.client.internal.Client; import org.elasticsearch.cluster.service.ClusterService; -import org.elasticsearch.common.breaker.CircuitBreaker; -import org.elasticsearch.common.breaker.NoopCircuitBreaker; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.ThreadContext; @@ -335,7 +333,7 @@ private SearchHit tokenHit(int idx, BytesReference source) { final Map accessToken = (Map) sourceMap.get("access_token"); @SuppressWarnings("unchecked") final Map userToken = (Map) accessToken.get("user_token"); - final SearchHit hit = new SearchHit(idx, "token_" + userToken.get("id"), new NoopCircuitBreaker(CircuitBreaker.REQUEST)); + final SearchHit hit = new SearchHit(idx, "token_" + userToken.get("id")); hit.sourceRef(source); return hit; } catch (IOException e) { diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/store/NativePrivilegeStoreTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/store/NativePrivilegeStoreTests.java index 60c7e2e6bc72c..ed3949450cb9f 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/store/NativePrivilegeStoreTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/store/NativePrivilegeStoreTests.java @@ -33,8 +33,6 @@ import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.Strings; import org.elasticsearch.common.UUIDs; -import org.elasticsearch.common.breaker.CircuitBreaker; -import org.elasticsearch.common.breaker.NoopCircuitBreaker; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.settings.ClusterSettings; @@ -923,11 +921,7 @@ private SearchHit[] buildHits(List sourcePrivile final SearchHit[] hits = new SearchHit[sourcePrivileges.size()]; for (int i = 0; i < hits.length; i++) { final ApplicationPrivilegeDescriptor p = sourcePrivileges.get(i); - hits[i] = new SearchHit( - i, - "application-privilege_" + p.getApplication() + ":" + p.getName(), - new NoopCircuitBreaker(CircuitBreaker.REQUEST) - ); + hits[i] = new SearchHit(i, "application-privilege_" + p.getApplication() + ":" + p.getName()); hits[i].sourceRef(new BytesArray(Strings.toString(p))); } return hits; diff --git a/x-pack/plugin/transform/src/test/java/org/elasticsearch/xpack/transform/transforms/TransformIndexerTests.java b/x-pack/plugin/transform/src/test/java/org/elasticsearch/xpack/transform/transforms/TransformIndexerTests.java index 01a6c7a0b228e..644518538638d 100644 --- a/x-pack/plugin/transform/src/test/java/org/elasticsearch/xpack/transform/transforms/TransformIndexerTests.java +++ b/x-pack/plugin/transform/src/test/java/org/elasticsearch/xpack/transform/transforms/TransformIndexerTests.java @@ -21,7 +21,6 @@ import org.elasticsearch.client.internal.Client; import org.elasticsearch.common.breaker.CircuitBreaker; import org.elasticsearch.common.breaker.CircuitBreakingException; -import org.elasticsearch.common.breaker.NoopCircuitBreaker; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.core.TimeValue; import org.elasticsearch.index.reindex.BulkByScrollResponse; @@ -83,11 +82,7 @@ public class TransformIndexerTests extends ESTestCase { private static final SearchResponse ONE_HIT_SEARCH_RESPONSE = new SearchResponse( - new SearchHits( - new SearchHit[] { new SearchHit(1, new NoopCircuitBreaker(CircuitBreaker.REQUEST)) }, - new TotalHits(1L, TotalHits.Relation.EQUAL_TO), - 1.0f - ), + new SearchHits(new SearchHit[] { new SearchHit(1) }, new TotalHits(1L, TotalHits.Relation.EQUAL_TO), 1.0f), // Simulate completely null aggs null, new Suggest(Collections.emptyList()), From 4083d640091c15eddc42ef5ceac9e45120c40931 Mon Sep 17 00:00:00 2001 From: Andrei Dan Date: Tue, 11 Feb 2025 12:47:02 +0000 Subject: [PATCH 14/56] Use unfiltered parent source ref counted --- .../enrich/action/EnrichShardMultiSearchAction.java | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/x-pack/plugin/enrich/src/main/java/org/elasticsearch/xpack/enrich/action/EnrichShardMultiSearchAction.java b/x-pack/plugin/enrich/src/main/java/org/elasticsearch/xpack/enrich/action/EnrichShardMultiSearchAction.java index ec4d1a74adb59..7e6585174b0fa 100644 --- a/x-pack/plugin/enrich/src/main/java/org/elasticsearch/xpack/enrich/action/EnrichShardMultiSearchAction.java +++ b/x-pack/plugin/enrich/src/main/java/org/elasticsearch/xpack/enrich/action/EnrichShardMultiSearchAction.java @@ -29,6 +29,7 @@ import org.elasticsearch.cluster.routing.Preference; import org.elasticsearch.cluster.routing.ShardsIterator; import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.MemoryAccountingBytesRefCounted; import org.elasticsearch.common.Strings; import org.elasticsearch.common.breaker.CircuitBreaker; import org.elasticsearch.common.breaker.CircuitBreakingException; @@ -264,10 +265,11 @@ protected MultiSearchResponse shardOperation(Request request, ShardId shardId) t }); final SearchHit hit = new SearchHit(scoreDoc.doc, visitor.id(), breaker); try { - BytesReference source = visitor.source(); - hit.unfilteredSourceRef(source); - breaker.addEstimateBytesAndMaybeBreak(source.length(), "enrich_msearch"); - hit.sourceRef(filterSource(fetchSourceContext, source)); + BytesReference sourceBytesRef = visitor.source(); + MemoryAccountingBytesRefCounted memAccountingSourceRef = MemoryAccountingBytesRefCounted + .createAndAccountForBytes(sourceBytesRef, breaker, "enrich source"); + hit.unfilteredSource(memAccountingSourceRef); + hit.sourceRef(filterSource(fetchSourceContext, sourceBytesRef)); hits[j] = hit; } catch (CircuitBreakingException e) { hit.decRef(); From 8831a33733c4dbe3cd431443aa42169df5a4e897 Mon Sep 17 00:00:00 2001 From: Andrei Dan Date: Tue, 11 Feb 2025 13:01:34 +0000 Subject: [PATCH 15/56] Compile --- .../xpack/enrich/action/EnrichShardMultiSearchAction.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/x-pack/plugin/enrich/src/main/java/org/elasticsearch/xpack/enrich/action/EnrichShardMultiSearchAction.java b/x-pack/plugin/enrich/src/main/java/org/elasticsearch/xpack/enrich/action/EnrichShardMultiSearchAction.java index 7e6585174b0fa..5ef47b47c9bf3 100644 --- a/x-pack/plugin/enrich/src/main/java/org/elasticsearch/xpack/enrich/action/EnrichShardMultiSearchAction.java +++ b/x-pack/plugin/enrich/src/main/java/org/elasticsearch/xpack/enrich/action/EnrichShardMultiSearchAction.java @@ -263,7 +263,7 @@ protected MultiSearchResponse shardOperation(Request request, ShardId shardId) t } return context.getFieldType(field); }); - final SearchHit hit = new SearchHit(scoreDoc.doc, visitor.id(), breaker); + final SearchHit hit = new SearchHit(scoreDoc.doc, visitor.id()); try { BytesReference sourceBytesRef = visitor.source(); MemoryAccountingBytesRefCounted memAccountingSourceRef = MemoryAccountingBytesRefCounted From 6ffec2d299d738d45e01cf19aa652e2bc3837144 Mon Sep 17 00:00:00 2001 From: Andrei Dan Date: Tue, 11 Feb 2025 13:29:28 +0000 Subject: [PATCH 16/56] Compile --- .../xpack/ml/job/persistence/MockClientBuilder.java | 4 +--- .../xpack/ml/job/persistence/StateStreamerTests.java | 6 +----- 2 files changed, 2 insertions(+), 8 deletions(-) diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/MockClientBuilder.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/MockClientBuilder.java index 5ee00058dedb8..ebc92eee61ea2 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/MockClientBuilder.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/MockClientBuilder.java @@ -25,8 +25,6 @@ import org.elasticsearch.client.internal.Client; import org.elasticsearch.client.internal.ClusterAdminClient; import org.elasticsearch.client.internal.IndicesAdminClient; -import org.elasticsearch.common.breaker.CircuitBreaker; -import org.elasticsearch.common.breaker.NoopCircuitBreaker; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.document.DocumentField; import org.elasticsearch.common.settings.Settings; @@ -132,7 +130,7 @@ public MockClientBuilder prepareSearch(String indexName, List do SearchHit hits[] = new SearchHit[docs.size()]; for (int i = 0; i < docs.size(); i++) { - SearchHit hit = new SearchHit(10, new NoopCircuitBreaker(CircuitBreaker.REQUEST)); + SearchHit hit = new SearchHit(10); hit.sourceRef(docs.get(i)); hits[i] = hit; } diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/StateStreamerTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/StateStreamerTests.java index 9b87edd5f564e..47f7d8c65a27a 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/StateStreamerTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/StateStreamerTests.java @@ -9,8 +9,6 @@ import org.elasticsearch.action.search.SearchRequestBuilder; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.client.internal.Client; -import org.elasticsearch.common.breaker.CircuitBreaker; -import org.elasticsearch.common.breaker.NoopCircuitBreaker; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.index.query.QueryBuilder; import org.elasticsearch.index.query.QueryBuilders; @@ -105,9 +103,7 @@ private static SearchResponse createSearchResponse(List> sou SearchHit[] hits = new SearchHit[source.size()]; int i = 0; for (Map s : source) { - SearchHit hit = new SearchHit(1, new NoopCircuitBreaker(CircuitBreaker.REQUEST)).sourceRef( - BytesReference.bytes(XContentFactory.jsonBuilder().map(s)) - ); + SearchHit hit = new SearchHit(1).sourceRef(BytesReference.bytes(XContentFactory.jsonBuilder().map(s))); hits[i++] = hit; } SearchHits searchHits = new SearchHits(hits, null, (float) 0.0); From f4e111b8e9ce9eaa51ae65dc8cb6d76feb89f0c9 Mon Sep 17 00:00:00 2001 From: Andrei Dan Date: Tue, 11 Feb 2025 14:26:37 +0000 Subject: [PATCH 17/56] Use the existing refCounted field in SearchHit --- .../MemoryAccountingBytesRefCounted.java | 24 ++++++------- .../org/elasticsearch/search/SearchHit.java | 15 +------- .../search/fetch/FetchPhase.java | 36 ++++++------------- .../action/EnrichShardMultiSearchAction.java | 7 ++-- 4 files changed, 26 insertions(+), 56 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/common/MemoryAccountingBytesRefCounted.java b/server/src/main/java/org/elasticsearch/common/MemoryAccountingBytesRefCounted.java index 2307e4639f06b..70a8273962e55 100644 --- a/server/src/main/java/org/elasticsearch/common/MemoryAccountingBytesRefCounted.java +++ b/server/src/main/java/org/elasticsearch/common/MemoryAccountingBytesRefCounted.java @@ -10,30 +10,30 @@ package org.elasticsearch.common; import org.elasticsearch.common.breaker.CircuitBreaker; -import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.core.AbstractRefCounted; public final class MemoryAccountingBytesRefCounted extends AbstractRefCounted { - private BytesReference bytes; - private CircuitBreaker breaker; + private long bytes; + private final CircuitBreaker breaker; - private MemoryAccountingBytesRefCounted(BytesReference bytes, CircuitBreaker breaker) { + private MemoryAccountingBytesRefCounted(long bytes, CircuitBreaker breaker) { this.bytes = bytes; this.breaker = breaker; } - public static MemoryAccountingBytesRefCounted createAndAccountForBytes( - BytesReference bytes, - CircuitBreaker breaker, - String memAccountingLabel - ) { - breaker.addEstimateBytesAndMaybeBreak(bytes.length(), memAccountingLabel); - return new MemoryAccountingBytesRefCounted(bytes, breaker); + public static MemoryAccountingBytesRefCounted create(CircuitBreaker breaker) { + return new MemoryAccountingBytesRefCounted(0L, breaker); + } + + public void account(long bytes, String label) { + breaker.addEstimateBytesAndMaybeBreak(bytes, label); + this.bytes += bytes; } @Override protected void closeInternal() { - breaker.addWithoutBreaking(-bytes.length()); + System.err.println("REMOVING " + bytes); + breaker.addWithoutBreaking(-bytes); } } diff --git a/server/src/main/java/org/elasticsearch/search/SearchHit.java b/server/src/main/java/org/elasticsearch/search/SearchHit.java index 4d29ce112ae94..535379d71c378 100644 --- a/server/src/main/java/org/elasticsearch/search/SearchHit.java +++ b/server/src/main/java/org/elasticsearch/search/SearchHit.java @@ -83,8 +83,6 @@ public final class SearchHit implements Writeable, ToXContentObject, RefCounted private long primaryTerm; private BytesReference source; - @Nullable - private RefCounted unfilteredSource; private final Map documentFields; private final Map metaFields; @@ -125,7 +123,7 @@ public SearchHit(int nestedTopDocId, String id, NestedIdentity nestedIdentity) { this(nestedTopDocId, id, nestedIdentity, null); } - private SearchHit(int nestedTopDocId, String id, NestedIdentity nestedIdentity, @Nullable RefCounted refCounted) { + public SearchHit(int nestedTopDocId, String id, NestedIdentity nestedIdentity, @Nullable RefCounted refCounted) { this( nestedTopDocId, DEFAULT_SCORE, @@ -449,17 +447,6 @@ public SearchHit sourceRef(BytesReference source) { return this; } - /** - * We track the unfiltered, entire, source so we can release the entire size from the - * circuit breakers when the hit is released. - * The regular source might be a subset of the unfiltered source due to either - * source filtering, field collapsing or inner hits. - */ - public SearchHit unfilteredSource(RefCounted source) { - this.unfilteredSource = source; - return this; - } - /** * Is the source available or not. A source with no fields will return true. This will return false if {@code fields} doesn't contain * {@code _source} or if source is disabled in the mapping. diff --git a/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java b/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java index 27834a848fb30..38a40ec71f20e 100644 --- a/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java +++ b/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java @@ -40,6 +40,7 @@ import org.elasticsearch.search.rank.RankDoc; import org.elasticsearch.search.rank.RankDocShardInfo; import org.elasticsearch.tasks.TaskCancelledException; +import org.elasticsearch.transport.LeakTracker; import org.elasticsearch.xcontent.XContentType; import java.io.IOException; @@ -302,23 +303,20 @@ private static HitContext prepareNonNestedHitContext( String id = idLoader.getId(subDocId); if (id == null) { - SearchHit hit = new SearchHit(docId); + MemoryAccountingBytesRefCounted memAccountingRefCounted = MemoryAccountingBytesRefCounted.create(circuitBreaker); + SearchHit hit = new SearchHit(docId, null, null, LeakTracker.wrap(memAccountingRefCounted)); // TODO: can we use real pooled buffers here as well? - Source source = Source.lazy(lazyStoredSourceLoader(profiler, subReaderContext, subDocId, hit, circuitBreaker)); + Source source = Source.lazy(lazyStoredSourceLoader(profiler, subReaderContext, subDocId, memAccountingRefCounted)); return new HitContext(hit, subReaderContext, subDocId, Map.of(), source, rankDoc); } else { - SearchHit hit = new SearchHit(docId, id); + MemoryAccountingBytesRefCounted memAccountingRefCounted = MemoryAccountingBytesRefCounted.create(circuitBreaker); + SearchHit hit = new SearchHit(docId, id, null, LeakTracker.wrap(memAccountingRefCounted)); Source source; if (requiresSource) { Timer timer = profiler.startLoadingSource(); try { source = sourceLoader.source(leafStoredFieldLoader, subDocId); - MemoryAccountingBytesRefCounted sourceRef = MemoryAccountingBytesRefCounted.createAndAccountForBytes( - source.internalSourceRef(), - circuitBreaker, - "fetch phase source loader" - ); - hit.unfilteredSource(sourceRef); + memAccountingRefCounted.account(source.internalSourceRef().length(), "fetch phase source loader"); } catch (CircuitBreakingException e) { hit.decRef(); throw e; @@ -328,7 +326,7 @@ private static HitContext prepareNonNestedHitContext( } } } else { - source = Source.lazy(lazyStoredSourceLoader(profiler, subReaderContext, subDocId, hit, circuitBreaker)); + source = Source.lazy(lazyStoredSourceLoader(profiler, subReaderContext, subDocId, memAccountingRefCounted)); } return new HitContext(hit, subReaderContext, subDocId, leafStoredFieldLoader.storedFields(), source, rankDoc); } @@ -338,8 +336,7 @@ private static Supplier lazyStoredSourceLoader( Profiler profiler, LeafReaderContext ctx, int doc, - SearchHit hit, - CircuitBreaker circuitBreaker + MemoryAccountingBytesRefCounted memAccountingRefCounted ) { return () -> { StoredFieldLoader rootLoader = profiler.storedFields(StoredFieldLoader.create(true, Collections.emptySet())); @@ -347,19 +344,7 @@ private static Supplier lazyStoredSourceLoader( LeafStoredFieldLoader leafRootLoader = rootLoader.getLoader(ctx, null); leafRootLoader.advanceTo(doc); BytesReference source = leafRootLoader.source(); - MemoryAccountingBytesRefCounted memAccountingSourceRef = MemoryAccountingBytesRefCounted.createAndAccountForBytes( - source, - circuitBreaker, - "fetch phase source loader" - ); - // Saving the entire source we loaded in the hit, so that we can release it entirely when the hit is released - // This is important for the circuit breaker accounting - note that this lazy loader can be triggered in the case of - // inner hits even though the top hit source is not requested (see {@link FetchPhase#prepareNestedHitContext} when - // the `nestedSource` is created), so we need to save the entire source on the hit - we account - // for the top level source via the {@link SearchHit#unfilteredSourceRef(BytesReference)} method because the - // {@link SearchHit#source()} method can be null when the top level source is not requested. - // NB all of the above also applies for source filtering and field collapsing - hit.unfilteredSource(memAccountingSourceRef); + memAccountingRefCounted.account(source.length(), "fetch phase source loader"); return Source.fromBytes(source); } catch (IOException e) { throw new UncheckedIOException(e); @@ -404,7 +389,6 @@ private static HitContext prepareNestedHitContext( if (requiresSource) { BytesReference source = leafRootLoader.source(); if (source != null) { - NOOP_CIRCUIT_BREAKER.addEstimateBytesAndMaybeBreak(source.length(), "fetch phase nested hit source loader"); rootSource = Source.fromBytes(source); } } diff --git a/x-pack/plugin/enrich/src/main/java/org/elasticsearch/xpack/enrich/action/EnrichShardMultiSearchAction.java b/x-pack/plugin/enrich/src/main/java/org/elasticsearch/xpack/enrich/action/EnrichShardMultiSearchAction.java index 5ef47b47c9bf3..6b30ebdfc7507 100644 --- a/x-pack/plugin/enrich/src/main/java/org/elasticsearch/xpack/enrich/action/EnrichShardMultiSearchAction.java +++ b/x-pack/plugin/enrich/src/main/java/org/elasticsearch/xpack/enrich/action/EnrichShardMultiSearchAction.java @@ -263,12 +263,11 @@ protected MultiSearchResponse shardOperation(Request request, ShardId shardId) t } return context.getFieldType(field); }); - final SearchHit hit = new SearchHit(scoreDoc.doc, visitor.id()); + MemoryAccountingBytesRefCounted memAccountingRefCounted = MemoryAccountingBytesRefCounted.create(breaker); + final SearchHit hit = new SearchHit(scoreDoc.doc, visitor.id(), null, memAccountingRefCounted); try { BytesReference sourceBytesRef = visitor.source(); - MemoryAccountingBytesRefCounted memAccountingSourceRef = MemoryAccountingBytesRefCounted - .createAndAccountForBytes(sourceBytesRef, breaker, "enrich source"); - hit.unfilteredSource(memAccountingSourceRef); + memAccountingRefCounted.account(sourceBytesRef.length(), "enrich source"); hit.sourceRef(filterSource(fetchSourceContext, sourceBytesRef)); hits[j] = hit; } catch (CircuitBreakingException e) { From b64bed521e95855c3972a9aca6d795e4d9a8c2c0 Mon Sep 17 00:00:00 2001 From: Andrei Dan Date: Tue, 11 Feb 2025 14:28:11 +0000 Subject: [PATCH 18/56] Leaktracker.wrap --- .../xpack/enrich/action/EnrichShardMultiSearchAction.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/x-pack/plugin/enrich/src/main/java/org/elasticsearch/xpack/enrich/action/EnrichShardMultiSearchAction.java b/x-pack/plugin/enrich/src/main/java/org/elasticsearch/xpack/enrich/action/EnrichShardMultiSearchAction.java index 6b30ebdfc7507..dabe85718ff15 100644 --- a/x-pack/plugin/enrich/src/main/java/org/elasticsearch/xpack/enrich/action/EnrichShardMultiSearchAction.java +++ b/x-pack/plugin/enrich/src/main/java/org/elasticsearch/xpack/enrich/action/EnrichShardMultiSearchAction.java @@ -57,6 +57,7 @@ import org.elasticsearch.search.builder.SearchSourceBuilder; import org.elasticsearch.search.fetch.subphase.FetchSourceContext; import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.LeakTracker; import org.elasticsearch.transport.TransportService; import org.elasticsearch.xcontent.DeprecationHandler; import org.elasticsearch.xcontent.NamedXContentRegistry; @@ -264,7 +265,7 @@ protected MultiSearchResponse shardOperation(Request request, ShardId shardId) t return context.getFieldType(field); }); MemoryAccountingBytesRefCounted memAccountingRefCounted = MemoryAccountingBytesRefCounted.create(breaker); - final SearchHit hit = new SearchHit(scoreDoc.doc, visitor.id(), null, memAccountingRefCounted); + final SearchHit hit = new SearchHit(scoreDoc.doc, visitor.id(), null, LeakTracker.wrap(memAccountingRefCounted)); try { BytesReference sourceBytesRef = visitor.source(); memAccountingRefCounted.account(sourceBytesRef.length(), "enrich source"); From 69644baaf136afcc5b3a04cc8ca1a8aebc89b018 Mon Sep 17 00:00:00 2001 From: Andrei Dan Date: Tue, 11 Feb 2025 14:32:41 +0000 Subject: [PATCH 19/56] compile --- server/src/main/java/org/elasticsearch/search/SearchHit.java | 5 ----- 1 file changed, 5 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/search/SearchHit.java b/server/src/main/java/org/elasticsearch/search/SearchHit.java index 535379d71c378..154b75d61fdbd 100644 --- a/server/src/main/java/org/elasticsearch/search/SearchHit.java +++ b/server/src/main/java/org/elasticsearch/search/SearchHit.java @@ -724,11 +724,6 @@ private void deallocate() { r.decRef(); } SearchHit.this.source = null; - - if (unfilteredSource != null) { - unfilteredSource.decRef(); - unfilteredSource = null; - } } @Override From 9aef36bf70df93cd221608d4cf797d0897da2d44 Mon Sep 17 00:00:00 2001 From: Andrei Dan Date: Tue, 11 Feb 2025 14:57:57 +0000 Subject: [PATCH 20/56] remove sout --- .../elasticsearch/common/MemoryAccountingBytesRefCounted.java | 1 - 1 file changed, 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/common/MemoryAccountingBytesRefCounted.java b/server/src/main/java/org/elasticsearch/common/MemoryAccountingBytesRefCounted.java index 70a8273962e55..a65d6d38f60e2 100644 --- a/server/src/main/java/org/elasticsearch/common/MemoryAccountingBytesRefCounted.java +++ b/server/src/main/java/org/elasticsearch/common/MemoryAccountingBytesRefCounted.java @@ -33,7 +33,6 @@ public void account(long bytes, String label) { @Override protected void closeInternal() { - System.err.println("REMOVING " + bytes); breaker.addWithoutBreaking(-bytes); } } From 87308c9a69faab08367db15693a9255b2d1738e5 Mon Sep 17 00:00:00 2001 From: Andrei Dan Date: Wed, 12 Feb 2025 09:42:49 +0000 Subject: [PATCH 21/56] Only call the breaker if bytes != 0 --- .../elasticsearch/common/MemoryAccountingBytesRefCounted.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/common/MemoryAccountingBytesRefCounted.java b/server/src/main/java/org/elasticsearch/common/MemoryAccountingBytesRefCounted.java index a65d6d38f60e2..8e56c50229f96 100644 --- a/server/src/main/java/org/elasticsearch/common/MemoryAccountingBytesRefCounted.java +++ b/server/src/main/java/org/elasticsearch/common/MemoryAccountingBytesRefCounted.java @@ -33,6 +33,8 @@ public void account(long bytes, String label) { @Override protected void closeInternal() { - breaker.addWithoutBreaking(-bytes); + if (bytes != 0L) { + breaker.addWithoutBreaking(-bytes); + } } } From 5a9c770d517c017601058b62c56be3849103264e Mon Sep 17 00:00:00 2001 From: Andrei Dan Date: Tue, 18 Feb 2025 07:49:24 +0000 Subject: [PATCH 22/56] Local accounting up to 32kb --- .../MemoryAccountingBytesRefCounted.java | 16 +-- .../common/settings/ClusterSettings.java | 1 + .../elasticsearch/search/SearchService.java | 33 +++++- .../metrics/TopHitsAggregator.java | 15 ++- .../search/fetch/FetchPhase.java | 107 +++++++++++++++--- .../search/fetch/subphase/InnerHitsPhase.java | 3 +- .../action/search/FetchSearchPhaseTests.java | 4 +- .../MemoryAccountingBytesRefCountedTests.java | 50 ++++++++ .../search/SearchServiceSingleNodeTests.java | 67 +++++++++++ 9 files changed, 262 insertions(+), 34 deletions(-) create mode 100644 server/src/test/java/org/elasticsearch/common/MemoryAccountingBytesRefCountedTests.java diff --git a/server/src/main/java/org/elasticsearch/common/MemoryAccountingBytesRefCounted.java b/server/src/main/java/org/elasticsearch/common/MemoryAccountingBytesRefCounted.java index 8e56c50229f96..d8cc638059093 100644 --- a/server/src/main/java/org/elasticsearch/common/MemoryAccountingBytesRefCounted.java +++ b/server/src/main/java/org/elasticsearch/common/MemoryAccountingBytesRefCounted.java @@ -12,29 +12,31 @@ import org.elasticsearch.common.breaker.CircuitBreaker; import org.elasticsearch.core.AbstractRefCounted; +/** + * A ref counted object that accounts for memory usage in bytes and releases the + * accounted memory from the circuit breaker when the reference count reaches zero. + */ public final class MemoryAccountingBytesRefCounted extends AbstractRefCounted { - private long bytes; + private int bytes; private final CircuitBreaker breaker; - private MemoryAccountingBytesRefCounted(long bytes, CircuitBreaker breaker) { + private MemoryAccountingBytesRefCounted(int bytes, CircuitBreaker breaker) { this.bytes = bytes; this.breaker = breaker; } public static MemoryAccountingBytesRefCounted create(CircuitBreaker breaker) { - return new MemoryAccountingBytesRefCounted(0L, breaker); + return new MemoryAccountingBytesRefCounted(0, breaker); } - public void account(long bytes, String label) { + public void account(int bytes, String label) { breaker.addEstimateBytesAndMaybeBreak(bytes, label); this.bytes += bytes; } @Override protected void closeInternal() { - if (bytes != 0L) { - breaker.addWithoutBreaking(-bytes); - } + breaker.addWithoutBreaking(-bytes); } } diff --git a/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java b/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java index 7397382866388..c09430e668a7f 100644 --- a/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java +++ b/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java @@ -535,6 +535,7 @@ public void apply(Settings value, Settings current, Settings previous) { SearchModule.SCRIPTED_METRICS_AGG_ALLOWED_STORED_SCRIPTS, SearchService.SEARCH_WORKER_THREADS_ENABLED, SearchService.QUERY_PHASE_PARALLEL_COLLECTION_ENABLED, + SearchService.MEMORY_ACCOUNTING_BUFFER_SIZE, ThreadPool.ESTIMATED_TIME_INTERVAL_SETTING, ThreadPool.LATE_TIME_INTERVAL_WARN_THRESHOLD_SETTING, ThreadPool.SLOW_SCHEDULER_TASK_WARN_THRESHOLD_SETTING, diff --git a/server/src/main/java/org/elasticsearch/search/SearchService.java b/server/src/main/java/org/elasticsearch/search/SearchService.java index 071312c25736c..74f3364259b27 100644 --- a/server/src/main/java/org/elasticsearch/search/SearchService.java +++ b/server/src/main/java/org/elasticsearch/search/SearchService.java @@ -274,6 +274,18 @@ public class SearchService extends AbstractLifecycleComponent implements IndexEv Property.NodeScope ); + /** + * The size of the buffer used for memory accounting. + * This buffer is used to locally track the memory accummulate during the executiong of + * a search request before submitting the accumulated value to the circuit breaker. + */ + public static final Setting MEMORY_ACCOUNTING_BUFFER_SIZE = Setting.byteSizeSetting( + "search.memory_accounting_buffer_size", + ByteSizeValue.of(32, ByteSizeUnit.KB), + Property.Dynamic, + Property.NodeScope + ); + public static final int DEFAULT_SIZE = 10; public static final int DEFAULT_FROM = 0; private static final StackTraceElement[] EMPTY_STACK_TRACE_ARRAY = new StackTraceElement[0]; @@ -311,6 +323,8 @@ public class SearchService extends AbstractLifecycleComponent implements IndexEv private volatile boolean enableRewriteAggsToFilterByFilter; + private volatile ByteSizeValue memoryAccountingBufferSize; + private final Cancellable keepAliveReaper; private final AtomicLong idGenerator = new AtomicLong(); @@ -388,6 +402,9 @@ public SearchService( enableQueryPhaseParallelCollection = QUERY_PHASE_PARALLEL_COLLECTION_ENABLED.get(settings); clusterService.getClusterSettings() .addSettingsUpdateConsumer(QUERY_PHASE_PARALLEL_COLLECTION_ENABLED, this::setEnableQueryPhaseParallelCollection); + + memoryAccountingBufferSize = MEMORY_ACCOUNTING_BUFFER_SIZE.get(settings); + clusterService.getClusterSettings().addSettingsUpdateConsumer(MEMORY_ACCOUNTING_BUFFER_SIZE, this::setMemoryAccountingBufferSize); } private void setEnableSearchWorkerThreads(boolean enableSearchWorkerThreads) { @@ -402,6 +419,10 @@ private void setEnableQueryPhaseParallelCollection(boolean enableQueryPhaseParal this.enableQueryPhaseParallelCollection = enableQueryPhaseParallelCollection; } + private void setMemoryAccountingBufferSize(ByteSizeValue memoryAccountingBufferSize) { + this.memoryAccountingBufferSize = memoryAccountingBufferSize; + } + private static void validateKeepAlives(TimeValue defaultKeepAlive, TimeValue maxKeepAlive) { if (defaultKeepAlive.millis() > maxKeepAlive.millis()) { throw new IllegalArgumentException( @@ -790,7 +811,7 @@ public void executeRankFeaturePhase(RankFeatureShardRequest request, SearchShard return searchContext.rankFeatureResult(); } RankFeatureShardPhase.prepareForFetch(searchContext, request); - fetchPhase.execute(searchContext, docIds, null, circuitBreaker); + fetchPhase.execute(searchContext, docIds, null, circuitBreaker, memoryAccountingBufferSize.getBytes()); RankFeatureShardPhase.processFetch(searchContext); var rankFeatureResult = searchContext.rankFeatureResult(); rankFeatureResult.incRef(); @@ -808,7 +829,7 @@ private QueryFetchSearchResult executeFetchPhase(ReaderContext reader, SearchCon Releasable scope = tracer.withScope(context.getTask()); SearchOperationListenerExecutor executor = new SearchOperationListenerExecutor(context, true, afterQueryTime) ) { - fetchPhase.execute(context, shortcutDocIdsToLoad(context), null, circuitBreaker); + fetchPhase.execute(context, shortcutDocIdsToLoad(context), null, circuitBreaker, memoryAccountingBufferSize.getBytes()); if (reader.singleSession()) { freeReaderContext(reader.id()); } @@ -974,7 +995,13 @@ public void executeFetchPhase(ShardFetchRequest request, CancellableTask task, A System.nanoTime() ) ) { - fetchPhase.execute(searchContext, request.docIds(), request.getRankDocks(), circuitBreaker); + fetchPhase.execute( + searchContext, + request.docIds(), + request.getRankDocks(), + circuitBreaker, + memoryAccountingBufferSize.getBytes() + ); if (readerContext.singleSession()) { freeReaderContext(request.contextId()); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/TopHitsAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/TopHitsAggregator.java index bc7043b652d4a..7cf2ee9f58916 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/TopHitsAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/TopHitsAggregator.java @@ -35,6 +35,7 @@ import org.elasticsearch.index.query.SearchExecutionContext; import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.SearchHits; +import org.elasticsearch.search.SearchService; import org.elasticsearch.search.aggregations.AggregationExecutionContext; import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.InternalAggregation; @@ -56,6 +57,8 @@ class TopHitsAggregator extends MetricsAggregator { + private final long memAccountingBufferSize; + private static class Collectors { public final TopDocsCollector topDocsCollector; public final MaxScoreCollector maxScoreCollector; @@ -87,6 +90,7 @@ private static class Collectors { this.subSearchContext = subSearchContext; this.topDocsCollectors = new LongObjectPagedHashMap<>(1, bigArrays); this.fetchProfiles = context.profiling() ? new ArrayList<>() : null; + this.memAccountingBufferSize = context.getClusterSettings().get(SearchService.MEMORY_ACCOUNTING_BUFFER_SIZE).getBytes(); } @Override @@ -197,7 +201,7 @@ public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOE for (int i = 0; i < topDocs.scoreDocs.length; i++) { docIdsToLoad[i] = topDocs.scoreDocs[i].doc; } - FetchSearchResult fetchResult = runFetchPhase(subSearchContext, docIdsToLoad, context.breaker()); + FetchSearchResult fetchResult = runFetchPhase(subSearchContext, docIdsToLoad, context.breaker(), memAccountingBufferSize); if (fetchProfiles != null) { fetchProfiles.add(fetchResult.profileResult()); } @@ -221,7 +225,12 @@ public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOE ); } - private static FetchSearchResult runFetchPhase(SubSearchContext subSearchContext, int[] docIdsToLoad, CircuitBreaker breaker) { + private static FetchSearchResult runFetchPhase( + SubSearchContext subSearchContext, + int[] docIdsToLoad, + CircuitBreaker breaker, + long memAccountingBufferSize + ) { // Fork the search execution context for each slice, because the fetch phase does not support concurrent execution yet. SearchExecutionContext searchExecutionContext = new SearchExecutionContext(subSearchContext.getSearchExecutionContext()); SubSearchContext fetchSubSearchContext = new SubSearchContext(subSearchContext) { @@ -230,7 +239,7 @@ public SearchExecutionContext getSearchExecutionContext() { return searchExecutionContext; } }; - fetchSubSearchContext.fetchPhase().execute(fetchSubSearchContext, docIdsToLoad, null, breaker); + fetchSubSearchContext.fetchPhase().execute(fetchSubSearchContext, docIdsToLoad, null, breaker, memAccountingBufferSize); return fetchSubSearchContext.fetchResult(); } diff --git a/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java b/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java index 38a40ec71f20e..d6d0088a2387f 100644 --- a/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java +++ b/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java @@ -16,8 +16,8 @@ import org.elasticsearch.common.MemoryAccountingBytesRefCounted; import org.elasticsearch.common.breaker.CircuitBreaker; import org.elasticsearch.common.breaker.CircuitBreakingException; -import org.elasticsearch.common.breaker.NoopCircuitBreaker; import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.core.RefCounted; import org.elasticsearch.index.fieldvisitor.LeafStoredFieldLoader; import org.elasticsearch.index.fieldvisitor.StoredFieldLoader; import org.elasticsearch.index.mapper.IdLoader; @@ -49,6 +49,7 @@ import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.function.Consumer; import java.util.function.Supplier; /** @@ -57,7 +58,6 @@ */ public final class FetchPhase { private static final Logger LOGGER = LogManager.getLogger(FetchPhase.class); - public static final NoopCircuitBreaker NOOP_CIRCUIT_BREAKER = new NoopCircuitBreaker("fetch_phase_nested_hits"); private final FetchSubPhase[] fetchSubPhases; @@ -66,7 +66,13 @@ public FetchPhase(List fetchSubPhases) { this.fetchSubPhases[fetchSubPhases.size()] = new InnerHitsPhase(this); } - public void execute(SearchContext context, int[] docIdsToLoad, RankDocShardInfo rankDocs, CircuitBreaker circuitBreaker) { + public void execute( + SearchContext context, + int[] docIdsToLoad, + RankDocShardInfo rankDocs, + CircuitBreaker circuitBreaker, + long memAccountingBufferSize + ) { if (LOGGER.isTraceEnabled()) { LOGGER.trace("{}", new SearchContextSourcePrinter(context)); } @@ -88,7 +94,7 @@ public void execute(SearchContext context, int[] docIdsToLoad, RankDocShardInfo : Profilers.startProfilingFetchPhase(); SearchHits hits = null; try { - hits = buildSearchHits(context, docIdsToLoad, profiler, rankDocs, circuitBreaker); + hits = buildSearchHits(context, docIdsToLoad, profiler, rankDocs, circuitBreaker, memAccountingBufferSize); } finally { // Always finish profiling ProfileResult profileResult = profiler.finish(); @@ -115,7 +121,8 @@ private SearchHits buildSearchHits( int[] docIdsToLoad, Profiler profiler, RankDocShardInfo rankDocs, - CircuitBreaker circuitBreaker + CircuitBreaker circuitBreaker, + long memAccountingBufferSize ) { FetchContext fetchContext = new FetchContext(context); @@ -152,11 +159,24 @@ private SearchHits buildSearchHits( LeafStoredFieldLoader leafStoredFieldLoader; SourceLoader.Leaf leafSourceLoader; IdLoader.Leaf leafIdLoader; + int accumulatedBytesInLeaf; + int docsInLeaf; + int processedDocs; + + private final Supplier getAndResetAccumulatedBytes = () -> { + int bytesToSubmit = this.accumulatedBytesInLeaf; + this.accumulatedBytesInLeaf = 0; + return bytesToSubmit; + }; + private final Consumer memoryUsageBytesAccumulator = (bytes) -> { this.accumulatedBytesInLeaf += bytes; }; @Override protected void setNextReader(LeafReaderContext ctx, int[] docsInLeaf) throws IOException { Timer timer = profiler.startNextReader(); this.ctx = ctx; + this.accumulatedBytesInLeaf = 0; + this.docsInLeaf = docsInLeaf.length; + this.processedDocs = 0; this.leafNestedDocuments = nestedDocuments.getLeafNestedDocuments(ctx); this.leafStoredFieldLoader = storedFieldLoader.getLoader(ctx, docsInLeaf); this.leafSourceLoader = sourceLoader.leaf(ctx.reader(), docsInLeaf); @@ -175,6 +195,12 @@ protected SearchHit nextDoc(int doc) throws IOException { if (context.isCancelled()) { throw new TaskCancelledException("cancelled"); } + ++processedDocs; + // indicates if we should submit the accounted memory to the circuit breaker. + // we do so whenever one of the following is true: + // 1. we have accumulated at least the size of the memory accounting buffer + // 2. we have reached the last document in the leaf + boolean enoughBytesOrLastDocInLeaf = (accumulatedBytesInLeaf >= memAccountingBufferSize) || (processedDocs == docsInLeaf); HitContext hit = prepareHitContext( context, requiresSource, @@ -186,7 +212,10 @@ protected SearchHit nextDoc(int doc) throws IOException { leafSourceLoader, leafIdLoader, rankDocs == null ? null : rankDocs.get(doc), - circuitBreaker + circuitBreaker, + enoughBytesOrLastDocInLeaf, + memoryUsageBytesAccumulator, + getAndResetAccumulatedBytes ); boolean success = false; try { @@ -251,7 +280,10 @@ private HitContext prepareHitContext( SourceLoader.Leaf sourceLoader, IdLoader.Leaf idLoader, RankDoc rankDoc, - CircuitBreaker circuitBreaker + CircuitBreaker circuitBreaker, + boolean submitToCb, + Consumer memoryUsageBytesAccumulator, + Supplier accumulatedBytesInLeaf ) throws IOException { if (nestedDocuments.advance(docId - subReaderContext.docBase) == null) { return prepareNonNestedHitContext( @@ -263,7 +295,10 @@ private HitContext prepareHitContext( sourceLoader, idLoader, rankDoc, - circuitBreaker + circuitBreaker, + submitToCb, + memoryUsageBytesAccumulator, + accumulatedBytesInLeaf ); } else { return prepareNestedHitContext( @@ -295,28 +330,48 @@ private static HitContext prepareNonNestedHitContext( SourceLoader.Leaf sourceLoader, IdLoader.Leaf idLoader, RankDoc rankDoc, - CircuitBreaker circuitBreaker + CircuitBreaker circuitBreaker, + boolean submitToCB, + Consumer memoryUsageBytesAccumulator, + Supplier accumulatedBytesInLeaf ) throws IOException { int subDocId = docId - subReaderContext.docBase; leafStoredFieldLoader.advanceTo(subDocId); + MemoryAccountingBytesRefCounted memAccountingRefCounted = null; + RefCounted refCountedHit = null; + if (submitToCB) { + memAccountingRefCounted = MemoryAccountingBytesRefCounted.create(circuitBreaker); + refCountedHit = LeakTracker.wrap(memAccountingRefCounted); + } String id = idLoader.getId(subDocId); if (id == null) { - MemoryAccountingBytesRefCounted memAccountingRefCounted = MemoryAccountingBytesRefCounted.create(circuitBreaker); - SearchHit hit = new SearchHit(docId, null, null, LeakTracker.wrap(memAccountingRefCounted)); + SearchHit hit = new SearchHit(docId, null, null, refCountedHit); // TODO: can we use real pooled buffers here as well? - Source source = Source.lazy(lazyStoredSourceLoader(profiler, subReaderContext, subDocId, memAccountingRefCounted)); + Source source = Source.lazy( + lazyStoredSourceLoader( + profiler, + subReaderContext, + subDocId, + memAccountingRefCounted, + submitToCB, + memoryUsageBytesAccumulator, + accumulatedBytesInLeaf + ) + ); return new HitContext(hit, subReaderContext, subDocId, Map.of(), source, rankDoc); } else { - MemoryAccountingBytesRefCounted memAccountingRefCounted = MemoryAccountingBytesRefCounted.create(circuitBreaker); - SearchHit hit = new SearchHit(docId, id, null, LeakTracker.wrap(memAccountingRefCounted)); + SearchHit hit = new SearchHit(docId, id, null, refCountedHit); Source source; if (requiresSource) { Timer timer = profiler.startLoadingSource(); try { source = sourceLoader.source(leafStoredFieldLoader, subDocId); - memAccountingRefCounted.account(source.internalSourceRef().length(), "fetch phase source loader"); + memoryUsageBytesAccumulator.accept(source.internalSourceRef().length()); + if (submitToCB) { + memAccountingRefCounted.account(accumulatedBytesInLeaf.get(), "fetch phase source loader"); + } } catch (CircuitBreakingException e) { hit.decRef(); throw e; @@ -326,7 +381,17 @@ private static HitContext prepareNonNestedHitContext( } } } else { - source = Source.lazy(lazyStoredSourceLoader(profiler, subReaderContext, subDocId, memAccountingRefCounted)); + source = Source.lazy( + lazyStoredSourceLoader( + profiler, + subReaderContext, + subDocId, + memAccountingRefCounted, + submitToCB, + memoryUsageBytesAccumulator, + accumulatedBytesInLeaf + ) + ); } return new HitContext(hit, subReaderContext, subDocId, leafStoredFieldLoader.storedFields(), source, rankDoc); } @@ -336,7 +401,10 @@ private static Supplier lazyStoredSourceLoader( Profiler profiler, LeafReaderContext ctx, int doc, - MemoryAccountingBytesRefCounted memAccountingRefCounted + MemoryAccountingBytesRefCounted memAccountingRefCounted, + boolean submitToCB, + Consumer memoryUsageAccumulator, + Supplier accumulatedBytesInLeaf ) { return () -> { StoredFieldLoader rootLoader = profiler.storedFields(StoredFieldLoader.create(true, Collections.emptySet())); @@ -344,7 +412,10 @@ private static Supplier lazyStoredSourceLoader( LeafStoredFieldLoader leafRootLoader = rootLoader.getLoader(ctx, null); leafRootLoader.advanceTo(doc); BytesReference source = leafRootLoader.source(); - memAccountingRefCounted.account(source.length(), "fetch phase source loader"); + memoryUsageAccumulator.accept(source.length()); + if (submitToCB) { + memAccountingRefCounted.account(accumulatedBytesInLeaf.get(), "lazy fetch phase source loader"); + } return Source.fromBytes(source); } catch (IOException e) { throw new UncheckedIOException(e); diff --git a/server/src/main/java/org/elasticsearch/search/fetch/subphase/InnerHitsPhase.java b/server/src/main/java/org/elasticsearch/search/fetch/subphase/InnerHitsPhase.java index 556f5f439d296..3212cbb86bab7 100644 --- a/server/src/main/java/org/elasticsearch/search/fetch/subphase/InnerHitsPhase.java +++ b/server/src/main/java/org/elasticsearch/search/fetch/subphase/InnerHitsPhase.java @@ -98,7 +98,8 @@ private void hitExecute(Map innerHi innerHitsContext.setRootId(hit.getId()); innerHitsContext.setRootLookup(rootSource); - fetchPhase.execute(innerHitsContext, docIdsToLoad, null, NOOP_CIRCUIT_BREAKER); + // running the inner hits fetch phase without memory accounting as the source size is already accounted for in the outer hits + fetchPhase.execute(innerHitsContext, docIdsToLoad, null, NOOP_CIRCUIT_BREAKER, 0L); FetchSearchResult fetchResult = innerHitsContext.fetchResult(); SearchHit[] internalHits = fetchResult.fetchResult().hits().getHits(); for (int j = 0; j < internalHits.length; j++) { diff --git a/server/src/test/java/org/elasticsearch/action/search/FetchSearchPhaseTests.java b/server/src/test/java/org/elasticsearch/action/search/FetchSearchPhaseTests.java index 3283cc9c063ef..5a563b11fc4f0 100644 --- a/server/src/test/java/org/elasticsearch/action/search/FetchSearchPhaseTests.java +++ b/server/src/test/java/org/elasticsearch/action/search/FetchSearchPhaseTests.java @@ -791,7 +791,7 @@ public void testFetchTimeoutWithPartialResults() throws IOException { ContextIndexSearcher contextIndexSearcher = createSearcher(r); try (SearchContext searchContext = createSearchContext(contextIndexSearcher, true)) { FetchPhase fetchPhase = createFetchPhase(contextIndexSearcher); - fetchPhase.execute(searchContext, new int[] { 0, 1, 2 }, null, new NoopCircuitBreaker(CircuitBreaker.REQUEST)); + fetchPhase.execute(searchContext, new int[] { 0, 1, 2 }, null, new NoopCircuitBreaker(CircuitBreaker.REQUEST), 0L); assertTrue(searchContext.queryResult().searchTimedOut()); assertEquals(1, searchContext.fetchResult().hits().getHits().length); } finally { @@ -814,7 +814,7 @@ public void testFetchTimeoutNoPartialResults() throws IOException { FetchPhase fetchPhase = createFetchPhase(contextIndexSearcher); expectThrows( SearchTimeoutException.class, - () -> fetchPhase.execute(searchContext, new int[] { 0, 1, 2 }, null, new NoopCircuitBreaker(CircuitBreaker.REQUEST)) + () -> fetchPhase.execute(searchContext, new int[] { 0, 1, 2 }, null, new NoopCircuitBreaker(CircuitBreaker.REQUEST), 0L) ); assertNull(searchContext.fetchResult().hits()); } finally { diff --git a/server/src/test/java/org/elasticsearch/common/MemoryAccountingBytesRefCountedTests.java b/server/src/test/java/org/elasticsearch/common/MemoryAccountingBytesRefCountedTests.java new file mode 100644 index 0000000000000..a808df0b2b82a --- /dev/null +++ b/server/src/test/java/org/elasticsearch/common/MemoryAccountingBytesRefCountedTests.java @@ -0,0 +1,50 @@ +/* + * 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", the "GNU Affero General Public License v3.0 only", and the "Server Side + * Public License v 1"; you may not use this file except in compliance with, at + * your election, the "Elastic License 2.0", the "GNU Affero General Public + * License v3.0 only", or the "Server Side Public License, v 1". + */ + +package org.elasticsearch.common; + +import org.elasticsearch.common.breaker.CircuitBreaker; +import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.common.util.MockBigArrays; +import org.elasticsearch.test.ESTestCase; + +public class MemoryAccountingBytesRefCountedTests extends ESTestCase { + + public void testNoMemoryAccounted() { + CircuitBreaker breaker = new MockBigArrays.LimitedBreaker("test", ByteSizeValue.ofGb(1)); + MemoryAccountingBytesRefCounted refCounted = MemoryAccountingBytesRefCounted.create(breaker); + refCounted.decRef(); + assertEquals(0, breaker.getUsed()); + } + + public void testMemoryAccounted() { + CircuitBreaker breaker = new MockBigArrays.LimitedBreaker("test", ByteSizeValue.ofGb(1)); + MemoryAccountingBytesRefCounted refCounted = MemoryAccountingBytesRefCounted.create(breaker); + refCounted.account(10, "test"); + assertEquals(10, breaker.getUsed()); + } + + public void testCloseInternalDecrementsBreaker() { + CircuitBreaker breaker = new MockBigArrays.LimitedBreaker("test", ByteSizeValue.ofGb(1)); + MemoryAccountingBytesRefCounted refCounted = MemoryAccountingBytesRefCounted.create(breaker); + refCounted.account(10, "test"); + refCounted.decRef(); + assertEquals(0, breaker.getUsed()); + } + + public void testBreakerNotDecrementedIfRefsRemaining() { + CircuitBreaker breaker = new MockBigArrays.LimitedBreaker("test", ByteSizeValue.ofGb(1)); + MemoryAccountingBytesRefCounted refCounted = MemoryAccountingBytesRefCounted.create(breaker); + refCounted.account(10, "test"); + refCounted.incRef(); // 2 refs + assertEquals(10, breaker.getUsed()); + refCounted.decRef(); // 1 ref remaining so no decrementing is executed + assertEquals(10, breaker.getUsed()); + } +} diff --git a/server/src/test/java/org/elasticsearch/search/SearchServiceSingleNodeTests.java b/server/src/test/java/org/elasticsearch/search/SearchServiceSingleNodeTests.java index fe602d2854c8c..c0487c6e42f67 100644 --- a/server/src/test/java/org/elasticsearch/search/SearchServiceSingleNodeTests.java +++ b/server/src/test/java/org/elasticsearch/search/SearchServiceSingleNodeTests.java @@ -49,6 +49,7 @@ import org.elasticsearch.cluster.routing.TestShardRouting; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.UUIDs; +import org.elasticsearch.common.breaker.CircuitBreaker; import org.elasticsearch.common.breaker.CircuitBreakingException; import org.elasticsearch.common.breaker.NoopCircuitBreaker; import org.elasticsearch.common.bytes.BytesArray; @@ -76,6 +77,7 @@ import org.elasticsearch.index.shard.SearchOperationListener; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.indices.IndicesService; +import org.elasticsearch.indices.breaker.CircuitBreakerService; import org.elasticsearch.indices.settings.InternalOrPrivateSettingsPlugin; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.plugins.SearchPlugin; @@ -155,6 +157,8 @@ import java.util.function.Function; import java.util.function.IntConsumer; import java.util.function.Supplier; +import java.util.stream.Collectors; +import java.util.stream.IntStream; import static java.util.Collections.emptyList; import static java.util.Collections.emptyMap; @@ -175,6 +179,7 @@ import static org.hamcrest.CoreMatchers.notNullValue; import static org.hamcrest.CoreMatchers.startsWith; import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.not; import static org.mockito.Mockito.mock; @@ -2927,6 +2932,68 @@ public void testSlicingBehaviourForParallelCollection() throws Exception { } } + public void testFetchPhaseAccountsForSourceMemoryUsage() throws Exception { + createIndex("index"); + for (int i = 0; i < 10; i++) { + prepareIndex("index").setId(String.valueOf(i)).setSource("field", randomAlphaOfLength(1000)).setRefreshPolicy(IMMEDIATE).get(); + } + + SearchService service = getInstanceFromNode(SearchService.class); + IndicesService indicesService = getInstanceFromNode(IndicesService.class); + IndexService indexService = indicesService.indexServiceSafe(resolveIndex("index")); + IndexShard indexShard = indexService.getShard(0); + CircuitBreakerService circuitBreakerService = getInstanceFromNode(CircuitBreakerService.class); + CircuitBreaker breaker = circuitBreakerService.getBreaker(CircuitBreaker.REQUEST); + + SearchRequest scrollSearchRequest = new SearchRequest().allowPartialSearchResults(true).scroll(TimeValue.timeValueMinutes(1)); + FetchSearchResult fetchSearchResult = null; + ReaderContext readerContext = null; + try { + readerContext = service.createAndPutReaderContext( + new ShardSearchRequest( + OriginalIndices.NONE, + scrollSearchRequest, + indexShard.shardId(), + 0, + 1, + AliasFilter.EMPTY, + 1.0f, + -1, + null + ), + indexService, + indexShard, + indexShard.acquireSearcherSupplier(), + 0 + ); + ShardFetchRequest req = new ShardFetchRequest( + readerContext.id(), + IntStream.range(0, 10).boxed().collect(Collectors.toList()), + null + ); + PlainActionFuture listener = new PlainActionFuture<>(); + long usedBeforeFetch = breaker.getUsed(); + service.executeFetchPhase(req, new SearchShardTask(123L, "", "", "", null, emptyMap()), listener.delegateFailure((l, r) -> { + r.incRef(); + l.onResponse(r); + })); + fetchSearchResult = listener.get(); + long usedAfterFetch = breaker.getUsed(); + assertThat(usedAfterFetch, greaterThan(usedBeforeFetch)); + logger.debug("--> usedBeforeFetch: [{}], usedAfterFetch: [{}]", usedBeforeFetch, usedAfterFetch); + // 10 docs with at least 1000 bytes in the source + assertThat((usedAfterFetch - usedBeforeFetch), greaterThan(10_000L)); + } finally { + if (fetchSearchResult != null) { + fetchSearchResult.decRef(); + assertThat(breaker.getUsed(), is(0L)); + } + if (readerContext != null) { + service.freeReaderContext(readerContext.id()); + } + } + } + private static ReaderContext createReaderContext(IndexService indexService, IndexShard indexShard) { return new ReaderContext( new ShardSearchContextId(UUIDs.randomBase64UUID(), randomNonNegativeLong()), From 558d4af14b3e3c3e5a8d4dfee5d6055f9c2954e4 Mon Sep 17 00:00:00 2001 From: Andrei Dan Date: Tue, 18 Feb 2025 09:09:20 +0000 Subject: [PATCH 23/56] Cosmetics and enhance test --- .../search/fetch/FetchPhase.java | 12 ++++++------ .../search/SearchServiceSingleNodeTests.java | 19 ++++++++++++++----- 2 files changed, 20 insertions(+), 11 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java b/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java index d6d0088a2387f..df6c4a9f0364c 100644 --- a/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java +++ b/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java @@ -168,7 +168,7 @@ private SearchHits buildSearchHits( this.accumulatedBytesInLeaf = 0; return bytesToSubmit; }; - private final Consumer memoryUsageBytesAccumulator = (bytes) -> { this.accumulatedBytesInLeaf += bytes; }; + private final Consumer memoryUsageBytesAccumulator = (bytes) -> this.accumulatedBytesInLeaf += bytes; @Override protected void setNextReader(LeafReaderContext ctx, int[] docsInLeaf) throws IOException { @@ -331,7 +331,7 @@ private static HitContext prepareNonNestedHitContext( IdLoader.Leaf idLoader, RankDoc rankDoc, CircuitBreaker circuitBreaker, - boolean submitToCB, + boolean accountMemoryWithCircuitBreaker, Consumer memoryUsageBytesAccumulator, Supplier accumulatedBytesInLeaf ) throws IOException { @@ -341,7 +341,7 @@ private static HitContext prepareNonNestedHitContext( MemoryAccountingBytesRefCounted memAccountingRefCounted = null; RefCounted refCountedHit = null; - if (submitToCB) { + if (accountMemoryWithCircuitBreaker) { memAccountingRefCounted = MemoryAccountingBytesRefCounted.create(circuitBreaker); refCountedHit = LeakTracker.wrap(memAccountingRefCounted); } @@ -355,7 +355,7 @@ private static HitContext prepareNonNestedHitContext( subReaderContext, subDocId, memAccountingRefCounted, - submitToCB, + accountMemoryWithCircuitBreaker, memoryUsageBytesAccumulator, accumulatedBytesInLeaf ) @@ -369,7 +369,7 @@ private static HitContext prepareNonNestedHitContext( try { source = sourceLoader.source(leafStoredFieldLoader, subDocId); memoryUsageBytesAccumulator.accept(source.internalSourceRef().length()); - if (submitToCB) { + if (accountMemoryWithCircuitBreaker) { memAccountingRefCounted.account(accumulatedBytesInLeaf.get(), "fetch phase source loader"); } } catch (CircuitBreakingException e) { @@ -387,7 +387,7 @@ private static HitContext prepareNonNestedHitContext( subReaderContext, subDocId, memAccountingRefCounted, - submitToCB, + accountMemoryWithCircuitBreaker, memoryUsageBytesAccumulator, accumulatedBytesInLeaf ) diff --git a/server/src/test/java/org/elasticsearch/search/SearchServiceSingleNodeTests.java b/server/src/test/java/org/elasticsearch/search/SearchServiceSingleNodeTests.java index 4ce55778f14c9..22b9565b4a668 100644 --- a/server/src/test/java/org/elasticsearch/search/SearchServiceSingleNodeTests.java +++ b/server/src/test/java/org/elasticsearch/search/SearchServiceSingleNodeTests.java @@ -180,6 +180,7 @@ import static org.hamcrest.CoreMatchers.startsWith; import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.not; import static org.mockito.Mockito.mock; @@ -2934,9 +2935,14 @@ public void testSlicingBehaviourForParallelCollection() throws Exception { public void testFetchPhaseAccountsForSourceMemoryUsage() throws Exception { createIndex("index"); - for (int i = 0; i < 10; i++) { + for (int i = 0; i < 48; i++) { prepareIndex("index").setId(String.valueOf(i)).setSource("field", randomAlphaOfLength(1000)).setRefreshPolicy(IMMEDIATE).get(); } + if (randomBoolean()) { + // 1 segment test is also useful so we enable the batching branch of the memory accounting + // we do local accounting up to 32kb (by default) before submitting to cb + indicesAdmin().prepareForceMerge("index").setMaxNumSegments(1).get(); + } SearchService service = getInstanceFromNode(SearchService.class); IndicesService indicesService = getInstanceFromNode(IndicesService.class); @@ -2968,7 +2974,7 @@ public void testFetchPhaseAccountsForSourceMemoryUsage() throws Exception { ); ShardFetchRequest req = new ShardFetchRequest( readerContext.id(), - IntStream.range(0, 10).boxed().collect(Collectors.toList()), + IntStream.range(0, 48).boxed().collect(Collectors.toList()), null ); PlainActionFuture listener = new PlainActionFuture<>(); @@ -2981,12 +2987,15 @@ public void testFetchPhaseAccountsForSourceMemoryUsage() throws Exception { long usedAfterFetch = breaker.getUsed(); assertThat(usedAfterFetch, greaterThan(usedBeforeFetch)); logger.debug("--> usedBeforeFetch: [{}], usedAfterFetch: [{}]", usedBeforeFetch, usedAfterFetch); - // 10 docs with at least 1000 bytes in the source - assertThat((usedAfterFetch - usedBeforeFetch), greaterThan(10_000L)); + // 48 docs with at least 1000 bytes in the source + assertThat((usedAfterFetch - usedBeforeFetch), greaterThanOrEqualTo(48_000L)); } finally { if (fetchSearchResult != null) { + long usedBeforeDecref = breaker.getUsed(); + assertThat(usedBeforeDecref, greaterThanOrEqualTo(48_000L)); fetchSearchResult.decRef(); - assertThat(breaker.getUsed(), is(0L)); + // when releasing the result references we should clear at least 48_000 bytes (48 hits with sources of at least 1000 bytes) + assertThat(usedBeforeDecref - breaker.getUsed(), greaterThanOrEqualTo(48_000L)); } if (readerContext != null) { service.freeReaderContext(readerContext.id()); From b793459261cf546d9b4f50a6c5f0d708222bca8f Mon Sep 17 00:00:00 2001 From: Andrei Dan Date: Tue, 18 Feb 2025 09:11:10 +0000 Subject: [PATCH 24/56] Revert enrich modifications --- .../action/EnrichShardMultiSearchAction.java | 21 ++----------------- 1 file changed, 2 insertions(+), 19 deletions(-) diff --git a/x-pack/plugin/enrich/src/main/java/org/elasticsearch/xpack/enrich/action/EnrichShardMultiSearchAction.java b/x-pack/plugin/enrich/src/main/java/org/elasticsearch/xpack/enrich/action/EnrichShardMultiSearchAction.java index dabe85718ff15..0b25030dd9998 100644 --- a/x-pack/plugin/enrich/src/main/java/org/elasticsearch/xpack/enrich/action/EnrichShardMultiSearchAction.java +++ b/x-pack/plugin/enrich/src/main/java/org/elasticsearch/xpack/enrich/action/EnrichShardMultiSearchAction.java @@ -29,10 +29,8 @@ import org.elasticsearch.cluster.routing.Preference; import org.elasticsearch.cluster.routing.ShardsIterator; import org.elasticsearch.cluster.service.ClusterService; -import org.elasticsearch.common.MemoryAccountingBytesRefCounted; import org.elasticsearch.common.Strings; import org.elasticsearch.common.breaker.CircuitBreaker; -import org.elasticsearch.common.breaker.CircuitBreakingException; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.NamedWriteableAwareStreamInput; @@ -57,7 +55,6 @@ import org.elasticsearch.search.builder.SearchSourceBuilder; import org.elasticsearch.search.fetch.subphase.FetchSourceContext; import org.elasticsearch.threadpool.ThreadPool; -import org.elasticsearch.transport.LeakTracker; import org.elasticsearch.transport.TransportService; import org.elasticsearch.xcontent.DeprecationHandler; import org.elasticsearch.xcontent.NamedXContentRegistry; @@ -264,22 +261,8 @@ protected MultiSearchResponse shardOperation(Request request, ShardId shardId) t } return context.getFieldType(field); }); - MemoryAccountingBytesRefCounted memAccountingRefCounted = MemoryAccountingBytesRefCounted.create(breaker); - final SearchHit hit = new SearchHit(scoreDoc.doc, visitor.id(), null, LeakTracker.wrap(memAccountingRefCounted)); - try { - BytesReference sourceBytesRef = visitor.source(); - memAccountingRefCounted.account(sourceBytesRef.length(), "enrich source"); - hit.sourceRef(filterSource(fetchSourceContext, sourceBytesRef)); - hits[j] = hit; - } catch (CircuitBreakingException e) { - hit.decRef(); - for (SearchHit searchHit : hits) { - if (searchHit != null) { - searchHit.decRef(); - } - } - throw e; - } + final SearchHit hit = new SearchHit(scoreDoc.doc, visitor.id()); + hit.sourceRef(filterSource(fetchSourceContext, visitor.source())); } items[i] = new MultiSearchResponse.Item(createSearchResponse(topDocs, hits), null); } From 1062778d93d0fa4609d8105a20cf4fe41993dee1 Mon Sep 17 00:00:00 2001 From: Andrei Dan Date: Tue, 18 Feb 2025 09:12:24 +0000 Subject: [PATCH 25/56] Revert enrich stuff --- .../xpack/enrich/action/EnrichShardMultiSearchAction.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/x-pack/plugin/enrich/src/main/java/org/elasticsearch/xpack/enrich/action/EnrichShardMultiSearchAction.java b/x-pack/plugin/enrich/src/main/java/org/elasticsearch/xpack/enrich/action/EnrichShardMultiSearchAction.java index 0b25030dd9998..a7167e176cbef 100644 --- a/x-pack/plugin/enrich/src/main/java/org/elasticsearch/xpack/enrich/action/EnrichShardMultiSearchAction.java +++ b/x-pack/plugin/enrich/src/main/java/org/elasticsearch/xpack/enrich/action/EnrichShardMultiSearchAction.java @@ -30,7 +30,6 @@ import org.elasticsearch.cluster.routing.ShardsIterator; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.Strings; -import org.elasticsearch.common.breaker.CircuitBreaker; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.NamedWriteableAwareStreamInput; @@ -222,7 +221,6 @@ protected ShardsIterator shards(ClusterState state, InternalRequest request) { protected MultiSearchResponse shardOperation(Request request, ShardId shardId) throws IOException { final IndexService indexService = indicesService.indexService(shardId.getIndex()); final IndexShard indexShard = indicesService.getShardOrNull(shardId); - final CircuitBreaker breaker = indicesService.getCircuitBreakerService().getBreaker(CircuitBreaker.REQUEST); try (Engine.Searcher searcher = indexShard.acquireSearcher("enrich_msearch")) { final FieldsVisitor visitor = new FieldsVisitor(true); /* @@ -263,6 +261,7 @@ protected MultiSearchResponse shardOperation(Request request, ShardId shardId) t }); final SearchHit hit = new SearchHit(scoreDoc.doc, visitor.id()); hit.sourceRef(filterSource(fetchSourceContext, visitor.source())); + hits[j] = hit; } items[i] = new MultiSearchResponse.Item(createSearchResponse(topDocs, hits), null); } From fbc4bb7ce5fd55be16f3d9dc49f9d2733101189e Mon Sep 17 00:00:00 2001 From: Andrei Dan Date: Tue, 18 Feb 2025 09:52:46 +0000 Subject: [PATCH 26/56] Release things before asserting in finally block --- .../search/SearchServiceSingleNodeTests.java | 16 +++++++++------- 1 file changed, 9 insertions(+), 7 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/search/SearchServiceSingleNodeTests.java b/server/src/test/java/org/elasticsearch/search/SearchServiceSingleNodeTests.java index 22b9565b4a668..d59e08702e57b 100644 --- a/server/src/test/java/org/elasticsearch/search/SearchServiceSingleNodeTests.java +++ b/server/src/test/java/org/elasticsearch/search/SearchServiceSingleNodeTests.java @@ -2986,19 +2986,21 @@ public void testFetchPhaseAccountsForSourceMemoryUsage() throws Exception { fetchSearchResult = listener.get(); long usedAfterFetch = breaker.getUsed(); assertThat(usedAfterFetch, greaterThan(usedBeforeFetch)); - logger.debug("--> usedBeforeFetch: [{}], usedAfterFetch: [{}]", usedBeforeFetch, usedAfterFetch); + logger.info("--> usedBeforeFetch: [{}], usedAfterFetch: [{}]", usedBeforeFetch, usedAfterFetch); // 48 docs with at least 1000 bytes in the source assertThat((usedAfterFetch - usedBeforeFetch), greaterThanOrEqualTo(48_000L)); } finally { + if (readerContext != null) { + service.freeReaderContext(readerContext.id()); + } if (fetchSearchResult != null) { - long usedBeforeDecref = breaker.getUsed(); - assertThat(usedBeforeDecref, greaterThanOrEqualTo(48_000L)); + long usedBeforeResultDecRef = breaker.getUsed(); fetchSearchResult.decRef(); + assertThat(usedBeforeResultDecRef, greaterThanOrEqualTo(48_000L)); // when releasing the result references we should clear at least 48_000 bytes (48 hits with sources of at least 1000 bytes) - assertThat(usedBeforeDecref - breaker.getUsed(), greaterThanOrEqualTo(48_000L)); - } - if (readerContext != null) { - service.freeReaderContext(readerContext.id()); + long usedAfterResultDecRef = breaker.getUsed(); + logger.info("--> usedBeforeResultDecRef: [{}], usedAfterResultDecRef: [{}]", usedBeforeResultDecRef, usedAfterResultDecRef); + assertThat(usedBeforeResultDecRef - usedAfterResultDecRef, greaterThanOrEqualTo(48_000L)); } } } From c89d1033609c7a017d94fe2c0866ac2027e34894 Mon Sep 17 00:00:00 2001 From: Andrei Dan Date: Tue, 18 Feb 2025 10:28:42 +0000 Subject: [PATCH 27/56] Remove unused field --- .../elasticsearch/action/search/ExpandSearchPhaseTests.java | 3 --- 1 file changed, 3 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/action/search/ExpandSearchPhaseTests.java b/server/src/test/java/org/elasticsearch/action/search/ExpandSearchPhaseTests.java index 3f8a12f9a8ef6..5f84739599514 100644 --- a/server/src/test/java/org/elasticsearch/action/search/ExpandSearchPhaseTests.java +++ b/server/src/test/java/org/elasticsearch/action/search/ExpandSearchPhaseTests.java @@ -12,8 +12,6 @@ import org.apache.lucene.search.TotalHits; import org.elasticsearch.action.ActionListener; import org.elasticsearch.common.Strings; -import org.elasticsearch.common.breaker.CircuitBreaker; -import org.elasticsearch.common.breaker.NoopCircuitBreaker; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.document.DocumentField; import org.elasticsearch.common.util.concurrent.AtomicArray; @@ -239,7 +237,6 @@ void sendExecuteMultiSearch(MultiSearchRequest request, SearchTask task, ActionL } }; - CircuitBreaker breaker = new NoopCircuitBreaker(CircuitBreaker.REQUEST); SearchHit hit1 = new SearchHit(1, "ID"); hit1.setDocumentField("someField", new DocumentField("someField", Collections.singletonList(null))); SearchHit hit2 = new SearchHit(2, "ID2"); From 35e8dceddd8d7d92d55317929063c8d41874346c Mon Sep 17 00:00:00 2001 From: Andrei Dan Date: Tue, 18 Feb 2025 10:31:30 +0000 Subject: [PATCH 28/56] Remove needless changes --- .../java/org/elasticsearch/search/fetch/FetchPhase.java | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java b/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java index df6c4a9f0364c..48d25b24deaa5 100644 --- a/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java +++ b/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java @@ -458,9 +458,8 @@ private static HitContext prepareNestedHitContext( rootId = leafRootLoader.id(); if (requiresSource) { - BytesReference source = leafRootLoader.source(); - if (source != null) { - rootSource = Source.fromBytes(source); + if (leafRootLoader.source() != null) { + rootSource = Source.fromBytes(leafRootLoader.source()); } } } @@ -471,8 +470,6 @@ private static HitContext prepareNestedHitContext( assert nestedIdentity != null; Source nestedSource = nestedIdentity.extractSource(rootSource); - // nested hits do not record their source size, as the top level hit will do so (nested hits will only reference part of the top - // level source) SearchHit nestedHit = new SearchHit(topDocId, rootId, nestedIdentity); return new HitContext(nestedHit, subReaderContext, nestedInfo.doc(), childFieldLoader.storedFields(), nestedSource, rankDoc); } From d926d33d8d85f87d3c98348b385521e43a2f5264 Mon Sep 17 00:00:00 2001 From: Andrei Dan Date: Tue, 18 Feb 2025 11:07:07 +0000 Subject: [PATCH 29/56] Update docs/changelog/121920.yaml --- docs/changelog/121920.yaml | 6 ++++++ 1 file changed, 6 insertions(+) create mode 100644 docs/changelog/121920.yaml diff --git a/docs/changelog/121920.yaml b/docs/changelog/121920.yaml new file mode 100644 index 0000000000000..6f0f4462d52ae --- /dev/null +++ b/docs/changelog/121920.yaml @@ -0,0 +1,6 @@ +pr: 121920 +summary: Account for the `SearchHit` source in circuit breaker +area: Search +type: enhancement +issues: + - 89656 From 3c57ced3792674e716b823ee4f3ab92255ddc647 Mon Sep 17 00:00:00 2001 From: Andrei Dan Date: Tue, 18 Feb 2025 11:41:50 +0000 Subject: [PATCH 30/56] Assert only if last ref in finally block --- .../search/SearchServiceSingleNodeTests.java | 18 ++++++++++++------ 1 file changed, 12 insertions(+), 6 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/search/SearchServiceSingleNodeTests.java b/server/src/test/java/org/elasticsearch/search/SearchServiceSingleNodeTests.java index d59e08702e57b..969408a37caa8 100644 --- a/server/src/test/java/org/elasticsearch/search/SearchServiceSingleNodeTests.java +++ b/server/src/test/java/org/elasticsearch/search/SearchServiceSingleNodeTests.java @@ -2995,12 +2995,18 @@ public void testFetchPhaseAccountsForSourceMemoryUsage() throws Exception { } if (fetchSearchResult != null) { long usedBeforeResultDecRef = breaker.getUsed(); - fetchSearchResult.decRef(); - assertThat(usedBeforeResultDecRef, greaterThanOrEqualTo(48_000L)); - // when releasing the result references we should clear at least 48_000 bytes (48 hits with sources of at least 1000 bytes) - long usedAfterResultDecRef = breaker.getUsed(); - logger.info("--> usedBeforeResultDecRef: [{}], usedAfterResultDecRef: [{}]", usedBeforeResultDecRef, usedAfterResultDecRef); - assertThat(usedBeforeResultDecRef - usedAfterResultDecRef, greaterThanOrEqualTo(48_000L)); + if (fetchSearchResult.decRef()) { + assertThat(usedBeforeResultDecRef, greaterThanOrEqualTo(48_000L)); + // when releasing the result references we should clear at least 48_000 bytes (48 hits with sources of at least 1000 + // bytes) + long usedAfterResultDecRef = breaker.getUsed(); + logger.info( + "--> usedBeforeResultDecRef: [{}], usedAfterResultDecRef: [{}]", + usedBeforeResultDecRef, + usedAfterResultDecRef + ); + assertThat(usedBeforeResultDecRef - usedAfterResultDecRef, greaterThanOrEqualTo(48_000L)); + } } } } From d3d2205dae72207b5902314b5adae417a41e4575 Mon Sep 17 00:00:00 2001 From: Andrei Dan Date: Tue, 18 Feb 2025 13:48:54 +0000 Subject: [PATCH 31/56] Update assert to account for fetch phase tripping the cb --- .../action/search/TransportSearchIT.java | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/server/src/internalClusterTest/java/org/elasticsearch/action/search/TransportSearchIT.java b/server/src/internalClusterTest/java/org/elasticsearch/action/search/TransportSearchIT.java index 5db2651c703d2..a396d61fdf576 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/action/search/TransportSearchIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/action/search/TransportSearchIT.java @@ -78,6 +78,7 @@ import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertResponse; import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.either; import static org.hamcrest.Matchers.equalTo; public class TransportSearchIT extends ESIntegTestCase { @@ -487,7 +488,10 @@ public void onFailure(Exception e) { Exception.class, client.prepareSearch("test").addAggregation(new TestAggregationBuilder("test")) ); - assertThat(exc.getCause().getMessage(), containsString("")); + assertThat( + exc.getCause().getMessage(), + either(containsString("")).or(containsString("fetch phase source loader")) + ); }); final AtomicArray exceptions = new AtomicArray<>(10); @@ -514,7 +518,10 @@ public void onFailure(Exception exc) { latch.await(); assertThat(exceptions.asList().size(), equalTo(10)); for (Exception exc : exceptions.asList()) { - assertThat(exc.getCause().getMessage(), containsString("")); + assertThat( + exc.getCause().getMessage(), + either(containsString("")).or(containsString("fetch phase source loader")) + ); } assertBusy(() -> assertThat(requestBreakerUsed(), equalTo(0L))); } finally { From ee6d444a0c8a0fb6345502dec5bbb0c312d8e7bd Mon Sep 17 00:00:00 2001 From: Andrei Dan Date: Wed, 19 Feb 2025 12:41:40 +0000 Subject: [PATCH 32/56] Default accounting buffer to 1M, drop int boxing. --- .../MemoryAccountingBytesRefCounted.java | 13 ++++++----- .../elasticsearch/search/SearchService.java | 14 +++++------ .../search/fetch/FetchPhase.java | 23 ++++++++++--------- .../search/SearchServiceSingleNodeTests.java | 17 +++++++++++++- 4 files changed, 42 insertions(+), 25 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/common/MemoryAccountingBytesRefCounted.java b/server/src/main/java/org/elasticsearch/common/MemoryAccountingBytesRefCounted.java index d8cc638059093..76cea992402b4 100644 --- a/server/src/main/java/org/elasticsearch/common/MemoryAccountingBytesRefCounted.java +++ b/server/src/main/java/org/elasticsearch/common/MemoryAccountingBytesRefCounted.java @@ -12,31 +12,32 @@ import org.elasticsearch.common.breaker.CircuitBreaker; import org.elasticsearch.core.AbstractRefCounted; +import java.util.concurrent.atomic.AtomicInteger; + /** * A ref counted object that accounts for memory usage in bytes and releases the * accounted memory from the circuit breaker when the reference count reaches zero. */ public final class MemoryAccountingBytesRefCounted extends AbstractRefCounted { - private int bytes; + private final AtomicInteger bytes = new AtomicInteger(0); private final CircuitBreaker breaker; - private MemoryAccountingBytesRefCounted(int bytes, CircuitBreaker breaker) { - this.bytes = bytes; + private MemoryAccountingBytesRefCounted(CircuitBreaker breaker) { this.breaker = breaker; } public static MemoryAccountingBytesRefCounted create(CircuitBreaker breaker) { - return new MemoryAccountingBytesRefCounted(0, breaker); + return new MemoryAccountingBytesRefCounted(breaker); } public void account(int bytes, String label) { breaker.addEstimateBytesAndMaybeBreak(bytes, label); - this.bytes += bytes; + this.bytes.addAndGet(bytes); } @Override protected void closeInternal() { - breaker.addWithoutBreaking(-bytes); + breaker.addWithoutBreaking(-bytes.get()); } } diff --git a/server/src/main/java/org/elasticsearch/search/SearchService.java b/server/src/main/java/org/elasticsearch/search/SearchService.java index 74f3364259b27..6b1b841fc4368 100644 --- a/server/src/main/java/org/elasticsearch/search/SearchService.java +++ b/server/src/main/java/org/elasticsearch/search/SearchService.java @@ -281,7 +281,7 @@ public class SearchService extends AbstractLifecycleComponent implements IndexEv */ public static final Setting MEMORY_ACCOUNTING_BUFFER_SIZE = Setting.byteSizeSetting( "search.memory_accounting_buffer_size", - ByteSizeValue.of(32, ByteSizeUnit.KB), + ByteSizeValue.of(1, ByteSizeUnit.MB), Property.Dynamic, Property.NodeScope ); @@ -323,7 +323,7 @@ public class SearchService extends AbstractLifecycleComponent implements IndexEv private volatile boolean enableRewriteAggsToFilterByFilter; - private volatile ByteSizeValue memoryAccountingBufferSize; + private volatile long memoryAccountingBufferSize; private final Cancellable keepAliveReaper; @@ -403,7 +403,7 @@ public SearchService( clusterService.getClusterSettings() .addSettingsUpdateConsumer(QUERY_PHASE_PARALLEL_COLLECTION_ENABLED, this::setEnableQueryPhaseParallelCollection); - memoryAccountingBufferSize = MEMORY_ACCOUNTING_BUFFER_SIZE.get(settings); + memoryAccountingBufferSize = MEMORY_ACCOUNTING_BUFFER_SIZE.get(settings).getBytes(); clusterService.getClusterSettings().addSettingsUpdateConsumer(MEMORY_ACCOUNTING_BUFFER_SIZE, this::setMemoryAccountingBufferSize); } @@ -420,7 +420,7 @@ private void setEnableQueryPhaseParallelCollection(boolean enableQueryPhaseParal } private void setMemoryAccountingBufferSize(ByteSizeValue memoryAccountingBufferSize) { - this.memoryAccountingBufferSize = memoryAccountingBufferSize; + this.memoryAccountingBufferSize = memoryAccountingBufferSize.getBytes(); } private static void validateKeepAlives(TimeValue defaultKeepAlive, TimeValue maxKeepAlive) { @@ -811,7 +811,7 @@ public void executeRankFeaturePhase(RankFeatureShardRequest request, SearchShard return searchContext.rankFeatureResult(); } RankFeatureShardPhase.prepareForFetch(searchContext, request); - fetchPhase.execute(searchContext, docIds, null, circuitBreaker, memoryAccountingBufferSize.getBytes()); + fetchPhase.execute(searchContext, docIds, null, circuitBreaker, memoryAccountingBufferSize); RankFeatureShardPhase.processFetch(searchContext); var rankFeatureResult = searchContext.rankFeatureResult(); rankFeatureResult.incRef(); @@ -829,7 +829,7 @@ private QueryFetchSearchResult executeFetchPhase(ReaderContext reader, SearchCon Releasable scope = tracer.withScope(context.getTask()); SearchOperationListenerExecutor executor = new SearchOperationListenerExecutor(context, true, afterQueryTime) ) { - fetchPhase.execute(context, shortcutDocIdsToLoad(context), null, circuitBreaker, memoryAccountingBufferSize.getBytes()); + fetchPhase.execute(context, shortcutDocIdsToLoad(context), null, circuitBreaker, memoryAccountingBufferSize); if (reader.singleSession()) { freeReaderContext(reader.id()); } @@ -1000,7 +1000,7 @@ public void executeFetchPhase(ShardFetchRequest request, CancellableTask task, A request.docIds(), request.getRankDocks(), circuitBreaker, - memoryAccountingBufferSize.getBytes() + memoryAccountingBufferSize ); if (readerContext.singleSession()) { freeReaderContext(request.contextId()); diff --git a/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java b/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java index 48d25b24deaa5..ce810205b04fb 100644 --- a/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java +++ b/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java @@ -49,7 +49,8 @@ import java.util.Collections; import java.util.List; import java.util.Map; -import java.util.function.Consumer; +import java.util.function.IntConsumer; +import java.util.function.IntSupplier; import java.util.function.Supplier; /** @@ -163,12 +164,12 @@ private SearchHits buildSearchHits( int docsInLeaf; int processedDocs; - private final Supplier getAndResetAccumulatedBytes = () -> { + private final IntSupplier getAndResetAccumulatedBytes = () -> { int bytesToSubmit = this.accumulatedBytesInLeaf; this.accumulatedBytesInLeaf = 0; return bytesToSubmit; }; - private final Consumer memoryUsageBytesAccumulator = (bytes) -> this.accumulatedBytesInLeaf += bytes; + private final IntConsumer memoryUsageBytesAccumulator = (bytes) -> this.accumulatedBytesInLeaf += bytes; @Override protected void setNextReader(LeafReaderContext ctx, int[] docsInLeaf) throws IOException { @@ -282,8 +283,8 @@ private HitContext prepareHitContext( RankDoc rankDoc, CircuitBreaker circuitBreaker, boolean submitToCb, - Consumer memoryUsageBytesAccumulator, - Supplier accumulatedBytesInLeaf + IntConsumer memoryUsageBytesAccumulator, + IntSupplier accumulatedBytesInLeaf ) throws IOException { if (nestedDocuments.advance(docId - subReaderContext.docBase) == null) { return prepareNonNestedHitContext( @@ -332,8 +333,8 @@ private static HitContext prepareNonNestedHitContext( RankDoc rankDoc, CircuitBreaker circuitBreaker, boolean accountMemoryWithCircuitBreaker, - Consumer memoryUsageBytesAccumulator, - Supplier accumulatedBytesInLeaf + IntConsumer memoryUsageBytesAccumulator, + IntSupplier accumulatedBytesInLeaf ) throws IOException { int subDocId = docId - subReaderContext.docBase; @@ -370,7 +371,7 @@ private static HitContext prepareNonNestedHitContext( source = sourceLoader.source(leafStoredFieldLoader, subDocId); memoryUsageBytesAccumulator.accept(source.internalSourceRef().length()); if (accountMemoryWithCircuitBreaker) { - memAccountingRefCounted.account(accumulatedBytesInLeaf.get(), "fetch phase source loader"); + memAccountingRefCounted.account(accumulatedBytesInLeaf.getAsInt(), "fetch phase source loader"); } } catch (CircuitBreakingException e) { hit.decRef(); @@ -403,8 +404,8 @@ private static Supplier lazyStoredSourceLoader( int doc, MemoryAccountingBytesRefCounted memAccountingRefCounted, boolean submitToCB, - Consumer memoryUsageAccumulator, - Supplier accumulatedBytesInLeaf + IntConsumer memoryUsageAccumulator, + IntSupplier accumulatedBytesInLeaf ) { return () -> { StoredFieldLoader rootLoader = profiler.storedFields(StoredFieldLoader.create(true, Collections.emptySet())); @@ -414,7 +415,7 @@ private static Supplier lazyStoredSourceLoader( BytesReference source = leafRootLoader.source(); memoryUsageAccumulator.accept(source.length()); if (submitToCB) { - memAccountingRefCounted.account(accumulatedBytesInLeaf.get(), "lazy fetch phase source loader"); + memAccountingRefCounted.account(accumulatedBytesInLeaf.getAsInt(), "lazy fetch phase source loader"); } return Source.fromBytes(source); } catch (IOException e) { diff --git a/server/src/test/java/org/elasticsearch/search/SearchServiceSingleNodeTests.java b/server/src/test/java/org/elasticsearch/search/SearchServiceSingleNodeTests.java index 969408a37caa8..0091b9585ee06 100644 --- a/server/src/test/java/org/elasticsearch/search/SearchServiceSingleNodeTests.java +++ b/server/src/test/java/org/elasticsearch/search/SearchServiceSingleNodeTests.java @@ -166,6 +166,7 @@ import static org.elasticsearch.action.support.WriteRequest.RefreshPolicy.IMMEDIATE; import static org.elasticsearch.indices.cluster.IndicesClusterStateService.AllocatedIndices.IndexRemovalReason.DELETED; import static org.elasticsearch.search.SearchService.DEFAULT_SIZE; +import static org.elasticsearch.search.SearchService.MEMORY_ACCOUNTING_BUFFER_SIZE; import static org.elasticsearch.search.SearchService.QUERY_PHASE_PARALLEL_COLLECTION_ENABLED; import static org.elasticsearch.search.SearchService.SEARCH_WORKER_THREADS_ENABLED; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; @@ -2940,8 +2941,16 @@ public void testFetchPhaseAccountsForSourceMemoryUsage() throws Exception { } if (randomBoolean()) { // 1 segment test is also useful so we enable the batching branch of the memory accounting - // we do local accounting up to 32kb (by default) before submitting to cb + indicesAdmin().prepareForceMerge("index").setMaxNumSegments(1).get(); + + // let's do local accounting up to 32kb before submitting to cb + ClusterUpdateSettingsResponse response = client().admin() + .cluster() + .prepareUpdateSettings(TEST_REQUEST_TIMEOUT, TEST_REQUEST_TIMEOUT) + .setPersistentSettings(Settings.builder().put(MEMORY_ACCOUNTING_BUFFER_SIZE.getKey(), "32k").build()) + .get(); + assertTrue(response.isAcknowledged()); } SearchService service = getInstanceFromNode(SearchService.class); @@ -2993,6 +3002,12 @@ public void testFetchPhaseAccountsForSourceMemoryUsage() throws Exception { if (readerContext != null) { service.freeReaderContext(readerContext.id()); } + // reset original default setting + client().admin() + .cluster() + .prepareUpdateSettings(TEST_REQUEST_TIMEOUT, TEST_REQUEST_TIMEOUT) + .setPersistentSettings(Settings.builder().putNull(SearchService.MEMORY_ACCOUNTING_BUFFER_SIZE.getKey()).build()) + .get(); if (fetchSearchResult != null) { long usedBeforeResultDecRef = breaker.getUsed(); if (fetchSearchResult.decRef()) { From a8e842a385c81229c6bbff76e5b125f774a58e62 Mon Sep 17 00:00:00 2001 From: Andrei Dan Date: Wed, 19 Feb 2025 13:02:56 +0000 Subject: [PATCH 33/56] Unify the in consumer and supplier in one interface --- .../search/fetch/FetchPhase.java | 76 +++++++++---------- 1 file changed, 38 insertions(+), 38 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java b/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java index ce810205b04fb..4672aaf924b8d 100644 --- a/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java +++ b/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java @@ -49,8 +49,6 @@ import java.util.Collections; import java.util.List; import java.util.Map; -import java.util.function.IntConsumer; -import java.util.function.IntSupplier; import java.util.function.Supplier; /** @@ -164,13 +162,6 @@ private SearchHits buildSearchHits( int docsInLeaf; int processedDocs; - private final IntSupplier getAndResetAccumulatedBytes = () -> { - int bytesToSubmit = this.accumulatedBytesInLeaf; - this.accumulatedBytesInLeaf = 0; - return bytesToSubmit; - }; - private final IntConsumer memoryUsageBytesAccumulator = (bytes) -> this.accumulatedBytesInLeaf += bytes; - @Override protected void setNextReader(LeafReaderContext ctx, int[] docsInLeaf) throws IOException { Timer timer = profiler.startNextReader(); @@ -215,8 +206,7 @@ protected SearchHit nextDoc(int doc) throws IOException { rankDocs == null ? null : rankDocs.get(doc), circuitBreaker, enoughBytesOrLastDocInLeaf, - memoryUsageBytesAccumulator, - getAndResetAccumulatedBytes + this::accumulateLocally ); boolean success = false; try { @@ -233,6 +223,24 @@ protected SearchHit nextDoc(int doc) throws IOException { } } } + + /** + * Accumulates the given bytes in the leaf and returns the accumulated bytes in the leaf. + * @param bytes the amount of bytes to accumulate locally + * @param resetLocallyAccumulatedBytes if the local counter for the accumulated bytes should be reset. Normally this will be true + * when submitting the accumulated bytes to the circuit breaker. + * @return the currently accumulated bytes in the leaf, including the provided bytes + */ + private int accumulateLocally(int bytes, boolean resetLocallyAccumulatedBytes) { + this.accumulatedBytesInLeaf += bytes; + if (resetLocallyAccumulatedBytes) { + int bytesToSubmit = this.accumulatedBytesInLeaf; + this.accumulatedBytesInLeaf = 0; + return bytesToSubmit; + } else { + return this.accumulatedBytesInLeaf; + } + } }; SearchHit[] hits = docsIterator.iterate( @@ -283,8 +291,7 @@ private HitContext prepareHitContext( RankDoc rankDoc, CircuitBreaker circuitBreaker, boolean submitToCb, - IntConsumer memoryUsageBytesAccumulator, - IntSupplier accumulatedBytesInLeaf + IntBooleanFunction memoryUsageAccumulator ) throws IOException { if (nestedDocuments.advance(docId - subReaderContext.docBase) == null) { return prepareNonNestedHitContext( @@ -298,8 +305,7 @@ private HitContext prepareHitContext( rankDoc, circuitBreaker, submitToCb, - memoryUsageBytesAccumulator, - accumulatedBytesInLeaf + memoryUsageAccumulator ); } else { return prepareNestedHitContext( @@ -332,9 +338,8 @@ private static HitContext prepareNonNestedHitContext( IdLoader.Leaf idLoader, RankDoc rankDoc, CircuitBreaker circuitBreaker, - boolean accountMemoryWithCircuitBreaker, - IntConsumer memoryUsageBytesAccumulator, - IntSupplier accumulatedBytesInLeaf + boolean submitToCB, + IntBooleanFunction memoryUsageAccumulator ) throws IOException { int subDocId = docId - subReaderContext.docBase; @@ -342,7 +347,7 @@ private static HitContext prepareNonNestedHitContext( MemoryAccountingBytesRefCounted memAccountingRefCounted = null; RefCounted refCountedHit = null; - if (accountMemoryWithCircuitBreaker) { + if (submitToCB) { memAccountingRefCounted = MemoryAccountingBytesRefCounted.create(circuitBreaker); refCountedHit = LeakTracker.wrap(memAccountingRefCounted); } @@ -351,15 +356,7 @@ private static HitContext prepareNonNestedHitContext( SearchHit hit = new SearchHit(docId, null, null, refCountedHit); // TODO: can we use real pooled buffers here as well? Source source = Source.lazy( - lazyStoredSourceLoader( - profiler, - subReaderContext, - subDocId, - memAccountingRefCounted, - accountMemoryWithCircuitBreaker, - memoryUsageBytesAccumulator, - accumulatedBytesInLeaf - ) + lazyStoredSourceLoader(profiler, subReaderContext, subDocId, memAccountingRefCounted, submitToCB, memoryUsageAccumulator) ); return new HitContext(hit, subReaderContext, subDocId, Map.of(), source, rankDoc); } else { @@ -369,9 +366,9 @@ private static HitContext prepareNonNestedHitContext( Timer timer = profiler.startLoadingSource(); try { source = sourceLoader.source(leafStoredFieldLoader, subDocId); - memoryUsageBytesAccumulator.accept(source.internalSourceRef().length()); - if (accountMemoryWithCircuitBreaker) { - memAccountingRefCounted.account(accumulatedBytesInLeaf.getAsInt(), "fetch phase source loader"); + int accumulatedInLeaf = memoryUsageAccumulator.apply(source.internalSourceRef().length(), submitToCB); + if (submitToCB) { + memAccountingRefCounted.account(accumulatedInLeaf, "fetch phase source loader"); } } catch (CircuitBreakingException e) { hit.decRef(); @@ -388,9 +385,8 @@ private static HitContext prepareNonNestedHitContext( subReaderContext, subDocId, memAccountingRefCounted, - accountMemoryWithCircuitBreaker, - memoryUsageBytesAccumulator, - accumulatedBytesInLeaf + submitToCB, + memoryUsageAccumulator ) ); } @@ -404,8 +400,7 @@ private static Supplier lazyStoredSourceLoader( int doc, MemoryAccountingBytesRefCounted memAccountingRefCounted, boolean submitToCB, - IntConsumer memoryUsageAccumulator, - IntSupplier accumulatedBytesInLeaf + IntBooleanFunction memoryUsageAccumulator ) { return () -> { StoredFieldLoader rootLoader = profiler.storedFields(StoredFieldLoader.create(true, Collections.emptySet())); @@ -413,9 +408,9 @@ private static Supplier lazyStoredSourceLoader( LeafStoredFieldLoader leafRootLoader = rootLoader.getLoader(ctx, null); leafRootLoader.advanceTo(doc); BytesReference source = leafRootLoader.source(); - memoryUsageAccumulator.accept(source.length()); + int accumulatedInLeaf = memoryUsageAccumulator.apply(source.length(), submitToCB); if (submitToCB) { - memAccountingRefCounted.account(accumulatedBytesInLeaf.getAsInt(), "lazy fetch phase source loader"); + memAccountingRefCounted.account(accumulatedInLeaf, "lazy fetch phase source loader"); } return Source.fromBytes(source); } catch (IOException e) { @@ -518,4 +513,9 @@ public String toString() { } }; } + + @FunctionalInterface + private interface IntBooleanFunction { + int apply(int i, boolean b); + } } From 64959b27c6353e72536f9234beffa8170b3dff6b Mon Sep 17 00:00:00 2001 From: Andrei Dan Date: Wed, 19 Feb 2025 13:03:05 +0000 Subject: [PATCH 34/56] use lambda --- .../main/java/org/elasticsearch/search/SearchService.java | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/search/SearchService.java b/server/src/main/java/org/elasticsearch/search/SearchService.java index 6b1b841fc4368..382d0c3686465 100644 --- a/server/src/main/java/org/elasticsearch/search/SearchService.java +++ b/server/src/main/java/org/elasticsearch/search/SearchService.java @@ -404,7 +404,8 @@ public SearchService( .addSettingsUpdateConsumer(QUERY_PHASE_PARALLEL_COLLECTION_ENABLED, this::setEnableQueryPhaseParallelCollection); memoryAccountingBufferSize = MEMORY_ACCOUNTING_BUFFER_SIZE.get(settings).getBytes(); - clusterService.getClusterSettings().addSettingsUpdateConsumer(MEMORY_ACCOUNTING_BUFFER_SIZE, this::setMemoryAccountingBufferSize); + clusterService.getClusterSettings() + .addSettingsUpdateConsumer(MEMORY_ACCOUNTING_BUFFER_SIZE, newValue -> this.memoryAccountingBufferSize = newValue.getBytes()); } private void setEnableSearchWorkerThreads(boolean enableSearchWorkerThreads) { @@ -419,10 +420,6 @@ private void setEnableQueryPhaseParallelCollection(boolean enableQueryPhaseParal this.enableQueryPhaseParallelCollection = enableQueryPhaseParallelCollection; } - private void setMemoryAccountingBufferSize(ByteSizeValue memoryAccountingBufferSize) { - this.memoryAccountingBufferSize = memoryAccountingBufferSize.getBytes(); - } - private static void validateKeepAlives(TimeValue defaultKeepAlive, TimeValue maxKeepAlive) { if (defaultKeepAlive.millis() > maxKeepAlive.millis()) { throw new IllegalArgumentException( From 6c3a59d7b49dbb7ef40bcb451cca113523feb418 Mon Sep 17 00:00:00 2001 From: Andrei Dan Date: Wed, 19 Feb 2025 13:31:24 +0000 Subject: [PATCH 35/56] Line length --- .../main/java/org/elasticsearch/search/fetch/FetchPhase.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java b/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java index 4672aaf924b8d..f190bb52d3362 100644 --- a/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java +++ b/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java @@ -227,8 +227,8 @@ protected SearchHit nextDoc(int doc) throws IOException { /** * Accumulates the given bytes in the leaf and returns the accumulated bytes in the leaf. * @param bytes the amount of bytes to accumulate locally - * @param resetLocallyAccumulatedBytes if the local counter for the accumulated bytes should be reset. Normally this will be true - * when submitting the accumulated bytes to the circuit breaker. + * @param resetLocallyAccumulatedBytes if the local counter for the accumulated bytes should be reset. Normally this will be + * true when submitting the accumulated bytes to the circuit breaker. * @return the currently accumulated bytes in the leaf, including the provided bytes */ private int accumulateLocally(int bytes, boolean resetLocallyAccumulatedBytes) { From f366e7f0d7729a21bed0f680f185b533b1d30b35 Mon Sep 17 00:00:00 2001 From: Andrei Dan Date: Wed, 19 Feb 2025 13:31:57 +0000 Subject: [PATCH 36/56] Use volatile int in `MemoryAccountingBytesRefCounted` --- .../common/MemoryAccountingBytesRefCounted.java | 10 ++++------ .../org/elasticsearch/search/fetch/FetchPhase.java | 4 ++-- .../common/MemoryAccountingBytesRefCountedTests.java | 6 +++--- 3 files changed, 9 insertions(+), 11 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/common/MemoryAccountingBytesRefCounted.java b/server/src/main/java/org/elasticsearch/common/MemoryAccountingBytesRefCounted.java index 76cea992402b4..b3c5749eb14ac 100644 --- a/server/src/main/java/org/elasticsearch/common/MemoryAccountingBytesRefCounted.java +++ b/server/src/main/java/org/elasticsearch/common/MemoryAccountingBytesRefCounted.java @@ -12,15 +12,13 @@ import org.elasticsearch.common.breaker.CircuitBreaker; import org.elasticsearch.core.AbstractRefCounted; -import java.util.concurrent.atomic.AtomicInteger; - /** * A ref counted object that accounts for memory usage in bytes and releases the * accounted memory from the circuit breaker when the reference count reaches zero. */ public final class MemoryAccountingBytesRefCounted extends AbstractRefCounted { - private final AtomicInteger bytes = new AtomicInteger(0); + private volatile int bytes; private final CircuitBreaker breaker; private MemoryAccountingBytesRefCounted(CircuitBreaker breaker) { @@ -31,13 +29,13 @@ public static MemoryAccountingBytesRefCounted create(CircuitBreaker breaker) { return new MemoryAccountingBytesRefCounted(breaker); } - public void account(int bytes, String label) { + public void setBytesAndAccount(int bytes, String label) { breaker.addEstimateBytesAndMaybeBreak(bytes, label); - this.bytes.addAndGet(bytes); + this.bytes = bytes; } @Override protected void closeInternal() { - breaker.addWithoutBreaking(-bytes.get()); + breaker.addWithoutBreaking(-bytes); } } diff --git a/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java b/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java index f190bb52d3362..e2ed442ee6b27 100644 --- a/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java +++ b/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java @@ -368,7 +368,7 @@ private static HitContext prepareNonNestedHitContext( source = sourceLoader.source(leafStoredFieldLoader, subDocId); int accumulatedInLeaf = memoryUsageAccumulator.apply(source.internalSourceRef().length(), submitToCB); if (submitToCB) { - memAccountingRefCounted.account(accumulatedInLeaf, "fetch phase source loader"); + memAccountingRefCounted.setBytesAndAccount(accumulatedInLeaf, "fetch phase source loader"); } } catch (CircuitBreakingException e) { hit.decRef(); @@ -410,7 +410,7 @@ private static Supplier lazyStoredSourceLoader( BytesReference source = leafRootLoader.source(); int accumulatedInLeaf = memoryUsageAccumulator.apply(source.length(), submitToCB); if (submitToCB) { - memAccountingRefCounted.account(accumulatedInLeaf, "lazy fetch phase source loader"); + memAccountingRefCounted.setBytesAndAccount(accumulatedInLeaf, "lazy fetch phase source loader"); } return Source.fromBytes(source); } catch (IOException e) { diff --git a/server/src/test/java/org/elasticsearch/common/MemoryAccountingBytesRefCountedTests.java b/server/src/test/java/org/elasticsearch/common/MemoryAccountingBytesRefCountedTests.java index a808df0b2b82a..30a4a1368f710 100644 --- a/server/src/test/java/org/elasticsearch/common/MemoryAccountingBytesRefCountedTests.java +++ b/server/src/test/java/org/elasticsearch/common/MemoryAccountingBytesRefCountedTests.java @@ -26,14 +26,14 @@ public void testNoMemoryAccounted() { public void testMemoryAccounted() { CircuitBreaker breaker = new MockBigArrays.LimitedBreaker("test", ByteSizeValue.ofGb(1)); MemoryAccountingBytesRefCounted refCounted = MemoryAccountingBytesRefCounted.create(breaker); - refCounted.account(10, "test"); + refCounted.setBytesAndAccount(10, "test"); assertEquals(10, breaker.getUsed()); } public void testCloseInternalDecrementsBreaker() { CircuitBreaker breaker = new MockBigArrays.LimitedBreaker("test", ByteSizeValue.ofGb(1)); MemoryAccountingBytesRefCounted refCounted = MemoryAccountingBytesRefCounted.create(breaker); - refCounted.account(10, "test"); + refCounted.setBytesAndAccount(10, "test"); refCounted.decRef(); assertEquals(0, breaker.getUsed()); } @@ -41,7 +41,7 @@ public void testCloseInternalDecrementsBreaker() { public void testBreakerNotDecrementedIfRefsRemaining() { CircuitBreaker breaker = new MockBigArrays.LimitedBreaker("test", ByteSizeValue.ofGb(1)); MemoryAccountingBytesRefCounted refCounted = MemoryAccountingBytesRefCounted.create(breaker); - refCounted.account(10, "test"); + refCounted.setBytesAndAccount(10, "test"); refCounted.incRef(); // 2 refs assertEquals(10, breaker.getUsed()); refCounted.decRef(); // 1 ref remaining so no decrementing is executed From 49df2a7d2f81d0d59fd129eeca87c7f8ef119aa8 Mon Sep 17 00:00:00 2001 From: Andrei Dan Date: Wed, 19 Feb 2025 19:46:03 +0000 Subject: [PATCH 37/56] Remove field volatile and add some docs as to why --- .../MemoryAccountingBytesRefCounted.java | 41 --------------- .../search/fetch/FetchPhase.java | 52 ++++++++++++++++--- .../MemoryAccountingBytesRefCountedTests.java | 9 ++-- 3 files changed, 50 insertions(+), 52 deletions(-) delete mode 100644 server/src/main/java/org/elasticsearch/common/MemoryAccountingBytesRefCounted.java rename server/src/test/java/org/elasticsearch/{common => search/fetch}/MemoryAccountingBytesRefCountedTests.java (89%) diff --git a/server/src/main/java/org/elasticsearch/common/MemoryAccountingBytesRefCounted.java b/server/src/main/java/org/elasticsearch/common/MemoryAccountingBytesRefCounted.java deleted file mode 100644 index b3c5749eb14ac..0000000000000 --- a/server/src/main/java/org/elasticsearch/common/MemoryAccountingBytesRefCounted.java +++ /dev/null @@ -1,41 +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", the "GNU Affero General Public License v3.0 only", and the "Server Side - * Public License v 1"; you may not use this file except in compliance with, at - * your election, the "Elastic License 2.0", the "GNU Affero General Public - * License v3.0 only", or the "Server Side Public License, v 1". - */ - -package org.elasticsearch.common; - -import org.elasticsearch.common.breaker.CircuitBreaker; -import org.elasticsearch.core.AbstractRefCounted; - -/** - * A ref counted object that accounts for memory usage in bytes and releases the - * accounted memory from the circuit breaker when the reference count reaches zero. - */ -public final class MemoryAccountingBytesRefCounted extends AbstractRefCounted { - - private volatile int bytes; - private final CircuitBreaker breaker; - - private MemoryAccountingBytesRefCounted(CircuitBreaker breaker) { - this.breaker = breaker; - } - - public static MemoryAccountingBytesRefCounted create(CircuitBreaker breaker) { - return new MemoryAccountingBytesRefCounted(breaker); - } - - public void setBytesAndAccount(int bytes, String label) { - breaker.addEstimateBytesAndMaybeBreak(bytes, label); - this.bytes = bytes; - } - - @Override - protected void closeInternal() { - breaker.addWithoutBreaking(-bytes); - } -} diff --git a/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java b/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java index e2ed442ee6b27..d4e9f9e653c21 100644 --- a/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java +++ b/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java @@ -13,10 +13,10 @@ import org.apache.logging.log4j.Logger; import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.search.TotalHits; -import org.elasticsearch.common.MemoryAccountingBytesRefCounted; import org.elasticsearch.common.breaker.CircuitBreaker; import org.elasticsearch.common.breaker.CircuitBreakingException; import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.core.AbstractRefCounted; import org.elasticsearch.core.RefCounted; import org.elasticsearch.index.fieldvisitor.LeafStoredFieldLoader; import org.elasticsearch.index.fieldvisitor.StoredFieldLoader; @@ -291,7 +291,7 @@ private HitContext prepareHitContext( RankDoc rankDoc, CircuitBreaker circuitBreaker, boolean submitToCb, - IntBooleanFunction memoryUsageAccumulator + MemoryUsageAccumulator memoryUsageAccumulator ) throws IOException { if (nestedDocuments.advance(docId - subReaderContext.docBase) == null) { return prepareNonNestedHitContext( @@ -339,7 +339,7 @@ private static HitContext prepareNonNestedHitContext( RankDoc rankDoc, CircuitBreaker circuitBreaker, boolean submitToCB, - IntBooleanFunction memoryUsageAccumulator + MemoryUsageAccumulator memoryUsageAccumulator ) throws IOException { int subDocId = docId - subReaderContext.docBase; @@ -368,7 +368,7 @@ private static HitContext prepareNonNestedHitContext( source = sourceLoader.source(leafStoredFieldLoader, subDocId); int accumulatedInLeaf = memoryUsageAccumulator.apply(source.internalSourceRef().length(), submitToCB); if (submitToCB) { - memAccountingRefCounted.setBytesAndAccount(accumulatedInLeaf, "fetch phase source loader"); + memAccountingRefCounted.account(accumulatedInLeaf, "fetch phase source loader"); } } catch (CircuitBreakingException e) { hit.decRef(); @@ -400,7 +400,7 @@ private static Supplier lazyStoredSourceLoader( int doc, MemoryAccountingBytesRefCounted memAccountingRefCounted, boolean submitToCB, - IntBooleanFunction memoryUsageAccumulator + MemoryUsageAccumulator memoryUsageAccumulator ) { return () -> { StoredFieldLoader rootLoader = profiler.storedFields(StoredFieldLoader.create(true, Collections.emptySet())); @@ -410,7 +410,7 @@ private static Supplier lazyStoredSourceLoader( BytesReference source = leafRootLoader.source(); int accumulatedInLeaf = memoryUsageAccumulator.apply(source.length(), submitToCB); if (submitToCB) { - memAccountingRefCounted.setBytesAndAccount(accumulatedInLeaf, "lazy fetch phase source loader"); + memAccountingRefCounted.account(accumulatedInLeaf, "lazy fetch phase source loader"); } return Source.fromBytes(source); } catch (IOException e) { @@ -515,7 +515,45 @@ public String toString() { } @FunctionalInterface - private interface IntBooleanFunction { + private interface MemoryUsageAccumulator { int apply(int i, boolean b); } + + /** + * A ref counted object that accounts for memory usage in bytes and releases the + * accounted memory from the circuit breaker when the reference count reaches zero. + */ + static final class MemoryAccountingBytesRefCounted extends AbstractRefCounted { + + // the bytes that we account for are not volatile because we only accumulate + // in the single threaded fetch phase and we release the reference after + // we write the response to the network (OutboundHandler). As with all other + // SearchHit fields this will be visible to the network thread that'll call #decRef. + private int bytes; + private final CircuitBreaker breaker; + + private MemoryAccountingBytesRefCounted(CircuitBreaker breaker) { + this.breaker = breaker; + } + + public static MemoryAccountingBytesRefCounted create(CircuitBreaker breaker) { + return new MemoryAccountingBytesRefCounted(breaker); + } + + /** + * This method increments the local counter for the accounted bytes and submits + * the accumulated bytes to the circuit breaker. + * This method is not thread-safe and should only be called from the single-threaded + * fetch phase. + */ + public void account(int bytes, String label) { + this.bytes += bytes; + breaker.addEstimateBytesAndMaybeBreak(bytes, label); + } + + @Override + protected void closeInternal() { + breaker.addWithoutBreaking(-bytes); + } + } } diff --git a/server/src/test/java/org/elasticsearch/common/MemoryAccountingBytesRefCountedTests.java b/server/src/test/java/org/elasticsearch/search/fetch/MemoryAccountingBytesRefCountedTests.java similarity index 89% rename from server/src/test/java/org/elasticsearch/common/MemoryAccountingBytesRefCountedTests.java rename to server/src/test/java/org/elasticsearch/search/fetch/MemoryAccountingBytesRefCountedTests.java index 30a4a1368f710..42c929d7067eb 100644 --- a/server/src/test/java/org/elasticsearch/common/MemoryAccountingBytesRefCountedTests.java +++ b/server/src/test/java/org/elasticsearch/search/fetch/MemoryAccountingBytesRefCountedTests.java @@ -7,11 +7,12 @@ * License v3.0 only", or the "Server Side Public License, v 1". */ -package org.elasticsearch.common; +package org.elasticsearch.search.fetch; import org.elasticsearch.common.breaker.CircuitBreaker; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.util.MockBigArrays; +import org.elasticsearch.search.fetch.FetchPhase.MemoryAccountingBytesRefCounted; import org.elasticsearch.test.ESTestCase; public class MemoryAccountingBytesRefCountedTests extends ESTestCase { @@ -26,14 +27,14 @@ public void testNoMemoryAccounted() { public void testMemoryAccounted() { CircuitBreaker breaker = new MockBigArrays.LimitedBreaker("test", ByteSizeValue.ofGb(1)); MemoryAccountingBytesRefCounted refCounted = MemoryAccountingBytesRefCounted.create(breaker); - refCounted.setBytesAndAccount(10, "test"); + refCounted.account(10, "test"); assertEquals(10, breaker.getUsed()); } public void testCloseInternalDecrementsBreaker() { CircuitBreaker breaker = new MockBigArrays.LimitedBreaker("test", ByteSizeValue.ofGb(1)); MemoryAccountingBytesRefCounted refCounted = MemoryAccountingBytesRefCounted.create(breaker); - refCounted.setBytesAndAccount(10, "test"); + refCounted.account(10, "test"); refCounted.decRef(); assertEquals(0, breaker.getUsed()); } @@ -41,7 +42,7 @@ public void testCloseInternalDecrementsBreaker() { public void testBreakerNotDecrementedIfRefsRemaining() { CircuitBreaker breaker = new MockBigArrays.LimitedBreaker("test", ByteSizeValue.ofGb(1)); MemoryAccountingBytesRefCounted refCounted = MemoryAccountingBytesRefCounted.create(breaker); - refCounted.setBytesAndAccount(10, "test"); + refCounted.account(10, "test"); refCounted.incRef(); // 2 refs assertEquals(10, breaker.getUsed()); refCounted.decRef(); // 1 ref remaining so no decrementing is executed From 86da7fdc5c0bbff02cec3e92875e03579d60e4fb Mon Sep 17 00:00:00 2001 From: Andrei Dan Date: Thu, 20 Feb 2025 07:53:37 +0000 Subject: [PATCH 38/56] Some docs for the functional interface --- .../java/org/elasticsearch/search/fetch/FetchPhase.java | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java b/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java index d4e9f9e653c21..4e1b06b171777 100644 --- a/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java +++ b/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java @@ -516,7 +516,12 @@ public String toString() { @FunctionalInterface private interface MemoryUsageAccumulator { - int apply(int i, boolean b); + /** + * Accumulates the provided bytes and returns the currently accumulated amount of bytes. + * When the reset parameter is true it accumulates the provided bytes, returns the + * accumulated amount but will also reset the local counter for the accumulated bytes. + */ + int apply(int bytes, boolean reset); } /** From ebcba6008a0166db6f1f786d5ced8caa32f433da Mon Sep 17 00:00:00 2001 From: Andrei Dan Date: Thu, 20 Feb 2025 08:21:01 +0000 Subject: [PATCH 39/56] Move the mem accounting buffer size in aggregationcontext --- .../AggConstructionContentionBenchmark.java | 5 +++++ .../org/elasticsearch/search/SearchService.java | 3 ++- .../aggregations/metrics/TopHitsAggregator.java | 11 ++++++----- .../aggregations/support/AggregationContext.java | 15 ++++++++++++++- .../index/mapper/MapperServiceTestCase.java | 5 +++++ .../search/aggregations/AggregatorTestCase.java | 5 ++++- 6 files changed, 36 insertions(+), 8 deletions(-) diff --git a/benchmarks/src/main/java/org/elasticsearch/benchmark/search/aggregations/AggConstructionContentionBenchmark.java b/benchmarks/src/main/java/org/elasticsearch/benchmark/search/aggregations/AggConstructionContentionBenchmark.java index 671564fcd424e..feff5ac06750e 100644 --- a/benchmarks/src/main/java/org/elasticsearch/benchmark/search/aggregations/AggConstructionContentionBenchmark.java +++ b/benchmarks/src/main/java/org/elasticsearch/benchmark/search/aggregations/AggConstructionContentionBenchmark.java @@ -366,6 +366,11 @@ public Set sourcePath(String fullName) { return Set.of(fullName); } + @Override + public long memoryAccountingBufferSize() { + return 1024 * 1024; + } + @Override public void close() { List releaseMe = new ArrayList<>(this.releaseMe); diff --git a/server/src/main/java/org/elasticsearch/search/SearchService.java b/server/src/main/java/org/elasticsearch/search/SearchService.java index 382d0c3686465..22ab64c1b411d 100644 --- a/server/src/main/java/org/elasticsearch/search/SearchService.java +++ b/server/src/main/java/org/elasticsearch/search/SearchService.java @@ -1441,7 +1441,8 @@ private void parseSource(DefaultSearchContext context, SearchSourceBuilder sourc context::isCancelled, context::buildFilteredQuery, enableRewriteAggsToFilterByFilter, - source.aggregations().isInSortOrderExecutionRequired() + source.aggregations().isInSortOrderExecutionRequired(), + memoryAccountingBufferSize ); context.addQuerySearchResultReleasable(aggContext); try { diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/TopHitsAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/TopHitsAggregator.java index 7cf2ee9f58916..dff7d93b7413b 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/TopHitsAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/TopHitsAggregator.java @@ -35,7 +35,6 @@ import org.elasticsearch.index.query.SearchExecutionContext; import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.SearchHits; -import org.elasticsearch.search.SearchService; import org.elasticsearch.search.aggregations.AggregationExecutionContext; import org.elasticsearch.search.aggregations.Aggregator; import org.elasticsearch.search.aggregations.InternalAggregation; @@ -57,8 +56,6 @@ class TopHitsAggregator extends MetricsAggregator { - private final long memAccountingBufferSize; - private static class Collectors { public final TopDocsCollector topDocsCollector; public final MaxScoreCollector maxScoreCollector; @@ -90,7 +87,6 @@ private static class Collectors { this.subSearchContext = subSearchContext; this.topDocsCollectors = new LongObjectPagedHashMap<>(1, bigArrays); this.fetchProfiles = context.profiling() ? new ArrayList<>() : null; - this.memAccountingBufferSize = context.getClusterSettings().get(SearchService.MEMORY_ACCOUNTING_BUFFER_SIZE).getBytes(); } @Override @@ -201,7 +197,12 @@ public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOE for (int i = 0; i < topDocs.scoreDocs.length; i++) { docIdsToLoad[i] = topDocs.scoreDocs[i].doc; } - FetchSearchResult fetchResult = runFetchPhase(subSearchContext, docIdsToLoad, context.breaker(), memAccountingBufferSize); + FetchSearchResult fetchResult = runFetchPhase( + subSearchContext, + docIdsToLoad, + context.breaker(), + context.memoryAccountingBufferSize() + ); if (fetchProfiles != null) { fetchProfiles.add(fetchResult.profileResult()); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/support/AggregationContext.java b/server/src/main/java/org/elasticsearch/search/aggregations/support/AggregationContext.java index 02f300df48385..e112513dda8ff 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/support/AggregationContext.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/support/AggregationContext.java @@ -308,6 +308,11 @@ public final AggregationUsageService getUsageService() { public abstract Set sourcePath(String fullName); + /** + * Return the amount of memory to buffer locally before accounting for it in the breaker. + */ + public abstract long memoryAccountingBufferSize(); + /** * Does this index have a {@code _doc_count} field in any segment? */ @@ -355,6 +360,7 @@ public static class ProductionAggregationContext extends AggregationContext { private final AnalysisRegistry analysisRegistry; private final List releaseMe = new ArrayList<>(); + private final long memoryAccountingBufferSize; public ProductionAggregationContext( AnalysisRegistry analysisRegistry, @@ -372,7 +378,8 @@ public ProductionAggregationContext( Supplier isCancelled, Function filterQuery, boolean enableRewriteToFilterByFilter, - boolean inSortOrderExecutionRequired + boolean inSortOrderExecutionRequired, + long memoryAccountingBufferSize ) { this.analysisRegistry = analysisRegistry; this.context = context; @@ -407,6 +414,7 @@ public ProductionAggregationContext( this.filterQuery = filterQuery; this.enableRewriteToFilterByFilter = enableRewriteToFilterByFilter; this.inSortOrderExecutionRequired = inSortOrderExecutionRequired; + this.memoryAccountingBufferSize = memoryAccountingBufferSize; } @Override @@ -612,6 +620,11 @@ public Set sourcePath(String fullName) { return context.sourcePath(fullName); } + @Override + public long memoryAccountingBufferSize() { + return memoryAccountingBufferSize; + } + @Override public void close() { /* diff --git a/test/framework/src/main/java/org/elasticsearch/index/mapper/MapperServiceTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/mapper/MapperServiceTestCase.java index b62e400826836..6200242aef2a4 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/mapper/MapperServiceTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/mapper/MapperServiceTestCase.java @@ -675,6 +675,11 @@ public Set sourcePath(String fullName) { return Set.of(fullName); } + @Override + public long memoryAccountingBufferSize() { + return 1024 * 1024; + } + @Override public void close() { throw new UnsupportedOperationException(); diff --git a/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java b/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java index d034e6e6679c1..6e388764d74e2 100644 --- a/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java @@ -68,6 +68,8 @@ import org.elasticsearch.common.network.NetworkAddress; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.unit.ByteSizeUnit; +import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.MockBigArrays; import org.elasticsearch.common.util.MockPageCacheRecycler; @@ -444,7 +446,8 @@ public Iterable dimensionFields() { isCancelled, q -> q, true, - isInSortOrderExecutionRequired + isInSortOrderExecutionRequired, + ByteSizeValue.of(1, ByteSizeUnit.MB).getBytes() ); return context; } From 83e37e64a6c6016bbc9043fb51ce46100a5ac917 Mon Sep 17 00:00:00 2001 From: Andrei Dan Date: Thu, 20 Feb 2025 09:11:58 +0000 Subject: [PATCH 40/56] AssertBusy to allow for the decRefs to be dec-ed to 0 --- .../search/SearchServiceSingleNodeTests.java | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/search/SearchServiceSingleNodeTests.java b/server/src/test/java/org/elasticsearch/search/SearchServiceSingleNodeTests.java index 0091b9585ee06..ef6d678a98cf8 100644 --- a/server/src/test/java/org/elasticsearch/search/SearchServiceSingleNodeTests.java +++ b/server/src/test/java/org/elasticsearch/search/SearchServiceSingleNodeTests.java @@ -3010,18 +3010,19 @@ public void testFetchPhaseAccountsForSourceMemoryUsage() throws Exception { .get(); if (fetchSearchResult != null) { long usedBeforeResultDecRef = breaker.getUsed(); - if (fetchSearchResult.decRef()) { - assertThat(usedBeforeResultDecRef, greaterThanOrEqualTo(48_000L)); - // when releasing the result references we should clear at least 48_000 bytes (48 hits with sources of at least 1000 - // bytes) + fetchSearchResult.decRef(); + assertBusy(() -> { long usedAfterResultDecRef = breaker.getUsed(); logger.info( "--> usedBeforeResultDecRef: [{}], usedAfterResultDecRef: [{}]", usedBeforeResultDecRef, usedAfterResultDecRef ); + assertThat(usedBeforeResultDecRef, greaterThanOrEqualTo(48_000L)); + // when releasing the result references we should clear at least 48_000 bytes (48 hits with sources of at least 1000 + // bytes) assertThat(usedBeforeResultDecRef - usedAfterResultDecRef, greaterThanOrEqualTo(48_000L)); - } + }); } } } From de7492d95580ac6b6af453c118dc024577c74586 Mon Sep 17 00:00:00 2001 From: Andrei Dan Date: Sat, 22 Feb 2025 19:28:44 +0000 Subject: [PATCH 41/56] Use the real memory circuit breaker in the fetch phase --- .../org/elasticsearch/index/IndexService.java | 4 + .../search/DefaultSearchContext.java | 16 +- .../org/elasticsearch/search/SearchHit.java | 2 +- .../elasticsearch/search/SearchService.java | 18 +- .../metrics/TopHitsAggregator.java | 17 +- .../support/AggregationContext.java | 15 +- .../search/fetch/FetchPhase.java | 174 +++--------------- .../search/fetch/subphase/InnerHitsPhase.java | 8 +- .../internal/FilteredSearchContext.java | 11 ++ .../search/internal/SearchContext.java | 11 ++ .../search/rank/RankSearchContext.java | 11 ++ .../action/search/FetchSearchPhaseTests.java | 7 +- .../search/DefaultSearchContextTests.java | 24 ++- .../search/SearchServiceSingleNodeTests.java | 100 ---------- .../MemoryAccountingBytesRefCountedTests.java | 51 ----- .../index/mapper/MapperServiceTestCase.java | 5 - .../aggregations/AggregatorTestCase.java | 5 +- .../elasticsearch/test/TestSearchContext.java | 11 ++ 18 files changed, 122 insertions(+), 368 deletions(-) delete mode 100644 server/src/test/java/org/elasticsearch/search/fetch/MemoryAccountingBytesRefCountedTests.java diff --git a/server/src/main/java/org/elasticsearch/index/IndexService.java b/server/src/main/java/org/elasticsearch/index/IndexService.java index baba9e94db7a7..6824f0d668c02 100644 --- a/server/src/main/java/org/elasticsearch/index/IndexService.java +++ b/server/src/main/java/org/elasticsearch/index/IndexService.java @@ -834,6 +834,10 @@ public ScriptService getScriptService() { return scriptService; } + public CircuitBreakerService breakerService() { + return circuitBreakerService; + } + List getIndexOperationListeners() { // pkg private for testing return indexingOperationListeners; } diff --git a/server/src/main/java/org/elasticsearch/search/DefaultSearchContext.java b/server/src/main/java/org/elasticsearch/search/DefaultSearchContext.java index 7095d3ec92c72..a6c15fc6b1a57 100644 --- a/server/src/main/java/org/elasticsearch/search/DefaultSearchContext.java +++ b/server/src/main/java/org/elasticsearch/search/DefaultSearchContext.java @@ -23,6 +23,7 @@ import org.apache.lucene.util.NumericUtils; import org.elasticsearch.action.search.SearchType; import org.elasticsearch.cluster.routing.IndexRouting; +import org.elasticsearch.common.breaker.CircuitBreaker; import org.elasticsearch.common.lucene.search.Queries; import org.elasticsearch.core.Nullable; import org.elasticsearch.core.Releasable; @@ -103,6 +104,7 @@ final class DefaultSearchContext extends SearchContext { private final IndexShard indexShard; private final IndexService indexService; private final ContextIndexSearcher searcher; + private final long memoryAccountingBufferSize; private DfsSearchResult dfsResult; private QuerySearchResult queryResult; private RankFeatureResult rankFeatureResult; @@ -168,7 +170,8 @@ final class DefaultSearchContext extends SearchContext { Executor executor, SearchService.ResultsType resultsType, boolean enableQueryPhaseParallelCollection, - int minimumDocsPerSlice + int minimumDocsPerSlice, + long memoryAccountingBufferSize ) throws IOException { this.readerContext = readerContext; this.request = request; @@ -179,6 +182,7 @@ final class DefaultSearchContext extends SearchContext { this.shardTarget = shardTarget; this.indexService = readerContext.indexService(); this.indexShard = readerContext.indexShard(); + this.memoryAccountingBufferSize = memoryAccountingBufferSize; Engine.Searcher engineSearcher = readerContext.acquireSearcher("search"); int maximumNumberOfSlices = determineMaximumNumberOfSlices( @@ -902,6 +906,16 @@ public Profilers getProfilers() { return profilers; } + @Override + public CircuitBreaker circuitBreaker() { + return indexService.breakerService().getBreaker(CircuitBreaker.REQUEST); + } + + @Override + public long memAccountingBufferSize() { + return memoryAccountingBufferSize; + } + public void setProfilers(Profilers profilers) { this.profilers = profilers; } diff --git a/server/src/main/java/org/elasticsearch/search/SearchHit.java b/server/src/main/java/org/elasticsearch/search/SearchHit.java index 154b75d61fdbd..b603b346e9b15 100644 --- a/server/src/main/java/org/elasticsearch/search/SearchHit.java +++ b/server/src/main/java/org/elasticsearch/search/SearchHit.java @@ -123,7 +123,7 @@ public SearchHit(int nestedTopDocId, String id, NestedIdentity nestedIdentity) { this(nestedTopDocId, id, nestedIdentity, null); } - public SearchHit(int nestedTopDocId, String id, NestedIdentity nestedIdentity, @Nullable RefCounted refCounted) { + private SearchHit(int nestedTopDocId, String id, NestedIdentity nestedIdentity, @Nullable RefCounted refCounted) { this( nestedTopDocId, DEFAULT_SCORE, diff --git a/server/src/main/java/org/elasticsearch/search/SearchService.java b/server/src/main/java/org/elasticsearch/search/SearchService.java index 22ab64c1b411d..2aa9829e9b132 100644 --- a/server/src/main/java/org/elasticsearch/search/SearchService.java +++ b/server/src/main/java/org/elasticsearch/search/SearchService.java @@ -808,7 +808,7 @@ public void executeRankFeaturePhase(RankFeatureShardRequest request, SearchShard return searchContext.rankFeatureResult(); } RankFeatureShardPhase.prepareForFetch(searchContext, request); - fetchPhase.execute(searchContext, docIds, null, circuitBreaker, memoryAccountingBufferSize); + fetchPhase.execute(searchContext, docIds, null); RankFeatureShardPhase.processFetch(searchContext); var rankFeatureResult = searchContext.rankFeatureResult(); rankFeatureResult.incRef(); @@ -826,7 +826,7 @@ private QueryFetchSearchResult executeFetchPhase(ReaderContext reader, SearchCon Releasable scope = tracer.withScope(context.getTask()); SearchOperationListenerExecutor executor = new SearchOperationListenerExecutor(context, true, afterQueryTime) ) { - fetchPhase.execute(context, shortcutDocIdsToLoad(context), null, circuitBreaker, memoryAccountingBufferSize); + fetchPhase.execute(context, shortcutDocIdsToLoad(context), null); if (reader.singleSession()) { freeReaderContext(reader.id()); } @@ -992,13 +992,7 @@ public void executeFetchPhase(ShardFetchRequest request, CancellableTask task, A System.nanoTime() ) ) { - fetchPhase.execute( - searchContext, - request.docIds(), - request.getRankDocks(), - circuitBreaker, - memoryAccountingBufferSize - ); + fetchPhase.execute(searchContext, request.docIds(), request.getRankDocks()); if (readerContext.singleSession()) { freeReaderContext(request.contextId()); } @@ -1217,7 +1211,8 @@ private DefaultSearchContext createSearchContext( searchExecutor, resultsType, enableQueryPhaseParallelCollection, - minimumDocsPerSlice + minimumDocsPerSlice, + memoryAccountingBufferSize ); // we clone the query shard context here just for rewriting otherwise we // might end up with incorrect state since we are using now() or script services @@ -1441,8 +1436,7 @@ private void parseSource(DefaultSearchContext context, SearchSourceBuilder sourc context::isCancelled, context::buildFilteredQuery, enableRewriteAggsToFilterByFilter, - source.aggregations().isInSortOrderExecutionRequired(), - memoryAccountingBufferSize + source.aggregations().isInSortOrderExecutionRequired() ); context.addQuerySearchResultReleasable(aggContext); try { diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/TopHitsAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/TopHitsAggregator.java index dff7d93b7413b..87d8f839dfca1 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/metrics/TopHitsAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/metrics/TopHitsAggregator.java @@ -25,7 +25,6 @@ import org.apache.lucene.search.TopScoreDocCollectorManager; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.action.search.MaxScoreCollector; -import org.elasticsearch.common.breaker.CircuitBreaker; import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.lucene.search.TopDocsAndMaxScore; import org.elasticsearch.common.util.BigArrays; @@ -197,12 +196,7 @@ public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOE for (int i = 0; i < topDocs.scoreDocs.length; i++) { docIdsToLoad[i] = topDocs.scoreDocs[i].doc; } - FetchSearchResult fetchResult = runFetchPhase( - subSearchContext, - docIdsToLoad, - context.breaker(), - context.memoryAccountingBufferSize() - ); + FetchSearchResult fetchResult = runFetchPhase(subSearchContext, docIdsToLoad); if (fetchProfiles != null) { fetchProfiles.add(fetchResult.profileResult()); } @@ -226,12 +220,7 @@ public InternalAggregation buildAggregation(long owningBucketOrdinal) throws IOE ); } - private static FetchSearchResult runFetchPhase( - SubSearchContext subSearchContext, - int[] docIdsToLoad, - CircuitBreaker breaker, - long memAccountingBufferSize - ) { + private static FetchSearchResult runFetchPhase(SubSearchContext subSearchContext, int[] docIdsToLoad) { // Fork the search execution context for each slice, because the fetch phase does not support concurrent execution yet. SearchExecutionContext searchExecutionContext = new SearchExecutionContext(subSearchContext.getSearchExecutionContext()); SubSearchContext fetchSubSearchContext = new SubSearchContext(subSearchContext) { @@ -240,7 +229,7 @@ public SearchExecutionContext getSearchExecutionContext() { return searchExecutionContext; } }; - fetchSubSearchContext.fetchPhase().execute(fetchSubSearchContext, docIdsToLoad, null, breaker, memAccountingBufferSize); + fetchSubSearchContext.fetchPhase().execute(fetchSubSearchContext, docIdsToLoad, null); return fetchSubSearchContext.fetchResult(); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/support/AggregationContext.java b/server/src/main/java/org/elasticsearch/search/aggregations/support/AggregationContext.java index e112513dda8ff..02f300df48385 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/support/AggregationContext.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/support/AggregationContext.java @@ -308,11 +308,6 @@ public final AggregationUsageService getUsageService() { public abstract Set sourcePath(String fullName); - /** - * Return the amount of memory to buffer locally before accounting for it in the breaker. - */ - public abstract long memoryAccountingBufferSize(); - /** * Does this index have a {@code _doc_count} field in any segment? */ @@ -360,7 +355,6 @@ public static class ProductionAggregationContext extends AggregationContext { private final AnalysisRegistry analysisRegistry; private final List releaseMe = new ArrayList<>(); - private final long memoryAccountingBufferSize; public ProductionAggregationContext( AnalysisRegistry analysisRegistry, @@ -378,8 +372,7 @@ public ProductionAggregationContext( Supplier isCancelled, Function filterQuery, boolean enableRewriteToFilterByFilter, - boolean inSortOrderExecutionRequired, - long memoryAccountingBufferSize + boolean inSortOrderExecutionRequired ) { this.analysisRegistry = analysisRegistry; this.context = context; @@ -414,7 +407,6 @@ public ProductionAggregationContext( this.filterQuery = filterQuery; this.enableRewriteToFilterByFilter = enableRewriteToFilterByFilter; this.inSortOrderExecutionRequired = inSortOrderExecutionRequired; - this.memoryAccountingBufferSize = memoryAccountingBufferSize; } @Override @@ -620,11 +612,6 @@ public Set sourcePath(String fullName) { return context.sourcePath(fullName); } - @Override - public long memoryAccountingBufferSize() { - return memoryAccountingBufferSize; - } - @Override public void close() { /* diff --git a/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java b/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java index 4e1b06b171777..545e2fd18a358 100644 --- a/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java +++ b/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java @@ -13,11 +13,7 @@ import org.apache.logging.log4j.Logger; import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.search.TotalHits; -import org.elasticsearch.common.breaker.CircuitBreaker; -import org.elasticsearch.common.breaker.CircuitBreakingException; import org.elasticsearch.common.bytes.BytesReference; -import org.elasticsearch.core.AbstractRefCounted; -import org.elasticsearch.core.RefCounted; import org.elasticsearch.index.fieldvisitor.LeafStoredFieldLoader; import org.elasticsearch.index.fieldvisitor.StoredFieldLoader; import org.elasticsearch.index.mapper.IdLoader; @@ -40,7 +36,6 @@ import org.elasticsearch.search.rank.RankDoc; import org.elasticsearch.search.rank.RankDocShardInfo; import org.elasticsearch.tasks.TaskCancelledException; -import org.elasticsearch.transport.LeakTracker; import org.elasticsearch.xcontent.XContentType; import java.io.IOException; @@ -65,13 +60,7 @@ public FetchPhase(List fetchSubPhases) { this.fetchSubPhases[fetchSubPhases.size()] = new InnerHitsPhase(this); } - public void execute( - SearchContext context, - int[] docIdsToLoad, - RankDocShardInfo rankDocs, - CircuitBreaker circuitBreaker, - long memAccountingBufferSize - ) { + public void execute(SearchContext context, int[] docIdsToLoad, RankDocShardInfo rankDocs) { if (LOGGER.isTraceEnabled()) { LOGGER.trace("{}", new SearchContextSourcePrinter(context)); } @@ -93,7 +82,7 @@ public void execute( : Profilers.startProfilingFetchPhase(); SearchHits hits = null; try { - hits = buildSearchHits(context, docIdsToLoad, profiler, rankDocs, circuitBreaker, memAccountingBufferSize); + hits = buildSearchHits(context, docIdsToLoad, profiler, rankDocs); } finally { // Always finish profiling ProfileResult profileResult = profiler.finish(); @@ -115,14 +104,7 @@ public Source getSource(LeafReaderContext ctx, int doc) { } } - private SearchHits buildSearchHits( - SearchContext context, - int[] docIdsToLoad, - Profiler profiler, - RankDocShardInfo rankDocs, - CircuitBreaker circuitBreaker, - long memAccountingBufferSize - ) { + private SearchHits buildSearchHits(SearchContext context, int[] docIdsToLoad, Profiler profiler, RankDocShardInfo rankDocs) { FetchContext fetchContext = new FetchContext(context); SourceLoader sourceLoader = context.newSourceLoader(); @@ -188,11 +170,17 @@ protected SearchHit nextDoc(int doc) throws IOException { throw new TaskCancelledException("cancelled"); } ++processedDocs; - // indicates if we should submit the accounted memory to the circuit breaker. + // indicates if we should check with the parent circuit breaker (the real memory breaker) if we have enough memory to + // continue the fetch phase. // we do so whenever one of the following is true: // 1. we have accumulated at least the size of the memory accounting buffer // 2. we have reached the last document in the leaf - boolean enoughBytesOrLastDocInLeaf = (accumulatedBytesInLeaf >= memAccountingBufferSize) || (processedDocs == docsInLeaf); + boolean enoughBytesOrLastDocInLeaf = (accumulatedBytesInLeaf >= context.memAccountingBufferSize()) + || (processedDocs == docsInLeaf); + if (enoughBytesOrLastDocInLeaf) { + context.circuitBreaker().addEstimateBytesAndMaybeBreak(0, "fetch source"); + accumulatedBytesInLeaf = 0; + } HitContext hit = prepareHitContext( context, requiresSource, @@ -203,10 +191,7 @@ protected SearchHit nextDoc(int doc) throws IOException { ctx, leafSourceLoader, leafIdLoader, - rankDocs == null ? null : rankDocs.get(doc), - circuitBreaker, - enoughBytesOrLastDocInLeaf, - this::accumulateLocally + rankDocs == null ? null : rankDocs.get(doc) ); boolean success = false; try { @@ -215,6 +200,12 @@ protected SearchHit nextDoc(int doc) throws IOException { for (FetchSubPhaseProcessor processor : processors) { processor.process(hit); } + + BytesReference sourceRef = hit.hit().getSourceRef(); + if (sourceRef != null) { + int sourceLength = sourceRef.length(); + this.accumulatedBytesInLeaf += sourceLength; + } success = true; return hit.hit(); } finally { @@ -223,24 +214,6 @@ protected SearchHit nextDoc(int doc) throws IOException { } } } - - /** - * Accumulates the given bytes in the leaf and returns the accumulated bytes in the leaf. - * @param bytes the amount of bytes to accumulate locally - * @param resetLocallyAccumulatedBytes if the local counter for the accumulated bytes should be reset. Normally this will be - * true when submitting the accumulated bytes to the circuit breaker. - * @return the currently accumulated bytes in the leaf, including the provided bytes - */ - private int accumulateLocally(int bytes, boolean resetLocallyAccumulatedBytes) { - this.accumulatedBytesInLeaf += bytes; - if (resetLocallyAccumulatedBytes) { - int bytesToSubmit = this.accumulatedBytesInLeaf; - this.accumulatedBytesInLeaf = 0; - return bytesToSubmit; - } else { - return this.accumulatedBytesInLeaf; - } - } }; SearchHit[] hits = docsIterator.iterate( @@ -288,10 +261,7 @@ private HitContext prepareHitContext( LeafReaderContext subReaderContext, SourceLoader.Leaf sourceLoader, IdLoader.Leaf idLoader, - RankDoc rankDoc, - CircuitBreaker circuitBreaker, - boolean submitToCb, - MemoryUsageAccumulator memoryUsageAccumulator + RankDoc rankDoc ) throws IOException { if (nestedDocuments.advance(docId - subReaderContext.docBase) == null) { return prepareNonNestedHitContext( @@ -302,10 +272,7 @@ private HitContext prepareHitContext( subReaderContext, sourceLoader, idLoader, - rankDoc, - circuitBreaker, - submitToCb, - memoryUsageAccumulator + rankDoc ); } else { return prepareNestedHitContext( @@ -336,83 +303,44 @@ private static HitContext prepareNonNestedHitContext( LeafReaderContext subReaderContext, SourceLoader.Leaf sourceLoader, IdLoader.Leaf idLoader, - RankDoc rankDoc, - CircuitBreaker circuitBreaker, - boolean submitToCB, - MemoryUsageAccumulator memoryUsageAccumulator + RankDoc rankDoc ) throws IOException { int subDocId = docId - subReaderContext.docBase; leafStoredFieldLoader.advanceTo(subDocId); - MemoryAccountingBytesRefCounted memAccountingRefCounted = null; - RefCounted refCountedHit = null; - if (submitToCB) { - memAccountingRefCounted = MemoryAccountingBytesRefCounted.create(circuitBreaker); - refCountedHit = LeakTracker.wrap(memAccountingRefCounted); - } String id = idLoader.getId(subDocId); if (id == null) { - SearchHit hit = new SearchHit(docId, null, null, refCountedHit); + SearchHit hit = new SearchHit(docId); // TODO: can we use real pooled buffers here as well? - Source source = Source.lazy( - lazyStoredSourceLoader(profiler, subReaderContext, subDocId, memAccountingRefCounted, submitToCB, memoryUsageAccumulator) - ); + Source source = Source.lazy(lazyStoredSourceLoader(profiler, subReaderContext, subDocId)); return new HitContext(hit, subReaderContext, subDocId, Map.of(), source, rankDoc); } else { - SearchHit hit = new SearchHit(docId, id, null, refCountedHit); + SearchHit hit = new SearchHit(docId, id, null); Source source; if (requiresSource) { Timer timer = profiler.startLoadingSource(); try { source = sourceLoader.source(leafStoredFieldLoader, subDocId); - int accumulatedInLeaf = memoryUsageAccumulator.apply(source.internalSourceRef().length(), submitToCB); - if (submitToCB) { - memAccountingRefCounted.account(accumulatedInLeaf, "fetch phase source loader"); - } - } catch (CircuitBreakingException e) { - hit.decRef(); - throw e; } finally { if (timer != null) { timer.stop(); } } } else { - source = Source.lazy( - lazyStoredSourceLoader( - profiler, - subReaderContext, - subDocId, - memAccountingRefCounted, - submitToCB, - memoryUsageAccumulator - ) - ); + source = Source.lazy(lazyStoredSourceLoader(profiler, subReaderContext, subDocId)); } return new HitContext(hit, subReaderContext, subDocId, leafStoredFieldLoader.storedFields(), source, rankDoc); } } - private static Supplier lazyStoredSourceLoader( - Profiler profiler, - LeafReaderContext ctx, - int doc, - MemoryAccountingBytesRefCounted memAccountingRefCounted, - boolean submitToCB, - MemoryUsageAccumulator memoryUsageAccumulator - ) { + private static Supplier lazyStoredSourceLoader(Profiler profiler, LeafReaderContext ctx, int doc) { return () -> { StoredFieldLoader rootLoader = profiler.storedFields(StoredFieldLoader.create(true, Collections.emptySet())); try { LeafStoredFieldLoader leafRootLoader = rootLoader.getLoader(ctx, null); leafRootLoader.advanceTo(doc); - BytesReference source = leafRootLoader.source(); - int accumulatedInLeaf = memoryUsageAccumulator.apply(source.length(), submitToCB); - if (submitToCB) { - memAccountingRefCounted.account(accumulatedInLeaf, "lazy fetch phase source loader"); - } - return Source.fromBytes(source); + return Source.fromBytes(leafRootLoader.source()); } catch (IOException e) { throw new UncheckedIOException(e); } @@ -513,52 +441,4 @@ public String toString() { } }; } - - @FunctionalInterface - private interface MemoryUsageAccumulator { - /** - * Accumulates the provided bytes and returns the currently accumulated amount of bytes. - * When the reset parameter is true it accumulates the provided bytes, returns the - * accumulated amount but will also reset the local counter for the accumulated bytes. - */ - int apply(int bytes, boolean reset); - } - - /** - * A ref counted object that accounts for memory usage in bytes and releases the - * accounted memory from the circuit breaker when the reference count reaches zero. - */ - static final class MemoryAccountingBytesRefCounted extends AbstractRefCounted { - - // the bytes that we account for are not volatile because we only accumulate - // in the single threaded fetch phase and we release the reference after - // we write the response to the network (OutboundHandler). As with all other - // SearchHit fields this will be visible to the network thread that'll call #decRef. - private int bytes; - private final CircuitBreaker breaker; - - private MemoryAccountingBytesRefCounted(CircuitBreaker breaker) { - this.breaker = breaker; - } - - public static MemoryAccountingBytesRefCounted create(CircuitBreaker breaker) { - return new MemoryAccountingBytesRefCounted(breaker); - } - - /** - * This method increments the local counter for the accounted bytes and submits - * the accumulated bytes to the circuit breaker. - * This method is not thread-safe and should only be called from the single-threaded - * fetch phase. - */ - public void account(int bytes, String label) { - this.bytes += bytes; - breaker.addEstimateBytesAndMaybeBreak(bytes, label); - } - - @Override - protected void closeInternal() { - breaker.addWithoutBreaking(-bytes); - } - } } diff --git a/server/src/main/java/org/elasticsearch/search/fetch/subphase/InnerHitsPhase.java b/server/src/main/java/org/elasticsearch/search/fetch/subphase/InnerHitsPhase.java index 3212cbb86bab7..374c96fdefe86 100644 --- a/server/src/main/java/org/elasticsearch/search/fetch/subphase/InnerHitsPhase.java +++ b/server/src/main/java/org/elasticsearch/search/fetch/subphase/InnerHitsPhase.java @@ -12,7 +12,6 @@ import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.search.FieldDoc; import org.apache.lucene.search.ScoreDoc; -import org.elasticsearch.common.breaker.NoopCircuitBreaker; import org.elasticsearch.common.lucene.search.TopDocsAndMaxScore; import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.SearchHits; @@ -33,10 +32,6 @@ public final class InnerHitsPhase implements FetchSubPhase { - // inner hits will potentially load deeper inner hits, so we need to make sure we don't count their source towards the breaker - // because the sub-top-hit-source they reference is already accounted for in the outer hits (i.e. the top level hit will account for - // the entire source which all inner hits will sub reference) - private static final NoopCircuitBreaker NOOP_CIRCUIT_BREAKER = new NoopCircuitBreaker("inner_hits_phase"); private final FetchPhase fetchPhase; public InnerHitsPhase(FetchPhase fetchPhase) { @@ -98,8 +93,7 @@ private void hitExecute(Map innerHi innerHitsContext.setRootId(hit.getId()); innerHitsContext.setRootLookup(rootSource); - // running the inner hits fetch phase without memory accounting as the source size is already accounted for in the outer hits - fetchPhase.execute(innerHitsContext, docIdsToLoad, null, NOOP_CIRCUIT_BREAKER, 0L); + fetchPhase.execute(innerHitsContext, docIdsToLoad, null); FetchSearchResult fetchResult = innerHitsContext.fetchResult(); SearchHit[] internalHits = fetchResult.fetchResult().hits().getHits(); for (int j = 0; j < internalHits.length; j++) { diff --git a/server/src/main/java/org/elasticsearch/search/internal/FilteredSearchContext.java b/server/src/main/java/org/elasticsearch/search/internal/FilteredSearchContext.java index 5bad06d08f96b..3ed2bf0fff5c0 100644 --- a/server/src/main/java/org/elasticsearch/search/internal/FilteredSearchContext.java +++ b/server/src/main/java/org/elasticsearch/search/internal/FilteredSearchContext.java @@ -13,6 +13,7 @@ import org.apache.lucene.search.Query; import org.apache.lucene.search.TotalHits; import org.elasticsearch.action.search.SearchType; +import org.elasticsearch.common.breaker.CircuitBreaker; import org.elasticsearch.core.TimeValue; import org.elasticsearch.index.cache.bitset.BitsetFilterCache; import org.elasticsearch.index.mapper.IdLoader; @@ -460,4 +461,14 @@ public SourceLoader newSourceLoader() { public IdLoader newIdLoader() { return in.newIdLoader(); } + + @Override + public CircuitBreaker circuitBreaker() { + return in.circuitBreaker(); + } + + @Override + public long memAccountingBufferSize() { + return in.memAccountingBufferSize(); + } } 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 5a8c280e12f85..2404112392bb9 100644 --- a/server/src/main/java/org/elasticsearch/search/internal/SearchContext.java +++ b/server/src/main/java/org/elasticsearch/search/internal/SearchContext.java @@ -12,6 +12,7 @@ import org.apache.lucene.search.Query; import org.apache.lucene.search.TotalHits; import org.elasticsearch.action.search.SearchType; +import org.elasticsearch.common.breaker.CircuitBreaker; import org.elasticsearch.core.Assertions; import org.elasticsearch.core.Nullable; import org.elasticsearch.core.Releasable; @@ -369,6 +370,16 @@ public Query rewrittenQuery() { */ public abstract Profilers getProfilers(); + /** + * The circuit breaker used to account for the search operation. + */ + public abstract CircuitBreaker circuitBreaker(); + + /** + * Return the amount of memory to buffer locally before accounting for it in the breaker. + */ + public abstract long memAccountingBufferSize(); + /** * Adds a releasable that will be freed when this context is closed. */ diff --git a/server/src/main/java/org/elasticsearch/search/rank/RankSearchContext.java b/server/src/main/java/org/elasticsearch/search/rank/RankSearchContext.java index 951a9b0cf3520..57a9b9282023f 100644 --- a/server/src/main/java/org/elasticsearch/search/rank/RankSearchContext.java +++ b/server/src/main/java/org/elasticsearch/search/rank/RankSearchContext.java @@ -13,6 +13,7 @@ import org.apache.lucene.search.Query; import org.apache.lucene.search.TotalHits; import org.elasticsearch.action.search.SearchType; +import org.elasticsearch.common.breaker.CircuitBreaker; import org.elasticsearch.core.TimeValue; import org.elasticsearch.index.cache.bitset.BitsetFilterCache; import org.elasticsearch.index.mapper.IdLoader; @@ -203,6 +204,16 @@ public Profilers getProfilers() { return null; } + @Override + public CircuitBreaker circuitBreaker() { + return parent.circuitBreaker(); + } + + @Override + public long memAccountingBufferSize() { + return parent.memAccountingBufferSize(); + } + @Override public long getRelativeTimeInMillis() { return parent.getRelativeTimeInMillis(); diff --git a/server/src/test/java/org/elasticsearch/action/search/FetchSearchPhaseTests.java b/server/src/test/java/org/elasticsearch/action/search/FetchSearchPhaseTests.java index 5a563b11fc4f0..fd60621c7e400 100644 --- a/server/src/test/java/org/elasticsearch/action/search/FetchSearchPhaseTests.java +++ b/server/src/test/java/org/elasticsearch/action/search/FetchSearchPhaseTests.java @@ -791,7 +791,7 @@ public void testFetchTimeoutWithPartialResults() throws IOException { ContextIndexSearcher contextIndexSearcher = createSearcher(r); try (SearchContext searchContext = createSearchContext(contextIndexSearcher, true)) { FetchPhase fetchPhase = createFetchPhase(contextIndexSearcher); - fetchPhase.execute(searchContext, new int[] { 0, 1, 2 }, null, new NoopCircuitBreaker(CircuitBreaker.REQUEST), 0L); + fetchPhase.execute(searchContext, new int[] { 0, 1, 2 }, null); assertTrue(searchContext.queryResult().searchTimedOut()); assertEquals(1, searchContext.fetchResult().hits().getHits().length); } finally { @@ -812,10 +812,7 @@ public void testFetchTimeoutNoPartialResults() throws IOException { try (SearchContext searchContext = createSearchContext(contextIndexSearcher, false)) { FetchPhase fetchPhase = createFetchPhase(contextIndexSearcher); - expectThrows( - SearchTimeoutException.class, - () -> fetchPhase.execute(searchContext, new int[] { 0, 1, 2 }, null, new NoopCircuitBreaker(CircuitBreaker.REQUEST), 0L) - ); + expectThrows(SearchTimeoutException.class, () -> fetchPhase.execute(searchContext, new int[] { 0, 1, 2 }, null)); assertNull(searchContext.fetchResult().hits()); } finally { r.close(); diff --git a/server/src/test/java/org/elasticsearch/search/DefaultSearchContextTests.java b/server/src/test/java/org/elasticsearch/search/DefaultSearchContextTests.java index 82737b13a1a46..a39dcc0a3175d 100644 --- a/server/src/test/java/org/elasticsearch/search/DefaultSearchContextTests.java +++ b/server/src/test/java/org/elasticsearch/search/DefaultSearchContextTests.java @@ -99,6 +99,8 @@ public class DefaultSearchContextTests extends MapperServiceTestCase { + private static final long MEMORY_ACCOUNTING_BUFFER_SIZE = 1024 * 1024L; + public void testPreProcess() throws Exception { TimeValue timeout = new TimeValue(randomIntBetween(1, 100)); ShardSearchRequest shardSearchRequest = mock(ShardSearchRequest.class); @@ -184,7 +186,8 @@ protected Engine.Searcher acquireSearcherInternal(String source) { null, randomFrom(SearchService.ResultsType.values()), randomBoolean(), - randomInt() + randomInt(), + MEMORY_ACCOUNTING_BUFFER_SIZE ); contextWithoutScroll.from(300); contextWithoutScroll.close(); @@ -226,7 +229,9 @@ protected Engine.Searcher acquireSearcherInternal(String source) { null, randomFrom(SearchService.ResultsType.values()), randomBoolean(), - randomInt() + randomInt(), + MEMORY_ACCOUNTING_BUFFER_SIZE + ) ) { context1.from(300); @@ -308,7 +313,8 @@ public ScrollContext scrollContext() { null, randomFrom(SearchService.ResultsType.values()), randomBoolean(), - randomInt() + randomInt(), + MEMORY_ACCOUNTING_BUFFER_SIZE ) ) { @@ -350,7 +356,8 @@ public ScrollContext scrollContext() { null, randomFrom(SearchService.ResultsType.values()), randomBoolean(), - randomInt() + randomInt(), + MEMORY_ACCOUNTING_BUFFER_SIZE ) ) { context3.sliceBuilder(null).parsedQuery(parsedQuery).preProcess(); @@ -381,7 +388,8 @@ public ScrollContext scrollContext() { null, randomFrom(SearchService.ResultsType.values()), randomBoolean(), - randomInt() + randomInt(), + MEMORY_ACCOUNTING_BUFFER_SIZE ) ) { context4.sliceBuilder(new SliceBuilder(1, 2)).parsedQuery(parsedQuery).preProcess(); @@ -452,7 +460,8 @@ protected Engine.Searcher acquireSearcherInternal(String source) { null, randomFrom(SearchService.ResultsType.values()), randomBoolean(), - randomInt() + randomInt(), + MEMORY_ACCOUNTING_BUFFER_SIZE ); assertThat(context.searcher().hasCancellations(), is(false)); @@ -1054,7 +1063,8 @@ protected Engine.Searcher acquireSearcherInternal(String source) { null, randomFrom(SearchService.ResultsType.values()), randomBoolean(), - randomInt() + randomInt(), + MEMORY_ACCOUNTING_BUFFER_SIZE ); } } diff --git a/server/src/test/java/org/elasticsearch/search/SearchServiceSingleNodeTests.java b/server/src/test/java/org/elasticsearch/search/SearchServiceSingleNodeTests.java index ef6d678a98cf8..9c3dc7ca8ab9d 100644 --- a/server/src/test/java/org/elasticsearch/search/SearchServiceSingleNodeTests.java +++ b/server/src/test/java/org/elasticsearch/search/SearchServiceSingleNodeTests.java @@ -49,7 +49,6 @@ import org.elasticsearch.cluster.routing.TestShardRouting; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.UUIDs; -import org.elasticsearch.common.breaker.CircuitBreaker; import org.elasticsearch.common.breaker.CircuitBreakingException; import org.elasticsearch.common.breaker.NoopCircuitBreaker; import org.elasticsearch.common.bytes.BytesArray; @@ -77,7 +76,6 @@ import org.elasticsearch.index.shard.SearchOperationListener; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.indices.IndicesService; -import org.elasticsearch.indices.breaker.CircuitBreakerService; import org.elasticsearch.indices.settings.InternalOrPrivateSettingsPlugin; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.plugins.SearchPlugin; @@ -157,8 +155,6 @@ import java.util.function.Function; import java.util.function.IntConsumer; import java.util.function.Supplier; -import java.util.stream.Collectors; -import java.util.stream.IntStream; import static java.util.Collections.emptyList; import static java.util.Collections.emptyMap; @@ -166,7 +162,6 @@ import static org.elasticsearch.action.support.WriteRequest.RefreshPolicy.IMMEDIATE; import static org.elasticsearch.indices.cluster.IndicesClusterStateService.AllocatedIndices.IndexRemovalReason.DELETED; import static org.elasticsearch.search.SearchService.DEFAULT_SIZE; -import static org.elasticsearch.search.SearchService.MEMORY_ACCOUNTING_BUFFER_SIZE; import static org.elasticsearch.search.SearchService.QUERY_PHASE_PARALLEL_COLLECTION_ENABLED; import static org.elasticsearch.search.SearchService.SEARCH_WORKER_THREADS_ENABLED; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; @@ -180,8 +175,6 @@ import static org.hamcrest.CoreMatchers.notNullValue; import static org.hamcrest.CoreMatchers.startsWith; import static org.hamcrest.Matchers.containsInAnyOrder; -import static org.hamcrest.Matchers.greaterThan; -import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.not; import static org.mockito.Mockito.mock; @@ -2934,99 +2927,6 @@ public void testSlicingBehaviourForParallelCollection() throws Exception { } } - public void testFetchPhaseAccountsForSourceMemoryUsage() throws Exception { - createIndex("index"); - for (int i = 0; i < 48; i++) { - prepareIndex("index").setId(String.valueOf(i)).setSource("field", randomAlphaOfLength(1000)).setRefreshPolicy(IMMEDIATE).get(); - } - if (randomBoolean()) { - // 1 segment test is also useful so we enable the batching branch of the memory accounting - - indicesAdmin().prepareForceMerge("index").setMaxNumSegments(1).get(); - - // let's do local accounting up to 32kb before submitting to cb - ClusterUpdateSettingsResponse response = client().admin() - .cluster() - .prepareUpdateSettings(TEST_REQUEST_TIMEOUT, TEST_REQUEST_TIMEOUT) - .setPersistentSettings(Settings.builder().put(MEMORY_ACCOUNTING_BUFFER_SIZE.getKey(), "32k").build()) - .get(); - assertTrue(response.isAcknowledged()); - } - - SearchService service = getInstanceFromNode(SearchService.class); - IndicesService indicesService = getInstanceFromNode(IndicesService.class); - IndexService indexService = indicesService.indexServiceSafe(resolveIndex("index")); - IndexShard indexShard = indexService.getShard(0); - CircuitBreakerService circuitBreakerService = getInstanceFromNode(CircuitBreakerService.class); - CircuitBreaker breaker = circuitBreakerService.getBreaker(CircuitBreaker.REQUEST); - - SearchRequest scrollSearchRequest = new SearchRequest().allowPartialSearchResults(true).scroll(TimeValue.timeValueMinutes(1)); - FetchSearchResult fetchSearchResult = null; - ReaderContext readerContext = null; - try { - readerContext = service.createAndPutReaderContext( - new ShardSearchRequest( - OriginalIndices.NONE, - scrollSearchRequest, - indexShard.shardId(), - 0, - 1, - AliasFilter.EMPTY, - 1.0f, - -1, - null - ), - indexService, - indexShard, - indexShard.acquireSearcherSupplier(), - 0 - ); - ShardFetchRequest req = new ShardFetchRequest( - readerContext.id(), - IntStream.range(0, 48).boxed().collect(Collectors.toList()), - null - ); - PlainActionFuture listener = new PlainActionFuture<>(); - long usedBeforeFetch = breaker.getUsed(); - service.executeFetchPhase(req, new SearchShardTask(123L, "", "", "", null, emptyMap()), listener.delegateFailure((l, r) -> { - r.incRef(); - l.onResponse(r); - })); - fetchSearchResult = listener.get(); - long usedAfterFetch = breaker.getUsed(); - assertThat(usedAfterFetch, greaterThan(usedBeforeFetch)); - logger.info("--> usedBeforeFetch: [{}], usedAfterFetch: [{}]", usedBeforeFetch, usedAfterFetch); - // 48 docs with at least 1000 bytes in the source - assertThat((usedAfterFetch - usedBeforeFetch), greaterThanOrEqualTo(48_000L)); - } finally { - if (readerContext != null) { - service.freeReaderContext(readerContext.id()); - } - // reset original default setting - client().admin() - .cluster() - .prepareUpdateSettings(TEST_REQUEST_TIMEOUT, TEST_REQUEST_TIMEOUT) - .setPersistentSettings(Settings.builder().putNull(SearchService.MEMORY_ACCOUNTING_BUFFER_SIZE.getKey()).build()) - .get(); - if (fetchSearchResult != null) { - long usedBeforeResultDecRef = breaker.getUsed(); - fetchSearchResult.decRef(); - assertBusy(() -> { - long usedAfterResultDecRef = breaker.getUsed(); - logger.info( - "--> usedBeforeResultDecRef: [{}], usedAfterResultDecRef: [{}]", - usedBeforeResultDecRef, - usedAfterResultDecRef - ); - assertThat(usedBeforeResultDecRef, greaterThanOrEqualTo(48_000L)); - // when releasing the result references we should clear at least 48_000 bytes (48 hits with sources of at least 1000 - // bytes) - assertThat(usedBeforeResultDecRef - usedAfterResultDecRef, greaterThanOrEqualTo(48_000L)); - }); - } - } - } - private static ReaderContext createReaderContext(IndexService indexService, IndexShard indexShard) { return new ReaderContext( new ShardSearchContextId(UUIDs.randomBase64UUID(), randomNonNegativeLong()), diff --git a/server/src/test/java/org/elasticsearch/search/fetch/MemoryAccountingBytesRefCountedTests.java b/server/src/test/java/org/elasticsearch/search/fetch/MemoryAccountingBytesRefCountedTests.java deleted file mode 100644 index 42c929d7067eb..0000000000000 --- a/server/src/test/java/org/elasticsearch/search/fetch/MemoryAccountingBytesRefCountedTests.java +++ /dev/null @@ -1,51 +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", the "GNU Affero General Public License v3.0 only", and the "Server Side - * Public License v 1"; you may not use this file except in compliance with, at - * your election, the "Elastic License 2.0", the "GNU Affero General Public - * License v3.0 only", or the "Server Side Public License, v 1". - */ - -package org.elasticsearch.search.fetch; - -import org.elasticsearch.common.breaker.CircuitBreaker; -import org.elasticsearch.common.unit.ByteSizeValue; -import org.elasticsearch.common.util.MockBigArrays; -import org.elasticsearch.search.fetch.FetchPhase.MemoryAccountingBytesRefCounted; -import org.elasticsearch.test.ESTestCase; - -public class MemoryAccountingBytesRefCountedTests extends ESTestCase { - - public void testNoMemoryAccounted() { - CircuitBreaker breaker = new MockBigArrays.LimitedBreaker("test", ByteSizeValue.ofGb(1)); - MemoryAccountingBytesRefCounted refCounted = MemoryAccountingBytesRefCounted.create(breaker); - refCounted.decRef(); - assertEquals(0, breaker.getUsed()); - } - - public void testMemoryAccounted() { - CircuitBreaker breaker = new MockBigArrays.LimitedBreaker("test", ByteSizeValue.ofGb(1)); - MemoryAccountingBytesRefCounted refCounted = MemoryAccountingBytesRefCounted.create(breaker); - refCounted.account(10, "test"); - assertEquals(10, breaker.getUsed()); - } - - public void testCloseInternalDecrementsBreaker() { - CircuitBreaker breaker = new MockBigArrays.LimitedBreaker("test", ByteSizeValue.ofGb(1)); - MemoryAccountingBytesRefCounted refCounted = MemoryAccountingBytesRefCounted.create(breaker); - refCounted.account(10, "test"); - refCounted.decRef(); - assertEquals(0, breaker.getUsed()); - } - - public void testBreakerNotDecrementedIfRefsRemaining() { - CircuitBreaker breaker = new MockBigArrays.LimitedBreaker("test", ByteSizeValue.ofGb(1)); - MemoryAccountingBytesRefCounted refCounted = MemoryAccountingBytesRefCounted.create(breaker); - refCounted.account(10, "test"); - refCounted.incRef(); // 2 refs - assertEquals(10, breaker.getUsed()); - refCounted.decRef(); // 1 ref remaining so no decrementing is executed - assertEquals(10, breaker.getUsed()); - } -} diff --git a/test/framework/src/main/java/org/elasticsearch/index/mapper/MapperServiceTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/mapper/MapperServiceTestCase.java index 6200242aef2a4..b62e400826836 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/mapper/MapperServiceTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/mapper/MapperServiceTestCase.java @@ -675,11 +675,6 @@ public Set sourcePath(String fullName) { return Set.of(fullName); } - @Override - public long memoryAccountingBufferSize() { - return 1024 * 1024; - } - @Override public void close() { throw new UnsupportedOperationException(); diff --git a/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java b/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java index 6e388764d74e2..d034e6e6679c1 100644 --- a/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java @@ -68,8 +68,6 @@ import org.elasticsearch.common.network.NetworkAddress; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.unit.ByteSizeUnit; -import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.MockBigArrays; import org.elasticsearch.common.util.MockPageCacheRecycler; @@ -446,8 +444,7 @@ public Iterable dimensionFields() { isCancelled, q -> q, true, - isInSortOrderExecutionRequired, - ByteSizeValue.of(1, ByteSizeUnit.MB).getBytes() + isInSortOrderExecutionRequired ); return context; } diff --git a/test/framework/src/main/java/org/elasticsearch/test/TestSearchContext.java b/test/framework/src/main/java/org/elasticsearch/test/TestSearchContext.java index c46442485ff9e..bf612ebc70bc4 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/TestSearchContext.java +++ b/test/framework/src/main/java/org/elasticsearch/test/TestSearchContext.java @@ -12,6 +12,7 @@ import org.apache.lucene.search.Query; import org.apache.lucene.search.TotalHits; import org.elasticsearch.action.search.SearchType; +import org.elasticsearch.common.breaker.CircuitBreaker; import org.elasticsearch.core.TimeValue; import org.elasticsearch.index.IndexService; import org.elasticsearch.index.cache.bitset.BitsetFilterCache; @@ -500,6 +501,16 @@ public Profilers getProfilers() { return null; // no profiling } + @Override + public CircuitBreaker circuitBreaker() { + return indexService.getBigArrays().breakerService().getBreaker(CircuitBreaker.REQUEST); + } + + @Override + public long memAccountingBufferSize() { + return 1024 * 1024; + } + @Override public SearchExecutionContext getSearchExecutionContext() { return searchExecutionContext; From e7f2a3c8e87047a7f9b5df11bf803de000e69ed1 Mon Sep 17 00:00:00 2001 From: Andrei Dan Date: Sat, 22 Feb 2025 20:16:22 +0000 Subject: [PATCH 42/56] Compile --- .../aggregations/AggConstructionContentionBenchmark.java | 5 ----- 1 file changed, 5 deletions(-) diff --git a/benchmarks/src/main/java/org/elasticsearch/benchmark/search/aggregations/AggConstructionContentionBenchmark.java b/benchmarks/src/main/java/org/elasticsearch/benchmark/search/aggregations/AggConstructionContentionBenchmark.java index feff5ac06750e..671564fcd424e 100644 --- a/benchmarks/src/main/java/org/elasticsearch/benchmark/search/aggregations/AggConstructionContentionBenchmark.java +++ b/benchmarks/src/main/java/org/elasticsearch/benchmark/search/aggregations/AggConstructionContentionBenchmark.java @@ -366,11 +366,6 @@ public Set sourcePath(String fullName) { return Set.of(fullName); } - @Override - public long memoryAccountingBufferSize() { - return 1024 * 1024; - } - @Override public void close() { List releaseMe = new ArrayList<>(this.releaseMe); From 1b0ca70b3b265bd8435452174cd430d3fcada97b Mon Sep 17 00:00:00 2001 From: Andrei Dan Date: Sat, 22 Feb 2025 20:17:16 +0000 Subject: [PATCH 43/56] static --- .../main/java/org/elasticsearch/search/fetch/FetchPhase.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java b/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java index 545e2fd18a358..6d6163faab31d 100644 --- a/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java +++ b/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java @@ -251,7 +251,7 @@ List getProcessors(SearchShardTarget target, FetchContex } } - private HitContext prepareHitContext( + private static HitContext prepareHitContext( SearchContext context, boolean requiresSource, Profiler profiler, From 979203a3f65eeda445987e83e9fc057c79102f76 Mon Sep 17 00:00:00 2001 From: Andrei Dan Date: Sat, 22 Feb 2025 20:18:03 +0000 Subject: [PATCH 44/56] Drop bool --- .../main/java/org/elasticsearch/search/fetch/FetchPhase.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java b/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java index 6d6163faab31d..4d32f1d96ea47 100644 --- a/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java +++ b/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java @@ -175,9 +175,7 @@ protected SearchHit nextDoc(int doc) throws IOException { // we do so whenever one of the following is true: // 1. we have accumulated at least the size of the memory accounting buffer // 2. we have reached the last document in the leaf - boolean enoughBytesOrLastDocInLeaf = (accumulatedBytesInLeaf >= context.memAccountingBufferSize()) - || (processedDocs == docsInLeaf); - if (enoughBytesOrLastDocInLeaf) { + if (accumulatedBytesInLeaf >= context.memAccountingBufferSize() || processedDocs == docsInLeaf) { context.circuitBreaker().addEstimateBytesAndMaybeBreak(0, "fetch source"); accumulatedBytesInLeaf = 0; } From 403720cf22749b8a1e76cab01bd829fd0fc0a1a8 Mon Sep 17 00:00:00 2001 From: Andrei Dan Date: Sun, 23 Feb 2025 11:09:33 +0000 Subject: [PATCH 45/56] Fix mocks --- .../search/aggregations/AggregatorTestCase.java | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java b/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java index d034e6e6679c1..03921dfb4359b 100644 --- a/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java @@ -437,7 +437,12 @@ public Iterable dimensionFields() { () -> query, null, maxBucket, - () -> buildSubSearchContext(indexSettings, searchExecutionContext, bitsetFilterCache), + () -> buildSubSearchContext( + indexSettings, + searchExecutionContext, + bitsetFilterCache, + breakerService.getBreaker(CircuitBreaker.REQUEST) + ), bitsetFilterCache, randomInt(), () -> 0L, @@ -471,7 +476,8 @@ protected List objectMappers() { private SubSearchContext buildSubSearchContext( IndexSettings indexSettings, SearchExecutionContext searchExecutionContext, - BitsetFilterCache bitsetFilterCache + BitsetFilterCache bitsetFilterCache, + CircuitBreaker breaker ) { SearchContext ctx = mock(SearchContext.class); try { @@ -488,7 +494,8 @@ private SubSearchContext buildSubSearchContext( throw new RuntimeException(e); } when(ctx.fetchPhase()).thenReturn(new FetchPhase(Arrays.asList(new FetchSourcePhase(), new FetchDocValuesPhase()))); - + when(ctx.circuitBreaker()).thenReturn(breaker); + when(ctx.memAccountingBufferSize()).thenReturn(1024 * 1024L); /* * Use a QueryShardContext that doesn't contain nested documents so we * don't try to fetch them which would require mocking a whole menagerie From 06dcb68127721dd9333e2b1429355439e13e1b40 Mon Sep 17 00:00:00 2001 From: Andrei Dan Date: Sun, 23 Feb 2025 11:18:57 +0000 Subject: [PATCH 46/56] Have a checkRealMemoryCB method on the SearchContext --- .../elasticsearch/search/fetch/FetchPhase.java | 10 +++------- .../search/internal/SearchContext.java | 17 +++++++++++++++++ 2 files changed, 20 insertions(+), 7 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java b/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java index 4d32f1d96ea47..4cede421a112b 100644 --- a/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java +++ b/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java @@ -170,15 +170,11 @@ protected SearchHit nextDoc(int doc) throws IOException { throw new TaskCancelledException("cancelled"); } ++processedDocs; - // indicates if we should check with the parent circuit breaker (the real memory breaker) if we have enough memory to - // continue the fetch phase. - // we do so whenever one of the following is true: - // 1. we have accumulated at least the size of the memory accounting buffer - // 2. we have reached the last document in the leaf - if (accumulatedBytesInLeaf >= context.memAccountingBufferSize() || processedDocs == docsInLeaf) { - context.circuitBreaker().addEstimateBytesAndMaybeBreak(0, "fetch source"); + if (context.checkRealMemoryCB(accumulatedBytesInLeaf, processedDocs == docsInLeaf, "fetch source")) { + // if we checked the real memory breaker, we restart our local accounting accumulatedBytesInLeaf = 0; } + HitContext hit = prepareHitContext( context, requiresSource, 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 2404112392bb9..ad7bb7a01bd7c 100644 --- a/server/src/main/java/org/elasticsearch/search/internal/SearchContext.java +++ b/server/src/main/java/org/elasticsearch/search/internal/SearchContext.java @@ -380,6 +380,23 @@ public Query rewrittenQuery() { */ public abstract long memAccountingBufferSize(); + /** + * Checks if the accumulated bytes are greater than the buffer size and if so, checks the available memory in the parent breaker + * (the real memory breaker). + * It also checks the available memory if the caller indicates that the local accounting is finished. + * @param locallyAccumulatedBytes the number of bytes accumulated locally + * @param localAccountingFinished if the local accounting is finished + * @param label the label to use in the breaker + * @return true if the real memory breaker is called and false otherwise + */ + public final boolean checkRealMemoryCB(int locallyAccumulatedBytes, boolean localAccountingFinished, String label) { + if (locallyAccumulatedBytes >= memAccountingBufferSize() || localAccountingFinished) { + circuitBreaker().addEstimateBytesAndMaybeBreak(0, label); + return true; + } + return false; + } + /** * Adds a releasable that will be freed when this context is closed. */ From 41b74cb5e61eecfb0aaa2ce2f753d1eb0091114c Mon Sep 17 00:00:00 2001 From: Andrei Dan Date: Sun, 23 Feb 2025 11:59:24 +0000 Subject: [PATCH 47/56] Revert use of fetch phase source loader trip --- .../action/search/TransportSearchIT.java | 11 ++--------- 1 file changed, 2 insertions(+), 9 deletions(-) diff --git a/server/src/internalClusterTest/java/org/elasticsearch/action/search/TransportSearchIT.java b/server/src/internalClusterTest/java/org/elasticsearch/action/search/TransportSearchIT.java index a396d61fdf576..5db2651c703d2 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/action/search/TransportSearchIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/action/search/TransportSearchIT.java @@ -78,7 +78,6 @@ import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertResponse; import static org.hamcrest.Matchers.containsString; -import static org.hamcrest.Matchers.either; import static org.hamcrest.Matchers.equalTo; public class TransportSearchIT extends ESIntegTestCase { @@ -488,10 +487,7 @@ public void onFailure(Exception e) { Exception.class, client.prepareSearch("test").addAggregation(new TestAggregationBuilder("test")) ); - assertThat( - exc.getCause().getMessage(), - either(containsString("")).or(containsString("fetch phase source loader")) - ); + assertThat(exc.getCause().getMessage(), containsString("")); }); final AtomicArray exceptions = new AtomicArray<>(10); @@ -518,10 +514,7 @@ public void onFailure(Exception exc) { latch.await(); assertThat(exceptions.asList().size(), equalTo(10)); for (Exception exc : exceptions.asList()) { - assertThat( - exc.getCause().getMessage(), - either(containsString("")).or(containsString("fetch phase source loader")) - ); + assertThat(exc.getCause().getMessage(), containsString("")); } assertBusy(() -> assertThat(requestBreakerUsed(), equalTo(0L))); } finally { From 6e60c6e37fceee953b2878983fdef93924c1440f Mon Sep 17 00:00:00 2001 From: Andrei Dan Date: Sun, 23 Feb 2025 12:44:21 +0000 Subject: [PATCH 48/56] Add unit test for the tracking memory in the fetch phase --- .../search/fetch/FetchPhase.java | 3 +- .../action/search/FetchSearchPhaseTests.java | 77 +++++++++++++++++++ 2 files changed, 78 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java b/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java index 4cede421a112b..0a7cd82e38179 100644 --- a/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java +++ b/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java @@ -197,8 +197,7 @@ protected SearchHit nextDoc(int doc) throws IOException { BytesReference sourceRef = hit.hit().getSourceRef(); if (sourceRef != null) { - int sourceLength = sourceRef.length(); - this.accumulatedBytesInLeaf += sourceLength; + this.accumulatedBytesInLeaf += sourceRef.length(); } success = true; return hit.hit(); diff --git a/server/src/test/java/org/elasticsearch/action/search/FetchSearchPhaseTests.java b/server/src/test/java/org/elasticsearch/action/search/FetchSearchPhaseTests.java index fd60621c7e400..8b54f67348a87 100644 --- a/server/src/test/java/org/elasticsearch/action/search/FetchSearchPhaseTests.java +++ b/server/src/test/java/org/elasticsearch/action/search/FetchSearchPhaseTests.java @@ -9,6 +9,9 @@ package org.elasticsearch.action.search; import org.apache.lucene.document.Document; +import org.apache.lucene.document.Field; +import org.apache.lucene.document.StoredField; +import org.apache.lucene.document.StringField; import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.search.Query; @@ -20,16 +23,19 @@ import org.apache.lucene.tests.index.RandomIndexWriter; import org.apache.lucene.tests.store.MockDirectoryWrapper; import org.apache.lucene.util.Accountable; +import org.apache.lucene.util.BytesRef; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.OriginalIndices; import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.breaker.CircuitBreaker; +import org.elasticsearch.common.breaker.CircuitBreakingException; import org.elasticsearch.common.breaker.NoopCircuitBreaker; import org.elasticsearch.common.lucene.search.TopDocsAndMaxScore; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.AtomicArray; import org.elasticsearch.common.util.concurrent.EsExecutors; +import org.elasticsearch.core.Nullable; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.IndexVersion; import org.elasticsearch.index.cache.bitset.BitsetFilterCache; @@ -55,6 +61,7 @@ import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.ShardSearchContextId; import org.elasticsearch.search.internal.ShardSearchRequest; +import org.elasticsearch.search.lookup.Source; import org.elasticsearch.search.profile.ProfileResult; import org.elasticsearch.search.profile.SearchProfileQueryPhaseResult; import org.elasticsearch.search.profile.SearchProfileShardResult; @@ -72,10 +79,12 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.BiFunction; +import java.util.stream.IntStream; import static org.hamcrest.Matchers.arrayWithSize; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.hasSize; +import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.nullValue; public class FetchSearchPhaseTests extends ESTestCase { @@ -820,6 +829,57 @@ public void testFetchTimeoutNoPartialResults() throws IOException { } } + public void testFetchPhaseChecksMemoryBreaker() throws IOException { + Directory dir = newDirectory(); + RandomIndexWriter w = new RandomIndexWriter(random(), dir); + + // we're indexing 100 documents with a field that is 48KB long so the fetch phase should check the memory breaker 5 times + // (every 22 documents that accumulate 1MiB in source sizes, and then a final time when we finished processing the one segment) + + String body = "{ \"thefield\": \" " + randomAlphaOfLength(48_000) + "\" }"; + for (int i = 0; i < 100; i++) { + Document document = new Document(); + document.add(new StringField("id", Integer.toString(i), Field.Store.YES)); + document.add(new StoredField("_source", new BytesRef(body))); + w.addDocument(document); + } + w.forceMerge(1); + IndexReader r = w.getReader(); + w.close(); + ContextIndexSearcher contextIndexSearcher = createSearcher(r); + AtomicInteger breakerCalledCount = new AtomicInteger(0); + NoopCircuitBreaker breakingCircuitBreaker = new NoopCircuitBreaker(CircuitBreaker.REQUEST) { + @Override + public void addEstimateBytesAndMaybeBreak(long bytes, String label) throws CircuitBreakingException { + breakerCalledCount.incrementAndGet(); + } + }; + try (SearchContext searchContext = createSearchContext(contextIndexSearcher, true, breakingCircuitBreaker)) { + FetchPhase fetchPhase = new FetchPhase(List.of(fetchContext -> new FetchSubPhaseProcessor() { + @Override + public void setNextReader(LeafReaderContext readerContext) throws IOException { + + } + + @Override + public void process(FetchSubPhase.HitContext hitContext) throws IOException { + Source source = hitContext.source(); + hitContext.hit().sourceRef(source.internalSourceRef()); + } + + @Override + public StoredFieldsSpec storedFieldsSpec() { + return StoredFieldsSpec.NEEDS_SOURCE; + } + })); + fetchPhase.execute(searchContext, IntStream.range(0, 100).toArray(), null); + assertThat(breakerCalledCount.get(), is(5)); + } finally { + r.close(); + dir.close(); + } + } + private static ContextIndexSearcher createSearcher(IndexReader reader) throws IOException { return new ContextIndexSearcher(reader, null, null, new QueryCachingPolicy() { @Override @@ -857,6 +917,14 @@ public StoredFieldsSpec storedFieldsSpec() { } private static SearchContext createSearchContext(ContextIndexSearcher contextIndexSearcher, boolean allowPartialResults) { + return createSearchContext(contextIndexSearcher, allowPartialResults, null); + } + + private static SearchContext createSearchContext( + ContextIndexSearcher contextIndexSearcher, + boolean allowPartialResults, + @Nullable CircuitBreaker circuitBreaker + ) { IndexSettings indexSettings = new IndexSettings( IndexMetadata.builder("index") .settings(Settings.builder().put(IndexMetadata.SETTING_VERSION_CREATED, IndexVersion.current())) @@ -929,6 +997,15 @@ public FetchSearchResult fetchResult() { public ShardSearchRequest request() { return request; } + + @Override + public CircuitBreaker circuitBreaker() { + if (circuitBreaker != null) { + return circuitBreaker; + } else { + return super.circuitBreaker(); + } + } }; searchContext.addReleasable(searchContext.fetchResult()::decRef); searchContext.setTask(new SearchShardTask(-1, "type", "action", "description", null, Collections.emptyMap())); From 1469c33b7927b3f5e25dee54250c2673085cd829 Mon Sep 17 00:00:00 2001 From: elasticsearchmachine Date: Sun, 23 Feb 2025 12:51:25 +0000 Subject: [PATCH 49/56] [CI] Auto commit changes from spotless --- .../org/elasticsearch/action/search/FetchSearchPhaseTests.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/test/java/org/elasticsearch/action/search/FetchSearchPhaseTests.java b/server/src/test/java/org/elasticsearch/action/search/FetchSearchPhaseTests.java index 8b54f67348a87..a0ad61879964f 100644 --- a/server/src/test/java/org/elasticsearch/action/search/FetchSearchPhaseTests.java +++ b/server/src/test/java/org/elasticsearch/action/search/FetchSearchPhaseTests.java @@ -917,7 +917,7 @@ public StoredFieldsSpec storedFieldsSpec() { } private static SearchContext createSearchContext(ContextIndexSearcher contextIndexSearcher, boolean allowPartialResults) { - return createSearchContext(contextIndexSearcher, allowPartialResults, null); + return createSearchContext(contextIndexSearcher, allowPartialResults, null); } private static SearchContext createSearchContext( From 39a0cc0edc0cb4465a019b60a4238c4a672e78bc Mon Sep 17 00:00:00 2001 From: Andrei Dan Date: Sun, 23 Feb 2025 12:54:30 +0000 Subject: [PATCH 50/56] Unit test the search context checkRealMemoryCB method --- .../search/DefaultSearchContextTests.java | 26 ++++++++++++++++++- 1 file changed, 25 insertions(+), 1 deletion(-) diff --git a/server/src/test/java/org/elasticsearch/search/DefaultSearchContextTests.java b/server/src/test/java/org/elasticsearch/search/DefaultSearchContextTests.java index a39dcc0a3175d..f0ba9f57e23ba 100644 --- a/server/src/test/java/org/elasticsearch/search/DefaultSearchContextTests.java +++ b/server/src/test/java/org/elasticsearch/search/DefaultSearchContextTests.java @@ -33,6 +33,8 @@ import org.elasticsearch.action.search.SearchType; import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.common.UUIDs; +import org.elasticsearch.common.breaker.CircuitBreaker; +import org.elasticsearch.common.breaker.NoopCircuitBreaker; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.EsExecutors; import org.elasticsearch.common.util.concurrent.ThreadContext; @@ -58,6 +60,7 @@ import org.elasticsearch.index.query.SearchExecutionContext; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.indices.breaker.CircuitBreakerService; import org.elasticsearch.indices.breaker.NoneCircuitBreakerService; import org.elasticsearch.search.aggregations.bucket.range.DateRangeAggregationBuilder; import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregationBuilder; @@ -972,6 +975,25 @@ public void testGetFieldCardinalityRuntimeField() { assertEquals(-1, DefaultSearchContext.getFieldCardinality("field", indexService, null)); } + public void testCheckRealMemoryCB() throws Exception { + IndexShard indexShard = null; + try (DefaultSearchContext context = createDefaultSearchContext(Settings.EMPTY)) { + indexShard = context.indexShard(); + // allocated more than the 1MiB buffer, local accounting not finished + assertThat(context.checkRealMemoryCB(1024 * 1800, false, "test"), is(true)); + // allocated more than the 1MiB buffer, local accounting finished + assertThat(context.checkRealMemoryCB(1024 * 1800, true, "test"), is(true)); + // allocated less than the 1MiB buffer, local accounting not finished + assertThat(context.checkRealMemoryCB(1024 * 5, false, "test"), is(false)); + // allocated less than the 1MiB buffer, local accounting finished + assertThat(context.checkRealMemoryCB(1024 * 5, true, "test"), is(true)); + } finally { + if (indexShard != null) { + indexShard.getThreadPool().shutdown(); + } + } + } + private DefaultSearchContext createDefaultSearchContext(Settings providedIndexSettings) throws IOException { return createDefaultSearchContext(providedIndexSettings, null); } @@ -999,7 +1021,9 @@ private DefaultSearchContext createDefaultSearchContext(Settings providedIndexSe SearchExecutionContext searchExecutionContext = mock(SearchExecutionContext.class); when(indexService.newSearchExecutionContext(eq(shardId.id()), eq(shardId.id()), any(), any(), nullable(String.class), any())) .thenReturn(searchExecutionContext); - + CircuitBreakerService breakerService = mock(CircuitBreakerService.class); + when(indexService.breakerService()).thenReturn(breakerService); + when(breakerService.getBreaker(anyString())).thenReturn(new NoopCircuitBreaker(CircuitBreaker.REQUEST)); IndexMetadata indexMetadata = IndexMetadata.builder("index").settings(settings).build(); IndexSettings indexSettings; MapperService mapperService; From abacdb16d580ba2eb0b11ca75546df50da420fb6 Mon Sep 17 00:00:00 2001 From: Andrei Dan Date: Sun, 23 Feb 2025 13:21:09 +0000 Subject: [PATCH 51/56] Only check the breaker at the end of the segment if the source is required --- .../main/java/org/elasticsearch/search/fetch/FetchPhase.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java b/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java index 0a7cd82e38179..e6654171fcb68 100644 --- a/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java +++ b/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java @@ -170,7 +170,8 @@ protected SearchHit nextDoc(int doc) throws IOException { throw new TaskCancelledException("cancelled"); } ++processedDocs; - if (context.checkRealMemoryCB(accumulatedBytesInLeaf, processedDocs == docsInLeaf, "fetch source")) { + // note that we only check the real memory breaker if we have a source to load + if (context.checkRealMemoryCB(accumulatedBytesInLeaf, (requiresSource && processedDocs == docsInLeaf), "fetch source")) { // if we checked the real memory breaker, we restart our local accounting accumulatedBytesInLeaf = 0; } From fab8b9bf8d5c2157e0913823214a1ab80f83c616 Mon Sep 17 00:00:00 2001 From: Andrei Dan Date: Mon, 24 Feb 2025 14:05:11 +0000 Subject: [PATCH 52/56] Account per fetch phase and add min value for the local acocunting buffer --- .../elasticsearch/search/SearchService.java | 2 ++ .../search/fetch/FetchPhase.java | 19 +++++++------------ .../action/search/FetchSearchPhaseTests.java | 5 ++++- 3 files changed, 13 insertions(+), 13 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/search/SearchService.java b/server/src/main/java/org/elasticsearch/search/SearchService.java index 2aa9829e9b132..b0554f12f6524 100644 --- a/server/src/main/java/org/elasticsearch/search/SearchService.java +++ b/server/src/main/java/org/elasticsearch/search/SearchService.java @@ -282,6 +282,8 @@ public class SearchService extends AbstractLifecycleComponent implements IndexEv public static final Setting MEMORY_ACCOUNTING_BUFFER_SIZE = Setting.byteSizeSetting( "search.memory_accounting_buffer_size", ByteSizeValue.of(1, ByteSizeUnit.MB), + ByteSizeValue.of(1, ByteSizeUnit.MB), + ByteSizeValue.ofBytes(Long.MAX_VALUE), Property.Dynamic, Property.NodeScope ); diff --git a/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java b/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java index e6654171fcb68..79354105d45df 100644 --- a/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java +++ b/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java @@ -83,6 +83,8 @@ public void execute(SearchContext context, int[] docIdsToLoad, RankDocShardInfo SearchHits hits = null; try { hits = buildSearchHits(context, docIdsToLoad, profiler, rankDocs); + // check the real memory breaker after finishing the fetch phase + context.checkRealMemoryCB(0, true, "fetch source"); } finally { // Always finish profiling ProfileResult profileResult = profiler.finish(); @@ -130,7 +132,7 @@ private SearchHits buildSearchHits(SearchContext context, int[] docIdsToLoad, Pr StoredFieldLoader storedFieldLoader = profiler.storedFields(StoredFieldLoader.fromSpec(storedFieldsSpec)); IdLoader idLoader = context.newIdLoader(); boolean requiresSource = storedFieldsSpec.requiresSource(); - + final int[] accumulatedBytesInLeaf = new int[1]; NestedDocuments nestedDocuments = context.getSearchExecutionContext().getNestedDocuments(); FetchPhaseDocsIterator docsIterator = new FetchPhaseDocsIterator() { @@ -140,17 +142,12 @@ private SearchHits buildSearchHits(SearchContext context, int[] docIdsToLoad, Pr LeafStoredFieldLoader leafStoredFieldLoader; SourceLoader.Leaf leafSourceLoader; IdLoader.Leaf leafIdLoader; - int accumulatedBytesInLeaf; - int docsInLeaf; - int processedDocs; + @Override protected void setNextReader(LeafReaderContext ctx, int[] docsInLeaf) throws IOException { Timer timer = profiler.startNextReader(); this.ctx = ctx; - this.accumulatedBytesInLeaf = 0; - this.docsInLeaf = docsInLeaf.length; - this.processedDocs = 0; this.leafNestedDocuments = nestedDocuments.getLeafNestedDocuments(ctx); this.leafStoredFieldLoader = storedFieldLoader.getLoader(ctx, docsInLeaf); this.leafSourceLoader = sourceLoader.leaf(ctx.reader(), docsInLeaf); @@ -169,11 +166,9 @@ protected SearchHit nextDoc(int doc) throws IOException { if (context.isCancelled()) { throw new TaskCancelledException("cancelled"); } - ++processedDocs; - // note that we only check the real memory breaker if we have a source to load - if (context.checkRealMemoryCB(accumulatedBytesInLeaf, (requiresSource && processedDocs == docsInLeaf), "fetch source")) { + if (context.checkRealMemoryCB(accumulatedBytesInLeaf[0], false, "fetch source")) { // if we checked the real memory breaker, we restart our local accounting - accumulatedBytesInLeaf = 0; + accumulatedBytesInLeaf[0] = 0; } HitContext hit = prepareHitContext( @@ -198,7 +193,7 @@ protected SearchHit nextDoc(int doc) throws IOException { BytesReference sourceRef = hit.hit().getSourceRef(); if (sourceRef != null) { - this.accumulatedBytesInLeaf += sourceRef.length(); + accumulatedBytesInLeaf[0] += sourceRef.length(); } success = true; return hit.hit(); diff --git a/server/src/test/java/org/elasticsearch/action/search/FetchSearchPhaseTests.java b/server/src/test/java/org/elasticsearch/action/search/FetchSearchPhaseTests.java index a0ad61879964f..dca70d06a21ea 100644 --- a/server/src/test/java/org/elasticsearch/action/search/FetchSearchPhaseTests.java +++ b/server/src/test/java/org/elasticsearch/action/search/FetchSearchPhaseTests.java @@ -843,7 +843,10 @@ public void testFetchPhaseChecksMemoryBreaker() throws IOException { document.add(new StoredField("_source", new BytesRef(body))); w.addDocument(document); } - w.forceMerge(1); + // we account per fetch phase so it doesn't matter if it's one or multiple segments, so let's test both + if (randomBoolean()) { + w.forceMerge(1); + } IndexReader r = w.getReader(); w.close(); ContextIndexSearcher contextIndexSearcher = createSearcher(r); From 01f4c90cb5b65bd9c361b67ea8ccce18e1d1ec1d Mon Sep 17 00:00:00 2001 From: Andrei Dan Date: Mon, 24 Feb 2025 14:06:28 +0000 Subject: [PATCH 53/56] Spotless --- .../src/main/java/org/elasticsearch/search/fetch/FetchPhase.java | 1 - 1 file changed, 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java b/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java index 79354105d45df..878791bc6a78d 100644 --- a/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java +++ b/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java @@ -143,7 +143,6 @@ private SearchHits buildSearchHits(SearchContext context, int[] docIdsToLoad, Pr SourceLoader.Leaf leafSourceLoader; IdLoader.Leaf leafIdLoader; - @Override protected void setNextReader(LeafReaderContext ctx, int[] docsInLeaf) throws IOException { Timer timer = profiler.startNextReader(); From 0be02e5dba32c4fd539d27b69ab320aa101a21f0 Mon Sep 17 00:00:00 2001 From: Andrei Dan Date: Mon, 24 Feb 2025 14:13:25 +0000 Subject: [PATCH 54/56] Drop the flag on finalizing local accounting --- .../org/elasticsearch/search/fetch/FetchPhase.java | 4 +--- .../elasticsearch/search/internal/SearchContext.java | 6 ++---- .../action/search/FetchSearchPhaseTests.java | 8 +++++--- .../search/DefaultSearchContextTests.java | 12 ++++-------- 4 files changed, 12 insertions(+), 18 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java b/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java index 878791bc6a78d..681b90330ac65 100644 --- a/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java +++ b/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java @@ -83,8 +83,6 @@ public void execute(SearchContext context, int[] docIdsToLoad, RankDocShardInfo SearchHits hits = null; try { hits = buildSearchHits(context, docIdsToLoad, profiler, rankDocs); - // check the real memory breaker after finishing the fetch phase - context.checkRealMemoryCB(0, true, "fetch source"); } finally { // Always finish profiling ProfileResult profileResult = profiler.finish(); @@ -165,7 +163,7 @@ protected SearchHit nextDoc(int doc) throws IOException { if (context.isCancelled()) { throw new TaskCancelledException("cancelled"); } - if (context.checkRealMemoryCB(accumulatedBytesInLeaf[0], false, "fetch source")) { + if (context.checkRealMemoryCB(accumulatedBytesInLeaf[0], "fetch source")) { // if we checked the real memory breaker, we restart our local accounting accumulatedBytesInLeaf[0] = 0; } 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 ad7bb7a01bd7c..d5e26b9c91a8d 100644 --- a/server/src/main/java/org/elasticsearch/search/internal/SearchContext.java +++ b/server/src/main/java/org/elasticsearch/search/internal/SearchContext.java @@ -383,14 +383,12 @@ public Query rewrittenQuery() { /** * Checks if the accumulated bytes are greater than the buffer size and if so, checks the available memory in the parent breaker * (the real memory breaker). - * It also checks the available memory if the caller indicates that the local accounting is finished. * @param locallyAccumulatedBytes the number of bytes accumulated locally - * @param localAccountingFinished if the local accounting is finished * @param label the label to use in the breaker * @return true if the real memory breaker is called and false otherwise */ - public final boolean checkRealMemoryCB(int locallyAccumulatedBytes, boolean localAccountingFinished, String label) { - if (locallyAccumulatedBytes >= memAccountingBufferSize() || localAccountingFinished) { + public final boolean checkRealMemoryCB(int locallyAccumulatedBytes, String label) { + if (locallyAccumulatedBytes >= memAccountingBufferSize()) { circuitBreaker().addEstimateBytesAndMaybeBreak(0, label); return true; } diff --git a/server/src/test/java/org/elasticsearch/action/search/FetchSearchPhaseTests.java b/server/src/test/java/org/elasticsearch/action/search/FetchSearchPhaseTests.java index dca70d06a21ea..53095010bffb4 100644 --- a/server/src/test/java/org/elasticsearch/action/search/FetchSearchPhaseTests.java +++ b/server/src/test/java/org/elasticsearch/action/search/FetchSearchPhaseTests.java @@ -833,8 +833,10 @@ public void testFetchPhaseChecksMemoryBreaker() throws IOException { Directory dir = newDirectory(); RandomIndexWriter w = new RandomIndexWriter(random(), dir); - // we're indexing 100 documents with a field that is 48KB long so the fetch phase should check the memory breaker 5 times - // (every 22 documents that accumulate 1MiB in source sizes, and then a final time when we finished processing the one segment) + // we're indexing 100 documents with a field that is 48KB long so the fetch phase should check the memory breaker 4 times + // (every 22 documents that accumulate 1MiB in source sizes, so we'll have 4 checks at roughly 4.1MiB and the last 12 documents will + // not + // accumulate 1MiB anymore so won't check the breaker anymore) String body = "{ \"thefield\": \" " + randomAlphaOfLength(48_000) + "\" }"; for (int i = 0; i < 100; i++) { @@ -876,7 +878,7 @@ public StoredFieldsSpec storedFieldsSpec() { } })); fetchPhase.execute(searchContext, IntStream.range(0, 100).toArray(), null); - assertThat(breakerCalledCount.get(), is(5)); + assertThat(breakerCalledCount.get(), is(4)); } finally { r.close(); dir.close(); diff --git a/server/src/test/java/org/elasticsearch/search/DefaultSearchContextTests.java b/server/src/test/java/org/elasticsearch/search/DefaultSearchContextTests.java index f0ba9f57e23ba..335815fcea445 100644 --- a/server/src/test/java/org/elasticsearch/search/DefaultSearchContextTests.java +++ b/server/src/test/java/org/elasticsearch/search/DefaultSearchContextTests.java @@ -979,14 +979,10 @@ public void testCheckRealMemoryCB() throws Exception { IndexShard indexShard = null; try (DefaultSearchContext context = createDefaultSearchContext(Settings.EMPTY)) { indexShard = context.indexShard(); - // allocated more than the 1MiB buffer, local accounting not finished - assertThat(context.checkRealMemoryCB(1024 * 1800, false, "test"), is(true)); - // allocated more than the 1MiB buffer, local accounting finished - assertThat(context.checkRealMemoryCB(1024 * 1800, true, "test"), is(true)); - // allocated less than the 1MiB buffer, local accounting not finished - assertThat(context.checkRealMemoryCB(1024 * 5, false, "test"), is(false)); - // allocated less than the 1MiB buffer, local accounting finished - assertThat(context.checkRealMemoryCB(1024 * 5, true, "test"), is(true)); + // allocated more than the 1MiB buffer + assertThat(context.checkRealMemoryCB(1024 * 1800, "test"), is(true)); + // allocated less than the 1MiB buffer + assertThat(context.checkRealMemoryCB(1024 * 5, "test"), is(false)); } finally { if (indexShard != null) { indexShard.getThreadPool().shutdown(); From cff3ad8fc6fabd87b9c34beeeb141b6271d01fe4 Mon Sep 17 00:00:00 2001 From: Andrei Dan Date: Mon, 24 Feb 2025 14:23:36 +0000 Subject: [PATCH 55/56] Drop leftover --- .../main/java/org/elasticsearch/search/fetch/FetchPhase.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java b/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java index 681b90330ac65..192c15cf92187 100644 --- a/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java +++ b/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java @@ -302,7 +302,7 @@ private static HitContext prepareNonNestedHitContext( Source source = Source.lazy(lazyStoredSourceLoader(profiler, subReaderContext, subDocId)); return new HitContext(hit, subReaderContext, subDocId, Map.of(), source, rankDoc); } else { - SearchHit hit = new SearchHit(docId, id, null); + SearchHit hit = new SearchHit(docId, id); Source source; if (requiresSource) { Timer timer = profiler.startLoadingSource(); From 13715a5d400ba69a6da50b062a183d526442e391 Mon Sep 17 00:00:00 2001 From: Andrei Dan Date: Mon, 24 Feb 2025 15:09:17 +0000 Subject: [PATCH 56/56] Renamings --- .../main/java/org/elasticsearch/search/SearchService.java | 2 +- .../java/org/elasticsearch/search/fetch/FetchPhase.java | 8 ++++---- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/search/SearchService.java b/server/src/main/java/org/elasticsearch/search/SearchService.java index b0554f12f6524..855e099a57d3f 100644 --- a/server/src/main/java/org/elasticsearch/search/SearchService.java +++ b/server/src/main/java/org/elasticsearch/search/SearchService.java @@ -276,7 +276,7 @@ public class SearchService extends AbstractLifecycleComponent implements IndexEv /** * The size of the buffer used for memory accounting. - * This buffer is used to locally track the memory accummulate during the executiong of + * This buffer is used to locally track the memory accummulated during the execution of * a search request before submitting the accumulated value to the circuit breaker. */ public static final Setting MEMORY_ACCOUNTING_BUFFER_SIZE = Setting.byteSizeSetting( diff --git a/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java b/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java index 192c15cf92187..dbf11ece9e82a 100644 --- a/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java +++ b/server/src/main/java/org/elasticsearch/search/fetch/FetchPhase.java @@ -130,7 +130,7 @@ private SearchHits buildSearchHits(SearchContext context, int[] docIdsToLoad, Pr StoredFieldLoader storedFieldLoader = profiler.storedFields(StoredFieldLoader.fromSpec(storedFieldsSpec)); IdLoader idLoader = context.newIdLoader(); boolean requiresSource = storedFieldsSpec.requiresSource(); - final int[] accumulatedBytesInLeaf = new int[1]; + final int[] locallyAccumulatedBytes = new int[1]; NestedDocuments nestedDocuments = context.getSearchExecutionContext().getNestedDocuments(); FetchPhaseDocsIterator docsIterator = new FetchPhaseDocsIterator() { @@ -163,9 +163,9 @@ protected SearchHit nextDoc(int doc) throws IOException { if (context.isCancelled()) { throw new TaskCancelledException("cancelled"); } - if (context.checkRealMemoryCB(accumulatedBytesInLeaf[0], "fetch source")) { + if (context.checkRealMemoryCB(locallyAccumulatedBytes[0], "fetch source")) { // if we checked the real memory breaker, we restart our local accounting - accumulatedBytesInLeaf[0] = 0; + locallyAccumulatedBytes[0] = 0; } HitContext hit = prepareHitContext( @@ -190,7 +190,7 @@ protected SearchHit nextDoc(int doc) throws IOException { BytesReference sourceRef = hit.hit().getSourceRef(); if (sourceRef != null) { - accumulatedBytesInLeaf[0] += sourceRef.length(); + locallyAccumulatedBytes[0] += sourceRef.length(); } success = true; return hit.hit();