This change introduces a new API in x-pack basic that allows to track the progress of a search. Users can submit an asynchronous search through a new endpoint called `_async_search` that works exactly the same as the `_search` endpoint but instead of blocking and returning the final response when available, it returns a response after a provided `wait_for_completion` time. ```` GET my_index_pattern*/_async_search?wait_for_completion=100ms { "aggs": { "date_histogram": { "field": "@timestamp", "fixed_interval": "1h" } } } ```` If after 100ms the final response is not available, a `partial_response` is included in the body: ```` { "id": "9N3J1m4BgyzUDzqgC15b", "version": 1, "is_running": true, "is_partial": true, "response": { "_shards": { "total": 100, "successful": 5, "failed": 0 }, "total_hits": { "value": 1653433, "relation": "eq" }, "aggs": { ... } } } ```` The partial response contains the total number of requested shards, the number of shards that successfully returned and the number of shards that failed. It also contains the total hits as well as partial aggregations computed from the successful shards. To continue to monitor the progress of the search users can call the get `_async_search` API like the following: ```` GET _async_search/9N3J1m4BgyzUDzqgC15b/?wait_for_completion=100ms ```` That returns a new response that can contain the same partial response than the previous call if the search didn't progress, in such case the returned `version` should be the same. If new partial results are available, the version is incremented and the `partial_response` contains the updated progress. Finally if the response is fully available while or after waiting for completion, the `partial_response` is replaced by a `response` section that contains the usual _search response: ```` { "id": "9N3J1m4BgyzUDzqgC15b", "version": 10, "is_running": false, "response": { "is_partial": false, ... } } ```` Asynchronous search are stored in a restricted index called `.async-search` if they survive (still running) after the initial submit. Each request has a keep alive that defaults to 5 days but this value can be changed/updated any time: ````` GET my_index_pattern*/_async_search?wait_for_completion=100ms&keep_alive=10d ````` The default can be changed when submitting the search, the example above raises the default value for the search to `10d`. ````` GET _async_search/9N3J1m4BgyzUDzqgC15b/?wait_for_completion=100ms&keep_alive=10d ````` The time to live for a specific search can be extended when getting the progress/result. In the example above we extend the keep alive to 10 more days. A background service that runs only on the node that holds the first primary shard of the `async-search` index is responsible for deleting the expired results. It runs every hour but the expiration is also checked by running queries (if they take longer than the keep_alive) and when getting a result. Like a normal `_search`, if the http channel that is used to submit a request is closed before getting a response, the search is automatically cancelled. Note that this behavior is only for the submit API, subsequent GET requests will not cancel if they are closed. Asynchronous search are not persistent, if the coordinator node crashes or is restarted during the search, the asynchronous search will stop. To know if the search is still running or not the response contains a field called `is_running` that indicates if the task is up or not. It is the responsibility of the user to resume an asynchronous search that didn't reach a final response by re-submitting the query. However final responses and failures are persisted in a system index that allows to retrieve a response even if the task finishes. ```` DELETE _async_search/9N3J1m4BgyzUDzqgC15b ```` The response is also not stored if the initial submit action returns a final response. This allows to not add any overhead to queries that completes within the initial `wait_for_completion`. The `.async-search` index is a restricted index (should be migrated to a system index in +8.0) that is accessible only through the async search APIs. These APIs also ensure that only the user that submitted the initial query can retrieve or delete the running search. Note that admins/superusers would still be able to cancel the search task through the task manager like any other tasks. Relates #49091 Co-authored-by: Luca Cavanna <javanna@users.noreply.github.com>
This commit is contained in:
parent
723034001c
commit
e6680be0b1
|
@ -90,7 +90,7 @@ abstract class AbstractSearchAsyncAction<Result extends SearchPhaseResult> exten
|
|||
private final SearchTimeProvider timeProvider;
|
||||
private final SearchResponse.Clusters clusters;
|
||||
|
||||
private final GroupShardsIterator<SearchShardIterator> toSkipShardsIts;
|
||||
protected final GroupShardsIterator<SearchShardIterator> toSkipShardsIts;
|
||||
protected final GroupShardsIterator<SearchShardIterator> shardsIts;
|
||||
private final int expectedTotalOps;
|
||||
private final AtomicInteger totalOps = new AtomicInteger();
|
||||
|
@ -375,6 +375,11 @@ abstract class AbstractSearchAsyncAction<Result extends SearchPhaseResult> exten
|
|||
// we do make sure to clean it on a successful response from a shard
|
||||
SearchShardTarget shardTarget = shardIt.newSearchShardTarget(nodeId);
|
||||
onShardFailure(shardIndex, shardTarget, e);
|
||||
final ShardRouting nextShard = shardIt.nextOrNull();
|
||||
final boolean lastShard = nextShard == null;
|
||||
if (lastShard) {
|
||||
onShardGroupFailure(shardIndex, shardTarget, e);
|
||||
}
|
||||
|
||||
if (totalOps.incrementAndGet() == expectedTotalOps) {
|
||||
if (logger.isDebugEnabled()) {
|
||||
|
@ -385,11 +390,8 @@ abstract class AbstractSearchAsyncAction<Result extends SearchPhaseResult> exten
|
|||
logger.trace(new ParameterizedMessage("{}: Failed to execute [{}]", shard, request), e);
|
||||
}
|
||||
}
|
||||
onShardGroupFailure(shardIndex, e);
|
||||
onPhaseDone();
|
||||
} else {
|
||||
final ShardRouting nextShard = shardIt.nextOrNull();
|
||||
final boolean lastShard = nextShard == null;
|
||||
// trace log this exception
|
||||
logger.trace(() -> new ParameterizedMessage(
|
||||
"{}: Failed to execute [{}] lastShard [{}]",
|
||||
|
@ -405,7 +407,6 @@ abstract class AbstractSearchAsyncAction<Result extends SearchPhaseResult> exten
|
|||
shard != null ? shard.shortSummary() : shardIt.shardId(), request, lastShard), e);
|
||||
}
|
||||
}
|
||||
onShardGroupFailure(shardIndex, e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -413,10 +414,11 @@ abstract class AbstractSearchAsyncAction<Result extends SearchPhaseResult> exten
|
|||
/**
|
||||
* Executed once for every {@link ShardId} that failed on all available shard routing.
|
||||
*
|
||||
* @param shardIndex the shard target that failed
|
||||
* @param exc the final failure reason
|
||||
* @param shardIndex the shard index that failed
|
||||
* @param shardTarget the last shard target for this failure
|
||||
* @param exc the last failure reason
|
||||
*/
|
||||
protected void onShardGroupFailure(int shardIndex, Exception exc) {}
|
||||
protected void onShardGroupFailure(int shardIndex, SearchShardTarget shardTarget, Exception exc) {}
|
||||
|
||||
/**
|
||||
* Executed once for every failed shard level request. This method is invoked before the next replica is tried for the given
|
||||
|
|
|
@ -22,7 +22,6 @@ import org.apache.logging.log4j.message.ParameterizedMessage;
|
|||
import org.elasticsearch.common.util.concurrent.AtomicArray;
|
||||
import org.elasticsearch.search.SearchPhaseResult;
|
||||
import org.elasticsearch.search.SearchShardTarget;
|
||||
import org.elasticsearch.search.builder.SearchSourceBuilder;
|
||||
import org.elasticsearch.search.dfs.AggregatedDfs;
|
||||
import org.elasticsearch.search.dfs.DfsSearchResult;
|
||||
import org.elasticsearch.search.query.QuerySearchRequest;
|
||||
|
@ -72,8 +71,6 @@ final class DfsQueryPhase extends SearchPhase {
|
|||
final CountedCollector<SearchPhaseResult> counter = new CountedCollector<>(queryResult::consumeResult,
|
||||
resultList.size(),
|
||||
() -> context.executeNextPhase(this, nextPhaseFactory.apply(queryResult)), context);
|
||||
final SearchSourceBuilder sourceBuilder = context.getRequest().source();
|
||||
progressListener.notifyListShards(progressListener.searchShards(resultList), sourceBuilder == null || sourceBuilder.size() != 0);
|
||||
for (final DfsSearchResult dfsResult : resultList) {
|
||||
final SearchShardTarget searchShardTarget = dfsResult.getSearchShardTarget();
|
||||
Transport.Connection connection = context.getConnection(searchShardTarget.getClusterAlias(), searchShardTarget.getNodeId());
|
||||
|
@ -97,7 +94,7 @@ final class DfsQueryPhase extends SearchPhase {
|
|||
try {
|
||||
context.getLogger().debug(() -> new ParameterizedMessage("[{}] Failed to execute query phase",
|
||||
querySearchRequest.contextId()), exception);
|
||||
progressListener.notifyQueryFailure(shardIndex, exception);
|
||||
progressListener.notifyQueryFailure(shardIndex, searchShardTarget, exception);
|
||||
counter.onFailure(shardIndex, searchShardTarget, exception);
|
||||
} finally {
|
||||
// the query might not have been executed at all (for example because thread pool rejected
|
||||
|
|
|
@ -24,6 +24,7 @@ import org.elasticsearch.action.ActionListener;
|
|||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.routing.GroupShardsIterator;
|
||||
import org.elasticsearch.cluster.routing.ShardRouting;
|
||||
import org.elasticsearch.search.builder.SearchSourceBuilder;
|
||||
import org.elasticsearch.search.dfs.DfsSearchResult;
|
||||
import org.elasticsearch.search.internal.AliasFilter;
|
||||
import org.elasticsearch.transport.Transport;
|
||||
|
@ -51,6 +52,10 @@ final class SearchDfsQueryThenFetchAsyncAction extends AbstractSearchAsyncAction
|
|||
shardsIts, timeProvider, clusterState, task, new ArraySearchPhaseResults<>(shardsIts.size()),
|
||||
request.getMaxConcurrentShardRequests(), clusters);
|
||||
this.searchPhaseController = searchPhaseController;
|
||||
SearchProgressListener progressListener = task.getProgressListener();
|
||||
SearchSourceBuilder sourceBuilder = request.source();
|
||||
progressListener.notifyListShards(progressListener.searchShards(this.shardsIts),
|
||||
progressListener.searchShards(toSkipShardsIts), clusters, sourceBuilder == null || sourceBuilder.size() != 0);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -664,9 +664,9 @@ public final class SearchPhaseController {
|
|||
}
|
||||
numReducePhases++;
|
||||
index = 1;
|
||||
if (hasAggs) {
|
||||
if (hasAggs || hasTopDocs) {
|
||||
progressListener.notifyPartialReduce(progressListener.searchShards(processedShards),
|
||||
topDocsStats.getTotalHits(), aggsBuffer[0], numReducePhases);
|
||||
topDocsStats.getTotalHits(), hasAggs ? aggsBuffer[0] : null, numReducePhases);
|
||||
}
|
||||
}
|
||||
final int i = index++;
|
||||
|
@ -696,7 +696,7 @@ public final class SearchPhaseController {
|
|||
ReducedQueryPhase reducePhase = controller.reducedQueryPhase(results.asList(),
|
||||
getRemainingAggs(), getRemainingTopDocs(), topDocsStats, numReducePhases, false, performFinalReduce);
|
||||
progressListener.notifyReduce(progressListener.searchShards(results.asList()),
|
||||
reducePhase.totalHits, reducePhase.aggregations);
|
||||
reducePhase.totalHits, reducePhase.aggregations, reducePhase.numReducePhases);
|
||||
return reducePhase;
|
||||
}
|
||||
|
||||
|
@ -751,7 +751,8 @@ public final class SearchPhaseController {
|
|||
List<SearchPhaseResult> resultList = results.asList();
|
||||
final ReducedQueryPhase reducePhase =
|
||||
reducedQueryPhase(resultList, isScrollRequest, trackTotalHitsUpTo, request.isFinalReduce());
|
||||
listener.notifyReduce(listener.searchShards(resultList), reducePhase.totalHits, reducePhase.aggregations);
|
||||
listener.notifyReduce(listener.searchShards(resultList), reducePhase.totalHits,
|
||||
reducePhase.aggregations, reducePhase.numReducePhases);
|
||||
return reducePhase;
|
||||
}
|
||||
};
|
||||
|
|
|
@ -23,6 +23,7 @@ import org.apache.logging.log4j.LogManager;
|
|||
import org.apache.logging.log4j.Logger;
|
||||
import org.apache.logging.log4j.message.ParameterizedMessage;
|
||||
import org.apache.lucene.search.TotalHits;
|
||||
import org.elasticsearch.action.search.SearchResponse.Clusters;
|
||||
import org.elasticsearch.cluster.routing.GroupShardsIterator;
|
||||
import org.elasticsearch.search.SearchPhaseResult;
|
||||
import org.elasticsearch.search.SearchShardTarget;
|
||||
|
@ -49,24 +50,27 @@ abstract class SearchProgressListener {
|
|||
* Executed when shards are ready to be queried.
|
||||
*
|
||||
* @param shards The list of shards to query.
|
||||
* @param skippedShards The list of skipped shards.
|
||||
* @param clusters The statistics for remote clusters included in the search.
|
||||
* @param fetchPhase <code>true</code> if the search needs a fetch phase, <code>false</code> otherwise.
|
||||
**/
|
||||
public void onListShards(List<SearchShard> shards, boolean fetchPhase) {}
|
||||
public void onListShards(List<SearchShard> shards, List<SearchShard> skippedShards, Clusters clusters, boolean fetchPhase) {}
|
||||
|
||||
/**
|
||||
* Executed when a shard returns a query result.
|
||||
*
|
||||
* @param shardIndex The index of the shard in the list provided by {@link SearchProgressListener#onListShards(List, boolean)} )}.
|
||||
* @param shardIndex The index of the shard in the list provided by {@link SearchProgressListener#onListShards} )}.
|
||||
*/
|
||||
public void onQueryResult(int shardIndex) {}
|
||||
|
||||
/**
|
||||
* Executed when a shard reports a query failure.
|
||||
*
|
||||
* @param shardIndex The index of the shard in the list provided by {@link SearchProgressListener#onListShards(List, boolean)})}.
|
||||
* @param shardIndex The index of the shard in the list provided by {@link SearchProgressListener#onListShards})}.
|
||||
* @param shardTarget The last shard target that thrown an exception.
|
||||
* @param exc The cause of the failure.
|
||||
*/
|
||||
public void onQueryFailure(int shardIndex, Exception exc) {}
|
||||
public void onQueryFailure(int shardIndex, SearchShardTarget shardTarget, Exception exc) {}
|
||||
|
||||
/**
|
||||
* Executed when a partial reduce is created. The number of partial reduce can be controlled via
|
||||
|
@ -75,9 +79,9 @@ abstract class SearchProgressListener {
|
|||
* @param shards The list of shards that are part of this reduce.
|
||||
* @param totalHits The total number of hits in this reduce.
|
||||
* @param aggs The partial result for aggregations.
|
||||
* @param version The version number for this reduce.
|
||||
* @param reducePhase The version number for this reduce.
|
||||
*/
|
||||
public void onPartialReduce(List<SearchShard> shards, TotalHits totalHits, InternalAggregations aggs, int version) {}
|
||||
public void onPartialReduce(List<SearchShard> shards, TotalHits totalHits, InternalAggregations aggs, int reducePhase) {}
|
||||
|
||||
/**
|
||||
* Executed once when the final reduce is created.
|
||||
|
@ -85,28 +89,29 @@ abstract class SearchProgressListener {
|
|||
* @param shards The list of shards that are part of this reduce.
|
||||
* @param totalHits The total number of hits in this reduce.
|
||||
* @param aggs The final result for aggregations.
|
||||
* @param reducePhase The version number for this reduce.
|
||||
*/
|
||||
public void onReduce(List<SearchShard> shards, TotalHits totalHits, InternalAggregations aggs) {}
|
||||
public void onReduce(List<SearchShard> shards, TotalHits totalHits, InternalAggregations aggs, int reducePhase) {}
|
||||
|
||||
/**
|
||||
* Executed when a shard returns a fetch result.
|
||||
*
|
||||
* @param shardIndex The index of the shard in the list provided by {@link SearchProgressListener#onListShards(List, boolean)})}.
|
||||
* @param shardIndex The index of the shard in the list provided by {@link SearchProgressListener#onListShards})}.
|
||||
*/
|
||||
public void onFetchResult(int shardIndex) {}
|
||||
|
||||
/**
|
||||
* Executed when a shard reports a fetch failure.
|
||||
*
|
||||
* @param shardIndex The index of the shard in the list provided by {@link SearchProgressListener#onListShards(List, boolean)})}.
|
||||
* @param shardIndex The index of the shard in the list provided by {@link SearchProgressListener#onListShards})}.
|
||||
* @param exc The cause of the failure.
|
||||
*/
|
||||
public void onFetchFailure(int shardIndex, Exception exc) {}
|
||||
|
||||
final void notifyListShards(List<SearchShard> shards, boolean fetchPhase) {
|
||||
final void notifyListShards(List<SearchShard> shards, List<SearchShard> skippedShards, Clusters clusters, boolean fetchPhase) {
|
||||
this.shards = shards;
|
||||
try {
|
||||
onListShards(shards, fetchPhase);
|
||||
onListShards(shards, skippedShards, clusters, fetchPhase);
|
||||
} catch (Exception e) {
|
||||
logger.warn(() -> new ParameterizedMessage("Failed to execute progress listener on list shards"), e);
|
||||
}
|
||||
|
@ -121,26 +126,26 @@ abstract class SearchProgressListener {
|
|||
}
|
||||
}
|
||||
|
||||
final void notifyQueryFailure(int shardIndex, Exception exc) {
|
||||
final void notifyQueryFailure(int shardIndex, SearchShardTarget shardTarget, Exception exc) {
|
||||
try {
|
||||
onQueryFailure(shardIndex, exc);
|
||||
onQueryFailure(shardIndex, shardTarget, exc);
|
||||
} catch (Exception e) {
|
||||
logger.warn(() -> new ParameterizedMessage("[{}] Failed to execute progress listener on query failure",
|
||||
shards.get(shardIndex)), e);
|
||||
}
|
||||
}
|
||||
|
||||
final void notifyPartialReduce(List<SearchShard> shards, TotalHits totalHits, InternalAggregations aggs, int version) {
|
||||
final void notifyPartialReduce(List<SearchShard> shards, TotalHits totalHits, InternalAggregations aggs, int reducePhase) {
|
||||
try {
|
||||
onPartialReduce(shards, totalHits, aggs, version);
|
||||
onPartialReduce(shards, totalHits, aggs, reducePhase);
|
||||
} catch (Exception e) {
|
||||
logger.warn(() -> new ParameterizedMessage("Failed to execute progress listener on partial reduce"), e);
|
||||
}
|
||||
}
|
||||
|
||||
final void notifyReduce(List<SearchShard> shards, TotalHits totalHits, InternalAggregations aggs) {
|
||||
final void notifyReduce(List<SearchShard> shards, TotalHits totalHits, InternalAggregations aggs, int reducePhase) {
|
||||
try {
|
||||
onReduce(shards, totalHits, aggs);
|
||||
onReduce(shards, totalHits, aggs, reducePhase);
|
||||
} catch (Exception e) {
|
||||
logger.warn(() -> new ParameterizedMessage("Failed to execute progress listener on reduce"), e);
|
||||
}
|
||||
|
|
|
@ -25,6 +25,7 @@ import org.elasticsearch.cluster.ClusterState;
|
|||
import org.elasticsearch.cluster.routing.GroupShardsIterator;
|
||||
import org.elasticsearch.cluster.routing.ShardRouting;
|
||||
import org.elasticsearch.search.SearchPhaseResult;
|
||||
import org.elasticsearch.search.SearchShardTarget;
|
||||
import org.elasticsearch.search.builder.SearchSourceBuilder;
|
||||
import org.elasticsearch.search.internal.AliasFilter;
|
||||
import org.elasticsearch.transport.Transport;
|
||||
|
@ -57,7 +58,7 @@ final class SearchQueryThenFetchAsyncAction extends AbstractSearchAsyncAction<Se
|
|||
final SearchProgressListener progressListener = task.getProgressListener();
|
||||
final SearchSourceBuilder sourceBuilder = request.source();
|
||||
progressListener.notifyListShards(progressListener.searchShards(this.shardsIts),
|
||||
sourceBuilder == null || sourceBuilder.size() != 0);
|
||||
progressListener.searchShards(toSkipShardsIts), clusters, sourceBuilder == null || sourceBuilder.size() != 0);
|
||||
}
|
||||
|
||||
protected void executePhaseOnShard(final SearchShardIterator shardIt, final ShardRouting shard,
|
||||
|
@ -67,8 +68,8 @@ final class SearchQueryThenFetchAsyncAction extends AbstractSearchAsyncAction<Se
|
|||
}
|
||||
|
||||
@Override
|
||||
protected void onShardGroupFailure(int shardIndex, Exception exc) {
|
||||
progressListener.notifyQueryFailure(shardIndex, exc);
|
||||
protected void onShardGroupFailure(int shardIndex, SearchShardTarget shardTarget, Exception exc) {
|
||||
progressListener.notifyQueryFailure(shardIndex, shardTarget, exc);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -33,7 +33,6 @@ import org.elasticsearch.common.xcontent.ToXContent;
|
|||
import org.elasticsearch.search.Scroll;
|
||||
import org.elasticsearch.search.builder.SearchSourceBuilder;
|
||||
import org.elasticsearch.search.internal.SearchContext;
|
||||
import org.elasticsearch.tasks.Task;
|
||||
import org.elasticsearch.tasks.TaskId;
|
||||
|
||||
import java.io.IOException;
|
||||
|
@ -56,9 +55,9 @@ import static org.elasticsearch.action.ValidateActions.addValidationError;
|
|||
* @see org.elasticsearch.client.Client#search(SearchRequest)
|
||||
* @see SearchResponse
|
||||
*/
|
||||
public final class SearchRequest extends ActionRequest implements IndicesRequest.Replaceable {
|
||||
public class SearchRequest extends ActionRequest implements IndicesRequest.Replaceable {
|
||||
|
||||
private static final ToXContent.Params FORMAT_PARAMS = new ToXContent.MapParams(Collections.singletonMap("pretty", "false"));
|
||||
public static final ToXContent.Params FORMAT_PARAMS = new ToXContent.MapParams(Collections.singletonMap("pretty", "false"));
|
||||
|
||||
public static final int DEFAULT_PRE_FILTER_SHARD_SIZE = 128;
|
||||
public static final int DEFAULT_BATCHED_REDUCE_SIZE = 512;
|
||||
|
@ -597,7 +596,7 @@ public final class SearchRequest extends ActionRequest implements IndicesRequest
|
|||
}
|
||||
|
||||
@Override
|
||||
public Task createTask(long id, String type, String action, TaskId parentTaskId, Map<String, String> headers) {
|
||||
public SearchTask createTask(long id, String type, String action, TaskId parentTaskId, Map<String, String> headers) {
|
||||
// generating description in a lazy way since source can be quite big
|
||||
return new SearchTask(id, type, action, null, parentTaskId, headers) {
|
||||
@Override
|
||||
|
|
|
@ -224,7 +224,7 @@ public class SearchRequestBuilder extends ActionRequestBuilder<SearchRequest, Se
|
|||
sourceBuilder().version(version);
|
||||
return this;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Should each {@link org.elasticsearch.search.SearchHit} be returned with the
|
||||
* sequence number and primary term of the last modification of the document.
|
||||
|
|
|
@ -118,6 +118,10 @@ public class SearchResponse extends ActionResponse implements StatusToXContentOb
|
|||
return RestStatus.status(successfulShards, totalShards, shardFailures);
|
||||
}
|
||||
|
||||
public SearchResponseSections getInternalResponse() {
|
||||
return internalResponse;
|
||||
}
|
||||
|
||||
/**
|
||||
* The search hits.
|
||||
*/
|
||||
|
|
|
@ -34,7 +34,7 @@ public class SearchShard implements Comparable<SearchShard> {
|
|||
private final String clusterAlias;
|
||||
private final ShardId shardId;
|
||||
|
||||
SearchShard(@Nullable String clusterAlias, ShardId shardId) {
|
||||
public SearchShard(@Nullable String clusterAlias, ShardId shardId) {
|
||||
this.clusterAlias = clusterAlias;
|
||||
this.shardId = shardId;
|
||||
}
|
||||
|
|
|
@ -23,12 +23,6 @@ import org.elasticsearch.action.ActionType;
|
|||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.ActionRequest;
|
||||
import org.elasticsearch.action.ActionResponse;
|
||||
import org.elasticsearch.action.search.SearchAction;
|
||||
import org.elasticsearch.action.search.SearchRequest;
|
||||
import org.elasticsearch.action.search.SearchResponse;
|
||||
import org.elasticsearch.action.search.SearchTask;
|
||||
import org.elasticsearch.action.search.SearchProgressActionListener;
|
||||
import org.elasticsearch.action.search.TransportSearchAction;
|
||||
import org.elasticsearch.action.support.TransportAction;
|
||||
import org.elasticsearch.client.Client;
|
||||
import org.elasticsearch.client.support.AbstractClient;
|
||||
|
@ -36,7 +30,6 @@ import org.elasticsearch.cluster.node.DiscoveryNode;
|
|||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.tasks.Task;
|
||||
import org.elasticsearch.tasks.TaskListener;
|
||||
import org.elasticsearch.tasks.TaskManager;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
import org.elasticsearch.transport.RemoteClusterService;
|
||||
|
||||
|
@ -49,9 +42,6 @@ import java.util.function.Supplier;
|
|||
public class NodeClient extends AbstractClient {
|
||||
|
||||
private Map<ActionType, TransportAction> actions;
|
||||
|
||||
private TaskManager taskManager;
|
||||
|
||||
/**
|
||||
* The id of the local {@link DiscoveryNode}. Useful for generating task ids from tasks returned by
|
||||
* {@link #executeLocally(ActionType, ActionRequest, TaskListener)}.
|
||||
|
@ -63,10 +53,9 @@ public class NodeClient extends AbstractClient {
|
|||
super(settings, threadPool);
|
||||
}
|
||||
|
||||
public void initialize(Map<ActionType, TransportAction> actions, TaskManager taskManager, Supplier<String> localNodeId,
|
||||
public void initialize(Map<ActionType, TransportAction> actions, Supplier<String> localNodeId,
|
||||
RemoteClusterService remoteClusterService) {
|
||||
this.actions = actions;
|
||||
this.taskManager = taskManager;
|
||||
this.localNodeId = localNodeId;
|
||||
this.remoteClusterService = remoteClusterService;
|
||||
}
|
||||
|
@ -104,38 +93,6 @@ public class NodeClient extends AbstractClient {
|
|||
return transportAction(action).execute(request, listener);
|
||||
}
|
||||
|
||||
/**
|
||||
* Execute a {@link SearchRequest} locally and track the progress of the request through
|
||||
* a {@link SearchProgressActionListener}.
|
||||
*/
|
||||
public SearchTask executeSearchLocally(SearchRequest request, SearchProgressActionListener listener) {
|
||||
// we cannot track the progress if remote cluster requests are splitted.
|
||||
request.setCcsMinimizeRoundtrips(false);
|
||||
TransportSearchAction action = (TransportSearchAction) actions.get(SearchAction.INSTANCE);
|
||||
SearchTask task = (SearchTask) taskManager.register("transport", action.actionName, request);
|
||||
task.setProgressListener(listener);
|
||||
action.execute(task, request, new ActionListener<SearchResponse>() {
|
||||
@Override
|
||||
public void onResponse(SearchResponse response) {
|
||||
try {
|
||||
taskManager.unregister(task);
|
||||
} finally {
|
||||
listener.onResponse(response);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onFailure(Exception e) {
|
||||
try {
|
||||
taskManager.unregister(task);
|
||||
} finally {
|
||||
listener.onFailure(e);
|
||||
}
|
||||
}
|
||||
});
|
||||
return task;
|
||||
}
|
||||
|
||||
/**
|
||||
* The id of the local {@link DiscoveryNode}. Useful for generating task ids from tasks returned by
|
||||
* {@link #executeLocally(ActionType, ActionRequest, TaskListener)}.
|
||||
|
|
|
@ -623,7 +623,7 @@ public class Node implements Closeable {
|
|||
resourcesToClose.addAll(pluginLifecycleComponents);
|
||||
resourcesToClose.add(injector.getInstance(PeerRecoverySourceService.class));
|
||||
this.pluginLifecycleComponents = Collections.unmodifiableList(pluginLifecycleComponents);
|
||||
client.initialize(injector.getInstance(new Key<Map<ActionType, TransportAction>>() {}), transportService.getTaskManager(),
|
||||
client.initialize(injector.getInstance(new Key<Map<ActionType, TransportAction>>() {}),
|
||||
() -> clusterService.localNode().getId(), transportService.getRemoteClusterService());
|
||||
|
||||
logger.debug("initializing HTTP handlers ...");
|
||||
|
|
|
@ -179,7 +179,7 @@ public class RestSearchAction extends BaseRestHandler {
|
|||
searchRequest.routing(request.param("routing"));
|
||||
searchRequest.preference(request.param("preference"));
|
||||
searchRequest.indicesOptions(IndicesOptions.fromRequest(request, searchRequest.indicesOptions()));
|
||||
searchRequest.setCcsMinimizeRoundtrips(request.paramAsBoolean("ccs_minimize_roundtrips", true));
|
||||
searchRequest.setCcsMinimizeRoundtrips(request.paramAsBoolean("ccs_minimize_roundtrips", searchRequest.isCcsMinimizeRoundtrips()));
|
||||
|
||||
checkRestTotalHits(request, searchRequest);
|
||||
}
|
||||
|
|
|
@ -140,6 +140,13 @@ public class Task {
|
|||
return startTime;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the task's start time in nanoseconds ({@link System#nanoTime()} style).
|
||||
*/
|
||||
public long getStartTimeNanos() {
|
||||
return startTimeNanos;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns id of the parent task or NO_PARENT_ID if the task doesn't have any parent tasks
|
||||
*/
|
||||
|
|
|
@ -818,18 +818,18 @@ public class SearchPhaseControllerTests extends ESTestCase {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void onQueryFailure(int shardIndex, Exception exc) {
|
||||
public void onQueryFailure(int shardIndex, SearchShardTarget shardTarget, Exception exc) {
|
||||
assertThat(shardIndex, lessThan(expectedNumResults));
|
||||
numQueryFailureListener.incrementAndGet();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onPartialReduce(List<SearchShard> shards, TotalHits totalHits, InternalAggregations aggs, int version) {
|
||||
assertEquals(numReduceListener.incrementAndGet(), version);
|
||||
public void onPartialReduce(List<SearchShard> shards, TotalHits totalHits, InternalAggregations aggs, int reducePhase) {
|
||||
assertEquals(numReduceListener.incrementAndGet(), reducePhase);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onReduce(List<SearchShard> shards, TotalHits totalHits, InternalAggregations aggs) {
|
||||
public void onReduce(List<SearchShard> shards, TotalHits totalHits, InternalAggregations aggs, int reducePhase) {
|
||||
totalHitsListener.set(totalHits);
|
||||
finalAggsListener.set(aggs);
|
||||
numReduceListener.incrementAndGet();
|
||||
|
|
|
@ -24,11 +24,13 @@ import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsResponse
|
|||
import org.elasticsearch.client.Client;
|
||||
import org.elasticsearch.client.node.NodeClient;
|
||||
import org.elasticsearch.index.query.QueryBuilders;
|
||||
import org.elasticsearch.search.SearchShardTarget;
|
||||
import org.elasticsearch.search.aggregations.AggregationBuilders;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregations;
|
||||
import org.elasticsearch.search.builder.SearchSourceBuilder;
|
||||
import org.elasticsearch.search.sort.FieldSortBuilder;
|
||||
import org.elasticsearch.search.sort.SortOrder;
|
||||
import org.elasticsearch.tasks.TaskId;
|
||||
import org.elasticsearch.test.ESSingleNodeTestCase;
|
||||
|
||||
import java.util.ArrayList;
|
||||
|
@ -37,6 +39,7 @@ import java.util.Collections;
|
|||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
|
@ -139,7 +142,8 @@ public class SearchProgressActionListenerIT extends ESSingleNodeTestCase {
|
|||
CountDownLatch latch = new CountDownLatch(1);
|
||||
SearchProgressActionListener listener = new SearchProgressActionListener() {
|
||||
@Override
|
||||
public void onListShards(List<SearchShard> shards, boolean fetchPhase) {
|
||||
public void onListShards(List<SearchShard> shards, List<SearchShard> skippedShards,
|
||||
SearchResponse.Clusters clusters, boolean fetchPhase) {
|
||||
shardsListener.set(shards);
|
||||
assertEquals(fetchPhase, hasFetchPhase);
|
||||
}
|
||||
|
@ -151,7 +155,7 @@ public class SearchProgressActionListenerIT extends ESSingleNodeTestCase {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void onQueryFailure(int shardIndex, Exception exc) {
|
||||
public void onQueryFailure(int shardIndex, SearchShardTarget shardTarget, Exception exc) {
|
||||
assertThat(shardIndex, lessThan(shardsListener.get().size()));
|
||||
numQueryFailures.incrementAndGet();
|
||||
}
|
||||
|
@ -169,12 +173,12 @@ public class SearchProgressActionListenerIT extends ESSingleNodeTestCase {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void onPartialReduce(List<SearchShard> shards, TotalHits totalHits, InternalAggregations aggs, int version) {
|
||||
public void onPartialReduce(List<SearchShard> shards, TotalHits totalHits, InternalAggregations aggs, int reducePhase) {
|
||||
numReduces.incrementAndGet();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onReduce(List<SearchShard> shards, TotalHits totalHits, InternalAggregations aggs) {
|
||||
public void onReduce(List<SearchShard> shards, TotalHits totalHits, InternalAggregations aggs, int reducePhase) {
|
||||
numReduces.incrementAndGet();
|
||||
}
|
||||
|
||||
|
@ -189,7 +193,14 @@ public class SearchProgressActionListenerIT extends ESSingleNodeTestCase {
|
|||
throw new AssertionError();
|
||||
}
|
||||
};
|
||||
client.executeSearchLocally(request, listener);
|
||||
client.executeLocally(SearchAction.INSTANCE, 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);
|
||||
task.setProgressListener(listener);
|
||||
return task;
|
||||
}
|
||||
}, listener);
|
||||
latch.await();
|
||||
|
||||
assertThat(shardsListener.get(), equalTo(expectedShards));
|
||||
|
|
|
@ -41,10 +41,9 @@ public class NodeClientHeadersTests extends AbstractClientHeadersTestCase {
|
|||
@Override
|
||||
protected Client buildClient(Settings headersSettings, ActionType[] testedActions) {
|
||||
Settings settings = HEADER_SETTINGS;
|
||||
TaskManager taskManager = new TaskManager(settings, threadPool, Collections.emptySet());
|
||||
Actions actions = new Actions(settings, threadPool, testedActions);
|
||||
NodeClient client = new NodeClient(settings, threadPool);
|
||||
client.initialize(actions, taskManager, () -> "test", null);
|
||||
client.initialize(actions, () -> "test", null);
|
||||
return client;
|
||||
}
|
||||
|
||||
|
|
|
@ -82,7 +82,7 @@ public class RestValidateQueryActionTests extends AbstractSearchTestCase {
|
|||
final Map<ActionType, TransportAction> actions = new HashMap<>();
|
||||
actions.put(ValidateQueryAction.INSTANCE, transportAction);
|
||||
|
||||
client.initialize(actions, taskManager, () -> "local", null);
|
||||
client.initialize(actions, () -> "local", null);
|
||||
controller.registerHandler(action);
|
||||
}
|
||||
|
||||
|
|
|
@ -1404,8 +1404,7 @@ public class SnapshotResiliencyTests extends ESTestCase {
|
|||
transportService, clusterService, threadPool,
|
||||
snapshotsService, actionFilters, indexNameExpressionResolver
|
||||
));
|
||||
client.initialize(actions, transportService.getTaskManager(),
|
||||
() -> clusterService.localNode().getId(), transportService.getRemoteClusterService());
|
||||
client.initialize(actions, () -> clusterService.localNode().getId(), transportService.getRemoteClusterService());
|
||||
}
|
||||
|
||||
private Repository.Factory getRepoFactory(Environment environment) {
|
||||
|
|
|
@ -1146,6 +1146,7 @@ public abstract class ESRestTestCase extends ESTestCase {
|
|||
case ".logstash-management":
|
||||
case "security_audit_log":
|
||||
case ".slm-history":
|
||||
case ".async-search":
|
||||
return true;
|
||||
default:
|
||||
return false;
|
||||
|
|
|
@ -0,0 +1,42 @@
|
|||
/*
|
||||
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||
* or more contributor license agreements. Licensed under the Elastic License;
|
||||
* you may not use this file except in compliance with the Elastic License.
|
||||
*/
|
||||
|
||||
evaluationDependsOn(xpackModule('core'))
|
||||
|
||||
apply plugin: 'elasticsearch.esplugin'
|
||||
esplugin {
|
||||
name 'x-pack-async-search'
|
||||
description 'A module which allows to track the progress of a search asynchronously.'
|
||||
classname 'org.elasticsearch.xpack.search.AsyncSearch'
|
||||
extendedPlugins = ['x-pack-core']
|
||||
}
|
||||
archivesBaseName = 'x-pack-async-search'
|
||||
|
||||
compileJava.options.compilerArgs << "-Xlint:-rawtypes"
|
||||
compileTestJava.options.compilerArgs << "-Xlint:-rawtypes"
|
||||
|
||||
integTest.enabled = false
|
||||
|
||||
// add all sub-projects of the qa sub-project
|
||||
gradle.projectsEvaluated {
|
||||
project.subprojects
|
||||
.find { it.path == project.path + ":qa" }
|
||||
.subprojects
|
||||
.findAll { it.path.startsWith(project.path + ":qa") }
|
||||
.each { check.dependsOn it.check }
|
||||
}
|
||||
|
||||
dependencies {
|
||||
compileOnly project(":server")
|
||||
|
||||
compileOnly project(path: xpackModule('core'), configuration: 'default')
|
||||
testCompile project(path: xpackModule('core'), configuration: 'testArtifacts')
|
||||
testCompile project(path: xpackModule('ilm'))
|
||||
}
|
||||
|
||||
dependencyLicenses {
|
||||
ignoreSha 'x-pack-core'
|
||||
}
|
|
@ -0,0 +1,8 @@
|
|||
import org.elasticsearch.gradle.test.RestIntegTestTask
|
||||
|
||||
apply plugin: 'elasticsearch.build'
|
||||
test.enabled = false
|
||||
|
||||
dependencies {
|
||||
compile project(':test:framework')
|
||||
}
|
|
@ -0,0 +1,30 @@
|
|||
import org.elasticsearch.gradle.test.RestIntegTestTask
|
||||
|
||||
apply plugin: 'elasticsearch.testclusters'
|
||||
apply plugin: 'elasticsearch.standalone-test'
|
||||
|
||||
restResources {
|
||||
restApi {
|
||||
includeXpack 'async_search'
|
||||
}
|
||||
}
|
||||
|
||||
dependencies {
|
||||
testCompile project(path: xpackModule('core'), configuration: 'testArtifacts')
|
||||
testCompile project(path: xpackModule('async-search'), configuration: 'runtime')
|
||||
}
|
||||
|
||||
task restTest(type: RestIntegTestTask) {
|
||||
mustRunAfter(precommit)
|
||||
}
|
||||
|
||||
testClusters.restTest {
|
||||
testDistribution = 'DEFAULT'
|
||||
setting 'xpack.ml.enabled', 'false'
|
||||
setting 'xpack.monitoring.enabled', 'false'
|
||||
setting 'xpack.security.enabled', 'true'
|
||||
user username: 'async-search-user', password: 'async-search-password'
|
||||
}
|
||||
|
||||
check.dependsOn restTest
|
||||
test.enabled = false
|
|
@ -0,0 +1,35 @@
|
|||
/*
|
||||
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||
* or more contributor license agreements. Licensed under the Elastic License;
|
||||
* you may not use this file except in compliance with the Elastic License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.xpack.search;
|
||||
|
||||
import com.carrotsearch.randomizedtesting.annotations.ParametersFactory;
|
||||
import org.elasticsearch.common.settings.SecureString;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.util.concurrent.ThreadContext;
|
||||
import org.elasticsearch.test.rest.yaml.ClientYamlTestCandidate;
|
||||
import org.elasticsearch.test.rest.yaml.ESClientYamlSuiteTestCase;
|
||||
|
||||
import static org.elasticsearch.xpack.core.security.authc.support.UsernamePasswordToken.basicAuthHeaderValue;
|
||||
|
||||
public class AsyncSearchRestIT extends ESClientYamlSuiteTestCase {
|
||||
|
||||
public AsyncSearchRestIT(final ClientYamlTestCandidate testCandidate) {
|
||||
super(testCandidate);
|
||||
}
|
||||
|
||||
@ParametersFactory
|
||||
public static Iterable<Object[]> parameters() throws Exception {
|
||||
return ESClientYamlSuiteTestCase.createParameters();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Settings restClientSettings() {
|
||||
final String userAuthHeaderValue = basicAuthHeaderValue("async-search-user",
|
||||
new SecureString("async-search-password".toCharArray()));
|
||||
return Settings.builder().put(ThreadContext.PREFIX + ".Authorization", userAuthHeaderValue).build();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,143 @@
|
|||
---
|
||||
"Async search":
|
||||
- do:
|
||||
indices.create:
|
||||
index: test-1
|
||||
body:
|
||||
settings:
|
||||
number_of_shards: "2"
|
||||
|
||||
- do:
|
||||
indices.create:
|
||||
index: test-2
|
||||
body:
|
||||
settings:
|
||||
number_of_shards: "1"
|
||||
|
||||
- do:
|
||||
indices.create:
|
||||
index: test-3
|
||||
body:
|
||||
settings:
|
||||
number_of_shards: "3"
|
||||
|
||||
- do:
|
||||
index:
|
||||
index: test-2
|
||||
body: { max: 2 }
|
||||
|
||||
- do:
|
||||
index:
|
||||
index: test-1
|
||||
body: { max: 1 }
|
||||
|
||||
- do:
|
||||
index:
|
||||
index: test-3
|
||||
body: { max: 3 }
|
||||
|
||||
- do:
|
||||
indices.refresh: {}
|
||||
|
||||
- do:
|
||||
async_search.submit:
|
||||
index: test-*
|
||||
batched_reduce_size: 2
|
||||
wait_for_completion: 10s
|
||||
body:
|
||||
query:
|
||||
match_all: {}
|
||||
aggs:
|
||||
1:
|
||||
max:
|
||||
field: max
|
||||
sort: max
|
||||
|
||||
- is_false: id
|
||||
- match: { version: 6 }
|
||||
- match: { is_partial: false }
|
||||
- length: { response.hits.hits: 3 }
|
||||
- match: { response.hits.hits.0._source.max: 1 }
|
||||
- match: { response.aggregations.1.value: 3.0 }
|
||||
|
||||
- do:
|
||||
async_search.submit:
|
||||
index: test-*
|
||||
batched_reduce_size: 2
|
||||
wait_for_completion: 10s
|
||||
clean_on_completion: false
|
||||
body:
|
||||
aggs:
|
||||
1:
|
||||
max:
|
||||
field: max
|
||||
sort: max
|
||||
|
||||
- set: { id: id }
|
||||
- match: { version: 6 }
|
||||
- match: { is_partial: false }
|
||||
- length: { response.hits.hits: 3 }
|
||||
- match: { response.hits.hits.0._source.max: 1 }
|
||||
- match: { response.aggregations.1.value: 3.0 }
|
||||
|
||||
# test with typed_keys:
|
||||
- do:
|
||||
async_search.submit:
|
||||
index: test-*
|
||||
batched_reduce_size: 2
|
||||
wait_for_completion: 10s
|
||||
clean_on_completion: false
|
||||
typed_keys: true
|
||||
body:
|
||||
aggs:
|
||||
1:
|
||||
max:
|
||||
field: max
|
||||
sort: max
|
||||
|
||||
- set: { id: id }
|
||||
- match: { version: 6 }
|
||||
- match: { is_partial: false }
|
||||
- length: { response.hits.hits: 3 }
|
||||
- match: { response.hits.hits.0._source.max: 1 }
|
||||
- match: { response.aggregations.max#1.value: 3.0 }
|
||||
|
||||
- do:
|
||||
async_search.get:
|
||||
id: "$id"
|
||||
|
||||
- match: { version: 6 }
|
||||
- match: { is_partial: false }
|
||||
- length: { response.hits.hits: 3 }
|
||||
- match: { response.hits.hits.0._source.max: 1 }
|
||||
- match: { response.aggregations.1.value: 3.0 }
|
||||
|
||||
# test with typed_keys:
|
||||
- do:
|
||||
async_search.get:
|
||||
id: "$id"
|
||||
typed_keys: true
|
||||
|
||||
- match: { version: 6 }
|
||||
- match: { is_partial: false }
|
||||
- length: { response.hits.hits: 3 }
|
||||
- match: { response.hits.hits.0._source.max: 1 }
|
||||
- match: { response.aggregations.max#1.value: 3.0 }
|
||||
|
||||
- do:
|
||||
async_search.delete:
|
||||
id: "$id"
|
||||
|
||||
- match: { acknowledged: true }
|
||||
|
||||
- do:
|
||||
catch: missing
|
||||
async_search.get:
|
||||
id: "$id"
|
||||
|
||||
- do:
|
||||
catch: missing
|
||||
async_search.delete:
|
||||
id: "$id"
|
||||
|
||||
|
|
@ -0,0 +1,19 @@
|
|||
apply plugin: 'elasticsearch.testclusters'
|
||||
apply plugin: 'elasticsearch.standalone-rest-test'
|
||||
apply plugin: 'elasticsearch.rest-test'
|
||||
|
||||
dependencies {
|
||||
testCompile project(path: xpackModule('core'), configuration: 'testArtifacts')
|
||||
testCompile project(path: xpackModule('async-search'), configuration: 'runtime')
|
||||
testCompile project(':x-pack:plugin:async-search:qa')
|
||||
}
|
||||
|
||||
testClusters.integTest {
|
||||
testDistribution = 'DEFAULT'
|
||||
setting 'xpack.license.self_generated.type', 'trial'
|
||||
setting 'xpack.security.enabled', 'true'
|
||||
extraConfigFile 'roles.yml', file('roles.yml')
|
||||
user username: "test-admin", password: 'x-pack-test-password', role: "test-admin"
|
||||
user username: "user1", password: 'x-pack-test-password', role: "user1"
|
||||
user username: "user2", password: 'x-pack-test-password', role: "user2"
|
||||
}
|
|
@ -0,0 +1,33 @@
|
|||
# All cluster rights
|
||||
# All operations on all indices
|
||||
# Run as all users
|
||||
test-admin:
|
||||
cluster:
|
||||
- all
|
||||
indices:
|
||||
- names: '*'
|
||||
privileges: [ all ]
|
||||
run_as:
|
||||
- '*'
|
||||
|
||||
user1:
|
||||
cluster:
|
||||
- cluster:monitor/main
|
||||
indices:
|
||||
- names: ['index-user1', 'index' ]
|
||||
privileges:
|
||||
- read
|
||||
- write
|
||||
- create_index
|
||||
- indices:admin/refresh
|
||||
|
||||
user2:
|
||||
cluster:
|
||||
- cluster:monitor/main
|
||||
indices:
|
||||
- names: [ 'index-user2', 'index' ]
|
||||
privileges:
|
||||
- read
|
||||
- write
|
||||
- create_index
|
||||
- indices:admin/refresh
|
|
@ -0,0 +1,172 @@
|
|||
/*
|
||||
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||
* or more contributor license agreements. Licensed under the Elastic License;
|
||||
* you may not use this file except in compliance with the Elastic License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.xpack.search;
|
||||
|
||||
import org.apache.http.util.EntityUtils;
|
||||
import org.elasticsearch.client.Request;
|
||||
import org.elasticsearch.client.RequestOptions;
|
||||
import org.elasticsearch.client.Response;
|
||||
import org.elasticsearch.client.ResponseException;
|
||||
import org.elasticsearch.common.Strings;
|
||||
import org.elasticsearch.common.settings.SecureString;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.common.util.concurrent.ThreadContext;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.common.xcontent.XContentHelper;
|
||||
import org.elasticsearch.common.xcontent.json.JsonXContent;
|
||||
import org.junit.Before;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Map;
|
||||
|
||||
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
|
||||
import static org.elasticsearch.xpack.core.security.authc.AuthenticationServiceField.RUN_AS_USER_HEADER;
|
||||
import static org.elasticsearch.xpack.core.security.authc.support.UsernamePasswordToken.basicAuthHeaderValue;
|
||||
import static org.elasticsearch.xpack.search.AsyncSearchIndexService.INDEX;
|
||||
import static org.hamcrest.Matchers.containsString;
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
|
||||
public class AsyncSearchSecurityIT extends AsyncSearchRestTestCase {
|
||||
/**
|
||||
* All tests run as a superuser but use <code>es-security-runas-user</code> to become a less privileged user.
|
||||
*/
|
||||
@Override
|
||||
protected Settings restClientSettings() {
|
||||
String token = basicAuthHeaderValue("test-admin", new SecureString("x-pack-test-password".toCharArray()));
|
||||
return Settings.builder()
|
||||
.put(ThreadContext.PREFIX + ".Authorization", token)
|
||||
.build();
|
||||
}
|
||||
|
||||
@Before
|
||||
private void indexDocuments() throws IOException {
|
||||
createIndex("index", Settings.EMPTY);
|
||||
index("index", "0", "foo", "bar");
|
||||
refresh("index");
|
||||
|
||||
createIndex("index-user1", Settings.EMPTY);
|
||||
index("index-user1", "0", "foo", "bar");
|
||||
refresh("index-user1");
|
||||
|
||||
createIndex("index-user2", Settings.EMPTY);
|
||||
index("index-user2", "0", "foo", "bar");
|
||||
refresh("index-user2");
|
||||
}
|
||||
|
||||
public void testWithUsers() throws Exception {
|
||||
testCase("user1", "user2");
|
||||
testCase("user2", "user1");
|
||||
}
|
||||
|
||||
private void testCase(String user, String other) throws Exception {
|
||||
for (String indexName : new String[] {"index", "index-" + user}) {
|
||||
Response submitResp = submitAsyncSearch(indexName, "foo:bar", TimeValue.timeValueSeconds(10), user);
|
||||
assertOK(submitResp);
|
||||
String id = extractResponseId(submitResp);
|
||||
Response getResp = getAsyncSearch(id, user);
|
||||
assertOK(getResp);
|
||||
|
||||
// other cannot access the result
|
||||
ResponseException exc = expectThrows(ResponseException.class, () -> getAsyncSearch(id, other));
|
||||
assertThat(exc.getResponse().getStatusLine().getStatusCode(), equalTo(404));
|
||||
|
||||
// other cannot delete the result
|
||||
exc = expectThrows(ResponseException.class, () -> deleteAsyncSearch(id, other));
|
||||
assertThat(exc.getResponse().getStatusLine().getStatusCode(), equalTo(404));
|
||||
|
||||
// other and user cannot access the result from direct get calls
|
||||
AsyncSearchId searchId = AsyncSearchId.decode(id);
|
||||
for (String runAs : new String[] {user, other}) {
|
||||
exc = expectThrows(ResponseException.class, () -> get(INDEX, searchId.getDocId(), runAs));
|
||||
assertThat(exc.getResponse().getStatusLine().getStatusCode(), equalTo(403));
|
||||
assertThat(exc.getMessage(), containsString("unauthorized"));
|
||||
}
|
||||
|
||||
Response delResp = deleteAsyncSearch(id, user);
|
||||
assertOK(delResp);
|
||||
}
|
||||
ResponseException exc = expectThrows(ResponseException.class,
|
||||
() -> submitAsyncSearch("index-" + other, "*", TimeValue.timeValueSeconds(10), user));
|
||||
assertThat(exc.getResponse().getStatusLine().getStatusCode(), equalTo(403));
|
||||
assertThat(exc.getMessage(), containsString("unauthorized"));
|
||||
}
|
||||
|
||||
static String extractResponseId(Response response) throws IOException {
|
||||
Map<String, Object> map = toMap(response);
|
||||
return (String) map.get("id");
|
||||
}
|
||||
|
||||
static void index(String index, String id, Object... fields) throws IOException {
|
||||
XContentBuilder document = jsonBuilder().startObject();
|
||||
for (int i = 0; i < fields.length; i += 2) {
|
||||
document.field((String) fields[i], fields[i + 1]);
|
||||
}
|
||||
document.endObject();
|
||||
final Request request = new Request("POST", "/" + index + "/_doc/" + id);
|
||||
request.setJsonEntity(Strings.toString(document));
|
||||
assertOK(client().performRequest(request));
|
||||
}
|
||||
|
||||
static void refresh(String index) throws IOException {
|
||||
assertOK(adminClient().performRequest(new Request("POST", "/" + index + "/_refresh")));
|
||||
}
|
||||
|
||||
static Response get(String index, String id, String user) throws IOException {
|
||||
final Request request = new Request("GET", "/" + index + "/_doc/" + id);
|
||||
setRunAsHeader(request, user);
|
||||
return client().performRequest(request);
|
||||
}
|
||||
|
||||
static Response submitAsyncSearch(String indexName, String query, String user) throws IOException {
|
||||
return submitAsyncSearch(indexName, query, TimeValue.MINUS_ONE, user);
|
||||
}
|
||||
|
||||
static Response submitAsyncSearch(String indexName, String query, TimeValue waitForCompletion, String user) throws IOException {
|
||||
final Request request = new Request("GET", indexName + "/_async_search");
|
||||
setRunAsHeader(request, user);
|
||||
request.addParameter("q", query);
|
||||
request.addParameter("wait_for_completion", waitForCompletion.toString());
|
||||
// we do the cleanup explicitly
|
||||
request.addParameter("clean_on_completion", "false");
|
||||
return client().performRequest(request);
|
||||
}
|
||||
|
||||
static Response search(String indexName, String query, String user) throws IOException {
|
||||
final Request request = new Request("GET", indexName + "/_search");
|
||||
setRunAsHeader(request, user);
|
||||
request.addParameter("q", query);
|
||||
return client().performRequest(request);
|
||||
}
|
||||
|
||||
static Response getAsyncSearch(String id, String user) throws IOException {
|
||||
final Request request = new Request("GET", "/_async_search/" + id);
|
||||
setRunAsHeader(request, user);
|
||||
request.addParameter("wait_for_completion", "0ms");
|
||||
return client().performRequest(request);
|
||||
}
|
||||
|
||||
static Response deleteAsyncSearch(String id, String user) throws IOException {
|
||||
final Request request = new Request("DELETE", "/_async_search/" + id);
|
||||
setRunAsHeader(request, user);
|
||||
return client().performRequest(request);
|
||||
}
|
||||
|
||||
static Map<String, Object> toMap(Response response) throws IOException {
|
||||
return toMap(EntityUtils.toString(response.getEntity()));
|
||||
}
|
||||
|
||||
static Map<String, Object> toMap(String response) {
|
||||
return XContentHelper.convertToMap(JsonXContent.jsonXContent, response, false);
|
||||
}
|
||||
|
||||
static void setRunAsHeader(Request request, String user) {
|
||||
final RequestOptions.Builder builder = RequestOptions.DEFAULT.toBuilder();
|
||||
builder.addHeader(RUN_AS_USER_HEADER, user);
|
||||
request.setOptions(builder);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,15 @@
|
|||
/*
|
||||
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||
* or more contributor license agreements. Licensed under the Elastic License;
|
||||
* you may not use this file except in compliance with the Elastic License.
|
||||
*/
|
||||
package org.elasticsearch.xpack.search;
|
||||
|
||||
import org.elasticsearch.test.rest.ESRestTestCase;
|
||||
|
||||
public class AsyncSearchRestTestCase extends ESRestTestCase {
|
||||
@Override
|
||||
protected boolean preserveClusterUponCompletion() {
|
||||
return true;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,92 @@
|
|||
/*
|
||||
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||
* or more contributor license agreements. Licensed under the Elastic License;
|
||||
* you may not use this file except in compliance with the Elastic License.
|
||||
*/
|
||||
package org.elasticsearch.xpack.search;
|
||||
|
||||
import org.elasticsearch.action.ActionRequest;
|
||||
import org.elasticsearch.action.ActionResponse;
|
||||
import org.elasticsearch.client.Client;
|
||||
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNodes;
|
||||
import org.elasticsearch.cluster.service.ClusterService;
|
||||
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
|
||||
import org.elasticsearch.common.settings.ClusterSettings;
|
||||
import org.elasticsearch.common.settings.IndexScopedSettings;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.settings.SettingsFilter;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
|
||||
import org.elasticsearch.env.Environment;
|
||||
import org.elasticsearch.env.NodeEnvironment;
|
||||
import org.elasticsearch.plugins.ActionPlugin;
|
||||
import org.elasticsearch.plugins.Plugin;
|
||||
import org.elasticsearch.rest.RestController;
|
||||
import org.elasticsearch.rest.RestHandler;
|
||||
import org.elasticsearch.script.ScriptService;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
import org.elasticsearch.watcher.ResourceWatcherService;
|
||||
import org.elasticsearch.xpack.core.search.action.DeleteAsyncSearchAction;
|
||||
import org.elasticsearch.xpack.core.search.action.GetAsyncSearchAction;
|
||||
import org.elasticsearch.xpack.core.search.action.SubmitAsyncSearchAction;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.function.Supplier;
|
||||
|
||||
public final class AsyncSearch extends Plugin implements ActionPlugin {
|
||||
private final Settings settings;
|
||||
|
||||
public AsyncSearch(Settings settings) {
|
||||
this.settings = settings;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<ActionHandler<? extends ActionRequest, ? extends ActionResponse>> getActions() {
|
||||
return Arrays.asList(
|
||||
new ActionHandler<>(SubmitAsyncSearchAction.INSTANCE, TransportSubmitAsyncSearchAction.class),
|
||||
new ActionHandler<>(GetAsyncSearchAction.INSTANCE, TransportGetAsyncSearchAction.class),
|
||||
new ActionHandler<>(DeleteAsyncSearchAction.INSTANCE, TransportDeleteAsyncSearchAction.class)
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<RestHandler> getRestHandlers(Settings settings, RestController restController, ClusterSettings clusterSettings,
|
||||
IndexScopedSettings indexScopedSettings, SettingsFilter settingsFilter,
|
||||
IndexNameExpressionResolver indexNameExpressionResolver,
|
||||
Supplier<DiscoveryNodes> nodesInCluster) {
|
||||
return Arrays.asList(
|
||||
new RestSubmitAsyncSearchAction(),
|
||||
new RestGetAsyncSearchAction(),
|
||||
new RestDeleteAsyncSearchAction()
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Collection<Object> createComponents(Client client,
|
||||
ClusterService clusterService,
|
||||
ThreadPool threadPool,
|
||||
ResourceWatcherService resourceWatcherService,
|
||||
ScriptService scriptService,
|
||||
NamedXContentRegistry xContentRegistry,
|
||||
Environment environment,
|
||||
NodeEnvironment nodeEnvironment,
|
||||
NamedWriteableRegistry namedWriteableRegistry,
|
||||
IndexNameExpressionResolver indexNameExpressionResolver) {
|
||||
if (DiscoveryNode.isDataNode(environment.settings())) {
|
||||
// only data nodes should be eligible to run the maintenance service.
|
||||
AsyncSearchIndexService indexService =
|
||||
new AsyncSearchIndexService(clusterService, threadPool.getThreadContext(), client, namedWriteableRegistry);
|
||||
AsyncSearchMaintenanceService maintenanceService =
|
||||
new AsyncSearchMaintenanceService(nodeEnvironment.nodeId(), threadPool, indexService, TimeValue.timeValueHours(1));
|
||||
clusterService.addListener(maintenanceService);
|
||||
return Collections.singletonList(maintenanceService);
|
||||
} else {
|
||||
return Collections.emptyList();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,106 @@
|
|||
/*
|
||||
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||
* or more contributor license agreements. Licensed under the Elastic License;
|
||||
* you may not use this file except in compliance with the Elastic License.
|
||||
*/
|
||||
package org.elasticsearch.xpack.search;
|
||||
|
||||
import org.elasticsearch.common.bytes.BytesReference;
|
||||
import org.elasticsearch.common.io.stream.ByteBufferStreamInput;
|
||||
import org.elasticsearch.common.io.stream.BytesStreamOutput;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.tasks.TaskId;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Base64;
|
||||
import java.util.Objects;
|
||||
|
||||
/**
|
||||
* A class that contains all information related to a submitted async search.
|
||||
*/
|
||||
class AsyncSearchId {
|
||||
private final String docId;
|
||||
private final TaskId taskId;
|
||||
private final String encoded;
|
||||
|
||||
AsyncSearchId(String docId, TaskId taskId) {
|
||||
this.docId = docId;
|
||||
this.taskId = taskId;
|
||||
this.encoded = encode(docId, taskId);
|
||||
}
|
||||
|
||||
/**
|
||||
* The document id of the response in the index if the task is not running.
|
||||
*/
|
||||
String getDocId() {
|
||||
return docId;
|
||||
}
|
||||
|
||||
/**
|
||||
* The {@link TaskId} of the async search in the task manager.
|
||||
*/
|
||||
TaskId getTaskId() {
|
||||
return taskId;
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the encoded string that represents this search.
|
||||
*/
|
||||
String getEncoded() {
|
||||
return encoded;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (this == o) return true;
|
||||
if (o == null || getClass() != o.getClass()) return false;
|
||||
AsyncSearchId searchId = (AsyncSearchId) o;
|
||||
return docId.equals(searchId.docId) &&
|
||||
taskId.equals(searchId.taskId);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hash(docId, taskId);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "AsyncSearchId{" +
|
||||
"docId='" + docId + '\'' +
|
||||
", taskId=" + taskId +
|
||||
'}';
|
||||
}
|
||||
|
||||
/**
|
||||
* Encodes the informations needed to retrieve a async search response
|
||||
* in a base64 encoded string.
|
||||
*/
|
||||
static String encode(String docId, TaskId taskId) {
|
||||
try (BytesStreamOutput out = new BytesStreamOutput()) {
|
||||
out.writeString(docId);
|
||||
out.writeString(taskId.toString());
|
||||
return Base64.getUrlEncoder().encodeToString(BytesReference.toBytes(out.bytes()));
|
||||
} catch (IOException e) {
|
||||
throw new IllegalArgumentException(e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Decodes a base64 encoded string into an {@link AsyncSearchId} that can be used
|
||||
* to retrieve the response of an async search.
|
||||
*/
|
||||
static AsyncSearchId decode(String id) {
|
||||
final AsyncSearchId searchId;
|
||||
try (StreamInput in = new ByteBufferStreamInput(ByteBuffer.wrap(Base64.getUrlDecoder().decode(id)))) {
|
||||
searchId = new AsyncSearchId(in.readString(), new TaskId(in.readString()));
|
||||
if (in.available() > 0) {
|
||||
throw new IllegalArgumentException("invalid id:[" + id + "]");
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new IllegalArgumentException("invalid id:[" + id + "]");
|
||||
}
|
||||
return searchId;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,342 @@
|
|||
/*
|
||||
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||
* or more contributor license agreements. Licensed under the Elastic License;
|
||||
* you may not use this file except in compliance with the Elastic License.
|
||||
*/
|
||||
package org.elasticsearch.xpack.search;
|
||||
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
import org.apache.logging.log4j.message.ParameterizedMessage;
|
||||
import org.elasticsearch.ExceptionsHelper;
|
||||
import org.elasticsearch.ResourceAlreadyExistsException;
|
||||
import org.elasticsearch.ResourceNotFoundException;
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.delete.DeleteRequest;
|
||||
import org.elasticsearch.action.get.GetRequest;
|
||||
import org.elasticsearch.action.index.IndexRequest;
|
||||
import org.elasticsearch.action.index.IndexResponse;
|
||||
import org.elasticsearch.action.support.master.AcknowledgedResponse;
|
||||
import org.elasticsearch.action.update.UpdateRequest;
|
||||
import org.elasticsearch.action.update.UpdateResponse;
|
||||
import org.elasticsearch.client.Client;
|
||||
import org.elasticsearch.client.OriginSettingClient;
|
||||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||
import org.elasticsearch.cluster.service.ClusterService;
|
||||
import org.elasticsearch.common.bytes.BytesReference;
|
||||
import org.elasticsearch.common.io.stream.ByteBufferStreamInput;
|
||||
import org.elasticsearch.common.io.stream.BytesStreamOutput;
|
||||
import org.elasticsearch.common.io.stream.NamedWriteableAwareStreamInput;
|
||||
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.util.concurrent.ThreadContext;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.common.xcontent.XContentType;
|
||||
import org.elasticsearch.rest.RestStatus;
|
||||
import org.elasticsearch.tasks.Task;
|
||||
import org.elasticsearch.tasks.TaskManager;
|
||||
import org.elasticsearch.xpack.core.search.action.AsyncSearchResponse;
|
||||
import org.elasticsearch.xpack.core.security.authc.Authentication;
|
||||
import org.elasticsearch.xpack.core.security.authc.support.AuthenticationContextSerializer;
|
||||
import org.elasticsearch.xpack.core.security.SecurityContext;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Base64;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder;
|
||||
import static org.elasticsearch.index.mapper.MapperService.SINGLE_MAPPING_NAME;
|
||||
import static org.elasticsearch.xpack.core.ClientHelper.ASYNC_SEARCH_ORIGIN;
|
||||
import static org.elasticsearch.xpack.core.security.authc.AuthenticationField.AUTHENTICATION_KEY;
|
||||
|
||||
/**
|
||||
* A service that exposes the CRUD operations for the async-search index.
|
||||
*/
|
||||
class AsyncSearchIndexService {
|
||||
private static final Logger logger = LogManager.getLogger(AsyncSearchIndexService.class);
|
||||
|
||||
public static final String INDEX = ".async-search";
|
||||
|
||||
public static final String HEADERS_FIELD = "headers";
|
||||
public static final String EXPIRATION_TIME_FIELD = "expiration_time";
|
||||
public static final String RESULT_FIELD = "result";
|
||||
|
||||
public static Settings settings() {
|
||||
return Settings.builder()
|
||||
.put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1)
|
||||
.put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 1)
|
||||
.build();
|
||||
}
|
||||
|
||||
public static XContentBuilder mappings() throws IOException {
|
||||
XContentBuilder builder = jsonBuilder()
|
||||
.startObject()
|
||||
.startObject(SINGLE_MAPPING_NAME)
|
||||
.startObject("_meta")
|
||||
.field("version", Version.CURRENT)
|
||||
.endObject()
|
||||
.field("dynamic", "strict")
|
||||
.startObject("properties")
|
||||
.startObject(HEADERS_FIELD)
|
||||
.field("type", "object")
|
||||
.field("enabled", "false")
|
||||
.endObject()
|
||||
.startObject(RESULT_FIELD)
|
||||
.field("type", "object")
|
||||
.field("enabled", "false")
|
||||
.endObject()
|
||||
.startObject(EXPIRATION_TIME_FIELD)
|
||||
.field("type", "long")
|
||||
.endObject()
|
||||
.endObject()
|
||||
.endObject()
|
||||
.endObject();
|
||||
return builder;
|
||||
}
|
||||
|
||||
private final ClusterService clusterService;
|
||||
private final Client client;
|
||||
private final SecurityContext securityContext;
|
||||
private final NamedWriteableRegistry registry;
|
||||
|
||||
AsyncSearchIndexService(ClusterService clusterService,
|
||||
ThreadContext threadContext,
|
||||
Client client,
|
||||
NamedWriteableRegistry registry) {
|
||||
this.clusterService = clusterService;
|
||||
this.securityContext = new SecurityContext(clusterService.getSettings(), threadContext);
|
||||
this.client = new OriginSettingClient(client, ASYNC_SEARCH_ORIGIN);
|
||||
this.registry = registry;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the internal client with origin.
|
||||
*/
|
||||
Client getClient() {
|
||||
return client;
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates the index with the expected settings and mappings if it doesn't exist.
|
||||
*/
|
||||
void createIndexIfNecessary(ActionListener<Void> listener) {
|
||||
if (clusterService.state().routingTable().hasIndex(AsyncSearchIndexService.INDEX) == false) {
|
||||
try {
|
||||
client.admin().indices().prepareCreate(INDEX)
|
||||
.setSettings(settings())
|
||||
.addMapping(SINGLE_MAPPING_NAME, mappings())
|
||||
.execute(ActionListener.wrap(
|
||||
resp -> listener.onResponse(null),
|
||||
exc -> {
|
||||
if (ExceptionsHelper.unwrapCause(exc) instanceof ResourceAlreadyExistsException) {
|
||||
listener.onResponse(null);
|
||||
} else {
|
||||
logger.error("failed to create async-search index", exc);
|
||||
listener.onFailure(exc);
|
||||
}
|
||||
}));
|
||||
} catch (Exception exc) {
|
||||
logger.error("failed to create async-search index", exc);
|
||||
listener.onFailure(exc);
|
||||
}
|
||||
} else {
|
||||
listener.onResponse(null);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Stores the initial response with the original headers of the authenticated user
|
||||
* and the expected expiration time.
|
||||
*/
|
||||
void storeInitialResponse(String docId,
|
||||
Map<String, String> headers,
|
||||
AsyncSearchResponse response,
|
||||
ActionListener<IndexResponse> listener) throws IOException {
|
||||
Map<String, Object> source = new HashMap<>();
|
||||
source.put(HEADERS_FIELD, headers);
|
||||
source.put(EXPIRATION_TIME_FIELD, response.getExpirationTime());
|
||||
source.put(RESULT_FIELD, encodeResponse(response));
|
||||
IndexRequest indexRequest = new IndexRequest(INDEX)
|
||||
.create(true)
|
||||
.id(docId)
|
||||
.source(source, XContentType.JSON);
|
||||
createIndexIfNecessary(ActionListener.wrap(v -> client.index(indexRequest, listener), listener::onFailure));
|
||||
}
|
||||
|
||||
/**
|
||||
* Stores the final response if the place-holder document is still present (update).
|
||||
*/
|
||||
void storeFinalResponse(String docId,
|
||||
AsyncSearchResponse response,
|
||||
ActionListener<UpdateResponse> listener) throws IOException {
|
||||
Map<String, Object> source = new HashMap<>();
|
||||
source.put(RESULT_FIELD, encodeResponse(response));
|
||||
UpdateRequest request = new UpdateRequest()
|
||||
.index(INDEX)
|
||||
.id(docId)
|
||||
.doc(source, XContentType.JSON);
|
||||
createIndexIfNecessary(ActionListener.wrap(v -> client.update(request, listener), listener::onFailure));
|
||||
}
|
||||
|
||||
/**
|
||||
* Updates the expiration time of the provided <code>docId</code> if the place-holder
|
||||
* document is still present (update).
|
||||
*/
|
||||
void updateExpirationTime(String docId,
|
||||
long expirationTimeMillis,
|
||||
ActionListener<UpdateResponse> listener) {
|
||||
Map<String, Object> source = Collections.singletonMap(EXPIRATION_TIME_FIELD, expirationTimeMillis);
|
||||
UpdateRequest request = new UpdateRequest().index(INDEX)
|
||||
.id(docId)
|
||||
.doc(source, XContentType.JSON);
|
||||
createIndexIfNecessary(ActionListener.wrap(v -> client.update(request, listener), listener::onFailure));
|
||||
}
|
||||
|
||||
/**
|
||||
* Deletes the provided <code>searchId</code> from the index if present.
|
||||
*/
|
||||
void deleteResponse(AsyncSearchId searchId,
|
||||
boolean failIfNotFound,
|
||||
ActionListener<AcknowledgedResponse> listener) {
|
||||
DeleteRequest request = new DeleteRequest(INDEX).id(searchId.getDocId());
|
||||
createIndexIfNecessary(
|
||||
ActionListener.wrap(v -> client.delete(request,
|
||||
ActionListener.wrap(
|
||||
resp -> {
|
||||
if (resp.status() == RestStatus.NOT_FOUND && failIfNotFound) {
|
||||
listener.onFailure(new ResourceNotFoundException(searchId.getEncoded()));
|
||||
} else {
|
||||
listener.onResponse(new AcknowledgedResponse(true));
|
||||
}
|
||||
},
|
||||
exc -> {
|
||||
logger.error(() -> new ParameterizedMessage("failed to clean async-search [{}]", searchId.getEncoded()), exc);
|
||||
listener.onFailure(exc);
|
||||
})),
|
||||
listener::onFailure));
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the {@link AsyncSearchTask} if the provided <code>searchId</code>
|
||||
* is registered in the task manager, <code>null</code> otherwise.
|
||||
*
|
||||
* This method throws a {@link ResourceNotFoundException} if the authenticated user
|
||||
* is not the creator of the original task.
|
||||
*/
|
||||
AsyncSearchTask getTask(TaskManager taskManager, AsyncSearchId searchId) throws IOException {
|
||||
Task task = taskManager.getTask(searchId.getTaskId().getId());
|
||||
if (task instanceof AsyncSearchTask == false) {
|
||||
return null;
|
||||
}
|
||||
AsyncSearchTask searchTask = (AsyncSearchTask) task;
|
||||
if (searchTask.getSearchId().equals(searchId) == false) {
|
||||
return null;
|
||||
}
|
||||
|
||||
// Check authentication for the user
|
||||
final Authentication auth = securityContext.getAuthentication();
|
||||
if (ensureAuthenticatedUserIsSame(searchTask.getOriginHeaders(), auth) == false) {
|
||||
throw new ResourceNotFoundException(searchId.getEncoded() + " not found");
|
||||
}
|
||||
return searchTask;
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the response from the index if present, or delegate a {@link ResourceNotFoundException}
|
||||
* failure to the provided listener if not.
|
||||
*/
|
||||
void getResponse(AsyncSearchId searchId,
|
||||
ActionListener<AsyncSearchResponse> listener) {
|
||||
final Authentication current = securityContext.getAuthentication();
|
||||
GetRequest internalGet = new GetRequest(INDEX)
|
||||
.preference(searchId.getEncoded())
|
||||
.id(searchId.getDocId());
|
||||
client.get(internalGet, ActionListener.wrap(
|
||||
get -> {
|
||||
if (get.isExists() == false) {
|
||||
listener.onFailure(new ResourceNotFoundException(searchId.getEncoded()));
|
||||
return;
|
||||
}
|
||||
|
||||
// check the authentication of the current user against the user that initiated the async search
|
||||
@SuppressWarnings("unchecked")
|
||||
Map<String, String> headers = (Map<String, String>) get.getSource().get(HEADERS_FIELD);
|
||||
if (ensureAuthenticatedUserIsSame(headers, current) == false) {
|
||||
listener.onFailure(new ResourceNotFoundException(searchId.getEncoded()));
|
||||
return;
|
||||
}
|
||||
|
||||
String encoded = (String) get.getSource().get(RESULT_FIELD);
|
||||
listener.onResponse(encoded != null ? decodeResponse(encoded) : null);
|
||||
},
|
||||
listener::onFailure
|
||||
));
|
||||
}
|
||||
|
||||
/**
|
||||
* Extracts the authentication from the original headers and checks that it matches
|
||||
* the current user. This function returns always <code>true</code> if the provided
|
||||
* <code>headers</code> do not contain any authentication.
|
||||
*/
|
||||
boolean ensureAuthenticatedUserIsSame(Map<String, String> originHeaders, Authentication current) throws IOException {
|
||||
if (originHeaders == null || originHeaders.containsKey(AUTHENTICATION_KEY) == false) {
|
||||
// no authorization attached to the original request
|
||||
return true;
|
||||
}
|
||||
if (current == null) {
|
||||
// origin is an authenticated user but current is not
|
||||
return false;
|
||||
}
|
||||
Authentication origin = AuthenticationContextSerializer.decode(originHeaders.get(AUTHENTICATION_KEY));
|
||||
return ensureAuthenticatedUserIsSame(origin, current);
|
||||
}
|
||||
|
||||
/**
|
||||
* Compares the {@link Authentication} that was used to create the {@link AsyncSearchId} with the
|
||||
* current authentication.
|
||||
*/
|
||||
boolean ensureAuthenticatedUserIsSame(Authentication original, Authentication current) {
|
||||
final boolean samePrincipal = original.getUser().principal().equals(current.getUser().principal());
|
||||
final boolean sameRealmType;
|
||||
if (original.getUser().isRunAs()) {
|
||||
if (current.getUser().isRunAs()) {
|
||||
sameRealmType = original.getLookedUpBy().getType().equals(current.getLookedUpBy().getType());
|
||||
} else {
|
||||
sameRealmType = original.getLookedUpBy().getType().equals(current.getAuthenticatedBy().getType());
|
||||
}
|
||||
} else if (current.getUser().isRunAs()) {
|
||||
sameRealmType = original.getAuthenticatedBy().getType().equals(current.getLookedUpBy().getType());
|
||||
} else {
|
||||
sameRealmType = original.getAuthenticatedBy().getType().equals(current.getAuthenticatedBy().getType());
|
||||
}
|
||||
return samePrincipal && sameRealmType;
|
||||
}
|
||||
|
||||
/**
|
||||
* Encode the provided response in a binary form using base64 encoding.
|
||||
*/
|
||||
String encodeResponse(AsyncSearchResponse response) throws IOException {
|
||||
try (BytesStreamOutput out = new BytesStreamOutput()) {
|
||||
Version.writeVersion(Version.CURRENT, out);
|
||||
response.writeTo(out);
|
||||
return Base64.getEncoder().encodeToString(BytesReference.toBytes(out.bytes()));
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Decode the provided base-64 bytes into a {@link AsyncSearchResponse}.
|
||||
*/
|
||||
AsyncSearchResponse decodeResponse(String value) throws IOException {
|
||||
try (ByteBufferStreamInput buf = new ByteBufferStreamInput(ByteBuffer.wrap(Base64.getDecoder().decode(value)))) {
|
||||
try (StreamInput in = new NamedWriteableAwareStreamInput(buf, registry)) {
|
||||
in.setVersion(Version.readVersion(in));
|
||||
return new AsyncSearchResponse(in);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,117 @@
|
|||
/*
|
||||
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||
* or more contributor license agreements. Licensed under the Elastic License;
|
||||
* you may not use this file except in compliance with the Elastic License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.xpack.search;
|
||||
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.cluster.ClusterChangedEvent;
|
||||
import org.elasticsearch.cluster.ClusterState;
|
||||
import org.elasticsearch.cluster.ClusterStateListener;
|
||||
import org.elasticsearch.cluster.routing.IndexRoutingTable;
|
||||
import org.elasticsearch.common.lease.Releasable;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException;
|
||||
import org.elasticsearch.gateway.GatewayService;
|
||||
import org.elasticsearch.index.query.QueryBuilders;
|
||||
import org.elasticsearch.index.reindex.DeleteByQueryAction;
|
||||
import org.elasticsearch.index.reindex.DeleteByQueryRequest;
|
||||
import org.elasticsearch.threadpool.Scheduler;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
|
||||
import static org.elasticsearch.xpack.search.AsyncSearchIndexService.EXPIRATION_TIME_FIELD;
|
||||
|
||||
/**
|
||||
* A service that runs a periodic cleanup over the async-search index.
|
||||
*/
|
||||
class AsyncSearchMaintenanceService implements Releasable, ClusterStateListener {
|
||||
private static final Logger logger = LogManager.getLogger(AsyncSearchMaintenanceService.class);
|
||||
|
||||
private final String localNodeId;
|
||||
private final ThreadPool threadPool;
|
||||
private final AsyncSearchIndexService indexService;
|
||||
private final TimeValue delay;
|
||||
|
||||
private final AtomicBoolean isCleanupRunning = new AtomicBoolean(false);
|
||||
private final AtomicBoolean isClosed = new AtomicBoolean(false);
|
||||
private volatile Scheduler.Cancellable cancellable;
|
||||
|
||||
AsyncSearchMaintenanceService(String localNodeId,
|
||||
ThreadPool threadPool,
|
||||
AsyncSearchIndexService indexService,
|
||||
TimeValue delay) {
|
||||
this.localNodeId = localNodeId;
|
||||
this.threadPool = threadPool;
|
||||
this.indexService = indexService;
|
||||
this.delay = delay;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void clusterChanged(ClusterChangedEvent event) {
|
||||
final ClusterState state = event.state();
|
||||
if (state.blocks().hasGlobalBlock(GatewayService.STATE_NOT_RECOVERED_BLOCK)) {
|
||||
// Wait until the gateway has recovered from disk.
|
||||
return;
|
||||
}
|
||||
tryStartCleanup(state);
|
||||
}
|
||||
|
||||
void tryStartCleanup(ClusterState state) {
|
||||
if (isClosed.get()) {
|
||||
return;
|
||||
}
|
||||
IndexRoutingTable indexRouting = state.routingTable().index(AsyncSearchIndexService.INDEX);
|
||||
if (indexRouting == null) {
|
||||
if (isCleanupRunning.compareAndSet(true, false)) {
|
||||
close();
|
||||
}
|
||||
return;
|
||||
}
|
||||
String primaryNodeId = indexRouting.shard(0).primaryShard().currentNodeId();
|
||||
if (localNodeId.equals(primaryNodeId)) {
|
||||
if (isCleanupRunning.compareAndSet(false, true)) {
|
||||
executeNextCleanup();
|
||||
}
|
||||
} else if (isCleanupRunning.compareAndSet(true, false)) {
|
||||
close();
|
||||
}
|
||||
}
|
||||
|
||||
synchronized void executeNextCleanup() {
|
||||
if (isClosed.get() == false && isCleanupRunning.get()) {
|
||||
long nowInMillis = System.currentTimeMillis();
|
||||
DeleteByQueryRequest toDelete = new DeleteByQueryRequest()
|
||||
.setQuery(QueryBuilders.rangeQuery(EXPIRATION_TIME_FIELD).lte(nowInMillis));
|
||||
indexService.getClient()
|
||||
.execute(DeleteByQueryAction.INSTANCE, toDelete, ActionListener.wrap(() -> scheduleNextCleanup()));
|
||||
}
|
||||
}
|
||||
|
||||
synchronized void scheduleNextCleanup() {
|
||||
if (isClosed.get() == false && isCleanupRunning.get()) {
|
||||
try {
|
||||
cancellable = threadPool.schedule(this::executeNextCleanup, delay, ThreadPool.Names.GENERIC);
|
||||
} catch (EsRejectedExecutionException e) {
|
||||
if (e.isExecutorShutdown()) {
|
||||
logger.debug("failed to schedule next maintenance task; shutting down", e);
|
||||
} else {
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
if (cancellable != null && cancellable.isCancelled() == false) {
|
||||
cancellable.cancel();
|
||||
}
|
||||
isClosed.compareAndSet(false, true);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,371 @@
|
|||
/*
|
||||
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||
* or more contributor license agreements. Licensed under the Elastic License;
|
||||
* you may not use this file except in compliance with the Elastic License.
|
||||
*/
|
||||
package org.elasticsearch.xpack.search;
|
||||
|
||||
import org.apache.lucene.search.TotalHits;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.admin.cluster.node.tasks.cancel.CancelTasksRequest;
|
||||
import org.elasticsearch.action.admin.cluster.node.tasks.cancel.CancelTasksResponse;
|
||||
import org.elasticsearch.action.search.SearchProgressActionListener;
|
||||
import org.elasticsearch.action.search.SearchRequest;
|
||||
import org.elasticsearch.action.search.SearchResponse;
|
||||
import org.elasticsearch.action.search.SearchResponse.Clusters;
|
||||
import org.elasticsearch.action.search.SearchShard;
|
||||
import org.elasticsearch.action.search.SearchTask;
|
||||
import org.elasticsearch.action.search.ShardSearchFailure;
|
||||
import org.elasticsearch.client.Client;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException;
|
||||
import org.elasticsearch.search.SearchHits;
|
||||
import org.elasticsearch.search.SearchShardTarget;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregation.ReduceContext;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregations;
|
||||
import org.elasticsearch.search.internal.InternalSearchResponse;
|
||||
import org.elasticsearch.tasks.TaskId;
|
||||
import org.elasticsearch.threadpool.Scheduler.Cancellable;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
import org.elasticsearch.xpack.core.search.action.AsyncSearchResponse;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
import java.util.function.Consumer;
|
||||
import java.util.function.Supplier;
|
||||
|
||||
/**
|
||||
* Task that tracks the progress of a currently running {@link SearchRequest}.
|
||||
*/
|
||||
class AsyncSearchTask extends SearchTask {
|
||||
private final AsyncSearchId searchId;
|
||||
private final Client client;
|
||||
private final ThreadPool threadPool;
|
||||
private final Supplier<ReduceContext> reduceContextSupplier;
|
||||
private final Listener progressListener;
|
||||
|
||||
private final Map<String, String> originHeaders;
|
||||
|
||||
private boolean hasInitialized;
|
||||
private boolean hasCompleted;
|
||||
private long completionId;
|
||||
private final List<Runnable> initListeners = new ArrayList<>();
|
||||
private final Map<Long, Consumer<AsyncSearchResponse>> completionListeners = new HashMap<>();
|
||||
|
||||
private volatile long expirationTimeMillis;
|
||||
private final AtomicBoolean isCancelling = new AtomicBoolean(false);
|
||||
|
||||
private AtomicReference<MutableSearchResponse> searchResponse;
|
||||
|
||||
/**
|
||||
* Creates an instance of {@link AsyncSearchTask}.
|
||||
*
|
||||
* @param id The id of the task.
|
||||
* @param type The type of the task.
|
||||
* @param action The action name.
|
||||
* @param parentTaskId The parent task id.
|
||||
* @param originHeaders All the request context headers.
|
||||
* @param taskHeaders The filtered request headers for the task.
|
||||
* @param searchId The {@link AsyncSearchId} of the task.
|
||||
* @param threadPool The threadPool to schedule runnable.
|
||||
* @param reduceContextSupplier A supplier to create final reduce contexts.
|
||||
*/
|
||||
AsyncSearchTask(long id,
|
||||
String type,
|
||||
String action,
|
||||
TaskId parentTaskId,
|
||||
TimeValue keepAlive,
|
||||
Map<String, String> originHeaders,
|
||||
Map<String, String> taskHeaders,
|
||||
AsyncSearchId searchId,
|
||||
Client client,
|
||||
ThreadPool threadPool,
|
||||
Supplier<ReduceContext> reduceContextSupplier) {
|
||||
super(id, type, action, "async_search", parentTaskId, taskHeaders);
|
||||
this.expirationTimeMillis = getStartTime() + keepAlive.getMillis();
|
||||
this.originHeaders = originHeaders;
|
||||
this.searchId = searchId;
|
||||
this.client = client;
|
||||
this.threadPool = threadPool;
|
||||
this.reduceContextSupplier = reduceContextSupplier;
|
||||
this.progressListener = new Listener();
|
||||
this.searchResponse = new AtomicReference<>();
|
||||
setProgressListener(progressListener);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns all of the request contexts headers
|
||||
*/
|
||||
Map<String, String> getOriginHeaders() {
|
||||
return originHeaders;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the {@link AsyncSearchId} of the task
|
||||
*/
|
||||
AsyncSearchId getSearchId() {
|
||||
return searchId;
|
||||
}
|
||||
|
||||
@Override
|
||||
public SearchProgressActionListener getProgressListener() {
|
||||
return progressListener;
|
||||
}
|
||||
|
||||
/**
|
||||
* Update the expiration time of the (partial) response.
|
||||
*/
|
||||
public void setExpirationTime(long expirationTimeMillis) {
|
||||
this.expirationTimeMillis = expirationTimeMillis;
|
||||
}
|
||||
|
||||
/**
|
||||
* Cancels the running task and its children.
|
||||
*/
|
||||
public void cancelTask(Runnable runnable) {
|
||||
if (isCancelled() == false && isCancelling.compareAndSet(false, true)) {
|
||||
CancelTasksRequest req = new CancelTasksRequest().setTaskId(searchId.getTaskId());
|
||||
client.admin().cluster().cancelTasks(req, new ActionListener<CancelTasksResponse>() {
|
||||
@Override
|
||||
public void onResponse(CancelTasksResponse cancelTasksResponse) {
|
||||
runnable.run();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onFailure(Exception exc) {
|
||||
// cancelling failed
|
||||
isCancelling.compareAndSet(true, false);
|
||||
runnable.run();
|
||||
}
|
||||
});
|
||||
} else {
|
||||
runnable.run();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void onCancelled() {
|
||||
super.onCancelled();
|
||||
isCancelling.compareAndSet(true, false);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a listener that listens for an {@link AsyncSearchResponse} and executes the
|
||||
* consumer when the task is finished or when the provided <code>waitForCompletion</code>
|
||||
* timeout occurs. In such case the consumed {@link AsyncSearchResponse} will contain partial results.
|
||||
*/
|
||||
public void addCompletionListener(ActionListener<AsyncSearchResponse> listener, TimeValue waitForCompletion) {
|
||||
boolean executeImmediately = false;
|
||||
long startTime = threadPool.relativeTimeInMillis();
|
||||
synchronized (this) {
|
||||
if (hasCompleted) {
|
||||
executeImmediately = true;
|
||||
} else {
|
||||
addInitListener(() -> {
|
||||
final TimeValue remainingWaitForCompletion;
|
||||
if (waitForCompletion.getMillis() > 0) {
|
||||
long elapsedTime = threadPool.relativeTimeInMillis() - startTime;
|
||||
// subtract the initialization time from the provided waitForCompletion.
|
||||
remainingWaitForCompletion = TimeValue.timeValueMillis(Math.max(0, waitForCompletion.getMillis() - elapsedTime));
|
||||
} else {
|
||||
remainingWaitForCompletion = TimeValue.ZERO;
|
||||
}
|
||||
internalAddCompletionListener(listener, remainingWaitForCompletion);
|
||||
});
|
||||
}
|
||||
}
|
||||
if (executeImmediately) {
|
||||
listener.onResponse(getResponse());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a listener that listens for an {@link AsyncSearchResponse} and executes the
|
||||
* consumer when the task is finished.
|
||||
*/
|
||||
public void addCompletionListener(Consumer<AsyncSearchResponse> listener) {
|
||||
boolean executeImmediately = false;
|
||||
synchronized (this) {
|
||||
if (hasCompleted) {
|
||||
executeImmediately = true;
|
||||
} else {
|
||||
completionListeners.put(completionId++, resp -> listener.accept(resp));
|
||||
}
|
||||
}
|
||||
if (executeImmediately) {
|
||||
listener.accept(getResponse());
|
||||
}
|
||||
}
|
||||
|
||||
private void internalAddCompletionListener(ActionListener<AsyncSearchResponse> listener, TimeValue waitForCompletion) {
|
||||
boolean executeImmediately = false;
|
||||
synchronized (this) {
|
||||
if (hasCompleted || waitForCompletion.getMillis() == 0) {
|
||||
executeImmediately = true;
|
||||
} else {
|
||||
// ensure that we consumes the listener only once
|
||||
AtomicBoolean hasRun = new AtomicBoolean(false);
|
||||
long id = completionId++;
|
||||
|
||||
final Cancellable cancellable;
|
||||
try {
|
||||
cancellable = threadPool.schedule(() -> {
|
||||
if (hasRun.compareAndSet(false, true)) {
|
||||
// timeout occurred before completion
|
||||
removeCompletionListener(id);
|
||||
listener.onResponse(getResponse());
|
||||
}
|
||||
}, waitForCompletion, "generic");
|
||||
} catch (EsRejectedExecutionException exc) {
|
||||
listener.onFailure(exc);
|
||||
return;
|
||||
}
|
||||
completionListeners.put(id, resp -> {
|
||||
if (hasRun.compareAndSet(false, true)) {
|
||||
// completion occurred before timeout
|
||||
cancellable.cancel();
|
||||
listener.onResponse(resp);
|
||||
}
|
||||
});
|
||||
}
|
||||
}
|
||||
if (executeImmediately) {
|
||||
listener.onResponse(getResponse());
|
||||
}
|
||||
}
|
||||
|
||||
private void removeCompletionListener(long id) {
|
||||
synchronized (this) {
|
||||
if (hasCompleted == false) {
|
||||
completionListeners.remove(id);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void addInitListener(Runnable listener) {
|
||||
boolean executeImmediately = false;
|
||||
synchronized (this) {
|
||||
if (hasInitialized) {
|
||||
executeImmediately = true;
|
||||
} else {
|
||||
initListeners.add(listener);
|
||||
}
|
||||
}
|
||||
if (executeImmediately) {
|
||||
listener.run();
|
||||
}
|
||||
}
|
||||
|
||||
private void executeInitListeners() {
|
||||
synchronized (this) {
|
||||
if (hasInitialized) {
|
||||
return;
|
||||
}
|
||||
hasInitialized = true;
|
||||
}
|
||||
for (Runnable listener : initListeners) {
|
||||
listener.run();
|
||||
}
|
||||
initListeners.clear();
|
||||
}
|
||||
|
||||
private void executeCompletionListeners() {
|
||||
synchronized (this) {
|
||||
if (hasCompleted) {
|
||||
return;
|
||||
}
|
||||
hasCompleted = true;
|
||||
}
|
||||
AsyncSearchResponse finalResponse = getResponse();
|
||||
for (Consumer<AsyncSearchResponse> listener : completionListeners.values()) {
|
||||
listener.accept(finalResponse);
|
||||
}
|
||||
completionListeners.clear();
|
||||
}
|
||||
|
||||
private AsyncSearchResponse getResponse() {
|
||||
assert searchResponse.get() != null;
|
||||
return searchResponse.get().toAsyncSearchResponse(this, expirationTimeMillis);
|
||||
}
|
||||
|
||||
// cancels the task if it expired
|
||||
private void checkExpiration() {
|
||||
long now = System.currentTimeMillis();
|
||||
if (expirationTimeMillis < now) {
|
||||
cancelTask(() -> {});
|
||||
}
|
||||
}
|
||||
|
||||
private class Listener extends SearchProgressActionListener {
|
||||
@Override
|
||||
public void onQueryResult(int shardIndex) {
|
||||
checkExpiration();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onFetchResult(int shardIndex) {
|
||||
checkExpiration();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onQueryFailure(int shardIndex, SearchShardTarget shardTarget, Exception exc) {
|
||||
// best effort to cancel expired tasks
|
||||
checkExpiration();
|
||||
searchResponse.get().addShardFailure(shardIndex, new ShardSearchFailure(exc, shardTarget));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onFetchFailure(int shardIndex, Exception exc) {
|
||||
checkExpiration();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onListShards(List<SearchShard> shards, List<SearchShard> skipped, Clusters clusters, boolean fetchPhase) {
|
||||
// best effort to cancel expired tasks
|
||||
checkExpiration();
|
||||
searchResponse.compareAndSet(null,
|
||||
new MutableSearchResponse(shards.size() + skipped.size(), skipped.size(), clusters, reduceContextSupplier));
|
||||
executeInitListeners();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onPartialReduce(List<SearchShard> shards, TotalHits totalHits, InternalAggregations aggs, int reducePhase) {
|
||||
// best effort to cancel expired tasks
|
||||
checkExpiration();
|
||||
searchResponse.get().updatePartialResponse(shards.size(),
|
||||
new InternalSearchResponse(new SearchHits(SearchHits.EMPTY, totalHits, Float.NaN), aggs,
|
||||
null, null, false, null, reducePhase), aggs == null);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onReduce(List<SearchShard> shards, TotalHits totalHits, InternalAggregations aggs, int reducePhase) {
|
||||
// best effort to cancel expired tasks
|
||||
checkExpiration();
|
||||
searchResponse.get().updatePartialResponse(shards.size(),
|
||||
new InternalSearchResponse(new SearchHits(SearchHits.EMPTY, totalHits, Float.NaN), aggs,
|
||||
null, null, false, null, reducePhase), true);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onResponse(SearchResponse response) {
|
||||
searchResponse.get().updateFinalResponse(response.getSuccessfulShards(), response.getInternalResponse());
|
||||
executeCompletionListeners();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onFailure(Exception exc) {
|
||||
if (searchResponse.get() == null) {
|
||||
// if the failure occurred before calling onListShards
|
||||
searchResponse.compareAndSet(null,
|
||||
new MutableSearchResponse(-1, -1, null, reduceContextSupplier));
|
||||
}
|
||||
searchResponse.get().updateWithFailure(exc);
|
||||
executeInitListeners();
|
||||
executeCompletionListeners();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,173 @@
|
|||
/*
|
||||
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||
* or more contributor license agreements. Licensed under the Elastic License;
|
||||
* you may not use this file except in compliance with the Elastic License.
|
||||
*/
|
||||
package org.elasticsearch.xpack.search;
|
||||
|
||||
import org.apache.lucene.search.TotalHits;
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.action.search.SearchResponse;
|
||||
import org.elasticsearch.action.search.SearchResponse.Clusters;
|
||||
import org.elasticsearch.action.search.SearchResponseSections;
|
||||
import org.elasticsearch.action.search.ShardSearchFailure;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.common.util.concurrent.AtomicArray;
|
||||
import org.elasticsearch.search.SearchHits;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregation.ReduceContext;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregations;
|
||||
import org.elasticsearch.search.internal.InternalSearchResponse;
|
||||
import org.elasticsearch.xpack.core.search.action.AsyncSearchResponse;
|
||||
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.function.Supplier;
|
||||
|
||||
import static java.util.Collections.singletonList;
|
||||
import static org.apache.lucene.search.TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO;
|
||||
import static org.elasticsearch.search.aggregations.InternalAggregations.topLevelReduce;
|
||||
|
||||
/**
|
||||
* A mutable search response that allows to update and create partial response synchronously.
|
||||
* Synchronized methods ensure that updates of the content are blocked if another thread is
|
||||
* creating an async response concurrently. This limits the number of final reduction that can
|
||||
* run concurrently to 1 and ensures that we pause the search progress when an {@link AsyncSearchResponse} is built.
|
||||
*/
|
||||
class MutableSearchResponse {
|
||||
private final int totalShards;
|
||||
private final int skippedShards;
|
||||
private final Clusters clusters;
|
||||
private final AtomicArray<ShardSearchFailure> shardFailures;
|
||||
private final Supplier<ReduceContext> reduceContextSupplier;
|
||||
|
||||
private int version;
|
||||
private boolean isPartial;
|
||||
private boolean isFinalReduce;
|
||||
private int successfulShards;
|
||||
private SearchResponseSections sections;
|
||||
private ElasticsearchException failure;
|
||||
|
||||
private boolean frozen;
|
||||
|
||||
/**
|
||||
* Creates a new mutable search response.
|
||||
*
|
||||
* @param totalShards The number of shards that participate in the request, or -1 to indicate a failure.
|
||||
* @param skippedShards The number of skipped shards, or -1 to indicate a failure.
|
||||
* @param clusters The remote clusters statistics.
|
||||
* @param reduceContextSupplier A supplier to run final reduce on partial aggregations.
|
||||
*/
|
||||
MutableSearchResponse(int totalShards, int skippedShards, Clusters clusters, Supplier<ReduceContext> reduceContextSupplier) {
|
||||
this.totalShards = totalShards;
|
||||
this.skippedShards = skippedShards;
|
||||
this.clusters = clusters;
|
||||
this.reduceContextSupplier = reduceContextSupplier;
|
||||
this.version = 0;
|
||||
this.shardFailures = totalShards == -1 ? null : new AtomicArray<>(totalShards-skippedShards);
|
||||
this.isPartial = true;
|
||||
this.sections = totalShards == -1 ? null : new InternalSearchResponse(
|
||||
new SearchHits(SearchHits.EMPTY, new TotalHits(0, GREATER_THAN_OR_EQUAL_TO), Float.NaN),
|
||||
null, null, null, false, null, 0);
|
||||
}
|
||||
|
||||
/**
|
||||
* Updates the response with the partial {@link SearchResponseSections} merged from #<code>successfulShards</code>
|
||||
* shards.
|
||||
*/
|
||||
synchronized void updatePartialResponse(int successfulShards, SearchResponseSections newSections, boolean isFinalReduce) {
|
||||
failIfFrozen();
|
||||
if (newSections.getNumReducePhases() < sections.getNumReducePhases()) {
|
||||
// should never happen since partial response are updated under a lock
|
||||
// in the search phase controller
|
||||
throw new IllegalStateException("received partial response out of order: "
|
||||
+ newSections.getNumReducePhases() + " < " + sections.getNumReducePhases());
|
||||
}
|
||||
++ version;
|
||||
this.successfulShards = successfulShards;
|
||||
this.sections = newSections;
|
||||
this.isPartial = true;
|
||||
this.isFinalReduce = isFinalReduce;
|
||||
}
|
||||
|
||||
/**
|
||||
* Updates the response with the final {@link SearchResponseSections} merged from #<code>successfulShards</code>
|
||||
* shards.
|
||||
*/
|
||||
synchronized void updateFinalResponse(int successfulShards, SearchResponseSections newSections) {
|
||||
failIfFrozen();
|
||||
++ version;
|
||||
this.successfulShards = successfulShards;
|
||||
this.sections = newSections;
|
||||
this.isPartial = false;
|
||||
this.isFinalReduce = true;
|
||||
this.frozen = true;
|
||||
}
|
||||
|
||||
/**
|
||||
* Updates the response with a fatal failure. This method preserves the partial response
|
||||
* received from previous updates
|
||||
*/
|
||||
synchronized void updateWithFailure(Exception exc) {
|
||||
failIfFrozen();
|
||||
++ version;
|
||||
this.isPartial = true;
|
||||
this.failure = ElasticsearchException.guessRootCauses(exc)[0];
|
||||
this.frozen = true;
|
||||
}
|
||||
|
||||
/**
|
||||
* Adds a shard failure concurrently (non-blocking).
|
||||
*/
|
||||
void addShardFailure(int shardIndex, ShardSearchFailure failure) {
|
||||
synchronized (this) {
|
||||
failIfFrozen();
|
||||
}
|
||||
shardFailures.set(shardIndex, failure);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates an {@link AsyncSearchResponse} based on the current state of the mutable response.
|
||||
* The final reduce of the aggregations is executed if needed (partial response).
|
||||
* This method is synchronized to ensure that we don't perform final reduces concurrently.
|
||||
*/
|
||||
synchronized AsyncSearchResponse toAsyncSearchResponse(AsyncSearchTask task, long expirationTime) {
|
||||
final SearchResponse resp;
|
||||
if (totalShards != -1) {
|
||||
if (sections.aggregations() != null && isFinalReduce == false) {
|
||||
InternalAggregations oldAggs = (InternalAggregations) sections.aggregations();
|
||||
InternalAggregations newAggs = topLevelReduce(singletonList(oldAggs), reduceContextSupplier.get());
|
||||
sections = new InternalSearchResponse(sections.hits(), newAggs, sections.suggest(),
|
||||
null, sections.timedOut(), sections.terminatedEarly(), sections.getNumReducePhases());
|
||||
isFinalReduce = true;
|
||||
}
|
||||
long tookInMillis = TimeValue.timeValueNanos(System.nanoTime() - task.getStartTimeNanos()).getMillis();
|
||||
resp = new SearchResponse(sections, null, totalShards, successfulShards,
|
||||
skippedShards, tookInMillis, buildShardFailures(), clusters);
|
||||
} else {
|
||||
resp = null;
|
||||
}
|
||||
return new AsyncSearchResponse(task.getSearchId().getEncoded(), version, resp, failure, isPartial,
|
||||
frozen == false, task.getStartTime(), expirationTime);
|
||||
}
|
||||
|
||||
private void failIfFrozen() {
|
||||
if (frozen) {
|
||||
throw new IllegalStateException("invalid update received after the completion of the request");
|
||||
}
|
||||
}
|
||||
|
||||
private ShardSearchFailure[] buildShardFailures() {
|
||||
if (shardFailures == null) {
|
||||
return new ShardSearchFailure[0];
|
||||
}
|
||||
List<ShardSearchFailure> failures = new ArrayList<>();
|
||||
for (int i = 0; i < shardFailures.length(); i++) {
|
||||
ShardSearchFailure failure = shardFailures.get(i);
|
||||
if (failure != null) {
|
||||
failures.add(failure);
|
||||
}
|
||||
}
|
||||
return failures.toArray(new ShardSearchFailure[0]);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,40 @@
|
|||
/*
|
||||
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||
* or more contributor license agreements. Licensed under the Elastic License;
|
||||
* you may not use this file except in compliance with the Elastic License.
|
||||
*/
|
||||
package org.elasticsearch.xpack.search;
|
||||
|
||||
import org.elasticsearch.client.node.NodeClient;
|
||||
import org.elasticsearch.rest.BaseRestHandler;
|
||||
import org.elasticsearch.rest.RestHandler.Route;
|
||||
import org.elasticsearch.rest.RestRequest;
|
||||
import org.elasticsearch.rest.action.RestToXContentListener;
|
||||
import org.elasticsearch.xpack.core.search.action.DeleteAsyncSearchAction;
|
||||
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
|
||||
import static java.util.Arrays.asList;
|
||||
import static java.util.Collections.unmodifiableList;
|
||||
import static org.elasticsearch.rest.RestRequest.Method.DELETE;
|
||||
|
||||
public class RestDeleteAsyncSearchAction extends BaseRestHandler {
|
||||
@Override
|
||||
public List<Route> routes() {
|
||||
return unmodifiableList(asList(
|
||||
new Route(DELETE, "/_async_search/{id}")));
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getName() {
|
||||
return "async_search_delete_action";
|
||||
}
|
||||
|
||||
@Override
|
||||
protected RestChannelConsumer prepareRequest(RestRequest request, NodeClient client) throws IOException {
|
||||
DeleteAsyncSearchAction.Request delete = new DeleteAsyncSearchAction.Request(request.param("id"));
|
||||
return channel -> client.execute(DeleteAsyncSearchAction.INSTANCE, delete, new RestToXContentListener<>(channel));
|
||||
}
|
||||
}
|
|
@ -0,0 +1,50 @@
|
|||
/*
|
||||
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||
* or more contributor license agreements. Licensed under the Elastic License;
|
||||
* you may not use this file except in compliance with the Elastic License.
|
||||
*/
|
||||
package org.elasticsearch.xpack.search;
|
||||
|
||||
import org.elasticsearch.client.node.NodeClient;
|
||||
import org.elasticsearch.rest.BaseRestHandler;
|
||||
import org.elasticsearch.rest.RestRequest;
|
||||
import org.elasticsearch.rest.action.RestStatusToXContentListener;
|
||||
import org.elasticsearch.xpack.core.search.action.GetAsyncSearchAction;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
|
||||
import static java.util.Arrays.asList;
|
||||
import static java.util.Collections.unmodifiableList;
|
||||
import static org.elasticsearch.rest.RestRequest.Method.GET;
|
||||
import static org.elasticsearch.xpack.search.RestSubmitAsyncSearchAction.RESPONSE_PARAMS;
|
||||
|
||||
public class RestGetAsyncSearchAction extends BaseRestHandler {
|
||||
@Override
|
||||
public List<Route> routes() {
|
||||
return unmodifiableList(asList(new Route(GET, "/_async_search/{id}")));
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public String getName() {
|
||||
return "async_search_get_action";
|
||||
}
|
||||
|
||||
@Override
|
||||
protected RestChannelConsumer prepareRequest(RestRequest request, NodeClient client) {
|
||||
GetAsyncSearchAction.Request get = new GetAsyncSearchAction.Request(request.param("id"));
|
||||
if (request.hasParam("wait_for_completion")) {
|
||||
get.setWaitForCompletion(request.paramAsTime("wait_for_completion", get.getWaitForCompletion()));
|
||||
}
|
||||
if (request.hasParam("keep_alive")) {
|
||||
get.setKeepAlive(request.paramAsTime("keep_alive", get.getKeepAlive()));
|
||||
}
|
||||
return channel -> client.execute(GetAsyncSearchAction.INSTANCE, get, new RestStatusToXContentListener<>(channel));
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Set<String> responseParams() {
|
||||
return RESPONSE_PARAMS;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,74 @@
|
|||
/*
|
||||
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||
* or more contributor license agreements. Licensed under the Elastic License;
|
||||
* you may not use this file except in compliance with the Elastic License.
|
||||
*/
|
||||
package org.elasticsearch.xpack.search;
|
||||
|
||||
import org.elasticsearch.client.node.NodeClient;
|
||||
import org.elasticsearch.rest.BaseRestHandler;
|
||||
import org.elasticsearch.rest.RestRequest;
|
||||
import org.elasticsearch.rest.action.RestCancellableNodeClient;
|
||||
import org.elasticsearch.rest.action.RestStatusToXContentListener;
|
||||
import org.elasticsearch.xpack.core.search.action.AsyncSearchResponse;
|
||||
import org.elasticsearch.xpack.core.search.action.SubmitAsyncSearchAction;
|
||||
import org.elasticsearch.xpack.core.search.action.SubmitAsyncSearchRequest;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.Set;
|
||||
import java.util.function.IntConsumer;
|
||||
import java.util.List;
|
||||
|
||||
import static java.util.Arrays.asList;
|
||||
import static java.util.Collections.unmodifiableList;
|
||||
import static org.elasticsearch.rest.RestRequest.Method.GET;
|
||||
import static org.elasticsearch.rest.RestRequest.Method.POST;
|
||||
import static org.elasticsearch.rest.action.search.RestSearchAction.parseSearchRequest;
|
||||
|
||||
public final class RestSubmitAsyncSearchAction extends BaseRestHandler {
|
||||
static final String TYPED_KEYS_PARAM = "typed_keys";
|
||||
static final Set<String> RESPONSE_PARAMS = Collections.singleton(TYPED_KEYS_PARAM);
|
||||
|
||||
@Override
|
||||
public List<Route> routes() {
|
||||
return unmodifiableList(asList(
|
||||
new Route(POST, "/_async_search"),
|
||||
new Route(GET, "/_async_search"),
|
||||
new Route(POST, "/{index}/_async_search"),
|
||||
new Route(GET, "/{index}/_async_search")));
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getName() {
|
||||
return "async_search_submit_action";
|
||||
}
|
||||
|
||||
@Override
|
||||
protected RestChannelConsumer prepareRequest(RestRequest request, NodeClient client) throws IOException {
|
||||
SubmitAsyncSearchRequest submit = new SubmitAsyncSearchRequest();
|
||||
IntConsumer setSize = size -> submit.getSearchRequest().source().size(size);
|
||||
request.withContentOrSourceParamParserOrNull(parser ->
|
||||
parseSearchRequest(submit.getSearchRequest(), request, parser, setSize));
|
||||
|
||||
if (request.hasParam("wait_for_completion")) {
|
||||
submit.setWaitForCompletion(request.paramAsTime("wait_for_completion", submit.getWaitForCompletion()));
|
||||
}
|
||||
if (request.hasParam("keep_alive")) {
|
||||
submit.setKeepAlive(request.paramAsTime("keep_alive", submit.getKeepAlive()));
|
||||
}
|
||||
if (request.hasParam("clean_on_completion")) {
|
||||
submit.setCleanOnCompletion(request.paramAsBoolean("clean_on_completion", submit.isCleanOnCompletion()));
|
||||
}
|
||||
return channel -> {
|
||||
RestStatusToXContentListener<AsyncSearchResponse> listener = new RestStatusToXContentListener<>(channel);
|
||||
RestCancellableNodeClient cancelClient = new RestCancellableNodeClient(client, request.getHttpChannel());
|
||||
cancelClient.execute(SubmitAsyncSearchAction.INSTANCE, submit, listener);
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Set<String> responseParams() {
|
||||
return RESPONSE_PARAMS;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,72 @@
|
|||
/*
|
||||
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||
* or more contributor license agreements. Licensed under the Elastic License;
|
||||
* you may not use this file except in compliance with the Elastic License.
|
||||
*/
|
||||
package org.elasticsearch.xpack.search;
|
||||
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.ActionListenerResponseHandler;
|
||||
import org.elasticsearch.action.support.ActionFilters;
|
||||
import org.elasticsearch.action.support.HandledTransportAction;
|
||||
import org.elasticsearch.action.support.master.AcknowledgedResponse;
|
||||
import org.elasticsearch.client.Client;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||
import org.elasticsearch.cluster.service.ClusterService;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
|
||||
import org.elasticsearch.tasks.Task;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
import org.elasticsearch.transport.TransportRequestOptions;
|
||||
import org.elasticsearch.transport.TransportService;
|
||||
import org.elasticsearch.xpack.core.search.action.DeleteAsyncSearchAction;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
public class TransportDeleteAsyncSearchAction extends HandledTransportAction<DeleteAsyncSearchAction.Request, AcknowledgedResponse> {
|
||||
private final ClusterService clusterService;
|
||||
private final TransportService transportService;
|
||||
private final AsyncSearchIndexService store;
|
||||
|
||||
@Inject
|
||||
public TransportDeleteAsyncSearchAction(TransportService transportService,
|
||||
ActionFilters actionFilters,
|
||||
ClusterService clusterService,
|
||||
ThreadPool threadPool,
|
||||
NamedWriteableRegistry registry,
|
||||
Client client) {
|
||||
super(DeleteAsyncSearchAction.NAME, transportService, actionFilters, DeleteAsyncSearchAction.Request::new);
|
||||
this.store = new AsyncSearchIndexService(clusterService, threadPool.getThreadContext(), client, registry);
|
||||
this.clusterService = clusterService;
|
||||
this.transportService = transportService;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doExecute(Task task, DeleteAsyncSearchAction.Request request, ActionListener<AcknowledgedResponse> listener) {
|
||||
try {
|
||||
AsyncSearchId searchId = AsyncSearchId.decode(request.getId());
|
||||
DiscoveryNode node = clusterService.state().nodes().get(searchId.getTaskId().getNodeId());
|
||||
if (clusterService.localNode().getId().equals(searchId.getTaskId().getNodeId()) || node == null) {
|
||||
cancelTaskAndDeleteResult(searchId, listener);
|
||||
} else {
|
||||
TransportRequestOptions.Builder builder = TransportRequestOptions.builder();
|
||||
transportService.sendRequest(node, DeleteAsyncSearchAction.NAME, request, builder.build(),
|
||||
new ActionListenerResponseHandler<>(listener, AcknowledgedResponse::new, ThreadPool.Names.SAME));
|
||||
}
|
||||
} catch (Exception exc) {
|
||||
listener.onFailure(exc);
|
||||
}
|
||||
}
|
||||
|
||||
private void cancelTaskAndDeleteResult(AsyncSearchId searchId, ActionListener<AcknowledgedResponse> listener) throws IOException {
|
||||
AsyncSearchTask task = store.getTask(taskManager, searchId);
|
||||
if (task != null) {
|
||||
task.cancelTask(() -> store.deleteResponse(searchId, false, listener));
|
||||
} else {
|
||||
// the task is not running anymore so we throw a not found exception if
|
||||
// the search id is also not present in the index (already deleted) or if the user
|
||||
// is not allowed to access it.
|
||||
store.getResponse(searchId, ActionListener.wrap(res -> store.deleteResponse(searchId, true, listener), listener::onFailure));
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,144 @@
|
|||
/*
|
||||
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||
* or more contributor license agreements. Licensed under the Elastic License;
|
||||
* you may not use this file except in compliance with the Elastic License.
|
||||
*/
|
||||
package org.elasticsearch.xpack.search;
|
||||
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
import org.elasticsearch.ResourceNotFoundException;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.ActionListenerResponseHandler;
|
||||
import org.elasticsearch.action.support.ActionFilters;
|
||||
import org.elasticsearch.action.support.HandledTransportAction;
|
||||
import org.elasticsearch.client.Client;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||
import org.elasticsearch.cluster.service.ClusterService;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
|
||||
import org.elasticsearch.index.engine.DocumentMissingException;
|
||||
import org.elasticsearch.tasks.Task;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
import org.elasticsearch.transport.TransportRequestOptions;
|
||||
import org.elasticsearch.transport.TransportService;
|
||||
import org.elasticsearch.xpack.core.search.action.AsyncSearchResponse;
|
||||
import org.elasticsearch.xpack.core.search.action.GetAsyncSearchAction;
|
||||
|
||||
public class TransportGetAsyncSearchAction extends HandledTransportAction<GetAsyncSearchAction.Request, AsyncSearchResponse> {
|
||||
private final Logger logger = LogManager.getLogger(TransportGetAsyncSearchAction.class);
|
||||
private final ClusterService clusterService;
|
||||
private final TransportService transportService;
|
||||
private final AsyncSearchIndexService store;
|
||||
|
||||
@Inject
|
||||
public TransportGetAsyncSearchAction(TransportService transportService,
|
||||
ActionFilters actionFilters,
|
||||
ClusterService clusterService,
|
||||
NamedWriteableRegistry registry,
|
||||
Client client,
|
||||
ThreadPool threadPool) {
|
||||
super(GetAsyncSearchAction.NAME, transportService, actionFilters, GetAsyncSearchAction.Request::new);
|
||||
this.clusterService = clusterService;
|
||||
this.transportService = transportService;
|
||||
this.store = new AsyncSearchIndexService(clusterService, threadPool.getThreadContext(), client, registry);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doExecute(Task task, GetAsyncSearchAction.Request request, ActionListener<AsyncSearchResponse> listener) {
|
||||
try {
|
||||
long nowInMillis = System.currentTimeMillis();
|
||||
AsyncSearchId searchId = AsyncSearchId.decode(request.getId());
|
||||
DiscoveryNode node = clusterService.state().nodes().get(searchId.getTaskId().getNodeId());
|
||||
if (clusterService.localNode().getId().equals(searchId.getTaskId().getNodeId()) || node == null) {
|
||||
if (request.getKeepAlive().getMillis() > 0) {
|
||||
long expirationTime = nowInMillis + request.getKeepAlive().getMillis();
|
||||
store.updateExpirationTime(searchId.getDocId(), expirationTime,
|
||||
ActionListener.wrap(
|
||||
p -> getSearchResponseFromTask(searchId, request, nowInMillis, expirationTime, listener),
|
||||
exc -> {
|
||||
if (exc.getCause() instanceof DocumentMissingException == false) {
|
||||
logger.error("failed to retrieve " + searchId.getEncoded(), exc);
|
||||
}
|
||||
listener.onFailure(new ResourceNotFoundException(searchId.getEncoded()));
|
||||
}
|
||||
));
|
||||
} else {
|
||||
getSearchResponseFromTask(searchId, request, nowInMillis, -1, listener);
|
||||
}
|
||||
} else {
|
||||
TransportRequestOptions.Builder builder = TransportRequestOptions.builder();
|
||||
transportService.sendRequest(node, GetAsyncSearchAction.NAME, request, builder.build(),
|
||||
new ActionListenerResponseHandler<>(listener, AsyncSearchResponse::new, ThreadPool.Names.SAME));
|
||||
}
|
||||
} catch (Exception exc) {
|
||||
listener.onFailure(exc);
|
||||
}
|
||||
}
|
||||
|
||||
private void getSearchResponseFromTask(AsyncSearchId searchId,
|
||||
GetAsyncSearchAction.Request request,
|
||||
long nowInMillis,
|
||||
long expirationTimeMillis,
|
||||
ActionListener<AsyncSearchResponse> listener) {
|
||||
try {
|
||||
final AsyncSearchTask task = store.getTask(taskManager, searchId);
|
||||
if (task == null) {
|
||||
getSearchResponseFromIndex(searchId, request, nowInMillis, listener);
|
||||
return;
|
||||
}
|
||||
|
||||
if (task.isCancelled()) {
|
||||
listener.onFailure(new ResourceNotFoundException(searchId.getEncoded()));
|
||||
return;
|
||||
}
|
||||
|
||||
if (expirationTimeMillis != -1) {
|
||||
task.setExpirationTime(expirationTimeMillis);
|
||||
}
|
||||
task.addCompletionListener(new ActionListener<AsyncSearchResponse>() {
|
||||
@Override
|
||||
public void onResponse(AsyncSearchResponse response) {
|
||||
sendFinalResponse(request, response, nowInMillis, listener);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onFailure(Exception exc) {
|
||||
listener.onFailure(exc);
|
||||
}
|
||||
}, request.getWaitForCompletion());
|
||||
} catch (Exception exc) {
|
||||
listener.onFailure(exc);
|
||||
}
|
||||
}
|
||||
|
||||
private void getSearchResponseFromIndex(AsyncSearchId searchId,
|
||||
GetAsyncSearchAction.Request request,
|
||||
long nowInMillis,
|
||||
ActionListener<AsyncSearchResponse> listener) {
|
||||
store.getResponse(searchId, new ActionListener<AsyncSearchResponse>() {
|
||||
@Override
|
||||
public void onResponse(AsyncSearchResponse response) {
|
||||
sendFinalResponse(request, response, nowInMillis, listener);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onFailure(Exception e) {
|
||||
listener.onFailure(e);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
private void sendFinalResponse(GetAsyncSearchAction.Request request,
|
||||
AsyncSearchResponse response,
|
||||
long nowInMillis,
|
||||
ActionListener<AsyncSearchResponse> listener) {
|
||||
// check if the result has expired
|
||||
if (response.getExpirationTime() < nowInMillis) {
|
||||
listener.onFailure(new ResourceNotFoundException(request.getId()));
|
||||
return;
|
||||
}
|
||||
|
||||
listener.onResponse(response);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,201 @@
|
|||
/*
|
||||
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||
* or more contributor license agreements. Licensed under the Elastic License;
|
||||
* you may not use this file except in compliance with the Elastic License.
|
||||
*/
|
||||
package org.elasticsearch.xpack.search;
|
||||
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
import org.apache.logging.log4j.message.ParameterizedMessage;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.index.IndexResponse;
|
||||
import org.elasticsearch.action.search.SearchAction;
|
||||
import org.elasticsearch.action.search.SearchRequest;
|
||||
import org.elasticsearch.action.search.TransportSearchAction;
|
||||
import org.elasticsearch.action.support.ActionFilters;
|
||||
import org.elasticsearch.action.support.HandledTransportAction;
|
||||
import org.elasticsearch.action.update.UpdateResponse;
|
||||
import org.elasticsearch.client.Client;
|
||||
import org.elasticsearch.client.node.NodeClient;
|
||||
import org.elasticsearch.cluster.service.ClusterService;
|
||||
import org.elasticsearch.common.UUIDs;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.index.engine.DocumentMissingException;
|
||||
import org.elasticsearch.search.SearchService;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregation.ReduceContext;
|
||||
import org.elasticsearch.tasks.CancellableTask;
|
||||
import org.elasticsearch.tasks.Task;
|
||||
import org.elasticsearch.tasks.TaskCancelledException;
|
||||
import org.elasticsearch.tasks.TaskId;
|
||||
import org.elasticsearch.transport.TransportService;
|
||||
import org.elasticsearch.xpack.core.search.action.AsyncSearchResponse;
|
||||
import org.elasticsearch.xpack.core.search.action.SubmitAsyncSearchAction;
|
||||
import org.elasticsearch.xpack.core.search.action.SubmitAsyncSearchRequest;
|
||||
|
||||
import java.util.Map;
|
||||
import java.util.function.Supplier;
|
||||
|
||||
public class TransportSubmitAsyncSearchAction extends HandledTransportAction<SubmitAsyncSearchRequest, AsyncSearchResponse> {
|
||||
private static final Logger logger = LogManager.getLogger(TransportSubmitAsyncSearchAction.class);
|
||||
|
||||
private final NodeClient nodeClient;
|
||||
private final Supplier<ReduceContext> reduceContextSupplier;
|
||||
private final TransportSearchAction searchAction;
|
||||
private final AsyncSearchIndexService store;
|
||||
|
||||
@Inject
|
||||
public TransportSubmitAsyncSearchAction(ClusterService clusterService,
|
||||
TransportService transportService,
|
||||
ActionFilters actionFilters,
|
||||
NamedWriteableRegistry registry,
|
||||
Client client,
|
||||
NodeClient nodeClient,
|
||||
SearchService searchService,
|
||||
TransportSearchAction searchAction) {
|
||||
super(SubmitAsyncSearchAction.NAME, transportService, actionFilters, SubmitAsyncSearchRequest::new);
|
||||
this.nodeClient = nodeClient;
|
||||
this.reduceContextSupplier = () -> searchService.createReduceContext(true);
|
||||
this.searchAction = searchAction;
|
||||
this.store = new AsyncSearchIndexService(clusterService, transportService.getThreadPool().getThreadContext(), client, registry);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doExecute(Task task, SubmitAsyncSearchRequest request, ActionListener<AsyncSearchResponse> submitListener) {
|
||||
CancellableTask submitTask = (CancellableTask) task;
|
||||
final SearchRequest searchRequest = createSearchRequest(request, submitTask.getId(), request.getKeepAlive());
|
||||
AsyncSearchTask searchTask = (AsyncSearchTask) taskManager.register("transport", SearchAction.INSTANCE.name(), searchRequest);
|
||||
searchAction.execute(searchTask, searchRequest, searchTask.getProgressListener());
|
||||
searchTask.addCompletionListener(
|
||||
new ActionListener<AsyncSearchResponse>() {
|
||||
@Override
|
||||
public void onResponse(AsyncSearchResponse searchResponse) {
|
||||
if (searchResponse.isRunning() || request.isCleanOnCompletion() == false) {
|
||||
// the task is still running and the user cannot wait more so we create
|
||||
// a document for further retrieval
|
||||
try {
|
||||
if (submitTask.isCancelled()) {
|
||||
// the user cancelled the submit so we don't store anything
|
||||
// and propagate the failure
|
||||
Exception cause = new TaskCancelledException(submitTask.getReasonCancelled());
|
||||
onFatalFailure(searchTask, cause, false, submitListener);
|
||||
} else {
|
||||
final String docId = searchTask.getSearchId().getDocId();
|
||||
// creates the fallback response if the node crashes/restarts in the middle of the request
|
||||
// TODO: store intermediate results ?
|
||||
AsyncSearchResponse initialResp = searchResponse.clone(searchResponse.getId());
|
||||
store.storeInitialResponse(docId, searchTask.getOriginHeaders(), initialResp,
|
||||
new ActionListener<IndexResponse>() {
|
||||
@Override
|
||||
public void onResponse(IndexResponse r) {
|
||||
if (searchResponse.isRunning()) {
|
||||
try {
|
||||
// store the final response on completion unless the submit is cancelled
|
||||
searchTask.addCompletionListener(finalResponse ->
|
||||
onFinalResponse(submitTask, searchTask, finalResponse, () -> {}));
|
||||
} finally {
|
||||
submitListener.onResponse(searchResponse);
|
||||
}
|
||||
} else {
|
||||
onFinalResponse(submitTask, searchTask, searchResponse,
|
||||
() -> submitListener.onResponse(searchResponse));
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onFailure(Exception exc) {
|
||||
onFatalFailure(searchTask, exc, searchResponse.isRunning(), submitListener);
|
||||
}
|
||||
});
|
||||
}
|
||||
} catch (Exception exc) {
|
||||
onFatalFailure(searchTask, exc, searchResponse.isRunning(), submitListener);
|
||||
}
|
||||
} else {
|
||||
// the task completed within the timeout so the response is sent back to the user
|
||||
// with a null id since nothing was stored on the cluster.
|
||||
taskManager.unregister(searchTask);
|
||||
submitListener.onResponse(searchResponse.clone(null));
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onFailure(Exception exc) {
|
||||
submitListener.onFailure(exc);
|
||||
}
|
||||
}, request.getWaitForCompletion());
|
||||
}
|
||||
|
||||
private SearchRequest createSearchRequest(SubmitAsyncSearchRequest request, long parentTaskId, TimeValue keepAlive) {
|
||||
String docID = UUIDs.randomBase64UUID();
|
||||
Map<String, String> originHeaders = nodeClient.threadPool().getThreadContext().getHeaders();
|
||||
SearchRequest searchRequest = new SearchRequest(request.getSearchRequest()) {
|
||||
@Override
|
||||
public AsyncSearchTask createTask(long id, String type, String action, TaskId parentTaskId, Map<String, String> taskHeaders) {
|
||||
AsyncSearchId searchId = new AsyncSearchId(docID, new TaskId(nodeClient.getLocalNodeId(), id));
|
||||
return new AsyncSearchTask(id, type, action, parentTaskId, keepAlive, originHeaders, taskHeaders, searchId,
|
||||
store.getClient(), nodeClient.threadPool(), reduceContextSupplier);
|
||||
}
|
||||
};
|
||||
searchRequest.setParentTask(new TaskId(nodeClient.getLocalNodeId(), parentTaskId));
|
||||
return searchRequest;
|
||||
}
|
||||
|
||||
private void onFatalFailure(AsyncSearchTask task, Exception error, boolean shouldCancel, ActionListener<AsyncSearchResponse> listener) {
|
||||
if (shouldCancel) {
|
||||
task.cancelTask(() -> {
|
||||
try {
|
||||
task.addCompletionListener(finalResponse -> taskManager.unregister(task));
|
||||
} finally {
|
||||
listener.onFailure(error);
|
||||
}
|
||||
});
|
||||
} else {
|
||||
try {
|
||||
task.addCompletionListener(finalResponse -> taskManager.unregister(task));
|
||||
} finally {
|
||||
listener.onFailure(error);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void onFinalResponse(CancellableTask submitTask,
|
||||
AsyncSearchTask searchTask,
|
||||
AsyncSearchResponse response,
|
||||
Runnable nextAction) {
|
||||
if (submitTask.isCancelled() || searchTask.isCancelled()) {
|
||||
// the user cancelled the submit so we ensure that there is nothing stored in the response index.
|
||||
store.deleteResponse(searchTask.getSearchId(), false, ActionListener.wrap(() -> {
|
||||
taskManager.unregister(searchTask);
|
||||
nextAction.run();
|
||||
}));
|
||||
return;
|
||||
}
|
||||
|
||||
try {
|
||||
store.storeFinalResponse(searchTask.getSearchId().getDocId(), response, new ActionListener<UpdateResponse>() {
|
||||
@Override
|
||||
public void onResponse(UpdateResponse updateResponse) {
|
||||
taskManager.unregister(searchTask);
|
||||
nextAction.run();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onFailure(Exception exc) {
|
||||
if (exc.getCause() instanceof DocumentMissingException == false) {
|
||||
logger.error(() -> new ParameterizedMessage("failed to store async-search [{}]",
|
||||
searchTask.getSearchId().getEncoded()), exc);
|
||||
}
|
||||
taskManager.unregister(searchTask);
|
||||
nextAction.run();
|
||||
}
|
||||
});
|
||||
} catch (Exception exc) {
|
||||
logger.error(() -> new ParameterizedMessage("failed to store async-search [{}]", searchTask.getSearchId().getEncoded()), exc);
|
||||
taskManager.unregister(searchTask);
|
||||
nextAction.run();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,255 @@
|
|||
/*
|
||||
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||
* or more contributor license agreements. Licensed under the Elastic License;
|
||||
* you may not use this file except in compliance with the Elastic License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.xpack.search;
|
||||
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.action.index.IndexRequestBuilder;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.search.aggregations.AggregationBuilders;
|
||||
import org.elasticsearch.search.aggregations.bucket.terms.InternalTerms;
|
||||
import org.elasticsearch.search.aggregations.bucket.terms.StringTerms;
|
||||
import org.elasticsearch.search.aggregations.metrics.InternalMax;
|
||||
import org.elasticsearch.search.aggregations.metrics.InternalMin;
|
||||
import org.elasticsearch.search.builder.SearchSourceBuilder;
|
||||
import org.elasticsearch.xpack.core.search.action.AsyncSearchResponse;
|
||||
import org.elasticsearch.xpack.core.search.action.SubmitAsyncSearchRequest;
|
||||
import org.junit.Before;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
import static org.hamcrest.Matchers.containsString;
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
|
||||
import static org.hamcrest.Matchers.lessThanOrEqualTo;
|
||||
|
||||
// TODO: add tests for keepAlive and expiration
|
||||
public class AsyncSearchActionTests extends AsyncSearchIntegTestCase {
|
||||
private String indexName;
|
||||
private int numShards;
|
||||
private int numDocs;
|
||||
|
||||
private int numKeywords;
|
||||
private Map<String, AtomicInteger> keywordFreqs;
|
||||
private float maxMetric = Float.NEGATIVE_INFINITY;
|
||||
private float minMetric = Float.POSITIVE_INFINITY;
|
||||
|
||||
@Before
|
||||
public void indexDocuments() throws InterruptedException {
|
||||
indexName = "test-async";
|
||||
numShards = randomIntBetween(internalCluster().numDataNodes(), internalCluster().numDataNodes()*10);
|
||||
int numDocs = randomIntBetween(numShards, numShards*3);
|
||||
createIndex(indexName, Settings.builder().put("index.number_of_shards", numShards).build());
|
||||
numKeywords = randomIntBetween(1, 100);
|
||||
keywordFreqs = new HashMap<>();
|
||||
Set<String> keywordSet = new HashSet<>();
|
||||
for (int i = 0; i < numKeywords; i++) {
|
||||
keywordSet.add(randomAlphaOfLengthBetween(10, 20));
|
||||
}
|
||||
numKeywords = keywordSet.size();
|
||||
String[] keywords = keywordSet.toArray(new String[0]);
|
||||
List<IndexRequestBuilder> reqs = new ArrayList<>();
|
||||
for (int i = 0; i < numDocs; i++) {
|
||||
float metric = randomFloat();
|
||||
maxMetric = Math.max(metric, maxMetric);
|
||||
minMetric = Math.min(metric, minMetric);
|
||||
String keyword = keywords[randomIntBetween(0, numKeywords-1)];
|
||||
keywordFreqs.compute(keyword,
|
||||
(k, v) -> {
|
||||
if (v == null) {
|
||||
return new AtomicInteger(1);
|
||||
}
|
||||
v.incrementAndGet();
|
||||
return v;
|
||||
});
|
||||
reqs.add(client().prepareIndex(indexName, "_doc").setSource("terms", keyword, "metric", metric));
|
||||
}
|
||||
indexRandom(true, true, reqs);
|
||||
ensureGreen("test-async");
|
||||
}
|
||||
|
||||
public void testMaxMinAggregation() throws Exception {
|
||||
int step = numShards > 2 ? randomIntBetween(2, numShards) : 2;
|
||||
int numFailures = randomBoolean() ? randomIntBetween(0, numShards) : 0;
|
||||
SearchSourceBuilder source = new SearchSourceBuilder()
|
||||
.aggregation(AggregationBuilders.min("min").field("metric"))
|
||||
.aggregation(AggregationBuilders.max("max").field("metric"));
|
||||
try (SearchResponseIterator it =
|
||||
assertBlockingIterator(indexName, source, numFailures, step)) {
|
||||
AsyncSearchResponse response = it.next();
|
||||
while (it.hasNext()) {
|
||||
response = it.next();
|
||||
assertNotNull(response.getSearchResponse());
|
||||
if (response.getSearchResponse().getSuccessfulShards() > 0) {
|
||||
assertNotNull(response.getSearchResponse().getAggregations());
|
||||
assertNotNull(response.getSearchResponse().getAggregations().get("max"));
|
||||
assertNotNull(response.getSearchResponse().getAggregations().get("min"));
|
||||
InternalMax max = response.getSearchResponse().getAggregations().get("max");
|
||||
InternalMin min = response.getSearchResponse().getAggregations().get("min");
|
||||
assertThat((float) min.getValue(), greaterThanOrEqualTo(minMetric));
|
||||
assertThat((float) max.getValue(), lessThanOrEqualTo(maxMetric));
|
||||
}
|
||||
}
|
||||
if (numFailures == numShards) {
|
||||
assertNotNull(response.getFailure());
|
||||
} else {
|
||||
assertNotNull(response.getSearchResponse());
|
||||
assertNotNull(response.getSearchResponse().getAggregations());
|
||||
assertNotNull(response.getSearchResponse().getAggregations().get("max"));
|
||||
assertNotNull(response.getSearchResponse().getAggregations().get("min"));
|
||||
InternalMax max = response.getSearchResponse().getAggregations().get("max");
|
||||
InternalMin min = response.getSearchResponse().getAggregations().get("min");
|
||||
if (numFailures == 0) {
|
||||
assertThat((float) min.getValue(), equalTo(minMetric));
|
||||
assertThat((float) max.getValue(), equalTo(maxMetric));
|
||||
} else {
|
||||
assertThat((float) min.getValue(), greaterThanOrEqualTo(minMetric));
|
||||
assertThat((float) max.getValue(), lessThanOrEqualTo(maxMetric));
|
||||
}
|
||||
}
|
||||
deleteAsyncSearch(response.getId());
|
||||
ensureTaskRemoval(response.getId());
|
||||
}
|
||||
}
|
||||
|
||||
public void testTermsAggregation() throws Exception {
|
||||
int step = numShards > 2 ? randomIntBetween(2, numShards) : 2;
|
||||
int numFailures = randomBoolean() ? randomIntBetween(0, numShards) : 0;
|
||||
SearchSourceBuilder source = new SearchSourceBuilder()
|
||||
.aggregation(AggregationBuilders.terms("terms").field("terms.keyword").size(numKeywords));
|
||||
try (SearchResponseIterator it =
|
||||
assertBlockingIterator(indexName, source, numFailures, step)) {
|
||||
AsyncSearchResponse response = it.next();
|
||||
while (it.hasNext()) {
|
||||
response = it.next();
|
||||
assertNotNull(response.getSearchResponse());
|
||||
if (response.getSearchResponse().getSuccessfulShards() > 0) {
|
||||
assertNotNull(response.getSearchResponse().getAggregations());
|
||||
assertNotNull(response.getSearchResponse().getAggregations().get("terms"));
|
||||
StringTerms terms = response.getSearchResponse().getAggregations().get("terms");
|
||||
assertThat(terms.getBuckets().size(), greaterThanOrEqualTo(0));
|
||||
assertThat(terms.getBuckets().size(), lessThanOrEqualTo(numKeywords));
|
||||
for (InternalTerms.Bucket bucket : terms.getBuckets()) {
|
||||
long count = keywordFreqs.getOrDefault(bucket.getKeyAsString(), new AtomicInteger(0)).get();
|
||||
assertThat(bucket.getDocCount(), lessThanOrEqualTo(count));
|
||||
}
|
||||
}
|
||||
}
|
||||
if (numFailures == numShards) {
|
||||
assertNotNull(response.getFailure());
|
||||
} else {
|
||||
assertNotNull(response.getSearchResponse());
|
||||
assertNotNull(response.getSearchResponse().getAggregations());
|
||||
assertNotNull(response.getSearchResponse().getAggregations().get("terms"));
|
||||
StringTerms terms = response.getSearchResponse().getAggregations().get("terms");
|
||||
assertThat(terms.getBuckets().size(), greaterThanOrEqualTo(0));
|
||||
assertThat(terms.getBuckets().size(), lessThanOrEqualTo(numKeywords));
|
||||
for (InternalTerms.Bucket bucket : terms.getBuckets()) {
|
||||
long count = keywordFreqs.getOrDefault(bucket.getKeyAsString(), new AtomicInteger(0)).get();
|
||||
if (numFailures > 0) {
|
||||
assertThat(bucket.getDocCount(), lessThanOrEqualTo(count));
|
||||
} else {
|
||||
assertThat(bucket.getDocCount(), equalTo(count));
|
||||
}
|
||||
}
|
||||
}
|
||||
deleteAsyncSearch(response.getId());
|
||||
ensureTaskRemoval(response.getId());
|
||||
}
|
||||
}
|
||||
|
||||
public void testRestartAfterCompletion() throws Exception {
|
||||
final AsyncSearchResponse initial;
|
||||
try (SearchResponseIterator it =
|
||||
assertBlockingIterator(indexName, new SearchSourceBuilder(), 0, 2)) {
|
||||
initial = it.next();
|
||||
}
|
||||
ensureTaskCompletion(initial.getId());
|
||||
restartTaskNode(initial.getId());
|
||||
AsyncSearchResponse response = getAsyncSearch(initial.getId());
|
||||
assertNotNull(response.getSearchResponse());
|
||||
assertFalse(response.isRunning());
|
||||
assertFalse(response.isPartial());
|
||||
deleteAsyncSearch(response.getId());
|
||||
ensureTaskRemoval(response.getId());
|
||||
}
|
||||
|
||||
public void testDeleteCancelRunningTask() throws Exception {
|
||||
final AsyncSearchResponse initial;
|
||||
SearchResponseIterator it =
|
||||
assertBlockingIterator(indexName, new SearchSourceBuilder(), randomBoolean() ? 1 : 0, 2);
|
||||
initial = it.next();
|
||||
deleteAsyncSearch(initial.getId());
|
||||
it.close();
|
||||
ensureTaskCompletion(initial.getId());
|
||||
ensureTaskRemoval(initial.getId());
|
||||
}
|
||||
|
||||
public void testDeleteCleanupIndex() throws Exception {
|
||||
SearchResponseIterator it =
|
||||
assertBlockingIterator(indexName, new SearchSourceBuilder(), randomBoolean() ? 1 : 0, 2);
|
||||
AsyncSearchResponse response = it.next();
|
||||
deleteAsyncSearch(response.getId());
|
||||
it.close();
|
||||
ensureTaskCompletion(response.getId());
|
||||
ensureTaskRemoval(response.getId());
|
||||
}
|
||||
|
||||
public void testCleanupOnFailure() throws Exception {
|
||||
final AsyncSearchResponse initial;
|
||||
try (SearchResponseIterator it =
|
||||
assertBlockingIterator(indexName, new SearchSourceBuilder(), numShards, 2)) {
|
||||
initial = it.next();
|
||||
}
|
||||
ensureTaskCompletion(initial.getId());
|
||||
AsyncSearchResponse response = getAsyncSearch(initial.getId());
|
||||
assertFalse(response.isRunning());
|
||||
assertNotNull(response.getFailure());
|
||||
assertTrue(response.isPartial());
|
||||
assertThat(response.getSearchResponse().getTotalShards(), equalTo(numShards));
|
||||
assertThat(response.getSearchResponse().getShardFailures().length, equalTo(numShards));
|
||||
deleteAsyncSearch(initial.getId());
|
||||
ensureTaskRemoval(initial.getId());
|
||||
}
|
||||
|
||||
public void testInvalidId() throws Exception {
|
||||
SearchResponseIterator it =
|
||||
assertBlockingIterator(indexName, new SearchSourceBuilder(), randomBoolean() ? 1 : 0, 2);
|
||||
AsyncSearchResponse response = it.next();
|
||||
ExecutionException exc = expectThrows(ExecutionException.class, () -> getAsyncSearch("invalid"));
|
||||
assertThat(exc.getMessage(), containsString("invalid id"));
|
||||
while (it.hasNext()) {
|
||||
response = it.next();
|
||||
}
|
||||
assertFalse(response.isRunning());
|
||||
}
|
||||
|
||||
public void testNoIndex() throws Exception {
|
||||
SubmitAsyncSearchRequest request = new SubmitAsyncSearchRequest(new String[] { "invalid-*" });
|
||||
request.setWaitForCompletion(TimeValue.timeValueMillis(1));
|
||||
AsyncSearchResponse response = submitAsyncSearch(request);
|
||||
assertNotNull(response.getSearchResponse());
|
||||
assertFalse(response.isRunning());
|
||||
assertThat(response.getSearchResponse().getTotalShards(), equalTo(0));
|
||||
|
||||
request = new SubmitAsyncSearchRequest(new String[] { "invalid" });
|
||||
request.setWaitForCompletion(TimeValue.timeValueMillis(1));
|
||||
response = submitAsyncSearch(request);
|
||||
assertNull(response.getSearchResponse());
|
||||
assertNotNull(response.getFailure());
|
||||
assertFalse(response.isRunning());
|
||||
ElasticsearchException exc = response.getFailure();
|
||||
assertThat(exc.getMessage(), containsString("no such index"));
|
||||
}
|
||||
}
|
|
@ -0,0 +1,42 @@
|
|||
/*
|
||||
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||
* or more contributor license agreements. Licensed under the Elastic License;
|
||||
* you may not use this file except in compliance with the Elastic License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.xpack.search;
|
||||
|
||||
import org.elasticsearch.common.UUIDs;
|
||||
import org.elasticsearch.tasks.TaskId;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
|
||||
public class AsyncSearchIdTests extends ESTestCase {
|
||||
public void testEncode() {
|
||||
for (int i = 0; i < 10; i++) {
|
||||
AsyncSearchId instance = new AsyncSearchId(UUIDs.randomBase64UUID(),
|
||||
new TaskId(randomAlphaOfLengthBetween(5, 20), randomNonNegativeLong()));
|
||||
String encoded = AsyncSearchId.encode(instance.getDocId(), instance.getTaskId());
|
||||
AsyncSearchId same = AsyncSearchId.decode(encoded);
|
||||
assertEquals(same, instance);
|
||||
|
||||
AsyncSearchId mutate = mutate(instance);
|
||||
assertNotEquals(mutate, instance);
|
||||
assertNotEquals(mutate, same);
|
||||
}
|
||||
}
|
||||
|
||||
private AsyncSearchId mutate(AsyncSearchId id) {
|
||||
int rand = randomIntBetween(0, 1);
|
||||
switch (rand) {
|
||||
case 0:
|
||||
return new AsyncSearchId(randomAlphaOfLength(id.getDocId().length()+1), id.getTaskId());
|
||||
|
||||
case 1:
|
||||
return new AsyncSearchId(id.getDocId(),
|
||||
new TaskId(randomAlphaOfLength(id.getTaskId().getNodeId().length()), randomNonNegativeLong()));
|
||||
|
||||
default:
|
||||
throw new AssertionError();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,103 @@
|
|||
/*
|
||||
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||
* or more contributor license agreements. Licensed under the Elastic License;
|
||||
* you may not use this file except in compliance with the Elastic License.
|
||||
*/
|
||||
package org.elasticsearch.xpack.search;
|
||||
|
||||
import org.elasticsearch.cluster.service.ClusterService;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.util.concurrent.ThreadContext;
|
||||
import org.elasticsearch.test.ESSingleNodeTestCase;
|
||||
import org.elasticsearch.transport.TransportService;
|
||||
import org.elasticsearch.xpack.core.search.action.AsyncSearchResponse;
|
||||
import org.elasticsearch.xpack.core.security.authc.Authentication;
|
||||
import org.elasticsearch.xpack.core.security.user.User;
|
||||
import org.junit.Before;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
|
||||
import static org.elasticsearch.xpack.search.AsyncSearchResponseTests.assertEqualResponses;
|
||||
import static org.elasticsearch.xpack.search.AsyncSearchResponseTests.randomAsyncSearchResponse;
|
||||
import static org.elasticsearch.xpack.search.AsyncSearchResponseTests.randomSearchResponse;
|
||||
import static org.elasticsearch.xpack.search.GetAsyncSearchRequestTests.randomSearchId;
|
||||
|
||||
// TODO: test CRUD operations
|
||||
public class AsyncSearchIndexServiceTests extends ESSingleNodeTestCase {
|
||||
private AsyncSearchIndexService indexService;
|
||||
|
||||
@Before
|
||||
public void setup() {
|
||||
ClusterService clusterService = getInstanceFromNode(ClusterService.class);
|
||||
TransportService transportService = getInstanceFromNode(TransportService.class);
|
||||
indexService = new AsyncSearchIndexService(clusterService, transportService.getThreadPool().getThreadContext(),
|
||||
client(), writableRegistry());
|
||||
}
|
||||
|
||||
public void testEncodeSearchResponse() throws IOException {
|
||||
for (int i = 0; i < 10; i++) {
|
||||
AsyncSearchResponse response = randomAsyncSearchResponse(randomSearchId(), randomSearchResponse());
|
||||
String encoded = indexService.encodeResponse(response);
|
||||
AsyncSearchResponse same = indexService.decodeResponse(encoded);
|
||||
assertEqualResponses(response, same);
|
||||
}
|
||||
}
|
||||
|
||||
public void testEnsuredAuthenticatedUserIsSame() throws IOException {
|
||||
Authentication original =
|
||||
new Authentication(new User("test", "role"), new Authentication.RealmRef("realm", "file", "node"), null);
|
||||
Authentication current = randomBoolean() ? original :
|
||||
new Authentication(new User("test", "role"), new Authentication.RealmRef("realm", "file", "node"), null);
|
||||
assertTrue(indexService.ensureAuthenticatedUserIsSame(original, current));
|
||||
ThreadContext threadContext = new ThreadContext(Settings.EMPTY);
|
||||
original.writeToContext(threadContext);
|
||||
assertTrue(indexService.ensureAuthenticatedUserIsSame(threadContext.getHeaders(), current));
|
||||
|
||||
// original is not set
|
||||
assertTrue(indexService.ensureAuthenticatedUserIsSame(Collections.emptyMap(), current));
|
||||
// current is not set
|
||||
assertFalse(indexService.ensureAuthenticatedUserIsSame(threadContext.getHeaders(), null));
|
||||
|
||||
// original user being run as
|
||||
User user = new User(new User("test", "role"), new User("authenticated", "runas"));
|
||||
current = new Authentication(user, new Authentication.RealmRef("realm", "file", "node"),
|
||||
new Authentication.RealmRef(randomAlphaOfLengthBetween(1, 16), "file", "node"));
|
||||
assertTrue(indexService.ensureAuthenticatedUserIsSame(original, current));
|
||||
assertTrue(indexService.ensureAuthenticatedUserIsSame(threadContext.getHeaders(), current));
|
||||
|
||||
// both user are run as
|
||||
current = new Authentication(user, new Authentication.RealmRef("realm", "file", "node"),
|
||||
new Authentication.RealmRef(randomAlphaOfLengthBetween(1, 16), "file", "node"));
|
||||
Authentication runAs = current;
|
||||
assertTrue(indexService.ensureAuthenticatedUserIsSame(runAs, current));
|
||||
threadContext = new ThreadContext(Settings.EMPTY);
|
||||
original.writeToContext(threadContext);
|
||||
assertTrue(indexService.ensureAuthenticatedUserIsSame(threadContext.getHeaders(), current));
|
||||
|
||||
// different authenticated by type
|
||||
Authentication differentRealmType =
|
||||
new Authentication(new User("test", "role"), new Authentication.RealmRef("realm", randomAlphaOfLength(5), "node"), null);
|
||||
threadContext = new ThreadContext(Settings.EMPTY);
|
||||
original.writeToContext(threadContext);
|
||||
assertFalse(indexService.ensureAuthenticatedUserIsSame(original, differentRealmType));
|
||||
assertFalse(indexService.ensureAuthenticatedUserIsSame(threadContext.getHeaders(), differentRealmType));
|
||||
|
||||
// wrong user
|
||||
Authentication differentUser =
|
||||
new Authentication(new User("test2", "role"), new Authentication.RealmRef("realm", "realm", "node"), null);
|
||||
assertFalse(indexService.ensureAuthenticatedUserIsSame(original, differentUser));
|
||||
|
||||
// run as different user
|
||||
Authentication diffRunAs = new Authentication(new User(new User("test2", "role"), new User("authenticated", "runas")),
|
||||
new Authentication.RealmRef("realm", "file", "node1"), new Authentication.RealmRef("realm", "file", "node1"));
|
||||
assertFalse(indexService.ensureAuthenticatedUserIsSame(original, diffRunAs));
|
||||
assertFalse(indexService.ensureAuthenticatedUserIsSame(threadContext.getHeaders(), diffRunAs));
|
||||
|
||||
// run as different looked up by type
|
||||
Authentication runAsDiffType = new Authentication(user, new Authentication.RealmRef("realm", "file", "node"),
|
||||
new Authentication.RealmRef(randomAlphaOfLengthBetween(1, 16), randomAlphaOfLengthBetween(5, 12), "node"));
|
||||
assertFalse(indexService.ensureAuthenticatedUserIsSame(original, runAsDiffType));
|
||||
assertFalse(indexService.ensureAuthenticatedUserIsSame(threadContext.getHeaders(), runAsDiffType));
|
||||
}
|
||||
}
|
|
@ -0,0 +1,417 @@
|
|||
/*
|
||||
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||
* or more contributor license agreements. Licensed under the Elastic License;
|
||||
* you may not use this file except in compliance with the Elastic License.
|
||||
*/
|
||||
package org.elasticsearch.xpack.search;
|
||||
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.search.ScoreMode;
|
||||
import org.apache.lucene.search.TotalHits;
|
||||
import org.apache.lucene.search.Weight;
|
||||
import org.elasticsearch.ResourceNotFoundException;
|
||||
import org.elasticsearch.action.admin.cluster.node.tasks.get.GetTaskResponse;
|
||||
import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsGroup;
|
||||
import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsResponse;
|
||||
import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse;
|
||||
import org.elasticsearch.action.get.GetResponse;
|
||||
import org.elasticsearch.action.support.master.AcknowledgedResponse;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||
import org.elasticsearch.common.ParsingException;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.lucene.search.Queries;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.common.xcontent.ObjectParser;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.common.xcontent.XContentParser;
|
||||
import org.elasticsearch.index.query.AbstractQueryBuilder;
|
||||
import org.elasticsearch.index.query.QueryShardContext;
|
||||
import org.elasticsearch.index.reindex.ReindexPlugin;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
import org.elasticsearch.plugins.Plugin;
|
||||
import org.elasticsearch.plugins.PluginsService;
|
||||
import org.elasticsearch.plugins.SearchPlugin;
|
||||
import org.elasticsearch.rest.RestStatus;
|
||||
import org.elasticsearch.search.builder.SearchSourceBuilder;
|
||||
import org.elasticsearch.tasks.TaskId;
|
||||
import org.elasticsearch.test.ESIntegTestCase;
|
||||
import org.elasticsearch.test.InternalTestCluster;
|
||||
import org.elasticsearch.xpack.core.LocalStateCompositeXPackPlugin;
|
||||
import org.elasticsearch.xpack.core.search.action.AsyncSearchResponse;
|
||||
import org.elasticsearch.xpack.core.search.action.DeleteAsyncSearchAction;
|
||||
import org.elasticsearch.xpack.core.search.action.GetAsyncSearchAction;
|
||||
import org.elasticsearch.xpack.core.search.action.SubmitAsyncSearchAction;
|
||||
import org.elasticsearch.xpack.core.search.action.SubmitAsyncSearchRequest;
|
||||
import org.elasticsearch.xpack.core.XPackClientPlugin;
|
||||
import org.elasticsearch.xpack.core.XPackSettings;
|
||||
import org.elasticsearch.xpack.ilm.IndexLifecycle;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.Comparator;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
import java.util.function.Function;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.elasticsearch.xpack.search.AsyncSearchIndexService.INDEX;
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
import static org.hamcrest.Matchers.lessThanOrEqualTo;
|
||||
|
||||
public abstract class AsyncSearchIntegTestCase extends ESIntegTestCase {
|
||||
interface SearchResponseIterator extends Iterator<AsyncSearchResponse>, Closeable {}
|
||||
|
||||
@Override
|
||||
protected Collection<Class<? extends Plugin>> nodePlugins() {
|
||||
return Arrays.asList(LocalStateCompositeXPackPlugin.class, AsyncSearch.class, IndexLifecycle.class,
|
||||
QueryBlockPlugin.class, ReindexPlugin.class);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Collection<Class<? extends Plugin>> transportClientPlugins() {
|
||||
return Collections.singletonList(XPackClientPlugin.class);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Settings transportClientSettings() {
|
||||
return Settings.builder().put(super.transportClientSettings())
|
||||
.put(XPackSettings.SECURITY_ENABLED.getKey(), false).build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Restart the node that runs the {@link TaskId} decoded from the provided {@link AsyncSearchId}.
|
||||
*/
|
||||
protected void restartTaskNode(String id) throws Exception {
|
||||
AsyncSearchId searchId = AsyncSearchId.decode(id);
|
||||
final ClusterStateResponse clusterState = client().admin().cluster()
|
||||
.prepareState().clear().setNodes(true).get();
|
||||
DiscoveryNode node = clusterState.getState().nodes().get(searchId.getTaskId().getNodeId());
|
||||
internalCluster().restartNode(node.getName(), new InternalTestCluster.RestartCallback() {
|
||||
@Override
|
||||
public Settings onNodeStopped(String nodeName) throws Exception {
|
||||
return super.onNodeStopped(nodeName);
|
||||
}
|
||||
});
|
||||
ensureYellow(INDEX);
|
||||
}
|
||||
|
||||
protected AsyncSearchResponse submitAsyncSearch(SubmitAsyncSearchRequest request) throws ExecutionException, InterruptedException {
|
||||
return client().execute(SubmitAsyncSearchAction.INSTANCE, request).get();
|
||||
}
|
||||
|
||||
protected AsyncSearchResponse getAsyncSearch(String id) throws ExecutionException, InterruptedException {
|
||||
return client().execute(GetAsyncSearchAction.INSTANCE, new GetAsyncSearchAction.Request(id)).get();
|
||||
}
|
||||
|
||||
protected AcknowledgedResponse deleteAsyncSearch(String id) throws ExecutionException, InterruptedException {
|
||||
return client().execute(DeleteAsyncSearchAction.INSTANCE, new DeleteAsyncSearchAction.Request(id)).get();
|
||||
}
|
||||
|
||||
/**
|
||||
* Wait the removal of the document decoded from the provided {@link AsyncSearchId}.
|
||||
*/
|
||||
protected void ensureTaskRemoval(String id) throws Exception {
|
||||
AsyncSearchId searchId = AsyncSearchId.decode(id);
|
||||
assertBusy(() -> {
|
||||
GetResponse resp = client().prepareGet()
|
||||
.setIndex(INDEX)
|
||||
.setId(searchId.getDocId())
|
||||
.get();
|
||||
assertFalse(resp.isExists());
|
||||
});
|
||||
}
|
||||
|
||||
/**
|
||||
* Wait the completion of the {@link TaskId} decoded from the provided {@link AsyncSearchId}.
|
||||
*/
|
||||
protected void ensureTaskCompletion(String id) throws Exception {
|
||||
assertBusy(() -> {
|
||||
TaskId taskId = AsyncSearchId.decode(id).getTaskId();
|
||||
try {
|
||||
GetTaskResponse resp = client().admin().cluster()
|
||||
.prepareGetTask(taskId).get();
|
||||
assertNull(resp.getTask());
|
||||
} catch (Exception exc) {
|
||||
if (exc.getCause() instanceof ResourceNotFoundException == false) {
|
||||
throw exc;
|
||||
}
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
protected SearchResponseIterator assertBlockingIterator(String indexName,
|
||||
SearchSourceBuilder source,
|
||||
int numFailures,
|
||||
int progressStep) throws Exception {
|
||||
SubmitAsyncSearchRequest request = new SubmitAsyncSearchRequest(source, indexName);
|
||||
request.setBatchedReduceSize(progressStep);
|
||||
request.setWaitForCompletion(TimeValue.timeValueMillis(1));
|
||||
ClusterSearchShardsResponse response = dataNodeClient().admin().cluster()
|
||||
.prepareSearchShards(request.getSearchRequest().indices()).get();
|
||||
AtomicInteger failures = new AtomicInteger(numFailures);
|
||||
Map<ShardId, ShardIdLatch> shardLatchMap = Arrays.stream(response.getGroups())
|
||||
.map(ClusterSearchShardsGroup::getShardId)
|
||||
.collect(
|
||||
Collectors.toMap(
|
||||
Function.identity(),
|
||||
id -> new ShardIdLatch(id, new CountDownLatch(1), failures.decrementAndGet() >= 0 ? true : false)
|
||||
)
|
||||
);
|
||||
ShardIdLatch[] shardLatchArray = shardLatchMap.values().stream()
|
||||
.sorted(Comparator.comparing(ShardIdLatch::shard))
|
||||
.toArray(ShardIdLatch[]::new);
|
||||
resetPluginsLatch(shardLatchMap);
|
||||
request.getSearchRequest().source().query(new BlockQueryBuilder(shardLatchMap));
|
||||
|
||||
final AsyncSearchResponse initial = client().execute(SubmitAsyncSearchAction.INSTANCE, request).get();
|
||||
|
||||
assertTrue(initial.isPartial());
|
||||
assertThat(initial.status(), equalTo(RestStatus.OK));
|
||||
assertThat(initial.getSearchResponse().getTotalShards(), equalTo(shardLatchArray.length));
|
||||
assertThat(initial.getSearchResponse().getSuccessfulShards(), equalTo(0));
|
||||
assertThat(initial.getSearchResponse().getShardFailures().length, equalTo(0));
|
||||
|
||||
return new SearchResponseIterator() {
|
||||
private AsyncSearchResponse response = initial;
|
||||
private int lastVersion = initial.getVersion();
|
||||
private int shardIndex = 0;
|
||||
private boolean isFirst = true;
|
||||
private int shardFailures = 0;
|
||||
|
||||
@Override
|
||||
public boolean hasNext() {
|
||||
return response.isRunning();
|
||||
}
|
||||
|
||||
@Override
|
||||
public AsyncSearchResponse next() {
|
||||
try {
|
||||
return doNext();
|
||||
} catch (Exception e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
private AsyncSearchResponse doNext() throws Exception {
|
||||
if (isFirst) {
|
||||
isFirst = false;
|
||||
return response;
|
||||
}
|
||||
AtomicReference<AsyncSearchResponse> atomic = new AtomicReference<>();
|
||||
int step = shardIndex == 0 ? progressStep+1 : progressStep-1;
|
||||
int index = 0;
|
||||
while (index < step && shardIndex < shardLatchArray.length) {
|
||||
if (shardLatchArray[shardIndex].shouldFail == false) {
|
||||
++index;
|
||||
} else {
|
||||
++shardFailures;
|
||||
}
|
||||
shardLatchArray[shardIndex++].countDown();
|
||||
}
|
||||
assertBusy(() -> {
|
||||
AsyncSearchResponse newResp = client().execute(GetAsyncSearchAction.INSTANCE,
|
||||
new GetAsyncSearchAction.Request(response.getId())
|
||||
.setWaitForCompletion(TimeValue.timeValueMillis(10))).get();
|
||||
atomic.set(newResp);
|
||||
assertNotEquals(lastVersion, newResp.getVersion());
|
||||
});
|
||||
AsyncSearchResponse newResponse = atomic.get();
|
||||
lastVersion = newResponse.getVersion();
|
||||
|
||||
if (newResponse.isRunning()) {
|
||||
assertThat(newResponse.status(), equalTo(RestStatus.OK));
|
||||
assertTrue(newResponse.isPartial());
|
||||
assertFalse(newResponse.getFailure() != null);
|
||||
assertNotNull(newResponse.getSearchResponse());
|
||||
assertThat(newResponse.getSearchResponse().getTotalShards(), equalTo(shardLatchArray.length));
|
||||
assertThat(newResponse.getSearchResponse().getShardFailures().length, lessThanOrEqualTo(numFailures));
|
||||
} else if (numFailures == shardLatchArray.length) {
|
||||
assertThat(newResponse.status(), equalTo(RestStatus.INTERNAL_SERVER_ERROR));
|
||||
assertTrue(newResponse.getFailure() != null);
|
||||
assertTrue(newResponse.isPartial());
|
||||
assertNotNull(newResponse.getSearchResponse());
|
||||
assertThat(newResponse.getSearchResponse().getTotalShards(), equalTo(shardLatchArray.length));
|
||||
assertThat(newResponse.getSearchResponse().getSuccessfulShards(), equalTo(0));
|
||||
assertThat(newResponse.getSearchResponse().getShardFailures().length, equalTo(numFailures));
|
||||
assertNull(newResponse.getSearchResponse().getAggregations());
|
||||
assertNotNull(newResponse.getSearchResponse().getHits().getTotalHits());
|
||||
assertThat(newResponse.getSearchResponse().getHits().getTotalHits().value, equalTo(0L));
|
||||
assertThat(newResponse.getSearchResponse().getHits().getTotalHits().relation,
|
||||
equalTo(TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO));
|
||||
} else {
|
||||
assertThat(newResponse.status(), equalTo(RestStatus.OK));
|
||||
assertNotNull(newResponse.getSearchResponse());
|
||||
assertFalse(newResponse.isPartial());
|
||||
assertThat(newResponse.status(), equalTo(RestStatus.OK));
|
||||
assertThat(newResponse.getSearchResponse().getTotalShards(), equalTo(shardLatchArray.length));
|
||||
assertThat(newResponse.getSearchResponse().getShardFailures().length, equalTo(numFailures));
|
||||
assertThat(newResponse.getSearchResponse().getSuccessfulShards(),
|
||||
equalTo(shardLatchArray.length-newResponse.getSearchResponse().getShardFailures().length));
|
||||
}
|
||||
return response = newResponse;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
Arrays.stream(shardLatchArray).forEach(shard -> {
|
||||
if (shard.latch.getCount() == 1) {
|
||||
shard.latch.countDown();
|
||||
}
|
||||
});
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
private void resetPluginsLatch(Map<ShardId, ShardIdLatch> newLatch) {
|
||||
for (PluginsService pluginsService : internalCluster().getDataNodeInstances(PluginsService.class)) {
|
||||
pluginsService.filterPlugins(QueryBlockPlugin.class).forEach(p -> p.reset(newLatch));
|
||||
}
|
||||
}
|
||||
|
||||
public static class QueryBlockPlugin extends Plugin implements SearchPlugin {
|
||||
private Map<ShardId, ShardIdLatch> shardsLatch;
|
||||
|
||||
public QueryBlockPlugin() {
|
||||
this.shardsLatch = null;
|
||||
}
|
||||
|
||||
public void reset(Map<ShardId, ShardIdLatch> newLatch) {
|
||||
shardsLatch = newLatch;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<QuerySpec<?>> getQueries() {
|
||||
return Collections.singletonList(
|
||||
new QuerySpec<>("block_match_all",
|
||||
in -> new BlockQueryBuilder(in, shardsLatch),
|
||||
p -> BlockQueryBuilder.fromXContent(p, shardsLatch))
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
private static class BlockQueryBuilder extends AbstractQueryBuilder<BlockQueryBuilder> {
|
||||
public static final String NAME = "block_match_all";
|
||||
private final Map<ShardId, ShardIdLatch> shardsLatch;
|
||||
|
||||
private BlockQueryBuilder(Map<ShardId, ShardIdLatch> shardsLatch) {
|
||||
super();
|
||||
this.shardsLatch = shardsLatch;
|
||||
}
|
||||
|
||||
BlockQueryBuilder(StreamInput in, Map<ShardId, ShardIdLatch> shardsLatch) throws IOException {
|
||||
super(in);
|
||||
this.shardsLatch = shardsLatch;
|
||||
}
|
||||
|
||||
private BlockQueryBuilder() {
|
||||
this.shardsLatch = null;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void doWriteTo(StreamOutput out) throws IOException {}
|
||||
|
||||
@Override
|
||||
protected void doXContent(XContentBuilder builder, Params params) throws IOException {
|
||||
builder.startObject(NAME);
|
||||
builder.endObject();
|
||||
}
|
||||
|
||||
private static final ObjectParser<BlockQueryBuilder, Void> PARSER = new ObjectParser<>(NAME, BlockQueryBuilder::new);
|
||||
|
||||
public static BlockQueryBuilder fromXContent(XContentParser parser, Map<ShardId, ShardIdLatch> shardsLatch) {
|
||||
try {
|
||||
PARSER.apply(parser, null);
|
||||
return new BlockQueryBuilder(shardsLatch);
|
||||
} catch (IllegalArgumentException e) {
|
||||
throw new ParsingException(parser.getTokenLocation(), e.getMessage(), e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Query doToQuery(QueryShardContext context) {
|
||||
final Query delegate = Queries.newMatchAllQuery();
|
||||
return new Query() {
|
||||
@Override
|
||||
public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) throws IOException {
|
||||
if (shardsLatch != null) {
|
||||
try {
|
||||
final ShardIdLatch latch = shardsLatch.get(new ShardId(context.index(), context.getShardId()));
|
||||
latch.await();
|
||||
if (latch.shouldFail) {
|
||||
throw new IOException("boum");
|
||||
}
|
||||
} catch (InterruptedException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
return delegate.createWeight(searcher, scoreMode, boost);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString(String field) {
|
||||
return delegate.toString(field);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object obj) {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return 0;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean doEquals(BlockQueryBuilder other) {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int doHashCode() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getWriteableName() {
|
||||
return NAME;
|
||||
}
|
||||
}
|
||||
|
||||
private static class ShardIdLatch {
|
||||
private final ShardId shard;
|
||||
private final CountDownLatch latch;
|
||||
private final boolean shouldFail;
|
||||
|
||||
private ShardIdLatch(ShardId shard, CountDownLatch latch, boolean shouldFail) {
|
||||
this.shard = shard;
|
||||
this.latch = latch;
|
||||
this.shouldFail = shouldFail;
|
||||
}
|
||||
|
||||
ShardId shard() {
|
||||
return shard;
|
||||
}
|
||||
|
||||
void countDown() {
|
||||
latch.countDown();
|
||||
}
|
||||
|
||||
void await() throws InterruptedException {
|
||||
latch.await();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,131 @@
|
|||
/*
|
||||
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||
* or more contributor license agreements. Licensed under the Elastic License;
|
||||
* you may not use this file except in compliance with the Elastic License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.xpack.search;
|
||||
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.action.search.SearchResponse;
|
||||
import org.elasticsearch.action.search.ShardSearchFailure;
|
||||
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
|
||||
import org.elasticsearch.common.io.stream.Writeable;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
|
||||
import org.elasticsearch.search.SearchModule;
|
||||
import org.elasticsearch.search.internal.InternalSearchResponse;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
import org.elasticsearch.xpack.core.search.action.AsyncSearchResponse;
|
||||
import org.elasticsearch.xpack.core.transform.TransformField;
|
||||
import org.elasticsearch.xpack.core.transform.TransformNamedXContentProvider;
|
||||
import org.elasticsearch.xpack.core.transform.transforms.SyncConfig;
|
||||
import org.elasticsearch.xpack.core.transform.transforms.TimeSyncConfig;
|
||||
import org.junit.Before;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
|
||||
import static java.util.Collections.emptyList;
|
||||
import static org.elasticsearch.xpack.search.GetAsyncSearchRequestTests.randomSearchId;
|
||||
|
||||
public class AsyncSearchResponseTests extends ESTestCase {
|
||||
private SearchResponse searchResponse = randomSearchResponse();
|
||||
private NamedWriteableRegistry namedWriteableRegistry;
|
||||
|
||||
@Before
|
||||
public void registerNamedObjects() {
|
||||
SearchModule searchModule = new SearchModule(Settings.EMPTY, false, emptyList());
|
||||
|
||||
List<NamedWriteableRegistry.Entry> namedWriteables = searchModule.getNamedWriteables();
|
||||
namedWriteables.add(new NamedWriteableRegistry.Entry(SyncConfig.class, TransformField.TIME_BASED_SYNC.getPreferredName(),
|
||||
TimeSyncConfig::new));
|
||||
|
||||
List<NamedXContentRegistry.Entry> namedXContents = searchModule.getNamedXContents();
|
||||
namedXContents.addAll(new TransformNamedXContentProvider().getNamedXContentParsers());
|
||||
|
||||
namedWriteableRegistry = new NamedWriteableRegistry(namedWriteables);
|
||||
}
|
||||
|
||||
|
||||
protected Writeable.Reader<AsyncSearchResponse> instanceReader() {
|
||||
return AsyncSearchResponse::new;
|
||||
}
|
||||
|
||||
protected AsyncSearchResponse createTestInstance() {
|
||||
return randomAsyncSearchResponse(randomSearchId(), searchResponse);
|
||||
}
|
||||
|
||||
protected void assertEqualInstances(AsyncSearchResponse expectedInstance, AsyncSearchResponse newInstance) {
|
||||
assertNotSame(newInstance, expectedInstance);
|
||||
assertEqualResponses(expectedInstance, newInstance);
|
||||
}
|
||||
|
||||
public final void testSerialization() throws IOException {
|
||||
for (int runs = 0; runs < 10; runs++) {
|
||||
AsyncSearchResponse testInstance = createTestInstance();
|
||||
assertSerialization(testInstance);
|
||||
}
|
||||
}
|
||||
|
||||
protected final AsyncSearchResponse assertSerialization(AsyncSearchResponse testInstance) throws IOException {
|
||||
return assertSerialization(testInstance, Version.CURRENT);
|
||||
}
|
||||
|
||||
protected final AsyncSearchResponse assertSerialization(AsyncSearchResponse testInstance, Version version) throws IOException {
|
||||
AsyncSearchResponse deserializedInstance = copyInstance(testInstance, version);
|
||||
assertEqualInstances(testInstance, deserializedInstance);
|
||||
return deserializedInstance;
|
||||
}
|
||||
|
||||
protected final AsyncSearchResponse copyInstance(AsyncSearchResponse instance) throws IOException {
|
||||
return copyInstance(instance, Version.CURRENT);
|
||||
}
|
||||
|
||||
protected AsyncSearchResponse copyInstance(AsyncSearchResponse instance, Version version) throws IOException {
|
||||
return copyWriteable(instance, namedWriteableRegistry, instanceReader(), version);
|
||||
}
|
||||
|
||||
static AsyncSearchResponse randomAsyncSearchResponse(String searchId, SearchResponse searchResponse) {
|
||||
int rand = randomIntBetween(0, 2);
|
||||
switch (rand) {
|
||||
case 0:
|
||||
return new AsyncSearchResponse(searchId, randomIntBetween(0, Integer.MAX_VALUE), randomBoolean(),
|
||||
randomBoolean(), randomNonNegativeLong(), randomNonNegativeLong());
|
||||
|
||||
case 1:
|
||||
return new AsyncSearchResponse(searchId, randomIntBetween(0, Integer.MAX_VALUE), searchResponse, null,
|
||||
randomBoolean(), randomBoolean(), randomNonNegativeLong(), randomNonNegativeLong());
|
||||
|
||||
case 2:
|
||||
return new AsyncSearchResponse(searchId, randomIntBetween(0, Integer.MAX_VALUE), searchResponse,
|
||||
new ElasticsearchException(new IOException("boum")), randomBoolean(), randomBoolean(),
|
||||
randomNonNegativeLong(), randomNonNegativeLong());
|
||||
|
||||
default:
|
||||
throw new AssertionError();
|
||||
}
|
||||
}
|
||||
|
||||
static SearchResponse randomSearchResponse() {
|
||||
long tookInMillis = randomNonNegativeLong();
|
||||
int totalShards = randomIntBetween(1, Integer.MAX_VALUE);
|
||||
int successfulShards = randomIntBetween(0, totalShards);
|
||||
int skippedShards = totalShards - successfulShards;
|
||||
InternalSearchResponse internalSearchResponse = InternalSearchResponse.empty();
|
||||
return new SearchResponse(internalSearchResponse, null, totalShards,
|
||||
successfulShards, skippedShards, tookInMillis, ShardSearchFailure.EMPTY_ARRAY, SearchResponse.Clusters.EMPTY);
|
||||
}
|
||||
|
||||
static void assertEqualResponses(AsyncSearchResponse expected, AsyncSearchResponse actual) {
|
||||
assertEquals(expected.getId(), actual.getId());
|
||||
assertEquals(expected.getVersion(), actual.getVersion());
|
||||
assertEquals(expected.status(), actual.status());
|
||||
assertEquals(expected.getFailure() == null, actual.getFailure() == null);
|
||||
assertEquals(expected.isRunning(), actual.isRunning());
|
||||
assertEquals(expected.isPartial(), actual.isPartial());
|
||||
assertEquals(expected.getStartTime(), actual.getStartTime());
|
||||
assertEquals(expected.getExpirationTime(), actual.getExpirationTime());
|
||||
}
|
||||
}
|
|
@ -0,0 +1,198 @@
|
|||
/*
|
||||
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||
* or more contributor license agreements. Licensed under the Elastic License;
|
||||
* you may not use this file except in compliance with the Elastic License.
|
||||
*/
|
||||
package org.elasticsearch.xpack.search;
|
||||
|
||||
import org.apache.lucene.search.TotalHits;
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.search.SearchResponse;
|
||||
import org.elasticsearch.action.search.SearchShard;
|
||||
import org.elasticsearch.action.search.ShardSearchFailure;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
import org.elasticsearch.search.SearchHits;
|
||||
import org.elasticsearch.search.aggregations.InternalAggregations;
|
||||
import org.elasticsearch.search.internal.InternalSearchResponse;
|
||||
import org.elasticsearch.tasks.TaskId;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
import org.elasticsearch.test.client.NoOpClient;
|
||||
import org.elasticsearch.threadpool.TestThreadPool;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
import org.elasticsearch.xpack.core.search.action.AsyncSearchResponse;
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
|
||||
public class AsyncSearchTaskTests extends ESTestCase {
|
||||
private ThreadPool threadPool;
|
||||
|
||||
@Before
|
||||
public void beforeTest() {
|
||||
threadPool = new TestThreadPool(getTestName());
|
||||
}
|
||||
|
||||
@After
|
||||
public void afterTest() {
|
||||
threadPool.shutdownNow();
|
||||
}
|
||||
|
||||
public void testWaitForInit() throws InterruptedException {
|
||||
AsyncSearchTask task = new AsyncSearchTask(0L, "", "", new TaskId("node1", 0), TimeValue.timeValueHours(1),
|
||||
Collections.emptyMap(), Collections.emptyMap(), new AsyncSearchId("0", new TaskId("node1", 1)),
|
||||
new NoOpClient(threadPool), threadPool, null);
|
||||
int numShards = randomIntBetween(0, 10);
|
||||
List<SearchShard> shards = new ArrayList<>();
|
||||
for (int i = 0; i < numShards; i++) {
|
||||
shards.add(new SearchShard(null, new ShardId("0", "0", 1)));
|
||||
}
|
||||
List<SearchShard> skippedShards = new ArrayList<>();
|
||||
int numSkippedShards = randomIntBetween(0, 10);
|
||||
for (int i = 0; i < numSkippedShards; i++) {
|
||||
skippedShards.add(new SearchShard(null, new ShardId("0", "0", 1)));
|
||||
}
|
||||
|
||||
List<Thread> threads = new ArrayList<>();
|
||||
int numThreads = randomIntBetween(1, 10);
|
||||
CountDownLatch latch = new CountDownLatch(numThreads);
|
||||
for (int i = 0; i < numThreads; i++) {
|
||||
Thread thread = new Thread(() -> task.addCompletionListener(new ActionListener<AsyncSearchResponse>() {
|
||||
@Override
|
||||
public void onResponse(AsyncSearchResponse resp) {
|
||||
assertThat(numShards + numSkippedShards, equalTo(resp.getSearchResponse().getTotalShards()));
|
||||
assertThat(numSkippedShards, equalTo(resp.getSearchResponse().getSkippedShards()));
|
||||
assertThat(0, equalTo(resp.getSearchResponse().getFailedShards()));
|
||||
latch.countDown();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onFailure(Exception e) {
|
||||
throw new AssertionError(e);
|
||||
|
||||
}
|
||||
}, TimeValue.timeValueMillis(1)));
|
||||
threads.add(thread);
|
||||
thread.start();
|
||||
}
|
||||
assertFalse(latch.await(numThreads*2, TimeUnit.MILLISECONDS));
|
||||
task.getProgressListener().onListShards(shards, skippedShards, SearchResponse.Clusters.EMPTY, false);
|
||||
latch.await();
|
||||
}
|
||||
|
||||
public void testWithFailure() throws InterruptedException {
|
||||
AsyncSearchTask task = new AsyncSearchTask(0L, "", "", new TaskId("node1", 0), TimeValue.timeValueHours(1),
|
||||
Collections.emptyMap(), Collections.emptyMap(), new AsyncSearchId("0", new TaskId("node1", 1)),
|
||||
new NoOpClient(threadPool), threadPool, null);
|
||||
int numShards = randomIntBetween(0, 10);
|
||||
List<SearchShard> shards = new ArrayList<>();
|
||||
for (int i = 0; i < numShards; i++) {
|
||||
shards.add(new SearchShard(null, new ShardId("0", "0", 1)));
|
||||
}
|
||||
List<SearchShard> skippedShards = new ArrayList<>();
|
||||
int numSkippedShards = randomIntBetween(0, 10);
|
||||
for (int i = 0; i < numSkippedShards; i++) {
|
||||
skippedShards.add(new SearchShard(null, new ShardId("0", "0", 1)));
|
||||
}
|
||||
|
||||
List<Thread> threads = new ArrayList<>();
|
||||
int numThreads = randomIntBetween(1, 10);
|
||||
CountDownLatch latch = new CountDownLatch(numThreads);
|
||||
for (int i = 0; i < numThreads; i++) {
|
||||
Thread thread = new Thread(() -> task.addCompletionListener(new ActionListener<AsyncSearchResponse>() {
|
||||
@Override
|
||||
public void onResponse(AsyncSearchResponse resp) {
|
||||
assertNull(resp.getSearchResponse());
|
||||
assertNotNull(resp.getFailure());
|
||||
assertTrue(resp.isPartial());
|
||||
latch.countDown();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onFailure(Exception e) {
|
||||
throw new AssertionError(e);
|
||||
}
|
||||
}, TimeValue.timeValueMillis(1)));
|
||||
threads.add(thread);
|
||||
thread.start();
|
||||
}
|
||||
assertFalse(latch.await(numThreads*2, TimeUnit.MILLISECONDS));
|
||||
task.getProgressListener().onFailure(new Exception("boom"));
|
||||
latch.await();
|
||||
}
|
||||
|
||||
public void testWaitForCompletion() throws InterruptedException {
|
||||
AsyncSearchTask task = new AsyncSearchTask(0L, "", "", new TaskId("node1", 0), TimeValue.timeValueHours(1),
|
||||
Collections.emptyMap(), Collections.emptyMap(), new AsyncSearchId("0", new TaskId("node1", 1)),
|
||||
new NoOpClient(threadPool), threadPool, null);
|
||||
int numShards = randomIntBetween(0, 10);
|
||||
List<SearchShard> shards = new ArrayList<>();
|
||||
for (int i = 0; i < numShards; i++) {
|
||||
shards.add(new SearchShard(null, new ShardId("0", "0", 1)));
|
||||
}
|
||||
List<SearchShard> skippedShards = new ArrayList<>();
|
||||
int numSkippedShards = randomIntBetween(0, 10);
|
||||
for (int i = 0; i < numSkippedShards; i++) {
|
||||
skippedShards.add(new SearchShard(null, new ShardId("0", "0", 1)));
|
||||
}
|
||||
|
||||
int numShardFailures = 0;
|
||||
task.getProgressListener().onListShards(shards, skippedShards, SearchResponse.Clusters.EMPTY, false);
|
||||
for (int i = 0; i < numShards; i++) {
|
||||
task.getProgressListener().onPartialReduce(shards.subList(i, i+1),
|
||||
new TotalHits(0, TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO), null, 0);
|
||||
assertCompletionListeners(task, numShards+numSkippedShards, numSkippedShards, numShardFailures, true);
|
||||
}
|
||||
task.getProgressListener().onReduce(shards,
|
||||
new TotalHits(0, TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO), null, 0);
|
||||
assertCompletionListeners(task, numShards+numSkippedShards, numSkippedShards, numShardFailures, true);
|
||||
task.getProgressListener().onResponse(newSearchResponse(numShards+numSkippedShards, numShards, numSkippedShards));
|
||||
assertCompletionListeners(task, numShards+numSkippedShards,
|
||||
numSkippedShards, numShardFailures, false);
|
||||
threadPool.shutdownNow();
|
||||
}
|
||||
|
||||
private SearchResponse newSearchResponse(int totalShards, int successfulShards, int skippedShards) {
|
||||
InternalSearchResponse response = new InternalSearchResponse(SearchHits.empty(),
|
||||
InternalAggregations.EMPTY, null, null, false, null, 1);
|
||||
return new SearchResponse(response, null, totalShards, successfulShards, skippedShards,
|
||||
100, ShardSearchFailure.EMPTY_ARRAY, SearchResponse.Clusters.EMPTY);
|
||||
}
|
||||
|
||||
private void assertCompletionListeners(AsyncSearchTask task,
|
||||
int expectedTotalShards,
|
||||
int expectedSkippedShards,
|
||||
int expectedShardFailures,
|
||||
boolean isPartial) throws InterruptedException {
|
||||
List<Thread> threads = new ArrayList<>();
|
||||
int numThreads = randomIntBetween(1, 10);
|
||||
CountDownLatch latch = new CountDownLatch(numThreads);
|
||||
for (int i = 0; i < numThreads; i++) {
|
||||
Thread thread = new Thread(() -> task.addCompletionListener(new ActionListener<AsyncSearchResponse>() {
|
||||
@Override
|
||||
public void onResponse(AsyncSearchResponse resp) {
|
||||
assertThat(resp.getSearchResponse().getTotalShards(), equalTo(expectedTotalShards));
|
||||
assertThat(resp.getSearchResponse().getSkippedShards(), equalTo(expectedSkippedShards));
|
||||
assertThat(resp.getSearchResponse().getFailedShards(), equalTo(expectedShardFailures));
|
||||
assertThat(resp.isPartial(), equalTo(isPartial));
|
||||
latch.countDown();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onFailure(Exception e) {
|
||||
throw new AssertionError(e);
|
||||
}
|
||||
}, TimeValue.timeValueMillis(1)));
|
||||
threads.add(thread);
|
||||
thread.start();
|
||||
}
|
||||
latch.await();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,24 @@
|
|||
/*
|
||||
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||
* or more contributor license agreements. Licensed under the Elastic License;
|
||||
* you may not use this file except in compliance with the Elastic License.
|
||||
*/
|
||||
package org.elasticsearch.xpack.search;
|
||||
|
||||
import org.elasticsearch.common.io.stream.Writeable;
|
||||
import org.elasticsearch.test.AbstractWireSerializingTestCase;
|
||||
import org.elasticsearch.xpack.core.search.action.DeleteAsyncSearchAction;
|
||||
|
||||
import static org.elasticsearch.xpack.search.GetAsyncSearchRequestTests.randomSearchId;
|
||||
|
||||
public class DeleteAsyncSearchRequestTests extends AbstractWireSerializingTestCase<DeleteAsyncSearchAction.Request> {
|
||||
@Override
|
||||
protected Writeable.Reader<DeleteAsyncSearchAction.Request> instanceReader() {
|
||||
return DeleteAsyncSearchAction.Request::new;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected DeleteAsyncSearchAction.Request createTestInstance() {
|
||||
return new DeleteAsyncSearchAction.Request(randomSearchId());
|
||||
}
|
||||
}
|
|
@ -0,0 +1,41 @@
|
|||
/*
|
||||
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||
* or more contributor license agreements. Licensed under the Elastic License;
|
||||
* you may not use this file except in compliance with the Elastic License.
|
||||
*/
|
||||
package org.elasticsearch.xpack.search;
|
||||
|
||||
import org.elasticsearch.common.UUIDs;
|
||||
import org.elasticsearch.common.io.stream.Writeable;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.tasks.TaskId;
|
||||
import org.elasticsearch.test.AbstractWireSerializingTestCase;
|
||||
import org.elasticsearch.xpack.core.search.action.GetAsyncSearchAction;
|
||||
|
||||
public class GetAsyncSearchRequestTests extends AbstractWireSerializingTestCase<GetAsyncSearchAction.Request> {
|
||||
@Override
|
||||
protected Writeable.Reader<GetAsyncSearchAction.Request> instanceReader() {
|
||||
return GetAsyncSearchAction.Request::new;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected GetAsyncSearchAction.Request createTestInstance() {
|
||||
GetAsyncSearchAction.Request req = new GetAsyncSearchAction.Request(randomSearchId());
|
||||
if (randomBoolean()) {
|
||||
req.setWaitForCompletion(TimeValue.timeValueMillis(randomIntBetween(1, 10000)));
|
||||
}
|
||||
if (randomBoolean()) {
|
||||
req.setKeepAlive(TimeValue.timeValueMillis(randomIntBetween(1, 10000)));
|
||||
}
|
||||
return req;
|
||||
}
|
||||
|
||||
static String randomSearchId() {
|
||||
return AsyncSearchId.encode(UUIDs.randomBase64UUID(),
|
||||
new TaskId(randomAlphaOfLengthBetween(10, 20), randomLongBetween(0, Long.MAX_VALUE)));
|
||||
}
|
||||
|
||||
public void testValidateWaitForCompletion() {
|
||||
|
||||
}
|
||||
}
|
|
@ -0,0 +1,112 @@
|
|||
/*
|
||||
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||
* or more contributor license agreements. Licensed under the Elastic License;
|
||||
* you may not use this file except in compliance with the Elastic License.
|
||||
*/
|
||||
package org.elasticsearch.xpack.search;
|
||||
|
||||
import org.elasticsearch.action.ActionRequestValidationException;
|
||||
import org.elasticsearch.action.search.SearchType;
|
||||
import org.elasticsearch.action.support.IndicesOptions;
|
||||
import org.elasticsearch.common.io.stream.Writeable;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.index.query.QueryBuilders;
|
||||
import org.elasticsearch.search.aggregations.AggregationBuilders;
|
||||
import org.elasticsearch.search.builder.SearchSourceBuilder;
|
||||
import org.elasticsearch.search.suggest.SuggestBuilder;
|
||||
import org.elasticsearch.xpack.core.search.action.SubmitAsyncSearchRequest;
|
||||
import org.elasticsearch.xpack.core.transform.action.AbstractWireSerializingTransformTestCase;
|
||||
|
||||
import static org.hamcrest.Matchers.containsString;
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
|
||||
public class SubmitAsyncSearchRequestTests extends AbstractWireSerializingTransformTestCase<SubmitAsyncSearchRequest> {
|
||||
@Override
|
||||
protected Writeable.Reader<SubmitAsyncSearchRequest> instanceReader() {
|
||||
return SubmitAsyncSearchRequest::new;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected SubmitAsyncSearchRequest createTestInstance() {
|
||||
final SubmitAsyncSearchRequest searchRequest;
|
||||
if (randomBoolean()) {
|
||||
searchRequest = new SubmitAsyncSearchRequest(generateRandomStringArray(10, 10, false, false));
|
||||
} else {
|
||||
searchRequest = new SubmitAsyncSearchRequest();
|
||||
}
|
||||
if (randomBoolean()) {
|
||||
searchRequest.setWaitForCompletion(TimeValue.parseTimeValue(randomPositiveTimeValue(), "wait_for_completion"));
|
||||
}
|
||||
searchRequest.setCleanOnCompletion(randomBoolean());
|
||||
if (randomBoolean()) {
|
||||
searchRequest.setKeepAlive(TimeValue.parseTimeValue(randomPositiveTimeValue(), "keep_alive"));
|
||||
}
|
||||
if (randomBoolean()) {
|
||||
searchRequest.getSearchRequest()
|
||||
.indicesOptions(IndicesOptions.fromOptions(randomBoolean(), randomBoolean(), randomBoolean(), randomBoolean()));
|
||||
}
|
||||
if (randomBoolean()) {
|
||||
searchRequest.getSearchRequest()
|
||||
.preference(randomAlphaOfLengthBetween(3, 10));
|
||||
}
|
||||
if (randomBoolean()) {
|
||||
searchRequest.getSearchRequest().requestCache(randomBoolean());
|
||||
}
|
||||
if (randomBoolean()) {
|
||||
searchRequest.getSearchRequest().searchType(randomFrom(SearchType.DFS_QUERY_THEN_FETCH, SearchType.QUERY_THEN_FETCH));
|
||||
}
|
||||
if (randomBoolean()) {
|
||||
searchRequest.getSearchRequest().source(randomSearchSourceBuilder());
|
||||
}
|
||||
return searchRequest;
|
||||
}
|
||||
|
||||
protected SearchSourceBuilder randomSearchSourceBuilder() {
|
||||
SearchSourceBuilder source = new SearchSourceBuilder();
|
||||
if (randomBoolean()) {
|
||||
source.query(QueryBuilders.termQuery("foo", "bar"));
|
||||
}
|
||||
if (randomBoolean()) {
|
||||
source.aggregation(AggregationBuilders.max("max").field("field"));
|
||||
}
|
||||
return source;
|
||||
}
|
||||
|
||||
public void testValidateCssMinimizeRoundtrips() {
|
||||
SubmitAsyncSearchRequest req = new SubmitAsyncSearchRequest();
|
||||
req.getSearchRequest().setCcsMinimizeRoundtrips(true);
|
||||
ActionRequestValidationException exc = req.validate();
|
||||
assertNotNull(exc);
|
||||
assertThat(exc.validationErrors().size(), equalTo(1));
|
||||
assertThat(exc.validationErrors().get(0), containsString("[ccs_minimize_roundtrips]"));
|
||||
}
|
||||
|
||||
public void testValidateScroll() {
|
||||
SubmitAsyncSearchRequest req = new SubmitAsyncSearchRequest();
|
||||
req.getSearchRequest().scroll(TimeValue.timeValueMinutes(5));
|
||||
ActionRequestValidationException exc = req.validate();
|
||||
assertNotNull(exc);
|
||||
assertThat(exc.validationErrors().size(), equalTo(2));
|
||||
// request_cache is activated by default
|
||||
assertThat(exc.validationErrors().get(0), containsString("[request_cache]"));
|
||||
assertThat(exc.validationErrors().get(1), containsString("[scroll]"));
|
||||
}
|
||||
|
||||
public void testValidateKeepAlive() {
|
||||
SubmitAsyncSearchRequest req = new SubmitAsyncSearchRequest();
|
||||
req.setKeepAlive(TimeValue.timeValueSeconds(randomIntBetween(1, 59)));
|
||||
ActionRequestValidationException exc = req.validate();
|
||||
assertNotNull(exc);
|
||||
assertThat(exc.validationErrors().size(), equalTo(1));
|
||||
assertThat(exc.validationErrors().get(0), containsString("[keep_alive]"));
|
||||
}
|
||||
|
||||
public void testValidateSuggestOnly() {
|
||||
SubmitAsyncSearchRequest req = new SubmitAsyncSearchRequest();
|
||||
req.getSearchRequest().source(new SearchSourceBuilder().suggest(new SuggestBuilder()));
|
||||
ActionRequestValidationException exc = req.validate();
|
||||
assertNotNull(exc);
|
||||
assertThat(exc.validationErrors().size(), equalTo(1));
|
||||
assertThat(exc.validationErrors().get(0), containsString("suggest"));
|
||||
}
|
||||
}
|
|
@ -39,6 +39,9 @@ import org.elasticsearch.transport.Transport;
|
|||
import org.elasticsearch.xpack.core.action.XPackInfoAction;
|
||||
import org.elasticsearch.xpack.core.action.XPackUsageAction;
|
||||
import org.elasticsearch.xpack.core.analytics.AnalyticsFeatureSetUsage;
|
||||
import org.elasticsearch.xpack.core.search.action.DeleteAsyncSearchAction;
|
||||
import org.elasticsearch.xpack.core.search.action.GetAsyncSearchAction;
|
||||
import org.elasticsearch.xpack.core.search.action.SubmitAsyncSearchAction;
|
||||
import org.elasticsearch.xpack.core.ccr.AutoFollowMetadata;
|
||||
import org.elasticsearch.xpack.core.ccr.CCRFeatureSet;
|
||||
import org.elasticsearch.xpack.core.deprecation.DeprecationInfoAction;
|
||||
|
@ -465,7 +468,11 @@ public class XPackClientPlugin extends Plugin implements ActionPlugin, NetworkPl
|
|||
DeleteEnrichPolicyAction.INSTANCE,
|
||||
ExecuteEnrichPolicyAction.INSTANCE,
|
||||
GetEnrichPolicyAction.INSTANCE,
|
||||
PutEnrichPolicyAction.INSTANCE
|
||||
PutEnrichPolicyAction.INSTANCE,
|
||||
// Async Search
|
||||
SubmitAsyncSearchAction.INSTANCE,
|
||||
GetAsyncSearchAction.INSTANCE,
|
||||
DeleteAsyncSearchAction.INSTANCE
|
||||
);
|
||||
}
|
||||
|
||||
|
|
|
@ -0,0 +1,208 @@
|
|||
/*
|
||||
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||
* or more contributor license agreements. Licensed under the Elastic License;
|
||||
* you may not use this file except in compliance with the Elastic License.
|
||||
*/
|
||||
package org.elasticsearch.xpack.core.search.action;
|
||||
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.action.ActionResponse;
|
||||
import org.elasticsearch.action.search.SearchResponse;
|
||||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.xcontent.StatusToXContentObject;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.rest.RestStatus;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import static org.elasticsearch.rest.RestStatus.OK;
|
||||
|
||||
/**
|
||||
* A response of an async search request.
|
||||
*/
|
||||
public class AsyncSearchResponse extends ActionResponse implements StatusToXContentObject {
|
||||
@Nullable
|
||||
private final String id;
|
||||
private final int version;
|
||||
private final SearchResponse searchResponse;
|
||||
private final ElasticsearchException error;
|
||||
private final boolean isRunning;
|
||||
private final boolean isPartial;
|
||||
|
||||
private final long startTimeMillis;
|
||||
private final long expirationTimeMillis;
|
||||
|
||||
/**
|
||||
* Creates an {@link AsyncSearchResponse} with meta-information only (not-modified).
|
||||
*/
|
||||
public AsyncSearchResponse(String id,
|
||||
int version,
|
||||
boolean isPartial,
|
||||
boolean isRunning,
|
||||
long startTimeMillis,
|
||||
long expirationTimeMillis) {
|
||||
this(id, version, null, null, isPartial, isRunning, startTimeMillis, expirationTimeMillis);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a new {@link AsyncSearchResponse}
|
||||
*
|
||||
* @param id The id of the search for further retrieval, <code>null</code> if not stored.
|
||||
* @param version The version number of this response.
|
||||
* @param searchResponse The actual search response.
|
||||
* @param error The error if the search failed, <code>null</code> if the search is running
|
||||
* or has completed without failure.
|
||||
* @param isPartial Whether the <code>searchResponse</code> contains partial results.
|
||||
* @param isRunning Whether the search is running in the cluster.
|
||||
* @param startTimeMillis The start date of the search in milliseconds since epoch.
|
||||
*/
|
||||
public AsyncSearchResponse(String id,
|
||||
int version,
|
||||
SearchResponse searchResponse,
|
||||
ElasticsearchException error,
|
||||
boolean isPartial,
|
||||
boolean isRunning,
|
||||
long startTimeMillis,
|
||||
long expirationTimeMillis) {
|
||||
this.id = id;
|
||||
this.version = version;
|
||||
this.error = error;
|
||||
this.searchResponse = searchResponse;
|
||||
this.isPartial = isPartial;
|
||||
this.isRunning = isRunning;
|
||||
this.startTimeMillis = startTimeMillis;
|
||||
this.expirationTimeMillis = expirationTimeMillis;
|
||||
}
|
||||
|
||||
public AsyncSearchResponse(StreamInput in) throws IOException {
|
||||
this.id = in.readOptionalString();
|
||||
this.version = in.readVInt();
|
||||
this.error = in.readOptionalWriteable(ElasticsearchException::new);
|
||||
this.searchResponse = in.readOptionalWriteable(SearchResponse::new);
|
||||
this.isPartial = in.readBoolean();
|
||||
this.isRunning = in.readBoolean();
|
||||
this.startTimeMillis = in.readLong();
|
||||
this.expirationTimeMillis = in.readLong();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
out.writeOptionalString(id);
|
||||
out.writeVInt(version);
|
||||
out.writeOptionalWriteable(error);
|
||||
out.writeOptionalWriteable(searchResponse);
|
||||
out.writeBoolean(isPartial);
|
||||
out.writeBoolean(isRunning);
|
||||
out.writeLong(startTimeMillis);
|
||||
out.writeLong(expirationTimeMillis);
|
||||
}
|
||||
|
||||
public AsyncSearchResponse clone(String id) {
|
||||
return new AsyncSearchResponse(id, version, searchResponse, error, isPartial, false, startTimeMillis, expirationTimeMillis);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the id of the async search request or null if the response is not stored in the cluster.
|
||||
*/
|
||||
@Nullable
|
||||
public String getId() {
|
||||
return id;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the version of this response.
|
||||
*/
|
||||
public int getVersion() {
|
||||
return version;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the current {@link SearchResponse} or <code>null</code> if not available.
|
||||
*
|
||||
* See {@link #isPartial()} to determine whether the response contains partial or complete
|
||||
* results.
|
||||
*/
|
||||
public SearchResponse getSearchResponse() {
|
||||
return searchResponse;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the failure reason or null if the query is running or has completed normally.
|
||||
*/
|
||||
public ElasticsearchException getFailure() {
|
||||
return error;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns <code>true</code> if the {@link SearchResponse} contains partial
|
||||
* results computed from a subset of the total shards.
|
||||
*/
|
||||
public boolean isPartial() {
|
||||
return isPartial;
|
||||
}
|
||||
|
||||
/**
|
||||
* Whether the search is still running in the cluster.
|
||||
*
|
||||
* A value of <code>false</code> indicates that the response is final
|
||||
* even if {@link #isPartial()} returns <code>true</code>. In such case,
|
||||
* the partial response represents the status of the search before a
|
||||
* non-recoverable failure.
|
||||
*/
|
||||
public boolean isRunning() {
|
||||
return isRunning;
|
||||
}
|
||||
|
||||
/**
|
||||
* When this response was created as a timestamp in milliseconds since epoch.
|
||||
*/
|
||||
public long getStartTime() {
|
||||
return startTimeMillis;
|
||||
}
|
||||
|
||||
/**
|
||||
* When this response will expired as a timestamp in milliseconds since epoch.
|
||||
*/
|
||||
public long getExpirationTime() {
|
||||
return expirationTimeMillis;
|
||||
}
|
||||
|
||||
@Override
|
||||
public RestStatus status() {
|
||||
if (searchResponse == null || isPartial) {
|
||||
// shard failures are not considered fatal for partial results so
|
||||
// we return OK until we get the final response even if we don't have
|
||||
// a single successful shard.
|
||||
return error != null ? error.status() : OK;
|
||||
} else {
|
||||
return searchResponse.status();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
|
||||
builder.startObject();
|
||||
if (id != null) {
|
||||
builder.field("id", id);
|
||||
}
|
||||
builder.field("version", version);
|
||||
builder.field("is_partial", isPartial);
|
||||
builder.field("is_running", isRunning);
|
||||
builder.field("start_time_in_millis", startTimeMillis);
|
||||
builder.field("expiration_time_in_millis", expirationTimeMillis);
|
||||
|
||||
if (searchResponse != null) {
|
||||
builder.field("response");
|
||||
searchResponse.toXContent(builder, params);
|
||||
}
|
||||
if (error != null) {
|
||||
builder.startObject("error");
|
||||
error.toXContent(builder, params);
|
||||
builder.endObject();
|
||||
}
|
||||
builder.endObject();
|
||||
return builder;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,72 @@
|
|||
/*
|
||||
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||
* or more contributor license agreements. Licensed under the Elastic License;
|
||||
* you may not use this file except in compliance with the Elastic License.
|
||||
*/
|
||||
package org.elasticsearch.xpack.core.search.action;
|
||||
|
||||
import org.elasticsearch.action.ActionRequest;
|
||||
import org.elasticsearch.action.ActionRequestValidationException;
|
||||
import org.elasticsearch.action.ActionType;
|
||||
import org.elasticsearch.action.support.master.AcknowledgedResponse;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.io.stream.Writeable;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Objects;
|
||||
|
||||
public class DeleteAsyncSearchAction extends ActionType<AcknowledgedResponse> {
|
||||
public static final DeleteAsyncSearchAction INSTANCE = new DeleteAsyncSearchAction();
|
||||
public static final String NAME = "indices:data/read/async_search/delete";
|
||||
|
||||
private DeleteAsyncSearchAction() {
|
||||
super(NAME, AcknowledgedResponse::new);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Writeable.Reader<AcknowledgedResponse> getResponseReader() {
|
||||
return AcknowledgedResponse::new;
|
||||
}
|
||||
|
||||
public static class Request extends ActionRequest {
|
||||
private final String id;
|
||||
|
||||
public Request(String id) {
|
||||
this.id = id;
|
||||
}
|
||||
|
||||
public Request(StreamInput in) throws IOException {
|
||||
super(in);
|
||||
this.id = in.readString();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
super.writeTo(out);
|
||||
out.writeString(id);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ActionRequestValidationException validate() {
|
||||
return null;
|
||||
}
|
||||
|
||||
public String getId() {
|
||||
return id;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (this == o) return true;
|
||||
if (o == null || getClass() != o.getClass()) return false;
|
||||
Request request = (Request) o;
|
||||
return id.equals(request.id);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hash(id);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,120 @@
|
|||
/*
|
||||
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||
* or more contributor license agreements. Licensed under the Elastic License;
|
||||
* you may not use this file except in compliance with the Elastic License.
|
||||
*/
|
||||
package org.elasticsearch.xpack.core.search.action;
|
||||
|
||||
import org.elasticsearch.action.ActionRequest;
|
||||
import org.elasticsearch.action.ActionRequestValidationException;
|
||||
import org.elasticsearch.action.ActionType;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.io.stream.Writeable;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Objects;
|
||||
|
||||
import static org.elasticsearch.action.ValidateActions.addValidationError;
|
||||
import static org.elasticsearch.xpack.core.search.action.SubmitAsyncSearchRequest.MIN_KEEP_ALIVE;
|
||||
|
||||
public class GetAsyncSearchAction extends ActionType<AsyncSearchResponse> {
|
||||
public static final GetAsyncSearchAction INSTANCE = new GetAsyncSearchAction();
|
||||
public static final String NAME = "indices:data/read/async_search/get";
|
||||
|
||||
private GetAsyncSearchAction() {
|
||||
super(NAME, AsyncSearchResponse::new);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Writeable.Reader<AsyncSearchResponse> getResponseReader() {
|
||||
return AsyncSearchResponse::new;
|
||||
}
|
||||
|
||||
public static class Request extends ActionRequest {
|
||||
private final String id;
|
||||
private TimeValue waitForCompletion = TimeValue.MINUS_ONE;
|
||||
private TimeValue keepAlive = TimeValue.MINUS_ONE;
|
||||
|
||||
/**
|
||||
* Creates a new request
|
||||
*
|
||||
* @param id The id of the search progress request.
|
||||
*/
|
||||
public Request(String id) {
|
||||
this.id = id;
|
||||
}
|
||||
|
||||
public Request(StreamInput in) throws IOException {
|
||||
super(in);
|
||||
this.id = in.readString();
|
||||
this.waitForCompletion = TimeValue.timeValueMillis(in.readLong());
|
||||
this.keepAlive = in.readTimeValue();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
super.writeTo(out);
|
||||
out.writeString(id);
|
||||
out.writeLong(waitForCompletion.millis());
|
||||
out.writeTimeValue(keepAlive);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ActionRequestValidationException validate() {
|
||||
ActionRequestValidationException validationException = null;
|
||||
if (keepAlive.getMillis() != -1 && keepAlive.getMillis() < MIN_KEEP_ALIVE) {
|
||||
validationException =
|
||||
addValidationError("keep_alive must be greater than 1 minute, got:" + keepAlive.toString(), validationException);
|
||||
}
|
||||
return validationException;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the id of the async search.
|
||||
*/
|
||||
public String getId() {
|
||||
return id;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the minimum time that the request should wait before returning a partial result (defaults to no wait).
|
||||
*/
|
||||
public Request setWaitForCompletion(TimeValue timeValue) {
|
||||
this.waitForCompletion = timeValue;
|
||||
return this;
|
||||
}
|
||||
|
||||
public TimeValue getWaitForCompletion() {
|
||||
return waitForCompletion;
|
||||
}
|
||||
|
||||
/**
|
||||
* Extends the amount of time after which the result will expire (defaults to no extension).
|
||||
*/
|
||||
public Request setKeepAlive(TimeValue timeValue) {
|
||||
this.keepAlive = timeValue;
|
||||
return this;
|
||||
}
|
||||
|
||||
public TimeValue getKeepAlive() {
|
||||
return keepAlive;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (this == o) return true;
|
||||
if (o == null || getClass() != o.getClass()) return false;
|
||||
Request request = (Request) o;
|
||||
return Objects.equals(id, request.id) &&
|
||||
waitForCompletion.equals(request.waitForCompletion) &&
|
||||
keepAlive.equals(request.keepAlive);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hash(id, waitForCompletion, keepAlive);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,17 @@
|
|||
/*
|
||||
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||
* or more contributor license agreements. Licensed under the Elastic License;
|
||||
* you may not use this file except in compliance with the Elastic License.
|
||||
*/
|
||||
package org.elasticsearch.xpack.core.search.action;
|
||||
|
||||
import org.elasticsearch.action.ActionType;
|
||||
|
||||
public final class SubmitAsyncSearchAction extends ActionType<AsyncSearchResponse> {
|
||||
public static final SubmitAsyncSearchAction INSTANCE = new SubmitAsyncSearchAction();
|
||||
public static final String NAME = "indices:data/read/async_search/submit";
|
||||
|
||||
private SubmitAsyncSearchAction() {
|
||||
super(NAME, AsyncSearchResponse::new);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,183 @@
|
|||
/*
|
||||
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||
* or more contributor license agreements. Licensed under the Elastic License;
|
||||
* you may not use this file except in compliance with the Elastic License.
|
||||
*/
|
||||
package org.elasticsearch.xpack.core.search.action;
|
||||
|
||||
import org.elasticsearch.action.ActionRequest;
|
||||
import org.elasticsearch.action.ActionRequestValidationException;
|
||||
import org.elasticsearch.action.search.SearchRequest;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.search.builder.SearchSourceBuilder;
|
||||
import org.elasticsearch.tasks.CancellableTask;
|
||||
import org.elasticsearch.tasks.Task;
|
||||
import org.elasticsearch.tasks.TaskId;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
|
||||
import static org.elasticsearch.action.ValidateActions.addValidationError;
|
||||
|
||||
/**
|
||||
* A request to track asynchronously the progress of a search against one or more indices.
|
||||
*
|
||||
* @see AsyncSearchResponse
|
||||
*/
|
||||
public class SubmitAsyncSearchRequest extends ActionRequest {
|
||||
public static long MIN_KEEP_ALIVE = TimeValue.timeValueMinutes(1).millis();
|
||||
|
||||
private TimeValue waitForCompletion = TimeValue.timeValueSeconds(1);
|
||||
private boolean cleanOnCompletion = true;
|
||||
private TimeValue keepAlive = TimeValue.timeValueDays(5);
|
||||
|
||||
private final SearchRequest request;
|
||||
|
||||
/**
|
||||
* Creates a new request
|
||||
*/
|
||||
public SubmitAsyncSearchRequest(String... indices) {
|
||||
this(new SearchSourceBuilder(), indices);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a new request
|
||||
*/
|
||||
public SubmitAsyncSearchRequest(SearchSourceBuilder source, String... indices) {
|
||||
this.request = new SearchRequest(indices, source);
|
||||
request.setCcsMinimizeRoundtrips(false);
|
||||
request.setPreFilterShardSize(1);
|
||||
request.setBatchedReduceSize(5);
|
||||
request.requestCache(true);
|
||||
}
|
||||
|
||||
public SubmitAsyncSearchRequest(StreamInput in) throws IOException {
|
||||
this.request = new SearchRequest(in);
|
||||
this.waitForCompletion = in.readTimeValue();
|
||||
this.keepAlive = in.readTimeValue();
|
||||
this.cleanOnCompletion = in.readBoolean();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
request.writeTo(out);
|
||||
out.writeTimeValue(waitForCompletion);
|
||||
out.writeTimeValue(keepAlive);
|
||||
out.writeBoolean(cleanOnCompletion);
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the number of shard results that should be returned to notify search progress (default to 5).
|
||||
*/
|
||||
public SubmitAsyncSearchRequest setBatchedReduceSize(int size) {
|
||||
request.setBatchedReduceSize(size);
|
||||
return this;
|
||||
}
|
||||
|
||||
public int getBatchReduceSize() {
|
||||
return request.getBatchedReduceSize();
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the minimum time that the request should wait before returning a partial result (defaults to 1 second).
|
||||
*/
|
||||
public SubmitAsyncSearchRequest setWaitForCompletion(TimeValue waitForCompletion) {
|
||||
this.waitForCompletion = waitForCompletion;
|
||||
return this;
|
||||
}
|
||||
|
||||
public TimeValue getWaitForCompletion() {
|
||||
return waitForCompletion;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the amount of time after which the result will expire (defaults to 5 days).
|
||||
*/
|
||||
public SubmitAsyncSearchRequest setKeepAlive(TimeValue keepAlive) {
|
||||
this.keepAlive = keepAlive;
|
||||
return this;
|
||||
}
|
||||
|
||||
public TimeValue getKeepAlive() {
|
||||
return keepAlive;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the underlying {@link SearchRequest}.
|
||||
*/
|
||||
public SearchRequest getSearchRequest() {
|
||||
return request;
|
||||
}
|
||||
|
||||
/**
|
||||
* Should the resource be removed on completion or failure (defaults to true).
|
||||
*/
|
||||
public SubmitAsyncSearchRequest setCleanOnCompletion(boolean value) {
|
||||
this.cleanOnCompletion = value;
|
||||
return this;
|
||||
}
|
||||
|
||||
public boolean isCleanOnCompletion() {
|
||||
return cleanOnCompletion;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ActionRequestValidationException validate() {
|
||||
ActionRequestValidationException validationException = request.validate();
|
||||
if (request.scroll() != null) {
|
||||
addValidationError("[scroll] queries are not supported", validationException);
|
||||
}
|
||||
if (request.isSuggestOnly()) {
|
||||
validationException = addValidationError("suggest-only queries are not supported", validationException);
|
||||
}
|
||||
if (keepAlive.getMillis() < MIN_KEEP_ALIVE) {
|
||||
validationException =
|
||||
addValidationError("[keep_alive] must be greater than 1 minute, got:" + keepAlive.toString(), validationException);
|
||||
}
|
||||
if (request.isCcsMinimizeRoundtrips()) {
|
||||
validationException =
|
||||
addValidationError("[ccs_minimize_roundtrips] is not supported on async search queries", validationException);
|
||||
}
|
||||
|
||||
return validationException;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Task createTask(long id, String type, String action, TaskId parentTaskId, Map<String, String> headers) {
|
||||
return new CancellableTask(id, type, action, toString(), parentTaskId, headers) {
|
||||
@Override
|
||||
public boolean shouldCancelChildrenOnCancellation() {
|
||||
return true;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (this == o) return true;
|
||||
if (o == null || getClass() != o.getClass()) return false;
|
||||
SubmitAsyncSearchRequest request1 = (SubmitAsyncSearchRequest) o;
|
||||
return cleanOnCompletion == request1.cleanOnCompletion &&
|
||||
waitForCompletion.equals(request1.waitForCompletion) &&
|
||||
keepAlive.equals(request1.keepAlive) &&
|
||||
request.equals(request1.request);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hash(waitForCompletion, cleanOnCompletion, keepAlive, request);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "SubmitAsyncSearchRequest{" +
|
||||
"waitForCompletion=" + waitForCompletion +
|
||||
", cleanOnCompletion=" + cleanOnCompletion +
|
||||
", keepAlive=" + keepAlive +
|
||||
", request=" + request +
|
||||
'}';
|
||||
}
|
||||
}
|
|
@ -23,7 +23,7 @@ public final class RestrictedIndicesNames {
|
|||
public static final String SECURITY_TOKENS_ALIAS = ".security-tokens";
|
||||
|
||||
// public for tests
|
||||
public static final String ASYNC_SEARCH_PREFIX = ".async-search-";
|
||||
public static final String ASYNC_SEARCH_PREFIX = ".async-search";
|
||||
private static final Automaton ASYNC_SEARCH_AUTOMATON = Automatons.patterns(ASYNC_SEARCH_PREFIX + "*");
|
||||
|
||||
// public for tests
|
||||
|
|
|
@ -32,6 +32,9 @@ import org.elasticsearch.common.settings.Settings;
|
|||
import org.elasticsearch.common.util.set.Sets;
|
||||
import org.elasticsearch.transport.TransportActionProxy;
|
||||
import org.elasticsearch.transport.TransportRequest;
|
||||
import org.elasticsearch.xpack.core.search.action.DeleteAsyncSearchAction;
|
||||
import org.elasticsearch.xpack.core.search.action.GetAsyncSearchAction;
|
||||
import org.elasticsearch.xpack.core.search.action.SubmitAsyncSearchAction;
|
||||
import org.elasticsearch.xpack.core.security.action.GetApiKeyAction;
|
||||
import org.elasticsearch.xpack.core.security.action.GetApiKeyRequest;
|
||||
import org.elasticsearch.xpack.core.security.action.user.AuthenticateAction;
|
||||
|
@ -239,17 +242,18 @@ public class RBACEngine implements AuthorizationEngine {
|
|||
// need to validate that the action is allowed and then move on
|
||||
authorizeIndexActionName(action, authorizationInfo, null, listener);
|
||||
} else if (request instanceof IndicesRequest == false && request instanceof IndicesAliasesRequest == false) {
|
||||
// scroll is special
|
||||
// some APIs are indices requests that are not actually associated with indices. For example,
|
||||
// search scroll request, is categorized under the indices context, but doesn't hold indices names
|
||||
// (in this case, the security check on the indices was done on the search request that initialized
|
||||
// the scroll. Given that scroll is implemented using a context on the node holding the shard, we
|
||||
// piggyback on it and enhance the context with the original authentication. This serves as our method
|
||||
// to validate the scroll id only stays with the same user!
|
||||
// note that clear scroll shard level actions can originate from a clear scroll all, which doesn't require any
|
||||
// indices permission as it's categorized under cluster. This is why the scroll check is performed
|
||||
// even before checking if the user has any indices permission.
|
||||
if (isScrollRelatedAction(action)) {
|
||||
// scroll is special
|
||||
// some APIs are indices requests that are not actually associated with indices. For example,
|
||||
// search scroll request, is categorized under the indices context, but doesn't hold indices names
|
||||
// (in this case, the security check on the indices was done on the search request that initialized
|
||||
// the scroll. Given that scroll is implemented using a context on the node holding the shard, we
|
||||
// piggyback on it and enhance the context with the original authentication. This serves as our method
|
||||
// to validate the scroll id only stays with the same user!
|
||||
// note that clear scroll shard level actions can originate from a clear scroll all, which doesn't require any
|
||||
// indices permission as it's categorized under cluster. This is why the scroll check is performed
|
||||
// even before checking if the user has any indices permission.
|
||||
|
||||
// if the action is a search scroll action, we first authorize that the user can execute the action for some
|
||||
// index and if they cannot, we can fail the request early before we allow the execution of the action and in
|
||||
// turn the shard actions
|
||||
|
@ -261,11 +265,22 @@ public class RBACEngine implements AuthorizationEngine {
|
|||
// information such as the index and the incoming address of the request
|
||||
listener.onResponse(new IndexAuthorizationResult(true, IndicesAccessControl.ALLOW_NO_INDICES));
|
||||
}
|
||||
} else if (isAsyncSearchRelatedAction(action)) {
|
||||
if (SubmitAsyncSearchAction.NAME.equals(action)) {
|
||||
// we check if the user has any indices permission when submitting an async-search request in order to be
|
||||
// able to fail the request early. Fine grained index-level permissions are handled by the search action
|
||||
// that is triggered internally by the submit API.
|
||||
authorizeIndexActionName(action, authorizationInfo, null, listener);
|
||||
} else {
|
||||
// async-search actions other than submit have a custom security layer that checks if the current user is
|
||||
// the same as the user that submitted the original request so we can skip security here.
|
||||
listener.onResponse(new IndexAuthorizationResult(true, IndicesAccessControl.ALLOW_NO_INDICES));
|
||||
}
|
||||
} else {
|
||||
assert false :
|
||||
"only scroll related requests are known indices api that don't support retrieving the indices they relate to";
|
||||
listener.onFailure(new IllegalStateException("only scroll related requests are known indices api that don't support " +
|
||||
"retrieving the indices they relate to"));
|
||||
assert false : "only scroll and async-search related requests are known indices api that don't " +
|
||||
"support retrieving the indices they relate to";
|
||||
listener.onFailure(new IllegalStateException("only scroll and async-search related requests are known indices " +
|
||||
"api that don't support retrieving the indices they relate to"));
|
||||
}
|
||||
} else if (request instanceof IndicesRequest &&
|
||||
IndicesAndAliasesResolver.allowsRemoteIndices((IndicesRequest) request)) {
|
||||
|
@ -567,4 +582,10 @@ public class RBACEngine implements AuthorizationEngine {
|
|||
action.equals("indices:data/read/sql/close_cursor") ||
|
||||
action.equals(SearchTransportService.CLEAR_SCROLL_CONTEXTS_ACTION_NAME);
|
||||
}
|
||||
|
||||
private static boolean isAsyncSearchRelatedAction(String action) {
|
||||
return action.equals(SubmitAsyncSearchAction.NAME) ||
|
||||
action.equals(GetAsyncSearchAction.NAME) ||
|
||||
action.equals(DeleteAsyncSearchAction.NAME);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,24 @@
|
|||
{
|
||||
"async_search.delete":{
|
||||
"documentation":{
|
||||
"url":"https://www.elastic.co/guide/en/elasticsearch/reference/current/async-search.html"
|
||||
},
|
||||
"stability":"experimental",
|
||||
"url":{
|
||||
"paths":[
|
||||
{
|
||||
"path":"/_async_search/{id}",
|
||||
"methods":[
|
||||
"DELETE"
|
||||
],
|
||||
"parts":{
|
||||
"id":{
|
||||
"type":"string",
|
||||
"description":"The async search ID"
|
||||
}
|
||||
}
|
||||
}
|
||||
]
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,40 @@
|
|||
{
|
||||
"async_search.get":{
|
||||
"documentation":{
|
||||
"url":"https://www.elastic.co/guide/en/elasticsearch/reference/current/async-search.html"
|
||||
},
|
||||
"stability":"experimental",
|
||||
"url":{
|
||||
"paths":[
|
||||
{
|
||||
"path":"/_async_search/{id}",
|
||||
"methods":[
|
||||
"GET"
|
||||
],
|
||||
"parts":{
|
||||
"id":{
|
||||
"type":"string",
|
||||
"description":"The async search ID"
|
||||
}
|
||||
}
|
||||
}
|
||||
]
|
||||
},
|
||||
"params":{
|
||||
"wait_for_completion":{
|
||||
"type":"time",
|
||||
"description":"Specify the time that the request should block waiting for the final response",
|
||||
"default": "1s"
|
||||
},
|
||||
"keep_alive": {
|
||||
"type": "time",
|
||||
"description": "Specify the time interval in which the results (partial or final) for this search will be available",
|
||||
"default": "5d"
|
||||
},
|
||||
"typed_keys":{
|
||||
"type":"boolean",
|
||||
"description":"Specify whether aggregation and suggester names should be prefixed by their respective types in the response"
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,227 @@
|
|||
{
|
||||
"async_search.submit":{
|
||||
"documentation":{
|
||||
"url":"https://www.elastic.co/guide/en/elasticsearch/reference/current/async-search.html"
|
||||
},
|
||||
"stability":"experimental",
|
||||
"url":{
|
||||
"paths":[
|
||||
{
|
||||
"path":"/_async_search",
|
||||
"methods":[
|
||||
"GET",
|
||||
"POST"
|
||||
]
|
||||
},
|
||||
{
|
||||
"path":"/{index}/_async_search",
|
||||
"methods":[
|
||||
"GET",
|
||||
"POST"
|
||||
],
|
||||
"parts":{
|
||||
"index":{
|
||||
"type":"list",
|
||||
"description":"A comma-separated list of index names to search; use `_all` or empty string to perform the operation on all indices"
|
||||
}
|
||||
}
|
||||
}
|
||||
]
|
||||
},
|
||||
"params":{
|
||||
"wait_for_completion":{
|
||||
"type":"time",
|
||||
"description":"Specify the time that the request should block waiting for the final response",
|
||||
"default": "1s"
|
||||
},
|
||||
"keep_alive": {
|
||||
"type": "time",
|
||||
"description": "Update the time interval in which the results (partial or final) for this search will be available"
|
||||
},
|
||||
"batched_reduce_size":{
|
||||
"type":"number",
|
||||
"description":"The number of shard results that should be reduced at once on the coordinating node. This value should be used as the granularity at which progress results will be made available.",
|
||||
"default":5
|
||||
},
|
||||
"analyzer":{
|
||||
"type":"string",
|
||||
"description":"The analyzer to use for the query string"
|
||||
},
|
||||
"analyze_wildcard":{
|
||||
"type":"boolean",
|
||||
"description":"Specify whether wildcard and prefix queries should be analyzed (default: false)"
|
||||
},
|
||||
"default_operator":{
|
||||
"type":"enum",
|
||||
"options":[
|
||||
"AND",
|
||||
"OR"
|
||||
],
|
||||
"default":"OR",
|
||||
"description":"The default operator for query string query (AND or OR)"
|
||||
},
|
||||
"df":{
|
||||
"type":"string",
|
||||
"description":"The field to use as default where no field prefix is given in the query string"
|
||||
},
|
||||
"explain":{
|
||||
"type":"boolean",
|
||||
"description":"Specify whether to return detailed information about score computation as part of a hit"
|
||||
},
|
||||
"stored_fields":{
|
||||
"type":"list",
|
||||
"description":"A comma-separated list of stored fields to return as part of a hit"
|
||||
},
|
||||
"docvalue_fields":{
|
||||
"type":"list",
|
||||
"description":"A comma-separated list of fields to return as the docvalue representation of a field for each hit"
|
||||
},
|
||||
"from":{
|
||||
"type":"number",
|
||||
"description":"Starting offset (default: 0)"
|
||||
},
|
||||
"ignore_unavailable":{
|
||||
"type":"boolean",
|
||||
"description":"Whether specified concrete indices should be ignored when unavailable (missing or closed)"
|
||||
},
|
||||
"ignore_throttled":{
|
||||
"type":"boolean",
|
||||
"description":"Whether specified concrete, expanded or aliased indices should be ignored when throttled"
|
||||
},
|
||||
"allow_no_indices":{
|
||||
"type":"boolean",
|
||||
"description":"Whether to ignore if a wildcard indices expression resolves into no concrete indices. (This includes `_all` string or when no indices have been specified)"
|
||||
},
|
||||
"expand_wildcards":{
|
||||
"type":"enum",
|
||||
"options":[
|
||||
"open",
|
||||
"closed",
|
||||
"none",
|
||||
"all"
|
||||
],
|
||||
"default":"open",
|
||||
"description":"Whether to expand wildcard expression to concrete indices that are open, closed or both."
|
||||
},
|
||||
"lenient":{
|
||||
"type":"boolean",
|
||||
"description":"Specify whether format-based query failures (such as providing text to a numeric field) should be ignored"
|
||||
},
|
||||
"preference":{
|
||||
"type":"string",
|
||||
"description":"Specify the node or shard the operation should be performed on (default: random)"
|
||||
},
|
||||
"q":{
|
||||
"type":"string",
|
||||
"description":"Query in the Lucene query string syntax"
|
||||
},
|
||||
"routing":{
|
||||
"type":"list",
|
||||
"description":"A comma-separated list of specific routing values"
|
||||
},
|
||||
"search_type":{
|
||||
"type":"enum",
|
||||
"options":[
|
||||
"query_then_fetch",
|
||||
"dfs_query_then_fetch"
|
||||
],
|
||||
"description":"Search operation type"
|
||||
},
|
||||
"size":{
|
||||
"type":"number",
|
||||
"description":"Number of hits to return (default: 10)"
|
||||
},
|
||||
"sort":{
|
||||
"type":"list",
|
||||
"description":"A comma-separated list of <field>:<direction> pairs"
|
||||
},
|
||||
"_source":{
|
||||
"type":"list",
|
||||
"description":"True or false to return the _source field or not, or a list of fields to return"
|
||||
},
|
||||
"_source_excludes":{
|
||||
"type":"list",
|
||||
"description":"A list of fields to exclude from the returned _source field"
|
||||
},
|
||||
"_source_includes":{
|
||||
"type":"list",
|
||||
"description":"A list of fields to extract and return from the _source field"
|
||||
},
|
||||
"terminate_after":{
|
||||
"type":"number",
|
||||
"description":"The maximum number of documents to collect for each shard, upon reaching which the query execution will terminate early."
|
||||
},
|
||||
"stats":{
|
||||
"type":"list",
|
||||
"description":"Specific 'tag' of the request for logging and statistical purposes"
|
||||
},
|
||||
"suggest_field":{
|
||||
"type":"string",
|
||||
"description":"Specify which field to use for suggestions"
|
||||
},
|
||||
"suggest_mode":{
|
||||
"type":"enum",
|
||||
"options":[
|
||||
"missing",
|
||||
"popular",
|
||||
"always"
|
||||
],
|
||||
"default":"missing",
|
||||
"description":"Specify suggest mode"
|
||||
},
|
||||
"suggest_size":{
|
||||
"type":"number",
|
||||
"description":"How many suggestions to return in response"
|
||||
},
|
||||
"suggest_text":{
|
||||
"type":"string",
|
||||
"description":"The source text for which the suggestions should be returned"
|
||||
},
|
||||
"timeout":{
|
||||
"type":"time",
|
||||
"description":"Explicit operation timeout"
|
||||
},
|
||||
"track_scores":{
|
||||
"type":"boolean",
|
||||
"description":"Whether to calculate and return scores even if they are not used for sorting"
|
||||
},
|
||||
"track_total_hits":{
|
||||
"type":"boolean",
|
||||
"description":"Indicate if the number of documents that match the query should be tracked"
|
||||
},
|
||||
"allow_partial_search_results":{
|
||||
"type":"boolean",
|
||||
"default":true,
|
||||
"description":"Indicate if an error should be returned if there is a partial search failure or timeout"
|
||||
},
|
||||
"typed_keys":{
|
||||
"type":"boolean",
|
||||
"description":"Specify whether aggregation and suggester names should be prefixed by their respective types in the response"
|
||||
},
|
||||
"version":{
|
||||
"type":"boolean",
|
||||
"description":"Specify whether to return document version as part of a hit"
|
||||
},
|
||||
"seq_no_primary_term":{
|
||||
"type":"boolean",
|
||||
"description":"Specify whether to return sequence number and primary term of the last modification of each hit"
|
||||
},
|
||||
"request_cache":{
|
||||
"type":"boolean",
|
||||
"description":"Specify if request cache should be used for this request or not, defaults to index level setting"
|
||||
},
|
||||
"max_concurrent_shard_requests":{
|
||||
"type":"number",
|
||||
"description":"The number of concurrent shard requests per node this search executes concurrently. This value should be used to limit the impact of the search on the cluster in order to limit the number of concurrent shard requests",
|
||||
"default":5
|
||||
},
|
||||
"clean_on_completion":{
|
||||
"type":"boolean",
|
||||
"description":"Control whether the response should not be stored in the cluster if it completed within the provided [wait_for_completion] time (default: true)"
|
||||
}
|
||||
},
|
||||
"body":{
|
||||
"description":"The search definition using the Query DSL"
|
||||
}
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue