11
11
import org .elasticsearch .action .support .ContextPreservingActionListener ;
12
12
import org .elasticsearch .action .support .SubscribableListener ;
13
13
import org .elasticsearch .common .util .concurrent .AbstractRunnable ;
14
+ import org .elasticsearch .common .util .concurrent .EsExecutors ;
14
15
import org .elasticsearch .common .util .concurrent .ThreadContext ;
15
16
import org .elasticsearch .compute .Describable ;
16
17
import org .elasticsearch .compute .data .Page ;
@@ -74,10 +75,12 @@ public class Driver implements Releasable, Describable {
74
75
private final long statusNanos ;
75
76
76
77
private final AtomicReference <String > cancelReason = new AtomicReference <>();
77
- private final AtomicReference <SubscribableListener <Void >> blocked = new AtomicReference <>();
78
78
79
79
private final AtomicBoolean started = new AtomicBoolean ();
80
+ private final AtomicBoolean earlyFinished = new AtomicBoolean ();
80
81
private final SubscribableListener <Void > completionListener = new SubscribableListener <>();
82
+ private final AtomicReference <SubscribableListener <Void >> blocked = new AtomicReference <>();
83
+ private final AtomicReference <Runnable > scheduledTask = new AtomicReference <>();
81
84
82
85
/**
83
86
* Status reported to the tasks API. We write the status at most once every
@@ -215,6 +218,23 @@ SubscribableListener<Void> run(TimeValue maxTime, int maxIterations, LongSupplie
215
218
}
216
219
}
217
220
221
+ private void earlyFinish () {
222
+ earlyFinished .set (true );
223
+ tryResumeDriver ();
224
+ }
225
+
226
+ private void tryResumeDriver () {
227
+ // Attempt to run the scheduled task on the current thread to finish the driver by closing operators
228
+ SubscribableListener <Void > blockingFuture = blocked .get ();
229
+ if (blockingFuture != null ) {
230
+ blockingFuture .onResponse (null );
231
+ }
232
+ Runnable task = scheduledTask .getAndSet (null );
233
+ if (task != null ) {
234
+ task .run ();
235
+ }
236
+ }
237
+
218
238
/**
219
239
* Whether the driver has run the chain of operators to completion.
220
240
*/
@@ -245,31 +265,33 @@ private IsBlockedResult runSingleLoopIteration() {
245
265
ensureNotCancelled ();
246
266
boolean movedPage = false ;
247
267
248
- for (int i = 0 ; i < activeOperators .size () - 1 ; i ++) {
249
- Operator op = activeOperators .get (i );
250
- Operator nextOp = activeOperators .get (i + 1 );
268
+ if (earlyFinished .get () == false ) {
269
+ for (int i = 0 ; i < activeOperators .size () - 1 ; i ++) {
270
+ Operator op = activeOperators .get (i );
271
+ Operator nextOp = activeOperators .get (i + 1 );
251
272
252
- // skip blocked operator
253
- if (op .isBlocked ().listener ().isDone () == false ) {
254
- continue ;
255
- }
273
+ // skip blocked operator
274
+ if (op .isBlocked ().listener ().isDone () == false ) {
275
+ continue ;
276
+ }
256
277
257
- if (op .isFinished () == false && nextOp .needsInput ()) {
258
- Page page = op .getOutput ();
259
- if (page == null ) {
260
- // No result, just move to the next iteration
261
- } else if (page .getPositionCount () == 0 ) {
262
- // Empty result, release any memory it holds immediately and move to the next iteration
263
- page .releaseBlocks ();
264
- } else {
265
- // Non-empty result from the previous operation, move it to the next operation
266
- nextOp .addInput (page );
267
- movedPage = true ;
278
+ if (op .isFinished () == false && nextOp .needsInput ()) {
279
+ Page page = op .getOutput ();
280
+ if (page == null ) {
281
+ // No result, just move to the next iteration
282
+ } else if (page .getPositionCount () == 0 ) {
283
+ // Empty result, release any memory it holds immediately and move to the next iteration
284
+ page .releaseBlocks ();
285
+ } else {
286
+ // Non-empty result from the previous operation, move it to the next operation
287
+ nextOp .addInput (page );
288
+ movedPage = true ;
289
+ }
268
290
}
269
- }
270
291
271
- if (op .isFinished ()) {
272
- nextOp .finish ();
292
+ if (op .isFinished ()) {
293
+ nextOp .finish ();
294
+ }
273
295
}
274
296
}
275
297
@@ -312,19 +334,10 @@ private IsBlockedResult runSingleLoopIteration() {
312
334
313
335
public void cancel (String reason ) {
314
336
if (cancelReason .compareAndSet (null , reason )) {
315
- synchronized (this ) {
316
- SubscribableListener <Void > fut = this .blocked .get ();
317
- if (fut != null ) {
318
- fut .onFailure (new TaskCancelledException (reason ));
319
- }
320
- }
337
+ tryResumeDriver ();
321
338
}
322
339
}
323
340
324
- private boolean isCancelled () {
325
- return cancelReason .get () != null ;
326
- }
327
-
328
341
private void ensureNotCancelled () {
329
342
String reason = cancelReason .get ();
330
343
if (reason != null ) {
@@ -342,6 +355,12 @@ public static void start(
342
355
driver .completionListener .addListener (listener );
343
356
if (driver .started .compareAndSet (false , true )) {
344
357
driver .updateStatus (0 , 0 , DriverStatus .Status .STARTING , "driver starting" );
358
+ if (driver .activeOperators .isEmpty () == false ) {
359
+ var onFinishedListener = driver .activeOperators .getLast ().onFinishedListener ();
360
+ if (onFinishedListener != null ) {
361
+ onFinishedListener .addListener (ActionListener .running (driver ::earlyFinish ));
362
+ }
363
+ }
345
364
schedule (DEFAULT_TIME_BEFORE_YIELDING , maxIterations , threadContext , executor , driver , driver .completionListener );
346
365
}
347
366
}
@@ -371,7 +390,7 @@ private static void schedule(
371
390
Driver driver ,
372
391
ActionListener <Void > listener
373
392
) {
374
- executor . execute ( new AbstractRunnable () {
393
+ final var task = new AbstractRunnable () {
375
394
376
395
@ Override
377
396
protected void doRun () {
@@ -383,16 +402,12 @@ protected void doRun() {
383
402
if (fut .isDone ()) {
384
403
schedule (maxTime , maxIterations , threadContext , executor , driver , listener );
385
404
} else {
386
- synchronized (driver ) {
387
- if (driver .isCancelled () == false ) {
388
- driver .blocked .set (fut );
389
- }
390
- }
391
405
ActionListener <Void > readyListener = ActionListener .wrap (
392
406
ignored -> schedule (maxTime , maxIterations , threadContext , executor , driver , listener ),
393
407
this ::onFailure
394
408
);
395
409
fut .addListener (ContextPreservingActionListener .wrapPreservingContext (readyListener , threadContext ));
410
+ driver .blocked .set (fut );
396
411
}
397
412
}
398
413
@@ -405,6 +420,17 @@ public void onFailure(Exception e) {
405
420
void onComplete (ActionListener <Void > listener ) {
406
421
driver .driverContext .waitForAsyncActions (ContextPreservingActionListener .wrapPreservingContext (listener , threadContext ));
407
422
}
423
+ };
424
+ final Runnable existing = driver .scheduledTask .getAndSet (task );
425
+ assert existing == null : existing ;
426
+ final boolean runOnCurrentThread = driver .earlyFinished .get () || driver .cancelReason .get () != null ;
427
+ final Executor executorToUse = runOnCurrentThread ? EsExecutors .DIRECT_EXECUTOR_SERVICE : executor ;
428
+ executorToUse .execute (() -> {
429
+ final Runnable next = driver .scheduledTask .getAndSet (null );
430
+ if (next != null ) {
431
+ assert next == task ;
432
+ next .run ();
433
+ }
408
434
});
409
435
}
410
436
0 commit comments