Browse Source

SQL: fix use of requestTimeout and pageTimeout query parameters (#79360)

Resolves #72151 The _sql endpoint offers a `page_timeout` parameter for
customizing how long scroll contexts should be kept open (if needed) and
a `request_timeout` parameter which the docs describe as "Timeout before
the request fails.". Currently, the value of the `page_timeout`
parameter is used as the `timeout` in subsequent _search requests and
not as the timeout in the `scroll` configuration. For the `scroll`
configuration, SQL uses the `request_timeout` parameter. This PR
addresses the issue by swapping the uses of `page_timeout` and
`request_timeout` in querier. Additionally, the PR removes some unused
artifacts that might have caused some confusion: - The `timeout` and
`keepAlive` fields in `Querier`. Instead, `Querier` directly uses the
according fields in `SqlConfiguration`. - The `SqlConfiguration`
parameter from `ScrollCursor.clear`, it's not used but required an
instance of `SqlConfiguration` with all default values. - One overloaded
constructor of `SqlConfiguration` that was only used for calling
`ScrollCursor.clear` (and some tests) and used default values for an
(arbitrary?) subset of the fields. - The fields related to async
requests in `SqlConfiguration`. I'm a bit unsure about this one but the
fields are never read and it does not seem like an SQL specific concern.
The whole creation of the async tasks is handled in
`TransportSqlQueryAction` and the downstream components do not require
the information.
Lukas Wegmann 4 years ago
parent
commit
9e66494106
22 changed files with 179 additions and 109 deletions
  1. 5 2
      docs/reference/sql/apis/sql-search-api.asciidoc
  2. 5 3
      docs/reference/sql/endpoints/jdbc.asciidoc
  3. 98 0
      x-pack/plugin/sql/src/internalClusterTest/java/org/elasticsearch/xpack/sql/action/SqlSearchPageTimeoutIT.java
  4. 2 2
      x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/execution/PlanExecutor.java
  5. 3 3
      x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/execution/search/CompositeAggCursor.java
  6. 4 14
      x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/execution/search/Querier.java
  7. 4 4
      x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/execution/search/ScrollCursor.java
  8. 3 3
      x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/plugin/TextFormatterCursor.java
  9. 3 10
      x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/plugin/TransportSqlClearCursorAction.java
  10. 3 4
      x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/plugin/TransportSqlQueryAction.java
  11. 1 1
      x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/plugin/TransportSqlTranslateAction.java
  12. 1 1
      x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/session/Cursor.java
  13. 1 1
      x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/session/EmptyCursor.java
  14. 1 1
      x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/session/ListCursor.java
  15. 21 45
      x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/session/SqlConfiguration.java
  16. 3 5
      x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/SqlTestUtils.java
  17. 1 1
      x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/expression/function/scalar/DatabaseFunctionTests.java
  18. 1 1
      x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/expression/function/scalar/UserFunctionTests.java
  19. 12 2
      x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/plan/logical/command/sys/SysColumnsTests.java
  20. 3 2
      x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/plan/logical/command/sys/SysTablesTests.java
  21. 2 1
      x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/plan/logical/command/sys/SysTypesTests.java
  22. 2 3
      x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/plugin/CursorTests.java

+ 5 - 2
docs/reference/sql/apis/sql-search-api.asciidoc

@@ -96,8 +96,11 @@ searches>> if you also specify the `wait_for_completion_timeout` parameter. If
 the `wait_for_completion_timeout`. Defaults to `false`.
 
 `page_timeout`::
-(Optional, <<time-units,time value>>) Timeout before a
-<<sql-pagination,pagination request>> fails. Defaults to `45s` (45 seconds).
+(Optional, <<time-units,time value>>) Minimum retention period for the scroll
+cursor. After this time period, a <<sql-pagination,pagination request>> might
+fail because the scroll cursor is no longer available. Subsequent scroll requests
+prolong the lifetime of the scroll cursor by the duration of `page_timeout` in
+the scroll request. Defaults to `45s` (45 seconds).
 
 `params`::
 (Optional, array) Values for parameters in the `query`. For syntax, see

+ 5 - 3
docs/reference/sql/endpoints/jdbc.asciidoc

@@ -94,12 +94,14 @@ Connection timeout (in milliseconds). That is the maximum amount of time waiting
 `network.timeout` (default `60000`)::
 Network timeout (in milliseconds). That is the maximum amount of time waiting for the network.
 
-`page.timeout` (default `45000`)::
-Page timeout (in milliseconds). That is the maximum amount of time waiting for a page.
-
 `page.size` (default `1000`)::
 Page size (in entries). The number of results returned per page by the server.
 
+`page.timeout` (default `45000`)::
+Page timeout (in milliseconds). Minimum retention period for the scroll cursor on the server. Queries that require
+a stateful scroll cursor on the server side might fail after this timeout. Hence, when scrolling through large result sets,
+processing `page.size` records should not take longer than `page.timeout` milliseconds.
+
 `query.timeout` (default `90000`)::
 Query timeout (in milliseconds). That is the maximum amount of time waiting for a query to return.
 

+ 98 - 0
x-pack/plugin/sql/src/internalClusterTest/java/org/elasticsearch/xpack/sql/action/SqlSearchPageTimeoutIT.java

@@ -0,0 +1,98 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License
+ * 2.0; you may not use this file except in compliance with the Elastic License
+ * 2.0.
+ */
+
+package org.elasticsearch.xpack.sql.action;
+
+import org.elasticsearch.action.index.IndexRequest;
+import org.elasticsearch.action.search.SearchPhaseExecutionException;
+import org.elasticsearch.action.support.WriteRequest;
+import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.core.TimeValue;
+import org.elasticsearch.search.SearchContextMissingException;
+import org.elasticsearch.search.SearchService;
+
+import java.util.Arrays;
+import java.util.concurrent.TimeUnit;
+
+import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
+import static org.hamcrest.Matchers.contains;
+import static org.hamcrest.Matchers.instanceOf;
+
+public class SqlSearchPageTimeoutIT extends AbstractSqlIntegTestCase {
+
+    @Override
+    protected Settings nodeSettings(int nodeOrdinal, Settings otherSettings) {
+        Settings.Builder settings = Settings.builder().put(super.nodeSettings(nodeOrdinal, otherSettings));
+        // use static low keepAlive interval to ensure obsolete search contexts are pruned soon enough
+        settings.put(SearchService.KEEPALIVE_INTERVAL_SETTING.getKey(), TimeValue.timeValueMillis(200));
+        return settings.build();
+    }
+
+    public void testSearchContextIsCleanedUpAfterPageTimeoutForHitsQueries() throws Exception {
+        setupTestIndex();
+
+        SqlQueryResponse response = new SqlQueryRequestBuilder(client(), SqlQueryAction.INSTANCE).query("SELECT field FROM test")
+            .fetchSize(1)
+            .pageTimeout(TimeValue.timeValueMillis(100))
+            .get();
+
+        assertEquals(1, response.size());
+        assertTrue(response.hasCursor());
+        assertEquals(1, getNumberOfSearchContexts());
+
+        assertBusy(() -> assertEquals(0, getNumberOfSearchContexts()), 3, TimeUnit.SECONDS);
+
+        SearchPhaseExecutionException exception = expectThrows(
+            SearchPhaseExecutionException.class,
+            () -> new SqlQueryRequestBuilder(client(), SqlQueryAction.INSTANCE).cursor(response.cursor()).get()
+        );
+
+        assertThat(Arrays.asList(exception.guessRootCauses()), contains(instanceOf(SearchContextMissingException.class)));
+    }
+
+    public void testNoSearchContextForAggregationQueries() throws InterruptedException {
+        setupTestIndex();
+
+        SqlQueryResponse response = new SqlQueryRequestBuilder(client(), SqlQueryAction.INSTANCE).query(
+            "SELECT COUNT(*) FROM test GROUP BY field"
+        ).fetchSize(1).pageTimeout(TimeValue.timeValueMillis(100)).get();
+
+        assertEquals(1, response.size());
+        assertTrue(response.hasCursor());
+        assertEquals(0, getNumberOfSearchContexts());
+
+        Thread.sleep(1000);
+
+        // since aggregation queries do not have a stateful search context, scrolling is still possible after page_timeout
+        response = new SqlQueryRequestBuilder(client(), SqlQueryAction.INSTANCE).cursor(response.cursor()).get();
+
+        assertEquals(1, response.size());
+    }
+
+    private void setupTestIndex() {
+        assertAcked(client().admin().indices().prepareCreate("test").get());
+        client().prepareBulk()
+            .add(new IndexRequest("test").id("1").source("field", "bar"))
+            .add(new IndexRequest("test").id("2").source("field", "baz"))
+            .setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE)
+            .get();
+        ensureYellow("test");
+    }
+
+    private long getNumberOfSearchContexts() {
+        return client().admin()
+            .indices()
+            .prepareStats("test")
+            .clear()
+            .setSearch(true)
+            .get()
+            .getIndex("test")
+            .getTotal()
+            .getSearch()
+            .getOpenContexts();
+    }
+}

