Remove some more ActionType subclasses (#102354)

Removing a couple more of these from the search package.

Co-authored-by: Joe Gallo <joegallo@gmail.com>
This commit is contained in:
Armin Braun 2023-11-27 19:11:19 +01:00 committed by GitHub
parent 0eca436728
commit c45b95d0eb
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
150 changed files with 1196 additions and 1049 deletions

View file

@ -12,9 +12,9 @@ import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.action.ActionFuture; import org.elasticsearch.action.ActionFuture;
import org.elasticsearch.action.DocWriteRequest; import org.elasticsearch.action.DocWriteRequest;
import org.elasticsearch.action.bulk.BulkRequestBuilder; import org.elasticsearch.action.bulk.BulkRequestBuilder;
import org.elasticsearch.action.search.SearchAction;
import org.elasticsearch.action.search.SearchPhaseExecutionException; import org.elasticsearch.action.search.SearchPhaseExecutionException;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.search.TransportSearchAction;
import org.elasticsearch.action.support.WriteRequest; import org.elasticsearch.action.support.WriteRequest;
import org.elasticsearch.aggregations.AggregationsPlugin; import org.elasticsearch.aggregations.AggregationsPlugin;
import org.elasticsearch.aggregations.bucket.timeseries.TimeSeriesAggregationBuilder; import org.elasticsearch.aggregations.bucket.timeseries.TimeSeriesAggregationBuilder;
@ -115,7 +115,7 @@ public class SearchCancellationIT extends AbstractSearchCancellationTestCase {
) )
.execute(); .execute();
awaitForBlock(plugins); awaitForBlock(plugins);
cancelSearch(SearchAction.NAME); cancelSearch(TransportSearchAction.TYPE.name());
disableBlocks(plugins); disableBlocks(plugins);
SearchPhaseExecutionException ex = expectThrows(SearchPhaseExecutionException.class, searchResponse::actionGet); SearchPhaseExecutionException ex = expectThrows(SearchPhaseExecutionException.class, searchResponse::actionGet);

View file

