mirror of
https://github.com/elastic/elasticsearch.git
synced 2025-04-24 23:27:25 -04:00
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:
parent
0eca436728
commit
c45b95d0eb
150 changed files with 1196 additions and 1049 deletions
|
@ -12,9 +12,9 @@ import org.elasticsearch.ExceptionsHelper;
|
|||
import org.elasticsearch.action.ActionFuture;
|
||||
import org.elasticsearch.action.DocWriteRequest;
|
||||
import org.elasticsearch.action.bulk.BulkRequestBuilder;
|
||||
import org.elasticsearch.action.search.SearchAction;
|
||||
import org.elasticsearch.action.search.SearchPhaseExecutionException;
|
||||
import org.elasticsearch.action.search.SearchResponse;
|
||||
import org.elasticsearch.action.search.TransportSearchAction;
|
||||
import org.elasticsearch.action.support.WriteRequest;
|
||||
import org.elasticsearch.aggregations.AggregationsPlugin;
|
||||
import org.elasticsearch.aggregations.bucket.timeseries.TimeSeriesAggregationBuilder;
|
||||
|
@ -115,7 +115,7 @@ public class SearchCancellationIT extends AbstractSearchCancellationTestCase {
|
|||
)
|
||||
.execute();
|
||||
awaitForBlock(plugins);
|
||||
cancelSearch(SearchAction.NAME);
|
||||
cancelSearch(TransportSearchAction.TYPE.name());
|
||||
disableBlocks(plugins);
|
||||
|
||||
SearchPhaseExecutionException ex = expectThrows(SearchPhaseExecutionException.class, searchResponse::actionGet);
|
||||
|
|
|
@ -14,11 +14,11 @@ import org.elasticsearch.action.ActionRequest;
|
|||
import org.elasticsearch.action.ActionResponse;
|
||||
import org.elasticsearch.action.ActionType;
|
||||
import org.elasticsearch.action.bulk.BackoffPolicy;
|
||||
import org.elasticsearch.action.search.SearchAction;
|
||||
import org.elasticsearch.action.search.SearchRequest;
|
||||
import org.elasticsearch.action.search.SearchResponse;
|
||||
import org.elasticsearch.action.search.SearchScrollAction;
|
||||
import org.elasticsearch.action.search.SearchScrollRequest;
|
||||
import org.elasticsearch.action.search.TransportSearchAction;
|
||||
import org.elasticsearch.action.search.TransportSearchScrollAction;
|
||||
import org.elasticsearch.action.support.PlainActionFuture;
|
||||
import org.elasticsearch.client.internal.ParentTaskAssigningClient;
|
||||
import org.elasticsearch.client.internal.support.AbstractClient;
|
||||
|
@ -102,17 +102,17 @@ public class ClientScrollableHitSourceTests extends ESTestCase {
|
|||
|
||||
hitSource.start();
|
||||
for (int retry = 0; retry < randomIntBetween(minFailures, maxFailures); ++retry) {
|
||||
client.fail(SearchAction.INSTANCE, new EsRejectedExecutionException());
|
||||
client.fail(TransportSearchAction.TYPE, new EsRejectedExecutionException());
|
||||
if (retry >= retries) {
|
||||
return;
|
||||
}
|
||||
client.awaitOperation();
|
||||
++expectedSearchRetries;
|
||||
}
|
||||
client.validateRequest(SearchAction.INSTANCE, (SearchRequest r) -> assertTrue(r.allowPartialSearchResults() == Boolean.FALSE));
|
||||
client.validateRequest(TransportSearchAction.TYPE, (SearchRequest r) -> assertTrue(r.allowPartialSearchResults() == Boolean.FALSE));
|
||||
SearchResponse searchResponse = createSearchResponse();
|
||||
try {
|
||||
client.respond(SearchAction.INSTANCE, searchResponse);
|
||||
client.respond(TransportSearchAction.TYPE, searchResponse);
|
||||
|
||||
for (int i = 0; i < randomIntBetween(1, 10); ++i) {
|
||||
ScrollableHitSource.AsyncResponse asyncResponse = responses.poll(10, TimeUnit.SECONDS);
|
||||
|
@ -122,13 +122,13 @@ public class ClientScrollableHitSourceTests extends ESTestCase {
|
|||
asyncResponse.done(TimeValue.ZERO);
|
||||
|
||||
for (int retry = 0; retry < randomIntBetween(minFailures, maxFailures); ++retry) {
|
||||
client.fail(SearchScrollAction.INSTANCE, new EsRejectedExecutionException());
|
||||
client.fail(TransportSearchScrollAction.TYPE, new EsRejectedExecutionException());
|
||||
client.awaitOperation();
|
||||
++expectedSearchRetries;
|
||||
}
|
||||
|
||||
searchResponse = createSearchResponse();
|
||||
client.respond(SearchScrollAction.INSTANCE, searchResponse);
|
||||
client.respond(TransportSearchScrollAction.TYPE, searchResponse);
|
||||
}
|
||||
|
||||
assertEquals(actualSearchRetries.get(), expectedSearchRetries);
|
||||
|
@ -154,7 +154,10 @@ public class ClientScrollableHitSourceTests extends ESTestCase {
|
|||
);
|
||||
|
||||
hitSource.startNextScroll(timeValueSeconds(100));
|
||||
client.validateRequest(SearchScrollAction.INSTANCE, (SearchScrollRequest r) -> assertEquals(r.scroll().keepAlive().seconds(), 110));
|
||||
client.validateRequest(
|
||||
TransportSearchScrollAction.TYPE,
|
||||
(SearchScrollRequest r) -> assertEquals(r.scroll().keepAlive().seconds(), 110)
|
||||
);
|
||||
}
|
||||
|
||||
private SearchResponse createSearchResponse() {
|
||||
|
|
|
@ -15,7 +15,7 @@ import org.elasticsearch.action.ActionListener;
|
|||
import org.elasticsearch.action.ActionRequest;
|
||||
import org.elasticsearch.action.ActionResponse;
|
||||
import org.elasticsearch.action.admin.cluster.node.info.NodeInfo;
|
||||
import org.elasticsearch.action.search.SearchAction;
|
||||
import org.elasticsearch.action.search.TransportSearchAction;
|
||||
import org.elasticsearch.action.support.ActionFilter;
|
||||
import org.elasticsearch.action.support.ActionFilterChain;
|
||||
import org.elasticsearch.action.support.WriteRequest.RefreshPolicy;
|
||||
|
@ -199,7 +199,7 @@ public class ReindexFromRemoteWithAuthTests extends ESSingleNodeTestCase {
|
|||
ActionListener<Response> listener,
|
||||
ActionFilterChain<Request, Response> chain
|
||||
) {
|
||||
if (false == action.equals(SearchAction.NAME)) {
|
||||
if (false == action.equals(TransportSearchAction.TYPE.name())) {
|
||||
chain.proceed(task, action, request, listener);
|
||||
return;
|
||||
}
|
||||
|
|
|
@ -16,13 +16,13 @@ import org.elasticsearch.TransportVersion;
|
|||
import org.elasticsearch.action.admin.cluster.state.ClusterStateAction;
|
||||
import org.elasticsearch.action.admin.cluster.state.ClusterStateRequest;
|
||||
import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse;
|
||||
import org.elasticsearch.action.search.SearchAction;
|
||||
import org.elasticsearch.action.search.SearchRequest;
|
||||
import org.elasticsearch.action.search.SearchResponse;
|
||||
import org.elasticsearch.action.search.SearchShardsAction;
|
||||
import org.elasticsearch.action.search.SearchShardsRequest;
|
||||
import org.elasticsearch.action.search.SearchShardsResponse;
|
||||
import org.elasticsearch.action.search.ShardSearchFailure;
|
||||
import org.elasticsearch.action.search.TransportSearchAction;
|
||||
import org.elasticsearch.action.search.TransportSearchShardsAction;
|
||||
import org.elasticsearch.client.Request;
|
||||
import org.elasticsearch.client.Response;
|
||||
import org.elasticsearch.client.ResponseException;
|
||||
|
@ -81,7 +81,7 @@ public class CrossClusterSearchUnavailableClusterIT extends ESRestTestCase {
|
|||
MockTransportService newService = MockTransportService.createNewService(s, version, transportVersion, threadPool, null);
|
||||
try {
|
||||
newService.registerRequestHandler(
|
||||
SearchShardsAction.NAME,
|
||||
TransportSearchShardsAction.TYPE.name(),
|
||||
EsExecutors.DIRECT_EXECUTOR_SERVICE,
|
||||
SearchShardsRequest::new,
|
||||
(request, channel, task) -> {
|
||||
|
@ -89,7 +89,7 @@ public class CrossClusterSearchUnavailableClusterIT extends ESRestTestCase {
|
|||
}
|
||||
);
|
||||
newService.registerRequestHandler(
|
||||
SearchAction.NAME,
|
||||
TransportSearchAction.TYPE.name(),
|
||||
EsExecutors.DIRECT_EXECUTOR_SERVICE,
|
||||
SearchRequest::new,
|
||||
(request, channel, task) -> {
|
||||
|
|
|
@ -15,10 +15,10 @@ import org.elasticsearch.action.admin.cluster.node.info.NodeInfo;
|
|||
import org.elasticsearch.action.admin.cluster.node.info.NodesInfoResponse;
|
||||
import org.elasticsearch.action.admin.cluster.node.tasks.list.ListTasksResponse;
|
||||
import org.elasticsearch.action.bulk.BulkRequestBuilder;
|
||||
import org.elasticsearch.action.search.MultiSearchAction;
|
||||
import org.elasticsearch.action.search.MultiSearchRequest;
|
||||
import org.elasticsearch.action.search.SearchAction;
|
||||
import org.elasticsearch.action.search.SearchRequest;
|
||||
import org.elasticsearch.action.search.TransportMultiSearchAction;
|
||||
import org.elasticsearch.action.search.TransportSearchAction;
|
||||
import org.elasticsearch.action.support.PlainActionFuture;
|
||||
import org.elasticsearch.action.support.WriteRequest;
|
||||
import org.elasticsearch.client.Cancellable;
|
||||
|
@ -73,7 +73,7 @@ public class SearchRestCancellationIT extends HttpSmokeTestCase {
|
|||
scriptQuery(new Script(ScriptType.INLINE, "mockscript", ScriptedBlockPlugin.SCRIPT_NAME, Collections.emptyMap()))
|
||||
);
|
||||
searchRequest.setJsonEntity(Strings.toString(searchSource));
|
||||
verifyCancellationDuringQueryPhase(SearchAction.NAME, searchRequest);
|
||||
verifyCancellationDuringQueryPhase(TransportSearchAction.TYPE.name(), searchRequest);
|
||||
}
|
||||
|
||||
public void testAutomaticCancellationMultiSearchDuringQueryPhase() throws Exception {
|
||||
|
@ -89,7 +89,7 @@ public class SearchRestCancellationIT extends HttpSmokeTestCase {
|
|||
Request restRequest = new Request("POST", "/_msearch");
|
||||
byte[] requestBody = MultiSearchRequest.writeMultiLineFormat(multiSearchRequest, contentType.xContent());
|
||||
restRequest.setEntity(new NByteArrayEntity(requestBody, createContentType(contentType)));
|
||||
verifyCancellationDuringQueryPhase(MultiSearchAction.NAME, restRequest);
|
||||
verifyCancellationDuringQueryPhase(TransportMultiSearchAction.TYPE.name(), restRequest);
|
||||
}
|
||||
|
||||
void verifyCancellationDuringQueryPhase(String searchAction, Request searchRequest) throws Exception {
|
||||
|
@ -116,7 +116,7 @@ public class SearchRestCancellationIT extends HttpSmokeTestCase {
|
|||
new Script(ScriptType.INLINE, "mockscript", ScriptedBlockPlugin.SCRIPT_NAME, Collections.emptyMap())
|
||||
);
|
||||
searchRequest.setJsonEntity(Strings.toString(searchSource));
|
||||
verifyCancellationDuringFetchPhase(SearchAction.NAME, searchRequest);
|
||||
verifyCancellationDuringFetchPhase(TransportSearchAction.TYPE.name(), searchRequest);
|
||||
}
|
||||
|
||||
public void testAutomaticCancellationMultiSearchDuringFetchPhase() throws Exception {
|
||||
|
@ -132,7 +132,7 @@ public class SearchRestCancellationIT extends HttpSmokeTestCase {
|
|||
Request restRequest = new Request("POST", "/_msearch");
|
||||
byte[] requestBody = MultiSearchRequest.writeMultiLineFormat(multiSearchRequest, contentType.xContent());
|
||||
restRequest.setEntity(new NByteArrayEntity(requestBody, createContentType(contentType)));
|
||||
verifyCancellationDuringFetchPhase(MultiSearchAction.NAME, restRequest);
|
||||
verifyCancellationDuringFetchPhase(TransportMultiSearchAction.TYPE.name(), restRequest);
|
||||
}
|
||||
|
||||
void verifyCancellationDuringFetchPhase(String searchAction, Request searchRequest) throws Exception {
|
||||
|
|
|
@ -45,8 +45,8 @@ import org.elasticsearch.action.admin.indices.validate.query.ValidateQueryReques
|
|||
import org.elasticsearch.action.bulk.BulkAction;
|
||||
import org.elasticsearch.action.bulk.BulkRequest;
|
||||
import org.elasticsearch.action.delete.DeleteRequest;
|
||||
import org.elasticsearch.action.explain.ExplainAction;
|
||||
import org.elasticsearch.action.explain.ExplainRequest;
|
||||
import org.elasticsearch.action.explain.TransportExplainAction;
|
||||
import org.elasticsearch.action.fieldcaps.FieldCapabilitiesAction;
|
||||
import org.elasticsearch.action.fieldcaps.FieldCapabilitiesRequest;
|
||||
import org.elasticsearch.action.get.GetAction;
|
||||
|
@ -317,7 +317,7 @@ public class IndicesRequestIT extends ESIntegTestCase {
|
|||
}
|
||||
|
||||
public void testExplain() {
|
||||
String explainShardAction = ExplainAction.NAME + "[s]";
|
||||
String explainShardAction = TransportExplainAction.TYPE.name() + "[s]";
|
||||
interceptTransportActions(explainShardAction);
|
||||
|
||||
ExplainRequest explainRequest = new ExplainRequest(randomIndexOrAlias(), "id").query(QueryBuilders.matchAllQuery());
|
||||
|
|
|
@ -26,8 +26,8 @@ import org.elasticsearch.action.admin.indices.refresh.RefreshAction;
|
|||
import org.elasticsearch.action.admin.indices.validate.query.ValidateQueryAction;
|
||||
import org.elasticsearch.action.bulk.BulkAction;
|
||||
import org.elasticsearch.action.index.IndexAction;
|
||||
import org.elasticsearch.action.search.SearchAction;
|
||||
import org.elasticsearch.action.search.SearchTransportService;
|
||||
import org.elasticsearch.action.search.TransportSearchAction;
|
||||
import org.elasticsearch.action.support.WriteRequest;
|
||||
import org.elasticsearch.action.support.replication.ReplicationResponse;
|
||||
import org.elasticsearch.action.support.replication.TransportReplicationActionTests;
|
||||
|
@ -349,8 +349,8 @@ public class TasksIT extends ESIntegTestCase {
|
|||
}
|
||||
|
||||
public void testSearchTaskDescriptions() {
|
||||
registerTaskManagerListeners(SearchAction.NAME); // main task
|
||||
registerTaskManagerListeners(SearchAction.NAME + "[*]"); // shard task
|
||||
registerTaskManagerListeners(TransportSearchAction.TYPE.name()); // main task
|
||||
registerTaskManagerListeners(TransportSearchAction.TYPE.name() + "[*]"); // shard task
|
||||
createIndex("test");
|
||||
ensureGreen("test"); // Make sure all shards are allocated to catch replication tasks
|
||||
prepareIndex("test").setId("test_id")
|
||||
|
@ -365,14 +365,14 @@ public class TasksIT extends ESIntegTestCase {
|
|||
assertNoFailures(client().filterWithHeader(headers).prepareSearch("test").setQuery(QueryBuilders.matchAllQuery()));
|
||||
|
||||
// the search operation should produce one main task
|
||||
List<TaskInfo> mainTask = findEvents(SearchAction.NAME, Tuple::v1);
|
||||
List<TaskInfo> mainTask = findEvents(TransportSearchAction.TYPE.name(), Tuple::v1);
|
||||
assertEquals(1, mainTask.size());
|
||||
assertThat(mainTask.get(0).description(), startsWith("indices[test], search_type["));
|
||||
assertThat(mainTask.get(0).description(), containsString("\"query\":{\"match_all\""));
|
||||
assertTaskHeaders(mainTask.get(0));
|
||||
|
||||
// check that if we have any shard-level requests they all have non-zero length description
|
||||
List<TaskInfo> shardTasks = findEvents(SearchAction.NAME + "[*]", Tuple::v1);
|
||||
List<TaskInfo> shardTasks = findEvents(TransportSearchAction.TYPE.name() + "[*]", Tuple::v1);
|
||||
for (TaskInfo taskInfo : shardTasks) {
|
||||
assertThat(taskInfo.parentTaskId(), notNullValue());
|
||||
assertEquals(mainTask.get(0).taskId(), taskInfo.parentTaskId());
|
||||
|
|
|
@ -202,11 +202,11 @@ public class CCSPointInTimeIT extends AbstractMultiClustersTestCase {
|
|||
|
||||
private String openPointInTime(String[] indices, TimeValue keepAlive) {
|
||||
OpenPointInTimeRequest request = new OpenPointInTimeRequest(indices).keepAlive(keepAlive);
|
||||
final OpenPointInTimeResponse response = client().execute(OpenPointInTimeAction.INSTANCE, request).actionGet();
|
||||
final OpenPointInTimeResponse response = client().execute(TransportOpenPointInTimeAction.TYPE, request).actionGet();
|
||||
return response.getPointInTimeId();
|
||||
}
|
||||
|
||||
private void closePointInTime(String readerId) {
|
||||
client().execute(ClosePointInTimeAction.INSTANCE, new ClosePointInTimeRequest(readerId)).actionGet();
|
||||
client().execute(TransportClosePointInTimeAction.TYPE, new ClosePointInTimeRequest(readerId)).actionGet();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -168,7 +168,7 @@ public class PointInTimeIT extends ESIntegTestCase {
|
|||
{
|
||||
|
||||
OpenPointInTimeRequest request = new OpenPointInTimeRequest("*").keepAlive(TimeValue.timeValueMinutes(2));
|
||||
final OpenPointInTimeResponse response = client().execute(OpenPointInTimeAction.INSTANCE, request).actionGet();
|
||||
final OpenPointInTimeResponse response = client().execute(TransportOpenPointInTimeAction.TYPE, request).actionGet();
|
||||
try {
|
||||
SearchContextId searchContextId = SearchContextId.decode(writableRegistry(), response.getPointInTimeId());
|
||||
String[] actualIndices = searchContextId.getActualIndices();
|
||||
|
@ -180,7 +180,7 @@ public class PointInTimeIT extends ESIntegTestCase {
|
|||
{
|
||||
OpenPointInTimeRequest request = new OpenPointInTimeRequest("*").keepAlive(TimeValue.timeValueMinutes(2));
|
||||
request.indexFilter(new RangeQueryBuilder("@timestamp").gte("2023-03-01"));
|
||||
final OpenPointInTimeResponse response = client().execute(OpenPointInTimeAction.INSTANCE, request).actionGet();
|
||||
final OpenPointInTimeResponse response = client().execute(TransportOpenPointInTimeAction.TYPE, request).actionGet();
|
||||
String pitId = response.getPointInTimeId();
|
||||
try {
|
||||
SearchContextId searchContextId = SearchContextId.decode(writableRegistry(), pitId);
|
||||
|
@ -333,8 +333,8 @@ public class PointInTimeIT extends ESIntegTestCase {
|
|||
public void testAllowNoIndex() {
|
||||
var request = new OpenPointInTimeRequest("my_index").indicesOptions(IndicesOptions.LENIENT_EXPAND_OPEN)
|
||||
.keepAlive(TimeValue.timeValueMinutes(between(1, 10)));
|
||||
String pit = client().execute(OpenPointInTimeAction.INSTANCE, request).actionGet().getPointInTimeId();
|
||||
var closeResp = client().execute(ClosePointInTimeAction.INSTANCE, new ClosePointInTimeRequest(pit)).actionGet();
|
||||
String pit = client().execute(TransportOpenPointInTimeAction.TYPE, request).actionGet().getPointInTimeId();
|
||||
var closeResp = client().execute(TransportClosePointInTimeAction.TYPE, new ClosePointInTimeRequest(pit)).actionGet();
|
||||
assertThat(closeResp.status(), equalTo(RestStatus.OK));
|
||||
}
|
||||
|
||||
|
@ -478,8 +478,11 @@ public class PointInTimeIT extends ESIntegTestCase {
|
|||
}
|
||||
|
||||
public void testCloseInvalidPointInTime() {
|
||||
expectThrows(Exception.class, () -> client().execute(ClosePointInTimeAction.INSTANCE, new ClosePointInTimeRequest("")).actionGet());
|
||||
List<TaskInfo> tasks = clusterAdmin().prepareListTasks().setActions(ClosePointInTimeAction.NAME).get().getTasks();
|
||||
expectThrows(
|
||||
Exception.class,
|
||||
() -> client().execute(TransportClosePointInTimeAction.TYPE, new ClosePointInTimeRequest("")).actionGet()
|
||||
);
|
||||
List<TaskInfo> tasks = clusterAdmin().prepareListTasks().setActions(TransportClosePointInTimeAction.TYPE.name()).get().getTasks();
|
||||
assertThat(tasks, empty());
|
||||
}
|
||||
|
||||
|
@ -520,7 +523,7 @@ public class PointInTimeIT extends ESIntegTestCase {
|
|||
OpenPointInTimeRequest request = new OpenPointInTimeRequest("test").keepAlive(TimeValue.timeValueMinutes(1));
|
||||
request.maxConcurrentShardRequests(maxConcurrentRequests);
|
||||
PlainActionFuture<OpenPointInTimeResponse> future = new PlainActionFuture<>();
|
||||
client().execute(OpenPointInTimeAction.INSTANCE, request, future);
|
||||
client().execute(TransportOpenPointInTimeAction.TYPE, request, future);
|
||||
assertTrue(sentLatch.await(1, TimeUnit.MINUTES));
|
||||
readyLatch.countDown();
|
||||
closePointInTime(future.actionGet().getPointInTimeId());
|
||||
|
@ -584,11 +587,11 @@ public class PointInTimeIT extends ESIntegTestCase {
|
|||
|
||||
private String openPointInTime(String[] indices, TimeValue keepAlive) {
|
||||
OpenPointInTimeRequest request = new OpenPointInTimeRequest(indices).keepAlive(keepAlive);
|
||||
final OpenPointInTimeResponse response = client().execute(OpenPointInTimeAction.INSTANCE, request).actionGet();
|
||||
final OpenPointInTimeResponse response = client().execute(TransportOpenPointInTimeAction.TYPE, request).actionGet();
|
||||
return response.getPointInTimeId();
|
||||
}
|
||||
|
||||
private void closePointInTime(String readerId) {
|
||||
client().execute(ClosePointInTimeAction.INSTANCE, new ClosePointInTimeRequest(readerId)).actionGet();
|
||||
client().execute(TransportClosePointInTimeAction.TYPE, new ClosePointInTimeRequest(readerId)).actionGet();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -162,7 +162,7 @@ public class SearchProgressActionListenerIT extends ESSingleNodeTestCase {
|
|||
throw new AssertionError();
|
||||
}
|
||||
};
|
||||
client.executeLocally(SearchAction.INSTANCE, new SearchRequest(request) {
|
||||
client.executeLocally(TransportSearchAction.TYPE, new SearchRequest(request) {
|
||||
@Override
|
||||
public SearchTask createTask(long id, String type, String action, TaskId parentTaskId, Map<String, String> headers) {
|
||||
SearchTask task = super.createTask(id, type, action, parentTaskId, headers);
|
||||
|
|
|
@ -70,7 +70,7 @@ public class SearchShardsIT extends ESIntegTestCase {
|
|||
randomBoolean(),
|
||||
randomBoolean() ? null : randomAlphaOfLength(10)
|
||||
);
|
||||
var resp = client().execute(SearchShardsAction.INSTANCE, request).actionGet();
|
||||
var resp = client().execute(TransportSearchShardsAction.TYPE, request).actionGet();
|
||||
assertThat(resp.getGroups(), hasSize(indicesWithData + indicesWithoutData));
|
||||
int skipped = 0;
|
||||
for (SearchShardsGroup g : resp.getGroups()) {
|
||||
|
@ -97,7 +97,7 @@ public class SearchShardsIT extends ESIntegTestCase {
|
|||
randomBoolean(),
|
||||
randomBoolean() ? null : randomAlphaOfLength(10)
|
||||
);
|
||||
SearchShardsResponse resp = client().execute(SearchShardsAction.INSTANCE, request).actionGet();
|
||||
SearchShardsResponse resp = client().execute(TransportSearchShardsAction.TYPE, request).actionGet();
|
||||
assertThat(resp.getGroups(), hasSize(indicesWithData + indicesWithoutData));
|
||||
for (SearchShardsGroup g : resp.getGroups()) {
|
||||
assertFalse(g.skipped());
|
||||
|
@ -137,7 +137,7 @@ public class SearchShardsIT extends ESIntegTestCase {
|
|||
randomBoolean(),
|
||||
randomBoolean() ? null : randomAlphaOfLength(10)
|
||||
);
|
||||
var searchShardsResponse = client().execute(SearchShardsAction.INSTANCE, searchShardsRequest).actionGet();
|
||||
var searchShardsResponse = client().execute(TransportSearchShardsAction.TYPE, searchShardsRequest).actionGet();
|
||||
|
||||
assertThat(searchShardsResponse.getGroups(), hasSize(searchResponse.getTotalShards()));
|
||||
long skippedShards = searchShardsResponse.getGroups().stream().filter(SearchShardsGroup::skipped).count();
|
||||
|
@ -182,7 +182,7 @@ public class SearchShardsIT extends ESIntegTestCase {
|
|||
randomBoolean(),
|
||||
null
|
||||
);
|
||||
SearchShardsResponse resp = client().execute(SearchShardsAction.INSTANCE, request).actionGet();
|
||||
SearchShardsResponse resp = client().execute(TransportSearchShardsAction.TYPE, request).actionGet();
|
||||
assertThat(resp.getGroups(), hasSize(totalShards));
|
||||
for (SearchShardsGroup group : resp.getGroups()) {
|
||||
assertFalse(group.skipped());
|
||||
|
|
|
@ -14,10 +14,10 @@ import org.elasticsearch.action.ActionResponse;
|
|||
import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsGroup;
|
||||
import org.elasticsearch.action.admin.indices.refresh.RefreshResponse;
|
||||
import org.elasticsearch.action.admin.indices.refresh.TransportUnpromotableShardRefreshAction;
|
||||
import org.elasticsearch.action.search.ClosePointInTimeAction;
|
||||
import org.elasticsearch.action.search.ClosePointInTimeRequest;
|
||||
import org.elasticsearch.action.search.OpenPointInTimeAction;
|
||||
import org.elasticsearch.action.search.OpenPointInTimeRequest;
|
||||
import org.elasticsearch.action.search.TransportClosePointInTimeAction;
|
||||
import org.elasticsearch.action.search.TransportOpenPointInTimeAction;
|
||||
import org.elasticsearch.cluster.ClusterChangedEvent;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.ClusterStateListener;
|
||||
|
@ -522,7 +522,7 @@ public class ShardRoutingRoleIT extends ESIntegTestCase {
|
|||
// do nothing
|
||||
}
|
||||
}
|
||||
String pitId = client().execute(OpenPointInTimeAction.INSTANCE, openRequest).actionGet().getPointInTimeId();
|
||||
String pitId = client().execute(TransportOpenPointInTimeAction.TYPE, openRequest).actionGet().getPointInTimeId();
|
||||
try {
|
||||
final var profileResults = prepareSearch().setPointInTime(new PointInTimeBuilder(pitId))
|
||||
.setProfile(true)
|
||||
|
@ -533,7 +533,7 @@ public class ShardRoutingRoleIT extends ESIntegTestCase {
|
|||
assertThat(profileKey, in(searchShardProfileKeys));
|
||||
}
|
||||
} finally {
|
||||
client().execute(ClosePointInTimeAction.INSTANCE, new ClosePointInTimeRequest(pitId));
|
||||
client().execute(TransportClosePointInTimeAction.TYPE, new ClosePointInTimeRequest(pitId));
|
||||
}
|
||||
}
|
||||
// search-shards API
|
||||
|
|
|
@ -11,17 +11,17 @@ package org.elasticsearch.search;
|
|||
import org.apache.lucene.tests.util.LuceneTestCase;
|
||||
import org.elasticsearch.ExceptionsHelper;
|
||||
import org.elasticsearch.action.ActionFuture;
|
||||
import org.elasticsearch.action.search.MultiSearchAction;
|
||||
import org.elasticsearch.action.search.MultiSearchResponse;
|
||||
import org.elasticsearch.action.search.SearchAction;
|
||||
import org.elasticsearch.action.search.SearchPhaseExecutionException;
|
||||
import org.elasticsearch.action.search.SearchResponse;
|
||||
import org.elasticsearch.action.search.SearchScrollAction;
|
||||
import org.elasticsearch.action.search.SearchShardTask;
|
||||
import org.elasticsearch.action.search.SearchTask;
|
||||
import org.elasticsearch.action.search.SearchTransportService;
|
||||
import org.elasticsearch.action.search.SearchType;
|
||||
import org.elasticsearch.action.search.ShardSearchFailure;
|
||||
import org.elasticsearch.action.search.TransportMultiSearchAction;
|
||||
import org.elasticsearch.action.search.TransportSearchAction;
|
||||
import org.elasticsearch.action.search.TransportSearchScrollAction;
|
||||
import org.elasticsearch.common.Strings;
|
||||
import org.elasticsearch.core.TimeValue;
|
||||
import org.elasticsearch.script.Script;
|
||||
|
@ -71,7 +71,7 @@ public class SearchCancellationIT extends AbstractSearchCancellationTestCase {
|
|||
).execute();
|
||||
|
||||
awaitForBlock(plugins);
|
||||
cancelSearch(SearchAction.NAME);
|
||||
cancelSearch(TransportSearchAction.TYPE.name());
|
||||
disableBlocks(plugins);
|
||||
logger.info("Segments {}", Strings.toString(indicesAdmin().prepareSegments("test").get()));
|
||||
ensureSearchWasCancelled(searchResponse);
|
||||
|
@ -89,7 +89,7 @@ public class SearchCancellationIT extends AbstractSearchCancellationTestCase {
|
|||
).execute();
|
||||
|
||||
awaitForBlock(plugins);
|
||||
cancelSearch(SearchAction.NAME);
|
||||
cancelSearch(TransportSearchAction.TYPE.name());
|
||||
disableBlocks(plugins);
|
||||
logger.info("Segments {}", Strings.toString(indicesAdmin().prepareSegments("test").get()));
|
||||
ensureSearchWasCancelled(searchResponse);
|
||||
|
@ -134,7 +134,7 @@ public class SearchCancellationIT extends AbstractSearchCancellationTestCase {
|
|||
)
|
||||
.execute();
|
||||
awaitForBlock(plugins);
|
||||
cancelSearch(SearchAction.NAME);
|
||||
cancelSearch(TransportSearchAction.TYPE.name());
|
||||
disableBlocks(plugins);
|
||||
ensureSearchWasCancelled(searchResponse);
|
||||
}
|
||||
|
@ -151,7 +151,7 @@ public class SearchCancellationIT extends AbstractSearchCancellationTestCase {
|
|||
.execute();
|
||||
|
||||
awaitForBlock(plugins);
|
||||
cancelSearch(SearchAction.NAME);
|
||||
cancelSearch(TransportSearchAction.TYPE.name());
|
||||
disableBlocks(plugins);
|
||||
SearchResponse response = ensureSearchWasCancelled(searchResponse);
|
||||
if (response != null) {
|
||||
|
@ -191,7 +191,7 @@ public class SearchCancellationIT extends AbstractSearchCancellationTestCase {
|
|||
.execute();
|
||||
|
||||
awaitForBlock(plugins);
|
||||
cancelSearch(SearchScrollAction.NAME);
|
||||
cancelSearch(TransportSearchScrollAction.TYPE.name());
|
||||
disableBlocks(plugins);
|
||||
|
||||
SearchResponse response = ensureSearchWasCancelled(scrollResponse);
|
||||
|
@ -215,7 +215,7 @@ public class SearchCancellationIT extends AbstractSearchCancellationTestCase {
|
|||
)
|
||||
.execute();
|
||||
awaitForBlock(plugins);
|
||||
cancelSearch(MultiSearchAction.NAME);
|
||||
cancelSearch(TransportMultiSearchAction.TYPE.name());
|
||||
disableBlocks(plugins);
|
||||
for (MultiSearchResponse.Item item : msearchResponse.actionGet()) {
|
||||
if (item.getFailure() != null) {
|
||||
|
@ -302,7 +302,7 @@ public class SearchCancellationIT extends AbstractSearchCancellationTestCase {
|
|||
for (String nodeName : internalCluster().getNodeNames()) {
|
||||
TransportService transportService = internalCluster().getInstance(TransportService.class, nodeName);
|
||||
for (Task task : transportService.getTaskManager().getCancellableTasks().values()) {
|
||||
if (task.getAction().equals(SearchAction.NAME)) {
|
||||
if (task.getAction().equals(TransportSearchAction.TYPE.name())) {
|
||||
tasks.add((SearchTask) task);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -13,13 +13,13 @@ import org.elasticsearch.action.ActionFuture;
|
|||
import org.elasticsearch.action.admin.cluster.node.tasks.cancel.CancelTasksRequest;
|
||||
import org.elasticsearch.action.admin.cluster.node.tasks.cancel.CancelTasksResponse;
|
||||
import org.elasticsearch.action.index.IndexRequest;
|
||||
import org.elasticsearch.action.search.SearchAction;
|
||||
import org.elasticsearch.action.search.SearchRequest;
|
||||
import org.elasticsearch.action.search.SearchResponse;
|
||||
import org.elasticsearch.action.search.SearchShardsAction;
|
||||
import org.elasticsearch.action.search.SearchShardsGroup;
|
||||
import org.elasticsearch.action.search.SearchShardsRequest;
|
||||
import org.elasticsearch.action.search.SearchShardsResponse;
|
||||
import org.elasticsearch.action.search.TransportSearchAction;
|
||||
import org.elasticsearch.action.search.TransportSearchShardsAction;
|
||||
import org.elasticsearch.action.support.IndicesOptions;
|
||||
import org.elasticsearch.action.support.PlainActionFuture;
|
||||
import org.elasticsearch.action.support.WriteRequest;
|
||||
|
@ -240,7 +240,7 @@ public class CrossClusterIT extends AbstractMultiClustersTestCase {
|
|||
final TaskInfo rootTask = client().admin()
|
||||
.cluster()
|
||||
.prepareListTasks()
|
||||
.setActions(SearchAction.INSTANCE.name())
|
||||
.setActions(TransportSearchAction.TYPE.name())
|
||||
.get()
|
||||
.getTasks()
|
||||
.stream()
|
||||
|
@ -274,7 +274,7 @@ public class CrossClusterIT extends AbstractMultiClustersTestCase {
|
|||
for (TransportService transportService : transportServices) {
|
||||
Collection<CancellableTask> cancellableTasks = transportService.getTaskManager().getCancellableTasks().values();
|
||||
for (CancellableTask cancellableTask : cancellableTasks) {
|
||||
if (cancellableTask.getAction().contains(SearchAction.INSTANCE.name())) {
|
||||
if (cancellableTask.getAction().contains(TransportSearchAction.TYPE.name())) {
|
||||
assertTrue(cancellableTask.getDescription(), cancellableTask.isCancelled());
|
||||
}
|
||||
}
|
||||
|
@ -528,7 +528,7 @@ public class CrossClusterIT extends AbstractMultiClustersTestCase {
|
|||
{
|
||||
QueryBuilder query = new TermQueryBuilder("_index", "cluster_a:my_index");
|
||||
SearchShardsRequest request = new SearchShardsRequest(indices, indicesOptions, query, null, null, randomBoolean(), "cluster_a");
|
||||
SearchShardsResponse resp = remoteClient.execute(SearchShardsAction.INSTANCE, request).actionGet();
|
||||
SearchShardsResponse resp = remoteClient.execute(TransportSearchShardsAction.TYPE, request).actionGet();
|
||||
assertThat(resp.getGroups(), hasSize(numShards));
|
||||
for (SearchShardsGroup group : resp.getGroups()) {
|
||||
assertFalse(group.skipped());
|
||||
|
@ -545,7 +545,7 @@ public class CrossClusterIT extends AbstractMultiClustersTestCase {
|
|||
randomBoolean(),
|
||||
randomFrom("cluster_b", null)
|
||||
);
|
||||
SearchShardsResponse resp = remoteClient.execute(SearchShardsAction.INSTANCE, request).actionGet();
|
||||
SearchShardsResponse resp = remoteClient.execute(TransportSearchShardsAction.TYPE, request).actionGet();
|
||||
assertThat(resp.getGroups(), hasSize(numShards));
|
||||
for (SearchShardsGroup group : resp.getGroups()) {
|
||||
assertTrue(group.skipped());
|
||||
|
@ -562,7 +562,7 @@ public class CrossClusterIT extends AbstractMultiClustersTestCase {
|
|||
randomBoolean(),
|
||||
randomFrom("cluster_a", "cluster_b", null)
|
||||
);
|
||||
SearchShardsResponse resp = remoteClient.execute(SearchShardsAction.INSTANCE, request).actionGet();
|
||||
SearchShardsResponse resp = remoteClient.execute(TransportSearchShardsAction.TYPE, request).actionGet();
|
||||
assertThat(resp.getGroups(), hasSize(numShards));
|
||||
for (SearchShardsGroup group : resp.getGroups()) {
|
||||
assertTrue(group.skipped());
|
||||
|
|
|
@ -12,15 +12,15 @@ import org.elasticsearch.action.admin.indices.create.CreateIndexRequestBuilder;
|
|||
import org.elasticsearch.action.bulk.BulkRequestBuilder;
|
||||
import org.elasticsearch.action.index.IndexRequest;
|
||||
import org.elasticsearch.action.index.IndexRequestBuilder;
|
||||
import org.elasticsearch.action.search.ClosePointInTimeAction;
|
||||
import org.elasticsearch.action.search.ClosePointInTimeRequest;
|
||||
import org.elasticsearch.action.search.OpenPointInTimeAction;
|
||||
import org.elasticsearch.action.search.OpenPointInTimeRequest;
|
||||
import org.elasticsearch.action.search.SearchPhaseExecutionException;
|
||||
import org.elasticsearch.action.search.SearchRequest;
|
||||
import org.elasticsearch.action.search.SearchRequestBuilder;
|
||||
import org.elasticsearch.action.search.SearchResponse;
|
||||
import org.elasticsearch.action.search.ShardSearchFailure;
|
||||
import org.elasticsearch.action.search.TransportClosePointInTimeAction;
|
||||
import org.elasticsearch.action.search.TransportOpenPointInTimeAction;
|
||||
import org.elasticsearch.action.support.WriteRequest;
|
||||
import org.elasticsearch.cluster.metadata.IndexMetadata;
|
||||
import org.elasticsearch.common.Randomness;
|
||||
|
@ -456,7 +456,7 @@ public class SearchAfterIT extends ESIntegTestCase {
|
|||
String pitID;
|
||||
{
|
||||
OpenPointInTimeRequest openPITRequest = new OpenPointInTimeRequest("test").keepAlive(TimeValue.timeValueMinutes(5));
|
||||
pitID = client().execute(OpenPointInTimeAction.INSTANCE, openPITRequest).actionGet().getPointInTimeId();
|
||||
pitID = client().execute(TransportOpenPointInTimeAction.TYPE, openPITRequest).actionGet().getPointInTimeId();
|
||||
SearchRequest searchRequest = new SearchRequest("test").source(
|
||||
new SearchSourceBuilder().pointInTimeBuilder(new PointInTimeBuilder(pitID).setKeepAlive(TimeValue.timeValueMinutes(5)))
|
||||
.sort("timestamp")
|
||||
|
@ -483,14 +483,14 @@ public class SearchAfterIT extends ESIntegTestCase {
|
|||
} while (resp.getHits().getHits().length > 0);
|
||||
assertThat(foundHits, equalTo(timestamps.size()));
|
||||
} finally {
|
||||
client().execute(ClosePointInTimeAction.INSTANCE, new ClosePointInTimeRequest(pitID)).actionGet();
|
||||
client().execute(TransportClosePointInTimeAction.TYPE, new ClosePointInTimeRequest(pitID)).actionGet();
|
||||
}
|
||||
}
|
||||
|
||||
// search_after without sort with point in time
|
||||
{
|
||||
OpenPointInTimeRequest openPITRequest = new OpenPointInTimeRequest("test").keepAlive(TimeValue.timeValueMinutes(5));
|
||||
pitID = client().execute(OpenPointInTimeAction.INSTANCE, openPITRequest).actionGet().getPointInTimeId();
|
||||
pitID = client().execute(TransportOpenPointInTimeAction.TYPE, openPITRequest).actionGet().getPointInTimeId();
|
||||
SearchRequest searchRequest = new SearchRequest("test").source(
|
||||
new SearchSourceBuilder().pointInTimeBuilder(new PointInTimeBuilder(pitID).setKeepAlive(TimeValue.timeValueMinutes(5)))
|
||||
.sort(SortBuilders.pitTiebreaker())
|
||||
|
@ -517,7 +517,7 @@ public class SearchAfterIT extends ESIntegTestCase {
|
|||
Collections.sort(foundSeqNos);
|
||||
assertThat(foundSeqNos, equalTo(timestamps));
|
||||
} finally {
|
||||
client().execute(ClosePointInTimeAction.INSTANCE, new ClosePointInTimeRequest(pitID)).actionGet();
|
||||
client().execute(TransportClosePointInTimeAction.TYPE, new ClosePointInTimeRequest(pitID)).actionGet();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -10,14 +10,14 @@ package org.elasticsearch.search.slice;
|
|||
|
||||
import org.elasticsearch.action.admin.indices.alias.IndicesAliasesRequest;
|
||||
import org.elasticsearch.action.index.IndexRequestBuilder;
|
||||
import org.elasticsearch.action.search.ClosePointInTimeAction;
|
||||
import org.elasticsearch.action.search.ClosePointInTimeRequest;
|
||||
import org.elasticsearch.action.search.OpenPointInTimeAction;
|
||||
import org.elasticsearch.action.search.OpenPointInTimeRequest;
|
||||
import org.elasticsearch.action.search.OpenPointInTimeResponse;
|
||||
import org.elasticsearch.action.search.SearchPhaseExecutionException;
|
||||
import org.elasticsearch.action.search.SearchRequestBuilder;
|
||||
import org.elasticsearch.action.search.SearchResponse;
|
||||
import org.elasticsearch.action.search.TransportClosePointInTimeAction;
|
||||
import org.elasticsearch.action.search.TransportOpenPointInTimeAction;
|
||||
import org.elasticsearch.common.Strings;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.core.TimeValue;
|
||||
|
@ -197,7 +197,7 @@ public class SearchSliceIT extends ESIntegTestCase {
|
|||
for (String field : new String[] { null, "random_int", "static_int" }) {
|
||||
// Open point-in-time reader
|
||||
OpenPointInTimeRequest request = new OpenPointInTimeRequest("test").keepAlive(TimeValue.timeValueSeconds(10));
|
||||
OpenPointInTimeResponse response = client().execute(OpenPointInTimeAction.INSTANCE, request).actionGet();
|
||||
OpenPointInTimeResponse response = client().execute(TransportOpenPointInTimeAction.TYPE, request).actionGet();
|
||||
String pointInTimeId = response.getPointInTimeId();
|
||||
|
||||
// Test sort on document IDs
|
||||
|
@ -206,7 +206,7 @@ public class SearchSliceIT extends ESIntegTestCase {
|
|||
assertSearchSlicesWithPointInTime(field, "random_int", pointInTimeId, max, numDocs);
|
||||
|
||||
// Close point-in-time reader
|
||||
client().execute(ClosePointInTimeAction.INSTANCE, new ClosePointInTimeRequest(pointInTimeId)).actionGet();
|
||||
client().execute(TransportClosePointInTimeAction.TYPE, new ClosePointInTimeRequest(pointInTimeId)).actionGet();
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -208,7 +208,6 @@ import org.elasticsearch.action.bulk.TransportShardBulkAction;
|
|||
import org.elasticsearch.action.bulk.TransportSimulateBulkAction;
|
||||
import org.elasticsearch.action.delete.DeleteAction;
|
||||
import org.elasticsearch.action.delete.TransportDeleteAction;
|
||||
import org.elasticsearch.action.explain.ExplainAction;
|
||||
import org.elasticsearch.action.explain.TransportExplainAction;
|
||||
import org.elasticsearch.action.fieldcaps.FieldCapabilitiesAction;
|
||||
import org.elasticsearch.action.fieldcaps.TransportFieldCapabilitiesAction;
|
||||
|
@ -227,15 +226,8 @@ import org.elasticsearch.action.ingest.PutPipelineAction;
|
|||
import org.elasticsearch.action.ingest.PutPipelineTransportAction;
|
||||
import org.elasticsearch.action.ingest.SimulatePipelineAction;
|
||||
import org.elasticsearch.action.ingest.SimulatePipelineTransportAction;
|
||||
import org.elasticsearch.action.search.ClearScrollAction;
|
||||
import org.elasticsearch.action.search.ClosePointInTimeAction;
|
||||
import org.elasticsearch.action.search.MultiSearchAction;
|
||||
import org.elasticsearch.action.search.OpenPointInTimeAction;
|
||||
import org.elasticsearch.action.search.RestClosePointInTimeAction;
|
||||
import org.elasticsearch.action.search.RestOpenPointInTimeAction;
|
||||
import org.elasticsearch.action.search.SearchAction;
|
||||
import org.elasticsearch.action.search.SearchScrollAction;
|
||||
import org.elasticsearch.action.search.SearchShardsAction;
|
||||
import org.elasticsearch.action.search.TransportClearScrollAction;
|
||||
import org.elasticsearch.action.search.TransportClosePointInTimeAction;
|
||||
import org.elasticsearch.action.search.TransportMultiSearchAction;
|
||||
|
@ -763,14 +755,14 @@ public class ActionModule extends AbstractModule {
|
|||
actions.register(BulkAction.INSTANCE, TransportBulkAction.class);
|
||||
actions.register(SimulateBulkAction.INSTANCE, TransportSimulateBulkAction.class);
|
||||
actions.register(TransportShardBulkAction.TYPE, TransportShardBulkAction.class);
|
||||
actions.register(SearchAction.INSTANCE, TransportSearchAction.class);
|
||||
actions.register(SearchScrollAction.INSTANCE, TransportSearchScrollAction.class);
|
||||
actions.register(OpenPointInTimeAction.INSTANCE, TransportOpenPointInTimeAction.class);
|
||||
actions.register(ClosePointInTimeAction.INSTANCE, TransportClosePointInTimeAction.class);
|
||||
actions.register(SearchShardsAction.INSTANCE, TransportSearchShardsAction.class);
|
||||
actions.register(MultiSearchAction.INSTANCE, TransportMultiSearchAction.class);
|
||||
actions.register(ExplainAction.INSTANCE, TransportExplainAction.class);
|
||||
actions.register(ClearScrollAction.INSTANCE, TransportClearScrollAction.class);
|
||||
actions.register(TransportSearchAction.TYPE, TransportSearchAction.class);
|
||||
actions.register(TransportSearchScrollAction.TYPE, TransportSearchScrollAction.class);
|
||||
actions.register(TransportOpenPointInTimeAction.TYPE, TransportOpenPointInTimeAction.class);
|
||||
actions.register(TransportClosePointInTimeAction.TYPE, TransportClosePointInTimeAction.class);
|
||||
actions.register(TransportSearchShardsAction.TYPE, TransportSearchShardsAction.class);
|
||||
actions.register(TransportMultiSearchAction.TYPE, TransportMultiSearchAction.class);
|
||||
actions.register(TransportExplainAction.TYPE, TransportExplainAction.class);
|
||||
actions.register(TransportClearScrollAction.TYPE, TransportClearScrollAction.class);
|
||||
actions.register(RecoveryAction.INSTANCE, TransportRecoveryAction.class);
|
||||
actions.register(NodesReloadSecureSettingsAction.INSTANCE, TransportNodesReloadSecureSettingsAction.class);
|
||||
actions.register(AutoCreateAction.INSTANCE, AutoCreateAction.TransportAction.class);
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
|
@ -8,6 +8,7 @@
|
|||
|
||||
package org.elasticsearch.action.explain;
|
||||
|
||||
import org.elasticsearch.action.ActionType;
|
||||
import org.elasticsearch.action.support.single.shard.SingleShardOperationRequestBuilder;
|
||||
import org.elasticsearch.client.internal.ElasticsearchClient;
|
||||
import org.elasticsearch.common.Strings;
|
||||
|
@ -20,11 +21,7 @@ import org.elasticsearch.search.fetch.subphase.FetchSourceContext;
|
|||
*/
|
||||
public class ExplainRequestBuilder extends SingleShardOperationRequestBuilder<ExplainRequest, ExplainResponse, ExplainRequestBuilder> {
|
||||
|
||||
ExplainRequestBuilder(ElasticsearchClient client, ExplainAction action) {
|
||||
super(client, action, new ExplainRequest());
|
||||
}
|
||||
|
||||
public ExplainRequestBuilder(ElasticsearchClient client, ExplainAction action, String index, String id) {
|
||||
public ExplainRequestBuilder(ElasticsearchClient client, ActionType<ExplainResponse> action, String index, String id) {
|
||||
super(client, action, new ExplainRequest().index(index).id(id));
|
||||
}
|
||||
|
||||
|
|
|
@ -11,6 +11,7 @@ package org.elasticsearch.action.explain;
|
|||
import org.apache.lucene.search.Explanation;
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.ActionType;
|
||||
import org.elasticsearch.action.support.ActionFilters;
|
||||
import org.elasticsearch.action.support.single.shard.TransportSingleShardAction;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
|
@ -48,6 +49,7 @@ import java.util.function.LongSupplier;
|
|||
// TODO: AggregatedDfs. Currently the idf can be different then when executing a normal search with explain.
|
||||
public class TransportExplainAction extends TransportSingleShardAction<ExplainRequest, ExplainResponse> {
|
||||
|
||||
public static final ActionType<ExplainResponse> TYPE = new ActionType<>("indices:data/read/explain", ExplainResponse::new);
|
||||
private final SearchService searchService;
|
||||
|
||||
@Inject
|
||||
|
@ -60,7 +62,7 @@ public class TransportExplainAction extends TransportSingleShardAction<ExplainRe
|
|||
IndexNameExpressionResolver indexNameExpressionResolver
|
||||
) {
|
||||
super(
|
||||
ExplainAction.NAME,
|
||||
TYPE.name(),
|
||||
threadPool,
|
||||
clusterService,
|
||||
transportService,
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -9,13 +9,14 @@
|
|||
package org.elasticsearch.action.search;
|
||||
|
||||
import org.elasticsearch.action.ActionRequestBuilder;
|
||||
import org.elasticsearch.action.ActionType;
|
||||
import org.elasticsearch.client.internal.ElasticsearchClient;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
public class ClearScrollRequestBuilder extends ActionRequestBuilder<ClearScrollRequest, ClearScrollResponse> {
|
||||
|
||||
public ClearScrollRequestBuilder(ElasticsearchClient client, ClearScrollAction action) {
|
||||
public ClearScrollRequestBuilder(ElasticsearchClient client, ActionType<ClearScrollResponse> action) {
|
||||
super(client, action, new ClearScrollRequest());
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -9,6 +9,7 @@
|
|||
package org.elasticsearch.action.search;
|
||||
|
||||
import org.elasticsearch.action.ActionRequestBuilder;
|
||||
import org.elasticsearch.action.ActionType;
|
||||
import org.elasticsearch.action.support.IndicesOptions;
|
||||
import org.elasticsearch.client.internal.ElasticsearchClient;
|
||||
|
||||
|
@ -17,7 +18,7 @@ import org.elasticsearch.client.internal.ElasticsearchClient;
|
|||
*/
|
||||
public class MultiSearchRequestBuilder extends ActionRequestBuilder<MultiSearchRequest, MultiSearchResponse> {
|
||||
|
||||
public MultiSearchRequestBuilder(ElasticsearchClient client, MultiSearchAction action) {
|
||||
public MultiSearchRequestBuilder(ElasticsearchClient client, ActionType<MultiSearchResponse> action) {
|
||||
super(client, action, new MultiSearchRequest());
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -41,7 +41,7 @@ public class RestClosePointInTimeAction extends BaseRestHandler {
|
|||
clearRequest = ClosePointInTimeRequest.fromXContent(parser);
|
||||
}
|
||||
return channel -> client.execute(
|
||||
ClosePointInTimeAction.INSTANCE,
|
||||
TransportClosePointInTimeAction.TYPE,
|
||||
clearRequest,
|
||||
new RestToXContentListener<>(channel, ClosePointInTimeResponse::status)
|
||||
);
|
||||
|
|
|
@ -54,13 +54,14 @@ public class RestOpenPointInTimeAction extends BaseRestHandler {
|
|||
);
|
||||
openRequest.maxConcurrentShardRequests(maxConcurrentShardRequests);
|
||||
}
|
||||
|
||||
request.withContentOrSourceParamParserOrNull(parser -> {
|
||||
if (parser != null) {
|
||||
PARSER.parse(parser, openRequest, null);
|
||||
}
|
||||
});
|
||||
|
||||
return channel -> client.execute(OpenPointInTimeAction.INSTANCE, openRequest, new RestToXContentListener<>(channel));
|
||||
return channel -> client.execute(TransportOpenPointInTimeAction.TYPE, openRequest, new RestToXContentListener<>(channel));
|
||||
}
|
||||
|
||||
private static final ObjectParser<OpenPointInTimeRequest, Void> PARSER = new ObjectParser<>("open_point_in_time_request");
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
|
@ -11,7 +11,7 @@ package org.elasticsearch.action.search;
|
|||
import org.elasticsearch.action.ActionListener;
|
||||
|
||||
/**
|
||||
* An {@link ActionListener} for search requests that allows to track progress of the {@link SearchAction}.
|
||||
* An {@link ActionListener} for search requests that allows to track progress of the {@link TransportSearchAction}.
|
||||
* See {@link SearchProgressListener}.
|
||||
*/
|
||||
public abstract class SearchProgressActionListener extends SearchProgressListener implements ActionListener<SearchResponse> {}
|
||||
|
|
|
@ -23,7 +23,7 @@ import java.util.Objects;
|
|||
import java.util.stream.StreamSupport;
|
||||
|
||||
/**
|
||||
* A listener that allows to track progress of the {@link SearchAction}.
|
||||
* A listener that allows to track progress of the {@link TransportSearchAction}.
|
||||
*/
|
||||
public abstract class SearchProgressListener {
|
||||
private static final Logger logger = LogManager.getLogger(SearchProgressListener.class);
|
||||
|
|
|
@ -9,6 +9,7 @@
|
|||
package org.elasticsearch.action.search;
|
||||
|
||||
import org.elasticsearch.action.ActionRequestBuilder;
|
||||
import org.elasticsearch.action.ActionType;
|
||||
import org.elasticsearch.action.support.IndicesOptions;
|
||||
import org.elasticsearch.client.internal.ElasticsearchClient;
|
||||
import org.elasticsearch.core.Nullable;
|
||||
|
@ -41,7 +42,7 @@ import java.util.Map;
|
|||
*/
|
||||
public class SearchRequestBuilder extends ActionRequestBuilder<SearchRequest, SearchResponse> {
|
||||
|
||||
public SearchRequestBuilder(ElasticsearchClient client, SearchAction action) {
|
||||
public SearchRequestBuilder(ElasticsearchClient client, ActionType<SearchResponse> action) {
|
||||
super(client, action, new SearchRequest());
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
|
@ -9,6 +9,7 @@
|
|||
package org.elasticsearch.action.search;
|
||||
|
||||
import org.elasticsearch.action.ActionRequestBuilder;
|
||||
import org.elasticsearch.action.ActionType;
|
||||
import org.elasticsearch.client.internal.ElasticsearchClient;
|
||||
import org.elasticsearch.core.TimeValue;
|
||||
import org.elasticsearch.search.Scroll;
|
||||
|
@ -18,11 +19,11 @@ import org.elasticsearch.search.Scroll;
|
|||
*/
|
||||
public class SearchScrollRequestBuilder extends ActionRequestBuilder<SearchScrollRequest, SearchResponse> {
|
||||
|
||||
public SearchScrollRequestBuilder(ElasticsearchClient client, SearchScrollAction action) {
|
||||
public SearchScrollRequestBuilder(ElasticsearchClient client, ActionType<SearchResponse> action) {
|
||||
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));
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -280,7 +280,7 @@ public class SearchTransportService {
|
|||
final Transport.Connection connection = transportService.getConnection(transportService.getLocalNode());
|
||||
transportService.sendChildRequest(
|
||||
connection,
|
||||
MultiSearchAction.NAME,
|
||||
TransportMultiSearchAction.TYPE.name(),
|
||||
request,
|
||||
task,
|
||||
new ConnectionCountingHandler<>(listener, MultiSearchResponse::new, clientConnections, connection.getNode().getId())
|
||||
|
|
|
@ -9,6 +9,7 @@
|
|||
package org.elasticsearch.action.search;
|
||||
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.ActionType;
|
||||
import org.elasticsearch.action.support.ActionFilters;
|
||||
import org.elasticsearch.action.support.HandledTransportAction;
|
||||
import org.elasticsearch.cluster.service.ClusterService;
|
||||
|
@ -19,6 +20,9 @@ import org.elasticsearch.transport.TransportService;
|
|||
|
||||
public class TransportClearScrollAction extends HandledTransportAction<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 SearchTransportService searchTransportService;
|
||||
|
||||
|
@ -29,7 +33,7 @@ public class TransportClearScrollAction extends HandledTransportAction<ClearScro
|
|||
ActionFilters actionFilters,
|
||||
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.searchTransportService = searchTransportService;
|
||||
}
|
||||
|
|
|
@ -9,6 +9,7 @@
|
|||
package org.elasticsearch.action.search;
|
||||
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.ActionType;
|
||||
import org.elasticsearch.action.support.ActionFilters;
|
||||
import org.elasticsearch.action.support.HandledTransportAction;
|
||||
import org.elasticsearch.cluster.service.ClusterService;
|
||||
|
@ -22,6 +23,10 @@ import java.util.Collection;
|
|||
|
||||
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 SearchTransportService searchTransportService;
|
||||
private final NamedWriteableRegistry namedWriteableRegistry;
|
||||
|
@ -34,13 +39,7 @@ public class TransportClosePointInTimeAction extends HandledTransportAction<Clos
|
|||
SearchTransportService searchTransportService,
|
||||
NamedWriteableRegistry namedWriteableRegistry
|
||||
) {
|
||||
super(
|
||||
ClosePointInTimeAction.NAME,
|
||||
transportService,
|
||||
actionFilters,
|
||||
ClosePointInTimeRequest::new,
|
||||
EsExecutors.DIRECT_EXECUTOR_SERVICE
|
||||
);
|
||||
super(TYPE.name(), transportService, actionFilters, ClosePointInTimeRequest::new, EsExecutors.DIRECT_EXECUTOR_SERVICE);
|
||||
this.clusterService = clusterService;
|
||||
this.searchTransportService = searchTransportService;
|
||||
this.namedWriteableRegistry = namedWriteableRegistry;
|
||||
|
|
|
@ -12,6 +12,7 @@ import org.apache.logging.log4j.LogManager;
|
|||
import org.apache.logging.log4j.Logger;
|
||||
import org.elasticsearch.ExceptionsHelper;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.ActionType;
|
||||
import org.elasticsearch.action.support.ActionFilters;
|
||||
import org.elasticsearch.action.support.HandledTransportAction;
|
||||
import org.elasticsearch.client.internal.node.NodeClient;
|
||||
|
@ -34,6 +35,8 @@ import java.util.function.LongSupplier;
|
|||
|
||||
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 final int allocatedProcessors;
|
||||
private final ThreadPool threadPool;
|
||||
|
@ -50,7 +53,7 @@ public class TransportMultiSearchAction extends HandledTransportAction<MultiSear
|
|||
ActionFilters actionFilters,
|
||||
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.clusterService = clusterService;
|
||||
this.allocatedProcessors = EsExecutors.allocatedProcessors(settings);
|
||||
|
@ -67,7 +70,7 @@ public class TransportMultiSearchAction extends HandledTransportAction<MultiSear
|
|||
LongSupplier relativeTimeProvider,
|
||||
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.clusterService = clusterService;
|
||||
this.allocatedProcessors = allocatedProcessors;
|
||||
|
|
|
@ -12,6 +12,7 @@ import org.apache.logging.log4j.LogManager;
|
|||
import org.apache.logging.log4j.Logger;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.ActionListenerResponseHandler;
|
||||
import org.elasticsearch.action.ActionType;
|
||||
import org.elasticsearch.action.IndicesRequest;
|
||||
import org.elasticsearch.action.OriginalIndices;
|
||||
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);
|
||||
|
||||
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 SearchTransportService searchTransportService;
|
||||
|
@ -68,13 +73,7 @@ public class TransportOpenPointInTimeAction extends HandledTransportAction<OpenP
|
|||
TransportSearchAction transportSearchAction,
|
||||
SearchTransportService searchTransportService
|
||||
) {
|
||||
super(
|
||||
OpenPointInTimeAction.NAME,
|
||||
transportService,
|
||||
actionFilters,
|
||||
OpenPointInTimeRequest::new,
|
||||
EsExecutors.DIRECT_EXECUTOR_SERVICE
|
||||
);
|
||||
super(TYPE.name(), transportService, actionFilters, OpenPointInTimeRequest::new, EsExecutors.DIRECT_EXECUTOR_SERVICE);
|
||||
this.transportService = transportService;
|
||||
this.transportSearchAction = transportSearchAction;
|
||||
this.searchService = searchService;
|
||||
|
|
|
@ -14,6 +14,7 @@ import org.elasticsearch.ExceptionsHelper;
|
|||
import org.elasticsearch.TransportVersions;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.ActionListenerResponseHandler;
|
||||
import org.elasticsearch.action.ActionType;
|
||||
import org.elasticsearch.action.IndicesRequest;
|
||||
import org.elasticsearch.action.OriginalIndices;
|
||||
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 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 DeprecationLogger DEPRECATION_LOGGER = DeprecationLogger.getLogger(TransportSearchAction.class);
|
||||
public static final String FROZEN_INDICES_DEPRECATION_MESSAGE = "Searching frozen indices [{}] is deprecated."
|
||||
|
@ -160,7 +163,7 @@ public class TransportSearchAction extends HandledTransportAction<SearchRequest,
|
|||
NamedWriteableRegistry namedWriteableRegistry,
|
||||
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.circuitBreaker = circuitBreakerService.getBreaker(CircuitBreaker.REQUEST);
|
||||
this.searchPhaseController = searchPhaseController;
|
||||
|
@ -722,7 +725,7 @@ public class TransportSearchAction extends HandledTransportAction<SearchRequest,
|
|||
);
|
||||
transportService.sendRequest(
|
||||
connection,
|
||||
SearchShardsAction.NAME,
|
||||
TransportSearchShardsAction.TYPE.name(),
|
||||
searchShardsRequest,
|
||||
TransportRequestOptions.EMPTY,
|
||||
new ActionListenerResponseHandler<>(singleListener, SearchShardsResponse::new, responseExecutor)
|
||||
|
|
|
@ -12,6 +12,7 @@ import org.apache.logging.log4j.LogManager;
|
|||
import org.apache.logging.log4j.Logger;
|
||||
import org.elasticsearch.ExceptionsHelper;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.ActionType;
|
||||
import org.elasticsearch.action.ShardOperationFailedException;
|
||||
import org.elasticsearch.action.support.ActionFilters;
|
||||
import org.elasticsearch.action.support.HandledTransportAction;
|
||||
|
@ -26,6 +27,7 @@ import static org.elasticsearch.action.search.ParsedScrollId.QUERY_THEN_FETCH_TY
|
|||
import static org.elasticsearch.action.search.TransportSearchHelper.parseScrollId;
|
||||
|
||||
public class TransportSearchScrollAction extends HandledTransportAction<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 final ClusterService clusterService;
|
||||
private final SearchTransportService searchTransportService;
|
||||
|
@ -37,7 +39,7 @@ public class TransportSearchScrollAction extends HandledTransportAction<SearchSc
|
|||
ActionFilters actionFilters,
|
||||
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.searchTransportService = searchTransportService;
|
||||
}
|
||||
|
|
|
@ -9,6 +9,7 @@
|
|||
package org.elasticsearch.action.search;
|
||||
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.ActionType;
|
||||
import org.elasticsearch.action.OriginalIndices;
|
||||
import org.elasticsearch.action.support.ActionFilters;
|
||||
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.
|
||||
*/
|
||||
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 TransportSearchAction transportSearchAction;
|
||||
private final SearchService searchService;
|
||||
|
@ -61,7 +65,7 @@ public class TransportSearchShardsAction extends HandledTransportAction<SearchSh
|
|||
IndexNameExpressionResolver indexNameExpressionResolver
|
||||
) {
|
||||
super(
|
||||
SearchShardsAction.NAME,
|
||||
TYPE.name(),
|
||||
transportService,
|
||||
actionFilters,
|
||||
SearchShardsRequest::new,
|
||||
|
|
|
@ -247,10 +247,10 @@ import org.elasticsearch.action.delete.DeleteAction;
|
|||
import org.elasticsearch.action.delete.DeleteRequest;
|
||||
import org.elasticsearch.action.delete.DeleteRequestBuilder;
|
||||
import org.elasticsearch.action.delete.DeleteResponse;
|
||||
import org.elasticsearch.action.explain.ExplainAction;
|
||||
import org.elasticsearch.action.explain.ExplainRequest;
|
||||
import org.elasticsearch.action.explain.ExplainRequestBuilder;
|
||||
import org.elasticsearch.action.explain.ExplainResponse;
|
||||
import org.elasticsearch.action.explain.TransportExplainAction;
|
||||
import org.elasticsearch.action.fieldcaps.FieldCapabilitiesAction;
|
||||
import org.elasticsearch.action.fieldcaps.FieldCapabilitiesRequest;
|
||||
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.SimulatePipelineRequestBuilder;
|
||||
import org.elasticsearch.action.ingest.SimulatePipelineResponse;
|
||||
import org.elasticsearch.action.search.ClearScrollAction;
|
||||
import org.elasticsearch.action.search.ClearScrollRequest;
|
||||
import org.elasticsearch.action.search.ClearScrollRequestBuilder;
|
||||
import org.elasticsearch.action.search.ClearScrollResponse;
|
||||
import org.elasticsearch.action.search.MultiSearchAction;
|
||||
import org.elasticsearch.action.search.MultiSearchRequest;
|
||||
import org.elasticsearch.action.search.MultiSearchRequestBuilder;
|
||||
import org.elasticsearch.action.search.MultiSearchResponse;
|
||||
import org.elasticsearch.action.search.SearchAction;
|
||||
import org.elasticsearch.action.search.SearchRequest;
|
||||
import org.elasticsearch.action.search.SearchRequestBuilder;
|
||||
import org.elasticsearch.action.search.SearchResponse;
|
||||
import org.elasticsearch.action.search.SearchScrollAction;
|
||||
import org.elasticsearch.action.search.SearchScrollRequest;
|
||||
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.master.AcknowledgedResponse;
|
||||
import org.elasticsearch.action.termvectors.MultiTermVectorsAction;
|
||||
|
@ -509,47 +509,47 @@ public abstract class AbstractClient implements Client {
|
|||
|
||||
@Override
|
||||
public ActionFuture<SearchResponse> search(final SearchRequest request) {
|
||||
return execute(SearchAction.INSTANCE, request);
|
||||
return execute(TransportSearchAction.TYPE, request);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void search(final SearchRequest request, final ActionListener<SearchResponse> listener) {
|
||||
execute(SearchAction.INSTANCE, request, listener);
|
||||
execute(TransportSearchAction.TYPE, request, listener);
|
||||
}
|
||||
|
||||
@Override
|
||||
public SearchRequestBuilder prepareSearch(String... indices) {
|
||||
return new SearchRequestBuilder(this, SearchAction.INSTANCE).setIndices(indices);
|
||||
return new SearchRequestBuilder(this, TransportSearchAction.TYPE).setIndices(indices);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ActionFuture<SearchResponse> searchScroll(final SearchScrollRequest request) {
|
||||
return execute(SearchScrollAction.INSTANCE, request);
|
||||
return execute(TransportSearchScrollAction.TYPE, request);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void searchScroll(final SearchScrollRequest request, final ActionListener<SearchResponse> listener) {
|
||||
execute(SearchScrollAction.INSTANCE, request, listener);
|
||||
execute(TransportSearchScrollAction.TYPE, request, listener);
|
||||
}
|
||||
|
||||
@Override
|
||||
public SearchScrollRequestBuilder prepareSearchScroll(String scrollId) {
|
||||
return new SearchScrollRequestBuilder(this, SearchScrollAction.INSTANCE, scrollId);
|
||||
return new SearchScrollRequestBuilder(this, TransportSearchScrollAction.TYPE, scrollId);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ActionFuture<MultiSearchResponse> multiSearch(MultiSearchRequest request) {
|
||||
return execute(MultiSearchAction.INSTANCE, request);
|
||||
return execute(TransportMultiSearchAction.TYPE, request);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void multiSearch(MultiSearchRequest request, ActionListener<MultiSearchResponse> listener) {
|
||||
execute(MultiSearchAction.INSTANCE, request, listener);
|
||||
execute(TransportMultiSearchAction.TYPE, request, listener);
|
||||
}
|
||||
|
||||
@Override
|
||||
public MultiSearchRequestBuilder prepareMultiSearch() {
|
||||
return new MultiSearchRequestBuilder(this, MultiSearchAction.INSTANCE);
|
||||
return new MultiSearchRequestBuilder(this, TransportMultiSearchAction.TYPE);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -589,32 +589,32 @@ public abstract class AbstractClient implements Client {
|
|||
|
||||
@Override
|
||||
public ExplainRequestBuilder prepareExplain(String index, String id) {
|
||||
return new ExplainRequestBuilder(this, ExplainAction.INSTANCE, index, id);
|
||||
return new ExplainRequestBuilder(this, TransportExplainAction.TYPE, index, id);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ActionFuture<ExplainResponse> explain(ExplainRequest request) {
|
||||
return execute(ExplainAction.INSTANCE, request);
|
||||
return execute(TransportExplainAction.TYPE, request);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void explain(ExplainRequest request, ActionListener<ExplainResponse> listener) {
|
||||
execute(ExplainAction.INSTANCE, request, listener);
|
||||
execute(TransportExplainAction.TYPE, request, listener);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void clearScroll(ClearScrollRequest request, ActionListener<ClearScrollResponse> listener) {
|
||||
execute(ClearScrollAction.INSTANCE, request, listener);
|
||||
execute(TransportClearScrollAction.TYPE, request, listener);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ActionFuture<ClearScrollResponse> clearScroll(ClearScrollRequest request) {
|
||||
return execute(ClearScrollAction.INSTANCE, request);
|
||||
return execute(TransportClearScrollAction.TYPE, request);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ClearScrollRequestBuilder prepareClearScroll() {
|
||||
return new ClearScrollRequestBuilder(this, ClearScrollAction.INSTANCE);
|
||||
return new ClearScrollRequestBuilder(this, TransportClearScrollAction.TYPE);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -9,14 +9,14 @@
|
|||
package org.elasticsearch.index.reindex;
|
||||
|
||||
import org.elasticsearch.action.ActionType;
|
||||
import org.elasticsearch.action.search.SearchAction;
|
||||
import org.elasticsearch.action.search.SearchRequestBuilder;
|
||||
import org.elasticsearch.action.search.TransportSearchAction;
|
||||
import org.elasticsearch.client.internal.ElasticsearchClient;
|
||||
|
||||
public class DeleteByQueryRequestBuilder extends AbstractBulkByScrollRequestBuilder<DeleteByQueryRequest, DeleteByQueryRequestBuilder> {
|
||||
|
||||
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) {
|
||||
|
|
|
@ -11,8 +11,8 @@ package org.elasticsearch.index.reindex;
|
|||
import org.elasticsearch.action.ActionType;
|
||||
import org.elasticsearch.action.index.IndexAction;
|
||||
import org.elasticsearch.action.index.IndexRequestBuilder;
|
||||
import org.elasticsearch.action.search.SearchAction;
|
||||
import org.elasticsearch.action.search.SearchRequestBuilder;
|
||||
import org.elasticsearch.action.search.TransportSearchAction;
|
||||
import org.elasticsearch.client.internal.ElasticsearchClient;
|
||||
|
||||
public class ReindexRequestBuilder extends AbstractBulkIndexByScrollRequestBuilder<ReindexRequest, ReindexRequestBuilder> {
|
||||
|
@ -22,7 +22,7 @@ public class ReindexRequestBuilder extends AbstractBulkIndexByScrollRequestBuild
|
|||
this(
|
||||
client,
|
||||
action,
|
||||
new SearchRequestBuilder(client, SearchAction.INSTANCE),
|
||||
new SearchRequestBuilder(client, TransportSearchAction.TYPE),
|
||||
new IndexRequestBuilder(client, IndexAction.INSTANCE)
|
||||
);
|
||||
}
|
||||
|
|
|
@ -9,8 +9,8 @@
|
|||
package org.elasticsearch.index.reindex;
|
||||
|
||||
import org.elasticsearch.action.ActionType;
|
||||
import org.elasticsearch.action.search.SearchAction;
|
||||
import org.elasticsearch.action.search.SearchRequestBuilder;
|
||||
import org.elasticsearch.action.search.TransportSearchAction;
|
||||
import org.elasticsearch.client.internal.ElasticsearchClient;
|
||||
|
||||
public class UpdateByQueryRequestBuilder extends AbstractBulkIndexByScrollRequestBuilder<
|
||||
|
@ -18,7 +18,7 @@ public class UpdateByQueryRequestBuilder extends AbstractBulkIndexByScrollReques
|
|||
UpdateByQueryRequestBuilder> {
|
||||
|
||||
public UpdateByQueryRequestBuilder(ElasticsearchClient client, ActionType<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) {
|
||||
|
|
|
@ -8,9 +8,9 @@
|
|||
|
||||
package org.elasticsearch.rest.action.search;
|
||||
|
||||
import org.elasticsearch.action.search.MultiSearchAction;
|
||||
import org.elasticsearch.action.search.MultiSearchRequest;
|
||||
import org.elasticsearch.action.search.SearchRequest;
|
||||
import org.elasticsearch.action.search.TransportMultiSearchAction;
|
||||
import org.elasticsearch.action.support.IndicesOptions;
|
||||
import org.elasticsearch.client.internal.node.NodeClient;
|
||||
import org.elasticsearch.common.CheckedBiConsumer;
|
||||
|
@ -83,7 +83,7 @@ public class RestMultiSearchAction extends BaseRestHandler {
|
|||
return channel -> {
|
||||
final RestCancellableNodeClient cancellableClient = new RestCancellableNodeClient(client, request.getHttpChannel());
|
||||
cancellableClient.execute(
|
||||
MultiSearchAction.INSTANCE,
|
||||
TransportMultiSearchAction.TYPE,
|
||||
multiSearchRequest,
|
||||
new RestRefCountedChunkedToXContentListener<>(channel)
|
||||
);
|
||||
|
|
|
@ -11,9 +11,9 @@ package org.elasticsearch.rest.action.search;
|
|||
import org.elasticsearch.ExceptionsHelper;
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.action.ActionRequestValidationException;
|
||||
import org.elasticsearch.action.search.SearchAction;
|
||||
import org.elasticsearch.action.search.SearchContextId;
|
||||
import org.elasticsearch.action.search.SearchRequest;
|
||||
import org.elasticsearch.action.search.TransportSearchAction;
|
||||
import org.elasticsearch.action.support.IndicesOptions;
|
||||
import org.elasticsearch.client.internal.node.NodeClient;
|
||||
import org.elasticsearch.common.Strings;
|
||||
|
@ -121,7 +121,7 @@ public class RestSearchAction extends BaseRestHandler {
|
|||
|
||||
return channel -> {
|
||||
RestCancellableNodeClient cancelClient = new RestCancellableNodeClient(client, request.getHttpChannel());
|
||||
cancelClient.execute(SearchAction.INSTANCE, searchRequest, new RestRefCountedChunkedToXContentListener<>(channel));
|
||||
cancelClient.execute(TransportSearchAction.TYPE, searchRequest, new RestRefCountedChunkedToXContentListener<>(channel));
|
||||
};
|
||||
}
|
||||
|
||||
|
|
|
@ -20,7 +20,7 @@ public class SearchRequestBuilderTests extends ESTestCase {
|
|||
|
||||
private SearchRequestBuilder makeBuilder() {
|
||||
ElasticsearchClient client = Mockito.mock(ElasticsearchClient.class);
|
||||
return new SearchRequestBuilder(client, SearchAction.INSTANCE);
|
||||
return new SearchRequestBuilder(client, TransportSearchAction.TYPE);
|
||||
}
|
||||
|
||||
public void testEmptySourceToString() {
|
||||
|
|
|
@ -528,7 +528,7 @@ public class SearchRequestTests extends AbstractSearchTestCase {
|
|||
}
|
||||
|
||||
private String toDescription(SearchRequest request) {
|
||||
return request.createTask(0, "test", SearchAction.NAME, TaskId.EMPTY_TASK_ID, emptyMap()).getDescription();
|
||||
return request.createTask(0, "test", TransportSearchAction.TYPE.name(), TaskId.EMPTY_TASK_ID, emptyMap()).getDescription();
|
||||
}
|
||||
|
||||
public void testForceSyntheticUnsupported() {
|
||||
|
|
|
@ -22,7 +22,7 @@ import org.elasticsearch.action.admin.indices.stats.IndicesStatsAction;
|
|||
import org.elasticsearch.action.delete.DeleteAction;
|
||||
import org.elasticsearch.action.get.GetAction;
|
||||
import org.elasticsearch.action.index.IndexAction;
|
||||
import org.elasticsearch.action.search.SearchAction;
|
||||
import org.elasticsearch.action.search.TransportSearchAction;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.util.concurrent.ThreadContext;
|
||||
import org.elasticsearch.env.Environment;
|
||||
|
@ -48,7 +48,7 @@ public abstract class AbstractClientHeadersTestCase extends ESTestCase {
|
|||
private static final ActionType<?>[] ACTIONS = new ActionType<?>[] {
|
||||
// client actions
|
||||
GetAction.INSTANCE,
|
||||
SearchAction.INSTANCE,
|
||||
TransportSearchAction.TYPE,
|
||||
DeleteAction.INSTANCE,
|
||||
DeleteStoredScriptAction.INSTANCE,
|
||||
IndexAction.INSTANCE,
|
||||
|
@ -97,7 +97,7 @@ public abstract class AbstractClientHeadersTestCase extends ESTestCase {
|
|||
|
||||
// choosing arbitrary top level actions to test
|
||||
client.prepareGet("idx", "id").execute(new AssertingActionListener<>(GetAction.NAME, client.threadPool()));
|
||||
client.prepareSearch().execute(new AssertingActionListener<>(SearchAction.NAME, client.threadPool()));
|
||||
client.prepareSearch().execute(new AssertingActionListener<>(TransportSearchAction.TYPE.name(), client.threadPool()));
|
||||
client.prepareDelete("idx", "id").execute(new AssertingActionListener<>(DeleteAction.NAME, client.threadPool()));
|
||||
client.admin()
|
||||
.cluster()
|
||||
|
|
|
@ -14,9 +14,9 @@ import org.elasticsearch.action.ActionResponse;
|
|||
import org.elasticsearch.action.ActionType;
|
||||
import org.elasticsearch.action.admin.cluster.node.tasks.cancel.CancelTasksAction;
|
||||
import org.elasticsearch.action.admin.cluster.node.tasks.cancel.CancelTasksRequest;
|
||||
import org.elasticsearch.action.search.SearchAction;
|
||||
import org.elasticsearch.action.search.SearchRequest;
|
||||
import org.elasticsearch.action.search.SearchResponse;
|
||||
import org.elasticsearch.action.search.TransportSearchAction;
|
||||
import org.elasticsearch.action.support.PlainActionFuture;
|
||||
import org.elasticsearch.client.internal.node.NodeClient;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
|
@ -76,7 +76,7 @@ public class RestCancellableNodeClientTests extends ESTestCase {
|
|||
for (int j = 0; j < numTasks; j++) {
|
||||
PlainActionFuture<SearchResponse> actionFuture = new PlainActionFuture<>();
|
||||
RestCancellableNodeClient client = new RestCancellableNodeClient(testClient, channel);
|
||||
threadPool.generic().submit(() -> client.execute(SearchAction.INSTANCE, new SearchRequest(), actionFuture));
|
||||
threadPool.generic().submit(() -> client.execute(TransportSearchAction.TYPE, new SearchRequest(), actionFuture));
|
||||
futures.add(actionFuture);
|
||||
}
|
||||
}
|
||||
|
@ -106,7 +106,7 @@ public class RestCancellableNodeClientTests extends ESTestCase {
|
|||
totalSearches += numTasks;
|
||||
RestCancellableNodeClient client = new RestCancellableNodeClient(nodeClient, channel);
|
||||
for (int j = 0; j < numTasks; j++) {
|
||||
client.execute(SearchAction.INSTANCE, new SearchRequest(), null);
|
||||
client.execute(TransportSearchAction.TYPE, new SearchRequest(), null);
|
||||
}
|
||||
assertEquals(numTasks, RestCancellableNodeClient.getNumTasks(channel));
|
||||
}
|
||||
|
@ -139,7 +139,7 @@ public class RestCancellableNodeClientTests extends ESTestCase {
|
|||
RestCancellableNodeClient client = new RestCancellableNodeClient(testClient, channel);
|
||||
for (int j = 0; j < numTasks; j++) {
|
||||
// here the channel will be first registered, then straight-away removed from the map as the close listener is invoked
|
||||
client.execute(SearchAction.INSTANCE, new SearchRequest(), null);
|
||||
client.execute(TransportSearchAction.TYPE, new SearchRequest(), null);
|
||||
}
|
||||
}
|
||||
assertEquals(initialHttpChannels, RestCancellableNodeClient.getNumChannels());
|
||||
|
@ -177,7 +177,7 @@ public class RestCancellableNodeClientTests extends ESTestCase {
|
|||
}
|
||||
return task;
|
||||
}
|
||||
case SearchAction.NAME -> {
|
||||
case TransportSearchAction.NAME -> {
|
||||
searchRequests.incrementAndGet();
|
||||
Task searchTask = request.createTask(counter.getAndIncrement(), "search", action.name(), null, Collections.emptyMap());
|
||||
if (timeout == false) {
|
||||
|
|
|
@ -25,9 +25,7 @@ import org.elasticsearch.action.DocWriteResponse;
|
|||
import org.elasticsearch.action.OriginalIndices;
|
||||
import org.elasticsearch.action.admin.cluster.settings.ClusterUpdateSettingsResponse;
|
||||
import org.elasticsearch.action.search.ClearScrollRequest;
|
||||
import org.elasticsearch.action.search.ClosePointInTimeAction;
|
||||
import org.elasticsearch.action.search.ClosePointInTimeRequest;
|
||||
import org.elasticsearch.action.search.OpenPointInTimeAction;
|
||||
import org.elasticsearch.action.search.OpenPointInTimeRequest;
|
||||
import org.elasticsearch.action.search.SearchPhaseExecutionException;
|
||||
import org.elasticsearch.action.search.SearchRequest;
|
||||
|
@ -35,6 +33,8 @@ import org.elasticsearch.action.search.SearchResponse;
|
|||
import org.elasticsearch.action.search.SearchScrollRequest;
|
||||
import org.elasticsearch.action.search.SearchShardTask;
|
||||
import org.elasticsearch.action.search.SearchType;
|
||||
import org.elasticsearch.action.search.TransportClosePointInTimeAction;
|
||||
import org.elasticsearch.action.search.TransportOpenPointInTimeAction;
|
||||
import org.elasticsearch.action.search.TransportSearchAction;
|
||||
import org.elasticsearch.action.support.IndicesOptions;
|
||||
import org.elasticsearch.action.support.PlainActionFuture;
|
||||
|
@ -1833,7 +1833,7 @@ public class SearchServiceTests extends ESSingleNodeTestCase {
|
|||
indicesAdmin().prepareRefresh("test").get();
|
||||
|
||||
String pitId = client().execute(
|
||||
OpenPointInTimeAction.INSTANCE,
|
||||
TransportOpenPointInTimeAction.TYPE,
|
||||
new OpenPointInTimeRequest("test").keepAlive(TimeValue.timeValueMinutes(10))
|
||||
).actionGet().getPointInTimeId();
|
||||
final MockSearchService searchService = (MockSearchService) getInstanceFromNode(SearchService.class);
|
||||
|
@ -1848,7 +1848,7 @@ public class SearchServiceTests extends ESSingleNodeTestCase {
|
|||
numDocs
|
||||
);
|
||||
} finally {
|
||||
client().execute(ClosePointInTimeAction.INSTANCE, new ClosePointInTimeRequest(pitId)).actionGet();
|
||||
client().execute(TransportClosePointInTimeAction.TYPE, new ClosePointInTimeRequest(pitId)).actionGet();
|
||||
}
|
||||
assertThat(shardRequests, not(emptyList()));
|
||||
for (ShardSearchRequest shardRequest : shardRequests) {
|
||||
|
|
|
@ -8,9 +8,9 @@
|
|||
|
||||
package org.elasticsearch.search.vectors;
|
||||
|
||||
import org.elasticsearch.action.search.SearchAction;
|
||||
import org.elasticsearch.action.search.SearchRequest;
|
||||
import org.elasticsearch.action.search.SearchRequestBuilder;
|
||||
import org.elasticsearch.action.search.TransportSearchAction;
|
||||
import org.elasticsearch.common.bytes.BytesReference;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.query.QueryBuilder;
|
||||
|
@ -224,7 +224,7 @@ public class KnnSearchRequestParserTests extends ESTestCase {
|
|||
.withContent(BytesReference.bytes(builder), builder.contentType())
|
||||
.build()
|
||||
);
|
||||
SearchRequestBuilder searchRequestBuilder = new SearchRequestBuilder(null, SearchAction.INSTANCE);
|
||||
SearchRequestBuilder searchRequestBuilder = new SearchRequestBuilder(null, TransportSearchAction.TYPE);
|
||||
knnRequestBuilder.toSearchRequest(searchRequestBuilder);
|
||||
return searchRequestBuilder;
|
||||
}
|
||||
|
|
|
@ -58,7 +58,6 @@ import org.elasticsearch.action.bulk.TransportBulkAction;
|
|||
import org.elasticsearch.action.bulk.TransportShardBulkAction;
|
||||
import org.elasticsearch.action.index.IndexRequest;
|
||||
import org.elasticsearch.action.resync.TransportResyncReplicationAction;
|
||||
import org.elasticsearch.action.search.SearchAction;
|
||||
import org.elasticsearch.action.search.SearchExecutionStatsCollector;
|
||||
import org.elasticsearch.action.search.SearchPhaseController;
|
||||
import org.elasticsearch.action.search.SearchRequest;
|
||||
|
@ -2000,7 +1999,7 @@ public class SnapshotResiliencyTests extends ESTestCase {
|
|||
|
||||
SearchPhaseController searchPhaseController = new SearchPhaseController(searchService::aggReduceContextBuilder);
|
||||
actions.put(
|
||||
SearchAction.INSTANCE,
|
||||
TransportSearchAction.TYPE,
|
||||
new TransportSearchAction(
|
||||
threadPool,
|
||||
new NoneCircuitBreakerService(),
|
||||
|
|
|
@ -10,9 +10,9 @@ package org.elasticsearch.transport;
|
|||
|
||||
import org.elasticsearch.TransportVersion;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.search.SearchShardsAction;
|
||||
import org.elasticsearch.action.search.SearchShardsRequest;
|
||||
import org.elasticsearch.action.search.SearchShardsResponse;
|
||||
import org.elasticsearch.action.search.TransportSearchShardsAction;
|
||||
import org.elasticsearch.action.support.IndicesOptions;
|
||||
import org.elasticsearch.action.support.PlainActionFuture;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||
|
@ -104,7 +104,7 @@ public class RemoteClusterAwareClientTests extends ESTestCase {
|
|||
);
|
||||
final SearchShardsResponse searchShardsResponse = PlainActionFuture.get(
|
||||
future -> client.execute(
|
||||
SearchShardsAction.INSTANCE,
|
||||
TransportSearchShardsAction.TYPE,
|
||||
searchShardsRequest,
|
||||
ActionListener.runBefore(
|
||||
future,
|
||||
|
@ -169,7 +169,7 @@ public class RemoteClusterAwareClientTests extends ESTestCase {
|
|||
null
|
||||
);
|
||||
client.execute(
|
||||
SearchShardsAction.INSTANCE,
|
||||
TransportSearchShardsAction.TYPE,
|
||||
searchShardsRequest,
|
||||
ActionListener.runBefore(
|
||||
future,
|
||||
|
|
|
@ -15,13 +15,13 @@ import org.elasticsearch.action.admin.cluster.remote.RemoteClusterNodesAction;
|
|||
import org.elasticsearch.action.admin.cluster.state.ClusterStateAction;
|
||||
import org.elasticsearch.action.admin.cluster.state.ClusterStateRequest;
|
||||
import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse;
|
||||
import org.elasticsearch.action.search.SearchAction;
|
||||
import org.elasticsearch.action.search.SearchRequest;
|
||||
import org.elasticsearch.action.search.SearchResponse;
|
||||
import org.elasticsearch.action.search.SearchShardsAction;
|
||||
import org.elasticsearch.action.search.SearchShardsRequest;
|
||||
import org.elasticsearch.action.search.SearchShardsResponse;
|
||||
import org.elasticsearch.action.search.ShardSearchFailure;
|
||||
import org.elasticsearch.action.search.TransportSearchAction;
|
||||
import org.elasticsearch.action.search.TransportSearchShardsAction;
|
||||
import org.elasticsearch.action.support.PlainActionFuture;
|
||||
import org.elasticsearch.cluster.ClusterName;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
|
@ -125,7 +125,7 @@ public class RemoteClusterConnectionTests extends ESTestCase {
|
|||
MockTransportService newService = MockTransportService.createNewService(s, version, transportVersion, threadPool, null);
|
||||
try {
|
||||
newService.registerRequestHandler(
|
||||
SearchShardsAction.NAME,
|
||||
TransportSearchShardsAction.TYPE.name(),
|
||||
EsExecutors.DIRECT_EXECUTOR_SERVICE,
|
||||
SearchShardsRequest::new,
|
||||
(request, channel, task) -> {
|
||||
|
@ -137,7 +137,7 @@ public class RemoteClusterConnectionTests extends ESTestCase {
|
|||
}
|
||||
);
|
||||
newService.registerRequestHandler(
|
||||
SearchAction.NAME,
|
||||
TransportSearchAction.TYPE.name(),
|
||||
EsExecutors.DIRECT_EXECUTOR_SERVICE,
|
||||
SearchRequest::new,
|
||||
(request, channel, task) -> {
|
||||
|
|
|
@ -12,10 +12,10 @@ import org.elasticsearch.ResourceNotFoundException;
|
|||
import org.elasticsearch.action.admin.cluster.node.tasks.get.GetTaskResponse;
|
||||
import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse;
|
||||
import org.elasticsearch.action.get.GetResponse;
|
||||
import org.elasticsearch.action.search.ClosePointInTimeAction;
|
||||
import org.elasticsearch.action.search.ClosePointInTimeRequest;
|
||||
import org.elasticsearch.action.search.OpenPointInTimeAction;
|
||||
import org.elasticsearch.action.search.OpenPointInTimeRequest;
|
||||
import org.elasticsearch.action.search.TransportClosePointInTimeAction;
|
||||
import org.elasticsearch.action.search.TransportOpenPointInTimeAction;
|
||||
import org.elasticsearch.action.support.master.AcknowledgedResponse;
|
||||
import org.elasticsearch.cluster.ClusterChangedEvent;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
|
@ -235,7 +235,7 @@ public abstract class AsyncSearchIntegTestCase extends ESIntegTestCase {
|
|||
final SubmitAsyncSearchRequest request;
|
||||
if (randomBoolean()) {
|
||||
OpenPointInTimeRequest openPIT = new OpenPointInTimeRequest(indexName).keepAlive(TimeValue.timeValueMinutes(between(5, 10)));
|
||||
pitId = client().execute(OpenPointInTimeAction.INSTANCE, openPIT).actionGet().getPointInTimeId();
|
||||
pitId = client().execute(TransportOpenPointInTimeAction.TYPE, openPIT).actionGet().getPointInTimeId();
|
||||
final PointInTimeBuilder pit = new PointInTimeBuilder(pitId);
|
||||
if (randomBoolean()) {
|
||||
pit.setKeepAlive(TimeValue.timeValueMillis(randomIntBetween(1, 3600)));
|
||||
|
@ -329,7 +329,7 @@ public abstract class AsyncSearchIntegTestCase extends ESIntegTestCase {
|
|||
public void close() {
|
||||
if (closed.compareAndSet(false, true)) {
|
||||
if (pitId != null) {
|
||||
client().execute(ClosePointInTimeAction.INSTANCE, new ClosePointInTimeRequest(pitId)).actionGet();
|
||||
client().execute(TransportClosePointInTimeAction.TYPE, new ClosePointInTimeRequest(pitId)).actionGet();
|
||||
}
|
||||
queryLatch.close();
|
||||
}
|
||||
|
|
|
@ -13,7 +13,6 @@ import org.elasticsearch.action.ActionFuture;
|
|||
import org.elasticsearch.action.admin.cluster.node.tasks.cancel.CancelTasksRequest;
|
||||
import org.elasticsearch.action.admin.cluster.node.tasks.cancel.CancelTasksResponse;
|
||||
import org.elasticsearch.action.admin.cluster.node.tasks.list.ListTasksResponse;
|
||||
import org.elasticsearch.action.search.SearchAction;
|
||||
import org.elasticsearch.action.search.SearchResponse;
|
||||
import org.elasticsearch.action.search.SearchType;
|
||||
import org.elasticsearch.action.search.ShardSearchFailure;
|
||||
|
@ -1266,7 +1265,7 @@ public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase {
|
|||
ListTasksResponse listTasksResponse = client(LOCAL_CLUSTER).admin()
|
||||
.cluster()
|
||||
.prepareListTasks()
|
||||
.setActions(SearchAction.INSTANCE.name())
|
||||
.setActions(TransportSearchAction.TYPE.name())
|
||||
.get();
|
||||
List<TaskInfo> tasks = listTasksResponse.getTasks();
|
||||
assertThat(tasks.size(), equalTo(1));
|
||||
|
@ -1280,7 +1279,7 @@ public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase {
|
|||
.get()
|
||||
.getTasks()
|
||||
.stream()
|
||||
.filter(t -> t.action().contains(SearchAction.NAME))
|
||||
.filter(t -> t.action().contains(TransportSearchAction.TYPE.name()))
|
||||
.collect(Collectors.toList());
|
||||
assertThat(remoteSearchTasks.size(), greaterThan(0));
|
||||
remoteClusterSearchTasks.set(remoteSearchTasks);
|
||||
|
@ -1298,7 +1297,7 @@ public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase {
|
|||
for (TransportService transportService : transportServices) {
|
||||
Collection<CancellableTask> cancellableTasks = transportService.getTaskManager().getCancellableTasks().values();
|
||||
for (CancellableTask cancellableTask : cancellableTasks) {
|
||||
if (cancellableTask.getAction().contains(SearchAction.INSTANCE.name())) {
|
||||
if (cancellableTask.getAction().contains(TransportSearchAction.TYPE.name())) {
|
||||
assertTrue(cancellableTask.getDescription(), cancellableTask.isCancelled());
|
||||
}
|
||||
}
|
||||
|
@ -1311,7 +1310,7 @@ public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase {
|
|||
.get()
|
||||
.getTasks()
|
||||
.stream()
|
||||
.filter(t -> t.action().contains(SearchAction.INSTANCE.name()))
|
||||
.filter(t -> t.action().contains(TransportSearchAction.TYPE.name()))
|
||||
.toList();
|
||||
for (TaskInfo taskInfo : remoteSearchTasksAfterCancellation) {
|
||||
assertTrue(taskInfo.description(), taskInfo.cancelled());
|
||||
|
@ -1385,7 +1384,7 @@ public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase {
|
|||
ListTasksResponse listTasksResponse = client(LOCAL_CLUSTER).admin()
|
||||
.cluster()
|
||||
.prepareListTasks()
|
||||
.setActions(SearchAction.INSTANCE.name())
|
||||
.setActions(TransportSearchAction.TYPE.name())
|
||||
.get();
|
||||
List<TaskInfo> tasks = listTasksResponse.getTasks();
|
||||
assertThat(tasks.size(), equalTo(1));
|
||||
|
@ -1398,7 +1397,7 @@ public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase {
|
|||
.get()
|
||||
.getTasks()
|
||||
.stream()
|
||||
.filter(t -> t.action().contains(SearchAction.NAME))
|
||||
.filter(t -> t.action().contains(TransportSearchAction.TYPE.name()))
|
||||
.collect(Collectors.toList());
|
||||
assertThat(remoteSearchTasks.size(), greaterThan(0));
|
||||
remoteClusterSearchTasks.set(remoteSearchTasks);
|
||||
|
@ -1416,7 +1415,7 @@ public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase {
|
|||
for (TransportService transportService : transportServices) {
|
||||
Collection<CancellableTask> cancellableTasks = transportService.getTaskManager().getCancellableTasks().values();
|
||||
for (CancellableTask cancellableTask : cancellableTasks) {
|
||||
if (cancellableTask.getAction().contains(SearchAction.INSTANCE.name())) {
|
||||
if (cancellableTask.getAction().contains(TransportSearchAction.TYPE.name())) {
|
||||
assertTrue(cancellableTask.getDescription(), cancellableTask.isCancelled());
|
||||
}
|
||||
}
|
||||
|
@ -1429,7 +1428,7 @@ public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase {
|
|||
.get()
|
||||
.getTasks()
|
||||
.stream()
|
||||
.filter(t -> t.action().contains(SearchAction.INSTANCE.name()))
|
||||
.filter(t -> t.action().contains(TransportSearchAction.TYPE.name()))
|
||||
.toList();
|
||||
for (TaskInfo taskInfo : remoteSearchTasksAfterCancellation) {
|
||||
assertTrue(taskInfo.description(), taskInfo.cancelled());
|
||||
|
@ -1489,7 +1488,7 @@ public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase {
|
|||
ListTasksResponse listTasksResponse = client(LOCAL_CLUSTER).admin()
|
||||
.cluster()
|
||||
.prepareListTasks()
|
||||
.setActions(SearchAction.INSTANCE.name())
|
||||
.setActions(TransportSearchAction.TYPE.name())
|
||||
.get();
|
||||
List<TaskInfo> tasks = listTasksResponse.getTasks();
|
||||
assertThat(tasks.size(), equalTo(1));
|
||||
|
@ -1502,7 +1501,7 @@ public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase {
|
|||
.get()
|
||||
.getTasks()
|
||||
.stream()
|
||||
.filter(t -> t.action().contains(SearchAction.NAME))
|
||||
.filter(t -> t.action().contains(TransportSearchAction.TYPE.name()))
|
||||
.collect(Collectors.toList());
|
||||
assertThat(remoteSearchTasks.size(), greaterThan(0));
|
||||
remoteClusterSearchTasks.set(remoteSearchTasks);
|
||||
|
@ -1524,7 +1523,7 @@ public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase {
|
|||
ListTasksResponse listTasksResponse = client(LOCAL_CLUSTER).admin()
|
||||
.cluster()
|
||||
.prepareListTasks()
|
||||
.setActions(SearchAction.INSTANCE.name())
|
||||
.setActions(TransportSearchAction.TYPE.name())
|
||||
.get();
|
||||
List<TaskInfo> tasks = listTasksResponse.getTasks();
|
||||
assertThat(tasks.size(), equalTo(0));
|
||||
|
@ -1532,7 +1531,7 @@ public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase {
|
|||
ListTasksResponse remoteTasksResponse = client(REMOTE_CLUSTER).admin()
|
||||
.cluster()
|
||||
.prepareListTasks()
|
||||
.setActions(SearchAction.INSTANCE.name())
|
||||
.setActions(TransportSearchAction.TYPE.name())
|
||||
.get();
|
||||
List<TaskInfo> remoteTasks = remoteTasksResponse.getTasks();
|
||||
assertThat(remoteTasks.size(), equalTo(0));
|
||||
|
@ -1554,7 +1553,7 @@ public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase {
|
|||
ListTasksResponse listTasksResponse = client(LOCAL_CLUSTER).admin()
|
||||
.cluster()
|
||||
.prepareListTasks()
|
||||
.setActions(SearchAction.INSTANCE.name())
|
||||
.setActions(TransportSearchAction.TYPE.name())
|
||||
.get();
|
||||
List<TaskInfo> tasks = listTasksResponse.getTasks();
|
||||
assertThat(tasks.size(), equalTo(0));
|
||||
|
@ -1562,7 +1561,7 @@ public class CrossClusterAsyncSearchIT extends AbstractMultiClustersTestCase {
|
|||
ListTasksResponse remoteTasksResponse = client(REMOTE_CLUSTER).admin()
|
||||
.cluster()
|
||||
.prepareListTasks()
|
||||
.setActions(SearchAction.INSTANCE.name())
|
||||
.setActions(TransportSearchAction.TYPE.name())
|
||||
.get();
|
||||
List<TaskInfo> remoteTasks = remoteTasksResponse.getTasks();
|
||||
assertThat(remoteTasks.size(), equalTo(0));
|
||||
|
|
|
@ -8,7 +8,6 @@ package org.elasticsearch.xpack.search;
|
|||
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.DocWriteResponse;
|
||||
import org.elasticsearch.action.search.SearchAction;
|
||||
import org.elasticsearch.action.search.SearchRequest;
|
||||
import org.elasticsearch.action.search.TransportSearchAction;
|
||||
import org.elasticsearch.action.support.ActionFilters;
|
||||
|
@ -87,7 +86,11 @@ public class TransportSubmitAsyncSearchAction extends HandledTransportAction<Sub
|
|||
protected void doExecute(Task submitTask, SubmitAsyncSearchRequest request, ActionListener<AsyncSearchResponse> submitListener) {
|
||||
final SearchRequest searchRequest = createSearchRequest(request, submitTask, request.getKeepAlive());
|
||||
try (var ignored = threadContext.newTraceContext()) {
|
||||
AsyncSearchTask searchTask = (AsyncSearchTask) taskManager.register("transport", SearchAction.INSTANCE.name(), searchRequest);
|
||||
AsyncSearchTask searchTask = (AsyncSearchTask) taskManager.register(
|
||||
"transport",
|
||||
TransportSearchAction.TYPE.name(),
|
||||
searchRequest
|
||||
);
|
||||
searchAction.execute(searchTask, searchRequest, searchTask.getSearchProgressActionListener());
|
||||
searchTask.addCompletionListener(new ActionListener<>() {
|
||||
@Override
|
||||
|
|
|
@ -29,7 +29,7 @@ import org.elasticsearch.action.datastreams.DeleteDataStreamAction;
|
|||
import org.elasticsearch.action.datastreams.GetDataStreamAction;
|
||||
import org.elasticsearch.action.datastreams.PromoteDataStreamAction;
|
||||
import org.elasticsearch.action.fieldcaps.FieldCapabilitiesAction;
|
||||
import org.elasticsearch.action.search.SearchShardsAction;
|
||||
import org.elasticsearch.action.search.TransportSearchShardsAction;
|
||||
import org.elasticsearch.common.Strings;
|
||||
import org.elasticsearch.index.seqno.RetentionLeaseActions;
|
||||
import org.elasticsearch.xpack.core.ccr.action.ForgetFollowerAction;
|
||||
|
@ -78,7 +78,7 @@ public final class IndexPrivilege extends Privilege {
|
|||
private static final Automaton READ_CROSS_CLUSTER_AUTOMATON = patterns(
|
||||
"internal:transport/proxy/indices:data/read/*",
|
||||
ClusterSearchShardsAction.NAME,
|
||||
SearchShardsAction.NAME
|
||||
TransportSearchShardsAction.TYPE.name()
|
||||
);
|
||||
private static final Automaton CREATE_AUTOMATON = patterns(
|
||||
"indices:data/write/index*",
|
||||
|
@ -119,7 +119,7 @@ public final class IndexPrivilege extends Privilege {
|
|||
GetFieldMappingsAction.NAME + "*",
|
||||
GetMappingsAction.NAME,
|
||||
ClusterSearchShardsAction.NAME,
|
||||
SearchShardsAction.NAME,
|
||||
TransportSearchShardsAction.TYPE.name(),
|
||||
ValidateQueryAction.NAME + "*",
|
||||
GetSettingsAction.NAME,
|
||||
ExplainLifecycleAction.NAME,
|
||||
|
|
|
@ -6,7 +6,7 @@
|
|||
*/
|
||||
package org.elasticsearch.xpack.core.security.authz.privilege;
|
||||
|
||||
import org.elasticsearch.action.search.SearchShardsAction;
|
||||
import org.elasticsearch.action.search.TransportSearchShardsAction;
|
||||
import org.elasticsearch.index.seqno.RetentionLeaseActions;
|
||||
import org.elasticsearch.index.seqno.RetentionLeaseBackgroundSyncAction;
|
||||
import org.elasticsearch.index.seqno.RetentionLeaseSyncAction;
|
||||
|
@ -43,7 +43,7 @@ public final class SystemPrivilege extends Privilege {
|
|||
"indices:data/read/*", // needed for SystemIndexMigrator
|
||||
"indices:admin/refresh", // needed for SystemIndexMigrator
|
||||
"indices:admin/aliases", // needed for SystemIndexMigrator
|
||||
SearchShardsAction.NAME // added so this API can be called with the system user by other APIs
|
||||
TransportSearchShardsAction.TYPE.name() // added so this API can be called with the system user by other APIs
|
||||
);
|
||||
|
||||
private static final Predicate<String> PREDICATE = (action) -> {
|
||||
|
|
|
@ -12,10 +12,10 @@ import org.elasticsearch.action.ActionListener;
|
|||
import org.elasticsearch.action.admin.cluster.health.ClusterHealthAction;
|
||||
import org.elasticsearch.action.admin.cluster.health.ClusterHealthRequest;
|
||||
import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse;
|
||||
import org.elasticsearch.action.search.SearchAction;
|
||||
import org.elasticsearch.action.search.SearchRequest;
|
||||
import org.elasticsearch.action.search.SearchResponse;
|
||||
import org.elasticsearch.action.search.ShardSearchFailure;
|
||||
import org.elasticsearch.action.search.TransportSearchAction;
|
||||
import org.elasticsearch.action.support.ActionTestUtils;
|
||||
import org.elasticsearch.action.support.PlainActionFuture;
|
||||
import org.elasticsearch.client.internal.Client;
|
||||
|
@ -170,7 +170,7 @@ public class ClientHelperTests extends ESTestCase {
|
|||
SearchRequest request = new SearchRequest("foo");
|
||||
|
||||
String originName = randomFrom(ClientHelper.ML_ORIGIN, ClientHelper.WATCHER_ORIGIN, ClientHelper.ROLLUP_ORIGIN);
|
||||
ClientHelper.executeWithHeadersAsync(Collections.emptyMap(), originName, client, SearchAction.INSTANCE, request, listener);
|
||||
ClientHelper.executeWithHeadersAsync(Collections.emptyMap(), originName, client, TransportSearchAction.TYPE, request, listener);
|
||||
|
||||
latch.await();
|
||||
}
|
||||
|
@ -201,7 +201,7 @@ public class ClientHelperTests extends ESTestCase {
|
|||
headers.put("bar", "bar");
|
||||
|
||||
String originName = randomFrom(ClientHelper.ML_ORIGIN, ClientHelper.WATCHER_ORIGIN, ClientHelper.ROLLUP_ORIGIN);
|
||||
ClientHelper.executeWithHeadersAsync(headers, originName, client, SearchAction.INSTANCE, request, listener);
|
||||
ClientHelper.executeWithHeadersAsync(headers, originName, client, TransportSearchAction.TYPE, request, listener);
|
||||
|
||||
latch.await();
|
||||
}
|
||||
|
@ -234,7 +234,7 @@ public class ClientHelperTests extends ESTestCase {
|
|||
headers.put("_xpack_security_authentication", "bar");
|
||||
|
||||
String originName = randomFrom(ClientHelper.ML_ORIGIN, ClientHelper.WATCHER_ORIGIN, ClientHelper.ROLLUP_ORIGIN);
|
||||
ClientHelper.executeWithHeadersAsync(headers, originName, client, SearchAction.INSTANCE, request, listener);
|
||||
ClientHelper.executeWithHeadersAsync(headers, originName, client, TransportSearchAction.TYPE, request, listener);
|
||||
|
||||
latch.await();
|
||||
}
|
||||
|
|
|
@ -8,9 +8,9 @@
|
|||
package org.elasticsearch.xpack.core.ilm;
|
||||
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.search.SearchAction;
|
||||
import org.elasticsearch.action.search.SearchRequest;
|
||||
import org.elasticsearch.action.search.SearchResponse;
|
||||
import org.elasticsearch.action.search.TransportSearchAction;
|
||||
import org.elasticsearch.action.support.ActionTestUtils;
|
||||
import org.elasticsearch.client.internal.Client;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
|
@ -57,7 +57,7 @@ public class LifecyclePolicyClientTests extends ESTestCase {
|
|||
SearchRequest request = new SearchRequest("foo");
|
||||
|
||||
final var policyClient = new LifecyclePolicySecurityClient(client, ClientHelper.INDEX_LIFECYCLE_ORIGIN, Collections.emptyMap());
|
||||
policyClient.execute(SearchAction.INSTANCE, request, listener);
|
||||
policyClient.execute(TransportSearchAction.TYPE, request, listener);
|
||||
|
||||
latch.await();
|
||||
}
|
||||
|
@ -89,7 +89,7 @@ public class LifecyclePolicyClientTests extends ESTestCase {
|
|||
headers.put("bar", "bar");
|
||||
|
||||
final var policyClient = new LifecyclePolicySecurityClient(client, ClientHelper.INDEX_LIFECYCLE_ORIGIN, headers);
|
||||
policyClient.execute(SearchAction.INSTANCE, request, listener);
|
||||
policyClient.execute(TransportSearchAction.TYPE, request, listener);
|
||||
|
||||
latch.await();
|
||||
}
|
||||
|
@ -123,7 +123,7 @@ public class LifecyclePolicyClientTests extends ESTestCase {
|
|||
headers.put("_xpack_security_authentication", "bar");
|
||||
|
||||
final var policyClient = new LifecyclePolicySecurityClient(client, ClientHelper.INDEX_LIFECYCLE_ORIGIN, headers);
|
||||
policyClient.execute(SearchAction.INSTANCE, request, listener);
|
||||
policyClient.execute(TransportSearchAction.TYPE, request, listener);
|
||||
|
||||
latch.await();
|
||||
}
|
||||
|
|
|
@ -11,7 +11,7 @@ import org.apache.lucene.util.automaton.Automaton;
|
|||
import org.elasticsearch.action.admin.indices.create.CreateIndexAction;
|
||||
import org.elasticsearch.action.admin.indices.delete.DeleteIndexAction;
|
||||
import org.elasticsearch.action.bulk.BulkAction;
|
||||
import org.elasticsearch.action.search.SearchAction;
|
||||
import org.elasticsearch.action.search.TransportSearchAction;
|
||||
import org.elasticsearch.cluster.metadata.AliasMetadata;
|
||||
import org.elasticsearch.cluster.metadata.IndexAbstraction;
|
||||
import org.elasticsearch.cluster.metadata.IndexMetadata;
|
||||
|
@ -299,7 +299,7 @@ public class LimitedRoleTests extends ESTestCase {
|
|||
.build();
|
||||
|
||||
IndicesAccessControl iac = fromRole.authorize(
|
||||
SearchAction.NAME,
|
||||
TransportSearchAction.TYPE.name(),
|
||||
Sets.newHashSet("_index", "_alias1"),
|
||||
md.getIndicesLookup(),
|
||||
fieldPermissionsCache
|
||||
|
@ -328,7 +328,7 @@ public class LimitedRoleTests extends ESTestCase {
|
|||
.add(IndexPrivilege.NONE, "_index1")
|
||||
.build();
|
||||
iac = limitedByRole.authorize(
|
||||
SearchAction.NAME,
|
||||
TransportSearchAction.TYPE.name(),
|
||||
Sets.newHashSet("_index", "_alias1"),
|
||||
md.getIndicesLookup(),
|
||||
fieldPermissionsCache
|
||||
|
@ -367,7 +367,12 @@ public class LimitedRoleTests extends ESTestCase {
|
|||
} else {
|
||||
role = fromRole.limitedBy(limitedByRole);
|
||||
}
|
||||
iac = role.authorize(SearchAction.NAME, Sets.newHashSet("_index", "_alias1"), md.getIndicesLookup(), fieldPermissionsCache);
|
||||
iac = role.authorize(
|
||||
TransportSearchAction.TYPE.name(),
|
||||
Sets.newHashSet("_index", "_alias1"),
|
||||
md.getIndicesLookup(),
|
||||
fieldPermissionsCache
|
||||
);
|
||||
assertThat(iac.isGranted(), is(false));
|
||||
assertThat(iac.getIndexPermissions("_index"), is(notNullValue()));
|
||||
assertThat(iac.hasIndexPermissions("_index"), is(true));
|
||||
|
@ -440,12 +445,12 @@ public class LimitedRoleTests extends ESTestCase {
|
|||
|
||||
public void testCheckIndicesAction() {
|
||||
Role fromRole = Role.builder(EMPTY_RESTRICTED_INDICES, "a-role").add(IndexPrivilege.READ, "ind-1").build();
|
||||
assertThat(fromRole.checkIndicesAction(SearchAction.NAME), is(true));
|
||||
assertThat(fromRole.checkIndicesAction(TransportSearchAction.TYPE.name()), is(true));
|
||||
assertThat(fromRole.checkIndicesAction(CreateIndexAction.NAME), is(false));
|
||||
|
||||
{
|
||||
Role limitedByRole = Role.builder(EMPTY_RESTRICTED_INDICES, "limited-role").add(IndexPrivilege.ALL, "ind-1").build();
|
||||
assertThat(limitedByRole.checkIndicesAction(SearchAction.NAME), is(true));
|
||||
assertThat(limitedByRole.checkIndicesAction(TransportSearchAction.TYPE.name()), is(true));
|
||||
assertThat(limitedByRole.checkIndicesAction(CreateIndexAction.NAME), is(true));
|
||||
Role role;
|
||||
if (randomBoolean()) {
|
||||
|
@ -453,64 +458,79 @@ public class LimitedRoleTests extends ESTestCase {
|
|||
} else {
|
||||
role = fromRole.limitedBy(limitedByRole);
|
||||
}
|
||||
assertThat(role.checkIndicesAction(SearchAction.NAME), is(true));
|
||||
assertThat(role.checkIndicesAction(TransportSearchAction.TYPE.name()), is(true));
|
||||
assertThat(role.checkIndicesAction(CreateIndexAction.NAME), is(false));
|
||||
}
|
||||
{
|
||||
Role limitedByRole = Role.builder(EMPTY_RESTRICTED_INDICES, "limited-role").add(IndexPrivilege.NONE, "ind-1").build();
|
||||
assertThat(limitedByRole.checkIndicesAction(SearchAction.NAME), is(false));
|
||||
assertThat(limitedByRole.checkIndicesAction(TransportSearchAction.TYPE.name()), is(false));
|
||||
Role role;
|
||||
if (randomBoolean()) {
|
||||
role = limitedByRole.limitedBy(fromRole);
|
||||
} else {
|
||||
role = fromRole.limitedBy(limitedByRole);
|
||||
}
|
||||
assertThat(role.checkIndicesAction(SearchAction.NAME), is(false));
|
||||
assertThat(role.checkIndicesAction(TransportSearchAction.TYPE.name()), is(false));
|
||||
assertThat(role.checkIndicesAction(CreateIndexAction.NAME), is(false));
|
||||
}
|
||||
}
|
||||
|
||||
public void testAllowedIndicesMatcher() {
|
||||
Role fromRole = Role.builder(EMPTY_RESTRICTED_INDICES, "a-role").add(IndexPrivilege.READ, "ind-1*").build();
|
||||
assertThat(fromRole.allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction("ind-1")), is(true));
|
||||
assertThat(fromRole.allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction("ind-11")), is(true));
|
||||
assertThat(fromRole.allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction("ind-2")), is(false));
|
||||
assertThat(fromRole.allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction("ind-1")), is(true));
|
||||
assertThat(fromRole.allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction("ind-11")), is(true));
|
||||
assertThat(fromRole.allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction("ind-2")), is(false));
|
||||
|
||||
{
|
||||
Role limitedByRole = Role.builder(EMPTY_RESTRICTED_INDICES, "limited-role").add(IndexPrivilege.READ, "ind-1", "ind-2").build();
|
||||
assertThat(limitedByRole.allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction("ind-1")), is(true));
|
||||
assertThat(limitedByRole.allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction("ind-11")), is(false));
|
||||
assertThat(limitedByRole.allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction("ind-2")), is(true));
|
||||
assertThat(
|
||||
limitedByRole.allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction("ind-1")),
|
||||
is(true)
|
||||
);
|
||||
assertThat(
|
||||
limitedByRole.allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction("ind-11")),
|
||||
is(false)
|
||||
);
|
||||
assertThat(
|
||||
limitedByRole.allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction("ind-2")),
|
||||
is(true)
|
||||
);
|
||||
Role role;
|
||||
if (randomBoolean()) {
|
||||
role = limitedByRole.limitedBy(fromRole);
|
||||
} else {
|
||||
role = fromRole.limitedBy(limitedByRole);
|
||||
}
|
||||
assertThat(role.allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction("ind-1")), is(true));
|
||||
assertThat(role.allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction("ind-11")), is(false));
|
||||
assertThat(role.allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction("ind-2")), is(false));
|
||||
assertThat(role.allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction("ind-1")), is(true));
|
||||
assertThat(role.allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction("ind-11")), is(false));
|
||||
assertThat(role.allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction("ind-2")), is(false));
|
||||
}
|
||||
{
|
||||
Role limitedByRole = Role.builder(EMPTY_RESTRICTED_INDICES, "limited-role").add(IndexPrivilege.READ, "ind-*").build();
|
||||
assertThat(limitedByRole.allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction("ind-1")), is(true));
|
||||
assertThat(limitedByRole.allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction("ind-2")), is(true));
|
||||
assertThat(
|
||||
limitedByRole.allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction("ind-1")),
|
||||
is(true)
|
||||
);
|
||||
assertThat(
|
||||
limitedByRole.allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction("ind-2")),
|
||||
is(true)
|
||||
);
|
||||
Role role;
|
||||
if (randomBoolean()) {
|
||||
role = limitedByRole.limitedBy(fromRole);
|
||||
} else {
|
||||
role = fromRole.limitedBy(limitedByRole);
|
||||
}
|
||||
assertThat(role.allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction("ind-1")), is(true));
|
||||
assertThat(role.allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction("ind-2")), is(false));
|
||||
assertThat(role.allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction("ind-1")), is(true));
|
||||
assertThat(role.allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction("ind-2")), is(false));
|
||||
}
|
||||
}
|
||||
|
||||
public void testAllowedIndicesMatcherWithNestedRole() {
|
||||
Role role = Role.builder(EMPTY_RESTRICTED_INDICES, "a-role").add(IndexPrivilege.READ, "ind-1*").build();
|
||||
assertThat(role.allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction("ind-1")), is(true));
|
||||
assertThat(role.allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction("ind-11")), is(true));
|
||||
assertThat(role.allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction("ind-2")), is(false));
|
||||
assertThat(role.allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction("ind-1")), is(true));
|
||||
assertThat(role.allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction("ind-11")), is(true));
|
||||
assertThat(role.allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction("ind-2")), is(false));
|
||||
|
||||
final int depth = randomIntBetween(2, 4);
|
||||
boolean index11Excluded = false;
|
||||
|
@ -526,9 +546,12 @@ public class LimitedRoleTests extends ESTestCase {
|
|||
} else {
|
||||
role = role.limitedBy(limitedByRole);
|
||||
}
|
||||
assertThat(role.allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction("ind-1")), is(true));
|
||||
assertThat(role.allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction("ind-11")), is(false == index11Excluded));
|
||||
assertThat(role.allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction("ind-2")), is(false));
|
||||
assertThat(role.allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction("ind-1")), is(true));
|
||||
assertThat(
|
||||
role.allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction("ind-11")),
|
||||
is(false == index11Excluded)
|
||||
);
|
||||
assertThat(role.allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction("ind-2")), is(false));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -540,13 +563,13 @@ public class LimitedRoleTests extends ESTestCase {
|
|||
.build();
|
||||
Automaton fromRoleAutomaton = fromRole.allowedActionsMatcher("index1");
|
||||
Predicate<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));
|
||||
|
||||
Role limitedByRole = Role.builder(EMPTY_RESTRICTED_INDICES, "limitedRole").add(IndexPrivilege.READ, "index1", "index2").build();
|
||||
Automaton limitedByRoleAutomaton = limitedByRole.allowedActionsMatcher("index1");
|
||||
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));
|
||||
Role role;
|
||||
if (randomBoolean()) {
|
||||
|
@ -557,17 +580,17 @@ public class LimitedRoleTests extends ESTestCase {
|
|||
|
||||
Automaton roleAutomaton = role.allowedActionsMatcher("index1");
|
||||
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));
|
||||
|
||||
roleAutomaton = role.allowedActionsMatcher("index2");
|
||||
rolePredicate = Automatons.predicate(roleAutomaton);
|
||||
assertThat(rolePredicate.test(SearchAction.NAME), is(true));
|
||||
assertThat(rolePredicate.test(TransportSearchAction.TYPE.name()), is(true));
|
||||
assertThat(rolePredicate.test(BulkAction.NAME), is(false));
|
||||
|
||||
roleAutomaton = role.allowedActionsMatcher("other");
|
||||
rolePredicate = Automatons.predicate(roleAutomaton);
|
||||
assertThat(rolePredicate.test(SearchAction.NAME), is(false));
|
||||
assertThat(rolePredicate.test(TransportSearchAction.TYPE.name()), is(false));
|
||||
assertThat(rolePredicate.test(BulkAction.NAME), is(false));
|
||||
}
|
||||
|
||||
|
|
|
@ -13,7 +13,7 @@ import org.elasticsearch.action.admin.indices.shrink.ShrinkAction;
|
|||
import org.elasticsearch.action.admin.indices.stats.IndicesStatsAction;
|
||||
import org.elasticsearch.action.delete.DeleteAction;
|
||||
import org.elasticsearch.action.index.IndexAction;
|
||||
import org.elasticsearch.action.search.SearchAction;
|
||||
import org.elasticsearch.action.search.TransportSearchAction;
|
||||
import org.elasticsearch.action.update.UpdateAction;
|
||||
import org.elasticsearch.common.util.iterable.Iterables;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
|
@ -59,7 +59,7 @@ public class IndexPrivilegeTests extends ESTestCase {
|
|||
}
|
||||
|
||||
public void testFindPrivilegesThatGrant() {
|
||||
assertThat(findPrivilegesThatGrant(SearchAction.NAME), equalTo(List.of("read", "all")));
|
||||
assertThat(findPrivilegesThatGrant(TransportSearchAction.TYPE.name()), equalTo(List.of("read", "all")));
|
||||
assertThat(findPrivilegesThatGrant(IndexAction.NAME), equalTo(List.of("create_doc", "create", "index", "write", "all")));
|
||||
assertThat(findPrivilegesThatGrant(UpdateAction.NAME), equalTo(List.of("index", "write", "all")));
|
||||
assertThat(findPrivilegesThatGrant(DeleteAction.NAME), equalTo(List.of("delete", "write", "all")));
|
||||
|
|
|
@ -52,9 +52,9 @@ import org.elasticsearch.action.ingest.DeletePipelineAction;
|
|||
import org.elasticsearch.action.ingest.GetPipelineAction;
|
||||
import org.elasticsearch.action.ingest.PutPipelineAction;
|
||||
import org.elasticsearch.action.ingest.SimulatePipelineAction;
|
||||
import org.elasticsearch.action.search.MultiSearchAction;
|
||||
import org.elasticsearch.action.search.SearchAction;
|
||||
import org.elasticsearch.action.search.SearchShardsAction;
|
||||
import org.elasticsearch.action.search.TransportMultiSearchAction;
|
||||
import org.elasticsearch.action.search.TransportSearchAction;
|
||||
import org.elasticsearch.action.search.TransportSearchShardsAction;
|
||||
import org.elasticsearch.action.support.PlainActionFuture;
|
||||
import org.elasticsearch.action.support.WriteRequest;
|
||||
import org.elasticsearch.action.update.UpdateAction;
|
||||
|
@ -615,8 +615,14 @@ public class ReservedRolesStoreTests extends ESTestCase {
|
|||
assertThat(kibanaRole.indices().allowedIndicesMatcher(IndexAction.NAME).test(mockIndexAbstraction(index)), is(false));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(DeleteAction.NAME).test(mockIndexAbstraction(index)), is(false));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(mockIndexAbstraction(index)), is(false));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(index)), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(MultiSearchAction.NAME).test(mockIndexAbstraction(index)), is(true));
|
||||
assertThat(
|
||||
kibanaRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction(index)),
|
||||
is(true)
|
||||
);
|
||||
assertThat(
|
||||
kibanaRole.indices().allowedIndicesMatcher(TransportMultiSearchAction.TYPE.name()).test(mockIndexAbstraction(index)),
|
||||
is(true)
|
||||
);
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(GetAction.NAME).test(mockIndexAbstraction(index)), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(READ_CROSS_CLUSTER_NAME).test(mockIndexAbstraction(index)), is(true));
|
||||
});
|
||||
|
@ -634,8 +640,14 @@ public class ReservedRolesStoreTests extends ESTestCase {
|
|||
assertThat(kibanaRole.indices().allowedIndicesMatcher(IndexAction.NAME).test(mockIndexAbstraction(index)), is(false));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(DeleteAction.NAME).test(mockIndexAbstraction(index)), is(false));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(mockIndexAbstraction(index)), is(false));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(index)), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(MultiSearchAction.NAME).test(mockIndexAbstraction(index)), is(true));
|
||||
assertThat(
|
||||
kibanaRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction(index)),
|
||||
is(true)
|
||||
);
|
||||
assertThat(
|
||||
kibanaRole.indices().allowedIndicesMatcher(TransportMultiSearchAction.TYPE.name()).test(mockIndexAbstraction(index)),
|
||||
is(true)
|
||||
);
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(GetAction.NAME).test(mockIndexAbstraction(index)), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(READ_CROSS_CLUSTER_NAME).test(mockIndexAbstraction(index)), is(false));
|
||||
});
|
||||
|
@ -653,8 +665,14 @@ public class ReservedRolesStoreTests extends ESTestCase {
|
|||
assertThat(kibanaRole.indices().allowedIndicesMatcher(IndexAction.NAME).test(mockIndexAbstraction(index)), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(DeleteAction.NAME).test(mockIndexAbstraction(index)), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(mockIndexAbstraction(index)), is(false));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(index)), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(MultiSearchAction.NAME).test(mockIndexAbstraction(index)), is(true));
|
||||
assertThat(
|
||||
kibanaRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction(index)),
|
||||
is(true)
|
||||
);
|
||||
assertThat(
|
||||
kibanaRole.indices().allowedIndicesMatcher(TransportMultiSearchAction.TYPE.name()).test(mockIndexAbstraction(index)),
|
||||
is(true)
|
||||
);
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(GetAction.NAME).test(mockIndexAbstraction(index)), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(READ_CROSS_CLUSTER_NAME).test(mockIndexAbstraction(index)), is(false));
|
||||
});
|
||||
|
@ -669,8 +687,14 @@ public class ReservedRolesStoreTests extends ESTestCase {
|
|||
assertThat(kibanaRole.indices().allowedIndicesMatcher(IndexAction.NAME).test(mockIndexAbstraction(index)), is(false));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(DeleteAction.NAME).test(mockIndexAbstraction(index)), is(false));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(mockIndexAbstraction(index)), is(false));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(index)), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(MultiSearchAction.NAME).test(mockIndexAbstraction(index)), is(true));
|
||||
assertThat(
|
||||
kibanaRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction(index)),
|
||||
is(true)
|
||||
);
|
||||
assertThat(
|
||||
kibanaRole.indices().allowedIndicesMatcher(TransportMultiSearchAction.TYPE.name()).test(mockIndexAbstraction(index)),
|
||||
is(true)
|
||||
);
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(GetAction.NAME).test(mockIndexAbstraction(index)), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(READ_CROSS_CLUSTER_NAME).test(mockIndexAbstraction(index)), is(true));
|
||||
});
|
||||
|
@ -688,8 +712,14 @@ public class ReservedRolesStoreTests extends ESTestCase {
|
|||
assertThat(kibanaRole.indices().allowedIndicesMatcher(CreateIndexAction.NAME).test(mockIndexAbstraction(index)), is(false));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(IndexAction.NAME).test(mockIndexAbstraction(index)), is(false));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(DeleteAction.NAME).test(mockIndexAbstraction(index)), is(false));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(index)), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(MultiSearchAction.NAME).test(mockIndexAbstraction(index)), is(true));
|
||||
assertThat(
|
||||
kibanaRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction(index)),
|
||||
is(true)
|
||||
);
|
||||
assertThat(
|
||||
kibanaRole.indices().allowedIndicesMatcher(TransportMultiSearchAction.TYPE.name()).test(mockIndexAbstraction(index)),
|
||||
is(true)
|
||||
);
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(GetAction.NAME).test(mockIndexAbstraction(index)), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(READ_CROSS_CLUSTER_NAME).test(mockIndexAbstraction(index)), is(true));
|
||||
});
|
||||
|
@ -702,8 +732,14 @@ public class ReservedRolesStoreTests extends ESTestCase {
|
|||
assertThat(kibanaRole.indices().allowedIndicesMatcher(CreateIndexAction.NAME).test(mockIndexAbstraction(index)), is(false));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(IndexAction.NAME).test(mockIndexAbstraction(index)), is(false));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(DeleteAction.NAME).test(mockIndexAbstraction(index)), is(false));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(index)), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(MultiSearchAction.NAME).test(mockIndexAbstraction(index)), is(true));
|
||||
assertThat(
|
||||
kibanaRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction(index)),
|
||||
is(true)
|
||||
);
|
||||
assertThat(
|
||||
kibanaRole.indices().allowedIndicesMatcher(TransportMultiSearchAction.TYPE.name()).test(mockIndexAbstraction(index)),
|
||||
is(true)
|
||||
);
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(GetAction.NAME).test(mockIndexAbstraction(index)), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(READ_CROSS_CLUSTER_NAME).test(mockIndexAbstraction(index)), is(false));
|
||||
|
||||
|
@ -725,8 +761,8 @@ public class ReservedRolesStoreTests extends ESTestCase {
|
|||
assertThat(kibanaRole.indices().allowedIndicesMatcher(CreateIndexAction.NAME).test(indexAbstraction), is(false));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(IndexAction.NAME).test(indexAbstraction), is(false));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(DeleteAction.NAME).test(indexAbstraction), is(false));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(MultiSearchAction.NAME).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(TransportMultiSearchAction.TYPE.name()).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(GetAction.NAME).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(READ_CROSS_CLUSTER_NAME).test(indexAbstraction), is(false));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(indexAbstraction), is(true));
|
||||
|
@ -744,8 +780,8 @@ public class ReservedRolesStoreTests extends ESTestCase {
|
|||
assertThat(kibanaRole.indices().allowedIndicesMatcher(CreateIndexAction.NAME).test(indexAbstraction), is(false));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(IndexAction.NAME).test(indexAbstraction), is(false));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(DeleteAction.NAME).test(indexAbstraction), is(false));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(MultiSearchAction.NAME).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(TransportMultiSearchAction.TYPE.name()).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(GetAction.NAME).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(READ_CROSS_CLUSTER_NAME).test(indexAbstraction), is(false));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(indexAbstraction), is(true));
|
||||
|
@ -770,8 +806,11 @@ public class ReservedRolesStoreTests extends ESTestCase {
|
|||
assertThat(kibanaRole.indices().allowedIndicesMatcher(CreateIndexAction.NAME).test(dotFleetSecretsIndex), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(IndexAction.NAME).test(dotFleetSecretsIndex), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(DeleteAction.NAME).test(dotFleetSecretsIndex), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(dotFleetSecretsIndex), is(false));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(MultiSearchAction.NAME).test(dotFleetSecretsIndex), is(false));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(dotFleetSecretsIndex), is(false));
|
||||
assertThat(
|
||||
kibanaRole.indices().allowedIndicesMatcher(TransportMultiSearchAction.TYPE.name()).test(dotFleetSecretsIndex),
|
||||
is(false)
|
||||
);
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(GetAction.NAME).test(dotFleetSecretsIndex), is(false));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(dotFleetSecretsIndex), is(false));
|
||||
|
||||
|
@ -787,8 +826,14 @@ public class ReservedRolesStoreTests extends ESTestCase {
|
|||
assertThat(kibanaRole.indices().allowedIndicesMatcher(CreateIndexAction.NAME).test(mockIndexAbstraction(index)), is(false));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(IndexAction.NAME).test(mockIndexAbstraction(index)), is(false));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(DeleteAction.NAME).test(mockIndexAbstraction(index)), is(false));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(index)), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(MultiSearchAction.NAME).test(mockIndexAbstraction(index)), is(true));
|
||||
assertThat(
|
||||
kibanaRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction(index)),
|
||||
is(true)
|
||||
);
|
||||
assertThat(
|
||||
kibanaRole.indices().allowedIndicesMatcher(TransportMultiSearchAction.TYPE.name()).test(mockIndexAbstraction(index)),
|
||||
is(true)
|
||||
);
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(GetAction.NAME).test(mockIndexAbstraction(index)), is(true));
|
||||
});
|
||||
|
||||
|
@ -802,8 +847,8 @@ public class ReservedRolesStoreTests extends ESTestCase {
|
|||
assertThat(kibanaRole.indices().allowedIndicesMatcher(CreateIndexAction.NAME).test(indexAbstraction), is(false));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(IndexAction.NAME).test(indexAbstraction), is(false));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(DeleteAction.NAME).test(indexAbstraction), is(false));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(MultiSearchAction.NAME).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(TransportMultiSearchAction.TYPE.name()).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(GetAction.NAME).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(READ_CROSS_CLUSTER_NAME).test(indexAbstraction), is(false));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(indexAbstraction), is(true));
|
||||
|
@ -820,8 +865,8 @@ public class ReservedRolesStoreTests extends ESTestCase {
|
|||
assertThat(kibanaRole.indices().allowedIndicesMatcher(CreateIndexAction.NAME).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(IndexAction.NAME).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(DeleteAction.NAME).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(MultiSearchAction.NAME).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(TransportMultiSearchAction.TYPE.name()).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(GetAction.NAME).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(READ_CROSS_CLUSTER_NAME).test(indexAbstraction), is(false));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(indexAbstraction), is(true));
|
||||
|
@ -839,8 +884,8 @@ public class ReservedRolesStoreTests extends ESTestCase {
|
|||
assertThat(kibanaRole.indices().allowedIndicesMatcher(CreateIndexAction.NAME).test(indexAbstraction), is(false));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(IndexAction.NAME).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(DeleteAction.NAME).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(MultiSearchAction.NAME).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(TransportMultiSearchAction.TYPE.name()).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(GetAction.NAME).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(READ_CROSS_CLUSTER_NAME).test(indexAbstraction), is(false));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(indexAbstraction), is(true));
|
||||
|
@ -858,8 +903,8 @@ public class ReservedRolesStoreTests extends ESTestCase {
|
|||
assertThat(kibanaRole.indices().allowedIndicesMatcher(CreateIndexAction.NAME).test(indexAbstraction), is(false));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(IndexAction.NAME).test(indexAbstraction), is(false));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(DeleteAction.NAME).test(indexAbstraction), is(false));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(MultiSearchAction.NAME).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(TransportMultiSearchAction.TYPE.name()).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(GetAction.NAME).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(READ_CROSS_CLUSTER_NAME).test(indexAbstraction), is(false));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(indexAbstraction), is(true));
|
||||
|
@ -880,8 +925,14 @@ public class ReservedRolesStoreTests extends ESTestCase {
|
|||
assertThat(kibanaRole.indices().allowedIndicesMatcher(IndexAction.NAME).test(mockIndexAbstraction(index)), is(false));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(DeleteAction.NAME).test(mockIndexAbstraction(index)), is(false));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(mockIndexAbstraction(index)), is(false));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(index)), is(false));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(MultiSearchAction.NAME).test(mockIndexAbstraction(index)), is(false));
|
||||
assertThat(
|
||||
kibanaRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction(index)),
|
||||
is(false)
|
||||
);
|
||||
assertThat(
|
||||
kibanaRole.indices().allowedIndicesMatcher(TransportMultiSearchAction.TYPE.name()).test(mockIndexAbstraction(index)),
|
||||
is(false)
|
||||
);
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(GetAction.NAME).test(mockIndexAbstraction(index)), is(false));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(READ_CROSS_CLUSTER_NAME).test(mockIndexAbstraction(index)), is(false));
|
||||
});
|
||||
|
@ -918,8 +969,8 @@ public class ReservedRolesStoreTests extends ESTestCase {
|
|||
assertThat(kibanaRole.indices().allowedIndicesMatcher(CreateIndexAction.NAME).test(indexAbstraction), is(false));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(IndexAction.NAME).test(indexAbstraction), is(false));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(DeleteAction.NAME).test(indexAbstraction), is(false));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(MultiSearchAction.NAME).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(TransportMultiSearchAction.TYPE.name()).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(GetAction.NAME).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(READ_CROSS_CLUSTER_NAME).test(indexAbstraction), is(false));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(indexAbstraction), is(true));
|
||||
|
@ -937,8 +988,8 @@ public class ReservedRolesStoreTests extends ESTestCase {
|
|||
assertThat(kibanaRole.indices().allowedIndicesMatcher(CreateIndexAction.NAME).test(indexAbstraction), is(false));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(IndexAction.NAME).test(indexAbstraction), is(false));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(DeleteAction.NAME).test(indexAbstraction), is(false));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(MultiSearchAction.NAME).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(TransportMultiSearchAction.TYPE.name()).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(GetAction.NAME).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(READ_CROSS_CLUSTER_NAME).test(indexAbstraction), is(false));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(indexAbstraction), is(true));
|
||||
|
@ -955,8 +1006,14 @@ public class ReservedRolesStoreTests extends ESTestCase {
|
|||
assertThat(kibanaRole.indices().allowedIndicesMatcher(IndexAction.NAME).test(mockIndexAbstraction(index)), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(DeleteAction.NAME).test(mockIndexAbstraction(index)), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(mockIndexAbstraction(index)), is(false));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(index)), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(MultiSearchAction.NAME).test(mockIndexAbstraction(index)), is(true));
|
||||
assertThat(
|
||||
kibanaRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction(index)),
|
||||
is(true)
|
||||
);
|
||||
assertThat(
|
||||
kibanaRole.indices().allowedIndicesMatcher(TransportMultiSearchAction.TYPE.name()).test(mockIndexAbstraction(index)),
|
||||
is(true)
|
||||
);
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(GetAction.NAME).test(mockIndexAbstraction(index)), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(READ_CROSS_CLUSTER_NAME).test(mockIndexAbstraction(index)), is(false));
|
||||
|
||||
|
@ -1029,8 +1086,14 @@ public class ReservedRolesStoreTests extends ESTestCase {
|
|||
|| indexName.startsWith(".logs-endpoint.heartbeat-")
|
||||
|| indexName.startsWith(".logs-osquery_manager.actions-");
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(GetAction.NAME).test(indexAbstraction), is(isAlsoReadIndex));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(indexAbstraction), is(isAlsoReadIndex));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(MultiSearchAction.NAME).test(indexAbstraction), is(isAlsoReadIndex));
|
||||
assertThat(
|
||||
kibanaRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(indexAbstraction),
|
||||
is(isAlsoReadIndex)
|
||||
);
|
||||
assertThat(
|
||||
kibanaRole.indices().allowedIndicesMatcher(TransportMultiSearchAction.TYPE.name()).test(indexAbstraction),
|
||||
is(isAlsoReadIndex)
|
||||
);
|
||||
|
||||
// Endpoint diagnostic, APM and Synthetics data streams also have an ILM policy with a delete action, all others should not.
|
||||
final boolean isAlsoIlmDeleteIndex = indexName.startsWith(".logs-endpoint.diagnostic.collection-")
|
||||
|
@ -1078,7 +1141,7 @@ public class ReservedRolesStoreTests extends ESTestCase {
|
|||
logger.info("index name [{}]", indexName);
|
||||
final IndexAbstraction indexAbstraction = mockIndexAbstraction(indexName);
|
||||
// Allow indexing
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(GetAction.NAME).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(IndexAction.NAME).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(UpdateAction.NAME).test(indexAbstraction), is(true));
|
||||
|
@ -1116,7 +1179,7 @@ public class ReservedRolesStoreTests extends ESTestCase {
|
|||
Arrays.asList("logs-ti_recordedfuture_latest.threat", "logs-ti_anomali_latest.threatstream").forEach(indexName -> {
|
||||
final IndexAbstraction indexAbstraction = mockIndexAbstraction(indexName);
|
||||
// Allow search and indexing
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(GetAction.NAME).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(IndexAction.NAME).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(UpdateAction.NAME).test(indexAbstraction), is(true));
|
||||
|
@ -1158,8 +1221,8 @@ public class ReservedRolesStoreTests extends ESTestCase {
|
|||
assertThat(kibanaRole.indices().allowedIndicesMatcher(PutMappingAction.NAME).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(RolloverAction.NAME).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(DeleteIndexAction.NAME).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(MultiSearchAction.NAME).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(TransportMultiSearchAction.TYPE.name()).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(GetAction.NAME).test(indexAbstraction), is(true));
|
||||
|
||||
// Implied by the overall view_index_metadata and monitor privilege
|
||||
|
@ -1177,7 +1240,7 @@ public class ReservedRolesStoreTests extends ESTestCase {
|
|||
logger.info("index name [{}]", indexName);
|
||||
final IndexAbstraction indexAbstraction = mockIndexAbstraction(indexName);
|
||||
// Allow indexing
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(GetAction.NAME).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(IndexAction.NAME).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(UpdateAction.NAME).test(indexAbstraction), is(true));
|
||||
|
@ -1205,8 +1268,8 @@ public class ReservedRolesStoreTests extends ESTestCase {
|
|||
assertThat(kibanaRole.indices().allowedIndicesMatcher(CreateIndexAction.NAME).test(indexAbstraction), is(false));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(IndexAction.NAME).test(indexAbstraction), is(false));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(DeleteAction.NAME).test(indexAbstraction), is(false));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(MultiSearchAction.NAME).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(TransportMultiSearchAction.TYPE.name()).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(GetAction.NAME).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(READ_CROSS_CLUSTER_NAME).test(indexAbstraction), is(false));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(indexAbstraction), is(true));
|
||||
|
@ -1223,8 +1286,8 @@ public class ReservedRolesStoreTests extends ESTestCase {
|
|||
assertThat(kibanaRole.indices().allowedIndicesMatcher(CreateIndexAction.NAME).test(indexAbstraction), is(false));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(IndexAction.NAME).test(indexAbstraction), is(false));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(DeleteAction.NAME).test(indexAbstraction), is(false));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(MultiSearchAction.NAME).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(TransportMultiSearchAction.TYPE.name()).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(GetAction.NAME).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(READ_CROSS_CLUSTER_NAME).test(indexAbstraction), is(false));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(indexAbstraction), is(true));
|
||||
|
@ -1243,7 +1306,7 @@ public class ReservedRolesStoreTests extends ESTestCase {
|
|||
logger.info("index name [{}]", indexName);
|
||||
final IndexAbstraction indexAbstraction = mockIndexAbstraction(indexName);
|
||||
// Allow indexing
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(GetAction.NAME).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(IndexAction.NAME).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(UpdateAction.NAME).test(indexAbstraction), is(true));
|
||||
|
@ -1280,8 +1343,8 @@ public class ReservedRolesStoreTests extends ESTestCase {
|
|||
assertThat(kibanaRole.indices().allowedIndicesMatcher(CreateIndexAction.NAME).test(indexAbstraction), is(false));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(IndexAction.NAME).test(indexAbstraction), is(false));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(DeleteAction.NAME).test(indexAbstraction), is(false));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(MultiSearchAction.NAME).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(TransportMultiSearchAction.TYPE.name()).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(GetAction.NAME).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(READ_CROSS_CLUSTER_NAME).test(indexAbstraction), is(false));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(indexAbstraction), is(true));
|
||||
|
@ -1318,7 +1381,7 @@ public class ReservedRolesStoreTests extends ESTestCase {
|
|||
Arrays.asList("kibana_sample_data_ecommerce", "kibana_sample_data_ecommerce_transform" + randomInt()).forEach(indexName -> {
|
||||
final IndexAbstraction indexAbstraction = mockIndexAbstraction(indexName);
|
||||
// Allow search and indexing
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(GetAction.NAME).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(IndexAction.NAME).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(UpdateAction.NAME).test(indexAbstraction), is(true));
|
||||
|
@ -1344,7 +1407,7 @@ public class ReservedRolesStoreTests extends ESTestCase {
|
|||
Arrays.asList(".asset-criticality.asset-criticality-" + randomAlphaOfLength(randomIntBetween(0, 13))).forEach(indexName -> {
|
||||
final IndexAbstraction indexAbstraction = mockIndexAbstraction(indexName);
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(CreateIndexAction.NAME).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(indexAbstraction), is(true));
|
||||
assertThat(kibanaRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(indexAbstraction), is(true));
|
||||
assertViewIndexMetadata(kibanaRole, indexName);
|
||||
});
|
||||
}
|
||||
|
@ -1522,12 +1585,18 @@ public class ReservedRolesStoreTests extends ESTestCase {
|
|||
|
||||
assertThat(monitoringUserRole.runAs().check(randomAlphaOfLengthBetween(1, 12)), is(false));
|
||||
|
||||
assertThat(monitoringUserRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction("foo")), is(false));
|
||||
assertThat(
|
||||
monitoringUserRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(".reporting")),
|
||||
monitoringUserRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction("foo")),
|
||||
is(false)
|
||||
);
|
||||
assertThat(
|
||||
monitoringUserRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction(".reporting")),
|
||||
is(false)
|
||||
);
|
||||
assertThat(
|
||||
monitoringUserRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction(".kibana")),
|
||||
is(false)
|
||||
);
|
||||
assertThat(monitoringUserRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(".kibana")), is(false));
|
||||
assertThat(
|
||||
monitoringUserRole.indices().allowedIndicesMatcher("indices:foo").test(mockIndexAbstraction(randomAlphaOfLengthBetween(8, 24))),
|
||||
is(false)
|
||||
|
@ -1556,7 +1625,10 @@ public class ReservedRolesStoreTests extends ESTestCase {
|
|||
monitoringUserRole.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(mockIndexAbstraction(index)),
|
||||
is(false)
|
||||
);
|
||||
assertThat(monitoringUserRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(index)), is(true));
|
||||
assertThat(
|
||||
monitoringUserRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction(index)),
|
||||
is(true)
|
||||
);
|
||||
assertThat(monitoringUserRole.indices().allowedIndicesMatcher(GetAction.NAME).test(mockIndexAbstraction(index)), is(true));
|
||||
assertThat(monitoringUserRole.indices().allowedIndicesMatcher(READ_CROSS_CLUSTER_NAME).test(mockIndexAbstraction(index)), is(true));
|
||||
|
||||
|
@ -1651,15 +1723,19 @@ public class ReservedRolesStoreTests extends ESTestCase {
|
|||
assertThat(remoteMonitoringAgentRole.runAs().check(randomAlphaOfLengthBetween(1, 12)), is(false));
|
||||
|
||||
assertThat(
|
||||
remoteMonitoringAgentRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction("foo")),
|
||||
remoteMonitoringAgentRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction("foo")),
|
||||
is(false)
|
||||
);
|
||||
assertThat(
|
||||
remoteMonitoringAgentRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(".reporting")),
|
||||
remoteMonitoringAgentRole.indices()
|
||||
.allowedIndicesMatcher(TransportSearchAction.TYPE.name())
|
||||
.test(mockIndexAbstraction(".reporting")),
|
||||
is(false)
|
||||
);
|
||||
assertThat(
|
||||
remoteMonitoringAgentRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(".kibana")),
|
||||
remoteMonitoringAgentRole.indices()
|
||||
.allowedIndicesMatcher(TransportSearchAction.TYPE.name())
|
||||
.test(mockIndexAbstraction(".kibana")),
|
||||
is(false)
|
||||
);
|
||||
assertThat(
|
||||
|
@ -1701,7 +1777,9 @@ public class ReservedRolesStoreTests extends ESTestCase {
|
|||
is(true)
|
||||
);
|
||||
assertThat(
|
||||
remoteMonitoringAgentRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(monitoringIndex)),
|
||||
remoteMonitoringAgentRole.indices()
|
||||
.allowedIndicesMatcher(TransportSearchAction.TYPE.name())
|
||||
.test(mockIndexAbstraction(monitoringIndex)),
|
||||
is(true)
|
||||
);
|
||||
assertThat(
|
||||
|
@ -1775,7 +1853,9 @@ public class ReservedRolesStoreTests extends ESTestCase {
|
|||
is(false)
|
||||
);
|
||||
assertThat(
|
||||
remoteMonitoringAgentRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(metricbeatIndex)),
|
||||
remoteMonitoringAgentRole.indices()
|
||||
.allowedIndicesMatcher(TransportSearchAction.TYPE.name())
|
||||
.test(mockIndexAbstraction(metricbeatIndex)),
|
||||
is(false)
|
||||
);
|
||||
assertThat(
|
||||
|
@ -1818,15 +1898,21 @@ public class ReservedRolesStoreTests extends ESTestCase {
|
|||
is(true)
|
||||
);
|
||||
assertThat(
|
||||
remoteMonitoringCollectorRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction("foo")),
|
||||
remoteMonitoringCollectorRole.indices()
|
||||
.allowedIndicesMatcher(TransportSearchAction.TYPE.name())
|
||||
.test(mockIndexAbstraction("foo")),
|
||||
is(false)
|
||||
);
|
||||
assertThat(
|
||||
remoteMonitoringCollectorRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(".reporting")),
|
||||
remoteMonitoringCollectorRole.indices()
|
||||
.allowedIndicesMatcher(TransportSearchAction.TYPE.name())
|
||||
.test(mockIndexAbstraction(".reporting")),
|
||||
is(false)
|
||||
);
|
||||
assertThat(
|
||||
remoteMonitoringCollectorRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(".kibana")),
|
||||
remoteMonitoringCollectorRole.indices()
|
||||
.allowedIndicesMatcher(TransportSearchAction.TYPE.name())
|
||||
.test(mockIndexAbstraction(".kibana")),
|
||||
is(true)
|
||||
);
|
||||
assertThat(
|
||||
|
@ -1888,7 +1974,9 @@ public class ReservedRolesStoreTests extends ESTestCase {
|
|||
is(false)
|
||||
);
|
||||
assertThat(
|
||||
remoteMonitoringCollectorRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(index)),
|
||||
remoteMonitoringCollectorRole.indices()
|
||||
.allowedIndicesMatcher(TransportSearchAction.TYPE.name())
|
||||
.test(mockIndexAbstraction(index)),
|
||||
is(false)
|
||||
);
|
||||
assertThat(
|
||||
|
@ -1966,13 +2054,13 @@ public class ReservedRolesStoreTests extends ESTestCase {
|
|||
|
||||
assertThat(
|
||||
remoteMonitoringCollectorRole.indices()
|
||||
.allowedIndicesMatcher(SearchAction.NAME)
|
||||
.allowedIndicesMatcher(TransportSearchAction.TYPE.name())
|
||||
.test(mockIndexAbstraction(randomFrom(TestRestrictedIndices.SAMPLE_RESTRICTED_NAMES))),
|
||||
is(false)
|
||||
);
|
||||
assertThat(
|
||||
remoteMonitoringCollectorRole.indices()
|
||||
.allowedIndicesMatcher(SearchAction.NAME)
|
||||
.allowedIndicesMatcher(TransportSearchAction.TYPE.name())
|
||||
.test(mockIndexAbstraction(XPackPlugin.ASYNC_RESULTS_INDEX + randomAlphaOfLengthBetween(0, 2))),
|
||||
is(false)
|
||||
);
|
||||
|
@ -2081,12 +2169,18 @@ public class ReservedRolesStoreTests extends ESTestCase {
|
|||
|
||||
assertThat(reportingUserRole.runAs().check(randomAlphaOfLengthBetween(1, 12)), is(false));
|
||||
|
||||
assertThat(reportingUserRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction("foo")), is(false));
|
||||
assertThat(
|
||||
reportingUserRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(".reporting")),
|
||||
reportingUserRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction("foo")),
|
||||
is(false)
|
||||
);
|
||||
assertThat(
|
||||
reportingUserRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction(".reporting")),
|
||||
is(false)
|
||||
);
|
||||
assertThat(
|
||||
reportingUserRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction(".kibana")),
|
||||
is(false)
|
||||
);
|
||||
assertThat(reportingUserRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(".kibana")), is(false));
|
||||
assertThat(
|
||||
reportingUserRole.indices().allowedIndicesMatcher("indices:foo").test(mockIndexAbstraction(randomAlphaOfLengthBetween(8, 24))),
|
||||
is(false)
|
||||
|
@ -2101,7 +2195,10 @@ public class ReservedRolesStoreTests extends ESTestCase {
|
|||
reportingUserRole.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(mockIndexAbstraction(index)),
|
||||
is(false)
|
||||
);
|
||||
assertThat(reportingUserRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(index)), is(false));
|
||||
assertThat(
|
||||
reportingUserRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction(index)),
|
||||
is(false)
|
||||
);
|
||||
assertThat(reportingUserRole.indices().allowedIndicesMatcher(GetAction.NAME).test(mockIndexAbstraction(index)), is(false));
|
||||
assertThat(reportingUserRole.indices().allowedIndicesMatcher(IndexAction.NAME).test(mockIndexAbstraction(index)), is(false));
|
||||
assertThat(reportingUserRole.indices().allowedIndicesMatcher(UpdateAction.NAME).test(mockIndexAbstraction(index)), is(false));
|
||||
|
@ -2170,7 +2267,7 @@ public class ReservedRolesStoreTests extends ESTestCase {
|
|||
FieldPermissionsCache fieldPermissionsCache = new FieldPermissionsCache(Settings.EMPTY);
|
||||
SortedMap<String, IndexAbstraction> lookup = metadata.getIndicesLookup();
|
||||
IndicesAccessControl iac = superuserRole.indices()
|
||||
.authorize(SearchAction.NAME, Sets.newHashSet("a1", "ba"), lookup, fieldPermissionsCache);
|
||||
.authorize(TransportSearchAction.TYPE.name(), Sets.newHashSet("a1", "ba"), lookup, fieldPermissionsCache);
|
||||
assertThat(iac.hasIndexPermissions("a1"), is(true));
|
||||
assertThat(iac.hasIndexPermissions("b"), is(true));
|
||||
iac = superuserRole.indices().authorize(DeleteIndexAction.NAME, Sets.newHashSet("a1", "ba"), lookup, fieldPermissionsCache);
|
||||
|
@ -2186,7 +2283,7 @@ public class ReservedRolesStoreTests extends ESTestCase {
|
|||
// Read security indices => allowed
|
||||
iac = superuserRole.indices()
|
||||
.authorize(
|
||||
randomFrom(SearchAction.NAME, GetIndexAction.NAME),
|
||||
randomFrom(TransportSearchAction.TYPE.name(), GetIndexAction.NAME),
|
||||
Sets.newHashSet(TestRestrictedIndices.SECURITY_MAIN_ALIAS),
|
||||
lookup,
|
||||
fieldPermissionsCache
|
||||
|
@ -2205,7 +2302,7 @@ public class ReservedRolesStoreTests extends ESTestCase {
|
|||
assertThat("For " + iac, iac.hasIndexPermissions(TestRestrictedIndices.SECURITY_MAIN_ALIAS), is(false));
|
||||
assertThat("For " + iac, iac.hasIndexPermissions(internalSecurityIndex), is(false));
|
||||
|
||||
assertTrue(superuserRole.indices().check(SearchAction.NAME));
|
||||
assertTrue(superuserRole.indices().check(TransportSearchAction.TYPE.name()));
|
||||
assertFalse(superuserRole.indices().check("unknown"));
|
||||
|
||||
assertThat(superuserRole.runAs().check(randomAlphaOfLengthBetween(1, 30)), is(true));
|
||||
|
@ -2320,8 +2417,14 @@ public class ReservedRolesStoreTests extends ESTestCase {
|
|||
assertThat(beatsAdminRole.indices().allowedIndicesMatcher(IndexAction.NAME).test(mockIndexAbstraction(index)), is(true));
|
||||
assertThat(beatsAdminRole.indices().allowedIndicesMatcher(DeleteAction.NAME).test(mockIndexAbstraction(index)), is(true));
|
||||
assertThat(beatsAdminRole.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(mockIndexAbstraction(index)), is(true));
|
||||
assertThat(beatsAdminRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(index)), is(true));
|
||||
assertThat(beatsAdminRole.indices().allowedIndicesMatcher(MultiSearchAction.NAME).test(mockIndexAbstraction(index)), is(true));
|
||||
assertThat(
|
||||
beatsAdminRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction(index)),
|
||||
is(true)
|
||||
);
|
||||
assertThat(
|
||||
beatsAdminRole.indices().allowedIndicesMatcher(TransportMultiSearchAction.TYPE.name()).test(mockIndexAbstraction(index)),
|
||||
is(true)
|
||||
);
|
||||
assertThat(beatsAdminRole.indices().allowedIndicesMatcher(GetAction.NAME).test(mockIndexAbstraction(index)), is(true));
|
||||
|
||||
assertNoAccessAllowed(beatsAdminRole, TestRestrictedIndices.SAMPLE_RESTRICTED_NAMES);
|
||||
|
@ -3228,8 +3331,11 @@ public class ReservedRolesStoreTests extends ESTestCase {
|
|||
assertThat(role.indices().allowedIndicesMatcher(IndexAction.NAME).test(mockIndexAbstraction(index)), is(true));
|
||||
assertThat(role.indices().allowedIndicesMatcher(DeleteAction.NAME).test(mockIndexAbstraction(index)), is(true));
|
||||
assertThat(role.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(mockIndexAbstraction(index)), is(true));
|
||||
assertThat(role.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(index)), is(true));
|
||||
assertThat(role.indices().allowedIndicesMatcher(MultiSearchAction.NAME).test(mockIndexAbstraction(index)), is(true));
|
||||
assertThat(role.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction(index)), is(true));
|
||||
assertThat(
|
||||
role.indices().allowedIndicesMatcher(TransportMultiSearchAction.TYPE.name()).test(mockIndexAbstraction(index)),
|
||||
is(true)
|
||||
);
|
||||
assertThat(role.indices().allowedIndicesMatcher(GetAction.NAME).test(mockIndexAbstraction(index)), is(true));
|
||||
// inherits from 'all'
|
||||
assertThat(role.indices().allowedIndicesMatcher(READ_CROSS_CLUSTER_NAME).test(mockIndexAbstraction(index)), is(true));
|
||||
|
@ -3237,7 +3343,7 @@ public class ReservedRolesStoreTests extends ESTestCase {
|
|||
|
||||
private void assertReadWriteDocsAndMaintenanceButNotDeleteIndexAllowed(Role role, String index) {
|
||||
assertThat(role.indices().allowedIndicesMatcher(DeleteIndexAction.NAME).test(mockIndexAbstraction(index)), is(false));
|
||||
assertThat(role.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(index)), is(true));
|
||||
assertThat(role.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction(index)), is(true));
|
||||
assertThat(role.indices().allowedIndicesMatcher(GetAction.NAME).test(mockIndexAbstraction(index)), is(true));
|
||||
assertThat(role.indices().allowedIndicesMatcher(IndexAction.NAME).test(mockIndexAbstraction(index)), is(true));
|
||||
assertThat(role.indices().allowedIndicesMatcher(UpdateAction.NAME).test(mockIndexAbstraction(index)), is(true));
|
||||
|
@ -3251,7 +3357,7 @@ public class ReservedRolesStoreTests extends ESTestCase {
|
|||
|
||||
private void assertReadWriteDocsButNotDeleteIndexAllowed(Role role, String index) {
|
||||
assertThat(role.indices().allowedIndicesMatcher(DeleteIndexAction.NAME).test(mockIndexAbstraction(index)), is(false));
|
||||
assertThat(role.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(index)), is(true));
|
||||
assertThat(role.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction(index)), is(true));
|
||||
assertThat(role.indices().allowedIndicesMatcher(GetAction.NAME).test(mockIndexAbstraction(index)), is(true));
|
||||
assertThat(role.indices().allowedIndicesMatcher(IndexAction.NAME).test(mockIndexAbstraction(index)), is(true));
|
||||
assertThat(role.indices().allowedIndicesMatcher(UpdateAction.NAME).test(mockIndexAbstraction(index)), is(true));
|
||||
|
@ -3263,7 +3369,7 @@ public class ReservedRolesStoreTests extends ESTestCase {
|
|||
assertThat(role.indices().allowedIndicesMatcher(DeleteIndexAction.NAME).test(mockIndexAbstraction(index)), is(false));
|
||||
assertThat(role.indices().allowedIndicesMatcher(CreateIndexAction.NAME).test(mockIndexAbstraction(index)), is(false));
|
||||
assertThat(role.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(mockIndexAbstraction(index)), is(false));
|
||||
assertThat(role.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(index)), is(true));
|
||||
assertThat(role.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction(index)), is(true));
|
||||
assertThat(role.indices().allowedIndicesMatcher(GetAction.NAME).test(mockIndexAbstraction(index)), is(true));
|
||||
assertThat(role.indices().allowedIndicesMatcher(IndexAction.NAME).test(mockIndexAbstraction(index)), is(false));
|
||||
assertThat(role.indices().allowedIndicesMatcher(UpdateAction.NAME).test(mockIndexAbstraction(index)), is(false));
|
||||
|
@ -3281,7 +3387,7 @@ public class ReservedRolesStoreTests extends ESTestCase {
|
|||
GetFieldMappingsAction.NAME + "*",
|
||||
GetMappingsAction.NAME,
|
||||
ClusterSearchShardsAction.NAME,
|
||||
SearchShardsAction.NAME,
|
||||
TransportSearchShardsAction.TYPE.name(),
|
||||
ValidateQueryAction.NAME + "*",
|
||||
GetSettingsAction.NAME,
|
||||
ExplainLifecycleAction.NAME,
|
||||
|
@ -3302,7 +3408,7 @@ public class ReservedRolesStoreTests extends ESTestCase {
|
|||
assertThat(role.indices().allowedIndicesMatcher(DeleteIndexAction.NAME).test(mockIndexAbstraction(index)), is(false));
|
||||
assertThat(role.indices().allowedIndicesMatcher(CreateIndexAction.NAME).test(mockIndexAbstraction(index)), is(false));
|
||||
assertThat(role.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(mockIndexAbstraction(index)), is(false));
|
||||
assertThat(role.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(index)), is(false));
|
||||
assertThat(role.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction(index)), is(false));
|
||||
assertThat(role.indices().allowedIndicesMatcher(GetAction.NAME).test(mockIndexAbstraction(index)), is(false));
|
||||
assertThat(role.indices().allowedIndicesMatcher(IndexAction.NAME).test(mockIndexAbstraction(index)), is(false));
|
||||
assertThat(role.indices().allowedIndicesMatcher(UpdateAction.NAME).test(mockIndexAbstraction(index)), is(false));
|
||||
|
@ -3350,8 +3456,14 @@ public class ReservedRolesStoreTests extends ESTestCase {
|
|||
assertThat(logstashAdminRole.indices().allowedIndicesMatcher(CreateIndexAction.NAME).test(mockIndexAbstraction(index)), is(true));
|
||||
assertThat(logstashAdminRole.indices().allowedIndicesMatcher(IndexAction.NAME).test(mockIndexAbstraction(index)), is(true));
|
||||
assertThat(logstashAdminRole.indices().allowedIndicesMatcher(GetAction.NAME).test(mockIndexAbstraction(index)), is(true));
|
||||
assertThat(logstashAdminRole.indices().allowedIndicesMatcher(SearchAction.NAME).test(mockIndexAbstraction(index)), is(true));
|
||||
assertThat(logstashAdminRole.indices().allowedIndicesMatcher(MultiSearchAction.NAME).test(mockIndexAbstraction(index)), is(true));
|
||||
assertThat(
|
||||
logstashAdminRole.indices().allowedIndicesMatcher(TransportSearchAction.TYPE.name()).test(mockIndexAbstraction(index)),
|
||||
is(true)
|
||||
);
|
||||
assertThat(
|
||||
logstashAdminRole.indices().allowedIndicesMatcher(TransportMultiSearchAction.TYPE.name()).test(mockIndexAbstraction(index)),
|
||||
is(true)
|
||||
);
|
||||
assertThat(
|
||||
logstashAdminRole.indices().allowedIndicesMatcher(UpdateSettingsAction.NAME).test(mockIndexAbstraction(index)),
|
||||
is(true)
|
||||
|
|
|
@ -8,9 +8,9 @@
|
|||
package org.elasticsearch.xpack.application.search.action;
|
||||
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.search.SearchAction;
|
||||
import org.elasticsearch.action.search.SearchRequest;
|
||||
import org.elasticsearch.action.search.SearchResponse;
|
||||
import org.elasticsearch.action.search.TransportSearchAction;
|
||||
import org.elasticsearch.action.support.ActionFilters;
|
||||
import org.elasticsearch.action.support.HandledTransportAction;
|
||||
import org.elasticsearch.client.internal.Client;
|
||||
|
@ -68,7 +68,7 @@ public class TransportQuerySearchApplicationAction extends HandledTransportActio
|
|||
SearchRequest searchRequest = new SearchRequest(searchApplication.name()).source(sourceBuilder);
|
||||
|
||||
client.execute(
|
||||
SearchAction.INSTANCE,
|
||||
TransportSearchAction.TYPE,
|
||||
searchRequest,
|
||||
listener.delegateFailure((l2, searchResponse) -> l2.onResponse(searchResponse))
|
||||
);
|
||||
|
|
|
@ -8,15 +8,15 @@
|
|||
package org.elasticsearch.xpack.eql.execution.search;
|
||||
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.search.ClosePointInTimeAction;
|
||||
import org.elasticsearch.action.search.ClosePointInTimeRequest;
|
||||
import org.elasticsearch.action.search.ClosePointInTimeResponse;
|
||||
import org.elasticsearch.action.search.MultiSearchRequest;
|
||||
import org.elasticsearch.action.search.MultiSearchResponse;
|
||||
import org.elasticsearch.action.search.OpenPointInTimeAction;
|
||||
import org.elasticsearch.action.search.OpenPointInTimeRequest;
|
||||
import org.elasticsearch.action.search.SearchRequest;
|
||||
import org.elasticsearch.action.search.SearchResponse;
|
||||
import org.elasticsearch.action.search.TransportClosePointInTimeAction;
|
||||
import org.elasticsearch.action.search.TransportOpenPointInTimeAction;
|
||||
import org.elasticsearch.common.Strings;
|
||||
import org.elasticsearch.common.util.CollectionUtils;
|
||||
import org.elasticsearch.core.TimeValue;
|
||||
|
@ -131,7 +131,7 @@ public class PITAwareQueryClient extends BasicQueryClient {
|
|||
private <Response> void openPIT(ActionListener<Response> listener, Runnable runnable) {
|
||||
OpenPointInTimeRequest request = new OpenPointInTimeRequest(indices).indicesOptions(IndexResolver.FIELD_CAPS_INDICES_OPTIONS)
|
||||
.keepAlive(keepAlive);
|
||||
client.execute(OpenPointInTimeAction.INSTANCE, request, listener.delegateFailureAndWrap((l, r) -> {
|
||||
client.execute(TransportOpenPointInTimeAction.TYPE, request, listener.delegateFailureAndWrap((l, r) -> {
|
||||
pitId = r.getPointInTimeId();
|
||||
runnable.run();
|
||||
}));
|
||||
|
@ -142,7 +142,7 @@ public class PITAwareQueryClient extends BasicQueryClient {
|
|||
// the pitId could be null as a consequence of a failure on openPIT
|
||||
if (pitId != null) {
|
||||
client.execute(
|
||||
ClosePointInTimeAction.INSTANCE,
|
||||
TransportClosePointInTimeAction.TYPE,
|
||||
new ClosePointInTimeRequest(pitId),
|
||||
map(listener, ClosePointInTimeResponse::isSucceeded)
|
||||
);
|
||||
|
|
|
@ -10,10 +10,10 @@ import org.elasticsearch.TransportVersion;
|
|||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.fieldcaps.FieldCapabilities;
|
||||
import org.elasticsearch.action.fieldcaps.FieldCapabilitiesResponse;
|
||||
import org.elasticsearch.action.search.SearchAction;
|
||||
import org.elasticsearch.action.search.SearchRequest;
|
||||
import org.elasticsearch.action.search.SearchRequestBuilder;
|
||||
import org.elasticsearch.action.search.SearchResponse;
|
||||
import org.elasticsearch.action.search.TransportSearchAction;
|
||||
import org.elasticsearch.client.internal.Client;
|
||||
import org.elasticsearch.cluster.ClusterName;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||
|
@ -211,7 +211,7 @@ public class CancellationTests extends ESTestCase {
|
|||
|
||||
// Emulation of search cancellation
|
||||
ArgumentCaptor<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 -> {
|
||||
@SuppressWarnings("unchecked")
|
||||
SearchRequest request = (SearchRequest) invocation.getArguments()[1];
|
||||
|
|
|
@ -10,10 +10,10 @@ package org.elasticsearch.xpack.esql.plugin;
|
|||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.ActionListenerResponseHandler;
|
||||
import org.elasticsearch.action.search.SearchRequest;
|
||||
import org.elasticsearch.action.search.SearchShardsAction;
|
||||
import org.elasticsearch.action.search.SearchShardsGroup;
|
||||
import org.elasticsearch.action.search.SearchShardsRequest;
|
||||
import org.elasticsearch.action.search.SearchShardsResponse;
|
||||
import org.elasticsearch.action.search.TransportSearchShardsAction;
|
||||
import org.elasticsearch.action.support.ContextPreservingActionListener;
|
||||
import org.elasticsearch.action.support.RefCountingListener;
|
||||
import org.elasticsearch.action.support.RefCountingRunnable;
|
||||
|
@ -393,7 +393,7 @@ public class ComputeService {
|
|||
);
|
||||
transportService.sendChildRequest(
|
||||
transportService.getLocalNode(),
|
||||
SearchShardsAction.NAME,
|
||||
TransportSearchShardsAction.TYPE.name(),
|
||||
searchShardsRequest,
|
||||
parentTask,
|
||||
TransportRequestOptions.EMPTY,
|
||||
|
|
|
@ -7,9 +7,9 @@
|
|||
|
||||
package org.elasticsearch.xpack.fleet.rest;
|
||||
|
||||
import org.elasticsearch.action.search.MultiSearchAction;
|
||||
import org.elasticsearch.action.search.MultiSearchRequest;
|
||||
import org.elasticsearch.action.search.SearchRequest;
|
||||
import org.elasticsearch.action.search.TransportMultiSearchAction;
|
||||
import org.elasticsearch.client.internal.node.NodeClient;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.core.TimeValue;
|
||||
|
@ -112,7 +112,7 @@ public class RestFleetMultiSearchAction extends BaseRestHandler {
|
|||
|
||||
return channel -> {
|
||||
final RestCancellableNodeClient cancellableClient = new RestCancellableNodeClient(client, request.getHttpChannel());
|
||||
cancellableClient.execute(MultiSearchAction.INSTANCE, multiSearchRequest, new RestChunkedToXContentListener<>(channel));
|
||||
cancellableClient.execute(TransportMultiSearchAction.TYPE, multiSearchRequest, new RestChunkedToXContentListener<>(channel));
|
||||
};
|
||||
}
|
||||
|
||||
|
|
|
@ -8,8 +8,8 @@
|
|||
package org.elasticsearch.xpack.fleet.rest;
|
||||
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.action.search.SearchAction;
|
||||
import org.elasticsearch.action.search.SearchRequest;
|
||||
import org.elasticsearch.action.search.TransportSearchAction;
|
||||
import org.elasticsearch.client.internal.node.NodeClient;
|
||||
import org.elasticsearch.common.Strings;
|
||||
import org.elasticsearch.core.TimeValue;
|
||||
|
@ -96,7 +96,7 @@ public class RestFleetSearchAction extends BaseRestHandler {
|
|||
|
||||
return channel -> {
|
||||
RestCancellableNodeClient cancelClient = new RestCancellableNodeClient(client, request.getHttpChannel());
|
||||
cancelClient.execute(SearchAction.INSTANCE, searchRequest, new RestChunkedToXContentListener<>(channel));
|
||||
cancelClient.execute(TransportSearchAction.TYPE, searchRequest, new RestChunkedToXContentListener<>(channel));
|
||||
};
|
||||
}
|
||||
|
||||
|
|
|
@ -9,11 +9,11 @@ package org.elasticsearch.index.engine.frozen;
|
|||
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.action.DocWriteResponse;
|
||||
import org.elasticsearch.action.search.ClosePointInTimeAction;
|
||||
import org.elasticsearch.action.search.ClosePointInTimeRequest;
|
||||
import org.elasticsearch.action.search.OpenPointInTimeAction;
|
||||
import org.elasticsearch.action.search.OpenPointInTimeRequest;
|
||||
import org.elasticsearch.action.search.SearchType;
|
||||
import org.elasticsearch.action.search.TransportClosePointInTimeAction;
|
||||
import org.elasticsearch.action.search.TransportOpenPointInTimeAction;
|
||||
import org.elasticsearch.action.support.ActiveShardCount;
|
||||
import org.elasticsearch.action.support.IndicesOptions;
|
||||
import org.elasticsearch.action.support.PlainActionFuture;
|
||||
|
@ -230,7 +230,7 @@ public class FrozenIndexIT extends ESIntegTestCase {
|
|||
final OpenPointInTimeRequest openPointInTimeRequest = new OpenPointInTimeRequest(indexName).indicesOptions(
|
||||
IndicesOptions.STRICT_EXPAND_OPEN_FORBID_CLOSED
|
||||
).keepAlive(TimeValue.timeValueMinutes(2));
|
||||
final String pitId = client().execute(OpenPointInTimeAction.INSTANCE, openPointInTimeRequest).actionGet().getPointInTimeId();
|
||||
final String pitId = client().execute(TransportOpenPointInTimeAction.TYPE, openPointInTimeRequest).actionGet().getPointInTimeId();
|
||||
try {
|
||||
assertNoFailuresAndResponse(
|
||||
prepareSearch().setIndices(indexName).setPreference(null).setPointInTime(new PointInTimeBuilder(pitId)),
|
||||
|
@ -257,7 +257,7 @@ public class FrozenIndexIT extends ESIntegTestCase {
|
|||
);
|
||||
} finally {
|
||||
assertAcked(client().execute(FreezeIndexAction.INSTANCE, new FreezeRequest(indexName).setFreeze(false)).actionGet());
|
||||
client().execute(ClosePointInTimeAction.INSTANCE, new ClosePointInTimeRequest(pitId)).actionGet();
|
||||
client().execute(TransportClosePointInTimeAction.TYPE, new ClosePointInTimeRequest(pitId)).actionGet();
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -282,7 +282,7 @@ public class FrozenIndexIT extends ESIntegTestCase {
|
|||
IndicesOptions.STRICT_EXPAND_OPEN_FORBID_CLOSED
|
||||
).keepAlive(TimeValue.timeValueMinutes(2));
|
||||
|
||||
final String pitId = client().execute(OpenPointInTimeAction.INSTANCE, openPointInTimeRequest).actionGet().getPointInTimeId();
|
||||
final String pitId = client().execute(TransportOpenPointInTimeAction.TYPE, openPointInTimeRequest).actionGet().getPointInTimeId();
|
||||
try {
|
||||
indicesAdmin().prepareDelete("index-1").get();
|
||||
// Return partial results if allow partial search result is allowed
|
||||
|
@ -299,7 +299,7 @@ public class FrozenIndexIT extends ESIntegTestCase {
|
|||
prepareSearch().setPreference(null).setAllowPartialSearchResults(false).setPointInTime(new PointInTimeBuilder(pitId))::get
|
||||
);
|
||||
} finally {
|
||||
client().execute(ClosePointInTimeAction.INSTANCE, new ClosePointInTimeRequest(pitId)).actionGet();
|
||||
client().execute(TransportClosePointInTimeAction.TYPE, new ClosePointInTimeRequest(pitId)).actionGet();
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -317,14 +317,16 @@ public class FrozenIndexIT extends ESIntegTestCase {
|
|||
final OpenPointInTimeRequest openPointInTimeRequest = new OpenPointInTimeRequest("test-*").indicesOptions(
|
||||
IndicesOptions.strictExpandOpenAndForbidClosed()
|
||||
).keepAlive(TimeValue.timeValueMinutes(2));
|
||||
final String pitId = client().execute(OpenPointInTimeAction.INSTANCE, openPointInTimeRequest).actionGet().getPointInTimeId();
|
||||
final String pitId = client().execute(TransportOpenPointInTimeAction.TYPE, openPointInTimeRequest)
|
||||
.actionGet()
|
||||
.getPointInTimeId();
|
||||
try {
|
||||
assertNoFailuresAndResponse(
|
||||
prepareSearch().setPreference(null).setPointInTime(new PointInTimeBuilder(pitId)),
|
||||
searchResponse -> assertHitCount(searchResponse, numDocs)
|
||||
);
|
||||
} finally {
|
||||
client().execute(ClosePointInTimeAction.INSTANCE, new ClosePointInTimeRequest(pitId)).actionGet();
|
||||
client().execute(TransportClosePointInTimeAction.TYPE, new ClosePointInTimeRequest(pitId)).actionGet();
|
||||
}
|
||||
}
|
||||
// exclude the frozen indices
|
||||
|
@ -332,11 +334,13 @@ public class FrozenIndexIT extends ESIntegTestCase {
|
|||
final OpenPointInTimeRequest openPointInTimeRequest = new OpenPointInTimeRequest("test-*").keepAlive(
|
||||
TimeValue.timeValueMinutes(2)
|
||||
);
|
||||
final String pitId = client().execute(OpenPointInTimeAction.INSTANCE, openPointInTimeRequest).actionGet().getPointInTimeId();
|
||||
final String pitId = client().execute(TransportOpenPointInTimeAction.TYPE, openPointInTimeRequest)
|
||||
.actionGet()
|
||||
.getPointInTimeId();
|
||||
try {
|
||||
assertHitCountAndNoFailures(prepareSearch().setPreference(null).setPointInTime(new PointInTimeBuilder(pitId)), 0);
|
||||
} finally {
|
||||
client().execute(ClosePointInTimeAction.INSTANCE, new ClosePointInTimeRequest(pitId)).actionGet();
|
||||
client().execute(TransportClosePointInTimeAction.TYPE, new ClosePointInTimeRequest(pitId)).actionGet();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -12,14 +12,14 @@ import org.elasticsearch.action.OriginalIndices;
|
|||
import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse;
|
||||
import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse;
|
||||
import org.elasticsearch.action.delete.DeleteResponse;
|
||||
import org.elasticsearch.action.search.ClosePointInTimeAction;
|
||||
import org.elasticsearch.action.search.ClosePointInTimeRequest;
|
||||
import org.elasticsearch.action.search.OpenPointInTimeAction;
|
||||
import org.elasticsearch.action.search.OpenPointInTimeRequest;
|
||||
import org.elasticsearch.action.search.OpenPointInTimeResponse;
|
||||
import org.elasticsearch.action.search.SearchRequest;
|
||||
import org.elasticsearch.action.search.SearchResponse;
|
||||
import org.elasticsearch.action.search.SearchType;
|
||||
import org.elasticsearch.action.search.TransportClosePointInTimeAction;
|
||||
import org.elasticsearch.action.search.TransportOpenPointInTimeAction;
|
||||
import org.elasticsearch.action.search.TransportSearchAction;
|
||||
import org.elasticsearch.action.support.IndicesOptions;
|
||||
import org.elasticsearch.action.support.PlainActionFuture;
|
||||
|
@ -86,7 +86,7 @@ public class FrozenIndexTests extends ESSingleNodeTestCase {
|
|||
String openReaders(TimeValue keepAlive, String... indices) {
|
||||
OpenPointInTimeRequest request = new OpenPointInTimeRequest(indices).indicesOptions(IndicesOptions.STRICT_EXPAND_OPEN_FORBID_CLOSED)
|
||||
.keepAlive(keepAlive);
|
||||
final OpenPointInTimeResponse response = client().execute(OpenPointInTimeAction.INSTANCE, request).actionGet();
|
||||
final OpenPointInTimeResponse response = client().execute(TransportOpenPointInTimeAction.TYPE, request).actionGet();
|
||||
return response.getPointInTimeId();
|
||||
}
|
||||
|
||||
|
@ -161,7 +161,7 @@ public class FrozenIndexTests extends ESSingleNodeTestCase {
|
|||
}
|
||||
assertWarnings(TransportSearchAction.FROZEN_INDICES_DEPRECATION_MESSAGE.replace("{}", indexName));
|
||||
} finally {
|
||||
client().execute(ClosePointInTimeAction.INSTANCE, new ClosePointInTimeRequest(pitId)).get();
|
||||
client().execute(TransportClosePointInTimeAction.TYPE, new ClosePointInTimeRequest(pitId)).get();
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -13,9 +13,9 @@ import org.elasticsearch.action.admin.indices.refresh.RefreshRequest;
|
|||
import org.elasticsearch.action.admin.indices.refresh.RefreshResponse;
|
||||
import org.elasticsearch.action.admin.indices.template.put.PutComposableIndexTemplateAction;
|
||||
import org.elasticsearch.action.datastreams.CreateDataStreamAction;
|
||||
import org.elasticsearch.action.search.SearchAction;
|
||||
import org.elasticsearch.action.search.SearchRequest;
|
||||
import org.elasticsearch.action.search.SearchRequestBuilder;
|
||||
import org.elasticsearch.action.search.TransportSearchAction;
|
||||
import org.elasticsearch.action.support.master.AcknowledgedResponse;
|
||||
import org.elasticsearch.client.internal.Client;
|
||||
import org.elasticsearch.cluster.ClusterModule;
|
||||
|
@ -311,12 +311,12 @@ abstract class MlNativeIntegTestCase extends ESIntegTestCase {
|
|||
RefreshResponse refreshResponse = client().execute(RefreshAction.INSTANCE, refreshRequest).actionGet();
|
||||
assertThat(refreshResponse.getStatus().getStatus(), anyOf(equalTo(200), equalTo(201)));
|
||||
|
||||
SearchRequest searchRequest = new SearchRequestBuilder(client(), SearchAction.INSTANCE).setIndices(
|
||||
SearchRequest searchRequest = new SearchRequestBuilder(client(), TransportSearchAction.TYPE).setIndices(
|
||||
NotificationsIndex.NOTIFICATIONS_INDEX
|
||||
).addSort("timestamp", SortOrder.ASC).setQuery(QueryBuilders.termQuery("job_id", jobId)).setSize(100).request();
|
||||
List<String> messages = new ArrayList<>();
|
||||
assertResponse(
|
||||
client().execute(SearchAction.INSTANCE, searchRequest),
|
||||
client().execute(TransportSearchAction.TYPE, searchRequest),
|
||||
searchResponse -> Arrays.stream(searchResponse.getHits().getHits())
|
||||
.map(hit -> (String) hit.getSourceAsMap().get("message"))
|
||||
.forEach(messages::add)
|
||||
|
|
|
@ -13,8 +13,8 @@ import org.elasticsearch.action.bulk.BulkRequest;
|
|||
import org.elasticsearch.action.bulk.BulkResponse;
|
||||
import org.elasticsearch.action.index.IndexAction;
|
||||
import org.elasticsearch.action.index.IndexRequest;
|
||||
import org.elasticsearch.action.search.SearchAction;
|
||||
import org.elasticsearch.action.search.SearchRequest;
|
||||
import org.elasticsearch.action.search.TransportSearchAction;
|
||||
import org.elasticsearch.action.support.PlainActionFuture;
|
||||
import org.elasticsearch.action.support.WriteRequest;
|
||||
import org.elasticsearch.action.support.master.MasterNodeRequest;
|
||||
|
@ -190,7 +190,7 @@ public class ModelSnapshotRetentionIT extends MlNativeAutodetectIntegTestCase {
|
|||
|
||||
private List<String> getDocIdsFromSearch(SearchRequest searchRequest) throws Exception {
|
||||
List<String> docIds = new ArrayList<>();
|
||||
assertResponse(client().execute(SearchAction.INSTANCE, searchRequest), searchResponse -> {
|
||||
assertResponse(client().execute(TransportSearchAction.TYPE, searchRequest), searchResponse -> {
|
||||
assertThat(searchResponse.getHits(), notNullValue());
|
||||
for (SearchHit searchHit : searchResponse.getHits().getHits()) {
|
||||
docIds.add(searchHit.getId());
|
||||
|
|
|
@ -13,9 +13,9 @@ import org.elasticsearch.ElasticsearchStatusException;
|
|||
import org.elasticsearch.ResourceNotFoundException;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.bulk.BulkItemResponse;
|
||||
import org.elasticsearch.action.search.SearchAction;
|
||||
import org.elasticsearch.action.search.SearchRequest;
|
||||
import org.elasticsearch.action.search.SearchResponse;
|
||||
import org.elasticsearch.action.search.TransportSearchAction;
|
||||
import org.elasticsearch.action.support.ActionFilters;
|
||||
import org.elasticsearch.action.support.HandledTransportAction;
|
||||
import org.elasticsearch.action.support.master.AcknowledgedResponse;
|
||||
|
@ -116,7 +116,7 @@ public class TransportDeleteForecastAction extends HandledTransportAction<Delete
|
|||
.query(query);
|
||||
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) {
|
||||
|
|
|
@ -7,8 +7,8 @@
|
|||
package org.elasticsearch.xpack.ml.action;
|
||||
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.search.SearchAction;
|
||||
import org.elasticsearch.action.search.SearchRequest;
|
||||
import org.elasticsearch.action.search.TransportSearchAction;
|
||||
import org.elasticsearch.action.support.ActionFilters;
|
||||
import org.elasticsearch.action.support.HandledTransportAction;
|
||||
import org.elasticsearch.client.internal.Client;
|
||||
|
@ -146,7 +146,7 @@ public class TransportEvaluateDataFrameAction extends HandledTransportAction<
|
|||
SearchRequest searchRequest = new SearchRequest(request.getIndices()).source(searchSourceBuilder);
|
||||
useSecondaryAuthIfAvailable(
|
||||
securityContext,
|
||||
() -> client.execute(SearchAction.INSTANCE, searchRequest, ActionListener.wrap(searchResponse -> {
|
||||
() -> client.execute(TransportSearchAction.TYPE, searchRequest, ActionListener.wrap(searchResponse -> {
|
||||
evaluation.process(searchResponse);
|
||||
if (evaluation.hasAllResults() == false) {
|
||||
add(nextTask());
|
||||
|
|
|
@ -11,10 +11,10 @@ import org.apache.logging.log4j.Logger;
|
|||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.FailedNodeException;
|
||||
import org.elasticsearch.action.TaskOperationFailure;
|
||||
import org.elasticsearch.action.search.MultiSearchAction;
|
||||
import org.elasticsearch.action.search.MultiSearchRequest;
|
||||
import org.elasticsearch.action.search.MultiSearchResponse;
|
||||
import org.elasticsearch.action.search.SearchRequest;
|
||||
import org.elasticsearch.action.search.TransportMultiSearchAction;
|
||||
import org.elasticsearch.action.support.ActionFilters;
|
||||
import org.elasticsearch.action.support.IndicesOptions;
|
||||
import org.elasticsearch.action.support.tasks.TransportTasksAction;
|
||||
|
@ -267,7 +267,7 @@ public class TransportGetDataFrameAnalyticsStatsAction extends TransportTasksAct
|
|||
executeAsyncWithOrigin(
|
||||
client,
|
||||
ML_ORIGIN,
|
||||
MultiSearchAction.INSTANCE,
|
||||
TransportMultiSearchAction.TYPE,
|
||||
multiSearchRequest,
|
||||
ActionListener.wrap(multiSearchResponse -> {
|
||||
MultiSearchResponse.Item[] itemResponses = multiSearchResponse.getResponses();
|
||||
|
|
|
@ -14,8 +14,8 @@ import org.elasticsearch.action.admin.cluster.node.stats.NodesStatsRequest;
|
|||
import org.elasticsearch.action.admin.cluster.node.stats.NodesStatsRequestParameters;
|
||||
import org.elasticsearch.action.admin.cluster.node.stats.NodesStatsResponse;
|
||||
import org.elasticsearch.action.admin.cluster.node.stats.TransportNodesStatsAction;
|
||||
import org.elasticsearch.action.search.SearchAction;
|
||||
import org.elasticsearch.action.search.SearchRequest;
|
||||
import org.elasticsearch.action.search.TransportSearchAction;
|
||||
import org.elasticsearch.action.support.ActionFilters;
|
||||
import org.elasticsearch.action.support.HandledTransportAction;
|
||||
import org.elasticsearch.client.internal.Client;
|
||||
|
@ -349,7 +349,7 @@ public class TransportGetTrainedModelsStatsAction extends HandledTransportAction
|
|||
.request();
|
||||
searchRequest.setParentTask(parentTaskId);
|
||||
|
||||
executeAsyncWithOrigin(client, ML_ORIGIN, SearchAction.INSTANCE, searchRequest, ActionListener.wrap(searchResponse -> {
|
||||
executeAsyncWithOrigin(client, ML_ORIGIN, TransportSearchAction.TYPE, searchRequest, ActionListener.wrap(searchResponse -> {
|
||||
Map<String, Long> totalDefinitionLengthByModelId = new HashMap<>();
|
||||
for (SearchHit hit : searchResponse.getHits().getHits()) {
|
||||
DocumentField modelIdField = hit.field(TrainedModelConfig.MODEL_ID.getPreferredName());
|
||||
|
|
|
@ -11,8 +11,8 @@ import org.apache.logging.log4j.Logger;
|
|||
import org.elasticsearch.ElasticsearchStatusException;
|
||||
import org.elasticsearch.ResourceAlreadyExistsException;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.search.SearchAction;
|
||||
import org.elasticsearch.action.search.SearchRequest;
|
||||
import org.elasticsearch.action.search.TransportSearchAction;
|
||||
import org.elasticsearch.action.support.ActionFilters;
|
||||
import org.elasticsearch.action.support.SubscribableListener;
|
||||
import org.elasticsearch.action.support.master.TransportMasterNodeAction;
|
||||
|
@ -429,7 +429,7 @@ public class TransportStartDataFrameAnalyticsAction extends TransportMasterNodeA
|
|||
startContext.config.getHeaders(),
|
||||
ClientHelper.ML_ORIGIN,
|
||||
parentTaskClient,
|
||||
SearchAction.INSTANCE,
|
||||
TransportSearchAction.TYPE,
|
||||
destEmptySearch,
|
||||
ActionListener.wrap(searchResponse -> {
|
||||
if (searchResponse.getHits().getTotalHits().value > 0) {
|
||||
|
|
|
@ -10,7 +10,7 @@ package org.elasticsearch.xpack.ml.datafeed;
|
|||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.search.SearchAction;
|
||||
import org.elasticsearch.action.search.TransportSearchAction;
|
||||
import org.elasticsearch.action.support.master.AcknowledgedResponse;
|
||||
import org.elasticsearch.client.internal.Client;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
|
@ -130,9 +130,9 @@ public final class DatafeedManager {
|
|||
|
||||
ActionListener<GetRollupIndexCapsAction.Response> getRollupIndexCapsActionHandler = ActionListener.wrap(response -> {
|
||||
if (response.getJobs().isEmpty()) { // This means no rollup indexes are in the config
|
||||
indicesPrivilegesBuilder.privileges(SearchAction.NAME);
|
||||
indicesPrivilegesBuilder.privileges(TransportSearchAction.TYPE.name());
|
||||
} else {
|
||||
indicesPrivilegesBuilder.privileges(SearchAction.NAME, RollupSearchAction.NAME);
|
||||
indicesPrivilegesBuilder.privileges(TransportSearchAction.TYPE.name(), RollupSearchAction.NAME);
|
||||
}
|
||||
if (indices.length == 0) {
|
||||
privResponseListener.onResponse(new HasPrivilegesResponse());
|
||||
|
@ -142,7 +142,7 @@ public final class DatafeedManager {
|
|||
}
|
||||
}, e -> {
|
||||
if (ExceptionsHelper.unwrapCause(e) instanceof IndexNotFoundException) {
|
||||
indicesPrivilegesBuilder.privileges(SearchAction.NAME);
|
||||
indicesPrivilegesBuilder.privileges(TransportSearchAction.TYPE.name());
|
||||
privRequest.indexPrivileges(indicesPrivilegesBuilder.build());
|
||||
client.execute(HasPrivilegesAction.INSTANCE, privRequest, privResponseListener);
|
||||
} else {
|
||||
|
|
|
@ -6,9 +6,9 @@
|
|||
*/
|
||||
package org.elasticsearch.xpack.ml.datafeed.delayeddatacheck;
|
||||
|
||||
import org.elasticsearch.action.search.SearchAction;
|
||||
import org.elasticsearch.action.search.SearchRequest;
|
||||
import org.elasticsearch.action.search.SearchResponse;
|
||||
import org.elasticsearch.action.search.TransportSearchAction;
|
||||
import org.elasticsearch.action.support.IndicesOptions;
|
||||
import org.elasticsearch.client.internal.Client;
|
||||
import org.elasticsearch.common.util.Maps;
|
||||
|
@ -134,7 +134,7 @@ public class DatafeedDelayedDataDetector implements DelayedDataDetector {
|
|||
|
||||
SearchRequest searchRequest = new SearchRequest(datafeedIndices).source(searchSourceBuilder).indicesOptions(indicesOptions);
|
||||
try (ThreadContext.StoredContext ignore = client.threadPool().getThreadContext().stashWithOrigin(ML_ORIGIN)) {
|
||||
SearchResponse response = client.execute(SearchAction.INSTANCE, searchRequest).actionGet();
|
||||
SearchResponse response = client.execute(TransportSearchAction.TYPE, searchRequest).actionGet();
|
||||
List<? extends Histogram.Bucket> buckets = ((Histogram) response.getAggregations().get(DATE_BUCKETS)).getBuckets();
|
||||
Map<Long, Long> hashMap = Maps.newMapWithExpectedSize(buckets.size());
|
||||
for (Histogram.Bucket bucket : buckets) {
|
||||
|
|
|
@ -6,8 +6,8 @@
|
|||
*/
|
||||
package org.elasticsearch.xpack.ml.datafeed.extractor.aggregation;
|
||||
|
||||
import org.elasticsearch.action.search.SearchAction;
|
||||
import org.elasticsearch.action.search.SearchRequestBuilder;
|
||||
import org.elasticsearch.action.search.TransportSearchAction;
|
||||
import org.elasticsearch.client.internal.Client;
|
||||
import org.elasticsearch.search.builder.SearchSourceBuilder;
|
||||
import org.elasticsearch.xpack.ml.datafeed.DatafeedTimingStatsReporter;
|
||||
|
@ -30,7 +30,7 @@ class AggregationDataExtractor extends AbstractAggregationDataExtractor<SearchRe
|
|||
|
||||
@Override
|
||||
protected SearchRequestBuilder buildSearchRequest(SearchSourceBuilder searchSourceBuilder) {
|
||||
return new SearchRequestBuilder(client, SearchAction.INSTANCE).setSource(searchSourceBuilder)
|
||||
return new SearchRequestBuilder(client, TransportSearchAction.TYPE).setSource(searchSourceBuilder)
|
||||
.setIndicesOptions(context.indicesOptions)
|
||||
.setAllowPartialSearchResults(false)
|
||||
.setIndices(context.indices);
|
||||
|
|
|
@ -6,8 +6,8 @@
|
|||
*/
|
||||
package org.elasticsearch.xpack.ml.datafeed.extractor.aggregation;
|
||||
|
||||
import org.elasticsearch.action.search.SearchAction;
|
||||
import org.elasticsearch.action.search.SearchRequestBuilder;
|
||||
import org.elasticsearch.action.search.TransportSearchAction;
|
||||
import org.elasticsearch.action.support.IndicesOptions;
|
||||
import org.elasticsearch.client.internal.Client;
|
||||
import org.elasticsearch.index.query.QueryBuilder;
|
||||
|
@ -30,7 +30,7 @@ public record AggregationDataExtractorFactory(
|
|||
) implements DataExtractorFactory {
|
||||
|
||||
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)
|
||||
.setAllowPartialSearchResults(false)
|
||||
.setIndices(indices);
|
||||
|
|
|
@ -9,10 +9,10 @@ package org.elasticsearch.xpack.ml.datafeed.extractor.chunked;
|
|||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
import org.elasticsearch.action.ActionRequestBuilder;
|
||||
import org.elasticsearch.action.search.SearchAction;
|
||||
import org.elasticsearch.action.search.SearchRequest;
|
||||
import org.elasticsearch.action.search.SearchRequestBuilder;
|
||||
import org.elasticsearch.action.search.SearchResponse;
|
||||
import org.elasticsearch.action.search.TransportSearchAction;
|
||||
import org.elasticsearch.client.internal.Client;
|
||||
import org.elasticsearch.search.aggregations.AggregationBuilders;
|
||||
import org.elasticsearch.search.aggregations.Aggregations;
|
||||
|
@ -279,7 +279,7 @@ public class ChunkedDataExtractor implements DataExtractor {
|
|||
}
|
||||
|
||||
private SearchRequestBuilder rangeSearchRequest() {
|
||||
return new SearchRequestBuilder(client, SearchAction.INSTANCE).setIndices(context.indices)
|
||||
return new SearchRequestBuilder(client, TransportSearchAction.TYPE).setIndices(context.indices)
|
||||
.setIndicesOptions(context.indicesOptions)
|
||||
.setSource(rangeSearchBuilder())
|
||||
.setAllowPartialSearchResults(false)
|
||||
|
|
|
@ -9,14 +9,14 @@ package org.elasticsearch.xpack.ml.datafeed.extractor.scroll;
|
|||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
import org.elasticsearch.ResourceNotFoundException;
|
||||
import org.elasticsearch.action.search.ClearScrollAction;
|
||||
import org.elasticsearch.action.search.ClearScrollRequest;
|
||||
import org.elasticsearch.action.search.SearchAction;
|
||||
import org.elasticsearch.action.search.SearchPhaseExecutionException;
|
||||
import org.elasticsearch.action.search.SearchRequestBuilder;
|
||||
import org.elasticsearch.action.search.SearchResponse;
|
||||
import org.elasticsearch.action.search.SearchScrollAction;
|
||||
import org.elasticsearch.action.search.SearchScrollRequestBuilder;
|
||||
import org.elasticsearch.action.search.TransportClearScrollAction;
|
||||
import org.elasticsearch.action.search.TransportSearchAction;
|
||||
import org.elasticsearch.action.search.TransportSearchScrollAction;
|
||||
import org.elasticsearch.client.internal.Client;
|
||||
import org.elasticsearch.common.io.stream.BytesStreamOutput;
|
||||
import org.elasticsearch.core.TimeValue;
|
||||
|
@ -154,7 +154,7 @@ class ScrollDataExtractor implements DataExtractor {
|
|||
.query(ExtractorUtils.wrapInTimeRangeQuery(context.query, context.extractedFields.timeField(), start, context.end))
|
||||
.runtimeMappings(context.runtimeMappings);
|
||||
|
||||
SearchRequestBuilder searchRequestBuilder = new SearchRequestBuilder(client, SearchAction.INSTANCE).setScroll(SCROLL_TIMEOUT)
|
||||
SearchRequestBuilder searchRequestBuilder = new SearchRequestBuilder(client, TransportSearchAction.TYPE).setScroll(SCROLL_TIMEOUT)
|
||||
.setIndices(context.indices)
|
||||
.setIndicesOptions(context.indicesOptions)
|
||||
.setAllowPartialSearchResults(false)
|
||||
|
@ -250,7 +250,9 @@ class ScrollDataExtractor implements DataExtractor {
|
|||
context.headers,
|
||||
ClientHelper.ML_ORIGIN,
|
||||
client,
|
||||
() -> new SearchScrollRequestBuilder(client, SearchScrollAction.INSTANCE).setScroll(SCROLL_TIMEOUT).setScrollId(scrollId).get()
|
||||
() -> new SearchScrollRequestBuilder(client, TransportSearchScrollAction.TYPE).setScroll(SCROLL_TIMEOUT)
|
||||
.setScrollId(scrollId)
|
||||
.get()
|
||||
);
|
||||
try {
|
||||
checkForSkippedClusters(searchResponse);
|
||||
|
@ -284,7 +286,7 @@ class ScrollDataExtractor implements DataExtractor {
|
|||
context.headers,
|
||||
ClientHelper.ML_ORIGIN,
|
||||
client,
|
||||
() -> client.execute(ClearScrollAction.INSTANCE, request).actionGet()
|
||||
() -> client.execute(TransportClearScrollAction.TYPE, request).actionGet()
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -13,9 +13,9 @@ import org.elasticsearch.action.ActionListener;
|
|||
import org.elasticsearch.action.DocWriteResponse;
|
||||
import org.elasticsearch.action.index.IndexAction;
|
||||
import org.elasticsearch.action.index.IndexRequest;
|
||||
import org.elasticsearch.action.search.SearchAction;
|
||||
import org.elasticsearch.action.search.SearchRequest;
|
||||
import org.elasticsearch.action.search.SearchResponse;
|
||||
import org.elasticsearch.action.search.TransportSearchAction;
|
||||
import org.elasticsearch.action.support.WriteRequest;
|
||||
import org.elasticsearch.client.internal.Client;
|
||||
import org.elasticsearch.client.internal.ParentTaskAssigningClient;
|
||||
|
@ -276,7 +276,7 @@ public class DataFrameAnalyticsTask extends LicensedAllocatedPersistentTask impl
|
|||
SearchRequest searchRequest = new SearchRequest(AnomalyDetectorsIndex.jobStateIndexPattern()).source(
|
||||
new SearchSourceBuilder().size(1).query(new IdsQueryBuilder().addIds(progressDocId))
|
||||
);
|
||||
executeAsyncWithOrigin(clientToUse, ML_ORIGIN, SearchAction.INSTANCE, searchRequest, searchFormerProgressDocListener);
|
||||
executeAsyncWithOrigin(clientToUse, ML_ORIGIN, TransportSearchAction.TYPE, searchRequest, searchFormerProgressDocListener);
|
||||
}, e -> {
|
||||
LOGGER.error(
|
||||
() -> format("[%s] cannot persist progress as an error occurred while updating task progress", taskParams.getId()),
|
||||
|
|
|
@ -9,9 +9,9 @@ package org.elasticsearch.xpack.ml.dataframe.extractor;
|
|||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.search.SearchAction;
|
||||
import org.elasticsearch.action.search.SearchRequestBuilder;
|
||||
import org.elasticsearch.action.search.SearchResponse;
|
||||
import org.elasticsearch.action.search.TransportSearchAction;
|
||||
import org.elasticsearch.client.internal.Client;
|
||||
import org.elasticsearch.common.util.CachedSupplier;
|
||||
import org.elasticsearch.core.Nullable;
|
||||
|
@ -127,7 +127,7 @@ public class DataFrameDataExtractor {
|
|||
*/
|
||||
public void preview(ActionListener<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
|
||||
.setAllowPartialSearchResults(false)
|
||||
.setIndices(context.indices)
|
||||
|
@ -146,7 +146,7 @@ public class DataFrameDataExtractor {
|
|||
context.headers,
|
||||
ClientHelper.ML_ORIGIN,
|
||||
client,
|
||||
SearchAction.INSTANCE,
|
||||
TransportSearchAction.TYPE,
|
||||
searchRequestBuilder.request(),
|
||||
ActionListener.wrap(searchResponse -> {
|
||||
if (searchResponse.getHits().getHits().length == 0) {
|
||||
|
@ -203,7 +203,7 @@ public class DataFrameDataExtractor {
|
|||
|
||||
LOGGER.trace(() -> format("[%s] Searching docs with [%s] in [%s, %s)", context.jobId, INCREMENTAL_ID, from, to));
|
||||
|
||||
SearchRequestBuilder searchRequestBuilder = new SearchRequestBuilder(client, SearchAction.INSTANCE)
|
||||
SearchRequestBuilder searchRequestBuilder = new SearchRequestBuilder(client, TransportSearchAction.TYPE)
|
||||
// This ensures the search throws if there are failures and the scroll context gets cleared automatically
|
||||
.setAllowPartialSearchResults(false)
|
||||
.addSort(DestinationIndex.INCREMENTAL_ID, SortOrder.ASC)
|
||||
|
@ -383,7 +383,7 @@ public class DataFrameDataExtractor {
|
|||
context.headers,
|
||||
ClientHelper.ML_ORIGIN,
|
||||
client,
|
||||
SearchAction.INSTANCE,
|
||||
TransportSearchAction.TYPE,
|
||||
searchRequestBuilder.request(),
|
||||
ActionListener.wrap(
|
||||
searchResponse -> dataSummaryActionListener.onResponse(
|
||||
|
@ -401,7 +401,7 @@ public class DataFrameDataExtractor {
|
|||
summaryQuery = QueryBuilders.boolQuery().filter(summaryQuery).filter(allExtractedFieldsExistQuery());
|
||||
}
|
||||
|
||||
return new SearchRequestBuilder(client, SearchAction.INSTANCE).setAllowPartialSearchResults(false)
|
||||
return new SearchRequestBuilder(client, TransportSearchAction.TYPE).setAllowPartialSearchResults(false)
|
||||
.setIndices(context.indices)
|
||||
.setSize(0)
|
||||
.setQuery(summaryQuery)
|
||||
|
|
|
@ -17,9 +17,9 @@ import org.elasticsearch.action.fieldcaps.FieldCapabilities;
|
|||
import org.elasticsearch.action.fieldcaps.FieldCapabilitiesAction;
|
||||
import org.elasticsearch.action.fieldcaps.FieldCapabilitiesRequest;
|
||||
import org.elasticsearch.action.fieldcaps.FieldCapabilitiesResponse;
|
||||
import org.elasticsearch.action.search.SearchAction;
|
||||
import org.elasticsearch.action.search.SearchRequest;
|
||||
import org.elasticsearch.action.search.SearchResponse;
|
||||
import org.elasticsearch.action.search.TransportSearchAction;
|
||||
import org.elasticsearch.action.support.IndicesOptions;
|
||||
import org.elasticsearch.client.internal.Client;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
|
@ -141,7 +141,14 @@ public class ExtractedFieldsDetectorFactory {
|
|||
);
|
||||
}
|
||||
SearchRequest searchRequest = new SearchRequest(index).source(searchSourceBuilder);
|
||||
ClientHelper.executeWithHeadersAsync(config.getHeaders(), ML_ORIGIN, client, SearchAction.INSTANCE, searchRequest, searchListener);
|
||||
ClientHelper.executeWithHeadersAsync(
|
||||
config.getHeaders(),
|
||||
ML_ORIGIN,
|
||||
client,
|
||||
TransportSearchAction.TYPE,
|
||||
searchRequest,
|
||||
searchListener
|
||||
);
|
||||
}
|
||||
|
||||
private static void buildFieldCardinalitiesMap(
|
||||
|
|
|
@ -12,8 +12,8 @@ import org.apache.logging.log4j.Logger;
|
|||
import org.elasticsearch.ResourceNotFoundException;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.admin.indices.refresh.RefreshResponse;
|
||||
import org.elasticsearch.action.search.SearchAction;
|
||||
import org.elasticsearch.action.search.SearchRequest;
|
||||
import org.elasticsearch.action.search.TransportSearchAction;
|
||||
import org.elasticsearch.client.internal.node.NodeClient;
|
||||
import org.elasticsearch.core.TimeValue;
|
||||
import org.elasticsearch.index.query.QueryBuilders;
|
||||
|
@ -122,7 +122,7 @@ public class InferenceStep extends AbstractDataFrameAnalyticsStep {
|
|||
executeAsyncWithOrigin(
|
||||
client,
|
||||
ML_ORIGIN,
|
||||
SearchAction.INSTANCE,
|
||||
TransportSearchAction.TYPE,
|
||||
searchRequest,
|
||||
ActionListener.wrap(
|
||||
searchResponse -> listener.onResponse(searchResponse.getHits().getTotalHits().value > 0),
|
||||
|
@ -142,7 +142,7 @@ public class InferenceStep extends AbstractDataFrameAnalyticsStep {
|
|||
SearchRequest searchRequest = new SearchRequest(InferenceIndexConstants.INDEX_PATTERN);
|
||||
searchRequest.source(searchSourceBuilder);
|
||||
|
||||
executeAsyncWithOrigin(client, ML_ORIGIN, SearchAction.INSTANCE, searchRequest, ActionListener.wrap(searchResponse -> {
|
||||
executeAsyncWithOrigin(client, ML_ORIGIN, TransportSearchAction.TYPE, searchRequest, ActionListener.wrap(searchResponse -> {
|
||||
SearchHit[] hits = searchResponse.getHits().getHits();
|
||||
if (hits.length == 0) {
|
||||
listener.onFailure(new ResourceNotFoundException("No model could be found to perform inference"));
|
||||
|
|
|
@ -12,8 +12,8 @@ import org.apache.logging.log4j.Logger;
|
|||
import org.apache.lucene.util.SetOnce;
|
||||
import org.elasticsearch.ResourceNotFoundException;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.search.SearchAction;
|
||||
import org.elasticsearch.action.search.SearchRequest;
|
||||
import org.elasticsearch.action.search.TransportSearchAction;
|
||||
import org.elasticsearch.action.support.master.AcknowledgedResponse;
|
||||
import org.elasticsearch.client.internal.Client;
|
||||
import org.elasticsearch.common.Strings;
|
||||
|
@ -174,29 +174,37 @@ public class DeploymentManager {
|
|||
task.init(nlpConfig);
|
||||
|
||||
SearchRequest searchRequest = vocabSearchRequest(nlpConfig.getVocabularyConfig(), modelConfig.getModelId());
|
||||
executeAsyncWithOrigin(client, ML_ORIGIN, SearchAction.INSTANCE, searchRequest, ActionListener.wrap(searchVocabResponse -> {
|
||||
if (searchVocabResponse.getHits().getHits().length == 0) {
|
||||
failedDeploymentListener.onFailure(
|
||||
new ResourceNotFoundException(
|
||||
Messages.getMessage(
|
||||
Messages.VOCABULARY_NOT_FOUND,
|
||||
modelConfig.getModelId(),
|
||||
VocabularyConfig.docId(modelConfig.getModelId())
|
||||
executeAsyncWithOrigin(
|
||||
client,
|
||||
ML_ORIGIN,
|
||||
TransportSearchAction.TYPE,
|
||||
searchRequest,
|
||||
ActionListener.wrap(searchVocabResponse -> {
|
||||
if (searchVocabResponse.getHits().getHits().length == 0) {
|
||||
failedDeploymentListener.onFailure(
|
||||
new ResourceNotFoundException(
|
||||
Messages.getMessage(
|
||||
Messages.VOCABULARY_NOT_FOUND,
|
||||
modelConfig.getModelId(),
|
||||
VocabularyConfig.docId(modelConfig.getModelId())
|
||||
)
|
||||
)
|
||||
)
|
||||
);
|
||||
return;
|
||||
}
|
||||
);
|
||||
return;
|
||||
}
|
||||
|
||||
Vocabulary vocabulary = parseVocabularyDocLeniently(searchVocabResponse.getHits().getAt(0));
|
||||
NlpTask nlpTask = new NlpTask(nlpConfig, vocabulary);
|
||||
NlpTask.Processor processor = nlpTask.createProcessor();
|
||||
processContext.nlpTaskProcessor.set(processor);
|
||||
// here, we are being called back on the searching thread, which MAY be a network thread
|
||||
// `startAndLoad` creates named pipes, blocking the calling thread, better to execute that in our utility
|
||||
// executor.
|
||||
executorServiceForDeployment.execute(() -> processContext.startAndLoad(modelConfig.getLocation(), modelLoadedListener));
|
||||
}, failedDeploymentListener::onFailure));
|
||||
Vocabulary vocabulary = parseVocabularyDocLeniently(searchVocabResponse.getHits().getAt(0));
|
||||
NlpTask nlpTask = new NlpTask(nlpConfig, vocabulary);
|
||||
NlpTask.Processor processor = nlpTask.createProcessor();
|
||||
processContext.nlpTaskProcessor.set(processor);
|
||||
// here, we are being called back on the searching thread, which MAY be a network thread
|
||||
// `startAndLoad` creates named pipes, blocking the calling thread, better to execute that in our utility
|
||||
// executor.
|
||||
executorServiceForDeployment.execute(
|
||||
() -> processContext.startAndLoad(modelConfig.getLocation(), modelLoadedListener)
|
||||
);
|
||||
}, failedDeploymentListener::onFailure)
|
||||
);
|
||||
} else {
|
||||
failedDeploymentListener.onFailure(
|
||||
new IllegalArgumentException(
|
||||
|
|
|
@ -25,9 +25,9 @@ import org.elasticsearch.action.index.IndexAction;
|
|||
import org.elasticsearch.action.index.IndexRequest;
|
||||
import org.elasticsearch.action.search.MultiSearchRequest;
|
||||
import org.elasticsearch.action.search.MultiSearchResponse;
|
||||
import org.elasticsearch.action.search.SearchAction;
|
||||
import org.elasticsearch.action.search.SearchRequest;
|
||||
import org.elasticsearch.action.search.SearchResponse;
|
||||
import org.elasticsearch.action.search.TransportSearchAction;
|
||||
import org.elasticsearch.action.support.IndicesOptions;
|
||||
import org.elasticsearch.action.support.WriteRequest;
|
||||
import org.elasticsearch.client.internal.Client;
|
||||
|
@ -400,7 +400,7 @@ public class TrainedModelProvider {
|
|||
if (parentTaskId != null) {
|
||||
searchRequest.setParentTask(parentTaskId);
|
||||
}
|
||||
executeAsyncWithOrigin(client, ML_ORIGIN, SearchAction.INSTANCE, searchRequest, ActionListener.wrap(searchResponse -> {
|
||||
executeAsyncWithOrigin(client, ML_ORIGIN, TransportSearchAction.TYPE, searchRequest, ActionListener.wrap(searchResponse -> {
|
||||
if (searchResponse.getHits().getHits().length == 0) {
|
||||
listener.onFailure(new ResourceNotFoundException(Messages.getMessage(Messages.MODEL_METADATA_NOT_FOUND, modelIds)));
|
||||
return;
|
||||
|
@ -691,7 +691,7 @@ public class TrainedModelProvider {
|
|||
executeAsyncWithOrigin(
|
||||
client,
|
||||
ML_ORIGIN,
|
||||
SearchAction.INSTANCE,
|
||||
TransportSearchAction.TYPE,
|
||||
ChunkedTrainedModelRestorer.buildSearch(
|
||||
client,
|
||||
modelId,
|
||||
|
@ -731,7 +731,7 @@ public class TrainedModelProvider {
|
|||
}, getTrainedModelListener::onFailure)
|
||||
);
|
||||
}, getTrainedModelListener::onFailure);
|
||||
executeAsyncWithOrigin(client, ML_ORIGIN, SearchAction.INSTANCE, trainedModelConfigSearch, trainedModelSearchHandler);
|
||||
executeAsyncWithOrigin(client, ML_ORIGIN, TransportSearchAction.TYPE, trainedModelConfigSearch, trainedModelSearchHandler);
|
||||
}
|
||||
|
||||
public void getTrainedModels(
|
||||
|
@ -872,7 +872,7 @@ public class TrainedModelProvider {
|
|||
getTrainedModelListener.onResponse(configs);
|
||||
}, getTrainedModelListener::onFailure);
|
||||
|
||||
executeAsyncWithOrigin(client, ML_ORIGIN, SearchAction.INSTANCE, searchRequest, configSearchHandler);
|
||||
executeAsyncWithOrigin(client, ML_ORIGIN, TransportSearchAction.TYPE, searchRequest, configSearchHandler);
|
||||
}
|
||||
|
||||
public void deleteTrainedModel(String modelId, ActionListener<Boolean> listener) {
|
||||
|
|
|
@ -17,11 +17,11 @@ import org.elasticsearch.action.admin.indices.refresh.RefreshAction;
|
|||
import org.elasticsearch.action.admin.indices.refresh.RefreshRequest;
|
||||
import org.elasticsearch.action.admin.indices.refresh.RefreshResponse;
|
||||
import org.elasticsearch.action.bulk.BulkItemResponse;
|
||||
import org.elasticsearch.action.search.MultiSearchAction;
|
||||
import org.elasticsearch.action.search.MultiSearchRequest;
|
||||
import org.elasticsearch.action.search.MultiSearchResponse;
|
||||
import org.elasticsearch.action.search.SearchRequest;
|
||||
import org.elasticsearch.action.search.SearchResponse;
|
||||
import org.elasticsearch.action.search.TransportMultiSearchAction;
|
||||
import org.elasticsearch.action.support.IndicesOptions;
|
||||
import org.elasticsearch.action.support.master.AcknowledgedResponse;
|
||||
import org.elasticsearch.client.internal.Client;
|
||||
|
@ -413,7 +413,7 @@ public class JobDataDeleter {
|
|||
);
|
||||
multiSearchRequest.add(new SearchRequest(indexName).source(source));
|
||||
}
|
||||
executeAsyncWithOrigin(client, ML_ORIGIN, MultiSearchAction.INSTANCE, multiSearchRequest, customIndexSearchHandler);
|
||||
executeAsyncWithOrigin(client, ML_ORIGIN, TransportMultiSearchAction.TYPE, multiSearchRequest, customIndexSearchHandler);
|
||||
}, failureHandler);
|
||||
|
||||
// Step 5. Get the job as the initial result index name is required
|
||||
|
|
|
@ -10,9 +10,9 @@ import org.apache.logging.log4j.LogManager;
|
|||
import org.apache.logging.log4j.Logger;
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.search.SearchAction;
|
||||
import org.elasticsearch.action.search.SearchRequest;
|
||||
import org.elasticsearch.action.search.SearchResponse;
|
||||
import org.elasticsearch.action.search.TransportSearchAction;
|
||||
import org.elasticsearch.action.support.ThreadedActionListener;
|
||||
import org.elasticsearch.client.internal.OriginSettingClient;
|
||||
import org.elasticsearch.index.query.BoolQueryBuilder;
|
||||
|
@ -98,7 +98,7 @@ public class ExpiredForecastsRemover implements MlDataRemover {
|
|||
searchRequest.source(source);
|
||||
searchRequest.setParentTask(parentTaskId);
|
||||
client.execute(
|
||||
SearchAction.INSTANCE,
|
||||
TransportSearchAction.TYPE,
|
||||
searchRequest,
|
||||
new ThreadedActionListener<>(threadPool.executor(MachineLearning.UTILITY_THREAD_POOL_NAME), forecastStatsHandler)
|
||||
);
|
||||
|
|
|
@ -7,12 +7,12 @@
|
|||
package org.elasticsearch.xpack.ml.datafeed.extractor.aggregation;
|
||||
|
||||
import org.elasticsearch.action.ActionRequestBuilder;
|
||||
import org.elasticsearch.action.search.SearchAction;
|
||||
import org.elasticsearch.action.search.SearchPhaseExecutionException;
|
||||
import org.elasticsearch.action.search.SearchRequest;
|
||||
import org.elasticsearch.action.search.SearchRequestBuilder;
|
||||
import org.elasticsearch.action.search.SearchResponse;
|
||||
import org.elasticsearch.action.search.ShardSearchFailure;
|
||||
import org.elasticsearch.action.search.TransportSearchAction;
|
||||
import org.elasticsearch.client.internal.Client;
|
||||
import org.elasticsearch.core.TimeValue;
|
||||
import org.elasticsearch.core.Tuple;
|
||||
|
@ -122,9 +122,10 @@ public class CompositeAggregationDataExtractorTests extends ESTestCase {
|
|||
.subAggregation(AggregationBuilders.avg("responsetime").field("responsetime"));
|
||||
runtimeMappings = Collections.emptyMap();
|
||||
timingStatsReporter = new DatafeedTimingStatsReporter(new DatafeedTimingStats(jobId), mock(DatafeedTimingStatsPersister.class));
|
||||
aggregatedSearchRequestBuilder = (searchSourceBuilder) -> new SearchRequestBuilder(testClient, SearchAction.INSTANCE).setSource(
|
||||
searchSourceBuilder
|
||||
).setAllowPartialSearchResults(false).setIndices(indices.toArray(String[]::new));
|
||||
aggregatedSearchRequestBuilder = (searchSourceBuilder) -> new SearchRequestBuilder(testClient, TransportSearchAction.TYPE)
|
||||
.setSource(searchSourceBuilder)
|
||||
.setAllowPartialSearchResults(false)
|
||||
.setIndices(indices.toArray(String[]::new));
|
||||
}
|
||||
|
||||
public void testExtraction() throws IOException {
|
||||
|
|
|
@ -9,7 +9,6 @@ package org.elasticsearch.xpack.ml.datafeed.extractor.scroll;
|
|||
import org.apache.lucene.search.TotalHits;
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.action.ActionFuture;
|
||||
import org.elasticsearch.action.search.ClearScrollAction;
|
||||
import org.elasticsearch.action.search.ClearScrollRequest;
|
||||
import org.elasticsearch.action.search.ClearScrollResponse;
|
||||
import org.elasticsearch.action.search.SearchPhaseExecutionException;
|
||||
|
@ -17,6 +16,7 @@ import org.elasticsearch.action.search.SearchRequest;
|
|||
import org.elasticsearch.action.search.SearchRequestBuilder;
|
||||
import org.elasticsearch.action.search.SearchResponse;
|
||||
import org.elasticsearch.action.search.ShardSearchFailure;
|
||||
import org.elasticsearch.action.search.TransportClearScrollAction;
|
||||
import org.elasticsearch.client.internal.Client;
|
||||
import org.elasticsearch.common.document.DocumentField;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
|
@ -173,7 +173,7 @@ public class ScrollDataExtractorTests extends ESTestCase {
|
|||
|
||||
clearScrollFuture = mock(ActionFuture.class);
|
||||
capturedClearScrollRequests = ArgumentCaptor.forClass(ClearScrollRequest.class);
|
||||
when(client.execute(same(ClearScrollAction.INSTANCE), capturedClearScrollRequests.capture())).thenReturn(clearScrollFuture);
|
||||
when(client.execute(same(TransportClearScrollAction.TYPE), capturedClearScrollRequests.capture())).thenReturn(clearScrollFuture);
|
||||
timingStatsReporter = new DatafeedTimingStatsReporter(new DatafeedTimingStats(jobId), mock(DatafeedTimingStatsPersister.class));
|
||||
}
|
||||
|
||||
|
|
|
@ -10,8 +10,8 @@ import org.elasticsearch.action.ActionListener;
|
|||
import org.elasticsearch.action.index.IndexAction;
|
||||
import org.elasticsearch.action.index.IndexRequest;
|
||||
import org.elasticsearch.action.index.IndexResponse;
|
||||
import org.elasticsearch.action.search.SearchAction;
|
||||
import org.elasticsearch.action.search.SearchResponse;
|
||||
import org.elasticsearch.action.search.TransportSearchAction;
|
||||
import org.elasticsearch.client.internal.Client;
|
||||
import org.elasticsearch.cluster.service.ClusterService;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
|
@ -188,7 +188,7 @@ public class DataFrameAnalyticsTaskTests extends ESTestCase {
|
|||
|
||||
SearchResponse searchResponse = mock(SearchResponse.class);
|
||||
when(searchResponse.getHits()).thenReturn(searchHits);
|
||||
doAnswer(withResponse(searchResponse)).when(client).execute(eq(SearchAction.INSTANCE), any(), any());
|
||||
doAnswer(withResponse(searchResponse)).when(client).execute(eq(TransportSearchAction.TYPE), any(), any());
|
||||
|
||||
IndexResponse indexResponse = mock(IndexResponse.class);
|
||||
doAnswer(withResponse(indexResponse)).when(client).execute(eq(IndexAction.INSTANCE), any(), any());
|
||||
|
@ -217,7 +217,7 @@ public class DataFrameAnalyticsTaskTests extends ESTestCase {
|
|||
ArgumentCaptor<IndexRequest> indexRequestCaptor = ArgumentCaptor.forClass(IndexRequest.class);
|
||||
|
||||
InOrder inOrder = inOrder(client, runnable);
|
||||
inOrder.verify(client).execute(eq(SearchAction.INSTANCE), any(), any());
|
||||
inOrder.verify(client).execute(eq(TransportSearchAction.TYPE), any(), any());
|
||||
inOrder.verify(client).execute(eq(IndexAction.INSTANCE), indexRequestCaptor.capture(), any());
|
||||
inOrder.verify(runnable).run();
|
||||
inOrder.verifyNoMoreInteractions();
|
||||
|
@ -284,7 +284,7 @@ public class DataFrameAnalyticsTaskTests extends ESTestCase {
|
|||
|
||||
SearchResponse searchResponse = mock(SearchResponse.class);
|
||||
when(searchResponse.getHits()).thenReturn(SearchHits.EMPTY_WITH_TOTAL_HITS);
|
||||
doAnswer(withResponse(searchResponse)).when(client).execute(eq(SearchAction.INSTANCE), any(), any());
|
||||
doAnswer(withResponse(searchResponse)).when(client).execute(eq(TransportSearchAction.TYPE), any(), any());
|
||||
|
||||
IndexResponse indexResponse = mock(IndexResponse.class);
|
||||
doAnswer(withResponse(indexResponse)).when(client).execute(eq(IndexAction.INSTANCE), any(), any());
|
||||
|
@ -315,7 +315,7 @@ public class DataFrameAnalyticsTaskTests extends ESTestCase {
|
|||
if (nodeShuttingDown == false) {
|
||||
// Verify progress was persisted
|
||||
ArgumentCaptor<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());
|
||||
|
||||
IndexRequest indexRequest = indexRequestCaptor.getValue();
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show more
Loading…
Add table
Add a link
Reference in a new issue