|
|
@@ -7,6 +7,8 @@
|
|
|
|
|
|
package org.elasticsearch.compute.operator;
|
|
|
|
|
|
+import org.elasticsearch.action.ActionListener;
|
|
|
+import org.elasticsearch.action.support.SubscribableListener;
|
|
|
import org.elasticsearch.common.breaker.CircuitBreaker;
|
|
|
import org.elasticsearch.common.util.BigArrays;
|
|
|
import org.elasticsearch.compute.data.BlockFactory;
|
|
|
@@ -17,6 +19,8 @@ import java.util.Collections;
|
|
|
import java.util.IdentityHashMap;
|
|
|
import java.util.Objects;
|
|
|
import java.util.Set;
|
|
|
+import java.util.concurrent.atomic.AtomicBoolean;
|
|
|
+import java.util.concurrent.atomic.AtomicInteger;
|
|
|
import java.util.concurrent.atomic.AtomicReference;
|
|
|
|
|
|
/**
|
|
|
@@ -34,7 +38,11 @@ import java.util.concurrent.atomic.AtomicReference;
|
|
|
* This allows to "transfer ownership" of a shared resource across operators (and even across
|
|
|
* Drivers), while ensuring that the resource can be correctly released when no longer needed.
|
|
|
*
|
|
|
- * Currently only supports releasables, but additional driver-local context can be added.
|
|
|
+ * DriverContext can also be used to track async actions. The driver may close an operator while
|
|
|
+ * some of its async actions are still running. To prevent the driver from finishing in this case,
|
|
|
+ * methods {@link #addAsyncAction()} and {@link #removeAsyncAction()} are provided for tracking
|
|
|
+ * such actions. Subsequently, the driver uses {@link #waitForAsyncActions(ActionListener)} to
|
|
|
+ * await the completion of all async actions before finalizing the Driver.
|
|
|
*/
|
|
|
public class DriverContext {
|
|
|
|
|
|
@@ -47,6 +55,8 @@ public class DriverContext {
|
|
|
|
|
|
private final BlockFactory blockFactory;
|
|
|
|
|
|
+ private final AsyncActions asyncActions = new AsyncActions();
|
|
|
+
|
|
|
public DriverContext(BigArrays bigArrays, BlockFactory blockFactory) {
|
|
|
Objects.requireNonNull(bigArrays);
|
|
|
Objects.requireNonNull(blockFactory);
|
|
|
@@ -119,6 +129,7 @@ public class DriverContext {
|
|
|
}
|
|
|
// must be called by the thread executing the driver.
|
|
|
// no more updates to this context.
|
|
|
+ asyncActions.finish();
|
|
|
var itr = workingSet.iterator();
|
|
|
workingSet = null;
|
|
|
Set<Releasable> releasableSet = Collections.newSetFromMap(new IdentityHashMap<>());
|
|
|
@@ -135,4 +146,45 @@ public class DriverContext {
|
|
|
throw new IllegalStateException("not finished");
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ public void waitForAsyncActions(ActionListener<Void> listener) {
|
|
|
+ asyncActions.addListener(listener);
|
|
|
+ }
|
|
|
+
|
|
|
+ public void addAsyncAction() {
|
|
|
+ asyncActions.addInstance();
|
|
|
+ }
|
|
|
+
|
|
|
+ public void removeAsyncAction() {
|
|
|
+ asyncActions.removeInstance();
|
|
|
+ }
|
|
|
+
|
|
|
+ private static class AsyncActions {
|
|
|
+ private final SubscribableListener<Void> completion = new SubscribableListener<>();
|
|
|
+ private final AtomicBoolean finished = new AtomicBoolean();
|
|
|
+ private final AtomicInteger instances = new AtomicInteger(1);
|
|
|
+
|
|
|
+ void addInstance() {
|
|
|
+ if (finished.get()) {
|
|
|
+ throw new IllegalStateException("DriverContext was finished already");
|
|
|
+ }
|
|
|
+ instances.incrementAndGet();
|
|
|
+ }
|
|
|
+
|
|
|
+ void removeInstance() {
|
|
|
+ if (instances.decrementAndGet() == 0) {
|
|
|
+ completion.onResponse(null);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ void addListener(ActionListener<Void> listener) {
|
|
|
+ completion.addListener(listener);
|
|
|
+ }
|
|
|
+
|
|
|
+ void finish() {
|
|
|
+ if (finished.compareAndSet(false, true)) {
|
|
|
+ removeInstance();
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|