@ -14,11 +14,11 @@ import org.elasticsearch.action.ActionRequest;
import org.elasticsearch.action.ActionResponse; import org.elasticsearch.action.ActionResponse;
import org.elasticsearch.action.ActionType; import org.elasticsearch.action.ActionType;
import org.elasticsearch.action.bulk.BackoffPolicy; import org.elasticsearch.action.bulk.BackoffPolicy;
import org.elasticsearch.action.search.SearchAction;
import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.search.SearchScrollAction;
import org.elasticsearch.action.search.SearchScrollRequest; 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.action.support.PlainActionFuture;
import org.elasticsearch.client.internal.ParentTaskAssigningClient; import org.elasticsearch.client.internal.ParentTaskAssigningClient;
import org.elasticsearch.client.internal.support.AbstractClient; import org.elasticsearch.client.internal.support.AbstractClient;
@ -102,17 +102,17 @@ public class ClientScrollableHitSourceTests extends ESTestCase {
hitSource.start(); hitSource.start();
for (int retry = 0; retry < randomIntBetween(minFailures, maxFailures); ++retry) { for (int retry = 0; retry < randomIntBetween(minFailures, maxFailures); ++retry) {
client.fail(SearchAction.INSTANCE, new EsRejectedExecutionException()); client.fail(TransportSearchAction.TYPE, new EsRejectedExecutionException());
if (retry >= retries) { if (retry >= retries) {
return; return;
} }
client.awaitOperation(); client.awaitOperation();
++expectedSearchRetries; ++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(); SearchResponse searchResponse = createSearchResponse();
try { try {
client.respond(SearchAction.INSTANCE, searchResponse); client.respond(TransportSearchAction.TYPE, searchResponse);
for (int i = 0; i < randomIntBetween(1, 10); ++i) { for (int i = 0; i < randomIntBetween(1, 10); ++i) {
ScrollableHitSource.AsyncResponse asyncResponse = responses.poll(10, TimeUnit.SECONDS); ScrollableHitSource.AsyncResponse asyncResponse = responses.poll(10, TimeUnit.SECONDS);
@ -122,13 +122,13 @@ public class ClientScrollableHitSourceTests extends ESTestCase {
asyncResponse.done(TimeValue.ZERO); asyncResponse.done(TimeValue.ZERO);
for (int retry = 0; retry < randomIntBetween(minFailures, maxFailures); ++retry) { for (int retry = 0; retry < randomIntBetween(minFailures, maxFailures); ++retry) {
client.fail(SearchScrollAction.INSTANCE, new EsRejectedExecutionException()); client.fail(TransportSearchScrollAction.TYPE, new EsRejectedExecutionException());
client.awaitOperation(); client.awaitOperation();
++expectedSearchRetries; ++expectedSearchRetries;
} }
searchResponse = createSearchResponse(); searchResponse = createSearchResponse();
client.respond(SearchScrollAction.INSTANCE, searchResponse); client.respond(TransportSearchScrollAction.TYPE, searchResponse);
} }
assertEquals(actualSearchRetries.get(), expectedSearchRetries); assertEquals(actualSearchRetries.get(), expectedSearchRetries);
@ -154,7 +154,10 @@ public class ClientScrollableHitSourceTests extends ESTestCase {
); );
hitSource.startNextScroll(timeValueSeconds(100)); 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() { private SearchResponse createSearchResponse() {

View file

@ -15,7 +15,7 @@ import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionRequest;
import org.elasticsearch.action.ActionResponse; import org.elasticsearch.action.ActionResponse;
import org.elasticsearch.action.admin.cluster.node.info.NodeInfo; 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.ActionFilter;
import org.elasticsearch.action.support.ActionFilterChain; import org.elasticsearch.action.support.ActionFilterChain;
import org.elasticsearch.action.support.WriteRequest.RefreshPolicy; import org.elasticsearch.action.support.WriteRequest.RefreshPolicy;
@ -199,7 +199,7 @@ public class ReindexFromRemoteWithAuthTests extends ESSingleNodeTestCase {
ActionListener<Response> listener, ActionListener<Response> listener,
ActionFilterChain<Request, Response> chain ActionFilterChain<Request, Response> chain
) { ) {
if (false == action.equals(SearchAction.NAME)) { if (false == action.equals(TransportSearchAction.TYPE.name())) {
chain.proceed(task, action, request, listener); chain.proceed(task, action, request, listener);
return; return;
} }

View file

@ -16,13 +16,13 @@ import org.elasticsearch.TransportVersion;
import org.elasticsearch.action.admin.cluster.state.ClusterStateAction; import org.elasticsearch.action.admin.cluster.state.ClusterStateAction;
import org.elasticsearch.action.admin.cluster.state.ClusterStateRequest; import org.elasticsearch.action.admin.cluster.state.ClusterStateRequest;
import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse; 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.SearchRequest;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.search.SearchShardsAction;
import org.elasticsearch.action.search.SearchShardsRequest; import org.elasticsearch.action.search.SearchShardsRequest;
import org.elasticsearch.action.search.SearchShardsResponse; import org.elasticsearch.action.search.SearchShardsResponse;
import org.elasticsearch.action.search.ShardSearchFailure; 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.Request;
import org.elasticsearch.client.Response; import org.elasticsearch.client.Response;
import org.elasticsearch.client.ResponseException; import org.elasticsearch.client.ResponseException;
@ -81,7 +81,7 @@ public class CrossClusterSearchUnavailableClusterIT extends ESRestTestCase {
MockTransportService newService = MockTransportService.createNewService(s, version, transportVersion, threadPool, null); MockTransportService newService = MockTransportService.createNewService(s, version, transportVersion, threadPool, null);
try { try {
newService.registerRequestHandler( newService.registerRequestHandler(
SearchShardsAction.NAME, TransportSearchShardsAction.TYPE.name(),
EsExecutors.DIRECT_EXECUTOR_SERVICE, EsExecutors.DIRECT_EXECUTOR_SERVICE,
SearchShardsRequest::new, SearchShardsRequest::new,
(request, channel, task) -> { (request, channel, task) -> {
@ -89,7 +89,7 @@ public class CrossClusterSearchUnavailableClusterIT extends ESRestTestCase {
} }
); );
newService.registerRequestHandler( newService.registerRequestHandler(
SearchAction.NAME, TransportSearchAction.TYPE.name(),
EsExecutors.DIRECT_EXECUTOR_SERVICE, EsExecutors.DIRECT_EXECUTOR_SERVICE,
SearchRequest::new, SearchRequest::new,
(request, channel, task) -> { (request, channel, task) -> {

View file

@ -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.info.NodesInfoResponse;
import org.elasticsearch.action.admin.cluster.node.tasks.list.ListTasksResponse; import org.elasticsearch.action.admin.cluster.node.tasks.list.ListTasksResponse;
import org.elasticsearch.action.bulk.BulkRequestBuilder; import org.elasticsearch.action.bulk.BulkRequestBuilder;
import org.elasticsearch.action.search.MultiSearchAction;
import org.elasticsearch.action.search.MultiSearchRequest; import org.elasticsearch.action.search.MultiSearchRequest;
import org.elasticsearch.action.search.SearchAction;
import org.elasticsearch.action.search.SearchRequest; 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.PlainActionFuture;
import org.elasticsearch.action.support.WriteRequest; import org.elasticsearch.action.support.WriteRequest;
import org.elasticsearch.client.Cancellable; 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())) scriptQuery(new Script(ScriptType.INLINE, "mockscript", ScriptedBlockPlugin.SCRIPT_NAME, Collections.emptyMap()))
); );
searchRequest.setJsonEntity(Strings.toString(searchSource)); searchRequest.setJsonEntity(Strings.toString(searchSource));
verifyCancellationDuringQueryPhase(SearchAction.NAME, searchRequest); verifyCancellationDuringQueryPhase(TransportSearchAction.TYPE.name(), searchRequest);
} }
public void testAutomaticCancellationMultiSearchDuringQueryPhase() throws Exception { public void testAutomaticCancellationMultiSearchDuringQueryPhase() throws Exception {
@ -89,7 +89,7 @@ public class SearchRestCancellationIT extends HttpSmokeTestCase {
Request restRequest = new Request("POST", "/_msearch"); Request restRequest = new Request("POST", "/_msearch");
byte[] requestBody = MultiSearchRequest.writeMultiLineFormat(multiSearchRequest, contentType.xContent()); byte[] requestBody = MultiSearchRequest.writeMultiLineFormat(multiSearchRequest, contentType.xContent());
restRequest.setEntity(new NByteArrayEntity(requestBody, createContentType(contentType))); restRequest.setEntity(new NByteArrayEntity(requestBody, createContentType(contentType)));
verifyCancellationDuringQueryPhase(MultiSearchAction.NAME, restRequest); verifyCancellationDuringQueryPhase(TransportMultiSearchAction.TYPE.name(), restRequest);
} }
void verifyCancellationDuringQueryPhase(String searchAction, Request searchRequest) throws Exception { 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()) new Script(ScriptType.INLINE, "mockscript", ScriptedBlockPlugin.SCRIPT_NAME, Collections.emptyMap())
); );
searchRequest.setJsonEntity(Strings.toString(searchSource)); searchRequest.setJsonEntity(Strings.toString(searchSource));
verifyCancellationDuringFetchPhase(SearchAction.NAME, searchRequest); verifyCancellationDuringFetchPhase(TransportSearchAction.TYPE.name(), searchRequest);
} }
public void testAutomaticCancellationMultiSearchDuringFetchPhase() throws Exception { public void testAutomaticCancellationMultiSearchDuringFetchPhase() throws Exception {
@ -132,7 +132,7 @@ public class SearchRestCancellationIT extends HttpSmokeTestCase {
Request restRequest = new Request("POST", "/_msearch"); Request restRequest = new Request("POST", "/_msearch");
byte[] requestBody = MultiSearchRequest.writeMultiLineFormat(multiSearchRequest, contentType.xContent()); byte[] requestBody = MultiSearchRequest.writeMultiLineFormat(multiSearchRequest, contentType.xContent());
restRequest.setEntity(new NByteArrayEntity(requestBody, createContentType(contentType))); restRequest.setEntity(new NByteArrayEntity(requestBody, createContentType(contentType)));
verifyCancellationDuringFetchPhase(MultiSearchAction.NAME, restRequest); verifyCancellationDuringFetchPhase(TransportMultiSearchAction.TYPE.name(), restRequest);
} }
void verifyCancellationDuringFetchPhase(String searchAction, Request searchRequest) throws Exception { void verifyCancellationDuringFetchPhase(String searchAction, Request searchRequest) throws Exception {

View file

@ -45,8 +45,8 @@ import org.elasticsearch.action.admin.indices.validate.query.ValidateQueryReques
import org.elasticsearch.action.bulk.BulkAction; import org.elasticsearch.action.bulk.BulkAction;
import org.elasticsearch.action.bulk.BulkRequest; import org.elasticsearch.action.bulk.BulkRequest;
import org.elasticsearch.action.delete.DeleteRequest; import org.elasticsearch.action.delete.DeleteRequest;
import org.elasticsearch.action.explain.ExplainAction;
import org.elasticsearch.action.explain.ExplainRequest; import org.elasticsearch.action.explain.ExplainRequest;
import org.elasticsearch.action.explain.TransportExplainAction;
import org.elasticsearch.action.fieldcaps.FieldCapabilitiesAction; import org.elasticsearch.action.fieldcaps.FieldCapabilitiesAction;
import org.elasticsearch.action.fieldcaps.FieldCapabilitiesRequest; import org.elasticsearch.action.fieldcaps.FieldCapabilitiesRequest;
import org.elasticsearch.action.get.GetAction; import org.elasticsearch.action.get.GetAction;
@ -317,7 +317,7 @@ public class IndicesRequestIT extends ESIntegTestCase {
} }
public void testExplain() { public void testExplain() {
String explainShardAction = ExplainAction.NAME + "[s]"; String explainShardAction = TransportExplainAction.TYPE.name() + "[s]";
interceptTransportActions(explainShardAction); interceptTransportActions(explainShardAction);
ExplainRequest explainRequest = new ExplainRequest(randomIndexOrAlias(), "id").query(QueryBuilders.matchAllQuery()); ExplainRequest explainRequest = new ExplainRequest(randomIndexOrAlias(), "id").query(QueryBuilders.matchAllQuery());

View file

@ -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.admin.indices.validate.query.ValidateQueryAction;
import org.elasticsearch.action.bulk.BulkAction; import org.elasticsearch.action.bulk.BulkAction;
import org.elasticsearch.action.index.IndexAction; import org.elasticsearch.action.index.IndexAction;
import org.elasticsearch.action.search.SearchAction;
import org.elasticsearch.action.search.SearchTransportService; import org.elasticsearch.action.search.SearchTransportService;
import org.elasticsearch.action.search.TransportSearchAction;
import org.elasticsearch.action.support.WriteRequest; import org.elasticsearch.action.support.WriteRequest;
import org.elasticsearch.action.support.replication.ReplicationResponse; import org.elasticsearch.action.support.replication.ReplicationResponse;
import org.elasticsearch.action.support.replication.TransportReplicationActionTests; import org.elasticsearch.action.support.replication.TransportReplicationActionTests;
@ -349,8 +349,8 @@ public class TasksIT extends ESIntegTestCase {
} }
public void testSearchTaskDescriptions() { public void testSearchTaskDescriptions() {
registerTaskManagerListeners(SearchAction.NAME); // main task registerTaskManagerListeners(TransportSearchAction.TYPE.name()); // main task
registerTaskManagerListeners(SearchAction.NAME + "[*]"); // shard task registerTaskManagerListeners(TransportSearchAction.TYPE.name() + "[*]"); // shard task
createIndex("test"); createIndex("test");
ensureGreen("test"); // Make sure all shards are allocated to catch replication tasks ensureGreen("test"); // Make sure all shards are allocated to catch replication tasks
prepareIndex("test").setId("test_id") prepareIndex("test").setId("test_id")
@ -365,14 +365,14 @@ public class TasksIT extends ESIntegTestCase {
assertNoFailures(client().filterWithHeader(headers).prepareSearch("test").setQuery(QueryBuilders.matchAllQuery())); assertNoFailures(client().filterWithHeader(headers).prepareSearch("test").setQuery(QueryBuilders.matchAllQuery()));
// the search operation should produce one main task // the search operation should produce one main task
List<TaskInfo> mainTask = findEvents(SearchAction.NAME, Tuple::v1); List<TaskInfo> mainTask = findEvents(TransportSearchAction.TYPE.name(), Tuple::v1);
assertEquals(1, mainTask.size()); assertEquals(1, mainTask.size());
assertThat(mainTask.get(0).description(), startsWith("indices[test], search_type[")); assertThat(mainTask.get(0).description(), startsWith("indices[test], search_type["));
assertThat(mainTask.get(0).description(), containsString("\"query\":{\"match_all\"")); assertThat(mainTask.get(0).description(), containsString("\"query\":{\"match_all\""));
assertTaskHeaders(mainTask.get(0)); assertTaskHeaders(mainTask.get(0));
// check that if we have any shard-level requests they all have non-zero length description // check that if we have any shard-level requests they all have non-zero length description
List<TaskInfo> shardTasks = findEvents(SearchAction.NAME + "[*]", Tuple::v1); List<TaskInfo> shardTasks = findEvents(TransportSearchAction.TYPE.name() + "[*]", Tuple::v1);
for (TaskInfo taskInfo : shardTasks) { for (TaskInfo taskInfo : shardTasks) {
assertThat(taskInfo.parentTaskId(), notNullValue()); assertThat(taskInfo.parentTaskId(), notNullValue());
assertEquals(mainTask.get(0).taskId(), taskInfo.parentTaskId()); assertEquals(mainTask.get(0).taskId(), taskInfo.parentTaskId());

View file

@ -202,11 +202,11 @@ public class CCSPointInTimeIT extends AbstractMultiClustersTestCase {
private String openPointInTime(String[] indices, TimeValue keepAlive) { private String openPointInTime(String[] indices, TimeValue keepAlive) {
OpenPointInTimeRequest request = new OpenPointInTimeRequest(indices).keepAlive(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(); return response.getPointInTimeId();
} }
private void closePointInTime(String readerId) { private void closePointInTime(String readerId) {
client().execute(ClosePointInTimeAction.INSTANCE, new ClosePointInTimeRequest(readerId)).actionGet(); client().execute(TransportClosePointInTimeAction.TYPE, new ClosePointInTimeRequest(readerId)).actionGet();
} }
} }

View file

@ -168,7 +168,7 @@ public class PointInTimeIT extends ESIntegTestCase {
{ {
OpenPointInTimeRequest request = new OpenPointInTimeRequest("*").keepAlive(TimeValue.timeValueMinutes(2)); 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 { try {
SearchContextId searchContextId = SearchContextId.decode(writableRegistry(), response.getPointInTimeId()); SearchContextId searchContextId = SearchContextId.decode(writableRegistry(), response.getPointInTimeId());
String[] actualIndices = searchContextId.getActualIndices(); String[] actualIndices = searchContextId.getActualIndices();
@ -180,7 +180,7 @@ public class PointInTimeIT extends ESIntegTestCase {
{ {
OpenPointInTimeRequest request = new OpenPointInTimeRequest("*").keepAlive(TimeValue.timeValueMinutes(2)); OpenPointInTimeRequest request = new OpenPointInTimeRequest("*").keepAlive(TimeValue.timeValueMinutes(2));
request.indexFilter(new RangeQueryBuilder("@timestamp").gte("2023-03-01")); 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(); String pitId = response.getPointInTimeId();
try { try {
SearchContextId searchContextId = SearchContextId.decode(writableRegistry(), pitId); SearchContextId searchContextId = SearchContextId.decode(writableRegistry(), pitId);
@ -333,8 +333,8 @@ public class PointInTimeIT extends ESIntegTestCase {
public void testAllowNoIndex() { public void testAllowNoIndex() {
var request = new OpenPointInTimeRequest("my_index").indicesOptions(IndicesOptions.LENIENT_EXPAND_OPEN) var request = new OpenPointInTimeRequest("my_index").indicesOptions(IndicesOptions.LENIENT_EXPAND_OPEN)
.keepAlive(TimeValue.timeValueMinutes(between(1, 10))); .keepAlive(TimeValue.timeValueMinutes(between(1, 10)));
String pit = client().execute(OpenPointInTimeAction.INSTANCE, request).actionGet().getPointInTimeId(); String pit = client().execute(TransportOpenPointInTimeAction.TYPE, request).actionGet().getPointInTimeId();
var closeResp = client().execute(ClosePointInTimeAction.INSTANCE, new ClosePointInTimeRequest(pit)).actionGet(); var closeResp = client().execute(TransportClosePointInTimeAction.TYPE, new ClosePointInTimeRequest(pit)).actionGet();
assertThat(closeResp.status(), equalTo(RestStatus.OK)); assertThat(closeResp.status(), equalTo(RestStatus.OK));
} }
@ -478,8 +478,11 @@ public class PointInTimeIT extends ESIntegTestCase {
} }
public void testCloseInvalidPointInTime() { public void testCloseInvalidPointInTime() {
expectThrows(Exception.class, () -> client().execute(ClosePointInTimeAction.INSTANCE, new ClosePointInTimeRequest("")).actionGet()); expectThrows(
List<TaskInfo> tasks = clusterAdmin().prepareListTasks().setActions(ClosePointInTimeAction.NAME).get().getTasks(); Exception.class,
() -> client().execute(TransportClosePointInTimeAction.TYPE, new ClosePointInTimeRequest("")).actionGet()
);
List<TaskInfo> tasks = clusterAdmin().prepareListTasks().setActions(TransportClosePointInTimeAction.TYPE.name()).get().getTasks();
assertThat(tasks, empty()); assertThat(tasks, empty());
} }
@ -520,7 +523,7 @@ public class PointInTimeIT extends ESIntegTestCase {
OpenPointInTimeRequest request = new OpenPointInTimeRequest("test").keepAlive(TimeValue.timeValueMinutes(1)); OpenPointInTimeRequest request = new OpenPointInTimeRequest("test").keepAlive(TimeValue.timeValueMinutes(1));
request.maxConcurrentShardRequests(maxConcurrentRequests); request.maxConcurrentShardRequests(maxConcurrentRequests);
PlainActionFuture<OpenPointInTimeResponse> future = new PlainActionFuture<>(); PlainActionFuture<OpenPointInTimeResponse> future = new PlainActionFuture<>();
client().execute(OpenPointInTimeAction.INSTANCE, request, future); client().execute(TransportOpenPointInTimeAction.TYPE, request, future);
assertTrue(sentLatch.await(1, TimeUnit.MINUTES)); assertTrue(sentLatch.await(1, TimeUnit.MINUTES));
readyLatch.countDown(); readyLatch.countDown();
closePointInTime(future.actionGet().getPointInTimeId()); closePointInTime(future.actionGet().getPointInTimeId());
@ -584,11 +587,11 @@ public class PointInTimeIT extends ESIntegTestCase {
private String openPointInTime(String[] indices, TimeValue keepAlive) { private String openPointInTime(String[] indices, TimeValue keepAlive) {
OpenPointInTimeRequest request = new OpenPointInTimeRequest(indices).keepAlive(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(); return response.getPointInTimeId();
} }
private void closePointInTime(String readerId) { private void closePointInTime(String readerId) {
client().execute(ClosePointInTimeAction.INSTANCE, new ClosePointInTimeRequest(readerId)).actionGet(); client().execute(TransportClosePointInTimeAction.TYPE, new ClosePointInTimeRequest(readerId)).actionGet();
} }
} }

View file

@ -162,7 +162,7 @@ public class SearchProgressActionListenerIT extends ESSingleNodeTestCase {
throw new AssertionError(); throw new AssertionError();
} }
}; };
client.executeLocally(SearchAction.INSTANCE, new SearchRequest(request) { client.executeLocally(TransportSearchAction.TYPE, new SearchRequest(request) {
@Override @Override
public SearchTask createTask(long id, String type, String action, TaskId parentTaskId, Map<String, String> headers) { public SearchTask createTask(long id, String type, String action, TaskId parentTaskId, Map<String, String> headers) {
SearchTask task = super.createTask(id, type, action, parentTaskId, headers); SearchTask task = super.createTask(id, type, action, parentTaskId, headers);

View file

@ -70,7 +70,7 @@ public class SearchShardsIT extends ESIntegTestCase {
randomBoolean(), randomBoolean(),
randomBoolean() ? null : randomAlphaOfLength(10) 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)); assertThat(resp.getGroups(), hasSize(indicesWithData + indicesWithoutData));
int skipped = 0; int skipped = 0;
for (SearchShardsGroup g : resp.getGroups()) { for (SearchShardsGroup g : resp.getGroups()) {
@ -97,7 +97,7 @@ public class SearchShardsIT extends ESIntegTestCase {
randomBoolean(), randomBoolean(),
randomBoolean() ? null : randomAlphaOfLength(10) 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)); assertThat(resp.getGroups(), hasSize(indicesWithData + indicesWithoutData));
for (SearchShardsGroup g : resp.getGroups()) { for (SearchShardsGroup g : resp.getGroups()) {
assertFalse(g.skipped()); assertFalse(g.skipped());
@ -137,7 +137,7 @@ public class SearchShardsIT extends ESIntegTestCase {
randomBoolean(), randomBoolean(),
randomBoolean() ? null : randomAlphaOfLength(10) 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())); assertThat(searchShardsResponse.getGroups(), hasSize(searchResponse.getTotalShards()));
long skippedShards = searchShardsResponse.getGroups().stream().filter(SearchShardsGroup::skipped).count(); long skippedShards = searchShardsResponse.getGroups().stream().filter(SearchShardsGroup::skipped).count();
@ -182,7 +182,7 @@ public class SearchShardsIT extends ESIntegTestCase {
randomBoolean(), randomBoolean(),
null null
); );
SearchShardsResponse resp = client().execute(SearchShardsAction.INSTANCE, request).actionGet(); SearchShardsResponse resp = client().execute(TransportSearchShardsAction.TYPE, request).actionGet();
assertThat(resp.getGroups(), hasSize(totalShards)); assertThat(resp.getGroups(), hasSize(totalShards));
for (SearchShardsGroup group : resp.getGroups()) { for (SearchShardsGroup group : resp.getGroups()) {
assertFalse(group.skipped()); assertFalse(group.skipped());

View file

@ -14,10 +14,10 @@ import org.elasticsearch.action.ActionResponse;
import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsGroup; import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsGroup;
import org.elasticsearch.action.admin.indices.refresh.RefreshResponse; import org.elasticsearch.action.admin.indices.refresh.RefreshResponse;
import org.elasticsearch.action.admin.indices.refresh.TransportUnpromotableShardRefreshAction; 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.ClosePointInTimeRequest;
import org.elasticsearch.action.search.OpenPointInTimeAction;
import org.elasticsearch.action.search.OpenPointInTimeRequest; 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.ClusterChangedEvent;
import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.ClusterStateListener; import org.elasticsearch.cluster.ClusterStateListener;
@ -522,7 +522,7 @@ public class ShardRoutingRoleIT extends ESIntegTestCase {
// do nothing // do nothing
} }
} }
String pitId = client().execute(OpenPointInTimeAction.INSTANCE, openRequest).actionGet().getPointInTimeId(); String pitId = client().execute(TransportOpenPointInTimeAction.TYPE, openRequest).actionGet().getPointInTimeId();
try { try {
final var profileResults = prepareSearch().setPointInTime(new PointInTimeBuilder(pitId)) final var profileResults = prepareSearch().setPointInTime(new PointInTimeBuilder(pitId))
.setProfile(true) .setProfile(true)
@ -533,7 +533,7 @@ public class ShardRoutingRoleIT extends ESIntegTestCase {
assertThat(profileKey, in(searchShardProfileKeys)); assertThat(profileKey, in(searchShardProfileKeys));
} }
} finally { } finally {
client().execute(ClosePointInTimeAction.INSTANCE, new ClosePointInTimeRequest(pitId)); client().execute(TransportClosePointInTimeAction.TYPE, new ClosePointInTimeRequest(pitId));
} }
} }
// search-shards API // search-shards API

View file

@ -11,17 +11,17 @@ package org.elasticsearch.search;
import org.apache.lucene.tests.util.LuceneTestCase; import org.apache.lucene.tests.util.LuceneTestCase;
import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.action.ActionFuture; import org.elasticsearch.action.ActionFuture;
import org.elasticsearch.action.search.MultiSearchAction;
import org.elasticsearch.action.search.MultiSearchResponse; import org.elasticsearch.action.search.MultiSearchResponse;
import org.elasticsearch.action.search.SearchAction;
import org.elasticsearch.action.search.SearchPhaseExecutionException; import org.elasticsearch.action.search.SearchPhaseExecutionException;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.search.SearchScrollAction;
import org.elasticsearch.action.search.SearchShardTask; import org.elasticsearch.action.search.SearchShardTask;
import org.elasticsearch.action.search.SearchTask; import org.elasticsearch.action.search.SearchTask;
import org.elasticsearch.action.search.SearchTransportService; import org.elasticsearch.action.search.SearchTransportService;
import org.elasticsearch.action.search.SearchType; import org.elasticsearch.action.search.SearchType;
import org.elasticsearch.action.search.ShardSearchFailure; 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.common.Strings;
import org.elasticsearch.core.TimeValue; import org.elasticsearch.core.TimeValue;
import org.elasticsearch.script.Script; import org.elasticsearch.script.Script;
@ -71,7 +71,7 @@ public class SearchCancellationIT extends AbstractSearchCancellationTestCase {
).execute(); ).execute();
awaitForBlock(plugins); awaitForBlock(plugins);
cancelSearch(SearchAction.NAME); cancelSearch(TransportSearchAction.TYPE.name());
disableBlocks(plugins); disableBlocks(plugins);
logger.info("Segments {}", Strings.toString(indicesAdmin().prepareSegments("test").get())); logger.info("Segments {}", Strings.toString(indicesAdmin().prepareSegments("test").get()));
ensureSearchWasCancelled(searchResponse); ensureSearchWasCancelled(searchResponse);
@ -89,7 +89,7 @@ public class SearchCancellationIT extends AbstractSearchCancellationTestCase {
).execute(); ).execute();
awaitForBlock(plugins); awaitForBlock(plugins);
cancelSearch(SearchAction.NAME); cancelSearch(TransportSearchAction.TYPE.name());
disableBlocks(plugins); disableBlocks(plugins);
logger.info("Segments {}", Strings.toString(indicesAdmin().prepareSegments("test").get())); logger.info("Segments {}", Strings.toString(indicesAdmin().prepareSegments("test").get()));
ensureSearchWasCancelled(searchResponse); ensureSearchWasCancelled(searchResponse);
@ -134,7 +134,7 @@ public class SearchCancellationIT extends AbstractSearchCancellationTestCase {
) )
.execute(); .execute();
awaitForBlock(plugins); awaitForBlock(plugins);
cancelSearch(SearchAction.NAME); cancelSearch(TransportSearchAction.TYPE.name());
disableBlocks(plugins); disableBlocks(plugins);
ensureSearchWasCancelled(searchResponse); ensureSearchWasCancelled(searchResponse);
} }
@ -151,7 +151,7 @@ public class SearchCancellationIT extends AbstractSearchCancellationTestCase {
.execute(); .execute();
awaitForBlock(plugins); awaitForBlock(plugins);
cancelSearch(SearchAction.NAME); cancelSearch(TransportSearchAction.TYPE.name());
disableBlocks(plugins); disableBlocks(plugins);
SearchResponse response = ensureSearchWasCancelled(searchResponse); SearchResponse response = ensureSearchWasCancelled(searchResponse);
if (response != null) { if (response != null) {
@ -191,7 +191,7 @@ public class SearchCancellationIT extends AbstractSearchCancellationTestCase {
.execute(); .execute();
awaitForBlock(plugins); awaitForBlock(plugins);
cancelSearch(SearchScrollAction.NAME); cancelSearch(TransportSearchScrollAction.TYPE.name());
disableBlocks(plugins); disableBlocks(plugins);
SearchResponse response = ensureSearchWasCancelled(scrollResponse); SearchResponse response = ensureSearchWasCancelled(scrollResponse);
@ -215,7 +215,7 @@ public class SearchCancellationIT extends AbstractSearchCancellationTestCase {
) )
.execute(); .execute();
awaitForBlock(plugins); awaitForBlock(plugins);
cancelSearch(MultiSearchAction.NAME); cancelSearch(TransportMultiSearchAction.TYPE.name());
disableBlocks(plugins); disableBlocks(plugins);
for (MultiSearchResponse.Item item : msearchResponse.actionGet()) { for (MultiSearchResponse.Item item : msearchResponse.actionGet()) {
if (item.getFailure() != null) { if (item.getFailure() != null) {
@ -302,7 +302,7 @@ public class SearchCancellationIT extends AbstractSearchCancellationTestCase {
for (String nodeName : internalCluster().getNodeNames()) { for (String nodeName : internalCluster().getNodeNames()) {
TransportService transportService = internalCluster().getInstance(TransportService.class, nodeName); TransportService transportService = internalCluster().getInstance(TransportService.class, nodeName);
for (Task task : transportService.getTaskManager().getCancellableTasks().values()) { for (Task task : transportService.getTaskManager().getCancellableTasks().values()) {
if (task.getAction().equals(SearchAction.NAME)) { if (task.getAction().equals(TransportSearchAction.TYPE.name())) {
tasks.add((SearchTask) task); tasks.add((SearchTask) task);
} }
} }

View file

@ -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.CancelTasksRequest;
import org.elasticsearch.action.admin.cluster.node.tasks.cancel.CancelTasksResponse; import org.elasticsearch.action.admin.cluster.node.tasks.cancel.CancelTasksResponse;
import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.action.search.SearchAction;
import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.search.SearchShardsAction;
import org.elasticsearch.action.search.SearchShardsGroup; import org.elasticsearch.action.search.SearchShardsGroup;
import org.elasticsearch.action.search.SearchShardsRequest; import org.elasticsearch.action.search.SearchShardsRequest;
import org.elasticsearch.action.search.SearchShardsResponse; 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.IndicesOptions;
import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.action.support.PlainActionFuture;
import org.elasticsearch.action.support.WriteRequest; import org.elasticsearch.action.support.WriteRequest;
@ -240,7 +240,7 @@ public class CrossClusterIT extends AbstractMultiClustersTestCase {
final TaskInfo rootTask = client().admin() final TaskInfo rootTask = client().admin()
.cluster() .cluster()
.prepareListTasks() .prepareListTasks()
.setActions(SearchAction.INSTANCE.name()) .setActions(TransportSearchAction.TYPE.name())
.get() .get()
.getTasks() .getTasks()
.stream() .stream()
@ -274,7 +274,7 @@ public class CrossClusterIT extends AbstractMultiClustersTestCase {
for (TransportService transportService : transportServices) { for (TransportService transportService : transportServices) {
Collection<CancellableTask> cancellableTasks = transportService.getTaskManager().getCancellableTasks().values(); Collection<CancellableTask> cancellableTasks = transportService.getTaskManager().getCancellableTasks().values();
for (CancellableTask cancellableTask : cancellableTasks) { for (CancellableTask cancellableTask : cancellableTasks) {
if (cancellableTask.getAction().contains(SearchAction.INSTANCE.name())) { if (cancellableTask.getAction().contains(TransportSearchAction.TYPE.name())) {
assertTrue(cancellableTask.getDescription(), cancellableTask.isCancelled()); assertTrue(cancellableTask.getDescription(), cancellableTask.isCancelled());
} }
} }
@ -528,7 +528,7 @@ public class CrossClusterIT extends AbstractMultiClustersTestCase {
{ {
QueryBuilder query = new TermQueryBuilder("_index", "cluster_a:my_index"); QueryBuilder query = new TermQueryBuilder("_index", "cluster_a:my_index");
SearchShardsRequest request = new SearchShardsRequest(indices, indicesOptions, query, null, null, randomBoolean(), "cluster_a"); 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)); assertThat(resp.getGroups(), hasSize(numShards));
for (SearchShardsGroup group : resp.getGroups()) { for (SearchShardsGroup group : resp.getGroups()) {
assertFalse(group.skipped()); assertFalse(group.skipped());
@ -545,7 +545,7 @@ public class CrossClusterIT extends AbstractMultiClustersTestCase {
randomBoolean(), randomBoolean(),
randomFrom("cluster_b", null) 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)); assertThat(resp.getGroups(), hasSize(numShards));
for (SearchShardsGroup group : resp.getGroups()) { for (SearchShardsGroup group : resp.getGroups()) {
assertTrue(group.skipped()); assertTrue(group.skipped());
@ -562,7 +562,7 @@ public class CrossClusterIT extends AbstractMultiClustersTestCase {
randomBoolean(), randomBoolean(),
randomFrom("cluster_a", "cluster_b", null) 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)); assertThat(resp.getGroups(), hasSize(numShards));
for (SearchShardsGroup group : resp.getGroups()) { for (SearchShardsGroup group : resp.getGroups()) {
assertTrue(group.skipped()); assertTrue(group.skipped());

View file

@ -12,15 +12,15 @@ import org.elasticsearch.action.admin.indices.create.CreateIndexRequestBuilder;
import org.elasticsearch.action.bulk.BulkRequestBuilder; import org.elasticsearch.action.bulk.BulkRequestBuilder;
import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.index.IndexRequestBuilder;
import org.elasticsearch.action.search.ClosePointInTimeAction;
import org.elasticsearch.action.search.ClosePointInTimeRequest; import org.elasticsearch.action.search.ClosePointInTimeRequest;
import org.elasticsearch.action.search.OpenPointInTimeAction;
import org.elasticsearch.action.search.OpenPointInTimeRequest; import org.elasticsearch.action.search.OpenPointInTimeRequest;
import org.elasticsearch.action.search.SearchPhaseExecutionException; import org.elasticsearch.action.search.SearchPhaseExecutionException;
import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.search.SearchRequestBuilder; import org.elasticsearch.action.search.SearchRequestBuilder;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.search.ShardSearchFailure; 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.action.support.WriteRequest;
import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.cluster.metadata.IndexMetadata;
import org.elasticsearch.common.Randomness; import org.elasticsearch.common.Randomness;
@ -456,7 +456,7 @@ public class SearchAfterIT extends ESIntegTestCase {
String pitID; String pitID;
{ {
OpenPointInTimeRequest openPITRequest = new OpenPointInTimeRequest("test").keepAlive(TimeValue.timeValueMinutes(5)); 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( SearchRequest searchRequest = new SearchRequest("test").source(
new SearchSourceBuilder().pointInTimeBuilder(new PointInTimeBuilder(pitID).setKeepAlive(TimeValue.timeValueMinutes(5))) new SearchSourceBuilder().pointInTimeBuilder(new PointInTimeBuilder(pitID).setKeepAlive(TimeValue.timeValueMinutes(5)))
.sort("timestamp") .sort("timestamp")
@ -483,14 +483,14 @@ public class SearchAfterIT extends ESIntegTestCase {
} while (resp.getHits().getHits().length > 0); } while (resp.getHits().getHits().length > 0);
assertThat(foundHits, equalTo(timestamps.size())); assertThat(foundHits, equalTo(timestamps.size()));
} finally { } 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 // search_after without sort with point in time
{ {
OpenPointInTimeRequest openPITRequest = new OpenPointInTimeRequest("test").keepAlive(TimeValue.timeValueMinutes(5)); 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( SearchRequest searchRequest = new SearchRequest("test").source(
new SearchSourceBuilder().pointInTimeBuilder(new PointInTimeBuilder(pitID).setKeepAlive(TimeValue.timeValueMinutes(5))) new SearchSourceBuilder().pointInTimeBuilder(new PointInTimeBuilder(pitID).setKeepAlive(TimeValue.timeValueMinutes(5)))
.sort(SortBuilders.pitTiebreaker()) .sort(SortBuilders.pitTiebreaker())
@ -517,7 +517,7 @@ public class SearchAfterIT extends ESIntegTestCase {
Collections.sort(foundSeqNos); Collections.sort(foundSeqNos);
assertThat(foundSeqNos, equalTo(timestamps)); assertThat(foundSeqNos, equalTo(timestamps));
} finally { } finally {
client().execute(ClosePointInTimeAction.INSTANCE, new ClosePointInTimeRequest(pitID)).actionGet(); client().execute(TransportClosePointInTimeAction.TYPE, new ClosePointInTimeRequest(pitID)).actionGet();
} }
} }
} }

View file

@ -10,14 +10,14 @@ package org.elasticsearch.search.slice;
import org.elasticsearch.action.admin.indices.alias.IndicesAliasesRequest; import org.elasticsearch.action.admin.indices.alias.IndicesAliasesRequest;
import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.index.IndexRequestBuilder;
import org.elasticsearch.action.search.ClosePointInTimeAction;
import org.elasticsearch.action.search.ClosePointInTimeRequest; import org.elasticsearch.action.search.ClosePointInTimeRequest;
import org.elasticsearch.action.search.OpenPointInTimeAction;
import org.elasticsearch.action.search.OpenPointInTimeRequest; import org.elasticsearch.action.search.OpenPointInTimeRequest;
import org.elasticsearch.action.search.OpenPointInTimeResponse; import org.elasticsearch.action.search.OpenPointInTimeResponse;
import org.elasticsearch.action.search.SearchPhaseExecutionException; import org.elasticsearch.action.search.SearchPhaseExecutionException;
import org.elasticsearch.action.search.SearchRequestBuilder; import org.elasticsearch.action.search.SearchRequestBuilder;
import org.elasticsearch.action.search.SearchResponse; 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.Strings;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.core.TimeValue; import org.elasticsearch.core.TimeValue;
@ -197,7 +197,7 @@ public class SearchSliceIT extends ESIntegTestCase {
for (String field : new String[] { null, "random_int", "static_int" }) { for (String field : new String[] { null, "random_int", "static_int" }) {
// Open point-in-time reader // Open point-in-time reader
OpenPointInTimeRequest request = new OpenPointInTimeRequest("test").keepAlive(TimeValue.timeValueSeconds(10)); 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(); String pointInTimeId = response.getPointInTimeId();
// Test sort on document IDs // Test sort on document IDs
@ -206,7 +206,7 @@ public class SearchSliceIT extends ESIntegTestCase {
assertSearchSlicesWithPointInTime(field, "random_int", pointInTimeId, max, numDocs); assertSearchSlicesWithPointInTime(field, "random_int", pointInTimeId, max, numDocs);
// Close point-in-time reader // Close point-in-time reader
client().execute(ClosePointInTimeAction.INSTANCE, new ClosePointInTimeRequest(pointInTimeId)).actionGet(); client().execute(TransportClosePointInTimeAction.TYPE, new ClosePointInTimeRequest(pointInTimeId)).actionGet();
} }
} }

View file

@ -208,7 +208,6 @@ import org.elasticsearch.action.bulk.TransportShardBulkAction;
import org.elasticsearch.action.bulk.TransportSimulateBulkAction; import org.elasticsearch.action.bulk.TransportSimulateBulkAction;
import org.elasticsearch.action.delete.DeleteAction; import org.elasticsearch.action.delete.DeleteAction;
import org.elasticsearch.action.delete.TransportDeleteAction; import org.elasticsearch.action.delete.TransportDeleteAction;
import org.elasticsearch.action.explain.ExplainAction;
import org.elasticsearch.action.explain.TransportExplainAction; import org.elasticsearch.action.explain.TransportExplainAction;
import org.elasticsearch.action.fieldcaps.FieldCapabilitiesAction; import org.elasticsearch.action.fieldcaps.FieldCapabilitiesAction;
import org.elasticsearch.action.fieldcaps.TransportFieldCapabilitiesAction; 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.PutPipelineTransportAction;
import org.elasticsearch.action.ingest.SimulatePipelineAction; import org.elasticsearch.action.ingest.SimulatePipelineAction;
import org.elasticsearch.action.ingest.SimulatePipelineTransportAction; 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.RestClosePointInTimeAction;
import org.elasticsearch.action.search.RestOpenPointInTimeAction; 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.TransportClearScrollAction;
import org.elasticsearch.action.search.TransportClosePointInTimeAction; import org.elasticsearch.action.search.TransportClosePointInTimeAction;
import org.elasticsearch.action.search.TransportMultiSearchAction; import org.elasticsearch.action.search.TransportMultiSearchAction;
@ -763,14 +755,14 @@ public class ActionModule extends AbstractModule {
actions.register(BulkAction.INSTANCE, TransportBulkAction.class); actions.register(BulkAction.INSTANCE, TransportBulkAction.class);
actions.register(SimulateBulkAction.INSTANCE, TransportSimulateBulkAction.class); actions.register(SimulateBulkAction.INSTANCE, TransportSimulateBulkAction.class);
actions.register(TransportShardBulkAction.TYPE, TransportShardBulkAction.class); actions.register(TransportShardBulkAction.TYPE, TransportShardBulkAction.class);
actions.register(SearchAction.INSTANCE, TransportSearchAction.class); actions.register(TransportSearchAction.TYPE, TransportSearchAction.class);
actions.register(SearchScrollAction.INSTANCE, TransportSearchScrollAction.class); actions.register(TransportSearchScrollAction.TYPE, TransportSearchScrollAction.class);
actions.register(OpenPointInTimeAction.INSTANCE, TransportOpenPointInTimeAction.class); actions.register(TransportOpenPointInTimeAction.TYPE, TransportOpenPointInTimeAction.class);
actions.register(ClosePointInTimeAction.INSTANCE, TransportClosePointInTimeAction.class); actions.register(TransportClosePointInTimeAction.TYPE, TransportClosePointInTimeAction.class);
actions.register(SearchShardsAction.INSTANCE, TransportSearchShardsAction.class); actions.register(TransportSearchShardsAction.TYPE, TransportSearchShardsAction.class);
actions.register(MultiSearchAction.INSTANCE, TransportMultiSearchAction.class); actions.register(TransportMultiSearchAction.TYPE, TransportMultiSearchAction.class);
actions.register(ExplainAction.INSTANCE, TransportExplainAction.class); actions.register(TransportExplainAction.TYPE, TransportExplainAction.class);
actions.register(ClearScrollAction.INSTANCE, TransportClearScrollAction.class); actions.register(TransportClearScrollAction.TYPE, TransportClearScrollAction.class);
actions.register(RecoveryAction.INSTANCE, TransportRecoveryAction.class); actions.register(RecoveryAction.INSTANCE, TransportRecoveryAction.class);
actions.register(NodesReloadSecureSettingsAction.INSTANCE, TransportNodesReloadSecureSettingsAction.class); actions.register(NodesReloadSecureSettingsAction.INSTANCE, TransportNodesReloadSecureSettingsAction.class);
actions.register(AutoCreateAction.INSTANCE, AutoCreateAction.TransportAction.class); actions.register(AutoCreateAction.INSTANCE, AutoCreateAction.TransportAction.class);

View file

@ -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<ExplainResponse> {
public static final ExplainAction INSTANCE = new ExplainAction();
public static final String NAME = "indices:data/read/explain";
private ExplainAction() {
super(NAME, ExplainResponse::new);
}
}

View file

@ -8,6 +8,7 @@
package org.elasticsearch.action.explain; package org.elasticsearch.action.explain;
import org.elasticsearch.action.ActionType;
import org.elasticsearch.action.support.single.shard.SingleShardOperationRequestBuilder; import org.elasticsearch.action.support.single.shard.SingleShardOperationRequestBuilder;
import org.elasticsearch.client.internal.ElasticsearchClient; import org.elasticsearch.client.internal.ElasticsearchClient;
import org.elasticsearch.common.Strings; import org.elasticsearch.common.Strings;
@ -20,11 +21,7 @@ import org.elasticsearch.search.fetch.subphase.FetchSourceContext;
*/ */
public class ExplainRequestBuilder extends SingleShardOperationRequestBuilder<ExplainRequest, ExplainResponse, ExplainRequestBuilder> { public class ExplainRequestBuilder extends SingleShardOperationRequestBuilder<ExplainRequest, ExplainResponse, ExplainRequestBuilder> {
ExplainRequestBuilder(ElasticsearchClient client, ExplainAction action) { public ExplainRequestBuilder(ElasticsearchClient client, ActionType<ExplainResponse> action, String index, String id) {
super(client, action, new ExplainRequest());
}
public ExplainRequestBuilder(ElasticsearchClient client, ExplainAction action, String index, String id) {
super(client, action, new ExplainRequest().index(index).id(id)); super(client, action, new ExplainRequest().index(index).id(id));
} }

View file

@ -11,6 +11,7 @@ package org.elasticsearch.action.explain;
import org.apache.lucene.search.Explanation; import org.apache.lucene.search.Explanation;
import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.ActionType;
import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.single.shard.TransportSingleShardAction; import org.elasticsearch.action.support.single.shard.TransportSingleShardAction;
import org.elasticsearch.cluster.ClusterState; 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. // TODO: AggregatedDfs. Currently the idf can be different then when executing a normal search with explain.
public class TransportExplainAction extends TransportSingleShardAction<ExplainRequest, ExplainResponse> { public class TransportExplainAction extends TransportSingleShardAction<ExplainRequest, ExplainResponse> {
public static final ActionType<ExplainResponse> TYPE = new ActionType<>("indices:data/read/explain", ExplainResponse::new);
private final SearchService searchService; private final SearchService searchService;
@Inject @Inject
@ -60,7 +62,7 @@ public class TransportExplainAction extends TransportSingleShardAction<ExplainRe
IndexNameExpressionResolver indexNameExpressionResolver IndexNameExpressionResolver indexNameExpressionResolver
) { ) {
super( super(
ExplainAction.NAME, TYPE.name(),
threadPool, threadPool,
clusterService, clusterService,
transportService, transportService,

View file

@ -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 ClearScrollAction extends ActionType<ClearScrollResponse> {
public static final ClearScrollAction INSTANCE = new ClearScrollAction();
public static final String NAME = "indices:data/read/scroll/clear";
private ClearScrollAction() {
super(NAME, ClearScrollResponse::new);
}
}

View file

@ -9,13 +9,14 @@
package org.elasticsearch.action.search; package org.elasticsearch.action.search;
import org.elasticsearch.action.ActionRequestBuilder; import org.elasticsearch.action.ActionRequestBuilder;
import org.elasticsearch.action.ActionType;
import org.elasticsearch.client.internal.ElasticsearchClient; import org.elasticsearch.client.internal.ElasticsearchClient;
import java.util.List; import java.util.List;
public class ClearScrollRequestBuilder extends ActionRequestBuilder<ClearScrollRequest, ClearScrollResponse> { public class ClearScrollRequestBuilder extends ActionRequestBuilder<ClearScrollRequest, ClearScrollResponse> {
public ClearScrollRequestBuilder(ElasticsearchClient client, ClearScrollAction action) { public ClearScrollRequestBuilder(ElasticsearchClient client, ActionType<ClearScrollResponse> action) {
super(client, action, new ClearScrollRequest()); super(client, action, new ClearScrollRequest());
} }

View file

@ -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<ClosePointInTimeResponse> {
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);
}
}

View file

@ -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<MultiSearchResponse> {
public static final MultiSearchAction INSTANCE = new MultiSearchAction();
public static final String NAME = "indices:data/read/msearch";
private MultiSearchAction() {
super(NAME, MultiSearchResponse::new);
}
}

View file

@ -9,6 +9,7 @@
package org.elasticsearch.action.search; package org.elasticsearch.action.search;
import org.elasticsearch.action.ActionRequestBuilder; import org.elasticsearch.action.ActionRequestBuilder;
import org.elasticsearch.action.ActionType;
import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.client.internal.ElasticsearchClient; import org.elasticsearch.client.internal.ElasticsearchClient;
@ -17,7 +18,7 @@ import org.elasticsearch.client.internal.ElasticsearchClient;
*/ */
public class MultiSearchRequestBuilder extends ActionRequestBuilder<MultiSearchRequest, MultiSearchResponse> { public class MultiSearchRequestBuilder extends ActionRequestBuilder<MultiSearchRequest, MultiSearchResponse> {
public MultiSearchRequestBuilder(ElasticsearchClient client, MultiSearchAction action) { public MultiSearchRequestBuilder(ElasticsearchClient client, ActionType<MultiSearchResponse> action) {
super(client, action, new MultiSearchRequest()); super(client, action, new MultiSearchRequest());
} }

View file

@ -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<OpenPointInTimeResponse> {
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);
}
}

View file

@ -41,7 +41,7 @@ public class RestClosePointInTimeAction extends BaseRestHandler {
clearRequest = ClosePointInTimeRequest.fromXContent(parser); clearRequest = ClosePointInTimeRequest.fromXContent(parser);
} }
return channel -> client.execute( return channel -> client.execute(
ClosePointInTimeAction.INSTANCE, TransportClosePointInTimeAction.TYPE,
clearRequest, clearRequest,
new RestToXContentListener<>(channel, ClosePointInTimeResponse::status) new RestToXContentListener<>(channel, ClosePointInTimeResponse::status)
); );

View file

@ -54,13 +54,14 @@ public class RestOpenPointInTimeAction extends BaseRestHandler {
); );
openRequest.maxConcurrentShardRequests(maxConcurrentShardRequests); openRequest.maxConcurrentShardRequests(maxConcurrentShardRequests);
} }
request.withContentOrSourceParamParserOrNull(parser -> { request.withContentOrSourceParamParserOrNull(parser -> {
if (parser != null) { if (parser != null) {
PARSER.parse(parser, openRequest, 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<OpenPointInTimeRequest, Void> PARSER = new ObjectParser<>("open_point_in_time_request"); private static final ObjectParser<OpenPointInTimeRequest, Void> PARSER = new ObjectParser<>("open_point_in_time_request");

View file

@ -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<SearchResponse> {
public static final SearchAction INSTANCE = new SearchAction();
public static final String NAME = "indices:data/read/search";
private SearchAction() {
super(NAME, SearchResponse::new);
}
}

View file

@ -11,7 +11,7 @@ package org.elasticsearch.action.search;
import org.elasticsearch.action.ActionListener; 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}. * See {@link SearchProgressListener}.
*/ */
public abstract class SearchProgressActionListener extends SearchProgressListener implements ActionListener<SearchResponse> {} public abstract class SearchProgressActionListener extends SearchProgressListener implements ActionListener<SearchResponse> {}

View file

@ -23,7 +23,7 @@ import java.util.Objects;
import java.util.stream.StreamSupport; 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 { public abstract class SearchProgressListener {
private static final Logger logger = LogManager.getLogger(SearchProgressListener.class); private static final Logger logger = LogManager.getLogger(SearchProgressListener.class);

View file

@ -9,6 +9,7 @@
package org.elasticsearch.action.search; package org.elasticsearch.action.search;
import org.elasticsearch.action.ActionRequestBuilder; import org.elasticsearch.action.ActionRequestBuilder;
import org.elasticsearch.action.ActionType;
import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.client.internal.ElasticsearchClient; import org.elasticsearch.client.internal.ElasticsearchClient;
import org.elasticsearch.core.Nullable; import org.elasticsearch.core.Nullable;
@ -41,7 +42,7 @@ import java.util.Map;
*/ */
public class SearchRequestBuilder extends ActionRequestBuilder<SearchRequest, SearchResponse> { public class SearchRequestBuilder extends ActionRequestBuilder<SearchRequest, SearchResponse> {
public SearchRequestBuilder(ElasticsearchClient client, SearchAction action) { public SearchRequestBuilder(ElasticsearchClient client, ActionType<SearchResponse> action) {
super(client, action, new SearchRequest()); super(client, action, new SearchRequest());
} }

View file

@ -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<SearchResponse> {
public static final SearchScrollAction INSTANCE = new SearchScrollAction();
public static final String NAME = "indices:data/read/scroll";
private SearchScrollAction() {
super(NAME, SearchResponse::new);
}
}

View file

@ -9,6 +9,7 @@
package org.elasticsearch.action.search; package org.elasticsearch.action.search;
import org.elasticsearch.action.ActionRequestBuilder; import org.elasticsearch.action.ActionRequestBuilder;
import org.elasticsearch.action.ActionType;
import org.elasticsearch.client.internal.ElasticsearchClient; import org.elasticsearch.client.internal.ElasticsearchClient;
import org.elasticsearch.core.TimeValue; import org.elasticsearch.core.TimeValue;
import org.elasticsearch.search.Scroll; import org.elasticsearch.search.Scroll;
@ -18,11 +19,11 @@ import org.elasticsearch.search.Scroll;
*/ */
public class SearchScrollRequestBuilder extends ActionRequestBuilder<SearchScrollRequest, SearchResponse> { public class SearchScrollRequestBuilder extends ActionRequestBuilder<SearchScrollRequest, SearchResponse> {
public SearchScrollRequestBuilder(ElasticsearchClient client, SearchScrollAction action) { public SearchScrollRequestBuilder(ElasticsearchClient client, ActionType<SearchResponse> action) {
super(client, action, new SearchScrollRequest()); super(client, action, new SearchScrollRequest());
} }
public SearchScrollRequestBuilder(ElasticsearchClient client, SearchScrollAction action, String scrollId) { public SearchScrollRequestBuilder(ElasticsearchClient client, ActionType<SearchResponse> action, String scrollId) {
super(client, action, new SearchScrollRequest(scrollId)); super(client, action, new SearchScrollRequest(scrollId));
} }

View file

@ -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<SearchShardsResponse> {
public static final String NAME = "indices:admin/search/search_shards";
public static final SearchShardsAction INSTANCE = new SearchShardsAction();
private SearchShardsAction() {
super(NAME, SearchShardsResponse::new);
}
}

View file

@ -280,7 +280,7 @@ public class SearchTransportService {
final Transport.Connection connection = transportService.getConnection(transportService.getLocalNode()); final Transport.Connection connection = transportService.getConnection(transportService.getLocalNode());
transportService.sendChildRequest( transportService.sendChildRequest(
connection, connection,
MultiSearchAction.NAME, TransportMultiSearchAction.TYPE.name(),
request, request,
task, task,
new ConnectionCountingHandler<>(listener, MultiSearchResponse::new, clientConnections, connection.getNode().getId()) new ConnectionCountingHandler<>(listener, MultiSearchResponse::new, clientConnections, connection.getNode().getId())

View file

@ -9,6 +9,7 @@
package org.elasticsearch.action.search; package org.elasticsearch.action.search;
import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.ActionType;
import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.HandledTransportAction; import org.elasticsearch.action.support.HandledTransportAction;
import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.cluster.service.ClusterService;
@ -19,6 +20,9 @@ import org.elasticsearch.transport.TransportService;
public class TransportClearScrollAction extends HandledTransportAction<ClearScrollRequest, ClearScrollResponse> { public class TransportClearScrollAction extends HandledTransportAction<ClearScrollRequest, ClearScrollResponse> {
public static final String NAME = "indices:data/read/scroll/clear";
public static final ActionType<ClearScrollResponse> TYPE = new ActionType<>(NAME, ClearScrollResponse::new);
private final ClusterService clusterService; private final ClusterService clusterService;
private final SearchTransportService searchTransportService; private final SearchTransportService searchTransportService;
@ -29,7 +33,7 @@ public class TransportClearScrollAction extends HandledTransportAction<ClearScro
ActionFilters actionFilters, ActionFilters actionFilters,
SearchTransportService searchTransportService SearchTransportService searchTransportService
) { ) {
super(ClearScrollAction.NAME, transportService, actionFilters, ClearScrollRequest::new, EsExecutors.DIRECT_EXECUTOR_SERVICE); super(TYPE.name(), transportService, actionFilters, ClearScrollRequest::new, EsExecutors.DIRECT_EXECUTOR_SERVICE);
this.clusterService = clusterService; this.clusterService = clusterService;
this.searchTransportService = searchTransportService; this.searchTransportService = searchTransportService;
} }

View file

@ -9,6 +9,7 @@
package org.elasticsearch.action.search; package org.elasticsearch.action.search;
import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.ActionType;
import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.HandledTransportAction; import org.elasticsearch.action.support.HandledTransportAction;
import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.cluster.service.ClusterService;
@ -22,6 +23,10 @@ import java.util.Collection;
public class TransportClosePointInTimeAction extends HandledTransportAction<ClosePointInTimeRequest, ClosePointInTimeResponse> { public class TransportClosePointInTimeAction extends HandledTransportAction<ClosePointInTimeRequest, ClosePointInTimeResponse> {
public static final ActionType<ClosePointInTimeResponse> TYPE = new ActionType<>(
"indices:data/read/close_point_in_time",
ClosePointInTimeResponse::new
);
private final ClusterService clusterService; private final ClusterService clusterService;
private final SearchTransportService searchTransportService; private final SearchTransportService searchTransportService;
private final NamedWriteableRegistry namedWriteableRegistry; private final NamedWriteableRegistry namedWriteableRegistry;
@ -34,13 +39,7 @@ public class TransportClosePointInTimeAction extends HandledTransportAction<Clos
SearchTransportService searchTransportService, SearchTransportService searchTransportService,
NamedWriteableRegistry namedWriteableRegistry NamedWriteableRegistry namedWriteableRegistry
) { ) {
super( super(TYPE.name(), transportService, actionFilters, ClosePointInTimeRequest::new, EsExecutors.DIRECT_EXECUTOR_SERVICE);
ClosePointInTimeAction.NAME,
transportService,
actionFilters,
ClosePointInTimeRequest::new,
EsExecutors.DIRECT_EXECUTOR_SERVICE
);
this.clusterService = clusterService; this.clusterService = clusterService;
this.searchTransportService = searchTransportService; this.searchTransportService = searchTransportService;
this.namedWriteableRegistry = namedWriteableRegistry; this.namedWriteableRegistry = namedWriteableRegistry;

View file

@ -12,6 +12,7 @@ import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.Logger;
import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.ActionType;
import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.HandledTransportAction; import org.elasticsearch.action.support.HandledTransportAction;
import org.elasticsearch.client.internal.node.NodeClient; import org.elasticsearch.client.internal.node.NodeClient;
@ -34,6 +35,8 @@ import java.util.function.LongSupplier;
public class TransportMultiSearchAction extends HandledTransportAction<MultiSearchRequest, MultiSearchResponse> { public class TransportMultiSearchAction extends HandledTransportAction<MultiSearchRequest, MultiSearchResponse> {
public static final String NAME = "indices:data/read/msearch";
public static final ActionType<MultiSearchResponse> TYPE = new ActionType<>(NAME, MultiSearchResponse::new);
private static final Logger logger = LogManager.getLogger(TransportMultiSearchAction.class); private static final Logger logger = LogManager.getLogger(TransportMultiSearchAction.class);
private final int allocatedProcessors; private final int allocatedProcessors;
private final ThreadPool threadPool; private final ThreadPool threadPool;
@ -50,7 +53,7 @@ public class TransportMultiSearchAction extends HandledTransportAction<MultiSear
ActionFilters actionFilters, ActionFilters actionFilters,
NodeClient client NodeClient client
) { ) {
super(MultiSearchAction.NAME, transportService, actionFilters, MultiSearchRequest::new, EsExecutors.DIRECT_EXECUTOR_SERVICE); super(TYPE.name(), transportService, actionFilters, MultiSearchRequest::new, EsExecutors.DIRECT_EXECUTOR_SERVICE);
this.threadPool = threadPool; this.threadPool = threadPool;
this.clusterService = clusterService; this.clusterService = clusterService;
this.allocatedProcessors = EsExecutors.allocatedProcessors(settings); this.allocatedProcessors = EsExecutors.allocatedProcessors(settings);
@ -67,7 +70,7 @@ public class TransportMultiSearchAction extends HandledTransportAction<MultiSear
LongSupplier relativeTimeProvider, LongSupplier relativeTimeProvider,
NodeClient client NodeClient client
) { ) {
super(MultiSearchAction.NAME, transportService, actionFilters, MultiSearchRequest::new, EsExecutors.DIRECT_EXECUTOR_SERVICE); super(TYPE.name(), transportService, actionFilters, MultiSearchRequest::new, EsExecutors.DIRECT_EXECUTOR_SERVICE);
this.threadPool = threadPool; this.threadPool = threadPool;
this.clusterService = clusterService; this.clusterService = clusterService;
this.allocatedProcessors = allocatedProcessors; this.allocatedProcessors = allocatedProcessors;

View file

@ -12,6 +12,7 @@ import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.Logger;
import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.ActionListenerResponseHandler; import org.elasticsearch.action.ActionListenerResponseHandler;
import org.elasticsearch.action.ActionType;
import org.elasticsearch.action.IndicesRequest; import org.elasticsearch.action.IndicesRequest;
import org.elasticsearch.action.OriginalIndices; import org.elasticsearch.action.OriginalIndices;
import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.ActionFilters;
@ -54,6 +55,10 @@ public class TransportOpenPointInTimeAction extends HandledTransportAction<OpenP
private static final Logger logger = LogManager.getLogger(TransportOpenPointInTimeAction.class); private static final Logger logger = LogManager.getLogger(TransportOpenPointInTimeAction.class);
public static final String OPEN_SHARD_READER_CONTEXT_NAME = "indices:data/read/open_reader_context"; public static final String OPEN_SHARD_READER_CONTEXT_NAME = "indices:data/read/open_reader_context";
public static final ActionType<OpenPointInTimeResponse> TYPE = new ActionType<>(
"indices:data/read/open_point_in_time",
OpenPointInTimeResponse::new
);
private final TransportSearchAction transportSearchAction; private final TransportSearchAction transportSearchAction;
private final SearchTransportService searchTransportService; private final SearchTransportService searchTransportService;
@ -68,13 +73,7 @@ public class TransportOpenPointInTimeAction extends HandledTransportAction<OpenP
TransportSearchAction transportSearchAction, TransportSearchAction transportSearchAction,
SearchTransportService searchTransportService SearchTransportService searchTransportService
) { ) {
super( super(TYPE.name(), transportService, actionFilters, OpenPointInTimeRequest::new, EsExecutors.DIRECT_EXECUTOR_SERVICE);
OpenPointInTimeAction.NAME,
transportService,
actionFilters,
OpenPointInTimeRequest::new,
EsExecutors.DIRECT_EXECUTOR_SERVICE
);
this.transportService = transportService; this.transportService = transportService;
this.transportSearchAction = transportSearchAction; this.transportSearchAction = transportSearchAction;
this.searchService = searchService; this.searchService = searchService;

View file

@ -14,6 +14,7 @@ import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.TransportVersions; import org.elasticsearch.TransportVersions;
import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.ActionListenerResponseHandler; import org.elasticsearch.action.ActionListenerResponseHandler;
import org.elasticsearch.action.ActionType;
import org.elasticsearch.action.IndicesRequest; import org.elasticsearch.action.IndicesRequest;
import org.elasticsearch.action.OriginalIndices; import org.elasticsearch.action.OriginalIndices;
import org.elasticsearch.action.ShardOperationFailedException; import org.elasticsearch.action.ShardOperationFailedException;
@ -111,6 +112,8 @@ import static org.elasticsearch.threadpool.ThreadPool.Names.SYSTEM_READ;
public class TransportSearchAction extends HandledTransportAction<SearchRequest, SearchResponse> { public class TransportSearchAction extends HandledTransportAction<SearchRequest, SearchResponse> {
public static final String NAME = "indices:data/read/search";
public static final ActionType<SearchResponse> TYPE = new ActionType<>(NAME, SearchResponse::new);
private static final Logger logger = LogManager.getLogger(TransportSearchAction.class); private static final Logger logger = LogManager.getLogger(TransportSearchAction.class);
private static final DeprecationLogger DEPRECATION_LOGGER = DeprecationLogger.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." public static final String FROZEN_INDICES_DEPRECATION_MESSAGE = "Searching frozen indices [{}] is deprecated."
@ -160,7 +163,7 @@ public class TransportSearchAction extends HandledTransportAction<SearchRequest,
NamedWriteableRegistry namedWriteableRegistry, NamedWriteableRegistry namedWriteableRegistry,
ExecutorSelector executorSelector ExecutorSelector executorSelector
) { ) {
super(SearchAction.NAME, transportService, actionFilters, SearchRequest::new, EsExecutors.DIRECT_EXECUTOR_SERVICE); super(TYPE.name(), transportService, actionFilters, SearchRequest::new, EsExecutors.DIRECT_EXECUTOR_SERVICE);
this.threadPool = threadPool; this.threadPool = threadPool;
this.circuitBreaker = circuitBreakerService.getBreaker(CircuitBreaker.REQUEST); this.circuitBreaker = circuitBreakerService.getBreaker(CircuitBreaker.REQUEST);
this.searchPhaseController = searchPhaseController; this.searchPhaseController = searchPhaseController;
@ -722,7 +725,7 @@ public class TransportSearchAction extends HandledTransportAction<SearchRequest,
); );
transportService.sendRequest( transportService.sendRequest(
connection, connection,
SearchShardsAction.NAME, TransportSearchShardsAction.TYPE.name(),
searchShardsRequest, searchShardsRequest,
TransportRequestOptions.EMPTY, TransportRequestOptions.EMPTY,
new ActionListenerResponseHandler<>(singleListener, SearchShardsResponse::new, responseExecutor) new ActionListenerResponseHandler<>(singleListener, SearchShardsResponse::new, responseExecutor)

View file

@ -12,6 +12,7 @@ import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.Logger;
import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.ActionType;
import org.elasticsearch.action.ShardOperationFailedException; import org.elasticsearch.action.ShardOperationFailedException;
import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.HandledTransportAction; 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; import static org.elasticsearch.action.search.TransportSearchHelper.parseScrollId;
public class TransportSearchScrollAction extends HandledTransportAction<SearchScrollRequest, SearchResponse> { public class TransportSearchScrollAction extends HandledTransportAction<SearchScrollRequest, SearchResponse> {
public static final ActionType<SearchResponse> TYPE = new ActionType<>("indices:data/read/scroll", SearchResponse::new);
private static final Logger logger = LogManager.getLogger(TransportSearchScrollAction.class); private static final Logger logger = LogManager.getLogger(TransportSearchScrollAction.class);
private final ClusterService clusterService; private final ClusterService clusterService;
private final SearchTransportService searchTransportService; private final SearchTransportService searchTransportService;
@ -37,7 +39,7 @@ public class TransportSearchScrollAction extends HandledTransportAction<SearchSc
ActionFilters actionFilters, ActionFilters actionFilters,
SearchTransportService searchTransportService SearchTransportService searchTransportService
) { ) {
super(SearchScrollAction.NAME, transportService, actionFilters, SearchScrollRequest::new, EsExecutors.DIRECT_EXECUTOR_SERVICE); super(TYPE.name(), transportService, actionFilters, SearchScrollRequest::new, EsExecutors.DIRECT_EXECUTOR_SERVICE);
this.clusterService = clusterService; this.clusterService = clusterService;
this.searchTransportService = searchTransportService; this.searchTransportService = searchTransportService;
} }

View file

@ -9,6 +9,7 @@
package org.elasticsearch.action.search; package org.elasticsearch.action.search;
import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.ActionType;
import org.elasticsearch.action.OriginalIndices; import org.elasticsearch.action.OriginalIndices;
import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.HandledTransportAction; import org.elasticsearch.action.support.HandledTransportAction;
@ -41,6 +42,9 @@ import java.util.Set;
* An internal search shards API performs the can_match phase and returns target shards of indices that might match a query. * An internal search shards API performs the can_match phase and returns target shards of indices that might match a query.
*/ */
public class TransportSearchShardsAction extends HandledTransportAction<SearchShardsRequest, SearchShardsResponse> { public class TransportSearchShardsAction extends HandledTransportAction<SearchShardsRequest, SearchShardsResponse> {
public static final String NAME = "indices:admin/search/search_shards";
public static final ActionType<SearchShardsResponse> TYPE = new ActionType<>(NAME, SearchShardsResponse::new);
private final TransportService transportService; private final TransportService transportService;
private final TransportSearchAction transportSearchAction; private final TransportSearchAction transportSearchAction;
private final SearchService searchService; private final SearchService searchService;
@ -61,7 +65,7 @@ public class TransportSearchShardsAction extends HandledTransportAction<SearchSh
IndexNameExpressionResolver indexNameExpressionResolver IndexNameExpressionResolver indexNameExpressionResolver
) { ) {
super( super(
SearchShardsAction.NAME, TYPE.name(),
transportService, transportService,
actionFilters, actionFilters,
SearchShardsRequest::new, SearchShardsRequest::new,

View file

@ -247,10 +247,10 @@ import org.elasticsearch.action.delete.DeleteAction;
import org.elasticsearch.action.delete.DeleteRequest; import org.elasticsearch.action.delete.DeleteRequest;
import org.elasticsearch.action.delete.DeleteRequestBuilder; import org.elasticsearch.action.delete.DeleteRequestBuilder;
import org.elasticsearch.action.delete.DeleteResponse; import org.elasticsearch.action.delete.DeleteResponse;
import org.elasticsearch.action.explain.ExplainAction;
import org.elasticsearch.action.explain.ExplainRequest; import org.elasticsearch.action.explain.ExplainRequest;
import org.elasticsearch.action.explain.ExplainRequestBuilder; import org.elasticsearch.action.explain.ExplainRequestBuilder;
import org.elasticsearch.action.explain.ExplainResponse; import org.elasticsearch.action.explain.ExplainResponse;
import org.elasticsearch.action.explain.TransportExplainAction;
import org.elasticsearch.action.fieldcaps.FieldCapabilitiesAction; import org.elasticsearch.action.fieldcaps.FieldCapabilitiesAction;
import org.elasticsearch.action.fieldcaps.FieldCapabilitiesRequest; import org.elasticsearch.action.fieldcaps.FieldCapabilitiesRequest;
import org.elasticsearch.action.fieldcaps.FieldCapabilitiesRequestBuilder; import org.elasticsearch.action.fieldcaps.FieldCapabilitiesRequestBuilder;
@ -280,21 +280,21 @@ import org.elasticsearch.action.ingest.SimulatePipelineAction;
import org.elasticsearch.action.ingest.SimulatePipelineRequest; import org.elasticsearch.action.ingest.SimulatePipelineRequest;
import org.elasticsearch.action.ingest.SimulatePipelineRequestBuilder; import org.elasticsearch.action.ingest.SimulatePipelineRequestBuilder;
import org.elasticsearch.action.ingest.SimulatePipelineResponse; import org.elasticsearch.action.ingest.SimulatePipelineResponse;
import org.elasticsearch.action.search.ClearScrollAction;
import org.elasticsearch.action.search.ClearScrollRequest; import org.elasticsearch.action.search.ClearScrollRequest;
import org.elasticsearch.action.search.ClearScrollRequestBuilder; import org.elasticsearch.action.search.ClearScrollRequestBuilder;
import org.elasticsearch.action.search.ClearScrollResponse; import org.elasticsearch.action.search.ClearScrollResponse;
import org.elasticsearch.action.search.MultiSearchAction;
import org.elasticsearch.action.search.MultiSearchRequest; import org.elasticsearch.action.search.MultiSearchRequest;
import org.elasticsearch.action.search.MultiSearchRequestBuilder; import org.elasticsearch.action.search.MultiSearchRequestBuilder;
import org.elasticsearch.action.search.MultiSearchResponse; import org.elasticsearch.action.search.MultiSearchResponse;
import org.elasticsearch.action.search.SearchAction;
import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.search.SearchRequestBuilder; import org.elasticsearch.action.search.SearchRequestBuilder;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.search.SearchScrollAction;
import org.elasticsearch.action.search.SearchScrollRequest; import org.elasticsearch.action.search.SearchScrollRequest;
import org.elasticsearch.action.search.SearchScrollRequestBuilder; import org.elasticsearch.action.search.SearchScrollRequestBuilder;
import org.elasticsearch.action.search.TransportClearScrollAction;
import org.elasticsearch.action.search.TransportMultiSearchAction;
import org.elasticsearch.action.search.TransportSearchAction;
import org.elasticsearch.action.search.TransportSearchScrollAction;
import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.action.support.PlainActionFuture;
import org.elasticsearch.action.support.master.AcknowledgedResponse; import org.elasticsearch.action.support.master.AcknowledgedResponse;
import org.elasticsearch.action.termvectors.MultiTermVectorsAction; import org.elasticsearch.action.termvectors.MultiTermVectorsAction;
@ -509,47 +509,47 @@ public abstract class AbstractClient implements Client {
@Override @Override
public ActionFuture<SearchResponse> search(final SearchRequest request) { public ActionFuture<SearchResponse> search(final SearchRequest request) {
return execute(SearchAction.INSTANCE, request); return execute(TransportSearchAction.TYPE, request);
} }
@Override @Override
public void search(final SearchRequest request, final ActionListener<SearchResponse> listener) { public void search(final SearchRequest request, final ActionListener<SearchResponse> listener) {
execute(SearchAction.INSTANCE, request, listener); execute(TransportSearchAction.TYPE, request, listener);
} }
@Override @Override
public SearchRequestBuilder prepareSearch(String... indices) { public SearchRequestBuilder prepareSearch(String... indices) {
return new SearchRequestBuilder(this, SearchAction.INSTANCE).setIndices(indices); return new SearchRequestBuilder(this, TransportSearchAction.TYPE).setIndices(indices);
} }
@Override @Override
public ActionFuture<SearchResponse> searchScroll(final SearchScrollRequest request) { public ActionFuture<SearchResponse> searchScroll(final SearchScrollRequest request) {
return execute(SearchScrollAction.INSTANCE, request); return execute(TransportSearchScrollAction.TYPE, request);
} }
@Override @Override
public void searchScroll(final SearchScrollRequest request, final ActionListener<SearchResponse> listener) { public void searchScroll(final SearchScrollRequest request, final ActionListener<SearchResponse> listener) {
execute(SearchScrollAction.INSTANCE, request, listener); execute(TransportSearchScrollAction.TYPE, request, listener);
} }
@Override @Override
public SearchScrollRequestBuilder prepareSearchScroll(String scrollId) { public SearchScrollRequestBuilder prepareSearchScroll(String scrollId) {
return new SearchScrollRequestBuilder(this, SearchScrollAction.INSTANCE, scrollId); return new SearchScrollRequestBuilder(this, TransportSearchScrollAction.TYPE, scrollId);
} }
@Override @Override
public ActionFuture<MultiSearchResponse> multiSearch(MultiSearchRequest request) { public ActionFuture<MultiSearchResponse> multiSearch(MultiSearchRequest request) {
return execute(MultiSearchAction.INSTANCE, request); return execute(TransportMultiSearchAction.TYPE, request);
} }
@Override @Override
public void multiSearch(MultiSearchRequest request, ActionListener<MultiSearchResponse> listener) { public void multiSearch(MultiSearchRequest request, ActionListener<MultiSearchResponse> listener) {
execute(MultiSearchAction.INSTANCE, request, listener); execute(TransportMultiSearchAction.TYPE, request, listener);
} }
@Override @Override
public MultiSearchRequestBuilder prepareMultiSearch() { public MultiSearchRequestBuilder prepareMultiSearch() {
return new MultiSearchRequestBuilder(this, MultiSearchAction.INSTANCE); return new MultiSearchRequestBuilder(this, TransportMultiSearchAction.TYPE);
} }
@Override @Override
@ -589,32 +589,32 @@ public abstract class AbstractClient implements Client {
@Override @Override
public ExplainRequestBuilder prepareExplain(String index, String id) { public ExplainRequestBuilder prepareExplain(String index, String id) {
return new ExplainRequestBuilder(this, ExplainAction.INSTANCE, index, id); return new ExplainRequestBuilder(this, TransportExplainAction.TYPE, index, id);
} }
@Override @Override
public ActionFuture<ExplainResponse> explain(ExplainRequest request) { public ActionFuture<ExplainResponse> explain(ExplainRequest request) {
return execute(ExplainAction.INSTANCE, request); return execute(TransportExplainAction.TYPE, request);
} }
@Override @Override
public void explain(ExplainRequest request, ActionListener<ExplainResponse> listener) { public void explain(ExplainRequest request, ActionListener<ExplainResponse> listener) {
execute(ExplainAction.INSTANCE, request, listener); execute(TransportExplainAction.TYPE, request, listener);
} }
@Override @Override
public void clearScroll(ClearScrollRequest request, ActionListener<ClearScrollResponse> listener) { public void clearScroll(ClearScrollRequest request, ActionListener<ClearScrollResponse> listener) {
execute(ClearScrollAction.INSTANCE, request, listener); execute(TransportClearScrollAction.TYPE, request, listener);
} }
@Override @Override
public ActionFuture<ClearScrollResponse> clearScroll(ClearScrollRequest request) { public ActionFuture<ClearScrollResponse> clearScroll(ClearScrollRequest request) {
return execute(ClearScrollAction.INSTANCE, request); return execute(TransportClearScrollAction.TYPE, request);
} }
@Override @Override
public ClearScrollRequestBuilder prepareClearScroll() { public ClearScrollRequestBuilder prepareClearScroll() {
return new ClearScrollRequestBuilder(this, ClearScrollAction.INSTANCE); return new ClearScrollRequestBuilder(this, TransportClearScrollAction.TYPE);
} }
@Override @Override

View file

@ -9,14 +9,14 @@
package org.elasticsearch.index.reindex; package org.elasticsearch.index.reindex;
import org.elasticsearch.action.ActionType; import org.elasticsearch.action.ActionType;
import org.elasticsearch.action.search.SearchAction;
import org.elasticsearch.action.search.SearchRequestBuilder; import org.elasticsearch.action.search.SearchRequestBuilder;
import org.elasticsearch.action.search.TransportSearchAction;
import org.elasticsearch.client.internal.ElasticsearchClient; import org.elasticsearch.client.internal.ElasticsearchClient;
public class DeleteByQueryRequestBuilder extends AbstractBulkByScrollRequestBuilder<DeleteByQueryRequest, DeleteByQueryRequestBuilder> { public class DeleteByQueryRequestBuilder extends AbstractBulkByScrollRequestBuilder<DeleteByQueryRequest, DeleteByQueryRequestBuilder> {
public DeleteByQueryRequestBuilder(ElasticsearchClient client, ActionType<BulkByScrollResponse> action) { public DeleteByQueryRequestBuilder(ElasticsearchClient client, ActionType<BulkByScrollResponse> action) {
this(client, action, new SearchRequestBuilder(client, SearchAction.INSTANCE)); this(client, action, new SearchRequestBuilder(client, TransportSearchAction.TYPE));
} }
private DeleteByQueryRequestBuilder(ElasticsearchClient client, ActionType<BulkByScrollResponse> action, SearchRequestBuilder search) { private DeleteByQueryRequestBuilder(ElasticsearchClient client, ActionType<BulkByScrollResponse> action, SearchRequestBuilder search) {

View file

@ -11,8 +11,8 @@ package org.elasticsearch.index.reindex;
import org.elasticsearch.action.ActionType; import org.elasticsearch.action.ActionType;
import org.elasticsearch.action.index.IndexAction; import org.elasticsearch.action.index.IndexAction;
import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.index.IndexRequestBuilder;
import org.elasticsearch.action.search.SearchAction;
import org.elasticsearch.action.search.SearchRequestBuilder; import org.elasticsearch.action.search.SearchRequestBuilder;
import org.elasticsearch.action.search.TransportSearchAction;
import org.elasticsearch.client.internal.ElasticsearchClient; import org.elasticsearch.client.internal.ElasticsearchClient;
public class ReindexRequestBuilder extends AbstractBulkIndexByScrollRequestBuilder<ReindexRequest, ReindexRequestBuilder> { public class ReindexRequestBuilder extends AbstractBulkIndexByScrollRequestBuilder<ReindexRequest, ReindexRequestBuilder> {
@ -22,7 +22,7 @@ public class ReindexRequestBuilder extends AbstractBulkIndexByScrollRequestBuild
this( this(
client, client,
action, action,
new SearchRequestBuilder(client, SearchAction.INSTANCE), new SearchRequestBuilder(client, TransportSearchAction.TYPE),
new IndexRequestBuilder(client, IndexAction.INSTANCE) new IndexRequestBuilder(client, IndexAction.INSTANCE)
); );
} }

View file

@ -9,8 +9,8 @@
package org.elasticsearch.index.reindex; package org.elasticsearch.index.reindex;
import org.elasticsearch.action.ActionType; import org.elasticsearch.action.ActionType;
import org.elasticsearch.action.search.SearchAction;
import org.elasticsearch.action.search.SearchRequestBuilder; import org.elasticsearch.action.search.SearchRequestBuilder;
import org.elasticsearch.action.search.TransportSearchAction;
import org.elasticsearch.client.internal.ElasticsearchClient; import org.elasticsearch.client.internal.ElasticsearchClient;
public class UpdateByQueryRequestBuilder extends AbstractBulkIndexByScrollRequestBuilder< public class UpdateByQueryRequestBuilder extends AbstractBulkIndexByScrollRequestBuilder<
@ -18,7 +18,7 @@ public class UpdateByQueryRequestBuilder extends AbstractBulkIndexByScrollReques
UpdateByQueryRequestBuilder> { UpdateByQueryRequestBuilder> {
public UpdateByQueryRequestBuilder(ElasticsearchClient client, ActionType<BulkByScrollResponse> action) { public UpdateByQueryRequestBuilder(ElasticsearchClient client, ActionType<BulkByScrollResponse> action) {
this(client, action, new SearchRequestBuilder(client, SearchAction.INSTANCE)); this(client, action, new SearchRequestBuilder(client, TransportSearchAction.TYPE));
} }
private UpdateByQueryRequestBuilder(ElasticsearchClient client, ActionType<BulkByScrollResponse> action, SearchRequestBuilder search) { private UpdateByQueryRequestBuilder(ElasticsearchClient client, ActionType<BulkByScrollResponse> action, SearchRequestBuilder search) {

View file

@ -8,9 +8,9 @@
package org.elasticsearch.rest.action.search; package org.elasticsearch.rest.action.search;
import org.elasticsearch.action.search.MultiSearchAction;
import org.elasticsearch.action.search.MultiSearchRequest; import org.elasticsearch.action.search.MultiSearchRequest;
import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.search.TransportMultiSearchAction;
import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.client.internal.node.NodeClient; import org.elasticsearch.client.internal.node.NodeClient;
import org.elasticsearch.common.CheckedBiConsumer; import org.elasticsearch.common.CheckedBiConsumer;
@ -83,7 +83,7 @@ public class RestMultiSearchAction extends BaseRestHandler {
return channel -> { return channel -> {
final RestCancellableNodeClient cancellableClient = new RestCancellableNodeClient(client, request.getHttpChannel()); final RestCancellableNodeClient cancellableClient = new RestCancellableNodeClient(client, request.getHttpChannel());
cancellableClient.execute( cancellableClient.execute(
MultiSearchAction.INSTANCE, TransportMultiSearchAction.TYPE,
multiSearchRequest, multiSearchRequest,
new RestRefCountedChunkedToXContentListener<>(channel) new RestRefCountedChunkedToXContentListener<>(channel)
); );

View file

@ -11,9 +11,9 @@ package org.elasticsearch.rest.action.search;
import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.Version; import org.elasticsearch.Version;
import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.ActionRequestValidationException;
import org.elasticsearch.action.search.SearchAction;
import org.elasticsearch.action.search.SearchContextId; import org.elasticsearch.action.search.SearchContextId;
import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.search.TransportSearchAction;
import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.client.internal.node.NodeClient; import org.elasticsearch.client.internal.node.NodeClient;
import org.elasticsearch.common.Strings; import org.elasticsearch.common.Strings;
@ -121,7 +121,7 @@ public class RestSearchAction extends BaseRestHandler {
return channel -> { return channel -> {
RestCancellableNodeClient cancelClient = new RestCancellableNodeClient(client, request.getHttpChannel()); RestCancellableNodeClient cancelClient = new RestCancellableNodeClient(client, request.getHttpChannel());
cancelClient.execute(SearchAction.INSTANCE, searchRequest, new RestRefCountedChunkedToXContentListener<>(channel)); cancelClient.execute(TransportSearchAction.TYPE, searchRequest, new RestRefCountedChunkedToXContentListener<>(channel));
}; };
} }

View file

@ -20,7 +20,7 @@ public class SearchRequestBuilderTests extends ESTestCase {
private SearchRequestBuilder makeBuilder() { private SearchRequestBuilder makeBuilder() {
ElasticsearchClient client = Mockito.mock(ElasticsearchClient.class); ElasticsearchClient client = Mockito.mock(ElasticsearchClient.class);
return new SearchRequestBuilder(client, SearchAction.INSTANCE); return new SearchRequestBuilder(client, TransportSearchAction.TYPE);
} }
public void testEmptySourceToString() { public void testEmptySourceToString() {

View file

@ -528,7 +528,7 @@ public class SearchRequestTests extends AbstractSearchTestCase {
} }
private String toDescription(SearchRequest request) { 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() { public void testForceSyntheticUnsupported() {

View file

@ -22,7 +22,7 @@ import org.elasticsearch.action.admin.indices.stats.IndicesStatsAction;
import org.elasticsearch.action.delete.DeleteAction; import org.elasticsearch.action.delete.DeleteAction;
import org.elasticsearch.action.get.GetAction; import org.elasticsearch.action.get.GetAction;
import org.elasticsearch.action.index.IndexAction; 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.settings.Settings;
import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.common.util.concurrent.ThreadContext;
import org.elasticsearch.env.Environment; import org.elasticsearch.env.Environment;
@ -48,7 +48,7 @@ public abstract class AbstractClientHeadersTestCase extends ESTestCase {
private static final ActionType<?>[] ACTIONS = new ActionType<?>[] { private static final ActionType<?>[] ACTIONS = new ActionType<?>[] {
// client actions // client actions
GetAction.INSTANCE, GetAction.INSTANCE,
SearchAction.INSTANCE, TransportSearchAction.TYPE,
DeleteAction.INSTANCE, DeleteAction.INSTANCE,
DeleteStoredScriptAction.INSTANCE, DeleteStoredScriptAction.INSTANCE,
IndexAction.INSTANCE, IndexAction.INSTANCE,
@ -97,7 +97,7 @@ public abstract class AbstractClientHeadersTestCase extends ESTestCase {
// choosing arbitrary top level actions to test // choosing arbitrary top level actions to test
client.prepareGet("idx", "id").execute(new AssertingActionListener<>(GetAction.NAME, client.threadPool())); 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.prepareDelete("idx", "id").execute(new AssertingActionListener<>(DeleteAction.NAME, client.threadPool()));
client.admin() client.admin()
.cluster() .cluster()

View file

@ -14,9 +14,9 @@ import org.elasticsearch.action.ActionResponse;
import org.elasticsearch.action.ActionType; import org.elasticsearch.action.ActionType;
import org.elasticsearch.action.admin.cluster.node.tasks.cancel.CancelTasksAction; import org.elasticsearch.action.admin.cluster.node.tasks.cancel.CancelTasksAction;
import org.elasticsearch.action.admin.cluster.node.tasks.cancel.CancelTasksRequest; 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.SearchRequest;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.search.TransportSearchAction;
import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.action.support.PlainActionFuture;
import org.elasticsearch.client.internal.node.NodeClient; import org.elasticsearch.client.internal.node.NodeClient;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
@ -76,7 +76,7 @@ public class RestCancellableNodeClientTests extends ESTestCase {
for (int j = 0; j < numTasks; j++) { for (int j = 0; j < numTasks; j++) {
PlainActionFuture<SearchResponse> actionFuture = new PlainActionFuture<>(); PlainActionFuture<SearchResponse> actionFuture = new PlainActionFuture<>();
RestCancellableNodeClient client = new RestCancellableNodeClient(testClient, channel); 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); futures.add(actionFuture);
} }
} }
@ -106,7 +106,7 @@ public class RestCancellableNodeClientTests extends ESTestCase {
totalSearches += numTasks; totalSearches += numTasks;
RestCancellableNodeClient client = new RestCancellableNodeClient(nodeClient, channel); RestCancellableNodeClient client = new RestCancellableNodeClient(nodeClient, channel);
for (int j = 0; j < numTasks; j++) { 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)); assertEquals(numTasks, RestCancellableNodeClient.getNumTasks(channel));
} }
@ -139,7 +139,7 @@ public class RestCancellableNodeClientTests extends ESTestCase {
RestCancellableNodeClient client = new RestCancellableNodeClient(testClient, channel); RestCancellableNodeClient client = new RestCancellableNodeClient(testClient, channel);
for (int j = 0; j < numTasks; j++) { 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 // 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()); assertEquals(initialHttpChannels, RestCancellableNodeClient.getNumChannels());
@ -177,7 +177,7 @@ public class RestCancellableNodeClientTests extends ESTestCase {
} }
return task; return task;
} }
case SearchAction.NAME -> { case TransportSearchAction.NAME -> {
searchRequests.incrementAndGet(); searchRequests.incrementAndGet();
Task searchTask = request.createTask(counter.getAndIncrement(), "search", action.name(), null, Collections.emptyMap()); Task searchTask = request.createTask(counter.getAndIncrement(), "search", action.name(), null, Collections.emptyMap());
if (timeout == false) { if (timeout == false) {

View file

@ -25,9 +25,7 @@ import org.elasticsearch.action.DocWriteResponse;
import org.elasticsearch.action.OriginalIndices; import org.elasticsearch.action.OriginalIndices;
import org.elasticsearch.action.admin.cluster.settings.ClusterUpdateSettingsResponse; import org.elasticsearch.action.admin.cluster.settings.ClusterUpdateSettingsResponse;
import org.elasticsearch.action.search.ClearScrollRequest; import org.elasticsearch.action.search.ClearScrollRequest;
import org.elasticsearch.action.search.ClosePointInTimeAction;
import org.elasticsearch.action.search.ClosePointInTimeRequest; import org.elasticsearch.action.search.ClosePointInTimeRequest;
import org.elasticsearch.action.search.OpenPointInTimeAction;
import org.elasticsearch.action.search.OpenPointInTimeRequest; import org.elasticsearch.action.search.OpenPointInTimeRequest;
import org.elasticsearch.action.search.SearchPhaseExecutionException; import org.elasticsearch.action.search.SearchPhaseExecutionException;
import org.elasticsearch.action.search.SearchRequest; 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.SearchScrollRequest;
import org.elasticsearch.action.search.SearchShardTask; import org.elasticsearch.action.search.SearchShardTask;
import org.elasticsearch.action.search.SearchType; 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.search.TransportSearchAction;
import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.action.support.PlainActionFuture;
@ -1833,7 +1833,7 @@ public class SearchServiceTests extends ESSingleNodeTestCase {
indicesAdmin().prepareRefresh("test").get(); indicesAdmin().prepareRefresh("test").get();
String pitId = client().execute( String pitId = client().execute(
OpenPointInTimeAction.INSTANCE, TransportOpenPointInTimeAction.TYPE,
new OpenPointInTimeRequest("test").keepAlive(TimeValue.timeValueMinutes(10)) new OpenPointInTimeRequest("test").keepAlive(TimeValue.timeValueMinutes(10))
).actionGet().getPointInTimeId(); ).actionGet().getPointInTimeId();
final MockSearchService searchService = (MockSearchService) getInstanceFromNode(SearchService.class); final MockSearchService searchService = (MockSearchService) getInstanceFromNode(SearchService.class);
@ -1848,7 +1848,7 @@ public class SearchServiceTests extends ESSingleNodeTestCase {
numDocs numDocs
); );
} finally { } finally {
client().execute(ClosePointInTimeAction.INSTANCE, new ClosePointInTimeRequest(pitId)).actionGet(); client().execute(TransportClosePointInTimeAction.TYPE, new ClosePointInTimeRequest(pitId)).actionGet();
} }
assertThat(shardRequests, not(emptyList())); assertThat(shardRequests, not(emptyList()));
for (ShardSearchRequest shardRequest : shardRequests) { for (ShardSearchRequest shardRequest : shardRequests) {

View file

@ -8,9 +8,9 @@
package org.elasticsearch.search.vectors; package org.elasticsearch.search.vectors;
import org.elasticsearch.action.search.SearchAction;
import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.search.SearchRequestBuilder; import org.elasticsearch.action.search.SearchRequestBuilder;
import org.elasticsearch.action.search.TransportSearchAction;
import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.query.QueryBuilder; import org.elasticsearch.index.query.QueryBuilder;
@ -224,7 +224,7 @@ public class KnnSearchRequestParserTests extends ESTestCase {
.withContent(BytesReference.bytes(builder), builder.contentType()) .withContent(BytesReference.bytes(builder), builder.contentType())
.build() .build()
); );
SearchRequestBuilder searchRequestBuilder = new SearchRequestBuilder(null, SearchAction.INSTANCE); SearchRequestBuilder searchRequestBuilder = new SearchRequestBuilder(null, TransportSearchAction.TYPE);
knnRequestBuilder.toSearchRequest(searchRequestBuilder); knnRequestBuilder.toSearchRequest(searchRequestBuilder);
return searchRequestBuilder; return searchRequestBuilder;
} }

View file

@ -58,7 +58,6 @@ import org.elasticsearch.action.bulk.TransportBulkAction;
import org.elasticsearch.action.bulk.TransportShardBulkAction; import org.elasticsearch.action.bulk.TransportShardBulkAction;
import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.action.resync.TransportResyncReplicationAction; import org.elasticsearch.action.resync.TransportResyncReplicationAction;
import org.elasticsearch.action.search.SearchAction;
import org.elasticsearch.action.search.SearchExecutionStatsCollector; import org.elasticsearch.action.search.SearchExecutionStatsCollector;
import org.elasticsearch.action.search.SearchPhaseController; import org.elasticsearch.action.search.SearchPhaseController;
import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchRequest;
@ -2000,7 +1999,7 @@ public class SnapshotResiliencyTests extends ESTestCase {
SearchPhaseController searchPhaseController = new SearchPhaseController(searchService::aggReduceContextBuilder); SearchPhaseController searchPhaseController = new SearchPhaseController(searchService::aggReduceContextBuilder);
actions.put( actions.put(
SearchAction.INSTANCE, TransportSearchAction.TYPE,
new TransportSearchAction( new TransportSearchAction(
threadPool, threadPool,
new NoneCircuitBreakerService(), new NoneCircuitBreakerService(),

View file

@ -10,9 +10,9 @@ package org.elasticsearch.transport;
import org.elasticsearch.TransportVersion; import org.elasticsearch.TransportVersion;
import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.search.SearchShardsAction;
import org.elasticsearch.action.search.SearchShardsRequest; import org.elasticsearch.action.search.SearchShardsRequest;
import org.elasticsearch.action.search.SearchShardsResponse; import org.elasticsearch.action.search.SearchShardsResponse;
import org.elasticsearch.action.search.TransportSearchShardsAction;
import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.action.support.PlainActionFuture;
import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNode;
@ -104,7 +104,7 @@ public class RemoteClusterAwareClientTests extends ESTestCase {
); );
final SearchShardsResponse searchShardsResponse = PlainActionFuture.get( final SearchShardsResponse searchShardsResponse = PlainActionFuture.get(
future -> client.execute( future -> client.execute(
SearchShardsAction.INSTANCE, TransportSearchShardsAction.TYPE,
searchShardsRequest, searchShardsRequest,
ActionListener.runBefore( ActionListener.runBefore(
future, future,
@ -169,7 +169,7 @@ public class RemoteClusterAwareClientTests extends ESTestCase {
null null
); );
client.execute( client.execute(
SearchShardsAction.INSTANCE, TransportSearchShardsAction.TYPE,
searchShardsRequest, searchShardsRequest,
ActionListener.runBefore( ActionListener.runBefore(
future, future,

View file

@ -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.ClusterStateAction;
import org.elasticsearch.action.admin.cluster.state.ClusterStateRequest; import org.elasticsearch.action.admin.cluster.state.ClusterStateRequest;
import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse; 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.SearchRequest;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.search.SearchShardsAction;
import org.elasticsearch.action.search.SearchShardsRequest; import org.elasticsearch.action.search.SearchShardsRequest;
import org.elasticsearch.action.search.SearchShardsResponse; import org.elasticsearch.action.search.SearchShardsResponse;
import org.elasticsearch.action.search.ShardSearchFailure; 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.action.support.PlainActionFuture;
import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.ClusterName;
import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterState;
@ -125,7 +125,7 @@ public class RemoteClusterConnectionTests extends ESTestCase {
MockTransportService newService = MockTransportService.createNewService(s, version, transportVersion, threadPool, null); MockTransportService newService = MockTransportService.createNewService(s, version, transportVersion, threadPool, null);
try { try {
newService.registerRequestHandler( newService.registerRequestHandler(
SearchShardsAction.NAME, TransportSearchShardsAction.TYPE.name(),
EsExecutors.DIRECT_EXECUTOR_SERVICE, EsExecutors.DIRECT_EXECUTOR_SERVICE,
SearchShardsRequest::new, SearchShardsRequest::new,
(request, channel, task) -> { (request, channel, task) -> {
@ -137,7 +137,7 @@ public class RemoteClusterConnectionTests extends ESTestCase {
} }
); );
newService.registerRequestHandler( newService.registerRequestHandler(
SearchAction.NAME, TransportSearchAction.TYPE.name(),
EsExecutors.DIRECT_EXECUTOR_SERVICE, EsExecutors.DIRECT_EXECUTOR_SERVICE,
SearchRequest::new, SearchRequest::new,
(request, channel, task) -> { (request, channel, task) -> {

View file

@ -12,10 +12,10 @@ import org.elasticsearch.ResourceNotFoundException;
import org.elasticsearch.action.admin.cluster.node.tasks.get.GetTaskResponse; import org.elasticsearch.action.admin.cluster.node.tasks.get.GetTaskResponse;
import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse; import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse;
import org.elasticsearch.action.get.GetResponse; import org.elasticsearch.action.get.GetResponse;
import org.elasticsearch.action.search.ClosePointInTimeAction;
import org.elasticsearch.action.search.ClosePointInTimeRequest; import org.elasticsearch.action.search.ClosePointInTimeRequest;
import org.elasticsearch.action.search.OpenPointInTimeAction;
import org.elasticsearch.action.search.OpenPointInTimeRequest; 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.action.support.master.AcknowledgedResponse;
import org.elasticsearch.cluster.ClusterChangedEvent; import org.elasticsearch.cluster.ClusterChangedEvent;
import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterState;
@ -235,7 +235,7 @@ public abstract class AsyncSearchIntegTestCase extends ESIntegTestCase {
final SubmitAsyncSearchRequest request; final SubmitAsyncSearchRequest request;
if (randomBoolean()) { if (randomBoolean()) {
OpenPointInTimeRequest openPIT = new OpenPointInTimeRequest(indexName).keepAlive(TimeValue.timeValueMinutes(between(5, 10))); 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); final PointInTimeBuilder pit = new PointInTimeBuilder(pitId);
if (randomBoolean()) { if (randomBoolean()) {
pit.setKeepAlive(TimeValue.timeValueMillis(randomIntBetween(1, 3600))); pit.setKeepAlive(TimeValue.timeValueMillis(randomIntBetween(1, 3600)));
@ -329,7 +329,7 @@ public abstract class AsyncSearchIntegTestCase extends ESIntegTestCase {
public void close() { public void close() {
if (closed.compareAndSet(false, true)) { if (closed.compareAndSet(false, true)) {
if (pitId != null) { if (pitId != null) {
client().execute(ClosePointInTimeAction.INSTANCE, new ClosePointInTimeRequest(pitId)).actionGet(); client().execute(TransportClosePointInTimeAction.TYPE, new ClosePointInTimeRequest(pitId)).actionGet();
} }
queryLatch.close(); queryLatch.close();
} }

View file

@ -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.CancelTasksRequest;
import org.elasticsearch.action.admin.cluster.node.tasks.cancel.CancelTasksResponse; import org.elasticsearch.action.admin.cluster.node.tasks.cancel.CancelTasksResponse;
import org.elasticsearch.action.admin.cluster.node.tasks.list.ListTasksResponse; 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.SearchResponse;
import org.elasticsearch.action.search.SearchType; import org.elasticsearch.action.search.SearchType;
import org.elasticsearch.action.search.ShardSearchFailure; import org.elasticsearch.action.search.ShardSearchFailure;
@ -1266,7 +1265,7 @@ public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase {
ListTasksResponse listTasksResponse = client(LOCAL_CLUSTER).admin() ListTasksResponse listTasksResponse = client(LOCAL_CLUSTER).admin()
.cluster() .cluster()
.prepareListTasks() .prepareListTasks()
.setActions(SearchAction.INSTANCE.name()) .setActions(TransportSearchAction.TYPE.name())
.get(); .get();
List<TaskInfo> tasks = listTasksResponse.getTasks(); List<TaskInfo> tasks = listTasksResponse.getTasks();
assertThat(tasks.size(), equalTo(1)); assertThat(tasks.size(), equalTo(1));
@ -1280,7 +1279,7 @@ public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase {
.get() .get()
.getTasks() .getTasks()
.stream() .stream()
.filter(t -> t.action().contains(SearchAction.NAME)) .filter(t -> t.action().contains(TransportSearchAction.TYPE.name()))
.collect(Collectors.toList()); .collect(Collectors.toList());
assertThat(remoteSearchTasks.size(), greaterThan(0)); assertThat(remoteSearchTasks.size(), greaterThan(0));
remoteClusterSearchTasks.set(remoteSearchTasks); remoteClusterSearchTasks.set(remoteSearchTasks);
@ -1298,7 +1297,7 @@ public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase {
for (TransportService transportService : transportServices) { for (TransportService transportService : transportServices) {
Collection<CancellableTask> cancellableTasks = transportService.getTaskManager().getCancellableTasks().values(); Collection<CancellableTask> cancellableTasks = transportService.getTaskManager().getCancellableTasks().values();
for (CancellableTask cancellableTask : cancellableTasks) { for (CancellableTask cancellableTask : cancellableTasks) {
if (cancellableTask.getAction().contains(SearchAction.INSTANCE.name())) { if (cancellableTask.getAction().contains(TransportSearchAction.TYPE.name())) {
assertTrue(cancellableTask.getDescription(), cancellableTask.isCancelled()); assertTrue(cancellableTask.getDescription(), cancellableTask.isCancelled());
} }
} }
@ -1311,7 +1310,7 @@ public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase {
.get() .get()
.getTasks() .getTasks()
.stream() .stream()
.filter(t -> t.action().contains(SearchAction.INSTANCE.name())) .filter(t -> t.action().contains(TransportSearchAction.TYPE.name()))
.toList(); .toList();
for (TaskInfo taskInfo : remoteSearchTasksAfterCancellation) { for (TaskInfo taskInfo : remoteSearchTasksAfterCancellation) {
assertTrue(taskInfo.description(), taskInfo.cancelled()); assertTrue(taskInfo.description(), taskInfo.cancelled());
@ -1385,7 +1384,7 @@ public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase {
ListTasksResponse listTasksResponse = client(LOCAL_CLUSTER).admin() ListTasksResponse listTasksResponse = client(LOCAL_CLUSTER).admin()
.cluster() .cluster()
.prepareListTasks() .prepareListTasks()
.setActions(SearchAction.INSTANCE.name()) .setActions(TransportSearchAction.TYPE.name())
.get(); .get();
List<TaskInfo> tasks = listTasksResponse.getTasks(); List<TaskInfo> tasks = listTasksResponse.getTasks();
assertThat(tasks.size(), equalTo(1)); assertThat(tasks.size(), equalTo(1));
@ -1398,7 +1397,7 @@ public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase {
.get() .get()
.getTasks() .getTasks()
.stream() .stream()
.filter(t -> t.action().contains(SearchAction.NAME)) .filter(t -> t.action().contains(TransportSearchAction.TYPE.name()))
.collect(Collectors.toList()); .collect(Collectors.toList());
assertThat(remoteSearchTasks.size(), greaterThan(0)); assertThat(remoteSearchTasks.size(), greaterThan(0));
remoteClusterSearchTasks.set(remoteSearchTasks); remoteClusterSearchTasks.set(remoteSearchTasks);
@ -1416,7 +1415,7 @@ public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase {
for (TransportService transportService : transportServices) { for (TransportService transportService : transportServices) {
Collection<CancellableTask> cancellableTasks = transportService.getTaskManager().getCancellableTasks().values(); Collection<CancellableTask> cancellableTasks = transportService.getTaskManager().getCancellableTasks().values();
for (CancellableTask cancellableTask : cancellableTasks) { for (CancellableTask cancellableTask : cancellableTasks) {
if (cancellableTask.getAction().contains(SearchAction.INSTANCE.name())) { if (cancellableTask.getAction().contains(TransportSearchAction.TYPE.name())) {
assertTrue(cancellableTask.getDescription(), cancellableTask.isCancelled()); assertTrue(cancellableTask.getDescription(), cancellableTask.isCancelled());
} }
} }
@ -1429,7 +1428,7 @@ public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase {
.get() .get()
.getTasks() .getTasks()
.stream() .stream()
.filter(t -> t.action().contains(SearchAction.INSTANCE.name())) .filter(t -> t.action().contains(TransportSearchAction.TYPE.name()))
.toList(); .toList();
for (TaskInfo taskInfo : remoteSearchTasksAfterCancellation) { for (TaskInfo taskInfo : remoteSearchTasksAfterCancellation) {
assertTrue(taskInfo.description(), taskInfo.cancelled()); assertTrue(taskInfo.description(), taskInfo.cancelled());
@ -1489,7 +1488,7 @@ public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase {
ListTasksResponse listTasksResponse = client(LOCAL_CLUSTER).admin() ListTasksResponse listTasksResponse = client(LOCAL_CLUSTER).admin()
.cluster() .cluster()
.prepareListTasks() .prepareListTasks()
.setActions(SearchAction.INSTANCE.name()) .setActions(TransportSearchAction.TYPE.name())
.get(); .get();
List<TaskInfo> tasks = listTasksResponse.getTasks(); List<TaskInfo> tasks = listTasksResponse.getTasks();
assertThat(tasks.size(), equalTo(1)); assertThat(tasks.size(), equalTo(1));
@ -1502,7 +1501,7 @@ public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase {
.get() .get()
.getTasks() .getTasks()
.stream() .stream()
.filter(t -> t.action().contains(SearchAction.NAME)) .filter(t -> t.action().contains(TransportSearchAction.TYPE.name()))
.collect(Collectors.toList()); .collect(Collectors.toList());
assertThat(remoteSearchTasks.size(), greaterThan(0)); assertThat(remoteSearchTasks.size(), greaterThan(0));
remoteClusterSearchTasks.set(remoteSearchTasks); remoteClusterSearchTasks.set(remoteSearchTasks);
@ -1524,7 +1523,7 @@ public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase {
ListTasksResponse listTasksResponse = client(LOCAL_CLUSTER).admin() ListTasksResponse listTasksResponse = client(LOCAL_CLUSTER).admin()
.cluster() .cluster()
.prepareListTasks() .prepareListTasks()
.setActions(SearchAction.INSTANCE.name()) .setActions(TransportSearchAction.TYPE.name())
.get(); .get();
List<TaskInfo> tasks = listTasksResponse.getTasks(); List<TaskInfo> tasks = listTasksResponse.getTasks();
assertThat(tasks.size(), equalTo(0)); assertThat(tasks.size(), equalTo(0));
@ -1532,7 +1531,7 @@ public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase {
ListTasksResponse remoteTasksResponse = client(REMOTE_CLUSTER).admin() ListTasksResponse remoteTasksResponse = client(REMOTE_CLUSTER).admin()
.cluster() .cluster()
.prepareListTasks() .prepareListTasks()
.setActions(SearchAction.INSTANCE.name()) .setActions(TransportSearchAction.TYPE.name())
.get(); .get();
List<TaskInfo> remoteTasks = remoteTasksResponse.getTasks(); List<TaskInfo> remoteTasks = remoteTasksResponse.getTasks();
assertThat(remoteTasks.size(), equalTo(0)); assertThat(remoteTasks.size(), equalTo(0));
@ -1554,7 +1553,7 @@ public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase {
ListTasksResponse listTasksResponse = client(LOCAL_CLUSTER).admin() ListTasksResponse listTasksResponse = client(LOCAL_CLUSTER).admin()
.cluster() .cluster()
.prepareListTasks() .prepareListTasks()
.setActions(SearchAction.INSTANCE.name()) .setActions(TransportSearchAction.TYPE.name())
.get(); .get();
List<TaskInfo> tasks = listTasksResponse.getTasks(); List<TaskInfo> tasks = listTasksResponse.getTasks();
assertThat(tasks.size(), equalTo(0)); assertThat(tasks.size(), equalTo(0));
@ -1562,7 +1561,7 @@ public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase {
ListTasksResponse remoteTasksResponse = client(REMOTE_CLUSTER).admin() ListTasksResponse remoteTasksResponse = client(REMOTE_CLUSTER).admin()
.cluster() .cluster()
.prepareListTasks() .prepareListTasks()
.setActions(SearchAction.INSTANCE.name()) .setActions(TransportSearchAction.TYPE.name())
.get(); .get();
List<TaskInfo> remoteTasks = remoteTasksResponse.getTasks(); List<TaskInfo> remoteTasks = remoteTasksResponse.getTasks();
assertThat(remoteTasks.size(), equalTo(0)); assertThat(remoteTasks.size(), equalTo(0));

View file

@ -8,7 +8,6 @@ package org.elasticsearch.xpack.search;
import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.DocWriteResponse;
import org.elasticsearch.action.search.SearchAction;
import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.action.search.TransportSearchAction;
import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.ActionFilters;
@ -87,7 +86,11 @@ public class TransportSubmitAsyncSearchAction extends HandledTransportAction<Sub
protected void doExecute(Task submitTask, SubmitAsyncSearchRequest request, ActionListener<AsyncSearchResponse> submitListener) { protected void doExecute(Task submitTask, SubmitAsyncSearchRequest request, ActionListener<AsyncSearchResponse> submitListener) {
final SearchRequest searchRequest = createSearchRequest(request, submitTask, request.getKeepAlive()); final SearchRequest searchRequest = createSearchRequest(request, submitTask, request.getKeepAlive());
try (var ignored = threadContext.newTraceContext()) { 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()); searchAction.execute(searchTask, searchRequest, searchTask.getSearchProgressActionListener());
searchTask.addCompletionListener(new ActionListener<>() { searchTask.addCompletionListener(new ActionListener<>() {
@Override @Override

View file

@ -29,7 +29,7 @@ import org.elasticsearch.action.datastreams.DeleteDataStreamAction;
import org.elasticsearch.action.datastreams.GetDataStreamAction; import org.elasticsearch.action.datastreams.GetDataStreamAction;
import org.elasticsearch.action.datastreams.PromoteDataStreamAction; import org.elasticsearch.action.datastreams.PromoteDataStreamAction;
import org.elasticsearch.action.fieldcaps.FieldCapabilitiesAction; 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.common.Strings;
import org.elasticsearch.index.seqno.RetentionLeaseActions; import org.elasticsearch.index.seqno.RetentionLeaseActions;
import org.elasticsearch.xpack.core.ccr.action.ForgetFollowerAction; 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( private static final Automaton READ_CROSS_CLUSTER_AUTOMATON = patterns(
"internal:transport/proxy/indices:data/read/*", "internal:transport/proxy/indices:data/read/*",
ClusterSearchShardsAction.NAME, ClusterSearchShardsAction.NAME,
SearchShardsAction.NAME TransportSearchShardsAction.TYPE.name()
); );
private static final Automaton CREATE_AUTOMATON = patterns( private static final Automaton CREATE_AUTOMATON = patterns(
"indices:data/write/index*", "indices:data/write/index*",
@ -119,7 +119,7 @@ public final class IndexPrivilege extends Privilege {
GetFieldMappingsAction.NAME + "*", GetFieldMappingsAction.NAME + "*",
GetMappingsAction.NAME, GetMappingsAction.NAME,
ClusterSearchShardsAction.NAME, ClusterSearchShardsAction.NAME,
SearchShardsAction.NAME, TransportSearchShardsAction.TYPE.name(),
ValidateQueryAction.NAME + "*", ValidateQueryAction.NAME + "*",
GetSettingsAction.NAME, GetSettingsAction.NAME,
ExplainLifecycleAction.NAME, ExplainLifecycleAction.NAME,

View file

@ -6,7 +6,7 @@
*/ */
package org.elasticsearch.xpack.core.security.authz.privilege; 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.RetentionLeaseActions;
import org.elasticsearch.index.seqno.RetentionLeaseBackgroundSyncAction; import org.elasticsearch.index.seqno.RetentionLeaseBackgroundSyncAction;
import org.elasticsearch.index.seqno.RetentionLeaseSyncAction; import org.elasticsearch.index.seqno.RetentionLeaseSyncAction;
@ -43,7 +43,7 @@ public final class SystemPrivilege extends Privilege {
"indices:data/read/*", // needed for SystemIndexMigrator "indices:data/read/*", // needed for SystemIndexMigrator
"indices:admin/refresh", // needed for SystemIndexMigrator "indices:admin/refresh", // needed for SystemIndexMigrator
"indices:admin/aliases", // 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<String> PREDICATE = (action) -> { private static final Predicate<String> PREDICATE = (action) -> {

View file

@ -12,10 +12,10 @@ import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.admin.cluster.health.ClusterHealthAction; import org.elasticsearch.action.admin.cluster.health.ClusterHealthAction;
import org.elasticsearch.action.admin.cluster.health.ClusterHealthRequest; import org.elasticsearch.action.admin.cluster.health.ClusterHealthRequest;
import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse; 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.SearchRequest;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.search.ShardSearchFailure; import org.elasticsearch.action.search.ShardSearchFailure;
import org.elasticsearch.action.search.TransportSearchAction;
import org.elasticsearch.action.support.ActionTestUtils; import org.elasticsearch.action.support.ActionTestUtils;
import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.action.support.PlainActionFuture;
import org.elasticsearch.client.internal.Client; import org.elasticsearch.client.internal.Client;
@ -170,7 +170,7 @@ public class ClientHelperTests extends ESTestCase {
SearchRequest request = new SearchRequest("foo"); SearchRequest request = new SearchRequest("foo");
String originName = randomFrom(ClientHelper.ML_ORIGIN, ClientHelper.WATCHER_ORIGIN, ClientHelper.ROLLUP_ORIGIN); 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(); latch.await();
} }
@ -201,7 +201,7 @@ public class ClientHelperTests extends ESTestCase {
headers.put("bar", "bar"); headers.put("bar", "bar");
String originName = randomFrom(ClientHelper.ML_ORIGIN, ClientHelper.WATCHER_ORIGIN, ClientHelper.ROLLUP_ORIGIN); 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(); latch.await();
} }
@ -234,7 +234,7 @@ public class ClientHelperTests extends ESTestCase {
headers.put("_xpack_security_authentication", "bar"); headers.put("_xpack_security_authentication", "bar");
String originName = randomFrom(ClientHelper.ML_ORIGIN, ClientHelper.WATCHER_ORIGIN, ClientHelper.ROLLUP_ORIGIN); 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(); latch.await();
} }

View file

@ -8,9 +8,9 @@
package org.elasticsearch.xpack.core.ilm; package org.elasticsearch.xpack.core.ilm;
import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.search.SearchAction;
import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.search.TransportSearchAction;
import org.elasticsearch.action.support.ActionTestUtils; import org.elasticsearch.action.support.ActionTestUtils;
import org.elasticsearch.client.internal.Client; import org.elasticsearch.client.internal.Client;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
@ -57,7 +57,7 @@ public class LifecyclePolicyClientTests extends ESTestCase {
SearchRequest request = new SearchRequest("foo"); SearchRequest request = new SearchRequest("foo");
final var policyClient = new LifecyclePolicySecurityClient(client, ClientHelper.INDEX_LIFECYCLE_ORIGIN, Collections.emptyMap()); 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(); latch.await();
} }
@ -89,7 +89,7 @@ public class LifecyclePolicyClientTests extends ESTestCase {
headers.put("bar", "bar"); headers.put("bar", "bar");
final var policyClient = new LifecyclePolicySecurityClient(client, ClientHelper.INDEX_LIFECYCLE_ORIGIN, headers); 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(); latch.await();
} }
@ -123,7 +123,7 @@ public class LifecyclePolicyClientTests extends ESTestCase {
headers.put("_xpack_security_authentication", "bar"); headers.put("_xpack_security_authentication", "bar");
final var policyClient = new LifecyclePolicySecurityClient(client, ClientHelper.INDEX_LIFECYCLE_ORIGIN, headers); 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(); latch.await();
} }

View file

@ -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.create.CreateIndexAction;
import org.elasticsearch.action.admin.indices.delete.DeleteIndexAction; import org.elasticsearch.action.admin.indices.delete.DeleteIndexAction;
import org.elasticsearch.action.bulk.BulkAction; 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.AliasMetadata;
import org.elasticsearch.cluster.metadata.IndexAbstraction; import org.elasticsearch.cluster.metadata.IndexAbstraction;
import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.cluster.metadata.IndexMetadata;
@ -299,7 +299,7 @@ public class LimitedRoleTests extends ESTestCase {
.build(); .build();
IndicesAccessControl iac = fromRole.authorize( IndicesAccessControl iac = fromRole.authorize(
SearchAction.NAME, TransportSearchAction.TYPE.name(),
Sets.newHashSet("_index", "_alias1"), Sets.newHashSet("_index", "_alias1"),
md.getIndicesLookup(), md.getIndicesLookup(),
fieldPermissionsCache fieldPermissionsCache
@ -328,7 +328,7 @@ public class LimitedRoleTests extends ESTestCase {
.add(IndexPrivilege.NONE, "_index1") .add(IndexPrivilege.NONE, "_index1")
.build(); .build();
iac = limitedByRole.authorize( iac = limitedByRole.authorize(
SearchAction.NAME, TransportSearchAction.TYPE.name(),
Sets.newHashSet("_index", "_alias1"), Sets.newHashSet("_index", "_alias1"),
md.getIndicesLookup(), md.getIndicesLookup(),
fieldPermissionsCache fieldPermissionsCache
@ -367,7 +367,12 @@ public class LimitedRoleTests extends ESTestCase {
} else { } else {
role = fromRole.limitedBy(limitedByRole); 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.isGranted(), is(false));
assertThat(iac.getIndexPermissions("_index"), is(notNullValue())); assertThat(iac.getIndexPermissions("_index"), is(notNullValue()));
assertThat(iac.hasIndexPermissions("_index"), is(true)); assertThat(iac.hasIndexPermissions("_index"), is(true));
@ -440,12 +445,12 @@ public class LimitedRoleTests extends ESTestCase {
public void testCheckIndicesAction() { public void testCheckIndicesAction() {
Role fromRole = Role.builder(EMPTY_RESTRICTED_INDICES, "a-role").add(IndexPrivilege.READ, "ind-1").build(); 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)); assertThat(fromRole.checkIndicesAction(CreateIndexAction.NAME), is(false));
{ {
Role limitedByRole = Role.builder(EMPTY_RESTRICTED_INDICES, "limited-role").add(IndexPrivilege.ALL, "ind-1").build(); 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)); assertThat(limitedByRole.checkIndicesAction(CreateIndexAction.NAME), is(true));
Role role; Role role;
if (randomBoolean()) { if (randomBoolean()) {
@ -453,64 +458,79 @@ public class LimitedRoleTests extends ESTestCase {
} else { } else {
role = fromRole.limitedBy(limitedByRole); 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)); assertThat(role.checkIndicesAction(CreateIndexAction.NAME), is(false));
} }
{ {
Role limitedByRole = Role.builder(EMPTY_RESTRICTED_INDICES, "limited-role").add(IndexPrivilege.NONE, "ind-1").build(); 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; Role role;
if (randomBoolean()) { if (randomBoolean()) {
role = limitedByRole.limitedBy(fromRole); role = limitedByRole.limitedBy(fromRole);
} else { } else {
role = fromRole.limitedBy(limitedByRole); 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)); assertThat(role.checkIndicesAction(CreateIndexAction.NAME), is(false));
} }
} }
public void testAllowedIndicesMatcher() { public void testAllowedIndicesMatcher() {
Role fromRole = Role.builder(EMPTY_RESTRICTED_INDICES, "a-role").add(IndexPrivilege.READ, "ind-1*").build(); 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(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction("ind-1")), is(true));
assertThat(fromRole.allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction("ind-11")), is(true)); assertThat(fromRole.allowedIndicesMatcher(TransportSearchAction.TYPE.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-2")), is(false));
{ {
Role limitedByRole = Role.builder(EMPTY_RESTRICTED_INDICES, "limited-role").add(IndexPrivilege.READ, "ind-1", "ind-2").build(); 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(
assertThat(limitedByRole.allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction("ind-11")), is(false)); limitedByRole.allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction("ind-1")),
assertThat(limitedByRole.allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction("ind-2")), is(true)); 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; Role role;
if (randomBoolean()) { if (randomBoolean()) {
role = limitedByRole.limitedBy(fromRole); role = limitedByRole.limitedBy(fromRole);
} else { } else {
role = fromRole.limitedBy(limitedByRole); role = fromRole.limitedBy(limitedByRole);
} }
assertThat(role.allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction("ind-1")), is(true)); assertThat(role.allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction("ind-1")), is(true));
assertThat(role.allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction("ind-11")), is(false)); assertThat(role.allowedIndicesMatcher(TransportSearchAction.TYPE.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-2")), is(false));
} }
{ {
Role limitedByRole = Role.builder(EMPTY_RESTRICTED_INDICES, "limited-role").add(IndexPrivilege.READ, "ind-*").build(); 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(
assertThat(limitedByRole.allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction("ind-2")), is(true)); 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; Role role;
if (randomBoolean()) { if (randomBoolean()) {
role = limitedByRole.limitedBy(fromRole); role = limitedByRole.limitedBy(fromRole);
} else { } else {
role = fromRole.limitedBy(limitedByRole); role = fromRole.limitedBy(limitedByRole);
} }
assertThat(role.allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction("ind-1")), is(true)); assertThat(role.allowedIndicesMatcher(TransportSearchAction.TYPE.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-2")), is(false));
} }
} }
public void testAllowedIndicesMatcherWithNestedRole() { public void testAllowedIndicesMatcherWithNestedRole() {
Role role = Role.builder(EMPTY_RESTRICTED_INDICES, "a-role").add(IndexPrivilege.READ, "ind-1*").build(); 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(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction("ind-1")), is(true));
assertThat(role.allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction("ind-11")), is(true)); assertThat(role.allowedIndicesMatcher(TransportSearchAction.TYPE.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-2")), is(false));
final int depth = randomIntBetween(2, 4); final int depth = randomIntBetween(2, 4);
boolean index11Excluded = false; boolean index11Excluded = false;
@ -526,9 +546,12 @@ public class LimitedRoleTests extends ESTestCase {
} else { } else {
role = role.limitedBy(limitedByRole); role = role.limitedBy(limitedByRole);
} }
assertThat(role.allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction("ind-1")), is(true)); assertThat(role.allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction("ind-1")), is(true));
assertThat(role.allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction("ind-11")), is(false == index11Excluded)); assertThat(
assertThat(role.allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction("ind-2")), is(false)); 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(); .build();
Automaton fromRoleAutomaton = fromRole.allowedActionsMatcher("index1"); Automaton fromRoleAutomaton = fromRole.allowedActionsMatcher("index1");
Predicate<String> fromRolePredicate = Automatons.predicate(fromRoleAutomaton); Predicate<String> 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)); assertThat(fromRolePredicate.test(BulkAction.NAME), is(true));
Role limitedByRole = Role.builder(EMPTY_RESTRICTED_INDICES, "limitedRole").add(IndexPrivilege.READ, "index1", "index2").build(); Role limitedByRole = Role.builder(EMPTY_RESTRICTED_INDICES, "limitedRole").add(IndexPrivilege.READ, "index1", "index2").build();
Automaton limitedByRoleAutomaton = limitedByRole.allowedActionsMatcher("index1"); Automaton limitedByRoleAutomaton = limitedByRole.allowedActionsMatcher("index1");
Predicate<String> limitedByRolePredicated = Automatons.predicate(limitedByRoleAutomaton); Predicate<String> 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)); assertThat(limitedByRolePredicated.test(BulkAction.NAME), is(false));
Role role; Role role;
if (randomBoolean()) { if (randomBoolean()) {
@ -557,17 +580,17 @@ public class LimitedRoleTests extends ESTestCase {
Automaton roleAutomaton = role.allowedActionsMatcher("index1"); Automaton roleAutomaton = role.allowedActionsMatcher("index1");
Predicate<String> rolePredicate = Automatons.predicate(roleAutomaton); Predicate<String> 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)); assertThat(rolePredicate.test(BulkAction.NAME), is(false));
roleAutomaton = role.allowedActionsMatcher("index2"); roleAutomaton = role.allowedActionsMatcher("index2");
rolePredicate = Automatons.predicate(roleAutomaton); 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)); assertThat(rolePredicate.test(BulkAction.NAME), is(false));
roleAutomaton = role.allowedActionsMatcher("other"); roleAutomaton = role.allowedActionsMatcher("other");
rolePredicate = Automatons.predicate(roleAutomaton); 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)); assertThat(rolePredicate.test(BulkAction.NAME), is(false));
} }

View file

@ -13,7 +13,7 @@ import org.elasticsearch.action.admin.indices.shrink.ShrinkAction;
import org.elasticsearch.action.admin.indices.stats.IndicesStatsAction; import org.elasticsearch.action.admin.indices.stats.IndicesStatsAction;
import org.elasticsearch.action.delete.DeleteAction; import org.elasticsearch.action.delete.DeleteAction;
import org.elasticsearch.action.index.IndexAction; 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.action.update.UpdateAction;
import org.elasticsearch.common.util.iterable.Iterables; import org.elasticsearch.common.util.iterable.Iterables;
import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.ESTestCase;
@ -59,7 +59,7 @@ public class IndexPrivilegeTests extends ESTestCase {
} }
public void testFindPrivilegesThatGrant() { 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(IndexAction.NAME), equalTo(List.of("create_doc", "create", "index", "write", "all")));
assertThat(findPrivilegesThatGrant(UpdateAction.NAME), equalTo(List.of("index", "write", "all"))); assertThat(findPrivilegesThatGrant(UpdateAction.NAME), equalTo(List.of("index", "write", "all")));
assertThat(findPrivilegesThatGrant(DeleteAction.NAME), equalTo(List.of("delete", "write", "all"))); assertThat(findPrivilegesThatGrant(DeleteAction.NAME), equalTo(List.of("delete", "write", "all")));

View file

@ -52,9 +52,9 @@ import org.elasticsearch.action.ingest.DeletePipelineAction;
import org.elasticsearch.action.ingest.GetPipelineAction; import org.elasticsearch.action.ingest.GetPipelineAction;
import org.elasticsearch.action.ingest.PutPipelineAction; import org.elasticsearch.action.ingest.PutPipelineAction;
import org.elasticsearch.action.ingest.SimulatePipelineAction; import org.elasticsearch.action.ingest.SimulatePipelineAction;
import org.elasticsearch.action.search.MultiSearchAction; import org.elasticsearch.action.search.TransportMultiSearchAction;
import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.TransportSearchAction;
import org.elasticsearch.action.search.SearchShardsAction; import org.elasticsearch.action.search.TransportSearchShardsAction;
import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.action.support.PlainActionFuture;
import org.elasticsearch.action.support.WriteRequest; import org.elasticsearch.action.support.WriteRequest;
import org.elasticsearch.action.update.UpdateAction; 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(IndexAction.NAME).test(mockIndexAbstraction(index)), is(false));
assertThat(kibanaRole.indices().allowedIndicesMatcher(DeleteAction.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(UpdateSettingsAction.NAME).test(mockIndexAbstraction(index)), is(false));
assertThat(kibanaRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(index)), is(true)); assertThat(
assertThat(kibanaRole.indices().allowedIndicesMatcher(MultiSearchAction.NAME).test(mockIndexAbstraction(index)), is(true)); 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(GetAction.NAME).test(mockIndexAbstraction(index)), is(true));
assertThat(kibanaRole.indices().allowedIndicesMatcher(READ_CROSS_CLUSTER_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(IndexAction.NAME).test(mockIndexAbstraction(index)), is(false));
assertThat(kibanaRole.indices().allowedIndicesMatcher(DeleteAction.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(UpdateSettingsAction.NAME).test(mockIndexAbstraction(index)), is(false));
assertThat(kibanaRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(index)), is(true)); assertThat(
assertThat(kibanaRole.indices().allowedIndicesMatcher(MultiSearchAction.NAME).test(mockIndexAbstraction(index)), is(true)); 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(GetAction.NAME).test(mockIndexAbstraction(index)), is(true));
assertThat(kibanaRole.indices().allowedIndicesMatcher(READ_CROSS_CLUSTER_NAME).test(mockIndexAbstraction(index)), is(false)); 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(IndexAction.NAME).test(mockIndexAbstraction(index)), is(true));
assertThat(kibanaRole.indices().allowedIndicesMatcher(DeleteAction.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(UpdateSettingsAction.NAME).test(mockIndexAbstraction(index)), is(false));
assertThat(kibanaRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(index)), is(true)); assertThat(
assertThat(kibanaRole.indices().allowedIndicesMatcher(MultiSearchAction.NAME).test(mockIndexAbstraction(index)), is(true)); 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(GetAction.NAME).test(mockIndexAbstraction(index)), is(true));
assertThat(kibanaRole.indices().allowedIndicesMatcher(READ_CROSS_CLUSTER_NAME).test(mockIndexAbstraction(index)), is(false)); 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(IndexAction.NAME).test(mockIndexAbstraction(index)), is(false));
assertThat(kibanaRole.indices().allowedIndicesMatcher(DeleteAction.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(UpdateSettingsAction.NAME).test(mockIndexAbstraction(index)), is(false));
assertThat(kibanaRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(index)), is(true)); assertThat(
assertThat(kibanaRole.indices().allowedIndicesMatcher(MultiSearchAction.NAME).test(mockIndexAbstraction(index)), is(true)); 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(GetAction.NAME).test(mockIndexAbstraction(index)), is(true));
assertThat(kibanaRole.indices().allowedIndicesMatcher(READ_CROSS_CLUSTER_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(CreateIndexAction.NAME).test(mockIndexAbstraction(index)), is(false));
assertThat(kibanaRole.indices().allowedIndicesMatcher(IndexAction.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(DeleteAction.NAME).test(mockIndexAbstraction(index)), is(false));
assertThat(kibanaRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(index)), is(true)); assertThat(
assertThat(kibanaRole.indices().allowedIndicesMatcher(MultiSearchAction.NAME).test(mockIndexAbstraction(index)), is(true)); 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(GetAction.NAME).test(mockIndexAbstraction(index)), is(true));
assertThat(kibanaRole.indices().allowedIndicesMatcher(READ_CROSS_CLUSTER_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(CreateIndexAction.NAME).test(mockIndexAbstraction(index)), is(false));
assertThat(kibanaRole.indices().allowedIndicesMatcher(IndexAction.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(DeleteAction.NAME).test(mockIndexAbstraction(index)), is(false));
assertThat(kibanaRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(index)), is(true)); assertThat(
assertThat(kibanaRole.indices().allowedIndicesMatcher(MultiSearchAction.NAME).test(mockIndexAbstraction(index)), is(true)); 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(GetAction.NAME).test(mockIndexAbstraction(index)), is(true));
assertThat(kibanaRole.indices().allowedIndicesMatcher(READ_CROSS_CLUSTER_NAME).test(mockIndexAbstraction(index)), is(false)); 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(CreateIndexAction.NAME).test(indexAbstraction), is(false));
assertThat(kibanaRole.indices().allowedIndicesMatcher(IndexAction.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(DeleteAction.NAME).test(indexAbstraction), is(false));
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(MultiSearchAction.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(GetAction.NAME).test(indexAbstraction), is(true));
assertThat(kibanaRole.indices().allowedIndicesMatcher(READ_CROSS_CLUSTER_NAME).test(indexAbstraction), is(false)); assertThat(kibanaRole.indices().allowedIndicesMatcher(READ_CROSS_CLUSTER_NAME).test(indexAbstraction), is(false));
assertThat(kibanaRole.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(indexAbstraction), is(true)); 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(CreateIndexAction.NAME).test(indexAbstraction), is(false));
assertThat(kibanaRole.indices().allowedIndicesMatcher(IndexAction.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(DeleteAction.NAME).test(indexAbstraction), is(false));
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(MultiSearchAction.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(GetAction.NAME).test(indexAbstraction), is(true));
assertThat(kibanaRole.indices().allowedIndicesMatcher(READ_CROSS_CLUSTER_NAME).test(indexAbstraction), is(false)); assertThat(kibanaRole.indices().allowedIndicesMatcher(READ_CROSS_CLUSTER_NAME).test(indexAbstraction), is(false));
assertThat(kibanaRole.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(indexAbstraction), is(true)); 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(CreateIndexAction.NAME).test(dotFleetSecretsIndex), is(true));
assertThat(kibanaRole.indices().allowedIndicesMatcher(IndexAction.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(DeleteAction.NAME).test(dotFleetSecretsIndex), is(true));
assertThat(kibanaRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(dotFleetSecretsIndex), is(false)); assertThat(kibanaRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(dotFleetSecretsIndex), is(false));
assertThat(kibanaRole.indices().allowedIndicesMatcher(MultiSearchAction.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(GetAction.NAME).test(dotFleetSecretsIndex), is(false));
assertThat(kibanaRole.indices().allowedIndicesMatcher(UpdateSettingsAction.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(CreateIndexAction.NAME).test(mockIndexAbstraction(index)), is(false));
assertThat(kibanaRole.indices().allowedIndicesMatcher(IndexAction.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(DeleteAction.NAME).test(mockIndexAbstraction(index)), is(false));
assertThat(kibanaRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(index)), is(true)); assertThat(
assertThat(kibanaRole.indices().allowedIndicesMatcher(MultiSearchAction.NAME).test(mockIndexAbstraction(index)), is(true)); 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(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(CreateIndexAction.NAME).test(indexAbstraction), is(false));
assertThat(kibanaRole.indices().allowedIndicesMatcher(IndexAction.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(DeleteAction.NAME).test(indexAbstraction), is(false));
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(MultiSearchAction.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(GetAction.NAME).test(indexAbstraction), is(true));
assertThat(kibanaRole.indices().allowedIndicesMatcher(READ_CROSS_CLUSTER_NAME).test(indexAbstraction), is(false)); assertThat(kibanaRole.indices().allowedIndicesMatcher(READ_CROSS_CLUSTER_NAME).test(indexAbstraction), is(false));
assertThat(kibanaRole.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(indexAbstraction), is(true)); 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(CreateIndexAction.NAME).test(indexAbstraction), is(true));
assertThat(kibanaRole.indices().allowedIndicesMatcher(IndexAction.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(DeleteAction.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));
assertThat(kibanaRole.indices().allowedIndicesMatcher(MultiSearchAction.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(GetAction.NAME).test(indexAbstraction), is(true));
assertThat(kibanaRole.indices().allowedIndicesMatcher(READ_CROSS_CLUSTER_NAME).test(indexAbstraction), is(false)); assertThat(kibanaRole.indices().allowedIndicesMatcher(READ_CROSS_CLUSTER_NAME).test(indexAbstraction), is(false));
assertThat(kibanaRole.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(indexAbstraction), is(true)); 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(CreateIndexAction.NAME).test(indexAbstraction), is(false));
assertThat(kibanaRole.indices().allowedIndicesMatcher(IndexAction.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(DeleteAction.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));
assertThat(kibanaRole.indices().allowedIndicesMatcher(MultiSearchAction.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(GetAction.NAME).test(indexAbstraction), is(true));
assertThat(kibanaRole.indices().allowedIndicesMatcher(READ_CROSS_CLUSTER_NAME).test(indexAbstraction), is(false)); assertThat(kibanaRole.indices().allowedIndicesMatcher(READ_CROSS_CLUSTER_NAME).test(indexAbstraction), is(false));
assertThat(kibanaRole.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(indexAbstraction), is(true)); 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(CreateIndexAction.NAME).test(indexAbstraction), is(false));
assertThat(kibanaRole.indices().allowedIndicesMatcher(IndexAction.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(DeleteAction.NAME).test(indexAbstraction), is(false));
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(MultiSearchAction.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(GetAction.NAME).test(indexAbstraction), is(true));
assertThat(kibanaRole.indices().allowedIndicesMatcher(READ_CROSS_CLUSTER_NAME).test(indexAbstraction), is(false)); assertThat(kibanaRole.indices().allowedIndicesMatcher(READ_CROSS_CLUSTER_NAME).test(indexAbstraction), is(false));
assertThat(kibanaRole.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(indexAbstraction), is(true)); 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(IndexAction.NAME).test(mockIndexAbstraction(index)), is(false));
assertThat(kibanaRole.indices().allowedIndicesMatcher(DeleteAction.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(UpdateSettingsAction.NAME).test(mockIndexAbstraction(index)), is(false));
assertThat(kibanaRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(index)), is(false)); assertThat(
assertThat(kibanaRole.indices().allowedIndicesMatcher(MultiSearchAction.NAME).test(mockIndexAbstraction(index)), is(false)); 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(GetAction.NAME).test(mockIndexAbstraction(index)), is(false));
assertThat(kibanaRole.indices().allowedIndicesMatcher(READ_CROSS_CLUSTER_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(CreateIndexAction.NAME).test(indexAbstraction), is(false));
assertThat(kibanaRole.indices().allowedIndicesMatcher(IndexAction.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(DeleteAction.NAME).test(indexAbstraction), is(false));
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(MultiSearchAction.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(GetAction.NAME).test(indexAbstraction), is(true));
assertThat(kibanaRole.indices().allowedIndicesMatcher(READ_CROSS_CLUSTER_NAME).test(indexAbstraction), is(false)); assertThat(kibanaRole.indices().allowedIndicesMatcher(READ_CROSS_CLUSTER_NAME).test(indexAbstraction), is(false));
assertThat(kibanaRole.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(indexAbstraction), is(true)); 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(CreateIndexAction.NAME).test(indexAbstraction), is(false));
assertThat(kibanaRole.indices().allowedIndicesMatcher(IndexAction.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(DeleteAction.NAME).test(indexAbstraction), is(false));
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(MultiSearchAction.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(GetAction.NAME).test(indexAbstraction), is(true));
assertThat(kibanaRole.indices().allowedIndicesMatcher(READ_CROSS_CLUSTER_NAME).test(indexAbstraction), is(false)); assertThat(kibanaRole.indices().allowedIndicesMatcher(READ_CROSS_CLUSTER_NAME).test(indexAbstraction), is(false));
assertThat(kibanaRole.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(indexAbstraction), is(true)); 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(IndexAction.NAME).test(mockIndexAbstraction(index)), is(true));
assertThat(kibanaRole.indices().allowedIndicesMatcher(DeleteAction.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(UpdateSettingsAction.NAME).test(mockIndexAbstraction(index)), is(false));
assertThat(kibanaRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(index)), is(true)); assertThat(
assertThat(kibanaRole.indices().allowedIndicesMatcher(MultiSearchAction.NAME).test(mockIndexAbstraction(index)), is(true)); 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(GetAction.NAME).test(mockIndexAbstraction(index)), is(true));
assertThat(kibanaRole.indices().allowedIndicesMatcher(READ_CROSS_CLUSTER_NAME).test(mockIndexAbstraction(index)), is(false)); 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-endpoint.heartbeat-")
|| indexName.startsWith(".logs-osquery_manager.actions-"); || indexName.startsWith(".logs-osquery_manager.actions-");
assertThat(kibanaRole.indices().allowedIndicesMatcher(GetAction.NAME).test(indexAbstraction), is(isAlsoReadIndex)); assertThat(kibanaRole.indices().allowedIndicesMatcher(GetAction.NAME).test(indexAbstraction), is(isAlsoReadIndex));
assertThat(kibanaRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(indexAbstraction), is(isAlsoReadIndex)); assertThat(
assertThat(kibanaRole.indices().allowedIndicesMatcher(MultiSearchAction.NAME).test(indexAbstraction), is(isAlsoReadIndex)); 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. // 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-") final boolean isAlsoIlmDeleteIndex = indexName.startsWith(".logs-endpoint.diagnostic.collection-")
@ -1078,7 +1141,7 @@ public class ReservedRolesStoreTests extends ESTestCase {
logger.info("index name [{}]", indexName); logger.info("index name [{}]", indexName);
final IndexAbstraction indexAbstraction = mockIndexAbstraction(indexName); final IndexAbstraction indexAbstraction = mockIndexAbstraction(indexName);
// Allow indexing // 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(GetAction.NAME).test(indexAbstraction), is(true));
assertThat(kibanaRole.indices().allowedIndicesMatcher(IndexAction.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)); 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 -> { Arrays.asList("logs-ti_recordedfuture_latest.threat", "logs-ti_anomali_latest.threatstream").forEach(indexName -> {
final IndexAbstraction indexAbstraction = mockIndexAbstraction(indexName); final IndexAbstraction indexAbstraction = mockIndexAbstraction(indexName);
// Allow search and indexing // 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(GetAction.NAME).test(indexAbstraction), is(true));
assertThat(kibanaRole.indices().allowedIndicesMatcher(IndexAction.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)); 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(PutMappingAction.NAME).test(indexAbstraction), is(true));
assertThat(kibanaRole.indices().allowedIndicesMatcher(RolloverAction.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(DeleteIndexAction.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));
assertThat(kibanaRole.indices().allowedIndicesMatcher(MultiSearchAction.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(GetAction.NAME).test(indexAbstraction), is(true));
// Implied by the overall view_index_metadata and monitor privilege // Implied by the overall view_index_metadata and monitor privilege
@ -1177,7 +1240,7 @@ public class ReservedRolesStoreTests extends ESTestCase {
logger.info("index name [{}]", indexName); logger.info("index name [{}]", indexName);
final IndexAbstraction indexAbstraction = mockIndexAbstraction(indexName); final IndexAbstraction indexAbstraction = mockIndexAbstraction(indexName);
// Allow indexing // 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(GetAction.NAME).test(indexAbstraction), is(true));
assertThat(kibanaRole.indices().allowedIndicesMatcher(IndexAction.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)); 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(CreateIndexAction.NAME).test(indexAbstraction), is(false));
assertThat(kibanaRole.indices().allowedIndicesMatcher(IndexAction.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(DeleteAction.NAME).test(indexAbstraction), is(false));
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(MultiSearchAction.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(GetAction.NAME).test(indexAbstraction), is(true));
assertThat(kibanaRole.indices().allowedIndicesMatcher(READ_CROSS_CLUSTER_NAME).test(indexAbstraction), is(false)); assertThat(kibanaRole.indices().allowedIndicesMatcher(READ_CROSS_CLUSTER_NAME).test(indexAbstraction), is(false));
assertThat(kibanaRole.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(indexAbstraction), is(true)); 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(CreateIndexAction.NAME).test(indexAbstraction), is(false));
assertThat(kibanaRole.indices().allowedIndicesMatcher(IndexAction.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(DeleteAction.NAME).test(indexAbstraction), is(false));
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(MultiSearchAction.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(GetAction.NAME).test(indexAbstraction), is(true));
assertThat(kibanaRole.indices().allowedIndicesMatcher(READ_CROSS_CLUSTER_NAME).test(indexAbstraction), is(false)); assertThat(kibanaRole.indices().allowedIndicesMatcher(READ_CROSS_CLUSTER_NAME).test(indexAbstraction), is(false));
assertThat(kibanaRole.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(indexAbstraction), is(true)); assertThat(kibanaRole.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(indexAbstraction), is(true));
@ -1243,7 +1306,7 @@ public class ReservedRolesStoreTests extends ESTestCase {
logger.info("index name [{}]", indexName); logger.info("index name [{}]", indexName);
final IndexAbstraction indexAbstraction = mockIndexAbstraction(indexName); final IndexAbstraction indexAbstraction = mockIndexAbstraction(indexName);
// Allow indexing // 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(GetAction.NAME).test(indexAbstraction), is(true));
assertThat(kibanaRole.indices().allowedIndicesMatcher(IndexAction.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)); 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(CreateIndexAction.NAME).test(indexAbstraction), is(false));
assertThat(kibanaRole.indices().allowedIndicesMatcher(IndexAction.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(DeleteAction.NAME).test(indexAbstraction), is(false));
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(MultiSearchAction.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(GetAction.NAME).test(indexAbstraction), is(true));
assertThat(kibanaRole.indices().allowedIndicesMatcher(READ_CROSS_CLUSTER_NAME).test(indexAbstraction), is(false)); assertThat(kibanaRole.indices().allowedIndicesMatcher(READ_CROSS_CLUSTER_NAME).test(indexAbstraction), is(false));
assertThat(kibanaRole.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(indexAbstraction), is(true)); 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 -> { Arrays.asList("kibana_sample_data_ecommerce", "kibana_sample_data_ecommerce_transform" + randomInt()).forEach(indexName -> {
final IndexAbstraction indexAbstraction = mockIndexAbstraction(indexName); final IndexAbstraction indexAbstraction = mockIndexAbstraction(indexName);
// Allow search and indexing // 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(GetAction.NAME).test(indexAbstraction), is(true));
assertThat(kibanaRole.indices().allowedIndicesMatcher(IndexAction.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)); 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 -> { Arrays.asList(".asset-criticality.asset-criticality-" + randomAlphaOfLength(randomIntBetween(0, 13))).forEach(indexName -> {
final IndexAbstraction indexAbstraction = mockIndexAbstraction(indexName); final IndexAbstraction indexAbstraction = mockIndexAbstraction(indexName);
assertThat(kibanaRole.indices().allowedIndicesMatcher(CreateIndexAction.NAME).test(indexAbstraction), is(true)); 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); assertViewIndexMetadata(kibanaRole, indexName);
}); });
} }
@ -1522,12 +1585,18 @@ public class ReservedRolesStoreTests extends ESTestCase {
assertThat(monitoringUserRole.runAs().check(randomAlphaOfLengthBetween(1, 12)), is(false)); assertThat(monitoringUserRole.runAs().check(randomAlphaOfLengthBetween(1, 12)), is(false));
assertThat(monitoringUserRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction("foo")), is(false));
assertThat( 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) is(false)
); );
assertThat(monitoringUserRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(".kibana")), is(false));
assertThat( assertThat(
monitoringUserRole.indices().allowedIndicesMatcher("indices:foo").test(mockIndexAbstraction(randomAlphaOfLengthBetween(8, 24))), monitoringUserRole.indices().allowedIndicesMatcher("indices:foo").test(mockIndexAbstraction(randomAlphaOfLengthBetween(8, 24))),
is(false) is(false)
@ -1556,7 +1625,10 @@ public class ReservedRolesStoreTests extends ESTestCase {
monitoringUserRole.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(mockIndexAbstraction(index)), monitoringUserRole.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(mockIndexAbstraction(index)),
is(false) 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(GetAction.NAME).test(mockIndexAbstraction(index)), is(true));
assertThat(monitoringUserRole.indices().allowedIndicesMatcher(READ_CROSS_CLUSTER_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.runAs().check(randomAlphaOfLengthBetween(1, 12)), is(false));
assertThat( assertThat(
remoteMonitoringAgentRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction("foo")), remoteMonitoringAgentRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction("foo")),
is(false) is(false)
); );
assertThat( assertThat(
remoteMonitoringAgentRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(".reporting")), remoteMonitoringAgentRole.indices()
.allowedIndicesMatcher(TransportSearchAction.TYPE.name())
.test(mockIndexAbstraction(".reporting")),
is(false) is(false)
); );
assertThat( assertThat(
remoteMonitoringAgentRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(".kibana")), remoteMonitoringAgentRole.indices()
.allowedIndicesMatcher(TransportSearchAction.TYPE.name())
.test(mockIndexAbstraction(".kibana")),
is(false) is(false)
); );
assertThat( assertThat(
@ -1701,7 +1777,9 @@ public class ReservedRolesStoreTests extends ESTestCase {
is(true) is(true)
); );
assertThat( assertThat(
remoteMonitoringAgentRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(monitoringIndex)), remoteMonitoringAgentRole.indices()
.allowedIndicesMatcher(TransportSearchAction.TYPE.name())
.test(mockIndexAbstraction(monitoringIndex)),
is(true) is(true)
); );
assertThat( assertThat(
@ -1775,7 +1853,9 @@ public class ReservedRolesStoreTests extends ESTestCase {
is(false) is(false)
); );
assertThat( assertThat(
remoteMonitoringAgentRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(metricbeatIndex)), remoteMonitoringAgentRole.indices()
.allowedIndicesMatcher(TransportSearchAction.TYPE.name())
.test(mockIndexAbstraction(metricbeatIndex)),
is(false) is(false)
); );
assertThat( assertThat(
@ -1818,15 +1898,21 @@ public class ReservedRolesStoreTests extends ESTestCase {
is(true) is(true)
); );
assertThat( assertThat(
remoteMonitoringCollectorRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction("foo")), remoteMonitoringCollectorRole.indices()
.allowedIndicesMatcher(TransportSearchAction.TYPE.name())
.test(mockIndexAbstraction("foo")),
is(false) is(false)
); );
assertThat( assertThat(
remoteMonitoringCollectorRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(".reporting")), remoteMonitoringCollectorRole.indices()
.allowedIndicesMatcher(TransportSearchAction.TYPE.name())
.test(mockIndexAbstraction(".reporting")),
is(false) is(false)
); );
assertThat( assertThat(
remoteMonitoringCollectorRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(".kibana")), remoteMonitoringCollectorRole.indices()
.allowedIndicesMatcher(TransportSearchAction.TYPE.name())
.test(mockIndexAbstraction(".kibana")),
is(true) is(true)
); );
assertThat( assertThat(
@ -1888,7 +1974,9 @@ public class ReservedRolesStoreTests extends ESTestCase {
is(false) is(false)
); );
assertThat( assertThat(
remoteMonitoringCollectorRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(index)), remoteMonitoringCollectorRole.indices()
.allowedIndicesMatcher(TransportSearchAction.TYPE.name())
.test(mockIndexAbstraction(index)),
is(false) is(false)
); );
assertThat( assertThat(
@ -1966,13 +2054,13 @@ public class ReservedRolesStoreTests extends ESTestCase {
assertThat( assertThat(
remoteMonitoringCollectorRole.indices() remoteMonitoringCollectorRole.indices()
.allowedIndicesMatcher(SearchAction.NAME) .allowedIndicesMatcher(TransportSearchAction.TYPE.name())
.test(mockIndexAbstraction(randomFrom(TestRestrictedIndices.SAMPLE_RESTRICTED_NAMES))), .test(mockIndexAbstraction(randomFrom(TestRestrictedIndices.SAMPLE_RESTRICTED_NAMES))),
is(false) is(false)
); );
assertThat( assertThat(
remoteMonitoringCollectorRole.indices() remoteMonitoringCollectorRole.indices()
.allowedIndicesMatcher(SearchAction.NAME) .allowedIndicesMatcher(TransportSearchAction.TYPE.name())
.test(mockIndexAbstraction(XPackPlugin.ASYNC_RESULTS_INDEX + randomAlphaOfLengthBetween(0, 2))), .test(mockIndexAbstraction(XPackPlugin.ASYNC_RESULTS_INDEX + randomAlphaOfLengthBetween(0, 2))),
is(false) is(false)
); );
@ -2081,12 +2169,18 @@ public class ReservedRolesStoreTests extends ESTestCase {
assertThat(reportingUserRole.runAs().check(randomAlphaOfLengthBetween(1, 12)), is(false)); assertThat(reportingUserRole.runAs().check(randomAlphaOfLengthBetween(1, 12)), is(false));
assertThat(reportingUserRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction("foo")), is(false));
assertThat( 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) is(false)
); );
assertThat(reportingUserRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(".kibana")), is(false));
assertThat( assertThat(
reportingUserRole.indices().allowedIndicesMatcher("indices:foo").test(mockIndexAbstraction(randomAlphaOfLengthBetween(8, 24))), reportingUserRole.indices().allowedIndicesMatcher("indices:foo").test(mockIndexAbstraction(randomAlphaOfLengthBetween(8, 24))),
is(false) is(false)
@ -2101,7 +2195,10 @@ public class ReservedRolesStoreTests extends ESTestCase {
reportingUserRole.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(mockIndexAbstraction(index)), reportingUserRole.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(mockIndexAbstraction(index)),
is(false) 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(GetAction.NAME).test(mockIndexAbstraction(index)), is(false));
assertThat(reportingUserRole.indices().allowedIndicesMatcher(IndexAction.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)); 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); FieldPermissionsCache fieldPermissionsCache = new FieldPermissionsCache(Settings.EMPTY);
SortedMap<String, IndexAbstraction> lookup = metadata.getIndicesLookup(); SortedMap<String, IndexAbstraction> lookup = metadata.getIndicesLookup();
IndicesAccessControl iac = superuserRole.indices() 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("a1"), is(true));
assertThat(iac.hasIndexPermissions("b"), is(true)); assertThat(iac.hasIndexPermissions("b"), is(true));
iac = superuserRole.indices().authorize(DeleteIndexAction.NAME, Sets.newHashSet("a1", "ba"), lookup, fieldPermissionsCache); 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 // Read security indices => allowed
iac = superuserRole.indices() iac = superuserRole.indices()
.authorize( .authorize(
randomFrom(SearchAction.NAME, GetIndexAction.NAME), randomFrom(TransportSearchAction.TYPE.name(), GetIndexAction.NAME),
Sets.newHashSet(TestRestrictedIndices.SECURITY_MAIN_ALIAS), Sets.newHashSet(TestRestrictedIndices.SECURITY_MAIN_ALIAS),
lookup, lookup,
fieldPermissionsCache 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(TestRestrictedIndices.SECURITY_MAIN_ALIAS), is(false));
assertThat("For " + iac, iac.hasIndexPermissions(internalSecurityIndex), 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")); assertFalse(superuserRole.indices().check("unknown"));
assertThat(superuserRole.runAs().check(randomAlphaOfLengthBetween(1, 30)), is(true)); 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(IndexAction.NAME).test(mockIndexAbstraction(index)), is(true));
assertThat(beatsAdminRole.indices().allowedIndicesMatcher(DeleteAction.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(UpdateSettingsAction.NAME).test(mockIndexAbstraction(index)), is(true));
assertThat(beatsAdminRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(index)), is(true)); assertThat(
assertThat(beatsAdminRole.indices().allowedIndicesMatcher(MultiSearchAction.NAME).test(mockIndexAbstraction(index)), is(true)); 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)); assertThat(beatsAdminRole.indices().allowedIndicesMatcher(GetAction.NAME).test(mockIndexAbstraction(index)), is(true));
assertNoAccessAllowed(beatsAdminRole, TestRestrictedIndices.SAMPLE_RESTRICTED_NAMES); 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(IndexAction.NAME).test(mockIndexAbstraction(index)), is(true));
assertThat(role.indices().allowedIndicesMatcher(DeleteAction.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(UpdateSettingsAction.NAME).test(mockIndexAbstraction(index)), is(true));
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(MultiSearchAction.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)); assertThat(role.indices().allowedIndicesMatcher(GetAction.NAME).test(mockIndexAbstraction(index)), is(true));
// inherits from 'all' // inherits from 'all'
assertThat(role.indices().allowedIndicesMatcher(READ_CROSS_CLUSTER_NAME).test(mockIndexAbstraction(index)), is(true)); 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) { private void assertReadWriteDocsAndMaintenanceButNotDeleteIndexAllowed(Role role, String index) {
assertThat(role.indices().allowedIndicesMatcher(DeleteIndexAction.NAME).test(mockIndexAbstraction(index)), is(false)); 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(GetAction.NAME).test(mockIndexAbstraction(index)), is(true));
assertThat(role.indices().allowedIndicesMatcher(IndexAction.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)); 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) { private void assertReadWriteDocsButNotDeleteIndexAllowed(Role role, String index) {
assertThat(role.indices().allowedIndicesMatcher(DeleteIndexAction.NAME).test(mockIndexAbstraction(index)), is(false)); 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(GetAction.NAME).test(mockIndexAbstraction(index)), is(true));
assertThat(role.indices().allowedIndicesMatcher(IndexAction.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)); 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(DeleteIndexAction.NAME).test(mockIndexAbstraction(index)), is(false));
assertThat(role.indices().allowedIndicesMatcher(CreateIndexAction.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(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(GetAction.NAME).test(mockIndexAbstraction(index)), is(true));
assertThat(role.indices().allowedIndicesMatcher(IndexAction.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)); assertThat(role.indices().allowedIndicesMatcher(UpdateAction.NAME).test(mockIndexAbstraction(index)), is(false));
@ -3281,7 +3387,7 @@ public class ReservedRolesStoreTests extends ESTestCase {
GetFieldMappingsAction.NAME + "*", GetFieldMappingsAction.NAME + "*",
GetMappingsAction.NAME, GetMappingsAction.NAME,
ClusterSearchShardsAction.NAME, ClusterSearchShardsAction.NAME,
SearchShardsAction.NAME, TransportSearchShardsAction.TYPE.name(),
ValidateQueryAction.NAME + "*", ValidateQueryAction.NAME + "*",
GetSettingsAction.NAME, GetSettingsAction.NAME,
ExplainLifecycleAction.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(DeleteIndexAction.NAME).test(mockIndexAbstraction(index)), is(false));
assertThat(role.indices().allowedIndicesMatcher(CreateIndexAction.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(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(GetAction.NAME).test(mockIndexAbstraction(index)), is(false));
assertThat(role.indices().allowedIndicesMatcher(IndexAction.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)); 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(CreateIndexAction.NAME).test(mockIndexAbstraction(index)), is(true));
assertThat(logstashAdminRole.indices().allowedIndicesMatcher(IndexAction.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(GetAction.NAME).test(mockIndexAbstraction(index)), is(true));
assertThat(logstashAdminRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(index)), is(true)); assertThat(
assertThat(logstashAdminRole.indices().allowedIndicesMatcher(MultiSearchAction.NAME).test(mockIndexAbstraction(index)), is(true)); logstashAdminRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction(index)),
is(true)
);
assertThat(
logstashAdminRole.indices().allowedIndicesMatcher(TransportMultiSearchAction.TYPE.name()).test(mockIndexAbstraction(index)),
is(true)
);
assertThat( assertThat(
logstashAdminRole.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(mockIndexAbstraction(index)), logstashAdminRole.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(mockIndexAbstraction(index)),
is(true) is(true)

View file

@ -8,9 +8,9 @@
package org.elasticsearch.xpack.application.search.action; package org.elasticsearch.xpack.application.search.action;
import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.search.SearchAction;
import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.search.TransportSearchAction;
import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.HandledTransportAction; import org.elasticsearch.action.support.HandledTransportAction;
import org.elasticsearch.client.internal.Client; import org.elasticsearch.client.internal.Client;
@ -68,7 +68,7 @@ public class TransportQuerySearchApplicationAction extends HandledTransportActio
SearchRequest searchRequest = new SearchRequest(searchApplication.name()).source(sourceBuilder); SearchRequest searchRequest = new SearchRequest(searchApplication.name()).source(sourceBuilder);
client.execute( client.execute(
SearchAction.INSTANCE, TransportSearchAction.TYPE,
searchRequest, searchRequest,
listener.delegateFailure((l2, searchResponse) -> l2.onResponse(searchResponse)) listener.delegateFailure((l2, searchResponse) -> l2.onResponse(searchResponse))
); );

View file

@ -8,15 +8,15 @@
package org.elasticsearch.xpack.eql.execution.search; package org.elasticsearch.xpack.eql.execution.search;
import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.search.ClosePointInTimeAction;
import org.elasticsearch.action.search.ClosePointInTimeRequest; import org.elasticsearch.action.search.ClosePointInTimeRequest;
import org.elasticsearch.action.search.ClosePointInTimeResponse; import org.elasticsearch.action.search.ClosePointInTimeResponse;
import org.elasticsearch.action.search.MultiSearchRequest; import org.elasticsearch.action.search.MultiSearchRequest;
import org.elasticsearch.action.search.MultiSearchResponse; import org.elasticsearch.action.search.MultiSearchResponse;
import org.elasticsearch.action.search.OpenPointInTimeAction;
import org.elasticsearch.action.search.OpenPointInTimeRequest; import org.elasticsearch.action.search.OpenPointInTimeRequest;
import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.search.SearchResponse; 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.Strings;
import org.elasticsearch.common.util.CollectionUtils; import org.elasticsearch.common.util.CollectionUtils;
import org.elasticsearch.core.TimeValue; import org.elasticsearch.core.TimeValue;
@ -131,7 +131,7 @@ public class PITAwareQueryClient extends BasicQueryClient {
private <Response> void openPIT(ActionListener<Response> listener, Runnable runnable) { private <Response> void openPIT(ActionListener<Response> listener, Runnable runnable) {
OpenPointInTimeRequest request = new OpenPointInTimeRequest(indices).indicesOptions(IndexResolver.FIELD_CAPS_INDICES_OPTIONS) OpenPointInTimeRequest request = new OpenPointInTimeRequest(indices).indicesOptions(IndexResolver.FIELD_CAPS_INDICES_OPTIONS)
.keepAlive(keepAlive); .keepAlive(keepAlive);
client.execute(OpenPointInTimeAction.INSTANCE, request, listener.delegateFailureAndWrap((l, r) -> { client.execute(TransportOpenPointInTimeAction.TYPE, request, listener.delegateFailureAndWrap((l, r) -> {
pitId = r.getPointInTimeId(); pitId = r.getPointInTimeId();
runnable.run(); runnable.run();
})); }));
@ -142,7 +142,7 @@ public class PITAwareQueryClient extends BasicQueryClient {
// the pitId could be null as a consequence of a failure on openPIT // the pitId could be null as a consequence of a failure on openPIT
if (pitId != null) { if (pitId != null) {
client.execute( client.execute(
ClosePointInTimeAction.INSTANCE, TransportClosePointInTimeAction.TYPE,
new ClosePointInTimeRequest(pitId), new ClosePointInTimeRequest(pitId),
map(listener, ClosePointInTimeResponse::isSucceeded) map(listener, ClosePointInTimeResponse::isSucceeded)
); );

View file

@ -10,10 +10,10 @@ import org.elasticsearch.TransportVersion;
import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.fieldcaps.FieldCapabilities; import org.elasticsearch.action.fieldcaps.FieldCapabilities;
import org.elasticsearch.action.fieldcaps.FieldCapabilitiesResponse; import org.elasticsearch.action.fieldcaps.FieldCapabilitiesResponse;
import org.elasticsearch.action.search.SearchAction;
import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.search.SearchRequestBuilder; import org.elasticsearch.action.search.SearchRequestBuilder;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.search.TransportSearchAction;
import org.elasticsearch.client.internal.Client; import org.elasticsearch.client.internal.Client;
import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.ClusterName;
import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNode;
@ -211,7 +211,7 @@ public class CancellationTests extends ESTestCase {
// Emulation of search cancellation // Emulation of search cancellation
ArgumentCaptor<SearchRequest> searchRequestCaptor = ArgumentCaptor.forClass(SearchRequest.class); ArgumentCaptor<SearchRequest> 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<Void>) invocation -> { doAnswer((Answer<Void>) invocation -> {
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
SearchRequest request = (SearchRequest) invocation.getArguments()[1]; SearchRequest request = (SearchRequest) invocation.getArguments()[1];

View file

@ -10,10 +10,10 @@ package org.elasticsearch.xpack.esql.plugin;
import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.ActionListenerResponseHandler; import org.elasticsearch.action.ActionListenerResponseHandler;
import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.search.SearchShardsAction;
import org.elasticsearch.action.search.SearchShardsGroup; import org.elasticsearch.action.search.SearchShardsGroup;
import org.elasticsearch.action.search.SearchShardsRequest; import org.elasticsearch.action.search.SearchShardsRequest;
import org.elasticsearch.action.search.SearchShardsResponse; import org.elasticsearch.action.search.SearchShardsResponse;
import org.elasticsearch.action.search.TransportSearchShardsAction;
import org.elasticsearch.action.support.ContextPreservingActionListener; import org.elasticsearch.action.support.ContextPreservingActionListener;
import org.elasticsearch.action.support.RefCountingListener; import org.elasticsearch.action.support.RefCountingListener;
import org.elasticsearch.action.support.RefCountingRunnable; import org.elasticsearch.action.support.RefCountingRunnable;
@ -393,7 +393,7 @@ public class ComputeService {
); );
transportService.sendChildRequest( transportService.sendChildRequest(
transportService.getLocalNode(), transportService.getLocalNode(),
SearchShardsAction.NAME, TransportSearchShardsAction.TYPE.name(),
searchShardsRequest, searchShardsRequest,
parentTask, parentTask,
TransportRequestOptions.EMPTY, TransportRequestOptions.EMPTY,

View file

@ -7,9 +7,9 @@
package org.elasticsearch.xpack.fleet.rest; package org.elasticsearch.xpack.fleet.rest;
import org.elasticsearch.action.search.MultiSearchAction;
import org.elasticsearch.action.search.MultiSearchRequest; import org.elasticsearch.action.search.MultiSearchRequest;
import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.search.TransportMultiSearchAction;
import org.elasticsearch.client.internal.node.NodeClient; import org.elasticsearch.client.internal.node.NodeClient;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.core.TimeValue; import org.elasticsearch.core.TimeValue;
@ -112,7 +112,7 @@ public class RestFleetMultiSearchAction extends BaseRestHandler {
return channel -> { return channel -> {
final RestCancellableNodeClient cancellableClient = new RestCancellableNodeClient(client, request.getHttpChannel()); final RestCancellableNodeClient cancellableClient = new RestCancellableNodeClient(client, request.getHttpChannel());
cancellableClient.execute(MultiSearchAction.INSTANCE, multiSearchRequest, new RestChunkedToXContentListener<>(channel)); cancellableClient.execute(TransportMultiSearchAction.TYPE, multiSearchRequest, new RestChunkedToXContentListener<>(channel));
}; };
} }

View file

@ -8,8 +8,8 @@
package org.elasticsearch.xpack.fleet.rest; package org.elasticsearch.xpack.fleet.rest;
import org.elasticsearch.Version; import org.elasticsearch.Version;
import org.elasticsearch.action.search.SearchAction;
import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.search.TransportSearchAction;
import org.elasticsearch.client.internal.node.NodeClient; import org.elasticsearch.client.internal.node.NodeClient;
import org.elasticsearch.common.Strings; import org.elasticsearch.common.Strings;
import org.elasticsearch.core.TimeValue; import org.elasticsearch.core.TimeValue;
@ -96,7 +96,7 @@ public class RestFleetSearchAction extends BaseRestHandler {
return channel -> { return channel -> {
RestCancellableNodeClient cancelClient = new RestCancellableNodeClient(client, request.getHttpChannel()); RestCancellableNodeClient cancelClient = new RestCancellableNodeClient(client, request.getHttpChannel());
cancelClient.execute(SearchAction.INSTANCE, searchRequest, new RestChunkedToXContentListener<>(channel)); cancelClient.execute(TransportSearchAction.TYPE, searchRequest, new RestChunkedToXContentListener<>(channel));
}; };
} }

View file

@ -9,11 +9,11 @@ package org.elasticsearch.index.engine.frozen;
import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.DocWriteResponse;
import org.elasticsearch.action.search.ClosePointInTimeAction;
import org.elasticsearch.action.search.ClosePointInTimeRequest; import org.elasticsearch.action.search.ClosePointInTimeRequest;
import org.elasticsearch.action.search.OpenPointInTimeAction;
import org.elasticsearch.action.search.OpenPointInTimeRequest; import org.elasticsearch.action.search.OpenPointInTimeRequest;
import org.elasticsearch.action.search.SearchType; 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.ActiveShardCount;
import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.action.support.PlainActionFuture;
@ -230,7 +230,7 @@ public class FrozenIndexIT extends ESIntegTestCase {
final OpenPointInTimeRequest openPointInTimeRequest = new OpenPointInTimeRequest(indexName).indicesOptions( final OpenPointInTimeRequest openPointInTimeRequest = new OpenPointInTimeRequest(indexName).indicesOptions(
IndicesOptions.STRICT_EXPAND_OPEN_FORBID_CLOSED IndicesOptions.STRICT_EXPAND_OPEN_FORBID_CLOSED
).keepAlive(TimeValue.timeValueMinutes(2)); ).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 { try {
assertNoFailuresAndResponse( assertNoFailuresAndResponse(
prepareSearch().setIndices(indexName).setPreference(null).setPointInTime(new PointInTimeBuilder(pitId)), prepareSearch().setIndices(indexName).setPreference(null).setPointInTime(new PointInTimeBuilder(pitId)),
@ -257,7 +257,7 @@ public class FrozenIndexIT extends ESIntegTestCase {
); );
} finally { } finally {
assertAcked(client().execute(FreezeIndexAction.INSTANCE, new FreezeRequest(indexName).setFreeze(false)).actionGet()); 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 IndicesOptions.STRICT_EXPAND_OPEN_FORBID_CLOSED
).keepAlive(TimeValue.timeValueMinutes(2)); ).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 { try {
indicesAdmin().prepareDelete("index-1").get(); indicesAdmin().prepareDelete("index-1").get();
// Return partial results if allow partial search result is allowed // 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 prepareSearch().setPreference(null).setAllowPartialSearchResults(false).setPointInTime(new PointInTimeBuilder(pitId))::get
); );
} finally { } 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( final OpenPointInTimeRequest openPointInTimeRequest = new OpenPointInTimeRequest("test-*").indicesOptions(
IndicesOptions.strictExpandOpenAndForbidClosed() IndicesOptions.strictExpandOpenAndForbidClosed()
).keepAlive(TimeValue.timeValueMinutes(2)); ).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 { try {
assertNoFailuresAndResponse( assertNoFailuresAndResponse(
prepareSearch().setPreference(null).setPointInTime(new PointInTimeBuilder(pitId)), prepareSearch().setPreference(null).setPointInTime(new PointInTimeBuilder(pitId)),
searchResponse -> assertHitCount(searchResponse, numDocs) searchResponse -> assertHitCount(searchResponse, numDocs)
); );
} finally { } finally {
client().execute(ClosePointInTimeAction.INSTANCE, new ClosePointInTimeRequest(pitId)).actionGet(); client().execute(TransportClosePointInTimeAction.TYPE, new ClosePointInTimeRequest(pitId)).actionGet();
} }
} }
// exclude the frozen indices // exclude the frozen indices
@ -332,11 +334,13 @@ public class FrozenIndexIT extends ESIntegTestCase {
final OpenPointInTimeRequest openPointInTimeRequest = new OpenPointInTimeRequest("test-*").keepAlive( final OpenPointInTimeRequest openPointInTimeRequest = new OpenPointInTimeRequest("test-*").keepAlive(
TimeValue.timeValueMinutes(2) TimeValue.timeValueMinutes(2)
); );
final String pitId = client().execute(OpenPointInTimeAction.INSTANCE, openPointInTimeRequest).actionGet().getPointInTimeId(); final String pitId = client().execute(TransportOpenPointInTimeAction.TYPE, openPointInTimeRequest)
.actionGet()
.getPointInTimeId();
try { try {
assertHitCountAndNoFailures(prepareSearch().setPreference(null).setPointInTime(new PointInTimeBuilder(pitId)), 0); assertHitCountAndNoFailures(prepareSearch().setPreference(null).setPointInTime(new PointInTimeBuilder(pitId)), 0);
} finally { } finally {
client().execute(ClosePointInTimeAction.INSTANCE, new ClosePointInTimeRequest(pitId)).actionGet(); client().execute(TransportClosePointInTimeAction.TYPE, new ClosePointInTimeRequest(pitId)).actionGet();
} }
} }
} }

View file

@ -12,14 +12,14 @@ import org.elasticsearch.action.OriginalIndices;
import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse; import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse;
import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse; import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse;
import org.elasticsearch.action.delete.DeleteResponse; import org.elasticsearch.action.delete.DeleteResponse;
import org.elasticsearch.action.search.ClosePointInTimeAction;
import org.elasticsearch.action.search.ClosePointInTimeRequest; import org.elasticsearch.action.search.ClosePointInTimeRequest;
import org.elasticsearch.action.search.OpenPointInTimeAction;
import org.elasticsearch.action.search.OpenPointInTimeRequest; import org.elasticsearch.action.search.OpenPointInTimeRequest;
import org.elasticsearch.action.search.OpenPointInTimeResponse; import org.elasticsearch.action.search.OpenPointInTimeResponse;
import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.search.SearchType; 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.search.TransportSearchAction;
import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.action.support.PlainActionFuture;
@ -86,7 +86,7 @@ public class FrozenIndexTests extends ESSingleNodeTestCase {
String openReaders(TimeValue keepAlive, String... indices) { String openReaders(TimeValue keepAlive, String... indices) {
OpenPointInTimeRequest request = new OpenPointInTimeRequest(indices).indicesOptions(IndicesOptions.STRICT_EXPAND_OPEN_FORBID_CLOSED) OpenPointInTimeRequest request = new OpenPointInTimeRequest(indices).indicesOptions(IndicesOptions.STRICT_EXPAND_OPEN_FORBID_CLOSED)
.keepAlive(keepAlive); .keepAlive(keepAlive);
final OpenPointInTimeResponse response = client().execute(OpenPointInTimeAction.INSTANCE, request).actionGet(); final OpenPointInTimeResponse response = client().execute(TransportOpenPointInTimeAction.TYPE, request).actionGet();
return response.getPointInTimeId(); return response.getPointInTimeId();
} }
@ -161,7 +161,7 @@ public class FrozenIndexTests extends ESSingleNodeTestCase {
} }
assertWarnings(TransportSearchAction.FROZEN_INDICES_DEPRECATION_MESSAGE.replace("{}", indexName)); assertWarnings(TransportSearchAction.FROZEN_INDICES_DEPRECATION_MESSAGE.replace("{}", indexName));
} finally { } finally {
client().execute(ClosePointInTimeAction.INSTANCE, new ClosePointInTimeRequest(pitId)).get(); client().execute(TransportClosePointInTimeAction.TYPE, new ClosePointInTimeRequest(pitId)).get();
} }
} }

View file

@ -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.refresh.RefreshResponse;
import org.elasticsearch.action.admin.indices.template.put.PutComposableIndexTemplateAction; import org.elasticsearch.action.admin.indices.template.put.PutComposableIndexTemplateAction;
import org.elasticsearch.action.datastreams.CreateDataStreamAction; import org.elasticsearch.action.datastreams.CreateDataStreamAction;
import org.elasticsearch.action.search.SearchAction;
import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.search.SearchRequestBuilder; import org.elasticsearch.action.search.SearchRequestBuilder;
import org.elasticsearch.action.search.TransportSearchAction;
import org.elasticsearch.action.support.master.AcknowledgedResponse; import org.elasticsearch.action.support.master.AcknowledgedResponse;
import org.elasticsearch.client.internal.Client; import org.elasticsearch.client.internal.Client;
import org.elasticsearch.cluster.ClusterModule; import org.elasticsearch.cluster.ClusterModule;
@ -311,12 +311,12 @@ abstract class MlNativeIntegTestCase extends ESIntegTestCase {
RefreshResponse refreshResponse = client().execute(RefreshAction.INSTANCE, refreshRequest).actionGet(); RefreshResponse refreshResponse = client().execute(RefreshAction.INSTANCE, refreshRequest).actionGet();
assertThat(refreshResponse.getStatus().getStatus(), anyOf(equalTo(200), equalTo(201))); 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 NotificationsIndex.NOTIFICATIONS_INDEX
).addSort("timestamp", SortOrder.ASC).setQuery(QueryBuilders.termQuery("job_id", jobId)).setSize(100).request(); ).addSort("timestamp", SortOrder.ASC).setQuery(QueryBuilders.termQuery("job_id", jobId)).setSize(100).request();
List<String> messages = new ArrayList<>(); List<String> messages = new ArrayList<>();
assertResponse( assertResponse(
client().execute(SearchAction.INSTANCE, searchRequest), client().execute(TransportSearchAction.TYPE, searchRequest),
searchResponse -> Arrays.stream(searchResponse.getHits().getHits()) searchResponse -> Arrays.stream(searchResponse.getHits().getHits())
.map(hit -> (String) hit.getSourceAsMap().get("message")) .map(hit -> (String) hit.getSourceAsMap().get("message"))
.forEach(messages::add) .forEach(messages::add)

View file

@ -13,8 +13,8 @@ import org.elasticsearch.action.bulk.BulkRequest;
import org.elasticsearch.action.bulk.BulkResponse; import org.elasticsearch.action.bulk.BulkResponse;
import org.elasticsearch.action.index.IndexAction; import org.elasticsearch.action.index.IndexAction;
import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.action.search.SearchAction;
import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.search.TransportSearchAction;
import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.action.support.PlainActionFuture;
import org.elasticsearch.action.support.WriteRequest; import org.elasticsearch.action.support.WriteRequest;
import org.elasticsearch.action.support.master.MasterNodeRequest; import org.elasticsearch.action.support.master.MasterNodeRequest;
@ -190,7 +190,7 @@ public class ModelSnapshotRetentionIT extends MlNativeAutodetectIntegTestCase {
private List<String> getDocIdsFromSearch(SearchRequest searchRequest) throws Exception { private List<String> getDocIdsFromSearch(SearchRequest searchRequest) throws Exception {
List<String> docIds = new ArrayList<>(); List<String> docIds = new ArrayList<>();
assertResponse(client().execute(SearchAction.INSTANCE, searchRequest), searchResponse -> { assertResponse(client().execute(TransportSearchAction.TYPE, searchRequest), searchResponse -> {
assertThat(searchResponse.getHits(), notNullValue()); assertThat(searchResponse.getHits(), notNullValue());
for (SearchHit searchHit : searchResponse.getHits().getHits()) { for (SearchHit searchHit : searchResponse.getHits().getHits()) {
docIds.add(searchHit.getId()); docIds.add(searchHit.getId());

View file

@ -13,9 +13,9 @@ import org.elasticsearch.ElasticsearchStatusException;
import org.elasticsearch.ResourceNotFoundException; import org.elasticsearch.ResourceNotFoundException;
import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.bulk.BulkItemResponse; import org.elasticsearch.action.bulk.BulkItemResponse;
import org.elasticsearch.action.search.SearchAction;
import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.search.TransportSearchAction;
import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.HandledTransportAction; import org.elasticsearch.action.support.HandledTransportAction;
import org.elasticsearch.action.support.master.AcknowledgedResponse; import org.elasticsearch.action.support.master.AcknowledgedResponse;
@ -116,7 +116,7 @@ public class TransportDeleteForecastAction extends HandledTransportAction<Delete
.query(query); .query(query);
SearchRequest searchRequest = new SearchRequest(AnomalyDetectorsIndex.jobResultsAliasedName(jobId)).source(source); SearchRequest searchRequest = new SearchRequest(AnomalyDetectorsIndex.jobResultsAliasedName(jobId)).source(source);
executeAsyncWithOrigin(client, ML_ORIGIN, SearchAction.INSTANCE, searchRequest, forecastStatsHandler); executeAsyncWithOrigin(client, ML_ORIGIN, TransportSearchAction.TYPE, searchRequest, forecastStatsHandler);
} }
static List<String> extractForecastIds(SearchHit[] forecastsToDelete, JobState jobState, String jobId) { static List<String> extractForecastIds(SearchHit[] forecastsToDelete, JobState jobState, String jobId) {

View file

@ -7,8 +7,8 @@
package org.elasticsearch.xpack.ml.action; package org.elasticsearch.xpack.ml.action;
import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.search.SearchAction;
import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.search.TransportSearchAction;
import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.HandledTransportAction; import org.elasticsearch.action.support.HandledTransportAction;
import org.elasticsearch.client.internal.Client; import org.elasticsearch.client.internal.Client;
@ -146,7 +146,7 @@ public class TransportEvaluateDataFrameAction extends HandledTransportAction<
SearchRequest searchRequest = new SearchRequest(request.getIndices()).source(searchSourceBuilder); SearchRequest searchRequest = new SearchRequest(request.getIndices()).source(searchSourceBuilder);
useSecondaryAuthIfAvailable( useSecondaryAuthIfAvailable(
securityContext, securityContext,
() -> client.execute(SearchAction.INSTANCE, searchRequest, ActionListener.wrap(searchResponse -> { () -> client.execute(TransportSearchAction.TYPE, searchRequest, ActionListener.wrap(searchResponse -> {
evaluation.process(searchResponse); evaluation.process(searchResponse);
if (evaluation.hasAllResults() == false) { if (evaluation.hasAllResults() == false) {
add(nextTask()); add(nextTask());

View file

@ -11,10 +11,10 @@ import org.apache.logging.log4j.Logger;
import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.FailedNodeException; import org.elasticsearch.action.FailedNodeException;
import org.elasticsearch.action.TaskOperationFailure; import org.elasticsearch.action.TaskOperationFailure;
import org.elasticsearch.action.search.MultiSearchAction;
import org.elasticsearch.action.search.MultiSearchRequest; import org.elasticsearch.action.search.MultiSearchRequest;
import org.elasticsearch.action.search.MultiSearchResponse; import org.elasticsearch.action.search.MultiSearchResponse;
import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.search.TransportMultiSearchAction;
import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.action.support.tasks.TransportTasksAction; import org.elasticsearch.action.support.tasks.TransportTasksAction;
@ -267,7 +267,7 @@ public class TransportGetDataFrameAnalyticsStatsAction extends TransportTasksAct
executeAsyncWithOrigin( executeAsyncWithOrigin(
client, client,
ML_ORIGIN, ML_ORIGIN,
MultiSearchAction.INSTANCE, TransportMultiSearchAction.TYPE,
multiSearchRequest, multiSearchRequest,
ActionListener.wrap(multiSearchResponse -> { ActionListener.wrap(multiSearchResponse -> {
MultiSearchResponse.Item[] itemResponses = multiSearchResponse.getResponses(); MultiSearchResponse.Item[] itemResponses = multiSearchResponse.getResponses();

View file

@ -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.NodesStatsRequestParameters;
import org.elasticsearch.action.admin.cluster.node.stats.NodesStatsResponse; import org.elasticsearch.action.admin.cluster.node.stats.NodesStatsResponse;
import org.elasticsearch.action.admin.cluster.node.stats.TransportNodesStatsAction; 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.SearchRequest;
import org.elasticsearch.action.search.TransportSearchAction;
import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.HandledTransportAction; import org.elasticsearch.action.support.HandledTransportAction;
import org.elasticsearch.client.internal.Client; import org.elasticsearch.client.internal.Client;
@ -349,7 +349,7 @@ public class TransportGetTrainedModelsStatsAction extends HandledTransportAction
.request(); .request();
searchRequest.setParentTask(parentTaskId); searchRequest.setParentTask(parentTaskId);
executeAsyncWithOrigin(client, ML_ORIGIN, SearchAction.INSTANCE, searchRequest, ActionListener.wrap(searchResponse -> { executeAsyncWithOrigin(client, ML_ORIGIN, TransportSearchAction.TYPE, searchRequest, ActionListener.wrap(searchResponse -> {
Map<String, Long> totalDefinitionLengthByModelId = new HashMap<>(); Map<String, Long> totalDefinitionLengthByModelId = new HashMap<>();
for (SearchHit hit : searchResponse.getHits().getHits()) { for (SearchHit hit : searchResponse.getHits().getHits()) {
DocumentField modelIdField = hit.field(TrainedModelConfig.MODEL_ID.getPreferredName()); DocumentField modelIdField = hit.field(TrainedModelConfig.MODEL_ID.getPreferredName());

View file

@ -11,8 +11,8 @@ import org.apache.logging.log4j.Logger;
import org.elasticsearch.ElasticsearchStatusException; import org.elasticsearch.ElasticsearchStatusException;
import org.elasticsearch.ResourceAlreadyExistsException; import org.elasticsearch.ResourceAlreadyExistsException;
import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.search.SearchAction;
import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.search.TransportSearchAction;
import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.SubscribableListener; import org.elasticsearch.action.support.SubscribableListener;
import org.elasticsearch.action.support.master.TransportMasterNodeAction; import org.elasticsearch.action.support.master.TransportMasterNodeAction;
@ -429,7 +429,7 @@ public class TransportStartDataFrameAnalyticsAction extends TransportMasterNodeA
startContext.config.getHeaders(), startContext.config.getHeaders(),
ClientHelper.ML_ORIGIN, ClientHelper.ML_ORIGIN,
parentTaskClient, parentTaskClient,
SearchAction.INSTANCE, TransportSearchAction.TYPE,
destEmptySearch, destEmptySearch,
ActionListener.wrap(searchResponse -> { ActionListener.wrap(searchResponse -> {
if (searchResponse.getHits().getTotalHits().value > 0) { if (searchResponse.getHits().getTotalHits().value > 0) {

View file

@ -10,7 +10,7 @@ package org.elasticsearch.xpack.ml.datafeed;
import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.Logger;
import org.elasticsearch.action.ActionListener; 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.action.support.master.AcknowledgedResponse;
import org.elasticsearch.client.internal.Client; import org.elasticsearch.client.internal.Client;
import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterState;
@ -130,9 +130,9 @@ public final class DatafeedManager {
ActionListener<GetRollupIndexCapsAction.Response> getRollupIndexCapsActionHandler = ActionListener.wrap(response -> { ActionListener<GetRollupIndexCapsAction.Response> getRollupIndexCapsActionHandler = ActionListener.wrap(response -> {
if (response.getJobs().isEmpty()) { // This means no rollup indexes are in the config if (response.getJobs().isEmpty()) { // This means no rollup indexes are in the config
indicesPrivilegesBuilder.privileges(SearchAction.NAME); indicesPrivilegesBuilder.privileges(TransportSearchAction.TYPE.name());
} else { } else {
indicesPrivilegesBuilder.privileges(SearchAction.NAME, RollupSearchAction.NAME); indicesPrivilegesBuilder.privileges(TransportSearchAction.TYPE.name(), RollupSearchAction.NAME);
} }
if (indices.length == 0) { if (indices.length == 0) {
privResponseListener.onResponse(new HasPrivilegesResponse()); privResponseListener.onResponse(new HasPrivilegesResponse());
@ -142,7 +142,7 @@ public final class DatafeedManager {
} }
}, e -> { }, e -> {
if (ExceptionsHelper.unwrapCause(e) instanceof IndexNotFoundException) { if (ExceptionsHelper.unwrapCause(e) instanceof IndexNotFoundException) {
indicesPrivilegesBuilder.privileges(SearchAction.NAME); indicesPrivilegesBuilder.privileges(TransportSearchAction.TYPE.name());
privRequest.indexPrivileges(indicesPrivilegesBuilder.build()); privRequest.indexPrivileges(indicesPrivilegesBuilder.build());
client.execute(HasPrivilegesAction.INSTANCE, privRequest, privResponseListener); client.execute(HasPrivilegesAction.INSTANCE, privRequest, privResponseListener);
} else { } else {

View file

@ -6,9 +6,9 @@
*/ */
package org.elasticsearch.xpack.ml.datafeed.delayeddatacheck; package org.elasticsearch.xpack.ml.datafeed.delayeddatacheck;
import org.elasticsearch.action.search.SearchAction;
import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.search.TransportSearchAction;
import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.client.internal.Client; import org.elasticsearch.client.internal.Client;
import org.elasticsearch.common.util.Maps; import org.elasticsearch.common.util.Maps;
@ -134,7 +134,7 @@ public class DatafeedDelayedDataDetector implements DelayedDataDetector {
SearchRequest searchRequest = new SearchRequest(datafeedIndices).source(searchSourceBuilder).indicesOptions(indicesOptions); SearchRequest searchRequest = new SearchRequest(datafeedIndices).source(searchSourceBuilder).indicesOptions(indicesOptions);
try (ThreadContext.StoredContext ignore = client.threadPool().getThreadContext().stashWithOrigin(ML_ORIGIN)) { 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<? extends Histogram.Bucket> buckets = ((Histogram) response.getAggregations().get(DATE_BUCKETS)).getBuckets(); List<? extends Histogram.Bucket> buckets = ((Histogram) response.getAggregations().get(DATE_BUCKETS)).getBuckets();
Map<Long, Long> hashMap = Maps.newMapWithExpectedSize(buckets.size()); Map<Long, Long> hashMap = Maps.newMapWithExpectedSize(buckets.size());
for (Histogram.Bucket bucket : buckets) { for (Histogram.Bucket bucket : buckets) {

View file

@ -6,8 +6,8 @@
*/ */
package org.elasticsearch.xpack.ml.datafeed.extractor.aggregation; 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.SearchRequestBuilder;
import org.elasticsearch.action.search.TransportSearchAction;
import org.elasticsearch.client.internal.Client; import org.elasticsearch.client.internal.Client;
import org.elasticsearch.search.builder.SearchSourceBuilder; import org.elasticsearch.search.builder.SearchSourceBuilder;
import org.elasticsearch.xpack.ml.datafeed.DatafeedTimingStatsReporter; import org.elasticsearch.xpack.ml.datafeed.DatafeedTimingStatsReporter;
@ -30,7 +30,7 @@ class AggregationDataExtractor extends AbstractAggregationDataExtractor<SearchRe
@Override @Override
protected SearchRequestBuilder buildSearchRequest(SearchSourceBuilder searchSourceBuilder) { protected SearchRequestBuilder buildSearchRequest(SearchSourceBuilder searchSourceBuilder) {
return new SearchRequestBuilder(client, SearchAction.INSTANCE).setSource(searchSourceBuilder) return new SearchRequestBuilder(client, TransportSearchAction.TYPE).setSource(searchSourceBuilder)
.setIndicesOptions(context.indicesOptions) .setIndicesOptions(context.indicesOptions)
.setAllowPartialSearchResults(false) .setAllowPartialSearchResults(false)
.setIndices(context.indices); .setIndices(context.indices);

View file

@ -6,8 +6,8 @@
*/ */
package org.elasticsearch.xpack.ml.datafeed.extractor.aggregation; 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.SearchRequestBuilder;
import org.elasticsearch.action.search.TransportSearchAction;
import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.client.internal.Client; import org.elasticsearch.client.internal.Client;
import org.elasticsearch.index.query.QueryBuilder; import org.elasticsearch.index.query.QueryBuilder;
@ -30,7 +30,7 @@ public record AggregationDataExtractorFactory(
) implements DataExtractorFactory { ) implements DataExtractorFactory {
public static AggregatedSearchRequestBuilder requestBuilder(Client client, String[] indices, IndicesOptions indicesOptions) { public static AggregatedSearchRequestBuilder requestBuilder(Client client, String[] indices, IndicesOptions indicesOptions) {
return (searchSourceBuilder) -> new SearchRequestBuilder(client, SearchAction.INSTANCE).setSource(searchSourceBuilder) return (searchSourceBuilder) -> new SearchRequestBuilder(client, TransportSearchAction.TYPE).setSource(searchSourceBuilder)
.setIndicesOptions(indicesOptions) .setIndicesOptions(indicesOptions)
.setAllowPartialSearchResults(false) .setAllowPartialSearchResults(false)
.setIndices(indices); .setIndices(indices);

View file

@ -9,10 +9,10 @@ package org.elasticsearch.xpack.ml.datafeed.extractor.chunked;
import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.Logger;
import org.elasticsearch.action.ActionRequestBuilder; import org.elasticsearch.action.ActionRequestBuilder;
import org.elasticsearch.action.search.SearchAction;
import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.search.SearchRequestBuilder; import org.elasticsearch.action.search.SearchRequestBuilder;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.search.TransportSearchAction;
import org.elasticsearch.client.internal.Client; import org.elasticsearch.client.internal.Client;
import org.elasticsearch.search.aggregations.AggregationBuilders; import org.elasticsearch.search.aggregations.AggregationBuilders;
import org.elasticsearch.search.aggregations.Aggregations; import org.elasticsearch.search.aggregations.Aggregations;
@ -279,7 +279,7 @@ public class ChunkedDataExtractor implements DataExtractor {
} }
private SearchRequestBuilder rangeSearchRequest() { private SearchRequestBuilder rangeSearchRequest() {
return new SearchRequestBuilder(client, SearchAction.INSTANCE).setIndices(context.indices) return new SearchRequestBuilder(client, TransportSearchAction.TYPE).setIndices(context.indices)
.setIndicesOptions(context.indicesOptions) .setIndicesOptions(context.indicesOptions)
.setSource(rangeSearchBuilder()) .setSource(rangeSearchBuilder())
.setAllowPartialSearchResults(false) .setAllowPartialSearchResults(false)

View file

@ -9,14 +9,14 @@ package org.elasticsearch.xpack.ml.datafeed.extractor.scroll;
import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.Logger;
import org.elasticsearch.ResourceNotFoundException; import org.elasticsearch.ResourceNotFoundException;
import org.elasticsearch.action.search.ClearScrollAction;
import org.elasticsearch.action.search.ClearScrollRequest; import org.elasticsearch.action.search.ClearScrollRequest;
import org.elasticsearch.action.search.SearchAction;
import org.elasticsearch.action.search.SearchPhaseExecutionException; import org.elasticsearch.action.search.SearchPhaseExecutionException;
import org.elasticsearch.action.search.SearchRequestBuilder; import org.elasticsearch.action.search.SearchRequestBuilder;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.search.SearchScrollAction;
import org.elasticsearch.action.search.SearchScrollRequestBuilder; 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.client.internal.Client;
import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.BytesStreamOutput;
import org.elasticsearch.core.TimeValue; import org.elasticsearch.core.TimeValue;
@ -154,7 +154,7 @@ class ScrollDataExtractor implements DataExtractor {
.query(ExtractorUtils.wrapInTimeRangeQuery(context.query, context.extractedFields.timeField(), start, context.end)) .query(ExtractorUtils.wrapInTimeRangeQuery(context.query, context.extractedFields.timeField(), start, context.end))
.runtimeMappings(context.runtimeMappings); .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) .setIndices(context.indices)
.setIndicesOptions(context.indicesOptions) .setIndicesOptions(context.indicesOptions)
.setAllowPartialSearchResults(false) .setAllowPartialSearchResults(false)
@ -250,7 +250,9 @@ class ScrollDataExtractor implements DataExtractor {
context.headers, context.headers,
ClientHelper.ML_ORIGIN, ClientHelper.ML_ORIGIN,
client, client,
() -> new SearchScrollRequestBuilder(client, SearchScrollAction.INSTANCE).setScroll(SCROLL_TIMEOUT).setScrollId(scrollId).get() () -> new SearchScrollRequestBuilder(client, TransportSearchScrollAction.TYPE).setScroll(SCROLL_TIMEOUT)
.setScrollId(scrollId)
.get()
); );
try { try {
checkForSkippedClusters(searchResponse); checkForSkippedClusters(searchResponse);
@ -284,7 +286,7 @@ class ScrollDataExtractor implements DataExtractor {
context.headers, context.headers,
ClientHelper.ML_ORIGIN, ClientHelper.ML_ORIGIN,
client, client,
() -> client.execute(ClearScrollAction.INSTANCE, request).actionGet() () -> client.execute(TransportClearScrollAction.TYPE, request).actionGet()
); );
} }
} }

View file

@ -13,9 +13,9 @@ import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.DocWriteResponse;
import org.elasticsearch.action.index.IndexAction; import org.elasticsearch.action.index.IndexAction;
import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.action.search.SearchAction;
import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.search.TransportSearchAction;
import org.elasticsearch.action.support.WriteRequest; import org.elasticsearch.action.support.WriteRequest;
import org.elasticsearch.client.internal.Client; import org.elasticsearch.client.internal.Client;
import org.elasticsearch.client.internal.ParentTaskAssigningClient; import org.elasticsearch.client.internal.ParentTaskAssigningClient;
@ -276,7 +276,7 @@ public class DataFrameAnalyticsTask extends LicensedAllocatedPersistentTask impl
SearchRequest searchRequest = new SearchRequest(AnomalyDetectorsIndex.jobStateIndexPattern()).source( SearchRequest searchRequest = new SearchRequest(AnomalyDetectorsIndex.jobStateIndexPattern()).source(
new SearchSourceBuilder().size(1).query(new IdsQueryBuilder().addIds(progressDocId)) 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 -> { }, e -> {
LOGGER.error( LOGGER.error(
() -> format("[%s] cannot persist progress as an error occurred while updating task progress", taskParams.getId()), () -> format("[%s] cannot persist progress as an error occurred while updating task progress", taskParams.getId()),

View file

@ -9,9 +9,9 @@ package org.elasticsearch.xpack.ml.dataframe.extractor;
import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.Logger;
import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.search.SearchAction;
import org.elasticsearch.action.search.SearchRequestBuilder; import org.elasticsearch.action.search.SearchRequestBuilder;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.search.TransportSearchAction;
import org.elasticsearch.client.internal.Client; import org.elasticsearch.client.internal.Client;
import org.elasticsearch.common.util.CachedSupplier; import org.elasticsearch.common.util.CachedSupplier;
import org.elasticsearch.core.Nullable; import org.elasticsearch.core.Nullable;
@ -127,7 +127,7 @@ public class DataFrameDataExtractor {
*/ */
public void preview(ActionListener<List<Row>> listener) { public void preview(ActionListener<List<Row>> 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 // This ensures the search throws if there are failures and the scroll context gets cleared automatically
.setAllowPartialSearchResults(false) .setAllowPartialSearchResults(false)
.setIndices(context.indices) .setIndices(context.indices)
@ -146,7 +146,7 @@ public class DataFrameDataExtractor {
context.headers, context.headers,
ClientHelper.ML_ORIGIN, ClientHelper.ML_ORIGIN,
client, client,
SearchAction.INSTANCE, TransportSearchAction.TYPE,
searchRequestBuilder.request(), searchRequestBuilder.request(),
ActionListener.wrap(searchResponse -> { ActionListener.wrap(searchResponse -> {
if (searchResponse.getHits().getHits().length == 0) { 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)); 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 // This ensures the search throws if there are failures and the scroll context gets cleared automatically
.setAllowPartialSearchResults(false) .setAllowPartialSearchResults(false)
.addSort(DestinationIndex.INCREMENTAL_ID, SortOrder.ASC) .addSort(DestinationIndex.INCREMENTAL_ID, SortOrder.ASC)
@ -383,7 +383,7 @@ public class DataFrameDataExtractor {
context.headers, context.headers,
ClientHelper.ML_ORIGIN, ClientHelper.ML_ORIGIN,
client, client,
SearchAction.INSTANCE, TransportSearchAction.TYPE,
searchRequestBuilder.request(), searchRequestBuilder.request(),
ActionListener.wrap( ActionListener.wrap(
searchResponse -> dataSummaryActionListener.onResponse( searchResponse -> dataSummaryActionListener.onResponse(
@ -401,7 +401,7 @@ public class DataFrameDataExtractor {
summaryQuery = QueryBuilders.boolQuery().filter(summaryQuery).filter(allExtractedFieldsExistQuery()); 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) .setIndices(context.indices)
.setSize(0) .setSize(0)
.setQuery(summaryQuery) .setQuery(summaryQuery)

View file

@ -17,9 +17,9 @@ import org.elasticsearch.action.fieldcaps.FieldCapabilities;
import org.elasticsearch.action.fieldcaps.FieldCapabilitiesAction; import org.elasticsearch.action.fieldcaps.FieldCapabilitiesAction;
import org.elasticsearch.action.fieldcaps.FieldCapabilitiesRequest; import org.elasticsearch.action.fieldcaps.FieldCapabilitiesRequest;
import org.elasticsearch.action.fieldcaps.FieldCapabilitiesResponse; import org.elasticsearch.action.fieldcaps.FieldCapabilitiesResponse;
import org.elasticsearch.action.search.SearchAction;
import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.search.TransportSearchAction;
import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.client.internal.Client; import org.elasticsearch.client.internal.Client;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
@ -141,7 +141,14 @@ public class ExtractedFieldsDetectorFactory {
); );
} }
SearchRequest searchRequest = new SearchRequest(index).source(searchSourceBuilder); 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( private static void buildFieldCardinalitiesMap(

View file

@ -12,8 +12,8 @@ import org.apache.logging.log4j.Logger;
import org.elasticsearch.ResourceNotFoundException; import org.elasticsearch.ResourceNotFoundException;
import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.admin.indices.refresh.RefreshResponse; 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.SearchRequest;
import org.elasticsearch.action.search.TransportSearchAction;
import org.elasticsearch.client.internal.node.NodeClient; import org.elasticsearch.client.internal.node.NodeClient;
import org.elasticsearch.core.TimeValue; import org.elasticsearch.core.TimeValue;
import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.index.query.QueryBuilders;
@ -122,7 +122,7 @@ public class InferenceStep extends AbstractDataFrameAnalyticsStep {
executeAsyncWithOrigin( executeAsyncWithOrigin(
client, client,
ML_ORIGIN, ML_ORIGIN,
SearchAction.INSTANCE, TransportSearchAction.TYPE,
searchRequest, searchRequest,
ActionListener.wrap( ActionListener.wrap(
searchResponse -> listener.onResponse(searchResponse.getHits().getTotalHits().value > 0), 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 searchRequest = new SearchRequest(InferenceIndexConstants.INDEX_PATTERN);
searchRequest.source(searchSourceBuilder); 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(); SearchHit[] hits = searchResponse.getHits().getHits();
if (hits.length == 0) { if (hits.length == 0) {
listener.onFailure(new ResourceNotFoundException("No model could be found to perform inference")); listener.onFailure(new ResourceNotFoundException("No model could be found to perform inference"));

View file

@ -12,8 +12,8 @@ import org.apache.logging.log4j.Logger;
import org.apache.lucene.util.SetOnce; import org.apache.lucene.util.SetOnce;
import org.elasticsearch.ResourceNotFoundException; import org.elasticsearch.ResourceNotFoundException;
import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.search.SearchAction;
import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.search.TransportSearchAction;
import org.elasticsearch.action.support.master.AcknowledgedResponse; import org.elasticsearch.action.support.master.AcknowledgedResponse;
import org.elasticsearch.client.internal.Client; import org.elasticsearch.client.internal.Client;
import org.elasticsearch.common.Strings; import org.elasticsearch.common.Strings;
@ -174,29 +174,37 @@ public class DeploymentManager {
task.init(nlpConfig); task.init(nlpConfig);
SearchRequest searchRequest = vocabSearchRequest(nlpConfig.getVocabularyConfig(), modelConfig.getModelId()); SearchRequest searchRequest = vocabSearchRequest(nlpConfig.getVocabularyConfig(), modelConfig.getModelId());
executeAsyncWithOrigin(client, ML_ORIGIN, SearchAction.INSTANCE, searchRequest, ActionListener.wrap(searchVocabResponse -> { executeAsyncWithOrigin(
if (searchVocabResponse.getHits().getHits().length == 0) { client,
failedDeploymentListener.onFailure( ML_ORIGIN,
new ResourceNotFoundException( TransportSearchAction.TYPE,
Messages.getMessage( searchRequest,
Messages.VOCABULARY_NOT_FOUND, ActionListener.wrap(searchVocabResponse -> {
modelConfig.getModelId(), if (searchVocabResponse.getHits().getHits().length == 0) {
VocabularyConfig.docId(modelConfig.getModelId()) 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)); Vocabulary vocabulary = parseVocabularyDocLeniently(searchVocabResponse.getHits().getAt(0));
NlpTask nlpTask = new NlpTask(nlpConfig, vocabulary); NlpTask nlpTask = new NlpTask(nlpConfig, vocabulary);
NlpTask.Processor processor = nlpTask.createProcessor(); NlpTask.Processor processor = nlpTask.createProcessor();
processContext.nlpTaskProcessor.set(processor); processContext.nlpTaskProcessor.set(processor);
// here, we are being called back on the searching thread, which MAY be a network thread // 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 // `startAndLoad` creates named pipes, blocking the calling thread, better to execute that in our utility
// executor. // executor.
executorServiceForDeployment.execute(() -> processContext.startAndLoad(modelConfig.getLocation(), modelLoadedListener)); executorServiceForDeployment.execute(
}, failedDeploymentListener::onFailure)); () -> processContext.startAndLoad(modelConfig.getLocation(), modelLoadedListener)
);
}, failedDeploymentListener::onFailure)
);
} else { } else {
failedDeploymentListener.onFailure( failedDeploymentListener.onFailure(
new IllegalArgumentException( new IllegalArgumentException(

View file

@ -25,9 +25,9 @@ import org.elasticsearch.action.index.IndexAction;
import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.action.search.MultiSearchRequest; import org.elasticsearch.action.search.MultiSearchRequest;
import org.elasticsearch.action.search.MultiSearchResponse; import org.elasticsearch.action.search.MultiSearchResponse;
import org.elasticsearch.action.search.SearchAction;
import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.search.TransportSearchAction;
import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.action.support.WriteRequest; import org.elasticsearch.action.support.WriteRequest;
import org.elasticsearch.client.internal.Client; import org.elasticsearch.client.internal.Client;
@ -400,7 +400,7 @@ public class TrainedModelProvider {
if (parentTaskId != null) { if (parentTaskId != null) {
searchRequest.setParentTask(parentTaskId); 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) { if (searchResponse.getHits().getHits().length == 0) {
listener.onFailure(new ResourceNotFoundException(Messages.getMessage(Messages.MODEL_METADATA_NOT_FOUND, modelIds))); listener.onFailure(new ResourceNotFoundException(Messages.getMessage(Messages.MODEL_METADATA_NOT_FOUND, modelIds)));
return; return;
@ -691,7 +691,7 @@ public class TrainedModelProvider {
executeAsyncWithOrigin( executeAsyncWithOrigin(
client, client,
ML_ORIGIN, ML_ORIGIN,
SearchAction.INSTANCE, TransportSearchAction.TYPE,
ChunkedTrainedModelRestorer.buildSearch( ChunkedTrainedModelRestorer.buildSearch(
client, client,
modelId, modelId,
@ -731,7 +731,7 @@ public class TrainedModelProvider {
}, getTrainedModelListener::onFailure) }, getTrainedModelListener::onFailure)
); );
}, getTrainedModelListener::onFailure); }, getTrainedModelListener::onFailure);
executeAsyncWithOrigin(client, ML_ORIGIN, SearchAction.INSTANCE, trainedModelConfigSearch, trainedModelSearchHandler); executeAsyncWithOrigin(client, ML_ORIGIN, TransportSearchAction.TYPE, trainedModelConfigSearch, trainedModelSearchHandler);
} }
public void getTrainedModels( public void getTrainedModels(
@ -872,7 +872,7 @@ public class TrainedModelProvider {
getTrainedModelListener.onResponse(configs); getTrainedModelListener.onResponse(configs);
}, getTrainedModelListener::onFailure); }, getTrainedModelListener::onFailure);
executeAsyncWithOrigin(client, ML_ORIGIN, SearchAction.INSTANCE, searchRequest, configSearchHandler); executeAsyncWithOrigin(client, ML_ORIGIN, TransportSearchAction.TYPE, searchRequest, configSearchHandler);
} }
public void deleteTrainedModel(String modelId, ActionListener<Boolean> listener) { public void deleteTrainedModel(String modelId, ActionListener<Boolean> listener) {

View file

@ -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.RefreshRequest;
import org.elasticsearch.action.admin.indices.refresh.RefreshResponse; import org.elasticsearch.action.admin.indices.refresh.RefreshResponse;
import org.elasticsearch.action.bulk.BulkItemResponse; import org.elasticsearch.action.bulk.BulkItemResponse;
import org.elasticsearch.action.search.MultiSearchAction;
import org.elasticsearch.action.search.MultiSearchRequest; import org.elasticsearch.action.search.MultiSearchRequest;
import org.elasticsearch.action.search.MultiSearchResponse; import org.elasticsearch.action.search.MultiSearchResponse;
import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.search.TransportMultiSearchAction;
import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.action.support.master.AcknowledgedResponse; import org.elasticsearch.action.support.master.AcknowledgedResponse;
import org.elasticsearch.client.internal.Client; import org.elasticsearch.client.internal.Client;
@ -413,7 +413,7 @@ public class JobDataDeleter {
); );
multiSearchRequest.add(new SearchRequest(indexName).source(source)); multiSearchRequest.add(new SearchRequest(indexName).source(source));
} }
executeAsyncWithOrigin(client, ML_ORIGIN, MultiSearchAction.INSTANCE, multiSearchRequest, customIndexSearchHandler); executeAsyncWithOrigin(client, ML_ORIGIN, TransportMultiSearchAction.TYPE, multiSearchRequest, customIndexSearchHandler);
}, failureHandler); }, failureHandler);
// Step 5. Get the job as the initial result index name is required // Step 5. Get the job as the initial result index name is required

View file

@ -10,9 +10,9 @@ import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.Logger;
import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.search.SearchAction;
import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.search.TransportSearchAction;
import org.elasticsearch.action.support.ThreadedActionListener; import org.elasticsearch.action.support.ThreadedActionListener;
import org.elasticsearch.client.internal.OriginSettingClient; import org.elasticsearch.client.internal.OriginSettingClient;
import org.elasticsearch.index.query.BoolQueryBuilder; import org.elasticsearch.index.query.BoolQueryBuilder;
@ -98,7 +98,7 @@ public class ExpiredForecastsRemover implements MlDataRemover {
searchRequest.source(source); searchRequest.source(source);
searchRequest.setParentTask(parentTaskId); searchRequest.setParentTask(parentTaskId);
client.execute( client.execute(
SearchAction.INSTANCE, TransportSearchAction.TYPE,
searchRequest, searchRequest,
new ThreadedActionListener<>(threadPool.executor(MachineLearning.UTILITY_THREAD_POOL_NAME), forecastStatsHandler) new ThreadedActionListener<>(threadPool.executor(MachineLearning.UTILITY_THREAD_POOL_NAME), forecastStatsHandler)
); );

View file

@ -7,12 +7,12 @@
package org.elasticsearch.xpack.ml.datafeed.extractor.aggregation; package org.elasticsearch.xpack.ml.datafeed.extractor.aggregation;
import org.elasticsearch.action.ActionRequestBuilder; import org.elasticsearch.action.ActionRequestBuilder;
import org.elasticsearch.action.search.SearchAction;
import org.elasticsearch.action.search.SearchPhaseExecutionException; import org.elasticsearch.action.search.SearchPhaseExecutionException;
import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchRequest;
import org.elasticsearch.action.search.SearchRequestBuilder; import org.elasticsearch.action.search.SearchRequestBuilder;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.search.ShardSearchFailure; import org.elasticsearch.action.search.ShardSearchFailure;
import org.elasticsearch.action.search.TransportSearchAction;
import org.elasticsearch.client.internal.Client; import org.elasticsearch.client.internal.Client;
import org.elasticsearch.core.TimeValue; import org.elasticsearch.core.TimeValue;
import org.elasticsearch.core.Tuple; import org.elasticsearch.core.Tuple;
@ -122,9 +122,10 @@ public class CompositeAggregationDataExtractorTests extends ESTestCase {
.subAggregation(AggregationBuilders.avg("responsetime").field("responsetime")); .subAggregation(AggregationBuilders.avg("responsetime").field("responsetime"));
runtimeMappings = Collections.emptyMap(); runtimeMappings = Collections.emptyMap();
timingStatsReporter = new DatafeedTimingStatsReporter(new DatafeedTimingStats(jobId), mock(DatafeedTimingStatsPersister.class)); timingStatsReporter = new DatafeedTimingStatsReporter(new DatafeedTimingStats(jobId), mock(DatafeedTimingStatsPersister.class));
aggregatedSearchRequestBuilder = (searchSourceBuilder) -> new SearchRequestBuilder(testClient, SearchAction.INSTANCE).setSource( aggregatedSearchRequestBuilder = (searchSourceBuilder) -> new SearchRequestBuilder(testClient, TransportSearchAction.TYPE)
searchSourceBuilder .setSource(searchSourceBuilder)
).setAllowPartialSearchResults(false).setIndices(indices.toArray(String[]::new)); .setAllowPartialSearchResults(false)
.setIndices(indices.toArray(String[]::new));
} }
public void testExtraction() throws IOException { public void testExtraction() throws IOException {

View file

@ -9,7 +9,6 @@ package org.elasticsearch.xpack.ml.datafeed.extractor.scroll;
import org.apache.lucene.search.TotalHits; import org.apache.lucene.search.TotalHits;
import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.action.ActionFuture; import org.elasticsearch.action.ActionFuture;
import org.elasticsearch.action.search.ClearScrollAction;
import org.elasticsearch.action.search.ClearScrollRequest; import org.elasticsearch.action.search.ClearScrollRequest;
import org.elasticsearch.action.search.ClearScrollResponse; import org.elasticsearch.action.search.ClearScrollResponse;
import org.elasticsearch.action.search.SearchPhaseExecutionException; 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.SearchRequestBuilder;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.search.ShardSearchFailure; import org.elasticsearch.action.search.ShardSearchFailure;
import org.elasticsearch.action.search.TransportClearScrollAction;
import org.elasticsearch.client.internal.Client; import org.elasticsearch.client.internal.Client;
import org.elasticsearch.common.document.DocumentField; import org.elasticsearch.common.document.DocumentField;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
@ -173,7 +173,7 @@ public class ScrollDataExtractorTests extends ESTestCase {
clearScrollFuture = mock(ActionFuture.class); clearScrollFuture = mock(ActionFuture.class);
capturedClearScrollRequests = ArgumentCaptor.forClass(ClearScrollRequest.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)); timingStatsReporter = new DatafeedTimingStatsReporter(new DatafeedTimingStats(jobId), mock(DatafeedTimingStatsPersister.class));
} }

View file

@ -10,8 +10,8 @@ import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.index.IndexAction; import org.elasticsearch.action.index.IndexAction;
import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.index.IndexResponse;
import org.elasticsearch.action.search.SearchAction;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.search.TransportSearchAction;
import org.elasticsearch.client.internal.Client; import org.elasticsearch.client.internal.Client;
import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
@ -188,7 +188,7 @@ public class DataFrameAnalyticsTaskTests extends ESTestCase {
SearchResponse searchResponse = mock(SearchResponse.class); SearchResponse searchResponse = mock(SearchResponse.class);
when(searchResponse.getHits()).thenReturn(searchHits); 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); IndexResponse indexResponse = mock(IndexResponse.class);
doAnswer(withResponse(indexResponse)).when(client).execute(eq(IndexAction.INSTANCE), any(), any()); doAnswer(withResponse(indexResponse)).when(client).execute(eq(IndexAction.INSTANCE), any(), any());
@ -217,7 +217,7 @@ public class DataFrameAnalyticsTaskTests extends ESTestCase {
ArgumentCaptor<IndexRequest> indexRequestCaptor = ArgumentCaptor.forClass(IndexRequest.class); ArgumentCaptor<IndexRequest> indexRequestCaptor = ArgumentCaptor.forClass(IndexRequest.class);
InOrder inOrder = inOrder(client, runnable); 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(client).execute(eq(IndexAction.INSTANCE), indexRequestCaptor.capture(), any());
inOrder.verify(runnable).run(); inOrder.verify(runnable).run();
inOrder.verifyNoMoreInteractions(); inOrder.verifyNoMoreInteractions();
@ -284,7 +284,7 @@ public class DataFrameAnalyticsTaskTests extends ESTestCase {
SearchResponse searchResponse = mock(SearchResponse.class); SearchResponse searchResponse = mock(SearchResponse.class);
when(searchResponse.getHits()).thenReturn(SearchHits.EMPTY_WITH_TOTAL_HITS); 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); IndexResponse indexResponse = mock(IndexResponse.class);
doAnswer(withResponse(indexResponse)).when(client).execute(eq(IndexAction.INSTANCE), any(), any()); doAnswer(withResponse(indexResponse)).when(client).execute(eq(IndexAction.INSTANCE), any(), any());
@ -315,7 +315,7 @@ public class DataFrameAnalyticsTaskTests extends ESTestCase {
if (nodeShuttingDown == false) { if (nodeShuttingDown == false) {
// Verify progress was persisted // Verify progress was persisted
ArgumentCaptor<IndexRequest> indexRequestCaptor = ArgumentCaptor.forClass(IndexRequest.class); ArgumentCaptor<IndexRequest> 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()); verify(client).execute(eq(IndexAction.INSTANCE), indexRequestCaptor.capture(), any());
IndexRequest indexRequest = indexRequestCaptor.getValue(); IndexRequest indexRequest = indexRequestCaptor.getValue();

Some files were not shown because too many files have changed in this diff Show more