Skip to content

Commit 25837c5

Browse files
committed
Add allow_partial_results cluster settings in ES|QL
1 parent 976c9f9 commit 25837c5

File tree

8 files changed

+40
-6
lines changed

8 files changed

+40
-6
lines changed

x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/esql/action/EsqlQueryRequestBuilder.java

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -39,4 +39,6 @@ public final ActionType<Response> action() {
3939

4040
public abstract EsqlQueryRequestBuilder<Request, Response> filter(QueryBuilder filter);
4141

42+
public abstract EsqlQueryRequestBuilder<Request, Response> allowPartialResults(boolean allowPartialResults);
43+
4244
}

x-pack/plugin/esql/qa/server/src/main/java/org/elasticsearch/xpack/esql/qa/rest/RestEsqlTestCase.java

Lines changed: 9 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -127,6 +127,7 @@ public static class RequestObjectBuilder {
127127
private Boolean includeCCSMetadata = null;
128128

129129
private CheckedConsumer<XContentBuilder, IOException> filter;
130+
private Boolean allPartialResults = null;
130131

131132
public RequestObjectBuilder() throws IOException {
132133
this(randomFrom(XContentType.values()));
@@ -204,6 +205,11 @@ public RequestObjectBuilder filter(CheckedConsumer<XContentBuilder, IOException>
204205
return this;
205206
}
206207

208+
public RequestObjectBuilder allPartialResults(boolean allPartialResults) {
209+
this.allPartialResults = allPartialResults;
210+
return this;
211+
}
212+
207213
public RequestObjectBuilder build() throws IOException {
208214
if (isBuilt == false) {
209215
if (tables != null) {
@@ -1151,6 +1157,9 @@ static Request prepareRequestWithOptions(RequestObjectBuilder requestObject, Mod
11511157
requestObject.build();
11521158
Request request = prepareRequest(mode);
11531159
String mediaType = attachBody(requestObject, request);
1160+
if (requestObject.allPartialResults != null) {
1161+
request.addParameter("allow_partial_results", String.valueOf(requestObject.allPartialResults));
1162+
}
11541163

11551164
RequestOptions.Builder options = request.getOptions().toBuilder();
11561165
options.setWarningsHandler(WarningsHandler.PERMISSIVE); // We assert the warnings ourselves

x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlQueryRequest.java

Lines changed: 4 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -52,7 +52,7 @@ public class EsqlQueryRequest extends org.elasticsearch.xpack.core.esql.action.E
5252
private boolean keepOnCompletion;
5353
private boolean onSnapshotBuild = Build.current().isSnapshot();
5454
private boolean acceptedPragmaRisks = false;
55-
private boolean allowPartialResults = false;
55+
private Boolean allowPartialResults = null;
5656

5757
/**
5858
* "Tables" provided in the request for use with things like {@code LOOKUP}.
@@ -232,12 +232,13 @@ public Map<String, Map<String, Column>> tables() {
232232
return tables;
233233
}
234234

235-
public boolean allowPartialResults() {
235+
public Boolean allowPartialResults() {
236236
return allowPartialResults;
237237
}
238238

239-
public void allowPartialResults(boolean allowPartialResults) {
239+
public EsqlQueryRequest allowPartialResults(boolean allowPartialResults) {
240240
this.allowPartialResults = allowPartialResults;
241+
return this;
241242
}
242243

243244
@Override

x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/EsqlQueryRequestBuilder.java

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -66,6 +66,12 @@ public EsqlQueryRequestBuilder keepOnCompletion(boolean keepOnCompletion) {
6666
return this;
6767
}
6868

69+
@Override
70+
public EsqlQueryRequestBuilder allowPartialResults(boolean allowPartialResults) {
71+
request.allowPartialResults(allowPartialResults);
72+
return this;
73+
}
74+
6975
static { // plumb access from x-pack core
7076
SharedSecrets.setEsqlQueryRequestBuilderAccess(EsqlQueryRequestBuilder::newSyncEsqlQueryRequestBuilder);
7177
}

x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/RequestXContent.java

Lines changed: 0 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -85,7 +85,6 @@ String fields() {
8585
static final ParseField WAIT_FOR_COMPLETION_TIMEOUT = new ParseField("wait_for_completion_timeout");
8686
static final ParseField KEEP_ALIVE = new ParseField("keep_alive");
8787
static final ParseField KEEP_ON_COMPLETION = new ParseField("keep_on_completion");
88-
static final ParseField ALLOW_PARTIAL_RESULTS = new ParseField("allow_partial_results");
8988

9089
private static final ObjectParser<EsqlQueryRequest, Void> SYNC_PARSER = objectParserSync(EsqlQueryRequest::syncEsqlQueryRequest);
9190
private static final ObjectParser<EsqlQueryRequest, Void> ASYNC_PARSER = objectParserAsync(EsqlQueryRequest::asyncEsqlQueryRequest);
@@ -115,7 +114,6 @@ private static void objectParserCommon(ObjectParser<EsqlQueryRequest, ?> parser)
115114
parser.declareString((request, localeTag) -> request.locale(Locale.forLanguageTag(localeTag)), LOCALE_FIELD);
116115
parser.declareBoolean(EsqlQueryRequest::profile, PROFILE_FIELD);
117116
parser.declareField((p, r, c) -> new ParseTables(r, p).parseTables(), TABLES_FIELD, ObjectParser.ValueType.OBJECT);
118-
parser.declareBoolean(EsqlQueryRequest::allowPartialResults, ALLOW_PARTIAL_RESULTS);
119117
}
120118

121119
private static ObjectParser<EsqlQueryRequest, Void> objectParserSync(Supplier<EsqlQueryRequest> supplier) {

x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/action/RestEsqlQueryAction.java

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -51,6 +51,10 @@ protected RestChannelConsumer prepareRequest(RestRequest request, NodeClient cli
5151
}
5252

5353
protected static RestChannelConsumer restChannelConsumer(EsqlQueryRequest esqlRequest, RestRequest request, NodeClient client) {
54+
final Boolean partialResults = request.paramAsBoolean("allow_partial_results", null);
55+
if (partialResults != null) {
56+
esqlRequest.allowPartialResults(partialResults);
57+
}
5458
LOGGER.debug("Beginning execution of ESQL query.\nQuery string: [{}]", esqlRequest.query());
5559

5660
return channel -> {

x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/EsqlPlugin.java

Lines changed: 8 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -102,6 +102,13 @@ public class EsqlPlugin extends Plugin implements ActionPlugin {
102102
Setting.Property.Dynamic
103103
);
104104

105+
public static final Setting<Boolean> QUERY_ALLOW_PARTIAL_RESULTS = Setting.boolSetting(
106+
"esql.query.allow_partial_results",
107+
false,
108+
Setting.Property.NodeScope,
109+
Setting.Property.Dynamic
110+
);
111+
105112
@Override
106113
public Collection<?> createComponents(PluginServices services) {
107114
CircuitBreaker circuitBreaker = services.indicesService().getBigArrays().breakerService().getBreaker("request");
@@ -151,7 +158,7 @@ protected XPackLicenseState getLicenseState() {
151158
*/
152159
@Override
153160
public List<Setting<?>> getSettings() {
154-
return List.of(QUERY_RESULT_TRUNCATION_DEFAULT_SIZE, QUERY_RESULT_TRUNCATION_MAX_SIZE);
161+
return List.of(QUERY_RESULT_TRUNCATION_DEFAULT_SIZE, QUERY_RESULT_TRUNCATION_MAX_SIZE, QUERY_ALLOW_PARTIAL_RESULTS);
155162
}
156163

157164
@Override

x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/TransportEsqlQueryAction.java

Lines changed: 7 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -80,6 +80,7 @@ public class TransportEsqlQueryAction extends HandledTransportAction<EsqlQueryRe
8080
private final RemoteClusterService remoteClusterService;
8181
private final UsageService usageService;
8282
private final TransportActionServices services;
83+
private volatile boolean defaultAllowPartialResults;
8384

8485
@Inject
8586
@SuppressWarnings("this-escape")
@@ -158,6 +159,9 @@ public TransportEsqlQueryAction(
158159
indexNameExpressionResolver,
159160
usageService
160161
);
162+
defaultAllowPartialResults = EsqlPlugin.QUERY_ALLOW_PARTIAL_RESULTS.get(clusterService.getSettings());
163+
clusterService.getClusterSettings()
164+
.addSettingsUpdateConsumer(EsqlPlugin.QUERY_ALLOW_PARTIAL_RESULTS, v -> defaultAllowPartialResults = v);
161165
}
162166

163167
@Override
@@ -194,6 +198,9 @@ public void execute(EsqlQueryRequest request, EsqlQueryTask task, ActionListener
194198
}
195199

196200
private void innerExecute(Task task, EsqlQueryRequest request, ActionListener<EsqlQueryResponse> listener) {
201+
if (request.allowPartialResults() == null) {
202+
request.allowPartialResults(defaultAllowPartialResults);
203+
}
197204
Configuration configuration = new Configuration(
198205
ZoneOffset.UTC,
199206
request.locale() != null ? request.locale() : Locale.US,

0 commit comments

Comments
 (0)