+ 2 - 2
x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/execution/PlanExecutor.java

@@ -114,8 +114,8 @@ public class PlanExecutor {
         }));
     }
 
-    public void cleanCursor(SqlConfiguration cfg, Cursor cursor, ActionListener<Boolean> listener) {
-        cursor.clear(cfg, client, listener);
+    public void cleanCursor(Cursor cursor, ActionListener<Boolean> listener) {
+        cursor.clear(client, listener);
     }
 
     public Client client() {

+ 3 - 3
x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/execution/search/CompositeAggCursor.java

@@ -28,9 +28,9 @@ import org.elasticsearch.xpack.ql.type.Schema;
 import org.elasticsearch.xpack.ql.util.StringUtils;
 import org.elasticsearch.xpack.sql.SqlIllegalArgumentException;
 import org.elasticsearch.xpack.sql.querydsl.agg.Aggs;
-import org.elasticsearch.xpack.sql.session.SqlConfiguration;
 import org.elasticsearch.xpack.sql.session.Cursor;
 import org.elasticsearch.xpack.sql.session.Rows;
+import org.elasticsearch.xpack.sql.session.SqlConfiguration;
 
 import java.io.IOException;
 import java.util.Arrays;
@@ -133,7 +133,7 @@ public class CompositeAggCursor implements Cursor {
             log.trace("About to execute composite query {} on {}", StringUtils.toString(query), indices);
         }
 
-        SearchRequest request = Querier.prepareRequest(client, query, cfg.pageTimeout(), includeFrozen, indices);
+        SearchRequest request = Querier.prepareRequest(query, cfg.requestTimeout(), includeFrozen, indices);
 
         client.search(request, new ActionListener.Delegating<>(listener) {
             @Override
@@ -267,7 +267,7 @@ public class CompositeAggCursor implements Cursor {
 
 
     @Override
-    public void clear(SqlConfiguration cfg, Client client, ActionListener<Boolean> listener) {
+    public void clear(Client client, ActionListener<Boolean> listener) {
         listener.onResponse(true);
     }
 

+ 4 - 14
x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/execution/search/Querier.java

@@ -16,7 +16,6 @@ import org.elasticsearch.action.search.ShardSearchFailure;
 import org.elasticsearch.client.Client;
 import org.elasticsearch.common.Strings;
 import org.elasticsearch.common.util.CollectionUtils;
-import org.elasticsearch.xcontent.XContentBuilder;
 import org.elasticsearch.core.Nullable;
 import org.elasticsearch.core.TimeValue;
 import org.elasticsearch.core.Tuple;
@@ -28,6 +27,7 @@ import org.elasticsearch.search.aggregations.bucket.MultiBucketsAggregation.Buck
 import org.elasticsearch.search.aggregations.bucket.filter.Filters;
 import org.elasticsearch.search.builder.SearchSourceBuilder;
 import org.elasticsearch.tasks.TaskCancelledException;
+import org.elasticsearch.xcontent.XContentBuilder;
 import org.elasticsearch.xpack.ql.execution.search.FieldExtraction;
 import org.elasticsearch.xpack.ql.execution.search.extractor.BucketExtractor;
 import org.elasticsearch.xpack.ql.execution.search.extractor.ComputingExtractor;
@@ -93,7 +93,6 @@ public class Querier {
 
     private final PlanExecutor planExecutor;
     private final SqlConfiguration cfg;
-    private final TimeValue keepAlive, timeout;
     private final int size;
     private final Client client;
     @Nullable
@@ -103,8 +102,6 @@ public class Querier {
         this.planExecutor = sqlSession.planExecutor();
         this.client = sqlSession.client();
         this.cfg = sqlSession.configuration();
-        this.keepAlive = cfg.requestTimeout();
-        this.timeout = cfg.pageTimeout();
         this.filter = cfg.filter();
         this.size = cfg.pageSize();
     }
@@ -112,12 +109,7 @@ public class Querier {
     public void query(List<Attribute> output, QueryContainer query, String index, ActionListener<Page> listener) {
         // prepare the request
         SearchSourceBuilder sourceBuilder = SourceGenerator.sourceBuilder(query, filter, size);
-        // set query timeout
-        if (timeout.getSeconds() > 0) {
-            sourceBuilder.timeout(timeout);
-        }
 
-        // set runtime mappings
         if (this.cfg.runtimeMappings() != null) {
             sourceBuilder.runtimeMappings(this.cfg.runtimeMappings());
         }
@@ -126,7 +118,7 @@ public class Querier {
             log.trace("About to execute query {} on {}", StringUtils.toString(sourceBuilder), index);
         }
 
-        SearchRequest search = prepareRequest(client, sourceBuilder, timeout, query.shouldIncludeFrozen(),
+        SearchRequest search = prepareRequest(sourceBuilder, cfg.requestTimeout(), query.shouldIncludeFrozen(),
                 Strings.commaDelimitedListToStringArray(index));
 
         @SuppressWarnings("rawtypes")
@@ -141,7 +133,7 @@ public class Querier {
                 l = new CompositeActionListener(listener, client, cfg, output, query, search);
             }
         } else {
-            search.scroll(keepAlive);
+            search.scroll(cfg.pageTimeout());
             l = new ScrollActionListener(listener, client, cfg, output, query);
         }
 
@@ -152,7 +144,7 @@ public class Querier {
         client.search(search, l);
     }
 
-    public static SearchRequest prepareRequest(Client client, SearchSourceBuilder source, TimeValue timeout, boolean includeFrozen,
+    public static SearchRequest prepareRequest(SearchSourceBuilder source, TimeValue timeout, boolean includeFrozen,
             String... indices) {
         source.timeout(timeout);
 
@@ -534,7 +526,6 @@ public class Querier {
 
         final Client client;
         final SqlConfiguration cfg;
-        final TimeValue keepAlive;
         final Schema schema;
 
         BaseActionListener(ActionListener<Page> listener, Client client, SqlConfiguration cfg, List<Attribute> output) {
@@ -542,7 +533,6 @@ public class Querier {
 
             this.client = client;
             this.cfg = cfg;
-            this.keepAlive = cfg.requestTimeout();
             this.schema = Rows.schema(output);
         }
 

+ 4 - 4
x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/execution/search/ScrollCursor.java

@@ -15,16 +15,16 @@ import org.elasticsearch.action.search.ClearScrollResponse;
 import org.elasticsearch.action.search.SearchResponse;
 import org.elasticsearch.action.search.SearchScrollRequest;
 import org.elasticsearch.client.Client;
-import org.elasticsearch.core.Tuple;
 import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
 import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
+import org.elasticsearch.core.Tuple;
 import org.elasticsearch.search.SearchHit;
 import org.elasticsearch.xpack.ql.execution.search.extractor.HitExtractor;
 import org.elasticsearch.xpack.ql.type.Schema;
-import org.elasticsearch.xpack.sql.session.SqlConfiguration;
 import org.elasticsearch.xpack.sql.session.Cursor;
 import org.elasticsearch.xpack.sql.session.Rows;
+import org.elasticsearch.xpack.sql.session.SqlConfiguration;
 
 import java.io.IOException;
 import java.util.BitSet;
@@ -100,13 +100,13 @@ public class ScrollCursor implements Cursor {
         client.searchScroll(request, wrap(response -> {
             handle(response, () -> new SearchHitRowSet(extractors, mask, limit, response),
                     p -> listener.onResponse(p),
-                    p -> clear(cfg, client, wrap(success -> listener.onResponse(p), listener::onFailure)),
+                    p -> clear(client, wrap(success -> listener.onResponse(p), listener::onFailure)),
                     Schema.EMPTY);
         }, listener::onFailure));
     }
 
     @Override
-    public void clear(SqlConfiguration cfg, Client client, ActionListener<Boolean> listener) {
+    public void clear(Client client, ActionListener<Boolean> listener) {
         cleanCursor(client, scrollId, wrap(
                         clearScrollResponse -> listener.onResponse(clearScrollResponse.isSucceeded()),
                         listener::onFailure));

+ 3 - 3
x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/plugin/TextFormatterCursor.java

@@ -12,8 +12,8 @@ import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
 import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.xpack.sql.action.BasicFormatter;
-import org.elasticsearch.xpack.sql.session.SqlConfiguration;
 import org.elasticsearch.xpack.sql.session.Cursor;
+import org.elasticsearch.xpack.sql.session.SqlConfiguration;
 
 import java.io.IOException;
 import java.util.Objects;
@@ -59,8 +59,8 @@ public class TextFormatterCursor implements Cursor {
     }
 
     @Override
-    public void clear(SqlConfiguration cfg, Client client, ActionListener<Boolean> listener) {
-        delegate.clear(cfg, client, listener);
+    public void clear(Client client, ActionListener<Boolean> listener) {
+        delegate.clear(client, listener);
     }
 
     @Override

+ 3 - 10
x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/plugin/TransportSqlClearCursorAction.java

@@ -12,17 +12,12 @@ import org.elasticsearch.action.support.HandledTransportAction;
 import org.elasticsearch.common.inject.Inject;
 import org.elasticsearch.tasks.Task;
 import org.elasticsearch.transport.TransportService;
-import org.elasticsearch.xpack.ql.util.StringUtils;
 import org.elasticsearch.xpack.sql.action.SqlClearCursorRequest;
 import org.elasticsearch.xpack.sql.action.SqlClearCursorResponse;
 import org.elasticsearch.xpack.sql.execution.PlanExecutor;
-import org.elasticsearch.xpack.sql.proto.Protocol;
 import org.elasticsearch.xpack.sql.session.Cursor;
 import org.elasticsearch.xpack.sql.session.Cursors;
-import org.elasticsearch.xpack.sql.session.SqlConfiguration;
-import org.elasticsearch.xpack.sql.util.DateUtils;
 
-import static java.util.Collections.emptyMap;
 import static org.elasticsearch.xpack.sql.action.SqlClearCursorAction.NAME;
 
 public class TransportSqlClearCursorAction extends HandledTransportAction<SqlClearCursorRequest, SqlClearCursorResponse> {
@@ -47,11 +42,9 @@ public class TransportSqlClearCursorAction extends HandledTransportAction<SqlCle
             ActionListener<SqlClearCursorResponse> listener) {
         Cursor cursor = Cursors.decodeFromStringWithZone(request.getCursor()).v1();
         planExecutor.cleanCursor(
-                new SqlConfiguration(DateUtils.UTC, null, Protocol.FETCH_SIZE, Protocol.REQUEST_TIMEOUT, Protocol.PAGE_TIMEOUT, null,
-                        emptyMap(), request.mode(), StringUtils.EMPTY, request.version(), StringUtils.EMPTY, StringUtils.EMPTY,
-                        Protocol.FIELD_MULTI_VALUE_LENIENCY, Protocol.INDEX_INCLUDE_FROZEN),
-                cursor, ActionListener.wrap(
-                success -> listener.onResponse(new SqlClearCursorResponse(success)), listener::onFailure));
+            cursor,
+            ActionListener.<Boolean>wrap(success -> listener.onResponse(new SqlClearCursorResponse(success)), listener::onFailure)
+        );
     }
 }
 

+ 3 - 4
x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/plugin/TransportSqlQueryAction.java

@@ -14,11 +14,11 @@ import org.elasticsearch.action.support.ActionFilters;
 import org.elasticsearch.action.support.HandledTransportAction;
 import org.elasticsearch.cluster.service.ClusterService;
 import org.elasticsearch.common.Strings;
-import org.elasticsearch.common.util.BigArrays;
-import org.elasticsearch.core.Tuple;
 import org.elasticsearch.common.inject.Inject;
 import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.common.util.BigArrays;
+import org.elasticsearch.core.Tuple;
 import org.elasticsearch.tasks.Task;
 import org.elasticsearch.tasks.TaskId;
 import org.elasticsearch.threadpool.ThreadPool;
@@ -112,8 +112,7 @@ public class TransportSqlQueryAction extends HandledTransportAction<SqlQueryRequ
         SqlConfiguration cfg = new SqlConfiguration(request.zoneId(), request.catalog(), request.fetchSize(), request.requestTimeout(),
                 request.pageTimeout(), request.filter(), request.runtimeMappings(), request.mode(), request.clientId(), request.version(),
                 username, clusterName(clusterService), request.fieldMultiValueLeniency(), request.indexIncludeFrozen(),
-                new TaskId(clusterService.localNode().getId(), task.getId()), task,
-                request.waitForCompletionTimeout(), request.keepOnCompletion(), request.keepAlive());
+                new TaskId(clusterService.localNode().getId(), task.getId()), task);
 
         if (Strings.hasText(request.cursor()) == false) {
             executeRequestWithRetryAttempt(clusterService, listener::onFailure,

+ 1 - 1
x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/plugin/TransportSqlTranslateAction.java

@@ -57,7 +57,7 @@ public class TransportSqlTranslateAction extends HandledTransportAction<SqlTrans
                 request.requestTimeout(), request.pageTimeout(), request.filter(), request.runtimeMappings(),
                 request.mode(), request.clientId(), request.version(),
                 username(securityContext), clusterName(clusterService), Protocol.FIELD_MULTI_VALUE_LENIENCY,
-                Protocol.INDEX_INCLUDE_FROZEN);
+                Protocol.INDEX_INCLUDE_FROZEN, null, null);
 
         planExecutor.searchSource(cfg, request.query(), request.params(), ActionListener.wrap(
                 searchSourceBuilder -> listener.onResponse(new SqlTranslateResponse(searchSourceBuilder)), listener::onFailure));

+ 1 - 1
x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/session/Cursor.java

@@ -48,5 +48,5 @@ public interface Cursor extends NamedWriteable {
     /**
      *  Cleans the resources associated with the cursor
      */
-    void clear(SqlConfiguration cfg, Client client, ActionListener<Boolean> listener);
+    void clear(Client client, ActionListener<Boolean> listener);
 }

+ 1 - 1
x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/session/EmptyCursor.java

@@ -38,7 +38,7 @@ class EmptyCursor implements Cursor {
     }
 
     @Override
-    public void clear(SqlConfiguration cfg, Client client, ActionListener<Boolean> listener) {
+    public void clear(Client client, ActionListener<Boolean> listener) {
         // There is nothing to clean
         listener.onResponse(false);
     }

+ 1 - 1
x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/session/ListCursor.java

@@ -89,7 +89,7 @@ public class ListCursor implements Cursor {
     }
 
     @Override
-    public void clear(SqlConfiguration cfg, Client client, ActionListener<Boolean> listener) {
+    public void clear(Client client, ActionListener<Boolean> listener) {
         listener.onResponse(true);
     }
 

+ 21 - 45
x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/session/SqlConfiguration.java

@@ -13,7 +13,6 @@ import org.elasticsearch.index.query.QueryBuilder;
 import org.elasticsearch.tasks.TaskId;
 import org.elasticsearch.xpack.sql.action.SqlQueryTask;
 import org.elasticsearch.xpack.sql.proto.Mode;
-import org.elasticsearch.xpack.sql.proto.Protocol;
 import org.elasticsearch.xpack.sql.proto.SqlVersion;
 
 import java.time.ZoneId;
@@ -33,9 +32,6 @@ public class SqlConfiguration extends org.elasticsearch.xpack.ql.session.Configu
     private final SqlVersion version;
     private final boolean multiValueFieldLeniency;
     private final boolean includeFrozenIndices;
-    private final TimeValue waitForCompletionTimeout;
-    private final boolean keepOnCompletion;
-    private final TimeValue keepAlive;
 
     @Nullable
     private final TaskId taskId;
@@ -43,22 +39,29 @@ public class SqlConfiguration extends org.elasticsearch.xpack.ql.session.Configu
     private final SqlQueryTask task;
 
     @Nullable
-    private QueryBuilder filter;
+    private final QueryBuilder filter;
 
     @Nullable
-    private Map<String, Object> runtimeMappings;
-
-    public SqlConfiguration(ZoneId zi, @Nullable String catalog, int pageSize, TimeValue requestTimeout, TimeValue pageTimeout,
-                            QueryBuilder filter,
-                         Map<String, Object> runtimeMappings,
-                         Mode mode, String clientId, SqlVersion version,
-                         String username, String clusterName,
-                         boolean multiValueFieldLeniency,
-                         boolean includeFrozen,
-                         @Nullable TaskId taskId,
-                         @Nullable SqlQueryTask task,
-                         TimeValue waitForCompletionTimeout, boolean keepOnCompletion, TimeValue keepAlive) {
-
+    private final Map<String, Object> runtimeMappings;
+
+    public SqlConfiguration(
+        ZoneId zi,
+        @Nullable String catalog,
+        int pageSize,
+        TimeValue requestTimeout,
+        TimeValue pageTimeout,
+        QueryBuilder filter,
+        Map<String, Object> runtimeMappings,
+        Mode mode,
+        String clientId,
+        SqlVersion version,
+        String username,
+        String clusterName,
+        boolean multiValueFieldLeniency,
+        boolean includeFrozen,
+        @Nullable TaskId taskId,
+        @Nullable SqlQueryTask task
+    ) {
         super(zi, username, clusterName, x -> Collections.emptySet());
 
         this.catalog = catalog;
@@ -74,21 +77,6 @@ public class SqlConfiguration extends org.elasticsearch.xpack.ql.session.Configu
         this.includeFrozenIndices = includeFrozen;
         this.taskId = taskId;
         this.task = task;
-        this.waitForCompletionTimeout = waitForCompletionTimeout;
-        this.keepOnCompletion = keepOnCompletion;
-        this.keepAlive = keepAlive;
-    }
-
-    public SqlConfiguration(ZoneId zi, @Nullable String catalog, int pageSize, TimeValue requestTimeout, TimeValue pageTimeout,
-                            QueryBuilder filter,
-                            Map<String, Object> runtimeMappings,
-                            Mode mode, String clientId, SqlVersion version,
-                            String username, String clusterName,
-                            boolean multiValueFieldLeniency,
-                            boolean includeFrozen) {
-        this(zi, catalog, pageSize, requestTimeout, pageTimeout, filter, runtimeMappings, mode, clientId, version, username, clusterName,
-            multiValueFieldLeniency, includeFrozen, null, null, Protocol.DEFAULT_WAIT_FOR_COMPLETION_TIMEOUT,
-            Protocol.DEFAULT_KEEP_ON_COMPLETION, Protocol.DEFAULT_KEEP_ALIVE);
     }
 
     public String catalog() {
@@ -142,16 +130,4 @@ public class SqlConfiguration extends org.elasticsearch.xpack.ql.session.Configu
     public SqlQueryTask task() {
         return task;
     }
-
-    public TimeValue waitForCompletionTimeout() {
-        return waitForCompletionTimeout;
-    }
-
-    public boolean keepOnCompletion() {
-        return keepOnCompletion;
-    }
-
-    public TimeValue keepAlive() {
-        return keepAlive;
-    }
 }

+ 3 - 5
x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/SqlTestUtils.java

@@ -45,7 +45,8 @@ public final class SqlTestUtils {
 
     public static final SqlConfiguration TEST_CFG = new SqlConfiguration(DateUtils.UTC, null, Protocol.FETCH_SIZE,
             Protocol.REQUEST_TIMEOUT, Protocol.PAGE_TIMEOUT, null, null, Mode.PLAIN,
-            null, null, null, null, false, false);
+            null, null, null, null, false, false,
+            null, null);
 
     public static SqlConfiguration randomConfiguration(ZoneId providedZoneId, SqlVersion sqlVersion) {
         Mode mode = randomFrom(Mode.values());
@@ -65,10 +66,7 @@ public final class SqlTestUtils {
             false,
             randomBoolean(),
             new TaskId(randomAlphaOfLength(10), taskId),
-            randomTask(taskId, mode, sqlVersion),
-            new TimeValue(randomNonNegativeLong()),
-            randomBoolean(),
-            new TimeValue(randomNonNegativeLong()));
+            randomTask(taskId, mode, sqlVersion));
     }
 
     public static SqlConfiguration randomConfiguration() {

+ 1 - 1
x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/expression/function/scalar/DatabaseFunctionTests.java

@@ -33,7 +33,7 @@ public class DatabaseFunctionTests extends ESTestCase {
         SqlConfiguration sqlConfig = new SqlConfiguration(DateUtils.UTC, null, Protocol.FETCH_SIZE, Protocol.REQUEST_TIMEOUT,
                 Protocol.PAGE_TIMEOUT, null, null,
                 randomFrom(Mode.values()), randomAlphaOfLength(10),
-                null, null, clusterName, randomBoolean(), randomBoolean());
+                null, null, clusterName, randomBoolean(), randomBoolean(), null, null);
         Analyzer analyzer = new Analyzer(
                 sqlConfig,
                 new SqlFunctionRegistry(),

+ 1 - 1
x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/expression/function/scalar/UserFunctionTests.java

@@ -33,7 +33,7 @@ public class UserFunctionTests extends ESTestCase {
                 Protocol.PAGE_TIMEOUT, null, null,
                 randomFrom(Mode.values()), randomAlphaOfLength(10),
                 null, null, randomAlphaOfLengthBetween(1, 15),
-                randomBoolean(), randomBoolean());
+                randomBoolean(), randomBoolean(), null, null);
         Analyzer analyzer = new Analyzer(
                 sqlConfig,
                 new SqlFunctionRegistry(),

+ 12 - 2
x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/plan/logical/command/sys/SysColumnsTests.java

@@ -39,6 +39,14 @@ import org.elasticsearch.xpack.sql.session.SqlSession;
 import org.elasticsearch.xpack.sql.stats.Metrics;
 import org.elasticsearch.xpack.sql.util.DateUtils;
 
+import java.sql.Types;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.function.Consumer;
+
 import static java.util.Collections.emptyList;
 import static java.util.Collections.singletonList;
 import static org.elasticsearch.action.ActionListener.wrap;
@@ -232,7 +240,8 @@ public class SysColumnsTests extends ESTestCase {
 
     private int executeCommandInOdbcModeAndCountRows(String sql) {
         final SqlConfiguration config = new SqlConfiguration(DateUtils.UTC, null, randomIntBetween(1, 15), Protocol.REQUEST_TIMEOUT,
-            Protocol.PAGE_TIMEOUT, null, null, Mode.ODBC, null, SqlVersion.fromId(Version.CURRENT.id), null, null, false, false);
+            Protocol.PAGE_TIMEOUT, null, null, Mode.ODBC, null, SqlVersion.fromId(Version.CURRENT.id), null, null,
+            false, false, null, null);
         Tuple<Command, SqlSession> tuple = sql(sql, emptyList(), config, MAPPING1);
 
         int[] rowCount = {0};
@@ -257,7 +266,8 @@ public class SysColumnsTests extends ESTestCase {
     private void executeCommand(String sql, List<SqlTypedParamValue> params, Mode mode, Consumer<SchemaRowSet> consumer,
                                 Map<String, EsField> mapping) {
         final SqlConfiguration config = new SqlConfiguration(DateUtils.UTC, null, Protocol.FETCH_SIZE, Protocol.REQUEST_TIMEOUT,
-            Protocol.PAGE_TIMEOUT, null, null, mode, null, SqlVersion.fromId(Version.CURRENT.id), null, null, false, false);
+            Protocol.PAGE_TIMEOUT, null, null, mode, null, SqlVersion.fromId(Version.CURRENT.id), null, null,
+            false, false, null, null);
         Tuple<Command, SqlSession> tuple = sql(sql, params, config, mapping);
 
         tuple.v1().execute(tuple.v2(), wrap(p -> consumer.accept((SchemaRowSet) p.rowSet()), ex -> fail(ex.getMessage())));

+ 3 - 2
x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/plan/logical/command/sys/SysTablesTests.java

@@ -25,8 +25,8 @@ import org.elasticsearch.xpack.sql.plan.logical.command.Command;
 import org.elasticsearch.xpack.sql.proto.Mode;
 import org.elasticsearch.xpack.sql.proto.Protocol;
 import org.elasticsearch.xpack.sql.proto.SqlTypedParamValue;
-import org.elasticsearch.xpack.sql.session.SqlConfiguration;
 import org.elasticsearch.xpack.sql.session.SchemaRowSet;
+import org.elasticsearch.xpack.sql.session.SqlConfiguration;
 import org.elasticsearch.xpack.sql.session.SqlSession;
 import org.elasticsearch.xpack.sql.stats.Metrics;
 import org.elasticsearch.xpack.sql.types.SqlTypesTests;
@@ -60,7 +60,8 @@ public class SysTablesTests extends ESTestCase {
     private final IndexInfo frozen = new IndexInfo(CLUSTER_NAME, "frozen", IndexType.FROZEN_INDEX);
 
     private final SqlConfiguration FROZEN_CFG = new SqlConfiguration(DateUtils.UTC, null, Protocol.FETCH_SIZE, Protocol.REQUEST_TIMEOUT,
-            Protocol.PAGE_TIMEOUT, null, null, Mode.PLAIN, null, null, null, null, false, true);
+            Protocol.PAGE_TIMEOUT, null, null, Mode.PLAIN, null, null, null, null,
+            false, true, null, null);
 
     //
     // catalog enumeration

+ 2 - 1
x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/plan/logical/command/sys/SysTypesTests.java

@@ -40,7 +40,8 @@ public class SysTypesTests extends ESTestCase {
 
     private Tuple<Command, SqlSession> sql(String sql, Mode mode, SqlVersion version) {
         SqlConfiguration configuration = new SqlConfiguration(DateUtils.UTC, null, Protocol.FETCH_SIZE,
-            Protocol.REQUEST_TIMEOUT, Protocol.PAGE_TIMEOUT, null, null, mode, null, version, null, null, false, false);
+            Protocol.REQUEST_TIMEOUT, Protocol.PAGE_TIMEOUT, null, null, mode, null, version, null,
+            null, false, false, null, null);
         EsIndex test = new EsIndex("test", SqlTypesTests.loadMapping("mapping-multi-field-with-nested.json", true));
         Analyzer analyzer = new Analyzer(configuration, new FunctionRegistry(), IndexResolution.valid(test), null);
         Command cmd = (Command) analyzer.analyze(parser.createStatement(sql), false);

+ 2 - 3
x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/plugin/CursorTests.java

@@ -14,7 +14,6 @@ import org.elasticsearch.client.Client;
 import org.elasticsearch.common.logging.LoggerMessageFormat;
 import org.elasticsearch.test.ESTestCase;
 import org.elasticsearch.xpack.sql.SqlIllegalArgumentException;
-import org.elasticsearch.xpack.sql.SqlTestUtils;
 import org.elasticsearch.xpack.sql.action.BasicFormatter;
 import org.elasticsearch.xpack.sql.action.SqlQueryResponse;
 import org.elasticsearch.xpack.sql.execution.search.ScrollCursor;
@@ -45,7 +44,7 @@ public class CursorTests extends ESTestCase {
         Client clientMock = mock(Client.class);
         Cursor cursor = Cursor.EMPTY;
         PlainActionFuture<Boolean> future = newFuture();
-        cursor.clear(SqlTestUtils.TEST_CFG, clientMock, future);
+        cursor.clear(clientMock, future);
         assertFalse(future.actionGet());
         verifyZeroInteractions(clientMock);
     }
@@ -57,7 +56,7 @@ public class CursorTests extends ESTestCase {
         String cursorString = randomAlphaOfLength(10);
         Cursor cursor = new ScrollCursor(cursorString, Collections.emptyList(), new BitSet(0), randomInt());
 
-        cursor.clear(SqlTestUtils.TEST_CFG, clientMock, listenerMock);
+        cursor.clear(clientMock, listenerMock);
 
         ArgumentCaptor<ClearScrollRequest> request = ArgumentCaptor.forClass(ClearScrollRequest.class);
         verify(clientMock).clearScroll(request.capture(), any(ActionListener.class));