From 1f450777a1a8a5e55e20740294c2bf466a0eda52 Mon Sep 17 00:00:00 2001 From: Luca Cavanna Date: Tue, 18 Jun 2019 15:18:22 +0200 Subject: [PATCH 01/28] Cancel search task on connection close --- .../action/search/HttpChannelTaskHandler.java | 75 +++++++++++++++++++ .../action/search/TransportSearchAction.java | 53 +++++++++++-- .../action/support/TransportAction.java | 32 +++++--- .../rest/action/RestActionListener.java | 36 ++++++++- .../snapshots/SnapshotResiliencyTests.java | 2 +- 5 files changed, 177 insertions(+), 21 deletions(-) create mode 100644 server/src/main/java/org/elasticsearch/action/search/HttpChannelTaskHandler.java diff --git a/server/src/main/java/org/elasticsearch/action/search/HttpChannelTaskHandler.java b/server/src/main/java/org/elasticsearch/action/search/HttpChannelTaskHandler.java new file mode 100644 index 0000000000000..da267c247bd13 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/action/search/HttpChannelTaskHandler.java @@ -0,0 +1,75 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.action.search; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.http.HttpChannel; +import org.elasticsearch.tasks.TaskId; + +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.function.Consumer; + +/** + * This class keeps track of which task came in from which {@link HttpChannel}, by allowing to associate + * an {@link HttpChannel} with a {@link TaskId}, and also removing the link once the task is complete. + * Additionally, it accepts a consumer that gets called whenever an http channel gets closed, which + * can be used to cancel the associated task when the underlying connection gets closed. + */ +final class HttpChannelTaskHandler { + private final Map httpChannels = new ConcurrentHashMap<>(); + private final Consumer onChannelClose; + + HttpChannelTaskHandler(Consumer onChannelClose) { + this.onChannelClose = onChannelClose; + } + + void linkChannelWithTask(HttpChannel httpChannel, TaskId taskId) { + TaskId previous = httpChannels.put(httpChannel, taskId); + if (previous == null) { + //Register the listener only once we see each channel for the first time. + //In case the channel is already closed when we register the listener, the listener will be immediately executed, which + //is fine as we have already added the channel to the map, hence the task will be cancelled straight-away + httpChannel.addCloseListener(ActionListener.wrap( + response -> { + //Assumption: when the channel gets closed it won't be reused, then we can remove it from the map + // as there is no chance we will register another close listener again to the same http channel + TaskId previousTaskId = httpChannels.remove(httpChannel); + assert previousTaskId != null : "channel not found in the map, already closed?"; + if (previousTaskId != TaskId.EMPTY_TASK_ID) { + onChannelClose.accept(previousTaskId); + } + }, + exception -> {})); + } else { + //Known channel: the close listener is already registered to it + assert previous == TaskId.EMPTY_TASK_ID : "http channel not expected to be running another task [" + previous + "]"; + //TODO if an already seen channel gets closed very quickly, before we have associated it with its task, its close listener + // may find the EMPTY_TASK_ID in the map in which case it will not cancel anything... + } + } + + void unlinkChannelFromTask(HttpChannel httpChannel, TaskId taskId) { + //Execution completed: leave the channel in the map, but unset its value as the corresponding task is completed + TaskId previous = httpChannels.put(httpChannel, TaskId.EMPTY_TASK_ID); + //It could happen that the task gets completed despite it was cancelled, in which case its channel will already have been removed + assert previous == null || taskId == previous : "channel was associated with task [" + previous + "] instead of [" + taskId + "]"; + } +} diff --git a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java index d4832fb0d7a10..0a9b26b80c887 100644 --- a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java @@ -21,6 +21,7 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.OriginalIndices; +import org.elasticsearch.action.admin.cluster.node.tasks.cancel.CancelTasksRequest; import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsGroup; import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsRequest; import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsResponse; @@ -28,6 +29,7 @@ import org.elasticsearch.action.support.HandledTransportAction; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.client.Client; +import org.elasticsearch.client.node.NodeClient; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.block.ClusterBlockException; import org.elasticsearch.cluster.block.ClusterBlockLevel; @@ -43,9 +45,11 @@ import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Setting.Property; import org.elasticsearch.common.util.concurrent.CountDown; +import org.elasticsearch.http.HttpChannel; import org.elasticsearch.index.Index; import org.elasticsearch.index.query.Rewriteable; import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.rest.action.RestActionListener; import org.elasticsearch.search.SearchPhaseResult; import org.elasticsearch.search.SearchService; import org.elasticsearch.search.aggregations.InternalAggregation; @@ -57,6 +61,7 @@ import org.elasticsearch.search.profile.ProfileShardResult; import org.elasticsearch.search.profile.SearchProfileShardResults; import org.elasticsearch.tasks.Task; +import org.elasticsearch.tasks.TaskId; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.RemoteClusterAware; import org.elasticsearch.transport.RemoteClusterService; @@ -97,12 +102,14 @@ public class TransportSearchAction extends HandledTransportAction) SearchRequest::new); this.threadPool = threadPool; this.searchPhaseController = searchPhaseController; @@ -112,6 +119,15 @@ public TransportSearchAction(ThreadPool threadPool, TransportService transportSe this.clusterService = clusterService; this.searchService = searchService; this.indexNameExpressionResolver = indexNameExpressionResolver; + this.client = client; + this.httpChannelTaskHandler = new HttpChannelTaskHandler(taskId -> { + CancelTasksRequest cancelTasksRequest = new CancelTasksRequest(); + cancelTasksRequest.setTaskId(taskId); + //TODO Note that cancel tasks fails if the user does not have the permissions to call it. + // It may make sense to cancel the task directly from task manager without an api call, but cancellation of children tasks + // is part of TransportCancelTasksAction. Maybe we should move that part to TaskManager? + client.admin().cluster().cancelTasks(cancelTasksRequest, ActionListener.wrap(r -> {}, e -> {})); + }); } private Map buildPerIndexAliasFilter(SearchRequest request, ClusterState clusterState, @@ -195,6 +211,31 @@ long buildTookInMillis() { @Override protected void doExecute(Task task, SearchRequest searchRequest, ActionListener listener) { + final ActionListener wrappedListener; + if (listener instanceof RestActionListener) { + HttpChannel httpChannel = ((RestActionListener) listener).getHttpChannel(); + TaskId currentTaskId = new TaskId(client.getLocalNodeId(), task.getId()); + httpChannelTaskHandler.linkChannelWithTask(httpChannel, currentTaskId); + wrappedListener = new ActionListener<>() { + //TODO verify that the order in which listeners get notified is guaranteed: this action listener needs to be notified + // BEFORE the on close listener (in case the underlying connection gets closed at completion), otherwise we end up + // cancelling tasks for requests that are about to return a response. + @Override + public void onResponse(SearchResponse searchResponse) { + httpChannelTaskHandler.unlinkChannelFromTask(httpChannel, currentTaskId); + listener.onResponse(searchResponse); + } + + @Override + public void onFailure(Exception e) { + httpChannelTaskHandler.unlinkChannelFromTask(httpChannel, currentTaskId); + listener.onFailure(e); + } + }; + } else { + wrappedListener = listener; + } + final long relativeStartNanos = System.nanoTime(); final SearchTimeProvider timeProvider = new SearchTimeProvider(searchRequest.getOrCreateAbsoluteStartMillis(), relativeStartNanos, System::nanoTime); @@ -209,11 +250,11 @@ protected void doExecute(Task task, SearchRequest searchRequest, ActionListener< searchRequest.indices()); OriginalIndices localIndices = remoteClusterIndices.remove(RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY); if (remoteClusterIndices.isEmpty()) { - executeLocalSearch(task, timeProvider, searchRequest, localIndices, clusterState, listener); + executeLocalSearch(task, timeProvider, searchRequest, localIndices, clusterState, wrappedListener); } else { if (shouldMinimizeRoundtrips(searchRequest)) { ccsRemoteReduce(searchRequest, localIndices, remoteClusterIndices, timeProvider, searchService::createReduceContext, - remoteClusterService, threadPool, listener, + remoteClusterService, threadPool, wrappedListener, (r, l) -> executeLocalSearch(task, timeProvider, r, localIndices, clusterState, l)); } else { AtomicInteger skippedClusters = new AtomicInteger(0); @@ -229,13 +270,13 @@ protected void doExecute(Task task, SearchRequest searchRequest, ActionListener< int totalClusters = remoteClusterIndices.size() + localClusters; int successfulClusters = searchShardsResponses.size() + localClusters; executeSearch((SearchTask) task, timeProvider, searchRequest, localIndices, - remoteShardIterators, clusterNodeLookup, clusterState, remoteAliasFilters, listener, + remoteShardIterators, clusterNodeLookup, clusterState, remoteAliasFilters, wrappedListener, new SearchResponse.Clusters(totalClusters, successfulClusters, skippedClusters.get())); }, - listener::onFailure)); + wrappedListener::onFailure)); } } - }, listener::onFailure); + }, wrappedListener::onFailure); if (searchRequest.source() == null) { rewriteListener.onResponse(searchRequest.source()); } else { diff --git a/server/src/main/java/org/elasticsearch/action/support/TransportAction.java b/server/src/main/java/org/elasticsearch/action/support/TransportAction.java index fcb4905b95914..73d9b5464f4b6 100644 --- a/server/src/main/java/org/elasticsearch/action/support/TransportAction.java +++ b/server/src/main/java/org/elasticsearch/action/support/TransportAction.java @@ -24,6 +24,7 @@ import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.ActionResponse; +import org.elasticsearch.rest.action.RestActionListener; import org.elasticsearch.tasks.Task; import org.elasticsearch.tasks.TaskListener; import org.elasticsearch.tasks.TaskManager; @@ -61,19 +62,26 @@ public final Task execute(Request request, ActionListener listener) { * this method. */ Task task = taskManager.register("transport", actionName, request); - execute(task, request, new ActionListener() { - @Override - public void onResponse(Response response) { - taskManager.unregister(task); - listener.onResponse(response); - } + final ActionListener wrappedListener; + if (listener instanceof RestActionListener) { + wrappedListener = RestActionListener.runBefore((RestActionListener) listener, () -> taskManager.unregister(task)); + } else { + wrappedListener = new ActionListener<>() { + @Override + public void onResponse(Response response) { + taskManager.unregister(task); + listener.onResponse(response); + } - @Override - public void onFailure(Exception e) { - taskManager.unregister(task); - listener.onFailure(e); - } - }); + @Override + public void onFailure(Exception e) { + taskManager.unregister(task); + listener.onFailure(e); + } + }; + } + + execute(task, request, wrappedListener); return task; } diff --git a/server/src/main/java/org/elasticsearch/rest/action/RestActionListener.java b/server/src/main/java/org/elasticsearch/rest/action/RestActionListener.java index 15e535ebfe201..f85000ededd43 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/RestActionListener.java +++ b/server/src/main/java/org/elasticsearch/rest/action/RestActionListener.java @@ -22,6 +22,7 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.http.HttpChannel; import org.elasticsearch.rest.BytesRestResponse; import org.elasticsearch.rest.RestChannel; @@ -42,7 +43,7 @@ protected RestActionListener(RestChannel channel) { } @Override - public final void onResponse(Response response) { + public void onResponse(Response response) { try { processResponse(response); } catch (Exception e) { @@ -53,7 +54,7 @@ public final void onResponse(Response response) { protected abstract void processResponse(Response response) throws Exception; @Override - public final void onFailure(Exception e) { + public void onFailure(Exception e) { try { channel.sendResponse(new BytesRestResponse(channel, e)); } catch (Exception inner) { @@ -61,4 +62,35 @@ public final void onFailure(Exception e) { logger.error("failed to send failure response", inner); } } + + public HttpChannel getHttpChannel() { + return channel.request().getHttpChannel(); + } + + public static RestActionListener runBefore(RestActionListener restActionListener, Runnable runnable) { + return new RestActionListener<>(restActionListener.channel) { + @Override + public void onResponse(Response response) { + try { + runnable.run(); + } finally { + restActionListener.onResponse(response); + } + } + + @Override + public void onFailure(Exception e) { + try { + runnable.run(); + } finally { + restActionListener.onFailure(e); + } + } + + @Override + protected void processResponse(Response response) throws Exception { + restActionListener.processResponse(response); + } + }; + } } diff --git a/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java b/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java index 6c6b136ba2751..2e7af2a021334 100644 --- a/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java +++ b/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java @@ -1165,7 +1165,7 @@ clusterService, indicesService, threadPool, shardStateAction, mappingUpdatedActi actions.put(SearchAction.INSTANCE, new TransportSearchAction(threadPool, transportService, searchService, searchTransportService, new SearchPhaseController(searchService::createReduceContext), clusterService, - actionFilters, indexNameExpressionResolver)); + actionFilters, indexNameExpressionResolver, client)); actions.put(RestoreSnapshotAction.INSTANCE, new TransportRestoreSnapshotAction(transportService, clusterService, threadPool, restoreService, actionFilters, indexNameExpressionResolver)); From 9361f230610548c4f144e1f26cc7686276963c6f Mon Sep 17 00:00:00 2001 From: Luca Cavanna Date: Mon, 15 Jul 2019 11:26:15 +0200 Subject: [PATCH 02/28] feedback --- .../action/search/TransportSearchAction.java | 48 +++-------------- .../action/support/TransportAction.java | 32 +++++------- .../rest/action/RestActionListener.java | 35 +------------ .../action/search/HttpChannelTaskHandler.java | 52 ++++++++++--------- .../rest/action/search/RestSearchAction.java | 46 +++++++++++++++- .../snapshots/SnapshotResiliencyTests.java | 2 +- 6 files changed, 94 insertions(+), 121 deletions(-) rename server/src/main/java/org/elasticsearch/{ => rest}/action/search/HttpChannelTaskHandler.java (50%) diff --git a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java index 0a9b26b80c887..6a5b7d6e22d2d 100644 --- a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java @@ -102,14 +102,12 @@ public class TransportSearchAction extends HandledTransportAction) SearchRequest::new); this.threadPool = threadPool; this.searchPhaseController = searchPhaseController; @@ -119,15 +117,6 @@ public TransportSearchAction(ThreadPool threadPool, TransportService transportSe this.clusterService = clusterService; this.searchService = searchService; this.indexNameExpressionResolver = indexNameExpressionResolver; - this.client = client; - this.httpChannelTaskHandler = new HttpChannelTaskHandler(taskId -> { - CancelTasksRequest cancelTasksRequest = new CancelTasksRequest(); - cancelTasksRequest.setTaskId(taskId); - //TODO Note that cancel tasks fails if the user does not have the permissions to call it. - // It may make sense to cancel the task directly from task manager without an api call, but cancellation of children tasks - // is part of TransportCancelTasksAction. Maybe we should move that part to TaskManager? - client.admin().cluster().cancelTasks(cancelTasksRequest, ActionListener.wrap(r -> {}, e -> {})); - }); } private Map buildPerIndexAliasFilter(SearchRequest request, ClusterState clusterState, @@ -211,31 +200,6 @@ long buildTookInMillis() { @Override protected void doExecute(Task task, SearchRequest searchRequest, ActionListener listener) { - final ActionListener wrappedListener; - if (listener instanceof RestActionListener) { - HttpChannel httpChannel = ((RestActionListener) listener).getHttpChannel(); - TaskId currentTaskId = new TaskId(client.getLocalNodeId(), task.getId()); - httpChannelTaskHandler.linkChannelWithTask(httpChannel, currentTaskId); - wrappedListener = new ActionListener<>() { - //TODO verify that the order in which listeners get notified is guaranteed: this action listener needs to be notified - // BEFORE the on close listener (in case the underlying connection gets closed at completion), otherwise we end up - // cancelling tasks for requests that are about to return a response. - @Override - public void onResponse(SearchResponse searchResponse) { - httpChannelTaskHandler.unlinkChannelFromTask(httpChannel, currentTaskId); - listener.onResponse(searchResponse); - } - - @Override - public void onFailure(Exception e) { - httpChannelTaskHandler.unlinkChannelFromTask(httpChannel, currentTaskId); - listener.onFailure(e); - } - }; - } else { - wrappedListener = listener; - } - final long relativeStartNanos = System.nanoTime(); final SearchTimeProvider timeProvider = new SearchTimeProvider(searchRequest.getOrCreateAbsoluteStartMillis(), relativeStartNanos, System::nanoTime); @@ -250,11 +214,11 @@ public void onFailure(Exception e) { searchRequest.indices()); OriginalIndices localIndices = remoteClusterIndices.remove(RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY); if (remoteClusterIndices.isEmpty()) { - executeLocalSearch(task, timeProvider, searchRequest, localIndices, clusterState, wrappedListener); + executeLocalSearch(task, timeProvider, searchRequest, localIndices, clusterState, listener); } else { if (shouldMinimizeRoundtrips(searchRequest)) { ccsRemoteReduce(searchRequest, localIndices, remoteClusterIndices, timeProvider, searchService::createReduceContext, - remoteClusterService, threadPool, wrappedListener, + remoteClusterService, threadPool, listener, (r, l) -> executeLocalSearch(task, timeProvider, r, localIndices, clusterState, l)); } else { AtomicInteger skippedClusters = new AtomicInteger(0); @@ -270,13 +234,13 @@ public void onFailure(Exception e) { int totalClusters = remoteClusterIndices.size() + localClusters; int successfulClusters = searchShardsResponses.size() + localClusters; executeSearch((SearchTask) task, timeProvider, searchRequest, localIndices, - remoteShardIterators, clusterNodeLookup, clusterState, remoteAliasFilters, wrappedListener, + remoteShardIterators, clusterNodeLookup, clusterState, remoteAliasFilters, listener, new SearchResponse.Clusters(totalClusters, successfulClusters, skippedClusters.get())); }, - wrappedListener::onFailure)); + listener::onFailure)); } } - }, wrappedListener::onFailure); + }, listener::onFailure); if (searchRequest.source() == null) { rewriteListener.onResponse(searchRequest.source()); } else { diff --git a/server/src/main/java/org/elasticsearch/action/support/TransportAction.java b/server/src/main/java/org/elasticsearch/action/support/TransportAction.java index 73d9b5464f4b6..f0db91f59aff8 100644 --- a/server/src/main/java/org/elasticsearch/action/support/TransportAction.java +++ b/server/src/main/java/org/elasticsearch/action/support/TransportAction.java @@ -24,7 +24,6 @@ import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.ActionResponse; -import org.elasticsearch.rest.action.RestActionListener; import org.elasticsearch.tasks.Task; import org.elasticsearch.tasks.TaskListener; import org.elasticsearch.tasks.TaskManager; @@ -62,26 +61,19 @@ public final Task execute(Request request, ActionListener listener) { * this method. */ Task task = taskManager.register("transport", actionName, request); - final ActionListener wrappedListener; - if (listener instanceof RestActionListener) { - wrappedListener = RestActionListener.runBefore((RestActionListener) listener, () -> taskManager.unregister(task)); - } else { - wrappedListener = new ActionListener<>() { - @Override - public void onResponse(Response response) { - taskManager.unregister(task); - listener.onResponse(response); - } - - @Override - public void onFailure(Exception e) { - taskManager.unregister(task); - listener.onFailure(e); - } - }; - } + execute(task, request, new ActionListener<>() { + @Override + public void onResponse(Response response) { + taskManager.unregister(task); + listener.onResponse(response); + } - execute(task, request, wrappedListener); + @Override + public void onFailure(Exception e) { + taskManager.unregister(task); + listener.onFailure(e); + } + }); return task; } diff --git a/server/src/main/java/org/elasticsearch/rest/action/RestActionListener.java b/server/src/main/java/org/elasticsearch/rest/action/RestActionListener.java index f85000ededd43..9a52aa8781932 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/RestActionListener.java +++ b/server/src/main/java/org/elasticsearch/rest/action/RestActionListener.java @@ -43,7 +43,7 @@ protected RestActionListener(RestChannel channel) { } @Override - public void onResponse(Response response) { + public final void onResponse(Response response) { try { processResponse(response); } catch (Exception e) { @@ -54,7 +54,7 @@ public void onResponse(Response response) { protected abstract void processResponse(Response response) throws Exception; @Override - public void onFailure(Exception e) { + public final void onFailure(Exception e) { try { channel.sendResponse(new BytesRestResponse(channel, e)); } catch (Exception inner) { @@ -62,35 +62,4 @@ public void onFailure(Exception e) { logger.error("failed to send failure response", inner); } } - - public HttpChannel getHttpChannel() { - return channel.request().getHttpChannel(); - } - - public static RestActionListener runBefore(RestActionListener restActionListener, Runnable runnable) { - return new RestActionListener<>(restActionListener.channel) { - @Override - public void onResponse(Response response) { - try { - runnable.run(); - } finally { - restActionListener.onResponse(response); - } - } - - @Override - public void onFailure(Exception e) { - try { - runnable.run(); - } finally { - restActionListener.onFailure(e); - } - } - - @Override - protected void processResponse(Response response) throws Exception { - restActionListener.processResponse(response); - } - }; - } } diff --git a/server/src/main/java/org/elasticsearch/action/search/HttpChannelTaskHandler.java b/server/src/main/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandler.java similarity index 50% rename from server/src/main/java/org/elasticsearch/action/search/HttpChannelTaskHandler.java rename to server/src/main/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandler.java index da267c247bd13..3db66edfaa58c 100644 --- a/server/src/main/java/org/elasticsearch/action/search/HttpChannelTaskHandler.java +++ b/server/src/main/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandler.java @@ -17,59 +17,63 @@ * under the License. */ -package org.elasticsearch.action.search; +package org.elasticsearch.rest.action.search; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.client.Client; import org.elasticsearch.http.HttpChannel; import org.elasticsearch.tasks.TaskId; +import java.util.HashSet; import java.util.Map; +import java.util.Set; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CopyOnWriteArraySet; +import java.util.function.BiConsumer; import java.util.function.Consumer; /** - * This class keeps track of which task came in from which {@link HttpChannel}, by allowing to associate + * This class keeps track of which tasks came in from which {@link HttpChannel}, by allowing to associate * an {@link HttpChannel} with a {@link TaskId}, and also removing the link once the task is complete. * Additionally, it accepts a consumer that gets called whenever an http channel gets closed, which * can be used to cancel the associated task when the underlying connection gets closed. */ final class HttpChannelTaskHandler { - private final Map httpChannels = new ConcurrentHashMap<>(); - private final Consumer onChannelClose; + private final Map> httpChannels = new ConcurrentHashMap<>(); + private final BiConsumer onChannelClose; - HttpChannelTaskHandler(Consumer onChannelClose) { + HttpChannelTaskHandler(BiConsumer onChannelClose) { this.onChannelClose = onChannelClose; } - void linkChannelWithTask(HttpChannel httpChannel, TaskId taskId) { - TaskId previous = httpChannels.put(httpChannel, taskId); - if (previous == null) { + void linkChannelWithTask(HttpChannel httpChannel, Client client, TaskId taskId) { + Set taskIds = httpChannels.computeIfAbsent(httpChannel, channel -> { //Register the listener only once we see each channel for the first time. //In case the channel is already closed when we register the listener, the listener will be immediately executed, which //is fine as we have already added the channel to the map, hence the task will be cancelled straight-away httpChannel.addCloseListener(ActionListener.wrap( response -> { - //Assumption: when the channel gets closed it won't be reused, then we can remove it from the map - // as there is no chance we will register another close listener again to the same http channel - TaskId previousTaskId = httpChannels.remove(httpChannel); - assert previousTaskId != null : "channel not found in the map, already closed?"; - if (previousTaskId != TaskId.EMPTY_TASK_ID) { - onChannelClose.accept(previousTaskId); + //When the channel gets closed it won't be reused: we can remove it from the map as there is no chance we will + //register another close listener to it later. + Set previousTaskIds = httpChannels.remove(httpChannel); + assert previousTaskIds != null : "channel not found in the map, already closed?"; + for (TaskId previousTaskId : previousTaskIds) { + onChannelClose.accept(client, previousTaskId); } }, exception -> {})); - } else { - //Known channel: the close listener is already registered to it - assert previous == TaskId.EMPTY_TASK_ID : "http channel not expected to be running another task [" + previous + "]"; - //TODO if an already seen channel gets closed very quickly, before we have associated it with its task, its close listener - // may find the EMPTY_TASK_ID in the map in which case it will not cancel anything... - } + return new CopyOnWriteArraySet<>(); + }); + taskIds.add(taskId); } void unlinkChannelFromTask(HttpChannel httpChannel, TaskId taskId) { - //Execution completed: leave the channel in the map, but unset its value as the corresponding task is completed - TaskId previous = httpChannels.put(httpChannel, TaskId.EMPTY_TASK_ID); - //It could happen that the task gets completed despite it was cancelled, in which case its channel will already have been removed - assert previous == null || taskId == previous : "channel was associated with task [" + previous + "] instead of [" + taskId + "]"; + //Execution completed: leave the channel in the map as it may be reused later and we need to make sure that we don't register + //another close listener to it. Unset its value as the corresponding task is completed + Set taskIds = httpChannels.get(httpChannel); + //It could happen that the task is completed even before it gets associated with its channel, in which case it won't be in the map + if (taskIds != null) { + taskIds.remove(taskId); + } } } diff --git a/server/src/main/java/org/elasticsearch/rest/action/search/RestSearchAction.java b/server/src/main/java/org/elasticsearch/rest/action/search/RestSearchAction.java index 95695bec4f0c1..48cc6a8a86165 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/search/RestSearchAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/search/RestSearchAction.java @@ -19,7 +19,11 @@ package org.elasticsearch.rest.action.search; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.admin.cluster.node.tasks.cancel.CancelTasksRequest; +import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchRequest; +import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.client.node.NodeClient; import org.elasticsearch.common.Booleans; @@ -40,6 +44,9 @@ import org.elasticsearch.search.sort.SortOrder; import org.elasticsearch.search.suggest.SuggestBuilder; import org.elasticsearch.search.suggest.term.TermSuggestionBuilder.SuggestMode; +import org.elasticsearch.tasks.Task; +import org.elasticsearch.tasks.TaskId; +import org.elasticsearch.tasks.TaskListener; import java.io.IOException; import java.util.Arrays; @@ -62,6 +69,8 @@ public class RestSearchAction extends BaseRestHandler { public static final String TYPED_KEYS_PARAM = "typed_keys"; private static final Set RESPONSE_PARAMS; + private final HttpChannelTaskHandler httpChannelTaskHandler; + static { final Set responseParams = new HashSet<>(Arrays.asList(TYPED_KEYS_PARAM, TOTAL_HITS_AS_INT_PARAM)); RESPONSE_PARAMS = Collections.unmodifiableSet(responseParams); @@ -73,6 +82,14 @@ public RestSearchAction(Settings settings, RestController controller) { controller.registerHandler(POST, "/_search", this); controller.registerHandler(GET, "/{index}/_search", this); controller.registerHandler(POST, "/{index}/_search", this); + this.httpChannelTaskHandler = new HttpChannelTaskHandler((client, taskId) -> { + CancelTasksRequest cancelTasksRequest = new CancelTasksRequest(); + cancelTasksRequest.setTaskId(taskId); + //TODO Note that cancel tasks fails if the user does not have the permissions to call it. + // It may make sense to cancel the task directly from task manager without an api call, but cancellation of children tasks + // is part of TransportCancelTasksAction. Maybe we should move that part to TaskManager? + client.admin().cluster().cancelTasks(cancelTasksRequest, ActionListener.wrap(r -> {}, e -> {})); + }); } @Override @@ -99,7 +116,34 @@ public RestChannelConsumer prepareRequest(final RestRequest request, final NodeC request.withContentOrSourceParamParserOrNull(parser -> parseSearchRequest(searchRequest, request, parser, setSize)); - return channel -> client.search(searchRequest, new RestStatusToXContentListener<>(channel)); + return channel -> { + RestStatusToXContentListener listener = new RestStatusToXContentListener<>(channel); + Task task = client.executeLocally(SearchAction.INSTANCE, searchRequest, new TaskListener<>() { + //TODO verify that the order in which listeners get notified is guaranteed: this action listener needs to be notified + // BEFORE the on close listener (in case the underlying connection gets closed at completion), otherwise we end up + // cancelling tasks for requests that are about to return a response. + @Override + public void onResponse(Task task, SearchResponse response) { + TaskId taskId = new TaskId(client.getLocalNodeId(), task.getId()); + //TODO this may be called even before the channel is linked with the task + httpChannelTaskHandler.unlinkChannelFromTask(request.getHttpChannel(), taskId); + listener.onResponse(response); + } + + @Override + public void onFailure(Task task, Throwable e) { + TaskId taskId = new TaskId(client.getLocalNodeId(), task.getId()); + httpChannelTaskHandler.unlinkChannelFromTask(request.getHttpChannel(), taskId); + if (e instanceof Exception) { + listener.onFailure((Exception)e); + } else { + listener.onFailure(new RuntimeException(e)); + } + } + }); + TaskId taskId = new TaskId(client.getLocalNodeId(), task.getId()); + httpChannelTaskHandler.linkChannelWithTask(request.getHttpChannel(), client, taskId); + }; } /** diff --git a/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java b/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java index af2239947a1be..289d707a1e4d3 100644 --- a/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java +++ b/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java @@ -1040,7 +1040,7 @@ clusterService, indicesService, threadPool, shardStateAction, mappingUpdatedActi actions.put(SearchAction.INSTANCE, new TransportSearchAction(threadPool, transportService, searchService, searchTransportService, new SearchPhaseController(searchService::createReduceContext), clusterService, - actionFilters, indexNameExpressionResolver, client)); + actionFilters, indexNameExpressionResolver)); actions.put(RestoreSnapshotAction.INSTANCE, new TransportRestoreSnapshotAction(transportService, clusterService, threadPool, restoreService, actionFilters, indexNameExpressionResolver)); From 71f424d482d248e124102f44255edcb2c4f924ea Mon Sep 17 00:00:00 2001 From: Luca Cavanna Date: Tue, 16 Jul 2019 17:29:54 +0200 Subject: [PATCH 03/28] wip --- .../action/search/FetchSearchPhase.java | 1 + .../action/search/HttpChannelTaskHandler.java | 13 ++-- .../search/HttpChannelTaskListener.java | 65 +++++++++++++++++++ .../action/search/RestMultiSearchAction.java | 14 +++- .../rest/action/search/RestSearchAction.java | 36 +--------- 5 files changed, 89 insertions(+), 40 deletions(-) create mode 100644 server/src/main/java/org/elasticsearch/rest/action/search/HttpChannelTaskListener.java diff --git a/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java b/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java index 2115b4fa99849..42d5d25cf8d54 100644 --- a/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java @@ -80,6 +80,7 @@ public void run() { context.execute(new AbstractRunnable() { @Override protected void doRun() throws Exception { + Thread.sleep(10000L); // we do the heavy lifting in this inner run method where we reduce aggs etc. that's why we fork this phase // off immediately instead of forking when we send back the response to the user since there we only need // to merge together the fetched results which is a linear operation. diff --git a/server/src/main/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandler.java b/server/src/main/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandler.java index 3db66edfaa58c..fae7bc29dc55f 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandler.java +++ b/server/src/main/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandler.java @@ -20,6 +20,7 @@ package org.elasticsearch.rest.action.search; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.admin.cluster.node.tasks.cancel.CancelTasksRequest; import org.elasticsearch.client.Client; import org.elasticsearch.http.HttpChannel; import org.elasticsearch.tasks.TaskId; @@ -40,11 +41,6 @@ */ final class HttpChannelTaskHandler { private final Map> httpChannels = new ConcurrentHashMap<>(); - private final BiConsumer onChannelClose; - - HttpChannelTaskHandler(BiConsumer onChannelClose) { - this.onChannelClose = onChannelClose; - } void linkChannelWithTask(HttpChannel httpChannel, Client client, TaskId taskId) { Set taskIds = httpChannels.computeIfAbsent(httpChannel, channel -> { @@ -58,7 +54,12 @@ void linkChannelWithTask(HttpChannel httpChannel, Client client, TaskId taskId) Set previousTaskIds = httpChannels.remove(httpChannel); assert previousTaskIds != null : "channel not found in the map, already closed?"; for (TaskId previousTaskId : previousTaskIds) { - onChannelClose.accept(client, previousTaskId); + CancelTasksRequest cancelTasksRequest = new CancelTasksRequest(); + cancelTasksRequest.setTaskId(previousTaskId); + //TODO Note that cancel tasks fails if the user does not have the permissions to call it. + // It may make sense to cancel the task directly from task manager without an api call, but cancellation of children tasks + // is part of TransportCancelTasksAction. Maybe we should move that part to TaskManager? + client.admin().cluster().cancelTasks(cancelTasksRequest, ActionListener.wrap(r -> {}, e -> {})); } }, exception -> {})); diff --git a/server/src/main/java/org/elasticsearch/rest/action/search/HttpChannelTaskListener.java b/server/src/main/java/org/elasticsearch/rest/action/search/HttpChannelTaskListener.java new file mode 100644 index 0000000000000..3ab361688af3c --- /dev/null +++ b/server/src/main/java/org/elasticsearch/rest/action/search/HttpChannelTaskListener.java @@ -0,0 +1,65 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.rest.action.search; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.http.HttpChannel; +import org.elasticsearch.tasks.Task; +import org.elasticsearch.tasks.TaskId; +import org.elasticsearch.tasks.TaskListener; + +/** + * {@link TaskListener} implementation that supports interaction with {@link HttpChannelTaskHandler} so that the link between the http + * channel and its corresponding task is removed once the request is completed. + */ +public final class HttpChannelTaskListener implements TaskListener { + private final HttpChannelTaskHandler httpChannelTaskHandler; + private final ActionListener delegateListener; + private final HttpChannel httpChannel; + private final String localNodeId; + + HttpChannelTaskListener(HttpChannelTaskHandler httpChannelTaskHandler, ActionListener delegateListener, + HttpChannel httpChannel, String localNodeId) { + this.httpChannelTaskHandler = httpChannelTaskHandler; + this.delegateListener = delegateListener; + this.httpChannel = httpChannel; + this.localNodeId = localNodeId; + } + + //TODO verify that the order in which listeners get notified is guaranteed: this action listener needs to be notified + // BEFORE the on close listener (in case the underlying connection gets closed at completion), otherwise we end up + // cancelling tasks for requests that are about to return a response. + @Override + public void onResponse(Task task, Response response) { + TaskId taskId = new TaskId(localNodeId, task.getId()); + //TODO this may be called even before the channel is linked with the task + httpChannelTaskHandler.unlinkChannelFromTask(httpChannel, taskId); + delegateListener.onResponse(response); + } + + @Override + public void onFailure(Task task, Throwable e) { + TaskId taskId = new TaskId(localNodeId, task.getId()); + httpChannelTaskHandler.unlinkChannelFromTask(httpChannel, taskId); + if (e instanceof Exception) { + delegateListener.onFailure(new RuntimeException(e)); + } + } +} diff --git a/server/src/main/java/org/elasticsearch/rest/action/search/RestMultiSearchAction.java b/server/src/main/java/org/elasticsearch/rest/action/search/RestMultiSearchAction.java index 1aef4aa5254a4..4ba0f44d8c059 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/search/RestMultiSearchAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/search/RestMultiSearchAction.java @@ -19,7 +19,9 @@ package org.elasticsearch.rest.action.search; +import org.elasticsearch.action.search.MultiSearchAction; import org.elasticsearch.action.search.MultiSearchRequest; +import org.elasticsearch.action.search.MultiSearchResponse; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.client.node.NodeClient; @@ -36,6 +38,8 @@ import org.elasticsearch.rest.RestRequest; import org.elasticsearch.rest.action.RestToXContentListener; import org.elasticsearch.search.builder.SearchSourceBuilder; +import org.elasticsearch.tasks.Task; +import org.elasticsearch.tasks.TaskId; import java.io.IOException; import java.util.Arrays; @@ -58,6 +62,7 @@ public class RestMultiSearchAction extends BaseRestHandler { RESPONSE_PARAMS = Collections.unmodifiableSet(responseParams); } + private final HttpChannelTaskHandler httpChannelTaskHandler = new HttpChannelTaskHandler(); private final boolean allowExplicitIndex; public RestMultiSearchAction(Settings settings, RestController controller) { @@ -78,7 +83,14 @@ public String getName() { @Override public RestChannelConsumer prepareRequest(final RestRequest request, final NodeClient client) throws IOException { MultiSearchRequest multiSearchRequest = parseRequest(request, allowExplicitIndex); - return channel -> client.multiSearch(multiSearchRequest, new RestToXContentListener<>(channel)); + + return channel -> { + RestToXContentListener listener = new RestToXContentListener<>(channel); + Task task = client.executeLocally(MultiSearchAction.INSTANCE, multiSearchRequest, + new HttpChannelTaskListener<>(httpChannelTaskHandler, listener, request.getHttpChannel(), client.getLocalNodeId())); + TaskId taskId = new TaskId(client.getLocalNodeId(), task.getId()); + httpChannelTaskHandler.linkChannelWithTask(request.getHttpChannel(), client, taskId); + }; } /** diff --git a/server/src/main/java/org/elasticsearch/rest/action/search/RestSearchAction.java b/server/src/main/java/org/elasticsearch/rest/action/search/RestSearchAction.java index 48cc6a8a86165..915b92ce6f925 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/search/RestSearchAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/search/RestSearchAction.java @@ -46,7 +46,6 @@ import org.elasticsearch.search.suggest.term.TermSuggestionBuilder.SuggestMode; import org.elasticsearch.tasks.Task; import org.elasticsearch.tasks.TaskId; -import org.elasticsearch.tasks.TaskListener; import java.io.IOException; import java.util.Arrays; @@ -69,7 +68,7 @@ public class RestSearchAction extends BaseRestHandler { public static final String TYPED_KEYS_PARAM = "typed_keys"; private static final Set RESPONSE_PARAMS; - private final HttpChannelTaskHandler httpChannelTaskHandler; + private final HttpChannelTaskHandler httpChannelTaskHandler = new HttpChannelTaskHandler(); static { final Set responseParams = new HashSet<>(Arrays.asList(TYPED_KEYS_PARAM, TOTAL_HITS_AS_INT_PARAM)); @@ -82,14 +81,6 @@ public RestSearchAction(Settings settings, RestController controller) { controller.registerHandler(POST, "/_search", this); controller.registerHandler(GET, "/{index}/_search", this); controller.registerHandler(POST, "/{index}/_search", this); - this.httpChannelTaskHandler = new HttpChannelTaskHandler((client, taskId) -> { - CancelTasksRequest cancelTasksRequest = new CancelTasksRequest(); - cancelTasksRequest.setTaskId(taskId); - //TODO Note that cancel tasks fails if the user does not have the permissions to call it. - // It may make sense to cancel the task directly from task manager without an api call, but cancellation of children tasks - // is part of TransportCancelTasksAction. Maybe we should move that part to TaskManager? - client.admin().cluster().cancelTasks(cancelTasksRequest, ActionListener.wrap(r -> {}, e -> {})); - }); } @Override @@ -118,29 +109,8 @@ public RestChannelConsumer prepareRequest(final RestRequest request, final NodeC return channel -> { RestStatusToXContentListener listener = new RestStatusToXContentListener<>(channel); - Task task = client.executeLocally(SearchAction.INSTANCE, searchRequest, new TaskListener<>() { - //TODO verify that the order in which listeners get notified is guaranteed: this action listener needs to be notified - // BEFORE the on close listener (in case the underlying connection gets closed at completion), otherwise we end up - // cancelling tasks for requests that are about to return a response. - @Override - public void onResponse(Task task, SearchResponse response) { - TaskId taskId = new TaskId(client.getLocalNodeId(), task.getId()); - //TODO this may be called even before the channel is linked with the task - httpChannelTaskHandler.unlinkChannelFromTask(request.getHttpChannel(), taskId); - listener.onResponse(response); - } - - @Override - public void onFailure(Task task, Throwable e) { - TaskId taskId = new TaskId(client.getLocalNodeId(), task.getId()); - httpChannelTaskHandler.unlinkChannelFromTask(request.getHttpChannel(), taskId); - if (e instanceof Exception) { - listener.onFailure((Exception)e); - } else { - listener.onFailure(new RuntimeException(e)); - } - } - }); + Task task = client.executeLocally(SearchAction.INSTANCE, searchRequest, + new HttpChannelTaskListener<>(httpChannelTaskHandler, listener, request.getHttpChannel(), client.getLocalNodeId())); TaskId taskId = new TaskId(client.getLocalNodeId(), task.getId()); httpChannelTaskHandler.linkChannelWithTask(request.getHttpChannel(), client, taskId); }; From f71dd5e81d513ab462bff75cdd48d137e33800a7 Mon Sep 17 00:00:00 2001 From: Luca Cavanna Date: Mon, 22 Jul 2019 17:23:07 +0200 Subject: [PATCH 04/28] speed up fetch phase by 10 seconds :) --- .../java/org/elasticsearch/action/search/FetchSearchPhase.java | 1 - 1 file changed, 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java b/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java index 42d5d25cf8d54..2115b4fa99849 100644 --- a/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java @@ -80,7 +80,6 @@ public void run() { context.execute(new AbstractRunnable() { @Override protected void doRun() throws Exception { - Thread.sleep(10000L); // we do the heavy lifting in this inner run method where we reduce aggs etc. that's why we fork this phase // off immediately instead of forking when we send back the response to the user since there we only need // to merge together the fetched results which is a linear operation. From 4dc5f9d6cc9618f646d558b512fabfb59288f530 Mon Sep 17 00:00:00 2001 From: Luca Cavanna Date: Mon, 22 Jul 2019 17:25:11 +0200 Subject: [PATCH 05/28] unused imports --- .../elasticsearch/action/search/TransportSearchAction.java | 5 ----- 1 file changed, 5 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java index 6a5b7d6e22d2d..d4832fb0d7a10 100644 --- a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java @@ -21,7 +21,6 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.OriginalIndices; -import org.elasticsearch.action.admin.cluster.node.tasks.cancel.CancelTasksRequest; import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsGroup; import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsRequest; import org.elasticsearch.action.admin.cluster.shards.ClusterSearchShardsResponse; @@ -29,7 +28,6 @@ import org.elasticsearch.action.support.HandledTransportAction; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.client.Client; -import org.elasticsearch.client.node.NodeClient; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.block.ClusterBlockException; import org.elasticsearch.cluster.block.ClusterBlockLevel; @@ -45,11 +43,9 @@ import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Setting.Property; import org.elasticsearch.common.util.concurrent.CountDown; -import org.elasticsearch.http.HttpChannel; import org.elasticsearch.index.Index; import org.elasticsearch.index.query.Rewriteable; import org.elasticsearch.index.shard.ShardId; -import org.elasticsearch.rest.action.RestActionListener; import org.elasticsearch.search.SearchPhaseResult; import org.elasticsearch.search.SearchService; import org.elasticsearch.search.aggregations.InternalAggregation; @@ -61,7 +57,6 @@ import org.elasticsearch.search.profile.ProfileShardResult; import org.elasticsearch.search.profile.SearchProfileShardResults; import org.elasticsearch.tasks.Task; -import org.elasticsearch.tasks.TaskId; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.RemoteClusterAware; import org.elasticsearch.transport.RemoteClusterService; From 9b101e54b1673ecc441cfd6d335ffdeda76778ce Mon Sep 17 00:00:00 2001 From: Luca Cavanna Date: Mon, 22 Jul 2019 17:26:17 +0200 Subject: [PATCH 06/28] revert unrelated change --- .../java/org/elasticsearch/action/support/TransportAction.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/action/support/TransportAction.java b/server/src/main/java/org/elasticsearch/action/support/TransportAction.java index f0db91f59aff8..fcb4905b95914 100644 --- a/server/src/main/java/org/elasticsearch/action/support/TransportAction.java +++ b/server/src/main/java/org/elasticsearch/action/support/TransportAction.java @@ -61,7 +61,7 @@ public final Task execute(Request request, ActionListener listener) { * this method. */ Task task = taskManager.register("transport", actionName, request); - execute(task, request, new ActionListener<>() { + execute(task, request, new ActionListener() { @Override public void onResponse(Response response) { taskManager.unregister(task); From 0654c915f3c588b9c771e3f66672b29fdd88ce2e Mon Sep 17 00:00:00 2001 From: Luca Cavanna Date: Mon, 22 Jul 2019 17:27:10 +0200 Subject: [PATCH 07/28] unused imports --- .../java/org/elasticsearch/rest/action/RestActionListener.java | 1 - 1 file changed, 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/rest/action/RestActionListener.java b/server/src/main/java/org/elasticsearch/rest/action/RestActionListener.java index 9a52aa8781932..15e535ebfe201 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/RestActionListener.java +++ b/server/src/main/java/org/elasticsearch/rest/action/RestActionListener.java @@ -22,7 +22,6 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.http.HttpChannel; import org.elasticsearch.rest.BytesRestResponse; import org.elasticsearch.rest.RestChannel; From 05a428d3dc7bc997c49802c9c7d7514f960cb48d Mon Sep 17 00:00:00 2001 From: Luca Cavanna Date: Mon, 22 Jul 2019 17:27:56 +0200 Subject: [PATCH 08/28] remove multisearch changes --- .../rest/action/search/RestMultiSearchAction.java | 14 +------------- 1 file changed, 1 insertion(+), 13 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/rest/action/search/RestMultiSearchAction.java b/server/src/main/java/org/elasticsearch/rest/action/search/RestMultiSearchAction.java index 4ba0f44d8c059..1aef4aa5254a4 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/search/RestMultiSearchAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/search/RestMultiSearchAction.java @@ -19,9 +19,7 @@ package org.elasticsearch.rest.action.search; -import org.elasticsearch.action.search.MultiSearchAction; import org.elasticsearch.action.search.MultiSearchRequest; -import org.elasticsearch.action.search.MultiSearchResponse; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.client.node.NodeClient; @@ -38,8 +36,6 @@ import org.elasticsearch.rest.RestRequest; import org.elasticsearch.rest.action.RestToXContentListener; import org.elasticsearch.search.builder.SearchSourceBuilder; -import org.elasticsearch.tasks.Task; -import org.elasticsearch.tasks.TaskId; import java.io.IOException; import java.util.Arrays; @@ -62,7 +58,6 @@ public class RestMultiSearchAction extends BaseRestHandler { RESPONSE_PARAMS = Collections.unmodifiableSet(responseParams); } - private final HttpChannelTaskHandler httpChannelTaskHandler = new HttpChannelTaskHandler(); private final boolean allowExplicitIndex; public RestMultiSearchAction(Settings settings, RestController controller) { @@ -83,14 +78,7 @@ public String getName() { @Override public RestChannelConsumer prepareRequest(final RestRequest request, final NodeClient client) throws IOException { MultiSearchRequest multiSearchRequest = parseRequest(request, allowExplicitIndex); - - return channel -> { - RestToXContentListener listener = new RestToXContentListener<>(channel); - Task task = client.executeLocally(MultiSearchAction.INSTANCE, multiSearchRequest, - new HttpChannelTaskListener<>(httpChannelTaskHandler, listener, request.getHttpChannel(), client.getLocalNodeId())); - TaskId taskId = new TaskId(client.getLocalNodeId(), task.getId()); - httpChannelTaskHandler.linkChannelWithTask(request.getHttpChannel(), client, taskId); - }; + return channel -> client.multiSearch(multiSearchRequest, new RestToXContentListener<>(channel)); } /** From 21572306e240305154a6625e3590df572903c889 Mon Sep 17 00:00:00 2001 From: Luca Cavanna Date: Thu, 25 Jul 2019 15:54:21 +0200 Subject: [PATCH 09/28] wip --- .../action/search/HttpChannelTaskHandler.java | 147 +++++++++++---- .../search/HttpChannelTaskListener.java | 65 ------- .../rest/action/search/RestSearchAction.java | 9 +- .../search/HttpChannelTaskHandlerTests.java | 170 ++++++++++++++++++ 4 files changed, 283 insertions(+), 108 deletions(-) delete mode 100644 server/src/main/java/org/elasticsearch/rest/action/search/HttpChannelTaskListener.java create mode 100644 server/src/test/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandlerTests.java diff --git a/server/src/main/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandler.java b/server/src/main/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandler.java index fae7bc29dc55f..40cc6b8484d41 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandler.java +++ b/server/src/main/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandler.java @@ -20,18 +20,23 @@ package org.elasticsearch.rest.action.search; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.action.ActionResponse; +import org.elasticsearch.action.ActionType; import org.elasticsearch.action.admin.cluster.node.tasks.cancel.CancelTasksRequest; import org.elasticsearch.client.Client; +import org.elasticsearch.client.node.NodeClient; import org.elasticsearch.http.HttpChannel; +import org.elasticsearch.tasks.Task; import org.elasticsearch.tasks.TaskId; +import org.elasticsearch.tasks.TaskListener; -import java.util.HashSet; import java.util.Map; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CopyOnWriteArraySet; -import java.util.function.BiConsumer; -import java.util.function.Consumer; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; /** * This class keeps track of which tasks came in from which {@link HttpChannel}, by allowing to associate @@ -40,41 +45,113 @@ * can be used to cancel the associated task when the underlying connection gets closed. */ final class HttpChannelTaskHandler { - private final Map> httpChannels = new ConcurrentHashMap<>(); - - void linkChannelWithTask(HttpChannel httpChannel, Client client, TaskId taskId) { - Set taskIds = httpChannels.computeIfAbsent(httpChannel, channel -> { - //Register the listener only once we see each channel for the first time. - //In case the channel is already closed when we register the listener, the listener will be immediately executed, which - //is fine as we have already added the channel to the map, hence the task will be cancelled straight-away - httpChannel.addCloseListener(ActionListener.wrap( - response -> { - //When the channel gets closed it won't be reused: we can remove it from the map as there is no chance we will - //register another close listener to it later. - Set previousTaskIds = httpChannels.remove(httpChannel); - assert previousTaskIds != null : "channel not found in the map, already closed?"; - for (TaskId previousTaskId : previousTaskIds) { - CancelTasksRequest cancelTasksRequest = new CancelTasksRequest(); - cancelTasksRequest.setTaskId(previousTaskId); - //TODO Note that cancel tasks fails if the user does not have the permissions to call it. - // It may make sense to cancel the task directly from task manager without an api call, but cancellation of children tasks - // is part of TransportCancelTasksAction. Maybe we should move that part to TaskManager? - client.admin().cluster().cancelTasks(cancelTasksRequest, ActionListener.wrap(r -> {}, e -> {})); + final Map httpChannels = new ConcurrentHashMap<>(); + + void execute(NodeClient client, HttpChannel httpChannel, ActionRequest request, + ActionType actionType, ActionListener listener) { + + AtomicBoolean linkEnabled = new AtomicBoolean(true); + Task task = client.executeLocally(actionType, request, + new TaskListener<>() { + @Override + public void onResponse(Task task, Response searchResponse) { + unlink(task); + listener.onResponse(searchResponse); + } + + @Override + public void onFailure(Task task, Throwable e) { + unlink(task); + if (e instanceof Exception) { + listener.onFailure((Exception)e); + } else { + //TODO should we rather throw in case of throwable instead of notifying the listener? + listener.onFailure(new RuntimeException(e)); + } + } + + private void unlink(Task task) { + //the synchronized blocks are to make sure that only link or unlink for a specific task can happen at a given time, + //they can't happen concurrently. The flag is needed because unlink can still be called before link, which would + //lead to piling up task ids that are never removed from the map. + //It may look like only either synchronized or the boolean flag are needed but they both are. In fact, the boolean flag + //is needed to ensure that we don't link a task if we have already unlinked it. But it's not enough as, once we start + //the linking, we do want its corresponding unlinking to happen, but only once the linking is completed. With only + //the boolean flag, we would just miss unlinking for some tasks that are being linked when onResponse is called. + synchronized(task) { + try { + //nothing to do if link was not called yet: it would not find the task anyways. + if (linkEnabled.compareAndSet(false, true)) { + CloseListener closeListener = httpChannels.get(httpChannel); + TaskId taskId = new TaskId(client.getLocalNodeId(), task.getId()); + closeListener.unregisterTask(taskId); + } + } catch(Exception e) { + listener.onFailure(e); + } } - }, - exception -> {})); - return new CopyOnWriteArraySet<>(); - }); - taskIds.add(taskId); + } + }); + + CloseListener closeListener = httpChannels.computeIfAbsent(httpChannel, channel -> new CloseListener(client)); + synchronized (task) { + //make sure that the link is made only if the task is not already completed, otherwise unlink would have already been called + if (linkEnabled.compareAndSet(true, false)) { + TaskId taskId = new TaskId(client.getLocalNodeId(), task.getId()); + closeListener.registerTask(httpChannel, taskId); + } + } + + //TODO test case where listener is registered, but no tasks have been added yet: + // - connection gets closed, channel will be removed, no tasks will be cancelled + // - unlink is called before, hence the task is removed (not found) before it gets added + + //TODO check that no tasks are left behind through assertions at node close } - void unlinkChannelFromTask(HttpChannel httpChannel, TaskId taskId) { - //Execution completed: leave the channel in the map as it may be reused later and we need to make sure that we don't register - //another close listener to it. Unset its value as the corresponding task is completed - Set taskIds = httpChannels.get(httpChannel); - //It could happen that the task is completed even before it gets associated with its channel, in which case it won't be in the map - if (taskIds != null) { - taskIds.remove(taskId); + final class CloseListener implements ActionListener { + private final Client client; + final Set taskIds = new CopyOnWriteArraySet<>(); + private final AtomicReference channel = new AtomicReference<>(); + + CloseListener(Client client) { + this.client = client; + } + + void registerTask(HttpChannel httpChannel, TaskId taskId) { + if (channel.compareAndSet(null, httpChannel)) { + //In case the channel is already closed when we register the listener, the listener will be immediately executed which will + //remove the channel from the map straight-away. That is why we do this in two stages. If we provided the channel at close + //listener initialization we would have to deal with close listeners calls before the channel is in the map. + httpChannel.addCloseListener(this); + } + this.taskIds.add(taskId); + } + + private void unregisterTask(TaskId taskId) { + this.taskIds.remove(taskId); + } + + @Override + public void onResponse(Void aVoid) { + //When the channel gets closed it won't be reused: we can remove it from the map as there is no chance we will + //register another close listener to it later. + //The channel reference may be null, if the connection gets closed before we set it. + //The channel must be found in the map though as this listener gets registered after the channel is added. + //TODO test channel null here? it can happen! + httpChannels.remove(channel.get()); + for (TaskId previousTaskId : taskIds) { + CancelTasksRequest cancelTasksRequest = new CancelTasksRequest(); + cancelTasksRequest.setTaskId(previousTaskId); + //We don't wait for cancel tasks to come back. Task cancellation is just best effort. + //Note that cancel tasks fails if the user sending the search request does not have the permissions to call it. + client.admin().cluster().cancelTasks(cancelTasksRequest, ActionListener.wrap(r -> {}, e -> {})); + } + } + + @Override + public void onFailure(Exception e) { + //nothing to do here } } } diff --git a/server/src/main/java/org/elasticsearch/rest/action/search/HttpChannelTaskListener.java b/server/src/main/java/org/elasticsearch/rest/action/search/HttpChannelTaskListener.java deleted file mode 100644 index 3ab361688af3c..0000000000000 --- a/server/src/main/java/org/elasticsearch/rest/action/search/HttpChannelTaskListener.java +++ /dev/null @@ -1,65 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.elasticsearch.rest.action.search; - -import org.elasticsearch.action.ActionListener; -import org.elasticsearch.http.HttpChannel; -import org.elasticsearch.tasks.Task; -import org.elasticsearch.tasks.TaskId; -import org.elasticsearch.tasks.TaskListener; - -/** - * {@link TaskListener} implementation that supports interaction with {@link HttpChannelTaskHandler} so that the link between the http - * channel and its corresponding task is removed once the request is completed. - */ -public final class HttpChannelTaskListener implements TaskListener { - private final HttpChannelTaskHandler httpChannelTaskHandler; - private final ActionListener delegateListener; - private final HttpChannel httpChannel; - private final String localNodeId; - - HttpChannelTaskListener(HttpChannelTaskHandler httpChannelTaskHandler, ActionListener delegateListener, - HttpChannel httpChannel, String localNodeId) { - this.httpChannelTaskHandler = httpChannelTaskHandler; - this.delegateListener = delegateListener; - this.httpChannel = httpChannel; - this.localNodeId = localNodeId; - } - - //TODO verify that the order in which listeners get notified is guaranteed: this action listener needs to be notified - // BEFORE the on close listener (in case the underlying connection gets closed at completion), otherwise we end up - // cancelling tasks for requests that are about to return a response. - @Override - public void onResponse(Task task, Response response) { - TaskId taskId = new TaskId(localNodeId, task.getId()); - //TODO this may be called even before the channel is linked with the task - httpChannelTaskHandler.unlinkChannelFromTask(httpChannel, taskId); - delegateListener.onResponse(response); - } - - @Override - public void onFailure(Task task, Throwable e) { - TaskId taskId = new TaskId(localNodeId, task.getId()); - httpChannelTaskHandler.unlinkChannelFromTask(httpChannel, taskId); - if (e instanceof Exception) { - delegateListener.onFailure(new RuntimeException(e)); - } - } -} diff --git a/server/src/main/java/org/elasticsearch/rest/action/search/RestSearchAction.java b/server/src/main/java/org/elasticsearch/rest/action/search/RestSearchAction.java index 915b92ce6f925..190dbfc9da9ea 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/search/RestSearchAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/search/RestSearchAction.java @@ -19,8 +19,6 @@ package org.elasticsearch.rest.action.search; -import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.admin.cluster.node.tasks.cancel.CancelTasksRequest; import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchResponse; @@ -44,8 +42,6 @@ import org.elasticsearch.search.sort.SortOrder; import org.elasticsearch.search.suggest.SuggestBuilder; import org.elasticsearch.search.suggest.term.TermSuggestionBuilder.SuggestMode; -import org.elasticsearch.tasks.Task; -import org.elasticsearch.tasks.TaskId; import java.io.IOException; import java.util.Arrays; @@ -109,10 +105,7 @@ public RestChannelConsumer prepareRequest(final RestRequest request, final NodeC return channel -> { RestStatusToXContentListener listener = new RestStatusToXContentListener<>(channel); - Task task = client.executeLocally(SearchAction.INSTANCE, searchRequest, - new HttpChannelTaskListener<>(httpChannelTaskHandler, listener, request.getHttpChannel(), client.getLocalNodeId())); - TaskId taskId = new TaskId(client.getLocalNodeId(), task.getId()); - httpChannelTaskHandler.linkChannelWithTask(request.getHttpChannel(), client, taskId); + httpChannelTaskHandler.execute(client, request.getHttpChannel(), searchRequest, SearchAction.INSTANCE, listener); }; } diff --git a/server/src/test/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandlerTests.java b/server/src/test/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandlerTests.java new file mode 100644 index 0000000000000..8eb756cbd91bc --- /dev/null +++ b/server/src/test/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandlerTests.java @@ -0,0 +1,170 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.rest.action.search; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.action.ActionResponse; +import org.elasticsearch.action.ActionType; +import org.elasticsearch.action.search.SearchAction; +import org.elasticsearch.action.search.SearchRequest; +import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.action.support.PlainListenableActionFuture; +import org.elasticsearch.client.node.NodeClient; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.http.HttpChannel; +import org.elasticsearch.http.HttpResponse; +import org.elasticsearch.tasks.Task; +import org.elasticsearch.tasks.TaskListener; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.threadpool.TestThreadPool; +import org.elasticsearch.threadpool.ThreadPool; +import org.junit.After; +import org.junit.Before; + +import java.net.InetSocketAddress; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; + +public class HttpChannelTaskHandlerTests extends ESTestCase { + + private ThreadPool threadPool; + + @Before + public void createThreadPool() { + threadPool = new TestThreadPool(HttpChannelTaskHandlerTests.class.getName()); + } + + @After + public void stopThreadPool() { + ThreadPool.terminate(threadPool, 5, TimeUnit.SECONDS); + } + + public void testLinkAndUnlink() throws Exception { + + try (TestClient testClient = new TestClient(Settings.EMPTY, threadPool)) { + HttpChannelTaskHandler httpChannelTaskHandler = new HttpChannelTaskHandler(); + List> futures = new ArrayList<>(); + int numChannels = 1;//randomIntBetween(1, 30); + System.out.println("channels " + numChannels); + + for (int i = 0; i < numChannels; i++) { + int numTasks = 3;//randomIntBetween(1, 30); + System.out.println("tasks " + numTasks); + TestHttpChannel channel = new TestHttpChannel(); + for (int j = 0; j < numTasks; j++) { + PlainListenableActionFuture actionFuture = PlainListenableActionFuture.newListenableFuture(); + threadPool.generic().submit(() -> httpChannelTaskHandler.execute(testClient, channel, new SearchRequest(), + SearchAction.INSTANCE, actionFuture)); + futures.add(actionFuture); + } + } + + for (Future future : futures) { + future.get(); + } + //no channels get closed in this test + assertEquals(numChannels, httpChannelTaskHandler.httpChannels.size()); + for (Map.Entry entry : httpChannelTaskHandler.httpChannels.entrySet()) { + assertEquals(0, entry.getValue().taskIds.size()); + } + } + } + + //TODO verify that we do not add stuff to the map when the channel is closed at link call + //same when unlink is called before link + + private static class TestClient extends NodeClient { + + private final AtomicLong counter = new AtomicLong(0); + + TestClient(Settings settings, ThreadPool threadPool) { + super(settings, threadPool); + } + + @Override + public Task executeLocally(ActionType action, + Request request, + TaskListener listener) { + Task task = request.createTask(counter.getAndIncrement(), "type", action.name(), null, Collections.emptyMap()); + if (rarely()) { + listener.onResponse(task, null); + } else { + threadPool().generic().submit(() -> listener.onResponse(task, null)); + } + return task; + } + + @Override + public String getLocalNodeId() { + return "node"; + } + } + + private static class TestHttpChannel implements HttpChannel { + private final AtomicBoolean open = new AtomicBoolean(true); + private final AtomicReference> closeListener = new AtomicReference<>(); + + @Override + public void sendResponse(HttpResponse response, ActionListener listener) { + + } + + @Override + public InetSocketAddress getLocalAddress() { + return null; + } + + @Override + public InetSocketAddress getRemoteAddress() { + return null; + } + + @Override + public void close() { + if (open.compareAndSet(true, false) == false) { + throw new IllegalStateException("channel already closed!"); + } + ActionListener listener = closeListener.get(); + if (listener != null) { + listener.onResponse(null); + } + } + + @Override + public boolean isOpen() { + return open.get(); + } + + @Override + public void addCloseListener(ActionListener listener) { + if (closeListener.compareAndSet(null, listener) == false) { + throw new IllegalStateException("close listener already set, only one is allowed!"); + } + } + } +} From f6d748923dfa69d17907fbf0bec06f0ce1436b3f Mon Sep 17 00:00:00 2001 From: Luca Cavanna Date: Thu, 25 Jul 2019 16:48:49 +0200 Subject: [PATCH 10/28] wip --- .../action/search/HttpChannelTaskHandler.java | 25 ++++++++++--------- .../search/HttpChannelTaskHandlerTests.java | 11 ++++---- 2 files changed, 18 insertions(+), 18 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandler.java b/server/src/main/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandler.java index 40cc6b8484d41..152d3e88daaf6 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandler.java +++ b/server/src/main/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandler.java @@ -35,7 +35,7 @@ import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CopyOnWriteArraySet; -import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; /** @@ -49,8 +49,8 @@ final class HttpChannelTaskHandler { void execute(NodeClient client, HttpChannel httpChannel, ActionRequest request, ActionType actionType, ActionListener listener) { - - AtomicBoolean linkEnabled = new AtomicBoolean(true); + //0: initial state, 1: either linked or already unlinked, 2: linked and unlinked + AtomicInteger link = new AtomicInteger(0); Task task = client.executeLocally(actionType, request, new TaskListener<>() { @Override @@ -72,16 +72,16 @@ public void onFailure(Task task, Throwable e) { private void unlink(Task task) { //the synchronized blocks are to make sure that only link or unlink for a specific task can happen at a given time, - //they can't happen concurrently. The flag is needed because unlink can still be called before link, which would + //they can't happen concurrently. The link flag is needed because unlink can still be called before link, which would //lead to piling up task ids that are never removed from the map. - //It may look like only either synchronized or the boolean flag are needed but they both are. In fact, the boolean flag - //is needed to ensure that we don't link a task if we have already unlinked it. But it's not enough as, once we start - //the linking, we do want its corresponding unlinking to happen, but only once the linking is completed. With only - //the boolean flag, we would just miss unlinking for some tasks that are being linked when onResponse is called. + //It may look like only either synchronized or the flag are needed but they both are. In fact, the flag is needed to + //ensure that we don't link a task if we have already unlinked it. But it's not enough as, once we start the linking, + //we do want its corresponding unlinking to happen, but only once the linking is completed. With only + //the flag, we would just miss unlinking for some tasks that are being linked while onResponse is called. synchronized(task) { try { - //nothing to do if link was not called yet: it would not find the task anyways. - if (linkEnabled.compareAndSet(false, true)) { + //nothing to do if link was not called yet: we would not find the task anyways. + if (link.getAndIncrement() > 0) { CloseListener closeListener = httpChannels.get(httpChannel); TaskId taskId = new TaskId(client.getLocalNodeId(), task.getId()); closeListener.unregisterTask(taskId); @@ -95,8 +95,9 @@ private void unlink(Task task) { CloseListener closeListener = httpChannels.computeIfAbsent(httpChannel, channel -> new CloseListener(client)); synchronized (task) { - //make sure that the link is made only if the task is not already completed, otherwise unlink would have already been called - if (linkEnabled.compareAndSet(true, false)) { + //the task will only be registered if it's not completed yet, meaning if its TaskListener has not been called yet. + //otherwise, given that its listener has already been called, the task id would never be removed. + if (link.getAndIncrement() == 0) { TaskId taskId = new TaskId(client.getLocalNodeId(), task.getId()); closeListener.registerTask(httpChannel, taskId); } diff --git a/server/src/test/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandlerTests.java b/server/src/test/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandlerTests.java index 8eb756cbd91bc..27c8bfb1e358b 100644 --- a/server/src/test/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandlerTests.java +++ b/server/src/test/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandlerTests.java @@ -50,6 +50,8 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; +import static org.hamcrest.Matchers.lessThanOrEqualTo; + public class HttpChannelTaskHandlerTests extends ESTestCase { private ThreadPool threadPool; @@ -69,12 +71,9 @@ public void testLinkAndUnlink() throws Exception { try (TestClient testClient = new TestClient(Settings.EMPTY, threadPool)) { HttpChannelTaskHandler httpChannelTaskHandler = new HttpChannelTaskHandler(); List> futures = new ArrayList<>(); - int numChannels = 1;//randomIntBetween(1, 30); - System.out.println("channels " + numChannels); - + int numChannels = randomIntBetween(1, 30); for (int i = 0; i < numChannels; i++) { - int numTasks = 3;//randomIntBetween(1, 30); - System.out.println("tasks " + numTasks); + int numTasks = randomIntBetween(1, 30); TestHttpChannel channel = new TestHttpChannel(); for (int j = 0; j < numTasks; j++) { PlainListenableActionFuture actionFuture = PlainListenableActionFuture.newListenableFuture(); @@ -88,7 +87,7 @@ public void testLinkAndUnlink() throws Exception { future.get(); } //no channels get closed in this test - assertEquals(numChannels, httpChannelTaskHandler.httpChannels.size()); + assertThat(httpChannelTaskHandler.httpChannels.size(), lessThanOrEqualTo(numChannels)); for (Map.Entry entry : httpChannelTaskHandler.httpChannels.entrySet()) { assertEquals(0, entry.getValue().taskIds.size()); } From d581ab65d71712fe6e6bb30f492191f298750f39 Mon Sep 17 00:00:00 2001 From: Luca Cavanna Date: Thu, 25 Jul 2019 17:00:12 +0200 Subject: [PATCH 11/28] wip --- .../rest/action/search/HttpChannelTaskHandler.java | 4 ++-- .../rest/action/search/HttpChannelTaskHandlerTests.java | 5 +---- 2 files changed, 3 insertions(+), 6 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandler.java b/server/src/main/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandler.java index 152d3e88daaf6..d1df0e441c04b 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandler.java +++ b/server/src/main/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandler.java @@ -49,7 +49,8 @@ final class HttpChannelTaskHandler { void execute(NodeClient client, HttpChannel httpChannel, ActionRequest request, ActionType actionType, ActionListener listener) { - //0: initial state, 1: either linked or already unlinked, 2: linked and unlinked + //0: initial state, 1: either linked or already unlinked without being linked first, 2: first linked and then unlinked + //link can only be done if it's the first thing that happens. unlink will only happen if link was done first. AtomicInteger link = new AtomicInteger(0); Task task = client.executeLocally(actionType, request, new TaskListener<>() { @@ -105,7 +106,6 @@ private void unlink(Task task) { //TODO test case where listener is registered, but no tasks have been added yet: // - connection gets closed, channel will be removed, no tasks will be cancelled - // - unlink is called before, hence the task is removed (not found) before it gets added //TODO check that no tasks are left behind through assertions at node close } diff --git a/server/src/test/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandlerTests.java b/server/src/test/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandlerTests.java index 27c8bfb1e358b..e9f1984374167 100644 --- a/server/src/test/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandlerTests.java +++ b/server/src/test/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandlerTests.java @@ -50,8 +50,6 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; -import static org.hamcrest.Matchers.lessThanOrEqualTo; - public class HttpChannelTaskHandlerTests extends ESTestCase { private ThreadPool threadPool; @@ -82,12 +80,11 @@ public void testLinkAndUnlink() throws Exception { futures.add(actionFuture); } } - for (Future future : futures) { future.get(); } //no channels get closed in this test - assertThat(httpChannelTaskHandler.httpChannels.size(), lessThanOrEqualTo(numChannels)); + assertEquals(numChannels, httpChannelTaskHandler.httpChannels.size()); for (Map.Entry entry : httpChannelTaskHandler.httpChannels.entrySet()) { assertEquals(0, entry.getValue().taskIds.size()); } From 40ae005dffaf8a191279b245c3e44dba52c0778f Mon Sep 17 00:00:00 2001 From: Luca Cavanna Date: Mon, 29 Jul 2019 16:23:42 +0200 Subject: [PATCH 12/28] add more tests and remove some TODO --- .../action/search/HttpChannelTaskHandler.java | 50 ++++---- .../search/HttpChannelTaskHandlerTests.java | 114 ++++++++++++++++-- 2 files changed, 131 insertions(+), 33 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandler.java b/server/src/main/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandler.java index d1df0e441c04b..0dec11af94f36 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandler.java +++ b/server/src/main/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandler.java @@ -66,8 +66,7 @@ public void onFailure(Task task, Throwable e) { if (e instanceof Exception) { listener.onFailure((Exception)e); } else { - //TODO should we rather throw in case of throwable instead of notifying the listener? - listener.onFailure(new RuntimeException(e)); + throw new RuntimeException(e); } } @@ -104,10 +103,8 @@ private void unlink(Task task) { } } - //TODO test case where listener is registered, but no tasks have been added yet: - // - connection gets closed, channel will be removed, no tasks will be cancelled - - //TODO check that no tasks are left behind through assertions at node close + //TODO check that no tasks are left behind through assertions at node close. Not sure how. Couldn't there be in-flight requests + //causing channels to be in the map when a node gets closed? } final class CloseListener implements ActionListener { @@ -122,8 +119,9 @@ final class CloseListener implements ActionListener { void registerTask(HttpChannel httpChannel, TaskId taskId) { if (channel.compareAndSet(null, httpChannel)) { //In case the channel is already closed when we register the listener, the listener will be immediately executed which will - //remove the channel from the map straight-away. That is why we do this in two stages. If we provided the channel at close - //listener initialization we would have to deal with close listeners calls before the channel is in the map. + //remove the channel from the map straight-away. That is why we first create the CloseListener and later we associate it + //with the channel. This guarantees that the close listener is already in the map when the it gets registered to its + //corresponding channel, hence it is always found in the map when it gets invoked if the channel gets closed. httpChannel.addCloseListener(this); } this.taskIds.add(taskId); @@ -135,24 +133,32 @@ private void unregisterTask(TaskId taskId) { @Override public void onResponse(Void aVoid) { - //When the channel gets closed it won't be reused: we can remove it from the map as there is no chance we will - //register another close listener to it later. - //The channel reference may be null, if the connection gets closed before we set it. - //The channel must be found in the map though as this listener gets registered after the channel is added. - //TODO test channel null here? it can happen! - httpChannels.remove(channel.get()); - for (TaskId previousTaskId : taskIds) { - CancelTasksRequest cancelTasksRequest = new CancelTasksRequest(); - cancelTasksRequest.setTaskId(previousTaskId); - //We don't wait for cancel tasks to come back. Task cancellation is just best effort. - //Note that cancel tasks fails if the user sending the search request does not have the permissions to call it. - client.admin().cluster().cancelTasks(cancelTasksRequest, ActionListener.wrap(r -> {}, e -> {})); - } + cancelTasks(); } @Override public void onFailure(Exception e) { - //nothing to do here + cancelTasks(); + } + + private void cancelTasks() { + //When the channel gets closed it won't be reused: we can remove it from the map as there is no chance we will + //register another close listener to it later. + //The channel reference may be null, if the connection gets closed before it got set. + HttpChannel channel = this.channel.get(); + //TODO is this enough to make sure that we only cancel tasks once? it could be that not all tasks have been registered yet + //when the close listener is notified. We remove the channel and cancel the tasks that are known up until then. + //if new tasks come in from the same channel, the channel will be added again to the map, but the close listener will + //be registered another time to it which is no good. tasks should not be left behind though. + if (channel != null && httpChannels.remove(channel) != null) { + for (TaskId previousTaskId : taskIds) { + CancelTasksRequest cancelTasksRequest = new CancelTasksRequest(); + cancelTasksRequest.setTaskId(previousTaskId); + //We don't wait for cancel tasks to come back. Task cancellation is just best effort. + //Note that cancel tasks fails if the user sending the search request does not have the permissions to call it. + client.admin().cluster().cancelTasks(cancelTasksRequest, ActionListener.wrap(r -> {}, e -> {})); + } + } } } } diff --git a/server/src/test/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandlerTests.java b/server/src/test/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandlerTests.java index e9f1984374167..b89cbae13313b 100644 --- a/server/src/test/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandlerTests.java +++ b/server/src/test/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandlerTests.java @@ -23,6 +23,8 @@ import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionResponse; import org.elasticsearch.action.ActionType; +import org.elasticsearch.action.admin.cluster.node.tasks.cancel.CancelTasksAction; +import org.elasticsearch.action.admin.cluster.node.tasks.cancel.CancelTasksRequest; import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchResponse; @@ -32,6 +34,7 @@ import org.elasticsearch.http.HttpChannel; import org.elasticsearch.http.HttpResponse; import org.elasticsearch.tasks.Task; +import org.elasticsearch.tasks.TaskId; import org.elasticsearch.tasks.TaskListener; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.threadpool.TestThreadPool; @@ -44,6 +47,9 @@ import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.Set; +import java.util.concurrent.CopyOnWriteArraySet; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; @@ -64,8 +70,12 @@ public void stopThreadPool() { ThreadPool.terminate(threadPool, 5, TimeUnit.SECONDS); } + /** + * This test verifies that no tasks are left in the map where channels and their corresponding tasks are tracked. + * Through the {@link TestClient} we simulate a scenario where the task listener can be called before the task has been + * associated with its channel. Either way, we need to make sure that no tasks are left in the map. + */ public void testLinkAndUnlink() throws Exception { - try (TestClient testClient = new TestClient(Settings.EMPTY, threadPool)) { HttpChannelTaskHandler httpChannelTaskHandler = new HttpChannelTaskHandler(); List> futures = new ArrayList<>(); @@ -91,12 +101,56 @@ public void testLinkAndUnlink() throws Exception { } } - //TODO verify that we do not add stuff to the map when the channel is closed at link call - //same when unlink is called before link + public void testChannelClose() throws Exception { + try (TestClient testClient = new TestClient(Settings.EMPTY, threadPool)) { + testClient.timeout.set(true); + HttpChannelTaskHandler httpChannelTaskHandler = new HttpChannelTaskHandler(); + int numChannels = randomIntBetween(1, 30); + int totalTasks = 0; + List channels = new ArrayList<>(numChannels); + for (int i = 0; i < numChannels; i++) { + TestHttpChannel channel = new TestHttpChannel(); + channels.add(channel); + int numTasks = randomIntBetween(1, 30); + totalTasks += numTasks; + for (int j = 0; j < numTasks; j++) { + httpChannelTaskHandler.execute(testClient, channel, new SearchRequest(), SearchAction.INSTANCE, null); + } + assertEquals(numTasks, httpChannelTaskHandler.httpChannels.get(channel).taskIds.size()); + } + assertEquals(numChannels, httpChannelTaskHandler.httpChannels.size()); + for (TestHttpChannel channel : channels) { + channel.awaitClose(); + } + assertEquals(0, httpChannelTaskHandler.httpChannels.size()); + assertEquals(totalTasks, testClient.cancelledTasks.size()); + } + } - private static class TestClient extends NodeClient { + public void testChannelAlreadyClosed() { + try (TestClient testClient = new TestClient(Settings.EMPTY, threadPool)) { + testClient.timeout.set(true); + int numChannels = randomIntBetween(1, 30); + HttpChannelTaskHandler httpChannelTaskHandler = new HttpChannelTaskHandler(); + for (int i = 0; i < numChannels; i++) { + TestHttpChannel channel = new TestHttpChannel(); + //no need to wait here, there will be no close listener registered, nothing to wait for. + channel.close(); + //here the channel will be first registered, then straight-away removed from the map as the close listener is invoked + //TODO is it possible that more tasks are started from a closed channel? In that case we would end up registering the close + //listener multiple times as the channel is unknown + httpChannelTaskHandler.execute(testClient, channel, new SearchRequest(), SearchAction.INSTANCE, null); + } + assertEquals(0, httpChannelTaskHandler.httpChannels.size()); + assertEquals(0, testClient.cancelledTasks.size()); + } + } + + private static class TestClient extends NodeClient { private final AtomicLong counter = new AtomicLong(0); + private final AtomicBoolean timeout = new AtomicBoolean(false); + private final Set cancelledTasks = new CopyOnWriteArraySet<>(); TestClient(Settings settings, ThreadPool threadPool) { super(settings, threadPool); @@ -106,12 +160,34 @@ private static class TestClient extends NodeClient { public Task executeLocally(ActionType action, Request request, TaskListener listener) { - Task task = request.createTask(counter.getAndIncrement(), "type", action.name(), null, Collections.emptyMap()); - if (rarely()) { - listener.onResponse(task, null); + assert action == SearchAction.INSTANCE; + Task task = request.createTask(counter.getAndIncrement(), "search", action.name(), null, Collections.emptyMap()); + if (timeout.get() == false) { + if (rarely()) { + //make sure that search is sometimes also called from the same thread before the task is returned + listener.onResponse(task, null); + } else { + threadPool().generic().submit(() -> listener.onResponse(task, null)); + } + } + return task; + } + + @Override + public Task executeLocally(ActionType action, + Request request, + ActionListener listener) { + assert action == CancelTasksAction.INSTANCE; + CancelTasksRequest cancelTasksRequest = (CancelTasksRequest) request; + assertTrue("tried to cancel the same task more than once", cancelledTasks.add(cancelTasksRequest.getTaskId())); + Task task = request.createTask(counter.getAndIncrement(), "cancel_task", action.name(), null, Collections.emptyMap()); + if (randomBoolean()) { + listener.onResponse(null); } else { - threadPool().generic().submit(() -> listener.onResponse(task, null)); + //test that cancel tasks is best effort, failure received are not propagated + listener.onFailure(new IllegalStateException()); } + return task; } @@ -121,13 +197,13 @@ public String getLocalNodeId() { } } - private static class TestHttpChannel implements HttpChannel { + private class TestHttpChannel implements HttpChannel { private final AtomicBoolean open = new AtomicBoolean(true); private final AtomicReference> closeListener = new AtomicReference<>(); + private final CountDownLatch closeLatch = new CountDownLatch(1); @Override public void sendResponse(HttpResponse response, ActionListener listener) { - } @Override @@ -147,10 +223,23 @@ public void close() { } ActionListener listener = closeListener.get(); if (listener != null) { - listener.onResponse(null); + boolean failure = randomBoolean(); + threadPool.generic().submit(() -> { + if (failure) { + listener.onFailure(new IllegalStateException()); + } else { + listener.onResponse(null); + } + closeLatch.countDown(); + }); } } + private void awaitClose() throws InterruptedException { + close(); + closeLatch.await(); + } + @Override public boolean isOpen() { return open.get(); @@ -161,6 +250,9 @@ public void addCloseListener(ActionListener listener) { if (closeListener.compareAndSet(null, listener) == false) { throw new IllegalStateException("close listener already set, only one is allowed!"); } + if (open.get() == false) { + listener.onResponse(null); + } } } } From 468abb88e04419587cf7cee484aed2f75cba04aa Mon Sep 17 00:00:00 2001 From: Luca Cavanna Date: Mon, 29 Jul 2019 16:55:44 +0200 Subject: [PATCH 13/28] adapt --- .../rest/action/search/HttpChannelTaskHandler.java | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandler.java b/server/src/main/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandler.java index 0dec11af94f36..20e3f6772ecc5 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandler.java +++ b/server/src/main/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandler.java @@ -61,13 +61,9 @@ public void onResponse(Task task, Response searchResponse) { } @Override - public void onFailure(Task task, Throwable e) { + public void onFailure(Task task, Exception e) { unlink(task); - if (e instanceof Exception) { - listener.onFailure((Exception)e); - } else { - throw new RuntimeException(e); - } + listener.onFailure(e); } private void unlink(Task task) { From b022f45f743d6084f2cc738f9d288bc9e5b2417b Mon Sep 17 00:00:00 2001 From: Luca Cavanna Date: Tue, 30 Jul 2019 17:28:07 +0200 Subject: [PATCH 14/28] iter --- .../action/search/HttpChannelTaskHandler.java | 114 ++++++++---------- .../search/HttpChannelTaskHandlerTests.java | 54 ++++----- 2 files changed, 73 insertions(+), 95 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandler.java b/server/src/main/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandler.java index 20e3f6772ecc5..6862c85fad241 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandler.java +++ b/server/src/main/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandler.java @@ -29,14 +29,11 @@ import org.elasticsearch.http.HttpChannel; import org.elasticsearch.tasks.Task; import org.elasticsearch.tasks.TaskId; -import org.elasticsearch.tasks.TaskListener; +import java.util.HashSet; import java.util.Map; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.CopyOnWriteArraySet; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicReference; /** * This class keeps track of which tasks came in from which {@link HttpChannel}, by allowing to associate @@ -49,55 +46,31 @@ final class HttpChannelTaskHandler { void execute(NodeClient client, HttpChannel httpChannel, ActionRequest request, ActionType actionType, ActionListener listener) { - //0: initial state, 1: either linked or already unlinked without being linked first, 2: first linked and then unlinked - //link can only be done if it's the first thing that happens. unlink will only happen if link was done first. - AtomicInteger link = new AtomicInteger(0); + + CloseListener closeListener = httpChannels.computeIfAbsent(httpChannel, channel -> new CloseListener(client)); + TaskHolder taskHolder = new TaskHolder(); Task task = client.executeLocally(actionType, request, - new TaskListener<>() { + new ActionListener<>() { @Override - public void onResponse(Task task, Response searchResponse) { - unlink(task); - listener.onResponse(searchResponse); + public void onResponse(Response searchResponse) { + try { + closeListener.unregisterTask(taskHolder); + } finally { + listener.onResponse(searchResponse); + } } @Override - public void onFailure(Task task, Exception e) { - unlink(task); - listener.onFailure(e); - } - - private void unlink(Task task) { - //the synchronized blocks are to make sure that only link or unlink for a specific task can happen at a given time, - //they can't happen concurrently. The link flag is needed because unlink can still be called before link, which would - //lead to piling up task ids that are never removed from the map. - //It may look like only either synchronized or the flag are needed but they both are. In fact, the flag is needed to - //ensure that we don't link a task if we have already unlinked it. But it's not enough as, once we start the linking, - //we do want its corresponding unlinking to happen, but only once the linking is completed. With only - //the flag, we would just miss unlinking for some tasks that are being linked while onResponse is called. - synchronized(task) { - try { - //nothing to do if link was not called yet: we would not find the task anyways. - if (link.getAndIncrement() > 0) { - CloseListener closeListener = httpChannels.get(httpChannel); - TaskId taskId = new TaskId(client.getLocalNodeId(), task.getId()); - closeListener.unregisterTask(taskId); - } - } catch(Exception e) { - listener.onFailure(e); - } + public void onFailure(Exception e) { + try { + closeListener.unregisterTask(taskHolder); + } finally { + listener.onFailure(e); } } }); - CloseListener closeListener = httpChannels.computeIfAbsent(httpChannel, channel -> new CloseListener(client)); - synchronized (task) { - //the task will only be registered if it's not completed yet, meaning if its TaskListener has not been called yet. - //otherwise, given that its listener has already been called, the task id would never be removed. - if (link.getAndIncrement() == 0) { - TaskId taskId = new TaskId(client.getLocalNodeId(), task.getId()); - closeListener.registerTask(httpChannel, taskId); - } - } + closeListener.registerTask(httpChannel, taskHolder, new TaskId(client.getLocalNodeId(), task.getId())); //TODO check that no tasks are left behind through assertions at node close. Not sure how. Couldn't there be in-flight requests //causing channels to be in the map when a node gets closed? @@ -105,48 +78,45 @@ private void unlink(Task task) { final class CloseListener implements ActionListener { private final Client client; - final Set taskIds = new CopyOnWriteArraySet<>(); - private final AtomicReference channel = new AtomicReference<>(); + final Set taskIds = new HashSet<>(); + private HttpChannel channel; CloseListener(Client client) { this.client = client; } - void registerTask(HttpChannel httpChannel, TaskId taskId) { - if (channel.compareAndSet(null, httpChannel)) { + synchronized void registerTask(HttpChannel httpChannel, TaskHolder taskHolder, TaskId taskId) { + if (channel == null) { + channel = httpChannel; //In case the channel is already closed when we register the listener, the listener will be immediately executed which will //remove the channel from the map straight-away. That is why we first create the CloseListener and later we associate it //with the channel. This guarantees that the close listener is already in the map when the it gets registered to its //corresponding channel, hence it is always found in the map when it gets invoked if the channel gets closed. httpChannel.addCloseListener(this); } - this.taskIds.add(taskId); - } - - private void unregisterTask(TaskId taskId) { - this.taskIds.remove(taskId); + taskHolder.taskId = taskId; + if (taskHolder.completed == false) { + this.taskIds.add(taskId); + } } - @Override - public void onResponse(Void aVoid) { - cancelTasks(); + private synchronized void unregisterTask(TaskHolder taskHolder) { + if (taskHolder.taskId != null) { + this.taskIds.remove(taskHolder.taskId); + } + taskHolder.completed = true; } @Override - public void onFailure(Exception e) { - cancelTasks(); - } - - private void cancelTasks() { + public synchronized void onResponse(Void aVoid) { //When the channel gets closed it won't be reused: we can remove it from the map as there is no chance we will //register another close listener to it later. //The channel reference may be null, if the connection gets closed before it got set. - HttpChannel channel = this.channel.get(); - //TODO is this enough to make sure that we only cancel tasks once? it could be that not all tasks have been registered yet - //when the close listener is notified. We remove the channel and cancel the tasks that are known up until then. - //if new tasks come in from the same channel, the channel will be added again to the map, but the close listener will - //be registered another time to it which is no good. tasks should not be left behind though. - if (channel != null && httpChannels.remove(channel) != null) { + //TODO Could it be that not all tasks have been registered yet when the close listener is notified. We remove the channel + // and cancel the tasks that are known up until then. if new tasks come in from the same channel, the channel will be added + // again to the map, but the close listener will be registered another time to it which is not good. + if (channel != null) { + httpChannels.remove(channel); for (TaskId previousTaskId : taskIds) { CancelTasksRequest cancelTasksRequest = new CancelTasksRequest(); cancelTasksRequest.setTaskId(previousTaskId); @@ -156,5 +126,15 @@ private void cancelTasks() { } } } + + @Override + public void onFailure(Exception e) { + onResponse(null); + } + } + + private static class TaskHolder { + private TaskId taskId; + private boolean completed = false; } } diff --git a/server/src/test/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandlerTests.java b/server/src/test/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandlerTests.java index b89cbae13313b..27f7c1903ddd8 100644 --- a/server/src/test/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandlerTests.java +++ b/server/src/test/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandlerTests.java @@ -35,7 +35,6 @@ import org.elasticsearch.http.HttpResponse; import org.elasticsearch.tasks.Task; import org.elasticsearch.tasks.TaskId; -import org.elasticsearch.tasks.TaskListener; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; @@ -156,39 +155,38 @@ private static class TestClient extends NodeClient { super(settings, threadPool); } - @Override - public Task executeLocally(ActionType action, - Request request, - TaskListener listener) { - assert action == SearchAction.INSTANCE; - Task task = request.createTask(counter.getAndIncrement(), "search", action.name(), null, Collections.emptyMap()); - if (timeout.get() == false) { - if (rarely()) { - //make sure that search is sometimes also called from the same thread before the task is returned - listener.onResponse(task, null); - } else { - threadPool().generic().submit(() -> listener.onResponse(task, null)); - } - } - return task; - } - @Override public Task executeLocally(ActionType action, Request request, ActionListener listener) { - assert action == CancelTasksAction.INSTANCE; - CancelTasksRequest cancelTasksRequest = (CancelTasksRequest) request; - assertTrue("tried to cancel the same task more than once", cancelledTasks.add(cancelTasksRequest.getTaskId())); - Task task = request.createTask(counter.getAndIncrement(), "cancel_task", action.name(), null, Collections.emptyMap()); - if (randomBoolean()) { - listener.onResponse(null); - } else { - //test that cancel tasks is best effort, failure received are not propagated - listener.onFailure(new IllegalStateException()); + switch(action.name()) { + case CancelTasksAction.NAME: + CancelTasksRequest cancelTasksRequest = (CancelTasksRequest) request; + assertTrue("tried to cancel the same task more than once", cancelledTasks.add(cancelTasksRequest.getTaskId())); + Task task = request.createTask(counter.getAndIncrement(), "cancel_task", action.name(), null, Collections.emptyMap()); + if (randomBoolean()) { + listener.onResponse(null); + } else { + //test that cancel tasks is best effort, failure received are not propagated + listener.onFailure(new IllegalStateException()); + } + + return task; + case SearchAction.NAME: + Task searchTask = request.createTask(counter.getAndIncrement(), "search", action.name(), null, Collections.emptyMap()); + if (timeout.get() == false) { + if (rarely()) { + //make sure that search is sometimes also called from the same thread before the task is returned + listener.onResponse(null); + } else { + threadPool().generic().submit(() -> listener.onResponse(null)); + } + } + return searchTask; + default: + throw new UnsupportedOperationException(); } - return task; } @Override From 2e6e0a24c93a1d6f8c503775875187dd547d6c33 Mon Sep 17 00:00:00 2001 From: Luca Cavanna Date: Tue, 30 Jul 2019 17:29:24 +0200 Subject: [PATCH 15/28] iter --- .../rest/action/search/HttpChannelTaskHandler.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandler.java b/server/src/main/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandler.java index 6862c85fad241..47da81089fe38 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandler.java +++ b/server/src/main/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandler.java @@ -37,9 +37,7 @@ /** * This class keeps track of which tasks came in from which {@link HttpChannel}, by allowing to associate - * an {@link HttpChannel} with a {@link TaskId}, and also removing the link once the task is complete. - * Additionally, it accepts a consumer that gets called whenever an http channel gets closed, which - * can be used to cancel the associated task when the underlying connection gets closed. + * an {@link HttpChannel} with a {@link TaskId}, and also to unregister the task once it's complete. */ final class HttpChannelTaskHandler { final Map httpChannels = new ConcurrentHashMap<>(); From 21f897f0b15bb29d0fa83f5f59454cf59a3bbd4b Mon Sep 17 00:00:00 2001 From: Luca Cavanna Date: Wed, 31 Jul 2019 12:43:48 +0200 Subject: [PATCH 16/28] move register channel to a separate method, that does not require locking --- .../action/search/HttpChannelTaskHandler.java | 37 ++++++++++--------- 1 file changed, 19 insertions(+), 18 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandler.java b/server/src/main/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandler.java index 47da81089fe38..2b84d67067804 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandler.java +++ b/server/src/main/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandler.java @@ -34,10 +34,11 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicReference; /** - * This class keeps track of which tasks came in from which {@link HttpChannel}, by allowing to associate - * an {@link HttpChannel} with a {@link TaskId}, and also to unregister the task once it's complete. + * This class executes a request and associates the corresponding {@link Task} with the {@link HttpChannel} that it was originated from, + * so that the tasks associated with a certain channel get cancelled when the underlying connection gets closed. */ final class HttpChannelTaskHandler { final Map httpChannels = new ConcurrentHashMap<>(); @@ -46,6 +47,7 @@ void execute(NodeClient client, HttpChannel ht ActionType actionType, ActionListener listener) { CloseListener closeListener = httpChannels.computeIfAbsent(httpChannel, channel -> new CloseListener(client)); + closeListener.maybeRegisterChannel(httpChannel); TaskHolder taskHolder = new TaskHolder(); Task task = client.executeLocally(actionType, request, new ActionListener<>() { @@ -68,7 +70,7 @@ public void onFailure(Exception e) { } }); - closeListener.registerTask(httpChannel, taskHolder, new TaskId(client.getLocalNodeId(), task.getId())); + closeListener.registerTask(taskHolder, new TaskId(client.getLocalNodeId(), task.getId())); //TODO check that no tasks are left behind through assertions at node close. Not sure how. Couldn't there be in-flight requests //causing channels to be in the map when a node gets closed? @@ -76,29 +78,31 @@ public void onFailure(Exception e) { final class CloseListener implements ActionListener { private final Client client; + private final AtomicReference channel = new AtomicReference<>(); final Set taskIds = new HashSet<>(); - private HttpChannel channel; CloseListener(Client client) { this.client = client; } - synchronized void registerTask(HttpChannel httpChannel, TaskHolder taskHolder, TaskId taskId) { - if (channel == null) { - channel = httpChannel; + void maybeRegisterChannel(HttpChannel httpChannel) { + if (channel.compareAndSet(null, httpChannel)) { //In case the channel is already closed when we register the listener, the listener will be immediately executed which will //remove the channel from the map straight-away. That is why we first create the CloseListener and later we associate it //with the channel. This guarantees that the close listener is already in the map when the it gets registered to its //corresponding channel, hence it is always found in the map when it gets invoked if the channel gets closed. httpChannel.addCloseListener(this); } + } + + synchronized void registerTask(TaskHolder taskHolder, TaskId taskId) { taskHolder.taskId = taskId; if (taskHolder.completed == false) { this.taskIds.add(taskId); } } - private synchronized void unregisterTask(TaskHolder taskHolder) { + synchronized void unregisterTask(TaskHolder taskHolder) { if (taskHolder.taskId != null) { this.taskIds.remove(taskHolder.taskId); } @@ -109,19 +113,16 @@ private synchronized void unregisterTask(TaskHolder taskHolder) { public synchronized void onResponse(Void aVoid) { //When the channel gets closed it won't be reused: we can remove it from the map as there is no chance we will //register another close listener to it later. - //The channel reference may be null, if the connection gets closed before it got set. //TODO Could it be that not all tasks have been registered yet when the close listener is notified. We remove the channel // and cancel the tasks that are known up until then. if new tasks come in from the same channel, the channel will be added // again to the map, but the close listener will be registered another time to it which is not good. - if (channel != null) { - httpChannels.remove(channel); - for (TaskId previousTaskId : taskIds) { - CancelTasksRequest cancelTasksRequest = new CancelTasksRequest(); - cancelTasksRequest.setTaskId(previousTaskId); - //We don't wait for cancel tasks to come back. Task cancellation is just best effort. - //Note that cancel tasks fails if the user sending the search request does not have the permissions to call it. - client.admin().cluster().cancelTasks(cancelTasksRequest, ActionListener.wrap(r -> {}, e -> {})); - } + httpChannels.remove(channel.get()); + for (TaskId previousTaskId : taskIds) { + CancelTasksRequest cancelTasksRequest = new CancelTasksRequest(); + cancelTasksRequest.setTaskId(previousTaskId); + //We don't wait for cancel tasks to come back. Task cancellation is just best effort. + //Note that cancel tasks fails if the user sending the search request does not have the permissions to call it. + client.admin().cluster().cancelTasks(cancelTasksRequest, ActionListener.wrap(r -> {}, e -> {})); } } From 7b9621c804102b3ab34a2b884136cf6df9ae454f Mon Sep 17 00:00:00 2001 From: Luca Cavanna Date: Thu, 1 Aug 2019 15:22:42 +0200 Subject: [PATCH 17/28] make handler static and harden tests --- .../action/search/HttpChannelTaskHandler.java | 27 +++--- .../rest/action/search/RestSearchAction.java | 4 +- .../search/HttpChannelTaskHandlerTests.java | 88 ++++++++++++------- .../elasticsearch/test/ESIntegTestCase.java | 3 + 4 files changed, 77 insertions(+), 45 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandler.java b/server/src/main/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandler.java index 2b84d67067804..d41cf7786461b 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandler.java +++ b/server/src/main/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandler.java @@ -40,14 +40,23 @@ * This class executes a request and associates the corresponding {@link Task} with the {@link HttpChannel} that it was originated from, * so that the tasks associated with a certain channel get cancelled when the underlying connection gets closed. */ -final class HttpChannelTaskHandler { +public final class HttpChannelTaskHandler { + + private static final HttpChannelTaskHandler INSTANCE = new HttpChannelTaskHandler(); + final Map httpChannels = new ConcurrentHashMap<>(); + private HttpChannelTaskHandler() { + } + + public static HttpChannelTaskHandler get() { + return INSTANCE; + } + void execute(NodeClient client, HttpChannel httpChannel, ActionRequest request, ActionType actionType, ActionListener listener) { CloseListener closeListener = httpChannels.computeIfAbsent(httpChannel, channel -> new CloseListener(client)); - closeListener.maybeRegisterChannel(httpChannel); TaskHolder taskHolder = new TaskHolder(); Task task = client.executeLocally(actionType, request, new ActionListener<>() { @@ -69,11 +78,12 @@ public void onFailure(Exception e) { } } }); - closeListener.registerTask(taskHolder, new TaskId(client.getLocalNodeId(), task.getId())); + closeListener.maybeRegisterChannel(httpChannel); + } - //TODO check that no tasks are left behind through assertions at node close. Not sure how. Couldn't there be in-flight requests - //causing channels to be in the map when a node gets closed? + public int getNumChannels() { + return httpChannels.size(); } final class CloseListener implements ActionListener { @@ -111,13 +121,10 @@ synchronized void unregisterTask(TaskHolder taskHolder) { @Override public synchronized void onResponse(Void aVoid) { - //When the channel gets closed it won't be reused: we can remove it from the map as there is no chance we will - //register another close listener to it later. - //TODO Could it be that not all tasks have been registered yet when the close listener is notified. We remove the channel - // and cancel the tasks that are known up until then. if new tasks come in from the same channel, the channel will be added - // again to the map, but the close listener will be registered another time to it which is not good. + //When the channel gets closed it won't be reused: we can remove it from the map and forget about it. httpChannels.remove(channel.get()); for (TaskId previousTaskId : taskIds) { + //TODO what thread context should this be run on? CancelTasksRequest cancelTasksRequest = new CancelTasksRequest(); cancelTasksRequest.setTaskId(previousTaskId); //We don't wait for cancel tasks to come back. Task cancellation is just best effort. diff --git a/server/src/main/java/org/elasticsearch/rest/action/search/RestSearchAction.java b/server/src/main/java/org/elasticsearch/rest/action/search/RestSearchAction.java index 190dbfc9da9ea..1af77a42288de 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/search/RestSearchAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/search/RestSearchAction.java @@ -64,8 +64,6 @@ public class RestSearchAction extends BaseRestHandler { public static final String TYPED_KEYS_PARAM = "typed_keys"; private static final Set RESPONSE_PARAMS; - private final HttpChannelTaskHandler httpChannelTaskHandler = new HttpChannelTaskHandler(); - static { final Set responseParams = new HashSet<>(Arrays.asList(TYPED_KEYS_PARAM, TOTAL_HITS_AS_INT_PARAM)); RESPONSE_PARAMS = Collections.unmodifiableSet(responseParams); @@ -105,7 +103,7 @@ public RestChannelConsumer prepareRequest(final RestRequest request, final NodeC return channel -> { RestStatusToXContentListener listener = new RestStatusToXContentListener<>(channel); - httpChannelTaskHandler.execute(client, request.getHttpChannel(), searchRequest, SearchAction.INSTANCE, listener); + HttpChannelTaskHandler.get().execute(client, request.getHttpChannel(), searchRequest, SearchAction.INSTANCE, listener); }; } diff --git a/server/src/test/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandlerTests.java b/server/src/test/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandlerTests.java index 27f7c1903ddd8..725a210ebf206 100644 --- a/server/src/test/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandlerTests.java +++ b/server/src/test/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandlerTests.java @@ -52,6 +52,7 @@ import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; @@ -71,17 +72,20 @@ public void stopThreadPool() { /** * This test verifies that no tasks are left in the map where channels and their corresponding tasks are tracked. - * Through the {@link TestClient} we simulate a scenario where the task listener can be called before the task has been - * associated with its channel. Either way, we need to make sure that no tasks are left in the map. + * Through the {@link TestClient} we simulate a scenario where the task may complete even before it has been + * associated with its corresponding channel. Either way, we need to make sure that no tasks are left in the map. */ - public void testLinkAndUnlink() throws Exception { - try (TestClient testClient = new TestClient(Settings.EMPTY, threadPool)) { - HttpChannelTaskHandler httpChannelTaskHandler = new HttpChannelTaskHandler(); + public void testCompletedTasks() throws Exception { + try (TestClient testClient = new TestClient(Settings.EMPTY, threadPool, false)) { + HttpChannelTaskHandler httpChannelTaskHandler = HttpChannelTaskHandler.get(); + int initialHttpChannels = httpChannelTaskHandler.httpChannels.size(); + int totalSearches = 0; List> futures = new ArrayList<>(); int numChannels = randomIntBetween(1, 30); for (int i = 0; i < numChannels; i++) { int numTasks = randomIntBetween(1, 30); TestHttpChannel channel = new TestHttpChannel(); + totalSearches += numTasks; for (int j = 0; j < numTasks; j++) { PlainListenableActionFuture actionFuture = PlainListenableActionFuture.newListenableFuture(); threadPool.generic().submit(() -> httpChannelTaskHandler.execute(testClient, channel, new SearchRequest(), @@ -92,67 +96,84 @@ public void testLinkAndUnlink() throws Exception { for (Future future : futures) { future.get(); } - //no channels get closed in this test - assertEquals(numChannels, httpChannelTaskHandler.httpChannels.size()); + //no channels get closed in this test, hence we expect as many channels as we created in the map + assertEquals(initialHttpChannels + numChannels, httpChannelTaskHandler.httpChannels.size()); for (Map.Entry entry : httpChannelTaskHandler.httpChannels.entrySet()) { assertEquals(0, entry.getValue().taskIds.size()); } + assertEquals(totalSearches, testClient.searchRequests.get()); } } - public void testChannelClose() throws Exception { - try (TestClient testClient = new TestClient(Settings.EMPTY, threadPool)) { - testClient.timeout.set(true); - HttpChannelTaskHandler httpChannelTaskHandler = new HttpChannelTaskHandler(); + /** + * This test verifies the behaviour when the channel gets closed. The channel is expected to be + * removed and all of its corresponding tasks get cancelled. + */ + public void testCancelledTasks() throws Exception { + try (TestClient testClient = new TestClient(Settings.EMPTY, threadPool, true)) { + HttpChannelTaskHandler httpChannelTaskHandler = HttpChannelTaskHandler.get(); + int initialHttpChannels = httpChannelTaskHandler.httpChannels.size(); int numChannels = randomIntBetween(1, 30); - int totalTasks = 0; + int totalSearches = 0; List channels = new ArrayList<>(numChannels); for (int i = 0; i < numChannels; i++) { TestHttpChannel channel = new TestHttpChannel(); channels.add(channel); int numTasks = randomIntBetween(1, 30); - totalTasks += numTasks; + totalSearches += numTasks; for (int j = 0; j < numTasks; j++) { httpChannelTaskHandler.execute(testClient, channel, new SearchRequest(), SearchAction.INSTANCE, null); } assertEquals(numTasks, httpChannelTaskHandler.httpChannels.get(channel).taskIds.size()); } - assertEquals(numChannels, httpChannelTaskHandler.httpChannels.size()); + assertEquals(initialHttpChannels + numChannels, httpChannelTaskHandler.httpChannels.size()); for (TestHttpChannel channel : channels) { channel.awaitClose(); } - assertEquals(0, httpChannelTaskHandler.httpChannels.size()); - assertEquals(totalTasks, testClient.cancelledTasks.size()); + assertEquals(initialHttpChannels, httpChannelTaskHandler.httpChannels.size()); + assertEquals(totalSearches, testClient.searchRequests.get()); + assertEquals(totalSearches, testClient.cancelledTasks.size()); } } + /** + * This test verified what happens when a request comes through yet its corresponding http channel is already closed. + * The close listener is straight-away executed, the task is cancelled. This can even happen multiple times, it's the only case + * where we may end up registering a close listener multiple times to the channel, but the channel is already closed hence only + * the newly added listener will be invoked at registration time. + */ public void testChannelAlreadyClosed() { - try (TestClient testClient = new TestClient(Settings.EMPTY, threadPool)) { - testClient.timeout.set(true); - + try (TestClient testClient = new TestClient(Settings.EMPTY, threadPool, true)) { + HttpChannelTaskHandler httpChannelTaskHandler = HttpChannelTaskHandler.get(); + int initialHttpChannels = httpChannelTaskHandler.httpChannels.size(); int numChannels = randomIntBetween(1, 30); - HttpChannelTaskHandler httpChannelTaskHandler = new HttpChannelTaskHandler(); + int totalSearches = 0; for (int i = 0; i < numChannels; i++) { TestHttpChannel channel = new TestHttpChannel(); //no need to wait here, there will be no close listener registered, nothing to wait for. channel.close(); - //here the channel will be first registered, then straight-away removed from the map as the close listener is invoked - //TODO is it possible that more tasks are started from a closed channel? In that case we would end up registering the close - //listener multiple times as the channel is unknown - httpChannelTaskHandler.execute(testClient, channel, new SearchRequest(), SearchAction.INSTANCE, null); + int numTasks = randomIntBetween(1, 5); + totalSearches += numTasks; + for (int j = 0; j < numTasks; j++) { + //here the channel will be first registered, then straight-away removed from the map as the close listener is invoked + httpChannelTaskHandler.execute(testClient, channel, new SearchRequest(), SearchAction.INSTANCE, null); + } } - assertEquals(0, httpChannelTaskHandler.httpChannels.size()); - assertEquals(0, testClient.cancelledTasks.size()); + assertEquals(initialHttpChannels, httpChannelTaskHandler.httpChannels.size()); + assertEquals(totalSearches, testClient.searchRequests.get()); + assertEquals(totalSearches, testClient.cancelledTasks.size()); } } private static class TestClient extends NodeClient { private final AtomicLong counter = new AtomicLong(0); - private final AtomicBoolean timeout = new AtomicBoolean(false); private final Set cancelledTasks = new CopyOnWriteArraySet<>(); + private final AtomicInteger searchRequests = new AtomicInteger(0); + private final boolean timeout; - TestClient(Settings settings, ThreadPool threadPool) { + TestClient(Settings settings, ThreadPool threadPool, boolean timeout) { super(settings, threadPool); + this.timeout = timeout; } @Override @@ -173,8 +194,9 @@ public Task exe return task; case SearchAction.NAME: + searchRequests.incrementAndGet(); Task searchTask = request.createTask(counter.getAndIncrement(), "search", action.name(), null, Collections.emptyMap()); - if (timeout.get() == false) { + if (timeout == false) { if (rarely()) { //make sure that search is sometimes also called from the same thread before the task is returned listener.onResponse(null); @@ -245,11 +267,13 @@ public boolean isOpen() { @Override public void addCloseListener(ActionListener listener) { - if (closeListener.compareAndSet(null, listener) == false) { - throw new IllegalStateException("close listener already set, only one is allowed!"); - } + //if the channel is already closed, the listener gets notified immediately, from the same thread. if (open.get() == false) { listener.onResponse(null); + } else { + if (closeListener.compareAndSet(null, listener) == false) { + throw new IllegalStateException("close listener already set, only one is allowed!"); + } } } } diff --git a/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java index 48d7d6a1a75e4..94c0cc6e3bf31 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java @@ -113,6 +113,7 @@ import org.elasticsearch.plugins.NetworkPlugin; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.rest.RestStatus; +import org.elasticsearch.rest.action.search.HttpChannelTaskHandler; import org.elasticsearch.script.ScriptService; import org.elasticsearch.search.MockSearchService; import org.elasticsearch.search.SearchHit; @@ -517,6 +518,8 @@ private static void clearClusters() throws IOException { restClient.close(); restClient = null; } + assertEquals(HttpChannelTaskHandler.get().getNumChannels() + " channels still being tracked in " + + HttpChannelTaskHandler.class.getSimpleName() + " while there should be none", 0, HttpChannelTaskHandler.get().getNumChannels()); } private void afterInternal(boolean afterClass) throws Exception { From f74d087c9f6d3435399be73030431256f21d8bb3 Mon Sep 17 00:00:00 2001 From: Luca Cavanna Date: Tue, 6 Aug 2019 10:49:38 +0200 Subject: [PATCH 18/28] add integration test --- .../http/SearchHttpCancellationIT.java | 223 ++++++++++++++++++ 1 file changed, 223 insertions(+) create mode 100644 qa/smoke-test-http/src/test/java/org/elasticsearch/http/SearchHttpCancellationIT.java diff --git a/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SearchHttpCancellationIT.java b/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SearchHttpCancellationIT.java new file mode 100644 index 0000000000000..7acaf8838c2c5 --- /dev/null +++ b/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SearchHttpCancellationIT.java @@ -0,0 +1,223 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.elasticsearch.http; + +import org.apache.http.HttpHost; +import org.apache.http.HttpResponse; +import org.apache.http.client.methods.HttpPost; +import org.apache.http.client.protocol.HttpClientContext; +import org.apache.http.entity.ContentType; +import org.apache.http.impl.nio.client.CloseableHttpAsyncClient; +import org.apache.http.impl.nio.client.HttpAsyncClientBuilder; +import org.apache.http.nio.client.methods.HttpAsyncMethods; +import org.apache.http.nio.entity.NStringEntity; +import org.apache.http.nio.protocol.HttpAsyncRequestProducer; +import org.apache.http.nio.protocol.HttpAsyncResponseConsumer; +import org.apache.logging.log4j.LogManager; +import org.apache.lucene.util.SetOnce; +import org.elasticsearch.action.admin.cluster.node.info.NodeInfo; +import org.elasticsearch.action.admin.cluster.node.info.NodesInfoResponse; +import org.elasticsearch.action.admin.cluster.node.tasks.list.ListTasksResponse; +import org.elasticsearch.action.bulk.BulkRequestBuilder; +import org.elasticsearch.action.search.SearchAction; +import org.elasticsearch.action.support.WriteRequest; +import org.elasticsearch.client.HttpAsyncResponseConsumerFactory; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.network.NetworkAddress; +import org.elasticsearch.common.transport.TransportAddress; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.plugins.PluginsService; +import org.elasticsearch.script.MockScriptPlugin; +import org.elasticsearch.script.Script; +import org.elasticsearch.script.ScriptType; +import org.elasticsearch.search.builder.SearchSourceBuilder; +import org.elasticsearch.search.lookup.LeafFieldsLookup; +import org.elasticsearch.tasks.CancellableTask; +import org.elasticsearch.tasks.Task; +import org.elasticsearch.tasks.TaskId; +import org.elasticsearch.tasks.TaskInfo; +import org.elasticsearch.tasks.TaskManager; +import org.elasticsearch.transport.TransportService; + +import java.net.InetSocketAddress; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CancellationException; +import java.util.concurrent.Future; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Function; + +import static org.elasticsearch.http.SearchHttpCancellationIT.ScriptedBlockPlugin.SCRIPT_NAME; +import static org.elasticsearch.index.query.QueryBuilders.scriptQuery; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoFailures; +import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.instanceOf; + +public class SearchHttpCancellationIT extends HttpSmokeTestCase { + + @Override + protected Collection> nodePlugins() { + List> plugins = new ArrayList<>(); + plugins.add(ScriptedBlockPlugin.class); + plugins.addAll(super.nodePlugins()); + return plugins; + } + + public void testAutomaticCancellationDuringQueryPhase() throws Exception { + List plugins = initBlockFactory(); + indexTestData(); + + HttpAsyncClientBuilder clientBuilder = HttpAsyncClientBuilder.create(); + + try (CloseableHttpAsyncClient httpClient = clientBuilder.build()) { + httpClient.start(); + + NodesInfoResponse nodesInfoResponse = client().admin().cluster().prepareNodesInfo().get(); + assertFalse(nodesInfoResponse.hasFailures()); + List hosts = new ArrayList<>(); + for (NodeInfo node : nodesInfoResponse.getNodes()) { + if (node.getHttp() != null) { + TransportAddress publishAddress = node.getHttp().address().publishAddress(); + InetSocketAddress address = publishAddress.address(); + hosts.add(new HttpHost(NetworkAddress.format(address.getAddress()), address.getPort(), "http")); + } + } + + SearchSourceBuilder searchSource = new SearchSourceBuilder().query(scriptQuery( + new Script(ScriptType.INLINE, "mockscript", SCRIPT_NAME, Collections.emptyMap()))); + + HttpPost httpPost = new HttpPost("/test/_search"); + httpPost.setEntity(new NStringEntity(Strings.toString(searchSource), ContentType.APPLICATION_JSON)); + + HttpAsyncRequestProducer requestProducer = HttpAsyncMethods.create(randomFrom(hosts), httpPost); + + HttpAsyncResponseConsumer httpAsyncResponseConsumer = + HttpAsyncResponseConsumerFactory.DEFAULT.createHttpAsyncResponseConsumer(); + HttpClientContext context = HttpClientContext.create(); + + Future future = httpClient.execute(requestProducer, httpAsyncResponseConsumer, context, null); + + awaitForBlock(plugins); + + SetOnce searchTask = new SetOnce<>(); + ListTasksResponse listTasksResponse = client().admin().cluster().prepareListTasks().get(); + for (TaskInfo task : listTasksResponse.getTasks()) { + if (task.getAction().equals(SearchAction.NAME)) { + searchTask.set(task); + } + } + assertNotNull(searchTask); + + httpPost.abort(); + + + + TaskId taskId = searchTask.get().getTaskId(); + NodesInfoResponse nodesInfo = client().admin().cluster().prepareNodesInfo(taskId.getNodeId()).get(); + String nodeName = nodesInfo.getNodes().get(0).getNode().getName(); + TaskManager taskManager = internalCluster().getInstance(TransportService.class, nodeName).getTaskManager(); + Task task = taskManager.getTask(taskId.getId()); + assertThat(task, instanceOf(CancellableTask.class)); + assertTrue(((CancellableTask)task).isCancelled()); + + disableBlocks(plugins); + expectThrows(CancellationException.class, future::get); + } + } + + private void indexTestData() { + for (int i = 0; i < 5; i++) { + // Make sure we have a few segments + BulkRequestBuilder bulkRequestBuilder = client().prepareBulk().setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE); + for (int j = 0; j < 20; j++) { + bulkRequestBuilder.add(client().prepareIndex("test", "type", Integer.toString(i * 5 + j)).setSource("field", "value")); + } + assertNoFailures(bulkRequestBuilder.get()); + } + } + + private List initBlockFactory() { + List plugins = new ArrayList<>(); + for (PluginsService pluginsService : internalCluster().getDataNodeInstances(PluginsService.class)) { + plugins.addAll(pluginsService.filterPlugins(ScriptedBlockPlugin.class)); + } + for (ScriptedBlockPlugin plugin : plugins) { + plugin.reset(); + plugin.enableBlock(); + } + return plugins; + } + + private void awaitForBlock(List plugins) throws Exception { + int numberOfShards = getNumShards("test").numPrimaries; + assertBusy(() -> { + int numberOfBlockedPlugins = 0; + for (ScriptedBlockPlugin plugin : plugins) { + numberOfBlockedPlugins += plugin.hits.get(); + } + logger.info("The plugin blocked on {} out of {} shards", numberOfBlockedPlugins, numberOfShards); + assertThat(numberOfBlockedPlugins, greaterThan(0)); + }); + } + + private void disableBlocks(List plugins) throws Exception { + for (ScriptedBlockPlugin plugin : plugins) { + plugin.disableBlock(); + } + } + + public static class ScriptedBlockPlugin extends MockScriptPlugin { + static final String SCRIPT_NAME = "search_block"; + + private final AtomicInteger hits = new AtomicInteger(); + + private final AtomicBoolean shouldBlock = new AtomicBoolean(true); + + void reset() { + hits.set(0); + } + + void disableBlock() { + shouldBlock.set(false); + } + + void enableBlock() { + shouldBlock.set(true); + } + + @Override + public Map, Object>> pluginScripts() { + return Collections.singletonMap(SCRIPT_NAME, params -> { + LeafFieldsLookup fieldsLookup = (LeafFieldsLookup) params.get("_fields"); + LogManager.getLogger(SearchHttpCancellationIT.class).info("Blocking on the document {}", fieldsLookup.get("_id")); + hits.incrementAndGet(); + try { + awaitBusy(() -> shouldBlock.get() == false); + } catch (Exception e) { + throw new RuntimeException(e); + } + return true; + }); + } + } +} From 75bf6c3d11437ca562d2be961698b2deccdb86ed Mon Sep 17 00:00:00 2001 From: Luca Cavanna Date: Tue, 6 Aug 2019 11:07:21 +0200 Subject: [PATCH 19/28] expand integ test --- .../http/SearchHttpCancellationIT.java | 96 ++++++++++++------- 1 file changed, 60 insertions(+), 36 deletions(-) diff --git a/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SearchHttpCancellationIT.java b/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SearchHttpCancellationIT.java index 7acaf8838c2c5..0a7067a60c211 100644 --- a/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SearchHttpCancellationIT.java +++ b/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SearchHttpCancellationIT.java @@ -21,14 +21,10 @@ import org.apache.http.HttpHost; import org.apache.http.HttpResponse; import org.apache.http.client.methods.HttpPost; -import org.apache.http.client.protocol.HttpClientContext; import org.apache.http.entity.ContentType; import org.apache.http.impl.nio.client.CloseableHttpAsyncClient; import org.apache.http.impl.nio.client.HttpAsyncClientBuilder; -import org.apache.http.nio.client.methods.HttpAsyncMethods; import org.apache.http.nio.entity.NStringEntity; -import org.apache.http.nio.protocol.HttpAsyncRequestProducer; -import org.apache.http.nio.protocol.HttpAsyncResponseConsumer; import org.apache.logging.log4j.LogManager; import org.apache.lucene.util.SetOnce; import org.elasticsearch.action.admin.cluster.node.info.NodeInfo; @@ -37,7 +33,6 @@ import org.elasticsearch.action.bulk.BulkRequestBuilder; import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.support.WriteRequest; -import org.elasticsearch.client.HttpAsyncResponseConsumerFactory; import org.elasticsearch.common.Strings; import org.elasticsearch.common.network.NetworkAddress; import org.elasticsearch.common.transport.TransportAddress; @@ -59,6 +54,7 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Collections; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.concurrent.CancellationException; @@ -92,16 +88,9 @@ public void testAutomaticCancellationDuringQueryPhase() throws Exception { try (CloseableHttpAsyncClient httpClient = clientBuilder.build()) { httpClient.start(); - NodesInfoResponse nodesInfoResponse = client().admin().cluster().prepareNodesInfo().get(); - assertFalse(nodesInfoResponse.hasFailures()); List hosts = new ArrayList<>(); - for (NodeInfo node : nodesInfoResponse.getNodes()) { - if (node.getHttp() != null) { - TransportAddress publishAddress = node.getHttp().address().publishAddress(); - InetSocketAddress address = publishAddress.address(); - hosts.add(new HttpHost(NetworkAddress.format(address.getAddress()), address.getPort(), "http")); - } - } + Map nodeIdToName = new HashMap<>(); + readNodesInfo(hosts, nodeIdToName); SearchSourceBuilder searchSource = new SearchSourceBuilder().query(scriptQuery( new Script(ScriptType.INLINE, "mockscript", SCRIPT_NAME, Collections.emptyMap()))); @@ -109,42 +98,77 @@ public void testAutomaticCancellationDuringQueryPhase() throws Exception { HttpPost httpPost = new HttpPost("/test/_search"); httpPost.setEntity(new NStringEntity(Strings.toString(searchSource), ContentType.APPLICATION_JSON)); - HttpAsyncRequestProducer requestProducer = HttpAsyncMethods.create(randomFrom(hosts), httpPost); + Future future = httpClient.execute(randomFrom(hosts), httpPost, null); + awaitForBlock(plugins); - HttpAsyncResponseConsumer httpAsyncResponseConsumer = - HttpAsyncResponseConsumerFactory.DEFAULT.createHttpAsyncResponseConsumer(); - HttpClientContext context = HttpClientContext.create(); + httpPost.abort(); + ensureSearchTaskIsCancelled(nodeIdToName::get); - Future future = httpClient.execute(requestProducer, httpAsyncResponseConsumer, context, null); + disableBlocks(plugins); + expectThrows(CancellationException.class, future::get); + } + } - awaitForBlock(plugins); + public void testAutomaticCancellationDuringFetchPhase() throws Exception { + List plugins = initBlockFactory(); + indexTestData(); - SetOnce searchTask = new SetOnce<>(); - ListTasksResponse listTasksResponse = client().admin().cluster().prepareListTasks().get(); - for (TaskInfo task : listTasksResponse.getTasks()) { - if (task.getAction().equals(SearchAction.NAME)) { - searchTask.set(task); - } - } - assertNotNull(searchTask); + HttpAsyncClientBuilder clientBuilder = HttpAsyncClientBuilder.create(); - httpPost.abort(); + try (CloseableHttpAsyncClient httpClient = clientBuilder.build()) { + httpClient.start(); + + List hosts = new ArrayList<>(); + Map nodeIdToName = new HashMap<>(); + readNodesInfo(hosts, nodeIdToName); + + SearchSourceBuilder searchSource = new SearchSourceBuilder().scriptField("test_field", + new Script(ScriptType.INLINE, "mockscript", SCRIPT_NAME, Collections.emptyMap())); + HttpPost httpPost = new HttpPost("/test/_search"); + httpPost.setEntity(new NStringEntity(Strings.toString(searchSource), ContentType.APPLICATION_JSON)); + Future future = httpClient.execute(randomFrom(hosts), httpPost, null); + awaitForBlock(plugins); - TaskId taskId = searchTask.get().getTaskId(); - NodesInfoResponse nodesInfo = client().admin().cluster().prepareNodesInfo(taskId.getNodeId()).get(); - String nodeName = nodesInfo.getNodes().get(0).getNode().getName(); - TaskManager taskManager = internalCluster().getInstance(TransportService.class, nodeName).getTaskManager(); - Task task = taskManager.getTask(taskId.getId()); - assertThat(task, instanceOf(CancellableTask.class)); - assertTrue(((CancellableTask)task).isCancelled()); + httpPost.abort(); + ensureSearchTaskIsCancelled(nodeIdToName::get); disableBlocks(plugins); expectThrows(CancellationException.class, future::get); } } + private static void readNodesInfo(List hosts, Map nodeIdToName) { + NodesInfoResponse nodesInfoResponse = client().admin().cluster().prepareNodesInfo().get(); + assertFalse(nodesInfoResponse.hasFailures()); + for (NodeInfo node : nodesInfoResponse.getNodes()) { + if (node.getHttp() != null) { + TransportAddress publishAddress = node.getHttp().address().publishAddress(); + InetSocketAddress address = publishAddress.address(); + hosts.add(new HttpHost(NetworkAddress.format(address.getAddress()), address.getPort(), "http")); + } + nodeIdToName.put(node.getNode().getId(), node.getNode().getName()); + } + } + + private static void ensureSearchTaskIsCancelled(Function nodeIdToName) { + SetOnce searchTask = new SetOnce<>(); + ListTasksResponse listTasksResponse = client().admin().cluster().prepareListTasks().get(); + for (TaskInfo task : listTasksResponse.getTasks()) { + if (task.getAction().equals(SearchAction.NAME)) { + searchTask.set(task); + } + } + assertNotNull(searchTask.get()); + TaskId taskId = searchTask.get().getTaskId(); + String nodeName = nodeIdToName.apply(taskId.getNodeId()); + TaskManager taskManager = internalCluster().getInstance(TransportService.class, nodeName).getTaskManager(); + Task task = taskManager.getTask(taskId.getId()); + assertThat(task, instanceOf(CancellableTask.class)); + assertTrue(((CancellableTask)task).isCancelled()); + } + private void indexTestData() { for (int i = 0; i < 5; i++) { // Make sure we have a few segments From 868e01a5c50314870054529127d3df4c95379092 Mon Sep 17 00:00:00 2001 From: Luca Cavanna Date: Tue, 6 Aug 2019 14:39:12 +0200 Subject: [PATCH 20/28] feedback --- .../action/search/HttpChannelTaskHandler.java | 33 +++++++++++-------- .../rest/action/search/RestSearchAction.java | 2 +- .../search/HttpChannelTaskHandlerTests.java | 20 +++++------ .../elasticsearch/test/ESIntegTestCase.java | 5 +-- 4 files changed, 33 insertions(+), 27 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandler.java b/server/src/main/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandler.java index d41cf7786461b..19b29fd3283d6 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandler.java +++ b/server/src/main/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandler.java @@ -24,8 +24,11 @@ import org.elasticsearch.action.ActionResponse; import org.elasticsearch.action.ActionType; import org.elasticsearch.action.admin.cluster.node.tasks.cancel.CancelTasksRequest; +import org.elasticsearch.action.admin.cluster.node.tasks.cancel.CancelTasksResponse; +import org.elasticsearch.action.support.ContextPreservingActionListener; import org.elasticsearch.client.Client; import org.elasticsearch.client.node.NodeClient; +import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.http.HttpChannel; import org.elasticsearch.tasks.Task; import org.elasticsearch.tasks.TaskId; @@ -42,17 +45,13 @@ */ public final class HttpChannelTaskHandler { - private static final HttpChannelTaskHandler INSTANCE = new HttpChannelTaskHandler(); - + public static final HttpChannelTaskHandler INSTANCE = new HttpChannelTaskHandler(); + //package private for testing final Map httpChannels = new ConcurrentHashMap<>(); private HttpChannelTaskHandler() { } - public static HttpChannelTaskHandler get() { - return INSTANCE; - } - void execute(NodeClient client, HttpChannel httpChannel, ActionRequest request, ActionType actionType, ActionListener listener) { @@ -122,14 +121,20 @@ synchronized void unregisterTask(TaskHolder taskHolder) { @Override public synchronized void onResponse(Void aVoid) { //When the channel gets closed it won't be reused: we can remove it from the map and forget about it. - httpChannels.remove(channel.get()); - for (TaskId previousTaskId : taskIds) { - //TODO what thread context should this be run on? - CancelTasksRequest cancelTasksRequest = new CancelTasksRequest(); - cancelTasksRequest.setTaskId(previousTaskId); - //We don't wait for cancel tasks to come back. Task cancellation is just best effort. - //Note that cancel tasks fails if the user sending the search request does not have the permissions to call it. - client.admin().cluster().cancelTasks(cancelTasksRequest, ActionListener.wrap(r -> {}, e -> {})); + CloseListener closeListener = httpChannels.remove(channel.get()); + assert closeListener != null : "channel not found in the map of tracked channels"; + for (TaskId taskId : taskIds) { + ThreadContext threadContext = client.threadPool().getThreadContext(); + try (ThreadContext.StoredContext ignore = threadContext.stashContext()) { + // we stash any context here since this is an internal execution and should not leak any existing context information + threadContext.markAsSystemContext(); + ContextPreservingActionListener contextPreservingListener = new ContextPreservingActionListener<>( + threadContext.newRestorableContext(false), ActionListener.wrap(r -> {}, e -> {})); + CancelTasksRequest cancelTasksRequest = new CancelTasksRequest(); + cancelTasksRequest.setTaskId(taskId); + //We don't wait for cancel tasks to come back. Task cancellation is just best effort. + client.admin().cluster().cancelTasks(cancelTasksRequest, contextPreservingListener); + } } } diff --git a/server/src/main/java/org/elasticsearch/rest/action/search/RestSearchAction.java b/server/src/main/java/org/elasticsearch/rest/action/search/RestSearchAction.java index 1af77a42288de..15e50f9ef4eb3 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/search/RestSearchAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/search/RestSearchAction.java @@ -103,7 +103,7 @@ public RestChannelConsumer prepareRequest(final RestRequest request, final NodeC return channel -> { RestStatusToXContentListener listener = new RestStatusToXContentListener<>(channel); - HttpChannelTaskHandler.get().execute(client, request.getHttpChannel(), searchRequest, SearchAction.INSTANCE, listener); + HttpChannelTaskHandler.INSTANCE.execute(client, request.getHttpChannel(), searchRequest, SearchAction.INSTANCE, listener); }; } diff --git a/server/src/test/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandlerTests.java b/server/src/test/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandlerTests.java index 725a210ebf206..d189afe471fd6 100644 --- a/server/src/test/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandlerTests.java +++ b/server/src/test/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandlerTests.java @@ -77,8 +77,8 @@ public void stopThreadPool() { */ public void testCompletedTasks() throws Exception { try (TestClient testClient = new TestClient(Settings.EMPTY, threadPool, false)) { - HttpChannelTaskHandler httpChannelTaskHandler = HttpChannelTaskHandler.get(); - int initialHttpChannels = httpChannelTaskHandler.httpChannels.size(); + HttpChannelTaskHandler httpChannelTaskHandler = HttpChannelTaskHandler.INSTANCE; + int initialHttpChannels = httpChannelTaskHandler.getNumChannels(); int totalSearches = 0; List> futures = new ArrayList<>(); int numChannels = randomIntBetween(1, 30); @@ -97,7 +97,7 @@ public void testCompletedTasks() throws Exception { future.get(); } //no channels get closed in this test, hence we expect as many channels as we created in the map - assertEquals(initialHttpChannels + numChannels, httpChannelTaskHandler.httpChannels.size()); + assertEquals(initialHttpChannels + numChannels, httpChannelTaskHandler.getNumChannels()); for (Map.Entry entry : httpChannelTaskHandler.httpChannels.entrySet()) { assertEquals(0, entry.getValue().taskIds.size()); } @@ -111,8 +111,8 @@ public void testCompletedTasks() throws Exception { */ public void testCancelledTasks() throws Exception { try (TestClient testClient = new TestClient(Settings.EMPTY, threadPool, true)) { - HttpChannelTaskHandler httpChannelTaskHandler = HttpChannelTaskHandler.get(); - int initialHttpChannels = httpChannelTaskHandler.httpChannels.size(); + HttpChannelTaskHandler httpChannelTaskHandler = HttpChannelTaskHandler.INSTANCE; + int initialHttpChannels = httpChannelTaskHandler.getNumChannels(); int numChannels = randomIntBetween(1, 30); int totalSearches = 0; List channels = new ArrayList<>(numChannels); @@ -126,11 +126,11 @@ public void testCancelledTasks() throws Exception { } assertEquals(numTasks, httpChannelTaskHandler.httpChannels.get(channel).taskIds.size()); } - assertEquals(initialHttpChannels + numChannels, httpChannelTaskHandler.httpChannels.size()); + assertEquals(initialHttpChannels + numChannels, httpChannelTaskHandler.getNumChannels()); for (TestHttpChannel channel : channels) { channel.awaitClose(); } - assertEquals(initialHttpChannels, httpChannelTaskHandler.httpChannels.size()); + assertEquals(initialHttpChannels, httpChannelTaskHandler.getNumChannels()); assertEquals(totalSearches, testClient.searchRequests.get()); assertEquals(totalSearches, testClient.cancelledTasks.size()); } @@ -144,8 +144,8 @@ public void testCancelledTasks() throws Exception { */ public void testChannelAlreadyClosed() { try (TestClient testClient = new TestClient(Settings.EMPTY, threadPool, true)) { - HttpChannelTaskHandler httpChannelTaskHandler = HttpChannelTaskHandler.get(); - int initialHttpChannels = httpChannelTaskHandler.httpChannels.size(); + HttpChannelTaskHandler httpChannelTaskHandler = HttpChannelTaskHandler.INSTANCE; + int initialHttpChannels = httpChannelTaskHandler.getNumChannels(); int numChannels = randomIntBetween(1, 30); int totalSearches = 0; for (int i = 0; i < numChannels; i++) { @@ -159,7 +159,7 @@ public void testChannelAlreadyClosed() { httpChannelTaskHandler.execute(testClient, channel, new SearchRequest(), SearchAction.INSTANCE, null); } } - assertEquals(initialHttpChannels, httpChannelTaskHandler.httpChannels.size()); + assertEquals(initialHttpChannels, httpChannelTaskHandler.getNumChannels()); assertEquals(totalSearches, testClient.searchRequests.get()); assertEquals(totalSearches, testClient.cancelledTasks.size()); } diff --git a/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java index 05d7eb6bed5e7..1a402577f407a 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java @@ -518,8 +518,9 @@ private static void clearClusters() throws IOException { restClient.close(); restClient = null; } - assertEquals(HttpChannelTaskHandler.get().getNumChannels() + " channels still being tracked in " + - HttpChannelTaskHandler.class.getSimpleName() + " while there should be none", 0, HttpChannelTaskHandler.get().getNumChannels()); + assertEquals(HttpChannelTaskHandler.INSTANCE.getNumChannels() + " channels still being tracked in " + + HttpChannelTaskHandler.class.getSimpleName() + " while there should be none", 0, + HttpChannelTaskHandler.INSTANCE.getNumChannels()); } private void afterInternal(boolean afterClass) throws Exception { From 2d886e6046b072cbf9fa164b64454c30c17e3b8f Mon Sep 17 00:00:00 2001 From: Luca Cavanna Date: Tue, 6 Aug 2019 14:41:08 +0200 Subject: [PATCH 21/28] address review comments --- .../rest/action/search/HttpChannelTaskHandler.java | 6 +++++- .../rest/action/search/HttpChannelTaskHandlerTests.java | 4 ++-- 2 files changed, 7 insertions(+), 3 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandler.java b/server/src/main/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandler.java index 19b29fd3283d6..5864551854fca 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandler.java +++ b/server/src/main/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandler.java @@ -88,12 +88,16 @@ public int getNumChannels() { final class CloseListener implements ActionListener { private final Client client; private final AtomicReference channel = new AtomicReference<>(); - final Set taskIds = new HashSet<>(); + private final Set taskIds = new HashSet<>(); CloseListener(Client client) { this.client = client; } + int getNumTasks() { + return taskIds.size(); + } + void maybeRegisterChannel(HttpChannel httpChannel) { if (channel.compareAndSet(null, httpChannel)) { //In case the channel is already closed when we register the listener, the listener will be immediately executed which will diff --git a/server/src/test/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandlerTests.java b/server/src/test/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandlerTests.java index d189afe471fd6..103981abdc41e 100644 --- a/server/src/test/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandlerTests.java +++ b/server/src/test/java/org/elasticsearch/rest/action/search/HttpChannelTaskHandlerTests.java @@ -99,7 +99,7 @@ public void testCompletedTasks() throws Exception { //no channels get closed in this test, hence we expect as many channels as we created in the map assertEquals(initialHttpChannels + numChannels, httpChannelTaskHandler.getNumChannels()); for (Map.Entry entry : httpChannelTaskHandler.httpChannels.entrySet()) { - assertEquals(0, entry.getValue().taskIds.size()); + assertEquals(0, entry.getValue().getNumTasks()); } assertEquals(totalSearches, testClient.searchRequests.get()); } @@ -124,7 +124,7 @@ public void testCancelledTasks() throws Exception { for (int j = 0; j < numTasks; j++) { httpChannelTaskHandler.execute(testClient, channel, new SearchRequest(), SearchAction.INSTANCE, null); } - assertEquals(numTasks, httpChannelTaskHandler.httpChannels.get(channel).taskIds.size()); + assertEquals(numTasks, httpChannelTaskHandler.httpChannels.get(channel).getNumTasks()); } assertEquals(initialHttpChannels + numChannels, httpChannelTaskHandler.getNumChannels()); for (TestHttpChannel channel : channels) { From 98e74b54cad2899a3401e2b8a8d1c0b60292e78e Mon Sep 17 00:00:00 2001 From: Luca Cavanna Date: Tue, 6 Aug 2019 21:10:50 +0200 Subject: [PATCH 22/28] iter --- .../http/SearchHttpCancellationIT.java | 91 +++++++++++-------- 1 file changed, 53 insertions(+), 38 deletions(-) diff --git a/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SearchHttpCancellationIT.java b/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SearchHttpCancellationIT.java index 0a7067a60c211..a14520d2681ae 100644 --- a/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SearchHttpCancellationIT.java +++ b/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SearchHttpCancellationIT.java @@ -21,10 +21,14 @@ import org.apache.http.HttpHost; import org.apache.http.HttpResponse; import org.apache.http.client.methods.HttpPost; +import org.apache.http.client.protocol.HttpClientContext; import org.apache.http.entity.ContentType; import org.apache.http.impl.nio.client.CloseableHttpAsyncClient; import org.apache.http.impl.nio.client.HttpAsyncClientBuilder; +import org.apache.http.nio.client.methods.HttpAsyncMethods; import org.apache.http.nio.entity.NStringEntity; +import org.apache.http.nio.protocol.HttpAsyncRequestProducer; +import org.apache.http.nio.protocol.HttpAsyncResponseConsumer; import org.apache.logging.log4j.LogManager; import org.apache.lucene.util.SetOnce; import org.elasticsearch.action.admin.cluster.node.info.NodeInfo; @@ -33,9 +37,11 @@ import org.elasticsearch.action.bulk.BulkRequestBuilder; import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.support.WriteRequest; +import org.elasticsearch.client.HttpAsyncResponseConsumerFactory; import org.elasticsearch.common.Strings; import org.elasticsearch.common.network.NetworkAddress; import org.elasticsearch.common.transport.TransportAddress; +import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.plugins.PluginsService; import org.elasticsearch.script.MockScriptPlugin; @@ -49,7 +55,10 @@ import org.elasticsearch.tasks.TaskInfo; import org.elasticsearch.tasks.TaskManager; import org.elasticsearch.transport.TransportService; +import org.junit.AfterClass; +import org.junit.Before; +import java.io.IOException; import java.net.InetSocketAddress; import java.util.ArrayList; import java.util.Collection; @@ -71,6 +80,10 @@ public class SearchHttpCancellationIT extends HttpSmokeTestCase { + private static CloseableHttpAsyncClient client; + private static List hosts = new ArrayList<>(); + private static Map nodeIdToName = new HashMap<>(); + @Override protected Collection> nodePlugins() { List> plugins = new ArrayList<>(); @@ -79,64 +92,66 @@ protected Collection> nodePlugins() { return plugins; } + @Before + public void init() { + if (client == null) { + client = HttpAsyncClientBuilder.create().build(); + client.start(); + readNodesInfo(hosts, nodeIdToName); + } + } + + @AfterClass + public static void closeClient() throws IOException { + IOUtils.close(client); + client = null; + } + public void testAutomaticCancellationDuringQueryPhase() throws Exception { List plugins = initBlockFactory(); indexTestData(); - HttpAsyncClientBuilder clientBuilder = HttpAsyncClientBuilder.create(); + SearchSourceBuilder searchSource = new SearchSourceBuilder().query(scriptQuery( + new Script(ScriptType.INLINE, "mockscript", SCRIPT_NAME, Collections.emptyMap()))); - try (CloseableHttpAsyncClient httpClient = clientBuilder.build()) { - httpClient.start(); + HttpPost httpPost = new HttpPost("/test/_search"); + httpPost.setEntity(new NStringEntity(Strings.toString(searchSource), ContentType.APPLICATION_JSON)); - List hosts = new ArrayList<>(); - Map nodeIdToName = new HashMap<>(); - readNodesInfo(hosts, nodeIdToName); + HttpAsyncRequestProducer requestProducer = HttpAsyncMethods.create(randomFrom(hosts), httpPost); + HttpAsyncResponseConsumer httpAsyncResponseConsumer = + HttpAsyncResponseConsumerFactory.DEFAULT.createHttpAsyncResponseConsumer(); + HttpClientContext context = HttpClientContext.create(); - SearchSourceBuilder searchSource = new SearchSourceBuilder().query(scriptQuery( - new Script(ScriptType.INLINE, "mockscript", SCRIPT_NAME, Collections.emptyMap()))); + Future future = client.execute(requestProducer, httpAsyncResponseConsumer, context, null); - HttpPost httpPost = new HttpPost("/test/_search"); - httpPost.setEntity(new NStringEntity(Strings.toString(searchSource), ContentType.APPLICATION_JSON)); + //Future future = httpClient.execute(randomFrom(hosts), httpPost, null); + awaitForBlock(plugins); - Future future = httpClient.execute(randomFrom(hosts), httpPost, null); - awaitForBlock(plugins); + httpPost.abort(); + ensureSearchTaskIsCancelled(nodeIdToName::get); - httpPost.abort(); - ensureSearchTaskIsCancelled(nodeIdToName::get); - - disableBlocks(plugins); - expectThrows(CancellationException.class, future::get); - } + disableBlocks(plugins); + expectThrows(CancellationException.class, future::get); } public void testAutomaticCancellationDuringFetchPhase() throws Exception { List plugins = initBlockFactory(); indexTestData(); - HttpAsyncClientBuilder clientBuilder = HttpAsyncClientBuilder.create(); + SearchSourceBuilder searchSource = new SearchSourceBuilder().scriptField("test_field", + new Script(ScriptType.INLINE, "mockscript", SCRIPT_NAME, Collections.emptyMap())); - try (CloseableHttpAsyncClient httpClient = clientBuilder.build()) { - httpClient.start(); - - List hosts = new ArrayList<>(); - Map nodeIdToName = new HashMap<>(); - readNodesInfo(hosts, nodeIdToName); + HttpPost httpPost = new HttpPost("/test/_search"); + httpPost.setEntity(new NStringEntity(Strings.toString(searchSource), ContentType.APPLICATION_JSON)); - SearchSourceBuilder searchSource = new SearchSourceBuilder().scriptField("test_field", - new Script(ScriptType.INLINE, "mockscript", SCRIPT_NAME, Collections.emptyMap())); + Future future = client.execute(randomFrom(hosts), httpPost, null); + awaitForBlock(plugins); - HttpPost httpPost = new HttpPost("/test/_search"); - httpPost.setEntity(new NStringEntity(Strings.toString(searchSource), ContentType.APPLICATION_JSON)); + httpPost.abort(); + ensureSearchTaskIsCancelled(nodeIdToName::get); - Future future = httpClient.execute(randomFrom(hosts), httpPost, null); - awaitForBlock(plugins); - - httpPost.abort(); - ensureSearchTaskIsCancelled(nodeIdToName::get); - - disableBlocks(plugins); - expectThrows(CancellationException.class, future::get); - } + disableBlocks(plugins); + expectThrows(CancellationException.class, future::get); } private static void readNodesInfo(List hosts, Map nodeIdToName) { From 85776589966f45c74311bab4ab5a737a4a3ff680 Mon Sep 17 00:00:00 2001 From: Luca Cavanna Date: Tue, 6 Aug 2019 21:34:11 +0200 Subject: [PATCH 23/28] iter --- .../http/SearchHttpCancellationIT.java | 35 ++++++++++++------- 1 file changed, 23 insertions(+), 12 deletions(-) diff --git a/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SearchHttpCancellationIT.java b/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SearchHttpCancellationIT.java index a14520d2681ae..d8a875bc71bb8 100644 --- a/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SearchHttpCancellationIT.java +++ b/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SearchHttpCancellationIT.java @@ -60,6 +60,9 @@ import java.io.IOException; import java.net.InetSocketAddress; +import java.security.AccessController; +import java.security.PrivilegedActionException; +import java.security.PrivilegedExceptionAction; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -68,6 +71,7 @@ import java.util.Map; import java.util.concurrent.CancellationException; import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Function; @@ -111,10 +115,9 @@ public void testAutomaticCancellationDuringQueryPhase() throws Exception { List plugins = initBlockFactory(); indexTestData(); + HttpPost httpPost = new HttpPost("/test/_search"); SearchSourceBuilder searchSource = new SearchSourceBuilder().query(scriptQuery( new Script(ScriptType.INLINE, "mockscript", SCRIPT_NAME, Collections.emptyMap()))); - - HttpPost httpPost = new HttpPost("/test/_search"); httpPost.setEntity(new NStringEntity(Strings.toString(searchSource), ContentType.APPLICATION_JSON)); HttpAsyncRequestProducer requestProducer = HttpAsyncMethods.create(randomFrom(hosts), httpPost); @@ -123,8 +126,7 @@ public void testAutomaticCancellationDuringQueryPhase() throws Exception { HttpClientContext context = HttpClientContext.create(); Future future = client.execute(requestProducer, httpAsyncResponseConsumer, context, null); - - //Future future = httpClient.execute(randomFrom(hosts), httpPost, null); + future.get(); //TODO remove this, it is here just to see the security manager error awaitForBlock(plugins); httpPost.abort(); @@ -138,13 +140,13 @@ public void testAutomaticCancellationDuringFetchPhase() throws Exception { List plugins = initBlockFactory(); indexTestData(); + HttpPost httpPost = new HttpPost("/test/_search"); SearchSourceBuilder searchSource = new SearchSourceBuilder().scriptField("test_field", new Script(ScriptType.INLINE, "mockscript", SCRIPT_NAME, Collections.emptyMap())); - - HttpPost httpPost = new HttpPost("/test/_search"); httpPost.setEntity(new NStringEntity(Strings.toString(searchSource), ContentType.APPLICATION_JSON)); - Future future = client.execute(randomFrom(hosts), httpPost, null); + Future future = sendRequest(randomFrom(hosts), httpPost); + future.get(); //TODO remove this, it is here just to see the security manager error awaitForBlock(plugins); httpPost.abort(); @@ -154,6 +156,15 @@ public void testAutomaticCancellationDuringFetchPhase() throws Exception { expectThrows(CancellationException.class, future::get); } + private static Future sendRequest(HttpHost httpHost, HttpPost httpPost) throws Exception { + try { + return AccessController.doPrivileged((PrivilegedExceptionAction>) + () -> client.execute(httpHost, httpPost, null)); + } catch (PrivilegedActionException e) { + throw (Exception) e.getCause(); + } + } + private static void readNodesInfo(List hosts, Map nodeIdToName) { NodesInfoResponse nodesInfoResponse = client().admin().cluster().prepareNodesInfo().get(); assertFalse(nodesInfoResponse.hasFailures()); @@ -184,18 +195,18 @@ private static void ensureSearchTaskIsCancelled(Function nodeIdT assertTrue(((CancellableTask)task).isCancelled()); } - private void indexTestData() { + private static void indexTestData() { for (int i = 0; i < 5; i++) { // Make sure we have a few segments BulkRequestBuilder bulkRequestBuilder = client().prepareBulk().setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE); for (int j = 0; j < 20; j++) { - bulkRequestBuilder.add(client().prepareIndex("test", "type", Integer.toString(i * 5 + j)).setSource("field", "value")); + bulkRequestBuilder.add(client().prepareIndex("test", "_doc", Integer.toString(i * 5 + j)).setSource("field", "value")); } assertNoFailures(bulkRequestBuilder.get()); } } - private List initBlockFactory() { + private static List initBlockFactory() { List plugins = new ArrayList<>(); for (PluginsService pluginsService : internalCluster().getDataNodeInstances(PluginsService.class)) { plugins.addAll(pluginsService.filterPlugins(ScriptedBlockPlugin.class)); @@ -216,10 +227,10 @@ private void awaitForBlock(List plugins) throws Exception { } logger.info("The plugin blocked on {} out of {} shards", numberOfBlockedPlugins, numberOfShards); assertThat(numberOfBlockedPlugins, greaterThan(0)); - }); + }, 20, TimeUnit.SECONDS); } - private void disableBlocks(List plugins) throws Exception { + private static void disableBlocks(List plugins) { for (ScriptedBlockPlugin plugin : plugins) { plugin.disableBlock(); } From adc710a4708b9aaeab3217c3b4a40ea61c9d41e3 Mon Sep 17 00:00:00 2001 From: Luca Cavanna Date: Wed, 7 Aug 2019 11:08:09 +0200 Subject: [PATCH 24/28] iter --- .../http/SearchHttpCancellationIT.java | 14 +++++++++++--- 1 file changed, 11 insertions(+), 3 deletions(-) diff --git a/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SearchHttpCancellationIT.java b/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SearchHttpCancellationIT.java index d8a875bc71bb8..9a96366f4cc62 100644 --- a/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SearchHttpCancellationIT.java +++ b/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SearchHttpCancellationIT.java @@ -97,10 +97,18 @@ protected Collection> nodePlugins() { } @Before - public void init() { + public void init() throws Exception { if (client == null) { - client = HttpAsyncClientBuilder.create().build(); - client.start(); + try { + client = AccessController.doPrivileged((PrivilegedExceptionAction) + () -> client = HttpAsyncClientBuilder.create().build()); + AccessController.doPrivileged((PrivilegedExceptionAction) () -> { + client.start(); + return null; + }); + } catch (PrivilegedActionException e) { + throw (Exception) e.getCause(); + } readNodesInfo(hosts, nodeIdToName); } } From 6f83fd8569f7d8e4fe5ae6fbdff86502334000b8 Mon Sep 17 00:00:00 2001 From: Luca Cavanna Date: Thu, 8 Aug 2019 17:08:15 +0200 Subject: [PATCH 25/28] abort requests --- .../org/elasticsearch/client/Cancellable.java | 39 +++++++++++++++++++ .../org/elasticsearch/client/RestClient.java | 7 +++- 2 files changed, 44 insertions(+), 2 deletions(-) create mode 100644 client/rest/src/main/java/org/elasticsearch/client/Cancellable.java diff --git a/client/rest/src/main/java/org/elasticsearch/client/Cancellable.java b/client/rest/src/main/java/org/elasticsearch/client/Cancellable.java new file mode 100644 index 0000000000000..4ec8a79eb57be --- /dev/null +++ b/client/rest/src/main/java/org/elasticsearch/client/Cancellable.java @@ -0,0 +1,39 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.elasticsearch.client; + +import java.util.concurrent.atomic.AtomicBoolean; + +public final class Cancellable { + + private final AtomicBoolean cancelled = new AtomicBoolean(false); + private final Runnable runOnCancel; + + Cancellable(Runnable runOnCancel) { + this.runOnCancel = runOnCancel; + } + + public boolean cancel() { + if (this.cancelled.compareAndSet(false, true)) { + this.runOnCancel.run(); + return true; + } + return false; + } +} diff --git a/client/rest/src/main/java/org/elasticsearch/client/RestClient.java b/client/rest/src/main/java/org/elasticsearch/client/RestClient.java index 38185ac960926..5867905f3f1d7 100644 --- a/client/rest/src/main/java/org/elasticsearch/client/RestClient.java +++ b/client/rest/src/main/java/org/elasticsearch/client/RestClient.java @@ -67,6 +67,7 @@ import java.util.Map; import java.util.Objects; import java.util.Set; +import java.util.concurrent.CancellationException; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ExecutionException; @@ -277,13 +278,15 @@ private ResponseOrResponseException convertResponse(InternalRequest request, Nod * @param responseListener the {@link ResponseListener} to notify when the * request is completed or fails */ - public void performRequestAsync(Request request, ResponseListener responseListener) { + public Cancellable performRequestAsync(Request request, ResponseListener responseListener) { try { FailureTrackingResponseListener failureTrackingResponseListener = new FailureTrackingResponseListener(responseListener); InternalRequest internalRequest = new InternalRequest(request); performRequestAsync(nextNodes(), internalRequest, failureTrackingResponseListener); + return new Cancellable(internalRequest.httpRequest::abort); } catch (Exception e) { responseListener.onFailure(e); + return new Cancellable(() -> {}); } } @@ -329,7 +332,7 @@ public void failed(Exception failure) { @Override public void cancelled() { - listener.onDefinitiveFailure(new ExecutionException("request was cancelled", null)); + listener.onDefinitiveFailure(new CancellationException("request was aborted")); } }); } From 862c1117724e6fd71577e3c658637d49b5d846f2 Mon Sep 17 00:00:00 2001 From: Luca Cavanna Date: Thu, 8 Aug 2019 17:08:31 +0200 Subject: [PATCH 26/28] update tests wip --- .../http/SearchHttpCancellationIT.java | 140 ++++++++++-- .../http/SearchRestCancellationIT.java | 216 ++++++++++++++++++ 2 files changed, 335 insertions(+), 21 deletions(-) create mode 100644 qa/smoke-test-http/src/test/java/org/elasticsearch/http/SearchRestCancellationIT.java diff --git a/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SearchHttpCancellationIT.java b/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SearchHttpCancellationIT.java index 9a96366f4cc62..9561021fccad8 100644 --- a/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SearchHttpCancellationIT.java +++ b/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SearchHttpCancellationIT.java @@ -20,15 +20,19 @@ import org.apache.http.HttpHost; import org.apache.http.HttpResponse; +import org.apache.http.client.config.RequestConfig; import org.apache.http.client.methods.HttpPost; +import org.apache.http.client.methods.HttpRequestBase; import org.apache.http.client.protocol.HttpClientContext; +import org.apache.http.concurrent.FutureCallback; import org.apache.http.entity.ContentType; +import org.apache.http.impl.client.TargetAuthenticationStrategy; import org.apache.http.impl.nio.client.CloseableHttpAsyncClient; import org.apache.http.impl.nio.client.HttpAsyncClientBuilder; import org.apache.http.nio.client.methods.HttpAsyncMethods; import org.apache.http.nio.entity.NStringEntity; +import org.apache.http.nio.protocol.BasicAsyncResponseConsumer; import org.apache.http.nio.protocol.HttpAsyncRequestProducer; -import org.apache.http.nio.protocol.HttpAsyncResponseConsumer; import org.apache.logging.log4j.LogManager; import org.apache.lucene.util.SetOnce; import org.elasticsearch.action.admin.cluster.node.info.NodeInfo; @@ -37,11 +41,9 @@ import org.elasticsearch.action.bulk.BulkRequestBuilder; import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.support.WriteRequest; -import org.elasticsearch.client.HttpAsyncResponseConsumerFactory; import org.elasticsearch.common.Strings; import org.elasticsearch.common.network.NetworkAddress; import org.elasticsearch.common.transport.TransportAddress; -import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.plugins.PluginsService; import org.elasticsearch.script.MockScriptPlugin; @@ -55,12 +57,10 @@ import org.elasticsearch.tasks.TaskInfo; import org.elasticsearch.tasks.TaskManager; import org.elasticsearch.transport.TransportService; -import org.junit.AfterClass; -import org.junit.Before; -import java.io.IOException; import java.net.InetSocketAddress; import java.security.AccessController; +import java.security.PrivilegedAction; import java.security.PrivilegedActionException; import java.security.PrivilegedExceptionAction; import java.util.ArrayList; @@ -69,11 +69,13 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.concurrent.CancellationException; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; import static org.elasticsearch.http.SearchHttpCancellationIT.ScriptedBlockPlugin.SCRIPT_NAME; @@ -85,7 +87,7 @@ public class SearchHttpCancellationIT extends HttpSmokeTestCase { private static CloseableHttpAsyncClient client; - private static List hosts = new ArrayList<>(); + private static HttpHost httpHost; private static Map nodeIdToName = new HashMap<>(); @Override @@ -96,6 +98,7 @@ protected Collection> nodePlugins() { return plugins; } +/* @Before public void init() throws Exception { if (client == null) { @@ -109,42 +112,135 @@ public void init() throws Exception { } catch (PrivilegedActionException e) { throw (Exception) e.getCause(); } - readNodesInfo(hosts, nodeIdToName); + readNodesInfo(nodeIdToName); } } +*/ +/* @AfterClass public static void closeClient() throws IOException { IOUtils.close(client); client = null; } +*/ public void testAutomaticCancellationDuringQueryPhase() throws Exception { List plugins = initBlockFactory(); indexTestData(); - HttpPost httpPost = new HttpPost("/test/_search"); SearchSourceBuilder searchSource = new SearchSourceBuilder().query(scriptQuery( new Script(ScriptType.INLINE, "mockscript", SCRIPT_NAME, Collections.emptyMap()))); - httpPost.setEntity(new NStringEntity(Strings.toString(searchSource), ContentType.APPLICATION_JSON)); - HttpAsyncRequestProducer requestProducer = HttpAsyncMethods.create(randomFrom(hosts), httpPost); + NodesInfoResponse nodesInfoResponse = client().admin().cluster().prepareNodesInfo().get(); + assertFalse(nodesInfoResponse.hasFailures()); + + HttpHost httpHost = null; + for (NodeInfo node : nodesInfoResponse.getNodes()) { + if (node.getHttp() != null) { + TransportAddress publishAddress = node.getHttp().address().publishAddress(); + InetSocketAddress address = publishAddress.address(); + httpHost = new HttpHost(NetworkAddress.format(address.getAddress()), address.getPort(), "http"); + break; + } + } + + + try (CloseableHttpAsyncClient httpClient = AccessController.doPrivileged( + (PrivilegedAction) this::createHttpClient)) { + httpClient.start(); + + HttpPost httpPost = new HttpPost("/test/_search"); + httpPost.setEntity(new NStringEntity(Strings.toString(searchSource), ContentType.APPLICATION_JSON)); + HttpAsyncRequestProducer requestProducer = HttpAsyncMethods.create(httpHost, httpPost); + BasicAsyncResponseConsumer responseConsumer = new BasicAsyncResponseConsumer(); + HttpClientContext context = HttpClientContext.create(); + AtomicReference error = new AtomicReference<>(); + CountDownLatch latch = new CountDownLatch(1); + httpClient.execute(requestProducer, responseConsumer, context, new FutureCallback() { + @Override + public void completed(HttpResponse result) { + latch.countDown(); + } + + @Override + public void failed(Exception ex) { + error.set(ex); + latch.countDown(); + } + + @Override + public void cancelled() { + + } + }); + latch.await(); + throw error.get(); + } + +/* + CountDownLatch latch = new CountDownLatch(1); + Request post = new Request("POST", "/test/_search"); + post.setJsonEntity(Strings.toString(searchSource)); + getRestClient().performRequestAsync(post, new ResponseListener() { + @Override + public void onSuccess(Response response) { + System.out.println("response onSuccess"); + latch.countDown(); + } + + @Override + public void onFailure(Exception exception) { + System.out.println("response onFailure"); + exception.printStackTrace(); + latch.countDown(); + } + }); + + awaitForBlock(plugins); + + disableBlocks(plugins); + + latch.await(); +*/ + + + //sendRequest(httpHost, new HttpGet("/")); + + /*HttpPost httpPost = new HttpPost("/test/_search"); + + httpPost.setEntity(new NStringEntity(Strings.toString(searchSource), ContentType.APPLICATION_JSON)); + HttpAsyncRequestProducer requestProducer = HttpAsyncMethods.create(httpHost, httpPost); HttpAsyncResponseConsumer httpAsyncResponseConsumer = HttpAsyncResponseConsumerFactory.DEFAULT.createHttpAsyncResponseConsumer(); HttpClientContext context = HttpClientContext.create(); Future future = client.execute(requestProducer, httpAsyncResponseConsumer, context, null); - future.get(); //TODO remove this, it is here just to see the security manager error + future.get();//TODO remove awaitForBlock(plugins); httpPost.abort(); ensureSearchTaskIsCancelled(nodeIdToName::get); disableBlocks(plugins); - expectThrows(CancellationException.class, future::get); + expectThrows(CancellationException.class, future::get);*/ } - public void testAutomaticCancellationDuringFetchPhase() throws Exception { + private CloseableHttpAsyncClient createHttpClient() { + //default timeouts are all infinite + RequestConfig.Builder requestConfigBuilder = RequestConfig.custom() + .setConnectTimeout(1000) + .setSocketTimeout(30_000); + + HttpAsyncClientBuilder httpClientBuilder = HttpAsyncClientBuilder.create().setDefaultRequestConfig(requestConfigBuilder.build()) + //default settings for connection pooling may be too constraining + .setMaxConnPerRoute(10).setMaxConnTotal(30) + .setTargetAuthenticationStrategy(TargetAuthenticationStrategy.INSTANCE); + httpClientBuilder.setThreadFactory(Executors.privilegedThreadFactory()); + return AccessController.doPrivileged((PrivilegedAction) httpClientBuilder::build); + } + +/* public void testAutomaticCancellationDuringFetchPhase() throws Exception { List plugins = initBlockFactory(); indexTestData(); @@ -153,8 +249,8 @@ public void testAutomaticCancellationDuringFetchPhase() throws Exception { new Script(ScriptType.INLINE, "mockscript", SCRIPT_NAME, Collections.emptyMap())); httpPost.setEntity(new NStringEntity(Strings.toString(searchSource), ContentType.APPLICATION_JSON)); - Future future = sendRequest(randomFrom(hosts), httpPost); - future.get(); //TODO remove this, it is here just to see the security manager error + Future future = sendRequest(httpHost, httpPost); + future.get();//TODO remove awaitForBlock(plugins); httpPost.abort(); @@ -162,20 +258,21 @@ public void testAutomaticCancellationDuringFetchPhase() throws Exception { disableBlocks(plugins); expectThrows(CancellationException.class, future::get); - } + }*/ - private static Future sendRequest(HttpHost httpHost, HttpPost httpPost) throws Exception { + private static Future sendRequest(HttpHost httpHost, HttpRequestBase httpRequest) throws Exception { try { return AccessController.doPrivileged((PrivilegedExceptionAction>) - () -> client.execute(httpHost, httpPost, null)); + () -> client.execute(httpHost, httpRequest, null)); } catch (PrivilegedActionException e) { throw (Exception) e.getCause(); } } - private static void readNodesInfo(List hosts, Map nodeIdToName) { + private static void readNodesInfo(Map nodeIdToName) { NodesInfoResponse nodesInfoResponse = client().admin().cluster().prepareNodesInfo().get(); assertFalse(nodesInfoResponse.hasFailures()); + List hosts = new ArrayList<>(); for (NodeInfo node : nodesInfoResponse.getNodes()) { if (node.getHttp() != null) { TransportAddress publishAddress = node.getHttp().address().publishAddress(); @@ -184,6 +281,7 @@ private static void readNodesInfo(List hosts, Map node } nodeIdToName.put(node.getNode().getId(), node.getNode().getName()); } + httpHost = randomFrom(hosts); } private static void ensureSearchTaskIsCancelled(Function nodeIdToName) { diff --git a/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SearchRestCancellationIT.java b/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SearchRestCancellationIT.java new file mode 100644 index 0000000000000..aa0d0261743b1 --- /dev/null +++ b/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SearchRestCancellationIT.java @@ -0,0 +1,216 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.elasticsearch.http; + +import org.apache.logging.log4j.LogManager; +import org.apache.lucene.util.SetOnce; +import org.elasticsearch.action.admin.cluster.node.info.NodeInfo; +import org.elasticsearch.action.admin.cluster.node.info.NodesInfoResponse; +import org.elasticsearch.action.admin.cluster.node.tasks.list.ListTasksResponse; +import org.elasticsearch.action.bulk.BulkRequestBuilder; +import org.elasticsearch.action.search.SearchAction; +import org.elasticsearch.action.support.WriteRequest; +import org.elasticsearch.client.Cancellable; +import org.elasticsearch.client.Request; +import org.elasticsearch.client.Response; +import org.elasticsearch.client.ResponseListener; +import org.elasticsearch.common.Strings; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.plugins.PluginsService; +import org.elasticsearch.script.MockScriptPlugin; +import org.elasticsearch.script.Script; +import org.elasticsearch.script.ScriptType; +import org.elasticsearch.search.builder.SearchSourceBuilder; +import org.elasticsearch.search.lookup.LeafFieldsLookup; +import org.elasticsearch.tasks.CancellableTask; +import org.elasticsearch.tasks.Task; +import org.elasticsearch.tasks.TaskId; +import org.elasticsearch.tasks.TaskInfo; +import org.elasticsearch.tasks.TaskManager; +import org.elasticsearch.transport.TransportService; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CancellationException; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Function; + +import static org.elasticsearch.index.query.QueryBuilders.scriptQuery; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoFailures; +import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.instanceOf; + +public class SearchRestCancellationIT extends HttpSmokeTestCase { + + @Override + protected Collection> nodePlugins() { + List> plugins = new ArrayList<>(); + plugins.add(ScriptedBlockPlugin.class); + plugins.addAll(super.nodePlugins()); + return plugins; + } + + public void testAutomaticCancellationDuringQueryPhase() throws Exception { + Map nodeIdToName = readNodesInfo(); + + List plugins = initBlockFactory(); + indexTestData(); + + Request searchRequest = new Request("GET", "/test/_search"); + SearchSourceBuilder searchSource = new SearchSourceBuilder().query(scriptQuery( + new Script(ScriptType.INLINE, "mockscript", ScriptedBlockPlugin.SCRIPT_NAME, Collections.emptyMap()))); + searchRequest.setJsonEntity(Strings.toString(searchSource)); + + CountDownLatch latch = new CountDownLatch(1); + AtomicReference error = new AtomicReference<>(); + Cancellable cancellable = getRestClient().performRequestAsync(searchRequest, new ResponseListener() { + @Override + public void onSuccess(Response response) { + latch.countDown(); + } + + @Override + public void onFailure(Exception exception) { + error.set(exception); + latch.countDown(); + } + }); + + awaitForBlock(plugins); + assertTrue(cancellable.cancel()); + ensureSearchTaskIsCancelled(nodeIdToName::get); + + disableBlocks(plugins); + latch.await(); + assertThat(error.get(), instanceOf(CancellationException.class)); + } + + //TODO test with retries! + + private static Map readNodesInfo() { + Map nodeIdToName = new HashMap<>(); + NodesInfoResponse nodesInfoResponse = client().admin().cluster().prepareNodesInfo().get(); + assertFalse(nodesInfoResponse.hasFailures()); + for (NodeInfo node : nodesInfoResponse.getNodes()) { + nodeIdToName.put(node.getNode().getId(), node.getNode().getName()); + } + return nodeIdToName; + } + + private static void ensureSearchTaskIsCancelled(Function nodeIdToName) { + SetOnce searchTask = new SetOnce<>(); + ListTasksResponse listTasksResponse = client().admin().cluster().prepareListTasks().get(); + for (TaskInfo task : listTasksResponse.getTasks()) { + if (task.getAction().equals(SearchAction.NAME)) { + searchTask.set(task); + } + } + assertNotNull(searchTask.get()); + TaskId taskId = searchTask.get().getTaskId(); + String nodeName = nodeIdToName.apply(taskId.getNodeId()); + TaskManager taskManager = internalCluster().getInstance(TransportService.class, nodeName).getTaskManager(); + Task task = taskManager.getTask(taskId.getId()); + assertThat(task, instanceOf(CancellableTask.class)); + assertTrue(((CancellableTask)task).isCancelled()); + } + + private static void indexTestData() { + for (int i = 0; i < 5; i++) { + // Make sure we have a few segments + BulkRequestBuilder bulkRequestBuilder = client().prepareBulk().setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE); + for (int j = 0; j < 20; j++) { + bulkRequestBuilder.add(client().prepareIndex("test", "_doc", Integer.toString(i * 5 + j)).setSource("field", "value")); + } + assertNoFailures(bulkRequestBuilder.get()); + } + } + + private static List initBlockFactory() { + List plugins = new ArrayList<>(); + for (PluginsService pluginsService : internalCluster().getDataNodeInstances(PluginsService.class)) { + plugins.addAll(pluginsService.filterPlugins(ScriptedBlockPlugin.class)); + } + for (ScriptedBlockPlugin plugin : plugins) { + plugin.reset(); + plugin.enableBlock(); + } + return plugins; + } + + private void awaitForBlock(List plugins) throws Exception { + int numberOfShards = getNumShards("test").numPrimaries; + assertBusy(() -> { + int numberOfBlockedPlugins = 0; + for (ScriptedBlockPlugin plugin : plugins) { + numberOfBlockedPlugins += plugin.hits.get(); + } + logger.info("The plugin blocked on {} out of {} shards", numberOfBlockedPlugins, numberOfShards); + assertThat(numberOfBlockedPlugins, greaterThan(0)); + }, 10, TimeUnit.SECONDS); + } + + private static void disableBlocks(List plugins) { + for (ScriptedBlockPlugin plugin : plugins) { + plugin.disableBlock(); + } + } + + public static class ScriptedBlockPlugin extends MockScriptPlugin { + static final String SCRIPT_NAME = "search_block"; + + private final AtomicInteger hits = new AtomicInteger(); + + private final AtomicBoolean shouldBlock = new AtomicBoolean(true); + + void reset() { + hits.set(0); + } + + void disableBlock() { + shouldBlock.set(false); + } + + void enableBlock() { + shouldBlock.set(true); + } + + @Override + public Map, Object>> pluginScripts() { + return Collections.singletonMap(SCRIPT_NAME, params -> { + LeafFieldsLookup fieldsLookup = (LeafFieldsLookup) params.get("_fields"); + LogManager.getLogger(SearchRestCancellationIT.class).info("Blocking on the document {}", fieldsLookup.get("_id")); + hits.incrementAndGet(); + try { + awaitBusy(() -> shouldBlock.get() == false); + } catch (Exception e) { + throw new RuntimeException(e); + } + return true; + }); + } + } +} From b4a7d5dbefe7c5e33de6f33fd722ba65d5713801 Mon Sep 17 00:00:00 2001 From: Luca Cavanna Date: Tue, 13 Aug 2019 15:21:52 +0200 Subject: [PATCH 27/28] extend test --- .../http/SearchHttpCancellationIT.java | 379 ------------------ .../http/SearchRestCancellationIT.java | 37 +- 2 files changed, 35 insertions(+), 381 deletions(-) delete mode 100644 qa/smoke-test-http/src/test/java/org/elasticsearch/http/SearchHttpCancellationIT.java diff --git a/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SearchHttpCancellationIT.java b/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SearchHttpCancellationIT.java deleted file mode 100644 index 9561021fccad8..0000000000000 --- a/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SearchHttpCancellationIT.java +++ /dev/null @@ -1,379 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.elasticsearch.http; - -import org.apache.http.HttpHost; -import org.apache.http.HttpResponse; -import org.apache.http.client.config.RequestConfig; -import org.apache.http.client.methods.HttpPost; -import org.apache.http.client.methods.HttpRequestBase; -import org.apache.http.client.protocol.HttpClientContext; -import org.apache.http.concurrent.FutureCallback; -import org.apache.http.entity.ContentType; -import org.apache.http.impl.client.TargetAuthenticationStrategy; -import org.apache.http.impl.nio.client.CloseableHttpAsyncClient; -import org.apache.http.impl.nio.client.HttpAsyncClientBuilder; -import org.apache.http.nio.client.methods.HttpAsyncMethods; -import org.apache.http.nio.entity.NStringEntity; -import org.apache.http.nio.protocol.BasicAsyncResponseConsumer; -import org.apache.http.nio.protocol.HttpAsyncRequestProducer; -import org.apache.logging.log4j.LogManager; -import org.apache.lucene.util.SetOnce; -import org.elasticsearch.action.admin.cluster.node.info.NodeInfo; -import org.elasticsearch.action.admin.cluster.node.info.NodesInfoResponse; -import org.elasticsearch.action.admin.cluster.node.tasks.list.ListTasksResponse; -import org.elasticsearch.action.bulk.BulkRequestBuilder; -import org.elasticsearch.action.search.SearchAction; -import org.elasticsearch.action.support.WriteRequest; -import org.elasticsearch.common.Strings; -import org.elasticsearch.common.network.NetworkAddress; -import org.elasticsearch.common.transport.TransportAddress; -import org.elasticsearch.plugins.Plugin; -import org.elasticsearch.plugins.PluginsService; -import org.elasticsearch.script.MockScriptPlugin; -import org.elasticsearch.script.Script; -import org.elasticsearch.script.ScriptType; -import org.elasticsearch.search.builder.SearchSourceBuilder; -import org.elasticsearch.search.lookup.LeafFieldsLookup; -import org.elasticsearch.tasks.CancellableTask; -import org.elasticsearch.tasks.Task; -import org.elasticsearch.tasks.TaskId; -import org.elasticsearch.tasks.TaskInfo; -import org.elasticsearch.tasks.TaskManager; -import org.elasticsearch.transport.TransportService; - -import java.net.InetSocketAddress; -import java.security.AccessController; -import java.security.PrivilegedAction; -import java.security.PrivilegedActionException; -import java.security.PrivilegedExceptionAction; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.Executors; -import java.util.concurrent.Future; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicReference; -import java.util.function.Function; - -import static org.elasticsearch.http.SearchHttpCancellationIT.ScriptedBlockPlugin.SCRIPT_NAME; -import static org.elasticsearch.index.query.QueryBuilders.scriptQuery; -import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoFailures; -import static org.hamcrest.Matchers.greaterThan; -import static org.hamcrest.Matchers.instanceOf; - -public class SearchHttpCancellationIT extends HttpSmokeTestCase { - - private static CloseableHttpAsyncClient client; - private static HttpHost httpHost; - private static Map nodeIdToName = new HashMap<>(); - - @Override - protected Collection> nodePlugins() { - List> plugins = new ArrayList<>(); - plugins.add(ScriptedBlockPlugin.class); - plugins.addAll(super.nodePlugins()); - return plugins; - } - -/* - @Before - public void init() throws Exception { - if (client == null) { - try { - client = AccessController.doPrivileged((PrivilegedExceptionAction) - () -> client = HttpAsyncClientBuilder.create().build()); - AccessController.doPrivileged((PrivilegedExceptionAction) () -> { - client.start(); - return null; - }); - } catch (PrivilegedActionException e) { - throw (Exception) e.getCause(); - } - readNodesInfo(nodeIdToName); - } - } -*/ - -/* - @AfterClass - public static void closeClient() throws IOException { - IOUtils.close(client); - client = null; - } -*/ - - public void testAutomaticCancellationDuringQueryPhase() throws Exception { - List plugins = initBlockFactory(); - indexTestData(); - - SearchSourceBuilder searchSource = new SearchSourceBuilder().query(scriptQuery( - new Script(ScriptType.INLINE, "mockscript", SCRIPT_NAME, Collections.emptyMap()))); - - NodesInfoResponse nodesInfoResponse = client().admin().cluster().prepareNodesInfo().get(); - assertFalse(nodesInfoResponse.hasFailures()); - - HttpHost httpHost = null; - for (NodeInfo node : nodesInfoResponse.getNodes()) { - if (node.getHttp() != null) { - TransportAddress publishAddress = node.getHttp().address().publishAddress(); - InetSocketAddress address = publishAddress.address(); - httpHost = new HttpHost(NetworkAddress.format(address.getAddress()), address.getPort(), "http"); - break; - } - } - - - try (CloseableHttpAsyncClient httpClient = AccessController.doPrivileged( - (PrivilegedAction) this::createHttpClient)) { - httpClient.start(); - - HttpPost httpPost = new HttpPost("/test/_search"); - httpPost.setEntity(new NStringEntity(Strings.toString(searchSource), ContentType.APPLICATION_JSON)); - HttpAsyncRequestProducer requestProducer = HttpAsyncMethods.create(httpHost, httpPost); - BasicAsyncResponseConsumer responseConsumer = new BasicAsyncResponseConsumer(); - HttpClientContext context = HttpClientContext.create(); - AtomicReference error = new AtomicReference<>(); - CountDownLatch latch = new CountDownLatch(1); - httpClient.execute(requestProducer, responseConsumer, context, new FutureCallback() { - @Override - public void completed(HttpResponse result) { - latch.countDown(); - } - - @Override - public void failed(Exception ex) { - error.set(ex); - latch.countDown(); - } - - @Override - public void cancelled() { - - } - }); - latch.await(); - throw error.get(); - } - -/* - CountDownLatch latch = new CountDownLatch(1); - Request post = new Request("POST", "/test/_search"); - post.setJsonEntity(Strings.toString(searchSource)); - getRestClient().performRequestAsync(post, new ResponseListener() { - @Override - public void onSuccess(Response response) { - System.out.println("response onSuccess"); - latch.countDown(); - } - - @Override - public void onFailure(Exception exception) { - System.out.println("response onFailure"); - exception.printStackTrace(); - latch.countDown(); - } - }); - - awaitForBlock(plugins); - - disableBlocks(plugins); - - latch.await(); -*/ - - - //sendRequest(httpHost, new HttpGet("/")); - - /*HttpPost httpPost = new HttpPost("/test/_search"); - - httpPost.setEntity(new NStringEntity(Strings.toString(searchSource), ContentType.APPLICATION_JSON)); - HttpAsyncRequestProducer requestProducer = HttpAsyncMethods.create(httpHost, httpPost); - HttpAsyncResponseConsumer httpAsyncResponseConsumer = - HttpAsyncResponseConsumerFactory.DEFAULT.createHttpAsyncResponseConsumer(); - HttpClientContext context = HttpClientContext.create(); - - Future future = client.execute(requestProducer, httpAsyncResponseConsumer, context, null); - future.get();//TODO remove - awaitForBlock(plugins); - - httpPost.abort(); - ensureSearchTaskIsCancelled(nodeIdToName::get); - - disableBlocks(plugins); - expectThrows(CancellationException.class, future::get);*/ - } - - private CloseableHttpAsyncClient createHttpClient() { - //default timeouts are all infinite - RequestConfig.Builder requestConfigBuilder = RequestConfig.custom() - .setConnectTimeout(1000) - .setSocketTimeout(30_000); - - HttpAsyncClientBuilder httpClientBuilder = HttpAsyncClientBuilder.create().setDefaultRequestConfig(requestConfigBuilder.build()) - //default settings for connection pooling may be too constraining - .setMaxConnPerRoute(10).setMaxConnTotal(30) - .setTargetAuthenticationStrategy(TargetAuthenticationStrategy.INSTANCE); - httpClientBuilder.setThreadFactory(Executors.privilegedThreadFactory()); - return AccessController.doPrivileged((PrivilegedAction) httpClientBuilder::build); - } - -/* public void testAutomaticCancellationDuringFetchPhase() throws Exception { - List plugins = initBlockFactory(); - indexTestData(); - - HttpPost httpPost = new HttpPost("/test/_search"); - SearchSourceBuilder searchSource = new SearchSourceBuilder().scriptField("test_field", - new Script(ScriptType.INLINE, "mockscript", SCRIPT_NAME, Collections.emptyMap())); - httpPost.setEntity(new NStringEntity(Strings.toString(searchSource), ContentType.APPLICATION_JSON)); - - Future future = sendRequest(httpHost, httpPost); - future.get();//TODO remove - awaitForBlock(plugins); - - httpPost.abort(); - ensureSearchTaskIsCancelled(nodeIdToName::get); - - disableBlocks(plugins); - expectThrows(CancellationException.class, future::get); - }*/ - - private static Future sendRequest(HttpHost httpHost, HttpRequestBase httpRequest) throws Exception { - try { - return AccessController.doPrivileged((PrivilegedExceptionAction>) - () -> client.execute(httpHost, httpRequest, null)); - } catch (PrivilegedActionException e) { - throw (Exception) e.getCause(); - } - } - - private static void readNodesInfo(Map nodeIdToName) { - NodesInfoResponse nodesInfoResponse = client().admin().cluster().prepareNodesInfo().get(); - assertFalse(nodesInfoResponse.hasFailures()); - List hosts = new ArrayList<>(); - for (NodeInfo node : nodesInfoResponse.getNodes()) { - if (node.getHttp() != null) { - TransportAddress publishAddress = node.getHttp().address().publishAddress(); - InetSocketAddress address = publishAddress.address(); - hosts.add(new HttpHost(NetworkAddress.format(address.getAddress()), address.getPort(), "http")); - } - nodeIdToName.put(node.getNode().getId(), node.getNode().getName()); - } - httpHost = randomFrom(hosts); - } - - private static void ensureSearchTaskIsCancelled(Function nodeIdToName) { - SetOnce searchTask = new SetOnce<>(); - ListTasksResponse listTasksResponse = client().admin().cluster().prepareListTasks().get(); - for (TaskInfo task : listTasksResponse.getTasks()) { - if (task.getAction().equals(SearchAction.NAME)) { - searchTask.set(task); - } - } - assertNotNull(searchTask.get()); - TaskId taskId = searchTask.get().getTaskId(); - String nodeName = nodeIdToName.apply(taskId.getNodeId()); - TaskManager taskManager = internalCluster().getInstance(TransportService.class, nodeName).getTaskManager(); - Task task = taskManager.getTask(taskId.getId()); - assertThat(task, instanceOf(CancellableTask.class)); - assertTrue(((CancellableTask)task).isCancelled()); - } - - private static void indexTestData() { - for (int i = 0; i < 5; i++) { - // Make sure we have a few segments - BulkRequestBuilder bulkRequestBuilder = client().prepareBulk().setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE); - for (int j = 0; j < 20; j++) { - bulkRequestBuilder.add(client().prepareIndex("test", "_doc", Integer.toString(i * 5 + j)).setSource("field", "value")); - } - assertNoFailures(bulkRequestBuilder.get()); - } - } - - private static List initBlockFactory() { - List plugins = new ArrayList<>(); - for (PluginsService pluginsService : internalCluster().getDataNodeInstances(PluginsService.class)) { - plugins.addAll(pluginsService.filterPlugins(ScriptedBlockPlugin.class)); - } - for (ScriptedBlockPlugin plugin : plugins) { - plugin.reset(); - plugin.enableBlock(); - } - return plugins; - } - - private void awaitForBlock(List plugins) throws Exception { - int numberOfShards = getNumShards("test").numPrimaries; - assertBusy(() -> { - int numberOfBlockedPlugins = 0; - for (ScriptedBlockPlugin plugin : plugins) { - numberOfBlockedPlugins += plugin.hits.get(); - } - logger.info("The plugin blocked on {} out of {} shards", numberOfBlockedPlugins, numberOfShards); - assertThat(numberOfBlockedPlugins, greaterThan(0)); - }, 20, TimeUnit.SECONDS); - } - - private static void disableBlocks(List plugins) { - for (ScriptedBlockPlugin plugin : plugins) { - plugin.disableBlock(); - } - } - - public static class ScriptedBlockPlugin extends MockScriptPlugin { - static final String SCRIPT_NAME = "search_block"; - - private final AtomicInteger hits = new AtomicInteger(); - - private final AtomicBoolean shouldBlock = new AtomicBoolean(true); - - void reset() { - hits.set(0); - } - - void disableBlock() { - shouldBlock.set(false); - } - - void enableBlock() { - shouldBlock.set(true); - } - - @Override - public Map, Object>> pluginScripts() { - return Collections.singletonMap(SCRIPT_NAME, params -> { - LeafFieldsLookup fieldsLookup = (LeafFieldsLookup) params.get("_fields"); - LogManager.getLogger(SearchHttpCancellationIT.class).info("Blocking on the document {}", fieldsLookup.get("_id")); - hits.incrementAndGet(); - try { - awaitBusy(() -> shouldBlock.get() == false); - } catch (Exception e) { - throw new RuntimeException(e); - } - return true; - }); - } - } -} diff --git a/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SearchRestCancellationIT.java b/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SearchRestCancellationIT.java index aa0d0261743b1..73ba8cdf8b331 100644 --- a/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SearchRestCancellationIT.java +++ b/qa/smoke-test-http/src/test/java/org/elasticsearch/http/SearchRestCancellationIT.java @@ -101,7 +101,7 @@ public void onFailure(Exception exception) { }); awaitForBlock(plugins); - assertTrue(cancellable.cancel()); + cancellable.cancel(); ensureSearchTaskIsCancelled(nodeIdToName::get); disableBlocks(plugins); @@ -109,7 +109,40 @@ public void onFailure(Exception exception) { assertThat(error.get(), instanceOf(CancellationException.class)); } - //TODO test with retries! + public void testAutomaticCancellationDuringFetchPhase() throws Exception { + Map nodeIdToName = readNodesInfo(); + + List plugins = initBlockFactory(); + indexTestData(); + + Request searchRequest = new Request("GET", "/test/_search"); + SearchSourceBuilder searchSource = new SearchSourceBuilder().scriptField("test_field", + new Script(ScriptType.INLINE, "mockscript", ScriptedBlockPlugin.SCRIPT_NAME, Collections.emptyMap())); + searchRequest.setJsonEntity(Strings.toString(searchSource)); + + CountDownLatch latch = new CountDownLatch(1); + AtomicReference error = new AtomicReference<>(); + Cancellable cancellable = getRestClient().performRequestAsync(searchRequest, new ResponseListener() { + @Override + public void onSuccess(Response response) { + latch.countDown(); + } + + @Override + public void onFailure(Exception exception) { + error.set(exception); + latch.countDown(); + } + }); + + awaitForBlock(plugins); + cancellable.cancel(); + ensureSearchTaskIsCancelled(nodeIdToName::get); + + disableBlocks(plugins); + latch.await(); + assertThat(error.get(), instanceOf(CancellationException.class)); + } private static Map readNodesInfo() { Map nodeIdToName = new HashMap<>(); From 83497abbe69f1f55962a7ce6365d7f779a07ae12 Mon Sep 17 00:00:00 2001 From: Luca Cavanna Date: Tue, 13 Aug 2019 19:06:56 +0200 Subject: [PATCH 28/28] Revert "abort requests" This reverts commit 6f83fd8569f7d8e4fe5ae6fbdff86502334000b8. --- .../org/elasticsearch/client/Cancellable.java | 39 ------------------- .../org/elasticsearch/client/RestClient.java | 7 +--- 2 files changed, 2 insertions(+), 44 deletions(-) delete mode 100644 client/rest/src/main/java/org/elasticsearch/client/Cancellable.java diff --git a/client/rest/src/main/java/org/elasticsearch/client/Cancellable.java b/client/rest/src/main/java/org/elasticsearch/client/Cancellable.java deleted file mode 100644 index 4ec8a79eb57be..0000000000000 --- a/client/rest/src/main/java/org/elasticsearch/client/Cancellable.java +++ /dev/null @@ -1,39 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.elasticsearch.client; - -import java.util.concurrent.atomic.AtomicBoolean; - -public final class Cancellable { - - private final AtomicBoolean cancelled = new AtomicBoolean(false); - private final Runnable runOnCancel; - - Cancellable(Runnable runOnCancel) { - this.runOnCancel = runOnCancel; - } - - public boolean cancel() { - if (this.cancelled.compareAndSet(false, true)) { - this.runOnCancel.run(); - return true; - } - return false; - } -} diff --git a/client/rest/src/main/java/org/elasticsearch/client/RestClient.java b/client/rest/src/main/java/org/elasticsearch/client/RestClient.java index 5867905f3f1d7..38185ac960926 100644 --- a/client/rest/src/main/java/org/elasticsearch/client/RestClient.java +++ b/client/rest/src/main/java/org/elasticsearch/client/RestClient.java @@ -67,7 +67,6 @@ import java.util.Map; import java.util.Objects; import java.util.Set; -import java.util.concurrent.CancellationException; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ExecutionException; @@ -278,15 +277,13 @@ private ResponseOrResponseException convertResponse(InternalRequest request, Nod * @param responseListener the {@link ResponseListener} to notify when the * request is completed or fails */ - public Cancellable performRequestAsync(Request request, ResponseListener responseListener) { + public void performRequestAsync(Request request, ResponseListener responseListener) { try { FailureTrackingResponseListener failureTrackingResponseListener = new FailureTrackingResponseListener(responseListener); InternalRequest internalRequest = new InternalRequest(request); performRequestAsync(nextNodes(), internalRequest, failureTrackingResponseListener); - return new Cancellable(internalRequest.httpRequest::abort); } catch (Exception e) { responseListener.onFailure(e); - return new Cancellable(() -> {}); } } @@ -332,7 +329,7 @@ public void failed(Exception failure) { @Override public void cancelled() { - listener.onDefinitiveFailure(new CancellationException("request was aborted")); + listener.onDefinitiveFailure(new ExecutionException("request was cancelled", null)); } }); }