17
17
* under the License.
18
18
*/
19
19
20
- package org .elasticsearch .rest .action . search ;
20
+ package org .elasticsearch .rest .action ;
21
21
22
22
import org .elasticsearch .action .ActionListener ;
23
23
import org .elasticsearch .action .ActionRequest ;
24
24
import org .elasticsearch .action .ActionResponse ;
25
25
import org .elasticsearch .action .ActionType ;
26
26
import org .elasticsearch .action .admin .cluster .node .tasks .cancel .CancelTasksRequest ;
27
- import org .elasticsearch .action .admin .cluster .node .tasks .cancel .CancelTasksResponse ;
28
- import org .elasticsearch .action .support .ContextPreservingActionListener ;
29
27
import org .elasticsearch .client .Client ;
28
+ import org .elasticsearch .client .FilterClient ;
29
+ import org .elasticsearch .client .OriginSettingClient ;
30
30
import org .elasticsearch .client .node .NodeClient ;
31
- import org .elasticsearch .common .util .concurrent .ThreadContext ;
32
31
import org .elasticsearch .http .HttpChannel ;
33
32
import org .elasticsearch .tasks .Task ;
34
33
import org .elasticsearch .tasks .TaskId ;
35
34
35
+ import java .util .ArrayList ;
36
36
import java .util .HashSet ;
37
+ import java .util .List ;
37
38
import java .util .Map ;
38
39
import java .util .Set ;
39
40
import java .util .concurrent .ConcurrentHashMap ;
40
41
import java .util .concurrent .atomic .AtomicReference ;
41
42
43
+ import static org .elasticsearch .action .admin .cluster .node .tasks .get .GetTaskAction .TASKS_ORIGIN ;
44
+
42
45
/**
43
- * This class executes a request and associates the corresponding {@link Task} with the {@link HttpChannel} that it was originated from,
44
- * so that the tasks associated with a certain channel get cancelled when the underlying connection gets closed .
46
+ * A {@linkplain Client} that cancels tasks executed locally when the provided {@link HttpChannel}
47
+ * is closed before completion .
45
48
*/
46
- public final class HttpChannelTaskHandler {
49
+ public class RestCancellableNodeClient extends FilterClient {
50
+ private static final Map <HttpChannel , CloseListener > httpChannels = new ConcurrentHashMap <>();
51
+
52
+ private final NodeClient client ;
53
+ private final HttpChannel httpChannel ;
47
54
48
- public static final HttpChannelTaskHandler INSTANCE = new HttpChannelTaskHandler ();
49
- //package private for testing
50
- final Map <HttpChannel , CloseListener > httpChannels = new ConcurrentHashMap <>();
55
+ public RestCancellableNodeClient (NodeClient client , HttpChannel httpChannel ) {
56
+ super (client );
57
+ this .client = client ;
58
+ this .httpChannel = httpChannel ;
59
+ }
60
+
61
+ /**
62
+ * Returns the number of channels tracked globally.
63
+ */
64
+ public static int getNumChannels () {
65
+ return httpChannels .size ();
66
+ }
51
67
52
- private HttpChannelTaskHandler () {
68
+ /**
69
+ * Returns the number of tasks tracked globally.
70
+ */
71
+ static int getNumTasks () {
72
+ return httpChannels .values ().stream ()
73
+ .mapToInt (CloseListener ::getNumTasks )
74
+ .sum ();
53
75
}
54
76
55
- <Response extends ActionResponse > void execute (NodeClient client , HttpChannel httpChannel , ActionRequest request ,
56
- ActionType <Response > actionType , ActionListener <Response > listener ) {
77
+ /**
78
+ * Returns the number of tasks tracked by the provided {@link HttpChannel}.
79
+ */
80
+ static int getNumTasks (HttpChannel channel ) {
81
+ CloseListener listener = httpChannels .get (channel );
82
+ return listener == null ? 0 : listener .getNumTasks ();
83
+ }
57
84
58
- CloseListener closeListener = httpChannels .computeIfAbsent (httpChannel , channel -> new CloseListener (client ));
85
+ @ Override
86
+ public <Request extends ActionRequest , Response extends ActionResponse > void doExecute (
87
+ ActionType <Response > action , Request request , ActionListener <Response > listener ) {
88
+ CloseListener closeListener = httpChannels .computeIfAbsent (httpChannel , channel -> new CloseListener ());
59
89
TaskHolder taskHolder = new TaskHolder ();
60
- Task task = client .executeLocally (actionType , request ,
90
+ Task task = client .executeLocally (action , request ,
61
91
new ActionListener <>() {
62
92
@ Override
63
- public void onResponse (Response searchResponse ) {
93
+ public void onResponse (Response response ) {
64
94
try {
65
95
closeListener .unregisterTask (taskHolder );
66
96
} finally {
67
- listener .onResponse (searchResponse );
97
+ listener .onResponse (response );
68
98
}
69
99
}
70
100
@@ -77,32 +107,35 @@ public void onFailure(Exception e) {
77
107
}
78
108
}
79
109
});
80
- closeListener .registerTask (taskHolder , new TaskId (client .getLocalNodeId (), task .getId ()));
110
+ final TaskId taskId = new TaskId (client .getLocalNodeId (), task .getId ());
111
+ closeListener .registerTask (taskHolder , taskId );
81
112
closeListener .maybeRegisterChannel (httpChannel );
82
113
}
83
114
84
- public int getNumChannels () {
85
- return httpChannels .size ();
115
+ private void cancelTask (TaskId taskId ) {
116
+ CancelTasksRequest req = new CancelTasksRequest ()
117
+ .setTaskId (taskId )
118
+ .setReason ("channel closed" );
119
+ // force the origin to execute the cancellation as a system user
120
+ new OriginSettingClient (client , TASKS_ORIGIN ).admin ().cluster ().cancelTasks (req , ActionListener .wrap (() -> {}));
86
121
}
87
122
88
- final class CloseListener implements ActionListener <Void > {
89
- private final Client client ;
123
+ private class CloseListener implements ActionListener <Void > {
90
124
private final AtomicReference <HttpChannel > channel = new AtomicReference <>();
91
- private final Set <TaskId > taskIds = new HashSet <>();
125
+ private final Set <TaskId > tasks = new HashSet <>();
92
126
93
- CloseListener (Client client ) {
94
- this .client = client ;
127
+ CloseListener () {
95
128
}
96
129
97
- int getNumTasks () {
98
- return taskIds .size ();
130
+ synchronized int getNumTasks () {
131
+ return tasks .size ();
99
132
}
100
133
101
134
void maybeRegisterChannel (HttpChannel httpChannel ) {
102
135
if (channel .compareAndSet (null , httpChannel )) {
103
136
//In case the channel is already closed when we register the listener, the listener will be immediately executed which will
104
137
//remove the channel from the map straight-away. That is why we first create the CloseListener and later we associate it
105
- //with the channel. This guarantees that the close listener is already in the map when the it gets registered to its
138
+ //with the channel. This guarantees that the close listener is already in the map when it gets registered to its
106
139
//corresponding channel, hence it is always found in the map when it gets invoked if the channel gets closed.
107
140
httpChannel .addCloseListener (this );
108
141
}
@@ -111,34 +144,31 @@ void maybeRegisterChannel(HttpChannel httpChannel) {
111
144
synchronized void registerTask (TaskHolder taskHolder , TaskId taskId ) {
112
145
taskHolder .taskId = taskId ;
113
146
if (taskHolder .completed == false ) {
114
- this .taskIds .add (taskId );
147
+ this .tasks .add (taskId );
115
148
}
116
149
}
117
150
118
151
synchronized void unregisterTask (TaskHolder taskHolder ) {
119
152
if (taskHolder .taskId != null ) {
120
- this .taskIds .remove (taskHolder .taskId );
153
+ this .tasks .remove (taskHolder .taskId );
121
154
}
122
155
taskHolder .completed = true ;
123
156
}
124
157
125
158
@ Override
126
- public synchronized void onResponse (Void aVoid ) {
127
- //When the channel gets closed it won't be reused: we can remove it from the map and forget about it.
128
- CloseListener closeListener = httpChannels .remove (channel .get ());
159
+ public void onResponse (Void aVoid ) {
160
+ final HttpChannel httpChannel = channel .get ();
161
+ assert httpChannel != null : "channel not registered" ;
162
+ // when the channel gets closed it won't be reused: we can remove it from the map and forget about it.
163
+ CloseListener closeListener = httpChannels .remove (httpChannel );
129
164
assert closeListener != null : "channel not found in the map of tracked channels" ;
130
- for (TaskId taskId : taskIds ) {
131
- ThreadContext threadContext = client .threadPool ().getThreadContext ();
132
- try (ThreadContext .StoredContext ignore = threadContext .stashContext ()) {
133
- // we stash any context here since this is an internal execution and should not leak any existing context information
134
- threadContext .markAsSystemContext ();
135
- ContextPreservingActionListener <CancelTasksResponse > contextPreservingListener = new ContextPreservingActionListener <>(
136
- threadContext .newRestorableContext (false ), ActionListener .wrap (r -> {}, e -> {}));
137
- CancelTasksRequest cancelTasksRequest = new CancelTasksRequest ();
138
- cancelTasksRequest .setTaskId (taskId );
139
- //We don't wait for cancel tasks to come back. Task cancellation is just best effort.
140
- client .admin ().cluster ().cancelTasks (cancelTasksRequest , contextPreservingListener );
141
- }
165
+ final List <TaskId > toCancel ;
166
+ synchronized (this ) {
167
+ toCancel = new ArrayList <>(tasks );
168
+ tasks .clear ();
169
+ }
170
+ for (TaskId taskId : toCancel ) {
171
+ cancelTask (taskId );
142
172
}
143
173
}
144
174
0 commit comments