You signed in with another tab or window. Reload to refresh your session.You signed out in another tab or window. Reload to refresh your session.You switched accounts on another tab or window. Reload to refresh your session.Dismiss alert
analytics-engine: drive reduce through stage tasks (#21723)
* analytics-engine: drive reduce through stage tasks + wire cancellation
Two related changes:
1. Drive reduce as a task. Introduces ReducingExchangeSink with a reduce/close
split (reduce() owns the work, close() owns cleanup); collapses three
reduce-execution variants into one ReduceStageExecution driven by the
sink's supportsEagerScheduling(). Streaming sink anchors native cleanup on
the reduce() caller via a SinkState machine.
2. Wire coordinator-side cancellation. Plumbs AnalyticsQueryTask.id through
ExchangeSinkContext, registers a cancellation token in the Rust
QUERY_REGISTRY, and has close() fire cancel_query for in-flight drains --
coordinator reduce now participates in task-level cancellation alongside
data-node scans.
Adds before-/after-first-batch cancellation tests in DatafusionReduceSinkTests.
Signed-off-by: Marc Handalian <marc.handalian@gmail.com>
* analytics-engine: fix race in DatafusionReduceSink.close()
close()'s prior get()+compareAndSet pattern had a window where reduce()
could transition READY→REDUCING between the read and the CAS, leaving
cancel_query unfired and the parked drain hung. Replaced with a single
compareAndExchange whose return value tells us which branch to take.
Also collapsed the close() override into closeImpl, removed the base's
auto-session-close (every concrete subclass already closed session, so
this was a double-close masked by idempotency), and gated session.close
on preparedState == null in both subclasses.
Signed-off-by: Marc Handalian <marc.handalian@gmail.com>
* spotless
Signed-off-by: Marc Handalian <marc.handalian@gmail.com>
* fix javadoc
Signed-off-by: Marc Handalian <marc.handalian@gmail.com>
* analytics-engine: fix outStream/session leak on cancel-during-REDUCING
External close() arriving while reduce() was parked set the base class's
`closed` flag and ran closeImpl() — which only fired cancel_query and
deferred teardown. When reduce()'s finally then called super.close() to
do the teardown, the base's `closed` short-circuit blocked re-entry and
closeImpl() never ran a second time. outStream/session leaked.
Have reduce()'s finally call closeImpl() directly, bypassing the base's
closed-gate. Add a `torndown` AtomicBoolean so concurrent close paths
(external + reduce-finally) can't both run the teardown body.
Regression test: testCancelBeforeFirstBatchUnwindsDrain now asserts
sink.torndown is set after the cancel — fails on the prior code.
Signed-off-by: Marc Handalian <marc.handalian@gmail.com>
---------
Signed-off-by: Marc Handalian <marc.handalian@gmail.com>
Copy file name to clipboardExpand all lines: sandbox/plugins/analytics-backend-datafusion/src/main/java/org/opensearch/be/datafusion/AbstractDatafusionReduceSink.java
Copy file name to clipboardExpand all lines: sandbox/plugins/analytics-backend-datafusion/src/main/java/org/opensearch/be/datafusion/DataFusionAnalyticsBackendPlugin.java
+1-1Lines changed: 1 addition & 1 deletion
Original file line number
Diff line number
Diff line change
@@ -725,7 +725,7 @@ public ExchangeSink createSink(ExchangeSinkContext ctx, BackendExecutionContext
Copy file name to clipboardExpand all lines: sandbox/plugins/analytics-backend-datafusion/src/main/java/org/opensearch/be/datafusion/DataFusionService.java
+2-23Lines changed: 2 additions & 23 deletions
Original file line number
Diff line number
Diff line change
@@ -19,9 +19,6 @@
19
19
20
20
importjava.io.IOException;
21
21
importjava.util.Collection;
22
-
importjava.util.concurrent.Executor;
23
-
importjava.util.concurrent.ExecutorService;
24
-
importjava.util.concurrent.Executors;
25
22
26
23
/**
27
24
* Node-level service managing the DataFusion native runtime lifecycle.
@@ -42,8 +39,6 @@ public class DataFusionService extends AbstractLifecycleComponent {
42
39
privatefinalintcpuThreads;
43
40
privatefinalClusterSettingsclusterSettings;
44
41
45
-
privatevolatileExecutorServicedrainExecutor;
46
-
47
42
/** Handle to the native DataFusion global runtime (memory pool + cache). */
0 commit comments