Skip to content

Commit 879279c

Browse files
dnhatnjimczi
andauthored
Introduce point in time APIs in x-pack basic (#61062)
This commit introduces a new API that manages point-in-times in x-pack basic. Elasticsearch pit (point in time) is a lightweight view into the state of the data as it existed when initiated. A search request by default executes against the most recent point in time. In some cases, it is preferred to perform multiple search requests using the same point in time. For example, if refreshes happen between search_after requests, then the results of those requests might not be consistent as changes happening between searches are only visible to the more recent point in time. A point in time must be opened before being used in search requests. The `keep_alive` parameter tells Elasticsearch how long it should keep a point in time around. ``` POST /my_index/_pit?keep_alive=1m ``` The response from the above request includes a `id`, which should be passed to the `id` of the `pit` parameter of search requests. ``` POST /_search { "query": { "match" : { "title" : "elasticsearch" } }, "pit": { "id": "46ToAwMDaWR4BXV1aWQxAgZub2RlXzEAAAAAAAAAAAEBYQNpZHkFdXVpZDIrBm5vZGVfMwAAAAAAAAAAKgFjA2lkeQV1dWlkMioGbm9kZV8yAAAAAAAAAAAMAWICBXV1aWQyAAAFdXVpZDEAAQltYXRjaF9hbGw_gAAAAA==", "keep_alive": "1m" } } ``` Point-in-times are automatically closed when the `keep_alive` is elapsed. However, keeping point-in-times has a cost; hence, point-in-times should be closed as soon as they are no longer used in search requests. ``` DELETE /_pit { "id" : "46ToAwMDaWR4BXV1aWQxAgZub2RlXzEAAAAAAAAAAAEBYQNpZHkFdXVpZDIrBm5vZGVfMwAAAAAAAAAAKgFjA2lkeQV1dWlkMioGbm9kZV8yAAAAAAAAAAAMAWIBBXV1aWQyAAA=" } ``` #### Notable works in this change: - Move the search state to the coordinating node: #52741 - Allow searches with a specific reader context: #53989 - Add the ability to acquire readers in IndexShard: #54966 Relates #46523 Relates #26472 Co-authored-by: Jim Ferenczi <[email protected]>
1 parent fc50e17 commit 879279c

File tree

121 files changed

+4366
-2116
lines changed

Some content is hidden

Large Commits have some content hidden by default. Use the searchbox below for content that may be hidden.

121 files changed

+4366
-2116
lines changed
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,116 @@
1+
[role="xpack"]
2+
[testenv="basic"]
3+
[[point-in-time]]
4+
==== Point in time
5+
6+
A search request by default executes against the most recent visible data of
7+
the target indices, which is called point in time. Elasticsearch pit (point in time)
8+
is a lightweight view into the state of the data as it existed when initiated.
9+
In some cases, it's preferred to perform multiple search requests using
10+
the same point in time. For example, if <<indices-refresh,refreshes>> happen between
11+
search_after requests, then the results of those requests might not be consistent as
12+
changes happening between searches are only visible to the more recent point in time.
13+
14+
A point in time must be opened explicitly before being used in search requests. The
15+
keep_alive parameter tells Elasticsearch how long it should keep a point in time alive,
16+
e.g. `?keep_alive=5m`.
17+
18+
[source,console]
19+
--------------------------------------------------
20+
POST /my-index-000001/_pit?keep_alive=1m
21+
--------------------------------------------------
22+
// TEST[setup:my_index]
23+
24+
The result from the above request includes a `id`, which should
25+
be passed to the `id` of the `pit` parameter of a search request.
26+
27+
[source,console]
28+
--------------------------------------------------
29+
POST /_search <1>
30+
{
31+
"size": 100,
32+
"query": {
33+
"match" : {
34+
"title" : "elasticsearch"
35+
}
36+
},
37+
"pit": {
38+
"id": "46ToAwMDaWR4BXV1aWQxAgZub2RlXzEAAAAAAAAAAAEBYQNpZHkFdXVpZDIrBm5vZGVfMwAAAAAAAAAAKgFjA2lkeQV1dWlkMioGbm9kZV8yAAAAAAAAAAAMAWICBXV1aWQyAAAFdXVpZDEAAQltYXRjaF9hbGw_gAAAAA==", <2>
39+
"keep_alive": "1m" <3>
40+
}
41+
}
42+
--------------------------------------------------
43+
// TEST[catch:missing]
44+
45+
<1> A search request with the `pit` parameter must not specify `index`, `routing`,
46+
and {ref}/search-request-body.html#request-body-search-preference[`preference`]
47+
as these parameters are copied from the point in time.
48+
<2> The `id` parameter tells Elasticsearch to execute the request using contexts
49+
from this point int time.
50+
<3> The `keep_alive` parameter tells Elasticsearch how long it should extend
51+
the time to live of the point in time.
52+
53+
IMPORTANT: The open point in time request and each subsequent search request can
54+
return different `id`; thus always use the most recently received `id` for the
55+
next search request.
56+
57+
[[point-in-time-keep-alive]]
58+
===== Keeping point in time alive
59+
The `keep_alive` parameter, which is passed to a open point in time request and
60+
search request, extends the time to live of the corresponding point in time.
61+
The value (e.g. `1m`, see <<time-units>>) does not need to be long enough to
62+
process all data -- it just needs to be long enough for the next request.
63+
64+
Normally, the background merge process optimizes the index by merging together
65+
smaller segments to create new, bigger segments. Once the smaller segments are
66+
no longer needed they are deleted. However, open point-in-times prevent the
67+
old segments from being deleted since they are still in use.
68+
69+
TIP: Keeping older segments alive means that more disk space and file handles
70+
are needed. Ensure that you have configured your nodes to have ample free file
71+
handles. See <<file-descriptors>>.
72+
73+
Additionally, if a segment contains deleted or updated documents then the
74+
point in time must keep track of whether each document in the segment was live at
75+
the time of the initial search request. Ensure that your nodes have sufficient heap
76+
space if you have many open point-in-times on an index that is subject to ongoing
77+
deletes or updates.
78+
79+
You can check how many point-in-times (i.e, search contexts) are open with the
80+
<<cluster-nodes-stats,nodes stats API>>:
81+
82+
[source,console]
83+
---------------------------------------
84+
GET /_nodes/stats/indices/search
85+
---------------------------------------
86+
87+
===== Close point in time API
88+
89+
Point-in-time is automatically closed when its `keep_alive` has
90+
been elapsed. However keeping point-in-times has a cost, as discussed in the
91+
<<point-in-time-keep-alive,previous section>>. Point-in-times should be closed
92+
as soon as they are no longer used in search requests.
93+
94+
[source,console]
95+
---------------------------------------
96+
DELETE /_pit
97+
{
98+
"id" : "46ToAwMDaWR4BXV1aWQxAgZub2RlXzEAAAAAAAAAAAEBYQNpZHkFdXVpZDIrBm5vZGVfMwAAAAAAAAAAKgFjA2lkeQV1dWlkMioGbm9kZV8yAAAAAAAAAAAMAWIBBXV1aWQyAAA="
99+
}
100+
---------------------------------------
101+
// TEST[catch:missing]
102+
103+
The API returns the following response:
104+
105+
[source,console-result]
106+
--------------------------------------------------
107+
{
108+
"succeeded": true, <1>
109+
"num_freed": 3 <2>
110+
}
111+
--------------------------------------------------
112+
// TESTRESPONSE[s/"succeeded": true/"succeeded": $body.succeeded/]
113+
// TESTRESPONSE[s/"num_freed": 3/"num_freed": $body.num_freed/]
114+
115+
<1> If true, all search contexts associated with the point-in-time id are successfully closed
116+
<2> The number of search contexts have been successfully closed

modules/parent-join/src/main/java/org/elasticsearch/join/query/ParentChildInnerHitContextBuilder.java

+2-6
Original file line numberDiff line numberDiff line change
@@ -152,12 +152,8 @@ public TopDocsAndMaxScore topDocs(SearchHit hit) throws IOException {
152152
topDocsCollector = TopScoreDocCollector.create(topN, Integer.MAX_VALUE);
153153
maxScoreCollector = new MaxScoreCollector();
154154
}
155-
try {
156-
for (LeafReaderContext ctx : context.searcher().getIndexReader().leaves()) {
157-
intersect(weight, innerHitQueryWeight, MultiCollector.wrap(topDocsCollector, maxScoreCollector), ctx);
158-
}
159-
} finally {
160-
clearReleasables(Lifetime.COLLECTION);
155+
for (LeafReaderContext ctx : context.searcher().getIndexReader().leaves()) {
156+
intersect(weight, innerHitQueryWeight, MultiCollector.wrap(topDocsCollector, maxScoreCollector), ctx);
161157
}
162158
TopDocs topDocs = topDocsCollector.topDocs(from(), size());
163159
float maxScore = Float.NaN;

rest-api-spec/src/main/resources/rest-api-spec/test/scroll/20_keep_alive.yml

+2-2
Original file line numberDiff line numberDiff line change
@@ -34,7 +34,7 @@
3434
search.max_keep_alive: "1m"
3535

3636
- do:
37-
catch: /.*Keep alive for scroll.*is too large.*/
37+
catch: /.*Keep alive for.*is too large.*/
3838
search:
3939
rest_total_hits_as_int: true
4040
index: test_scroll
@@ -61,7 +61,7 @@
6161
- length: {hits.hits: 1 }
6262

6363
- do:
64-
catch: /.*Keep alive for scroll.*is too large.*/
64+
catch: /.*Keep alive for.*is too large.*/
6565
scroll:
6666
rest_total_hits_as_int: true
6767
scroll_id: $scroll_id

server/src/internalClusterTest/java/org/elasticsearch/search/scroll/SearchScrollIT.java

+2-2
Original file line numberDiff line numberDiff line change
@@ -602,7 +602,7 @@ public void testInvalidScrollKeepAlive() throws IOException {
602602
IllegalArgumentException illegalArgumentException =
603603
(IllegalArgumentException) ExceptionsHelper.unwrap(exc, IllegalArgumentException.class);
604604
assertNotNull(illegalArgumentException);
605-
assertThat(illegalArgumentException.getMessage(), containsString("Keep alive for scroll (2h) is too large"));
605+
assertThat(illegalArgumentException.getMessage(), containsString("Keep alive for request (2h) is too large"));
606606

607607
SearchResponse searchResponse = client().prepareSearch()
608608
.setQuery(matchAllQuery())
@@ -619,7 +619,7 @@ public void testInvalidScrollKeepAlive() throws IOException {
619619
illegalArgumentException =
620620
(IllegalArgumentException) ExceptionsHelper.unwrap(exc, IllegalArgumentException.class);
621621
assertNotNull(illegalArgumentException);
622-
assertThat(illegalArgumentException.getMessage(), containsString("Keep alive for scroll (3h) is too large"));
622+
assertThat(illegalArgumentException.getMessage(), containsString("Keep alive for request (3h) is too large"));
623623
}
624624

625625
/**

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

+20-15
Original file line numberDiff line numberDiff line change
@@ -24,6 +24,7 @@
2424
import org.apache.lucene.util.SetOnce;
2525
import org.elasticsearch.ElasticsearchException;
2626
import org.elasticsearch.ExceptionsHelper;
27+
import org.elasticsearch.Version;
2728
import org.elasticsearch.action.ActionListener;
2829
import org.elasticsearch.action.NoShardAvailableActionException;
2930
import org.elasticsearch.action.ShardOperationFailedException;
@@ -163,7 +164,7 @@ public final void start() {
163164
// total hits is null in the response if the tracking of total hits is disabled
164165
boolean withTotalHits = trackTotalHitsUpTo != SearchContext.TRACK_TOTAL_HITS_DISABLED;
165166
listener.onResponse(new SearchResponse(InternalSearchResponse.empty(withTotalHits), null, 0, 0, 0, buildTookInMillis(),
166-
ShardSearchFailure.EMPTY_ARRAY, clusters));
167+
ShardSearchFailure.EMPTY_ARRAY, clusters, null));
167168
return;
168169
}
169170
executePhase(this);
@@ -514,22 +515,29 @@ public final SearchRequest getRequest() {
514515
return request;
515516
}
516517

517-
protected final SearchResponse buildSearchResponse(InternalSearchResponse internalSearchResponse,
518-
String scrollId,
519-
ShardSearchFailure[] failures) {
518+
protected final SearchResponse buildSearchResponse(InternalSearchResponse internalSearchResponse, ShardSearchFailure[] failures,
519+
String scrollId, String searchContextId) {
520520
return new SearchResponse(internalSearchResponse, scrollId, getNumShards(), successfulOps.get(),
521-
skippedOps.get(), buildTookInMillis(), failures, clusters);
521+
skippedOps.get(), buildTookInMillis(), failures, clusters, searchContextId);
522+
}
523+
524+
boolean includeSearchContextInResponse() {
525+
return request.pointInTimeBuilder() != null;
522526
}
523527

524528
@Override
525-
public void sendSearchResponse(InternalSearchResponse internalSearchResponse, String scrollId) {
529+
public void sendSearchResponse(InternalSearchResponse internalSearchResponse, AtomicArray<SearchPhaseResult> queryResults) {
526530
ShardSearchFailure[] failures = buildShardFailures();
527531
Boolean allowPartialResults = request.allowPartialSearchResults();
528532
assert allowPartialResults != null : "SearchRequest missing setting for allowPartialSearchResults";
529-
if (allowPartialResults == false && failures.length > 0){
533+
if (request.pointInTimeBuilder() == null && allowPartialResults == false && failures.length > 0) {
530534
raisePhaseFailure(new SearchPhaseExecutionException("", "Shard failures", null, failures));
531535
} else {
532-
listener.onResponse(buildSearchResponse(internalSearchResponse, scrollId, failures));
536+
final Version minNodeVersion = clusterState.nodes().getMinNodeVersion();
537+
final String scrollId = request.scroll() != null ? TransportSearchHelper.buildScrollId(queryResults, minNodeVersion) : null;
538+
final String searchContextId =
539+
includeSearchContextInResponse() ? SearchContextId.encode(queryResults.asList(), aliasFilter, minNodeVersion) : null;
540+
listener.onResponse(buildSearchResponse(internalSearchResponse, failures, scrollId, searchContextId));
533541
}
534542
}
535543

@@ -598,12 +606,13 @@ public final ShardSearchRequest buildShardSearchRequest(SearchShardIterator shar
598606
final String[] routings = indexRoutings.getOrDefault(indexName, Collections.emptySet())
599607
.toArray(new String[0]);
600608
ShardSearchRequest shardRequest = new ShardSearchRequest(shardIt.getOriginalIndices(), request, shardIt.shardId(), getNumShards(),
601-
filter, indexBoost, timeProvider.getAbsoluteStartMillis(), shardIt.getClusterAlias(), routings);
609+
filter, indexBoost, timeProvider.getAbsoluteStartMillis(), shardIt.getClusterAlias(), routings,
610+
shardIt.getSearchContextId(), shardIt.getSearchContextKeepAlive());
602611
// if we already received a search result we can inform the shard that it
603612
// can return a null response if the request rewrites to match none rather
604613
// than creating an empty response in the search thread pool.
605-
// Note that, we have to disable this shortcut for scroll queries.
606-
shardRequest.canReturnNullResponseIfMatchNoDocs(hasShardResponse.get() && request.scroll() == null);
614+
// Note that, we have to disable this shortcut for queries that create a context (scroll and search context).
615+
shardRequest.canReturnNullResponseIfMatchNoDocs(hasShardResponse.get() && shardRequest.scroll() == null);
607616
return shardRequest;
608617
}
609618

@@ -673,8 +682,4 @@ private synchronized Runnable tryQueue(Runnable runnable) {
673682
return toExecute;
674683
}
675684
}
676-
677-
protected ClusterState clusterState() {
678-
return clusterState;
679-
}
680685
}

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

+62-14
Original file line numberDiff line numberDiff line change
@@ -21,20 +21,28 @@
2121
import org.apache.logging.log4j.Logger;
2222
import org.apache.logging.log4j.message.ParameterizedMessage;
2323
import org.elasticsearch.action.ActionListener;
24+
import org.elasticsearch.action.StepListener;
25+
import org.elasticsearch.action.support.GroupedActionListener;
2426
import org.elasticsearch.cluster.node.DiscoveryNode;
2527
import org.elasticsearch.cluster.node.DiscoveryNodes;
28+
import org.elasticsearch.common.Strings;
2629
import org.elasticsearch.common.util.concurrent.CountDown;
2730
import org.elasticsearch.transport.Transport;
2831
import org.elasticsearch.transport.TransportResponse;
2932

3033
import java.util.ArrayList;
34+
import java.util.Collection;
35+
import java.util.Collections;
3136
import java.util.List;
37+
import java.util.Set;
3238
import java.util.concurrent.atomic.AtomicBoolean;
3339
import java.util.concurrent.atomic.AtomicInteger;
40+
import java.util.function.BiFunction;
41+
import java.util.stream.Collectors;
3442

3543
import static org.elasticsearch.action.search.TransportSearchHelper.parseScrollId;
3644

37-
final class ClearScrollController implements Runnable {
45+
public final class ClearScrollController implements Runnable {
3846
private final DiscoveryNodes nodes;
3947
private final SearchTransportService searchTransportService;
4048
private final CountDown expectedOps;
@@ -56,19 +64,18 @@ final class ClearScrollController implements Runnable {
5664
expectedOps = nodes.getSize();
5765
runner = this::cleanAllScrolls;
5866
} else {
59-
List<ScrollIdForNode> parsedScrollIds = new ArrayList<>();
60-
for (String parsedScrollId : request.getScrollIds()) {
61-
ScrollIdForNode[] context = parseScrollId(parsedScrollId).getContext();
62-
for (ScrollIdForNode id : context) {
63-
parsedScrollIds.add(id);
64-
}
67+
// TODO: replace this with #closeContexts
68+
List<SearchContextIdForNode> contexts = new ArrayList<>();
69+
for (String scrollId : request.getScrollIds()) {
70+
SearchContextIdForNode[] context = parseScrollId(scrollId).getContext();
71+
Collections.addAll(contexts, context);
6572
}
66-
if (parsedScrollIds.isEmpty()) {
73+
if (contexts.isEmpty()) {
6774
expectedOps = 0;
6875
runner = () -> listener.onResponse(new ClearScrollResponse(true, 0));
6976
} else {
70-
expectedOps = parsedScrollIds.size();
71-
runner = () -> cleanScrollIds(parsedScrollIds);
77+
expectedOps = contexts.size();
78+
runner = () -> cleanScrollIds(contexts);
7279
}
7380
}
7481
this.expectedOps = new CountDown(expectedOps);
@@ -101,17 +108,17 @@ public void onFailure(Exception e) {
101108
}
102109
}
103110

104-
void cleanScrollIds(List<ScrollIdForNode> parsedScrollIds) {
105-
SearchScrollAsyncAction.collectNodesAndRun(parsedScrollIds, nodes, searchTransportService, ActionListener.wrap(
111+
void cleanScrollIds(List<SearchContextIdForNode> contextIds) {
112+
SearchScrollAsyncAction.collectNodesAndRun(contextIds, nodes, searchTransportService, ActionListener.wrap(
106113
lookup -> {
107-
for (ScrollIdForNode target : parsedScrollIds) {
114+
for (SearchContextIdForNode target : contextIds) {
108115
final DiscoveryNode node = lookup.apply(target.getClusterAlias(), target.getNode());
109116
if (node == null) {
110117
onFreedContext(false);
111118
} else {
112119
try {
113120
Transport.Connection connection = searchTransportService.getConnection(target.getClusterAlias(), node);
114-
searchTransportService.sendFreeContext(connection, target.getContextId(),
121+
searchTransportService.sendFreeContext(connection, target.getSearchContextId(),
115122
ActionListener.wrap(freed -> onFreedContext(freed.isFreed()), e -> onFailedFreedContext(e, node)));
116123
} catch (Exception e) {
117124
onFailedFreedContext(e, node);
@@ -142,4 +149,45 @@ private void onFailedFreedContext(Throwable e, DiscoveryNode node) {
142149
listener.onResponse(new ClearScrollResponse(false, freedSearchContexts.get()));
143150
}
144151
}
152+
153+
/**
154+
* Closes the given context id and reports the number of freed contexts via the listener
155+
*/
156+
public static void closeContexts(DiscoveryNodes nodes, SearchTransportService searchTransportService,
157+
Collection<SearchContextIdForNode> contextIds,
158+
ActionListener<Integer> listener) {
159+
if (contextIds.isEmpty()) {
160+
listener.onResponse(0);
161+
return;
162+
}
163+
final Set<String> clusters = contextIds.stream()
164+
.filter(ctx -> Strings.isEmpty(ctx.getClusterAlias()) == false)
165+
.map(SearchContextIdForNode::getClusterAlias).collect(Collectors.toSet());
166+
final StepListener<BiFunction<String, String, DiscoveryNode>> lookupListener = new StepListener<>();
167+
if (clusters.isEmpty() == false) {
168+
searchTransportService.getRemoteClusterService().collectNodes(clusters, lookupListener);
169+
} else {
170+
lookupListener.onResponse((cluster, nodeId) -> nodes.get(nodeId));
171+
}
172+
lookupListener.whenComplete(nodeLookup -> {
173+
final GroupedActionListener<Boolean> groupedListener = new GroupedActionListener<>(
174+
ActionListener.delegateFailure(listener, (l, rs) -> l.onResponse(Math.toIntExact(rs.stream().filter(r -> r).count()))),
175+
contextIds.size()
176+
);
177+
for (SearchContextIdForNode contextId : contextIds) {
178+
final DiscoveryNode node = nodeLookup.apply(contextId.getClusterAlias(), contextId.getNode());
179+
if (node == null) {
180+
groupedListener.onResponse(false);
181+
} else {
182+
try {
183+
final Transport.Connection connection = searchTransportService.getConnection(contextId.getClusterAlias(), node);
184+
searchTransportService.sendFreeContext(connection, contextId.getSearchContextId(),
185+
ActionListener.wrap(r -> groupedListener.onResponse(r.isFreed()), e -> groupedListener.onResponse(false)));
186+
} catch (Exception e) {
187+
groupedListener.onResponse(false);
188+
}
189+
}
190+
}
191+
}, listener::onFailure);
192+
}
145193
}

0 commit comments

Comments
 (0)