Skip to content

Commit 51f9542

Browse files
authored
Adds the ability to acquire readers in IndexShard (#54966)
This change adds the ability to acquire a point in time reader on an engine. This is needed for frozen indices that lazily loads the reader on every phase of the search requests. Acquiring a reader on a frozen index ensures that the engine will not be closed until the reader is released, leaving the directory reader unopened until a call to acquire a searcher is made. When the searcher is closed, the underlyinng directory reader is also closed unless another requests on the same frozen shard is in-flight. This ensures that the directory reader of frozen indices is opened only when requests are executed (they consume a thread in the search throttled pool).
1 parent 15f18b1 commit 51f9542

File tree

26 files changed

+847
-1014
lines changed

26 files changed

+847
-1014
lines changed

server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java

+2-2
Original file line numberDiff line numberDiff line change
@@ -374,9 +374,9 @@ public static void registerRequestHandler(TransportService transportService, Sea
374374
transportService.registerRequestHandler(SHARD_OPEN_READER_NAME, ThreadPool.Names.SAME,
375375
TransportOpenReaderAction.ShardOpenReaderRequest::new,
376376
(request, channel, task) -> {
377-
searchService.openReaderContext(request.searchShardTarget.getShardId(), request.keepAlive,
377+
searchService.openReaderContext(request.getShardId(), request.keepAlive,
378378
ActionListener.map(new ChannelActionListener<>(channel, SHARD_OPEN_READER_NAME, request),
379-
contextId -> new TransportOpenReaderAction.ShardOpenReaderResponse(contextId, request.searchShardTarget)));
379+
contextId -> new TransportOpenReaderAction.ShardOpenReaderResponse(contextId)));
380380
});
381381
TransportActionProxy.registerProxyAction(
382382
transportService, SHARD_OPEN_READER_NAME, TransportOpenReaderAction.ShardOpenReaderResponse::new);

server/src/main/java/org/elasticsearch/action/search/TransportOpenReaderAction.java

+20-10
Original file line numberDiff line numberDiff line change
@@ -23,6 +23,7 @@
2323
import org.elasticsearch.action.ActionListener;
2424
import org.elasticsearch.action.ActionType;
2525
import org.elasticsearch.action.IndicesRequest;
26+
import org.elasticsearch.action.OriginalIndices;
2627
import org.elasticsearch.action.support.ActionFilters;
2728
import org.elasticsearch.action.support.HandledTransportAction;
2829
import org.elasticsearch.action.support.IndicesOptions;
@@ -34,6 +35,7 @@
3435
import org.elasticsearch.common.io.stream.StreamOutput;
3536
import org.elasticsearch.common.unit.TimeValue;
3637
import org.elasticsearch.common.util.concurrent.AtomicArray;
38+
import org.elasticsearch.index.shard.ShardId;
3739
import org.elasticsearch.search.SearchPhaseResult;
3840
import org.elasticsearch.search.SearchShardTarget;
3941
import org.elasticsearch.search.internal.AliasFilter;
@@ -93,42 +95,49 @@ public AbstractSearchAsyncAction<? extends SearchPhaseResult> asyncSearchAction(
9395
}
9496

9597
static final class ShardOpenReaderRequest extends TransportRequest implements IndicesRequest {
96-
final SearchShardTarget searchShardTarget;
98+
final ShardId shardId;
99+
final OriginalIndices originalIndices;
97100
final TimeValue keepAlive;
98101

99-
ShardOpenReaderRequest(SearchShardTarget searchShardTarget, TimeValue keepAlive) {
100-
this.searchShardTarget = searchShardTarget;
102+
ShardOpenReaderRequest(ShardId shardId, OriginalIndices originalIndices, TimeValue keepAlive) {
103+
this.shardId = shardId;
104+
this.originalIndices = originalIndices;
101105
this.keepAlive = keepAlive;
102106
}
103107

104108
ShardOpenReaderRequest(StreamInput in) throws IOException {
105109
super(in);
106-
searchShardTarget = new SearchShardTarget(in);
110+
shardId = new ShardId(in);
111+
originalIndices = OriginalIndices.readOriginalIndices(in);
107112
keepAlive = in.readTimeValue();
108113
}
109114

110115
@Override
111116
public void writeTo(StreamOutput out) throws IOException {
112117
super.writeTo(out);
113-
searchShardTarget.writeTo(out);
118+
shardId.writeTo(out);
119+
OriginalIndices.writeOriginalIndices(originalIndices, out);
114120
out.writeTimeValue(keepAlive);
115121
}
116122

123+
public ShardId getShardId() {
124+
return shardId;
125+
}
126+
117127
@Override
118128
public String[] indices() {
119-
return searchShardTarget.getOriginalIndices().indices();
129+
return originalIndices.indices();
120130
}
121131

122132
@Override
123133
public IndicesOptions indicesOptions() {
124-
return searchShardTarget.getOriginalIndices().indicesOptions();
134+
return originalIndices.indicesOptions();
125135
}
126136
}
127137

128138
static final class ShardOpenReaderResponse extends SearchPhaseResult {
129-
ShardOpenReaderResponse(SearchContextId contextId, SearchShardTarget searchShardTarget) {
139+
ShardOpenReaderResponse(SearchContextId contextId) {
130140
this.contextId = contextId;
131-
setSearchShardTarget(searchShardTarget);
132141
}
133142

134143
ShardOpenReaderResponse(StreamInput in) throws IOException {
@@ -163,7 +172,8 @@ protected void executePhaseOnShard(SearchShardIterator shardIt, ShardRouting sha
163172
SearchActionListener<SearchPhaseResult> listener) {
164173
final Transport.Connection connection = getConnection(shardIt.getClusterAlias(), shard.currentNodeId());
165174
final SearchShardTarget searchShardTarget = shardIt.newSearchShardTarget(shard.currentNodeId());
166-
final ShardOpenReaderRequest shardRequest = new ShardOpenReaderRequest(searchShardTarget, openReaderRequest.keepAlive());
175+
final ShardOpenReaderRequest shardRequest = new ShardOpenReaderRequest(searchShardTarget.getShardId(),
176+
searchShardTarget.getOriginalIndices(), openReaderRequest.keepAlive());
167177
getSearchTransport().sendShardOpenReader(connection, getTask(), shardRequest, ActionListener.map(listener, r -> r));
168178
}
169179

server/src/main/java/org/elasticsearch/index/engine/Engine.java

+70-39
Original file line numberDiff line numberDiff line change
@@ -95,6 +95,7 @@
9595
import java.util.concurrent.locks.ReentrantLock;
9696
import java.util.concurrent.locks.ReentrantReadWriteLock;
9797
import java.util.function.BiFunction;
98+
import java.util.function.Function;
9899
import java.util.stream.Stream;
99100

100101
import static org.elasticsearch.index.seqno.SequenceNumbers.UNASSIGNED_PRIMARY_TERM;
@@ -588,31 +589,17 @@ protected final GetResult getFromSearcher(Get get, BiFunction<String, SearcherSc
588589

589590
public abstract GetResult get(Get get, BiFunction<String, SearcherScope, Searcher> searcherFactory) throws EngineException;
590591

591-
592592
/**
593-
* Returns a new searcher instance. The consumer of this
594-
* API is responsible for releasing the returned searcher in a
595-
* safe manner, preferably in a try/finally block.
596-
*
597-
* @param source the source API or routing that triggers this searcher acquire
598-
*
599-
* @see Searcher#close()
593+
* Acquires a point-in-time reader that can be used to create {@link Engine.Searcher}s on demand.
600594
*/
601-
public final Searcher acquireSearcher(String source) throws EngineException {
602-
return acquireSearcher(source, SearcherScope.EXTERNAL);
595+
public final SearcherSupplier acquireSearcherSupplier(Function<Searcher, Searcher> wrapper) throws EngineException {
596+
return acquireSearcherSupplier(wrapper, SearcherScope.EXTERNAL);
603597
}
604598

605599
/**
606-
* Returns a new searcher instance. The consumer of this
607-
* API is responsible for releasing the returned searcher in a
608-
* safe manner, preferably in a try/finally block.
609-
*
610-
* @param source the source API or routing that triggers this searcher acquire
611-
* @param scope the scope of this searcher ie. if the searcher will be used for get or search purposes
612-
*
613-
* @see Searcher#close()
600+
* Acquires a point-in-time reader that can be used to create {@link Engine.Searcher}s on demand.
614601
*/
615-
public Searcher acquireSearcher(String source, SearcherScope scope) throws EngineException {
602+
public SearcherSupplier acquireSearcherSupplier(Function<Searcher, Searcher> wrapper, SearcherScope scope) throws EngineException {
616603
/* Acquire order here is store -> manager since we need
617604
* to make sure that the store is not closed before
618605
* the searcher is acquired. */
@@ -621,35 +608,64 @@ public Searcher acquireSearcher(String source, SearcherScope scope) throws Engin
621608
}
622609
Releasable releasable = store::decRef;
623610
try {
624-
assert assertSearcherIsWarmedUp(source, scope);
625611
ReferenceManager<ElasticsearchDirectoryReader> referenceManager = getReferenceManager(scope);
626-
final ElasticsearchDirectoryReader acquire = referenceManager.acquire();
612+
ElasticsearchDirectoryReader acquire = referenceManager.acquire();
627613
AtomicBoolean released = new AtomicBoolean(false);
628-
Searcher engineSearcher = new Searcher(source, acquire,
629-
engineConfig.getSimilarity(), engineConfig.getQueryCache(), engineConfig.getQueryCachingPolicy(),
630-
() -> {
631-
if (released.compareAndSet(false, true)) {
632-
try {
633-
referenceManager.release(acquire);
634-
} finally {
635-
store.decRef();
614+
SearcherSupplier reader = new SearcherSupplier(wrapper) {
615+
@Override
616+
public Searcher acquireSearcherInternal(String source) {
617+
assert assertSearcherIsWarmedUp(source, scope);
618+
return new Searcher(source, acquire, engineConfig.getSimilarity(), engineConfig.getQueryCache(),
619+
engineConfig.getQueryCachingPolicy(), () -> {});
620+
}
621+
622+
@Override
623+
public void close() {
624+
if (released.compareAndSet(false, true)) {
625+
try {
626+
referenceManager.release(acquire);
627+
} catch (IOException e) {
628+
throw new UncheckedIOException("failed to close", e);
629+
} catch (AlreadyClosedException e) {
630+
// This means there's a bug somewhere: don't suppress it
631+
throw new AssertionError(e);
632+
} finally {
633+
store.decRef();
634+
}
635+
} else {
636+
/* In general, readers should never be released twice or this would break reference counting. There is one rare case
637+
* when it might happen though: when the request and the Reaper thread would both try to release it in a very short
638+
* amount of time, this is why we only log a warning instead of throwing an exception. */
639+
logger.warn("Reader was released twice", new IllegalStateException("Double release"));
636640
}
637-
} else {
638-
/* In general, readers should never be released twice or this would break reference counting. There is one rare case
639-
* when it might happen though: when the request and the Reaper thread would both try to release it in a very short
640-
* amount of time, this is why we only log a warning instead of throwing an exception. */
641-
logger.warn("Searcher was released twice", new IllegalStateException("Double release"));
642641
}
643-
});
642+
};
644643
releasable = null; // success - hand over the reference to the engine reader
645-
return engineSearcher;
644+
return reader;
646645
} catch (AlreadyClosedException ex) {
647646
throw ex;
648647
} catch (Exception ex) {
649-
maybeFailEngine("acquire_searcher", ex);
648+
maybeFailEngine("acquire_reader", ex);
650649
ensureOpen(ex); // throw EngineCloseException here if we are already closed
651-
logger.error(() -> new ParameterizedMessage("failed to acquire searcher, source {}", source), ex);
652-
throw new EngineException(shardId, "failed to acquire searcher, source " + source, ex);
650+
logger.error(() -> new ParameterizedMessage("failed to acquire reader"), ex);
651+
throw new EngineException(shardId, "failed to acquire reader", ex);
652+
} finally {
653+
Releasables.close(releasable);
654+
}
655+
}
656+
657+
public final Searcher acquireSearcher(String source) throws EngineException {
658+
return acquireSearcher(source, SearcherScope.EXTERNAL);
659+
}
660+
661+
public Searcher acquireSearcher(String source, SearcherScope scope) throws EngineException {
662+
SearcherSupplier releasable = null;
663+
try {
664+
SearcherSupplier reader = releasable = acquireSearcherSupplier(Function.identity(), scope);
665+
Searcher searcher = reader.acquireSearcher(source);
666+
releasable = null;
667+
return new Searcher(source, searcher.getDirectoryReader(), searcher.getSimilarity(),
668+
searcher.getQueryCache(), searcher.getQueryCachingPolicy(), () -> Releasables.close(searcher, reader));
653669
} finally {
654670
Releasables.close(releasable);
655671
}
@@ -1158,6 +1174,21 @@ default void onFailedEngine(String reason, @Nullable Exception e) {
11581174
}
11591175
}
11601176

1177+
public abstract static class SearcherSupplier implements Releasable {
1178+
private final Function<Searcher, Searcher> wrapper;
1179+
1180+
public SearcherSupplier(Function<Searcher, Searcher> wrapper) {
1181+
this.wrapper = wrapper;
1182+
}
1183+
1184+
public final Searcher acquireSearcher(String source) {
1185+
final Searcher searcher = acquireSearcherInternal(source);
1186+
return "can_match".equals(source) ? searcher : wrapper.apply(searcher);
1187+
}
1188+
1189+
protected abstract Searcher acquireSearcherInternal(String source);
1190+
}
1191+
11611192
public static final class Searcher extends IndexSearcher implements Releasable {
11621193
private final String source;
11631194
private final Closeable onClose;

server/src/main/java/org/elasticsearch/index/search/stats/ShardSearchStats.java

+3-4
Original file line numberDiff line numberDiff line change
@@ -25,7 +25,6 @@
2525
import org.elasticsearch.common.util.Maps;
2626
import org.elasticsearch.index.shard.SearchOperationListener;
2727
import org.elasticsearch.search.internal.ReaderContext;
28-
import org.elasticsearch.search.internal.ScrollContext;
2928
import org.elasticsearch.search.internal.SearchContext;
3029

3130
import java.util.HashMap;
@@ -158,15 +157,15 @@ public void onFreeReaderContext(ReaderContext readerContext) {
158157
}
159158

160159
@Override
161-
public void onNewScrollContext(ScrollContext scrollContext) {
160+
public void onNewScrollContext(ReaderContext readerContext) {
162161
totalStats.scrollCurrent.inc();
163162
}
164163

165164
@Override
166-
public void onFreeScrollContext(ScrollContext scrollContext) {
165+
public void onFreeScrollContext(ReaderContext readerContext) {
167166
totalStats.scrollCurrent.dec();
168167
assert totalStats.scrollCurrent.count() >= 0;
169-
totalStats.scrollMetric.inc(TimeUnit.NANOSECONDS.toMicros(System.nanoTime() - scrollContext.getStartTimeInNano()));
168+
totalStats.scrollMetric.inc(TimeUnit.NANOSECONDS.toMicros(System.nanoTime() - readerContext.getStartTimeInNano()));
170169
}
171170

172171
static final class StatsHolder {

server/src/main/java/org/elasticsearch/index/shard/IndexShard.java

+12-4
Original file line numberDiff line numberDiff line change
@@ -1197,12 +1197,20 @@ public void failShard(String reason, @Nullable Exception e) {
11971197
}
11981198

11991199
/**
1200-
* Acquire a lightweight searcher which can be used to rewrite shard search requests.
1200+
* Acquires a point-in-time reader that can be used to create {@link Engine.Searcher}s on demand.
12011201
*/
1202-
public Engine.Searcher acquireCanMatchSearcher() {
1202+
public Engine.SearcherSupplier acquireSearcherSupplier() {
1203+
return acquireSearcherSupplier(Engine.SearcherScope.EXTERNAL);
1204+
}
1205+
1206+
/**
1207+
* Acquires a point-in-time reader that can be used to create {@link Engine.Searcher}s on demand.
1208+
*/
1209+
public Engine.SearcherSupplier acquireSearcherSupplier(Engine.SearcherScope scope) {
12031210
readAllowed();
12041211
markSearcherAccessed();
1205-
return getEngine().acquireSearcher("can_match", Engine.SearcherScope.EXTERNAL);
1212+
final Engine engine = getEngine();
1213+
return engine.acquireSearcherSupplier(this::wrapSearcher, scope);
12061214
}
12071215

12081216
public Engine.Searcher acquireSearcher(String source) {
@@ -1218,7 +1226,7 @@ private Engine.Searcher acquireSearcher(String source, Engine.SearcherScope scop
12181226
markSearcherAccessed();
12191227
final Engine engine = getEngine();
12201228
final Engine.Searcher searcher = engine.acquireSearcher(source, scope);
1221-
return wrapSearcher(searcher);
1229+
return "can_match".equals(source) ? searcher : wrapSearcher(searcher);
12221230
}
12231231

12241232
private Engine.Searcher wrapSearcher(Engine.Searcher searcher) {

server/src/main/java/org/elasticsearch/index/shard/SearchOperationListener.java

+14-14
Original file line numberDiff line numberDiff line change
@@ -22,7 +22,6 @@
2222
import org.apache.logging.log4j.message.ParameterizedMessage;
2323
import org.elasticsearch.ExceptionsHelper;
2424
import org.elasticsearch.search.internal.ReaderContext;
25-
import org.elasticsearch.search.internal.ScrollContext;
2625
import org.elasticsearch.search.internal.SearchContext;
2726
import org.elasticsearch.transport.TransportRequest;
2827

@@ -93,28 +92,29 @@ default void onNewReaderContext(ReaderContext readerContext) {}
9392
default void onFreeReaderContext(ReaderContext readerContext) {}
9493

9594
/**
96-
* Executed when a new scroll search {@link SearchContext} was created
97-
* @param scrollContext the created search context
95+
* Executed when a new scroll search {@link ReaderContext} was created
96+
* @param readerContext the created reader context
9897
*/
99-
default void onNewScrollContext(ScrollContext scrollContext) {}
98+
default void onNewScrollContext(ReaderContext readerContext) {}
10099

101100
/**
102101
* Executed when a scroll search {@link SearchContext} is freed.
103102
* This happens either when the scroll search execution finishes, if the
104103
* execution failed or if the search context as idle for and needs to be
105104
* cleaned up.
106-
* @param scrollContext the freed search context
105+
* @param readerContext the freed search context
107106
*/
108-
default void onFreeScrollContext(ScrollContext scrollContext) {}
107+
default void onFreeScrollContext(ReaderContext readerContext) {}
109108

110109
/**
111110
* Executed prior to using a {@link SearchContext} that has been retrieved
112111
* from the active contexts. If the context is deemed invalid a runtime
113112
* exception can be thrown, which will prevent the context from being used.
114-
* @param context the context retrieved from the active contexts
113+
* @param readerContext The reader context used by this request.
114+
* @param searchContext The newly created {@link SearchContext}.
115115
* @param transportRequest the request that is going to use the search context
116116
*/
117-
default void validateSearchContext(SearchContext context, TransportRequest transportRequest) {}
117+
default void validateSearchContext(ReaderContext readerContext, SearchContext searchContext, TransportRequest transportRequest) {}
118118

119119
/**
120120
* Executed when a search context was freed. The implementor can implement
@@ -225,33 +225,33 @@ public void onFreeReaderContext(ReaderContext readerContext) {
225225
}
226226

227227
@Override
228-
public void onNewScrollContext(ScrollContext scrollContext) {
228+
public void onNewScrollContext(ReaderContext readerContext) {
229229
for (SearchOperationListener listener : listeners) {
230230
try {
231-
listener.onNewScrollContext(scrollContext);
231+
listener.onNewScrollContext(readerContext);
232232
} catch (Exception e) {
233233
logger.warn(() -> new ParameterizedMessage("onNewScrollContext listener [{}] failed", listener), e);
234234
}
235235
}
236236
}
237237

238238
@Override
239-
public void onFreeScrollContext(ScrollContext scrollContext) {
239+
public void onFreeScrollContext(ReaderContext readerContext) {
240240
for (SearchOperationListener listener : listeners) {
241241
try {
242-
listener.onFreeScrollContext(scrollContext);
242+
listener.onFreeScrollContext(readerContext);
243243
} catch (Exception e) {
244244
logger.warn(() -> new ParameterizedMessage("onFreeScrollContext listener [{}] failed", listener), e);
245245
}
246246
}
247247
}
248248

249249
@Override
250-
public void validateSearchContext(SearchContext context, TransportRequest request) {
250+
public void validateSearchContext(ReaderContext readerContext, SearchContext searchContext, TransportRequest request) {
251251
Exception exception = null;
252252
for (SearchOperationListener listener : listeners) {
253253
try {
254-
listener.validateSearchContext(context, request);
254+
listener.validateSearchContext(readerContext, searchContext, request);
255255
} catch (Exception e) {
256256
exception = ExceptionsHelper.useOrSuppress(exception, e);
257257
}

0 commit comments

Comments
 (0)