diff --git a/modules/aggregations/src/internalClusterTest/java/org/elasticsearch/aggregations/bucket/SearchCancellationIT.java b/modules/aggregations/src/internalClusterTest/java/org/elasticsearch/aggregations/bucket/SearchCancellationIT.java index 714af4d8c40d..a6e530a9d66c 100644 --- a/modules/aggregations/src/internalClusterTest/java/org/elasticsearch/aggregations/bucket/SearchCancellationIT.java +++ b/modules/aggregations/src/internalClusterTest/java/org/elasticsearch/aggregations/bucket/SearchCancellationIT.java @@ -12,9 +12,9 @@ import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.action.ActionFuture; import org.elasticsearch.action.DocWriteRequest; import org.elasticsearch.action.bulk.BulkRequestBuilder; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchPhaseExecutionException; import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.action.support.WriteRequest; import org.elasticsearch.aggregations.AggregationsPlugin; import org.elasticsearch.aggregations.bucket.timeseries.TimeSeriesAggregationBuilder; @@ -115,7 +115,7 @@ public class SearchCancellationIT extends AbstractSearchCancellationTestCase { ) .execute(); awaitForBlock(plugins); - cancelSearch(SearchAction.NAME); + cancelSearch(TransportSearchAction.TYPE.name()); disableBlocks(plugins); SearchPhaseExecutionException ex = expectThrows(SearchPhaseExecutionException.class, searchResponse::actionGet); diff --git a/modules/reindex/src/test/java/org/elasticsearch/reindex/ClientScrollableHitSourceTests.java b/modules/reindex/src/test/java/org/elasticsearch/reindex/ClientScrollableHitSourceTests.java index 406b815bdd44..0ad1867e7505 100644 --- a/modules/reindex/src/test/java/org/elasticsearch/reindex/ClientScrollableHitSourceTests.java +++ b/modules/reindex/src/test/java/org/elasticsearch/reindex/ClientScrollableHitSourceTests.java @@ -14,11 +14,11 @@ import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionResponse; import org.elasticsearch.action.ActionType; import org.elasticsearch.action.bulk.BackoffPolicy; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchResponse; -import org.elasticsearch.action.search.SearchScrollAction; import org.elasticsearch.action.search.SearchScrollRequest; +import org.elasticsearch.action.search.TransportSearchAction; +import org.elasticsearch.action.search.TransportSearchScrollAction; import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.client.internal.ParentTaskAssigningClient; import org.elasticsearch.client.internal.support.AbstractClient; @@ -102,17 +102,17 @@ public class ClientScrollableHitSourceTests extends ESTestCase { hitSource.start(); for (int retry = 0; retry < randomIntBetween(minFailures, maxFailures); ++retry) { - client.fail(SearchAction.INSTANCE, new EsRejectedExecutionException()); + client.fail(TransportSearchAction.TYPE, new EsRejectedExecutionException()); if (retry >= retries) { return; } client.awaitOperation(); ++expectedSearchRetries; } - client.validateRequest(SearchAction.INSTANCE, (SearchRequest r) -> assertTrue(r.allowPartialSearchResults() == Boolean.FALSE)); + client.validateRequest(TransportSearchAction.TYPE, (SearchRequest r) -> assertTrue(r.allowPartialSearchResults() == Boolean.FALSE)); SearchResponse searchResponse = createSearchResponse(); try { - client.respond(SearchAction.INSTANCE, searchResponse); + client.respond(TransportSearchAction.TYPE, searchResponse); for (int i = 0; i < randomIntBetween(1, 10); ++i) { ScrollableHitSource.AsyncResponse asyncResponse = responses.poll(10, TimeUnit.SECONDS); @@ -122,13 +122,13 @@ public class ClientScrollableHitSourceTests extends ESTestCase { asyncResponse.done(TimeValue.ZERO); for (int retry = 0; retry < randomIntBetween(minFailures, maxFailures); ++retry) { - client.fail(SearchScrollAction.INSTANCE, new EsRejectedExecutionException()); + client.fail(TransportSearchScrollAction.TYPE, new EsRejectedExecutionException()); client.awaitOperation(); ++expectedSearchRetries; } searchResponse = createSearchResponse(); - client.respond(SearchScrollAction.INSTANCE, searchResponse); + client.respond(TransportSearchScrollAction.TYPE, searchResponse); } assertEquals(actualSearchRetries.get(), expectedSearchRetries); @@ -154,7 +154,10 @@ public class ClientScrollableHitSourceTests extends ESTestCase { ); hitSource.startNextScroll(timeValueSeconds(100)); - client.validateRequest(SearchScrollAction.INSTANCE, (SearchScrollRequest r) -> assertEquals(r.scroll().keepAlive().seconds(), 110)); + client.validateRequest( + TransportSearchScrollAction.TYPE, + (SearchScrollRequest r) -> assertEquals(r.scroll().keepAlive().seconds(), 110) + ); } private SearchResponse createSearchResponse() { diff --git a/modules/reindex/src/test/java/org/elasticsearch/reindex/ReindexFromRemoteWithAuthTests.java b/modules/reindex/src/test/java/org/elasticsearch/reindex/ReindexFromRemoteWithAuthTests.java index cf06a6d8dcb0..a68c390b8bd8 100644 --- a/modules/reindex/src/test/java/org/elasticsearch/reindex/ReindexFromRemoteWithAuthTests.java +++ b/modules/reindex/src/test/java/org/elasticsearch/reindex/ReindexFromRemoteWithAuthTests.java @@ -15,7 +15,7 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionResponse; import org.elasticsearch.action.admin.cluster.node.info.NodeInfo; -import org.elasticsearch.action.search.SearchAction; +import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.action.support.ActionFilter; import org.elasticsearch.action.support.ActionFilterChain; import org.elasticsearch.action.support.WriteRequest.RefreshPolicy; @@ -199,7 +199,7 @@ public class ReindexFromRemoteWithAuthTests extends ESSingleNodeTestCase { ActionListener listener, ActionFilterChain chain ) { - if (false == action.equals(SearchAction.NAME)) { + if (false == action.equals(TransportSearchAction.TYPE.name())) { chain.proceed(task, action, request, listener); return; } diff --git a/qa/ccs-unavailable-clusters/src/javaRestTest/java/org/elasticsearch/search/CrossClusterSearchUnavailableClusterIT.java b/qa/ccs-unavailable-clusters/src/javaRestTest/java/org/elasticsearch/search/CrossClusterSearchUnavailableClusterIT.java index c93ed4e39829..b17b81b6ac18 100644 --- a/qa/ccs-unavailable-clusters/src/javaRestTest/java/org/elasticsearch/search/CrossClusterSearchUnavailableClusterIT.java +++ b/qa/ccs-unavailable-clusters/src/javaRestTest/java/org/elasticsearch/search/CrossClusterSearchUnavailableClusterIT.java @@ -16,13 +16,13 @@ import org.elasticsearch.TransportVersion; import org.elasticsearch.action.admin.cluster.state.ClusterStateAction; import org.elasticsearch.action.admin.cluster.state.ClusterStateRequest; import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchResponse; -import org.elasticsearch.action.search.SearchShardsAction; import org.elasticsearch.action.search.SearchShardsRequest; import org.elasticsearch.action.search.SearchShardsResponse; import org.elasticsearch.action.search.ShardSearchFailure; +import org.elasticsearch.action.search.TransportSearchAction; +import org.elasticsearch.action.search.TransportSearchShardsAction; import org.elasticsearch.client.Request; import org.elasticsearch.client.Response; import org.elasticsearch.client.ResponseException; @@ -81,7 +81,7 @@ public class CrossClusterSearchUnavailableClusterIT extends ESRestTestCase { MockTransportService newService = MockTransportService.createNewService(s, version, transportVersion, threadPool, null); try { newService.registerRequestHandler( - SearchShardsAction.NAME, + TransportSearchShardsAction.TYPE.name(), EsExecutors.DIRECT_EXECUTOR_SERVICE, SearchShardsRequest::new, (request, channel, task) -> { @@ -89,7 +89,7 @@ public class CrossClusterSearchUnavailableClusterIT extends ESRestTestCase { } ); newService.registerRequestHandler( - SearchAction.NAME, + TransportSearchAction.TYPE.name(), EsExecutors.DIRECT_EXECUTOR_SERVICE, SearchRequest::new, (request, channel, task) -> { diff --git a/qa/smoke-test-http/src/javaRestTest/java/org/elasticsearch/http/SearchRestCancellationIT.java b/qa/smoke-test-http/src/javaRestTest/java/org/elasticsearch/http/SearchRestCancellationIT.java index 798464bf6d2c..73dd1525f8a0 100644 --- a/qa/smoke-test-http/src/javaRestTest/java/org/elasticsearch/http/SearchRestCancellationIT.java +++ b/qa/smoke-test-http/src/javaRestTest/java/org/elasticsearch/http/SearchRestCancellationIT.java @@ -15,10 +15,10 @@ import org.elasticsearch.action.admin.cluster.node.info.NodeInfo; import org.elasticsearch.action.admin.cluster.node.info.NodesInfoResponse; import org.elasticsearch.action.admin.cluster.node.tasks.list.ListTasksResponse; import org.elasticsearch.action.bulk.BulkRequestBuilder; -import org.elasticsearch.action.search.MultiSearchAction; import org.elasticsearch.action.search.MultiSearchRequest; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchRequest; +import org.elasticsearch.action.search.TransportMultiSearchAction; +import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.action.support.WriteRequest; import org.elasticsearch.client.Cancellable; @@ -73,7 +73,7 @@ public class SearchRestCancellationIT extends HttpSmokeTestCase { scriptQuery(new Script(ScriptType.INLINE, "mockscript", ScriptedBlockPlugin.SCRIPT_NAME, Collections.emptyMap())) ); searchRequest.setJsonEntity(Strings.toString(searchSource)); - verifyCancellationDuringQueryPhase(SearchAction.NAME, searchRequest); + verifyCancellationDuringQueryPhase(TransportSearchAction.TYPE.name(), searchRequest); } public void testAutomaticCancellationMultiSearchDuringQueryPhase() throws Exception { @@ -89,7 +89,7 @@ public class SearchRestCancellationIT extends HttpSmokeTestCase { Request restRequest = new Request("POST", "/_msearch"); byte[] requestBody = MultiSearchRequest.writeMultiLineFormat(multiSearchRequest, contentType.xContent()); restRequest.setEntity(new NByteArrayEntity(requestBody, createContentType(contentType))); - verifyCancellationDuringQueryPhase(MultiSearchAction.NAME, restRequest); + verifyCancellationDuringQueryPhase(TransportMultiSearchAction.TYPE.name(), restRequest); } void verifyCancellationDuringQueryPhase(String searchAction, Request searchRequest) throws Exception { @@ -116,7 +116,7 @@ public class SearchRestCancellationIT extends HttpSmokeTestCase { new Script(ScriptType.INLINE, "mockscript", ScriptedBlockPlugin.SCRIPT_NAME, Collections.emptyMap()) ); searchRequest.setJsonEntity(Strings.toString(searchSource)); - verifyCancellationDuringFetchPhase(SearchAction.NAME, searchRequest); + verifyCancellationDuringFetchPhase(TransportSearchAction.TYPE.name(), searchRequest); } public void testAutomaticCancellationMultiSearchDuringFetchPhase() throws Exception { @@ -132,7 +132,7 @@ public class SearchRestCancellationIT extends HttpSmokeTestCase { Request restRequest = new Request("POST", "/_msearch"); byte[] requestBody = MultiSearchRequest.writeMultiLineFormat(multiSearchRequest, contentType.xContent()); restRequest.setEntity(new NByteArrayEntity(requestBody, createContentType(contentType))); - verifyCancellationDuringFetchPhase(MultiSearchAction.NAME, restRequest); + verifyCancellationDuringFetchPhase(TransportMultiSearchAction.TYPE.name(), restRequest); } void verifyCancellationDuringFetchPhase(String searchAction, Request searchRequest) throws Exception { diff --git a/server/src/internalClusterTest/java/org/elasticsearch/action/IndicesRequestIT.java b/server/src/internalClusterTest/java/org/elasticsearch/action/IndicesRequestIT.java index fab886350364..e01241da4db9 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/action/IndicesRequestIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/action/IndicesRequestIT.java @@ -45,8 +45,8 @@ import org.elasticsearch.action.admin.indices.validate.query.ValidateQueryReques import org.elasticsearch.action.bulk.BulkAction; import org.elasticsearch.action.bulk.BulkRequest; import org.elasticsearch.action.delete.DeleteRequest; -import org.elasticsearch.action.explain.ExplainAction; import org.elasticsearch.action.explain.ExplainRequest; +import org.elasticsearch.action.explain.TransportExplainAction; import org.elasticsearch.action.fieldcaps.FieldCapabilitiesAction; import org.elasticsearch.action.fieldcaps.FieldCapabilitiesRequest; import org.elasticsearch.action.get.GetAction; @@ -317,7 +317,7 @@ public class IndicesRequestIT extends ESIntegTestCase { } public void testExplain() { - String explainShardAction = ExplainAction.NAME + "[s]"; + String explainShardAction = TransportExplainAction.TYPE.name() + "[s]"; interceptTransportActions(explainShardAction); ExplainRequest explainRequest = new ExplainRequest(randomIndexOrAlias(), "id").query(QueryBuilders.matchAllQuery()); diff --git a/server/src/internalClusterTest/java/org/elasticsearch/action/admin/cluster/node/tasks/TasksIT.java b/server/src/internalClusterTest/java/org/elasticsearch/action/admin/cluster/node/tasks/TasksIT.java index b7789eca6b3f..502c60b4a340 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/action/admin/cluster/node/tasks/TasksIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/action/admin/cluster/node/tasks/TasksIT.java @@ -26,8 +26,8 @@ import org.elasticsearch.action.admin.indices.refresh.RefreshAction; import org.elasticsearch.action.admin.indices.validate.query.ValidateQueryAction; import org.elasticsearch.action.bulk.BulkAction; import org.elasticsearch.action.index.IndexAction; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchTransportService; +import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.action.support.WriteRequest; import org.elasticsearch.action.support.replication.ReplicationResponse; import org.elasticsearch.action.support.replication.TransportReplicationActionTests; @@ -349,8 +349,8 @@ public class TasksIT extends ESIntegTestCase { } public void testSearchTaskDescriptions() { - registerTaskManagerListeners(SearchAction.NAME); // main task - registerTaskManagerListeners(SearchAction.NAME + "[*]"); // shard task + registerTaskManagerListeners(TransportSearchAction.TYPE.name()); // main task + registerTaskManagerListeners(TransportSearchAction.TYPE.name() + "[*]"); // shard task createIndex("test"); ensureGreen("test"); // Make sure all shards are allocated to catch replication tasks prepareIndex("test").setId("test_id") @@ -365,14 +365,14 @@ public class TasksIT extends ESIntegTestCase { assertNoFailures(client().filterWithHeader(headers).prepareSearch("test").setQuery(QueryBuilders.matchAllQuery())); // the search operation should produce one main task - List mainTask = findEvents(SearchAction.NAME, Tuple::v1); + List mainTask = findEvents(TransportSearchAction.TYPE.name(), Tuple::v1); assertEquals(1, mainTask.size()); assertThat(mainTask.get(0).description(), startsWith("indices[test], search_type[")); assertThat(mainTask.get(0).description(), containsString("\"query\":{\"match_all\"")); assertTaskHeaders(mainTask.get(0)); // check that if we have any shard-level requests they all have non-zero length description - List shardTasks = findEvents(SearchAction.NAME + "[*]", Tuple::v1); + List shardTasks = findEvents(TransportSearchAction.TYPE.name() + "[*]", Tuple::v1); for (TaskInfo taskInfo : shardTasks) { assertThat(taskInfo.parentTaskId(), notNullValue()); assertEquals(mainTask.get(0).taskId(), taskInfo.parentTaskId()); diff --git a/server/src/internalClusterTest/java/org/elasticsearch/action/search/CCSPointInTimeIT.java b/server/src/internalClusterTest/java/org/elasticsearch/action/search/CCSPointInTimeIT.java index 36e544af90bc..eff681f1f281 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/action/search/CCSPointInTimeIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/action/search/CCSPointInTimeIT.java @@ -202,11 +202,11 @@ public class CCSPointInTimeIT extends AbstractMultiClustersTestCase { private String openPointInTime(String[] indices, TimeValue keepAlive) { OpenPointInTimeRequest request = new OpenPointInTimeRequest(indices).keepAlive(keepAlive); - final OpenPointInTimeResponse response = client().execute(OpenPointInTimeAction.INSTANCE, request).actionGet(); + final OpenPointInTimeResponse response = client().execute(TransportOpenPointInTimeAction.TYPE, request).actionGet(); return response.getPointInTimeId(); } private void closePointInTime(String readerId) { - client().execute(ClosePointInTimeAction.INSTANCE, new ClosePointInTimeRequest(readerId)).actionGet(); + client().execute(TransportClosePointInTimeAction.TYPE, new ClosePointInTimeRequest(readerId)).actionGet(); } } diff --git a/server/src/internalClusterTest/java/org/elasticsearch/action/search/PointInTimeIT.java b/server/src/internalClusterTest/java/org/elasticsearch/action/search/PointInTimeIT.java index f517f82dcbe7..d3e312e173c2 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/action/search/PointInTimeIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/action/search/PointInTimeIT.java @@ -168,7 +168,7 @@ public class PointInTimeIT extends ESIntegTestCase { { OpenPointInTimeRequest request = new OpenPointInTimeRequest("*").keepAlive(TimeValue.timeValueMinutes(2)); - final OpenPointInTimeResponse response = client().execute(OpenPointInTimeAction.INSTANCE, request).actionGet(); + final OpenPointInTimeResponse response = client().execute(TransportOpenPointInTimeAction.TYPE, request).actionGet(); try { SearchContextId searchContextId = SearchContextId.decode(writableRegistry(), response.getPointInTimeId()); String[] actualIndices = searchContextId.getActualIndices(); @@ -180,7 +180,7 @@ public class PointInTimeIT extends ESIntegTestCase { { OpenPointInTimeRequest request = new OpenPointInTimeRequest("*").keepAlive(TimeValue.timeValueMinutes(2)); request.indexFilter(new RangeQueryBuilder("@timestamp").gte("2023-03-01")); - final OpenPointInTimeResponse response = client().execute(OpenPointInTimeAction.INSTANCE, request).actionGet(); + final OpenPointInTimeResponse response = client().execute(TransportOpenPointInTimeAction.TYPE, request).actionGet(); String pitId = response.getPointInTimeId(); try { SearchContextId searchContextId = SearchContextId.decode(writableRegistry(), pitId); @@ -333,8 +333,8 @@ public class PointInTimeIT extends ESIntegTestCase { public void testAllowNoIndex() { var request = new OpenPointInTimeRequest("my_index").indicesOptions(IndicesOptions.LENIENT_EXPAND_OPEN) .keepAlive(TimeValue.timeValueMinutes(between(1, 10))); - String pit = client().execute(OpenPointInTimeAction.INSTANCE, request).actionGet().getPointInTimeId(); - var closeResp = client().execute(ClosePointInTimeAction.INSTANCE, new ClosePointInTimeRequest(pit)).actionGet(); + String pit = client().execute(TransportOpenPointInTimeAction.TYPE, request).actionGet().getPointInTimeId(); + var closeResp = client().execute(TransportClosePointInTimeAction.TYPE, new ClosePointInTimeRequest(pit)).actionGet(); assertThat(closeResp.status(), equalTo(RestStatus.OK)); } @@ -478,8 +478,11 @@ public class PointInTimeIT extends ESIntegTestCase { } public void testCloseInvalidPointInTime() { - expectThrows(Exception.class, () -> client().execute(ClosePointInTimeAction.INSTANCE, new ClosePointInTimeRequest("")).actionGet()); - List tasks = clusterAdmin().prepareListTasks().setActions(ClosePointInTimeAction.NAME).get().getTasks(); + expectThrows( + Exception.class, + () -> client().execute(TransportClosePointInTimeAction.TYPE, new ClosePointInTimeRequest("")).actionGet() + ); + List tasks = clusterAdmin().prepareListTasks().setActions(TransportClosePointInTimeAction.TYPE.name()).get().getTasks(); assertThat(tasks, empty()); } @@ -520,7 +523,7 @@ public class PointInTimeIT extends ESIntegTestCase { OpenPointInTimeRequest request = new OpenPointInTimeRequest("test").keepAlive(TimeValue.timeValueMinutes(1)); request.maxConcurrentShardRequests(maxConcurrentRequests); PlainActionFuture future = new PlainActionFuture<>(); - client().execute(OpenPointInTimeAction.INSTANCE, request, future); + client().execute(TransportOpenPointInTimeAction.TYPE, request, future); assertTrue(sentLatch.await(1, TimeUnit.MINUTES)); readyLatch.countDown(); closePointInTime(future.actionGet().getPointInTimeId()); @@ -584,11 +587,11 @@ public class PointInTimeIT extends ESIntegTestCase { private String openPointInTime(String[] indices, TimeValue keepAlive) { OpenPointInTimeRequest request = new OpenPointInTimeRequest(indices).keepAlive(keepAlive); - final OpenPointInTimeResponse response = client().execute(OpenPointInTimeAction.INSTANCE, request).actionGet(); + final OpenPointInTimeResponse response = client().execute(TransportOpenPointInTimeAction.TYPE, request).actionGet(); return response.getPointInTimeId(); } private void closePointInTime(String readerId) { - client().execute(ClosePointInTimeAction.INSTANCE, new ClosePointInTimeRequest(readerId)).actionGet(); + client().execute(TransportClosePointInTimeAction.TYPE, new ClosePointInTimeRequest(readerId)).actionGet(); } } diff --git a/server/src/internalClusterTest/java/org/elasticsearch/action/search/SearchProgressActionListenerIT.java b/server/src/internalClusterTest/java/org/elasticsearch/action/search/SearchProgressActionListenerIT.java index df36f1babd36..227a3b861233 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/action/search/SearchProgressActionListenerIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/action/search/SearchProgressActionListenerIT.java @@ -162,7 +162,7 @@ public class SearchProgressActionListenerIT extends ESSingleNodeTestCase { throw new AssertionError(); } }; - client.executeLocally(SearchAction.INSTANCE, new SearchRequest(request) { + client.executeLocally(TransportSearchAction.TYPE, new SearchRequest(request) { @Override public SearchTask createTask(long id, String type, String action, TaskId parentTaskId, Map headers) { SearchTask task = super.createTask(id, type, action, parentTaskId, headers); diff --git a/server/src/internalClusterTest/java/org/elasticsearch/action/search/SearchShardsIT.java b/server/src/internalClusterTest/java/org/elasticsearch/action/search/SearchShardsIT.java index e1957c822288..8b1acf11a7a5 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/action/search/SearchShardsIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/action/search/SearchShardsIT.java @@ -70,7 +70,7 @@ public class SearchShardsIT extends ESIntegTestCase { randomBoolean(), randomBoolean() ? null : randomAlphaOfLength(10) ); - var resp = client().execute(SearchShardsAction.INSTANCE, request).actionGet(); + var resp = client().execute(TransportSearchShardsAction.TYPE, request).actionGet(); assertThat(resp.getGroups(), hasSize(indicesWithData + indicesWithoutData)); int skipped = 0; for (SearchShardsGroup g : resp.getGroups()) { @@ -97,7 +97,7 @@ public class SearchShardsIT extends ESIntegTestCase { randomBoolean(), randomBoolean() ? null : randomAlphaOfLength(10) ); - SearchShardsResponse resp = client().execute(SearchShardsAction.INSTANCE, request).actionGet(); + SearchShardsResponse resp = client().execute(TransportSearchShardsAction.TYPE, request).actionGet(); assertThat(resp.getGroups(), hasSize(indicesWithData + indicesWithoutData)); for (SearchShardsGroup g : resp.getGroups()) { assertFalse(g.skipped()); @@ -137,7 +137,7 @@ public class SearchShardsIT extends ESIntegTestCase { randomBoolean(), randomBoolean() ? null : randomAlphaOfLength(10) ); - var searchShardsResponse = client().execute(SearchShardsAction.INSTANCE, searchShardsRequest).actionGet(); + var searchShardsResponse = client().execute(TransportSearchShardsAction.TYPE, searchShardsRequest).actionGet(); assertThat(searchShardsResponse.getGroups(), hasSize(searchResponse.getTotalShards())); long skippedShards = searchShardsResponse.getGroups().stream().filter(SearchShardsGroup::skipped).count(); @@ -182,7 +182,7 @@ public class SearchShardsIT extends ESIntegTestCase { randomBoolean(), null ); - SearchShardsResponse resp = client().execute(SearchShardsAction.INSTANCE, request).actionGet(); + SearchShardsResponse resp = client().execute(TransportSearchShardsAction.TYPE, request).actionGet(); assertThat(resp.getGroups(), hasSize(totalShards)); for (SearchShardsGroup group : resp.getGroups()) { assertFalse(group.skipped()); diff --git a/server/src/internalClusterTest/java/org/elasticsearch/cluster/routing/ShardRoutingRoleIT.java b/server/src/internalClusterTest/java/org/elasticsearch/cluster/routing/ShardRoutingRoleIT.java index 1319de634b42..3418874bd590 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/cluster/routing/ShardRoutingRoleIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/cluster/routing/ShardRoutingRoleIT.java @@ -14,10 +14,10 @@ import org.elasticsearch.action.ActionResponse; import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsGroup; import org.elasticsearch.action.admin.indices.refresh.RefreshResponse; import org.elasticsearch.action.admin.indices.refresh.TransportUnpromotableShardRefreshAction; -import org.elasticsearch.action.search.ClosePointInTimeAction; import org.elasticsearch.action.search.ClosePointInTimeRequest; -import org.elasticsearch.action.search.OpenPointInTimeAction; import org.elasticsearch.action.search.OpenPointInTimeRequest; +import org.elasticsearch.action.search.TransportClosePointInTimeAction; +import org.elasticsearch.action.search.TransportOpenPointInTimeAction; import org.elasticsearch.cluster.ClusterChangedEvent; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterStateListener; @@ -522,7 +522,7 @@ public class ShardRoutingRoleIT extends ESIntegTestCase { // do nothing } } - String pitId = client().execute(OpenPointInTimeAction.INSTANCE, openRequest).actionGet().getPointInTimeId(); + String pitId = client().execute(TransportOpenPointInTimeAction.TYPE, openRequest).actionGet().getPointInTimeId(); try { final var profileResults = prepareSearch().setPointInTime(new PointInTimeBuilder(pitId)) .setProfile(true) @@ -533,7 +533,7 @@ public class ShardRoutingRoleIT extends ESIntegTestCase { assertThat(profileKey, in(searchShardProfileKeys)); } } finally { - client().execute(ClosePointInTimeAction.INSTANCE, new ClosePointInTimeRequest(pitId)); + client().execute(TransportClosePointInTimeAction.TYPE, new ClosePointInTimeRequest(pitId)); } } // search-shards API diff --git a/server/src/internalClusterTest/java/org/elasticsearch/search/SearchCancellationIT.java b/server/src/internalClusterTest/java/org/elasticsearch/search/SearchCancellationIT.java index 3dd90b096c63..ad610954e86b 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/search/SearchCancellationIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/search/SearchCancellationIT.java @@ -11,17 +11,17 @@ package org.elasticsearch.search; import org.apache.lucene.tests.util.LuceneTestCase; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.action.ActionFuture; -import org.elasticsearch.action.search.MultiSearchAction; import org.elasticsearch.action.search.MultiSearchResponse; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchPhaseExecutionException; import org.elasticsearch.action.search.SearchResponse; -import org.elasticsearch.action.search.SearchScrollAction; import org.elasticsearch.action.search.SearchShardTask; import org.elasticsearch.action.search.SearchTask; import org.elasticsearch.action.search.SearchTransportService; import org.elasticsearch.action.search.SearchType; import org.elasticsearch.action.search.ShardSearchFailure; +import org.elasticsearch.action.search.TransportMultiSearchAction; +import org.elasticsearch.action.search.TransportSearchAction; +import org.elasticsearch.action.search.TransportSearchScrollAction; import org.elasticsearch.common.Strings; import org.elasticsearch.core.TimeValue; import org.elasticsearch.script.Script; @@ -71,7 +71,7 @@ public class SearchCancellationIT extends AbstractSearchCancellationTestCase { ).execute(); awaitForBlock(plugins); - cancelSearch(SearchAction.NAME); + cancelSearch(TransportSearchAction.TYPE.name()); disableBlocks(plugins); logger.info("Segments {}", Strings.toString(indicesAdmin().prepareSegments("test").get())); ensureSearchWasCancelled(searchResponse); @@ -89,7 +89,7 @@ public class SearchCancellationIT extends AbstractSearchCancellationTestCase { ).execute(); awaitForBlock(plugins); - cancelSearch(SearchAction.NAME); + cancelSearch(TransportSearchAction.TYPE.name()); disableBlocks(plugins); logger.info("Segments {}", Strings.toString(indicesAdmin().prepareSegments("test").get())); ensureSearchWasCancelled(searchResponse); @@ -134,7 +134,7 @@ public class SearchCancellationIT extends AbstractSearchCancellationTestCase { ) .execute(); awaitForBlock(plugins); - cancelSearch(SearchAction.NAME); + cancelSearch(TransportSearchAction.TYPE.name()); disableBlocks(plugins); ensureSearchWasCancelled(searchResponse); } @@ -151,7 +151,7 @@ public class SearchCancellationIT extends AbstractSearchCancellationTestCase { .execute(); awaitForBlock(plugins); - cancelSearch(SearchAction.NAME); + cancelSearch(TransportSearchAction.TYPE.name()); disableBlocks(plugins); SearchResponse response = ensureSearchWasCancelled(searchResponse); if (response != null) { @@ -191,7 +191,7 @@ public class SearchCancellationIT extends AbstractSearchCancellationTestCase { .execute(); awaitForBlock(plugins); - cancelSearch(SearchScrollAction.NAME); + cancelSearch(TransportSearchScrollAction.TYPE.name()); disableBlocks(plugins); SearchResponse response = ensureSearchWasCancelled(scrollResponse); @@ -215,7 +215,7 @@ public class SearchCancellationIT extends AbstractSearchCancellationTestCase { ) .execute(); awaitForBlock(plugins); - cancelSearch(MultiSearchAction.NAME); + cancelSearch(TransportMultiSearchAction.TYPE.name()); disableBlocks(plugins); for (MultiSearchResponse.Item item : msearchResponse.actionGet()) { if (item.getFailure() != null) { @@ -302,7 +302,7 @@ public class SearchCancellationIT extends AbstractSearchCancellationTestCase { for (String nodeName : internalCluster().getNodeNames()) { TransportService transportService = internalCluster().getInstance(TransportService.class, nodeName); for (Task task : transportService.getTaskManager().getCancellableTasks().values()) { - if (task.getAction().equals(SearchAction.NAME)) { + if (task.getAction().equals(TransportSearchAction.TYPE.name())) { tasks.add((SearchTask) task); } } diff --git a/server/src/internalClusterTest/java/org/elasticsearch/search/ccs/CrossClusterIT.java b/server/src/internalClusterTest/java/org/elasticsearch/search/ccs/CrossClusterIT.java index cc40003c1001..cf8d81f406f9 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/search/ccs/CrossClusterIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/search/ccs/CrossClusterIT.java @@ -13,13 +13,13 @@ import org.elasticsearch.action.ActionFuture; import org.elasticsearch.action.admin.cluster.node.tasks.cancel.CancelTasksRequest; import org.elasticsearch.action.admin.cluster.node.tasks.cancel.CancelTasksResponse; import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchResponse; -import org.elasticsearch.action.search.SearchShardsAction; import org.elasticsearch.action.search.SearchShardsGroup; import org.elasticsearch.action.search.SearchShardsRequest; import org.elasticsearch.action.search.SearchShardsResponse; +import org.elasticsearch.action.search.TransportSearchAction; +import org.elasticsearch.action.search.TransportSearchShardsAction; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.action.support.WriteRequest; @@ -240,7 +240,7 @@ public class CrossClusterIT extends AbstractMultiClustersTestCase { final TaskInfo rootTask = client().admin() .cluster() .prepareListTasks() - .setActions(SearchAction.INSTANCE.name()) + .setActions(TransportSearchAction.TYPE.name()) .get() .getTasks() .stream() @@ -274,7 +274,7 @@ public class CrossClusterIT extends AbstractMultiClustersTestCase { for (TransportService transportService : transportServices) { Collection cancellableTasks = transportService.getTaskManager().getCancellableTasks().values(); for (CancellableTask cancellableTask : cancellableTasks) { - if (cancellableTask.getAction().contains(SearchAction.INSTANCE.name())) { + if (cancellableTask.getAction().contains(TransportSearchAction.TYPE.name())) { assertTrue(cancellableTask.getDescription(), cancellableTask.isCancelled()); } } @@ -528,7 +528,7 @@ public class CrossClusterIT extends AbstractMultiClustersTestCase { { QueryBuilder query = new TermQueryBuilder("_index", "cluster_a:my_index"); SearchShardsRequest request = new SearchShardsRequest(indices, indicesOptions, query, null, null, randomBoolean(), "cluster_a"); - SearchShardsResponse resp = remoteClient.execute(SearchShardsAction.INSTANCE, request).actionGet(); + SearchShardsResponse resp = remoteClient.execute(TransportSearchShardsAction.TYPE, request).actionGet(); assertThat(resp.getGroups(), hasSize(numShards)); for (SearchShardsGroup group : resp.getGroups()) { assertFalse(group.skipped()); @@ -545,7 +545,7 @@ public class CrossClusterIT extends AbstractMultiClustersTestCase { randomBoolean(), randomFrom("cluster_b", null) ); - SearchShardsResponse resp = remoteClient.execute(SearchShardsAction.INSTANCE, request).actionGet(); + SearchShardsResponse resp = remoteClient.execute(TransportSearchShardsAction.TYPE, request).actionGet(); assertThat(resp.getGroups(), hasSize(numShards)); for (SearchShardsGroup group : resp.getGroups()) { assertTrue(group.skipped()); @@ -562,7 +562,7 @@ public class CrossClusterIT extends AbstractMultiClustersTestCase { randomBoolean(), randomFrom("cluster_a", "cluster_b", null) ); - SearchShardsResponse resp = remoteClient.execute(SearchShardsAction.INSTANCE, request).actionGet(); + SearchShardsResponse resp = remoteClient.execute(TransportSearchShardsAction.TYPE, request).actionGet(); assertThat(resp.getGroups(), hasSize(numShards)); for (SearchShardsGroup group : resp.getGroups()) { assertTrue(group.skipped()); diff --git a/server/src/internalClusterTest/java/org/elasticsearch/search/searchafter/SearchAfterIT.java b/server/src/internalClusterTest/java/org/elasticsearch/search/searchafter/SearchAfterIT.java index 8852da02a975..6219c1b72253 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/search/searchafter/SearchAfterIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/search/searchafter/SearchAfterIT.java @@ -12,15 +12,15 @@ import org.elasticsearch.action.admin.indices.create.CreateIndexRequestBuilder; import org.elasticsearch.action.bulk.BulkRequestBuilder; import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.index.IndexRequestBuilder; -import org.elasticsearch.action.search.ClosePointInTimeAction; import org.elasticsearch.action.search.ClosePointInTimeRequest; -import org.elasticsearch.action.search.OpenPointInTimeAction; import org.elasticsearch.action.search.OpenPointInTimeRequest; import org.elasticsearch.action.search.SearchPhaseExecutionException; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchRequestBuilder; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.ShardSearchFailure; +import org.elasticsearch.action.search.TransportClosePointInTimeAction; +import org.elasticsearch.action.search.TransportOpenPointInTimeAction; import org.elasticsearch.action.support.WriteRequest; import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.common.Randomness; @@ -456,7 +456,7 @@ public class SearchAfterIT extends ESIntegTestCase { String pitID; { OpenPointInTimeRequest openPITRequest = new OpenPointInTimeRequest("test").keepAlive(TimeValue.timeValueMinutes(5)); - pitID = client().execute(OpenPointInTimeAction.INSTANCE, openPITRequest).actionGet().getPointInTimeId(); + pitID = client().execute(TransportOpenPointInTimeAction.TYPE, openPITRequest).actionGet().getPointInTimeId(); SearchRequest searchRequest = new SearchRequest("test").source( new SearchSourceBuilder().pointInTimeBuilder(new PointInTimeBuilder(pitID).setKeepAlive(TimeValue.timeValueMinutes(5))) .sort("timestamp") @@ -483,14 +483,14 @@ public class SearchAfterIT extends ESIntegTestCase { } while (resp.getHits().getHits().length > 0); assertThat(foundHits, equalTo(timestamps.size())); } finally { - client().execute(ClosePointInTimeAction.INSTANCE, new ClosePointInTimeRequest(pitID)).actionGet(); + client().execute(TransportClosePointInTimeAction.TYPE, new ClosePointInTimeRequest(pitID)).actionGet(); } } // search_after without sort with point in time { OpenPointInTimeRequest openPITRequest = new OpenPointInTimeRequest("test").keepAlive(TimeValue.timeValueMinutes(5)); - pitID = client().execute(OpenPointInTimeAction.INSTANCE, openPITRequest).actionGet().getPointInTimeId(); + pitID = client().execute(TransportOpenPointInTimeAction.TYPE, openPITRequest).actionGet().getPointInTimeId(); SearchRequest searchRequest = new SearchRequest("test").source( new SearchSourceBuilder().pointInTimeBuilder(new PointInTimeBuilder(pitID).setKeepAlive(TimeValue.timeValueMinutes(5))) .sort(SortBuilders.pitTiebreaker()) @@ -517,7 +517,7 @@ public class SearchAfterIT extends ESIntegTestCase { Collections.sort(foundSeqNos); assertThat(foundSeqNos, equalTo(timestamps)); } finally { - client().execute(ClosePointInTimeAction.INSTANCE, new ClosePointInTimeRequest(pitID)).actionGet(); + client().execute(TransportClosePointInTimeAction.TYPE, new ClosePointInTimeRequest(pitID)).actionGet(); } } } diff --git a/server/src/internalClusterTest/java/org/elasticsearch/search/slice/SearchSliceIT.java b/server/src/internalClusterTest/java/org/elasticsearch/search/slice/SearchSliceIT.java index 2c836e7763ff..527d8bed8bc6 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/search/slice/SearchSliceIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/search/slice/SearchSliceIT.java @@ -10,14 +10,14 @@ package org.elasticsearch.search.slice; import org.elasticsearch.action.admin.indices.alias.IndicesAliasesRequest; import org.elasticsearch.action.index.IndexRequestBuilder; -import org.elasticsearch.action.search.ClosePointInTimeAction; import org.elasticsearch.action.search.ClosePointInTimeRequest; -import org.elasticsearch.action.search.OpenPointInTimeAction; import org.elasticsearch.action.search.OpenPointInTimeRequest; import org.elasticsearch.action.search.OpenPointInTimeResponse; import org.elasticsearch.action.search.SearchPhaseExecutionException; import org.elasticsearch.action.search.SearchRequestBuilder; import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.action.search.TransportClosePointInTimeAction; +import org.elasticsearch.action.search.TransportOpenPointInTimeAction; import org.elasticsearch.common.Strings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.core.TimeValue; @@ -197,7 +197,7 @@ public class SearchSliceIT extends ESIntegTestCase { for (String field : new String[] { null, "random_int", "static_int" }) { // Open point-in-time reader OpenPointInTimeRequest request = new OpenPointInTimeRequest("test").keepAlive(TimeValue.timeValueSeconds(10)); - OpenPointInTimeResponse response = client().execute(OpenPointInTimeAction.INSTANCE, request).actionGet(); + OpenPointInTimeResponse response = client().execute(TransportOpenPointInTimeAction.TYPE, request).actionGet(); String pointInTimeId = response.getPointInTimeId(); // Test sort on document IDs @@ -206,7 +206,7 @@ public class SearchSliceIT extends ESIntegTestCase { assertSearchSlicesWithPointInTime(field, "random_int", pointInTimeId, max, numDocs); // Close point-in-time reader - client().execute(ClosePointInTimeAction.INSTANCE, new ClosePointInTimeRequest(pointInTimeId)).actionGet(); + client().execute(TransportClosePointInTimeAction.TYPE, new ClosePointInTimeRequest(pointInTimeId)).actionGet(); } } diff --git a/server/src/main/java/org/elasticsearch/action/ActionModule.java b/server/src/main/java/org/elasticsearch/action/ActionModule.java index 220090a132ec..e0f01405bcf0 100644 --- a/server/src/main/java/org/elasticsearch/action/ActionModule.java +++ b/server/src/main/java/org/elasticsearch/action/ActionModule.java @@ -208,7 +208,6 @@ import org.elasticsearch.action.bulk.TransportShardBulkAction; import org.elasticsearch.action.bulk.TransportSimulateBulkAction; import org.elasticsearch.action.delete.DeleteAction; import org.elasticsearch.action.delete.TransportDeleteAction; -import org.elasticsearch.action.explain.ExplainAction; import org.elasticsearch.action.explain.TransportExplainAction; import org.elasticsearch.action.fieldcaps.FieldCapabilitiesAction; import org.elasticsearch.action.fieldcaps.TransportFieldCapabilitiesAction; @@ -227,15 +226,8 @@ import org.elasticsearch.action.ingest.PutPipelineAction; import org.elasticsearch.action.ingest.PutPipelineTransportAction; import org.elasticsearch.action.ingest.SimulatePipelineAction; import org.elasticsearch.action.ingest.SimulatePipelineTransportAction; -import org.elasticsearch.action.search.ClearScrollAction; -import org.elasticsearch.action.search.ClosePointInTimeAction; -import org.elasticsearch.action.search.MultiSearchAction; -import org.elasticsearch.action.search.OpenPointInTimeAction; import org.elasticsearch.action.search.RestClosePointInTimeAction; import org.elasticsearch.action.search.RestOpenPointInTimeAction; -import org.elasticsearch.action.search.SearchAction; -import org.elasticsearch.action.search.SearchScrollAction; -import org.elasticsearch.action.search.SearchShardsAction; import org.elasticsearch.action.search.TransportClearScrollAction; import org.elasticsearch.action.search.TransportClosePointInTimeAction; import org.elasticsearch.action.search.TransportMultiSearchAction; @@ -763,14 +755,14 @@ public class ActionModule extends AbstractModule { actions.register(BulkAction.INSTANCE, TransportBulkAction.class); actions.register(SimulateBulkAction.INSTANCE, TransportSimulateBulkAction.class); actions.register(TransportShardBulkAction.TYPE, TransportShardBulkAction.class); - actions.register(SearchAction.INSTANCE, TransportSearchAction.class); - actions.register(SearchScrollAction.INSTANCE, TransportSearchScrollAction.class); - actions.register(OpenPointInTimeAction.INSTANCE, TransportOpenPointInTimeAction.class); - actions.register(ClosePointInTimeAction.INSTANCE, TransportClosePointInTimeAction.class); - actions.register(SearchShardsAction.INSTANCE, TransportSearchShardsAction.class); - actions.register(MultiSearchAction.INSTANCE, TransportMultiSearchAction.class); - actions.register(ExplainAction.INSTANCE, TransportExplainAction.class); - actions.register(ClearScrollAction.INSTANCE, TransportClearScrollAction.class); + actions.register(TransportSearchAction.TYPE, TransportSearchAction.class); + actions.register(TransportSearchScrollAction.TYPE, TransportSearchScrollAction.class); + actions.register(TransportOpenPointInTimeAction.TYPE, TransportOpenPointInTimeAction.class); + actions.register(TransportClosePointInTimeAction.TYPE, TransportClosePointInTimeAction.class); + actions.register(TransportSearchShardsAction.TYPE, TransportSearchShardsAction.class); + actions.register(TransportMultiSearchAction.TYPE, TransportMultiSearchAction.class); + actions.register(TransportExplainAction.TYPE, TransportExplainAction.class); + actions.register(TransportClearScrollAction.TYPE, TransportClearScrollAction.class); actions.register(RecoveryAction.INSTANCE, TransportRecoveryAction.class); actions.register(NodesReloadSecureSettingsAction.INSTANCE, TransportNodesReloadSecureSettingsAction.class); actions.register(AutoCreateAction.INSTANCE, AutoCreateAction.TransportAction.class); diff --git a/server/src/main/java/org/elasticsearch/action/explain/ExplainAction.java b/server/src/main/java/org/elasticsearch/action/explain/ExplainAction.java deleted file mode 100644 index 7668a48d623d..000000000000 --- a/server/src/main/java/org/elasticsearch/action/explain/ExplainAction.java +++ /dev/null @@ -1,25 +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 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 or the Server - * Side Public License, v 1. - */ - -package org.elasticsearch.action.explain; - -import org.elasticsearch.action.ActionType; - -/** - * Entry point for the explain feature. - */ -public class ExplainAction extends ActionType { - - public static final ExplainAction INSTANCE = new ExplainAction(); - public static final String NAME = "indices:data/read/explain"; - - private ExplainAction() { - super(NAME, ExplainResponse::new); - } - -} diff --git a/server/src/main/java/org/elasticsearch/action/explain/ExplainRequestBuilder.java b/server/src/main/java/org/elasticsearch/action/explain/ExplainRequestBuilder.java index 4a9ae67c60e1..9ae05687649e 100644 --- a/server/src/main/java/org/elasticsearch/action/explain/ExplainRequestBuilder.java +++ b/server/src/main/java/org/elasticsearch/action/explain/ExplainRequestBuilder.java @@ -8,6 +8,7 @@ package org.elasticsearch.action.explain; +import org.elasticsearch.action.ActionType; import org.elasticsearch.action.support.single.shard.SingleShardOperationRequestBuilder; import org.elasticsearch.client.internal.ElasticsearchClient; import org.elasticsearch.common.Strings; @@ -20,11 +21,7 @@ import org.elasticsearch.search.fetch.subphase.FetchSourceContext; */ public class ExplainRequestBuilder extends SingleShardOperationRequestBuilder { - ExplainRequestBuilder(ElasticsearchClient client, ExplainAction action) { - super(client, action, new ExplainRequest()); - } - - public ExplainRequestBuilder(ElasticsearchClient client, ExplainAction action, String index, String id) { + public ExplainRequestBuilder(ElasticsearchClient client, ActionType action, String index, String id) { super(client, action, new ExplainRequest().index(index).id(id)); } diff --git a/server/src/main/java/org/elasticsearch/action/explain/TransportExplainAction.java b/server/src/main/java/org/elasticsearch/action/explain/TransportExplainAction.java index 0b6a0a327664..d889f8fac811 100644 --- a/server/src/main/java/org/elasticsearch/action/explain/TransportExplainAction.java +++ b/server/src/main/java/org/elasticsearch/action/explain/TransportExplainAction.java @@ -11,6 +11,7 @@ package org.elasticsearch.action.explain; import org.apache.lucene.search.Explanation; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.ActionType; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.single.shard.TransportSingleShardAction; import org.elasticsearch.cluster.ClusterState; @@ -48,6 +49,7 @@ import java.util.function.LongSupplier; // TODO: AggregatedDfs. Currently the idf can be different then when executing a normal search with explain. public class TransportExplainAction extends TransportSingleShardAction { + public static final ActionType TYPE = new ActionType<>("indices:data/read/explain", ExplainResponse::new); private final SearchService searchService; @Inject @@ -60,7 +62,7 @@ public class TransportExplainAction extends TransportSingleShardAction { - - public static final ClearScrollAction INSTANCE = new ClearScrollAction(); - public static final String NAME = "indices:data/read/scroll/clear"; - - private ClearScrollAction() { - super(NAME, ClearScrollResponse::new); - } -} diff --git a/server/src/main/java/org/elasticsearch/action/search/ClearScrollRequestBuilder.java b/server/src/main/java/org/elasticsearch/action/search/ClearScrollRequestBuilder.java index 42b734715bd8..2311a5f65eb4 100644 --- a/server/src/main/java/org/elasticsearch/action/search/ClearScrollRequestBuilder.java +++ b/server/src/main/java/org/elasticsearch/action/search/ClearScrollRequestBuilder.java @@ -9,13 +9,14 @@ package org.elasticsearch.action.search; import org.elasticsearch.action.ActionRequestBuilder; +import org.elasticsearch.action.ActionType; import org.elasticsearch.client.internal.ElasticsearchClient; import java.util.List; public class ClearScrollRequestBuilder extends ActionRequestBuilder { - public ClearScrollRequestBuilder(ElasticsearchClient client, ClearScrollAction action) { + public ClearScrollRequestBuilder(ElasticsearchClient client, ActionType action) { super(client, action, new ClearScrollRequest()); } diff --git a/server/src/main/java/org/elasticsearch/action/search/ClosePointInTimeAction.java b/server/src/main/java/org/elasticsearch/action/search/ClosePointInTimeAction.java deleted file mode 100644 index ae9757b5b516..000000000000 --- a/server/src/main/java/org/elasticsearch/action/search/ClosePointInTimeAction.java +++ /dev/null @@ -1,21 +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 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 or the Server - * Side Public License, v 1. - */ - -package org.elasticsearch.action.search; - -import org.elasticsearch.action.ActionType; - -public class ClosePointInTimeAction extends ActionType { - - public static final ClosePointInTimeAction INSTANCE = new ClosePointInTimeAction(); - public static final String NAME = "indices:data/read/close_point_in_time"; - - private ClosePointInTimeAction() { - super(NAME, ClosePointInTimeResponse::new); - } -} diff --git a/server/src/main/java/org/elasticsearch/action/search/MultiSearchAction.java b/server/src/main/java/org/elasticsearch/action/search/MultiSearchAction.java deleted file mode 100644 index faea4b88e5c7..000000000000 --- a/server/src/main/java/org/elasticsearch/action/search/MultiSearchAction.java +++ /dev/null @@ -1,21 +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 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 or the Server - * Side Public License, v 1. - */ - -package org.elasticsearch.action.search; - -import org.elasticsearch.action.ActionType; - -public class MultiSearchAction extends ActionType { - - public static final MultiSearchAction INSTANCE = new MultiSearchAction(); - public static final String NAME = "indices:data/read/msearch"; - - private MultiSearchAction() { - super(NAME, MultiSearchResponse::new); - } -} diff --git a/server/src/main/java/org/elasticsearch/action/search/MultiSearchRequestBuilder.java b/server/src/main/java/org/elasticsearch/action/search/MultiSearchRequestBuilder.java index 57c536f3d371..20888d652c8a 100644 --- a/server/src/main/java/org/elasticsearch/action/search/MultiSearchRequestBuilder.java +++ b/server/src/main/java/org/elasticsearch/action/search/MultiSearchRequestBuilder.java @@ -9,6 +9,7 @@ package org.elasticsearch.action.search; import org.elasticsearch.action.ActionRequestBuilder; +import org.elasticsearch.action.ActionType; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.client.internal.ElasticsearchClient; @@ -17,7 +18,7 @@ import org.elasticsearch.client.internal.ElasticsearchClient; */ public class MultiSearchRequestBuilder extends ActionRequestBuilder { - public MultiSearchRequestBuilder(ElasticsearchClient client, MultiSearchAction action) { + public MultiSearchRequestBuilder(ElasticsearchClient client, ActionType action) { super(client, action, new MultiSearchRequest()); } diff --git a/server/src/main/java/org/elasticsearch/action/search/OpenPointInTimeAction.java b/server/src/main/java/org/elasticsearch/action/search/OpenPointInTimeAction.java deleted file mode 100644 index 560f8aea1da5..000000000000 --- a/server/src/main/java/org/elasticsearch/action/search/OpenPointInTimeAction.java +++ /dev/null @@ -1,20 +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 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 or the Server - * Side Public License, v 1. - */ - -package org.elasticsearch.action.search; - -import org.elasticsearch.action.ActionType; - -public class OpenPointInTimeAction extends ActionType { - public static final String NAME = "indices:data/read/open_point_in_time"; - public static final OpenPointInTimeAction INSTANCE = new OpenPointInTimeAction(); - - private OpenPointInTimeAction() { - super(NAME, OpenPointInTimeResponse::new); - } -} diff --git a/server/src/main/java/org/elasticsearch/action/search/RestClosePointInTimeAction.java b/server/src/main/java/org/elasticsearch/action/search/RestClosePointInTimeAction.java index a9da16bd6202..64702501581e 100644 --- a/server/src/main/java/org/elasticsearch/action/search/RestClosePointInTimeAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/RestClosePointInTimeAction.java @@ -41,7 +41,7 @@ public class RestClosePointInTimeAction extends BaseRestHandler { clearRequest = ClosePointInTimeRequest.fromXContent(parser); } return channel -> client.execute( - ClosePointInTimeAction.INSTANCE, + TransportClosePointInTimeAction.TYPE, clearRequest, new RestToXContentListener<>(channel, ClosePointInTimeResponse::status) ); diff --git a/server/src/main/java/org/elasticsearch/action/search/RestOpenPointInTimeAction.java b/server/src/main/java/org/elasticsearch/action/search/RestOpenPointInTimeAction.java index 627fdd88cc30..0e7f3f911184 100644 --- a/server/src/main/java/org/elasticsearch/action/search/RestOpenPointInTimeAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/RestOpenPointInTimeAction.java @@ -54,13 +54,14 @@ public class RestOpenPointInTimeAction extends BaseRestHandler { ); openRequest.maxConcurrentShardRequests(maxConcurrentShardRequests); } + request.withContentOrSourceParamParserOrNull(parser -> { if (parser != null) { PARSER.parse(parser, openRequest, null); } }); - return channel -> client.execute(OpenPointInTimeAction.INSTANCE, openRequest, new RestToXContentListener<>(channel)); + return channel -> client.execute(TransportOpenPointInTimeAction.TYPE, openRequest, new RestToXContentListener<>(channel)); } private static final ObjectParser PARSER = new ObjectParser<>("open_point_in_time_request"); diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchAction.java deleted file mode 100644 index 61d7a3355dc8..000000000000 --- a/server/src/main/java/org/elasticsearch/action/search/SearchAction.java +++ /dev/null @@ -1,22 +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 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 or the Server - * Side Public License, v 1. - */ - -package org.elasticsearch.action.search; - -import org.elasticsearch.action.ActionType; - -public class SearchAction extends ActionType { - - public static final SearchAction INSTANCE = new SearchAction(); - public static final String NAME = "indices:data/read/search"; - - private SearchAction() { - super(NAME, SearchResponse::new); - } - -} diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchProgressActionListener.java b/server/src/main/java/org/elasticsearch/action/search/SearchProgressActionListener.java index 26466215a3e8..95e9b4cedeba 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchProgressActionListener.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchProgressActionListener.java @@ -11,7 +11,7 @@ package org.elasticsearch.action.search; import org.elasticsearch.action.ActionListener; /** - * An {@link ActionListener} for search requests that allows to track progress of the {@link SearchAction}. + * An {@link ActionListener} for search requests that allows to track progress of the {@link TransportSearchAction}. * See {@link SearchProgressListener}. */ public abstract class SearchProgressActionListener extends SearchProgressListener implements ActionListener {} diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchProgressListener.java b/server/src/main/java/org/elasticsearch/action/search/SearchProgressListener.java index c6b002259317..096f2606d3f0 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchProgressListener.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchProgressListener.java @@ -23,7 +23,7 @@ import java.util.Objects; import java.util.stream.StreamSupport; /** - * A listener that allows to track progress of the {@link SearchAction}. + * A listener that allows to track progress of the {@link TransportSearchAction}. */ public abstract class SearchProgressListener { private static final Logger logger = LogManager.getLogger(SearchProgressListener.class); diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchRequestBuilder.java b/server/src/main/java/org/elasticsearch/action/search/SearchRequestBuilder.java index a59557832285..4588c707c3be 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchRequestBuilder.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchRequestBuilder.java @@ -9,6 +9,7 @@ package org.elasticsearch.action.search; import org.elasticsearch.action.ActionRequestBuilder; +import org.elasticsearch.action.ActionType; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.client.internal.ElasticsearchClient; import org.elasticsearch.core.Nullable; @@ -41,7 +42,7 @@ import java.util.Map; */ public class SearchRequestBuilder extends ActionRequestBuilder { - public SearchRequestBuilder(ElasticsearchClient client, SearchAction action) { + public SearchRequestBuilder(ElasticsearchClient client, ActionType action) { super(client, action, new SearchRequest()); } diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchScrollAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchScrollAction.java deleted file mode 100644 index 25f0daab932d..000000000000 --- a/server/src/main/java/org/elasticsearch/action/search/SearchScrollAction.java +++ /dev/null @@ -1,22 +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 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 or the Server - * Side Public License, v 1. - */ - -package org.elasticsearch.action.search; - -import org.elasticsearch.action.ActionType; - -public class SearchScrollAction extends ActionType { - - public static final SearchScrollAction INSTANCE = new SearchScrollAction(); - public static final String NAME = "indices:data/read/scroll"; - - private SearchScrollAction() { - super(NAME, SearchResponse::new); - } - -} diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchScrollRequestBuilder.java b/server/src/main/java/org/elasticsearch/action/search/SearchScrollRequestBuilder.java index e8348d189fcb..4de27b843041 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchScrollRequestBuilder.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchScrollRequestBuilder.java @@ -9,6 +9,7 @@ package org.elasticsearch.action.search; import org.elasticsearch.action.ActionRequestBuilder; +import org.elasticsearch.action.ActionType; import org.elasticsearch.client.internal.ElasticsearchClient; import org.elasticsearch.core.TimeValue; import org.elasticsearch.search.Scroll; @@ -18,11 +19,11 @@ import org.elasticsearch.search.Scroll; */ public class SearchScrollRequestBuilder extends ActionRequestBuilder { - public SearchScrollRequestBuilder(ElasticsearchClient client, SearchScrollAction action) { + public SearchScrollRequestBuilder(ElasticsearchClient client, ActionType action) { super(client, action, new SearchScrollRequest()); } - public SearchScrollRequestBuilder(ElasticsearchClient client, SearchScrollAction action, String scrollId) { + public SearchScrollRequestBuilder(ElasticsearchClient client, ActionType action, String scrollId) { super(client, action, new SearchScrollRequest(scrollId)); } diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchShardsAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchShardsAction.java deleted file mode 100644 index f4bfc2623fe1..000000000000 --- a/server/src/main/java/org/elasticsearch/action/search/SearchShardsAction.java +++ /dev/null @@ -1,20 +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 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 or the Server - * Side Public License, v 1. - */ - -package org.elasticsearch.action.search; - -import org.elasticsearch.action.ActionType; - -public class SearchShardsAction extends ActionType { - public static final String NAME = "indices:admin/search/search_shards"; - public static final SearchShardsAction INSTANCE = new SearchShardsAction(); - - private SearchShardsAction() { - super(NAME, SearchShardsResponse::new); - } -} diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java b/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java index 96824844d4ba..e46d26c3532a 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java @@ -280,7 +280,7 @@ public class SearchTransportService { final Transport.Connection connection = transportService.getConnection(transportService.getLocalNode()); transportService.sendChildRequest( connection, - MultiSearchAction.NAME, + TransportMultiSearchAction.TYPE.name(), request, task, new ConnectionCountingHandler<>(listener, MultiSearchResponse::new, clientConnections, connection.getNode().getId()) diff --git a/server/src/main/java/org/elasticsearch/action/search/TransportClearScrollAction.java b/server/src/main/java/org/elasticsearch/action/search/TransportClearScrollAction.java index 8de2815a9d41..e1a6bb6c42b2 100644 --- a/server/src/main/java/org/elasticsearch/action/search/TransportClearScrollAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/TransportClearScrollAction.java @@ -9,6 +9,7 @@ package org.elasticsearch.action.search; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.ActionType; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.HandledTransportAction; import org.elasticsearch.cluster.service.ClusterService; @@ -19,6 +20,9 @@ import org.elasticsearch.transport.TransportService; public class TransportClearScrollAction extends HandledTransportAction { + public static final String NAME = "indices:data/read/scroll/clear"; + + public static final ActionType TYPE = new ActionType<>(NAME, ClearScrollResponse::new); private final ClusterService clusterService; private final SearchTransportService searchTransportService; @@ -29,7 +33,7 @@ public class TransportClearScrollAction extends HandledTransportAction { + public static final ActionType TYPE = new ActionType<>( + "indices:data/read/close_point_in_time", + ClosePointInTimeResponse::new + ); private final ClusterService clusterService; private final SearchTransportService searchTransportService; private final NamedWriteableRegistry namedWriteableRegistry; @@ -34,13 +39,7 @@ public class TransportClosePointInTimeAction extends HandledTransportAction { + public static final String NAME = "indices:data/read/msearch"; + public static final ActionType TYPE = new ActionType<>(NAME, MultiSearchResponse::new); private static final Logger logger = LogManager.getLogger(TransportMultiSearchAction.class); private final int allocatedProcessors; private final ThreadPool threadPool; @@ -50,7 +53,7 @@ public class TransportMultiSearchAction extends HandledTransportAction TYPE = new ActionType<>( + "indices:data/read/open_point_in_time", + OpenPointInTimeResponse::new + ); private final TransportSearchAction transportSearchAction; private final SearchTransportService searchTransportService; @@ -68,13 +73,7 @@ public class TransportOpenPointInTimeAction extends HandledTransportAction { + public static final String NAME = "indices:data/read/search"; + public static final ActionType TYPE = new ActionType<>(NAME, SearchResponse::new); private static final Logger logger = LogManager.getLogger(TransportSearchAction.class); private static final DeprecationLogger DEPRECATION_LOGGER = DeprecationLogger.getLogger(TransportSearchAction.class); public static final String FROZEN_INDICES_DEPRECATION_MESSAGE = "Searching frozen indices [{}] is deprecated." @@ -160,7 +163,7 @@ public class TransportSearchAction extends HandledTransportAction(singleListener, SearchShardsResponse::new, responseExecutor) diff --git a/server/src/main/java/org/elasticsearch/action/search/TransportSearchScrollAction.java b/server/src/main/java/org/elasticsearch/action/search/TransportSearchScrollAction.java index 9874bcfb56c6..0a2b496a5eb8 100644 --- a/server/src/main/java/org/elasticsearch/action/search/TransportSearchScrollAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/TransportSearchScrollAction.java @@ -12,6 +12,7 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.ActionType; import org.elasticsearch.action.ShardOperationFailedException; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.HandledTransportAction; @@ -26,6 +27,7 @@ import static org.elasticsearch.action.search.ParsedScrollId.QUERY_THEN_FETCH_TY import static org.elasticsearch.action.search.TransportSearchHelper.parseScrollId; public class TransportSearchScrollAction extends HandledTransportAction { + public static final ActionType TYPE = new ActionType<>("indices:data/read/scroll", SearchResponse::new); private static final Logger logger = LogManager.getLogger(TransportSearchScrollAction.class); private final ClusterService clusterService; private final SearchTransportService searchTransportService; @@ -37,7 +39,7 @@ public class TransportSearchScrollAction extends HandledTransportAction { + + public static final String NAME = "indices:admin/search/search_shards"; + public static final ActionType TYPE = new ActionType<>(NAME, SearchShardsResponse::new); private final TransportService transportService; private final TransportSearchAction transportSearchAction; private final SearchService searchService; @@ -61,7 +65,7 @@ public class TransportSearchShardsAction extends HandledTransportAction search(final SearchRequest request) { - return execute(SearchAction.INSTANCE, request); + return execute(TransportSearchAction.TYPE, request); } @Override public void search(final SearchRequest request, final ActionListener listener) { - execute(SearchAction.INSTANCE, request, listener); + execute(TransportSearchAction.TYPE, request, listener); } @Override public SearchRequestBuilder prepareSearch(String... indices) { - return new SearchRequestBuilder(this, SearchAction.INSTANCE).setIndices(indices); + return new SearchRequestBuilder(this, TransportSearchAction.TYPE).setIndices(indices); } @Override public ActionFuture searchScroll(final SearchScrollRequest request) { - return execute(SearchScrollAction.INSTANCE, request); + return execute(TransportSearchScrollAction.TYPE, request); } @Override public void searchScroll(final SearchScrollRequest request, final ActionListener listener) { - execute(SearchScrollAction.INSTANCE, request, listener); + execute(TransportSearchScrollAction.TYPE, request, listener); } @Override public SearchScrollRequestBuilder prepareSearchScroll(String scrollId) { - return new SearchScrollRequestBuilder(this, SearchScrollAction.INSTANCE, scrollId); + return new SearchScrollRequestBuilder(this, TransportSearchScrollAction.TYPE, scrollId); } @Override public ActionFuture multiSearch(MultiSearchRequest request) { - return execute(MultiSearchAction.INSTANCE, request); + return execute(TransportMultiSearchAction.TYPE, request); } @Override public void multiSearch(MultiSearchRequest request, ActionListener listener) { - execute(MultiSearchAction.INSTANCE, request, listener); + execute(TransportMultiSearchAction.TYPE, request, listener); } @Override public MultiSearchRequestBuilder prepareMultiSearch() { - return new MultiSearchRequestBuilder(this, MultiSearchAction.INSTANCE); + return new MultiSearchRequestBuilder(this, TransportMultiSearchAction.TYPE); } @Override @@ -589,32 +589,32 @@ public abstract class AbstractClient implements Client { @Override public ExplainRequestBuilder prepareExplain(String index, String id) { - return new ExplainRequestBuilder(this, ExplainAction.INSTANCE, index, id); + return new ExplainRequestBuilder(this, TransportExplainAction.TYPE, index, id); } @Override public ActionFuture explain(ExplainRequest request) { - return execute(ExplainAction.INSTANCE, request); + return execute(TransportExplainAction.TYPE, request); } @Override public void explain(ExplainRequest request, ActionListener listener) { - execute(ExplainAction.INSTANCE, request, listener); + execute(TransportExplainAction.TYPE, request, listener); } @Override public void clearScroll(ClearScrollRequest request, ActionListener listener) { - execute(ClearScrollAction.INSTANCE, request, listener); + execute(TransportClearScrollAction.TYPE, request, listener); } @Override public ActionFuture clearScroll(ClearScrollRequest request) { - return execute(ClearScrollAction.INSTANCE, request); + return execute(TransportClearScrollAction.TYPE, request); } @Override public ClearScrollRequestBuilder prepareClearScroll() { - return new ClearScrollRequestBuilder(this, ClearScrollAction.INSTANCE); + return new ClearScrollRequestBuilder(this, TransportClearScrollAction.TYPE); } @Override diff --git a/server/src/main/java/org/elasticsearch/index/reindex/DeleteByQueryRequestBuilder.java b/server/src/main/java/org/elasticsearch/index/reindex/DeleteByQueryRequestBuilder.java index 3a7f316de6f4..7f6465e9bce8 100644 --- a/server/src/main/java/org/elasticsearch/index/reindex/DeleteByQueryRequestBuilder.java +++ b/server/src/main/java/org/elasticsearch/index/reindex/DeleteByQueryRequestBuilder.java @@ -9,14 +9,14 @@ package org.elasticsearch.index.reindex; import org.elasticsearch.action.ActionType; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchRequestBuilder; +import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.client.internal.ElasticsearchClient; public class DeleteByQueryRequestBuilder extends AbstractBulkByScrollRequestBuilder { public DeleteByQueryRequestBuilder(ElasticsearchClient client, ActionType action) { - this(client, action, new SearchRequestBuilder(client, SearchAction.INSTANCE)); + this(client, action, new SearchRequestBuilder(client, TransportSearchAction.TYPE)); } private DeleteByQueryRequestBuilder(ElasticsearchClient client, ActionType action, SearchRequestBuilder search) { diff --git a/server/src/main/java/org/elasticsearch/index/reindex/ReindexRequestBuilder.java b/server/src/main/java/org/elasticsearch/index/reindex/ReindexRequestBuilder.java index 68a0b948ef32..e79d06ceba0b 100644 --- a/server/src/main/java/org/elasticsearch/index/reindex/ReindexRequestBuilder.java +++ b/server/src/main/java/org/elasticsearch/index/reindex/ReindexRequestBuilder.java @@ -11,8 +11,8 @@ package org.elasticsearch.index.reindex; import org.elasticsearch.action.ActionType; import org.elasticsearch.action.index.IndexAction; import org.elasticsearch.action.index.IndexRequestBuilder; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchRequestBuilder; +import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.client.internal.ElasticsearchClient; public class ReindexRequestBuilder extends AbstractBulkIndexByScrollRequestBuilder { @@ -22,7 +22,7 @@ public class ReindexRequestBuilder extends AbstractBulkIndexByScrollRequestBuild this( client, action, - new SearchRequestBuilder(client, SearchAction.INSTANCE), + new SearchRequestBuilder(client, TransportSearchAction.TYPE), new IndexRequestBuilder(client, IndexAction.INSTANCE) ); } diff --git a/server/src/main/java/org/elasticsearch/index/reindex/UpdateByQueryRequestBuilder.java b/server/src/main/java/org/elasticsearch/index/reindex/UpdateByQueryRequestBuilder.java index d5410d8d9aff..2dfd78637836 100644 --- a/server/src/main/java/org/elasticsearch/index/reindex/UpdateByQueryRequestBuilder.java +++ b/server/src/main/java/org/elasticsearch/index/reindex/UpdateByQueryRequestBuilder.java @@ -9,8 +9,8 @@ package org.elasticsearch.index.reindex; import org.elasticsearch.action.ActionType; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchRequestBuilder; +import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.client.internal.ElasticsearchClient; public class UpdateByQueryRequestBuilder extends AbstractBulkIndexByScrollRequestBuilder< @@ -18,7 +18,7 @@ public class UpdateByQueryRequestBuilder extends AbstractBulkIndexByScrollReques UpdateByQueryRequestBuilder> { public UpdateByQueryRequestBuilder(ElasticsearchClient client, ActionType action) { - this(client, action, new SearchRequestBuilder(client, SearchAction.INSTANCE)); + this(client, action, new SearchRequestBuilder(client, TransportSearchAction.TYPE)); } private UpdateByQueryRequestBuilder(ElasticsearchClient client, ActionType action, SearchRequestBuilder search) { diff --git a/server/src/main/java/org/elasticsearch/rest/action/search/RestMultiSearchAction.java b/server/src/main/java/org/elasticsearch/rest/action/search/RestMultiSearchAction.java index 60e797a62013..c232e1a30c55 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/search/RestMultiSearchAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/search/RestMultiSearchAction.java @@ -8,9 +8,9 @@ package org.elasticsearch.rest.action.search; -import org.elasticsearch.action.search.MultiSearchAction; import org.elasticsearch.action.search.MultiSearchRequest; import org.elasticsearch.action.search.SearchRequest; +import org.elasticsearch.action.search.TransportMultiSearchAction; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.client.internal.node.NodeClient; import org.elasticsearch.common.CheckedBiConsumer; @@ -83,7 +83,7 @@ public class RestMultiSearchAction extends BaseRestHandler { return channel -> { final RestCancellableNodeClient cancellableClient = new RestCancellableNodeClient(client, request.getHttpChannel()); cancellableClient.execute( - MultiSearchAction.INSTANCE, + TransportMultiSearchAction.TYPE, multiSearchRequest, new RestRefCountedChunkedToXContentListener<>(channel) ); diff --git a/server/src/main/java/org/elasticsearch/rest/action/search/RestSearchAction.java b/server/src/main/java/org/elasticsearch/rest/action/search/RestSearchAction.java index 7ce488d62a1c..41102a3568e3 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/search/RestSearchAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/search/RestSearchAction.java @@ -11,9 +11,9 @@ package org.elasticsearch.rest.action.search; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.Version; import org.elasticsearch.action.ActionRequestValidationException; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchContextId; import org.elasticsearch.action.search.SearchRequest; +import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.client.internal.node.NodeClient; import org.elasticsearch.common.Strings; @@ -121,7 +121,7 @@ public class RestSearchAction extends BaseRestHandler { return channel -> { RestCancellableNodeClient cancelClient = new RestCancellableNodeClient(client, request.getHttpChannel()); - cancelClient.execute(SearchAction.INSTANCE, searchRequest, new RestRefCountedChunkedToXContentListener<>(channel)); + cancelClient.execute(TransportSearchAction.TYPE, searchRequest, new RestRefCountedChunkedToXContentListener<>(channel)); }; } diff --git a/server/src/test/java/org/elasticsearch/action/search/SearchRequestBuilderTests.java b/server/src/test/java/org/elasticsearch/action/search/SearchRequestBuilderTests.java index 1747886c48da..bb2b6f7903a6 100644 --- a/server/src/test/java/org/elasticsearch/action/search/SearchRequestBuilderTests.java +++ b/server/src/test/java/org/elasticsearch/action/search/SearchRequestBuilderTests.java @@ -20,7 +20,7 @@ public class SearchRequestBuilderTests extends ESTestCase { private SearchRequestBuilder makeBuilder() { ElasticsearchClient client = Mockito.mock(ElasticsearchClient.class); - return new SearchRequestBuilder(client, SearchAction.INSTANCE); + return new SearchRequestBuilder(client, TransportSearchAction.TYPE); } public void testEmptySourceToString() { diff --git a/server/src/test/java/org/elasticsearch/action/search/SearchRequestTests.java b/server/src/test/java/org/elasticsearch/action/search/SearchRequestTests.java index 8005f2f41269..0c8496081ff1 100644 --- a/server/src/test/java/org/elasticsearch/action/search/SearchRequestTests.java +++ b/server/src/test/java/org/elasticsearch/action/search/SearchRequestTests.java @@ -528,7 +528,7 @@ public class SearchRequestTests extends AbstractSearchTestCase { } private String toDescription(SearchRequest request) { - return request.createTask(0, "test", SearchAction.NAME, TaskId.EMPTY_TASK_ID, emptyMap()).getDescription(); + return request.createTask(0, "test", TransportSearchAction.TYPE.name(), TaskId.EMPTY_TASK_ID, emptyMap()).getDescription(); } public void testForceSyntheticUnsupported() { diff --git a/server/src/test/java/org/elasticsearch/client/internal/AbstractClientHeadersTestCase.java b/server/src/test/java/org/elasticsearch/client/internal/AbstractClientHeadersTestCase.java index 32e9b214ab53..a776f74febc1 100644 --- a/server/src/test/java/org/elasticsearch/client/internal/AbstractClientHeadersTestCase.java +++ b/server/src/test/java/org/elasticsearch/client/internal/AbstractClientHeadersTestCase.java @@ -22,7 +22,7 @@ import org.elasticsearch.action.admin.indices.stats.IndicesStatsAction; import org.elasticsearch.action.delete.DeleteAction; import org.elasticsearch.action.get.GetAction; import org.elasticsearch.action.index.IndexAction; -import org.elasticsearch.action.search.SearchAction; +import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.env.Environment; @@ -48,7 +48,7 @@ public abstract class AbstractClientHeadersTestCase extends ESTestCase { private static final ActionType[] ACTIONS = new ActionType[] { // client actions GetAction.INSTANCE, - SearchAction.INSTANCE, + TransportSearchAction.TYPE, DeleteAction.INSTANCE, DeleteStoredScriptAction.INSTANCE, IndexAction.INSTANCE, @@ -97,7 +97,7 @@ public abstract class AbstractClientHeadersTestCase extends ESTestCase { // choosing arbitrary top level actions to test client.prepareGet("idx", "id").execute(new AssertingActionListener<>(GetAction.NAME, client.threadPool())); - client.prepareSearch().execute(new AssertingActionListener<>(SearchAction.NAME, client.threadPool())); + client.prepareSearch().execute(new AssertingActionListener<>(TransportSearchAction.TYPE.name(), client.threadPool())); client.prepareDelete("idx", "id").execute(new AssertingActionListener<>(DeleteAction.NAME, client.threadPool())); client.admin() .cluster() diff --git a/server/src/test/java/org/elasticsearch/rest/action/RestCancellableNodeClientTests.java b/server/src/test/java/org/elasticsearch/rest/action/RestCancellableNodeClientTests.java index a21eab1d9591..3f425ac202b6 100644 --- a/server/src/test/java/org/elasticsearch/rest/action/RestCancellableNodeClientTests.java +++ b/server/src/test/java/org/elasticsearch/rest/action/RestCancellableNodeClientTests.java @@ -14,9 +14,9 @@ import org.elasticsearch.action.ActionResponse; import org.elasticsearch.action.ActionType; import org.elasticsearch.action.admin.cluster.node.tasks.cancel.CancelTasksAction; import org.elasticsearch.action.admin.cluster.node.tasks.cancel.CancelTasksRequest; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.client.internal.node.NodeClient; import org.elasticsearch.common.settings.Settings; @@ -76,7 +76,7 @@ public class RestCancellableNodeClientTests extends ESTestCase { for (int j = 0; j < numTasks; j++) { PlainActionFuture actionFuture = new PlainActionFuture<>(); RestCancellableNodeClient client = new RestCancellableNodeClient(testClient, channel); - threadPool.generic().submit(() -> client.execute(SearchAction.INSTANCE, new SearchRequest(), actionFuture)); + threadPool.generic().submit(() -> client.execute(TransportSearchAction.TYPE, new SearchRequest(), actionFuture)); futures.add(actionFuture); } } @@ -106,7 +106,7 @@ public class RestCancellableNodeClientTests extends ESTestCase { totalSearches += numTasks; RestCancellableNodeClient client = new RestCancellableNodeClient(nodeClient, channel); for (int j = 0; j < numTasks; j++) { - client.execute(SearchAction.INSTANCE, new SearchRequest(), null); + client.execute(TransportSearchAction.TYPE, new SearchRequest(), null); } assertEquals(numTasks, RestCancellableNodeClient.getNumTasks(channel)); } @@ -139,7 +139,7 @@ public class RestCancellableNodeClientTests extends ESTestCase { RestCancellableNodeClient client = new RestCancellableNodeClient(testClient, channel); for (int j = 0; j < numTasks; j++) { // here the channel will be first registered, then straight-away removed from the map as the close listener is invoked - client.execute(SearchAction.INSTANCE, new SearchRequest(), null); + client.execute(TransportSearchAction.TYPE, new SearchRequest(), null); } } assertEquals(initialHttpChannels, RestCancellableNodeClient.getNumChannels()); @@ -177,7 +177,7 @@ public class RestCancellableNodeClientTests extends ESTestCase { } return task; } - case SearchAction.NAME -> { + case TransportSearchAction.NAME -> { searchRequests.incrementAndGet(); Task searchTask = request.createTask(counter.getAndIncrement(), "search", action.name(), null, Collections.emptyMap()); if (timeout == false) { diff --git a/server/src/test/java/org/elasticsearch/search/SearchServiceTests.java b/server/src/test/java/org/elasticsearch/search/SearchServiceTests.java index 8472e5849051..270ab3003a1f 100644 --- a/server/src/test/java/org/elasticsearch/search/SearchServiceTests.java +++ b/server/src/test/java/org/elasticsearch/search/SearchServiceTests.java @@ -25,9 +25,7 @@ import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.OriginalIndices; import org.elasticsearch.action.admin.cluster.settings.ClusterUpdateSettingsResponse; import org.elasticsearch.action.search.ClearScrollRequest; -import org.elasticsearch.action.search.ClosePointInTimeAction; import org.elasticsearch.action.search.ClosePointInTimeRequest; -import org.elasticsearch.action.search.OpenPointInTimeAction; import org.elasticsearch.action.search.OpenPointInTimeRequest; import org.elasticsearch.action.search.SearchPhaseExecutionException; import org.elasticsearch.action.search.SearchRequest; @@ -35,6 +33,8 @@ import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchScrollRequest; import org.elasticsearch.action.search.SearchShardTask; import org.elasticsearch.action.search.SearchType; +import org.elasticsearch.action.search.TransportClosePointInTimeAction; +import org.elasticsearch.action.search.TransportOpenPointInTimeAction; import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.support.PlainActionFuture; @@ -1833,7 +1833,7 @@ public class SearchServiceTests extends ESSingleNodeTestCase { indicesAdmin().prepareRefresh("test").get(); String pitId = client().execute( - OpenPointInTimeAction.INSTANCE, + TransportOpenPointInTimeAction.TYPE, new OpenPointInTimeRequest("test").keepAlive(TimeValue.timeValueMinutes(10)) ).actionGet().getPointInTimeId(); final MockSearchService searchService = (MockSearchService) getInstanceFromNode(SearchService.class); @@ -1848,7 +1848,7 @@ public class SearchServiceTests extends ESSingleNodeTestCase { numDocs ); } finally { - client().execute(ClosePointInTimeAction.INSTANCE, new ClosePointInTimeRequest(pitId)).actionGet(); + client().execute(TransportClosePointInTimeAction.TYPE, new ClosePointInTimeRequest(pitId)).actionGet(); } assertThat(shardRequests, not(emptyList())); for (ShardSearchRequest shardRequest : shardRequests) { diff --git a/server/src/test/java/org/elasticsearch/search/vectors/KnnSearchRequestParserTests.java b/server/src/test/java/org/elasticsearch/search/vectors/KnnSearchRequestParserTests.java index 37fbb6c5fcea..5271289e37b7 100644 --- a/server/src/test/java/org/elasticsearch/search/vectors/KnnSearchRequestParserTests.java +++ b/server/src/test/java/org/elasticsearch/search/vectors/KnnSearchRequestParserTests.java @@ -8,9 +8,9 @@ package org.elasticsearch.search.vectors; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchRequestBuilder; +import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.query.QueryBuilder; @@ -224,7 +224,7 @@ public class KnnSearchRequestParserTests extends ESTestCase { .withContent(BytesReference.bytes(builder), builder.contentType()) .build() ); - SearchRequestBuilder searchRequestBuilder = new SearchRequestBuilder(null, SearchAction.INSTANCE); + SearchRequestBuilder searchRequestBuilder = new SearchRequestBuilder(null, TransportSearchAction.TYPE); knnRequestBuilder.toSearchRequest(searchRequestBuilder); return searchRequestBuilder; } diff --git a/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java b/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java index f4898e1fe2c8..6da5b887a3f3 100644 --- a/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java +++ b/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java @@ -58,7 +58,6 @@ import org.elasticsearch.action.bulk.TransportBulkAction; import org.elasticsearch.action.bulk.TransportShardBulkAction; import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.resync.TransportResyncReplicationAction; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchExecutionStatsCollector; import org.elasticsearch.action.search.SearchPhaseController; import org.elasticsearch.action.search.SearchRequest; @@ -2000,7 +1999,7 @@ public class SnapshotResiliencyTests extends ESTestCase { SearchPhaseController searchPhaseController = new SearchPhaseController(searchService::aggReduceContextBuilder); actions.put( - SearchAction.INSTANCE, + TransportSearchAction.TYPE, new TransportSearchAction( threadPool, new NoneCircuitBreakerService(), diff --git a/server/src/test/java/org/elasticsearch/transport/RemoteClusterAwareClientTests.java b/server/src/test/java/org/elasticsearch/transport/RemoteClusterAwareClientTests.java index e606da040bab..b745756eece0 100644 --- a/server/src/test/java/org/elasticsearch/transport/RemoteClusterAwareClientTests.java +++ b/server/src/test/java/org/elasticsearch/transport/RemoteClusterAwareClientTests.java @@ -10,9 +10,9 @@ package org.elasticsearch.transport; import org.elasticsearch.TransportVersion; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.search.SearchShardsAction; import org.elasticsearch.action.search.SearchShardsRequest; import org.elasticsearch.action.search.SearchShardsResponse; +import org.elasticsearch.action.search.TransportSearchShardsAction; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.cluster.node.DiscoveryNode; @@ -104,7 +104,7 @@ public class RemoteClusterAwareClientTests extends ESTestCase { ); final SearchShardsResponse searchShardsResponse = PlainActionFuture.get( future -> client.execute( - SearchShardsAction.INSTANCE, + TransportSearchShardsAction.TYPE, searchShardsRequest, ActionListener.runBefore( future, @@ -169,7 +169,7 @@ public class RemoteClusterAwareClientTests extends ESTestCase { null ); client.execute( - SearchShardsAction.INSTANCE, + TransportSearchShardsAction.TYPE, searchShardsRequest, ActionListener.runBefore( future, diff --git a/server/src/test/java/org/elasticsearch/transport/RemoteClusterConnectionTests.java b/server/src/test/java/org/elasticsearch/transport/RemoteClusterConnectionTests.java index 44b48b60a9c5..d4f03f102783 100644 --- a/server/src/test/java/org/elasticsearch/transport/RemoteClusterConnectionTests.java +++ b/server/src/test/java/org/elasticsearch/transport/RemoteClusterConnectionTests.java @@ -15,13 +15,13 @@ import org.elasticsearch.action.admin.cluster.remote.RemoteClusterNodesAction; import org.elasticsearch.action.admin.cluster.state.ClusterStateAction; import org.elasticsearch.action.admin.cluster.state.ClusterStateRequest; import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchResponse; -import org.elasticsearch.action.search.SearchShardsAction; import org.elasticsearch.action.search.SearchShardsRequest; import org.elasticsearch.action.search.SearchShardsResponse; import org.elasticsearch.action.search.ShardSearchFailure; +import org.elasticsearch.action.search.TransportSearchAction; +import org.elasticsearch.action.search.TransportSearchShardsAction; import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.ClusterState; @@ -125,7 +125,7 @@ public class RemoteClusterConnectionTests extends ESTestCase { MockTransportService newService = MockTransportService.createNewService(s, version, transportVersion, threadPool, null); try { newService.registerRequestHandler( - SearchShardsAction.NAME, + TransportSearchShardsAction.TYPE.name(), EsExecutors.DIRECT_EXECUTOR_SERVICE, SearchShardsRequest::new, (request, channel, task) -> { @@ -137,7 +137,7 @@ public class RemoteClusterConnectionTests extends ESTestCase { } ); newService.registerRequestHandler( - SearchAction.NAME, + TransportSearchAction.TYPE.name(), EsExecutors.DIRECT_EXECUTOR_SERVICE, SearchRequest::new, (request, channel, task) -> { diff --git a/x-pack/plugin/async-search/src/internalClusterTest/java/org/elasticsearch/xpack/search/AsyncSearchIntegTestCase.java b/x-pack/plugin/async-search/src/internalClusterTest/java/org/elasticsearch/xpack/search/AsyncSearchIntegTestCase.java index 794763247ebf..3f888685f33d 100644 --- a/x-pack/plugin/async-search/src/internalClusterTest/java/org/elasticsearch/xpack/search/AsyncSearchIntegTestCase.java +++ b/x-pack/plugin/async-search/src/internalClusterTest/java/org/elasticsearch/xpack/search/AsyncSearchIntegTestCase.java @@ -12,10 +12,10 @@ import org.elasticsearch.ResourceNotFoundException; import org.elasticsearch.action.admin.cluster.node.tasks.get.GetTaskResponse; import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse; import org.elasticsearch.action.get.GetResponse; -import org.elasticsearch.action.search.ClosePointInTimeAction; import org.elasticsearch.action.search.ClosePointInTimeRequest; -import org.elasticsearch.action.search.OpenPointInTimeAction; import org.elasticsearch.action.search.OpenPointInTimeRequest; +import org.elasticsearch.action.search.TransportClosePointInTimeAction; +import org.elasticsearch.action.search.TransportOpenPointInTimeAction; import org.elasticsearch.action.support.master.AcknowledgedResponse; import org.elasticsearch.cluster.ClusterChangedEvent; import org.elasticsearch.cluster.ClusterState; @@ -235,7 +235,7 @@ public abstract class AsyncSearchIntegTestCase extends ESIntegTestCase { final SubmitAsyncSearchRequest request; if (randomBoolean()) { OpenPointInTimeRequest openPIT = new OpenPointInTimeRequest(indexName).keepAlive(TimeValue.timeValueMinutes(between(5, 10))); - pitId = client().execute(OpenPointInTimeAction.INSTANCE, openPIT).actionGet().getPointInTimeId(); + pitId = client().execute(TransportOpenPointInTimeAction.TYPE, openPIT).actionGet().getPointInTimeId(); final PointInTimeBuilder pit = new PointInTimeBuilder(pitId); if (randomBoolean()) { pit.setKeepAlive(TimeValue.timeValueMillis(randomIntBetween(1, 3600))); @@ -329,7 +329,7 @@ public abstract class AsyncSearchIntegTestCase extends ESIntegTestCase { public void close() { if (closed.compareAndSet(false, true)) { if (pitId != null) { - client().execute(ClosePointInTimeAction.INSTANCE, new ClosePointInTimeRequest(pitId)).actionGet(); + client().execute(TransportClosePointInTimeAction.TYPE, new ClosePointInTimeRequest(pitId)).actionGet(); } queryLatch.close(); } diff --git a/x-pack/plugin/async-search/src/internalClusterTest/java/org/elasticsearch/xpack/search/CrossClusterAsyncSearchIT.java b/x-pack/plugin/async-search/src/internalClusterTest/java/org/elasticsearch/xpack/search/CrossClusterAsyncSearchIT.java index 5c7795ecbd18..218a99a23be0 100644 --- a/x-pack/plugin/async-search/src/internalClusterTest/java/org/elasticsearch/xpack/search/CrossClusterAsyncSearchIT.java +++ b/x-pack/plugin/async-search/src/internalClusterTest/java/org/elasticsearch/xpack/search/CrossClusterAsyncSearchIT.java @@ -13,7 +13,6 @@ import org.elasticsearch.action.ActionFuture; import org.elasticsearch.action.admin.cluster.node.tasks.cancel.CancelTasksRequest; import org.elasticsearch.action.admin.cluster.node.tasks.cancel.CancelTasksResponse; import org.elasticsearch.action.admin.cluster.node.tasks.list.ListTasksResponse; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchType; import org.elasticsearch.action.search.ShardSearchFailure; @@ -1266,7 +1265,7 @@ public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase { ListTasksResponse listTasksResponse = client(LOCAL_CLUSTER).admin() .cluster() .prepareListTasks() - .setActions(SearchAction.INSTANCE.name()) + .setActions(TransportSearchAction.TYPE.name()) .get(); List tasks = listTasksResponse.getTasks(); assertThat(tasks.size(), equalTo(1)); @@ -1280,7 +1279,7 @@ public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase { .get() .getTasks() .stream() - .filter(t -> t.action().contains(SearchAction.NAME)) + .filter(t -> t.action().contains(TransportSearchAction.TYPE.name())) .collect(Collectors.toList()); assertThat(remoteSearchTasks.size(), greaterThan(0)); remoteClusterSearchTasks.set(remoteSearchTasks); @@ -1298,7 +1297,7 @@ public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase { for (TransportService transportService : transportServices) { Collection cancellableTasks = transportService.getTaskManager().getCancellableTasks().values(); for (CancellableTask cancellableTask : cancellableTasks) { - if (cancellableTask.getAction().contains(SearchAction.INSTANCE.name())) { + if (cancellableTask.getAction().contains(TransportSearchAction.TYPE.name())) { assertTrue(cancellableTask.getDescription(), cancellableTask.isCancelled()); } } @@ -1311,7 +1310,7 @@ public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase { .get() .getTasks() .stream() - .filter(t -> t.action().contains(SearchAction.INSTANCE.name())) + .filter(t -> t.action().contains(TransportSearchAction.TYPE.name())) .toList(); for (TaskInfo taskInfo : remoteSearchTasksAfterCancellation) { assertTrue(taskInfo.description(), taskInfo.cancelled()); @@ -1385,7 +1384,7 @@ public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase { ListTasksResponse listTasksResponse = client(LOCAL_CLUSTER).admin() .cluster() .prepareListTasks() - .setActions(SearchAction.INSTANCE.name()) + .setActions(TransportSearchAction.TYPE.name()) .get(); List tasks = listTasksResponse.getTasks(); assertThat(tasks.size(), equalTo(1)); @@ -1398,7 +1397,7 @@ public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase { .get() .getTasks() .stream() - .filter(t -> t.action().contains(SearchAction.NAME)) + .filter(t -> t.action().contains(TransportSearchAction.TYPE.name())) .collect(Collectors.toList()); assertThat(remoteSearchTasks.size(), greaterThan(0)); remoteClusterSearchTasks.set(remoteSearchTasks); @@ -1416,7 +1415,7 @@ public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase { for (TransportService transportService : transportServices) { Collection cancellableTasks = transportService.getTaskManager().getCancellableTasks().values(); for (CancellableTask cancellableTask : cancellableTasks) { - if (cancellableTask.getAction().contains(SearchAction.INSTANCE.name())) { + if (cancellableTask.getAction().contains(TransportSearchAction.TYPE.name())) { assertTrue(cancellableTask.getDescription(), cancellableTask.isCancelled()); } } @@ -1429,7 +1428,7 @@ public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase { .get() .getTasks() .stream() - .filter(t -> t.action().contains(SearchAction.INSTANCE.name())) + .filter(t -> t.action().contains(TransportSearchAction.TYPE.name())) .toList(); for (TaskInfo taskInfo : remoteSearchTasksAfterCancellation) { assertTrue(taskInfo.description(), taskInfo.cancelled()); @@ -1489,7 +1488,7 @@ public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase { ListTasksResponse listTasksResponse = client(LOCAL_CLUSTER).admin() .cluster() .prepareListTasks() - .setActions(SearchAction.INSTANCE.name()) + .setActions(TransportSearchAction.TYPE.name()) .get(); List tasks = listTasksResponse.getTasks(); assertThat(tasks.size(), equalTo(1)); @@ -1502,7 +1501,7 @@ public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase { .get() .getTasks() .stream() - .filter(t -> t.action().contains(SearchAction.NAME)) + .filter(t -> t.action().contains(TransportSearchAction.TYPE.name())) .collect(Collectors.toList()); assertThat(remoteSearchTasks.size(), greaterThan(0)); remoteClusterSearchTasks.set(remoteSearchTasks); @@ -1524,7 +1523,7 @@ public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase { ListTasksResponse listTasksResponse = client(LOCAL_CLUSTER).admin() .cluster() .prepareListTasks() - .setActions(SearchAction.INSTANCE.name()) + .setActions(TransportSearchAction.TYPE.name()) .get(); List tasks = listTasksResponse.getTasks(); assertThat(tasks.size(), equalTo(0)); @@ -1532,7 +1531,7 @@ public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase { ListTasksResponse remoteTasksResponse = client(REMOTE_CLUSTER).admin() .cluster() .prepareListTasks() - .setActions(SearchAction.INSTANCE.name()) + .setActions(TransportSearchAction.TYPE.name()) .get(); List remoteTasks = remoteTasksResponse.getTasks(); assertThat(remoteTasks.size(), equalTo(0)); @@ -1554,7 +1553,7 @@ public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase { ListTasksResponse listTasksResponse = client(LOCAL_CLUSTER).admin() .cluster() .prepareListTasks() - .setActions(SearchAction.INSTANCE.name()) + .setActions(TransportSearchAction.TYPE.name()) .get(); List tasks = listTasksResponse.getTasks(); assertThat(tasks.size(), equalTo(0)); @@ -1562,7 +1561,7 @@ public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase { ListTasksResponse remoteTasksResponse = client(REMOTE_CLUSTER).admin() .cluster() .prepareListTasks() - .setActions(SearchAction.INSTANCE.name()) + .setActions(TransportSearchAction.TYPE.name()) .get(); List remoteTasks = remoteTasksResponse.getTasks(); assertThat(remoteTasks.size(), equalTo(0)); diff --git a/x-pack/plugin/async-search/src/main/java/org/elasticsearch/xpack/search/TransportSubmitAsyncSearchAction.java b/x-pack/plugin/async-search/src/main/java/org/elasticsearch/xpack/search/TransportSubmitAsyncSearchAction.java index 6368f6a10853..7cbd0662302d 100644 --- a/x-pack/plugin/async-search/src/main/java/org/elasticsearch/xpack/search/TransportSubmitAsyncSearchAction.java +++ b/x-pack/plugin/async-search/src/main/java/org/elasticsearch/xpack/search/TransportSubmitAsyncSearchAction.java @@ -8,7 +8,6 @@ package org.elasticsearch.xpack.search; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.DocWriteResponse; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.action.support.ActionFilters; @@ -87,7 +86,11 @@ public class TransportSubmitAsyncSearchAction extends HandledTransportAction submitListener) { final SearchRequest searchRequest = createSearchRequest(request, submitTask, request.getKeepAlive()); try (var ignored = threadContext.newTraceContext()) { - AsyncSearchTask searchTask = (AsyncSearchTask) taskManager.register("transport", SearchAction.INSTANCE.name(), searchRequest); + AsyncSearchTask searchTask = (AsyncSearchTask) taskManager.register( + "transport", + TransportSearchAction.TYPE.name(), + searchRequest + ); searchAction.execute(searchTask, searchRequest, searchTask.getSearchProgressActionListener()); searchTask.addCompletionListener(new ActionListener<>() { @Override diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authz/privilege/IndexPrivilege.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authz/privilege/IndexPrivilege.java index c09a661a6492..35c32780d2e4 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authz/privilege/IndexPrivilege.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authz/privilege/IndexPrivilege.java @@ -29,7 +29,7 @@ import org.elasticsearch.action.datastreams.DeleteDataStreamAction; import org.elasticsearch.action.datastreams.GetDataStreamAction; import org.elasticsearch.action.datastreams.PromoteDataStreamAction; import org.elasticsearch.action.fieldcaps.FieldCapabilitiesAction; -import org.elasticsearch.action.search.SearchShardsAction; +import org.elasticsearch.action.search.TransportSearchShardsAction; import org.elasticsearch.common.Strings; import org.elasticsearch.index.seqno.RetentionLeaseActions; import org.elasticsearch.xpack.core.ccr.action.ForgetFollowerAction; @@ -78,7 +78,7 @@ public final class IndexPrivilege extends Privilege { private static final Automaton READ_CROSS_CLUSTER_AUTOMATON = patterns( "internal:transport/proxy/indices:data/read/*", ClusterSearchShardsAction.NAME, - SearchShardsAction.NAME + TransportSearchShardsAction.TYPE.name() ); private static final Automaton CREATE_AUTOMATON = patterns( "indices:data/write/index*", @@ -119,7 +119,7 @@ public final class IndexPrivilege extends Privilege { GetFieldMappingsAction.NAME + "*", GetMappingsAction.NAME, ClusterSearchShardsAction.NAME, - SearchShardsAction.NAME, + TransportSearchShardsAction.TYPE.name(), ValidateQueryAction.NAME + "*", GetSettingsAction.NAME, ExplainLifecycleAction.NAME, diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authz/privilege/SystemPrivilege.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authz/privilege/SystemPrivilege.java index b15212d979d0..bc4263250725 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authz/privilege/SystemPrivilege.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authz/privilege/SystemPrivilege.java @@ -6,7 +6,7 @@ */ package org.elasticsearch.xpack.core.security.authz.privilege; -import org.elasticsearch.action.search.SearchShardsAction; +import org.elasticsearch.action.search.TransportSearchShardsAction; import org.elasticsearch.index.seqno.RetentionLeaseActions; import org.elasticsearch.index.seqno.RetentionLeaseBackgroundSyncAction; import org.elasticsearch.index.seqno.RetentionLeaseSyncAction; @@ -43,7 +43,7 @@ public final class SystemPrivilege extends Privilege { "indices:data/read/*", // needed for SystemIndexMigrator "indices:admin/refresh", // needed for SystemIndexMigrator "indices:admin/aliases", // needed for SystemIndexMigrator - SearchShardsAction.NAME // added so this API can be called with the system user by other APIs + TransportSearchShardsAction.TYPE.name() // added so this API can be called with the system user by other APIs ); private static final Predicate PREDICATE = (action) -> { diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ClientHelperTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ClientHelperTests.java index 3f977228edcb..0f3a58350c36 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ClientHelperTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ClientHelperTests.java @@ -12,10 +12,10 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.admin.cluster.health.ClusterHealthAction; import org.elasticsearch.action.admin.cluster.health.ClusterHealthRequest; import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.ShardSearchFailure; +import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.action.support.ActionTestUtils; import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.client.internal.Client; @@ -170,7 +170,7 @@ public class ClientHelperTests extends ESTestCase { SearchRequest request = new SearchRequest("foo"); String originName = randomFrom(ClientHelper.ML_ORIGIN, ClientHelper.WATCHER_ORIGIN, ClientHelper.ROLLUP_ORIGIN); - ClientHelper.executeWithHeadersAsync(Collections.emptyMap(), originName, client, SearchAction.INSTANCE, request, listener); + ClientHelper.executeWithHeadersAsync(Collections.emptyMap(), originName, client, TransportSearchAction.TYPE, request, listener); latch.await(); } @@ -201,7 +201,7 @@ public class ClientHelperTests extends ESTestCase { headers.put("bar", "bar"); String originName = randomFrom(ClientHelper.ML_ORIGIN, ClientHelper.WATCHER_ORIGIN, ClientHelper.ROLLUP_ORIGIN); - ClientHelper.executeWithHeadersAsync(headers, originName, client, SearchAction.INSTANCE, request, listener); + ClientHelper.executeWithHeadersAsync(headers, originName, client, TransportSearchAction.TYPE, request, listener); latch.await(); } @@ -234,7 +234,7 @@ public class ClientHelperTests extends ESTestCase { headers.put("_xpack_security_authentication", "bar"); String originName = randomFrom(ClientHelper.ML_ORIGIN, ClientHelper.WATCHER_ORIGIN, ClientHelper.ROLLUP_ORIGIN); - ClientHelper.executeWithHeadersAsync(headers, originName, client, SearchAction.INSTANCE, request, listener); + ClientHelper.executeWithHeadersAsync(headers, originName, client, TransportSearchAction.TYPE, request, listener); latch.await(); } diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ilm/LifecyclePolicyClientTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ilm/LifecyclePolicyClientTests.java index 162794865ba5..753edfbe334b 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ilm/LifecyclePolicyClientTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ilm/LifecyclePolicyClientTests.java @@ -8,9 +8,9 @@ package org.elasticsearch.xpack.core.ilm; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.action.support.ActionTestUtils; import org.elasticsearch.client.internal.Client; import org.elasticsearch.common.settings.Settings; @@ -57,7 +57,7 @@ public class LifecyclePolicyClientTests extends ESTestCase { SearchRequest request = new SearchRequest("foo"); final var policyClient = new LifecyclePolicySecurityClient(client, ClientHelper.INDEX_LIFECYCLE_ORIGIN, Collections.emptyMap()); - policyClient.execute(SearchAction.INSTANCE, request, listener); + policyClient.execute(TransportSearchAction.TYPE, request, listener); latch.await(); } @@ -89,7 +89,7 @@ public class LifecyclePolicyClientTests extends ESTestCase { headers.put("bar", "bar"); final var policyClient = new LifecyclePolicySecurityClient(client, ClientHelper.INDEX_LIFECYCLE_ORIGIN, headers); - policyClient.execute(SearchAction.INSTANCE, request, listener); + policyClient.execute(TransportSearchAction.TYPE, request, listener); latch.await(); } @@ -123,7 +123,7 @@ public class LifecyclePolicyClientTests extends ESTestCase { headers.put("_xpack_security_authentication", "bar"); final var policyClient = new LifecyclePolicySecurityClient(client, ClientHelper.INDEX_LIFECYCLE_ORIGIN, headers); - policyClient.execute(SearchAction.INSTANCE, request, listener); + policyClient.execute(TransportSearchAction.TYPE, request, listener); latch.await(); } diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/permission/LimitedRoleTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/permission/LimitedRoleTests.java index 6a3883672023..7ab450fc2e19 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/permission/LimitedRoleTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/permission/LimitedRoleTests.java @@ -11,7 +11,7 @@ import org.apache.lucene.util.automaton.Automaton; import org.elasticsearch.action.admin.indices.create.CreateIndexAction; import org.elasticsearch.action.admin.indices.delete.DeleteIndexAction; import org.elasticsearch.action.bulk.BulkAction; -import org.elasticsearch.action.search.SearchAction; +import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.cluster.metadata.AliasMetadata; import org.elasticsearch.cluster.metadata.IndexAbstraction; import org.elasticsearch.cluster.metadata.IndexMetadata; @@ -299,7 +299,7 @@ public class LimitedRoleTests extends ESTestCase { .build(); IndicesAccessControl iac = fromRole.authorize( - SearchAction.NAME, + TransportSearchAction.TYPE.name(), Sets.newHashSet("_index", "_alias1"), md.getIndicesLookup(), fieldPermissionsCache @@ -328,7 +328,7 @@ public class LimitedRoleTests extends ESTestCase { .add(IndexPrivilege.NONE, "_index1") .build(); iac = limitedByRole.authorize( - SearchAction.NAME, + TransportSearchAction.TYPE.name(), Sets.newHashSet("_index", "_alias1"), md.getIndicesLookup(), fieldPermissionsCache @@ -367,7 +367,12 @@ public class LimitedRoleTests extends ESTestCase { } else { role = fromRole.limitedBy(limitedByRole); } - iac = role.authorize(SearchAction.NAME, Sets.newHashSet("_index", "_alias1"), md.getIndicesLookup(), fieldPermissionsCache); + iac = role.authorize( + TransportSearchAction.TYPE.name(), + Sets.newHashSet("_index", "_alias1"), + md.getIndicesLookup(), + fieldPermissionsCache + ); assertThat(iac.isGranted(), is(false)); assertThat(iac.getIndexPermissions("_index"), is(notNullValue())); assertThat(iac.hasIndexPermissions("_index"), is(true)); @@ -440,12 +445,12 @@ public class LimitedRoleTests extends ESTestCase { public void testCheckIndicesAction() { Role fromRole = Role.builder(EMPTY_RESTRICTED_INDICES, "a-role").add(IndexPrivilege.READ, "ind-1").build(); - assertThat(fromRole.checkIndicesAction(SearchAction.NAME), is(true)); + assertThat(fromRole.checkIndicesAction(TransportSearchAction.TYPE.name()), is(true)); assertThat(fromRole.checkIndicesAction(CreateIndexAction.NAME), is(false)); { Role limitedByRole = Role.builder(EMPTY_RESTRICTED_INDICES, "limited-role").add(IndexPrivilege.ALL, "ind-1").build(); - assertThat(limitedByRole.checkIndicesAction(SearchAction.NAME), is(true)); + assertThat(limitedByRole.checkIndicesAction(TransportSearchAction.TYPE.name()), is(true)); assertThat(limitedByRole.checkIndicesAction(CreateIndexAction.NAME), is(true)); Role role; if (randomBoolean()) { @@ -453,64 +458,79 @@ public class LimitedRoleTests extends ESTestCase { } else { role = fromRole.limitedBy(limitedByRole); } - assertThat(role.checkIndicesAction(SearchAction.NAME), is(true)); + assertThat(role.checkIndicesAction(TransportSearchAction.TYPE.name()), is(true)); assertThat(role.checkIndicesAction(CreateIndexAction.NAME), is(false)); } { Role limitedByRole = Role.builder(EMPTY_RESTRICTED_INDICES, "limited-role").add(IndexPrivilege.NONE, "ind-1").build(); - assertThat(limitedByRole.checkIndicesAction(SearchAction.NAME), is(false)); + assertThat(limitedByRole.checkIndicesAction(TransportSearchAction.TYPE.name()), is(false)); Role role; if (randomBoolean()) { role = limitedByRole.limitedBy(fromRole); } else { role = fromRole.limitedBy(limitedByRole); } - assertThat(role.checkIndicesAction(SearchAction.NAME), is(false)); + assertThat(role.checkIndicesAction(TransportSearchAction.TYPE.name()), is(false)); assertThat(role.checkIndicesAction(CreateIndexAction.NAME), is(false)); } } public void testAllowedIndicesMatcher() { Role fromRole = Role.builder(EMPTY_RESTRICTED_INDICES, "a-role").add(IndexPrivilege.READ, "ind-1*").build(); - assertThat(fromRole.allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction("ind-1")), is(true)); - assertThat(fromRole.allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction("ind-11")), is(true)); - assertThat(fromRole.allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction("ind-2")), is(false)); + assertThat(fromRole.allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction("ind-1")), is(true)); + assertThat(fromRole.allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction("ind-11")), is(true)); + assertThat(fromRole.allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction("ind-2")), is(false)); { Role limitedByRole = Role.builder(EMPTY_RESTRICTED_INDICES, "limited-role").add(IndexPrivilege.READ, "ind-1", "ind-2").build(); - assertThat(limitedByRole.allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction("ind-1")), is(true)); - assertThat(limitedByRole.allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction("ind-11")), is(false)); - assertThat(limitedByRole.allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction("ind-2")), is(true)); + assertThat( + limitedByRole.allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction("ind-1")), + is(true) + ); + assertThat( + limitedByRole.allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction("ind-11")), + is(false) + ); + assertThat( + limitedByRole.allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction("ind-2")), + is(true) + ); Role role; if (randomBoolean()) { role = limitedByRole.limitedBy(fromRole); } else { role = fromRole.limitedBy(limitedByRole); } - assertThat(role.allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction("ind-1")), is(true)); - assertThat(role.allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction("ind-11")), is(false)); - assertThat(role.allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction("ind-2")), is(false)); + assertThat(role.allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction("ind-1")), is(true)); + assertThat(role.allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction("ind-11")), is(false)); + assertThat(role.allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction("ind-2")), is(false)); } { Role limitedByRole = Role.builder(EMPTY_RESTRICTED_INDICES, "limited-role").add(IndexPrivilege.READ, "ind-*").build(); - assertThat(limitedByRole.allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction("ind-1")), is(true)); - assertThat(limitedByRole.allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction("ind-2")), is(true)); + assertThat( + limitedByRole.allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction("ind-1")), + is(true) + ); + assertThat( + limitedByRole.allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction("ind-2")), + is(true) + ); Role role; if (randomBoolean()) { role = limitedByRole.limitedBy(fromRole); } else { role = fromRole.limitedBy(limitedByRole); } - assertThat(role.allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction("ind-1")), is(true)); - assertThat(role.allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction("ind-2")), is(false)); + assertThat(role.allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction("ind-1")), is(true)); + assertThat(role.allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction("ind-2")), is(false)); } } public void testAllowedIndicesMatcherWithNestedRole() { Role role = Role.builder(EMPTY_RESTRICTED_INDICES, "a-role").add(IndexPrivilege.READ, "ind-1*").build(); - assertThat(role.allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction("ind-1")), is(true)); - assertThat(role.allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction("ind-11")), is(true)); - assertThat(role.allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction("ind-2")), is(false)); + assertThat(role.allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction("ind-1")), is(true)); + assertThat(role.allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction("ind-11")), is(true)); + assertThat(role.allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction("ind-2")), is(false)); final int depth = randomIntBetween(2, 4); boolean index11Excluded = false; @@ -526,9 +546,12 @@ public class LimitedRoleTests extends ESTestCase { } else { role = role.limitedBy(limitedByRole); } - assertThat(role.allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction("ind-1")), is(true)); - assertThat(role.allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction("ind-11")), is(false == index11Excluded)); - assertThat(role.allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction("ind-2")), is(false)); + assertThat(role.allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction("ind-1")), is(true)); + assertThat( + role.allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction("ind-11")), + is(false == index11Excluded) + ); + assertThat(role.allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction("ind-2")), is(false)); } } @@ -540,13 +563,13 @@ public class LimitedRoleTests extends ESTestCase { .build(); Automaton fromRoleAutomaton = fromRole.allowedActionsMatcher("index1"); Predicate fromRolePredicate = Automatons.predicate(fromRoleAutomaton); - assertThat(fromRolePredicate.test(SearchAction.NAME), is(true)); + assertThat(fromRolePredicate.test(TransportSearchAction.TYPE.name()), is(true)); assertThat(fromRolePredicate.test(BulkAction.NAME), is(true)); Role limitedByRole = Role.builder(EMPTY_RESTRICTED_INDICES, "limitedRole").add(IndexPrivilege.READ, "index1", "index2").build(); Automaton limitedByRoleAutomaton = limitedByRole.allowedActionsMatcher("index1"); Predicate limitedByRolePredicated = Automatons.predicate(limitedByRoleAutomaton); - assertThat(limitedByRolePredicated.test(SearchAction.NAME), is(true)); + assertThat(limitedByRolePredicated.test(TransportSearchAction.TYPE.name()), is(true)); assertThat(limitedByRolePredicated.test(BulkAction.NAME), is(false)); Role role; if (randomBoolean()) { @@ -557,17 +580,17 @@ public class LimitedRoleTests extends ESTestCase { Automaton roleAutomaton = role.allowedActionsMatcher("index1"); Predicate rolePredicate = Automatons.predicate(roleAutomaton); - assertThat(rolePredicate.test(SearchAction.NAME), is(true)); + assertThat(rolePredicate.test(TransportSearchAction.TYPE.name()), is(true)); assertThat(rolePredicate.test(BulkAction.NAME), is(false)); roleAutomaton = role.allowedActionsMatcher("index2"); rolePredicate = Automatons.predicate(roleAutomaton); - assertThat(rolePredicate.test(SearchAction.NAME), is(true)); + assertThat(rolePredicate.test(TransportSearchAction.TYPE.name()), is(true)); assertThat(rolePredicate.test(BulkAction.NAME), is(false)); roleAutomaton = role.allowedActionsMatcher("other"); rolePredicate = Automatons.predicate(roleAutomaton); - assertThat(rolePredicate.test(SearchAction.NAME), is(false)); + assertThat(rolePredicate.test(TransportSearchAction.TYPE.name()), is(false)); assertThat(rolePredicate.test(BulkAction.NAME), is(false)); } diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/privilege/IndexPrivilegeTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/privilege/IndexPrivilegeTests.java index 37f899903141..9dde59465336 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/privilege/IndexPrivilegeTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/privilege/IndexPrivilegeTests.java @@ -13,7 +13,7 @@ import org.elasticsearch.action.admin.indices.shrink.ShrinkAction; import org.elasticsearch.action.admin.indices.stats.IndicesStatsAction; import org.elasticsearch.action.delete.DeleteAction; import org.elasticsearch.action.index.IndexAction; -import org.elasticsearch.action.search.SearchAction; +import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.action.update.UpdateAction; import org.elasticsearch.common.util.iterable.Iterables; import org.elasticsearch.test.ESTestCase; @@ -59,7 +59,7 @@ public class IndexPrivilegeTests extends ESTestCase { } public void testFindPrivilegesThatGrant() { - assertThat(findPrivilegesThatGrant(SearchAction.NAME), equalTo(List.of("read", "all"))); + assertThat(findPrivilegesThatGrant(TransportSearchAction.TYPE.name()), equalTo(List.of("read", "all"))); assertThat(findPrivilegesThatGrant(IndexAction.NAME), equalTo(List.of("create_doc", "create", "index", "write", "all"))); assertThat(findPrivilegesThatGrant(UpdateAction.NAME), equalTo(List.of("index", "write", "all"))); assertThat(findPrivilegesThatGrant(DeleteAction.NAME), equalTo(List.of("delete", "write", "all"))); diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/store/ReservedRolesStoreTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/store/ReservedRolesStoreTests.java index a7b503cfc2ed..6ee70173f505 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/store/ReservedRolesStoreTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/store/ReservedRolesStoreTests.java @@ -52,9 +52,9 @@ import org.elasticsearch.action.ingest.DeletePipelineAction; import org.elasticsearch.action.ingest.GetPipelineAction; import org.elasticsearch.action.ingest.PutPipelineAction; import org.elasticsearch.action.ingest.SimulatePipelineAction; -import org.elasticsearch.action.search.MultiSearchAction; -import org.elasticsearch.action.search.SearchAction; -import org.elasticsearch.action.search.SearchShardsAction; +import org.elasticsearch.action.search.TransportMultiSearchAction; +import org.elasticsearch.action.search.TransportSearchAction; +import org.elasticsearch.action.search.TransportSearchShardsAction; import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.action.support.WriteRequest; import org.elasticsearch.action.update.UpdateAction; @@ -615,8 +615,14 @@ public class ReservedRolesStoreTests extends ESTestCase { assertThat(kibanaRole.indices().allowedIndicesMatcher(IndexAction.NAME).test(mockIndexAbstraction(index)), is(false)); assertThat(kibanaRole.indices().allowedIndicesMatcher(DeleteAction.NAME).test(mockIndexAbstraction(index)), is(false)); assertThat(kibanaRole.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(mockIndexAbstraction(index)), is(false)); - assertThat(kibanaRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(index)), is(true)); - assertThat(kibanaRole.indices().allowedIndicesMatcher(MultiSearchAction.NAME).test(mockIndexAbstraction(index)), is(true)); + assertThat( + kibanaRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction(index)), + is(true) + ); + assertThat( + kibanaRole.indices().allowedIndicesMatcher(TransportMultiSearchAction.TYPE.name()).test(mockIndexAbstraction(index)), + is(true) + ); assertThat(kibanaRole.indices().allowedIndicesMatcher(GetAction.NAME).test(mockIndexAbstraction(index)), is(true)); assertThat(kibanaRole.indices().allowedIndicesMatcher(READ_CROSS_CLUSTER_NAME).test(mockIndexAbstraction(index)), is(true)); }); @@ -634,8 +640,14 @@ public class ReservedRolesStoreTests extends ESTestCase { assertThat(kibanaRole.indices().allowedIndicesMatcher(IndexAction.NAME).test(mockIndexAbstraction(index)), is(false)); assertThat(kibanaRole.indices().allowedIndicesMatcher(DeleteAction.NAME).test(mockIndexAbstraction(index)), is(false)); assertThat(kibanaRole.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(mockIndexAbstraction(index)), is(false)); - assertThat(kibanaRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(index)), is(true)); - assertThat(kibanaRole.indices().allowedIndicesMatcher(MultiSearchAction.NAME).test(mockIndexAbstraction(index)), is(true)); + assertThat( + kibanaRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction(index)), + is(true) + ); + assertThat( + kibanaRole.indices().allowedIndicesMatcher(TransportMultiSearchAction.TYPE.name()).test(mockIndexAbstraction(index)), + is(true) + ); assertThat(kibanaRole.indices().allowedIndicesMatcher(GetAction.NAME).test(mockIndexAbstraction(index)), is(true)); assertThat(kibanaRole.indices().allowedIndicesMatcher(READ_CROSS_CLUSTER_NAME).test(mockIndexAbstraction(index)), is(false)); }); @@ -653,8 +665,14 @@ public class ReservedRolesStoreTests extends ESTestCase { assertThat(kibanaRole.indices().allowedIndicesMatcher(IndexAction.NAME).test(mockIndexAbstraction(index)), is(true)); assertThat(kibanaRole.indices().allowedIndicesMatcher(DeleteAction.NAME).test(mockIndexAbstraction(index)), is(true)); assertThat(kibanaRole.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(mockIndexAbstraction(index)), is(false)); - assertThat(kibanaRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(index)), is(true)); - assertThat(kibanaRole.indices().allowedIndicesMatcher(MultiSearchAction.NAME).test(mockIndexAbstraction(index)), is(true)); + assertThat( + kibanaRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction(index)), + is(true) + ); + assertThat( + kibanaRole.indices().allowedIndicesMatcher(TransportMultiSearchAction.TYPE.name()).test(mockIndexAbstraction(index)), + is(true) + ); assertThat(kibanaRole.indices().allowedIndicesMatcher(GetAction.NAME).test(mockIndexAbstraction(index)), is(true)); assertThat(kibanaRole.indices().allowedIndicesMatcher(READ_CROSS_CLUSTER_NAME).test(mockIndexAbstraction(index)), is(false)); }); @@ -669,8 +687,14 @@ public class ReservedRolesStoreTests extends ESTestCase { assertThat(kibanaRole.indices().allowedIndicesMatcher(IndexAction.NAME).test(mockIndexAbstraction(index)), is(false)); assertThat(kibanaRole.indices().allowedIndicesMatcher(DeleteAction.NAME).test(mockIndexAbstraction(index)), is(false)); assertThat(kibanaRole.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(mockIndexAbstraction(index)), is(false)); - assertThat(kibanaRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(index)), is(true)); - assertThat(kibanaRole.indices().allowedIndicesMatcher(MultiSearchAction.NAME).test(mockIndexAbstraction(index)), is(true)); + assertThat( + kibanaRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction(index)), + is(true) + ); + assertThat( + kibanaRole.indices().allowedIndicesMatcher(TransportMultiSearchAction.TYPE.name()).test(mockIndexAbstraction(index)), + is(true) + ); assertThat(kibanaRole.indices().allowedIndicesMatcher(GetAction.NAME).test(mockIndexAbstraction(index)), is(true)); assertThat(kibanaRole.indices().allowedIndicesMatcher(READ_CROSS_CLUSTER_NAME).test(mockIndexAbstraction(index)), is(true)); }); @@ -688,8 +712,14 @@ public class ReservedRolesStoreTests extends ESTestCase { assertThat(kibanaRole.indices().allowedIndicesMatcher(CreateIndexAction.NAME).test(mockIndexAbstraction(index)), is(false)); assertThat(kibanaRole.indices().allowedIndicesMatcher(IndexAction.NAME).test(mockIndexAbstraction(index)), is(false)); assertThat(kibanaRole.indices().allowedIndicesMatcher(DeleteAction.NAME).test(mockIndexAbstraction(index)), is(false)); - assertThat(kibanaRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(index)), is(true)); - assertThat(kibanaRole.indices().allowedIndicesMatcher(MultiSearchAction.NAME).test(mockIndexAbstraction(index)), is(true)); + assertThat( + kibanaRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction(index)), + is(true) + ); + assertThat( + kibanaRole.indices().allowedIndicesMatcher(TransportMultiSearchAction.TYPE.name()).test(mockIndexAbstraction(index)), + is(true) + ); assertThat(kibanaRole.indices().allowedIndicesMatcher(GetAction.NAME).test(mockIndexAbstraction(index)), is(true)); assertThat(kibanaRole.indices().allowedIndicesMatcher(READ_CROSS_CLUSTER_NAME).test(mockIndexAbstraction(index)), is(true)); }); @@ -702,8 +732,14 @@ public class ReservedRolesStoreTests extends ESTestCase { assertThat(kibanaRole.indices().allowedIndicesMatcher(CreateIndexAction.NAME).test(mockIndexAbstraction(index)), is(false)); assertThat(kibanaRole.indices().allowedIndicesMatcher(IndexAction.NAME).test(mockIndexAbstraction(index)), is(false)); assertThat(kibanaRole.indices().allowedIndicesMatcher(DeleteAction.NAME).test(mockIndexAbstraction(index)), is(false)); - assertThat(kibanaRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(index)), is(true)); - assertThat(kibanaRole.indices().allowedIndicesMatcher(MultiSearchAction.NAME).test(mockIndexAbstraction(index)), is(true)); + assertThat( + kibanaRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction(index)), + is(true) + ); + assertThat( + kibanaRole.indices().allowedIndicesMatcher(TransportMultiSearchAction.TYPE.name()).test(mockIndexAbstraction(index)), + is(true) + ); assertThat(kibanaRole.indices().allowedIndicesMatcher(GetAction.NAME).test(mockIndexAbstraction(index)), is(true)); assertThat(kibanaRole.indices().allowedIndicesMatcher(READ_CROSS_CLUSTER_NAME).test(mockIndexAbstraction(index)), is(false)); @@ -725,8 +761,8 @@ public class ReservedRolesStoreTests extends ESTestCase { assertThat(kibanaRole.indices().allowedIndicesMatcher(CreateIndexAction.NAME).test(indexAbstraction), is(false)); assertThat(kibanaRole.indices().allowedIndicesMatcher(IndexAction.NAME).test(indexAbstraction), is(false)); assertThat(kibanaRole.indices().allowedIndicesMatcher(DeleteAction.NAME).test(indexAbstraction), is(false)); - assertThat(kibanaRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(indexAbstraction), is(true)); - assertThat(kibanaRole.indices().allowedIndicesMatcher(MultiSearchAction.NAME).test(indexAbstraction), is(true)); + assertThat(kibanaRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(indexAbstraction), is(true)); + assertThat(kibanaRole.indices().allowedIndicesMatcher(TransportMultiSearchAction.TYPE.name()).test(indexAbstraction), is(true)); assertThat(kibanaRole.indices().allowedIndicesMatcher(GetAction.NAME).test(indexAbstraction), is(true)); assertThat(kibanaRole.indices().allowedIndicesMatcher(READ_CROSS_CLUSTER_NAME).test(indexAbstraction), is(false)); assertThat(kibanaRole.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(indexAbstraction), is(true)); @@ -744,8 +780,8 @@ public class ReservedRolesStoreTests extends ESTestCase { assertThat(kibanaRole.indices().allowedIndicesMatcher(CreateIndexAction.NAME).test(indexAbstraction), is(false)); assertThat(kibanaRole.indices().allowedIndicesMatcher(IndexAction.NAME).test(indexAbstraction), is(false)); assertThat(kibanaRole.indices().allowedIndicesMatcher(DeleteAction.NAME).test(indexAbstraction), is(false)); - assertThat(kibanaRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(indexAbstraction), is(true)); - assertThat(kibanaRole.indices().allowedIndicesMatcher(MultiSearchAction.NAME).test(indexAbstraction), is(true)); + assertThat(kibanaRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(indexAbstraction), is(true)); + assertThat(kibanaRole.indices().allowedIndicesMatcher(TransportMultiSearchAction.TYPE.name()).test(indexAbstraction), is(true)); assertThat(kibanaRole.indices().allowedIndicesMatcher(GetAction.NAME).test(indexAbstraction), is(true)); assertThat(kibanaRole.indices().allowedIndicesMatcher(READ_CROSS_CLUSTER_NAME).test(indexAbstraction), is(false)); assertThat(kibanaRole.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(indexAbstraction), is(true)); @@ -770,8 +806,11 @@ public class ReservedRolesStoreTests extends ESTestCase { assertThat(kibanaRole.indices().allowedIndicesMatcher(CreateIndexAction.NAME).test(dotFleetSecretsIndex), is(true)); assertThat(kibanaRole.indices().allowedIndicesMatcher(IndexAction.NAME).test(dotFleetSecretsIndex), is(true)); assertThat(kibanaRole.indices().allowedIndicesMatcher(DeleteAction.NAME).test(dotFleetSecretsIndex), is(true)); - assertThat(kibanaRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(dotFleetSecretsIndex), is(false)); - assertThat(kibanaRole.indices().allowedIndicesMatcher(MultiSearchAction.NAME).test(dotFleetSecretsIndex), is(false)); + assertThat(kibanaRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(dotFleetSecretsIndex), is(false)); + assertThat( + kibanaRole.indices().allowedIndicesMatcher(TransportMultiSearchAction.TYPE.name()).test(dotFleetSecretsIndex), + is(false) + ); assertThat(kibanaRole.indices().allowedIndicesMatcher(GetAction.NAME).test(dotFleetSecretsIndex), is(false)); assertThat(kibanaRole.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(dotFleetSecretsIndex), is(false)); @@ -787,8 +826,14 @@ public class ReservedRolesStoreTests extends ESTestCase { assertThat(kibanaRole.indices().allowedIndicesMatcher(CreateIndexAction.NAME).test(mockIndexAbstraction(index)), is(false)); assertThat(kibanaRole.indices().allowedIndicesMatcher(IndexAction.NAME).test(mockIndexAbstraction(index)), is(false)); assertThat(kibanaRole.indices().allowedIndicesMatcher(DeleteAction.NAME).test(mockIndexAbstraction(index)), is(false)); - assertThat(kibanaRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(index)), is(true)); - assertThat(kibanaRole.indices().allowedIndicesMatcher(MultiSearchAction.NAME).test(mockIndexAbstraction(index)), is(true)); + assertThat( + kibanaRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction(index)), + is(true) + ); + assertThat( + kibanaRole.indices().allowedIndicesMatcher(TransportMultiSearchAction.TYPE.name()).test(mockIndexAbstraction(index)), + is(true) + ); assertThat(kibanaRole.indices().allowedIndicesMatcher(GetAction.NAME).test(mockIndexAbstraction(index)), is(true)); }); @@ -802,8 +847,8 @@ public class ReservedRolesStoreTests extends ESTestCase { assertThat(kibanaRole.indices().allowedIndicesMatcher(CreateIndexAction.NAME).test(indexAbstraction), is(false)); assertThat(kibanaRole.indices().allowedIndicesMatcher(IndexAction.NAME).test(indexAbstraction), is(false)); assertThat(kibanaRole.indices().allowedIndicesMatcher(DeleteAction.NAME).test(indexAbstraction), is(false)); - assertThat(kibanaRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(indexAbstraction), is(true)); - assertThat(kibanaRole.indices().allowedIndicesMatcher(MultiSearchAction.NAME).test(indexAbstraction), is(true)); + assertThat(kibanaRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(indexAbstraction), is(true)); + assertThat(kibanaRole.indices().allowedIndicesMatcher(TransportMultiSearchAction.TYPE.name()).test(indexAbstraction), is(true)); assertThat(kibanaRole.indices().allowedIndicesMatcher(GetAction.NAME).test(indexAbstraction), is(true)); assertThat(kibanaRole.indices().allowedIndicesMatcher(READ_CROSS_CLUSTER_NAME).test(indexAbstraction), is(false)); assertThat(kibanaRole.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(indexAbstraction), is(true)); @@ -820,8 +865,8 @@ public class ReservedRolesStoreTests extends ESTestCase { assertThat(kibanaRole.indices().allowedIndicesMatcher(CreateIndexAction.NAME).test(indexAbstraction), is(true)); assertThat(kibanaRole.indices().allowedIndicesMatcher(IndexAction.NAME).test(indexAbstraction), is(true)); assertThat(kibanaRole.indices().allowedIndicesMatcher(DeleteAction.NAME).test(indexAbstraction), is(true)); - assertThat(kibanaRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(indexAbstraction), is(true)); - assertThat(kibanaRole.indices().allowedIndicesMatcher(MultiSearchAction.NAME).test(indexAbstraction), is(true)); + assertThat(kibanaRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(indexAbstraction), is(true)); + assertThat(kibanaRole.indices().allowedIndicesMatcher(TransportMultiSearchAction.TYPE.name()).test(indexAbstraction), is(true)); assertThat(kibanaRole.indices().allowedIndicesMatcher(GetAction.NAME).test(indexAbstraction), is(true)); assertThat(kibanaRole.indices().allowedIndicesMatcher(READ_CROSS_CLUSTER_NAME).test(indexAbstraction), is(false)); assertThat(kibanaRole.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(indexAbstraction), is(true)); @@ -839,8 +884,8 @@ public class ReservedRolesStoreTests extends ESTestCase { assertThat(kibanaRole.indices().allowedIndicesMatcher(CreateIndexAction.NAME).test(indexAbstraction), is(false)); assertThat(kibanaRole.indices().allowedIndicesMatcher(IndexAction.NAME).test(indexAbstraction), is(true)); assertThat(kibanaRole.indices().allowedIndicesMatcher(DeleteAction.NAME).test(indexAbstraction), is(true)); - assertThat(kibanaRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(indexAbstraction), is(true)); - assertThat(kibanaRole.indices().allowedIndicesMatcher(MultiSearchAction.NAME).test(indexAbstraction), is(true)); + assertThat(kibanaRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(indexAbstraction), is(true)); + assertThat(kibanaRole.indices().allowedIndicesMatcher(TransportMultiSearchAction.TYPE.name()).test(indexAbstraction), is(true)); assertThat(kibanaRole.indices().allowedIndicesMatcher(GetAction.NAME).test(indexAbstraction), is(true)); assertThat(kibanaRole.indices().allowedIndicesMatcher(READ_CROSS_CLUSTER_NAME).test(indexAbstraction), is(false)); assertThat(kibanaRole.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(indexAbstraction), is(true)); @@ -858,8 +903,8 @@ public class ReservedRolesStoreTests extends ESTestCase { assertThat(kibanaRole.indices().allowedIndicesMatcher(CreateIndexAction.NAME).test(indexAbstraction), is(false)); assertThat(kibanaRole.indices().allowedIndicesMatcher(IndexAction.NAME).test(indexAbstraction), is(false)); assertThat(kibanaRole.indices().allowedIndicesMatcher(DeleteAction.NAME).test(indexAbstraction), is(false)); - assertThat(kibanaRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(indexAbstraction), is(true)); - assertThat(kibanaRole.indices().allowedIndicesMatcher(MultiSearchAction.NAME).test(indexAbstraction), is(true)); + assertThat(kibanaRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(indexAbstraction), is(true)); + assertThat(kibanaRole.indices().allowedIndicesMatcher(TransportMultiSearchAction.TYPE.name()).test(indexAbstraction), is(true)); assertThat(kibanaRole.indices().allowedIndicesMatcher(GetAction.NAME).test(indexAbstraction), is(true)); assertThat(kibanaRole.indices().allowedIndicesMatcher(READ_CROSS_CLUSTER_NAME).test(indexAbstraction), is(false)); assertThat(kibanaRole.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(indexAbstraction), is(true)); @@ -880,8 +925,14 @@ public class ReservedRolesStoreTests extends ESTestCase { assertThat(kibanaRole.indices().allowedIndicesMatcher(IndexAction.NAME).test(mockIndexAbstraction(index)), is(false)); assertThat(kibanaRole.indices().allowedIndicesMatcher(DeleteAction.NAME).test(mockIndexAbstraction(index)), is(false)); assertThat(kibanaRole.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(mockIndexAbstraction(index)), is(false)); - assertThat(kibanaRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(index)), is(false)); - assertThat(kibanaRole.indices().allowedIndicesMatcher(MultiSearchAction.NAME).test(mockIndexAbstraction(index)), is(false)); + assertThat( + kibanaRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction(index)), + is(false) + ); + assertThat( + kibanaRole.indices().allowedIndicesMatcher(TransportMultiSearchAction.TYPE.name()).test(mockIndexAbstraction(index)), + is(false) + ); assertThat(kibanaRole.indices().allowedIndicesMatcher(GetAction.NAME).test(mockIndexAbstraction(index)), is(false)); assertThat(kibanaRole.indices().allowedIndicesMatcher(READ_CROSS_CLUSTER_NAME).test(mockIndexAbstraction(index)), is(false)); }); @@ -918,8 +969,8 @@ public class ReservedRolesStoreTests extends ESTestCase { assertThat(kibanaRole.indices().allowedIndicesMatcher(CreateIndexAction.NAME).test(indexAbstraction), is(false)); assertThat(kibanaRole.indices().allowedIndicesMatcher(IndexAction.NAME).test(indexAbstraction), is(false)); assertThat(kibanaRole.indices().allowedIndicesMatcher(DeleteAction.NAME).test(indexAbstraction), is(false)); - assertThat(kibanaRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(indexAbstraction), is(true)); - assertThat(kibanaRole.indices().allowedIndicesMatcher(MultiSearchAction.NAME).test(indexAbstraction), is(true)); + assertThat(kibanaRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(indexAbstraction), is(true)); + assertThat(kibanaRole.indices().allowedIndicesMatcher(TransportMultiSearchAction.TYPE.name()).test(indexAbstraction), is(true)); assertThat(kibanaRole.indices().allowedIndicesMatcher(GetAction.NAME).test(indexAbstraction), is(true)); assertThat(kibanaRole.indices().allowedIndicesMatcher(READ_CROSS_CLUSTER_NAME).test(indexAbstraction), is(false)); assertThat(kibanaRole.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(indexAbstraction), is(true)); @@ -937,8 +988,8 @@ public class ReservedRolesStoreTests extends ESTestCase { assertThat(kibanaRole.indices().allowedIndicesMatcher(CreateIndexAction.NAME).test(indexAbstraction), is(false)); assertThat(kibanaRole.indices().allowedIndicesMatcher(IndexAction.NAME).test(indexAbstraction), is(false)); assertThat(kibanaRole.indices().allowedIndicesMatcher(DeleteAction.NAME).test(indexAbstraction), is(false)); - assertThat(kibanaRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(indexAbstraction), is(true)); - assertThat(kibanaRole.indices().allowedIndicesMatcher(MultiSearchAction.NAME).test(indexAbstraction), is(true)); + assertThat(kibanaRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(indexAbstraction), is(true)); + assertThat(kibanaRole.indices().allowedIndicesMatcher(TransportMultiSearchAction.TYPE.name()).test(indexAbstraction), is(true)); assertThat(kibanaRole.indices().allowedIndicesMatcher(GetAction.NAME).test(indexAbstraction), is(true)); assertThat(kibanaRole.indices().allowedIndicesMatcher(READ_CROSS_CLUSTER_NAME).test(indexAbstraction), is(false)); assertThat(kibanaRole.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(indexAbstraction), is(true)); @@ -955,8 +1006,14 @@ public class ReservedRolesStoreTests extends ESTestCase { assertThat(kibanaRole.indices().allowedIndicesMatcher(IndexAction.NAME).test(mockIndexAbstraction(index)), is(true)); assertThat(kibanaRole.indices().allowedIndicesMatcher(DeleteAction.NAME).test(mockIndexAbstraction(index)), is(true)); assertThat(kibanaRole.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(mockIndexAbstraction(index)), is(false)); - assertThat(kibanaRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(index)), is(true)); - assertThat(kibanaRole.indices().allowedIndicesMatcher(MultiSearchAction.NAME).test(mockIndexAbstraction(index)), is(true)); + assertThat( + kibanaRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction(index)), + is(true) + ); + assertThat( + kibanaRole.indices().allowedIndicesMatcher(TransportMultiSearchAction.TYPE.name()).test(mockIndexAbstraction(index)), + is(true) + ); assertThat(kibanaRole.indices().allowedIndicesMatcher(GetAction.NAME).test(mockIndexAbstraction(index)), is(true)); assertThat(kibanaRole.indices().allowedIndicesMatcher(READ_CROSS_CLUSTER_NAME).test(mockIndexAbstraction(index)), is(false)); @@ -1029,8 +1086,14 @@ public class ReservedRolesStoreTests extends ESTestCase { || indexName.startsWith(".logs-endpoint.heartbeat-") || indexName.startsWith(".logs-osquery_manager.actions-"); assertThat(kibanaRole.indices().allowedIndicesMatcher(GetAction.NAME).test(indexAbstraction), is(isAlsoReadIndex)); - assertThat(kibanaRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(indexAbstraction), is(isAlsoReadIndex)); - assertThat(kibanaRole.indices().allowedIndicesMatcher(MultiSearchAction.NAME).test(indexAbstraction), is(isAlsoReadIndex)); + assertThat( + kibanaRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(indexAbstraction), + is(isAlsoReadIndex) + ); + assertThat( + kibanaRole.indices().allowedIndicesMatcher(TransportMultiSearchAction.TYPE.name()).test(indexAbstraction), + is(isAlsoReadIndex) + ); // Endpoint diagnostic, APM and Synthetics data streams also have an ILM policy with a delete action, all others should not. final boolean isAlsoIlmDeleteIndex = indexName.startsWith(".logs-endpoint.diagnostic.collection-") @@ -1078,7 +1141,7 @@ public class ReservedRolesStoreTests extends ESTestCase { logger.info("index name [{}]", indexName); final IndexAbstraction indexAbstraction = mockIndexAbstraction(indexName); // Allow indexing - assertThat(kibanaRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(indexAbstraction), is(true)); + assertThat(kibanaRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(indexAbstraction), is(true)); assertThat(kibanaRole.indices().allowedIndicesMatcher(GetAction.NAME).test(indexAbstraction), is(true)); assertThat(kibanaRole.indices().allowedIndicesMatcher(IndexAction.NAME).test(indexAbstraction), is(true)); assertThat(kibanaRole.indices().allowedIndicesMatcher(UpdateAction.NAME).test(indexAbstraction), is(true)); @@ -1116,7 +1179,7 @@ public class ReservedRolesStoreTests extends ESTestCase { Arrays.asList("logs-ti_recordedfuture_latest.threat", "logs-ti_anomali_latest.threatstream").forEach(indexName -> { final IndexAbstraction indexAbstraction = mockIndexAbstraction(indexName); // Allow search and indexing - assertThat(kibanaRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(indexAbstraction), is(true)); + assertThat(kibanaRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(indexAbstraction), is(true)); assertThat(kibanaRole.indices().allowedIndicesMatcher(GetAction.NAME).test(indexAbstraction), is(true)); assertThat(kibanaRole.indices().allowedIndicesMatcher(IndexAction.NAME).test(indexAbstraction), is(true)); assertThat(kibanaRole.indices().allowedIndicesMatcher(UpdateAction.NAME).test(indexAbstraction), is(true)); @@ -1158,8 +1221,8 @@ public class ReservedRolesStoreTests extends ESTestCase { assertThat(kibanaRole.indices().allowedIndicesMatcher(PutMappingAction.NAME).test(indexAbstraction), is(true)); assertThat(kibanaRole.indices().allowedIndicesMatcher(RolloverAction.NAME).test(indexAbstraction), is(true)); assertThat(kibanaRole.indices().allowedIndicesMatcher(DeleteIndexAction.NAME).test(indexAbstraction), is(true)); - assertThat(kibanaRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(indexAbstraction), is(true)); - assertThat(kibanaRole.indices().allowedIndicesMatcher(MultiSearchAction.NAME).test(indexAbstraction), is(true)); + assertThat(kibanaRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(indexAbstraction), is(true)); + assertThat(kibanaRole.indices().allowedIndicesMatcher(TransportMultiSearchAction.TYPE.name()).test(indexAbstraction), is(true)); assertThat(kibanaRole.indices().allowedIndicesMatcher(GetAction.NAME).test(indexAbstraction), is(true)); // Implied by the overall view_index_metadata and monitor privilege @@ -1177,7 +1240,7 @@ public class ReservedRolesStoreTests extends ESTestCase { logger.info("index name [{}]", indexName); final IndexAbstraction indexAbstraction = mockIndexAbstraction(indexName); // Allow indexing - assertThat(kibanaRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(indexAbstraction), is(true)); + assertThat(kibanaRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(indexAbstraction), is(true)); assertThat(kibanaRole.indices().allowedIndicesMatcher(GetAction.NAME).test(indexAbstraction), is(true)); assertThat(kibanaRole.indices().allowedIndicesMatcher(IndexAction.NAME).test(indexAbstraction), is(true)); assertThat(kibanaRole.indices().allowedIndicesMatcher(UpdateAction.NAME).test(indexAbstraction), is(true)); @@ -1205,8 +1268,8 @@ public class ReservedRolesStoreTests extends ESTestCase { assertThat(kibanaRole.indices().allowedIndicesMatcher(CreateIndexAction.NAME).test(indexAbstraction), is(false)); assertThat(kibanaRole.indices().allowedIndicesMatcher(IndexAction.NAME).test(indexAbstraction), is(false)); assertThat(kibanaRole.indices().allowedIndicesMatcher(DeleteAction.NAME).test(indexAbstraction), is(false)); - assertThat(kibanaRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(indexAbstraction), is(true)); - assertThat(kibanaRole.indices().allowedIndicesMatcher(MultiSearchAction.NAME).test(indexAbstraction), is(true)); + assertThat(kibanaRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(indexAbstraction), is(true)); + assertThat(kibanaRole.indices().allowedIndicesMatcher(TransportMultiSearchAction.TYPE.name()).test(indexAbstraction), is(true)); assertThat(kibanaRole.indices().allowedIndicesMatcher(GetAction.NAME).test(indexAbstraction), is(true)); assertThat(kibanaRole.indices().allowedIndicesMatcher(READ_CROSS_CLUSTER_NAME).test(indexAbstraction), is(false)); assertThat(kibanaRole.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(indexAbstraction), is(true)); @@ -1223,8 +1286,8 @@ public class ReservedRolesStoreTests extends ESTestCase { assertThat(kibanaRole.indices().allowedIndicesMatcher(CreateIndexAction.NAME).test(indexAbstraction), is(false)); assertThat(kibanaRole.indices().allowedIndicesMatcher(IndexAction.NAME).test(indexAbstraction), is(false)); assertThat(kibanaRole.indices().allowedIndicesMatcher(DeleteAction.NAME).test(indexAbstraction), is(false)); - assertThat(kibanaRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(indexAbstraction), is(true)); - assertThat(kibanaRole.indices().allowedIndicesMatcher(MultiSearchAction.NAME).test(indexAbstraction), is(true)); + assertThat(kibanaRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(indexAbstraction), is(true)); + assertThat(kibanaRole.indices().allowedIndicesMatcher(TransportMultiSearchAction.TYPE.name()).test(indexAbstraction), is(true)); assertThat(kibanaRole.indices().allowedIndicesMatcher(GetAction.NAME).test(indexAbstraction), is(true)); assertThat(kibanaRole.indices().allowedIndicesMatcher(READ_CROSS_CLUSTER_NAME).test(indexAbstraction), is(false)); assertThat(kibanaRole.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(indexAbstraction), is(true)); @@ -1243,7 +1306,7 @@ public class ReservedRolesStoreTests extends ESTestCase { logger.info("index name [{}]", indexName); final IndexAbstraction indexAbstraction = mockIndexAbstraction(indexName); // Allow indexing - assertThat(kibanaRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(indexAbstraction), is(true)); + assertThat(kibanaRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(indexAbstraction), is(true)); assertThat(kibanaRole.indices().allowedIndicesMatcher(GetAction.NAME).test(indexAbstraction), is(true)); assertThat(kibanaRole.indices().allowedIndicesMatcher(IndexAction.NAME).test(indexAbstraction), is(true)); assertThat(kibanaRole.indices().allowedIndicesMatcher(UpdateAction.NAME).test(indexAbstraction), is(true)); @@ -1280,8 +1343,8 @@ public class ReservedRolesStoreTests extends ESTestCase { assertThat(kibanaRole.indices().allowedIndicesMatcher(CreateIndexAction.NAME).test(indexAbstraction), is(false)); assertThat(kibanaRole.indices().allowedIndicesMatcher(IndexAction.NAME).test(indexAbstraction), is(false)); assertThat(kibanaRole.indices().allowedIndicesMatcher(DeleteAction.NAME).test(indexAbstraction), is(false)); - assertThat(kibanaRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(indexAbstraction), is(true)); - assertThat(kibanaRole.indices().allowedIndicesMatcher(MultiSearchAction.NAME).test(indexAbstraction), is(true)); + assertThat(kibanaRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(indexAbstraction), is(true)); + assertThat(kibanaRole.indices().allowedIndicesMatcher(TransportMultiSearchAction.TYPE.name()).test(indexAbstraction), is(true)); assertThat(kibanaRole.indices().allowedIndicesMatcher(GetAction.NAME).test(indexAbstraction), is(true)); assertThat(kibanaRole.indices().allowedIndicesMatcher(READ_CROSS_CLUSTER_NAME).test(indexAbstraction), is(false)); assertThat(kibanaRole.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(indexAbstraction), is(true)); @@ -1318,7 +1381,7 @@ public class ReservedRolesStoreTests extends ESTestCase { Arrays.asList("kibana_sample_data_ecommerce", "kibana_sample_data_ecommerce_transform" + randomInt()).forEach(indexName -> { final IndexAbstraction indexAbstraction = mockIndexAbstraction(indexName); // Allow search and indexing - assertThat(kibanaRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(indexAbstraction), is(true)); + assertThat(kibanaRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(indexAbstraction), is(true)); assertThat(kibanaRole.indices().allowedIndicesMatcher(GetAction.NAME).test(indexAbstraction), is(true)); assertThat(kibanaRole.indices().allowedIndicesMatcher(IndexAction.NAME).test(indexAbstraction), is(true)); assertThat(kibanaRole.indices().allowedIndicesMatcher(UpdateAction.NAME).test(indexAbstraction), is(true)); @@ -1344,7 +1407,7 @@ public class ReservedRolesStoreTests extends ESTestCase { Arrays.asList(".asset-criticality.asset-criticality-" + randomAlphaOfLength(randomIntBetween(0, 13))).forEach(indexName -> { final IndexAbstraction indexAbstraction = mockIndexAbstraction(indexName); assertThat(kibanaRole.indices().allowedIndicesMatcher(CreateIndexAction.NAME).test(indexAbstraction), is(true)); - assertThat(kibanaRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(indexAbstraction), is(true)); + assertThat(kibanaRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(indexAbstraction), is(true)); assertViewIndexMetadata(kibanaRole, indexName); }); } @@ -1522,12 +1585,18 @@ public class ReservedRolesStoreTests extends ESTestCase { assertThat(monitoringUserRole.runAs().check(randomAlphaOfLengthBetween(1, 12)), is(false)); - assertThat(monitoringUserRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction("foo")), is(false)); assertThat( - monitoringUserRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(".reporting")), + monitoringUserRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction("foo")), + is(false) + ); + assertThat( + monitoringUserRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction(".reporting")), + is(false) + ); + assertThat( + monitoringUserRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction(".kibana")), is(false) ); - assertThat(monitoringUserRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(".kibana")), is(false)); assertThat( monitoringUserRole.indices().allowedIndicesMatcher("indices:foo").test(mockIndexAbstraction(randomAlphaOfLengthBetween(8, 24))), is(false) @@ -1556,7 +1625,10 @@ public class ReservedRolesStoreTests extends ESTestCase { monitoringUserRole.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(mockIndexAbstraction(index)), is(false) ); - assertThat(monitoringUserRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(index)), is(true)); + assertThat( + monitoringUserRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction(index)), + is(true) + ); assertThat(monitoringUserRole.indices().allowedIndicesMatcher(GetAction.NAME).test(mockIndexAbstraction(index)), is(true)); assertThat(monitoringUserRole.indices().allowedIndicesMatcher(READ_CROSS_CLUSTER_NAME).test(mockIndexAbstraction(index)), is(true)); @@ -1651,15 +1723,19 @@ public class ReservedRolesStoreTests extends ESTestCase { assertThat(remoteMonitoringAgentRole.runAs().check(randomAlphaOfLengthBetween(1, 12)), is(false)); assertThat( - remoteMonitoringAgentRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction("foo")), + remoteMonitoringAgentRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction("foo")), is(false) ); assertThat( - remoteMonitoringAgentRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(".reporting")), + remoteMonitoringAgentRole.indices() + .allowedIndicesMatcher(TransportSearchAction.TYPE.name()) + .test(mockIndexAbstraction(".reporting")), is(false) ); assertThat( - remoteMonitoringAgentRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(".kibana")), + remoteMonitoringAgentRole.indices() + .allowedIndicesMatcher(TransportSearchAction.TYPE.name()) + .test(mockIndexAbstraction(".kibana")), is(false) ); assertThat( @@ -1701,7 +1777,9 @@ public class ReservedRolesStoreTests extends ESTestCase { is(true) ); assertThat( - remoteMonitoringAgentRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(monitoringIndex)), + remoteMonitoringAgentRole.indices() + .allowedIndicesMatcher(TransportSearchAction.TYPE.name()) + .test(mockIndexAbstraction(monitoringIndex)), is(true) ); assertThat( @@ -1775,7 +1853,9 @@ public class ReservedRolesStoreTests extends ESTestCase { is(false) ); assertThat( - remoteMonitoringAgentRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(metricbeatIndex)), + remoteMonitoringAgentRole.indices() + .allowedIndicesMatcher(TransportSearchAction.TYPE.name()) + .test(mockIndexAbstraction(metricbeatIndex)), is(false) ); assertThat( @@ -1818,15 +1898,21 @@ public class ReservedRolesStoreTests extends ESTestCase { is(true) ); assertThat( - remoteMonitoringCollectorRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction("foo")), + remoteMonitoringCollectorRole.indices() + .allowedIndicesMatcher(TransportSearchAction.TYPE.name()) + .test(mockIndexAbstraction("foo")), is(false) ); assertThat( - remoteMonitoringCollectorRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(".reporting")), + remoteMonitoringCollectorRole.indices() + .allowedIndicesMatcher(TransportSearchAction.TYPE.name()) + .test(mockIndexAbstraction(".reporting")), is(false) ); assertThat( - remoteMonitoringCollectorRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(".kibana")), + remoteMonitoringCollectorRole.indices() + .allowedIndicesMatcher(TransportSearchAction.TYPE.name()) + .test(mockIndexAbstraction(".kibana")), is(true) ); assertThat( @@ -1888,7 +1974,9 @@ public class ReservedRolesStoreTests extends ESTestCase { is(false) ); assertThat( - remoteMonitoringCollectorRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(index)), + remoteMonitoringCollectorRole.indices() + .allowedIndicesMatcher(TransportSearchAction.TYPE.name()) + .test(mockIndexAbstraction(index)), is(false) ); assertThat( @@ -1966,13 +2054,13 @@ public class ReservedRolesStoreTests extends ESTestCase { assertThat( remoteMonitoringCollectorRole.indices() - .allowedIndicesMatcher(SearchAction.NAME) + .allowedIndicesMatcher(TransportSearchAction.TYPE.name()) .test(mockIndexAbstraction(randomFrom(TestRestrictedIndices.SAMPLE_RESTRICTED_NAMES))), is(false) ); assertThat( remoteMonitoringCollectorRole.indices() - .allowedIndicesMatcher(SearchAction.NAME) + .allowedIndicesMatcher(TransportSearchAction.TYPE.name()) .test(mockIndexAbstraction(XPackPlugin.ASYNC_RESULTS_INDEX + randomAlphaOfLengthBetween(0, 2))), is(false) ); @@ -2081,12 +2169,18 @@ public class ReservedRolesStoreTests extends ESTestCase { assertThat(reportingUserRole.runAs().check(randomAlphaOfLengthBetween(1, 12)), is(false)); - assertThat(reportingUserRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction("foo")), is(false)); assertThat( - reportingUserRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(".reporting")), + reportingUserRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction("foo")), + is(false) + ); + assertThat( + reportingUserRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction(".reporting")), + is(false) + ); + assertThat( + reportingUserRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction(".kibana")), is(false) ); - assertThat(reportingUserRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(".kibana")), is(false)); assertThat( reportingUserRole.indices().allowedIndicesMatcher("indices:foo").test(mockIndexAbstraction(randomAlphaOfLengthBetween(8, 24))), is(false) @@ -2101,7 +2195,10 @@ public class ReservedRolesStoreTests extends ESTestCase { reportingUserRole.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(mockIndexAbstraction(index)), is(false) ); - assertThat(reportingUserRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(index)), is(false)); + assertThat( + reportingUserRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction(index)), + is(false) + ); assertThat(reportingUserRole.indices().allowedIndicesMatcher(GetAction.NAME).test(mockIndexAbstraction(index)), is(false)); assertThat(reportingUserRole.indices().allowedIndicesMatcher(IndexAction.NAME).test(mockIndexAbstraction(index)), is(false)); assertThat(reportingUserRole.indices().allowedIndicesMatcher(UpdateAction.NAME).test(mockIndexAbstraction(index)), is(false)); @@ -2170,7 +2267,7 @@ public class ReservedRolesStoreTests extends ESTestCase { FieldPermissionsCache fieldPermissionsCache = new FieldPermissionsCache(Settings.EMPTY); SortedMap lookup = metadata.getIndicesLookup(); IndicesAccessControl iac = superuserRole.indices() - .authorize(SearchAction.NAME, Sets.newHashSet("a1", "ba"), lookup, fieldPermissionsCache); + .authorize(TransportSearchAction.TYPE.name(), Sets.newHashSet("a1", "ba"), lookup, fieldPermissionsCache); assertThat(iac.hasIndexPermissions("a1"), is(true)); assertThat(iac.hasIndexPermissions("b"), is(true)); iac = superuserRole.indices().authorize(DeleteIndexAction.NAME, Sets.newHashSet("a1", "ba"), lookup, fieldPermissionsCache); @@ -2186,7 +2283,7 @@ public class ReservedRolesStoreTests extends ESTestCase { // Read security indices => allowed iac = superuserRole.indices() .authorize( - randomFrom(SearchAction.NAME, GetIndexAction.NAME), + randomFrom(TransportSearchAction.TYPE.name(), GetIndexAction.NAME), Sets.newHashSet(TestRestrictedIndices.SECURITY_MAIN_ALIAS), lookup, fieldPermissionsCache @@ -2205,7 +2302,7 @@ public class ReservedRolesStoreTests extends ESTestCase { assertThat("For " + iac, iac.hasIndexPermissions(TestRestrictedIndices.SECURITY_MAIN_ALIAS), is(false)); assertThat("For " + iac, iac.hasIndexPermissions(internalSecurityIndex), is(false)); - assertTrue(superuserRole.indices().check(SearchAction.NAME)); + assertTrue(superuserRole.indices().check(TransportSearchAction.TYPE.name())); assertFalse(superuserRole.indices().check("unknown")); assertThat(superuserRole.runAs().check(randomAlphaOfLengthBetween(1, 30)), is(true)); @@ -2320,8 +2417,14 @@ public class ReservedRolesStoreTests extends ESTestCase { assertThat(beatsAdminRole.indices().allowedIndicesMatcher(IndexAction.NAME).test(mockIndexAbstraction(index)), is(true)); assertThat(beatsAdminRole.indices().allowedIndicesMatcher(DeleteAction.NAME).test(mockIndexAbstraction(index)), is(true)); assertThat(beatsAdminRole.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(mockIndexAbstraction(index)), is(true)); - assertThat(beatsAdminRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(index)), is(true)); - assertThat(beatsAdminRole.indices().allowedIndicesMatcher(MultiSearchAction.NAME).test(mockIndexAbstraction(index)), is(true)); + assertThat( + beatsAdminRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction(index)), + is(true) + ); + assertThat( + beatsAdminRole.indices().allowedIndicesMatcher(TransportMultiSearchAction.TYPE.name()).test(mockIndexAbstraction(index)), + is(true) + ); assertThat(beatsAdminRole.indices().allowedIndicesMatcher(GetAction.NAME).test(mockIndexAbstraction(index)), is(true)); assertNoAccessAllowed(beatsAdminRole, TestRestrictedIndices.SAMPLE_RESTRICTED_NAMES); @@ -3228,8 +3331,11 @@ public class ReservedRolesStoreTests extends ESTestCase { assertThat(role.indices().allowedIndicesMatcher(IndexAction.NAME).test(mockIndexAbstraction(index)), is(true)); assertThat(role.indices().allowedIndicesMatcher(DeleteAction.NAME).test(mockIndexAbstraction(index)), is(true)); assertThat(role.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(mockIndexAbstraction(index)), is(true)); - assertThat(role.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(index)), is(true)); - assertThat(role.indices().allowedIndicesMatcher(MultiSearchAction.NAME).test(mockIndexAbstraction(index)), is(true)); + assertThat(role.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction(index)), is(true)); + assertThat( + role.indices().allowedIndicesMatcher(TransportMultiSearchAction.TYPE.name()).test(mockIndexAbstraction(index)), + is(true) + ); assertThat(role.indices().allowedIndicesMatcher(GetAction.NAME).test(mockIndexAbstraction(index)), is(true)); // inherits from 'all' assertThat(role.indices().allowedIndicesMatcher(READ_CROSS_CLUSTER_NAME).test(mockIndexAbstraction(index)), is(true)); @@ -3237,7 +3343,7 @@ public class ReservedRolesStoreTests extends ESTestCase { private void assertReadWriteDocsAndMaintenanceButNotDeleteIndexAllowed(Role role, String index) { assertThat(role.indices().allowedIndicesMatcher(DeleteIndexAction.NAME).test(mockIndexAbstraction(index)), is(false)); - assertThat(role.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(index)), is(true)); + assertThat(role.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction(index)), is(true)); assertThat(role.indices().allowedIndicesMatcher(GetAction.NAME).test(mockIndexAbstraction(index)), is(true)); assertThat(role.indices().allowedIndicesMatcher(IndexAction.NAME).test(mockIndexAbstraction(index)), is(true)); assertThat(role.indices().allowedIndicesMatcher(UpdateAction.NAME).test(mockIndexAbstraction(index)), is(true)); @@ -3251,7 +3357,7 @@ public class ReservedRolesStoreTests extends ESTestCase { private void assertReadWriteDocsButNotDeleteIndexAllowed(Role role, String index) { assertThat(role.indices().allowedIndicesMatcher(DeleteIndexAction.NAME).test(mockIndexAbstraction(index)), is(false)); - assertThat(role.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(index)), is(true)); + assertThat(role.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction(index)), is(true)); assertThat(role.indices().allowedIndicesMatcher(GetAction.NAME).test(mockIndexAbstraction(index)), is(true)); assertThat(role.indices().allowedIndicesMatcher(IndexAction.NAME).test(mockIndexAbstraction(index)), is(true)); assertThat(role.indices().allowedIndicesMatcher(UpdateAction.NAME).test(mockIndexAbstraction(index)), is(true)); @@ -3263,7 +3369,7 @@ public class ReservedRolesStoreTests extends ESTestCase { assertThat(role.indices().allowedIndicesMatcher(DeleteIndexAction.NAME).test(mockIndexAbstraction(index)), is(false)); assertThat(role.indices().allowedIndicesMatcher(CreateIndexAction.NAME).test(mockIndexAbstraction(index)), is(false)); assertThat(role.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(mockIndexAbstraction(index)), is(false)); - assertThat(role.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(index)), is(true)); + assertThat(role.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction(index)), is(true)); assertThat(role.indices().allowedIndicesMatcher(GetAction.NAME).test(mockIndexAbstraction(index)), is(true)); assertThat(role.indices().allowedIndicesMatcher(IndexAction.NAME).test(mockIndexAbstraction(index)), is(false)); assertThat(role.indices().allowedIndicesMatcher(UpdateAction.NAME).test(mockIndexAbstraction(index)), is(false)); @@ -3281,7 +3387,7 @@ public class ReservedRolesStoreTests extends ESTestCase { GetFieldMappingsAction.NAME + "*", GetMappingsAction.NAME, ClusterSearchShardsAction.NAME, - SearchShardsAction.NAME, + TransportSearchShardsAction.TYPE.name(), ValidateQueryAction.NAME + "*", GetSettingsAction.NAME, ExplainLifecycleAction.NAME, @@ -3302,7 +3408,7 @@ public class ReservedRolesStoreTests extends ESTestCase { assertThat(role.indices().allowedIndicesMatcher(DeleteIndexAction.NAME).test(mockIndexAbstraction(index)), is(false)); assertThat(role.indices().allowedIndicesMatcher(CreateIndexAction.NAME).test(mockIndexAbstraction(index)), is(false)); assertThat(role.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(mockIndexAbstraction(index)), is(false)); - assertThat(role.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(index)), is(false)); + assertThat(role.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction(index)), is(false)); assertThat(role.indices().allowedIndicesMatcher(GetAction.NAME).test(mockIndexAbstraction(index)), is(false)); assertThat(role.indices().allowedIndicesMatcher(IndexAction.NAME).test(mockIndexAbstraction(index)), is(false)); assertThat(role.indices().allowedIndicesMatcher(UpdateAction.NAME).test(mockIndexAbstraction(index)), is(false)); @@ -3350,8 +3456,14 @@ public class ReservedRolesStoreTests extends ESTestCase { assertThat(logstashAdminRole.indices().allowedIndicesMatcher(CreateIndexAction.NAME).test(mockIndexAbstraction(index)), is(true)); assertThat(logstashAdminRole.indices().allowedIndicesMatcher(IndexAction.NAME).test(mockIndexAbstraction(index)), is(true)); assertThat(logstashAdminRole.indices().allowedIndicesMatcher(GetAction.NAME).test(mockIndexAbstraction(index)), is(true)); - assertThat(logstashAdminRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(index)), is(true)); - assertThat(logstashAdminRole.indices().allowedIndicesMatcher(MultiSearchAction.NAME).test(mockIndexAbstraction(index)), is(true)); + assertThat( + logstashAdminRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction(index)), + is(true) + ); + assertThat( + logstashAdminRole.indices().allowedIndicesMatcher(TransportMultiSearchAction.TYPE.name()).test(mockIndexAbstraction(index)), + is(true) + ); assertThat( logstashAdminRole.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(mockIndexAbstraction(index)), is(true) diff --git a/x-pack/plugin/ent-search/src/main/java/org/elasticsearch/xpack/application/search/action/TransportQuerySearchApplicationAction.java b/x-pack/plugin/ent-search/src/main/java/org/elasticsearch/xpack/application/search/action/TransportQuerySearchApplicationAction.java index ac4b15fd4356..89e670f9b1de 100644 --- a/x-pack/plugin/ent-search/src/main/java/org/elasticsearch/xpack/application/search/action/TransportQuerySearchApplicationAction.java +++ b/x-pack/plugin/ent-search/src/main/java/org/elasticsearch/xpack/application/search/action/TransportQuerySearchApplicationAction.java @@ -8,9 +8,9 @@ package org.elasticsearch.xpack.application.search.action; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.HandledTransportAction; import org.elasticsearch.client.internal.Client; @@ -68,7 +68,7 @@ public class TransportQuerySearchApplicationAction extends HandledTransportActio SearchRequest searchRequest = new SearchRequest(searchApplication.name()).source(sourceBuilder); client.execute( - SearchAction.INSTANCE, + TransportSearchAction.TYPE, searchRequest, listener.delegateFailure((l2, searchResponse) -> l2.onResponse(searchResponse)) ); diff --git a/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/execution/search/PITAwareQueryClient.java b/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/execution/search/PITAwareQueryClient.java index b5ac09cc39b9..707964a93ab9 100644 --- a/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/execution/search/PITAwareQueryClient.java +++ b/x-pack/plugin/eql/src/main/java/org/elasticsearch/xpack/eql/execution/search/PITAwareQueryClient.java @@ -8,15 +8,15 @@ package org.elasticsearch.xpack.eql.execution.search; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.search.ClosePointInTimeAction; import org.elasticsearch.action.search.ClosePointInTimeRequest; import org.elasticsearch.action.search.ClosePointInTimeResponse; import org.elasticsearch.action.search.MultiSearchRequest; import org.elasticsearch.action.search.MultiSearchResponse; -import org.elasticsearch.action.search.OpenPointInTimeAction; import org.elasticsearch.action.search.OpenPointInTimeRequest; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.action.search.TransportClosePointInTimeAction; +import org.elasticsearch.action.search.TransportOpenPointInTimeAction; import org.elasticsearch.common.Strings; import org.elasticsearch.common.util.CollectionUtils; import org.elasticsearch.core.TimeValue; @@ -131,7 +131,7 @@ public class PITAwareQueryClient extends BasicQueryClient { private void openPIT(ActionListener listener, Runnable runnable) { OpenPointInTimeRequest request = new OpenPointInTimeRequest(indices).indicesOptions(IndexResolver.FIELD_CAPS_INDICES_OPTIONS) .keepAlive(keepAlive); - client.execute(OpenPointInTimeAction.INSTANCE, request, listener.delegateFailureAndWrap((l, r) -> { + client.execute(TransportOpenPointInTimeAction.TYPE, request, listener.delegateFailureAndWrap((l, r) -> { pitId = r.getPointInTimeId(); runnable.run(); })); @@ -142,7 +142,7 @@ public class PITAwareQueryClient extends BasicQueryClient { // the pitId could be null as a consequence of a failure on openPIT if (pitId != null) { client.execute( - ClosePointInTimeAction.INSTANCE, + TransportClosePointInTimeAction.TYPE, new ClosePointInTimeRequest(pitId), map(listener, ClosePointInTimeResponse::isSucceeded) ); diff --git a/x-pack/plugin/eql/src/test/java/org/elasticsearch/xpack/eql/analysis/CancellationTests.java b/x-pack/plugin/eql/src/test/java/org/elasticsearch/xpack/eql/analysis/CancellationTests.java index eba52ff91c2a..f6369be44283 100644 --- a/x-pack/plugin/eql/src/test/java/org/elasticsearch/xpack/eql/analysis/CancellationTests.java +++ b/x-pack/plugin/eql/src/test/java/org/elasticsearch/xpack/eql/analysis/CancellationTests.java @@ -10,10 +10,10 @@ import org.elasticsearch.TransportVersion; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.fieldcaps.FieldCapabilities; import org.elasticsearch.action.fieldcaps.FieldCapabilitiesResponse; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchRequestBuilder; import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.client.internal.Client; import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.node.DiscoveryNode; @@ -211,7 +211,7 @@ public class CancellationTests extends ESTestCase { // Emulation of search cancellation ArgumentCaptor searchRequestCaptor = ArgumentCaptor.forClass(SearchRequest.class); - when(client.prepareSearch(any())).thenReturn(new SearchRequestBuilder(client, SearchAction.INSTANCE).setIndices(indices)); + when(client.prepareSearch(any())).thenReturn(new SearchRequestBuilder(client, TransportSearchAction.TYPE).setIndices(indices)); doAnswer((Answer) invocation -> { @SuppressWarnings("unchecked") SearchRequest request = (SearchRequest) invocation.getArguments()[1]; diff --git a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/ComputeService.java b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/ComputeService.java index dabce913279e..23cfe522c782 100644 --- a/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/ComputeService.java +++ b/x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/plugin/ComputeService.java @@ -10,10 +10,10 @@ package org.elasticsearch.xpack.esql.plugin; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListenerResponseHandler; import org.elasticsearch.action.search.SearchRequest; -import org.elasticsearch.action.search.SearchShardsAction; import org.elasticsearch.action.search.SearchShardsGroup; import org.elasticsearch.action.search.SearchShardsRequest; import org.elasticsearch.action.search.SearchShardsResponse; +import org.elasticsearch.action.search.TransportSearchShardsAction; import org.elasticsearch.action.support.ContextPreservingActionListener; import org.elasticsearch.action.support.RefCountingListener; import org.elasticsearch.action.support.RefCountingRunnable; @@ -393,7 +393,7 @@ public class ComputeService { ); transportService.sendChildRequest( transportService.getLocalNode(), - SearchShardsAction.NAME, + TransportSearchShardsAction.TYPE.name(), searchShardsRequest, parentTask, TransportRequestOptions.EMPTY, diff --git a/x-pack/plugin/fleet/src/main/java/org/elasticsearch/xpack/fleet/rest/RestFleetMultiSearchAction.java b/x-pack/plugin/fleet/src/main/java/org/elasticsearch/xpack/fleet/rest/RestFleetMultiSearchAction.java index 308af9b89e07..29636436af57 100644 --- a/x-pack/plugin/fleet/src/main/java/org/elasticsearch/xpack/fleet/rest/RestFleetMultiSearchAction.java +++ b/x-pack/plugin/fleet/src/main/java/org/elasticsearch/xpack/fleet/rest/RestFleetMultiSearchAction.java @@ -7,9 +7,9 @@ package org.elasticsearch.xpack.fleet.rest; -import org.elasticsearch.action.search.MultiSearchAction; import org.elasticsearch.action.search.MultiSearchRequest; import org.elasticsearch.action.search.SearchRequest; +import org.elasticsearch.action.search.TransportMultiSearchAction; import org.elasticsearch.client.internal.node.NodeClient; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.core.TimeValue; @@ -112,7 +112,7 @@ public class RestFleetMultiSearchAction extends BaseRestHandler { return channel -> { final RestCancellableNodeClient cancellableClient = new RestCancellableNodeClient(client, request.getHttpChannel()); - cancellableClient.execute(MultiSearchAction.INSTANCE, multiSearchRequest, new RestChunkedToXContentListener<>(channel)); + cancellableClient.execute(TransportMultiSearchAction.TYPE, multiSearchRequest, new RestChunkedToXContentListener<>(channel)); }; } diff --git a/x-pack/plugin/fleet/src/main/java/org/elasticsearch/xpack/fleet/rest/RestFleetSearchAction.java b/x-pack/plugin/fleet/src/main/java/org/elasticsearch/xpack/fleet/rest/RestFleetSearchAction.java index 178db0229ca5..3a09fe1d1838 100644 --- a/x-pack/plugin/fleet/src/main/java/org/elasticsearch/xpack/fleet/rest/RestFleetSearchAction.java +++ b/x-pack/plugin/fleet/src/main/java/org/elasticsearch/xpack/fleet/rest/RestFleetSearchAction.java @@ -8,8 +8,8 @@ package org.elasticsearch.xpack.fleet.rest; import org.elasticsearch.Version; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchRequest; +import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.client.internal.node.NodeClient; import org.elasticsearch.common.Strings; import org.elasticsearch.core.TimeValue; @@ -96,7 +96,7 @@ public class RestFleetSearchAction extends BaseRestHandler { return channel -> { RestCancellableNodeClient cancelClient = new RestCancellableNodeClient(client, request.getHttpChannel()); - cancelClient.execute(SearchAction.INSTANCE, searchRequest, new RestChunkedToXContentListener<>(channel)); + cancelClient.execute(TransportSearchAction.TYPE, searchRequest, new RestChunkedToXContentListener<>(channel)); }; } diff --git a/x-pack/plugin/frozen-indices/src/internalClusterTest/java/org/elasticsearch/index/engine/frozen/FrozenIndexIT.java b/x-pack/plugin/frozen-indices/src/internalClusterTest/java/org/elasticsearch/index/engine/frozen/FrozenIndexIT.java index 7f2035bef643..5f219bd8ce59 100644 --- a/x-pack/plugin/frozen-indices/src/internalClusterTest/java/org/elasticsearch/index/engine/frozen/FrozenIndexIT.java +++ b/x-pack/plugin/frozen-indices/src/internalClusterTest/java/org/elasticsearch/index/engine/frozen/FrozenIndexIT.java @@ -9,11 +9,11 @@ package org.elasticsearch.index.engine.frozen; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.action.DocWriteResponse; -import org.elasticsearch.action.search.ClosePointInTimeAction; import org.elasticsearch.action.search.ClosePointInTimeRequest; -import org.elasticsearch.action.search.OpenPointInTimeAction; import org.elasticsearch.action.search.OpenPointInTimeRequest; import org.elasticsearch.action.search.SearchType; +import org.elasticsearch.action.search.TransportClosePointInTimeAction; +import org.elasticsearch.action.search.TransportOpenPointInTimeAction; import org.elasticsearch.action.support.ActiveShardCount; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.support.PlainActionFuture; @@ -230,7 +230,7 @@ public class FrozenIndexIT extends ESIntegTestCase { final OpenPointInTimeRequest openPointInTimeRequest = new OpenPointInTimeRequest(indexName).indicesOptions( IndicesOptions.STRICT_EXPAND_OPEN_FORBID_CLOSED ).keepAlive(TimeValue.timeValueMinutes(2)); - final String pitId = client().execute(OpenPointInTimeAction.INSTANCE, openPointInTimeRequest).actionGet().getPointInTimeId(); + final String pitId = client().execute(TransportOpenPointInTimeAction.TYPE, openPointInTimeRequest).actionGet().getPointInTimeId(); try { assertNoFailuresAndResponse( prepareSearch().setIndices(indexName).setPreference(null).setPointInTime(new PointInTimeBuilder(pitId)), @@ -257,7 +257,7 @@ public class FrozenIndexIT extends ESIntegTestCase { ); } finally { assertAcked(client().execute(FreezeIndexAction.INSTANCE, new FreezeRequest(indexName).setFreeze(false)).actionGet()); - client().execute(ClosePointInTimeAction.INSTANCE, new ClosePointInTimeRequest(pitId)).actionGet(); + client().execute(TransportClosePointInTimeAction.TYPE, new ClosePointInTimeRequest(pitId)).actionGet(); } } @@ -282,7 +282,7 @@ public class FrozenIndexIT extends ESIntegTestCase { IndicesOptions.STRICT_EXPAND_OPEN_FORBID_CLOSED ).keepAlive(TimeValue.timeValueMinutes(2)); - final String pitId = client().execute(OpenPointInTimeAction.INSTANCE, openPointInTimeRequest).actionGet().getPointInTimeId(); + final String pitId = client().execute(TransportOpenPointInTimeAction.TYPE, openPointInTimeRequest).actionGet().getPointInTimeId(); try { indicesAdmin().prepareDelete("index-1").get(); // Return partial results if allow partial search result is allowed @@ -299,7 +299,7 @@ public class FrozenIndexIT extends ESIntegTestCase { prepareSearch().setPreference(null).setAllowPartialSearchResults(false).setPointInTime(new PointInTimeBuilder(pitId))::get ); } finally { - client().execute(ClosePointInTimeAction.INSTANCE, new ClosePointInTimeRequest(pitId)).actionGet(); + client().execute(TransportClosePointInTimeAction.TYPE, new ClosePointInTimeRequest(pitId)).actionGet(); } } @@ -317,14 +317,16 @@ public class FrozenIndexIT extends ESIntegTestCase { final OpenPointInTimeRequest openPointInTimeRequest = new OpenPointInTimeRequest("test-*").indicesOptions( IndicesOptions.strictExpandOpenAndForbidClosed() ).keepAlive(TimeValue.timeValueMinutes(2)); - final String pitId = client().execute(OpenPointInTimeAction.INSTANCE, openPointInTimeRequest).actionGet().getPointInTimeId(); + final String pitId = client().execute(TransportOpenPointInTimeAction.TYPE, openPointInTimeRequest) + .actionGet() + .getPointInTimeId(); try { assertNoFailuresAndResponse( prepareSearch().setPreference(null).setPointInTime(new PointInTimeBuilder(pitId)), searchResponse -> assertHitCount(searchResponse, numDocs) ); } finally { - client().execute(ClosePointInTimeAction.INSTANCE, new ClosePointInTimeRequest(pitId)).actionGet(); + client().execute(TransportClosePointInTimeAction.TYPE, new ClosePointInTimeRequest(pitId)).actionGet(); } } // exclude the frozen indices @@ -332,11 +334,13 @@ public class FrozenIndexIT extends ESIntegTestCase { final OpenPointInTimeRequest openPointInTimeRequest = new OpenPointInTimeRequest("test-*").keepAlive( TimeValue.timeValueMinutes(2) ); - final String pitId = client().execute(OpenPointInTimeAction.INSTANCE, openPointInTimeRequest).actionGet().getPointInTimeId(); + final String pitId = client().execute(TransportOpenPointInTimeAction.TYPE, openPointInTimeRequest) + .actionGet() + .getPointInTimeId(); try { assertHitCountAndNoFailures(prepareSearch().setPreference(null).setPointInTime(new PointInTimeBuilder(pitId)), 0); } finally { - client().execute(ClosePointInTimeAction.INSTANCE, new ClosePointInTimeRequest(pitId)).actionGet(); + client().execute(TransportClosePointInTimeAction.TYPE, new ClosePointInTimeRequest(pitId)).actionGet(); } } } diff --git a/x-pack/plugin/frozen-indices/src/internalClusterTest/java/org/elasticsearch/index/engine/frozen/FrozenIndexTests.java b/x-pack/plugin/frozen-indices/src/internalClusterTest/java/org/elasticsearch/index/engine/frozen/FrozenIndexTests.java index 896c0050cd5c..faeaee2da930 100644 --- a/x-pack/plugin/frozen-indices/src/internalClusterTest/java/org/elasticsearch/index/engine/frozen/FrozenIndexTests.java +++ b/x-pack/plugin/frozen-indices/src/internalClusterTest/java/org/elasticsearch/index/engine/frozen/FrozenIndexTests.java @@ -12,14 +12,14 @@ import org.elasticsearch.action.OriginalIndices; import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse; import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse; import org.elasticsearch.action.delete.DeleteResponse; -import org.elasticsearch.action.search.ClosePointInTimeAction; import org.elasticsearch.action.search.ClosePointInTimeRequest; -import org.elasticsearch.action.search.OpenPointInTimeAction; import org.elasticsearch.action.search.OpenPointInTimeRequest; import org.elasticsearch.action.search.OpenPointInTimeResponse; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchType; +import org.elasticsearch.action.search.TransportClosePointInTimeAction; +import org.elasticsearch.action.search.TransportOpenPointInTimeAction; import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.support.PlainActionFuture; @@ -86,7 +86,7 @@ public class FrozenIndexTests extends ESSingleNodeTestCase { String openReaders(TimeValue keepAlive, String... indices) { OpenPointInTimeRequest request = new OpenPointInTimeRequest(indices).indicesOptions(IndicesOptions.STRICT_EXPAND_OPEN_FORBID_CLOSED) .keepAlive(keepAlive); - final OpenPointInTimeResponse response = client().execute(OpenPointInTimeAction.INSTANCE, request).actionGet(); + final OpenPointInTimeResponse response = client().execute(TransportOpenPointInTimeAction.TYPE, request).actionGet(); return response.getPointInTimeId(); } @@ -161,7 +161,7 @@ public class FrozenIndexTests extends ESSingleNodeTestCase { } assertWarnings(TransportSearchAction.FROZEN_INDICES_DEPRECATION_MESSAGE.replace("{}", indexName)); } finally { - client().execute(ClosePointInTimeAction.INSTANCE, new ClosePointInTimeRequest(pitId)).get(); + client().execute(TransportClosePointInTimeAction.TYPE, new ClosePointInTimeRequest(pitId)).get(); } } diff --git a/x-pack/plugin/ml/qa/native-multi-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/ml/integration/MlNativeIntegTestCase.java b/x-pack/plugin/ml/qa/native-multi-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/ml/integration/MlNativeIntegTestCase.java index 276f60a28ccb..209f12c2e90c 100644 --- a/x-pack/plugin/ml/qa/native-multi-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/ml/integration/MlNativeIntegTestCase.java +++ b/x-pack/plugin/ml/qa/native-multi-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/ml/integration/MlNativeIntegTestCase.java @@ -13,9 +13,9 @@ import org.elasticsearch.action.admin.indices.refresh.RefreshRequest; import org.elasticsearch.action.admin.indices.refresh.RefreshResponse; import org.elasticsearch.action.admin.indices.template.put.PutComposableIndexTemplateAction; import org.elasticsearch.action.datastreams.CreateDataStreamAction; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchRequestBuilder; +import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.action.support.master.AcknowledgedResponse; import org.elasticsearch.client.internal.Client; import org.elasticsearch.cluster.ClusterModule; @@ -311,12 +311,12 @@ abstract class MlNativeIntegTestCase extends ESIntegTestCase { RefreshResponse refreshResponse = client().execute(RefreshAction.INSTANCE, refreshRequest).actionGet(); assertThat(refreshResponse.getStatus().getStatus(), anyOf(equalTo(200), equalTo(201))); - SearchRequest searchRequest = new SearchRequestBuilder(client(), SearchAction.INSTANCE).setIndices( + SearchRequest searchRequest = new SearchRequestBuilder(client(), TransportSearchAction.TYPE).setIndices( NotificationsIndex.NOTIFICATIONS_INDEX ).addSort("timestamp", SortOrder.ASC).setQuery(QueryBuilders.termQuery("job_id", jobId)).setSize(100).request(); List messages = new ArrayList<>(); assertResponse( - client().execute(SearchAction.INSTANCE, searchRequest), + client().execute(TransportSearchAction.TYPE, searchRequest), searchResponse -> Arrays.stream(searchResponse.getHits().getHits()) .map(hit -> (String) hit.getSourceAsMap().get("message")) .forEach(messages::add) diff --git a/x-pack/plugin/ml/qa/native-multi-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/ml/integration/ModelSnapshotRetentionIT.java b/x-pack/plugin/ml/qa/native-multi-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/ml/integration/ModelSnapshotRetentionIT.java index bc4d9e82ef83..11ab23bf665b 100644 --- a/x-pack/plugin/ml/qa/native-multi-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/ml/integration/ModelSnapshotRetentionIT.java +++ b/x-pack/plugin/ml/qa/native-multi-node-tests/src/javaRestTest/java/org/elasticsearch/xpack/ml/integration/ModelSnapshotRetentionIT.java @@ -13,8 +13,8 @@ import org.elasticsearch.action.bulk.BulkRequest; import org.elasticsearch.action.bulk.BulkResponse; import org.elasticsearch.action.index.IndexAction; import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchRequest; +import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.action.support.WriteRequest; import org.elasticsearch.action.support.master.MasterNodeRequest; @@ -190,7 +190,7 @@ public class ModelSnapshotRetentionIT extends MlNativeAutodetectIntegTestCase { private List getDocIdsFromSearch(SearchRequest searchRequest) throws Exception { List docIds = new ArrayList<>(); - assertResponse(client().execute(SearchAction.INSTANCE, searchRequest), searchResponse -> { + assertResponse(client().execute(TransportSearchAction.TYPE, searchRequest), searchResponse -> { assertThat(searchResponse.getHits(), notNullValue()); for (SearchHit searchHit : searchResponse.getHits().getHits()) { docIds.add(searchHit.getId()); diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportDeleteForecastAction.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportDeleteForecastAction.java index 11ad47c00ebd..5aa85a6331c2 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportDeleteForecastAction.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportDeleteForecastAction.java @@ -13,9 +13,9 @@ import org.elasticsearch.ElasticsearchStatusException; import org.elasticsearch.ResourceNotFoundException; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.bulk.BulkItemResponse; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.HandledTransportAction; import org.elasticsearch.action.support.master.AcknowledgedResponse; @@ -116,7 +116,7 @@ public class TransportDeleteForecastAction extends HandledTransportAction extractForecastIds(SearchHit[] forecastsToDelete, JobState jobState, String jobId) { diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportEvaluateDataFrameAction.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportEvaluateDataFrameAction.java index 8fb4be69b676..4336489ce5d2 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportEvaluateDataFrameAction.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportEvaluateDataFrameAction.java @@ -7,8 +7,8 @@ package org.elasticsearch.xpack.ml.action; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchRequest; +import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.HandledTransportAction; import org.elasticsearch.client.internal.Client; @@ -146,7 +146,7 @@ public class TransportEvaluateDataFrameAction extends HandledTransportAction< SearchRequest searchRequest = new SearchRequest(request.getIndices()).source(searchSourceBuilder); useSecondaryAuthIfAvailable( securityContext, - () -> client.execute(SearchAction.INSTANCE, searchRequest, ActionListener.wrap(searchResponse -> { + () -> client.execute(TransportSearchAction.TYPE, searchRequest, ActionListener.wrap(searchResponse -> { evaluation.process(searchResponse); if (evaluation.hasAllResults() == false) { add(nextTask()); diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportGetDataFrameAnalyticsStatsAction.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportGetDataFrameAnalyticsStatsAction.java index c0c4a5cb5b3b..38da82124e77 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportGetDataFrameAnalyticsStatsAction.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportGetDataFrameAnalyticsStatsAction.java @@ -11,10 +11,10 @@ import org.apache.logging.log4j.Logger; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.FailedNodeException; import org.elasticsearch.action.TaskOperationFailure; -import org.elasticsearch.action.search.MultiSearchAction; import org.elasticsearch.action.search.MultiSearchRequest; import org.elasticsearch.action.search.MultiSearchResponse; import org.elasticsearch.action.search.SearchRequest; +import org.elasticsearch.action.search.TransportMultiSearchAction; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.support.tasks.TransportTasksAction; @@ -267,7 +267,7 @@ public class TransportGetDataFrameAnalyticsStatsAction extends TransportTasksAct executeAsyncWithOrigin( client, ML_ORIGIN, - MultiSearchAction.INSTANCE, + TransportMultiSearchAction.TYPE, multiSearchRequest, ActionListener.wrap(multiSearchResponse -> { MultiSearchResponse.Item[] itemResponses = multiSearchResponse.getResponses(); diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportGetTrainedModelsStatsAction.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportGetTrainedModelsStatsAction.java index 1139d383980d..3c9ba3700dc8 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportGetTrainedModelsStatsAction.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportGetTrainedModelsStatsAction.java @@ -14,8 +14,8 @@ import org.elasticsearch.action.admin.cluster.node.stats.NodesStatsRequest; import org.elasticsearch.action.admin.cluster.node.stats.NodesStatsRequestParameters; import org.elasticsearch.action.admin.cluster.node.stats.NodesStatsResponse; import org.elasticsearch.action.admin.cluster.node.stats.TransportNodesStatsAction; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchRequest; +import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.HandledTransportAction; import org.elasticsearch.client.internal.Client; @@ -349,7 +349,7 @@ public class TransportGetTrainedModelsStatsAction extends HandledTransportAction .request(); searchRequest.setParentTask(parentTaskId); - executeAsyncWithOrigin(client, ML_ORIGIN, SearchAction.INSTANCE, searchRequest, ActionListener.wrap(searchResponse -> { + executeAsyncWithOrigin(client, ML_ORIGIN, TransportSearchAction.TYPE, searchRequest, ActionListener.wrap(searchResponse -> { Map totalDefinitionLengthByModelId = new HashMap<>(); for (SearchHit hit : searchResponse.getHits().getHits()) { DocumentField modelIdField = hit.field(TrainedModelConfig.MODEL_ID.getPreferredName()); diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportStartDataFrameAnalyticsAction.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportStartDataFrameAnalyticsAction.java index 525ee7c4aa21..5351023a803e 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportStartDataFrameAnalyticsAction.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportStartDataFrameAnalyticsAction.java @@ -11,8 +11,8 @@ import org.apache.logging.log4j.Logger; import org.elasticsearch.ElasticsearchStatusException; import org.elasticsearch.ResourceAlreadyExistsException; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchRequest; +import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.SubscribableListener; import org.elasticsearch.action.support.master.TransportMasterNodeAction; @@ -429,7 +429,7 @@ public class TransportStartDataFrameAnalyticsAction extends TransportMasterNodeA startContext.config.getHeaders(), ClientHelper.ML_ORIGIN, parentTaskClient, - SearchAction.INSTANCE, + TransportSearchAction.TYPE, destEmptySearch, ActionListener.wrap(searchResponse -> { if (searchResponse.getHits().getTotalHits().value > 0) { diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/datafeed/DatafeedManager.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/datafeed/DatafeedManager.java index bdc143c7dde4..999d85b6dd54 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/datafeed/DatafeedManager.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/datafeed/DatafeedManager.java @@ -10,7 +10,7 @@ package org.elasticsearch.xpack.ml.datafeed; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.search.SearchAction; +import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.action.support.master.AcknowledgedResponse; import org.elasticsearch.client.internal.Client; import org.elasticsearch.cluster.ClusterState; @@ -130,9 +130,9 @@ public final class DatafeedManager { ActionListener getRollupIndexCapsActionHandler = ActionListener.wrap(response -> { if (response.getJobs().isEmpty()) { // This means no rollup indexes are in the config - indicesPrivilegesBuilder.privileges(SearchAction.NAME); + indicesPrivilegesBuilder.privileges(TransportSearchAction.TYPE.name()); } else { - indicesPrivilegesBuilder.privileges(SearchAction.NAME, RollupSearchAction.NAME); + indicesPrivilegesBuilder.privileges(TransportSearchAction.TYPE.name(), RollupSearchAction.NAME); } if (indices.length == 0) { privResponseListener.onResponse(new HasPrivilegesResponse()); @@ -142,7 +142,7 @@ public final class DatafeedManager { } }, e -> { if (ExceptionsHelper.unwrapCause(e) instanceof IndexNotFoundException) { - indicesPrivilegesBuilder.privileges(SearchAction.NAME); + indicesPrivilegesBuilder.privileges(TransportSearchAction.TYPE.name()); privRequest.indexPrivileges(indicesPrivilegesBuilder.build()); client.execute(HasPrivilegesAction.INSTANCE, privRequest, privResponseListener); } else { diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/datafeed/delayeddatacheck/DatafeedDelayedDataDetector.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/datafeed/delayeddatacheck/DatafeedDelayedDataDetector.java index 321277df02db..341746a097bb 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/datafeed/delayeddatacheck/DatafeedDelayedDataDetector.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/datafeed/delayeddatacheck/DatafeedDelayedDataDetector.java @@ -6,9 +6,9 @@ */ package org.elasticsearch.xpack.ml.datafeed.delayeddatacheck; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.client.internal.Client; import org.elasticsearch.common.util.Maps; @@ -134,7 +134,7 @@ public class DatafeedDelayedDataDetector implements DelayedDataDetector { SearchRequest searchRequest = new SearchRequest(datafeedIndices).source(searchSourceBuilder).indicesOptions(indicesOptions); try (ThreadContext.StoredContext ignore = client.threadPool().getThreadContext().stashWithOrigin(ML_ORIGIN)) { - SearchResponse response = client.execute(SearchAction.INSTANCE, searchRequest).actionGet(); + SearchResponse response = client.execute(TransportSearchAction.TYPE, searchRequest).actionGet(); List buckets = ((Histogram) response.getAggregations().get(DATE_BUCKETS)).getBuckets(); Map hashMap = Maps.newMapWithExpectedSize(buckets.size()); for (Histogram.Bucket bucket : buckets) { diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/datafeed/extractor/aggregation/AggregationDataExtractor.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/datafeed/extractor/aggregation/AggregationDataExtractor.java index f1b817c7835a..42766f6ebf12 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/datafeed/extractor/aggregation/AggregationDataExtractor.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/datafeed/extractor/aggregation/AggregationDataExtractor.java @@ -6,8 +6,8 @@ */ package org.elasticsearch.xpack.ml.datafeed.extractor.aggregation; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchRequestBuilder; +import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.client.internal.Client; import org.elasticsearch.search.builder.SearchSourceBuilder; import org.elasticsearch.xpack.ml.datafeed.DatafeedTimingStatsReporter; @@ -30,7 +30,7 @@ class AggregationDataExtractor extends AbstractAggregationDataExtractor new SearchRequestBuilder(client, SearchAction.INSTANCE).setSource(searchSourceBuilder) + return (searchSourceBuilder) -> new SearchRequestBuilder(client, TransportSearchAction.TYPE).setSource(searchSourceBuilder) .setIndicesOptions(indicesOptions) .setAllowPartialSearchResults(false) .setIndices(indices); diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/datafeed/extractor/chunked/ChunkedDataExtractor.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/datafeed/extractor/chunked/ChunkedDataExtractor.java index 669370f1f017..03dda0a8201b 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/datafeed/extractor/chunked/ChunkedDataExtractor.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/datafeed/extractor/chunked/ChunkedDataExtractor.java @@ -9,10 +9,10 @@ package org.elasticsearch.xpack.ml.datafeed.extractor.chunked; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.elasticsearch.action.ActionRequestBuilder; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchRequestBuilder; import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.client.internal.Client; import org.elasticsearch.search.aggregations.AggregationBuilders; import org.elasticsearch.search.aggregations.Aggregations; @@ -279,7 +279,7 @@ public class ChunkedDataExtractor implements DataExtractor { } private SearchRequestBuilder rangeSearchRequest() { - return new SearchRequestBuilder(client, SearchAction.INSTANCE).setIndices(context.indices) + return new SearchRequestBuilder(client, TransportSearchAction.TYPE).setIndices(context.indices) .setIndicesOptions(context.indicesOptions) .setSource(rangeSearchBuilder()) .setAllowPartialSearchResults(false) diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/datafeed/extractor/scroll/ScrollDataExtractor.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/datafeed/extractor/scroll/ScrollDataExtractor.java index f633a69bf807..8757e1afd812 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/datafeed/extractor/scroll/ScrollDataExtractor.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/datafeed/extractor/scroll/ScrollDataExtractor.java @@ -9,14 +9,14 @@ package org.elasticsearch.xpack.ml.datafeed.extractor.scroll; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.elasticsearch.ResourceNotFoundException; -import org.elasticsearch.action.search.ClearScrollAction; import org.elasticsearch.action.search.ClearScrollRequest; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchPhaseExecutionException; import org.elasticsearch.action.search.SearchRequestBuilder; import org.elasticsearch.action.search.SearchResponse; -import org.elasticsearch.action.search.SearchScrollAction; import org.elasticsearch.action.search.SearchScrollRequestBuilder; +import org.elasticsearch.action.search.TransportClearScrollAction; +import org.elasticsearch.action.search.TransportSearchAction; +import org.elasticsearch.action.search.TransportSearchScrollAction; import org.elasticsearch.client.internal.Client; import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.core.TimeValue; @@ -154,7 +154,7 @@ class ScrollDataExtractor implements DataExtractor { .query(ExtractorUtils.wrapInTimeRangeQuery(context.query, context.extractedFields.timeField(), start, context.end)) .runtimeMappings(context.runtimeMappings); - SearchRequestBuilder searchRequestBuilder = new SearchRequestBuilder(client, SearchAction.INSTANCE).setScroll(SCROLL_TIMEOUT) + SearchRequestBuilder searchRequestBuilder = new SearchRequestBuilder(client, TransportSearchAction.TYPE).setScroll(SCROLL_TIMEOUT) .setIndices(context.indices) .setIndicesOptions(context.indicesOptions) .setAllowPartialSearchResults(false) @@ -250,7 +250,9 @@ class ScrollDataExtractor implements DataExtractor { context.headers, ClientHelper.ML_ORIGIN, client, - () -> new SearchScrollRequestBuilder(client, SearchScrollAction.INSTANCE).setScroll(SCROLL_TIMEOUT).setScrollId(scrollId).get() + () -> new SearchScrollRequestBuilder(client, TransportSearchScrollAction.TYPE).setScroll(SCROLL_TIMEOUT) + .setScrollId(scrollId) + .get() ); try { checkForSkippedClusters(searchResponse); @@ -284,7 +286,7 @@ class ScrollDataExtractor implements DataExtractor { context.headers, ClientHelper.ML_ORIGIN, client, - () -> client.execute(ClearScrollAction.INSTANCE, request).actionGet() + () -> client.execute(TransportClearScrollAction.TYPE, request).actionGet() ); } } diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/dataframe/DataFrameAnalyticsTask.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/dataframe/DataFrameAnalyticsTask.java index 85ab838d52ec..7f50be8a663f 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/dataframe/DataFrameAnalyticsTask.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/dataframe/DataFrameAnalyticsTask.java @@ -13,9 +13,9 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.index.IndexAction; import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.action.support.WriteRequest; import org.elasticsearch.client.internal.Client; import org.elasticsearch.client.internal.ParentTaskAssigningClient; @@ -276,7 +276,7 @@ public class DataFrameAnalyticsTask extends LicensedAllocatedPersistentTask impl SearchRequest searchRequest = new SearchRequest(AnomalyDetectorsIndex.jobStateIndexPattern()).source( new SearchSourceBuilder().size(1).query(new IdsQueryBuilder().addIds(progressDocId)) ); - executeAsyncWithOrigin(clientToUse, ML_ORIGIN, SearchAction.INSTANCE, searchRequest, searchFormerProgressDocListener); + executeAsyncWithOrigin(clientToUse, ML_ORIGIN, TransportSearchAction.TYPE, searchRequest, searchFormerProgressDocListener); }, e -> { LOGGER.error( () -> format("[%s] cannot persist progress as an error occurred while updating task progress", taskParams.getId()), diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/dataframe/extractor/DataFrameDataExtractor.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/dataframe/extractor/DataFrameDataExtractor.java index f93988681f60..7ea6fdf55ca0 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/dataframe/extractor/DataFrameDataExtractor.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/dataframe/extractor/DataFrameDataExtractor.java @@ -9,9 +9,9 @@ package org.elasticsearch.xpack.ml.dataframe.extractor; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchRequestBuilder; import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.client.internal.Client; import org.elasticsearch.common.util.CachedSupplier; import org.elasticsearch.core.Nullable; @@ -127,7 +127,7 @@ public class DataFrameDataExtractor { */ public void preview(ActionListener> listener) { - SearchRequestBuilder searchRequestBuilder = new SearchRequestBuilder(client, SearchAction.INSTANCE) + SearchRequestBuilder searchRequestBuilder = new SearchRequestBuilder(client, TransportSearchAction.TYPE) // This ensures the search throws if there are failures and the scroll context gets cleared automatically .setAllowPartialSearchResults(false) .setIndices(context.indices) @@ -146,7 +146,7 @@ public class DataFrameDataExtractor { context.headers, ClientHelper.ML_ORIGIN, client, - SearchAction.INSTANCE, + TransportSearchAction.TYPE, searchRequestBuilder.request(), ActionListener.wrap(searchResponse -> { if (searchResponse.getHits().getHits().length == 0) { @@ -203,7 +203,7 @@ public class DataFrameDataExtractor { LOGGER.trace(() -> format("[%s] Searching docs with [%s] in [%s, %s)", context.jobId, INCREMENTAL_ID, from, to)); - SearchRequestBuilder searchRequestBuilder = new SearchRequestBuilder(client, SearchAction.INSTANCE) + SearchRequestBuilder searchRequestBuilder = new SearchRequestBuilder(client, TransportSearchAction.TYPE) // This ensures the search throws if there are failures and the scroll context gets cleared automatically .setAllowPartialSearchResults(false) .addSort(DestinationIndex.INCREMENTAL_ID, SortOrder.ASC) @@ -383,7 +383,7 @@ public class DataFrameDataExtractor { context.headers, ClientHelper.ML_ORIGIN, client, - SearchAction.INSTANCE, + TransportSearchAction.TYPE, searchRequestBuilder.request(), ActionListener.wrap( searchResponse -> dataSummaryActionListener.onResponse( @@ -401,7 +401,7 @@ public class DataFrameDataExtractor { summaryQuery = QueryBuilders.boolQuery().filter(summaryQuery).filter(allExtractedFieldsExistQuery()); } - return new SearchRequestBuilder(client, SearchAction.INSTANCE).setAllowPartialSearchResults(false) + return new SearchRequestBuilder(client, TransportSearchAction.TYPE).setAllowPartialSearchResults(false) .setIndices(context.indices) .setSize(0) .setQuery(summaryQuery) diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/dataframe/extractor/ExtractedFieldsDetectorFactory.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/dataframe/extractor/ExtractedFieldsDetectorFactory.java index db68f49b7842..9e925ff3f8fe 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/dataframe/extractor/ExtractedFieldsDetectorFactory.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/dataframe/extractor/ExtractedFieldsDetectorFactory.java @@ -17,9 +17,9 @@ import org.elasticsearch.action.fieldcaps.FieldCapabilities; import org.elasticsearch.action.fieldcaps.FieldCapabilitiesAction; import org.elasticsearch.action.fieldcaps.FieldCapabilitiesRequest; import org.elasticsearch.action.fieldcaps.FieldCapabilitiesResponse; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.client.internal.Client; import org.elasticsearch.common.settings.Settings; @@ -141,7 +141,14 @@ public class ExtractedFieldsDetectorFactory { ); } SearchRequest searchRequest = new SearchRequest(index).source(searchSourceBuilder); - ClientHelper.executeWithHeadersAsync(config.getHeaders(), ML_ORIGIN, client, SearchAction.INSTANCE, searchRequest, searchListener); + ClientHelper.executeWithHeadersAsync( + config.getHeaders(), + ML_ORIGIN, + client, + TransportSearchAction.TYPE, + searchRequest, + searchListener + ); } private static void buildFieldCardinalitiesMap( diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/dataframe/steps/InferenceStep.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/dataframe/steps/InferenceStep.java index 3e77f051a451..65ac2b678d93 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/dataframe/steps/InferenceStep.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/dataframe/steps/InferenceStep.java @@ -12,8 +12,8 @@ import org.apache.logging.log4j.Logger; import org.elasticsearch.ResourceNotFoundException; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.admin.indices.refresh.RefreshResponse; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchRequest; +import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.client.internal.node.NodeClient; import org.elasticsearch.core.TimeValue; import org.elasticsearch.index.query.QueryBuilders; @@ -122,7 +122,7 @@ public class InferenceStep extends AbstractDataFrameAnalyticsStep { executeAsyncWithOrigin( client, ML_ORIGIN, - SearchAction.INSTANCE, + TransportSearchAction.TYPE, searchRequest, ActionListener.wrap( searchResponse -> listener.onResponse(searchResponse.getHits().getTotalHits().value > 0), @@ -142,7 +142,7 @@ public class InferenceStep extends AbstractDataFrameAnalyticsStep { SearchRequest searchRequest = new SearchRequest(InferenceIndexConstants.INDEX_PATTERN); searchRequest.source(searchSourceBuilder); - executeAsyncWithOrigin(client, ML_ORIGIN, SearchAction.INSTANCE, searchRequest, ActionListener.wrap(searchResponse -> { + executeAsyncWithOrigin(client, ML_ORIGIN, TransportSearchAction.TYPE, searchRequest, ActionListener.wrap(searchResponse -> { SearchHit[] hits = searchResponse.getHits().getHits(); if (hits.length == 0) { listener.onFailure(new ResourceNotFoundException("No model could be found to perform inference")); diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/inference/deployment/DeploymentManager.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/inference/deployment/DeploymentManager.java index f2421fbc6ded..f48e67f37781 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/inference/deployment/DeploymentManager.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/inference/deployment/DeploymentManager.java @@ -12,8 +12,8 @@ import org.apache.logging.log4j.Logger; import org.apache.lucene.util.SetOnce; import org.elasticsearch.ResourceNotFoundException; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchRequest; +import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.action.support.master.AcknowledgedResponse; import org.elasticsearch.client.internal.Client; import org.elasticsearch.common.Strings; @@ -174,29 +174,37 @@ public class DeploymentManager { task.init(nlpConfig); SearchRequest searchRequest = vocabSearchRequest(nlpConfig.getVocabularyConfig(), modelConfig.getModelId()); - executeAsyncWithOrigin(client, ML_ORIGIN, SearchAction.INSTANCE, searchRequest, ActionListener.wrap(searchVocabResponse -> { - if (searchVocabResponse.getHits().getHits().length == 0) { - failedDeploymentListener.onFailure( - new ResourceNotFoundException( - Messages.getMessage( - Messages.VOCABULARY_NOT_FOUND, - modelConfig.getModelId(), - VocabularyConfig.docId(modelConfig.getModelId()) + executeAsyncWithOrigin( + client, + ML_ORIGIN, + TransportSearchAction.TYPE, + searchRequest, + ActionListener.wrap(searchVocabResponse -> { + if (searchVocabResponse.getHits().getHits().length == 0) { + failedDeploymentListener.onFailure( + new ResourceNotFoundException( + Messages.getMessage( + Messages.VOCABULARY_NOT_FOUND, + modelConfig.getModelId(), + VocabularyConfig.docId(modelConfig.getModelId()) + ) ) - ) - ); - return; - } + ); + return; + } - Vocabulary vocabulary = parseVocabularyDocLeniently(searchVocabResponse.getHits().getAt(0)); - NlpTask nlpTask = new NlpTask(nlpConfig, vocabulary); - NlpTask.Processor processor = nlpTask.createProcessor(); - processContext.nlpTaskProcessor.set(processor); - // here, we are being called back on the searching thread, which MAY be a network thread - // `startAndLoad` creates named pipes, blocking the calling thread, better to execute that in our utility - // executor. - executorServiceForDeployment.execute(() -> processContext.startAndLoad(modelConfig.getLocation(), modelLoadedListener)); - }, failedDeploymentListener::onFailure)); + Vocabulary vocabulary = parseVocabularyDocLeniently(searchVocabResponse.getHits().getAt(0)); + NlpTask nlpTask = new NlpTask(nlpConfig, vocabulary); + NlpTask.Processor processor = nlpTask.createProcessor(); + processContext.nlpTaskProcessor.set(processor); + // here, we are being called back on the searching thread, which MAY be a network thread + // `startAndLoad` creates named pipes, blocking the calling thread, better to execute that in our utility + // executor. + executorServiceForDeployment.execute( + () -> processContext.startAndLoad(modelConfig.getLocation(), modelLoadedListener) + ); + }, failedDeploymentListener::onFailure) + ); } else { failedDeploymentListener.onFailure( new IllegalArgumentException( diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/inference/persistence/TrainedModelProvider.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/inference/persistence/TrainedModelProvider.java index b704dc37bfc2..02a841e44585 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/inference/persistence/TrainedModelProvider.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/inference/persistence/TrainedModelProvider.java @@ -25,9 +25,9 @@ import org.elasticsearch.action.index.IndexAction; import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.search.MultiSearchRequest; import org.elasticsearch.action.search.MultiSearchResponse; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.support.WriteRequest; import org.elasticsearch.client.internal.Client; @@ -400,7 +400,7 @@ public class TrainedModelProvider { if (parentTaskId != null) { searchRequest.setParentTask(parentTaskId); } - executeAsyncWithOrigin(client, ML_ORIGIN, SearchAction.INSTANCE, searchRequest, ActionListener.wrap(searchResponse -> { + executeAsyncWithOrigin(client, ML_ORIGIN, TransportSearchAction.TYPE, searchRequest, ActionListener.wrap(searchResponse -> { if (searchResponse.getHits().getHits().length == 0) { listener.onFailure(new ResourceNotFoundException(Messages.getMessage(Messages.MODEL_METADATA_NOT_FOUND, modelIds))); return; @@ -691,7 +691,7 @@ public class TrainedModelProvider { executeAsyncWithOrigin( client, ML_ORIGIN, - SearchAction.INSTANCE, + TransportSearchAction.TYPE, ChunkedTrainedModelRestorer.buildSearch( client, modelId, @@ -731,7 +731,7 @@ public class TrainedModelProvider { }, getTrainedModelListener::onFailure) ); }, getTrainedModelListener::onFailure); - executeAsyncWithOrigin(client, ML_ORIGIN, SearchAction.INSTANCE, trainedModelConfigSearch, trainedModelSearchHandler); + executeAsyncWithOrigin(client, ML_ORIGIN, TransportSearchAction.TYPE, trainedModelConfigSearch, trainedModelSearchHandler); } public void getTrainedModels( @@ -872,7 +872,7 @@ public class TrainedModelProvider { getTrainedModelListener.onResponse(configs); }, getTrainedModelListener::onFailure); - executeAsyncWithOrigin(client, ML_ORIGIN, SearchAction.INSTANCE, searchRequest, configSearchHandler); + executeAsyncWithOrigin(client, ML_ORIGIN, TransportSearchAction.TYPE, searchRequest, configSearchHandler); } public void deleteTrainedModel(String modelId, ActionListener listener) { diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobDataDeleter.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobDataDeleter.java index 9e357ce7c1de..ac16948e32ed 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobDataDeleter.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobDataDeleter.java @@ -17,11 +17,11 @@ import org.elasticsearch.action.admin.indices.refresh.RefreshAction; import org.elasticsearch.action.admin.indices.refresh.RefreshRequest; import org.elasticsearch.action.admin.indices.refresh.RefreshResponse; import org.elasticsearch.action.bulk.BulkItemResponse; -import org.elasticsearch.action.search.MultiSearchAction; import org.elasticsearch.action.search.MultiSearchRequest; import org.elasticsearch.action.search.MultiSearchResponse; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.action.search.TransportMultiSearchAction; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.support.master.AcknowledgedResponse; import org.elasticsearch.client.internal.Client; @@ -413,7 +413,7 @@ public class JobDataDeleter { ); multiSearchRequest.add(new SearchRequest(indexName).source(source)); } - executeAsyncWithOrigin(client, ML_ORIGIN, MultiSearchAction.INSTANCE, multiSearchRequest, customIndexSearchHandler); + executeAsyncWithOrigin(client, ML_ORIGIN, TransportMultiSearchAction.TYPE, multiSearchRequest, customIndexSearchHandler); }, failureHandler); // Step 5. Get the job as the initial result index name is required diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/retention/ExpiredForecastsRemover.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/retention/ExpiredForecastsRemover.java index 10a12273baff..ed4e6875e260 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/retention/ExpiredForecastsRemover.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/retention/ExpiredForecastsRemover.java @@ -10,9 +10,9 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.action.support.ThreadedActionListener; import org.elasticsearch.client.internal.OriginSettingClient; import org.elasticsearch.index.query.BoolQueryBuilder; @@ -98,7 +98,7 @@ public class ExpiredForecastsRemover implements MlDataRemover { searchRequest.source(source); searchRequest.setParentTask(parentTaskId); client.execute( - SearchAction.INSTANCE, + TransportSearchAction.TYPE, searchRequest, new ThreadedActionListener<>(threadPool.executor(MachineLearning.UTILITY_THREAD_POOL_NAME), forecastStatsHandler) ); diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/datafeed/extractor/aggregation/CompositeAggregationDataExtractorTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/datafeed/extractor/aggregation/CompositeAggregationDataExtractorTests.java index 9ea246adf95c..93d20c0c7be4 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/datafeed/extractor/aggregation/CompositeAggregationDataExtractorTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/datafeed/extractor/aggregation/CompositeAggregationDataExtractorTests.java @@ -7,12 +7,12 @@ package org.elasticsearch.xpack.ml.datafeed.extractor.aggregation; import org.elasticsearch.action.ActionRequestBuilder; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchPhaseExecutionException; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchRequestBuilder; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.ShardSearchFailure; +import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.client.internal.Client; import org.elasticsearch.core.TimeValue; import org.elasticsearch.core.Tuple; @@ -122,9 +122,10 @@ public class CompositeAggregationDataExtractorTests extends ESTestCase { .subAggregation(AggregationBuilders.avg("responsetime").field("responsetime")); runtimeMappings = Collections.emptyMap(); timingStatsReporter = new DatafeedTimingStatsReporter(new DatafeedTimingStats(jobId), mock(DatafeedTimingStatsPersister.class)); - aggregatedSearchRequestBuilder = (searchSourceBuilder) -> new SearchRequestBuilder(testClient, SearchAction.INSTANCE).setSource( - searchSourceBuilder - ).setAllowPartialSearchResults(false).setIndices(indices.toArray(String[]::new)); + aggregatedSearchRequestBuilder = (searchSourceBuilder) -> new SearchRequestBuilder(testClient, TransportSearchAction.TYPE) + .setSource(searchSourceBuilder) + .setAllowPartialSearchResults(false) + .setIndices(indices.toArray(String[]::new)); } public void testExtraction() throws IOException { diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/datafeed/extractor/scroll/ScrollDataExtractorTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/datafeed/extractor/scroll/ScrollDataExtractorTests.java index 13a7552e409c..7ffb3231331a 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/datafeed/extractor/scroll/ScrollDataExtractorTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/datafeed/extractor/scroll/ScrollDataExtractorTests.java @@ -9,7 +9,6 @@ package org.elasticsearch.xpack.ml.datafeed.extractor.scroll; import org.apache.lucene.search.TotalHits; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.action.ActionFuture; -import org.elasticsearch.action.search.ClearScrollAction; import org.elasticsearch.action.search.ClearScrollRequest; import org.elasticsearch.action.search.ClearScrollResponse; import org.elasticsearch.action.search.SearchPhaseExecutionException; @@ -17,6 +16,7 @@ import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchRequestBuilder; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.ShardSearchFailure; +import org.elasticsearch.action.search.TransportClearScrollAction; import org.elasticsearch.client.internal.Client; import org.elasticsearch.common.document.DocumentField; import org.elasticsearch.common.settings.Settings; @@ -173,7 +173,7 @@ public class ScrollDataExtractorTests extends ESTestCase { clearScrollFuture = mock(ActionFuture.class); capturedClearScrollRequests = ArgumentCaptor.forClass(ClearScrollRequest.class); - when(client.execute(same(ClearScrollAction.INSTANCE), capturedClearScrollRequests.capture())).thenReturn(clearScrollFuture); + when(client.execute(same(TransportClearScrollAction.TYPE), capturedClearScrollRequests.capture())).thenReturn(clearScrollFuture); timingStatsReporter = new DatafeedTimingStatsReporter(new DatafeedTimingStats(jobId), mock(DatafeedTimingStatsPersister.class)); } diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/dataframe/DataFrameAnalyticsTaskTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/dataframe/DataFrameAnalyticsTaskTests.java index a2b796921069..70bacebd3609 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/dataframe/DataFrameAnalyticsTaskTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/dataframe/DataFrameAnalyticsTaskTests.java @@ -10,8 +10,8 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.index.IndexAction; import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.index.IndexResponse; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.client.internal.Client; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.settings.Settings; @@ -188,7 +188,7 @@ public class DataFrameAnalyticsTaskTests extends ESTestCase { SearchResponse searchResponse = mock(SearchResponse.class); when(searchResponse.getHits()).thenReturn(searchHits); - doAnswer(withResponse(searchResponse)).when(client).execute(eq(SearchAction.INSTANCE), any(), any()); + doAnswer(withResponse(searchResponse)).when(client).execute(eq(TransportSearchAction.TYPE), any(), any()); IndexResponse indexResponse = mock(IndexResponse.class); doAnswer(withResponse(indexResponse)).when(client).execute(eq(IndexAction.INSTANCE), any(), any()); @@ -217,7 +217,7 @@ public class DataFrameAnalyticsTaskTests extends ESTestCase { ArgumentCaptor indexRequestCaptor = ArgumentCaptor.forClass(IndexRequest.class); InOrder inOrder = inOrder(client, runnable); - inOrder.verify(client).execute(eq(SearchAction.INSTANCE), any(), any()); + inOrder.verify(client).execute(eq(TransportSearchAction.TYPE), any(), any()); inOrder.verify(client).execute(eq(IndexAction.INSTANCE), indexRequestCaptor.capture(), any()); inOrder.verify(runnable).run(); inOrder.verifyNoMoreInteractions(); @@ -284,7 +284,7 @@ public class DataFrameAnalyticsTaskTests extends ESTestCase { SearchResponse searchResponse = mock(SearchResponse.class); when(searchResponse.getHits()).thenReturn(SearchHits.EMPTY_WITH_TOTAL_HITS); - doAnswer(withResponse(searchResponse)).when(client).execute(eq(SearchAction.INSTANCE), any(), any()); + doAnswer(withResponse(searchResponse)).when(client).execute(eq(TransportSearchAction.TYPE), any(), any()); IndexResponse indexResponse = mock(IndexResponse.class); doAnswer(withResponse(indexResponse)).when(client).execute(eq(IndexAction.INSTANCE), any(), any()); @@ -315,7 +315,7 @@ public class DataFrameAnalyticsTaskTests extends ESTestCase { if (nodeShuttingDown == false) { // Verify progress was persisted ArgumentCaptor indexRequestCaptor = ArgumentCaptor.forClass(IndexRequest.class); - verify(client).execute(eq(SearchAction.INSTANCE), any(), any()); + verify(client).execute(eq(TransportSearchAction.TYPE), any(), any()); verify(client).execute(eq(IndexAction.INSTANCE), indexRequestCaptor.capture(), any()); IndexRequest indexRequest = indexRequestCaptor.getValue(); diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsPersisterTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsPersisterTests.java index fe82cfcb00d2..696c32f9863e 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsPersisterTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsPersisterTests.java @@ -14,8 +14,8 @@ import org.elasticsearch.action.bulk.BulkResponse; import org.elasticsearch.action.index.IndexAction; import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.index.IndexResponse; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.action.support.ActionTestUtils; import org.elasticsearch.action.support.WriteRequest; import org.elasticsearch.client.internal.Client; @@ -334,13 +334,13 @@ public class JobResultsPersisterTests extends ESTestCase { SearchResponse searchResponse = mock(SearchResponse.class); when(searchResponse.status()).thenReturn(RestStatus.OK); when(searchResponse.getHits()).thenReturn(searchHits); - doAnswer(withResponse(searchResponse)).when(client).execute(eq(SearchAction.INSTANCE), any(), any()); + doAnswer(withResponse(searchResponse)).when(client).execute(eq(TransportSearchAction.TYPE), any(), any()); Quantiles quantiles = new Quantiles("foo", new Date(), "bar"); persister.persistQuantiles(quantiles, () -> false); InOrder inOrder = inOrder(client); - inOrder.verify(client).execute(eq(SearchAction.INSTANCE), any(), any()); + inOrder.verify(client).execute(eq(TransportSearchAction.TYPE), any(), any()); inOrder.verify(client).execute(eq(BulkAction.INSTANCE), bulkRequestCaptor.capture(), any()); inOrder.verifyNoMoreInteractions(); @@ -370,7 +370,7 @@ public class JobResultsPersisterTests extends ESTestCase { SearchResponse searchResponse = mock(SearchResponse.class); when(searchResponse.getHits()).thenReturn(searchHits); - doAnswer(withResponse(searchResponse)).when(client).execute(eq(SearchAction.INSTANCE), any(), any()); + doAnswer(withResponse(searchResponse)).when(client).execute(eq(TransportSearchAction.TYPE), any(), any()); IndexResponse indexResponse = mock(IndexResponse.class); doAnswer(withResponse(indexResponse)).when(client).execute(eq(IndexAction.INSTANCE), any(), any()); @@ -380,7 +380,7 @@ public class JobResultsPersisterTests extends ESTestCase { persister.persistQuantiles(quantiles, WriteRequest.RefreshPolicy.IMMEDIATE, indexResponseListener); InOrder inOrder = inOrder(client, indexResponseListener); - inOrder.verify(client).execute(eq(SearchAction.INSTANCE), any(), any()); + inOrder.verify(client).execute(eq(TransportSearchAction.TYPE), any(), any()); inOrder.verify(client).execute(eq(IndexAction.INSTANCE), indexRequestCaptor.capture(), any()); inOrder.verify(indexResponseListener).onResponse(any()); inOrder.verifyNoMoreInteractions(); 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 b563694b9209..e9cc63e4dd96 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 @@ -10,14 +10,14 @@ import org.apache.lucene.search.TotalHits; import org.apache.lucene.util.SetOnce; import org.elasticsearch.ResourceNotFoundException; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.search.MultiSearchAction; import org.elasticsearch.action.search.MultiSearchRequest; import org.elasticsearch.action.search.MultiSearchRequestBuilder; import org.elasticsearch.action.search.MultiSearchResponse; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchRequestBuilder; import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.action.search.TransportMultiSearchAction; +import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.action.support.ActionTestUtils; import org.elasticsearch.client.internal.Client; import org.elasticsearch.cluster.metadata.IndexMetadata; @@ -757,7 +757,7 @@ public class JobResultsProviderTests extends ESTestCase { try { Client client = getBasicMockedClient(); - when(client.prepareMultiSearch()).thenReturn(new MultiSearchRequestBuilder(client, MultiSearchAction.INSTANCE)); + when(client.prepareMultiSearch()).thenReturn(new MultiSearchRequestBuilder(client, TransportMultiSearchAction.TYPE)); doAnswer(invocationOnMock -> { MultiSearchRequest multiSearchRequest = (MultiSearchRequest) invocationOnMock.getArguments()[0]; assertThat(multiSearchRequest.requests(), hasSize(2)); @@ -770,10 +770,10 @@ public class JobResultsProviderTests extends ESTestCase { return null; }).when(client).multiSearch(any(), any()); when(client.prepareSearch(AnomalyDetectorsIndex.jobResultsAliasedName("foo"))).thenReturn( - new SearchRequestBuilder(client, SearchAction.INSTANCE).setIndices(AnomalyDetectorsIndex.jobResultsAliasedName("foo")) + new SearchRequestBuilder(client, TransportSearchAction.TYPE).setIndices(AnomalyDetectorsIndex.jobResultsAliasedName("foo")) ); when(client.prepareSearch(AnomalyDetectorsIndex.jobResultsAliasedName("bar"))).thenReturn( - new SearchRequestBuilder(client, SearchAction.INSTANCE).setIndices(AnomalyDetectorsIndex.jobResultsAliasedName("bar")) + new SearchRequestBuilder(client, TransportSearchAction.TYPE).setIndices(AnomalyDetectorsIndex.jobResultsAliasedName("bar")) ); JobResultsProvider provider = createProvider(client); @@ -842,7 +842,9 @@ public class JobResultsProviderTests extends ESTestCase { SearchResponse response = createSearchResponse(source); Client client = getMockedClient(queryBuilder -> assertThat(queryBuilder.getName(), equalTo("ids")), response); - when(client.prepareSearch(indexName)).thenReturn(new SearchRequestBuilder(client, SearchAction.INSTANCE).setIndices(indexName)); + when(client.prepareSearch(indexName)).thenReturn( + new SearchRequestBuilder(client, TransportSearchAction.TYPE).setIndices(indexName) + ); JobResultsProvider provider = createProvider(client); ExponentialAverageCalculationContext contextFoo = new ExponentialAverageCalculationContext( 600.0, @@ -869,7 +871,9 @@ public class JobResultsProviderTests extends ESTestCase { SearchResponse response = createSearchResponse(source); Client client = getMockedClient(queryBuilder -> assertThat(queryBuilder.getName(), equalTo("ids")), response); - when(client.prepareSearch(indexName)).thenReturn(new SearchRequestBuilder(client, SearchAction.INSTANCE).setIndices(indexName)); + when(client.prepareSearch(indexName)).thenReturn( + new SearchRequestBuilder(client, TransportSearchAction.TYPE).setIndices(indexName) + ); JobResultsProvider provider = createProvider(client); provider.datafeedTimingStats("foo", stats -> assertThat(stats, equalTo(new DatafeedTimingStats("foo"))), e -> { throw new AssertionError("Failure getting datafeed timing stats", e); diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/retention/ExpiredAnnotationsRemoverTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/retention/ExpiredAnnotationsRemoverTests.java index f0eb02624999..ad0719011c92 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/retention/ExpiredAnnotationsRemoverTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/retention/ExpiredAnnotationsRemoverTests.java @@ -7,8 +7,8 @@ package org.elasticsearch.xpack.ml.job.retention; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.client.internal.Client; import org.elasticsearch.client.internal.OriginSettingClient; import org.elasticsearch.index.reindex.BulkByScrollResponse; @@ -180,7 +180,7 @@ public class ExpiredAnnotationsRemoverTests extends ESTestCase { ActionListener listener = (ActionListener) invocationOnMock.getArguments()[2]; listener.onResponse(AbstractExpiredJobDataRemoverTests.createSearchResponse(Collections.singletonList(bucket))); return null; - }).when(client).execute(eq(SearchAction.INSTANCE), any(), any()); + }).when(client).execute(eq(TransportSearchAction.TYPE), any(), any()); } private ExpiredAnnotationsRemover createExpiredAnnotationsRemover(Iterator jobIterator) { diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/retention/ExpiredModelSnapshotsRemoverTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/retention/ExpiredModelSnapshotsRemoverTests.java index 5be897df538e..5b3168a42502 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/retention/ExpiredModelSnapshotsRemoverTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/retention/ExpiredModelSnapshotsRemoverTests.java @@ -7,8 +7,8 @@ package org.elasticsearch.xpack.ml.job.retention; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.action.support.master.AcknowledgedResponse; import org.elasticsearch.client.internal.Client; import org.elasticsearch.client.internal.OriginSettingClient; @@ -99,7 +99,7 @@ public class ExpiredModelSnapshotsRemoverTests extends ESTestCase { listener.waitToCompletion(); assertThat(listener.success, is(true)); - verify(client, times(1)).execute(eq(SearchAction.INSTANCE), any(), any()); + verify(client, times(1)).execute(eq(TransportSearchAction.TYPE), any(), any()); } public void testRemove_GivenJobsWithMixedRetentionPolicies() { @@ -351,7 +351,7 @@ public class ExpiredModelSnapshotsRemoverTests extends ESTestCase { } return null; } - }).when(client).execute(same(SearchAction.INSTANCE), any(), any()); + }).when(client).execute(same(TransportSearchAction.TYPE), any(), any()); doAnswer(invocationOnMock -> { capturedDeleteModelSnapshotRequests.add((DeleteByQueryRequest) invocationOnMock.getArguments()[1]); diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/retention/ExpiredResultsRemoverTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/retention/ExpiredResultsRemoverTests.java index 258ec71505ca..5aa5b847b26b 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/retention/ExpiredResultsRemoverTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/retention/ExpiredResultsRemoverTests.java @@ -7,8 +7,8 @@ package org.elasticsearch.xpack.ml.job.retention; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.client.internal.Client; import org.elasticsearch.client.internal.OriginSettingClient; import org.elasticsearch.index.reindex.BulkByScrollResponse; @@ -180,7 +180,7 @@ public class ExpiredResultsRemoverTests extends ESTestCase { ActionListener listener = (ActionListener) invocationOnMock.getArguments()[2]; listener.onResponse(AbstractExpiredJobDataRemoverTests.createSearchResponse(Collections.singletonList(bucket))); return null; - }).when(client).execute(eq(SearchAction.INSTANCE), any(), any()); + }).when(client).execute(eq(TransportSearchAction.TYPE), any(), any()); } private ExpiredResultsRemover createExpiredResultsRemover(Iterator jobIterator) { diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/utils/persistence/BatchedDocumentsIteratorTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/utils/persistence/BatchedDocumentsIteratorTests.java index d0efe69e8ac4..4f1308e9295c 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/utils/persistence/BatchedDocumentsIteratorTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/utils/persistence/BatchedDocumentsIteratorTests.java @@ -8,13 +8,13 @@ package org.elasticsearch.xpack.ml.utils.persistence; import org.apache.lucene.search.TotalHits; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.search.ClearScrollAction; import org.elasticsearch.action.search.ClearScrollResponse; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchResponse; -import org.elasticsearch.action.search.SearchScrollAction; import org.elasticsearch.action.search.SearchScrollRequest; +import org.elasticsearch.action.search.TransportClearScrollAction; +import org.elasticsearch.action.search.TransportSearchAction; +import org.elasticsearch.action.search.TransportSearchScrollAction; import org.elasticsearch.client.internal.Client; import org.elasticsearch.client.internal.OriginSettingClient; import org.elasticsearch.core.TimeValue; @@ -142,7 +142,7 @@ public class BatchedDocumentsIteratorTests extends ESTestCase { wasScrollCleared = true; listener.onResponse(mock(ClearScrollResponse.class)); return null; - }).when(client).execute(eq(ClearScrollAction.INSTANCE), any(), any()); + }).when(client).execute(eq(TransportClearScrollAction.TYPE), any(), any()); } abstract static class ResponsesMocker { @@ -227,7 +227,7 @@ public class BatchedDocumentsIteratorTests extends ESTestCase { ActionListener listener = (ActionListener) invocationOnMock.getArguments()[2]; listener.onResponse(responses.get(responseIndex.getAndIncrement())); return null; - }).when(client).execute(eq(SearchScrollAction.INSTANCE), searchScrollRequestCaptor.capture(), any()); + }).when(client).execute(eq(TransportSearchScrollAction.TYPE), searchScrollRequestCaptor.capture(), any()); return this; } @@ -240,7 +240,7 @@ public class BatchedDocumentsIteratorTests extends ESTestCase { ActionListener listener = (ActionListener) invocationOnMock.getArguments()[2]; listener.onResponse(searchResponse); return null; - }).when(client).execute(eq(SearchAction.INSTANCE), searchRequestCaptor.capture(), any()); + }).when(client).execute(eq(TransportSearchAction.TYPE), searchRequestCaptor.capture(), any()); } } @@ -258,7 +258,7 @@ public class BatchedDocumentsIteratorTests extends ESTestCase { ActionListener listener = (ActionListener) invocationOnMock.getArguments()[2]; listener.onResponse(createSearchResponseWithHits()); return null; - }).when(client).execute(eq(SearchAction.INSTANCE), searchRequestCaptor.capture(), any()); + }).when(client).execute(eq(TransportSearchAction.TYPE), searchRequestCaptor.capture(), any()); return this; } @@ -271,7 +271,7 @@ public class BatchedDocumentsIteratorTests extends ESTestCase { ActionListener listener = (ActionListener) invocationOnMock.getArguments()[2]; listener.onResponse(responses.get(responseIndex.getAndIncrement())); return null; - }).when(client).execute(eq(SearchAction.INSTANCE), searchRequestCaptor.capture(), any()); + }).when(client).execute(eq(TransportSearchAction.TYPE), searchRequestCaptor.capture(), any()); return this; } diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/utils/persistence/ResultsPersisterServiceTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/utils/persistence/ResultsPersisterServiceTests.java index ec07d0424c84..9e2f14aaabd8 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/utils/persistence/ResultsPersisterServiceTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/utils/persistence/ResultsPersisterServiceTests.java @@ -16,10 +16,10 @@ import org.elasticsearch.action.bulk.BulkRequest; import org.elasticsearch.action.bulk.BulkResponse; import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.index.IndexResponse; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.ShardSearchFailure; +import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.client.internal.Client; import org.elasticsearch.client.internal.OriginSettingClient; import org.elasticsearch.cluster.routing.OperationRouting; @@ -131,45 +131,45 @@ public class ResultsPersisterServiceTests extends ESTestCase { } public void testSearchWithRetries_ImmediateSuccess() { - doAnswer(withResponse(SEARCH_RESPONSE_SUCCESS)).when(client).execute(eq(SearchAction.INSTANCE), eq(SEARCH_REQUEST), any()); + doAnswer(withResponse(SEARCH_RESPONSE_SUCCESS)).when(client).execute(eq(TransportSearchAction.TYPE), eq(SEARCH_REQUEST), any()); List messages = new ArrayList<>(); SearchResponse searchResponse = resultsPersisterService.searchWithRetry(SEARCH_REQUEST, JOB_ID, () -> true, messages::add); assertThat(searchResponse, is(SEARCH_RESPONSE_SUCCESS)); assertThat(messages, is(empty())); - verify(client).execute(eq(SearchAction.INSTANCE), eq(SEARCH_REQUEST), any()); + verify(client).execute(eq(TransportSearchAction.TYPE), eq(SEARCH_REQUEST), any()); } public void testSearchWithRetries_SuccessAfterRetry() { doAnswerWithResponses(SEARCH_RESPONSE_FAILURE, SEARCH_RESPONSE_SUCCESS).when(client) - .execute(eq(SearchAction.INSTANCE), eq(SEARCH_REQUEST), any()); + .execute(eq(TransportSearchAction.TYPE), eq(SEARCH_REQUEST), any()); List messages = new ArrayList<>(); SearchResponse searchResponse = resultsPersisterService.searchWithRetry(SEARCH_REQUEST, JOB_ID, () -> true, messages::add); assertThat(searchResponse, is(SEARCH_RESPONSE_SUCCESS)); assertThat(messages, hasSize(1)); - verify(client, times(2)).execute(eq(SearchAction.INSTANCE), eq(SEARCH_REQUEST), any()); + verify(client, times(2)).execute(eq(TransportSearchAction.TYPE), eq(SEARCH_REQUEST), any()); } public void testSearchWithRetries_SuccessAfterRetryDueToException() { doAnswer(withFailure(new IndexPrimaryShardNotAllocatedException(new Index("my-index", "UUID")))).doAnswer( withResponse(SEARCH_RESPONSE_SUCCESS) - ).when(client).execute(eq(SearchAction.INSTANCE), eq(SEARCH_REQUEST), any()); + ).when(client).execute(eq(TransportSearchAction.TYPE), eq(SEARCH_REQUEST), any()); List messages = new ArrayList<>(); SearchResponse searchResponse = resultsPersisterService.searchWithRetry(SEARCH_REQUEST, JOB_ID, () -> true, messages::add); assertThat(searchResponse, is(SEARCH_RESPONSE_SUCCESS)); assertThat(messages, hasSize(1)); - verify(client, times(2)).execute(eq(SearchAction.INSTANCE), eq(SEARCH_REQUEST), any()); + verify(client, times(2)).execute(eq(TransportSearchAction.TYPE), eq(SEARCH_REQUEST), any()); } private void testSearchWithRetries_FailureAfterTooManyRetries(int maxFailureRetries) { resultsPersisterService.setMaxFailureRetries(maxFailureRetries); - doAnswer(withResponse(SEARCH_RESPONSE_FAILURE)).when(client).execute(eq(SearchAction.INSTANCE), eq(SEARCH_REQUEST), any()); + doAnswer(withResponse(SEARCH_RESPONSE_FAILURE)).when(client).execute(eq(TransportSearchAction.TYPE), eq(SEARCH_REQUEST), any()); List messages = new ArrayList<>(); ElasticsearchException e = expectThrows( @@ -179,7 +179,7 @@ public class ResultsPersisterServiceTests extends ESTestCase { assertThat(e.getMessage(), containsString("search failed with status")); assertThat(messages, hasSize(maxFailureRetries)); - verify(client, times(maxFailureRetries + 1)).execute(eq(SearchAction.INSTANCE), eq(SEARCH_REQUEST), any()); + verify(client, times(maxFailureRetries + 1)).execute(eq(TransportSearchAction.TYPE), eq(SEARCH_REQUEST), any()); } public void testSearchWithRetries_FailureAfterTooManyRetries_0() { @@ -195,7 +195,7 @@ public class ResultsPersisterServiceTests extends ESTestCase { } public void testSearchWithRetries_Failure_ShouldNotRetryFromTheBeginning() { - doAnswer(withResponse(SEARCH_RESPONSE_FAILURE)).when(client).execute(eq(SearchAction.INSTANCE), eq(SEARCH_REQUEST), any()); + doAnswer(withResponse(SEARCH_RESPONSE_FAILURE)).when(client).execute(eq(TransportSearchAction.TYPE), eq(SEARCH_REQUEST), any()); List messages = new ArrayList<>(); ElasticsearchException e = expectThrows( @@ -205,14 +205,14 @@ public class ResultsPersisterServiceTests extends ESTestCase { assertThat(e.getMessage(), containsString("search failed with status SERVICE_UNAVAILABLE")); assertThat(messages, empty()); - verify(client, times(1)).execute(eq(SearchAction.INSTANCE), eq(SEARCH_REQUEST), any()); + verify(client, times(1)).execute(eq(TransportSearchAction.TYPE), eq(SEARCH_REQUEST), any()); } public void testSearchWithRetries_Failure_ShouldNotRetryAfterRandomNumberOfRetries() { int maxFailureRetries = 10; resultsPersisterService.setMaxFailureRetries(maxFailureRetries); - doAnswer(withResponse(SEARCH_RESPONSE_FAILURE)).when(client).execute(eq(SearchAction.INSTANCE), eq(SEARCH_REQUEST), any()); + doAnswer(withResponse(SEARCH_RESPONSE_FAILURE)).when(client).execute(eq(TransportSearchAction.TYPE), eq(SEARCH_REQUEST), any()); int maxRetries = randomIntBetween(1, maxFailureRetries); List messages = new ArrayList<>(); @@ -223,14 +223,14 @@ public class ResultsPersisterServiceTests extends ESTestCase { assertThat(e.getMessage(), containsString("search failed with status SERVICE_UNAVAILABLE")); assertThat(messages, hasSize(maxRetries)); - verify(client, times(maxRetries + 1)).execute(eq(SearchAction.INSTANCE), eq(SEARCH_REQUEST), any()); + verify(client, times(maxRetries + 1)).execute(eq(TransportSearchAction.TYPE), eq(SEARCH_REQUEST), any()); } public void testSearchWithRetries_FailureOnIrrecoverableError() { resultsPersisterService.setMaxFailureRetries(5); doAnswer(withFailure(new ElasticsearchStatusException("bad search request", RestStatus.BAD_REQUEST))).when(client) - .execute(eq(SearchAction.INSTANCE), eq(SEARCH_REQUEST), any()); + .execute(eq(TransportSearchAction.TYPE), eq(SEARCH_REQUEST), any()); ElasticsearchException e = expectThrows( ElasticsearchException.class, @@ -238,7 +238,7 @@ public class ResultsPersisterServiceTests extends ESTestCase { ); assertThat(e.getMessage(), containsString("bad search request")); - verify(client, times(1)).execute(eq(SearchAction.INSTANCE), eq(SEARCH_REQUEST), any()); + verify(client, times(1)).execute(eq(TransportSearchAction.TYPE), eq(SEARCH_REQUEST), any()); } private static Supplier shouldRetryUntil(int maxRetries) { diff --git a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupJobTask.java b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupJobTask.java index aac47ca1d2ea..68b5b8953ccb 100644 --- a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupJobTask.java +++ b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/job/RollupJobTask.java @@ -15,8 +15,8 @@ import org.elasticsearch.action.admin.indices.refresh.RefreshResponse; import org.elasticsearch.action.bulk.BulkAction; import org.elasticsearch.action.bulk.BulkRequest; import org.elasticsearch.action.bulk.BulkResponse; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.client.internal.Client; import org.elasticsearch.client.internal.ParentTaskAssigningClient; import org.elasticsearch.common.scheduler.SchedulerEngine; @@ -130,7 +130,7 @@ public class RollupJobTask extends AllocatedPersistentTask implements SchedulerE job.getHeaders(), ClientHelper.ROLLUP_ORIGIN, client, - SearchAction.INSTANCE, + TransportSearchAction.TYPE, buildSearchRequest(), nextPhase ); diff --git a/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/RollupJobTaskTests.java b/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/RollupJobTaskTests.java index 5befaafba0f8..7fcde59f7308 100644 --- a/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/RollupJobTaskTests.java +++ b/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/RollupJobTaskTests.java @@ -12,9 +12,9 @@ import org.elasticsearch.action.ActionResponse; import org.elasticsearch.action.ActionType; import org.elasticsearch.action.admin.indices.refresh.RefreshAction; import org.elasticsearch.action.admin.indices.refresh.RefreshResponse; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.ShardSearchFailure; +import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.client.internal.Client; import org.elasticsearch.common.scheduler.SchedulerEngine; import org.elasticsearch.common.settings.Settings; @@ -619,7 +619,7 @@ public class RollupJobTaskTests extends ESTestCase { ((ActionListener) invocationOnMock.getArguments()[2]).onResponse(r); return null; - }).when(client).execute(eq(SearchAction.INSTANCE), any(), any()); + }).when(client).execute(eq(TransportSearchAction.TYPE), any(), any()); SchedulerEngine schedulerEngine = mock(SchedulerEngine.class); TaskId taskId = new TaskId("node", 123); @@ -728,7 +728,7 @@ public class RollupJobTaskTests extends ESTestCase { ((ActionListener) invocationOnMock.getArguments()[2]).onResponse(r); return null; - }).when(client).execute(eq(SearchAction.INSTANCE), any(), any()); + }).when(client).execute(eq(TransportSearchAction.TYPE), any(), any()); SchedulerEngine schedulerEngine = mock(SchedulerEngine.class); TaskId taskId = new TaskId("node", 123); @@ -838,7 +838,7 @@ public class RollupJobTaskTests extends ESTestCase { ((ActionListener) invocationOnMock.getArguments()[2]).onResponse(r); return null; - }).when(client).execute(eq(SearchAction.INSTANCE), any(), any()); + }).when(client).execute(eq(TransportSearchAction.TYPE), any(), any()); SchedulerEngine schedulerEngine = mock(SchedulerEngine.class); RollupJobStatus status = new RollupJobStatus(IndexerState.STOPPED, null); diff --git a/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/RetrySearchIntegTests.java b/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/RetrySearchIntegTests.java index e11dee6684a4..0551ac3007f1 100644 --- a/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/RetrySearchIntegTests.java +++ b/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/RetrySearchIntegTests.java @@ -7,12 +7,12 @@ package org.elasticsearch.xpack.searchablesnapshots; import org.elasticsearch.action.index.IndexRequestBuilder; -import org.elasticsearch.action.search.ClosePointInTimeAction; import org.elasticsearch.action.search.ClosePointInTimeRequest; -import org.elasticsearch.action.search.OpenPointInTimeAction; import org.elasticsearch.action.search.OpenPointInTimeRequest; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchType; +import org.elasticsearch.action.search.TransportClosePointInTimeAction; +import org.elasticsearch.action.search.TransportOpenPointInTimeAction; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.common.settings.Settings; @@ -142,7 +142,7 @@ public class RetrySearchIntegTests extends BaseSearchableSnapshotsIntegTestCase final OpenPointInTimeRequest openRequest = new OpenPointInTimeRequest(indexName).indicesOptions( IndicesOptions.STRICT_EXPAND_OPEN_FORBID_CLOSED ).keepAlive(TimeValue.timeValueMinutes(2)); - final String pitId = client().execute(OpenPointInTimeAction.INSTANCE, openRequest).actionGet().getPointInTimeId(); + final String pitId = client().execute(TransportOpenPointInTimeAction.TYPE, openRequest).actionGet().getPointInTimeId(); try { SearchResponse resp = prepareSearch().setIndices(indexName) .setPreference(null) @@ -169,7 +169,7 @@ public class RetrySearchIntegTests extends BaseSearchableSnapshotsIntegTestCase assertThat(resp.pointInTimeId(), equalTo(pitId)); assertHitCount(resp, docCount); } finally { - client().execute(ClosePointInTimeAction.INSTANCE, new ClosePointInTimeRequest(pitId)).actionGet(); + client().execute(TransportClosePointInTimeAction.TYPE, new ClosePointInTimeRequest(pitId)).actionGet(); } } } diff --git a/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsCanMatchOnCoordinatorIntegTests.java b/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsCanMatchOnCoordinatorIntegTests.java index 402cb4816ba7..32c031f80177 100644 --- a/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsCanMatchOnCoordinatorIntegTests.java +++ b/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsCanMatchOnCoordinatorIntegTests.java @@ -12,10 +12,10 @@ import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.search.SearchPhaseExecutionException; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchResponse; -import org.elasticsearch.action.search.SearchShardsAction; import org.elasticsearch.action.search.SearchShardsGroup; import org.elasticsearch.action.search.SearchShardsRequest; import org.elasticsearch.action.search.SearchShardsResponse; +import org.elasticsearch.action.search.TransportSearchShardsAction; import org.elasticsearch.blobcache.shared.SharedBlobCacheService; import org.elasticsearch.cluster.metadata.DataStream; import org.elasticsearch.cluster.metadata.IndexMetadata; @@ -209,7 +209,8 @@ public class SearchableSnapshotsCanMatchOnCoordinatorIntegTests extends BaseFroz ); if (includeIndexCoveringSearchRangeInSearchRequest) { - SearchShardsResponse searchShardsResponse = client().execute(SearchShardsAction.INSTANCE, searchShardsRequest).actionGet(); + SearchShardsResponse searchShardsResponse = client().execute(TransportSearchShardsAction.TYPE, searchShardsRequest) + .actionGet(); assertThat(searchShardsResponse.getGroups().size(), equalTo(totalShards)); List> partitionedBySkipped = searchShardsResponse.getGroups() .stream() @@ -227,7 +228,7 @@ public class SearchableSnapshotsCanMatchOnCoordinatorIntegTests extends BaseFroz } else { SearchShardsResponse searchShardsResponse = null; try { - searchShardsResponse = client().execute(SearchShardsAction.INSTANCE, searchShardsRequest).actionGet(); + searchShardsResponse = client().execute(TransportSearchShardsAction.TYPE, searchShardsRequest).actionGet(); } catch (SearchPhaseExecutionException e) { // ignore as this is expected to happen } @@ -290,7 +291,8 @@ public class SearchableSnapshotsCanMatchOnCoordinatorIntegTests extends BaseFroz null ); - SearchShardsResponse searchShardsResponse = client().execute(SearchShardsAction.INSTANCE, searchShardsRequest).actionGet(); + SearchShardsResponse searchShardsResponse = client().execute(TransportSearchShardsAction.TYPE, searchShardsRequest) + .actionGet(); assertThat(searchShardsResponse.getGroups().size(), equalTo(totalShards)); List> partitionedBySkipped = searchShardsResponse.getGroups() .stream() @@ -324,7 +326,7 @@ public class SearchableSnapshotsCanMatchOnCoordinatorIntegTests extends BaseFroz SearchShardsResponse searchShardsResponse = null; try { - searchShardsResponse = client().execute(SearchShardsAction.INSTANCE, searchShardsRequest).actionGet(); + searchShardsResponse = client().execute(TransportSearchShardsAction.TYPE, searchShardsRequest).actionGet(); } catch (SearchPhaseExecutionException e) { // ignore as this is what should happen } @@ -356,7 +358,7 @@ public class SearchableSnapshotsCanMatchOnCoordinatorIntegTests extends BaseFroz null ); - SearchShardsResponse searchShardsResponse = client().execute(SearchShardsAction.INSTANCE, searchShardsRequest) + SearchShardsResponse searchShardsResponse = client().execute(TransportSearchShardsAction.TYPE, searchShardsRequest) .actionGet(); assertThat(searchShardsResponse.getGroups().size(), equalTo(indexOutsideSearchRangeShardCount)); List> partitionedBySkipped = searchShardsResponse.getGroups() @@ -469,7 +471,7 @@ public class SearchableSnapshotsCanMatchOnCoordinatorIntegTests extends BaseFroz null ); - SearchShardsResponse searchShardsResponse = client().execute(SearchShardsAction.INSTANCE, searchShardsRequest).actionGet(); + SearchShardsResponse searchShardsResponse = client().execute(TransportSearchShardsAction.TYPE, searchShardsRequest).actionGet(); assertThat(searchShardsResponse.getGroups().size(), equalTo(totalShards)); List> partitionedBySkipped = searchShardsResponse.getGroups() .stream() @@ -535,7 +537,7 @@ public class SearchableSnapshotsCanMatchOnCoordinatorIntegTests extends BaseFroz null ); - SearchShardsResponse searchShardsResponse = client().execute(SearchShardsAction.INSTANCE, searchShardsRequest).actionGet(); + SearchShardsResponse searchShardsResponse = client().execute(TransportSearchShardsAction.TYPE, searchShardsRequest).actionGet(); assertThat(searchShardsResponse.getGroups().size(), equalTo(totalShards)); List> partitionedBySkipped = searchShardsResponse.getGroups() .stream() @@ -637,7 +639,7 @@ public class SearchableSnapshotsCanMatchOnCoordinatorIntegTests extends BaseFroz { SearchShardsResponse searchShardsResponse = null; try { - client().execute(SearchShardsAction.INSTANCE, searchShardsRequest).actionGet(); + client().execute(TransportSearchShardsAction.TYPE, searchShardsRequest).actionGet(); } catch (SearchPhaseExecutionException e) { // ignore as this is expected to happen } @@ -678,7 +680,7 @@ public class SearchableSnapshotsCanMatchOnCoordinatorIntegTests extends BaseFroz { SearchShardsResponse searchShardsResponse = null; try { - client().execute(SearchShardsAction.INSTANCE, searchShardsRequest).actionGet(); + client().execute(TransportSearchShardsAction.TYPE, searchShardsRequest).actionGet(); } catch (SearchPhaseExecutionException e) { // ignore as this is expected to happen } diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/cache/blob/BlobStoreCacheMaintenanceService.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/cache/blob/BlobStoreCacheMaintenanceService.java index 1b09ea10d36a..64508e1d4995 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/cache/blob/BlobStoreCacheMaintenanceService.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/cache/blob/BlobStoreCacheMaintenanceService.java @@ -16,15 +16,15 @@ import org.elasticsearch.action.bulk.BulkRequest; import org.elasticsearch.action.bulk.BulkResponse; import org.elasticsearch.action.delete.DeleteRequest; import org.elasticsearch.action.delete.DeleteResponse; -import org.elasticsearch.action.search.ClosePointInTimeAction; import org.elasticsearch.action.search.ClosePointInTimeRequest; import org.elasticsearch.action.search.ClosePointInTimeResponse; -import org.elasticsearch.action.search.OpenPointInTimeAction; import org.elasticsearch.action.search.OpenPointInTimeRequest; import org.elasticsearch.action.search.OpenPointInTimeResponse; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.action.search.TransportClosePointInTimeAction; +import org.elasticsearch.action.search.TransportOpenPointInTimeAction; +import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.action.support.TransportActions; import org.elasticsearch.client.internal.Client; import org.elasticsearch.client.internal.OriginSettingClient; @@ -435,7 +435,7 @@ public class BlobStoreCacheMaintenanceService implements ClusterStateListener { if (pointIntTimeId == null) { final OpenPointInTimeRequest openRequest = new OpenPointInTimeRequest(SNAPSHOT_BLOB_CACHE_INDEX); openRequest.keepAlive(keepAlive); - clientWithOrigin.execute(OpenPointInTimeAction.INSTANCE, openRequest, new ActionListener<>() { + clientWithOrigin.execute(TransportOpenPointInTimeAction.TYPE, openRequest, new ActionListener<>() { @Override public void onResponse(OpenPointInTimeResponse response) { logger.trace("periodic maintenance task initialized with point-in-time id [{}]", response.getPointInTimeId()); @@ -476,7 +476,7 @@ public class BlobStoreCacheMaintenanceService implements ClusterStateListener { searchSource.pointInTimeBuilder(pointInTime); final SearchRequest searchRequest = new SearchRequest(); searchRequest.source(searchSource); - clientWithOrigin.execute(SearchAction.INSTANCE, searchRequest, new ActionListener<>() { + clientWithOrigin.execute(TransportSearchAction.TYPE, searchRequest, new ActionListener<>() { @Override public void onResponse(SearchResponse response) { if (searchAfter == null) { @@ -652,21 +652,25 @@ public class BlobStoreCacheMaintenanceService implements ClusterStateListener { final String pitId = pointIntTimeId; if (Strings.hasLength(pitId)) { final ClosePointInTimeRequest closeRequest = new ClosePointInTimeRequest(pitId); - clientWithOrigin.execute(ClosePointInTimeAction.INSTANCE, closeRequest, ActionListener.runAfter(new ActionListener<>() { - @Override - public void onResponse(ClosePointInTimeResponse response) { - if (response.isSucceeded()) { - logger.debug("periodic maintenance task successfully closed point-in-time id [{}]", pitId); - } else { - logger.debug("point-in-time id [{}] not found", pitId); + clientWithOrigin.execute( + TransportClosePointInTimeAction.TYPE, + closeRequest, + ActionListener.runAfter(new ActionListener<>() { + @Override + public void onResponse(ClosePointInTimeResponse response) { + if (response.isSucceeded()) { + logger.debug("periodic maintenance task successfully closed point-in-time id [{}]", pitId); + } else { + logger.debug("point-in-time id [{}] not found", pitId); + } } - } - @Override - public void onFailure(Exception e) { - logger.warn(() -> "failed to close point-in-time id [" + pitId + "]", e); - } - }, () -> Releasables.close(releasable))); + @Override + public void onFailure(Exception e) { + logger.warn(() -> "failed to close point-in-time id [" + pitId + "]", e); + } + }, () -> Releasables.close(releasable)) + ); waitForRelease = true; } } finally { diff --git a/x-pack/plugin/security/qa/security-trial/src/javaRestTest/java/org/elasticsearch/xpack/security/crossclusteraccess/CrossClusterAccessHeadersForCcsRestIT.java b/x-pack/plugin/security/qa/security-trial/src/javaRestTest/java/org/elasticsearch/xpack/security/crossclusteraccess/CrossClusterAccessHeadersForCcsRestIT.java index 9788042ac5ec..97b52a699749 100644 --- a/x-pack/plugin/security/qa/security-trial/src/javaRestTest/java/org/elasticsearch/xpack/security/crossclusteraccess/CrossClusterAccessHeadersForCcsRestIT.java +++ b/x-pack/plugin/security/qa/security-trial/src/javaRestTest/java/org/elasticsearch/xpack/security/crossclusteraccess/CrossClusterAccessHeadersForCcsRestIT.java @@ -16,13 +16,13 @@ import org.elasticsearch.action.admin.cluster.remote.RemoteClusterNodesAction; import org.elasticsearch.action.admin.cluster.state.ClusterStateAction; import org.elasticsearch.action.admin.cluster.state.ClusterStateRequest; import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchResponse; -import org.elasticsearch.action.search.SearchShardsAction; import org.elasticsearch.action.search.SearchShardsRequest; import org.elasticsearch.action.search.SearchShardsResponse; import org.elasticsearch.action.search.ShardSearchFailure; +import org.elasticsearch.action.search.TransportSearchAction; +import org.elasticsearch.action.search.TransportSearchShardsAction; import org.elasticsearch.client.Request; import org.elasticsearch.client.Response; import org.elasticsearch.cluster.ClusterName; @@ -1035,9 +1035,9 @@ public class CrossClusterAccessHeadersForCcsRestIT extends SecurityOnTrialLicens ) throws IOException { final Set expectedActions = new HashSet<>(); if (minimizeRoundtrips) { - expectedActions.add(SearchAction.NAME); + expectedActions.add(TransportSearchAction.TYPE.name()); } else { - expectedActions.add(SearchShardsAction.NAME); + expectedActions.add(TransportSearchShardsAction.TYPE.name()); } if (false == useProxyMode) { expectedActions.add(RemoteClusterNodesAction.TYPE.name()); @@ -1066,7 +1066,7 @@ public class CrossClusterAccessHeadersForCcsRestIT extends SecurityOnTrialLicens ); assertThat(actualCrossClusterAccessSubjectInfo, equalTo(expectedCrossClusterAccessSubjectInfo)); } - case SearchAction.NAME, SearchShardsAction.NAME -> { + case TransportSearchAction.NAME, TransportSearchShardsAction.NAME -> { assertContainsHeadersExpectedForCrossClusterAccess(actual.headers()); assertContainsCrossClusterAccessCredentialsHeader(encodedCredential, actual); final var actualCrossClusterAccessSubjectInfo = CrossClusterAccessSubjectInfo.decode( @@ -1132,7 +1132,7 @@ public class CrossClusterAccessHeadersForCcsRestIT extends SecurityOnTrialLicens } ); service.registerRequestHandler( - SearchShardsAction.NAME, + TransportSearchShardsAction.TYPE.name(), EsExecutors.DIRECT_EXECUTOR_SERVICE, SearchShardsRequest::new, (request, channel, task) -> { @@ -1143,7 +1143,7 @@ public class CrossClusterAccessHeadersForCcsRestIT extends SecurityOnTrialLicens } ); service.registerRequestHandler( - SearchAction.NAME, + TransportSearchAction.TYPE.name(), EsExecutors.DIRECT_EXECUTOR_SERVICE, SearchRequest::new, (request, channel, task) -> { diff --git a/x-pack/plugin/security/qa/security-trial/src/javaRestTest/java/org/elasticsearch/xpack/security/role/RoleWithRemoteIndicesPrivilegesRestIT.java b/x-pack/plugin/security/qa/security-trial/src/javaRestTest/java/org/elasticsearch/xpack/security/role/RoleWithRemoteIndicesPrivilegesRestIT.java index 9fc236ff8859..4e3a520678f7 100644 --- a/x-pack/plugin/security/qa/security-trial/src/javaRestTest/java/org/elasticsearch/xpack/security/role/RoleWithRemoteIndicesPrivilegesRestIT.java +++ b/x-pack/plugin/security/qa/security-trial/src/javaRestTest/java/org/elasticsearch/xpack/security/role/RoleWithRemoteIndicesPrivilegesRestIT.java @@ -7,7 +7,7 @@ package org.elasticsearch.xpack.security.role; -import org.elasticsearch.action.search.SearchAction; +import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.client.Request; import org.elasticsearch.client.RequestOptions; import org.elasticsearch.client.Response; @@ -101,7 +101,7 @@ public class RoleWithRemoteIndicesPrivilegesRestIT extends SecurityOnTrialLicens ); final ResponseException e = expectThrows(ResponseException.class, () -> client().performRequest(searchRequest)); assertEquals(403, e.getResponse().getStatusLine().getStatusCode()); - assertThat(e.getMessage(), containsString("action [" + SearchAction.NAME + "] is unauthorized for user")); + assertThat(e.getMessage(), containsString("action [" + TransportSearchAction.TYPE.name() + "] is unauthorized for user")); // Add local privileges and check local authorization works putRoleRequest = new Request("PUT", "_security/role/" + REMOTE_SEARCH_ROLE); diff --git a/x-pack/plugin/security/src/internalClusterTest/java/org/elasticsearch/integration/DocumentLevelSecurityTests.java b/x-pack/plugin/security/src/internalClusterTest/java/org/elasticsearch/integration/DocumentLevelSecurityTests.java index aa7435f8ea84..aa8dec69b910 100644 --- a/x-pack/plugin/security/src/internalClusterTest/java/org/elasticsearch/integration/DocumentLevelSecurityTests.java +++ b/x-pack/plugin/security/src/internalClusterTest/java/org/elasticsearch/integration/DocumentLevelSecurityTests.java @@ -14,11 +14,11 @@ import org.elasticsearch.action.bulk.BulkItemResponse; import org.elasticsearch.action.bulk.BulkResponse; import org.elasticsearch.action.get.GetResponse; import org.elasticsearch.action.get.MultiGetResponse; -import org.elasticsearch.action.search.ClosePointInTimeAction; import org.elasticsearch.action.search.ClosePointInTimeRequest; import org.elasticsearch.action.search.MultiSearchResponse; import org.elasticsearch.action.search.SearchRequestBuilder; import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.action.search.TransportClosePointInTimeAction; import org.elasticsearch.action.support.WriteRequest; import org.elasticsearch.action.termvectors.MultiTermVectorsResponse; import org.elasticsearch.action.termvectors.TermVectorsRequest; @@ -1172,7 +1172,7 @@ public class DocumentLevelSecurityTests extends SecurityIntegTestCase { assertThat(response.getHits().getAt(0).getSourceAsMap().get("field1"), is("value1")); } } finally { - client().execute(ClosePointInTimeAction.INSTANCE, new ClosePointInTimeRequest(response.pointInTimeId())).actionGet(); + client().execute(TransportClosePointInTimeAction.TYPE, new ClosePointInTimeRequest(response.pointInTimeId())).actionGet(); } } diff --git a/x-pack/plugin/security/src/internalClusterTest/java/org/elasticsearch/integration/FieldLevelSecurityTests.java b/x-pack/plugin/security/src/internalClusterTest/java/org/elasticsearch/integration/FieldLevelSecurityTests.java index e9ac3926a32d..9f396524ff0d 100644 --- a/x-pack/plugin/security/src/internalClusterTest/java/org/elasticsearch/integration/FieldLevelSecurityTests.java +++ b/x-pack/plugin/security/src/internalClusterTest/java/org/elasticsearch/integration/FieldLevelSecurityTests.java @@ -12,15 +12,15 @@ import org.elasticsearch.action.bulk.BulkItemResponse; import org.elasticsearch.action.bulk.BulkResponse; import org.elasticsearch.action.get.GetResponse; import org.elasticsearch.action.get.MultiGetResponse; -import org.elasticsearch.action.search.ClosePointInTimeAction; import org.elasticsearch.action.search.ClosePointInTimeRequest; import org.elasticsearch.action.search.MultiSearchResponse; -import org.elasticsearch.action.search.OpenPointInTimeAction; import org.elasticsearch.action.search.OpenPointInTimeRequest; import org.elasticsearch.action.search.OpenPointInTimeResponse; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchRequestBuilder; import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.action.search.TransportClosePointInTimeAction; +import org.elasticsearch.action.search.TransportOpenPointInTimeAction; import org.elasticsearch.action.support.WriteRequest; import org.elasticsearch.action.termvectors.MultiTermVectorsResponse; import org.elasticsearch.action.termvectors.TermVectorsRequest; @@ -1161,7 +1161,7 @@ public class FieldLevelSecurityTests extends SecurityIntegTestCase { OpenPointInTimeRequest request = new OpenPointInTimeRequest(indices).keepAlive(keepAlive); final OpenPointInTimeResponse response = client().filterWithHeader( Collections.singletonMap(BASIC_AUTH_HEADER, basicAuthHeaderValue(userName, USERS_PASSWD)) - ).execute(OpenPointInTimeAction.INSTANCE, request).actionGet(); + ).execute(TransportOpenPointInTimeAction.TYPE, request).actionGet(); return response.getPointInTimeId(); } @@ -1198,7 +1198,7 @@ public class FieldLevelSecurityTests extends SecurityIntegTestCase { assertThat(response.getHits().getAt(0).getSourceAsMap().get("field1"), is("value1")); } } finally { - client().execute(ClosePointInTimeAction.INSTANCE, new ClosePointInTimeRequest(pitId)).actionGet(); + client().execute(TransportClosePointInTimeAction.TYPE, new ClosePointInTimeRequest(pitId)).actionGet(); } } diff --git a/x-pack/plugin/security/src/internalClusterTest/java/org/elasticsearch/xpack/security/authz/ReadActionsTests.java b/x-pack/plugin/security/src/internalClusterTest/java/org/elasticsearch/xpack/security/authz/ReadActionsTests.java index 7c0cb64848f3..922187029c4d 100644 --- a/x-pack/plugin/security/src/internalClusterTest/java/org/elasticsearch/xpack/security/authz/ReadActionsTests.java +++ b/x-pack/plugin/security/src/internalClusterTest/java/org/elasticsearch/xpack/security/authz/ReadActionsTests.java @@ -11,10 +11,10 @@ import org.elasticsearch.action.get.GetAction; import org.elasticsearch.action.get.MultiGetAction; import org.elasticsearch.action.get.MultiGetResponse; import org.elasticsearch.action.search.MultiSearchResponse; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchRequestBuilder; import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.termvectors.MultiTermVectorsAction; import org.elasticsearch.action.termvectors.MultiTermVectorsResponse; @@ -133,12 +133,12 @@ public class ReadActionsTests extends SecurityIntegTestCase { public void testExplicitNonAuthorizedIndex() { createIndicesWithRandomAliases("test1", "test2", "index1"); - assertThrowsAuthorizationExceptionDefaultUsers(() -> trySearch("test*", "index1").get(), SearchAction.NAME); + assertThrowsAuthorizationExceptionDefaultUsers(() -> trySearch("test*", "index1").get(), TransportSearchAction.TYPE.name()); } public void testIndexNotFound() { createIndicesWithRandomAliases("test1", "test2", "index1"); - assertThrowsAuthorizationExceptionDefaultUsers(() -> trySearch("missing").get(), SearchAction.NAME); + assertThrowsAuthorizationExceptionDefaultUsers(() -> trySearch("missing").get(), TransportSearchAction.TYPE.name()); } public void testIndexNotFoundIgnoreUnavailable() { @@ -220,7 +220,7 @@ public class ReadActionsTests extends SecurityIntegTestCase { assertTrue(multiSearchResponse.getResponses()[1].isFailure()); Exception exception = multiSearchResponse.getResponses()[1].getFailure(); assertThat(exception, instanceOf(ElasticsearchSecurityException.class)); - assertAuthorizationExceptionDefaultUsers(exception, SearchAction.NAME); + assertAuthorizationExceptionDefaultUsers(exception, TransportSearchAction.TYPE.name()); } finally { multiSearchResponse.decRef(); } @@ -260,7 +260,7 @@ public class ReadActionsTests extends SecurityIntegTestCase { assertTrue(multiSearchResponse.getResponses()[1].isFailure()); Exception exception = multiSearchResponse.getResponses()[1].getFailure(); assertThat(exception, instanceOf(ElasticsearchSecurityException.class)); - assertAuthorizationExceptionDefaultUsers(exception, SearchAction.NAME); + assertAuthorizationExceptionDefaultUsers(exception, TransportSearchAction.TYPE.name()); } finally { multiSearchResponse.decRef(); } diff --git a/x-pack/plugin/security/src/internalClusterTest/java/org/elasticsearch/xpack/security/profile/ProfileCancellationIntegTests.java b/x-pack/plugin/security/src/internalClusterTest/java/org/elasticsearch/xpack/security/profile/ProfileCancellationIntegTests.java index 84d4bc070057..87a5146113f7 100644 --- a/x-pack/plugin/security/src/internalClusterTest/java/org/elasticsearch/xpack/security/profile/ProfileCancellationIntegTests.java +++ b/x-pack/plugin/security/src/internalClusterTest/java/org/elasticsearch/xpack/security/profile/ProfileCancellationIntegTests.java @@ -10,7 +10,7 @@ package org.elasticsearch.xpack.security.profile; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.search.SearchAction; +import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.client.Cancellable; import org.elasticsearch.client.Request; import org.elasticsearch.client.RequestOptions; @@ -129,7 +129,11 @@ public class ProfileCancellationIntegTests extends AbstractProfileIntegTestCase final List taskActions = tasks.stream().map(Task::getAction).toList(); assertThat( taskActions, - hasItems(equalTo(SuggestProfilesAction.NAME), equalTo(SearchAction.NAME), startsWith(SearchAction.NAME)) + hasItems( + equalTo(SuggestProfilesAction.NAME), + equalTo(TransportSearchAction.TYPE.name()), + startsWith(TransportSearchAction.TYPE.name()) + ) ); assertThat(isShardSearchBlocked(), is(true)); tasks.forEach(t -> { diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/action/SecurityActionMapper.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/action/SecurityActionMapper.java index 6432ccae9bf5..f4ab867a3966 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/action/SecurityActionMapper.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/action/SecurityActionMapper.java @@ -7,8 +7,8 @@ package org.elasticsearch.xpack.security.action; import org.elasticsearch.action.admin.indices.analyze.AnalyzeAction; -import org.elasticsearch.action.search.ClearScrollAction; import org.elasticsearch.action.search.ClearScrollRequest; +import org.elasticsearch.action.search.TransportClearScrollAction; import org.elasticsearch.action.support.single.shard.SingleShardRequest; import org.elasticsearch.transport.TransportRequest; @@ -29,7 +29,7 @@ public class SecurityActionMapper { */ public static String action(String action, TransportRequest request) { switch (action) { - case ClearScrollAction.NAME -> { + case TransportClearScrollAction.NAME -> { assert request instanceof ClearScrollRequest; boolean isClearAllScrollRequest = ((ClearScrollRequest) request).scrollIds().contains("_all"); if (isClearAllScrollRequest) { diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/ApiKeyService.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/ApiKeyService.java index fc8cc79e419b..6f5c7f2f87cd 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/ApiKeyService.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/ApiKeyService.java @@ -29,8 +29,8 @@ import org.elasticsearch.action.get.GetRequest; import org.elasticsearch.action.get.GetResponse; import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.index.IndexResponse; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchRequest; +import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.action.support.ContextPreservingActionListener; import org.elasticsearch.action.support.WriteRequest.RefreshPolicy; import org.elasticsearch.action.update.UpdateRequest; @@ -1945,7 +1945,7 @@ public class ApiKeyService { () -> executeAsyncWithOrigin( client, SECURITY_ORIGIN, - SearchAction.INSTANCE, + TransportSearchAction.TYPE, searchRequest, ActionListener.wrap(searchResponse -> { final long total = searchResponse.getHits().getTotalHits().value; diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/PreAuthorizationUtils.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/PreAuthorizationUtils.java index 459af5f9a143..aeb6bfc8de79 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/PreAuthorizationUtils.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/PreAuthorizationUtils.java @@ -10,8 +10,8 @@ package org.elasticsearch.xpack.security.authz; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.elasticsearch.action.IndicesRequest; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchTransportService; +import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.xpack.core.security.SecurityContext; import org.elasticsearch.xpack.core.security.authz.AuthorizationEngine.AuthorizationContext; import org.elasticsearch.xpack.core.security.authz.AuthorizationEngine.AuthorizationInfo; @@ -38,7 +38,7 @@ public final class PreAuthorizationUtils { * on a remote node as they only access a subset of resources. */ public static final Map> CHILD_ACTIONS_PRE_AUTHORIZED_BY_PARENT = Map.of( - SearchAction.NAME, + TransportSearchAction.TYPE.name(), Set.of( SearchTransportService.FREE_CONTEXT_ACTION_NAME, SearchTransportService.DFS_ACTION_NAME, diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/RBACEngine.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/RBACEngine.java index 2df8edcfcb21..cdbb690098cb 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/RBACEngine.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/RBACEngine.java @@ -24,12 +24,12 @@ import org.elasticsearch.action.bulk.SimulateBulkAction; import org.elasticsearch.action.delete.DeleteAction; import org.elasticsearch.action.get.MultiGetAction; import org.elasticsearch.action.index.IndexAction; -import org.elasticsearch.action.search.ClearScrollAction; -import org.elasticsearch.action.search.ClosePointInTimeAction; -import org.elasticsearch.action.search.MultiSearchAction; -import org.elasticsearch.action.search.SearchScrollAction; import org.elasticsearch.action.search.SearchScrollRequest; import org.elasticsearch.action.search.SearchTransportService; +import org.elasticsearch.action.search.TransportClearScrollAction; +import org.elasticsearch.action.search.TransportClosePointInTimeAction; +import org.elasticsearch.action.search.TransportMultiSearchAction; +import org.elasticsearch.action.search.TransportSearchScrollAction; import org.elasticsearch.action.termvectors.MultiTermVectorsAction; import org.elasticsearch.cluster.metadata.IndexAbstraction; import org.elasticsearch.common.Strings; @@ -261,7 +261,7 @@ public class RBACEngine implements AuthorizationEngine { case DELETE_SUB_REQUEST_REPLICA: case MultiGetAction.NAME: case MultiTermVectorsAction.NAME: - case MultiSearchAction.NAME: + case TransportMultiSearchAction.NAME: case "indices:data/read/mpercolate": case "indices:data/read/msearch/template": case "indices:data/read/search/template": @@ -326,7 +326,7 @@ public class RBACEngine implements AuthorizationEngine { // if the action is a search scroll action, we first authorize that the user can execute the action for some // index and if they cannot, we can fail the request early before we allow the execution of the action and in // turn the shard actions - if (SearchScrollAction.NAME.equals(action)) { + if (TransportSearchScrollAction.TYPE.name().equals(action)) { ActionRunnable.supply(listener.delegateFailureAndWrap((l, parsedScrollId) -> { if (parsedScrollId.hasLocalIndices()) { l.onResponse( @@ -358,7 +358,7 @@ public class RBACEngine implements AuthorizationEngine { // the same as the user that submitted the original request so no additional checks are needed here. listener.onResponse(IndexAuthorizationResult.ALLOW_NO_INDICES); } - } else if (action.equals(ClosePointInTimeAction.NAME)) { + } else if (action.equals(TransportClosePointInTimeAction.TYPE.name())) { listener.onResponse(IndexAuthorizationResult.ALLOW_NO_INDICES); } else { assert false @@ -948,12 +948,12 @@ public class RBACEngine implements AuthorizationEngine { } private static boolean isScrollRelatedAction(String action) { - return action.equals(SearchScrollAction.NAME) + return action.equals(TransportSearchScrollAction.TYPE.name()) || action.equals(SearchTransportService.FETCH_ID_SCROLL_ACTION_NAME) || action.equals(SearchTransportService.QUERY_FETCH_SCROLL_ACTION_NAME) || action.equals(SearchTransportService.QUERY_SCROLL_ACTION_NAME) || action.equals(SearchTransportService.FREE_CONTEXT_SCROLL_ACTION_NAME) - || action.equals(ClearScrollAction.NAME) + || action.equals(TransportClearScrollAction.NAME) || action.equals("indices:data/read/sql/close_cursor") || action.equals(SearchTransportService.CLEAR_SCROLL_CONTEXTS_ACTION_NAME); } diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/profile/ProfileService.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/profile/ProfileService.java index d76100f1bc83..d572932670f2 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/profile/ProfileService.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/profile/ProfileService.java @@ -24,11 +24,11 @@ import org.elasticsearch.action.get.GetAction; import org.elasticsearch.action.get.GetRequest; import org.elasticsearch.action.get.MultiGetItemResponse; import org.elasticsearch.action.index.IndexResponse; -import org.elasticsearch.action.search.MultiSearchAction; import org.elasticsearch.action.search.MultiSearchRequest; import org.elasticsearch.action.search.MultiSearchResponse; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchRequest; +import org.elasticsearch.action.search.TransportMultiSearchAction; +import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.action.support.WriteRequest.RefreshPolicy; import org.elasticsearch.action.support.master.AcknowledgedResponse; import org.elasticsearch.action.update.UpdateAction; @@ -271,7 +271,7 @@ public class ProfileService { () -> executeAsyncWithOrigin( client, getActionOrigin(), - SearchAction.INSTANCE, + TransportSearchAction.TYPE, searchRequest, ActionListener.wrap(searchResponse -> { final SearchHits searchHits = searchResponse.getHits(); @@ -373,7 +373,7 @@ public class ProfileService { () -> executeAsyncWithOrigin( client, getActionOrigin(), - MultiSearchAction.INSTANCE, + TransportMultiSearchAction.TYPE, multiSearchRequest, ActionListener.wrap(multiSearchResponse -> { final MultiSearchResponse.Item[] items = multiSearchResponse.getResponses(); @@ -553,7 +553,7 @@ public class ProfileService { executeAsyncWithOrigin( client, getActionOrigin(), - MultiSearchAction.INSTANCE, + TransportMultiSearchAction.TYPE, multiSearchRequest, ActionListener.wrap( multiSearchResponse -> listener.onResponse(convertSubjectMultiSearchResponse(multiSearchResponse, subjects)), diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/action/SecurityActionMapperTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/action/SecurityActionMapperTests.java index 696932931b57..eca1503019dd 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/action/SecurityActionMapperTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/action/SecurityActionMapperTests.java @@ -8,8 +8,8 @@ package org.elasticsearch.xpack.security.action; import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.admin.indices.analyze.AnalyzeAction; -import org.elasticsearch.action.search.ClearScrollAction; import org.elasticsearch.action.search.ClearScrollRequest; +import org.elasticsearch.action.search.TransportClearScrollAction; import org.elasticsearch.action.support.single.shard.SingleShardRequest; import org.elasticsearch.test.ESTestCase; @@ -42,7 +42,7 @@ public class SecurityActionMapperTests extends ESTestCase { String randomAction = actionNameBuilder.toString(); assumeFalse( "Random action is one of the known mapped values: " + randomAction, - randomAction.equals(ClearScrollAction.NAME) + randomAction.equals(TransportClearScrollAction.NAME) || randomAction.equals(AnalyzeAction.NAME) || randomAction.equals(AnalyzeAction.NAME + "[s]") ); @@ -56,7 +56,10 @@ public class SecurityActionMapperTests extends ESTestCase { for (int i = 0; i < scrollIds; i++) { clearScrollRequest.addScrollId(randomAlphaOfLength(randomIntBetween(1, 30))); } - assertThat(SecurityActionMapper.action(ClearScrollAction.NAME, clearScrollRequest), equalTo(ClearScrollAction.NAME)); + assertThat( + SecurityActionMapper.action(TransportClearScrollAction.NAME, clearScrollRequest), + equalTo(TransportClearScrollAction.NAME) + ); } public void testClearScrollAll() { @@ -70,7 +73,7 @@ public class SecurityActionMapperTests extends ESTestCase { Collections.shuffle(clearScrollRequest.getScrollIds(), random()); assertThat( - SecurityActionMapper.action(ClearScrollAction.NAME, clearScrollRequest), + SecurityActionMapper.action(TransportClearScrollAction.NAME, clearScrollRequest), equalTo(SecurityActionMapper.CLUSTER_PERMISSION_SCROLL_CLEAR_ALL_NAME) ); } 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 a748de0c8941..28f8a77c422f 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 @@ -24,15 +24,15 @@ import org.elasticsearch.action.bulk.BulkResponse; import org.elasticsearch.action.index.IndexAction; import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.index.IndexResponse; -import org.elasticsearch.action.search.ClearScrollAction; import org.elasticsearch.action.search.ClearScrollRequest; import org.elasticsearch.action.search.ClearScrollResponse; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponseSections; -import org.elasticsearch.action.search.SearchScrollAction; import org.elasticsearch.action.search.SearchScrollRequest; +import org.elasticsearch.action.search.TransportClearScrollAction; +import org.elasticsearch.action.search.TransportSearchAction; +import org.elasticsearch.action.search.TransportSearchScrollAction; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.action.update.UpdateRequest; @@ -193,7 +193,7 @@ public class TransportSamlInvalidateSessionActionTests extends SamlTestCase { } BulkResponse response = new BulkResponse(bulkItemResponses, 1); listener.onResponse((Response) response); - } else if (SearchAction.NAME.equals(action.name())) { + } else if (TransportSearchAction.TYPE.name().equals(action.name())) { assertThat(request, instanceOf(SearchRequest.class)); SearchRequest searchRequest = (SearchRequest) request; searchRequests.add(searchRequest); @@ -217,7 +217,7 @@ public class TransportSamlInvalidateSessionActionTests extends SamlTestCase { null ); listener.onResponse((Response) response); - } else if (SearchScrollAction.NAME.equals(action.name())) { + } else if (TransportSearchScrollAction.TYPE.name().equals(action.name())) { assertThat(request, instanceOf(SearchScrollRequest.class)); final SearchHit[] hits = new SearchHit[0]; final SearchResponse response = new SearchResponse( @@ -239,7 +239,7 @@ public class TransportSamlInvalidateSessionActionTests extends SamlTestCase { null ); listener.onResponse((Response) response); - } else if (ClearScrollAction.NAME.equals(action.name())) { + } else if (TransportClearScrollAction.NAME.equals(action.name())) { assertThat(request, instanceOf(ClearScrollRequest.class)); ClearScrollRequest scrollRequest = (ClearScrollRequest) request; assertEquals("_scrollId1", scrollRequest.getScrollIds().get(0)); diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/ApiKeyServiceTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/ApiKeyServiceTests.java index 90f2520b6347..5dd3bda1686d 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/ApiKeyServiceTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/ApiKeyServiceTests.java @@ -28,10 +28,10 @@ import org.elasticsearch.action.index.IndexAction; import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.index.IndexResponse; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchRequestBuilder; import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.action.update.UpdateAction; import org.elasticsearch.action.update.UpdateRequestBuilder; @@ -271,7 +271,7 @@ public class ApiKeyServiceTests extends ESTestCase { when(clock.instant()).thenReturn(Instant.ofEpochMilli(now)); final Settings settings = Settings.builder().put(XPackSettings.API_KEY_SERVICE_ENABLED_SETTING.getKey(), true).build(); when(client.threadPool()).thenReturn(threadPool); - SearchRequestBuilder searchRequestBuilder = Mockito.spy(new SearchRequestBuilder(client, SearchAction.INSTANCE)); + SearchRequestBuilder searchRequestBuilder = Mockito.spy(new SearchRequestBuilder(client, TransportSearchAction.TYPE)); when(client.prepareSearch(eq(SECURITY_MAIN_ALIAS))).thenReturn(searchRequestBuilder); final ApiKeyService service = createApiKeyService(settings); final AtomicReference searchRequest = new AtomicReference<>(); @@ -332,7 +332,7 @@ public class ApiKeyServiceTests extends ESTestCase { public void testInvalidateApiKeys() throws Exception { final Settings settings = Settings.builder().put(XPackSettings.API_KEY_SERVICE_ENABLED_SETTING.getKey(), true).build(); when(client.threadPool()).thenReturn(threadPool); - SearchRequestBuilder searchRequestBuilder = Mockito.spy(new SearchRequestBuilder(client, SearchAction.INSTANCE)); + SearchRequestBuilder searchRequestBuilder = Mockito.spy(new SearchRequestBuilder(client, TransportSearchAction.TYPE)); when(client.prepareSearch(eq(SECURITY_MAIN_ALIAS))).thenReturn(searchRequestBuilder); final ApiKeyService service = createApiKeyService(settings); final AtomicReference searchRequest = new AtomicReference<>(); @@ -406,7 +406,7 @@ public class ApiKeyServiceTests extends ESTestCase { // Mock the search request for keys to invalidate when(client.threadPool()).thenReturn(threadPool); - when(client.prepareSearch(eq(SECURITY_MAIN_ALIAS))).thenReturn(new SearchRequestBuilder(client, SearchAction.INSTANCE)); + when(client.prepareSearch(eq(SECURITY_MAIN_ALIAS))).thenReturn(new SearchRequestBuilder(client, TransportSearchAction.TYPE)); doAnswer(invocation -> { final var listener = (ActionListener) invocation.getArguments()[1]; final var searchHit = new SearchHit(docId, apiKeyId); @@ -739,7 +739,7 @@ public class ApiKeyServiceTests extends ESTestCase { final Instant now = Instant.now(); when(clock.instant()).thenReturn(now); when(client.threadPool()).thenReturn(threadPool); - SearchRequestBuilder searchRequestBuilder = Mockito.spy(new SearchRequestBuilder(client, SearchAction.INSTANCE)); + SearchRequestBuilder searchRequestBuilder = Mockito.spy(new SearchRequestBuilder(client, TransportSearchAction.TYPE)); when(client.prepareSearch(eq(SECURITY_MAIN_ALIAS))).thenReturn(searchRequestBuilder); final List searchHits = new ArrayList<>(); diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/TokenServiceTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/TokenServiceTests.java index 35335fd5e4a5..0188907462fc 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/TokenServiceTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/TokenServiceTests.java @@ -29,10 +29,10 @@ import org.elasticsearch.action.index.IndexAction; import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.index.IndexResponse; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchRequestBuilder; import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.action.update.UpdateAction; import org.elasticsearch.action.update.UpdateRequestBuilder; @@ -174,7 +174,7 @@ public class TokenServiceTests extends ESTestCase { final String id = (String) inv.getArguments()[1]; return new UpdateRequestBuilder(client, UpdateAction.INSTANCE).setIndex(index).setId(id); }); - when(client.prepareSearch(any(String.class))).thenReturn(new SearchRequestBuilder(client, SearchAction.INSTANCE)); + when(client.prepareSearch(any(String.class))).thenReturn(new SearchRequestBuilder(client, TransportSearchAction.TYPE)); doAnswer(invocationOnMock -> { @SuppressWarnings("unchecked") ActionListener responseActionListener = (ActionListener) invocationOnMock.getArguments()[2]; diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/service/ElasticServiceAccountsTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/service/ElasticServiceAccountsTests.java index aa82cd320a20..fa37a2abee77 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/service/ElasticServiceAccountsTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/service/ElasticServiceAccountsTests.java @@ -24,8 +24,8 @@ import org.elasticsearch.action.delete.DeleteAction; import org.elasticsearch.action.get.GetAction; import org.elasticsearch.action.get.MultiGetAction; import org.elasticsearch.action.index.IndexAction; -import org.elasticsearch.action.search.MultiSearchAction; -import org.elasticsearch.action.search.SearchAction; +import org.elasticsearch.action.search.TransportMultiSearchAction; +import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.cluster.metadata.IndexAbstraction; import org.elasticsearch.common.Strings; import org.elasticsearch.common.settings.Settings; @@ -140,8 +140,8 @@ public class ElasticServiceAccountsTests extends ESTestCase { assertThat(role.indices().allowedIndicesMatcher(DeleteIndexAction.NAME).test(index), is(false)); assertThat(role.indices().allowedIndicesMatcher(GetAction.NAME).test(index), is(false)); assertThat(role.indices().allowedIndicesMatcher(MultiGetAction.NAME).test(index), is(false)); - assertThat(role.indices().allowedIndicesMatcher(SearchAction.NAME).test(index), is(false)); - assertThat(role.indices().allowedIndicesMatcher(MultiSearchAction.NAME).test(index), is(false)); + assertThat(role.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(index), is(false)); + assertThat(role.indices().allowedIndicesMatcher(TransportMultiSearchAction.TYPE.name()).test(index), is(false)); assertThat(role.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(index), is(false)); }); @@ -155,8 +155,8 @@ public class ElasticServiceAccountsTests extends ESTestCase { assertThat(role.indices().allowedIndicesMatcher(DeleteIndexAction.NAME).test(profilingIndex), is(false)); assertThat(role.indices().allowedIndicesMatcher(GetAction.NAME).test(profilingIndex), is(true)); assertThat(role.indices().allowedIndicesMatcher(MultiGetAction.NAME).test(profilingIndex), is(true)); - assertThat(role.indices().allowedIndicesMatcher(SearchAction.NAME).test(profilingIndex), is(true)); - assertThat(role.indices().allowedIndicesMatcher(MultiSearchAction.NAME).test(profilingIndex), is(true)); + assertThat(role.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(profilingIndex), is(true)); + assertThat(role.indices().allowedIndicesMatcher(TransportMultiSearchAction.TYPE.name()).test(profilingIndex), is(true)); assertThat(role.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(profilingIndex), is(false)); List.of("synthetics-" + randomAlphaOfLengthBetween(1, 20)).stream().map(this::mockIndexAbstraction).forEach(index -> { @@ -169,8 +169,8 @@ public class ElasticServiceAccountsTests extends ESTestCase { assertThat(role.indices().allowedIndicesMatcher(DeleteIndexAction.NAME).test(index), is(false)); assertThat(role.indices().allowedIndicesMatcher(GetAction.NAME).test(index), is(true)); assertThat(role.indices().allowedIndicesMatcher(MultiGetAction.NAME).test(index), is(true)); - assertThat(role.indices().allowedIndicesMatcher(SearchAction.NAME).test(index), is(true)); - assertThat(role.indices().allowedIndicesMatcher(MultiSearchAction.NAME).test(index), is(true)); + assertThat(role.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(index), is(true)); + assertThat(role.indices().allowedIndicesMatcher(TransportMultiSearchAction.TYPE.name()).test(index), is(true)); assertThat(role.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(index), is(false)); }); @@ -192,8 +192,8 @@ public class ElasticServiceAccountsTests extends ESTestCase { assertThat(role.indices().allowedIndicesMatcher(BulkAction.NAME).test(dotFleetIndex), is(true)); assertThat(role.indices().allowedIndicesMatcher(GetAction.NAME).test(dotFleetIndex), is(true)); assertThat(role.indices().allowedIndicesMatcher(MultiGetAction.NAME).test(dotFleetIndex), is(true)); - assertThat(role.indices().allowedIndicesMatcher(SearchAction.NAME).test(dotFleetIndex), is(true)); - assertThat(role.indices().allowedIndicesMatcher(MultiSearchAction.NAME).test(dotFleetIndex), is(true)); + assertThat(role.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(dotFleetIndex), is(true)); + assertThat(role.indices().allowedIndicesMatcher(TransportMultiSearchAction.TYPE.name()).test(dotFleetIndex), is(true)); assertThat(role.indices().allowedIndicesMatcher(IndicesStatsAction.NAME).test(dotFleetIndex), is(true)); assertThat(role.indices().allowedIndicesMatcher(DeleteIndexAction.NAME).test(dotFleetIndex), is(false)); assertThat(role.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(dotFleetIndex), is(false)); @@ -207,8 +207,8 @@ public class ElasticServiceAccountsTests extends ESTestCase { assertThat(role.indices().allowedIndicesMatcher(BulkAction.NAME).test(dotFleetSecretsIndex), is(false)); assertThat(role.indices().allowedIndicesMatcher(GetAction.NAME).test(dotFleetSecretsIndex), is(true)); assertThat(role.indices().allowedIndicesMatcher(MultiGetAction.NAME).test(dotFleetSecretsIndex), is(true)); - assertThat(role.indices().allowedIndicesMatcher(SearchAction.NAME).test(dotFleetSecretsIndex), is(true)); - assertThat(role.indices().allowedIndicesMatcher(MultiSearchAction.NAME).test(dotFleetSecretsIndex), is(true)); + assertThat(role.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(dotFleetSecretsIndex), is(true)); + assertThat(role.indices().allowedIndicesMatcher(TransportMultiSearchAction.TYPE.name()).test(dotFleetSecretsIndex), is(true)); assertThat(role.indices().allowedIndicesMatcher(IndicesStatsAction.NAME).test(dotFleetSecretsIndex), is(false)); assertThat(role.indices().allowedIndicesMatcher(DeleteIndexAction.NAME).test(dotFleetSecretsIndex), is(false)); assertThat(role.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(dotFleetSecretsIndex), is(false)); @@ -226,8 +226,8 @@ public class ElasticServiceAccountsTests extends ESTestCase { assertThat(role.indices().allowedIndicesMatcher(BulkAction.NAME).test(apmSampledTracesIndex), is(true)); assertThat(role.indices().allowedIndicesMatcher(GetAction.NAME).test(apmSampledTracesIndex), is(true)); assertThat(role.indices().allowedIndicesMatcher(MultiGetAction.NAME).test(apmSampledTracesIndex), is(true)); - assertThat(role.indices().allowedIndicesMatcher(SearchAction.NAME).test(apmSampledTracesIndex), is(true)); - assertThat(role.indices().allowedIndicesMatcher(MultiSearchAction.NAME).test(apmSampledTracesIndex), is(true)); + assertThat(role.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(apmSampledTracesIndex), is(true)); + assertThat(role.indices().allowedIndicesMatcher(TransportMultiSearchAction.TYPE.name()).test(apmSampledTracesIndex), is(true)); assertThat(role.indices().allowedIndicesMatcher(IndicesStatsAction.NAME).test(apmSampledTracesIndex), is(true)); assertThat(role.indices().allowedIndicesMatcher(DeleteIndexAction.NAME).test(apmSampledTracesIndex), is(false)); assertThat(role.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(apmSampledTracesIndex), is(false)); @@ -372,8 +372,8 @@ public class ElasticServiceAccountsTests extends ESTestCase { assertThat(role.indices().allowedIndicesMatcher(BulkAction.NAME).test(enterpriseSearchIndex), is(true)); assertThat(role.indices().allowedIndicesMatcher(GetAction.NAME).test(enterpriseSearchIndex), is(true)); assertThat(role.indices().allowedIndicesMatcher(MultiGetAction.NAME).test(enterpriseSearchIndex), is(true)); - assertThat(role.indices().allowedIndicesMatcher(SearchAction.NAME).test(enterpriseSearchIndex), is(true)); - assertThat(role.indices().allowedIndicesMatcher(MultiSearchAction.NAME).test(enterpriseSearchIndex), is(true)); + assertThat(role.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(enterpriseSearchIndex), is(true)); + assertThat(role.indices().allowedIndicesMatcher(TransportMultiSearchAction.TYPE.name()).test(enterpriseSearchIndex), is(true)); assertThat(role.indices().allowedIndicesMatcher(IndicesStatsAction.NAME).test(enterpriseSearchIndex), is(true)); assertThat(role.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(enterpriseSearchIndex), is(true)); assertThat(role.indices().allowedIndicesMatcher(RefreshAction.NAME).test(enterpriseSearchIndex), is(true)); @@ -389,5 +389,4 @@ public class ElasticServiceAccountsTests extends ESTestCase { ); return mock; } - } diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/support/mapper/NativeRoleMappingStoreTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/support/mapper/NativeRoleMappingStoreTests.java index efc97ca30cd1..d083c1700c30 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/support/mapper/NativeRoleMappingStoreTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/support/mapper/NativeRoleMappingStoreTests.java @@ -8,10 +8,10 @@ package org.elasticsearch.xpack.security.authc.support.mapper; import org.apache.lucene.search.TotalHits; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchRequestBuilder; import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.client.internal.Client; import org.elasticsearch.cluster.ClusterName; @@ -195,7 +195,7 @@ public class NativeRoleMappingStoreTests extends ESTestCase { when(mockThreadPool.getThreadContext()).thenReturn(new ThreadContext(Settings.EMPTY)); when(client.threadPool()).thenReturn(mockThreadPool); when(client.prepareSearch(eq(SECURITY_MAIN_ALIAS))).thenReturn( - Mockito.spy(new SearchRequestBuilder(client, SearchAction.INSTANCE)) + Mockito.spy(new SearchRequestBuilder(client, TransportSearchAction.TYPE)) ); final ExpressionRoleMapping mapping = new ExpressionRoleMapping( "mapping", @@ -239,7 +239,7 @@ public class NativeRoleMappingStoreTests extends ESTestCase { when(mockThreadPool.getThreadContext()).thenReturn(new ThreadContext(Settings.EMPTY)); when(client.threadPool()).thenReturn(mockThreadPool); when(client.prepareSearch(eq(SECURITY_MAIN_ALIAS))).thenReturn( - Mockito.spy(new SearchRequestBuilder(client, SearchAction.INSTANCE)) + Mockito.spy(new SearchRequestBuilder(client, TransportSearchAction.TYPE)) ); final ExpressionRoleMapping mapping = new ExpressionRoleMapping( "mapping", @@ -303,7 +303,7 @@ public class NativeRoleMappingStoreTests extends ESTestCase { when(mockThreadPool.getThreadContext()).thenReturn(new ThreadContext(Settings.EMPTY)); when(client.threadPool()).thenReturn(mockThreadPool); when(client.prepareSearch(eq(SECURITY_MAIN_ALIAS))).thenReturn( - Mockito.spy(new SearchRequestBuilder(client, SearchAction.INSTANCE)) + Mockito.spy(new SearchRequestBuilder(client, TransportSearchAction.TYPE)) ); final ExpressionRoleMapping mapping = new ExpressionRoleMapping( "mapping", diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/AuthorizationServiceTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/AuthorizationServiceTests.java index edff46cef16e..bf358f03e16a 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/AuthorizationServiceTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/AuthorizationServiceTests.java @@ -56,20 +56,20 @@ import org.elasticsearch.action.get.MultiGetAction; import org.elasticsearch.action.get.MultiGetRequest; import org.elasticsearch.action.index.IndexAction; import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.action.search.ClearScrollAction; import org.elasticsearch.action.search.ClearScrollRequest; -import org.elasticsearch.action.search.ClosePointInTimeAction; import org.elasticsearch.action.search.ClosePointInTimeRequest; -import org.elasticsearch.action.search.MultiSearchAction; import org.elasticsearch.action.search.MultiSearchRequest; -import org.elasticsearch.action.search.OpenPointInTimeAction; import org.elasticsearch.action.search.OpenPointInTimeRequest; import org.elasticsearch.action.search.ParsedScrollId; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchRequest; -import org.elasticsearch.action.search.SearchScrollAction; import org.elasticsearch.action.search.SearchScrollRequest; import org.elasticsearch.action.search.SearchTransportService; +import org.elasticsearch.action.search.TransportClearScrollAction; +import org.elasticsearch.action.search.TransportClosePointInTimeAction; +import org.elasticsearch.action.search.TransportMultiSearchAction; +import org.elasticsearch.action.search.TransportOpenPointInTimeAction; +import org.elasticsearch.action.search.TransportSearchAction; +import org.elasticsearch.action.search.TransportSearchScrollAction; import org.elasticsearch.action.support.ActionTestUtils; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.support.PlainActionFuture; @@ -746,11 +746,11 @@ public class AuthorizationServiceTests extends ESTestCase { final Authentication authentication = createAuthentication(new User("test user")); mockEmptyMetadata(); final String requestId = AuditUtil.getOrGenerateRequestId(threadContext); - authorize(authentication, SearchAction.NAME, request); + authorize(authentication, TransportSearchAction.TYPE.name(), request); verify(auditTrail).accessGranted( eq(requestId), eq(authentication), - eq(SearchAction.NAME), + eq(TransportSearchAction.TYPE.name()), eq(request), authzInfoRoles(Role.EMPTY.names()) ); @@ -768,7 +768,7 @@ public class AuthorizationServiceTests extends ESTestCase { when(parsedScrollId.hasLocalIndices()).thenReturn(hasLocalIndices); if (hasLocalIndices) { assertThrowsAuthorizationException( - () -> authorize(authentication, SearchScrollAction.NAME, searchScrollRequest), + () -> authorize(authentication, TransportSearchScrollAction.TYPE.name(), searchScrollRequest), "indices:data/read/scroll", "test user" ); @@ -780,11 +780,11 @@ public class AuthorizationServiceTests extends ESTestCase { authzInfoRoles(Role.EMPTY.names()) ); } else { - authorize(authentication, SearchScrollAction.NAME, searchScrollRequest); + authorize(authentication, TransportSearchScrollAction.TYPE.name(), searchScrollRequest); verify(auditTrail).accessGranted( eq(requestId), eq(authentication), - eq(SearchScrollAction.NAME), + eq(TransportSearchScrollAction.TYPE.name()), eq(searchScrollRequest), authzInfoRoles(Role.EMPTY.names()) ); @@ -804,11 +804,15 @@ public class AuthorizationServiceTests extends ESTestCase { final Authentication authentication = createAuthentication(new User("test user")); mockEmptyMetadata(); final String requestId = AuditUtil.getOrGenerateRequestId(threadContext); - assertThrowsAuthorizationException(() -> authorize(authentication, SearchAction.NAME, request), SearchAction.NAME, "test user"); + assertThrowsAuthorizationException( + () -> authorize(authentication, TransportSearchAction.TYPE.name(), request), + TransportSearchAction.TYPE.name(), + "test user" + ); verify(auditTrail).accessDenied( eq(requestId), eq(authentication), - eq(SearchAction.NAME), + eq(TransportSearchAction.TYPE.name()), eq(request), authzInfoRoles(Role.EMPTY.names()) ); @@ -825,11 +829,15 @@ public class AuthorizationServiceTests extends ESTestCase { final Authentication authentication = createAuthentication(new User("test user")); mockEmptyMetadata(); final String requestId = AuditUtil.getOrGenerateRequestId(threadContext); - assertThrowsAuthorizationException(() -> authorize(authentication, SearchAction.NAME, request), SearchAction.NAME, "test user"); + assertThrowsAuthorizationException( + () -> authorize(authentication, TransportSearchAction.TYPE.name(), request), + TransportSearchAction.TYPE.name(), + "test user" + ); verify(auditTrail).accessDenied( eq(requestId), eq(authentication), - eq(SearchAction.NAME), + eq(TransportSearchAction.TYPE.name()), eq(request), authzInfoRoles(Role.EMPTY.names()) ); @@ -898,7 +906,7 @@ public class AuthorizationServiceTests extends ESTestCase { } if (hasLocalIndices) { assertThrowsAuthorizationException( - () -> authorize(authentication, OpenPointInTimeAction.NAME, openPointInTimeRequest), + () -> authorize(authentication, TransportOpenPointInTimeAction.TYPE.name(), openPointInTimeRequest), "indices:data/read/open_point_in_time", "test user" ); @@ -910,7 +918,7 @@ public class AuthorizationServiceTests extends ESTestCase { authzInfoRoles(Role.EMPTY.names()) ); } else { - authorize(authentication, OpenPointInTimeAction.NAME, openPointInTimeRequest); + authorize(authentication, TransportOpenPointInTimeAction.TYPE.name(), openPointInTimeRequest); verify(auditTrail).accessGranted( eq(requestId), eq(authentication), @@ -928,7 +936,7 @@ public class AuthorizationServiceTests extends ESTestCase { final Authentication authentication = createAuthentication(new User("test user")); mockEmptyMetadata(); final String requestId = AuditUtil.getOrGenerateRequestId(threadContext); - authorize(authentication, ClosePointInTimeAction.NAME, closePointInTimeRequest); + authorize(authentication, TransportClosePointInTimeAction.TYPE.name(), closePointInTimeRequest); verify(auditTrail).accessGranted( eq(requestId), eq(authentication), @@ -941,7 +949,10 @@ public class AuthorizationServiceTests extends ESTestCase { public void testUnknownRoleCausesDenial() { Tuple tuple = randomFrom( - asList(new Tuple<>(SearchAction.NAME, new SearchRequest()), new Tuple<>(SqlQueryAction.NAME, new SqlQueryRequest())) + asList( + new Tuple<>(TransportSearchAction.TYPE.name(), new SearchRequest()), + new Tuple<>(SqlQueryAction.NAME, new SqlQueryRequest()) + ) ); String action = tuple.v1(); TransportRequest request = tuple.v2(); @@ -973,7 +984,10 @@ public class AuthorizationServiceTests extends ESTestCase { public void testServiceAccountDenial() { Tuple tuple = randomFrom( - asList(new Tuple<>(SearchAction.NAME, new SearchRequest()), new Tuple<>(SqlQueryAction.NAME, new SqlQueryRequest())) + asList( + new Tuple<>(TransportSearchAction.TYPE.name(), new SearchRequest()), + new Tuple<>(SqlQueryAction.NAME, new SqlQueryRequest()) + ) ); String action = tuple.v1(); TransportRequest request = tuple.v2(); @@ -1037,7 +1051,7 @@ public class AuthorizationServiceTests extends ESTestCase { public void testThatRoleWithNoIndicesIsDenied() { Tuple tuple = randomFrom( - new Tuple<>(SearchAction.NAME, new SearchRequest()), + new Tuple<>(TransportSearchAction.TYPE.name(), new SearchRequest()), new Tuple<>(SqlQueryAction.NAME, new SqlQueryRequest()) ); String action = tuple.v1(); @@ -1104,14 +1118,14 @@ public class AuthorizationServiceTests extends ESTestCase { ); assertThrowsAuthorizationException( - () -> authorize(authentication, SearchAction.NAME, searchRequest), - SearchAction.NAME, + () -> authorize(authentication, TransportSearchAction.TYPE.name(), searchRequest), + TransportSearchAction.TYPE.name(), "test user" ); verify(auditTrail).accessDenied( eq(requestId), eq(authentication), - eq(SearchAction.NAME), + eq(TransportSearchAction.TYPE.name()), eq(searchRequest), authzInfoRoles(new String[] { role.getName() }) ); @@ -1135,12 +1149,17 @@ public class AuthorizationServiceTests extends ESTestCase { assertFalse(indexAccessControl.getDocumentPermissions().hasDocumentLevelPermissions()); }); final CountDownLatch latch = new CountDownLatch(1); - authorizationService.authorize(authentication, SearchAction.NAME, searchRequest, new LatchedActionListener<>(listener, latch)); + authorizationService.authorize( + authentication, + TransportSearchAction.TYPE.name(), + searchRequest, + new LatchedActionListener<>(listener, latch) + ); latch.await(); verify(auditTrail).accessGranted( eq(requestId), eq(authentication), - eq(SearchAction.NAME), + eq(TransportSearchAction.TYPE.name()), eq(searchRequest), authzInfoRoles(new String[] { role.getName() }) ); @@ -1176,11 +1195,11 @@ public class AuthorizationServiceTests extends ESTestCase { null ); this.setFakeOriginatingAction = false; - authorize(authentication, SearchAction.NAME, searchRequest, true, () -> { + authorize(authentication, TransportSearchAction.TYPE.name(), searchRequest, true, () -> { verify(rolesStore).getRoles(Mockito.same(authentication), Mockito.any()); IndicesAccessControl iac = threadContext.getTransient(AuthorizationServiceField.INDICES_PERMISSIONS_KEY); // Successful search action authorization should set a parent authorization header. - assertThat(securityContext.getParentAuthorization().action(), equalTo(SearchAction.NAME)); + assertThat(securityContext.getParentAuthorization().action(), equalTo(TransportSearchAction.TYPE.name())); // Within the action handler, execute a child action (the query phase of search) authorize(authentication, SearchTransportService.QUERY_ACTION_NAME, shardRequest, false, () -> { // This child action triggers a second interaction with the role store (which is cached) @@ -1196,7 +1215,7 @@ public class AuthorizationServiceTests extends ESTestCase { verify(auditTrail).accessGranted( eq(requestId), eq(authentication), - eq(SearchAction.NAME), + eq(TransportSearchAction.TYPE.name()), eq(searchRequest), authzInfoRoles(new String[] { role.getName() }) ); @@ -1223,11 +1242,11 @@ public class AuthorizationServiceTests extends ESTestCase { final String requestId = AuditUtil.getOrGenerateRequestId(threadContext); final ClearScrollRequest clearScrollRequest = new ClearScrollRequest(); - authorize(authentication, ClearScrollAction.NAME, clearScrollRequest); + authorize(authentication, TransportClearScrollAction.NAME, clearScrollRequest); verify(auditTrail).accessGranted( eq(requestId), eq(authentication), - eq(ClearScrollAction.NAME), + eq(TransportClearScrollAction.NAME), eq(clearScrollRequest), authzInfoRoles(new String[] { role.getName() }) ); @@ -1236,11 +1255,11 @@ public class AuthorizationServiceTests extends ESTestCase { when(parsedScrollId.hasLocalIndices()).thenReturn(true); final SearchScrollRequest searchScrollRequest = mock(SearchScrollRequest.class); when(searchScrollRequest.parseScrollId()).thenReturn(parsedScrollId); - authorize(authentication, SearchScrollAction.NAME, searchScrollRequest); + authorize(authentication, TransportSearchScrollAction.TYPE.name(), searchScrollRequest); verify(auditTrail).accessGranted( eq(requestId), eq(authentication), - eq(SearchScrollAction.NAME), + eq(TransportSearchScrollAction.TYPE.name()), eq(searchScrollRequest), authzInfoRoles(new String[] { role.getName() }) ); @@ -1415,14 +1434,14 @@ public class AuthorizationServiceTests extends ESTestCase { ElasticsearchSecurityException securityException = expectThrows( ElasticsearchSecurityException.class, - () -> authorize(authentication, SearchAction.NAME, request) + () -> authorize(authentication, TransportSearchAction.TYPE.name(), request) ); assertThat( securityException, throwableWithMessage( containsString( "[" - + SearchAction.NAME + + TransportSearchAction.TYPE.name() + "] is unauthorized" + " for user [" + user.principal() @@ -1963,7 +1982,12 @@ public class AuthorizationServiceTests extends ESTestCase { requests.add( new Tuple<>(BulkAction.NAME + "[s]", new IndexRequest(randomFrom(SECURITY_MAIN_ALIAS, INTERNAL_SECURITY_MAIN_INDEX_7))) ); - requests.add(new Tuple<>(SearchAction.NAME, new SearchRequest(randomFrom(SECURITY_MAIN_ALIAS, INTERNAL_SECURITY_MAIN_INDEX_7)))); + requests.add( + new Tuple<>( + TransportSearchAction.TYPE.name(), + new SearchRequest(randomFrom(SECURITY_MAIN_ALIAS, INTERNAL_SECURITY_MAIN_INDEX_7)) + ) + ); requests.add( new Tuple<>( TermVectorsAction.NAME, @@ -2050,7 +2074,7 @@ public class AuthorizationServiceTests extends ESTestCase { verifyNoMoreInteractions(auditTrail); final SearchRequest searchRequest = new SearchRequest("_all"); - authorize(authentication, SearchAction.NAME, searchRequest); + authorize(authentication, TransportSearchAction.TYPE.name(), searchRequest); assertEquals(2, searchRequest.indices().length); assertEquals(IndicesAndAliasesResolverField.NO_INDICES_OR_ALIASES_LIST, Arrays.asList(searchRequest.indices())); } @@ -2144,7 +2168,12 @@ public class AuthorizationServiceTests extends ESTestCase { final String requestId = AuditUtil.getOrGenerateRequestId(threadContext); List> requests = new ArrayList<>(); - requests.add(new Tuple<>(SearchAction.NAME, new SearchRequest(randomFrom(SECURITY_MAIN_ALIAS, INTERNAL_SECURITY_MAIN_INDEX_7)))); + requests.add( + new Tuple<>( + TransportSearchAction.TYPE.name(), + new SearchRequest(randomFrom(SECURITY_MAIN_ALIAS, INTERNAL_SECURITY_MAIN_INDEX_7)) + ) + ); requests.add( new Tuple<>( TermVectorsAction.NAME, @@ -2274,7 +2303,7 @@ public class AuthorizationServiceTests extends ESTestCase { ); final String requestId = AuditUtil.getOrGenerateRequestId(threadContext); - String action = SearchAction.NAME; + String action = TransportSearchAction.TYPE.name(); SearchRequest request = new SearchRequest("_all"); authorize(authentication, action, request); verify(auditTrail).accessGranted(eq(requestId), eq(authentication), eq(action), eq(request), authzInfoRoles(superuser.roles())); @@ -2360,7 +2389,7 @@ public class AuthorizationServiceTests extends ESTestCase { } case 1 -> { // reindex, msearch, search template, and multi search template delegate to search - action = SearchAction.NAME; + action = TransportSearchAction.TYPE.name(); request = mockRequest; } case 2 -> { @@ -2955,7 +2984,7 @@ public class AuthorizationServiceTests extends ESTestCase { private static Tuple randomCompositeRequest() { return switch (randomIntBetween(0, 7)) { case 0 -> Tuple.tuple(MultiGetAction.NAME, new MultiGetRequest().add("index", "id")); - case 1 -> Tuple.tuple(MultiSearchAction.NAME, new MultiSearchRequest().add(new SearchRequest())); + case 1 -> Tuple.tuple(TransportMultiSearchAction.TYPE.name(), new MultiSearchRequest().add(new SearchRequest())); case 2 -> Tuple.tuple(MultiTermVectorsAction.NAME, new MultiTermVectorsRequest().add("index", "id")); case 3 -> Tuple.tuple(BulkAction.NAME, new BulkRequest().add(new DeleteRequest("index", "id"))); case 4 -> Tuple.tuple("indices:data/read/mpercolate", new MockCompositeIndicesRequest()); @@ -3461,7 +3490,10 @@ public class AuthorizationServiceTests extends ESTestCase { public void testRoleRestrictionAccessDenial() { Tuple tuple = randomFrom( - asList(new Tuple<>(SearchAction.NAME, new SearchRequest()), new Tuple<>(SqlQueryAction.NAME, new SqlQueryRequest())) + asList( + new Tuple<>(TransportSearchAction.TYPE.name(), new SearchRequest()), + new Tuple<>(SqlQueryAction.NAME, new SqlQueryRequest()) + ) ); String action = tuple.v1(); TransportRequest request = tuple.v2(); diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/AuthorizedIndicesTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/AuthorizedIndicesTests.java index 35071628603c..fd2c0c7c6e8d 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/AuthorizedIndicesTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/AuthorizedIndicesTests.java @@ -7,7 +7,7 @@ package org.elasticsearch.xpack.security.authz; import org.elasticsearch.action.admin.indices.resolve.ResolveIndexAction; -import org.elasticsearch.action.search.SearchAction; +import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.cluster.metadata.AliasMetadata; import org.elasticsearch.cluster.metadata.DataStream; @@ -110,7 +110,7 @@ public class AuthorizedIndicesTests extends ESTestCase { Role roles = future.actionGet(); AuthorizedIndices authorizedIndices = RBACEngine.resolveAuthorizedIndicesFromRole( roles, - getRequestInfo(SearchAction.NAME), + getRequestInfo(TransportSearchAction.TYPE.name()), metadata.getIndicesLookup(), () -> ignore -> {} ); @@ -129,7 +129,7 @@ public class AuthorizedIndicesTests extends ESTestCase { Role role = Role.builder(RESTRICTED_INDICES, "role").add(IndexPrivilege.ALL, "*").build(); AuthorizedIndices authorizedIndices = RBACEngine.resolveAuthorizedIndicesFromRole( role, - getRequestInfo(SearchAction.NAME), + getRequestInfo(TransportSearchAction.TYPE.name()), Metadata.EMPTY_METADATA.getIndicesLookup(), () -> ignore -> {} ); @@ -140,7 +140,7 @@ public class AuthorizedIndicesTests extends ESTestCase { Role role = Role.builder(RESTRICTED_INDICES, "user_role").add(IndexPrivilege.ALL, "*").cluster(Set.of("all"), Set.of()).build(); AuthorizedIndices authorizedIndices = RBACEngine.resolveAuthorizedIndicesFromRole( role, - getRequestInfo(SearchAction.NAME), + getRequestInfo(TransportSearchAction.TYPE.name()), Metadata.EMPTY_METADATA.getIndicesLookup(), () -> ignore -> {} ); @@ -172,7 +172,7 @@ public class AuthorizedIndicesTests extends ESTestCase { AuthorizedIndices authorizedIndices = RBACEngine.resolveAuthorizedIndicesFromRole( role, - getRequestInfo(SearchAction.NAME), + getRequestInfo(TransportSearchAction.TYPE.name()), metadata.getIndicesLookup(), () -> ignore -> {} ); @@ -210,7 +210,7 @@ public class AuthorizedIndicesTests extends ESTestCase { AuthorizedIndices authorizedIndices = RBACEngine.resolveAuthorizedIndicesFromRole( role, - getRequestInfo(SearchAction.NAME), + getRequestInfo(TransportSearchAction.TYPE.name()), metadata.getIndicesLookup(), () -> ignore -> {} ); @@ -221,7 +221,7 @@ public class AuthorizedIndicesTests extends ESTestCase { AuthorizedIndices authorizedIndicesSuperUser = RBACEngine.resolveAuthorizedIndicesFromRole( role, - getRequestInfo(SearchAction.NAME), + getRequestInfo(TransportSearchAction.TYPE.name()), metadata.getIndicesLookup(), () -> ignore -> {} ); @@ -292,7 +292,7 @@ public class AuthorizedIndicesTests extends ESTestCase { Role roles = future.actionGet(); AuthorizedIndices authorizedIndices = RBACEngine.resolveAuthorizedIndicesFromRole( roles, - getRequestInfo(SearchAction.NAME), + getRequestInfo(TransportSearchAction.TYPE.name()), metadata.getIndicesLookup(), () -> ignore -> {} ); @@ -374,7 +374,7 @@ public class AuthorizedIndicesTests extends ESTestCase { ); Role roles = future.actionGet(); TransportRequest request = new ResolveIndexAction.Request(new String[] { "a*" }); - AuthorizationEngine.RequestInfo requestInfo = getRequestInfo(request, SearchAction.NAME); + AuthorizationEngine.RequestInfo requestInfo = getRequestInfo(request, TransportSearchAction.TYPE.name()); AuthorizedIndices authorizedIndices = RBACEngine.resolveAuthorizedIndicesFromRole( roles, requestInfo, diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/IndicesAndAliasesResolverTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/IndicesAndAliasesResolverTests.java index 81bc002b4ca7..0709e775776f 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/IndicesAndAliasesResolverTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/IndicesAndAliasesResolverTests.java @@ -24,12 +24,12 @@ import org.elasticsearch.action.bulk.BulkRequest; import org.elasticsearch.action.fieldcaps.FieldCapabilitiesAction; import org.elasticsearch.action.fieldcaps.FieldCapabilitiesRequest; import org.elasticsearch.action.get.MultiGetRequest; -import org.elasticsearch.action.search.MultiSearchAction; import org.elasticsearch.action.search.MultiSearchRequest; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchRequest; -import org.elasticsearch.action.search.SearchShardsAction; import org.elasticsearch.action.search.SearchShardsRequest; +import org.elasticsearch.action.search.TransportMultiSearchAction; +import org.elasticsearch.action.search.TransportSearchAction; +import org.elasticsearch.action.search.TransportSearchShardsAction; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.action.termvectors.MultiTermVectorsRequest; @@ -404,8 +404,10 @@ public class IndicesAndAliasesResolverTests extends ESTestCase { // aliases with names starting with '-' or '+' can be created up to version 5.x and can be around in 6.x ShardSearchRequest request = mock(ShardSearchRequest.class); when(request.indices()).thenReturn(new String[] { "-index10", "-index20", "+index30" }); - List indices = IndicesAndAliasesResolver.resolveIndicesAndAliasesWithoutWildcards(SearchAction.NAME + "[s]", request) - .getLocal(); + List indices = IndicesAndAliasesResolver.resolveIndicesAndAliasesWithoutWildcards( + TransportSearchAction.TYPE.name() + "[s]", + request + ).getLocal(); String[] expectedIndices = new String[] { "-index10", "-index20", "+index30" }; assertThat(indices, hasSize(expectedIndices.length)); assertThat(indices, hasItems(expectedIndices)); @@ -416,7 +418,7 @@ public class IndicesAndAliasesResolverTests extends ESTestCase { when(request.indices()).thenReturn(new String[] { "index*" }); IllegalArgumentException exception = expectThrows( IllegalArgumentException.class, - () -> IndicesAndAliasesResolver.resolveIndicesAndAliasesWithoutWildcards(SearchAction.NAME + "[s]", request) + () -> IndicesAndAliasesResolver.resolveIndicesAndAliasesWithoutWildcards(TransportSearchAction.TYPE.name() + "[s]", request) ); assertThat( exception, @@ -441,7 +443,7 @@ public class IndicesAndAliasesResolverTests extends ESTestCase { } IllegalArgumentException exception = expectThrows( IllegalArgumentException.class, - () -> IndicesAndAliasesResolver.resolveIndicesAndAliasesWithoutWildcards(SearchAction.NAME + "[s]", request) + () -> IndicesAndAliasesResolver.resolveIndicesAndAliasesWithoutWildcards(TransportSearchAction.TYPE.name() + "[s]", request) ); assertThat( @@ -457,7 +459,8 @@ public class IndicesAndAliasesResolverTests extends ESTestCase { public void testExplicitDashIndices() { SearchRequest request = new SearchRequest("-index10", "-index20"); - List indices = resolveIndices(request, buildAuthorizedIndices(userDashIndices, SearchAction.NAME)).getLocal(); + List indices = resolveIndices(request, buildAuthorizedIndices(userDashIndices, TransportSearchAction.TYPE.name())) + .getLocal(); String[] expectedIndices = new String[] { "-index10", "-index20" }; assertThat(indices, hasSize(expectedIndices.length)); assertThat(request.indices().length, equalTo(expectedIndices.length)); @@ -472,7 +475,8 @@ public class IndicesAndAliasesResolverTests extends ESTestCase { } else { request = new SearchRequest("*", "--index20"); } - List indices = resolveIndices(request, buildAuthorizedIndices(userDashIndices, SearchAction.NAME)).getLocal(); + List indices = resolveIndices(request, buildAuthorizedIndices(userDashIndices, TransportSearchAction.TYPE.name())) + .getLocal(); String[] expectedIndices = new String[] { "-index10", "-index11", "-index21" }; assertThat(indices, hasSize(expectedIndices.length)); assertThat(request.indices().length, equalTo(expectedIndices.length)); @@ -482,7 +486,8 @@ public class IndicesAndAliasesResolverTests extends ESTestCase { public void testExplicitMixedWildcardDashIndices() { SearchRequest request = new SearchRequest("-index21", "-does_not_exist", "-index1*", "--index11"); - List indices = resolveIndices(request, buildAuthorizedIndices(userDashIndices, SearchAction.NAME)).getLocal(); + List indices = resolveIndices(request, buildAuthorizedIndices(userDashIndices, TransportSearchAction.TYPE.name())) + .getLocal(); String[] expectedIndices = new String[] { "-index10", "-index21", "-does_not_exist" }; assertThat(indices, hasSize(expectedIndices.length)); assertThat(request.indices().length, equalTo(expectedIndices.length)); @@ -493,7 +498,8 @@ public class IndicesAndAliasesResolverTests extends ESTestCase { public void testDashIndicesNoExpandWildcard() { SearchRequest request = new SearchRequest("-index1*", "--index11"); request.indicesOptions(IndicesOptions.fromOptions(false, randomBoolean(), false, false)); - List indices = resolveIndices(request, buildAuthorizedIndices(userDashIndices, SearchAction.NAME)).getLocal(); + List indices = resolveIndices(request, buildAuthorizedIndices(userDashIndices, TransportSearchAction.TYPE.name())) + .getLocal(); String[] expectedIndices = new String[] { "-index1*", "--index11" }; assertThat(indices, hasSize(expectedIndices.length)); assertThat(request.indices().length, equalTo(expectedIndices.length)); @@ -504,7 +510,8 @@ public class IndicesAndAliasesResolverTests extends ESTestCase { public void testDashIndicesMinus() { SearchRequest request = new SearchRequest("-index10", "-index11", "--index11", "-index20"); request.indicesOptions(IndicesOptions.fromOptions(false, randomBoolean(), randomBoolean(), randomBoolean())); - List indices = resolveIndices(request, buildAuthorizedIndices(userDashIndices, SearchAction.NAME)).getLocal(); + List indices = resolveIndices(request, buildAuthorizedIndices(userDashIndices, TransportSearchAction.TYPE.name())) + .getLocal(); String[] expectedIndices = new String[] { "-index10", "-index11", "--index11", "-index20" }; assertThat(indices, hasSize(expectedIndices.length)); assertThat(request.indices().length, equalTo(expectedIndices.length)); @@ -517,14 +524,15 @@ public class IndicesAndAliasesResolverTests extends ESTestCase { request.indicesOptions(IndicesOptions.fromOptions(true, false, randomBoolean(), randomBoolean())); expectThrows( IndexNotFoundException.class, - () -> resolveIndices(request, buildAuthorizedIndices(userDashIndices, SearchAction.NAME)) + () -> resolveIndices(request, buildAuthorizedIndices(userDashIndices, TransportSearchAction.TYPE.name())) ); } public void testDashNotExistingIndex() { SearchRequest request = new SearchRequest("-does_not_exist"); request.indicesOptions(IndicesOptions.fromOptions(false, randomBoolean(), randomBoolean(), randomBoolean())); - List indices = resolveIndices(request, buildAuthorizedIndices(userDashIndices, SearchAction.NAME)).getLocal(); + List indices = resolveIndices(request, buildAuthorizedIndices(userDashIndices, TransportSearchAction.TYPE.name())) + .getLocal(); String[] expectedIndices = new String[] { "-does_not_exist" }; assertThat(indices, hasSize(expectedIndices.length)); assertThat(request.indices().length, equalTo(expectedIndices.length)); @@ -535,7 +543,7 @@ public class IndicesAndAliasesResolverTests extends ESTestCase { public void testResolveEmptyIndicesExpandWilcardsOpenAndClosed() { SearchRequest request = new SearchRequest(); request.indicesOptions(IndicesOptions.fromOptions(randomBoolean(), randomBoolean(), true, true)); - List indices = resolveIndices(request, buildAuthorizedIndices(user, SearchAction.NAME)).getLocal(); + List indices = resolveIndices(request, buildAuthorizedIndices(user, TransportSearchAction.TYPE.name())).getLocal(); String[] replacedIndices = new String[] { "bar", "bar-closed", "foofoobar", "foobarfoo", "foofoo", "foofoo-closed" }; assertThat(indices, hasSize(replacedIndices.length)); assertThat(request.indices().length, equalTo(replacedIndices.length)); @@ -546,7 +554,7 @@ public class IndicesAndAliasesResolverTests extends ESTestCase { public void testResolveEmptyIndicesExpandWilcardsOpen() { SearchRequest request = new SearchRequest(); request.indicesOptions(IndicesOptions.fromOptions(randomBoolean(), randomBoolean(), true, false)); - List indices = resolveIndices(request, buildAuthorizedIndices(user, SearchAction.NAME)).getLocal(); + List indices = resolveIndices(request, buildAuthorizedIndices(user, TransportSearchAction.TYPE.name())).getLocal(); String[] replacedIndices = new String[] { "bar", "foofoobar", "foobarfoo", "foofoo" }; assertSameValues(indices, replacedIndices); assertThat(request.indices(), arrayContainingInAnyOrder(replacedIndices)); @@ -555,7 +563,7 @@ public class IndicesAndAliasesResolverTests extends ESTestCase { public void testResolveAllExpandWilcardsOpenAndClosed() { SearchRequest request = new SearchRequest("_all"); request.indicesOptions(IndicesOptions.fromOptions(randomBoolean(), randomBoolean(), true, true)); - List indices = resolveIndices(request, buildAuthorizedIndices(user, SearchAction.NAME)).getLocal(); + List indices = resolveIndices(request, buildAuthorizedIndices(user, TransportSearchAction.TYPE.name())).getLocal(); String[] replacedIndices = new String[] { "bar", "bar-closed", "foofoobar", "foobarfoo", "foofoo", "foofoo-closed" }; assertThat(indices, hasSize(replacedIndices.length)); assertThat(request.indices().length, equalTo(replacedIndices.length)); @@ -566,7 +574,7 @@ public class IndicesAndAliasesResolverTests extends ESTestCase { public void testResolveAllExpandWilcardsOpen() { SearchRequest request = new SearchRequest("_all"); request.indicesOptions(IndicesOptions.fromOptions(randomBoolean(), randomBoolean(), true, false)); - List indices = resolveIndices(request, buildAuthorizedIndices(user, SearchAction.NAME)).getLocal(); + List indices = resolveIndices(request, buildAuthorizedIndices(user, TransportSearchAction.TYPE.name())).getLocal(); String[] replacedIndices = new String[] { "bar", "foofoobar", "foobarfoo", "foofoo" }; assertThat(indices, hasSize(replacedIndices.length)); assertThat(request.indices().length, equalTo(replacedIndices.length)); @@ -577,7 +585,7 @@ public class IndicesAndAliasesResolverTests extends ESTestCase { public void testResolveWildcardsStrictExpand() { SearchRequest request = new SearchRequest("barbaz", "foofoo*"); request.indicesOptions(IndicesOptions.fromOptions(false, randomBoolean(), true, true)); - List indices = resolveIndices(request, buildAuthorizedIndices(user, SearchAction.NAME)).getLocal(); + List indices = resolveIndices(request, buildAuthorizedIndices(user, TransportSearchAction.TYPE.name())).getLocal(); String[] replacedIndices = new String[] { "barbaz", "foofoobar", "foofoo", "foofoo-closed" }; assertThat(indices, hasSize(replacedIndices.length)); assertThat(request.indices().length, equalTo(replacedIndices.length)); @@ -588,7 +596,7 @@ public class IndicesAndAliasesResolverTests extends ESTestCase { public void testResolveWildcardsExpandOpenAndClosedIgnoreUnavailable() { SearchRequest request = new SearchRequest("barbaz", "foofoo*"); request.indicesOptions(IndicesOptions.fromOptions(true, randomBoolean(), true, true)); - List indices = resolveIndices(request, buildAuthorizedIndices(user, SearchAction.NAME)).getLocal(); + List indices = resolveIndices(request, buildAuthorizedIndices(user, TransportSearchAction.TYPE.name())).getLocal(); String[] replacedIndices = new String[] { "foofoobar", "foofoo", "foofoo-closed" }; assertThat(indices, hasSize(replacedIndices.length)); assertThat(request.indices().length, equalTo(replacedIndices.length)); @@ -599,7 +607,7 @@ public class IndicesAndAliasesResolverTests extends ESTestCase { public void testResolveWildcardsStrictExpandOpen() { SearchRequest request = new SearchRequest("barbaz", "foofoo*"); request.indicesOptions(IndicesOptions.fromOptions(false, randomBoolean(), true, false)); - List indices = resolveIndices(request, buildAuthorizedIndices(user, SearchAction.NAME)).getLocal(); + List indices = resolveIndices(request, buildAuthorizedIndices(user, TransportSearchAction.TYPE.name())).getLocal(); String[] replacedIndices = new String[] { "barbaz", "foofoobar", "foofoo" }; assertThat(indices, hasSize(replacedIndices.length)); assertThat(request.indices().length, equalTo(replacedIndices.length)); @@ -610,7 +618,7 @@ public class IndicesAndAliasesResolverTests extends ESTestCase { public void testResolveWildcardsLenientExpandOpen() { SearchRequest request = new SearchRequest("barbaz", "foofoo*"); request.indicesOptions(IndicesOptions.fromOptions(true, randomBoolean(), true, false)); - List indices = resolveIndices(request, buildAuthorizedIndices(user, SearchAction.NAME)).getLocal(); + List indices = resolveIndices(request, buildAuthorizedIndices(user, TransportSearchAction.TYPE.name())).getLocal(); String[] replacedIndices = new String[] { "foofoobar", "foofoo" }; assertThat(indices, hasSize(replacedIndices.length)); assertThat(request.indices().length, equalTo(replacedIndices.length)); @@ -621,7 +629,7 @@ public class IndicesAndAliasesResolverTests extends ESTestCase { public void testResolveWildcardsMinusExpandWilcardsOpen() { SearchRequest request = new SearchRequest("*", "-foofoo*"); request.indicesOptions(IndicesOptions.fromOptions(randomBoolean(), randomBoolean(), true, false)); - List indices = resolveIndices(request, buildAuthorizedIndices(user, SearchAction.NAME)).getLocal(); + List indices = resolveIndices(request, buildAuthorizedIndices(user, TransportSearchAction.TYPE.name())).getLocal(); String[] replacedIndices = new String[] { "bar", "foobarfoo" }; assertThat(indices, hasSize(replacedIndices.length)); assertThat(request.indices().length, equalTo(replacedIndices.length)); @@ -632,7 +640,7 @@ public class IndicesAndAliasesResolverTests extends ESTestCase { public void testResolveWildcardsMinusExpandWilcardsOpenAndClosed() { SearchRequest request = new SearchRequest("*", "-foofoo*"); request.indicesOptions(IndicesOptions.fromOptions(randomBoolean(), randomBoolean(), true, true)); - List indices = resolveIndices(request, buildAuthorizedIndices(user, SearchAction.NAME)).getLocal(); + List indices = resolveIndices(request, buildAuthorizedIndices(user, TransportSearchAction.TYPE.name())).getLocal(); String[] replacedIndices = new String[] { "bar", "foobarfoo", "bar-closed" }; assertThat(indices, hasSize(replacedIndices.length)); assertThat(request.indices().length, equalTo(replacedIndices.length)); @@ -644,20 +652,20 @@ public class IndicesAndAliasesResolverTests extends ESTestCase { SearchRequest request = new SearchRequest("*", "-foofoo*"); // no wildcard expand and no ignore unavailable request.indicesOptions(IndicesOptions.fromOptions(false, randomBoolean(), false, false)); - ResolvedIndices indices = resolveIndices(request, buildAuthorizedIndices(user, SearchAction.NAME)); + ResolvedIndices indices = resolveIndices(request, buildAuthorizedIndices(user, TransportSearchAction.TYPE.name())); String[] replacedIndices = new String[] { "*", "-foofoo*" }; assertThat(indices.getLocal(), containsInAnyOrder(replacedIndices)); assertThat(request.indices(), arrayContainingInAnyOrder(replacedIndices)); // no wildcard expand but ignore unavailable request = new SearchRequest("*", "-foofoo*"); request.indicesOptions(IndicesOptions.fromOptions(true, true, false, false)); - indices = resolveIndices(request, buildAuthorizedIndices(user, SearchAction.NAME)); + indices = resolveIndices(request, buildAuthorizedIndices(user, TransportSearchAction.TYPE.name())); assertNoIndices(request, indices); SearchRequest disallowNoIndicesRequest = new SearchRequest("*", "-foofoo*"); disallowNoIndicesRequest.indicesOptions(IndicesOptions.fromOptions(true, false, false, false)); IndexNotFoundException e = expectThrows( IndexNotFoundException.class, - () -> resolveIndices(disallowNoIndicesRequest, buildAuthorizedIndices(user, SearchAction.NAME)) + () -> resolveIndices(disallowNoIndicesRequest, buildAuthorizedIndices(user, TransportSearchAction.TYPE.name())) ); assertEquals("no such index [[*, -foofoo*]]", e.getMessage()); } @@ -665,7 +673,7 @@ public class IndicesAndAliasesResolverTests extends ESTestCase { public void testResolveWildcardsExclusionsExpandWilcardsOpenStrict() { SearchRequest request = new SearchRequest("*", "-foofoo*", "barbaz", "foob*"); request.indicesOptions(IndicesOptions.fromOptions(false, true, true, false)); - List indices = resolveIndices(request, buildAuthorizedIndices(user, SearchAction.NAME)).getLocal(); + List indices = resolveIndices(request, buildAuthorizedIndices(user, TransportSearchAction.TYPE.name())).getLocal(); String[] replacedIndices = new String[] { "bar", "foobarfoo", "barbaz" }; assertSameValues(indices, replacedIndices); assertThat(request.indices(), arrayContainingInAnyOrder("bar", "foobarfoo", "barbaz", "foobarfoo")); @@ -674,7 +682,7 @@ public class IndicesAndAliasesResolverTests extends ESTestCase { public void testResolveWildcardsPlusAndMinusExpandWilcardsOpenIgnoreUnavailable() { SearchRequest request = new SearchRequest("*", "-foofoo*", "+barbaz", "+foob*"); request.indicesOptions(IndicesOptions.fromOptions(true, true, true, false)); - List indices = resolveIndices(request, buildAuthorizedIndices(user, SearchAction.NAME)).getLocal(); + List indices = resolveIndices(request, buildAuthorizedIndices(user, TransportSearchAction.TYPE.name())).getLocal(); String[] replacedIndices = new String[] { "bar", "foobarfoo" }; assertThat(indices, hasSize(replacedIndices.length)); assertThat(request.indices().length, equalTo(replacedIndices.length)); @@ -685,7 +693,7 @@ public class IndicesAndAliasesResolverTests extends ESTestCase { public void testResolveWildcardsExclusionExpandWilcardsOpenAndClosedStrict() { SearchRequest request = new SearchRequest("*", "-foofoo*", "barbaz"); request.indicesOptions(IndicesOptions.fromOptions(false, randomBoolean(), true, true)); - List indices = resolveIndices(request, buildAuthorizedIndices(user, SearchAction.NAME)).getLocal(); + List indices = resolveIndices(request, buildAuthorizedIndices(user, TransportSearchAction.TYPE.name())).getLocal(); String[] replacedIndices = new String[] { "bar", "bar-closed", "barbaz", "foobarfoo" }; assertSameValues(indices, replacedIndices); assertThat(request.indices(), arrayContainingInAnyOrder(replacedIndices)); @@ -694,7 +702,7 @@ public class IndicesAndAliasesResolverTests extends ESTestCase { public void testResolveWildcardsExclusionExpandWilcardsOpenAndClosedIgnoreUnavailable() { SearchRequest request = new SearchRequest("*", "-foofoo*", "barbaz"); request.indicesOptions(IndicesOptions.fromOptions(true, randomBoolean(), true, true)); - List indices = resolveIndices(request, buildAuthorizedIndices(user, SearchAction.NAME)).getLocal(); + List indices = resolveIndices(request, buildAuthorizedIndices(user, TransportSearchAction.TYPE.name())).getLocal(); String[] replacedIndices = new String[] { "bar", "bar-closed", "foobarfoo" }; assertThat(indices, hasSize(replacedIndices.length)); assertThat(indices, hasItems(replacedIndices)); @@ -704,7 +712,7 @@ public class IndicesAndAliasesResolverTests extends ESTestCase { public void testResolveNonMatchingIndicesAllowNoIndices() { SearchRequest request = new SearchRequest("missing*"); request.indicesOptions(IndicesOptions.fromOptions(randomBoolean(), true, true, randomBoolean())); - assertNoIndices(request, resolveIndices(request, buildAuthorizedIndices(user, SearchAction.NAME))); + assertNoIndices(request, resolveIndices(request, buildAuthorizedIndices(user, TransportSearchAction.TYPE.name()))); } public void testResolveNonMatchingIndicesDisallowNoIndices() { @@ -712,7 +720,7 @@ public class IndicesAndAliasesResolverTests extends ESTestCase { request.indicesOptions(IndicesOptions.fromOptions(randomBoolean(), false, true, randomBoolean())); IndexNotFoundException e = expectThrows( IndexNotFoundException.class, - () -> resolveIndices(request, buildAuthorizedIndices(user, SearchAction.NAME)) + () -> resolveIndices(request, buildAuthorizedIndices(user, TransportSearchAction.TYPE.name())) ); assertEquals("no such index [missing*]", e.getMessage()); } @@ -720,7 +728,7 @@ public class IndicesAndAliasesResolverTests extends ESTestCase { public void testResolveExplicitIndicesStrict() { SearchRequest request = new SearchRequest("missing", "bar", "barbaz"); request.indicesOptions(IndicesOptions.fromOptions(false, randomBoolean(), randomBoolean(), randomBoolean())); - List indices = resolveIndices(request, buildAuthorizedIndices(user, SearchAction.NAME)).getLocal(); + List indices = resolveIndices(request, buildAuthorizedIndices(user, TransportSearchAction.TYPE.name())).getLocal(); String[] replacedIndices = new String[] { "missing", "bar", "barbaz" }; assertThat(indices, hasSize(replacedIndices.length)); assertThat(request.indices().length, equalTo(replacedIndices.length)); @@ -731,7 +739,7 @@ public class IndicesAndAliasesResolverTests extends ESTestCase { public void testResolveExplicitIndicesIgnoreUnavailable() { SearchRequest request = new SearchRequest("missing", "missing-and-unauthorized", "bar", "barbaz"); request.indicesOptions(IndicesOptions.fromOptions(true, randomBoolean(), randomBoolean(), randomBoolean())); - List indices = resolveIndices(request, buildAuthorizedIndices(user, SearchAction.NAME)).getLocal(); + List indices = resolveIndices(request, buildAuthorizedIndices(user, TransportSearchAction.TYPE.name())).getLocal(); assertThat(indices, containsInAnyOrder("bar", "missing")); assertThat(request.indices(), arrayContainingInAnyOrder("bar", "missing")); } @@ -739,7 +747,7 @@ public class IndicesAndAliasesResolverTests extends ESTestCase { public void testResolveNoAuthorizedIndicesAllowNoIndices() { SearchRequest request = new SearchRequest(); request.indicesOptions(IndicesOptions.fromOptions(randomBoolean(), true, true, randomBoolean())); - assertNoIndices(request, resolveIndices(request, buildAuthorizedIndices(userNoIndices, SearchAction.NAME))); + assertNoIndices(request, resolveIndices(request, buildAuthorizedIndices(userNoIndices, TransportSearchAction.TYPE.name()))); } public void testResolveNoAuthorizedIndicesDisallowNoIndices() { @@ -747,7 +755,7 @@ public class IndicesAndAliasesResolverTests extends ESTestCase { request.indicesOptions(IndicesOptions.fromOptions(randomBoolean(), false, true, randomBoolean())); IndexNotFoundException e = expectThrows( IndexNotFoundException.class, - () -> resolveIndices(request, buildAuthorizedIndices(userNoIndices, SearchAction.NAME)) + () -> resolveIndices(request, buildAuthorizedIndices(userNoIndices, TransportSearchAction.TYPE.name())) ); assertEquals("no such index [[]]", e.getMessage()); } @@ -755,7 +763,7 @@ public class IndicesAndAliasesResolverTests extends ESTestCase { public void testResolveMissingIndexStrict() { SearchRequest request = new SearchRequest("bar*", "missing"); request.indicesOptions(IndicesOptions.fromOptions(false, true, true, false)); - List indices = resolveIndices(request, buildAuthorizedIndices(user, SearchAction.NAME)).getLocal(); + List indices = resolveIndices(request, buildAuthorizedIndices(user, TransportSearchAction.TYPE.name())).getLocal(); String[] expectedIndices = new String[] { "bar", "missing" }; assertThat(indices, hasSize(expectedIndices.length)); assertThat(request.indices().length, equalTo(expectedIndices.length)); @@ -766,7 +774,7 @@ public class IndicesAndAliasesResolverTests extends ESTestCase { public void testResolveMissingIndexIgnoreUnavailable() { SearchRequest request = new SearchRequest("bar*", "missing", "missing-and-unauthorized"); request.indicesOptions(IndicesOptions.fromOptions(true, randomBoolean(), true, false)); - List indices = resolveIndices(request, buildAuthorizedIndices(user, SearchAction.NAME)).getLocal(); + List indices = resolveIndices(request, buildAuthorizedIndices(user, TransportSearchAction.TYPE.name())).getLocal(); assertThat(indices, containsInAnyOrder("bar", "missing")); assertThat(request.indices(), arrayContainingInAnyOrder("bar", "missing")); } @@ -774,7 +782,7 @@ public class IndicesAndAliasesResolverTests extends ESTestCase { public void testResolveNonMatchingIndicesAndExplicit() { SearchRequest request = new SearchRequest("missing*", "bar"); request.indicesOptions(IndicesOptions.fromOptions(randomBoolean(), true, true, randomBoolean())); - List indices = resolveIndices(request, buildAuthorizedIndices(user, SearchAction.NAME)).getLocal(); + List indices = resolveIndices(request, buildAuthorizedIndices(user, TransportSearchAction.TYPE.name())).getLocal(); String[] expectedIndices = new String[] { "bar" }; assertThat(indices.toArray(new String[indices.size()]), equalTo(expectedIndices)); assertThat(request.indices(), equalTo(expectedIndices)); @@ -783,7 +791,7 @@ public class IndicesAndAliasesResolverTests extends ESTestCase { public void testResolveNoExpandStrict() { SearchRequest request = new SearchRequest("missing*"); request.indicesOptions(IndicesOptions.fromOptions(false, randomBoolean(), false, false)); - List indices = resolveIndices(request, buildAuthorizedIndices(user, SearchAction.NAME)).getLocal(); + List indices = resolveIndices(request, buildAuthorizedIndices(user, TransportSearchAction.TYPE.name())).getLocal(); String[] expectedIndices = new String[] { "missing*" }; assertThat(indices.toArray(new String[indices.size()]), equalTo(expectedIndices)); assertThat(request.indices(), equalTo(expectedIndices)); @@ -792,13 +800,13 @@ public class IndicesAndAliasesResolverTests extends ESTestCase { public void testResolveNoExpandIgnoreUnavailable() { SearchRequest request = new SearchRequest("missing*"); request.indicesOptions(IndicesOptions.fromOptions(true, true, false, false)); - assertNoIndices(request, resolveIndices(request, buildAuthorizedIndices(user, SearchAction.NAME))); + assertNoIndices(request, resolveIndices(request, buildAuthorizedIndices(user, TransportSearchAction.TYPE.name()))); } public void testSearchWithRemoteIndex() { SearchRequest request = new SearchRequest("remote:indexName"); request.indicesOptions(IndicesOptions.fromOptions(randomBoolean(), randomBoolean(), randomBoolean(), randomBoolean())); - final ResolvedIndices resolved = resolveIndices(request, buildAuthorizedIndices(user, SearchAction.NAME)); + final ResolvedIndices resolved = resolveIndices(request, buildAuthorizedIndices(user, TransportSearchAction.TYPE.name())); assertThat(resolved.getLocal(), emptyIterable()); assertThat(resolved.getRemote(), containsInAnyOrder("remote:indexName")); assertThat(request.indices(), arrayContaining("remote:indexName")); @@ -807,7 +815,7 @@ public class IndicesAndAliasesResolverTests extends ESTestCase { public void testSearchWithRemoteAndLocalIndices() { SearchRequest request = new SearchRequest("remote:indexName", "bar", "bar2"); request.indicesOptions(IndicesOptions.fromOptions(true, randomBoolean(), randomBoolean(), randomBoolean())); - final ResolvedIndices resolved = resolveIndices(request, buildAuthorizedIndices(user, SearchAction.NAME)); + final ResolvedIndices resolved = resolveIndices(request, buildAuthorizedIndices(user, TransportSearchAction.TYPE.name())); assertThat(resolved.getLocal(), containsInAnyOrder("bar")); assertThat(resolved.getRemote(), containsInAnyOrder("remote:indexName")); assertThat(request.indices(), arrayContainingInAnyOrder("remote:indexName", "bar")); @@ -816,7 +824,7 @@ public class IndicesAndAliasesResolverTests extends ESTestCase { public void testSearchWithRemoteAndLocalWildcards() { SearchRequest request = new SearchRequest("*:foo", "r*:bar*", "remote:baz*", "bar*", "foofoo"); request.indicesOptions(IndicesOptions.fromOptions(randomBoolean(), randomBoolean(), true, false)); - final AuthorizedIndices authorizedIndices = buildAuthorizedIndices(user, SearchAction.NAME); + final AuthorizedIndices authorizedIndices = buildAuthorizedIndices(user, TransportSearchAction.TYPE.name()); final ResolvedIndices resolved = resolveIndices(request, authorizedIndices); assertThat(resolved.getRemote(), containsInAnyOrder("remote:foo", "other_remote:foo", "remote:bar*", "remote:baz*")); assertThat(resolved.getLocal(), containsInAnyOrder("bar", "foofoo")); @@ -1503,7 +1511,7 @@ public class IndicesAndAliasesResolverTests extends ESTestCase { public void testRemotableRequestsAllowRemoteIndices() { IndicesOptions options = IndicesOptions.fromOptions(true, false, false, false); Tuple tuple = randomFrom( - new Tuple(new SearchRequest("remote:foo").indicesOptions(options), SearchAction.NAME), + new Tuple(new SearchRequest("remote:foo").indicesOptions(options), TransportSearchAction.TYPE.name()), new Tuple( new FieldCapabilitiesRequest().indices("remote:foo").indicesOptions(options), FieldCapabilitiesAction.NAME @@ -1555,7 +1563,10 @@ public class IndicesAndAliasesResolverTests extends ESTestCase { new MultiTermVectorsRequest(), new BulkRequest() ); - expectThrows(IllegalStateException.class, () -> resolveIndices(request, buildAuthorizedIndices(user, MultiSearchAction.NAME))); + expectThrows( + IllegalStateException.class, + () -> resolveIndices(request, buildAuthorizedIndices(user, TransportMultiSearchAction.TYPE.name())) + ); } public void testResolveAdminAction() { @@ -1581,7 +1592,10 @@ public class IndicesAndAliasesResolverTests extends ESTestCase { public void testXPackSecurityUserHasAccessToSecurityIndex() { SearchRequest request = new SearchRequest(); { - final AuthorizedIndices authorizedIndices = buildAuthorizedIndices(InternalUsers.XPACK_SECURITY_USER, SearchAction.NAME); + final AuthorizedIndices authorizedIndices = buildAuthorizedIndices( + InternalUsers.XPACK_SECURITY_USER, + TransportSearchAction.TYPE.name() + ); List indices = resolveIndices(request, authorizedIndices).getLocal(); assertThat(indices, hasItem(SECURITY_MAIN_ALIAS)); } @@ -1599,7 +1613,7 @@ public class IndicesAndAliasesResolverTests extends ESTestCase { public void testXPackUserDoesNotHaveAccessToSecurityIndex() { SearchRequest request = new SearchRequest(); - final AuthorizedIndices authorizedIndices = buildAuthorizedIndices(InternalUsers.XPACK_USER, SearchAction.NAME); + final AuthorizedIndices authorizedIndices = buildAuthorizedIndices(InternalUsers.XPACK_USER, TransportSearchAction.TYPE.name()); List indices = resolveIndices(request, authorizedIndices).getLocal(); assertThat(indices, not(hasItem(SECURITY_MAIN_ALIAS))); } @@ -1618,7 +1632,7 @@ public class IndicesAndAliasesResolverTests extends ESTestCase { { SearchRequest request = new SearchRequest(); - final AuthorizedIndices authorizedIndices = buildAuthorizedIndices(allAccessUser, SearchAction.NAME); + final AuthorizedIndices authorizedIndices = buildAuthorizedIndices(allAccessUser, TransportSearchAction.TYPE.name()); List indices = resolveIndices(request, authorizedIndices).getLocal(); assertThat(indices, not(hasItem(SECURITY_MAIN_ALIAS))); } @@ -1635,7 +1649,7 @@ public class IndicesAndAliasesResolverTests extends ESTestCase { public void testUnauthorizedDateMathExpressionIgnoreUnavailable() { SearchRequest request = new SearchRequest(""); request.indicesOptions(IndicesOptions.fromOptions(true, true, randomBoolean(), randomBoolean())); - assertNoIndices(request, resolveIndices(request, buildAuthorizedIndices(user, SearchAction.NAME))); + assertNoIndices(request, resolveIndices(request, buildAuthorizedIndices(user, TransportSearchAction.TYPE.name()))); } public void testUnauthorizedDateMathExpressionIgnoreUnavailableDisallowNoIndices() { @@ -1643,7 +1657,7 @@ public class IndicesAndAliasesResolverTests extends ESTestCase { request.indicesOptions(IndicesOptions.fromOptions(true, false, randomBoolean(), randomBoolean())); IndexNotFoundException e = expectThrows( IndexNotFoundException.class, - () -> resolveIndices(request, buildAuthorizedIndices(user, SearchAction.NAME)) + () -> resolveIndices(request, buildAuthorizedIndices(user, TransportSearchAction.TYPE.name())) ); assertEquals("no such index [[]]", e.getMessage()); } @@ -1653,7 +1667,7 @@ public class IndicesAndAliasesResolverTests extends ESTestCase { + DateTimeFormatter.ofPattern("uuuu.MM.dd", Locale.ROOT).format(ZonedDateTime.now(ZoneOffset.UTC).withDayOfMonth(1)); SearchRequest request = new SearchRequest(""); request.indicesOptions(IndicesOptions.fromOptions(false, randomBoolean(), randomBoolean(), randomBoolean())); - List indices = resolveIndices(request, buildAuthorizedIndices(user, SearchAction.NAME)).getLocal(); + List indices = resolveIndices(request, buildAuthorizedIndices(user, TransportSearchAction.TYPE.name())).getLocal(); assertThat(indices, contains(expectedIndex)); } @@ -1677,7 +1691,7 @@ public class IndicesAndAliasesResolverTests extends ESTestCase { final boolean expandIndicesOpen = Regex.isSimpleMatchPattern(pattern) ? true : randomBoolean(); request.indicesOptions(IndicesOptions.fromOptions(randomBoolean(), randomBoolean(), expandIndicesOpen, randomBoolean())); } - List indices = resolveIndices(request, buildAuthorizedIndices(user, SearchAction.NAME)).getLocal(); + List indices = resolveIndices(request, buildAuthorizedIndices(user, TransportSearchAction.TYPE.name())).getLocal(); assertThat(indices, hasSize(1)); assertThat(request.indices()[0], equalTo(dateTimeIndex)); } @@ -1685,7 +1699,7 @@ public class IndicesAndAliasesResolverTests extends ESTestCase { public void testMissingDateMathExpressionIgnoreUnavailable() { SearchRequest request = new SearchRequest(""); request.indicesOptions(IndicesOptions.fromOptions(true, true, randomBoolean(), randomBoolean())); - assertNoIndices(request, resolveIndices(request, buildAuthorizedIndices(user, SearchAction.NAME))); + assertNoIndices(request, resolveIndices(request, buildAuthorizedIndices(user, TransportSearchAction.TYPE.name()))); } public void testMissingDateMathExpressionIgnoreUnavailableDisallowNoIndices() { @@ -1693,7 +1707,7 @@ public class IndicesAndAliasesResolverTests extends ESTestCase { request.indicesOptions(IndicesOptions.fromOptions(true, false, randomBoolean(), randomBoolean())); IndexNotFoundException e = expectThrows( IndexNotFoundException.class, - () -> resolveIndices(request, buildAuthorizedIndices(user, SearchAction.NAME)) + () -> resolveIndices(request, buildAuthorizedIndices(user, TransportSearchAction.TYPE.name())) ); assertEquals("no such index [[]]", e.getMessage()); } @@ -1703,7 +1717,7 @@ public class IndicesAndAliasesResolverTests extends ESTestCase { + DateTimeFormatter.ofPattern("uuuu.MM.dd", Locale.ROOT).format(ZonedDateTime.now(ZoneOffset.UTC).withDayOfMonth(1)); SearchRequest request = new SearchRequest(""); request.indicesOptions(IndicesOptions.fromOptions(false, randomBoolean(), randomBoolean(), randomBoolean())); - List indices = resolveIndices(request, buildAuthorizedIndices(user, SearchAction.NAME)).getLocal(); + List indices = resolveIndices(request, buildAuthorizedIndices(user, TransportSearchAction.TYPE.name())).getLocal(); assertThat(indices, contains(expectedIndex)); } @@ -1774,9 +1788,9 @@ public class IndicesAndAliasesResolverTests extends ESTestCase { public void testHiddenIndicesResolution() { SearchRequest searchRequest = new SearchRequest(); searchRequest.indicesOptions(IndicesOptions.fromOptions(false, false, true, true, true)); - AuthorizedIndices authorizedIndices = buildAuthorizedIndices(user, SearchAction.NAME); + AuthorizedIndices authorizedIndices = buildAuthorizedIndices(user, TransportSearchAction.TYPE.name()); ResolvedIndices resolvedIndices = defaultIndicesResolver.resolveIndicesAndAliases( - SearchAction.NAME, + TransportSearchAction.TYPE.name(), searchRequest, metadata, authorizedIndices @@ -1803,7 +1817,12 @@ public class IndicesAndAliasesResolverTests extends ESTestCase { // open + hidden searchRequest = new SearchRequest(); searchRequest.indicesOptions(IndicesOptions.fromOptions(false, false, true, false, true)); - resolvedIndices = defaultIndicesResolver.resolveIndicesAndAliases(SearchAction.NAME, searchRequest, metadata, authorizedIndices); + resolvedIndices = defaultIndicesResolver.resolveIndicesAndAliases( + TransportSearchAction.TYPE.name(), + searchRequest, + metadata, + authorizedIndices + ); assertThat( resolvedIndices.getLocal(), containsInAnyOrder( @@ -1822,52 +1841,77 @@ public class IndicesAndAliasesResolverTests extends ESTestCase { // open + implicit hidden for . indices searchRequest = new SearchRequest(randomFrom(".h*", ".hid*")); searchRequest.indicesOptions(IndicesOptions.fromOptions(false, false, true, false, false)); - authorizedIndices = buildAuthorizedIndices(user, SearchAction.NAME); - resolvedIndices = defaultIndicesResolver.resolveIndicesAndAliases(SearchAction.NAME, searchRequest, metadata, authorizedIndices); + authorizedIndices = buildAuthorizedIndices(user, TransportSearchAction.TYPE.name()); + resolvedIndices = defaultIndicesResolver.resolveIndicesAndAliases( + TransportSearchAction.TYPE.name(), + searchRequest, + metadata, + authorizedIndices + ); assertThat(resolvedIndices.getLocal(), containsInAnyOrder(".hidden-open")); assertThat(resolvedIndices.getRemote(), emptyIterable()); // closed + hidden, ignore aliases searchRequest = new SearchRequest(); searchRequest.indicesOptions(IndicesOptions.fromOptions(false, false, false, true, true, true, false, true, false)); - authorizedIndices = buildAuthorizedIndices(user, SearchAction.NAME); - resolvedIndices = defaultIndicesResolver.resolveIndicesAndAliases(SearchAction.NAME, searchRequest, metadata, authorizedIndices); + authorizedIndices = buildAuthorizedIndices(user, TransportSearchAction.TYPE.name()); + resolvedIndices = defaultIndicesResolver.resolveIndicesAndAliases( + TransportSearchAction.TYPE.name(), + searchRequest, + metadata, + authorizedIndices + ); assertThat(resolvedIndices.getLocal(), containsInAnyOrder("bar-closed", "foofoo-closed", "hidden-closed", ".hidden-closed")); assertThat(resolvedIndices.getRemote(), emptyIterable()); // closed + implicit hidden for . indices searchRequest = new SearchRequest(randomFrom(".h*", ".hid*")); searchRequest.indicesOptions(IndicesOptions.fromOptions(false, false, false, true, false)); - authorizedIndices = buildAuthorizedIndices(user, SearchAction.NAME); - resolvedIndices = defaultIndicesResolver.resolveIndicesAndAliases(SearchAction.NAME, searchRequest, metadata, authorizedIndices); + authorizedIndices = buildAuthorizedIndices(user, TransportSearchAction.TYPE.name()); + resolvedIndices = defaultIndicesResolver.resolveIndicesAndAliases( + TransportSearchAction.TYPE.name(), + searchRequest, + metadata, + authorizedIndices + ); assertThat(resolvedIndices.getLocal(), containsInAnyOrder(".hidden-closed")); assertThat(resolvedIndices.getRemote(), emptyIterable()); // allow no indices, do not expand to open or closed, expand hidden, ignore aliases searchRequest = new SearchRequest(); searchRequest.indicesOptions(IndicesOptions.fromOptions(false, true, false, false, false, true, false, true, false)); - authorizedIndices = buildAuthorizedIndices(user, SearchAction.NAME); - resolvedIndices = defaultIndicesResolver.resolveIndicesAndAliases(SearchAction.NAME, searchRequest, metadata, authorizedIndices); + authorizedIndices = buildAuthorizedIndices(user, TransportSearchAction.TYPE.name()); + resolvedIndices = defaultIndicesResolver.resolveIndicesAndAliases( + TransportSearchAction.TYPE.name(), + searchRequest, + metadata, + authorizedIndices + ); assertThat(resolvedIndices.getLocal(), contains("-*")); assertThat(resolvedIndices.getRemote(), emptyIterable()); // date math with default indices options searchRequest = new SearchRequest(""); - authorizedIndices = buildAuthorizedIndices(user, SearchAction.NAME); - resolvedIndices = defaultIndicesResolver.resolveIndicesAndAliases(SearchAction.NAME, searchRequest, metadata, authorizedIndices); + authorizedIndices = buildAuthorizedIndices(user, TransportSearchAction.TYPE.name()); + resolvedIndices = defaultIndicesResolver.resolveIndicesAndAliases( + TransportSearchAction.TYPE.name(), + searchRequest, + metadata, + authorizedIndices + ); assertThat(resolvedIndices.getLocal(), contains(oneOf("date-hidden-" + todaySuffix, "date-hidden-" + tomorrowSuffix))); assertThat(resolvedIndices.getRemote(), emptyIterable()); } public void testHiddenAliasesResolution() { final User user = new User("hidden-alias-tester", "hidden_alias_test"); - final AuthorizedIndices authorizedIndices = buildAuthorizedIndices(user, SearchAction.NAME); + final AuthorizedIndices authorizedIndices = buildAuthorizedIndices(user, TransportSearchAction.TYPE.name()); // Visible only SearchRequest searchRequest = new SearchRequest(); searchRequest.indicesOptions(IndicesOptions.fromOptions(false, false, true, false, false)); ResolvedIndices resolvedIndices = defaultIndicesResolver.resolveIndicesAndAliases( - SearchAction.NAME, + TransportSearchAction.TYPE.name(), searchRequest, metadata, authorizedIndices @@ -1878,7 +1922,12 @@ public class IndicesAndAliasesResolverTests extends ESTestCase { // Include hidden explicitly searchRequest = new SearchRequest(); searchRequest.indicesOptions(IndicesOptions.fromOptions(false, false, true, false, true)); - resolvedIndices = defaultIndicesResolver.resolveIndicesAndAliases(SearchAction.NAME, searchRequest, metadata, authorizedIndices); + resolvedIndices = defaultIndicesResolver.resolveIndicesAndAliases( + TransportSearchAction.TYPE.name(), + searchRequest, + metadata, + authorizedIndices + ); assertThat( resolvedIndices.getLocal(), containsInAnyOrder( @@ -1896,28 +1945,48 @@ public class IndicesAndAliasesResolverTests extends ESTestCase { // Include hidden with a wildcard searchRequest = new SearchRequest("alias-h*"); searchRequest.indicesOptions(IndicesOptions.fromOptions(false, false, true, false, true)); - resolvedIndices = defaultIndicesResolver.resolveIndicesAndAliases(SearchAction.NAME, searchRequest, metadata, authorizedIndices); + resolvedIndices = defaultIndicesResolver.resolveIndicesAndAliases( + TransportSearchAction.TYPE.name(), + searchRequest, + metadata, + authorizedIndices + ); assertThat(resolvedIndices.getLocal(), containsInAnyOrder("alias-hidden", "alias-hidden-datemath-" + todaySuffix)); assertThat(resolvedIndices.getRemote(), emptyIterable()); // Dot prefix, implicitly including hidden searchRequest = new SearchRequest(".a*"); searchRequest.indicesOptions(IndicesOptions.fromOptions(false, false, true, false, false)); - resolvedIndices = defaultIndicesResolver.resolveIndicesAndAliases(SearchAction.NAME, searchRequest, metadata, authorizedIndices); + resolvedIndices = defaultIndicesResolver.resolveIndicesAndAliases( + TransportSearchAction.TYPE.name(), + searchRequest, + metadata, + authorizedIndices + ); assertThat(resolvedIndices.getLocal(), containsInAnyOrder(".alias-hidden", ".alias-hidden-datemath-" + todaySuffix)); assertThat(resolvedIndices.getRemote(), emptyIterable()); // Make sure ignoring aliases works (visible only) searchRequest = new SearchRequest(); searchRequest.indicesOptions(IndicesOptions.fromOptions(false, true, true, false, false, true, false, true, false)); - resolvedIndices = defaultIndicesResolver.resolveIndicesAndAliases(SearchAction.NAME, searchRequest, metadata, authorizedIndices); + resolvedIndices = defaultIndicesResolver.resolveIndicesAndAliases( + TransportSearchAction.TYPE.name(), + searchRequest, + metadata, + authorizedIndices + ); assertThat(resolvedIndices.getLocal(), contains("-*")); assertThat(resolvedIndices.getRemote(), emptyIterable()); // Make sure ignoring aliases works (including hidden) searchRequest = new SearchRequest(); searchRequest.indicesOptions(IndicesOptions.fromOptions(false, false, true, false, true, true, false, true, false)); - resolvedIndices = defaultIndicesResolver.resolveIndicesAndAliases(SearchAction.NAME, searchRequest, metadata, authorizedIndices); + resolvedIndices = defaultIndicesResolver.resolveIndicesAndAliases( + TransportSearchAction.TYPE.name(), + searchRequest, + metadata, + authorizedIndices + ); assertThat(resolvedIndices.getLocal(), containsInAnyOrder("hidden-open")); assertThat(resolvedIndices.getRemote(), emptyIterable()); } @@ -1930,9 +1999,9 @@ public class IndicesAndAliasesResolverTests extends ESTestCase { SearchRequest searchRequest = new SearchRequest(); searchRequest.indices("logs-*"); searchRequest.indicesOptions(IndicesOptions.fromOptions(false, false, true, false, false, true, true, true, true)); - final AuthorizedIndices authorizedIndices = buildAuthorizedIndices(user, SearchAction.NAME, searchRequest); + final AuthorizedIndices authorizedIndices = buildAuthorizedIndices(user, TransportSearchAction.TYPE.name(), searchRequest); ResolvedIndices resolvedIndices = defaultIndicesResolver.resolveIndicesAndAliases( - SearchAction.NAME, + TransportSearchAction.TYPE.name(), searchRequest, metadata, authorizedIndices @@ -1945,7 +2014,7 @@ public class IndicesAndAliasesResolverTests extends ESTestCase { searchRequest.indices("logs-*"); searchRequest.indicesOptions(IndicesOptions.fromOptions(false, true, true, false, false, true, true, true, true)); resolvedIndices = defaultIndicesResolver.resolveIndicesAndAliases( - SearchAction.NAME, + TransportSearchAction.TYPE.name(), searchRequest, metadata, authorizedIndices @@ -1961,9 +2030,9 @@ public class IndicesAndAliasesResolverTests extends ESTestCase { SearchRequest searchRequest = new SearchRequest(); searchRequest.indices("logs-*"); searchRequest.indicesOptions(IndicesOptions.fromOptions(false, false, true, false, false, true, true, true, true)); - final AuthorizedIndices authorizedIndices = buildAuthorizedIndices(user, SearchAction.NAME, searchRequest); + final AuthorizedIndices authorizedIndices = buildAuthorizedIndices(user, TransportSearchAction.TYPE.name(), searchRequest); ResolvedIndices resolvedIndices = defaultIndicesResolver.resolveIndicesAndAliases( - SearchAction.NAME, + TransportSearchAction.TYPE.name(), searchRequest, metadata, authorizedIndices @@ -2055,7 +2124,7 @@ public class IndicesAndAliasesResolverTests extends ESTestCase { // data streams and their backing indices should be in the authorized list List expectedDataStreams = List.of("logs-foo", "logs-foobar"); - final AuthorizedIndices authorizedIndices = buildAuthorizedIndices(user, SearchAction.NAME, request); + final AuthorizedIndices authorizedIndices = buildAuthorizedIndices(user, TransportSearchAction.TYPE.name(), request); for (String dsName : expectedDataStreams) { DataStream dataStream = metadata.dataStreams().get(dsName); assertThat(authorizedIndices.all().get(), hasItem(dsName)); @@ -2069,7 +2138,7 @@ public class IndicesAndAliasesResolverTests extends ESTestCase { // data streams without their backing indices will be in the resolved list since the backing indices do not match the requested // pattern ResolvedIndices resolvedIndices = defaultIndicesResolver.resolveIndicesAndAliases( - SearchAction.NAME, + TransportSearchAction.TYPE.name(), request, metadata, authorizedIndices @@ -2097,7 +2166,7 @@ public class IndicesAndAliasesResolverTests extends ESTestCase { assertThat(request, instanceOf(IndicesRequest.Replaceable.class)); assertThat(request.includeDataStreams(), is(true)); - final AuthorizedIndices authorizedIndices = buildAuthorizedIndices(user, SearchAction.NAME, request); + final AuthorizedIndices authorizedIndices = buildAuthorizedIndices(user, TransportSearchAction.TYPE.name(), request); // data streams and their backing indices should be in the authorized list assertThat(authorizedIndices.all().get(), hasItem(dataStreamName)); assertThat(authorizedIndices.check(dataStreamName), is(true)); @@ -2107,7 +2176,7 @@ public class IndicesAndAliasesResolverTests extends ESTestCase { } ResolvedIndices resolvedIndices = defaultIndicesResolver.resolveIndicesAndAliases( - SearchAction.NAME, + TransportSearchAction.TYPE.name(), request, metadata, authorizedIndices @@ -2128,7 +2197,7 @@ public class IndicesAndAliasesResolverTests extends ESTestCase { // data streams and their backing indices should be included in the authorized list List expectedDataStreams = List.of("logs-foo", "logs-foobar"); - final AuthorizedIndices authorizedIndices = buildAuthorizedIndices(user, SearchAction.NAME, request); + final AuthorizedIndices authorizedIndices = buildAuthorizedIndices(user, TransportSearchAction.TYPE.name(), request); for (String dsName : expectedDataStreams) { DataStream dataStream = metadata.dataStreams().get(dsName); assertThat(authorizedIndices.all().get(), hasItem(dsName)); @@ -2142,7 +2211,7 @@ public class IndicesAndAliasesResolverTests extends ESTestCase { // data streams should _not_ be included in the resolved list because they do not match the pattern but their backing indices // should be in the resolved list because they match the pattern and are authorized via extension from their parent data stream ResolvedIndices resolvedIndices = defaultIndicesResolver.resolveIndicesAndAliases( - SearchAction.NAME, + TransportSearchAction.TYPE.name(), request, metadata, authorizedIndices @@ -2214,7 +2283,7 @@ public class IndicesAndAliasesResolverTests extends ESTestCase { // only the backing indices will be in the resolved list since the request does not support data streams // but the backing indices match the requested pattern ResolvedIndices resolvedIndices = defaultIndicesResolver.resolveIndicesAndAliases( - SearchAction.NAME, + TransportSearchAction.TYPE.name(), request, metadata, authorizedIndices @@ -2243,7 +2312,7 @@ public class IndicesAndAliasesResolverTests extends ESTestCase { // only the single backing index will be in the resolved list since the request does not support data streams // but one of the backing indices matched the requested pattern ResolvedIndices resolvedIndices = defaultIndicesResolver.resolveIndicesAndAliases( - SearchAction.NAME, + TransportSearchAction.TYPE.name(), request, metadata, authorizedIndices @@ -2326,9 +2395,9 @@ public class IndicesAndAliasesResolverTests extends ESTestCase { randomBoolean(), null ); - final AuthorizedIndices authorizedIndices = buildAuthorizedIndices(user, SearchShardsAction.NAME, request); + final AuthorizedIndices authorizedIndices = buildAuthorizedIndices(user, TransportSearchShardsAction.TYPE.name(), request); final ResolvedIndices resolvedIndices = defaultIndicesResolver.resolveIndicesAndAliases( - SearchShardsAction.NAME, + TransportSearchShardsAction.TYPE.name(), request, metadata, authorizedIndices @@ -2348,9 +2417,9 @@ public class IndicesAndAliasesResolverTests extends ESTestCase { randomBoolean(), null ); - final AuthorizedIndices authorizedIndices = buildAuthorizedIndices(user, SearchAction.NAME, request); + final AuthorizedIndices authorizedIndices = buildAuthorizedIndices(user, TransportSearchAction.TYPE.name(), request); final ResolvedIndices resolvedIndices = defaultIndicesResolver.resolveIndicesAndAliases( - SearchShardsAction.NAME, + TransportSearchShardsAction.TYPE.name(), request, metadata, authorizedIndices diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/LoadAuthorizedIndicesTimeCheckerTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/LoadAuthorizedIndicesTimeCheckerTests.java index 1e669417c02e..ed9250cb8282 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/LoadAuthorizedIndicesTimeCheckerTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/LoadAuthorizedIndicesTimeCheckerTests.java @@ -10,8 +10,8 @@ package org.elasticsearch.xpack.security.authz; import org.apache.logging.log4j.Level; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchRequest; +import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.common.logging.Loggers; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; @@ -135,7 +135,7 @@ public class LoadAuthorizedIndicesTimeCheckerTests extends ESTestCase { "WARN-Slow Index Resolution", timerLogger.getName(), Level.WARN, - Pattern.quote("Resolving [0] indices for action [" + SearchAction.NAME + "] and user [slow-user] took [") + Pattern.quote("Resolving [0] indices for action [" + TransportSearchAction.TYPE.name() + "] and user [slow-user] took [") + "\\d{3}" + Pattern.quote( "ms] which is greater than the threshold of " @@ -163,7 +163,7 @@ public class LoadAuthorizedIndicesTimeCheckerTests extends ESTestCase { Level.INFO, Pattern.quote("Took [") + "\\d{2,3}" - + Pattern.quote("ms] to resolve [0] indices for action [" + SearchAction.NAME + "] and user [slow-user]") + + Pattern.quote("ms] to resolve [0] indices for action [" + TransportSearchAction.TYPE.name() + "] and user [slow-user]") ); testLogging(thresholds, elapsedMs, expectation); @@ -182,7 +182,7 @@ public class LoadAuthorizedIndicesTimeCheckerTests extends ESTestCase { final AuthorizationEngine.RequestInfo requestInfo = new AuthorizationEngine.RequestInfo( authentication, new SearchRequest(), - SearchAction.NAME, + TransportSearchAction.TYPE.name(), null ); diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/PreAuthorizationUtilsTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/PreAuthorizationUtilsTests.java index 866626e7d01f..ff92b7a1e7dc 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/PreAuthorizationUtilsTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/PreAuthorizationUtilsTests.java @@ -7,8 +7,8 @@ package org.elasticsearch.xpack.security.authz; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchRequest; +import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.test.ESTestCase; @@ -39,7 +39,7 @@ import static org.hamcrest.Matchers.nullValue; public class PreAuthorizationUtilsTests extends ESTestCase { public void testMaybeSkipChildrenActionAuthorizationAddsParentAuthorizationHeader() { - String action = SearchAction.NAME; + String action = TransportSearchAction.TYPE.name(); Role role = Role.builder(RESTRICTED_INDICES, "test-role").add(IndexPrivilege.READ, "test-*").build(); @@ -64,7 +64,7 @@ public class PreAuthorizationUtilsTests extends ESTestCase { } public void testShouldRemoveParentAuthorizationFromThreadContext() { - final String parentAction = SearchAction.NAME; + final String parentAction = TransportSearchAction.TYPE.name(); SecurityContext securityContextWithParentAuthorization = new SecurityContext(Settings.EMPTY, new ThreadContext(Settings.EMPTY)); securityContextWithParentAuthorization.setParentAuthorization(new ParentActionAuthorization(parentAction)); @@ -113,7 +113,7 @@ public class PreAuthorizationUtilsTests extends ESTestCase { } public void testShouldPreAuthorizeChildByParentAction() { - final String parentAction = SearchAction.NAME; + final String parentAction = TransportSearchAction.TYPE.name(); final String childAction = randomWhitelistedChildAction(parentAction); ParentActionAuthorization parentAuthorization = new ParentActionAuthorization(parentAction); @@ -130,7 +130,7 @@ public class PreAuthorizationUtilsTests extends ESTestCase { } public void testShouldPreAuthorizeChildByParentActionWhenParentAndChildAreSame() { - final String parentAction = SearchAction.NAME; + final String parentAction = TransportSearchAction.TYPE.name(); final String childAction = parentAction; ParentActionAuthorization parentAuthorization = new ParentActionAuthorization(parentAction); diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/RBACEngineTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/RBACEngineTests.java index 2420d7c2269a..251b692f4282 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/RBACEngineTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/RBACEngineTests.java @@ -16,8 +16,8 @@ import org.elasticsearch.action.admin.indices.mapping.put.PutMappingAction; import org.elasticsearch.action.admin.indices.mapping.put.PutMappingRequest; import org.elasticsearch.action.delete.DeleteAction; import org.elasticsearch.action.index.IndexAction; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchRequest; +import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.client.internal.Client; import org.elasticsearch.client.internal.ElasticsearchClient; @@ -1385,7 +1385,7 @@ public class RBACEngineTests extends ESTestCase { SearchRequest request = new SearchRequest("*"); AuthorizedIndices authorizedIndices = RBACEngine.resolveAuthorizedIndicesFromRole( role, - getRequestInfo(request, SearchAction.NAME), + getRequestInfo(request, TransportSearchAction.TYPE.name()), lookup, () -> ignore -> {} ); @@ -1773,8 +1773,10 @@ public class RBACEngineTests extends ESTestCase { final String[] indices = { "test-index" }; final Role role = Mockito.spy(Role.builder(RESTRICTED_INDICES, "test-role").add(IndexPrivilege.READ, indices).build()); - final String action = randomFrom(PreAuthorizationUtils.CHILD_ACTIONS_PRE_AUTHORIZED_BY_PARENT.get(SearchAction.NAME)); - final ParentActionAuthorization parentAuthorization = new ParentActionAuthorization(SearchAction.NAME); + final String action = randomFrom( + PreAuthorizationUtils.CHILD_ACTIONS_PRE_AUTHORIZED_BY_PARENT.get(TransportSearchAction.TYPE.name()) + ); + final ParentActionAuthorization parentAuthorization = new ParentActionAuthorization(TransportSearchAction.TYPE.name()); authorizeIndicesAction(indices, role, action, parentAuthorization, new ActionListener() { @Override @@ -1796,7 +1798,9 @@ public class RBACEngineTests extends ESTestCase { final String[] indices = { "test-index" }; final Role role = Mockito.spy(Role.builder(RESTRICTED_INDICES, "test-role").add(IndexPrivilege.READ, indices).build()); - final String action = randomFrom(PreAuthorizationUtils.CHILD_ACTIONS_PRE_AUTHORIZED_BY_PARENT.get(SearchAction.NAME)); + final String action = randomFrom( + PreAuthorizationUtils.CHILD_ACTIONS_PRE_AUTHORIZED_BY_PARENT.get(TransportSearchAction.TYPE.name()) + ); final ParentActionAuthorization parentAuthorization = null; authorizeIndicesAction(indices, role, action, parentAuthorization, new ActionListener() { @@ -1830,8 +1834,10 @@ public class RBACEngineTests extends ESTestCase { .build() ); - final String action = randomFrom(PreAuthorizationUtils.CHILD_ACTIONS_PRE_AUTHORIZED_BY_PARENT.get(SearchAction.NAME)); - final ParentActionAuthorization parentAuthorization = new ParentActionAuthorization(SearchAction.NAME); + final String action = randomFrom( + PreAuthorizationUtils.CHILD_ACTIONS_PRE_AUTHORIZED_BY_PARENT.get(TransportSearchAction.TYPE.name()) + ); + final ParentActionAuthorization parentAuthorization = new ParentActionAuthorization(TransportSearchAction.TYPE.name()); authorizeIndicesAction(indices, role, action, parentAuthorization, new ActionListener() { @Override @@ -1852,8 +1858,8 @@ public class RBACEngineTests extends ESTestCase { final String[] indices = { "test-index" }; final Role role = Mockito.spy(Role.builder(RESTRICTED_INDICES, "test-role").add(IndexPrivilege.READ, indices).build()); - final String action = SearchAction.NAME + "[" + randomAlphaOfLength(3) + "]"; - final ParentActionAuthorization parentAuthorization = new ParentActionAuthorization(SearchAction.NAME); + final String action = TransportSearchAction.TYPE.name() + "[" + randomAlphaOfLength(3) + "]"; + final ParentActionAuthorization parentAuthorization = new ParentActionAuthorization(TransportSearchAction.TYPE.name()); authorizeIndicesAction(indices, role, action, parentAuthorization, new ActionListener() { @Override diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/accesscontrol/IndicesPermissionTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/accesscontrol/IndicesPermissionTests.java index aa89cdad6eb7..afc1d0931547 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/accesscontrol/IndicesPermissionTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/accesscontrol/IndicesPermissionTests.java @@ -10,7 +10,7 @@ import org.elasticsearch.ElasticsearchSecurityException; import org.elasticsearch.TransportVersion; import org.elasticsearch.action.admin.indices.mapping.put.AutoPutMappingAction; import org.elasticsearch.action.admin.indices.mapping.put.PutMappingAction; -import org.elasticsearch.action.search.SearchAction; +import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.cluster.metadata.AliasMetadata; import org.elasticsearch.cluster.metadata.DataStream; import org.elasticsearch.cluster.metadata.DataStreamTestHelper; @@ -74,7 +74,12 @@ public class IndicesPermissionTests extends ESTestCase { Role role = Role.builder(RESTRICTED_INDICES, "_role") .add(new FieldPermissions(fieldPermissionDef(fields, null)), query, IndexPrivilege.ALL, randomBoolean(), "_index") .build(); - IndicesAccessControl permissions = role.authorize(SearchAction.NAME, Sets.newHashSet("_index"), lookup, fieldPermissionsCache); + IndicesAccessControl permissions = role.authorize( + TransportSearchAction.TYPE.name(), + Sets.newHashSet("_index"), + lookup, + fieldPermissionsCache + ); assertThat(permissions.getIndexPermissions("_index"), notNullValue()); assertTrue(permissions.getIndexPermissions("_index").getFieldPermissions().grantsAccessTo("_field")); assertTrue(permissions.getIndexPermissions("_index").getFieldPermissions().hasFieldLevelSecurity()); @@ -86,7 +91,7 @@ public class IndicesPermissionTests extends ESTestCase { role = Role.builder(RESTRICTED_INDICES, "_role") .add(new FieldPermissions(fieldPermissionDef(fields, null)), null, IndexPrivilege.ALL, randomBoolean(), "_index") .build(); - permissions = role.authorize(SearchAction.NAME, Sets.newHashSet("_index"), lookup, fieldPermissionsCache); + permissions = role.authorize(TransportSearchAction.TYPE.name(), Sets.newHashSet("_index"), lookup, fieldPermissionsCache); assertThat(permissions.getIndexPermissions("_index"), notNullValue()); assertTrue(permissions.getIndexPermissions("_index").getFieldPermissions().grantsAccessTo("_field")); assertTrue(permissions.getIndexPermissions("_index").getFieldPermissions().hasFieldLevelSecurity()); @@ -97,7 +102,7 @@ public class IndicesPermissionTests extends ESTestCase { role = Role.builder(RESTRICTED_INDICES, "_role") .add(FieldPermissions.DEFAULT, query, IndexPrivilege.ALL, randomBoolean(), "_index") .build(); - permissions = role.authorize(SearchAction.NAME, Sets.newHashSet("_index"), lookup, fieldPermissionsCache); + permissions = role.authorize(TransportSearchAction.TYPE.name(), Sets.newHashSet("_index"), lookup, fieldPermissionsCache); assertThat(permissions.getIndexPermissions("_index"), notNullValue()); assertFalse(permissions.getIndexPermissions("_index").getFieldPermissions().hasFieldLevelSecurity()); assertThat(permissions.getIndexPermissions("_index").getDocumentPermissions().hasDocumentLevelPermissions(), is(true)); @@ -108,7 +113,7 @@ public class IndicesPermissionTests extends ESTestCase { role = Role.builder(RESTRICTED_INDICES, "_role") .add(new FieldPermissions(fieldPermissionDef(fields, null)), query, IndexPrivilege.ALL, randomBoolean(), "_alias") .build(); - permissions = role.authorize(SearchAction.NAME, Sets.newHashSet("_alias"), lookup, fieldPermissionsCache); + permissions = role.authorize(TransportSearchAction.TYPE.name(), Sets.newHashSet("_alias"), lookup, fieldPermissionsCache); assertThat(permissions.getIndexPermissions("_index"), notNullValue()); assertTrue(permissions.getIndexPermissions("_index").getFieldPermissions().grantsAccessTo("_field")); assertTrue(permissions.getIndexPermissions("_index").getFieldPermissions().hasFieldLevelSecurity()); @@ -132,7 +137,7 @@ public class IndicesPermissionTests extends ESTestCase { role = Role.builder(RESTRICTED_INDICES, "_role") .add(new FieldPermissions(fieldPermissionDef(allFields, null)), query, IndexPrivilege.ALL, randomBoolean(), "_alias") .build(); - permissions = role.authorize(SearchAction.NAME, Sets.newHashSet("_alias"), lookup, fieldPermissionsCache); + permissions = role.authorize(TransportSearchAction.TYPE.name(), Sets.newHashSet("_alias"), lookup, fieldPermissionsCache); assertThat(permissions.getIndexPermissions("_index"), notNullValue()); assertFalse(permissions.getIndexPermissions("_index").getFieldPermissions().hasFieldLevelSecurity()); assertThat(permissions.getIndexPermissions("_index").getDocumentPermissions().hasDocumentLevelPermissions(), is(true)); @@ -158,7 +163,7 @@ public class IndicesPermissionTests extends ESTestCase { .add(new FieldPermissions(fieldPermissionDef(allFields, null)), fooQuery, IndexPrivilege.ALL, randomBoolean(), "_alias") .add(new FieldPermissions(fieldPermissionDef(allFields, null)), query, IndexPrivilege.ALL, randomBoolean(), "_alias") .build(); - permissions = role.authorize(SearchAction.NAME, Sets.newHashSet("_alias"), lookup, fieldPermissionsCache); + permissions = role.authorize(TransportSearchAction.TYPE.name(), Sets.newHashSet("_alias"), lookup, fieldPermissionsCache); Set bothQueries = Sets.union(fooQuery, query); assertThat(permissions.getIndexPermissions("_index"), notNullValue()); assertFalse(permissions.getIndexPermissions("_index").getFieldPermissions().hasFieldLevelSecurity()); @@ -194,7 +199,12 @@ public class IndicesPermissionTests extends ESTestCase { .add(new FieldPermissions(fieldPermissionDef(fields, null)), query, IndexPrivilege.ALL, randomBoolean(), "_index") .add(new FieldPermissions(fieldPermissionDef(null, null)), null, IndexPrivilege.ALL, randomBoolean(), "*") .build(); - IndicesAccessControl permissions = role.authorize(SearchAction.NAME, Sets.newHashSet("_index"), lookup, fieldPermissionsCache); + IndicesAccessControl permissions = role.authorize( + TransportSearchAction.TYPE.name(), + Sets.newHashSet("_index"), + lookup, + fieldPermissionsCache + ); assertThat(permissions.getIndexPermissions("_index"), notNullValue()); assertTrue(permissions.getIndexPermissions("_index").getFieldPermissions().grantsAccessTo("_field")); assertFalse(permissions.getIndexPermissions("_index").getFieldPermissions().hasFieldLevelSecurity()); @@ -261,13 +271,18 @@ public class IndicesPermissionTests extends ESTestCase { "a1" ) .build(); - IndicesAccessControl iac = core.authorize(SearchAction.NAME, Sets.newHashSet("a1", "ba"), lookup, fieldPermissionsCache); + IndicesAccessControl iac = core.authorize( + TransportSearchAction.TYPE.name(), + Sets.newHashSet("a1", "ba"), + lookup, + fieldPermissionsCache + ); assertTrue(iac.getIndexPermissions("a1").getFieldPermissions().grantsAccessTo("denied_field")); assertTrue(iac.getIndexPermissions("a1").getFieldPermissions().grantsAccessTo(randomAlphaOfLength(5))); // did not define anything for ba so we allow all assertFalse(iac.hasIndexPermissions("ba")); - assertTrue(core.check(SearchAction.NAME)); + assertTrue(core.check(TransportSearchAction.TYPE.name())); assertTrue(core.check(PutMappingAction.NAME)); assertTrue(core.check(AutoPutMappingAction.NAME)); assertFalse(core.check("unknown")); @@ -302,7 +317,7 @@ public class IndicesPermissionTests extends ESTestCase { "a2" ) .build(); - iac = core.authorize(SearchAction.NAME, Sets.newHashSet("a1", "a2"), lookup, fieldPermissionsCache); + iac = core.authorize(TransportSearchAction.TYPE.name(), Sets.newHashSet("a1", "a2"), lookup, fieldPermissionsCache); assertFalse(iac.getIndexPermissions("a1").getFieldPermissions().hasFieldLevelSecurity()); assertFalse(iac.getIndexPermissions("a2").getFieldPermissions().grantsAccessTo("denied_field2")); assertFalse(iac.getIndexPermissions("a2").getFieldPermissions().grantsAccessTo("denied_field")); @@ -310,7 +325,7 @@ public class IndicesPermissionTests extends ESTestCase { assertTrue(iac.getIndexPermissions("a2").getFieldPermissions().grantsAccessTo(randomAlphaOfLength(5) + "_field2")); assertTrue(iac.getIndexPermissions("a2").getFieldPermissions().hasFieldLevelSecurity()); - assertTrue(core.check(SearchAction.NAME)); + assertTrue(core.check(TransportSearchAction.TYPE.name())); assertTrue(core.check(PutMappingAction.NAME)); assertTrue(core.check(AutoPutMappingAction.NAME)); assertFalse(core.check("unknown")); @@ -364,7 +379,7 @@ public class IndicesPermissionTests extends ESTestCase { "*" ).build(); IndicesAccessControl iac = indicesPermission.authorize( - SearchAction.NAME, + TransportSearchAction.TYPE.name(), Sets.newHashSet(internalSecurityIndex, SecuritySystemIndices.SECURITY_MAIN_ALIAS), lookup, fieldPermissionsCache @@ -384,7 +399,7 @@ public class IndicesPermissionTests extends ESTestCase { "*" ).build(); iac = indicesPermission.authorize( - SearchAction.NAME, + TransportSearchAction.TYPE.name(), Sets.newHashSet(internalSecurityIndex, SecuritySystemIndices.SECURITY_MAIN_ALIAS), lookup, fieldPermissionsCache @@ -415,7 +430,7 @@ public class IndicesPermissionTests extends ESTestCase { "*" ).build(); IndicesAccessControl iac = indicesPermission.authorize( - SearchAction.NAME, + TransportSearchAction.TYPE.name(), Sets.newHashSet(asyncSearchIndex), lookup, fieldPermissionsCache @@ -432,7 +447,12 @@ public class IndicesPermissionTests extends ESTestCase { true, "*" ).build(); - iac = indicesPermission.authorize(SearchAction.NAME, Sets.newHashSet(asyncSearchIndex), lookup, fieldPermissionsCache); + iac = indicesPermission.authorize( + TransportSearchAction.TYPE.name(), + Sets.newHashSet(asyncSearchIndex), + lookup, + fieldPermissionsCache + ); assertThat(iac.isGranted(), is(true)); assertThat(iac.hasIndexPermissions(asyncSearchIndex), is(true)); assertThat(iac.getIndexPermissions(asyncSearchIndex), is(notNullValue())); @@ -466,7 +486,7 @@ public class IndicesPermissionTests extends ESTestCase { dataStreamName ).build(); IndicesAccessControl iac = indicesPermission.authorize( - SearchAction.NAME, + TransportSearchAction.TYPE.name(), Sets.newHashSet(backingIndices.stream().map(im -> im.getIndex().getName()).collect(Collectors.toList())), lookup, fieldPermissionsCache diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/interceptor/SearchRequestCacheDisablingInterceptorTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/interceptor/SearchRequestCacheDisablingInterceptorTests.java index 19f4faf62064..2c4a03b7df50 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/interceptor/SearchRequestCacheDisablingInterceptorTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/interceptor/SearchRequestCacheDisablingInterceptorTests.java @@ -8,8 +8,8 @@ package org.elasticsearch.xpack.security.authz.interceptor; import org.elasticsearch.Version; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchRequest; +import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.node.DiscoveryNodes; @@ -82,7 +82,7 @@ public class SearchRequestCacheDisablingInterceptorTests extends ESTestCase { RequestInfo requestInfo = new RequestInfo( Authentication.newAnonymousAuthentication(new AnonymousUser(Settings.EMPTY), randomAlphaOfLengthBetween(3, 8)), searchRequest, - SearchAction.NAME, + TransportSearchAction.TYPE.name(), null ); diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/store/CompositeRolesStoreTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/store/CompositeRolesStoreTests.java index 5c21edd7226c..6e2351d6bbde 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/store/CompositeRolesStoreTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/store/CompositeRolesStoreTests.java @@ -21,7 +21,7 @@ import org.elasticsearch.action.admin.indices.create.CreateIndexAction; import org.elasticsearch.action.delete.DeleteAction; import org.elasticsearch.action.get.GetAction; import org.elasticsearch.action.index.IndexAction; -import org.elasticsearch.action.search.SearchAction; +import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.client.internal.Client; import org.elasticsearch.cluster.health.ClusterHealthStatus; @@ -542,17 +542,17 @@ public class CompositeRolesStoreTests extends ESTestCase { ); assertThat(role.cluster().privileges(), containsInAnyOrder(ClusterPrivilegeResolver.ALL)); - assertThat(role.indices().check(SearchAction.NAME), Matchers.is(true)); + assertThat(role.indices().check(TransportSearchAction.TYPE.name()), Matchers.is(true)); assertThat(role.indices().check(IndexAction.NAME), Matchers.is(true)); final Predicate indexActionPredicate = Automatons.predicate( role.indices().allowedActionsMatcher("index-" + randomAlphaOfLengthBetween(1, 12)) ); - assertThat(indexActionPredicate.test(SearchAction.NAME), is(true)); + assertThat(indexActionPredicate.test(TransportSearchAction.TYPE.name()), is(true)); assertThat(indexActionPredicate.test(IndexAction.NAME), is(true)); final Predicate securityActionPredicate = Automatons.predicate(role.indices().allowedActionsMatcher(".security")); - assertThat(securityActionPredicate.test(SearchAction.NAME), is(true)); + assertThat(securityActionPredicate.test(TransportSearchAction.TYPE.name()), is(true)); assertThat(securityActionPredicate.test(IndexAction.NAME), is(false)); } @@ -2127,7 +2127,8 @@ public class CompositeRolesStoreTests extends ESTestCase { .build(); final var emptyCache = new FieldPermissionsCache(Settings.EMPTY); assertThat( - role.authorize(SearchAction.NAME, Sets.newHashSet("index1"), indexMetadata.getIndicesLookup(), emptyCache).isGranted(), + role.authorize(TransportSearchAction.TYPE.name(), Sets.newHashSet("index1"), indexMetadata.getIndicesLookup(), emptyCache) + .isGranted(), is(false == emptyRemoteRole) ); assertThat( @@ -2135,7 +2136,8 @@ public class CompositeRolesStoreTests extends ESTestCase { is(false) ); assertThat( - role.authorize(SearchAction.NAME, Sets.newHashSet("index2"), indexMetadata.getIndicesLookup(), emptyCache).isGranted(), + role.authorize(TransportSearchAction.TYPE.name(), Sets.newHashSet("index2"), indexMetadata.getIndicesLookup(), emptyCache) + .isGranted(), is(false) ); } @@ -2277,7 +2279,7 @@ public class CompositeRolesStoreTests extends ESTestCase { Role role = future1.actionGet(); assertThat(role.hasWorkflowsRestriction(), equalTo(true)); assertThat(role, not(sameInstance(Role.EMPTY_RESTRICTED_BY_WORKFLOW))); - assertThat(role.checkIndicesAction(SearchAction.NAME), is(true)); + assertThat(role.checkIndicesAction(TransportSearchAction.TYPE.name()), is(true)); } // 2. an "empty-restricted" role if originating workflow does not match (or is null) @@ -2389,7 +2391,7 @@ public class CompositeRolesStoreTests extends ESTestCase { Role role = future1.actionGet(); assertThat(role.hasWorkflowsRestriction(), equalTo(false)); assertThat(role, not(sameInstance(Role.EMPTY_RESTRICTED_BY_WORKFLOW))); - assertThat(role.checkIndicesAction(SearchAction.NAME), is(true)); + assertThat(role.checkIndicesAction(TransportSearchAction.TYPE.name()), is(true)); } } @@ -2586,7 +2588,7 @@ public class CompositeRolesStoreTests extends ESTestCase { } public void testXPackSecurityUserCanAccessAnyIndex() { - for (String action : Arrays.asList(GetAction.NAME, DeleteAction.NAME, SearchAction.NAME, IndexAction.NAME)) { + for (String action : Arrays.asList(GetAction.NAME, DeleteAction.NAME, TransportSearchAction.TYPE.name(), IndexAction.NAME)) { IsResourceAuthorizedPredicate predicate = getXPackSecurityRole().indices().allowedIndicesMatcher(action); IndexAbstraction index = mockIndexAbstraction(randomAlphaOfLengthBetween(3, 12)); @@ -2601,7 +2603,7 @@ public class CompositeRolesStoreTests extends ESTestCase { } public void testSecurityProfileUserHasAccessForOnlyProfileIndex() { - for (String action : Arrays.asList(GetAction.NAME, DeleteAction.NAME, SearchAction.NAME, IndexAction.NAME)) { + for (String action : Arrays.asList(GetAction.NAME, DeleteAction.NAME, TransportSearchAction.TYPE.name(), IndexAction.NAME)) { IsResourceAuthorizedPredicate predicate = getSecurityProfileRole().indices().allowedIndicesMatcher(action); List.of( @@ -2625,7 +2627,7 @@ public class CompositeRolesStoreTests extends ESTestCase { } public void testXPackUserCanAccessNonRestrictedIndices() { - for (String action : Arrays.asList(GetAction.NAME, DeleteAction.NAME, SearchAction.NAME, IndexAction.NAME)) { + for (String action : Arrays.asList(GetAction.NAME, DeleteAction.NAME, TransportSearchAction.TYPE.name(), IndexAction.NAME)) { IsResourceAuthorizedPredicate predicate = getXPackUserRole().indices().allowedIndicesMatcher(action); IndexAbstraction index = mockIndexAbstraction(randomAlphaOfLengthBetween(3, 12)); if (false == TestRestrictedIndices.RESTRICTED_INDICES.isRestricted(index.getName())) { @@ -2639,7 +2641,7 @@ public class CompositeRolesStoreTests extends ESTestCase { } public void testXPackUserCannotAccessSecurityOrAsyncSearch() { - for (String action : Arrays.asList(GetAction.NAME, DeleteAction.NAME, SearchAction.NAME, IndexAction.NAME)) { + for (String action : Arrays.asList(GetAction.NAME, DeleteAction.NAME, TransportSearchAction.TYPE.name(), IndexAction.NAME)) { IsResourceAuthorizedPredicate predicate = getXPackUserRole().indices().allowedIndicesMatcher(action); for (String index : TestRestrictedIndices.SAMPLE_RESTRICTED_NAMES) { assertThat(predicate.test(mockIndexAbstraction(index)), Matchers.is(false)); @@ -2652,7 +2654,7 @@ public class CompositeRolesStoreTests extends ESTestCase { } public void testAsyncSearchUserCannotAccessNonRestrictedIndices() { - for (String action : Arrays.asList(GetAction.NAME, DeleteAction.NAME, SearchAction.NAME, IndexAction.NAME)) { + for (String action : Arrays.asList(GetAction.NAME, DeleteAction.NAME, TransportSearchAction.TYPE.name(), IndexAction.NAME)) { IsResourceAuthorizedPredicate predicate = getAsyncSearchUserRole().indices().allowedIndicesMatcher(action); IndexAbstraction index = mockIndexAbstraction(randomAlphaOfLengthBetween(3, 12)); if (false == TestRestrictedIndices.RESTRICTED_INDICES.isRestricted(index.getName())) { @@ -2666,7 +2668,7 @@ public class CompositeRolesStoreTests extends ESTestCase { } public void testAsyncSearchUserCanAccessOnlyAsyncSearchRestrictedIndices() { - for (String action : Arrays.asList(GetAction.NAME, DeleteAction.NAME, SearchAction.NAME, IndexAction.NAME)) { + for (String action : Arrays.asList(GetAction.NAME, DeleteAction.NAME, TransportSearchAction.TYPE.name(), IndexAction.NAME)) { final IsResourceAuthorizedPredicate predicate = getAsyncSearchUserRole().indices().allowedIndicesMatcher(action); for (String index : TestRestrictedIndices.SAMPLE_RESTRICTED_NAMES) { assertThat(predicate.test(mockIndexAbstraction(index)), Matchers.is(false)); diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/profile/ProfileServiceTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/profile/ProfileServiceTests.java index 5718922bb9be..447153630821 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/profile/ProfileServiceTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/profile/ProfileServiceTests.java @@ -22,14 +22,14 @@ import org.elasticsearch.action.get.MultiGetRequest; import org.elasticsearch.action.get.MultiGetResponse; import org.elasticsearch.action.index.IndexAction; import org.elasticsearch.action.index.IndexRequestBuilder; -import org.elasticsearch.action.search.MultiSearchAction; import org.elasticsearch.action.search.MultiSearchRequest; import org.elasticsearch.action.search.MultiSearchRequestBuilder; import org.elasticsearch.action.search.MultiSearchResponse; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchRequestBuilder; import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.action.search.TransportMultiSearchAction; +import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.action.update.UpdateAction; import org.elasticsearch.action.update.UpdateRequest; @@ -197,7 +197,7 @@ public class ProfileServiceTests extends ESTestCase { this.client = mock(Client.class); when(client.threadPool()).thenReturn(threadPool); when(client.prepareSearch(SECURITY_PROFILE_ALIAS)).thenReturn( - new SearchRequestBuilder(client, SearchAction.INSTANCE).setIndices(SECURITY_PROFILE_ALIAS) + new SearchRequestBuilder(client, TransportSearchAction.TYPE).setIndices(SECURITY_PROFILE_ALIAS) ); this.profileIndex = SecurityMocks.mockSecurityIndexManager(SECURITY_PROFILE_ALIAS); final ClusterService clusterService = mock(ClusterService.class); @@ -593,7 +593,7 @@ public class ProfileServiceTests extends ESTestCase { ) ); return null; - }).when(client).execute(eq(MultiSearchAction.INSTANCE), any(MultiSearchRequest.class), anyActionListener()); + }).when(client).execute(eq(TransportMultiSearchAction.TYPE), any(MultiSearchRequest.class), anyActionListener()); when(client.prepareIndex(SECURITY_PROFILE_ALIAS)).thenReturn( new IndexRequestBuilder(client, IndexAction.INSTANCE, SECURITY_PROFILE_ALIAS) @@ -639,7 +639,7 @@ public class ProfileServiceTests extends ESTestCase { final ActionListener listener = (ActionListener) invocation.getArguments()[2]; listener.onFailure(expectedException); return null; - }).when(client).execute(eq(SearchAction.INSTANCE), any(SearchRequest.class), anyActionListener()); + }).when(client).execute(eq(TransportSearchAction.TYPE), any(SearchRequest.class), anyActionListener()); final PlainActionFuture future3 = new PlainActionFuture<>(); profileService.suggestProfile( new SuggestProfilesRequest(Set.of(), "", 1, null), @@ -1064,9 +1064,9 @@ public class ProfileServiceTests extends ESTestCase { final var listener = (ActionListener) invocation.getArgument(2); listener.onResponse(multiSearchResponse); return null; - }).when(client).execute(eq(MultiSearchAction.INSTANCE), any(MultiSearchRequest.class), anyActionListener()); + }).when(client).execute(eq(TransportMultiSearchAction.TYPE), any(MultiSearchRequest.class), anyActionListener()); - when(client.prepareMultiSearch()).thenReturn(new MultiSearchRequestBuilder(client, MultiSearchAction.INSTANCE)); + when(client.prepareMultiSearch()).thenReturn(new MultiSearchRequestBuilder(client, TransportMultiSearchAction.TYPE)); final PlainActionFuture> future = new PlainActionFuture<>(); profileService.usageStats(future); assertThat(future.actionGet(), equalTo(metrics)); diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/ServerTransportFilterTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/ServerTransportFilterTests.java index 03e64f1e616f..c22892df2ce5 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/ServerTransportFilterTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/ServerTransportFilterTests.java @@ -13,7 +13,7 @@ import org.elasticsearch.action.MockIndicesRequest; import org.elasticsearch.action.admin.indices.close.CloseIndexAction; import org.elasticsearch.action.admin.indices.delete.DeleteIndexAction; import org.elasticsearch.action.admin.indices.open.OpenIndexAction; -import org.elasticsearch.action.search.SearchAction; +import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.action.support.DestructiveOperations; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.support.PlainActionFuture; @@ -268,7 +268,7 @@ public class ServerTransportFilterTests extends ESTestCase { ServerTransportFilter filter = crossClusterAccess ? getNodeCrossClusterAccessFilter() : getNodeFilter(); TransportRequest request = mock(TransportRequest.class); Authentication authentication = AuthenticationTestHelper.builder().build(); - String action = SearchAction.NAME; + String action = TransportSearchAction.TYPE.name(); doAnswer(getAnswer(authentication)).when(authcService).authenticate(eq(action), eq(request), eq(true), anyActionListener()); doAnswer(getAnswer(authentication, true)).when(crossClusterAccessAuthcService) .authenticate(eq(action), eq(request), anyActionListener()); diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/execution/search/Querier.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/execution/search/Querier.java index 65492f29d9a0..a0da67f3006a 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/execution/search/Querier.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/execution/search/Querier.java @@ -12,13 +12,13 @@ import org.apache.lucene.search.TotalHits; import org.apache.lucene.util.PriorityQueue; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.DelegatingActionListener; -import org.elasticsearch.action.search.ClosePointInTimeAction; import org.elasticsearch.action.search.ClosePointInTimeRequest; -import org.elasticsearch.action.search.OpenPointInTimeAction; import org.elasticsearch.action.search.OpenPointInTimeRequest; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.ShardSearchFailure; +import org.elasticsearch.action.search.TransportClosePointInTimeAction; +import org.elasticsearch.action.search.TransportOpenPointInTimeAction; import org.elasticsearch.client.internal.Client; import org.elasticsearch.client.internal.ParentTaskAssigningClient; import org.elasticsearch.common.Strings; @@ -157,7 +157,7 @@ public class Querier { .keepAlive(cfg.pageTimeout()); client.execute( - OpenPointInTimeAction.INSTANCE, + TransportOpenPointInTimeAction.TYPE, openPitRequest, listener.delegateFailureAndWrap((delegate, openPointInTimeResponse) -> { String pitId = openPointInTimeResponse.getPointInTimeId(); @@ -188,7 +188,7 @@ public class Querier { client = client instanceof ParentTaskAssigningClient wrapperClient ? wrapperClient.unwrap() : client; client.execute( - ClosePointInTimeAction.INSTANCE, + TransportClosePointInTimeAction.TYPE, new ClosePointInTimeRequest(pointInTimeId), listener.delegateFailureAndWrap((l, clearPointInTimeResponse) -> l.onResponse(clearPointInTimeResponse.isSucceeded())) ); diff --git a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/analysis/CancellationTests.java b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/analysis/CancellationTests.java index 9eb6b7a6f978..1fbe85d87395 100644 --- a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/analysis/CancellationTests.java +++ b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/analysis/CancellationTests.java @@ -9,15 +9,15 @@ package org.elasticsearch.xpack.sql.analysis; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.fieldcaps.FieldCapabilities; import org.elasticsearch.action.fieldcaps.FieldCapabilitiesResponse; -import org.elasticsearch.action.search.ClosePointInTimeAction; import org.elasticsearch.action.search.ClosePointInTimeRequest; import org.elasticsearch.action.search.ClosePointInTimeResponse; -import org.elasticsearch.action.search.OpenPointInTimeAction; import org.elasticsearch.action.search.OpenPointInTimeResponse; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchRequestBuilder; import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.action.search.TransportClosePointInTimeAction; +import org.elasticsearch.action.search.TransportOpenPointInTimeAction; +import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.client.internal.Client; import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.node.DiscoveryNode; @@ -192,11 +192,11 @@ public class CancellationTests extends ESTestCase { ActionListener listener = (ActionListener) invocation.getArguments()[2]; listener.onResponse(new OpenPointInTimeResponse(pitId)); return null; - }).when(client).execute(eq(OpenPointInTimeAction.INSTANCE), any(), any()); + }).when(client).execute(eq(TransportOpenPointInTimeAction.TYPE), any(), any()); // Emulation of search cancellation ArgumentCaptor searchRequestCaptor = ArgumentCaptor.forClass(SearchRequest.class); - when(client.prepareSearch(any())).thenReturn(new SearchRequestBuilder(client, SearchAction.INSTANCE).setIndices(indices)); + when(client.prepareSearch(any())).thenReturn(new SearchRequestBuilder(client, TransportSearchAction.TYPE).setIndices(indices)); doAnswer((Answer) invocation -> { @SuppressWarnings("unchecked") SearchRequest request = (SearchRequest) invocation.getArguments()[1]; @@ -209,7 +209,7 @@ public class CancellationTests extends ESTestCase { ActionListener listener = (ActionListener) invocation.getArguments()[2]; listener.onFailure(new TaskCancelledException("cancelled")); return null; - }).when(client).execute(eq(SearchAction.INSTANCE), searchRequestCaptor.capture(), any()); + }).when(client).execute(eq(TransportSearchAction.TYPE), searchRequestCaptor.capture(), any()); // Emulation of close pit doAnswer(invocation -> { @@ -220,7 +220,7 @@ public class CancellationTests extends ESTestCase { ActionListener listener = (ActionListener) invocation.getArguments()[2]; listener.onResponse(new ClosePointInTimeResponse(true, 1)); return null; - }).when(client).execute(eq(ClosePointInTimeAction.INSTANCE), any(), any()); + }).when(client).execute(eq(TransportClosePointInTimeAction.TYPE), any(), any()); IndexResolver indexResolver = indexResolver(client); PlanExecutor planExecutor = new PlanExecutor(client, indexResolver, new NamedWriteableRegistry(Collections.emptyList())); @@ -242,9 +242,9 @@ public class CancellationTests extends ESTestCase { assertTrue(countDownLatch.await(5, TimeUnit.SECONDS)); // Final verification to ensure no more interaction verify(client).fieldCaps(any(), any()); - verify(client, times(1)).execute(eq(OpenPointInTimeAction.INSTANCE), any(), any()); - verify(client, times(1)).execute(eq(SearchAction.INSTANCE), any(), any()); - verify(client, times(1)).execute(eq(ClosePointInTimeAction.INSTANCE), any(), any()); + verify(client, times(1)).execute(eq(TransportOpenPointInTimeAction.TYPE), any(), any()); + verify(client, times(1)).execute(eq(TransportSearchAction.TYPE), any(), any()); + verify(client, times(1)).execute(eq(TransportClosePointInTimeAction.TYPE), any(), any()); verify(client, times(1)).settings(); verify(client, times(1)).threadPool(); verifyNoMoreInteractions(client); diff --git a/x-pack/plugin/transform/src/main/java/org/elasticsearch/xpack/transform/action/TransportGetCheckpointAction.java b/x-pack/plugin/transform/src/main/java/org/elasticsearch/xpack/transform/action/TransportGetCheckpointAction.java index 675d71a5d1db..bf18f7257d90 100644 --- a/x-pack/plugin/transform/src/main/java/org/elasticsearch/xpack/transform/action/TransportGetCheckpointAction.java +++ b/x-pack/plugin/transform/src/main/java/org/elasticsearch/xpack/transform/action/TransportGetCheckpointAction.java @@ -15,10 +15,10 @@ import org.elasticsearch.action.NoShardAvailableActionException; import org.elasticsearch.action.OriginalIndices; import org.elasticsearch.action.UnavailableShardsException; import org.elasticsearch.action.search.SearchRequest; -import org.elasticsearch.action.search.SearchShardsAction; import org.elasticsearch.action.search.SearchShardsGroup; import org.elasticsearch.action.search.SearchShardsRequest; import org.elasticsearch.action.search.SearchShardsResponse; +import org.elasticsearch.action.search.TransportSearchShardsAction; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.GroupedActionListener; import org.elasticsearch.action.support.HandledTransportAction; @@ -126,7 +126,7 @@ public class TransportGetCheckpointAction extends HandledTransportAction { Map> filteredNodesAndShards = filterOutSkippedShards(nodesAndShards, searchShardsResponse); diff --git a/x-pack/plugin/transform/src/main/java/org/elasticsearch/xpack/transform/checkpoint/TimeBasedCheckpointProvider.java b/x-pack/plugin/transform/src/main/java/org/elasticsearch/xpack/transform/checkpoint/TimeBasedCheckpointProvider.java index 7b83af1dc140..ec4cc2dcbcbf 100644 --- a/x-pack/plugin/transform/src/main/java/org/elasticsearch/xpack/transform/checkpoint/TimeBasedCheckpointProvider.java +++ b/x-pack/plugin/transform/src/main/java/org/elasticsearch/xpack/transform/checkpoint/TimeBasedCheckpointProvider.java @@ -10,8 +10,8 @@ package org.elasticsearch.xpack.transform.checkpoint; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchRequest; +import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.client.internal.ParentTaskAssigningClient; import org.elasticsearch.index.query.BoolQueryBuilder; @@ -81,7 +81,7 @@ class TimeBasedCheckpointProvider extends DefaultCheckpointProvider { transformConfig.getHeaders(), ClientHelper.TRANSFORM_ORIGIN, client, - SearchAction.INSTANCE, + TransportSearchAction.TYPE, searchRequest, ActionListener.wrap(r -> listener.onResponse(r.getHits().getTotalHits().value > 0L), listener::onFailure) ); diff --git a/x-pack/plugin/transform/src/main/java/org/elasticsearch/xpack/transform/persistence/IndexBasedTransformConfigManager.java b/x-pack/plugin/transform/src/main/java/org/elasticsearch/xpack/transform/persistence/IndexBasedTransformConfigManager.java index 107c3c2b5bd6..5d9a3971ad08 100644 --- a/x-pack/plugin/transform/src/main/java/org/elasticsearch/xpack/transform/persistence/IndexBasedTransformConfigManager.java +++ b/x-pack/plugin/transform/src/main/java/org/elasticsearch/xpack/transform/persistence/IndexBasedTransformConfigManager.java @@ -22,9 +22,9 @@ import org.elasticsearch.action.admin.indices.refresh.RefreshResponse; import org.elasticsearch.action.bulk.BulkItemResponse; import org.elasticsearch.action.index.IndexAction; import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.support.WriteRequest; import org.elasticsearch.client.internal.Client; @@ -389,7 +389,7 @@ public class IndexBasedTransformConfigManager implements TransformConfigManager executeAsyncWithOrigin( client, TRANSFORM_ORIGIN, - SearchAction.INSTANCE, + TransportSearchAction.TYPE, searchRequest, ActionListener.wrap(searchResponse -> { if (searchResponse.getHits().getHits().length == 0) { @@ -426,7 +426,7 @@ public class IndexBasedTransformConfigManager implements TransformConfigManager executeAsyncWithOrigin( client, TRANSFORM_ORIGIN, - SearchAction.INSTANCE, + TransportSearchAction.TYPE, searchRequest, ActionListener.wrap(searchResponse -> { if (searchResponse.getHits().getHits().length == 0) { @@ -470,7 +470,7 @@ public class IndexBasedTransformConfigManager implements TransformConfigManager executeAsyncWithOrigin( client, TRANSFORM_ORIGIN, - SearchAction.INSTANCE, + TransportSearchAction.TYPE, searchRequest, ActionListener.wrap(searchResponse -> { if (searchResponse.getHits().getHits().length == 0) { @@ -503,7 +503,7 @@ public class IndexBasedTransformConfigManager implements TransformConfigManager .seqNoAndPrimaryTerm(true) .request(); - executeAsyncWithOrigin(client, TRANSFORM_ORIGIN, SearchAction.INSTANCE, searchRequest, ActionListener.wrap(searchResponse -> { + executeAsyncWithOrigin(client, TRANSFORM_ORIGIN, TransportSearchAction.TYPE, searchRequest, ActionListener.wrap(searchResponse -> { if (searchResponse.getHits().getHits().length == 0) { configAndVersionListener.onFailure( new ResourceNotFoundException(TransformMessages.getMessage(TransformMessages.REST_UNKNOWN_TRANSFORM, transformId)) @@ -637,7 +637,7 @@ public class IndexBasedTransformConfigManager implements TransformConfigManager .query(QueryBuilders.termQuery(TransformField.ID.getPreferredName(), transformId)) .trackTotalHitsUpTo(1) ); - executeAsyncWithOrigin(client, TRANSFORM_ORIGIN, SearchAction.INSTANCE, searchRequest, ActionListener.wrap(searchResponse -> { + executeAsyncWithOrigin(client, TRANSFORM_ORIGIN, TransportSearchAction.TYPE, searchRequest, ActionListener.wrap(searchResponse -> { if (searchResponse.getHits().getTotalHits().value == 0) { listener.onFailure( new ResourceNotFoundException(TransformMessages.getMessage(TransformMessages.REST_UNKNOWN_TRANSFORM, transformId)) @@ -762,7 +762,7 @@ public class IndexBasedTransformConfigManager implements TransformConfigManager executeAsyncWithOrigin( client, TRANSFORM_ORIGIN, - SearchAction.INSTANCE, + TransportSearchAction.TYPE, searchRequest, ActionListener.wrap(searchResponse -> { if (searchResponse.getHits().getHits().length == 0) { diff --git a/x-pack/plugin/transform/src/main/java/org/elasticsearch/xpack/transform/transforms/ClientTransformIndexer.java b/x-pack/plugin/transform/src/main/java/org/elasticsearch/xpack/transform/transforms/ClientTransformIndexer.java index f50d12f58ec6..29be02b87cbd 100644 --- a/x-pack/plugin/transform/src/main/java/org/elasticsearch/xpack/transform/transforms/ClientTransformIndexer.java +++ b/x-pack/plugin/transform/src/main/java/org/elasticsearch/xpack/transform/transforms/ClientTransformIndexer.java @@ -18,13 +18,13 @@ import org.elasticsearch.action.bulk.BulkAction; import org.elasticsearch.action.bulk.BulkItemResponse; import org.elasticsearch.action.bulk.BulkRequest; import org.elasticsearch.action.bulk.BulkResponse; -import org.elasticsearch.action.search.ClosePointInTimeAction; import org.elasticsearch.action.search.ClosePointInTimeRequest; -import org.elasticsearch.action.search.OpenPointInTimeAction; import org.elasticsearch.action.search.OpenPointInTimeRequest; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.action.search.TransportClosePointInTimeAction; +import org.elasticsearch.action.search.TransportOpenPointInTimeAction; +import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.action.support.master.AcknowledgedRequest; import org.elasticsearch.client.internal.ParentTaskAssigningClient; import org.elasticsearch.common.logging.LoggerMessageFormat; @@ -277,7 +277,7 @@ class ClientTransformIndexer extends TransformIndexer { transformConfig.getHeaders(), ClientHelper.TRANSFORM_ORIGIN, client, - SearchAction.INSTANCE, + TransportSearchAction.TYPE, request, responseListener ); @@ -433,7 +433,7 @@ class ClientTransformIndexer extends TransformIndexer { transformConfig.getHeaders(), ClientHelper.TRANSFORM_ORIGIN, client, - ClosePointInTimeAction.INSTANCE, + TransportClosePointInTimeAction.TYPE, closePitRequest, ActionListener.wrap(response -> { logger.trace("[{}] closed pit search context [{}]", getJobId(), oldPit); @@ -471,7 +471,7 @@ class ClientTransformIndexer extends TransformIndexer { transformConfig.getHeaders(), ClientHelper.TRANSFORM_ORIGIN, client, - OpenPointInTimeAction.INSTANCE, + TransportOpenPointInTimeAction.TYPE, pitRequest, ActionListener.wrap(response -> { PointInTimeBuilder newPit = new PointInTimeBuilder(response.getPointInTimeId()).setKeepAlive(PIT_KEEP_ALIVE); @@ -534,7 +534,7 @@ class ClientTransformIndexer extends TransformIndexer { transformConfig.getHeaders(), ClientHelper.TRANSFORM_ORIGIN, client, - SearchAction.INSTANCE, + TransportSearchAction.TYPE, searchRequest, ActionListener.wrap(response -> { // did the pit change? @@ -560,7 +560,7 @@ class ClientTransformIndexer extends TransformIndexer { transformConfig.getHeaders(), ClientHelper.TRANSFORM_ORIGIN, client, - SearchAction.INSTANCE, + TransportSearchAction.TYPE, searchRequest, listener ); @@ -579,7 +579,7 @@ class ClientTransformIndexer extends TransformIndexer { transformConfig.getHeaders(), ClientHelper.TRANSFORM_ORIGIN, client, - SearchAction.INSTANCE, + TransportSearchAction.TYPE, searchRequest, listener ); diff --git a/x-pack/plugin/transform/src/main/java/org/elasticsearch/xpack/transform/transforms/common/AbstractCompositeAggFunction.java b/x-pack/plugin/transform/src/main/java/org/elasticsearch/xpack/transform/transforms/common/AbstractCompositeAggFunction.java index 80eb482da6b6..189fb26e1f96 100644 --- a/x-pack/plugin/transform/src/main/java/org/elasticsearch/xpack/transform/transforms/common/AbstractCompositeAggFunction.java +++ b/x-pack/plugin/transform/src/main/java/org/elasticsearch/xpack/transform/transforms/common/AbstractCompositeAggFunction.java @@ -11,9 +11,9 @@ import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchStatusException; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.client.internal.Client; import org.elasticsearch.common.ValidationException; @@ -77,7 +77,7 @@ public abstract class AbstractCompositeAggFunction implements Function { headers, ClientHelper.TRANSFORM_ORIGIN, client, - SearchAction.INSTANCE, + TransportSearchAction.TYPE, buildSearchRequest(sourceConfig, timeout, numberOfBuckets), ActionListener.wrap(r -> { try { @@ -121,7 +121,7 @@ public abstract class AbstractCompositeAggFunction implements Function { headers, ClientHelper.TRANSFORM_ORIGIN, client, - SearchAction.INSTANCE, + TransportSearchAction.TYPE, searchRequest, ActionListener.wrap(response -> { if (response == null) { diff --git a/x-pack/plugin/transform/src/test/java/org/elasticsearch/xpack/transform/checkpoint/TimeBasedCheckpointProviderTests.java b/x-pack/plugin/transform/src/test/java/org/elasticsearch/xpack/transform/checkpoint/TimeBasedCheckpointProviderTests.java index 5f1c0e6bb7f7..bed646b9ddeb 100644 --- a/x-pack/plugin/transform/src/test/java/org/elasticsearch/xpack/transform/checkpoint/TimeBasedCheckpointProviderTests.java +++ b/x-pack/plugin/transform/src/test/java/org/elasticsearch/xpack/transform/checkpoint/TimeBasedCheckpointProviderTests.java @@ -11,11 +11,11 @@ import org.apache.lucene.search.TotalHits; import org.apache.lucene.util.SetOnce; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.LatchedActionListener; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponseSections; import org.elasticsearch.action.search.ShardSearchFailure; +import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.client.internal.Client; import org.elasticsearch.client.internal.ParentTaskAssigningClient; import org.elasticsearch.common.settings.ClusterSettings; @@ -179,7 +179,7 @@ public class TimeBasedCheckpointProviderTests extends ESTestCase { TimeValue delay, Tuple expectedRangeQueryBounds ) throws InterruptedException { - doAnswer(withResponse(newSearchResponse(totalHits))).when(client).execute(eq(SearchAction.INSTANCE), any(), any()); + doAnswer(withResponse(newSearchResponse(totalHits))).when(client).execute(eq(TransportSearchAction.TYPE), any(), any()); String transformId = getTestName(); TransformConfig transformConfig = newTransformConfigWithDateHistogram( transformId, @@ -200,7 +200,7 @@ public class TimeBasedCheckpointProviderTests extends ESTestCase { assertThat(latch.await(100, TimeUnit.MILLISECONDS), is(true)); ArgumentCaptor searchRequestArgumentCaptor = ArgumentCaptor.forClass(SearchRequest.class); - verify(client).execute(eq(SearchAction.INSTANCE), searchRequestArgumentCaptor.capture(), any()); + verify(client).execute(eq(TransportSearchAction.TYPE), searchRequestArgumentCaptor.capture(), any()); SearchRequest searchRequest = searchRequestArgumentCaptor.getValue(); BoolQueryBuilder boolQuery = (BoolQueryBuilder) searchRequest.source().query(); RangeQueryBuilder rangeQuery = (RangeQueryBuilder) boolQuery.filter().get(1); diff --git a/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/WatcherServiceTests.java b/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/WatcherServiceTests.java index 87c7c6e9748f..1dede3f4e135 100644 --- a/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/WatcherServiceTests.java +++ b/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/WatcherServiceTests.java @@ -12,16 +12,16 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.admin.indices.refresh.RefreshAction; import org.elasticsearch.action.admin.indices.refresh.RefreshRequest; import org.elasticsearch.action.admin.indices.refresh.RefreshResponse; -import org.elasticsearch.action.search.ClearScrollAction; import org.elasticsearch.action.search.ClearScrollRequest; import org.elasticsearch.action.search.ClearScrollResponse; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponseSections; -import org.elasticsearch.action.search.SearchScrollAction; import org.elasticsearch.action.search.SearchScrollRequest; import org.elasticsearch.action.search.ShardSearchFailure; +import org.elasticsearch.action.search.TransportClearScrollAction; +import org.elasticsearch.action.search.TransportSearchAction; +import org.elasticsearch.action.search.TransportSearchScrollAction; import org.elasticsearch.client.internal.Client; import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.ClusterState; @@ -196,7 +196,7 @@ public class WatcherServiceTests extends ESTestCase { ActionListener listener = (ActionListener) invocation.getArguments()[2]; listener.onResponse(scrollSearchResponse); return null; - }).when(client).execute(eq(SearchScrollAction.INSTANCE), any(SearchScrollRequest.class), anyActionListener()); + }).when(client).execute(eq(TransportSearchScrollAction.TYPE), any(SearchScrollRequest.class), anyActionListener()); // one search response containing active and inactive watches int count = randomIntBetween(2, 200); @@ -236,13 +236,13 @@ public class WatcherServiceTests extends ESTestCase { ActionListener listener = (ActionListener) invocation.getArguments()[2]; listener.onResponse(searchResponse); return null; - }).when(client).execute(eq(SearchAction.INSTANCE), any(SearchRequest.class), anyActionListener()); + }).when(client).execute(eq(TransportSearchAction.TYPE), any(SearchRequest.class), anyActionListener()); doAnswer(invocation -> { ActionListener listener = (ActionListener) invocation.getArguments()[2]; listener.onResponse(new ClearScrollResponse(true, 1)); return null; - }).when(client).execute(eq(ClearScrollAction.INSTANCE), any(ClearScrollRequest.class), anyActionListener()); + }).when(client).execute(eq(TransportClearScrollAction.TYPE), any(ClearScrollRequest.class), anyActionListener()); service.start(clusterState, () -> {}, exception -> {}); diff --git a/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/execution/TriggeredWatchStoreTests.java b/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/execution/TriggeredWatchStoreTests.java index 01547b898e4b..0f47df9dff12 100644 --- a/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/execution/TriggeredWatchStoreTests.java +++ b/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/execution/TriggeredWatchStoreTests.java @@ -18,12 +18,12 @@ import org.elasticsearch.action.bulk.BulkProcessor2; import org.elasticsearch.action.bulk.BulkRequest; import org.elasticsearch.action.bulk.BulkResponse; import org.elasticsearch.action.index.IndexResponse; -import org.elasticsearch.action.search.ClearScrollAction; import org.elasticsearch.action.search.ClearScrollResponse; -import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchResponse; -import org.elasticsearch.action.search.SearchScrollAction; import org.elasticsearch.action.search.SearchScrollRequest; +import org.elasticsearch.action.search.TransportClearScrollAction; +import org.elasticsearch.action.search.TransportSearchAction; +import org.elasticsearch.action.search.TransportSearchScrollAction; import org.elasticsearch.client.internal.Client; import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.ClusterState; @@ -223,7 +223,7 @@ public class TriggeredWatchStoreTests extends ESTestCase { ActionListener listener = (ActionListener) invocation.getArguments()[2]; listener.onResponse(searchResponse1); return null; - }).when(client).execute(eq(SearchAction.INSTANCE), any(), any()); + }).when(client).execute(eq(TransportSearchAction.TYPE), any(), any()); // First return a scroll response with a single hit and then with no hits hit = new SearchHit(0, "second_foo"); @@ -264,7 +264,7 @@ public class TriggeredWatchStoreTests extends ESTestCase { listener.onFailure(new ElasticsearchException("test issue")); } return null; - }).when(client).execute(eq(SearchScrollAction.INSTANCE), any(), any()); + }).when(client).execute(eq(TransportSearchScrollAction.TYPE), any(), any()); TriggeredWatch triggeredWatch = mock(TriggeredWatch.class); when(parser.parse(eq("_id"), eq(1L), any(BytesReference.class))).thenReturn(triggeredWatch); @@ -275,7 +275,7 @@ public class TriggeredWatchStoreTests extends ESTestCase { listener.onResponse(new ClearScrollResponse(true, 1)); return null; - }).when(client).execute(eq(ClearScrollAction.INSTANCE), any(), any()); + }).when(client).execute(eq(TransportClearScrollAction.TYPE), any(), any()); assertThat(TriggeredWatchStore.validate(cs), is(true)); ZonedDateTime now = ZonedDateTime.now(ZoneOffset.UTC); @@ -301,9 +301,9 @@ public class TriggeredWatchStoreTests extends ESTestCase { assertThat(triggeredWatches, hasSize(watches.size())); verify(client, times(1)).execute(eq(RefreshAction.INSTANCE), any(), any()); - verify(client, times(1)).execute(eq(SearchAction.INSTANCE), any(), any()); - verify(client, times(2)).execute(eq(SearchScrollAction.INSTANCE), any(), any()); - verify(client, times(1)).execute(eq(ClearScrollAction.INSTANCE), any(), any()); + verify(client, times(1)).execute(eq(TransportSearchAction.TYPE), any(), any()); + verify(client, times(2)).execute(eq(TransportSearchScrollAction.TYPE), any(), any()); + verify(client, times(1)).execute(eq(TransportClearScrollAction.TYPE), any(), any()); } // the elasticsearch migration helper is doing reindex using aliases, so we have to