From 3c1674d7348a8554c4d676a46041c5ea8063d533 Mon Sep 17 00:00:00 2001 From: Gonzalo Ortiz <1913993+gortiz@users.noreply.github.com> Date: Thu, 30 Apr 2026 19:35:46 +0200 Subject: [PATCH 01/31] [multistage] Add SubmitWithStream RPC + MultiStageStatsTree proto definitions MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Phase A.0 of #18375. Purely additive proto changes — new long-lived bidi RPC alongside the existing unary Submit/Cancel, plus the message types (BrokerToServer, ServerToBroker, OpChainComplete, ServerDone, MultiStageStatsTree, StageStatsNode) needed by the upcoming stream-mode stats reporting path. No behavior change yet: nothing implements or invokes the new RPC. Generated Java + gRPC stubs verified to compile. --- pinot-common/src/main/proto/worker.proto | 85 ++++++++++++++++++++++++ 1 file changed, 85 insertions(+) diff --git a/pinot-common/src/main/proto/worker.proto b/pinot-common/src/main/proto/worker.proto index 285821b0c13e..8ad979566721 100644 --- a/pinot-common/src/main/proto/worker.proto +++ b/pinot-common/src/main/proto/worker.proto @@ -30,6 +30,13 @@ service PinotQueryWorker { rpc Cancel(CancelRequest) returns (CancelResponse); rpc Explain(QueryRequest) returns (stream ExplainResponse); + + // Long-lived bidi RPC used by the "stream" stats-reporting mode (S3-variant). + // The broker keeps the stream open for the query lifetime; the server pushes one + // OpChainComplete per stage it ran on this server, followed by a ServerDone. + // Stats no longer travel through the mailbox in this mode. + // See OpChainComplete / MultiStageStatsTree below for the payload shape. + rpc SubmitWithStream(stream BrokerToServer) returns (stream ServerToBroker); } message CancelRequest { @@ -100,3 +107,81 @@ message MailboxInfo { message Properties { map property = 1; } + +// ============================================================================= +// SubmitWithStream — stream-mode stats reporting messages +// ============================================================================= + +// Messages flowing from broker to server on the SubmitWithStream stream. +// First message MUST be a `submit`. Subsequent messages may carry cancel. +message BrokerToServer { + oneof payload { + // Plan submission. Same shape as today's unary Submit request. + QueryRequest submit = 1; + // Optional cancel message. In Phase A the broker still uses the unary + // Cancel RPC; this field is reserved and processed by the server but only + // emitted by the broker in Phase B. + CancelRequest cancel = 2; + } +} + +// Messages flowing from server to broker on the SubmitWithStream stream. +// The first server message is a `submit_ack`. Each opchain that runs on this +// server emits exactly one `opchain` message when it finishes (success or error). +// After the last opchain has reported, the server emits `done` and half-closes. +message ServerToBroker { + oneof payload { + // Synchronous-style ack for the submission. Same shape as today's unary + // QueryResponse. Carries early errors (plan parsing, malformed metadata). + QueryResponse submit_ack = 1; + // Per-stage stats from one opchain that ran on this server. + OpChainComplete opchain = 2; + // Final marker — the server has no more messages for this query. + ServerDone done = 3; + } +} + +// Sent once per opchain that ran on this server, regardless of success/failure. +message OpChainComplete { + int32 stage_id = 1; + int32 worker_id = 2; + bool success = 3; + string error_msg = 4; // populated when success = false + MultiStageStatsTree stats = 5; // structured, tree-shaped stats payload +} + +// Final marker on the server-to-broker stream. +message ServerDone { +} + +// Multi-stage stats produced by one opchain. Carries the current stage as an +// explicit operator tree, plus any upstream-stage trees this opchain +// accumulated (today these reach an opchain via pipeline-breaker stats and via +// mailbox-receive merges; in stream mode each upstream stage is also reported +// by its own server, so the broker merges duplicates per stage). +message MultiStageStatsTree { + int32 current_stage_id = 1; + StageStatsNode current_stage = 2; + // Sparse map of upstream stage id -> tree. + map upstream_stages = 3; +} + +// One node of the operator tree for a single stage. +message StageStatsNode { + // The MultiStageOperator.Type id, same byte we serialize today via + // MultiStageOperator.Type.getId(). + int32 operator_type_id = 1; + // Plan node ids that compile down to this operator. One-to-many; the leaf + // operator typically owns the whole sub-tree of v1 plan nodes below the leaf + // boundary. Stable per request via PlanNode.getNodeId(). + repeated int32 plan_node_ids = 2; + // Opaque StatMap bytes -- same encoding the existing StatMap.serialize + // produces. The broker decodes by looking up the StatMap key class via + // MultiStageOperator.Type.fromId(operator_type_id). + bytes stat_map = 3; + // Children in left-to-right order. Arity is whatever the operator dictates + // (1 for transforms, 2 for joins, N for set ops). The tree shape on the wire + // is canonical -- legacy mode keeps using its inorder/implicit-arity + // encoding. + repeated StageStatsNode children = 4; +} From 99eef1913897cf57d0189b9835f022688d3a44c3 Mon Sep 17 00:00:00 2001 From: Gonzalo Ortiz <1913993+gortiz@users.noreply.github.com> Date: Mon, 4 May 2026 09:06:38 +0200 Subject: [PATCH 02/31] [multistage] Capture op->PlanNode mapping on OpChainExecutionContext Phase A.1.a of #18375. The existing PlanNodeToOpChain BiConsumer tracker now also populates a per-opchain Map> on OpChainExecutionContext. Cardinality is one-to-many: intermediate operators map 1:1 to their PlanNode; the leaf operator records the whole sub-tree of v1 plan nodes below the leaf-stage boundary (walked once at construction). This will be consumed by the upcoming MultiStageStatsTreeEncoder (stream-mode stats reporting). No behavior change yet for legacy mode. --- .../runtime/plan/OpChainExecutionContext.java | 30 ++++++++++++++ .../query/runtime/plan/PlanNodeToOpChain.java | 40 ++++++++++++++++++- 2 files changed, 68 insertions(+), 2 deletions(-) diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/OpChainExecutionContext.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/OpChainExecutionContext.java index e917adb678f4..bdcd9482d4fc 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/OpChainExecutionContext.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/OpChainExecutionContext.java @@ -20,14 +20,18 @@ import com.google.common.annotations.VisibleForTesting; import java.util.Collections; +import java.util.IdentityHashMap; +import java.util.List; import java.util.Map; import javax.annotation.Nullable; import org.apache.pinot.core.instance.context.BrokerContext; import org.apache.pinot.core.instance.context.ServerContext; import org.apache.pinot.query.mailbox.MailboxService; +import org.apache.pinot.query.planner.plannode.PlanNode; import org.apache.pinot.query.routing.StageMetadata; import org.apache.pinot.query.routing.VirtualServerAddress; import org.apache.pinot.query.routing.WorkerMetadata; +import org.apache.pinot.query.runtime.operator.MultiStageOperator; import org.apache.pinot.query.runtime.operator.OpChainId; import org.apache.pinot.query.runtime.operator.factory.DefaultQueryOperatorFactoryProvider; import org.apache.pinot.query.runtime.operator.factory.QueryOperatorFactoryProvider; @@ -63,6 +67,16 @@ public class OpChainExecutionContext { private ServerPlanRequestContext _leafStageContext; private final boolean _sendStats; private final boolean _keepPipelineBreakerStats; + /** + * Map of MultiStageOperator -> PlanNodes that compile down to that operator. Populated by + * {@link org.apache.pinot.query.runtime.plan.PlanNodeToOpChain} during opchain construction. Cardinality is + * one-to-many: an intermediate operator maps to a single PlanNode, but the leaf operator maps to the whole sub-tree + * of v1 plan nodes below the leaf-stage boundary. Used by the stream-mode stats reporting path + * ({@code MultiStageStatsTreeEncoder}) to attach plan-node identifiers to each operator's stats. + *

+ * Identity-based (IdentityHashMap) because PlanNode equality is structural and two distinct nodes can compare equal. + */ + private final Map> _operatorToPlanNodes = new IdentityHashMap<>(); @VisibleForTesting public OpChainExecutionContext(MailboxService mailboxService, long requestId, String cid, long activeDeadlineMs, @@ -207,6 +221,22 @@ public boolean isKeepPipelineBreakerStats() { return _keepPipelineBreakerStats; } + /** + * Records which PlanNodes compiled down to the given MultiStageOperator. Should be called once per operator as the + * opchain is constructed; subsequent calls overwrite. {@code planNodes} is captured by reference; callers should not + * mutate it after passing it in. + */ + public void recordPlanNodesForOperator(MultiStageOperator operator, List planNodes) { + _operatorToPlanNodes.put(operator, planNodes); + } + + /** + * Returns the PlanNodes that compiled down to the given operator, or an empty list if no mapping was recorded. + */ + public List getPlanNodesForOperator(MultiStageOperator operator) { + return _operatorToPlanNodes.getOrDefault(operator, List.of()); + } + private static QueryOperatorFactoryProvider getDefaultQueryOperatorFactoryProvider() { // Prefer server context when explicitly configured, otherwise fall back to broker, then default. Object serverProvider = ServerContext.getInstance().getQueryOperatorFactoryProvider(); diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/PlanNodeToOpChain.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/PlanNodeToOpChain.java index 6e5da76bc4e3..40e7572f0dd6 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/PlanNodeToOpChain.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/PlanNodeToOpChain.java @@ -96,16 +96,34 @@ public static OpChain convert(PlanNode node, OpChainExecutionContext context) { */ public static OpChain convert(PlanNode node, OpChainExecutionContext context, BiConsumer tracker) { - MyVisitor visitor = new MyVisitor(tracker); + MyVisitor visitor = new MyVisitor(context, tracker); MultiStageOperator root = node.visit(visitor, context); + visitor.record(node, root); tracker.accept(node, root); return new OpChain(context, root); } + /** + * Recursively collects all PlanNodes in the sub-tree rooted at {@code root} (including {@code root} itself), in + * pre-order (root first, then children left-to-right). + *

+ * Used to record the leaf operator's full one-to-many mapping: the leaf operator's tracker fires once with the + * leaf-stage boundary PlanNode, but the leaf actually represents the whole sub-tree of v1 plan nodes below that + * boundary. We walk the boundary's sub-tree once at construction and store the full list on the operator. + */ + private static void collectPlanNodeSubTree(PlanNode root, List out) { + out.add(root); + for (PlanNode child : root.getInputs()) { + collectPlanNodeSubTree(child, out); + } + } + private static class MyVisitor implements PlanNodeVisitor { + private final OpChainExecutionContext _context; private final BiConsumer _tracker; - public MyVisitor(BiConsumer tracker) { + public MyVisitor(OpChainExecutionContext context, BiConsumer tracker) { + _context = context; _tracker = tracker; } @@ -127,10 +145,28 @@ private MultiStageOperator visit(T node, OpChainExecutionCo } else { result = node.visit(this, context); } + record(node, result); _tracker.accept(node, result); return result; } + /** + * Records the operator-to-PlanNode mapping on the execution context. For non-leaf operators this is a 1:1 mapping + * to {@code node}. For the leaf operator we walk the sub-tree below the leaf-stage boundary and record every + * PlanNode encountered (one-to-many: a leaf operator owns the whole v1 sub-plan below it). + */ + void record(PlanNode node, MultiStageOperator operator) { + List mapping; + ServerPlanRequestContext leafStageContext = _context.getLeafStageContext(); + if (leafStageContext != null && leafStageContext.getLeafStageBoundaryNode() == node) { + mapping = new ArrayList<>(); + collectPlanNodeSubTree(node, mapping); + } else { + mapping = List.of(node); + } + _context.recordPlanNodesForOperator(operator, mapping); + } + @Override public MultiStageOperator visitMailboxReceive(MailboxReceiveNode node, OpChainExecutionContext context) { try { From 49ba40ec2e2140207f319e17b4db47502361ac82 Mon Sep 17 00:00:00 2001 From: Gonzalo Ortiz <1913993+gortiz@users.noreply.github.com> Date: Mon, 4 May 2026 09:15:38 +0200 Subject: [PATCH 03/31] [multistage] Add MultiStageStatsTreeEncoder for stream-mode stats reporting Phase A.1.b of #18375. Walks an opchain's live operator tree in lock-step with the flat MultiStageQueryStats lists (already maintained in inorder) to produce a tree-shaped MultiStageStatsTree proto. Each node carries the operator type id, opaque StatMap bytes, recursive children, and the stage-scoped plan-node ids gathered during opchain construction. PlanNodeToOpChain now also performs a deterministic pre-order walk over the stage's plan tree, assigning each PlanNode a sequential integer id on OpChainExecutionContext. Both broker and server perform the same walk on the same plan structure, so the ids match without being serialized on the wire. The encoder has a Function> overload used by tests so they don't need to construct a full OpChainExecutionContext. Production calls go through the OpChainExecutionContext-resolving overload. Coverage: linear chain ordering, N-ary set-op (3 inputs) with left-to-right child order, leaf operator one-to-many planNodeIds fan-out, tree/flat-list mismatch detection. --- .../plan/MultiStageStatsTreeEncoder.java | 136 +++++++++++ .../runtime/plan/OpChainExecutionContext.java | 25 ++ .../query/runtime/plan/PlanNodeToOpChain.java | 18 ++ .../plan/MultiStageStatsTreeEncoderTest.java | 224 ++++++++++++++++++ 4 files changed, 403 insertions(+) create mode 100644 pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/MultiStageStatsTreeEncoder.java create mode 100644 pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/plan/MultiStageStatsTreeEncoderTest.java diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/MultiStageStatsTreeEncoder.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/MultiStageStatsTreeEncoder.java new file mode 100644 index 000000000000..1df9d118f82e --- /dev/null +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/MultiStageStatsTreeEncoder.java @@ -0,0 +1,136 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.pinot.query.runtime.plan; + +import com.google.protobuf.ByteString; +import java.io.DataOutputStream; +import java.io.IOException; +import java.util.List; +import java.util.function.Function; +import org.apache.commons.io.output.UnsynchronizedByteArrayOutputStream; +import org.apache.pinot.common.datatable.StatMap; +import org.apache.pinot.common.proto.Worker; +import org.apache.pinot.query.planner.plannode.PlanNode; +import org.apache.pinot.query.runtime.operator.MultiStageOperator; + + +/** + * Builds a {@link Worker.MultiStageStatsTree} proto from an opchain's live operator tree and its accumulated + * {@link MultiStageQueryStats}. Used by the stream-mode stats reporting path (gRPC {@code SubmitWithStream} RPC). + * + *

The encoder walks the operator tree in inorder (leftmost-leaf-first) in lock-step with the flat + * {@link MultiStageQueryStats.StageStats#_operatorTypes _operatorTypes} / + * {@link MultiStageQueryStats.StageStats#_operatorStats _operatorStats} lists, which are already maintained in + * inorder (each operator appends its entry via + * {@link MultiStageQueryStats.StageStats.Open#addLastOperator addLastOperator} just before emitting EOS). For each + * operator the encoder produces a {@link Worker.StageStatsNode} carrying the operator type id, the serialized + * {@link StatMap} bytes, the recursive children, and the stage-scoped plan-node ids gathered from the + * {@link OpChainExecutionContext} during opchain construction. + * + *

Currently only the current-stage tree is encoded. Upstream stages reach the broker via the upstream opchains' + * own reports in stream mode, so per-opchain reporting of upstream-stage trees is not needed except for the + * pipeline-breaker case (handled in a follow-up). + */ +public final class MultiStageStatsTreeEncoder { + private MultiStageStatsTreeEncoder() { + } + + /** + * Convenience overload that resolves plan-node ids from the {@link OpChainExecutionContext}. This is the form + * called from production code (the opchain completion callback in {@code QueryServer}). + */ + public static Worker.MultiStageStatsTree encode(MultiStageOperator root, MultiStageQueryStats stats, + OpChainExecutionContext context) + throws IOException { + return encode(root, stats, op -> resolvePlanNodeIds(op, context)); + } + + /** + * Encodes the current-stage operator tree + stats into a {@link Worker.MultiStageStatsTree}. Tests use this entry + * point with a custom {@code planNodeIdResolver} so they don't need to construct a full + * {@link OpChainExecutionContext}. + * + * @throws IllegalStateException if the operator tree shape does not align with the flat stats list (missing + * entries — typically caused by an operator that failed before emitting EOS). + */ + public static Worker.MultiStageStatsTree encode(MultiStageOperator root, MultiStageQueryStats stats, + Function> planNodeIdResolver) + throws IOException { + MultiStageQueryStats.StageStats.Open openStats = stats.getCurrentStats(); + int treeSize = countOperators(root); + int flatSize = openStats.getLastOperatorIndex() + 1; + if (treeSize != flatSize) { + throw new IllegalStateException("Operator tree size (" + treeSize + ") does not match flat stats list size (" + + flatSize + ") for stage " + stats.getCurrentStageId() + + ". This usually means an operator failed before emitting EOS."); + } + int[] cursor = new int[]{0}; + Worker.StageStatsNode rootNode = encodeNode(root, openStats, cursor, planNodeIdResolver); + return Worker.MultiStageStatsTree.newBuilder() + .setCurrentStageId(stats.getCurrentStageId()) + .setCurrentStage(rootNode) + .build(); + } + + private static Worker.StageStatsNode encodeNode(MultiStageOperator op, + MultiStageQueryStats.StageStats openStats, int[] cursor, + Function> planNodeIdResolver) + throws IOException { + Worker.StageStatsNode.Builder builder = Worker.StageStatsNode.newBuilder(); + // Inorder: encode children first. + for (MultiStageOperator child : op.getChildOperators()) { + builder.addChildren(encodeNode(child, openStats, cursor, planNodeIdResolver)); + } + int idx = cursor[0]++; + MultiStageOperator.Type type = openStats.getOperatorType(idx); + StatMap statMap = openStats.getOperatorStats(idx); + builder.setOperatorTypeId(type.getId()); + builder.setStatMap(serializeStatMap(statMap)); + for (Integer id : planNodeIdResolver.apply(op)) { + if (id != null && id >= 0) { + builder.addPlanNodeIds(id); + } + } + return builder.build(); + } + + private static int countOperators(MultiStageOperator op) { + int count = 1; + for (MultiStageOperator child : op.getChildOperators()) { + count += countOperators(child); + } + return count; + } + + private static ByteString serializeStatMap(StatMap statMap) + throws IOException { + try (UnsynchronizedByteArrayOutputStream baos = new UnsynchronizedByteArrayOutputStream.Builder().get(); + DataOutputStream output = new DataOutputStream(baos)) { + statMap.serialize(output); + output.flush(); + return ByteString.copyFrom(baos.toByteArray()); + } + } + + private static List resolvePlanNodeIds(MultiStageOperator op, OpChainExecutionContext context) { + return context.getPlanNodesForOperator(op).stream() + .map((PlanNode pn) -> context.getPlanNodeId(pn)) + .toList(); + } +} diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/OpChainExecutionContext.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/OpChainExecutionContext.java index bdcd9482d4fc..c58e5f21f046 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/OpChainExecutionContext.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/OpChainExecutionContext.java @@ -77,6 +77,15 @@ public class OpChainExecutionContext { * Identity-based (IdentityHashMap) because PlanNode equality is structural and two distinct nodes can compare equal. */ private final Map> _operatorToPlanNodes = new IdentityHashMap<>(); + /** + * Stage-scoped plan-node ids: each PlanNode reachable from the opchain's root receives a sequential integer id + * assigned by a deterministic pre-order walk. Both broker and server perform the same walk over the same plan + * structure, so the ids match without being serialized on the wire. Used by {@code MultiStageStatsTreeEncoder} to + * populate {@code StageStatsNode.plan_node_ids}. + *

+ * Identity-based for the same reason as {@link #_operatorToPlanNodes}. + */ + private final Map _planNodeIds = new IdentityHashMap<>(); @VisibleForTesting public OpChainExecutionContext(MailboxService mailboxService, long requestId, String cid, long activeDeadlineMs, @@ -237,6 +246,22 @@ public List getPlanNodesForOperator(MultiStageOperator operator) { return _operatorToPlanNodes.getOrDefault(operator, List.of()); } + /** + * Records the stage-scoped id assigned to the given PlanNode. Should only be called once per node by the + * {@link org.apache.pinot.query.runtime.plan.PlanNodeToOpChain} pre-walk. + */ + public void recordPlanNodeId(PlanNode node, int id) { + _planNodeIds.put(node, id); + } + + /** + * Returns the stage-scoped id assigned to the given PlanNode, or {@code -1} if no id was recorded for it. + */ + public int getPlanNodeId(PlanNode node) { + Integer id = _planNodeIds.get(node); + return id != null ? id : -1; + } + private static QueryOperatorFactoryProvider getDefaultQueryOperatorFactoryProvider() { // Prefer server context when explicitly configured, otherwise fall back to broker, then default. Object serverProvider = ServerContext.getInstance().getQueryOperatorFactoryProvider(); diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/PlanNodeToOpChain.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/PlanNodeToOpChain.java index 40e7572f0dd6..53cf4d0739a9 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/PlanNodeToOpChain.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/PlanNodeToOpChain.java @@ -96,6 +96,10 @@ public static OpChain convert(PlanNode node, OpChainExecutionContext context) { */ public static OpChain convert(PlanNode node, OpChainExecutionContext context, BiConsumer tracker) { + // Assign deterministic stage-scoped ids to every PlanNode reachable from the root before constructing operators, + // so the encoder can attach plan_node_ids to each StageStatsNode without needing to mutate or re-walk the plan. + // Both broker and server perform this same pre-walk over the same plan structure, producing matching ids. + assignPlanNodeIds(node, context); MyVisitor visitor = new MyVisitor(context, tracker); MultiStageOperator root = node.visit(visitor, context); visitor.record(node, root); @@ -103,6 +107,20 @@ public static OpChain convert(PlanNode node, OpChainExecutionContext context, return new OpChain(context, root); } + /** + * Pre-order walk that assigns each PlanNode in the sub-tree a sequential integer id, recorded on the context. + */ + private static void assignPlanNodeIds(PlanNode root, OpChainExecutionContext context) { + assignPlanNodeIds(root, context, new int[]{0}); + } + + private static void assignPlanNodeIds(PlanNode node, OpChainExecutionContext context, int[] counter) { + context.recordPlanNodeId(node, counter[0]++); + for (PlanNode child : node.getInputs()) { + assignPlanNodeIds(child, context, counter); + } + } + /** * Recursively collects all PlanNodes in the sub-tree rooted at {@code root} (including {@code root} itself), in * pre-order (root first, then children left-to-right). diff --git a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/plan/MultiStageStatsTreeEncoderTest.java b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/plan/MultiStageStatsTreeEncoderTest.java new file mode 100644 index 000000000000..e6a89b232ba5 --- /dev/null +++ b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/plan/MultiStageStatsTreeEncoderTest.java @@ -0,0 +1,224 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.pinot.query.runtime.plan; + +import com.google.protobuf.ByteString; +import java.io.DataInputStream; +import java.io.IOException; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.function.Function; +import org.apache.pinot.common.datatable.StatMap; +import org.apache.pinot.common.proto.Worker; +import org.apache.pinot.query.runtime.operator.AggregateOperator; +import org.apache.pinot.query.runtime.operator.BaseMailboxReceiveOperator; +import org.apache.pinot.query.runtime.operator.MailboxSendOperator; +import org.apache.pinot.query.runtime.operator.MultiStageOperator; +import org.apache.pinot.query.runtime.operator.SortOperator; +import org.mockito.Mockito; +import org.testng.Assert; +import org.testng.annotations.Test; + + +/** + * Unit tests for {@link MultiStageStatsTreeEncoder}. + * + *

The operator tree is built with mocked {@link MultiStageOperator} instances; only {@link + * MultiStageOperator#getChildOperators()} is stubbed because that's the only method the encoder reads. Plan-node ids + * are supplied via the {@code Function>} test entry point so we don't need to build + * a full {@link OpChainExecutionContext}. + */ +public class MultiStageStatsTreeEncoderTest { + + /** + * Linear chain: MailboxReceive -> Sort -> MailboxSend (root). Verifies inorder ordering between operator tree and + * flat stats list, and that stat map bytes round-trip. + */ + @Test + public void testLinearChainEncode() + throws IOException { + MultiStageOperator receive = mockOperator(); + MultiStageOperator sort = mockOperator(receive); + MultiStageOperator send = mockOperator(sort); + + StatMap receiveStat = + new StatMap<>(BaseMailboxReceiveOperator.StatKey.class) + .merge(BaseMailboxReceiveOperator.StatKey.EXECUTION_TIME_MS, 100) + .merge(BaseMailboxReceiveOperator.StatKey.EMITTED_ROWS, 10); + StatMap sortStat = + new StatMap<>(SortOperator.StatKey.class) + .merge(SortOperator.StatKey.EXECUTION_TIME_MS, 5) + .merge(SortOperator.StatKey.EMITTED_ROWS, 8); + StatMap sendStat = + new StatMap<>(MailboxSendOperator.StatKey.class) + .merge(MailboxSendOperator.StatKey.EXECUTION_TIME_MS, 3); + + MultiStageQueryStats stats = new MultiStageQueryStats.Builder(2) + .customizeOpen(open -> open + .addLastOperator(MultiStageOperator.Type.MAILBOX_RECEIVE, receiveStat) + .addLastOperator(MultiStageOperator.Type.SORT_OR_LIMIT, sortStat) + .addLastOperator(MultiStageOperator.Type.MAILBOX_SEND, sendStat)) + .build(); + + Map> idMap = new HashMap<>(); + idMap.put(receive, List.of(2)); + idMap.put(sort, List.of(1)); + idMap.put(send, List.of(0)); + + Worker.MultiStageStatsTree tree = MultiStageStatsTreeEncoder.encode(send, stats, asResolver(idMap)); + + Assert.assertEquals(tree.getCurrentStageId(), 2); + Worker.StageStatsNode rootNode = tree.getCurrentStage(); + Assert.assertEquals(rootNode.getOperatorTypeId(), MultiStageOperator.Type.MAILBOX_SEND.getId()); + Assert.assertEquals(rootNode.getPlanNodeIdsList(), List.of(0)); + Assert.assertEquals(rootNode.getChildrenCount(), 1); + + Worker.StageStatsNode sortNode = rootNode.getChildren(0); + Assert.assertEquals(sortNode.getOperatorTypeId(), MultiStageOperator.Type.SORT_OR_LIMIT.getId()); + Assert.assertEquals(sortNode.getPlanNodeIdsList(), List.of(1)); + Assert.assertEquals(sortNode.getChildrenCount(), 1); + + Worker.StageStatsNode receiveNode = sortNode.getChildren(0); + Assert.assertEquals(receiveNode.getOperatorTypeId(), MultiStageOperator.Type.MAILBOX_RECEIVE.getId()); + Assert.assertEquals(receiveNode.getPlanNodeIdsList(), List.of(2)); + Assert.assertEquals(receiveNode.getChildrenCount(), 0); + + StatMap deserializedReceiveStat = deserialize( + receiveNode.getStatMap(), BaseMailboxReceiveOperator.StatKey.class); + Assert.assertEquals(deserializedReceiveStat, receiveStat); + } + + /** + * N-ary set op (3 inputs). Verifies the encoder records all children regardless of arity, in left-to-right order, + * and that the flat stats list is consumed in inorder (leftmost-leaf-first). + */ + @Test + public void testNaryEncode() + throws IOException { + MultiStageOperator left = mockOperator(); + MultiStageOperator mid = mockOperator(); + MultiStageOperator right = mockOperator(); + MultiStageOperator union = mockOperator(left, mid, right); + + StatMap receiveStatLeft = + new StatMap<>(BaseMailboxReceiveOperator.StatKey.class) + .merge(BaseMailboxReceiveOperator.StatKey.EMITTED_ROWS, 1); + StatMap receiveStatMid = + new StatMap<>(BaseMailboxReceiveOperator.StatKey.class) + .merge(BaseMailboxReceiveOperator.StatKey.EMITTED_ROWS, 2); + StatMap receiveStatRight = + new StatMap<>(BaseMailboxReceiveOperator.StatKey.class) + .merge(BaseMailboxReceiveOperator.StatKey.EMITTED_ROWS, 3); + StatMap unionStat = + new StatMap<>(AggregateOperator.StatKey.class) + .merge(AggregateOperator.StatKey.EMITTED_ROWS, 6); + + MultiStageQueryStats stats = new MultiStageQueryStats.Builder(1) + .customizeOpen(open -> open + .addLastOperator(MultiStageOperator.Type.MAILBOX_RECEIVE, receiveStatLeft) + .addLastOperator(MultiStageOperator.Type.MAILBOX_RECEIVE, receiveStatMid) + .addLastOperator(MultiStageOperator.Type.MAILBOX_RECEIVE, receiveStatRight) + .addLastOperator(MultiStageOperator.Type.AGGREGATE, unionStat)) + .build(); + + Map> idMap = new HashMap<>(); + idMap.put(left, List.of(1)); + idMap.put(mid, List.of(2)); + idMap.put(right, List.of(3)); + idMap.put(union, List.of(0)); + + Worker.MultiStageStatsTree tree = MultiStageStatsTreeEncoder.encode(union, stats, asResolver(idMap)); + + Worker.StageStatsNode rootNode = tree.getCurrentStage(); + Assert.assertEquals(rootNode.getOperatorTypeId(), MultiStageOperator.Type.AGGREGATE.getId()); + Assert.assertEquals(rootNode.getChildrenCount(), 3); + Assert.assertEquals(rootNode.getChildren(0).getPlanNodeIdsList(), List.of(1)); + Assert.assertEquals(rootNode.getChildren(1).getPlanNodeIdsList(), List.of(2)); + Assert.assertEquals(rootNode.getChildren(2).getPlanNodeIdsList(), List.of(3)); + + StatMap roundTripped = deserialize( + rootNode.getChildren(1).getStatMap(), BaseMailboxReceiveOperator.StatKey.class); + Assert.assertEquals(roundTripped, receiveStatMid); + } + + /** + * One-to-many planNodeIds for a leaf operator: a single operator records multiple plan-node ids. + */ + @Test + public void testLeafPlanNodeFanOut() + throws IOException { + MultiStageOperator leaf = mockOperator(); + + StatMap leafStat = new StatMap<>(AggregateOperator.StatKey.class) + .merge(AggregateOperator.StatKey.EMITTED_ROWS, 42); + MultiStageQueryStats stats = new MultiStageQueryStats.Builder(0) + .customizeOpen(open -> open.addLastOperator(MultiStageOperator.Type.AGGREGATE, leafStat)) + .build(); + + Map> idMap = new HashMap<>(); + idMap.put(leaf, Arrays.asList(10, 11, 12, 13)); + + Worker.MultiStageStatsTree tree = MultiStageStatsTreeEncoder.encode(leaf, stats, asResolver(idMap)); + + Assert.assertEquals(tree.getCurrentStage().getPlanNodeIdsList(), List.of(10, 11, 12, 13)); + } + + /** + * Tree-vs-flat-list size mismatch: encoder must throw rather than emit a malformed tree. + */ + @Test(expectedExceptions = IllegalStateException.class) + public void testTreeFlatListMismatchThrows() + throws IOException { + MultiStageOperator leaf = mockOperator(); + MultiStageOperator root = mockOperator(leaf); + + // Only one entry in the flat list, but tree has two operators. + MultiStageQueryStats stats = new MultiStageQueryStats.Builder(0) + .customizeOpen(open -> open.addLastOperator(MultiStageOperator.Type.AGGREGATE, + new StatMap<>(AggregateOperator.StatKey.class))) + .build(); + + MultiStageStatsTreeEncoder.encode(root, stats, op -> List.of()); + } + + // ---- helpers ---- + + private static MultiStageOperator mockOperator(MultiStageOperator... children) { + MultiStageOperator op = Mockito.mock(MultiStageOperator.class); + Mockito.when(op.getChildOperators()).thenReturn(children.length == 0 + ? Collections.emptyList() + : Arrays.asList(children)); + return op; + } + + private static Function> asResolver( + Map> idMap) { + return op -> idMap.getOrDefault(op, List.of()); + } + + private static & StatMap.Key> StatMap deserialize(ByteString bytes, Class keyClass) + throws IOException { + try (DataInputStream input = new DataInputStream(bytes.newInput())) { + return StatMap.deserialize(input, keyClass); + } + } +} From 0f6ab03cd1de185eb5fcc50a538f4bd95b6f2b7b Mon Sep 17 00:00:00 2001 From: Gonzalo Ortiz <1913993+gortiz@users.noreply.github.com> Date: Mon, 4 May 2026 09:21:32 +0200 Subject: [PATCH 04/31] [multistage] Add MultiStageStatsTree decoder + tree-shape merge MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Phase A.1.c of #18375. Broker-side counterpart to MultiStageStatsTreeEncoder: - StageStatsTreeNode: in-memory view of a decoded per-stage operator tree, mutable, with a recursive merge(StageStatsTreeNode) that sums StatMaps node-by-node. Throws ShapeMismatchException when the two trees disagree on operator type or arity at any position. - MultiStageStatsTreeDecoder: turns a Worker.MultiStageStatsTree proto into a Decoded(currentStageId, currentStage, upstreamStages) value. Throws DecodeFailedException on unknown operator type ids — gives the broker a typed signal to mark mergeFailed for that stage and continue. Coverage: - decode round-trip preserves type, planNodeIds, statMap, children - merge same-shape sums counters - merge type/arity mismatch throws ShapeMismatchException - decode unknown operator type id throws DecodeFailedException (mixed-version safety: newer server, older broker) --- .../plan/MultiStageStatsTreeDecoder.java | 133 ++++++++++++++ .../runtime/plan/StageStatsTreeNode.java | 121 +++++++++++++ .../plan/MultiStageStatsTreeDecoderTest.java | 164 ++++++++++++++++++ 3 files changed, 418 insertions(+) create mode 100644 pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/MultiStageStatsTreeDecoder.java create mode 100644 pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/StageStatsTreeNode.java create mode 100644 pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/plan/MultiStageStatsTreeDecoderTest.java diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/MultiStageStatsTreeDecoder.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/MultiStageStatsTreeDecoder.java new file mode 100644 index 000000000000..3ed8fa1437b1 --- /dev/null +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/MultiStageStatsTreeDecoder.java @@ -0,0 +1,133 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.pinot.query.runtime.plan; + +import com.google.protobuf.ByteString; +import java.io.DataInputStream; +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.apache.pinot.common.datatable.StatMap; +import org.apache.pinot.common.proto.Worker; +import org.apache.pinot.query.runtime.operator.MultiStageOperator; + + +/** + * Broker-side decoder turning {@link Worker.MultiStageStatsTree} payloads into {@link StageStatsTreeNode} instances + * the broker accumulates by stage id (and merges across worker reports for the same stage). + * + *

Pairs with {@link MultiStageStatsTreeEncoder} on the server side. + */ +public final class MultiStageStatsTreeDecoder { + private MultiStageStatsTreeDecoder() { + } + + /** + * Thrown when a payload cannot be decoded — usually because the operator type id is unknown to this broker + * (newer-server / older-broker case). The broker logs and marks the stage {@code mergeFailed}; the query continues. + */ + public static class DecodeFailedException extends Exception { + public DecodeFailedException(String message) { + super(message); + } + + public DecodeFailedException(String message, Throwable cause) { + super(message, cause); + } + } + + /** + * Decodes a single {@link Worker.StageStatsNode} (recursive). Used directly when the caller already has a node and + * a known stage id. + */ + public static StageStatsTreeNode decodeNode(Worker.StageStatsNode node) + throws DecodeFailedException { + MultiStageOperator.Type type = MultiStageOperator.Type.fromId(node.getOperatorTypeId()); + if (type == null) { + throw new DecodeFailedException("Unknown operator type id: " + node.getOperatorTypeId()); + } + StatMap statMap; + try { + statMap = deserializeStatMap(node.getStatMap(), type); + } catch (IOException e) { + throw new DecodeFailedException("Failed to deserialize StatMap for operator type " + type, e); + } + List children = new ArrayList<>(node.getChildrenCount()); + for (Worker.StageStatsNode child : node.getChildrenList()) { + children.add(decodeNode(child)); + } + return new StageStatsTreeNode(type, node.getPlanNodeIdsList(), statMap, children); + } + + /** + * Result of decoding a {@link Worker.MultiStageStatsTree}: the current-stage tree plus any upstream-stage trees the + * opchain attached to its report (e.g. from pipeline-breaker stats). + */ + public static final class Decoded { + private final int _currentStageId; + private final StageStatsTreeNode _currentStage; + private final Map _upstreamStages; + + public Decoded(int currentStageId, StageStatsTreeNode currentStage, + Map upstreamStages) { + _currentStageId = currentStageId; + _currentStage = currentStage; + _upstreamStages = upstreamStages; + } + + public int getCurrentStageId() { + return _currentStageId; + } + + public StageStatsTreeNode getCurrentStage() { + return _currentStage; + } + + public Map getUpstreamStages() { + return _upstreamStages; + } + } + + /** + * Decodes a full {@link Worker.MultiStageStatsTree} into a {@link Decoded} containing the current-stage tree and a + * map of upstream-stage trees keyed by stage id. Throws {@link DecodeFailedException} on any decode error; the + * caller is responsible for logging and marking {@code mergeFailed}. + */ + public static Decoded decode(Worker.MultiStageStatsTree proto) + throws DecodeFailedException { + StageStatsTreeNode currentStage = decodeNode(proto.getCurrentStage()); + Map upstreamStages = new HashMap<>(proto.getUpstreamStagesCount()); + for (Map.Entry entry : proto.getUpstreamStagesMap().entrySet()) { + upstreamStages.put(entry.getKey(), decodeNode(entry.getValue())); + } + return new Decoded(proto.getCurrentStageId(), currentStage, upstreamStages); + } + + // The Type -> StatKey class relationship is preserved by MultiStageOperator.Type, but Java's type system can't + // express the dependent type, so we suppress the resulting unchecked warning here. + @SuppressWarnings({"unchecked", "rawtypes"}) + private static StatMap deserializeStatMap(ByteString bytes, MultiStageOperator.Type type) + throws IOException { + try (DataInputStream input = new DataInputStream(bytes.newInput())) { + return StatMap.deserialize(input, (Class) type.getStatKeyClass()); + } + } +} diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/StageStatsTreeNode.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/StageStatsTreeNode.java new file mode 100644 index 000000000000..a2bc46f80e4e --- /dev/null +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/StageStatsTreeNode.java @@ -0,0 +1,121 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.pinot.query.runtime.plan; + +import com.google.common.collect.ImmutableList; +import java.util.List; +import java.util.Objects; +import org.apache.pinot.common.datatable.StatMap; +import org.apache.pinot.query.runtime.operator.MultiStageOperator; + + +/** + * Broker-side, in-memory view of the structured per-stage stats tree decoded from + * {@link org.apache.pinot.common.proto.Worker.StageStatsNode}. Mirrors the proto shape but stores a deserialized + * {@link StatMap} so that merging across multiple opchain reports (different workers of the same stage) is just a + * recursive walk + {@link StatMap#merge(StatMap)} per node. + * + *

Used by the stream-mode stats reporting path. Mutable: {@link #merge(StageStatsTreeNode)} sums the other tree's + * stat maps into this one in place. + */ +public class StageStatsTreeNode { + + /** + * Thrown by {@link StageStatsTreeNode#merge(StageStatsTreeNode)} when the two trees disagree on shape (operator type + * at any position differs, or children-count differs). The broker catches this, logs, and marks the stage as + * {@code mergeFailed} in the per-stage coverage structure. + */ + public static class ShapeMismatchException extends Exception { + public ShapeMismatchException(String message) { + super(message); + } + } + + private final MultiStageOperator.Type _type; + private final List _planNodeIds; + private final StatMap _statMap; + private final List _children; + + public StageStatsTreeNode(MultiStageOperator.Type type, List planNodeIds, StatMap statMap, + List children) { + _type = type; + _planNodeIds = ImmutableList.copyOf(planNodeIds); + _statMap = statMap; + _children = ImmutableList.copyOf(children); + } + + public MultiStageOperator.Type getType() { + return _type; + } + + public List getPlanNodeIds() { + return _planNodeIds; + } + + public StatMap getStatMap() { + return _statMap; + } + + public List getChildren() { + return _children; + } + + /** + * Sums the other tree's stats into this one. Both trees must have identical shape (same operator type at every + * position, same arity). On mismatch this method throws {@link ShapeMismatchException} and leaves this tree in an + * unspecified partially-merged state — callers should drop it. + */ + // We can prove the StatMaps share a key class because their operator type matches, but Java's type system can't + // express that, so we suppress and rely on the runtime check inside StatMap.merge. + @SuppressWarnings({"unchecked", "rawtypes"}) + public void merge(StageStatsTreeNode other) + throws ShapeMismatchException { + if (_type != other._type) { + throw new ShapeMismatchException("Operator type mismatch: " + _type + " vs " + other._type); + } + if (_children.size() != other._children.size()) { + throw new ShapeMismatchException("Children count mismatch for " + _type + ": " + _children.size() + " vs " + + other._children.size()); + } + ((StatMap) _statMap).merge(other._statMap); + for (int i = 0; i < _children.size(); i++) { + _children.get(i).merge(other._children.get(i)); + } + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof StageStatsTreeNode)) { + return false; + } + StageStatsTreeNode that = (StageStatsTreeNode) o; + return _type == that._type + && Objects.equals(_planNodeIds, that._planNodeIds) + && Objects.equals(_statMap, that._statMap) + && Objects.equals(_children, that._children); + } + + @Override + public int hashCode() { + return Objects.hash(_type, _planNodeIds, _statMap, _children); + } +} diff --git a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/plan/MultiStageStatsTreeDecoderTest.java b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/plan/MultiStageStatsTreeDecoderTest.java new file mode 100644 index 000000000000..a36822d4191e --- /dev/null +++ b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/plan/MultiStageStatsTreeDecoderTest.java @@ -0,0 +1,164 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.pinot.query.runtime.plan; + +import com.google.protobuf.ByteString; +import java.io.ByteArrayOutputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.util.List; +import org.apache.pinot.common.datatable.StatMap; +import org.apache.pinot.common.proto.Worker; +import org.apache.pinot.query.runtime.operator.AggregateOperator; +import org.apache.pinot.query.runtime.operator.BaseMailboxReceiveOperator; +import org.apache.pinot.query.runtime.operator.MailboxSendOperator; +import org.apache.pinot.query.runtime.operator.MultiStageOperator; +import org.apache.pinot.query.runtime.operator.SortOperator; +import org.testng.Assert; +import org.testng.annotations.Test; + + +/** + * Unit tests for {@link MultiStageStatsTreeDecoder} and {@link StageStatsTreeNode#merge(StageStatsTreeNode)}. + */ +public class MultiStageStatsTreeDecoderTest { + + /** + * Round-trip: build a proto tree by hand, decode it, verify shape, planNodeIds, and stat-map contents. + */ + @Test + public void testDecodeRoundTrip() + throws Exception { + StatMap receiveStat = + new StatMap<>(BaseMailboxReceiveOperator.StatKey.class) + .merge(BaseMailboxReceiveOperator.StatKey.EXECUTION_TIME_MS, 100) + .merge(BaseMailboxReceiveOperator.StatKey.EMITTED_ROWS, 10); + StatMap sendStat = + new StatMap<>(MailboxSendOperator.StatKey.class) + .merge(MailboxSendOperator.StatKey.EXECUTION_TIME_MS, 3); + + Worker.StageStatsNode receiveNode = Worker.StageStatsNode.newBuilder() + .setOperatorTypeId(MultiStageOperator.Type.MAILBOX_RECEIVE.getId()) + .addPlanNodeIds(7) + .setStatMap(serialize(receiveStat)) + .build(); + Worker.StageStatsNode sendNode = Worker.StageStatsNode.newBuilder() + .setOperatorTypeId(MultiStageOperator.Type.MAILBOX_SEND.getId()) + .addPlanNodeIds(0) + .setStatMap(serialize(sendStat)) + .addChildren(receiveNode) + .build(); + Worker.MultiStageStatsTree proto = Worker.MultiStageStatsTree.newBuilder() + .setCurrentStageId(2) + .setCurrentStage(sendNode) + .build(); + + MultiStageStatsTreeDecoder.Decoded decoded = MultiStageStatsTreeDecoder.decode(proto); + Assert.assertEquals(decoded.getCurrentStageId(), 2); + StageStatsTreeNode root = decoded.getCurrentStage(); + Assert.assertEquals(root.getType(), MultiStageOperator.Type.MAILBOX_SEND); + Assert.assertEquals(root.getPlanNodeIds(), List.of(0)); + Assert.assertEquals(root.getStatMap(), sendStat); + Assert.assertEquals(root.getChildren().size(), 1); + + StageStatsTreeNode receive = root.getChildren().get(0); + Assert.assertEquals(receive.getType(), MultiStageOperator.Type.MAILBOX_RECEIVE); + Assert.assertEquals(receive.getPlanNodeIds(), List.of(7)); + Assert.assertEquals(receive.getStatMap(), receiveStat); + Assert.assertTrue(receive.getChildren().isEmpty()); + } + + /** + * Merging two trees of the same shape sums the counters per node. + */ + @Test + public void testMergeSameShape() + throws Exception { + StageStatsTreeNode a = leafNode(MultiStageOperator.Type.AGGREGATE, + new StatMap<>(AggregateOperator.StatKey.class).merge(AggregateOperator.StatKey.EMITTED_ROWS, 5)); + StageStatsTreeNode b = leafNode(MultiStageOperator.Type.AGGREGATE, + new StatMap<>(AggregateOperator.StatKey.class).merge(AggregateOperator.StatKey.EMITTED_ROWS, 7)); + + a.merge(b); + @SuppressWarnings("unchecked") + StatMap merged = (StatMap) a.getStatMap(); + Assert.assertEquals(merged.getLong(AggregateOperator.StatKey.EMITTED_ROWS), 12); + } + + /** + * Merging a tree with mismatched operator type at the root throws ShapeMismatchException. + */ + @Test(expectedExceptions = StageStatsTreeNode.ShapeMismatchException.class) + public void testMergeTypeMismatchThrows() + throws Exception { + StageStatsTreeNode a = leafNode(MultiStageOperator.Type.AGGREGATE, + new StatMap<>(AggregateOperator.StatKey.class)); + StageStatsTreeNode b = leafNode(MultiStageOperator.Type.SORT_OR_LIMIT, + new StatMap<>(SortOperator.StatKey.class)); + a.merge(b); + } + + /** + * Merging a tree with a different number of children throws ShapeMismatchException. + */ + @Test(expectedExceptions = StageStatsTreeNode.ShapeMismatchException.class) + public void testMergeArityMismatchThrows() + throws Exception { + StageStatsTreeNode a = new StageStatsTreeNode(MultiStageOperator.Type.AGGREGATE, List.of(), + new StatMap<>(AggregateOperator.StatKey.class), List.of( + leafNode(MultiStageOperator.Type.MAILBOX_RECEIVE, + new StatMap<>(BaseMailboxReceiveOperator.StatKey.class)))); + StageStatsTreeNode b = leafNode(MultiStageOperator.Type.AGGREGATE, + new StatMap<>(AggregateOperator.StatKey.class)); + a.merge(b); + } + + /** + * Decoder rejects an unknown operator type id with DecodeFailedException — exercises mixed-version safety where a + * newer server emits a type id this older broker doesn't recognise. + */ + @Test(expectedExceptions = MultiStageStatsTreeDecoder.DecodeFailedException.class) + public void testDecodeUnknownTypeThrows() + throws Exception { + Worker.StageStatsNode bad = Worker.StageStatsNode.newBuilder() + .setOperatorTypeId(Integer.MAX_VALUE) + .build(); + Worker.MultiStageStatsTree proto = Worker.MultiStageStatsTree.newBuilder() + .setCurrentStageId(0) + .setCurrentStage(bad) + .build(); + MultiStageStatsTreeDecoder.decode(proto); + } + + // ---- helpers ---- + + private static StageStatsTreeNode leafNode(MultiStageOperator.Type type, StatMap statMap) { + return new StageStatsTreeNode(type, List.of(), statMap, List.of()); + } + + private static ByteString serialize(StatMap statMap) + throws IOException { + try (ByteArrayOutputStream baos = new ByteArrayOutputStream(); + DataOutputStream output = new DataOutputStream(baos)) { + statMap.serialize(output); + output.flush(); + return ByteString.copyFrom(baos.toByteArray()); + } + } +} From 8bea7efcd2b674afc52603a98e2aa838f804fe9f Mon Sep 17 00:00:00 2001 From: Gonzalo Ortiz <1913993+gortiz@users.noreply.github.com> Date: Tue, 5 May 2026 14:49:15 +0100 Subject: [PATCH 05/31] [multistage] Add SubmitWithStream handler skeleton on QueryServer MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Phase A.2.0 of #18375. Wires up the new bidi RPC's gRPC mechanics: - New per-call SubmitWithStreamObserver handles inbound BrokerToServer messages. First message must be `submit`; subsequent `cancel` is routed to QueryRunner.cancel; broker stream-close also cancels. - Plan submission runs on the existing _submissionExecutorService via submitInternal, identical to the unary Submit path. The submit_ack message replaces today's unary QueryResponse. - Response stream onNext calls are serialised under a per-call lock (gRPC requires onNext to be called serially). OpChainComplete / ServerDone emission is deferred: that needs a per- opchain completion hook on OpChainSchedulerService which is added in the next sub-commit. Today this skeleton runs the query end-to-end but loses stream-mode stats — only the broker-side ack and cancel-via- stream-close paths are useful as-is. --- .../query/service/server/QueryServer.java | 167 ++++++++++++++++++ 1 file changed, 167 insertions(+) diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/server/QueryServer.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/server/QueryServer.java index de88051cfb54..2e1c8bd091ae 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/server/QueryServer.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/server/QueryServer.java @@ -36,6 +36,7 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import javax.annotation.Nullable; import org.apache.commons.io.output.UnsynchronizedByteArrayOutputStream; import org.apache.pinot.common.config.TlsConfig; @@ -450,6 +451,172 @@ private void submitTimeSeriesInternal(Worker.TimeSeriesQueryRequest request, } } + /// Stream-mode submission handler. The broker keeps the stream open for the query lifetime; the server replies + /// with a {@code submit_ack} as the first message and (in subsequent commits) per-opchain + /// {@link Worker.OpChainComplete} messages followed by a final {@link Worker.ServerDone}. + /// + /// This skeleton wires up the gRPC mechanics + plan submission via the existing submission path. It does NOT yet + /// emit OpChainComplete / ServerDone — those need a per-opchain completion hook on + /// {@link org.apache.pinot.query.runtime.executor.OpChainSchedulerService}, which is layered on next. + /// Cancel still routes through the existing unary {@link #cancel(Worker.CancelRequest, StreamObserver)} RPC; broker + /// stream-close also triggers a cancel here. + @Override + public StreamObserver submitWithStream( + StreamObserver responseObserver) { + return new SubmitWithStreamObserver(responseObserver); + } + + /// Per-query state for an open {@code SubmitWithStream} call. Owns the response stream and serialises every + /// {@code onNext} call on it via a {@code synchronized} block — gRPC requires {@code StreamObserver.onNext} to be + /// called serially. + /// + /// All blocking work (plan deserialization, opchain construction) runs on + /// {@link QueryServer#_submissionExecutorService}. + private final class SubmitWithStreamObserver implements StreamObserver { + private final StreamObserver _responseObserver; + /// Serialises onNext calls on the response stream and guards mutable session state. + private final Object _streamLock = new Object(); + /// True once we've received the first {@code submit} and dispatched it. + private final AtomicBoolean _submitted = new AtomicBoolean(false); + /// True once we've completed the response stream (success or error). Idempotent guard. + private final AtomicBoolean _completed = new AtomicBoolean(false); + /// Set once we successfully parse the request id from the submit metadata. Used by cancel-via-stream. + private volatile long _requestId = -1; + + SubmitWithStreamObserver(StreamObserver responseObserver) { + _responseObserver = responseObserver; + } + + @Override + public void onNext(Worker.BrokerToServer message) { + switch (message.getPayloadCase()) { + case SUBMIT: + handleSubmit(message.getSubmit()); + break; + case CANCEL: + handleCancel(message.getCancel()); + break; + case PAYLOAD_NOT_SET: + default: + sendErrorAndComplete("Unexpected BrokerToServer payload: " + message.getPayloadCase()); + break; + } + } + + @Override + public void onError(Throwable t) { + // Broker-side stream error / disconnect. Treat like a cancel and clean up; do not reply on the response stream + // (the underlying transport is gone). + LOGGER.warn("SubmitWithStream stream error for request {}: {}", _requestId, t.getMessage()); + _completed.set(true); + cancelIfSubmitted(); + } + + @Override + public void onCompleted() { + // Broker has half-closed (no more inbound messages). At this point the server may still be running opchains. + // Until we have the per-opchain completion hook (next sub-commit), we have no way to know when stats are ready, + // so we just complete the response stream immediately. This means stream-mode queries currently lose stats — + // resolved by the next commit. + sendDoneAndComplete(); + } + + private void handleSubmit(Worker.QueryRequest request) { + if (!_submitted.compareAndSet(false, true)) { + sendErrorAndComplete("Multiple submit messages on the same stream are not allowed"); + return; + } + ServerMetrics.get().addMeteredGlobalValue(ServerMeter.MSE_QUERIES, 1L); + Map reqMetadata; + try { + reqMetadata = QueryPlanSerDeUtils.fromProtoProperties(request.getMetadata()); + } catch (Exception e) { + LOGGER.error("Caught exception while deserializing request metadata", e); + sendErrorAndComplete("Caught exception while deserializing request metadata: " + e.getMessage()); + return; + } + try { + _requestId = Long.parseLong(reqMetadata.get(MetadataKeys.REQUEST_ID)); + } catch (Exception ignored) { + // _requestId stays at -1; cancel-on-stream-close will just be a no-op. + } + long timeoutMs = Long.parseLong(reqMetadata.get(QueryOptionKey.TIMEOUT_MS)); + CompletableFuture.runAsync(() -> submitInternal(request, reqMetadata), _submissionExecutorService) + .orTimeout(timeoutMs, TimeUnit.MILLISECONDS) + .whenComplete((result, error) -> { + if (error != null) { + LOGGER.error("Caught exception while submitting request: {}", _requestId, error); + sendSubmitAck(buildErrorResponse("Caught exception while submitting request: " + error.getMessage())); + } else { + sendSubmitAck(buildOkResponse()); + } + }); + } + + private void handleCancel(Worker.CancelRequest cancel) { + // The broker still uses the unary Cancel RPC in Phase A; cancel-via-stream is plumbed through here so that + // future Phase B brokers can use the same channel without server-side changes. + try { + _queryRunner.cancel(cancel.getRequestId()); + } catch (Throwable t) { + LOGGER.warn("Caught exception cancelling request {} via SubmitWithStream", cancel.getRequestId(), t); + } + } + + private void cancelIfSubmitted() { + if (_submitted.get() && _requestId >= 0) { + try { + _queryRunner.cancel(_requestId); + } catch (Throwable t) { + LOGGER.warn("Caught exception cancelling request {} after stream error", _requestId, t); + } + } + } + + private void sendSubmitAck(Worker.QueryResponse ack) { + synchronized (_streamLock) { + if (_completed.get()) { + return; + } + _responseObserver.onNext(Worker.ServerToBroker.newBuilder().setSubmitAck(ack).build()); + } + } + + private void sendDoneAndComplete() { + synchronized (_streamLock) { + if (_completed.compareAndSet(false, true)) { + _responseObserver.onNext(Worker.ServerToBroker.newBuilder() + .setDone(Worker.ServerDone.getDefaultInstance()) + .build()); + _responseObserver.onCompleted(); + } + } + } + + private void sendErrorAndComplete(String errorMsg) { + synchronized (_streamLock) { + if (_completed.compareAndSet(false, true)) { + _responseObserver.onNext(Worker.ServerToBroker.newBuilder() + .setSubmitAck(buildErrorResponse(errorMsg)) + .build()); + _responseObserver.onCompleted(); + } + } + } + + private Worker.QueryResponse buildOkResponse() { + return Worker.QueryResponse.newBuilder() + .putMetadata(CommonConstants.Query.Response.ServerResponseStatus.STATUS_OK, "") + .build(); + } + + private Worker.QueryResponse buildErrorResponse(String errorMsg) { + return Worker.QueryResponse.newBuilder() + .putMetadata(CommonConstants.Query.Response.ServerResponseStatus.STATUS_ERROR, errorMsg) + .build(); + } + } + /// Executes a cancel request. /// /// Cancel requests should always be executed on different threads than the submission threads to be sure that From ec5645067be4e04d804180a7d2e1fdbbfea4598a Mon Sep 17 00:00:00 2001 From: Gonzalo Ortiz <1913993+gortiz@users.noreply.github.com> Date: Tue, 5 May 2026 15:01:35 +0100 Subject: [PATCH 06/31] [multistage] Wire opchain completion listener into SubmitWithStream Phase A.2.1 of #18375. Adds the per-opchain completion hook that the stream-mode handler needs to emit OpChainComplete and ServerDone. OpChainSchedulerService: - New ConcurrentMap registry. - registerCompletionListener / unregisterCompletionListener methods. - registerInternal captures stats in an AtomicReference inside the TraceRunnable and the FutureCallback invokes the listener with the captured stats on both success and error paths. Listener exceptions never block opchain teardown. QueryRunner exposes registerOpChainCompletionListener / unregisterOpChainCompletionListener as passthroughs. QueryServer.SubmitWithStreamObserver: - Counts expected opchains for the request (sum of WorkerMetadata across all stage plans). - Registers the listener BEFORE submitting (avoids race where short opchains finish before registration). - Each event encodes via MultiStageStatsTreeEncoder, builds an OpChainComplete proto, and emits on the response stream under the per-call lock. - Encoding failures (e.g. tree/flat-list mismatch on error path) are surfaced on OpChainComplete with success=false + error_msg, never block stream completion. - After all expected opchains have reported, unregisters the listener and emits ServerDone, closing the stream. Tests cover listener fire on success, fire on error, and no-fire after unregister. --- .../pinot/query/runtime/QueryRunner.java | 18 ++++ .../executor/OpChainCompletionListener.java | 58 ++++++++++++ .../executor/OpChainSchedulerService.java | 48 ++++++++++ .../query/service/server/QueryServer.java | 92 ++++++++++++++++++- .../executor/OpChainSchedulerServiceTest.java | 76 +++++++++++++++ 5 files changed, 287 insertions(+), 5 deletions(-) create mode 100644 pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/executor/OpChainCompletionListener.java diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/QueryRunner.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/QueryRunner.java index 53398023be7b..03a9e13b302b 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/QueryRunner.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/QueryRunner.java @@ -52,6 +52,7 @@ import org.apache.pinot.query.routing.StagePlan; import org.apache.pinot.query.routing.WorkerMetadata; import org.apache.pinot.query.runtime.blocks.ErrorMseBlock; +import org.apache.pinot.query.runtime.executor.OpChainCompletionListener; import org.apache.pinot.query.runtime.executor.OpChainSchedulerService; import org.apache.pinot.query.runtime.operator.LeafOperator; import org.apache.pinot.query.runtime.operator.MultiStageOperator; @@ -527,6 +528,23 @@ public Map cancel(long requestI return _opChainScheduler.cancel(requestId); } + /** + * Registers an opchain completion listener for the given request id. Used by the stream-mode stats reporting path + * (gRPC {@code SubmitWithStream}) so the {@link org.apache.pinot.query.service.server.QueryServer} can be notified + * each time an opchain finishes and emit a corresponding {@code OpChainComplete} message on the broker stream. + * + *

The listener fires once per opchain that runs on this server for the request and must be unregistered by the + * caller (typically when the per-request opchain count reaches the expected total) via + * {@link #unregisterOpChainCompletionListener(long)}. + */ + public void registerOpChainCompletionListener(long requestId, OpChainCompletionListener listener) { + _opChainScheduler.registerCompletionListener(requestId, listener); + } + + public void unregisterOpChainCompletionListener(long requestId) { + _opChainScheduler.unregisterCompletionListener(requestId); + } + public StagePlan explainQuery(WorkerMetadata workerMetadata, StagePlan stagePlan, Map requestMetadata) { if (!workerMetadata.isLeafStageWorker()) { diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/executor/OpChainCompletionListener.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/executor/OpChainCompletionListener.java new file mode 100644 index 000000000000..1207e0da97ee --- /dev/null +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/executor/OpChainCompletionListener.java @@ -0,0 +1,58 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.pinot.query.runtime.executor; + +import javax.annotation.Nullable; +import org.apache.pinot.query.runtime.operator.MultiStageOperator; +import org.apache.pinot.query.runtime.operator.OpChainId; +import org.apache.pinot.query.runtime.plan.MultiStageQueryStats; +import org.apache.pinot.query.runtime.plan.OpChainExecutionContext; + + +/** + * Listener invoked by {@link OpChainSchedulerService} once an opchain finishes (either successfully or with an + * error). Used by the stream-mode stats-reporting path to encode and emit + * {@code OpChainComplete} messages on the corresponding broker→server gRPC stream. + * + *

Listeners are registered against a request id and fire for every opchain that runs on this server for that + * request. The listener implementation is responsible for cleaning itself up (calling + * {@link OpChainSchedulerService#unregisterCompletionListener(long)}) once it has received all the events it + * expects, typically when the per-request opchain count reaches the expected total. + * + *

The listener is invoked on the gRPC opchain executor thread that ran the opchain. Implementations must not + * block. + */ +@FunctionalInterface +public interface OpChainCompletionListener { + + /** + * Invoked once per opchain completion. + * + * @param opChainId stage + worker + request id of the finished opchain + * @param rootOperator the root operator of the finished opchain — still alive at the time of the call but the + * scheduler will close it as soon as the listener returns. The listener may walk it but must not retain a + * reference past the call + * @param stats the stats accumulated by this opchain. {@code null} when the opchain failed before reaching the + * point where stats are calculated + * @param context the execution context for the opchain (carries the op→PlanNode map for stats encoding) + * @param error {@code null} on success, the failure cause on error + */ + void onOpChainComplete(OpChainId opChainId, MultiStageOperator rootOperator, + @Nullable MultiStageQueryStats stats, OpChainExecutionContext context, @Nullable Throwable error); +} diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/executor/OpChainSchedulerService.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/executor/OpChainSchedulerService.java index 599d052cf24e..bda6bf9594cc 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/executor/OpChainSchedulerService.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/executor/OpChainSchedulerService.java @@ -28,11 +28,15 @@ import java.util.ArrayList; import java.util.HashMap; import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; +import javax.annotation.Nullable; import org.apache.commons.lang3.tuple.Pair; import org.apache.pinot.common.datatable.StatMap; import org.apache.pinot.common.metrics.ServerMeter; @@ -69,6 +73,10 @@ public class OpChainSchedulerService { private final ReadWriteLock[] _queryLocks; private final Cache _cancelledQueryCache; private final Metrics _metrics = new Metrics(); + /// Per-request opchain completion listeners used by the stream-mode stats-reporting path. A listener is registered + /// before the broker submits the query and fires once for every opchain that runs on this server for that request. + /// Listeners are responsible for unregistering themselves once they've consumed all expected events. + private final ConcurrentMap _completionListeners = new ConcurrentHashMap<>(); public OpChainSchedulerService(String instanceId, ExecutorService executorService, PinotConfiguration config) { this(instanceId, executorService, config.getProperty(MultiStageQueryRunner.KEY_OF_OP_STATS_CACHE_SIZE, @@ -152,6 +160,10 @@ private void registerInternal(OpChain operatorChain, QueryExecutionContext execu OpChainId opChainId = operatorChain.getId(); MultiStageOperator rootOperator = operatorChain.getRoot(); _opChainCache.put(opChainId, Pair.of(rootOperator, executionContext)); + // Captured by the runJob and read by the FutureCallback so we can hand the calculated stats to the per-request + // completion listener (stream-mode stats reporting). On error we may not have stats; the FutureCallback handles + // that by passing null. + AtomicReference statsRef = new AtomicReference<>(); // Create a ListenableFutureTask to ensure the opChain is cancelled even if the task is not scheduled ListenableFutureTask listenableFutureTask = ListenableFutureTask.create(new TraceRunnable() { @@ -164,6 +176,7 @@ public void runJob() { result = rootOperator.nextBlock(); } MultiStageQueryStats stats = rootOperator.calculateStats(); + statsRef.set(stats); if (result.isError()) { ErrorMseBlock errorBlock = (ErrorMseBlock) result; throw errorBlock.getMainErrorCode().asException("Error block " @@ -180,6 +193,7 @@ public void runJob() { @Override public void onSuccess(Void result) { _metrics.onOpChainFinished(rootOperator); + notifyCompletionListener(opChainId, operatorChain, statsRef.get(), null); operatorChain.close(); } @@ -200,6 +214,7 @@ public void onFailure(Throwable t) { } else { LOGGER.error(logMsg, t); } + notifyCompletionListener(opChainId, operatorChain, statsRef.get(), t); operatorChain.cancel(t); operatorChain.close(); } @@ -208,6 +223,39 @@ public void onFailure(Throwable t) { _executorService.submit(listenableFutureTask); } + private void notifyCompletionListener(OpChainId opChainId, OpChain operatorChain, + @Nullable MultiStageQueryStats stats, @Nullable Throwable error) { + OpChainCompletionListener listener = _completionListeners.get(opChainId.getRequestId()); + if (listener == null) { + return; + } + try { + listener.onOpChainComplete(opChainId, operatorChain.getRoot(), stats, operatorChain.getContext(), error); + } catch (Throwable listenerError) { + // The listener is best-effort observability; never let it interfere with opchain teardown. + LOGGER.warn("OpChain completion listener threw for {}", opChainId, listenerError); + } + } + + /** + * Registers an {@link OpChainCompletionListener} that fires once per opchain finishing for the given request id. + * Used by the stream-mode stats reporting path. Returns the previously-registered listener, or {@code null} if + * none — callers should always pass {@code null}. + */ + @Nullable + public OpChainCompletionListener registerCompletionListener(long requestId, OpChainCompletionListener listener) { + return _completionListeners.put(requestId, listener); + } + + /** + * Removes any registered listener for the given request id. Idempotent — safe to call even if no listener was + * registered. Returns the removed listener, or {@code null}. + */ + @Nullable + public OpChainCompletionListener unregisterCompletionListener(long requestId) { + return _completionListeners.remove(requestId); + } + public Map cancel(long requestId) { QueryExecutionContext cancelledExecutionContext = null; Map cancelledOperators = new HashMap<>(); diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/server/QueryServer.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/server/QueryServer.java index 2e1c8bd091ae..6da8338b7282 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/server/QueryServer.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/server/QueryServer.java @@ -37,6 +37,7 @@ import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; import javax.annotation.Nullable; import org.apache.commons.io.output.UnsynchronizedByteArrayOutputStream; import org.apache.pinot.common.config.TlsConfig; @@ -58,7 +59,11 @@ import org.apache.pinot.query.routing.StagePlan; import org.apache.pinot.query.routing.WorkerMetadata; import org.apache.pinot.query.runtime.QueryRunner; +import org.apache.pinot.query.runtime.operator.MultiStageOperator; +import org.apache.pinot.query.runtime.operator.OpChainId; import org.apache.pinot.query.runtime.plan.MultiStageQueryStats; +import org.apache.pinot.query.runtime.plan.MultiStageStatsTreeEncoder; +import org.apache.pinot.query.runtime.plan.OpChainExecutionContext; import org.apache.pinot.spi.accounting.ThreadAccountant; import org.apache.pinot.spi.accounting.ThreadAccountantUtils; import org.apache.pinot.spi.env.PinotConfiguration; @@ -470,6 +475,12 @@ public StreamObserver submitWithStream( /// {@code onNext} call on it via a {@code synchronized} block — gRPC requires {@code StreamObserver.onNext} to be /// called serially. /// + /// Tracks the expected number of opchains for the request (sum of WorkerMetadata across all stages). An + /// {@link OpChainCompletionListener} registered with {@link QueryRunner#registerOpChainCompletionListener} + /// fires once per opchain finishing, encodes its stats via {@link MultiStageStatsTreeEncoder}, and emits an + /// {@link Worker.OpChainComplete} on the response stream. When the per-request completed-count reaches the + /// expected total, {@link Worker.ServerDone} is emitted and the stream is closed. + /// /// All blocking work (plan deserialization, opchain construction) runs on /// {@link QueryServer#_submissionExecutorService}. private final class SubmitWithStreamObserver implements StreamObserver { @@ -480,6 +491,10 @@ private final class SubmitWithStreamObserver implements StreamObserver= 0 && expected > 0) { + _queryRunner.registerOpChainCompletionListener(_requestId, this::onOpChainComplete); + } + long timeoutMs = Long.parseLong(reqMetadata.get(QueryOptionKey.TIMEOUT_MS)); CompletableFuture.runAsync(() -> submitInternal(request, reqMetadata), _submissionExecutorService) .orTimeout(timeoutMs, TimeUnit.MILLISECONDS) @@ -547,12 +576,65 @@ private void handleSubmit(Worker.QueryRequest request) { if (error != null) { LOGGER.error("Caught exception while submitting request: {}", _requestId, error); sendSubmitAck(buildErrorResponse("Caught exception while submitting request: " + error.getMessage())); + // Submission failed — no opchains will run, so emit ServerDone immediately and clean up. + cleanupListener(); + sendDoneAndComplete(); } else { sendSubmitAck(buildOkResponse()); + // If for some reason expected was 0 (empty plan), close the stream now. + if (expected == 0) { + cleanupListener(); + sendDoneAndComplete(); + } } }); } + /** + * Fires once per opchain on this server completing. Encodes the stats into a {@link Worker.MultiStageStatsTree}, + * emits an {@link Worker.OpChainComplete}, and emits {@link Worker.ServerDone} once all expected opchains have + * reported. + */ + private void onOpChainComplete(OpChainId opChainId, MultiStageOperator rootOperator, + @Nullable MultiStageQueryStats stats, OpChainExecutionContext context, @Nullable Throwable error) { + Worker.OpChainComplete.Builder builder = Worker.OpChainComplete.newBuilder() + .setStageId(opChainId.getStageId()) + .setWorkerId(opChainId.getVirtualServerId()) + .setSuccess(error == null); + if (error != null) { + builder.setErrorMsg(error.getMessage() == null ? error.getClass().getSimpleName() : error.getMessage()); + } + if (stats != null) { + try { + builder.setStats(MultiStageStatsTreeEncoder.encode(rootOperator, stats, context)); + } catch (Throwable t) { + // Encoding failed (operator tree / flat list mismatch on error path is the most likely cause). Surface the + // error on the OpChainComplete so the broker can mark mergeFailed, but don't block stream completion. + LOGGER.warn("Failed to encode stats tree for opchain {}", opChainId, t); + builder.setSuccess(false); + builder.setErrorMsg(builder.getErrorMsg().isEmpty() ? "stats encode failed: " + t.getMessage() + : builder.getErrorMsg() + "; stats encode failed: " + t.getMessage()); + } + } + Worker.ServerToBroker message = Worker.ServerToBroker.newBuilder().setOpchain(builder).build(); + synchronized (_streamLock) { + if (!_completed.get()) { + _responseObserver.onNext(message); + } + } + // After all expected opchains have reported, emit ServerDone and close. + if (_completedOpChains.incrementAndGet() >= _expectedOpChains.get()) { + cleanupListener(); + sendDoneAndComplete(); + } + } + + private void cleanupListener() { + if (_requestId >= 0) { + _queryRunner.unregisterOpChainCompletionListener(_requestId); + } + } + private void handleCancel(Worker.CancelRequest cancel) { // The broker still uses the unary Cancel RPC in Phase A; cancel-via-stream is plumbed through here so that // future Phase B brokers can use the same channel without server-side changes. diff --git a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/executor/OpChainSchedulerServiceTest.java b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/executor/OpChainSchedulerServiceTest.java index 196e43eb2694..403e635a3610 100644 --- a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/executor/OpChainSchedulerServiceTest.java +++ b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/executor/OpChainSchedulerServiceTest.java @@ -24,6 +24,9 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; import org.apache.pinot.common.datatable.StatMap; import org.apache.pinot.common.utils.NamedThreadFactory; import org.apache.pinot.query.mailbox.MailboxService; @@ -49,6 +52,7 @@ import static org.mockito.Mockito.clearInvocations; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; +import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; @@ -191,6 +195,78 @@ public void shouldCallCancelOnOpChainsWhenItIsCancelledByDispatch() Mockito.verify(_operatorA, Mockito.times(1)).cancel(Mockito.any()); } + @Test + public void shouldFireCompletionListenerOnSuccess() + throws InterruptedException { + CountDownLatch completed = new CountDownLatch(1); + Mockito.when(_operatorA.nextBlock()).thenReturn(SuccessMseBlock.INSTANCE); + Mockito.doAnswer(inv -> MultiStageQueryStats.emptyStats(0)).when(_operatorA).calculateStats(); + + OpChainSchedulerService schedulerService = new OpChainSchedulerService(_executor); + AtomicBoolean errorWasNull = new AtomicBoolean(false); + AtomicReference seenStats = + new AtomicReference<>(); + schedulerService.registerCompletionListener(123L, (id, root, stats, ctx, error) -> { + errorWasNull.set(error == null); + seenStats.set(stats); + completed.countDown(); + }); + try (QueryThreadContext ignore = QueryThreadContext.openForMseTest()) { + schedulerService.register(getChain(_operatorA)); + } + + assertTrue(completed.await(10, TimeUnit.SECONDS), "expected completion listener to fire"); + assertTrue(errorWasNull.get(), "expected error to be null on successful opchain"); + assertTrue(seenStats.get() != null, "expected stats to be passed to listener"); + } + + @Test + public void shouldFireCompletionListenerOnError() + throws InterruptedException { + CountDownLatch completed = new CountDownLatch(1); + Mockito.when(_operatorA.nextBlock()).thenThrow(new RuntimeException("boom")); + + OpChainSchedulerService schedulerService = new OpChainSchedulerService(_executor); + AtomicReference seenError = + new AtomicReference<>(); + schedulerService.registerCompletionListener(123L, (id, root, stats, ctx, error) -> { + seenError.set(error); + completed.countDown(); + }); + try (QueryThreadContext ignore = QueryThreadContext.openForMseTest()) { + schedulerService.register(getChain(_operatorA)); + } + + assertTrue(completed.await(10, TimeUnit.SECONDS), "expected completion listener to fire on error"); + assertTrue(seenError.get() != null, "expected error to be propagated to listener"); + } + + @Test + public void shouldNotFireCompletionListenerAfterUnregister() + throws InterruptedException { + CountDownLatch opChainCloseLatch = new CountDownLatch(1); + Mockito.when(_operatorA.nextBlock()).thenReturn(SuccessMseBlock.INSTANCE); + Mockito.doAnswer(inv -> MultiStageQueryStats.emptyStats(0)).when(_operatorA).calculateStats(); + Mockito.doAnswer(inv -> { + opChainCloseLatch.countDown(); + return null; + }).when(_operatorA).close(); + + OpChainSchedulerService schedulerService = new OpChainSchedulerService(_executor); + AtomicInteger fired = new AtomicInteger(); + OpChainCompletionListener listener = (id, root, stats, ctx, error) -> fired.incrementAndGet(); + schedulerService.registerCompletionListener(123L, listener); + schedulerService.unregisterCompletionListener(123L); + + try (QueryThreadContext ignore = QueryThreadContext.openForMseTest()) { + schedulerService.register(getChain(_operatorA)); + } + + assertTrue(opChainCloseLatch.await(10, TimeUnit.SECONDS), "expected opchain to finish"); + // Listener was unregistered before the opchain ran, so it must not fire. + assertEquals(fired.get(), 0, "listener should not fire after unregister"); + } + @Test public void shouldCallCancelOnOpChainsThatThrow() throws InterruptedException { From 923cd243814fbf499a5ecb922c4f54702e49a2ab Mon Sep 17 00:00:00 2001 From: Gonzalo Ortiz <1913993+gortiz@users.noreply.github.com> Date: Tue, 5 May 2026 15:19:00 +0100 Subject: [PATCH 07/31] [multistage] Suppress mailbox stats when SubmitWithStream is in use Phase A.2.2 of #18375. In stream mode the broker collects stats out-of- band on the bidi RPC via OpChainComplete, so paying the cost of also serializing them onto the mailbox EOS path is wasteful. Per-request override: - New CommonConstants.MultiStageQueryRunner.KEY_OF_STATS_REPORTING_MODE metadata key with values STATS_REPORTING_MODE_STREAM / _LEGACY. - QueryServer.SubmitWithStreamObserver.handleSubmit injects the key with value "stream" into reqMetadata before invoking submitInternal. - QueryRunner.processQueryBlocking computes effectiveSendStats(...) from the metadata, forcing sendStats=false when the key is "stream". All downstream consumers (PipelineBreakerExecutor, OpChainExecution- Context, error path) use the effective value. - Comment added on QueryRunner._sendStats noting the per-request override; effectiveSendStats() Javadoc points back to the metadata key. Not user-facing: brokers do not set this key; it exists purely as an internal channel from the SubmitWithStream handler to the runner. --- .../pinot/query/runtime/QueryRunner.java | 34 +++++++++++++++++-- .../query/service/server/QueryServer.java | 11 ++++-- .../pinot/spi/utils/CommonConstants.java | 13 +++++++ 3 files changed, 53 insertions(+), 5 deletions(-) diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/QueryRunner.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/QueryRunner.java index 03a9e13b302b..8a40d20784ea 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/QueryRunner.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/QueryRunner.java @@ -132,6 +132,9 @@ public class QueryRunner { private WindowOverFlowMode _windowOverflowMode; @Nullable private PhysicalTimeSeriesServerPlanVisitor _timeSeriesPhysicalPlanVisitor; + /// Cluster-level decision on whether to send stats over the mailbox path, driven by the {@code SendStatsPredicate} + /// at startup time. May be overridden per-request via the {@code KEY_OF_STATS_REPORTING_MODE} metadata key — + /// see {@link #effectiveSendStats(Map)}. private BooleanSupplier _sendStats; private BooleanSupplier _keepPipelineBreakerStats; @@ -299,10 +302,15 @@ private void processQueryBlocking(WorkerMetadata workerMetadata, StagePlan stage StageMetadata stageMetadata = stagePlan.getStageMetadata(); Map opChainMetadata = consolidateMetadata(stageMetadata.getCustomProperties(), requestMetadata); + // The cluster-level _sendStats decision can be overridden per-request by the SubmitWithStream RPC handler via + // MultiStageQueryRunner.KEY_OF_STATS_REPORTING_MODE; in stream mode stats travel out-of-band + // and we suppress the mailbox-side path to avoid duplication. + boolean sendStats = effectiveSendStats(requestMetadata); + // run pre-stage execution for all pipeline breakers PipelineBreakerResult pipelineBreakerResult = PipelineBreakerExecutor.executePipelineBreakers( _opChainScheduler, _mailboxService, workerMetadata, stagePlan, opChainMetadata, - _sendStats.getAsBoolean(), _keepPipelineBreakerStats.getAsBoolean()); + sendStats, _keepPipelineBreakerStats.getAsBoolean()); // Send error block to all the receivers if pipeline breaker fails if (pipelineBreakerResult != null && pipelineBreakerResult.getErrorBlock() != null) { @@ -315,7 +323,7 @@ private void processQueryBlocking(WorkerMetadata workerMetadata, StagePlan stage // run OpChain OpChainExecutionContext executionContext = OpChainExecutionContext.fromQueryContext(_mailboxService, opChainMetadata, stageMetadata, workerMetadata, - pipelineBreakerResult, _sendStats.getAsBoolean(), _keepPipelineBreakerStats.getAsBoolean()); + pipelineBreakerResult, sendStats, _keepPipelineBreakerStats.getAsBoolean()); try { OpChain opChain; if (workerMetadata.isLeafStageWorker()) { @@ -337,6 +345,10 @@ private void processQueryBlocking(WorkerMetadata workerMetadata, StagePlan stage /** * Attempts to propagate stage failures through the active {@link OpChainConverter}. + *

+ * Note: the cluster-level {@code _sendStats} decision can be overridden per-request via + * {@link CommonConstants.MultiStageQueryRunner#KEY_OF_STATS_REPORTING_MODE} — see + * {@link #effectiveSendStats(Map)}. */ private void tryPropagateErrorViaOpChainConverter(WorkerMetadata workerMetadata, StagePlan stagePlan, Map opChainMetadata, @Nullable PipelineBreakerResult pipelineBreakerResult, @@ -349,7 +361,7 @@ private void tryPropagateErrorViaOpChainConverter(WorkerMetadata workerMetadata, try { OpChainExecutionContext executionContext = OpChainExecutionContext.fromQueryContext(_mailboxService, opChainMetadata, stageMetadata, workerMetadata, - pipelineBreakerResult, _sendStats.getAsBoolean(), _keepPipelineBreakerStats.getAsBoolean()); + pipelineBreakerResult, effectiveSendStats(opChainMetadata), _keepPipelineBreakerStats.getAsBoolean()); OpChain errorOpChain = OpChainConverterDispatcher.sendEarlyError(executionContext, stagePlan, errorBlock); _opChainScheduler.register(errorOpChain); } catch (RuntimeException e) { @@ -358,6 +370,22 @@ private void tryPropagateErrorViaOpChainConverter(WorkerMetadata workerMetadata, } } + /** + * Returns the effective {@code sendStats} flag for the current request: starts from the cluster-level + * {@link #_sendStats} decision and forces it to {@code false} when + * {@link CommonConstants.MultiStageQueryRunner#KEY_OF_STATS_REPORTING_MODE} is set to + * {@link CommonConstants.MultiStageQueryRunner#STATS_REPORTING_MODE_STREAM} on the request metadata. The + * stream-mode handler injects this key when stats are being collected out-of-band on the bidi RPC, so the + * mailbox-side path can be skipped. + */ + private boolean effectiveSendStats(Map requestMetadata) { + String mode = requestMetadata.get(MultiStageQueryRunner.KEY_OF_STATS_REPORTING_MODE); + if (MultiStageQueryRunner.STATS_REPORTING_MODE_STREAM.equals(mode)) { + return false; + } + return _sendStats.getAsBoolean(); + } + /** * Receives a serialized plan sent by the broker, and runs it to completion, blocking the thread until the execution * is complete. diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/server/QueryServer.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/server/QueryServer.java index 6da8338b7282..f6b5a8d21d1b 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/server/QueryServer.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/server/QueryServer.java @@ -28,6 +28,7 @@ import java.io.DataOutputStream; import java.util.ArrayDeque; import java.util.ArrayList; +import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -542,14 +543,20 @@ private void handleSubmit(Worker.QueryRequest request) { return; } ServerMetrics.get().addMeteredGlobalValue(ServerMeter.MSE_QUERIES, 1L); - Map reqMetadata; + Map deserializedMetadata; try { - reqMetadata = QueryPlanSerDeUtils.fromProtoProperties(request.getMetadata()); + deserializedMetadata = QueryPlanSerDeUtils.fromProtoProperties(request.getMetadata()); } catch (Exception e) { LOGGER.error("Caught exception while deserializing request metadata", e); sendErrorAndComplete("Caught exception while deserializing request metadata: " + e.getMessage()); return; } + // Override the cluster-level _sendStats decision for this request: stats travel out-of-band on the bidi stream + // (via the OpChainCompletionListener), so we suppress the mailbox-side stats path. The override is read by + // QueryRunner.effectiveSendStats(...). + Map reqMetadata = new HashMap<>(deserializedMetadata); + reqMetadata.put(CommonConstants.MultiStageQueryRunner.KEY_OF_STATS_REPORTING_MODE, + CommonConstants.MultiStageQueryRunner.STATS_REPORTING_MODE_STREAM); try { _requestId = Long.parseLong(reqMetadata.get(MetadataKeys.REQUEST_ID)); } catch (Exception ignored) { diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/CommonConstants.java b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/CommonConstants.java index 916ca208a4aa..086e928fa569 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/CommonConstants.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/CommonConstants.java @@ -2153,6 +2153,19 @@ public static class MultiStageQueryRunner { public static final String KEY_OF_SEND_STATS_MODE = "pinot.query.mse.stats.mode"; public static final String DEFAULT_SEND_STATS_MODE = "SAFE"; + /// Per-request metadata key that overrides the cluster-level send-stats decision for the duration of a single + /// query. Set automatically by the {@code SubmitWithStream} bidi RPC handler on the server: when stats travel + /// out-of-band on the bidi stream there is no point in also paying the cost of serializing them onto the mailbox + /// path, so the mailbox-side {@code sendStats} flag is forced to {@code false} for that request. + /// + /// This is **not** a user-facing option — it exists purely as a server-internal channel from the + /// {@code SubmitWithStream} handler down to {@code QueryRunner.processQueryBlocking}. Brokers do not set it. + public static final String KEY_OF_STATS_REPORTING_MODE = "pinot.query.mse.statsReportingMode"; + /// Value indicating the new bidi-stream stats reporting path is in use; mailbox-side stats are suppressed. + public static final String STATS_REPORTING_MODE_STREAM = "stream"; + /// Value indicating today's legacy mailbox-piggyback stats reporting path. Equivalent to leaving the key unset. + public static final String STATS_REPORTING_MODE_LEGACY = "legacy"; + /// Used to indicate whether MSE pipeline breaker stats should be included in the queryStats field. /// This flag was introduced in 1.5.0. Before 1.5.0, MSE pipeline breaker stats were not kept. Starting from 1.5.0, /// they are not included by default but can be included by setting this flag to false (upper or lower case). From 63a76b81e56ee699c6f65247297d22b5f90bbc7c Mon Sep 17 00:00:00 2001 From: Gonzalo Ortiz <1913993+gortiz@users.noreply.github.com> Date: Tue, 5 May 2026 15:33:20 +0100 Subject: [PATCH 08/31] [multistage] Broker-side streaming session + completion semantics MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Phase A.3.0 of #18375. Per-query state container for the broker's SubmitWithStream dispatch path: - StreamingQuerySession owns the per-stage tree accumulator (Map), per-stage counters (responded / mergeFailed), open-streams set, and a CountDownLatch initialised to expectedOpChains. - Completion semantics: awaitCompletion(timeout, unit) returns true as soon as every expected opchain has reported (early completion — the common case in stream mode), and false if the timeout fires first. The dispatcher will only call this AFTER the broker receiving mailbox has finished, so a `true` return means "data done AND stats fully accounted for". - recordOpChainComplete decodes the MultiStageStatsTree, merges into the per-stage accumulator (StatMap.merge), and on shape mismatch / decode failure marks the stage mergeFailed and continues. success= false reports trigger fan-out cancel exactly once. - recordStreamError variant lets the dispatcher drain the latch by the number of opchains a dead server still owed; also fires fan-out cancel. - StreamingServerHandle is the abstraction over an open server stream used for cancel fan-out, so the session doesn't depend on concrete gRPC stub types. Coverage (snapshotCoverage()) is exposed as an immutable view of responded / mergeFailed counts plus the merged accumulator. Missing opchains are computed by the caller against the expected total. Tests cover early completion (returns immediately when all reports arrived before the wait window), timeout fall-through, cross-worker stats sum, fan-out cancel idempotency on peer error, stream-error draining the latch by remainingExpected, and concurrent reports across 50 threads. --- .../streaming/StreamingQuerySession.java | 316 ++++++++++++++++++ .../streaming/StreamingServerHandle.java | 32 ++ .../streaming/StreamingQuerySessionTest.java | 232 +++++++++++++ 3 files changed, 580 insertions(+) create mode 100644 pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/streaming/StreamingQuerySession.java create mode 100644 pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/streaming/StreamingServerHandle.java create mode 100644 pinot-query-runtime/src/test/java/org/apache/pinot/query/service/dispatch/streaming/StreamingQuerySessionTest.java diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/streaming/StreamingQuerySession.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/streaming/StreamingQuerySession.java new file mode 100644 index 000000000000..1bd8ff1c7985 --- /dev/null +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/streaming/StreamingQuerySession.java @@ -0,0 +1,316 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.pinot.query.service.dispatch.streaming; + +import java.util.Collections; +import java.util.HashMap; +import java.util.LinkedHashSet; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.ReentrantLock; +import javax.annotation.Nullable; +import org.apache.pinot.common.proto.Worker; +import org.apache.pinot.query.runtime.plan.MultiStageStatsTreeDecoder; +import org.apache.pinot.query.runtime.plan.StageStatsTreeNode; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * Broker-side per-query state for the {@code SubmitWithStream} dispatch path. Owns the per-stage tree accumulator, + * the outstanding-opchain count, the per-stage coverage counters, and the set of open server streams. + * + *

Concurrency model — all mutating methods acquire the per-session lock, so the accumulator and counters need no + * additional internal synchronization. gRPC client {@code onNext} callbacks land on I/O threads and call into this + * session directly; the work per call is short (decode + merge + decrement) so doing it on the I/O thread is fine. + * + *

Completion semantics — {@link #awaitCompletion(long, TimeUnit)} returns {@code true} as soon as every expected + * opchain has reported (early completion), and {@code false} if the timeout fires first. The dispatcher should call + * it only after the broker receiving mailbox has finished, so that a successful return means both + * "data done" and "stats fully accounted for". When it returns {@code false} the per-stage coverage exposes which + * stages are missing. + */ +public class StreamingQuerySession { + private static final Logger LOGGER = LoggerFactory.getLogger(StreamingQuerySession.class); + + private final long _requestId; + private final int _expectedOpChains; + private final CountDownLatch _completionLatch; + private final ReentrantLock _lock = new ReentrantLock(); + + /** Per-stage merged accumulator. Mutated under {@link #_lock}. */ + private final Map _stageAccumulator = new HashMap<>(); + /** Per-stage count of opchains that have responded successfully and merged cleanly. */ + private final Map _respondedByStage = new HashMap<>(); + /** Per-stage count of opchains that responded but the broker couldn't merge their payload. */ + private final Map _mergeFailedByStage = new HashMap<>(); + + /** Set of open server streams. Iteration order is insertion order so cancel fan-out is deterministic. */ + private final Set _openStreams = new LinkedHashSet<>(); + + /** True after the first peer error (success=false OpChainComplete or stream onError). Used to trigger fan-out + * cancel idempotently. */ + private boolean _peerErrorObserved = false; + + public StreamingQuerySession(long requestId, int expectedOpChains) { + _requestId = requestId; + _expectedOpChains = expectedOpChains; + _completionLatch = new CountDownLatch(expectedOpChains); + } + + public long getRequestId() { + return _requestId; + } + + public int getExpectedOpChains() { + return _expectedOpChains; + } + + /** + * Registers an open server stream so the session can iterate them later for fan-out cancel. Must be called by the + * dispatcher when the {@code SubmitWithStream} call is opened. + */ + public void registerStream(StreamingServerHandle stream) { + _lock.lock(); + try { + _openStreams.add(stream); + } finally { + _lock.unlock(); + } + } + + /** + * Removes a stream from the open-streams set. Called when the server emits {@code ServerDone} (clean close) or the + * stream errors. Idempotent. + */ + public void unregisterStream(StreamingServerHandle stream) { + _lock.lock(); + try { + _openStreams.remove(stream); + } finally { + _lock.unlock(); + } + } + + /** + * Records an {@link Worker.OpChainComplete} message decoded from a server stream. Decrements the outstanding count + * and merges the contained tree into the per-stage accumulator (or marks the stage {@code mergeFailed} on a shape + * mismatch / decode failure). Also records {@code success=false} reports as peer errors so fan-out cancel can fire. + */ + public void recordOpChainComplete(Worker.OpChainComplete message) { + int stageId = message.getStageId(); + boolean isSuccess = message.getSuccess(); + Worker.MultiStageStatsTree statsTree = message.getStats(); + + boolean shouldFanOutCancel = false; + _lock.lock(); + try { + if (!isSuccess) { + if (!_peerErrorObserved) { + _peerErrorObserved = true; + shouldFanOutCancel = true; + } + } + if (statsTree.hasCurrentStage()) { + try { + MultiStageStatsTreeDecoder.Decoded decoded = MultiStageStatsTreeDecoder.decode(statsTree); + mergeIntoAccumulatorLocked(decoded.getCurrentStageId(), decoded.getCurrentStage()); + for (Map.Entry upstream : decoded.getUpstreamStages().entrySet()) { + mergeIntoAccumulatorLocked(upstream.getKey(), upstream.getValue()); + } + incrementLocked(_respondedByStage, stageId); + } catch (MultiStageStatsTreeDecoder.DecodeFailedException e) { + LOGGER.warn("Decode failed for opchain stage={} worker={} on request {}: {}", + stageId, message.getWorkerId(), _requestId, e.getMessage()); + incrementLocked(_mergeFailedByStage, stageId); + } + } else { + // Successful opchain that produced no stats tree (rare but possible — e.g. an empty plan). Still counts as + // "responded" so we can finalize. + incrementLocked(_respondedByStage, stageId); + } + } finally { + _lock.unlock(); + } + _completionLatch.countDown(); + if (shouldFanOutCancel) { + fanOutCancel(); + } + } + + private void mergeIntoAccumulatorLocked(int stageId, StageStatsTreeNode incoming) { + StageStatsTreeNode existing = _stageAccumulator.get(stageId); + if (existing == null) { + _stageAccumulator.put(stageId, incoming); + return; + } + try { + existing.merge(incoming); + } catch (StageStatsTreeNode.ShapeMismatchException e) { + LOGGER.warn("Shape mismatch merging stage {} on request {}: {}", stageId, _requestId, e.getMessage()); + incrementLocked(_mergeFailedByStage, stageId); + } + } + + private static void incrementLocked(Map counter, int stageId) { + counter.merge(stageId, 1, Integer::sum); + } + + /** + * Records a transport-level error on one of the server streams (gRPC {@code onError}). Treated as a fatal report + * for that opchain (drains the latch by 1) and triggers fan-out cancel if not already triggered. + * + *

Idempotent w.r.t. the same stream — if the stream already errored, subsequent calls are no-ops on the latch + * but may still trigger fan-out cancel if it hasn't fired yet. + */ + public void recordStreamError(StreamingServerHandle stream, @Nullable Throwable error) { + boolean shouldFanOutCancel = false; + boolean wasOpen; + _lock.lock(); + try { + wasOpen = _openStreams.remove(stream); + if (!_peerErrorObserved) { + _peerErrorObserved = true; + shouldFanOutCancel = true; + } + } finally { + _lock.unlock(); + } + if (wasOpen) { + // Drain one pending opchain from the latch on the assumption that this stream represented at least one opchain + // that will not now report. The dispatcher passes per-server expected counts to {@link #recordStreamErrorWith} + // when finer accounting is needed. + _completionLatch.countDown(); + } + LOGGER.warn("Stream error on request {} (open={}): {}", _requestId, wasOpen, + error == null ? "" : error.getMessage()); + if (shouldFanOutCancel) { + fanOutCancel(); + } + } + + /** + * Variant of {@link #recordStreamError} that drains exactly {@code remainingExpected} entries from the latch. Used + * by the dispatcher when it knows how many opchains a server still owed before its stream broke. + */ + public void recordStreamError(StreamingServerHandle stream, @Nullable Throwable error, int remainingExpected) { + boolean shouldFanOutCancel = false; + _lock.lock(); + try { + _openStreams.remove(stream); + if (!_peerErrorObserved) { + _peerErrorObserved = true; + shouldFanOutCancel = true; + } + } finally { + _lock.unlock(); + } + for (int i = 0; i < remainingExpected; i++) { + _completionLatch.countDown(); + } + LOGGER.warn("Stream error on request {} draining {} pending: {}", + _requestId, remainingExpected, error == null ? "" : error.getMessage()); + if (shouldFanOutCancel) { + fanOutCancel(); + } + } + + /** + * Sends {@code BrokerToServer.cancel} on every other open server stream. Called once on the first peer error + * observed. Failures are swallowed — cancel is best-effort. + */ + private void fanOutCancel() { + Set snapshot; + _lock.lock(); + try { + snapshot = new LinkedHashSet<>(_openStreams); + } finally { + _lock.unlock(); + } + for (StreamingServerHandle stream : snapshot) { + try { + stream.cancel(_requestId); + } catch (Throwable t) { + LOGGER.warn("Failed to fan out cancel on request {}", _requestId, t); + } + } + } + + /** + * Blocks the calling thread until either every expected opchain has reported, or the timeout fires. + * + *

Returns {@code true} when all opchains reported before the timeout (early completion is the common case in + * stream mode). Returns {@code false} when the timeout fired first; the per-stage coverage exposed via + * {@link #snapshotCoverage()} indicates which stages are missing or had merge failures. + * + *

The dispatcher should only call this after the broker receiving mailbox has finished. That + * way a {@code true} return means both "data done" and "stats fully accounted for". + */ + public boolean awaitCompletion(long timeout, TimeUnit unit) + throws InterruptedException { + return _completionLatch.await(timeout, unit); + } + + /** + * Returns a snapshot of the per-stage coverage. Stage ids that received any responses (successful or + * merge-failed) appear in the map; missing stages are computed by the caller against the expected total. + */ + public Coverage snapshotCoverage() { + _lock.lock(); + try { + return new Coverage(new HashMap<>(_respondedByStage), new HashMap<>(_mergeFailedByStage), + new HashMap<>(_stageAccumulator)); + } finally { + _lock.unlock(); + } + } + + /** Snapshot of the accumulator and per-stage counters at a point in time. */ + public static final class Coverage { + private final Map _respondedByStage; + private final Map _mergeFailedByStage; + private final Map _stageAccumulator; + + public Coverage(Map respondedByStage, Map mergeFailedByStage, + Map stageAccumulator) { + _respondedByStage = Collections.unmodifiableMap(respondedByStage); + _mergeFailedByStage = Collections.unmodifiableMap(mergeFailedByStage); + _stageAccumulator = Collections.unmodifiableMap(stageAccumulator); + } + + public Map getRespondedByStage() { + return _respondedByStage; + } + + public Map getMergeFailedByStage() { + return _mergeFailedByStage; + } + + public Map getStageAccumulator() { + return _stageAccumulator; + } + } + + /** Returns the number of opchains still expected to report. Visible for tests. */ + public long getOutstandingCount() { + return _completionLatch.getCount(); + } +} diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/streaming/StreamingServerHandle.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/streaming/StreamingServerHandle.java new file mode 100644 index 000000000000..b0564c8e1402 --- /dev/null +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/streaming/StreamingServerHandle.java @@ -0,0 +1,32 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.pinot.query.service.dispatch.streaming; + +/** + * Broker-side handle to a single open {@code SubmitWithStream} call. Lets the + * {@link StreamingQuerySession} send {@code BrokerToServer.cancel} fan-outs without depending on the concrete gRPC + * stub type. + */ +public interface StreamingServerHandle { + /** + * Sends a {@code BrokerToServer.cancel} message on this stream. Best-effort: the stream may already be closed or + * broken; implementations should swallow transport failures and never throw. + */ + void cancel(long requestId); +} diff --git a/pinot-query-runtime/src/test/java/org/apache/pinot/query/service/dispatch/streaming/StreamingQuerySessionTest.java b/pinot-query-runtime/src/test/java/org/apache/pinot/query/service/dispatch/streaming/StreamingQuerySessionTest.java new file mode 100644 index 000000000000..0abc9d7d056f --- /dev/null +++ b/pinot-query-runtime/src/test/java/org/apache/pinot/query/service/dispatch/streaming/StreamingQuerySessionTest.java @@ -0,0 +1,232 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.pinot.query.service.dispatch.streaming; + +import com.google.protobuf.ByteString; +import java.io.ByteArrayOutputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.pinot.common.datatable.StatMap; +import org.apache.pinot.common.proto.Worker; +import org.apache.pinot.query.runtime.operator.AggregateOperator; +import org.apache.pinot.query.runtime.operator.MultiStageOperator; +import org.testng.Assert; +import org.testng.annotations.Test; + + +/** + * Unit tests for {@link StreamingQuerySession}. Covers the broker-side accumulator, early completion (returns true + * as soon as all expected opchains have reported, without waiting for the timeout), timeout fall-through, and + * fan-out cancel behaviour. + */ +public class StreamingQuerySessionTest { + + /** + * Early completion: when all expected opchains report before the wait window, awaitCompletion returns true + * immediately rather than burning the full timeout. + */ + @Test + public void testEarlyCompletion() + throws Exception { + StreamingQuerySession session = new StreamingQuerySession(1L, 3); + session.recordOpChainComplete(buildOpChainComplete(0, 0, 1, 5)); + session.recordOpChainComplete(buildOpChainComplete(0, 1, 1, 10)); + session.recordOpChainComplete(buildOpChainComplete(0, 2, 1, 15)); + + long start = System.nanoTime(); + boolean done = session.awaitCompletion(10, TimeUnit.SECONDS); + long elapsedMs = (System.nanoTime() - start) / 1_000_000; + + Assert.assertTrue(done, "expected early completion"); + Assert.assertTrue(elapsedMs < 1000, "expected immediate return, took " + elapsedMs + "ms"); + Assert.assertEquals(session.getOutstandingCount(), 0L); + } + + /** + * Timeout fall-through: awaitCompletion returns false when the timeout fires before all opchains have reported, + * and the per-stage coverage shows the missing opchains. + */ + @Test + public void testTimeoutFallThrough() + throws Exception { + StreamingQuerySession session = new StreamingQuerySession(1L, 3); + session.recordOpChainComplete(buildOpChainComplete(0, 0, 1, 5)); + // Only 1 of 3 reports. + boolean done = session.awaitCompletion(50, TimeUnit.MILLISECONDS); + Assert.assertFalse(done, "expected timeout, not early completion"); + Assert.assertEquals(session.getOutstandingCount(), 2L); + + StreamingQuerySession.Coverage coverage = session.snapshotCoverage(); + Assert.assertEquals((int) coverage.getRespondedByStage().getOrDefault(0, 0), 1); + Assert.assertEquals((int) coverage.getMergeFailedByStage().getOrDefault(0, 0), 0); + } + + /** + * Cross-worker stats sum: two opchains for the same stage merge into one accumulator entry by tree-shape match. + */ + @Test + public void testStatsAccumulationAcrossWorkers() + throws Exception { + StreamingQuerySession session = new StreamingQuerySession(1L, 2); + session.recordOpChainComplete(buildOpChainComplete(0, 0, 1, 5)); + session.recordOpChainComplete(buildOpChainComplete(0, 1, 1, 7)); + + Assert.assertTrue(session.awaitCompletion(1, TimeUnit.SECONDS)); + StreamingQuerySession.Coverage coverage = session.snapshotCoverage(); + Assert.assertEquals((int) coverage.getRespondedByStage().get(0), 2); + @SuppressWarnings("unchecked") + StatMap merged = + (StatMap) coverage.getStageAccumulator().get(0).getStatMap(); + Assert.assertEquals(merged.getLong(AggregateOperator.StatKey.EMITTED_ROWS), 12); + } + + /** + * A failed opchain (success=false) records as merge-failed-or-responded depending on whether stats are present, + * and triggers fan-out cancel exactly once across remaining streams. + */ + @Test + public void testFanOutCancelOnPeerError() + throws Exception { + StreamingQuerySession session = new StreamingQuerySession(1L, 3); + AtomicInteger cancelCalls = new AtomicInteger(); + StreamingServerHandle a = req -> cancelCalls.incrementAndGet(); + StreamingServerHandle b = req -> cancelCalls.incrementAndGet(); + StreamingServerHandle c = req -> cancelCalls.incrementAndGet(); + session.registerStream(a); + session.registerStream(b); + session.registerStream(c); + + // Server B reports an error; should fan-out cancel to A and C (and B itself, since fan-out walks all open + // streams — they're best-effort anyway). + session.recordOpChainComplete(buildErrorOpChainComplete(0, 1, "boom")); + Assert.assertEquals(cancelCalls.get(), 3); + + // A second error does not re-fire fan-out (idempotent). + session.recordOpChainComplete(buildErrorOpChainComplete(0, 2, "boom2")); + Assert.assertEquals(cancelCalls.get(), 3); + } + + /** + * Stream onError (transport failure) drains the latch and triggers fan-out cancel; the dispatcher's + * remainingExpected variant lets it account for opchains that the dead server still owed. + */ + @Test + public void testStreamErrorDrainsLatchAndCancels() + throws Exception { + StreamingQuerySession session = new StreamingQuerySession(1L, 4); + AtomicInteger cancelCalls = new AtomicInteger(); + StreamingServerHandle dead = req -> cancelCalls.incrementAndGet(); + StreamingServerHandle other = req -> cancelCalls.incrementAndGet(); + session.registerStream(dead); + session.registerStream(other); + + // 'dead' owed 3 opchains. + session.recordStreamError(dead, new RuntimeException("transport"), 3); + // 'other' delivered its 1 opchain. + session.recordOpChainComplete(buildOpChainComplete(0, 0, 1, 1)); + + Assert.assertTrue(session.awaitCompletion(1, TimeUnit.SECONDS), "expected completion after dead drained"); + Assert.assertEquals(cancelCalls.get(), 1, "fan-out cancel should hit only 'other' (dead removed first)"); + } + + /** + * Concurrent opchain reports across many threads: latch drains correctly with no lost updates. + */ + @Test + public void testConcurrentReports() + throws Exception { + int n = 50; + StreamingQuerySession session = new StreamingQuerySession(1L, n); + CountDownLatch start = new CountDownLatch(1); + List threads = new ArrayList<>(n); + for (int i = 0; i < n; i++) { + int workerId = i; + Thread t = new Thread(() -> { + try { + start.await(); + } catch (InterruptedException ignored) { + } + try { + session.recordOpChainComplete(buildOpChainComplete(0, workerId, 1, 1)); + } catch (Exception e) { + throw new RuntimeException(e); + } + }); + t.start(); + threads.add(t); + } + start.countDown(); + Assert.assertTrue(session.awaitCompletion(5, TimeUnit.SECONDS)); + for (Thread t : threads) { + t.join(); + } + StreamingQuerySession.Coverage coverage = session.snapshotCoverage(); + Assert.assertEquals((int) coverage.getRespondedByStage().get(0), n); + @SuppressWarnings("unchecked") + StatMap merged = + (StatMap) coverage.getStageAccumulator().get(0).getStatMap(); + Assert.assertEquals(merged.getLong(AggregateOperator.StatKey.EMITTED_ROWS), n); + } + + // ---- helpers ---- + + private static Worker.OpChainComplete buildOpChainComplete(int stageId, int workerId, int planNodeId, long emitted) + throws IOException { + StatMap stat = new StatMap<>(AggregateOperator.StatKey.class) + .merge(AggregateOperator.StatKey.EMITTED_ROWS, emitted); + Worker.StageStatsNode rootNode = Worker.StageStatsNode.newBuilder() + .setOperatorTypeId(MultiStageOperator.Type.AGGREGATE.getId()) + .addPlanNodeIds(planNodeId) + .setStatMap(serialize(stat)) + .build(); + return Worker.OpChainComplete.newBuilder() + .setStageId(stageId) + .setWorkerId(workerId) + .setSuccess(true) + .setStats(Worker.MultiStageStatsTree.newBuilder() + .setCurrentStageId(stageId) + .setCurrentStage(rootNode) + .build()) + .build(); + } + + private static Worker.OpChainComplete buildErrorOpChainComplete(int stageId, int workerId, String errorMsg) { + return Worker.OpChainComplete.newBuilder() + .setStageId(stageId) + .setWorkerId(workerId) + .setSuccess(false) + .setErrorMsg(errorMsg) + .build(); + } + + private static ByteString serialize(StatMap statMap) + throws IOException { + try (ByteArrayOutputStream baos = new ByteArrayOutputStream(); + DataOutputStream output = new DataOutputStream(baos)) { + statMap.serialize(output); + output.flush(); + return ByteString.copyFrom(baos.toByteArray()); + } + } +} From 065e0e7764e46f24b9cacf7a10beb6f1c405bab3 Mon Sep 17 00:00:00 2001 From: Gonzalo Ortiz <1913993+gortiz@users.noreply.github.com> Date: Tue, 5 May 2026 15:37:44 +0100 Subject: [PATCH 09/31] [multistage] DispatchClient.submitWithStream + StreamingDispatchObserver MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Phase A.3.1 of #18375. Broker-side gRPC plumbing for opening SubmitWithStream bidi calls. StreamingDispatchObserver: - Implements StreamObserver for the inbound side and StreamingServerHandle for the outbound side (cancel fan-out). - First ServerToBroker message must be submit_ack — passed to the caller's BiConsumer. Subsequent OpChain messages forward to the session; ServerDone unregisters this stream from the session's open-set; onError drains the session's latch by the per-server remaining-expected count. DispatchClient.submitWithStream: - Opens the bidi RPC, sends the initial submit, registers the observer with the session, returns the observer for the caller to track. - Uses BiConsumer rather than the package-private AsyncResponse so the streaming subpackage doesn't depend on AsyncResponse visibility. No high-level integration into submitAndReduce yet — that's A.3.2. --- .../service/dispatch/DispatchClient.java | 33 ++++ .../streaming/StreamingDispatchObserver.java | 156 ++++++++++++++++++ 2 files changed, 189 insertions(+) create mode 100644 pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/streaming/StreamingDispatchObserver.java diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/DispatchClient.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/DispatchClient.java index 4beea8ae25b6..7d6bee8da64a 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/DispatchClient.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/DispatchClient.java @@ -35,6 +35,8 @@ import org.apache.pinot.common.proto.Worker; import org.apache.pinot.common.utils.grpc.ServerGrpcQueryClient; import org.apache.pinot.query.routing.QueryServerInstance; +import org.apache.pinot.query.service.dispatch.streaming.StreamingDispatchObserver; +import org.apache.pinot.query.service.dispatch.streaming.StreamingQuerySession; /** @@ -130,6 +132,37 @@ public void submit(Worker.QueryRequest request, QueryServerInstance virtualServe _dispatchStub.withDeadline(deadline).submit(request, new LastValueDispatchObserver<>(virtualServer, callback)); } + /** + * Opens a {@code SubmitWithStream} bidi RPC for one server, sends the initial {@code submit}, and registers the + * resulting {@link StreamingDispatchObserver} with {@code session} for cancel fan-out and {@code OpChainComplete} + * accumulation. + * + *

The submit-ack callback is invoked exactly once: with the {@link Worker.QueryResponse} on the first + * {@code submit_ack} from the server, or with a non-null {@link Throwable} if the stream errors before the ack + * arrives. + * + * @param request the plan submission + * @param virtualServer server identity (used in callbacks for routing decisions on failure) + * @param deadline gRPC deadline for the call + * @param session broker-side streaming session — the returned observer registers itself here + * @param expectedOpChainsForThisServer number of opchains this server is expected to report; used to drain the + * session latch correctly when the stream errors before all opchains have responded + * @param ackCallback receives the submit-ack response or a failure throwable + * @return the observer, also exposed as {@link org.apache.pinot.query.service.dispatch.streaming.StreamingServerHandle} + * on the session for cancel fan-out + */ + public StreamingDispatchObserver submitWithStream(Worker.QueryRequest request, QueryServerInstance virtualServer, + Deadline deadline, StreamingQuerySession session, int expectedOpChainsForThisServer, + java.util.function.BiConsumer ackCallback) { + StreamingDispatchObserver observer = new StreamingDispatchObserver(virtualServer, session, + expectedOpChainsForThisServer, ackCallback); + StreamObserver outbound = _dispatchStub.withDeadline(deadline).submitWithStream(observer); + observer.attachOutboundStream(outbound); + session.registerStream(observer); + observer.sendSubmit(request); + return observer; + } + public void cancelAsync(long requestId) { Worker.CancelRequest cancelRequest = Worker.CancelRequest.newBuilder().setRequestId(requestId).build(); _dispatchStub.cancel(cancelRequest, NO_OP_CANCEL_STREAM_OBSERVER); diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/streaming/StreamingDispatchObserver.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/streaming/StreamingDispatchObserver.java new file mode 100644 index 000000000000..8c1464bd300a --- /dev/null +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/streaming/StreamingDispatchObserver.java @@ -0,0 +1,156 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.pinot.query.service.dispatch.streaming; + +import io.grpc.stub.StreamObserver; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.BiConsumer; +import javax.annotation.Nullable; +import org.apache.pinot.common.proto.Worker; +import org.apache.pinot.query.routing.QueryServerInstance; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * Broker-side gRPC client {@code StreamObserver} for the {@code SubmitWithStream} bidi RPC. Routes inbound + * {@link Worker.ServerToBroker} messages to either the {@code submit_ack} callback (first message) or the + * {@link StreamingQuerySession} (subsequent {@code OpChainComplete}s), and adapts the inbound side of the stream + * (the {@link StreamObserver} we use to send {@code BrokerToServer} messages) into a + * {@link StreamingServerHandle} so the session can fan out cancel. + * + *

Lifecycle — created when the broker opens a {@code SubmitWithStream} call to one server, then registered with + * the session via {@link StreamingQuerySession#registerStream}. Receives: + *

    + *
  1. Exactly one {@code submit_ack} (always the first server→broker message).
  2. + *
  3. Zero or more {@code OpChainComplete} messages (one per opchain that ran on this server).
  4. + *
  5. Exactly one {@code ServerDone} after the last opchain has reported.
  6. + *
+ * + *

On {@code onError} or unexpected message order, drains the latch by {@code remainingExpected} via the session + * so {@link StreamingQuerySession#awaitCompletion} can still finalize. + */ +public class StreamingDispatchObserver + implements StreamObserver, StreamingServerHandle { + private static final Logger LOGGER = LoggerFactory.getLogger(StreamingDispatchObserver.class); + + private final QueryServerInstance _server; + private final StreamingQuerySession _session; + /// Receives the submit-ack response or a failure throwable. Called exactly once per call (either with response on + /// the first ServerToBroker.submit_ack, or with an error if the stream breaks before the ack arrives). + private final BiConsumer _ackCallback; + private final int _expectedOpChainsForThisServer; + private final AtomicBoolean _ackReceived = new AtomicBoolean(false); + + /** + * Counts how many opchains we've already drained from the session for this server, so an onError doesn't + * double-drain after some opchains already reported successfully. + */ + private int _opChainsReportedForThisServer = 0; + + /** + * The inbound side of the bidi stream — used to send {@code submit} (initial) and {@code cancel} (fan-out) from + * the broker. Set once via {@link #attachOutboundStream} after the gRPC stub is asked to start the stream; lives + * for the duration of the call. + */ + private volatile StreamObserver _outbound; + + public StreamingDispatchObserver(QueryServerInstance server, StreamingQuerySession session, + int expectedOpChainsForThisServer, BiConsumer ackCallback) { + _server = server; + _session = session; + _expectedOpChainsForThisServer = expectedOpChainsForThisServer; + _ackCallback = ackCallback; + } + + public QueryServerInstance getServer() { + return _server; + } + + /** Wires the outbound side of the bidi stream once the gRPC stub returns it. */ + public void attachOutboundStream(StreamObserver outbound) { + _outbound = outbound; + } + + /** Sends the initial {@code BrokerToServer.submit} message on the outbound side. */ + public void sendSubmit(Worker.QueryRequest request) { + StreamObserver outbound = _outbound; + if (outbound == null) { + throw new IllegalStateException("attachOutboundStream must be called before sendSubmit"); + } + outbound.onNext(Worker.BrokerToServer.newBuilder().setSubmit(request).build()); + } + + @Override + public void onNext(Worker.ServerToBroker message) { + switch (message.getPayloadCase()) { + case SUBMIT_ACK: + if (_ackReceived.compareAndSet(false, true)) { + _ackCallback.accept(message.getSubmitAck(), null); + } else { + LOGGER.warn("Ignoring duplicate submit_ack from {}", _server); + } + break; + case OPCHAIN: + _session.recordOpChainComplete(message.getOpchain()); + _opChainsReportedForThisServer++; + break; + case DONE: + _session.unregisterStream(this); + break; + case PAYLOAD_NOT_SET: + default: + LOGGER.warn("Unexpected ServerToBroker payload {} from {}", message.getPayloadCase(), _server); + break; + } + } + + @Override + public void onError(@Nullable Throwable t) { + int remaining = Math.max(0, _expectedOpChainsForThisServer - _opChainsReportedForThisServer); + _session.recordStreamError(this, t, remaining); + if (_ackReceived.compareAndSet(false, true)) { + // The submit_ack never arrived — surface the error to the ack callback so the dispatcher's submit-ack wait + // doesn't hang. + _ackCallback.accept(null, t); + } + } + + @Override + public void onCompleted() { + _session.unregisterStream(this); + } + + // --- StreamingServerHandle ------------------------------------------------- + + @Override + public void cancel(long requestId) { + StreamObserver outbound = _outbound; + if (outbound == null) { + return; + } + try { + outbound.onNext(Worker.BrokerToServer.newBuilder() + .setCancel(Worker.CancelRequest.newBuilder().setRequestId(requestId).build()) + .build()); + } catch (Throwable t) { + LOGGER.debug("Cancel send failed on already-closed stream to {}: {}", _server, t.getMessage()); + } + } +} From 6475f7464d69812d0e41482aa11d3f9828c15ca5 Mon Sep 17 00:00:00 2001 From: Gonzalo Ortiz <1913993+gortiz@users.noreply.github.com> Date: Tue, 5 May 2026 15:43:20 +0100 Subject: [PATCH 10/31] [multistage] StreamingDispatchObserver tests MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Phase A.3 follow-up. Unit-level coverage for the broker-side gRPC observer added in the previous commit: - Happy path: submit_ack → 2 OpChainComplete → ServerDone, with the ack callback firing once and the session latch draining to zero. - onError before submit_ack: ack callback receives the error so the dispatcher's ack-await doesn't hang; latch drains by remaining- expected. - Cancel emits a BrokerToServer.cancel onto the outbound stream. - onError after partial reports drains only the remaining-expected count, not the full per-server count (no double-decrement). --- .../StreamingDispatchObserverTest.java | 193 ++++++++++++++++++ 1 file changed, 193 insertions(+) create mode 100644 pinot-query-runtime/src/test/java/org/apache/pinot/query/service/dispatch/streaming/StreamingDispatchObserverTest.java diff --git a/pinot-query-runtime/src/test/java/org/apache/pinot/query/service/dispatch/streaming/StreamingDispatchObserverTest.java b/pinot-query-runtime/src/test/java/org/apache/pinot/query/service/dispatch/streaming/StreamingDispatchObserverTest.java new file mode 100644 index 000000000000..817d54e6cba2 --- /dev/null +++ b/pinot-query-runtime/src/test/java/org/apache/pinot/query/service/dispatch/streaming/StreamingDispatchObserverTest.java @@ -0,0 +1,193 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.pinot.query.service.dispatch.streaming; + +import com.google.protobuf.ByteString; +import io.grpc.stub.StreamObserver; +import java.io.ByteArrayOutputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; +import org.apache.pinot.common.datatable.StatMap; +import org.apache.pinot.common.proto.Worker; +import org.apache.pinot.query.routing.QueryServerInstance; +import org.apache.pinot.query.runtime.operator.AggregateOperator; +import org.apache.pinot.query.runtime.operator.MultiStageOperator; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; +import org.testng.Assert; +import org.testng.annotations.Test; + + +/** + * Unit tests for {@link StreamingDispatchObserver}. Drives ServerToBroker messages directly into + * {@link StreamObserver#onNext} on a real {@link StreamingQuerySession} and verifies the right session methods are + * called and the session completes when expected. + */ +public class StreamingDispatchObserverTest { + + /** + * Happy path: submit_ack, then 2 OpChainCompletes, then ServerDone. ackCallback fires once with the response, the + * session's latch drains to zero, and the stream is unregistered. + */ + @Test + public void testHappyPath() + throws Exception { + StreamingQuerySession session = new StreamingQuerySession(1L, 2); + AtomicReference ackResponse = new AtomicReference<>(); + AtomicReference ackError = new AtomicReference<>(); + + QueryServerInstance server = mockServer(); + StreamingDispatchObserver observer = new StreamingDispatchObserver(server, session, 2, + (resp, err) -> { + ackResponse.set(resp); + ackError.set(err); + }); + + StreamObserver outbound = Mockito.mock(StreamObserver.class); + observer.attachOutboundStream(outbound); + session.registerStream(observer); + + Worker.QueryResponse okResponse = Worker.QueryResponse.newBuilder().putMetadata("STATUS_OK", "").build(); + observer.onNext(Worker.ServerToBroker.newBuilder().setSubmitAck(okResponse).build()); + Assert.assertEquals(ackResponse.get(), okResponse); + Assert.assertNull(ackError.get()); + + observer.onNext(Worker.ServerToBroker.newBuilder().setOpchain(buildOpChainComplete(1, 0, 5)).build()); + observer.onNext(Worker.ServerToBroker.newBuilder().setOpchain(buildOpChainComplete(1, 1, 7)).build()); + + Assert.assertTrue(session.awaitCompletion(1, TimeUnit.SECONDS)); + + observer.onNext(Worker.ServerToBroker.newBuilder().setDone(Worker.ServerDone.getDefaultInstance()).build()); + StreamingQuerySession.Coverage coverage = session.snapshotCoverage(); + Assert.assertEquals((int) coverage.getRespondedByStage().get(1), 2); + } + + /** + * onError before submit_ack: ack callback receives the error (so the dispatcher's ack-await doesn't hang) and the + * session latch drains by the per-server remaining-expected count. + */ + @Test + public void testErrorBeforeAckSurfacesToCallback() + throws Exception { + StreamingQuerySession session = new StreamingQuerySession(1L, 4); + AtomicReference ackError = new AtomicReference<>(); + + StreamingDispatchObserver observer = new StreamingDispatchObserver(mockServer(), session, 4, + (resp, err) -> ackError.set(err)); + StreamObserver outbound = Mockito.mock(StreamObserver.class); + observer.attachOutboundStream(outbound); + session.registerStream(observer); + + RuntimeException transport = new RuntimeException("transport fault"); + observer.onError(transport); + + Assert.assertSame(ackError.get(), transport); + // Latch should be drained by remaining-expected (4 since none reported before error). + Assert.assertTrue(session.awaitCompletion(1, TimeUnit.SECONDS), + "expected session to complete after stream error drained latch"); + } + + /** + * Cancel via the StreamingServerHandle interface writes a BrokerToServer.cancel onto the outbound stream. + */ + @Test + public void testCancelEmitsBrokerToServerCancel() + throws Exception { + StreamingQuerySession session = new StreamingQuerySession(42L, 0); + StreamingDispatchObserver observer = new StreamingDispatchObserver(mockServer(), session, 0, + (resp, err) -> { }); + + @SuppressWarnings("unchecked") + StreamObserver outbound = Mockito.mock(StreamObserver.class); + observer.attachOutboundStream(outbound); + + observer.cancel(42L); + + ArgumentCaptor captor = ArgumentCaptor.forClass(Worker.BrokerToServer.class); + Mockito.verify(outbound).onNext(captor.capture()); + Worker.BrokerToServer sent = captor.getValue(); + Assert.assertTrue(sent.hasCancel()); + Assert.assertEquals(sent.getCancel().getRequestId(), 42L); + } + + /** + * Partial reports + onError: the latch is drained by exactly remaining-expected, not the full per-server count + * (so we don't double-decrement). + */ + @Test + public void testErrorAfterPartialReportsDrainsRemainingOnly() + throws Exception { + StreamingQuerySession session = new StreamingQuerySession(1L, 5); + StreamingDispatchObserver observer = new StreamingDispatchObserver(mockServer(), session, 5, + (resp, err) -> { }); + StreamObserver outbound = Mockito.mock(StreamObserver.class); + observer.attachOutboundStream(outbound); + session.registerStream(observer); + + Worker.QueryResponse okResponse = Worker.QueryResponse.newBuilder().putMetadata("STATUS_OK", "").build(); + observer.onNext(Worker.ServerToBroker.newBuilder().setSubmitAck(okResponse).build()); + observer.onNext(Worker.ServerToBroker.newBuilder().setOpchain(buildOpChainComplete(1, 0, 1)).build()); + observer.onNext(Worker.ServerToBroker.newBuilder().setOpchain(buildOpChainComplete(1, 1, 1)).build()); + // 3 of 5 still owed; stream errors. + observer.onError(new RuntimeException("died")); + + Assert.assertTrue(session.awaitCompletion(1, TimeUnit.SECONDS)); + Assert.assertEquals(session.getOutstandingCount(), 0L); + } + + // ---- helpers ---- + + private static QueryServerInstance mockServer() { + QueryServerInstance s = Mockito.mock(QueryServerInstance.class); + Mockito.when(s.toString()).thenReturn("test-server"); + return s; + } + + private static Worker.OpChainComplete buildOpChainComplete(int stageId, int workerId, long emitted) + throws IOException { + StatMap stat = new StatMap<>(AggregateOperator.StatKey.class) + .merge(AggregateOperator.StatKey.EMITTED_ROWS, emitted); + Worker.StageStatsNode rootNode = Worker.StageStatsNode.newBuilder() + .setOperatorTypeId(MultiStageOperator.Type.AGGREGATE.getId()) + .setStatMap(serialize(stat)) + .build(); + return Worker.OpChainComplete.newBuilder() + .setStageId(stageId) + .setWorkerId(workerId) + .setSuccess(true) + .setStats(Worker.MultiStageStatsTree.newBuilder() + .setCurrentStageId(stageId) + .setCurrentStage(rootNode) + .build()) + .build(); + } + + private static ByteString serialize(StatMap statMap) + throws IOException { + try (ByteArrayOutputStream baos = new ByteArrayOutputStream(); + DataOutputStream output = new DataOutputStream(baos)) { + statMap.serialize(output); + output.flush(); + return ByteString.copyFrom(baos.toByteArray()); + } + } + +} From f1692d0b7209233a63b639a3be9908b508b398bc Mon Sep 17 00:00:00 2001 From: Gonzalo Ortiz <1913993+gortiz@users.noreply.github.com> Date: Wed, 6 May 2026 10:02:33 +0100 Subject: [PATCH 11/31] [multistage] Wire SubmitWithStream into broker QueryDispatcher Phase A.3.2 of #18375. Broker-side end-to-end integration of the streaming dispatch path, gated by a per-query option. Per-query option: - New QueryOptionKey.USE_STREAM_STATS_REPORTING ("useStreamStatsReporting") - QueryOptionsUtils.isUseStreamStatsReporting reads it. QueryDispatcher.submitAndReduce branches on the option: - Legacy path unchanged. - New submitAndReduceWithStream: 1. Sizes a StreamingQuerySession with expectedOpChains = sum across non-root stages of total workers. 2. Calls submitWithStream(...) which opens one SubmitWithStream bidi RPC per server via DispatchClient.submitWithStream (computing the per-server expected opchain count for correct latch draining on stream errors). Awaits all submit_acks via the existing processResults/ack-queue pattern. 3. Runs runReducer() -- broker's stage 0 mailbox-receive -- exactly as today. 4. After mailbox EOS, awaits stats with early-completion semantics: session.awaitCompletion(remainingTimeoutMs, MILLISECONDS) returns true the moment every expected opchain has reported, false on timeout. Either way the broker proceeds to build the result. 5. Merges session stats into broker's local stage-0 stats via the new StageStatsTreeNode.flattenInorder() helper and a new QueryResult constructor that takes a pre-built per-stage list. Where both broker (pipeline-breaker) and session (server-reported) have an entry for the same stage, the session wins -- avoids double- counting stats that the upstream server also reported. Default for the option is false, so legacy behavior is preserved unless the broker explicitly opts in. --- .../utils/config/QueryOptionsUtils.java | 9 + .../runtime/plan/StageStatsTreeNode.java | 26 +++ .../service/dispatch/QueryDispatcher.java | 167 ++++++++++++++++++ .../pinot/spi/utils/CommonConstants.java | 9 + 4 files changed, 211 insertions(+) diff --git a/pinot-common/src/main/java/org/apache/pinot/common/utils/config/QueryOptionsUtils.java b/pinot-common/src/main/java/org/apache/pinot/common/utils/config/QueryOptionsUtils.java index 11c507639cf8..3510b56c97ab 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/utils/config/QueryOptionsUtils.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/utils/config/QueryOptionsUtils.java @@ -528,6 +528,15 @@ public static boolean isUsePhysicalOptimizer(Map queryOptions, b return option != null ? Boolean.parseBoolean(option) : defaultValue; } + /** + * Reads the {@code useStreamStatsReporting} query option that opts a single query into the {@code SubmitWithStream} + * dispatch path. See {@link QueryOptionKey#USE_STREAM_STATS_REPORTING}. + */ + public static boolean isUseStreamStatsReporting(Map queryOptions, boolean defaultValue) { + String option = queryOptions.get(QueryOptionKey.USE_STREAM_STATS_REPORTING); + return option != null ? Boolean.parseBoolean(option) : defaultValue; + } + public static boolean isMultiClusterRoutingEnabled(Map queryOptions, boolean defaultValue) { String option = queryOptions.get(QueryOptionKey.ENABLE_MULTI_CLUSTER_ROUTING); return option != null ? Boolean.parseBoolean(option) : defaultValue; diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/StageStatsTreeNode.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/StageStatsTreeNode.java index a2bc46f80e4e..feeef1c35bda 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/StageStatsTreeNode.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/StageStatsTreeNode.java @@ -19,6 +19,7 @@ package org.apache.pinot.query.runtime.plan; import com.google.common.collect.ImmutableList; +import java.util.ArrayList; import java.util.List; import java.util.Objects; import org.apache.pinot.common.datatable.StatMap; @@ -99,6 +100,31 @@ public void merge(StageStatsTreeNode other) } } + /** + * Flattens this tree into a {@link MultiStageQueryStats.StageStats.Closed} via inorder traversal (leftmost-leaf + * first), mirroring the order in which operators emit their stats today + * ({@link MultiStageQueryStats.StageStats.Open#addLastOperator} call order). The result is compatible with the + * legacy {@link QueryDispatcher.QueryResult} contract that exposes per-stage stats as a flat + * {@code List}. + * + *

The flattening is lossy in the same way the legacy format is: tree shape is not preserved. Callers that need + * tree shape should use this {@link StageStatsTreeNode} directly. + */ + public MultiStageQueryStats.StageStats.Closed flattenInorder() { + List types = new ArrayList<>(); + List> stats = new ArrayList<>(); + flattenInto(types, stats); + return new MultiStageQueryStats.StageStats.Closed(types, stats); + } + + private void flattenInto(List types, List> stats) { + for (StageStatsTreeNode child : _children) { + child.flattenInto(types, stats); + } + types.add(_type); + stats.add(_statMap); + } + @Override public boolean equals(Object o) { if (this == o) { diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/QueryDispatcher.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/QueryDispatcher.java index 0ee05dcfe1d9..9e71cd1643b0 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/QueryDispatcher.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/QueryDispatcher.java @@ -60,6 +60,7 @@ import org.apache.pinot.common.response.broker.ResultTable; import org.apache.pinot.common.utils.DataSchema; import org.apache.pinot.common.utils.DataSchema.ColumnDataType; +import org.apache.pinot.common.utils.config.QueryOptionsUtils; import org.apache.pinot.common.utils.grpc.ServerGrpcQueryClient; import org.apache.pinot.core.instance.context.BrokerContext; import org.apache.pinot.core.transport.ServerInstance; @@ -86,6 +87,8 @@ import org.apache.pinot.query.runtime.plan.MultiStageQueryStats; import org.apache.pinot.query.runtime.plan.OpChainConverterDispatcher; import org.apache.pinot.query.runtime.plan.OpChainExecutionContext; +import org.apache.pinot.query.runtime.plan.StageStatsTreeNode; +import org.apache.pinot.query.service.dispatch.streaming.StreamingQuerySession; import org.apache.pinot.spi.exception.QueryErrorCode; import org.apache.pinot.spi.exception.QueryException; import org.apache.pinot.spi.query.QueryExecutionContext; @@ -167,6 +170,9 @@ public void start() { public QueryResult submitAndReduce(RequestContext context, DispatchableSubPlan dispatchableSubPlan, long timeoutMs, Map queryOptions) throws Exception { + if (QueryOptionsUtils.isUseStreamStatsReporting(queryOptions, false)) { + return submitAndReduceWithStream(context, dispatchableSubPlan, timeoutMs, queryOptions); + } long requestId = context.getRequestId(); Set servers = new HashSet<>(); try { @@ -189,6 +195,154 @@ public QueryResult submitAndReduce(RequestContext context, DispatchableSubPlan d } } + /// Streaming variant of {@link #submitAndReduce}: opens one {@code SubmitWithStream} bidi RPC per server, runs the + /// broker's stage 0 reducer, and once the receiving mailbox finishes awaits the per-stage stats with early + /// completion (returns as soon as every expected opchain has reported, or when the wait window fires — whichever + /// happens first). Stats from the session accumulator are then merged into the broker's local stage 0 stats to + /// build the final {@link QueryResult}. + /// + /// The wait window is bounded by the query's remaining timeout: if {@code submitWithStream + runReducer} consumed + /// most of the budget, the per-stage stats may end up partial (visible via the per-stage {@code mergeFailed} / + /// {@code missing} counts the session exposes). + private QueryResult submitAndReduceWithStream(RequestContext context, DispatchableSubPlan dispatchableSubPlan, + long timeoutMs, Map queryOptions) + throws Exception { + long requestId = context.getRequestId(); + long deadlineMs = System.currentTimeMillis() + timeoutMs; + Set servers = new HashSet<>(); + + // The session's expected-opchain count must equal the total number of opchains across every (server, non-root + // stage) pair — that's how many OpChainComplete messages we expect to receive. + Set stagePlansWithoutRoot = dispatchableSubPlan.getQueryStagesWithoutRoot(); + int totalExpected = 0; + for (DispatchablePlanFragment stagePlan : stagePlansWithoutRoot) { + for (List workerIds : stagePlan.getServerInstanceToWorkerIdMap().values()) { + totalExpected += workerIds.size(); + } + } + StreamingQuerySession session = new StreamingQuerySession(requestId, totalExpected); + + try { + submitWithStream(requestId, dispatchableSubPlan, timeoutMs, servers, queryOptions, session); + QueryResult brokerResult = runReducer(dispatchableSubPlan, queryOptions, _mailboxService); + + // Receiving mailbox finished. Wait for stats: returns true as soon as every opchain has reported, or false + // when the timeout fires. + long remainingMs = Math.max(0, deadlineMs - System.currentTimeMillis()); + boolean fullCoverage = session.awaitCompletion(remainingMs, TimeUnit.MILLISECONDS); + if (!fullCoverage) { + LOGGER.warn("Stream-mode request {} timed out waiting for stats after mailbox EOS; coverage may be partial", + requestId); + } + + if (brokerResult.getProcessingException() != null) { + cancel(requestId); + } + return mergeSessionStatsIntoResult(brokerResult, session); + } catch (Exception ex) { + return tryRecover(requestId, servers, ex); + } catch (Throwable e) { + cancel(requestId); + throw e; + } finally { + if (isQueryCancellationEnabled()) { + _serversByQuery.remove(requestId); + } + } + } + + /// Streaming variant of {@link #submit}: opens one {@code SubmitWithStream} bidi RPC per server, registers each + /// open stream with {@code session} (so cancel fan-out and {@code OpChainComplete} accumulation work), and waits + /// for every server's submit-ack before returning. Errors during ack-await trigger {@link #cancel} on all peers. + @VisibleForTesting + void submitWithStream(long requestId, DispatchableSubPlan dispatchableSubPlan, long timeoutMs, + Set serversOut, Map queryOptions, StreamingQuerySession session) + throws Exception { + Deadline deadline = Deadline.after(timeoutMs, TimeUnit.MILLISECONDS); + + Set plansWithoutRoot = dispatchableSubPlan.getQueryStagesWithoutRoot(); + Map stageInfos = serializePlanFragments(plansWithoutRoot, serversOut); + if (serversOut.isEmpty()) { + return; + } + + Map requestMetadata = + prepareRequestMetadata(QueryThreadContext.get().getExecutionContext(), queryOptions, deadline); + ByteString protoRequestMetadata = QueryPlanSerDeUtils.toProtoProperties(requestMetadata); + + // Per-server expected opchain count = sum across the server's non-root stages of (workers on this server in + // that stage). The streaming observer uses this to drain the session latch correctly when its stream errors + // before all opchains have responded. + BlockingQueue> ackQueue = new ArrayBlockingQueue<>(serversOut.size()); + for (QueryServerInstance server : serversOut) { + Worker.QueryRequest request = createRequest(server, stageInfos, protoRequestMetadata); + int expectedForServer = 0; + for (DispatchablePlanFragment stagePlan : plansWithoutRoot) { + List workerIds = stagePlan.getServerInstanceToWorkerIdMap().get(server); + if (workerIds != null) { + expectedForServer += workerIds.size(); + } + } + DispatchClient client = getOrCreateDispatchClient(server); + try { + client.submitWithStream(request, server, deadline, session, expectedForServer, + (resp, err) -> ackQueue.offer(new AsyncResponse<>(server, resp, err))); + } catch (Throwable t) { + LOGGER.warn("Caught exception while opening stream to server: {}", server, t); + ackQueue.offer(new AsyncResponse<>(server, null, t)); + _failureDetector.markServerUnhealthy(server.getInstanceId(), server.getHostname()); + } + } + + processResults(requestId, serversOut.size(), (response, server) -> { + if (response.containsMetadata(ServerResponseStatus.STATUS_ERROR)) { + cancel(requestId, serversOut); + throw new RuntimeException( + String.format("Unable to execute query plan for request: %d on server: %s, ERROR: %s", requestId, server, + response.getMetadataOrDefault(ServerResponseStatus.STATUS_ERROR, "null"))); + } + }, deadline, ackQueue); + + if (isQueryCancellationEnabled()) { + _serversByQuery.put(requestId, serversOut); + } + } + + /// Builds the final {@link QueryResult} for a stream-mode query: takes the broker's local stage-0 stats from + /// {@code brokerResult} and overlays the per-stage trees from the session accumulator (flattened to + /// {@link MultiStageQueryStats.StageStats.Closed} via inorder traversal so the resulting list shape matches the + /// legacy {@link QueryResult#_queryStats} contract). + /// + /// In stream mode the broker's local mailbox path is suppressed for stages 1..N, so brokerResult's _queryStats + /// list typically only contains stage 0 plus any pipeline-breaker stages. The session's accumulator carries + /// stages 1..N. Where both have an entry for the same stage id, the session wins (avoids double-counting + /// pipeline-breaker stats that the upstream server also reported). + private QueryResult mergeSessionStatsIntoResult(QueryResult brokerResult, StreamingQuerySession session) { + StreamingQuerySession.Coverage coverage = session.snapshotCoverage(); + Map accumulator = coverage.getStageAccumulator(); + + int maxStageId = brokerResult.getQueryStats().size() - 1; + for (Integer stageId : accumulator.keySet()) { + if (stageId > maxStageId) { + maxStageId = stageId; + } + } + + List merged = new ArrayList<>(maxStageId + 1); + for (int i = 0; i <= maxStageId; i++) { + StageStatsTreeNode sessionTree = accumulator.get(i); + if (sessionTree != null) { + merged.add(sessionTree.flattenInorder()); + } else if (i < brokerResult.getQueryStats().size()) { + merged.add(brokerResult.getQueryStats().get(i)); + } else { + merged.add(null); + } + } + return new QueryResult(brokerResult.getResultTable(), brokerResult.getProcessingException(), merged, + brokerResult.getBrokerReduceTimeMs()); + } + /// Tries to recover from an exception thrown during query dispatching. /// /// [QueryException] and [TimeoutException] are handled by returning a [QueryResult] with the error code and stats, @@ -757,6 +911,19 @@ public QueryResult(QueryProcessingException processingException, MultiStageQuery } } + /** + * Creates a query result from a pre-built per-stage stats list. Used by the {@code SubmitWithStream} path so + * the caller can merge the broker's local stage-0 stats with the session accumulator's stages 1..N before + * constructing the result. + */ + public QueryResult(@Nullable ResultTable resultTable, @Nullable QueryProcessingException processingException, + List queryStats, long brokerReduceTimeMs) { + _resultTable = resultTable; + _processingException = processingException; + _queryStats = queryStats; + _brokerReduceTimeMs = brokerReduceTimeMs; + } + @Nullable public ResultTable getResultTable() { return _resultTable; diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/CommonConstants.java b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/CommonConstants.java index 086e928fa569..7f38b60431b6 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/CommonConstants.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/CommonConstants.java @@ -730,6 +730,15 @@ public static class QueryOptionKey { public static final String APPLICATION_NAME = "applicationName"; public static final String USE_SPOOLS = "useSpools"; public static final String USE_PHYSICAL_OPTIMIZER = "usePhysicalOptimizer"; + /** + * When set to true, the broker uses the long-lived {@code SubmitWithStream} bidi RPC to dispatch the query, + * receiving stage stats out-of-band as {@code OpChainComplete} messages instead of via mailbox EOS. The + * broker awaits stats completion as soon as the receiving mailbox finishes, with a configurable wait window + * for any outstanding opchains. + * + *

When unset / false, the legacy unary {@code Submit} path is used and stats travel via mailbox EOS. + */ + public static final String USE_STREAM_STATS_REPORTING = "useStreamStatsReporting"; /** * If set, changes the explain behavior in multi-stage engine. * From 87f00a8e9809b19f2db84a90ffa4167dd6bdd90c Mon Sep 17 00:00:00 2001 From: Gonzalo Ortiz <1913993+gortiz@users.noreply.github.com> Date: Wed, 6 May 2026 10:18:32 +0100 Subject: [PATCH 12/31] [multistage] Document mixed-version policy for stream-mode dispatch MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Phase A.3 follow-up. Make the no-fallback design choice explicit in both the per-query option Javadoc and on submitAndReduceWithStream: - Enabling useStreamStatsReporting requires every server in the cluster to support SubmitWithStream. Operators are responsible for setting it only after the fleet is fully upgraded. - If any server returns UNIMPLEMENTED (or any other transport error) during dispatch, the broker cancels the query and surfaces the error to the client. No code change — the existing path already does this: the failing ack lands on the queue with a non-null throwable, processResults throws, tryRecover catches and calls cancel(requestId, servers) before rethrowing. Just clarifying that this is the intended behavior rather than a fallback to add later. --- .../pinot/query/service/dispatch/QueryDispatcher.java | 8 ++++++++ .../org/apache/pinot/spi/utils/CommonConstants.java | 10 ++++++++-- 2 files changed, 16 insertions(+), 2 deletions(-) diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/QueryDispatcher.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/QueryDispatcher.java index 9e71cd1643b0..56968ee9c775 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/QueryDispatcher.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/QueryDispatcher.java @@ -204,6 +204,14 @@ public QueryResult submitAndReduce(RequestContext context, DispatchableSubPlan d /// The wait window is bounded by the query's remaining timeout: if {@code submitWithStream + runReducer} consumed /// most of the budget, the per-stage stats may end up partial (visible via the per-stage {@code mergeFailed} / /// {@code missing} counts the session exposes). + /// + /// Mixed-version policy. No automatic fallback to the unary {@link #submit} path. Enabling + /// {@link CommonConstants.Broker.Request.QueryOptionKey#USE_STREAM_STATS_REPORTING} requires every server in the + /// cluster to implement {@code SubmitWithStream}; if any server returns {@code UNIMPLEMENTED} or any other + /// transport error during dispatch, {@link #submitWithStream} surfaces the throwable through the ack queue, + /// {@link #processResults} throws, this method's {@code catch} cancels every other server via {@link #tryRecover} + /// and propagates the failure to the caller. Mixed-version safety is the operator's responsibility — only enable + /// this option once the whole fleet has been upgraded. private QueryResult submitAndReduceWithStream(RequestContext context, DispatchableSubPlan dispatchableSubPlan, long timeoutMs, Map queryOptions) throws Exception { diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/CommonConstants.java b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/CommonConstants.java index 7f38b60431b6..8749c45dcf01 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/CommonConstants.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/CommonConstants.java @@ -733,10 +733,16 @@ public static class QueryOptionKey { /** * When set to true, the broker uses the long-lived {@code SubmitWithStream} bidi RPC to dispatch the query, * receiving stage stats out-of-band as {@code OpChainComplete} messages instead of via mailbox EOS. The - * broker awaits stats completion as soon as the receiving mailbox finishes, with a configurable wait window - * for any outstanding opchains. + * broker awaits stats completion as soon as the receiving mailbox finishes (early completion), bounded by + * the query's remaining timeout. * *

When unset / false, the legacy unary {@code Submit} path is used and stats travel via mailbox EOS. + * + *

Mixed-version note. All servers in the cluster must support {@code SubmitWithStream} when this + * option is enabled. Operators are responsible for setting it only after the entire fleet has been upgraded + * — there is no automatic fallback to the unary path. If any server returns {@code UNIMPLEMENTED} (or any + * other transport error) during dispatch, the broker cancels the query and surfaces the error to the + * client. */ public static final String USE_STREAM_STATS_REPORTING = "useStreamStatsReporting"; /** From a26def64ebd142a34adfe5f99be57ea349142cf1 Mon Sep 17 00:00:00 2001 From: Gonzalo Ortiz <1913993+gortiz@users.noreply.github.com> Date: Wed, 6 May 2026 15:26:29 +0100 Subject: [PATCH 13/31] [multistage] Add cluster-level config for stream-mode stats reporting Adds pinot.broker.multistage.use.stream.stats.reporting (default false) so operators can enable stream-mode stats for all queries cluster-wide without requiring per-query opt-in. Individual queries can still override via the useStreamStatsReporting query option. Co-Authored-By: Claude Opus 4.7 (1M context) --- .../MultiStageBrokerRequestHandler.java | 15 ++++++++++++++- .../apache/pinot/spi/utils/CommonConstants.java | 9 +++++++++ 2 files changed, 23 insertions(+), 1 deletion(-) diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/MultiStageBrokerRequestHandler.java b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/MultiStageBrokerRequestHandler.java index 9ccfad0a6121..61a5b02106b3 100644 --- a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/MultiStageBrokerRequestHandler.java +++ b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/MultiStageBrokerRequestHandler.java @@ -25,6 +25,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; +import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Locale; @@ -138,6 +139,7 @@ public class MultiStageBrokerRequestHandler extends BaseBrokerRequestHandler { private final Set _defaultDisabledPlannerRules; protected final long _extraPassiveTimeoutMs; protected final boolean _enableQueryFingerprinting; + private final boolean _useStreamStatsReporting; protected final PinotMeter _stagesStartedMeter = BrokerMeter.MSE_STAGES_STARTED.getGlobalMeter(); protected final PinotMeter _stagesFinishedMeter = BrokerMeter.MSE_STAGES_COMPLETED.getGlobalMeter(); @@ -203,6 +205,9 @@ tlsConfig, isQueryCancellationEnabled(), cancelTimeout, dispatchKeepAliveTimeMs, _enableQueryFingerprinting = _config.getProperty( CommonConstants.Broker.CONFIG_OF_BROKER_ENABLE_QUERY_FINGERPRINTING, CommonConstants.Broker.DEFAULT_BROKER_ENABLE_QUERY_FINGERPRINTING); + _useStreamStatsReporting = _config.getProperty( + CommonConstants.Broker.CONFIG_OF_USE_STREAM_STATS_REPORTING, + CommonConstants.Broker.DEFAULT_USE_STREAM_STATS_REPORTING); } @Override @@ -650,10 +655,18 @@ private BrokerResponse query(QueryEnvironment.CompiledQuery query, long requestI _stagesStartedMeter.mark(stageCount); _opchainsStartedMeter.mark(opChainCount); + // Inject the cluster-default stream-stats mode unless the query already overrides it. + Map effectiveOptions = query.getOptions(); + if (_useStreamStatsReporting && !effectiveOptions.containsKey( + CommonConstants.Broker.Request.QueryOptionKey.USE_STREAM_STATS_REPORTING)) { + effectiveOptions = new HashMap<>(effectiveOptions); + effectiveOptions.put(CommonConstants.Broker.Request.QueryOptionKey.USE_STREAM_STATS_REPORTING, "true"); + } + QueryDispatcher.QueryResult queryResults; try { queryResults = _queryDispatcher.submitAndReduce(requestContext, dispatchableSubPlan, timer.getRemainingTimeMs(), - query.getOptions()); + effectiveOptions); } catch (QueryException e) { throw e; } catch (Throwable t) { diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/CommonConstants.java b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/CommonConstants.java index 8749c45dcf01..69115645d783 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/CommonConstants.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/CommonConstants.java @@ -525,6 +525,15 @@ public static class Broker { "pinot.broker.mse.use.leaf.server.for.intermediate.stage"; public static final boolean DEFAULT_USE_LEAF_SERVER_FOR_INTERMEDIATE_STAGE = false; + /// Cluster-level default for stream-mode stats reporting. When {@code true} the broker opens a + /// {@code SubmitWithStream} bidi RPC for every multi-stage query instead of the legacy unary Submit, enabling + /// reliable per-operator stats delivery even on the error path. Individual queries may override this default + /// via the {@link Request.QueryOptionKey#USE_STREAM_STATS_REPORTING} query option. Requires all servers to + /// implement the {@code SubmitWithStream} RPC; enabling it on a mixed-version cluster will cause query failures. + public static final String CONFIG_OF_USE_STREAM_STATS_REPORTING = + "pinot.broker.multistage.use.stream.stats.reporting"; + public static final boolean DEFAULT_USE_STREAM_STATS_REPORTING = false; + public static final String CONFIG_OF_USE_FIXED_REPLICA = "pinot.broker.use.fixed.replica"; public static final boolean DEFAULT_USE_FIXED_REPLICA = false; From 0284485080017d4a46b54f903ba0a700647b48a0 Mon Sep 17 00:00:00 2001 From: Gonzalo Ortiz <1913993+gortiz@users.noreply.github.com> Date: Wed, 6 May 2026 15:30:37 +0100 Subject: [PATCH 14/31] [multistage] Expose per-stage stats coverage on QueryResult and broker response Stream-mode queries now record how many opchain reports were received, how many failed to merge (shape mismatch / decode error), and how many went missing (timed out or stream error). The counts are carried by a new QueryResult.StageCoverage list and rendered as a streamStatsCoverage JSON array in BrokerResponseNativeV2, indexed by stage id, so consumers can see exactly which stages delivered partial stats. Co-Authored-By: Claude Opus 4.7 (1M context) --- .../MultiStageBrokerRequestHandler.java | 21 ++++- .../broker/BrokerResponseNativeV2.java | 23 +++++- .../service/dispatch/QueryDispatcher.java | 81 +++++++++++++++++-- 3 files changed, 117 insertions(+), 8 deletions(-) diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/MultiStageBrokerRequestHandler.java b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/MultiStageBrokerRequestHandler.java index 61a5b02106b3..617b61e40e71 100644 --- a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/MultiStageBrokerRequestHandler.java +++ b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/MultiStageBrokerRequestHandler.java @@ -19,6 +19,7 @@ package org.apache.pinot.broker.requesthandler; import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.node.ArrayNode; import com.fasterxml.jackson.databind.node.JsonNodeFactory; import com.fasterxml.jackson.databind.node.ObjectNode; import java.time.Duration; @@ -736,7 +737,8 @@ private BrokerResponse query(QueryEnvironment.CompiledQuery query, long requestI } } - fillOldBrokerResponseStats(brokerResponse, queryResults.getQueryStats(), dispatchableSubPlan); + fillOldBrokerResponseStats(brokerResponse, queryResults.getQueryStats(), dispatchableSubPlan, + queryResults.getStageCoverage()); long totalTimeMs = System.currentTimeMillis() - requestContext.getRequestArrivalTimeMillis(); _brokerMetrics.addTimedValue(BrokerTimer.MULTI_STAGE_QUERY_TOTAL_TIME_MS, totalTimeMs, TimeUnit.MILLISECONDS); @@ -826,7 +828,8 @@ private Collection requestPhysicalPlan(DispatchablePlanFragment fragme } private void fillOldBrokerResponseStats(BrokerResponseNativeV2 brokerResponse, - List queryStats, DispatchableSubPlan dispatchableSubPlan) { + List queryStats, DispatchableSubPlan dispatchableSubPlan, + @Nullable List stageCoverage) { try { Map queryStageMap = dispatchableSubPlan.getQueryStageMap(); @@ -842,6 +845,20 @@ private void fillOldBrokerResponseStats(BrokerResponseNativeV2 brokerResponse, brokerResponse.setStageStats(JsonNodeFactory.instance.objectNode() .put("error", "Error encountered while collecting multi-stage stats - " + e)); } + if (stageCoverage != null) { + ArrayNode coverageArray = JsonNodeFactory.instance.arrayNode(); + for (QueryDispatcher.QueryResult.StageCoverage sc : stageCoverage) { + if (sc == null) { + coverageArray.addNull(); + } else { + coverageArray.addObject() + .put("responded", sc.getResponded()) + .put("mergeFailed", sc.getMergeFailed()) + .put("missing", sc.getMissing()); + } + } + brokerResponse.setStreamStatsCoverage(coverageArray); + } } private BrokerResponse constructMultistageExplainPlan(String sql, String plan, Map extraFields) { diff --git a/pinot-common/src/main/java/org/apache/pinot/common/response/broker/BrokerResponseNativeV2.java b/pinot-common/src/main/java/org/apache/pinot/common/response/broker/BrokerResponseNativeV2.java index e485912feea2..c25433a1d454 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/response/broker/BrokerResponseNativeV2.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/response/broker/BrokerResponseNativeV2.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonPropertyOrder; +import com.fasterxml.jackson.databind.node.ArrayNode; import com.fasterxml.jackson.databind.node.ObjectNode; import java.util.ArrayList; import java.util.List; @@ -39,7 +40,7 @@ @JsonPropertyOrder({ "resultTable", "numRowsResultSet", "partialResult", "exceptions", "numGroupsLimitReached", "numGroupsWarningLimitReached", "numGroups", "maxRowsInJoinReached", "maxRowsInJoin", - "maxRowsInWindowReached", "maxRowsInWindow", "timeUsedMs", "stageStats", + "maxRowsInWindowReached", "maxRowsInWindow", "timeUsedMs", "stageStats", "streamStatsCoverage", "maxRowsInOperator", "requestId", "clientRequestId", "brokerId", "numDocsScanned", "totalDocs", "numEntriesScannedInFilter", "numEntriesScannedPostFilter", "numServersQueried", "numServersResponded", "numSegmentsQueried", "numSegmentsProcessed", "numSegmentsMatched", "numConsumingSegmentsQueried", @@ -68,6 +69,13 @@ public class BrokerResponseNativeV2 implements BrokerResponse { * Statistics for each stage of the query execution. */ private ObjectNode _stageStats; + /** + * Stream-mode stats coverage, populated only when the query used {@code SubmitWithStream}. An array indexed by stage + * id; each element is an object with {@code responded}, {@code mergeFailed}, and {@code missing} counters, or + * {@code null} for stages that have no coverage info (e.g. stage 0 which runs broker-local). + */ + @JsonInclude(JsonInclude.Include.NON_NULL) + private ArrayNode _streamStatsCoverage; /** * The max number of rows seen at runtime. *

@@ -208,6 +216,19 @@ public void setStageStats(ObjectNode stageStats) { _stageStats = stageStats; } + /** + * Returns the stream-mode stats coverage, or {@code null} when the query ran in legacy mode. Array indexed by stage + * id; elements may be {@code null} for stages with no coverage (e.g. stage 0). + */ + @Nullable + public ArrayNode getStreamStatsCoverage() { + return _streamStatsCoverage; + } + + public void setStreamStatsCoverage(ArrayNode streamStatsCoverage) { + _streamStatsCoverage = streamStatsCoverage; + } + /** * Returns the maximum number of rows seen by a single operator in the query processing chain. */ diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/QueryDispatcher.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/QueryDispatcher.java index 56968ee9c775..07f10ee25bf4 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/QueryDispatcher.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/QueryDispatcher.java @@ -223,10 +223,15 @@ private QueryResult submitAndReduceWithStream(RequestContext context, Dispatchab // stage) pair — that's how many OpChainComplete messages we expect to receive. Set stagePlansWithoutRoot = dispatchableSubPlan.getQueryStagesWithoutRoot(); int totalExpected = 0; + Map expectedByStage = new HashMap<>(); for (DispatchablePlanFragment stagePlan : stagePlansWithoutRoot) { + int stageId = stagePlan.getPlanFragment().getFragmentId(); + int stageCount = 0; for (List workerIds : stagePlan.getServerInstanceToWorkerIdMap().values()) { - totalExpected += workerIds.size(); + stageCount += workerIds.size(); } + totalExpected += stageCount; + expectedByStage.put(stageId, stageCount); } StreamingQuerySession session = new StreamingQuerySession(requestId, totalExpected); @@ -246,7 +251,7 @@ private QueryResult submitAndReduceWithStream(RequestContext context, Dispatchab if (brokerResult.getProcessingException() != null) { cancel(requestId); } - return mergeSessionStatsIntoResult(brokerResult, session); + return mergeSessionStatsIntoResult(brokerResult, session, expectedByStage); } catch (Exception ex) { return tryRecover(requestId, servers, ex); } catch (Throwable e) { @@ -325,7 +330,11 @@ void submitWithStream(long requestId, DispatchableSubPlan dispatchableSubPlan, l /// list typically only contains stage 0 plus any pipeline-breaker stages. The session's accumulator carries /// stages 1..N. Where both have an entry for the same stage id, the session wins (avoids double-counting /// pipeline-breaker stats that the upstream server also reported). - private QueryResult mergeSessionStatsIntoResult(QueryResult brokerResult, StreamingQuerySession session) { + /// + /// @param expectedByStage map from stage id to the number of opchain reports expected for that stage (used to + /// compute the {@link QueryResult.StageCoverage#getMissing()} count per stage) + private QueryResult mergeSessionStatsIntoResult(QueryResult brokerResult, StreamingQuerySession session, + Map expectedByStage) { StreamingQuerySession.Coverage coverage = session.snapshotCoverage(); Map accumulator = coverage.getStageAccumulator(); @@ -335,8 +344,14 @@ private QueryResult mergeSessionStatsIntoResult(QueryResult brokerResult, Stream maxStageId = stageId; } } + for (Integer stageId : expectedByStage.keySet()) { + if (stageId > maxStageId) { + maxStageId = stageId; + } + } List merged = new ArrayList<>(maxStageId + 1); + List stageCoverage = new ArrayList<>(maxStageId + 1); for (int i = 0; i <= maxStageId; i++) { StageStatsTreeNode sessionTree = accumulator.get(i); if (sessionTree != null) { @@ -346,9 +361,15 @@ private QueryResult mergeSessionStatsIntoResult(QueryResult brokerResult, Stream } else { merged.add(null); } + int responded = coverage.getRespondedByStage().getOrDefault(i, 0); + int mergeFailed = coverage.getMergeFailedByStage().getOrDefault(i, 0); + int expected = expectedByStage.getOrDefault(i, 0); + int missing = Math.max(0, expected - responded - mergeFailed); + // Stage 0 is broker-local and not tracked by the session; leave its entry null. + stageCoverage.add(expected == 0 ? null : new QueryResult.StageCoverage(responded, mergeFailed, missing)); } return new QueryResult(brokerResult.getResultTable(), brokerResult.getProcessingException(), merged, - brokerResult.getBrokerReduceTimeMs()); + brokerResult.getBrokerReduceTimeMs(), stageCoverage); } /// Tries to recover from an exception thrown during query dispatching. @@ -881,6 +902,12 @@ public static class QueryResult { private final QueryProcessingException _processingException; private final List _queryStats; private final long _brokerReduceTimeMs; + /** + * Non-null only in stream-mode queries. Indexed by stage id; entries may be null for stages with no coverage data + * (e.g. stage 0 which runs broker-local and is not tracked by the session). + */ + @Nullable + private final List _stageCoverage; /** * Creates a successful query result. @@ -897,6 +924,7 @@ public QueryResult(ResultTable resultTable, MultiStageQueryStats queryStats, lon } _brokerReduceTimeMs = brokerReduceTimeMs; _processingException = null; + _stageCoverage = null; } /** @@ -917,6 +945,7 @@ public QueryResult(QueryProcessingException processingException, MultiStageQuery for (int i = 1; i < numStages; i++) { _queryStats.add(queryStats.getUpstreamStageStats(i)); } + _stageCoverage = null; } /** @@ -925,11 +954,13 @@ public QueryResult(QueryProcessingException processingException, MultiStageQuery * constructing the result. */ public QueryResult(@Nullable ResultTable resultTable, @Nullable QueryProcessingException processingException, - List queryStats, long brokerReduceTimeMs) { + List queryStats, long brokerReduceTimeMs, + @Nullable List stageCoverage) { _resultTable = resultTable; _processingException = processingException; _queryStats = queryStats; _brokerReduceTimeMs = brokerReduceTimeMs; + _stageCoverage = stageCoverage; } @Nullable @@ -949,6 +980,46 @@ public List getQueryStats() { public long getBrokerReduceTimeMs() { return _brokerReduceTimeMs; } + + /** + * Returns per-stage coverage data from the stream-mode session, or {@code null} when the query ran in legacy mode. + * The list is indexed by stage id; entries may be {@code null} for stages with no coverage info (e.g. stage 0). + */ + @Nullable + public List getStageCoverage() { + return _stageCoverage; + } + + /** + * Per-stage stats coverage for a stream-mode query. Captures how many opchain reports the broker received vs. + * expected, and how many it couldn't merge (version-skew or shape mismatch). + */ + public static final class StageCoverage { + private final int _responded; + private final int _mergeFailed; + private final int _missing; + + public StageCoverage(int responded, int mergeFailed, int missing) { + _responded = responded; + _mergeFailed = mergeFailed; + _missing = missing; + } + + /** Opchains that reported and whose stats were merged successfully. */ + public int getResponded() { + return _responded; + } + + /** Opchains that reported but whose stats the broker could not merge (shape mismatch / decode error). */ + public int getMergeFailed() { + return _mergeFailed; + } + + /** Opchains that were expected but never reported (timed out or stream error before reporting). */ + public int getMissing() { + return _missing; + } + } } private interface SendRequest { From 3a092bee52e28c4de8c31c8ba74dbcd8721ba57a Mon Sep 17 00:00:00 2001 From: Gonzalo Ortiz <1913993+gortiz@users.noreply.github.com> Date: Wed, 6 May 2026 15:48:33 +0100 Subject: [PATCH 15/31] [multistage] Add StreamStatsReportingIntegrationTest + fix config namespace Integration test exercises the SubmitWithStream stats path end-to-end: - Simple aggregation, join (multi-stage), and three-way UNION (N-ary set op) all run with useStreamStatsReporting=true and verify results are correct and streamStatsCoverage shows full coverage (responded>0, missing=0, mergeFailed=0). - Cluster-level config test uses overrideBrokerConf to enable stream stats for all queries without a per-query option. Also fixes the config key namespace: rename from pinot.broker.multistage.use.stream.stats.reporting to pinot.broker.mse.use.stream.stats.reporting, consistent with all other MSE broker config keys in CommonConstants. Co-Authored-By: Claude Opus 4.7 (1M context) --- .../StreamStatsReportingIntegrationTest.java | 231 ++++++++++++++++++ .../pinot/spi/utils/CommonConstants.java | 2 +- 2 files changed, 232 insertions(+), 1 deletion(-) create mode 100644 pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/StreamStatsReportingIntegrationTest.java diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/StreamStatsReportingIntegrationTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/StreamStatsReportingIntegrationTest.java new file mode 100644 index 000000000000..71f605122853 --- /dev/null +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/StreamStatsReportingIntegrationTest.java @@ -0,0 +1,231 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.pinot.integration.tests; + +import com.fasterxml.jackson.databind.JsonNode; +import java.io.File; +import java.util.List; +import org.apache.commons.io.FileUtils; +import org.apache.pinot.spi.config.table.TableConfig; +import org.apache.pinot.spi.data.Schema; +import org.apache.pinot.spi.env.PinotConfiguration; +import org.apache.pinot.spi.utils.CommonConstants; +import org.apache.pinot.util.TestUtils; +import org.intellij.lang.annotations.Language; +import org.testng.annotations.AfterClass; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertTrue; + + +/** + * Integration tests for the {@code SubmitWithStream} stats-reporting path. Verifies that: + *

    + *
  1. Queries run to completion with correct results when stream stats are enabled per-query via the + * {@code useStreamStatsReporting} query option.
  2. + *
  3. The broker response contains a non-null {@code streamStatsCoverage} array indexed by stage id with + * {@code responded} > 0 and {@code missing} = {@code mergeFailed} = 0 on the success path.
  4. + *
  5. An N-ary set operation (three-way UNION) produces a correct tree-shaped stats payload — regression coverage + * for the known loss in the legacy flat-binary / inorder format when a set op has more than two inputs.
  6. + *
  7. The cluster-level config ({@code pinot.broker.multistage.use.stream.stats.reporting}) activates stream mode + * for all queries without a per-query option.
  8. + *
+ */ +public class StreamStatsReportingIntegrationTest extends BaseClusterIntegrationTestSet { + + private static final String STREAM_OPTION = "useStreamStatsReporting=true"; + + @BeforeClass + public void setUp() + throws Exception { + TestUtils.ensureDirectoriesExistAndEmpty(_tempDir, _segmentDir, _tarDir); + + startZk(); + startController(); + startBroker(); + startServer(); + + Schema schema = createSchema(); + addSchema(schema); + TableConfig tableConfig = createOfflineTableConfig(); + addTableConfig(tableConfig); + + List avroFiles = unpackAvroData(_tempDir); + ClusterIntegrationTestUtils.buildSegmentsFromAvro(avroFiles, tableConfig, schema, 0, _segmentDir, _tarDir); + uploadSegments(getTableName(), _tarDir); + + waitForAllDocsLoaded(600_000L); + } + + @AfterClass + public void tearDown() + throws Exception { + stopServer(); + stopBroker(); + stopController(); + stopZk(); + FileUtils.deleteDirectory(_tempDir); + } + + @Override + public boolean useMultiStageQueryEngine() { + return true; + } + + // ─── helpers ────────────────────────────────────────────────────────────────── + + /** + * Posts a multi-stage query with the {@code useStreamStatsReporting=true} option. + */ + private JsonNode postWithStreamStats(@Language("sql") String sql) + throws Exception { + return postQueryWithOptions(sql, STREAM_OPTION); + } + + /** + * Asserts that the {@code streamStatsCoverage} array in the response is well-formed: every non-null entry must have + * responded > 0, mergeFailed = 0, and missing = 0. + */ + private static void assertFullCoverage(JsonNode response) { + JsonNode coverage = response.get("streamStatsCoverage"); + assertNotNull(coverage, "streamStatsCoverage should be present in stream-mode response"); + assertTrue(coverage.isArray(), "streamStatsCoverage must be an array"); + boolean anyNonNull = false; + for (JsonNode entry : coverage) { + if (entry == null || entry.isNull()) { + continue; + } + anyNonNull = true; + int responded = entry.get("responded").asInt(); + int mergeFailed = entry.get("mergeFailed").asInt(); + int missing = entry.get("missing").asInt(); + assertTrue(responded > 0, "responded must be > 0 for a tracked stage, got: " + responded); + assertEquals(mergeFailed, 0, "mergeFailed must be 0 on success path"); + assertEquals(missing, 0, "missing must be 0 on success path"); + } + assertTrue(anyNonNull, "at least one non-null entry expected in streamStatsCoverage"); + } + + // ─── tests ──────────────────────────────────────────────────────────────────── + + /** + * Simple aggregation — exercises the one-stage path and verifies results + coverage. + */ + @Test + public void testSimpleAggregation() + throws Exception { + @Language("sql") + String sql = "SELECT COUNT(*) FROM mytable LIMIT 1"; + JsonNode response = postWithStreamStats(sql); + + assertFalse(hasExceptions(response), "Query should succeed without exceptions"); + JsonNode resultTable = response.get("resultTable"); + assertNotNull(resultTable); + assertTrue(resultTable.get("rows").get(0).get(0).asLong() > 0, "count(*) should be > 0"); + + assertFullCoverage(response); + } + + /** + * Join query — creates at least two non-root stages and verifies that stats arrive for all of them. + */ + @Test + public void testJoinQuery() + throws Exception { + @Language("sql") + String sql = "SELECT a.AirlineID, COUNT(*) " + + "FROM mytable a JOIN mytable b ON a.AirlineID = b.AirlineID " + + "WHERE a.DaysSinceEpoch > 16000 " + + "GROUP BY a.AirlineID " + + "LIMIT 5"; + JsonNode response = postWithStreamStats(sql); + + assertFalse(hasExceptions(response), "Join query should succeed without exceptions"); + assertNotNull(response.get("resultTable")); + + assertFullCoverage(response); + // A join plan has at least 3 stages: root (0), join (1), leaf (2). Coverage array must be at least length 3. + JsonNode coverage = response.get("streamStatsCoverage"); + assertTrue(coverage.size() >= 3, + "Join should produce at least 3 stages, got coverage of size: " + coverage.size()); + } + + /** + * Three-way UNION — exercises the N-ary set-op path that the legacy flat-binary format reconstructed incorrectly. + * Verifies that results are correct and that coverage is full (no missing stats from any branch). + */ + @Test + public void testThreeWayUnion() + throws Exception { + @Language("sql") + String sql = + "SELECT AirlineID FROM mytable WHERE DaysSinceEpoch = 16101 " + + "UNION " + + "SELECT AirlineID FROM mytable WHERE DaysSinceEpoch = 16102 " + + "UNION " + + "SELECT AirlineID FROM mytable WHERE DaysSinceEpoch = 16103"; + JsonNode response = postWithStreamStats(sql); + + assertFalse(hasExceptions(response), "Three-way UNION should succeed without exceptions"); + assertNotNull(response.get("resultTable"), "Result table must be present"); + + assertFullCoverage(response); + // A three-way UNION plan has leaf stages for each branch plus the union stage and root — at least 4 stages. + JsonNode coverage = response.get("streamStatsCoverage"); + assertTrue(coverage.size() >= 4, + "Three-way UNION should produce at least 4 stages, coverage size: " + coverage.size()); + } + + /** + * Verifies that the cluster-level config {@code pinot.broker.multistage.use.stream.stats.reporting=true} + * activates stream mode for queries that do not carry the per-query option. The broker is restarted with the config + * set for this test class via {@link #overrideBrokerConf}. + */ + @Test + public void testClusterLevelConfigActivatesStreamMode() + throws Exception { + // Post WITHOUT the per-query option — stream mode should still be active because the broker was started + // with CONFIG_OF_USE_STREAM_STATS_REPORTING=true in overrideBrokerConf. + @Language("sql") + String sql = "SELECT COUNT(*) FROM mytable LIMIT 1"; + JsonNode response = postQuery(sql); + + assertFalse(hasExceptions(response), "Cluster-default stream mode query should succeed"); + assertNotNull(response.get("resultTable")); + assertFullCoverage(response); + } + + @Override + protected void overrideBrokerConf(PinotConfiguration brokerConf) { + // Enable stream-mode cluster-wide so testClusterLevelConfigActivatesStreamMode can run without a per-query option. + brokerConf.setProperty(CommonConstants.Broker.CONFIG_OF_USE_STREAM_STATS_REPORTING, true); + super.overrideBrokerConf(brokerConf); + } + + // ─── private helpers ────────────────────────────────────────────────────────── + + private static boolean hasExceptions(JsonNode response) { + JsonNode exceptions = response.get("exceptions"); + return exceptions != null && exceptions.isArray() && !exceptions.isEmpty(); + } +} diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/CommonConstants.java b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/CommonConstants.java index 69115645d783..98339ac4e45d 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/CommonConstants.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/CommonConstants.java @@ -531,7 +531,7 @@ public static class Broker { /// via the {@link Request.QueryOptionKey#USE_STREAM_STATS_REPORTING} query option. Requires all servers to /// implement the {@code SubmitWithStream} RPC; enabling it on a mixed-version cluster will cause query failures. public static final String CONFIG_OF_USE_STREAM_STATS_REPORTING = - "pinot.broker.multistage.use.stream.stats.reporting"; + "pinot.broker.mse.use.stream.stats.reporting"; public static final boolean DEFAULT_USE_STREAM_STATS_REPORTING = false; public static final String CONFIG_OF_USE_FIXED_REPLICA = "pinot.broker.use.fixed.replica"; From 3c3dffc218edb059dc8248219f5250881fd6977d Mon Sep 17 00:00:00 2001 From: Gonzalo Ortiz <1913993+gortiz@users.noreply.github.com> Date: Mon, 11 May 2026 15:08:21 +0200 Subject: [PATCH 16/31] [multistage] Phase B: cancel-via-stream + O(1) cancel in OpChainSchedulerService Folds query cancellation onto the SubmitWithStream bidi RPC so the broker does not need a separate unary Cancel call in stream mode, and improves cancel throughput for all modes via direct QueryExecutionContext lookup. Broker side: - fanOutCancel() is now public on StreamingQuerySession and is used everywhere the broker previously called the unary Cancel RPC while in stream mode (ack errors, processing exceptions, query timeout). - tryRecoverWithStream() fans out cancel then waits for remaining OpChainComplete stats up to the query deadline before building the error QueryResult. - cancelWithStats() removed; on the legacy path cancel no longer collects stats synchronously (error-path stats arrive via OpChainCompletionListener in stream mode). Server side: - SubmitWithStream handler already routes BrokerToServer.cancel to QueryRunner.cancel(); unary Cancel RPC is simplified (no longer serialises stats into the response). - QueryRunner.cancel() / OpChainSchedulerService.cancel() return void. OpChainSchedulerService O(1) cancel (all modes): - Two new maps (_executionContextByRequest, _activeOpChainsByRequest) replace the previous O(n) opChainCache scan in cancel(). - All mutations of both maps for the same requestId are serialised via ConcurrentHashMap.compute() on the counter map; cancel() acquires the query write lock before compute() to close the window where a concurrent register() could slip through the cache check. - _opChainCache entries are invalidated promptly on cancel to release operator memory without waiting for TTL expiry. - Belt-and-suspenders: _cancelledQueryCache is always written on cancel regardless of whether a live context was found. - New test shouldCleanUpContextAfterAllOpChainsComplete verifies that the context map is empty after all opchains for a multi-opchain request complete; existing cancel test asserts activeRequestCount()==0 immediately after cancel() returns. The unary Cancel RPC is kept for one release of overlap per plan. Co-Authored-By: Claude Sonnet 4.6 --- .../StreamStatsReportingIntegrationTest.java | 2 +- .../pinot/query/runtime/QueryRunner.java | 5 +- .../executor/OpChainSchedulerService.java | 137 +++++++++++++----- .../service/dispatch/QueryDispatcher.java | 104 ++++++------- .../streaming/StreamingQuerySession.java | 7 +- .../query/service/server/QueryServer.java | 21 +-- .../executor/OpChainSchedulerServiceTest.java | 47 +++++- 7 files changed, 201 insertions(+), 122 deletions(-) diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/StreamStatsReportingIntegrationTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/StreamStatsReportingIntegrationTest.java index 71f605122853..7a1f8b368fe2 100644 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/StreamStatsReportingIntegrationTest.java +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/StreamStatsReportingIntegrationTest.java @@ -155,7 +155,7 @@ public void testJoinQuery() @Language("sql") String sql = "SELECT a.AirlineID, COUNT(*) " + "FROM mytable a JOIN mytable b ON a.AirlineID = b.AirlineID " - + "WHERE a.DaysSinceEpoch > 16000 " + + "WHERE a.DaysSinceEpoch = 16101 AND b.DaysSinceEpoch = 16101 " + "GROUP BY a.AirlineID " + "LIMIT 5"; JsonNode response = postWithStreamStats(sql); diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/QueryRunner.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/QueryRunner.java index 8a40d20784ea..79c65729e17c 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/QueryRunner.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/QueryRunner.java @@ -57,7 +57,6 @@ import org.apache.pinot.query.runtime.operator.LeafOperator; import org.apache.pinot.query.runtime.operator.MultiStageOperator; import org.apache.pinot.query.runtime.operator.OpChain; -import org.apache.pinot.query.runtime.plan.MultiStageQueryStats; import org.apache.pinot.query.runtime.plan.OpChainConverterDispatcher; import org.apache.pinot.query.runtime.plan.OpChainExecutionContext; import org.apache.pinot.query.runtime.plan.pipeline.PipelineBreakerExecutor; @@ -552,8 +551,8 @@ public MailboxService getMailboxService() { return _mailboxService; } - public Map cancel(long requestId) { - return _opChainScheduler.cancel(requestId); + public void cancel(long requestId) { + _opChainScheduler.cancel(requestId); } /** diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/executor/OpChainSchedulerService.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/executor/OpChainSchedulerService.java index bda6bf9594cc..6c8c0d8de0b8 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/executor/OpChainSchedulerService.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/executor/OpChainSchedulerService.java @@ -26,12 +26,11 @@ import com.google.common.util.concurrent.ListenableFutureTask; import com.google.common.util.concurrent.MoreExecutors; import java.util.ArrayList; -import java.util.HashMap; -import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReadWriteLock; @@ -77,6 +76,21 @@ public class OpChainSchedulerService { /// before the broker submits the query and fires once for every opchain that runs on this server for that request. /// Listeners are responsible for unregistering themselves once they've consumed all expected events. private final ConcurrentMap _completionListeners = new ConcurrentHashMap<>(); + /// Maps requestId → QueryExecutionContext for O(1) cancel. An entry is added when the first opchain for a request + /// registers and removed when cancel() fires or when the last opchain for that request completes. The counter map + /// below tracks how many opchains are still active per request so we know when to clean up. + /// + /// Consistency invariant: all mutations of BOTH maps for the same requestId must occur inside + /// `_activeOpChainsByRequest.compute(requestId, …)`. The compute() bin-lock for the requestId key serializes + /// all register and decrement operations, keeping the two maps coherent. cancel() acquires the query write lock + /// BEFORE calling compute() so that register()'s read lock cannot overlap with the eviction window. + /// + /// NOTE: `_opChainCache.put()` in registerInternal must stay inside the read lock. cancel()'s cache-invalidation + /// forEach runs OUTSIDE the write lock (after the cancelled-query cache is written). It relies on the read lock + /// exclusion to guarantee that no register() call is mid-flight between the cache.put and the counter increment + /// when the forEach observes the cache entry. + private final ConcurrentMap _executionContextByRequest = new ConcurrentHashMap<>(); + private final ConcurrentMap _activeOpChainsByRequest = new ConcurrentHashMap<>(); public OpChainSchedulerService(String instanceId, ExecutorService executorService, PinotConfiguration config) { this(instanceId, executorService, config.getProperty(MultiStageQueryRunner.KEY_OF_OP_STATS_CACHE_SIZE, @@ -158,8 +172,21 @@ private void checkTermination(OpChain operatorChain, QueryExecutionContext execu private void registerInternal(OpChain operatorChain, QueryExecutionContext executionContext) { OpChainId opChainId = operatorChain.getId(); + long requestId = opChainId.getRequestId(); MultiStageOperator rootOperator = operatorChain.getRoot(); _opChainCache.put(opChainId, Pair.of(rootOperator, executionContext)); + // Track the context for O(1) cancel and increment the per-request active opchain count. + // Both operations are performed inside a single compute() call so that a concurrent decrementActiveOpChains() + // that observes count==0 and removes the context entry cannot race with a new putIfAbsent arriving between the + // counter update and the context-map update. + _activeOpChainsByRequest.compute(requestId, (k, v) -> { + _executionContextByRequest.putIfAbsent(requestId, executionContext); + if (v == null) { + return new AtomicInteger(1); + } + v.incrementAndGet(); + return v; + }); // Captured by the runJob and read by the FutureCallback so we can hand the calculated stats to the per-request // completion listener (stream-mode stats reporting). On error we may not have stats; the FutureCallback handles // that by passing null. @@ -193,6 +220,7 @@ public void runJob() { @Override public void onSuccess(Void result) { _metrics.onOpChainFinished(rootOperator); + decrementActiveOpChains(requestId); notifyCompletionListener(opChainId, operatorChain, statsRef.get(), null); operatorChain.close(); } @@ -214,6 +242,7 @@ public void onFailure(Throwable t) { } else { LOGGER.error(logMsg, t); } + decrementActiveOpChains(requestId); notifyCompletionListener(opChainId, operatorChain, statsRef.get(), t); operatorChain.cancel(t); operatorChain.close(); @@ -223,6 +252,27 @@ public void onFailure(Throwable t) { _executorService.submit(listenableFutureTask); } + private void decrementActiveOpChains(long requestId) { + // Use compute() so the "decrement-to-zero → remove" step is atomic with a concurrent registerInternal() + // that would otherwise interleave a putIfAbsent + increment between our remove() calls. + _activeOpChainsByRequest.compute(requestId, (k, counter) -> { + if (counter == null) { + return null; + } + if (counter.decrementAndGet() <= 0) { + _executionContextByRequest.remove(requestId); + return null; + } + return counter; + }); + } + + /** Returns the number of requestIds with at least one active opchain. Exposed for tests only. */ + @VisibleForTesting + int activeRequestCount() { + return _executionContextByRequest.size(); + } + private void notifyCompletionListener(OpChainId opChainId, OpChain operatorChain, @Nullable MultiStageQueryStats stats, @Nullable Throwable error) { OpChainCompletionListener listener = _completionListeners.get(opChainId.getRequestId()); @@ -256,45 +306,54 @@ public OpChainCompletionListener unregisterCompletionListener(long requestId) { return _completionListeners.remove(requestId); } - public Map cancel(long requestId) { - QueryExecutionContext cancelledExecutionContext = null; - Map cancelledOperators = new HashMap<>(); - for (Map.Entry> entry : _opChainCache.asMap() - .entrySet()) { - if (entry.getKey().getRequestId() == requestId) { - Pair pair = entry.getValue(); - cancelledOperators.put(entry.getKey(), pair.getLeft()); - cancelledExecutionContext = pair.getRight(); - } + /** + * Cancels all opchains registered for {@code requestId} by terminating the shared + * {@link QueryExecutionContext} for that request. The cancel is O(1) via a direct context look-up; no + * per-opchain scan is needed. + * + *

Stats for cancelled opchains are NOT returned synchronously. In stream mode, each cancelled opchain + * delivers its (partial) stats asynchronously via {@link OpChainCompletionListener#onOpChainComplete} once the + * opchain finishes after being interrupted. In legacy mode, cancel-path stats are not collected. + * + *

If no opchains are currently registered (pre-registration cancel race), the requestId is marked in the + * cancelled-query cache so that any future registrations for that requestId are immediately rejected. + */ + public void cancel(long requestId) { + // Acquire the write lock BEFORE eviction so that register()'s readLock cannot overlap with the window + // between the context eviction and the cancelled-query cache write. Without the write lock first, a + // concurrent register() could pass the cache check (cache not yet written) after the context was already + // evicted, slipping through and starting an opchain that should have been rejected. + AtomicReference ctxRef = new AtomicReference<>(); + Lock writeLock = getQueryLock(requestId).writeLock(); + writeLock.lock(); + try { + // Atomically evict the counter and the shared QueryExecutionContext inside one compute() so this cannot + // race with a concurrent registerInternal() that holds the same compute()-bin lock while calling + // putIfAbsent() on _executionContextByRequest. + _activeOpChainsByRequest.compute(requestId, (k, counter) -> { + ctxRef.set(_executionContextByRequest.remove(requestId)); + return null; // unconditionally evict the counter on cancel + }); + _cancelledQueryCache.put(requestId, Boolean.TRUE); + } finally { + writeLock.unlock(); } - - if (cancelledExecutionContext != null) { - cancelledExecutionContext.terminate(QueryErrorCode.QUERY_CANCELLATION, "Cancelled on: " + _instanceId); - _opChainCache.invalidateAll(cancelledOperators.keySet()); - Map statsMap = new HashMap<>(); - for (Map.Entry entry : cancelledOperators.entrySet()) { - int stageId = entry.getKey().getStageId(); - MultiStageQueryStats.StageStats.Closed stats = entry.getValue().calculateStats().getCurrentStats().close(); - statsMap.merge(stageId, stats, (s1, s2) -> { - s1.merge(s2); - return s1; - }); - } - return statsMap; - } else { - // When no query execution context is found, it means there is no actively running operator chain (registered but - // not done). To prevent future registration for a cancelled query, add the query to the cancelled query cache. - - // Acquire write lock for the query to ensure that the query is not cancelled while scheduling the operator chain. - Lock writeLock = getQueryLock(requestId).writeLock(); - writeLock.lock(); - try { - _cancelledQueryCache.put(requestId, Boolean.TRUE); - } finally { - writeLock.unlock(); + // Promptly release memory held by cancelled opchain entries; without explicit invalidation they would linger + // in the cache until TTL expiry. Running this forEach outside the write lock is safe: by the time the write + // lock was released, the cancelledQueryCache entry was already written, so no new register() calls for this + // requestId can add entries. Any register() that was already holding the read lock when cancel() began must + // have completed (it released the read lock for cancel() to acquire the write lock), so its _opChainCache + // entry is already visible to this iterator. Double-invalidation of an already-evicted entry is a no-op. + _opChainCache.asMap().forEach((id, pair) -> { + if (id.getRequestId() == requestId) { + _opChainCache.invalidate(id); } - - return Map.of(); + }); + QueryExecutionContext context = ctxRef.get(); + if (context != null) { + // terminate() interrupts all registered tasks (via addTask) and sets the termination flag so that + // checkTermination() blocks any subsequent registrations for the same requestId. + context.terminate(QueryErrorCode.QUERY_CANCELLATION, "Cancelled on: " + _instanceId); } } diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/QueryDispatcher.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/QueryDispatcher.java index 07f10ee25bf4..265a8a1d8459 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/QueryDispatcher.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/QueryDispatcher.java @@ -26,8 +26,6 @@ import io.grpc.ConnectivityState; import io.grpc.Deadline; import io.grpc.netty.shaded.io.netty.handler.ssl.SslContext; -import java.io.DataInputStream; -import java.io.InputStream; import java.time.Duration; import java.util.ArrayList; import java.util.HashMap; @@ -52,7 +50,6 @@ import org.apache.commons.lang3.tuple.Pair; import org.apache.pinot.common.config.TlsConfig; import org.apache.pinot.common.datablock.DataBlock; -import org.apache.pinot.common.datatable.StatMap; import org.apache.pinot.common.failuredetector.FailureDetector; import org.apache.pinot.common.proto.Plan; import org.apache.pinot.common.proto.Worker; @@ -81,7 +78,6 @@ import org.apache.pinot.query.runtime.blocks.MseBlock; import org.apache.pinot.query.runtime.blocks.RowHeapDataBlock; import org.apache.pinot.query.runtime.blocks.SerializedDataBlock; -import org.apache.pinot.query.runtime.operator.BaseMailboxReceiveOperator; import org.apache.pinot.query.runtime.operator.MultiStageOperator; import org.apache.pinot.query.runtime.operator.OpChain; import org.apache.pinot.query.runtime.plan.MultiStageQueryStats; @@ -205,13 +201,15 @@ public QueryResult submitAndReduce(RequestContext context, DispatchableSubPlan d /// most of the budget, the per-stage stats may end up partial (visible via the per-stage {@code mergeFailed} / /// {@code missing} counts the session exposes). /// + /// Cancel is handled via {@link StreamingQuerySession#fanOutCancel()} — no unary Cancel RPCs are issued for this + /// query path. On any error, fan-out cancel is broadcast over the open streams, then the broker waits for remaining + /// stats before building the final result. + /// /// Mixed-version policy. No automatic fallback to the unary {@link #submit} path. Enabling /// {@link CommonConstants.Broker.Request.QueryOptionKey#USE_STREAM_STATS_REPORTING} requires every server in the /// cluster to implement {@code SubmitWithStream}; if any server returns {@code UNIMPLEMENTED} or any other /// transport error during dispatch, {@link #submitWithStream} surfaces the throwable through the ack queue, - /// {@link #processResults} throws, this method's {@code catch} cancels every other server via {@link #tryRecover} - /// and propagates the failure to the caller. Mixed-version safety is the operator's responsibility — only enable - /// this option once the whole fleet has been upgraded. + /// {@link #processResults} throws, and this method fans out cancel via the session before propagating the failure. private QueryResult submitAndReduceWithStream(RequestContext context, DispatchableSubPlan dispatchableSubPlan, long timeoutMs, Map queryOptions) throws Exception { @@ -249,13 +247,13 @@ private QueryResult submitAndReduceWithStream(RequestContext context, Dispatchab } if (brokerResult.getProcessingException() != null) { - cancel(requestId); + session.fanOutCancel(); } return mergeSessionStatsIntoResult(brokerResult, session, expectedByStage); } catch (Exception ex) { - return tryRecover(requestId, servers, ex); + return tryRecoverWithStream(session, expectedByStage, deadlineMs, ex); } catch (Throwable e) { - cancel(requestId); + session.fanOutCancel(); throw e; } finally { if (isQueryCancellationEnabled()) { @@ -309,7 +307,7 @@ void submitWithStream(long requestId, DispatchableSubPlan dispatchableSubPlan, l processResults(requestId, serversOut.size(), (response, server) -> { if (response.containsMetadata(ServerResponseStatus.STATUS_ERROR)) { - cancel(requestId, serversOut); + session.fanOutCancel(); throw new RuntimeException( String.format("Unable to execute query plan for request: %d on server: %s, ERROR: %s", requestId, server, response.getMetadataOrDefault(ServerResponseStatus.STATUS_ERROR, "null"))); @@ -372,10 +370,10 @@ private QueryResult mergeSessionStatsIntoResult(QueryResult brokerResult, Stream brokerResult.getBrokerReduceTimeMs(), stageCoverage); } - /// Tries to recover from an exception thrown during query dispatching. + /// Tries to recover from an exception thrown during legacy (non-streaming) query dispatching. /// - /// [QueryException] and [TimeoutException] are handled by returning a [QueryResult] with the error code and stats, - /// while other exceptions are not known, so they are directly rethrown. + /// [QueryException] and [TimeoutException] are handled by returning a [QueryResult] with the error code and empty + /// stats, while other exceptions are directly rethrown. Stats are not collected on the legacy cancel path. private QueryResult tryRecover(long requestId, Set servers, Exception ex) throws Exception { if (servers.isEmpty()) { @@ -390,19 +388,48 @@ private QueryResult tryRecover(long requestId, Set servers, } else if (ex instanceof QueryException) { errorCode = ((QueryException) ex).getErrorCode(); } else { - // in case of unknown exceptions, the exception will be rethrown, so we don't need stats cancel(requestId, servers); throw ex; } - // in case of known exceptions (timeout or query exception), we need can build here the erroneous QueryResult - // that include the stats. - LOGGER.warn("Query failed with a known exception. Trying to cancel the other opchains"); - MultiStageQueryStats stats = cancelWithStats(requestId, servers); - if (stats == null) { + LOGGER.warn("Query failed with a known exception. Cancelling remaining opchains."); + cancel(requestId, servers); + QueryProcessingException processingException = new QueryProcessingException(errorCode, ex.getMessage()); + return new QueryResult(processingException, MultiStageQueryStats.emptyStats(0), 0L); + } + + /// Tries to recover from an exception thrown during stream-mode ({@code SubmitWithStream}) query dispatching. + /// + /// Fans out cancel over the open streams, waits briefly for any remaining {@code OpChainComplete} messages (up to + /// the query deadline), and builds a {@link QueryResult} that includes whatever stats arrived before the deadline. + /// Stats from before the error are available because servers push {@code OpChainComplete} even on failure. + /// + /// Unknown exceptions (not {@link TimeoutException} or {@link QueryException}) are re-thrown after cancel fan-out. + private QueryResult tryRecoverWithStream(StreamingQuerySession session, Map expectedByStage, + long deadlineMs, Exception ex) + throws Exception { + if (ex instanceof ExecutionException && ex.getCause() instanceof Exception) { + ex = (Exception) ex.getCause(); + } + QueryErrorCode errorCode; + if (ex instanceof TimeoutException) { + errorCode = QueryErrorCode.EXECUTION_TIMEOUT; + } else if (ex instanceof QueryException) { + errorCode = ((QueryException) ex).getErrorCode(); + } else { + session.fanOutCancel(); throw ex; } + LOGGER.warn("Stream-mode query failed with a known exception. Fanning out cancel and waiting for stats."); + session.fanOutCancel(); + long remainingMs = Math.max(0, deadlineMs - System.currentTimeMillis()); + try { + session.awaitCompletion(remainingMs, TimeUnit.MILLISECONDS); + } catch (InterruptedException ie) { + Thread.currentThread().interrupt(); + } QueryProcessingException processingException = new QueryProcessingException(errorCode, ex.getMessage()); - return new QueryResult(processingException, stats, 0L); + QueryResult errorResult = new QueryResult(processingException, MultiStageQueryStats.emptyStats(0), 0L); + return mergeSessionStatsIntoResult(errorResult, session, expectedByStage); } public List explain(RequestContext context, DispatchablePlanFragment fragment, long timeoutMs, @@ -683,41 +710,6 @@ private boolean cancel(long requestId, @Nullable Set server return true; } - @Nullable - private MultiStageQueryStats cancelWithStats(long requestId, @Nullable Set servers) { - if (servers == null) { - return null; - } - - Deadline deadline = Deadline.after(_cancelTimeout.toMillis(), TimeUnit.MILLISECONDS); - SendRequest sendRequest = DispatchClient::cancel; - BlockingQueue> dispatchCallbacks = - dispatch(sendRequest, servers, deadline, serverInstance -> requestId); - - MultiStageQueryStats stats = MultiStageQueryStats.emptyStats(0); - StatMap rootStats = new StatMap<>(BaseMailboxReceiveOperator.StatKey.class); - stats.getCurrentStats().addLastOperator(MultiStageOperator.Type.MAILBOX_RECEIVE, rootStats); - try { - processResults(requestId, servers.size(), (response, server) -> { - Map statsByStage = response.getStatsByStageMap(); - for (Map.Entry entry : statsByStage.entrySet()) { - try (InputStream is = entry.getValue().newInput(); DataInputStream dis = new DataInputStream(is)) { - MultiStageQueryStats.StageStats.Closed closed = MultiStageQueryStats.StageStats.Closed.deserialize(dis); - stats.mergeUpstream(entry.getKey(), closed); - } catch (Exception e) { - LOGGER.debug("Caught exception while deserializing stats on server: {}", server, e); - } - } - }, deadline, dispatchCallbacks); - return stats; - } catch (InterruptedException e) { - throw QueryErrorCode.INTERNAL.asException("Interrupted while waiting for cancel response", e); - } catch (TimeoutException e) { - LOGGER.debug("Timed out waiting for cancel response", e); - return stats; - } - } - private DispatchClient getOrCreateDispatchClient(QueryServerInstance queryServerInstance) { String hostname = queryServerInstance.getHostname(); int port = queryServerInstance.getQueryServicePort(); diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/streaming/StreamingQuerySession.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/streaming/StreamingQuerySession.java index 1bd8ff1c7985..223bf4398048 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/streaming/StreamingQuerySession.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/streaming/StreamingQuerySession.java @@ -234,10 +234,11 @@ public void recordStreamError(StreamingServerHandle stream, @Nullable Throwable } /** - * Sends {@code BrokerToServer.cancel} on every other open server stream. Called once on the first peer error - * observed. Failures are swallowed — cancel is best-effort. + * Sends {@code BrokerToServer.cancel} on every open server stream. Used on the first peer error observed and when + * the broker's data mailbox reports a processing exception. Failures are swallowed — cancel is best-effort. + * Idempotent w.r.t. concurrent calls. */ - private void fanOutCancel() { + public void fanOutCancel() { Set snapshot; _lock.lock(); try { diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/server/QueryServer.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/server/QueryServer.java index f6b5a8d21d1b..4bbf51a0cfb9 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/server/QueryServer.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/server/QueryServer.java @@ -20,12 +20,10 @@ import com.google.common.annotations.VisibleForTesting; import com.google.protobuf.ByteString; -import com.google.protobuf.UnsafeByteOperations; import io.grpc.Server; import io.grpc.netty.shaded.io.grpc.netty.NettyServerBuilder; import io.grpc.netty.shaded.io.netty.handler.ssl.SslContext; import io.grpc.stub.StreamObserver; -import java.io.DataOutputStream; import java.util.ArrayDeque; import java.util.ArrayList; import java.util.HashMap; @@ -40,7 +38,6 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import javax.annotation.Nullable; -import org.apache.commons.io.output.UnsynchronizedByteArrayOutputStream; import org.apache.pinot.common.config.TlsConfig; import org.apache.pinot.common.metrics.ServerMeter; import org.apache.pinot.common.metrics.ServerMetrics; @@ -715,26 +712,12 @@ private Worker.QueryResponse buildErrorResponse(String errorMsg) { public void cancel(Worker.CancelRequest request, StreamObserver responseObserver) { long requestId = request.getRequestId(); try { - Map stats = _queryRunner.cancel(requestId); - - Worker.CancelResponse.Builder cancelBuilder = Worker.CancelResponse.newBuilder(); - for (Map.Entry statEntry : stats.entrySet()) { - // even we are using output streams here, these calls are non-blocking because we use in memory output streams - try (UnsynchronizedByteArrayOutputStream baos = new UnsynchronizedByteArrayOutputStream.Builder().get(); - DataOutputStream daos = new DataOutputStream(baos)) { - statEntry.getValue().serialize(daos); - - daos.flush(); - byte[] byteArray = baos.toByteArray(); - ByteString bytes = UnsafeByteOperations.unsafeWrap(byteArray); - cancelBuilder.putStatsByStage(statEntry.getKey(), bytes); - } - } - responseObserver.onNext(cancelBuilder.build()); + _queryRunner.cancel(requestId); } catch (Throwable t) { LOGGER.error("Caught exception while cancelling opChain for request: {}", requestId, t); } // we always return completed even if cancel attempt fails, server will self clean up in this case. + responseObserver.onNext(Worker.CancelResponse.getDefaultInstance()); responseObserver.onCompleted(); } diff --git a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/executor/OpChainSchedulerServiceTest.java b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/executor/OpChainSchedulerServiceTest.java index 403e635a3610..699e8ded5522 100644 --- a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/executor/OpChainSchedulerServiceTest.java +++ b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/executor/OpChainSchedulerServiceTest.java @@ -84,12 +84,16 @@ public void beforeMethod() { } private OpChain getChain(MultiStageOperator operator) { + return getChain(operator, 0); + } + + private OpChain getChain(MultiStageOperator operator, int stageId) { MailboxService mailboxService = mock(MailboxService.class); when(mailboxService.getHostname()).thenReturn("localhost"); when(mailboxService.getPort()).thenReturn(1234); WorkerMetadata workerMetadata = new WorkerMetadata(0, Map.of(), Map.of()); OpChainExecutionContext context = OpChainExecutionContext.fromQueryContext(mailboxService, Map.of(), - new StageMetadata(0, List.of(workerMetadata), Map.of()), workerMetadata, null, true, true); + new StageMetadata(stageId, List.of(workerMetadata), Map.of()), workerMetadata, null, true, true); return new OpChain(context, operator); } @@ -190,6 +194,10 @@ public void shouldCallCancelOnOpChainsWhenItIsCancelledByDispatch() // now cancel the request. schedulerService.cancel(123L); + // The eviction inside cancel() happens under the write lock (before context.terminate()), so the context + // map must be empty as soon as cancel() returns — no need to wait for the opchain to finish. + assertEquals(schedulerService.activeRequestCount(), 0, + "context map should be empty immediately after cancel() returns"); assertTrue(cancelLatch.await(10, TimeUnit.SECONDS), "expected OpChain to be cancelled"); Mockito.verify(_operatorA, Mockito.times(1)).cancel(Mockito.any()); @@ -310,6 +318,43 @@ public void shouldThrowQueryCancelledExceptionWhenRegisteringOpChainAfterQueryCa } } + /** + * Registers two opchains for the same request, waits for both to complete, and verifies that the per-request + * context map entry is removed once the last opchain finishes. Regression coverage for the TOCTOU race in + * decrementActiveOpChains that could leave a stale entry in _executionContextByRequest. + */ + @Test + public void shouldCleanUpContextAfterAllOpChainsComplete() + throws InterruptedException { + CountDownLatch allClosed = new CountDownLatch(2); + MultiStageOperator operatorB = Mockito.mock(MultiStageOperator.class); + Mockito.when(operatorB.copyStatMaps()).thenAnswer(inv -> new StatMap<>(MailboxSendOperator.StatKey.class)); + + Mockito.when(_operatorA.nextBlock()).thenReturn(SuccessMseBlock.INSTANCE); + Mockito.doAnswer(inv -> MultiStageQueryStats.emptyStats(0)).when(_operatorA).calculateStats(); + Mockito.doAnswer(inv -> { + allClosed.countDown(); + return null; + }).when(_operatorA).close(); + + Mockito.when(operatorB.nextBlock()).thenReturn(SuccessMseBlock.INSTANCE); + Mockito.doAnswer(inv -> MultiStageQueryStats.emptyStats(1)).when(operatorB).calculateStats(); + Mockito.doAnswer(inv -> { + allClosed.countDown(); + return null; + }).when(operatorB).close(); + + OpChainSchedulerService schedulerService = new OpChainSchedulerService(_executor); + try (QueryThreadContext ignore = QueryThreadContext.openForMseTest()) { + schedulerService.register(getChain(_operatorA, 0)); + schedulerService.register(getChain(operatorB, 1)); + } + + assertTrue(allClosed.await(10, TimeUnit.SECONDS), "expected both opchains to complete within 10 s"); + assertEquals(schedulerService.activeRequestCount(), 0, + "context map should be empty after all opchains for a request complete"); + } + @Test public void shouldHandleConcurrentCancellationAndRegistration() throws InterruptedException { From 09f56dad104e040cebd5b04699df14986cb0f787 Mon Sep 17 00:00:00 2001 From: Gonzalo Ortiz <1913993+gortiz@users.noreply.github.com> Date: Mon, 11 May 2026 17:24:41 +0200 Subject: [PATCH 17/31] [multistage] Fix 3 checkstyle violations caught by GHA linter - StageStatsTreeNode: replace ImmutableList.copyOf with List.copyOf to eliminate the banned Guava import - DispatchClient: wrap @return Javadoc line to stay under 120 chars - StreamingDispatchObserverTest: remove blank line before closing brace Co-Authored-By: Claude Sonnet 4.6 --- .../apache/pinot/query/runtime/plan/StageStatsTreeNode.java | 5 ++--- .../apache/pinot/query/service/dispatch/DispatchClient.java | 5 +++-- .../dispatch/streaming/StreamingDispatchObserverTest.java | 1 - 3 files changed, 5 insertions(+), 6 deletions(-) diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/StageStatsTreeNode.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/StageStatsTreeNode.java index feeef1c35bda..75f6cfbd0e2f 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/StageStatsTreeNode.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/StageStatsTreeNode.java @@ -18,7 +18,6 @@ */ package org.apache.pinot.query.runtime.plan; -import com.google.common.collect.ImmutableList; import java.util.ArrayList; import java.util.List; import java.util.Objects; @@ -56,9 +55,9 @@ public ShapeMismatchException(String message) { public StageStatsTreeNode(MultiStageOperator.Type type, List planNodeIds, StatMap statMap, List children) { _type = type; - _planNodeIds = ImmutableList.copyOf(planNodeIds); + _planNodeIds = List.copyOf(planNodeIds); _statMap = statMap; - _children = ImmutableList.copyOf(children); + _children = List.copyOf(children); } public MultiStageOperator.Type getType() { diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/DispatchClient.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/DispatchClient.java index 7d6bee8da64a..e76d20dd0b84 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/DispatchClient.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/DispatchClient.java @@ -148,8 +148,9 @@ public void submit(Worker.QueryRequest request, QueryServerInstance virtualServe * @param expectedOpChainsForThisServer number of opchains this server is expected to report; used to drain the * session latch correctly when the stream errors before all opchains have responded * @param ackCallback receives the submit-ack response or a failure throwable - * @return the observer, also exposed as {@link org.apache.pinot.query.service.dispatch.streaming.StreamingServerHandle} - * on the session for cancel fan-out + * @return the observer, also exposed as + * {@link org.apache.pinot.query.service.dispatch.streaming.StreamingServerHandle} on the session for + * cancel fan-out */ public StreamingDispatchObserver submitWithStream(Worker.QueryRequest request, QueryServerInstance virtualServer, Deadline deadline, StreamingQuerySession session, int expectedOpChainsForThisServer, diff --git a/pinot-query-runtime/src/test/java/org/apache/pinot/query/service/dispatch/streaming/StreamingDispatchObserverTest.java b/pinot-query-runtime/src/test/java/org/apache/pinot/query/service/dispatch/streaming/StreamingDispatchObserverTest.java index 817d54e6cba2..79bc59fbb75f 100644 --- a/pinot-query-runtime/src/test/java/org/apache/pinot/query/service/dispatch/streaming/StreamingDispatchObserverTest.java +++ b/pinot-query-runtime/src/test/java/org/apache/pinot/query/service/dispatch/streaming/StreamingDispatchObserverTest.java @@ -189,5 +189,4 @@ private static ByteString serialize(StatMap statMap) return ByteString.copyFrom(baos.toByteArray()); } } - } From 08295d95cde66662950fb54b0178a54811182832 Mon Sep 17 00:00:00 2001 From: Gonzalo Ortiz <1913993+gortiz@users.noreply.github.com> Date: Tue, 12 May 2026 15:34:49 +0200 Subject: [PATCH 18/31] [multistage] OperatorTypeDescriptor SPI + OperatorTypeRegistry MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Extract MultiStageOperator.Type into an OperatorTypeDescriptor interface so plugin-defined operators can participate in the stream-mode stats path without modifying core. The Type enum implements the interface unchanged; a new OperatorTypeRegistry loads built-in types at startup and discovers plugins via ServiceLoader (META-INF/services). Plugin ids must be >= 256. Changed files: - OperatorTypeDescriptor (new) — SPI interface - OperatorTypeRegistry (new) — static registry, ServiceLoader-discovered - MultiStageOperator.Type — implements OperatorTypeDescriptor - MultiStageOperator.getOperatorType() — return type widened to interface - MultiStageQueryStats._operatorTypes — List - MultiStageStatsTreeDecoder — uses OperatorTypeRegistry.fromId() - MultiStageStatsTreeEncoder — local type var is OperatorTypeDescriptor - StageStatsTreeNode — _type is OperatorTypeDescriptor; merge() uses getId() - InStageStatsTreeBuilder — local vars widened to OperatorTypeDescriptor - OperatorTypeRegistryTest (new) — verifies all 16 built-ins present, unknown id returns null Co-Authored-By: Claude Sonnet 4.6 --- .../runtime/InStageStatsTreeBuilder.java | 21 ++--- .../runtime/operator/MultiStageOperator.java | 4 +- .../operator/OperatorTypeDescriptor.java | 80 +++++++++++++++++++ .../operator/OperatorTypeRegistry.java | 75 +++++++++++++++++ .../runtime/plan/MultiStageQueryStats.java | 21 ++--- .../plan/MultiStageStatsTreeDecoder.java | 9 ++- .../plan/MultiStageStatsTreeEncoder.java | 3 +- .../runtime/plan/StageStatsTreeNode.java | 18 ++--- .../operator/OperatorTypeRegistryTest.java | 59 ++++++++++++++ 9 files changed, 254 insertions(+), 36 deletions(-) create mode 100644 pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/OperatorTypeDescriptor.java create mode 100644 pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/OperatorTypeRegistry.java create mode 100644 pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/OperatorTypeRegistryTest.java diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/InStageStatsTreeBuilder.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/InStageStatsTreeBuilder.java index dfc042f755d3..3018ffefef4b 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/InStageStatsTreeBuilder.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/InStageStatsTreeBuilder.java @@ -49,6 +49,7 @@ import org.apache.pinot.query.planner.plannode.WindowNode; import org.apache.pinot.query.runtime.operator.MailboxSendOperator; import org.apache.pinot.query.runtime.operator.MultiStageOperator; +import org.apache.pinot.query.runtime.operator.OperatorTypeDescriptor; import org.apache.pinot.query.runtime.plan.MultiStageQueryStats; import org.apache.pinot.spi.utils.JsonUtils; import org.slf4j.Logger; @@ -69,11 +70,11 @@ public InStageStatsTreeBuilder(MultiStageQueryStats.StageStats stageStats, IntFu _jsonStatsByStage = jsonStatsByStage; } - private ObjectNode selfNode(MultiStageOperator.Type type, Context context) { + private ObjectNode selfNode(OperatorTypeDescriptor type, Context context) { return selfNode(type, context, _index, new JsonNode[0]); } - private ObjectNode selfNode(MultiStageOperator.Type type, Context context, int index, JsonNode[] childrenArr) { + private ObjectNode selfNode(OperatorTypeDescriptor type, Context context, int index, JsonNode[] childrenArr) { return selfNode(type, context, index, childrenArr, true); } @@ -90,13 +91,13 @@ private ObjectNode selfNode(MultiStageOperator.Type type, Context context, int i * @return The constructed JSON node representing the operator and its statistics, including children. */ private ObjectNode selfNode( - MultiStageOperator.Type type, + OperatorTypeDescriptor type, Context context, int index, JsonNode[] childrenArr, boolean adjustWithChildren) { ObjectNode json = JsonUtils.newObjectNode(); - json.put("type", type.toString()); + json.put("type", type.name()); for (Map.Entry entry : _stageStats.getOperatorStats(index).asJson().properties()) { json.set(entry.getKey(), entry.getValue()); } @@ -117,7 +118,7 @@ private ObjectNode selfNode( return json; } - private void addClockTimeMs(MultiStageOperator.Type type, ObjectNode selfNode, JsonNode[] children, Context context) { + private void addClockTimeMs(OperatorTypeDescriptor type, ObjectNode selfNode, JsonNode[] children, Context context) { JsonNode executionTimeMs = selfNode.get("executionTimeMs"); long cpuTimeMs = executionTimeMs == null ? 0 : executionTimeMs.asLong(0); @@ -132,7 +133,7 @@ private void addClockTimeMs(MultiStageOperator.Type type, ObjectNode selfNode, J } private void addSelfAllocatedBytes( - MultiStageOperator.Type type, ObjectNode selfNode, JsonNode[] children, Context context) { + OperatorTypeDescriptor type, ObjectNode selfNode, JsonNode[] children, Context context) { JsonNode allocatedBytes = selfNode.get("allocatedMemoryBytes"); long totalAllocatedBytes = allocatedBytes == null ? 0 : allocatedBytes.asLong(0); @@ -142,7 +143,7 @@ private void addSelfAllocatedBytes( } } - private void addSelfGcTime(MultiStageOperator.Type type, ObjectNode selfNode, JsonNode[] children, Context context) { + private void addSelfGcTime(OperatorTypeDescriptor type, ObjectNode selfNode, JsonNode[] children, Context context) { JsonNode gcTimeMs = selfNode.get("gcTimeMs"); long totalGcTimeMs = gcTimeMs == null ? 0 : gcTimeMs.asLong(0); @@ -152,7 +153,7 @@ private void addSelfGcTime(MultiStageOperator.Type type, ObjectNode selfNode, Js } } - private long getChildrenStat(MultiStageOperator.Type type, JsonNode[] children, String key) { + private long getChildrenStat(OperatorTypeDescriptor type, JsonNode[] children, String key) { if (children == null) { return 0; } @@ -179,7 +180,7 @@ private MailboxReceiveNode getPipelineBreakerNode(BasePlanNode node) { if (_index == 0) { return null; } - MultiStageOperator.Type nextOperatorType = _stageStats.getOperatorType(_index - 1); + OperatorTypeDescriptor nextOperatorType = _stageStats.getOperatorType(_index - 1); if (nextOperatorType != MultiStageOperator.Type.PIPELINE_BREAKER) { // even if the plan may say there is a pipeline breaker, the stats do not have it return null; @@ -208,7 +209,7 @@ private boolean isPipelineBreakerNode(JoinNode joinNode) { } private ObjectNode recursiveCase(BasePlanNode node, MultiStageOperator.Type expectedType, Context context) { - MultiStageOperator.Type type = _stageStats.getOperatorType(_index); + OperatorTypeDescriptor type = _stageStats.getOperatorType(_index); /* Sometimes the operator type is not what we expect, but we can still build the tree This always happen in stage 0, in which case we have two operators but we only have stats for the receive diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/MultiStageOperator.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/MultiStageOperator.java index b33391d12847..2012ac18aa2c 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/MultiStageOperator.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/MultiStageOperator.java @@ -72,7 +72,7 @@ public MultiStageOperator(OpChainExecutionContext context) { */ protected abstract Logger logger(); - public abstract Type getOperatorType(); + public abstract OperatorTypeDescriptor getOperatorType(); public abstract void registerExecution(long time, int numRows, long memoryUsedBytes, long gcTimeMs); @@ -242,7 +242,7 @@ private long getGcTimeMillis() { * always append them at the end and assign the next available ID. Never reuse or change existing IDs as this * would break backward compatibility with older versions. */ - public enum Type { + public enum Type implements OperatorTypeDescriptor { AGGREGATE(0, AggregateOperator.StatKey.class) { @Override public void mergeInto(BrokerResponseNativeV2 response, StatMap map) { diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/OperatorTypeDescriptor.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/OperatorTypeDescriptor.java new file mode 100644 index 000000000000..cd3d2e36127a --- /dev/null +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/OperatorTypeDescriptor.java @@ -0,0 +1,80 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.pinot.query.runtime.operator; + +import org.apache.pinot.common.datatable.StatMap; +import org.apache.pinot.common.metrics.ServerMetrics; +import org.apache.pinot.common.response.broker.BrokerResponseNativeV2; + + +/** + * SPI for MSE operator type descriptors. Built-in types are the entries of {@link MultiStageOperator.Type}. Plugin + * authors can implement this interface and register descriptors via + * {@code META-INF/services/org.apache.pinot.query.runtime.operator.OperatorTypeDescriptor} to extend the set of known + * operator types without modifying core. + * + *

Instances are registered in {@link OperatorTypeRegistry} at startup. Descriptors should be effectively singletons + * within a JVM — the registry stores one instance per id, and stage-stats equality checks + * ({@link org.apache.pinot.query.runtime.plan.MultiStageQueryStats.StageStats#equals}) rely on + * {@link Object#equals} comparing the two descriptor lists element-by-element. Built-in enum constants satisfy this + * naturally via identity equality; plugin implementations should either be singletons or override {@link #equals}. + */ +public interface OperatorTypeDescriptor { + + /** + * First id available to plugin-defined operator types. Built-in types ({@link MultiStageOperator.Type}) use ids 0–15; + * ids 16–255 are reserved for future built-ins. Plugin ids must be ≥ this value. + */ + int PLUGIN_ID_FLOOR = 256; + + /** + * Stable numeric id used in the gRPC wire format and the legacy binary stat format. Must be unique across all + * descriptors loaded in the same JVM. + * + *

Built-in types use ids 0–255. Plugin types must use ids ≥ {@value #PLUGIN_ID_FLOOR}. + */ + int getId(); + + /** Display name used in logs and stats JSON output. */ + String name(); + + /** + * Returns the key class for the operator's {@link StatMap}. + * + *

The returned class must be {@code Class} where {@code K extends Enum & StatMap.Key}. The generic + * parameter is erased at runtime, so the raw {@code Class} type is used here to remain compatible with the + * {@link MultiStageOperator.Type} enum's existing {@code getStatKeyClass()} signature. + */ + @SuppressWarnings("rawtypes") + Class getStatKeyClass(); + + /** + * Merges this operator's stats into the broker response. Each implementation casts {@code map} to its specific + * {@link StatMap} key type (which is guaranteed by the serialization contract that the key class matches + * {@link #getStatKeyClass()}). + */ + void mergeInto(BrokerResponseNativeV2 response, StatMap map); + + /** + * Updates server-side metrics from this operator's stats. Default implementation is a no-op; override when the + * operator produces metrics that should be recorded on the server at query completion. + */ + default void updateServerMetrics(StatMap map, ServerMetrics serverMetrics) { + } +} diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/OperatorTypeRegistry.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/OperatorTypeRegistry.java new file mode 100644 index 000000000000..820d4d4dc700 --- /dev/null +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/operator/OperatorTypeRegistry.java @@ -0,0 +1,75 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.pinot.query.runtime.operator; + +import com.google.common.annotations.VisibleForTesting; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.ServiceLoader; +import javax.annotation.Nullable; + + +/** + * Registry of all known MSE {@link OperatorTypeDescriptor}s. Built-in types ({@link MultiStageOperator.Type} enum + * entries) are always present. Plugin-defined types are discovered at class-loading time via {@link ServiceLoader}: + * any jar on the classpath that ships a + * {@code META-INF/services/org.apache.pinot.query.runtime.operator.OperatorTypeDescriptor} file will have its + * descriptors automatically registered without configuration. + * + *

Thread-safe: the registry map is built once in a static initializer and never mutated afterward. + */ +public final class OperatorTypeRegistry { + + private static final Map ID_TO_DESCRIPTOR; + + static { + Map map = new HashMap<>(); + for (MultiStageOperator.Type builtIn : MultiStageOperator.Type.values()) { + map.put(builtIn.getId(), builtIn); + } + for (OperatorTypeDescriptor plugin : ServiceLoader.load(OperatorTypeDescriptor.class)) { + OperatorTypeDescriptor prev = map.put(plugin.getId(), plugin); + if (prev != null) { + throw new IllegalStateException( + "Duplicate operator type id " + plugin.getId() + ": " + prev.name() + " vs " + plugin.name()); + } + } + ID_TO_DESCRIPTOR = Collections.unmodifiableMap(map); + } + + private OperatorTypeRegistry() { + } + + /** + * Returns the descriptor registered for the given id, or {@code null} if no descriptor has that id. + * Built-in types (ids 0–15) are always present. Plugin types are available if their jar was on the classpath at + * startup. + */ + @Nullable + public static OperatorTypeDescriptor fromId(int id) { + return ID_TO_DESCRIPTOR.get(id); + } + + /** Returns the total number of registered descriptors (built-ins + plugins). */ + @VisibleForTesting + static int size() { + return ID_TO_DESCRIPTOR.size(); + } +} diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/MultiStageQueryStats.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/MultiStageQueryStats.java index b4f6ad02c769..29508fd18b01 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/MultiStageQueryStats.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/MultiStageQueryStats.java @@ -38,7 +38,8 @@ import javax.annotation.Nullable; import org.apache.commons.io.output.UnsynchronizedByteArrayOutputStream; import org.apache.pinot.common.datatable.StatMap; -import org.apache.pinot.query.runtime.operator.MultiStageOperator; +import org.apache.pinot.query.runtime.operator.OperatorTypeDescriptor; +import org.apache.pinot.query.runtime.operator.OperatorTypeRegistry; import org.apache.pinot.segment.spi.memory.DataBuffer; import org.apache.pinot.segment.spi.memory.PinotByteBuffer; import org.apache.pinot.segment.spi.memory.PinotInputStream; @@ -406,7 +407,7 @@ public abstract static class StageStats { *

* This list contains no null values. */ - protected final List _operatorTypes; + protected final List _operatorTypes; /** * The stats associated with the given operator index. *

@@ -421,7 +422,7 @@ private StageStats() { this(new ArrayList<>(), new ArrayList<>()); } - private StageStats(List operatorTypes, List> operatorStats) { + private StageStats(List operatorTypes, List> operatorStats) { Preconditions.checkArgument(operatorTypes.size() == operatorStats.size(), "Operator types and stats must have the same size (%s != %s)", operatorTypes.size(), operatorStats.size()); @@ -466,7 +467,7 @@ public StatMap getOperatorStats(int operatorIdx) { return _operatorStats.get(operatorIdx); } - public MultiStageOperator.Type getOperatorType(int index) { + public OperatorTypeDescriptor getOperatorType(int index) { return _operatorTypes.get(index); } @@ -482,8 +483,8 @@ public boolean isEmpty() { return _operatorTypes.isEmpty(); } - public void forEach(BiConsumer> consumer) { - Iterator typeIterator = _operatorTypes.iterator(); + public void forEach(BiConsumer> consumer) { + Iterator typeIterator = _operatorTypes.iterator(); Iterator> statIterator = _operatorStats.iterator(); while (typeIterator.hasNext()) { consumer.accept(typeIterator.next(), statIterator.next()); @@ -551,7 +552,7 @@ public String toString() { * never add entries for itself in upstream stats. */ public static class Closed extends StageStats { - public Closed(List operatorTypes, List> operatorStats) { + public Closed(List operatorTypes, List> operatorStats) { super(operatorTypes, operatorStats); } @@ -619,13 +620,13 @@ public static Closed deserialize(DataInput input) public static Closed deserialize(DataInput input, int numOperators) throws IOException { - List operatorTypes = new ArrayList<>(numOperators); + List operatorTypes = new ArrayList<>(numOperators); List> operatorStats = new ArrayList<>(numOperators); try { for (int i = 0; i < numOperators; i++) { int typeId = input.readByte(); - MultiStageOperator.Type type = MultiStageOperator.Type.fromId(typeId); + OperatorTypeDescriptor type = OperatorTypeRegistry.fromId(typeId); if (type == null) { throw new IllegalStateException( "Invalid operator type id " + typeId + " at index " + i + ". " + "Deserialized so far: " @@ -662,7 +663,7 @@ private Open() { /// /// @param statMap The stats for the operator to add. The ownership of this map will be transferred to this /// object, so the caller should not modify it after calling this method. - public Open addLastOperator(MultiStageOperator.Type type, StatMap statMap) { + public Open addLastOperator(OperatorTypeDescriptor type, StatMap statMap) { Preconditions.checkArgument(statMap.getKeyClass().equals(type.getStatKeyClass()), "Expected stats of class %s for type %s but found class %s", type.getStatKeyClass(), type, statMap.getKeyClass()); diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/MultiStageStatsTreeDecoder.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/MultiStageStatsTreeDecoder.java index 3ed8fa1437b1..2f9966b16193 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/MultiStageStatsTreeDecoder.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/MultiStageStatsTreeDecoder.java @@ -27,7 +27,8 @@ import java.util.Map; import org.apache.pinot.common.datatable.StatMap; import org.apache.pinot.common.proto.Worker; -import org.apache.pinot.query.runtime.operator.MultiStageOperator; +import org.apache.pinot.query.runtime.operator.OperatorTypeDescriptor; +import org.apache.pinot.query.runtime.operator.OperatorTypeRegistry; /** @@ -60,7 +61,7 @@ public DecodeFailedException(String message, Throwable cause) { */ public static StageStatsTreeNode decodeNode(Worker.StageStatsNode node) throws DecodeFailedException { - MultiStageOperator.Type type = MultiStageOperator.Type.fromId(node.getOperatorTypeId()); + OperatorTypeDescriptor type = OperatorTypeRegistry.fromId(node.getOperatorTypeId()); if (type == null) { throw new DecodeFailedException("Unknown operator type id: " + node.getOperatorTypeId()); } @@ -68,7 +69,7 @@ public static StageStatsTreeNode decodeNode(Worker.StageStatsNode node) try { statMap = deserializeStatMap(node.getStatMap(), type); } catch (IOException e) { - throw new DecodeFailedException("Failed to deserialize StatMap for operator type " + type, e); + throw new DecodeFailedException("Failed to deserialize StatMap for operator type " + type.name(), e); } List children = new ArrayList<>(node.getChildrenCount()); for (Worker.StageStatsNode child : node.getChildrenList()) { @@ -124,7 +125,7 @@ public static Decoded decode(Worker.MultiStageStatsTree proto) // The Type -> StatKey class relationship is preserved by MultiStageOperator.Type, but Java's type system can't // express the dependent type, so we suppress the resulting unchecked warning here. @SuppressWarnings({"unchecked", "rawtypes"}) - private static StatMap deserializeStatMap(ByteString bytes, MultiStageOperator.Type type) + private static StatMap deserializeStatMap(ByteString bytes, OperatorTypeDescriptor type) throws IOException { try (DataInputStream input = new DataInputStream(bytes.newInput())) { return StatMap.deserialize(input, (Class) type.getStatKeyClass()); diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/MultiStageStatsTreeEncoder.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/MultiStageStatsTreeEncoder.java index 1df9d118f82e..c9483c78367d 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/MultiStageStatsTreeEncoder.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/MultiStageStatsTreeEncoder.java @@ -28,6 +28,7 @@ import org.apache.pinot.common.proto.Worker; import org.apache.pinot.query.planner.plannode.PlanNode; import org.apache.pinot.query.runtime.operator.MultiStageOperator; +import org.apache.pinot.query.runtime.operator.OperatorTypeDescriptor; /** @@ -98,7 +99,7 @@ private static Worker.StageStatsNode encodeNode(MultiStageOperator op, builder.addChildren(encodeNode(child, openStats, cursor, planNodeIdResolver)); } int idx = cursor[0]++; - MultiStageOperator.Type type = openStats.getOperatorType(idx); + OperatorTypeDescriptor type = openStats.getOperatorType(idx); StatMap statMap = openStats.getOperatorStats(idx); builder.setOperatorTypeId(type.getId()); builder.setStatMap(serializeStatMap(statMap)); diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/StageStatsTreeNode.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/StageStatsTreeNode.java index 75f6cfbd0e2f..e52922eee786 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/StageStatsTreeNode.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/StageStatsTreeNode.java @@ -22,7 +22,7 @@ import java.util.List; import java.util.Objects; import org.apache.pinot.common.datatable.StatMap; -import org.apache.pinot.query.runtime.operator.MultiStageOperator; +import org.apache.pinot.query.runtime.operator.OperatorTypeDescriptor; /** @@ -47,12 +47,12 @@ public ShapeMismatchException(String message) { } } - private final MultiStageOperator.Type _type; + private final OperatorTypeDescriptor _type; private final List _planNodeIds; private final StatMap _statMap; private final List _children; - public StageStatsTreeNode(MultiStageOperator.Type type, List planNodeIds, StatMap statMap, + public StageStatsTreeNode(OperatorTypeDescriptor type, List planNodeIds, StatMap statMap, List children) { _type = type; _planNodeIds = List.copyOf(planNodeIds); @@ -60,7 +60,7 @@ public StageStatsTreeNode(MultiStageOperator.Type type, List planNodeId _children = List.copyOf(children); } - public MultiStageOperator.Type getType() { + public OperatorTypeDescriptor getType() { return _type; } @@ -86,8 +86,8 @@ public List getChildren() { @SuppressWarnings({"unchecked", "rawtypes"}) public void merge(StageStatsTreeNode other) throws ShapeMismatchException { - if (_type != other._type) { - throw new ShapeMismatchException("Operator type mismatch: " + _type + " vs " + other._type); + if (_type.getId() != other._type.getId()) { + throw new ShapeMismatchException("Operator type mismatch: " + _type.name() + " vs " + other._type.name()); } if (_children.size() != other._children.size()) { throw new ShapeMismatchException("Children count mismatch for " + _type + ": " + _children.size() + " vs " @@ -110,13 +110,13 @@ public void merge(StageStatsTreeNode other) * tree shape should use this {@link StageStatsTreeNode} directly. */ public MultiStageQueryStats.StageStats.Closed flattenInorder() { - List types = new ArrayList<>(); + List types = new ArrayList<>(); List> stats = new ArrayList<>(); flattenInto(types, stats); return new MultiStageQueryStats.StageStats.Closed(types, stats); } - private void flattenInto(List types, List> stats) { + private void flattenInto(List types, List> stats) { for (StageStatsTreeNode child : _children) { child.flattenInto(types, stats); } @@ -133,7 +133,7 @@ public boolean equals(Object o) { return false; } StageStatsTreeNode that = (StageStatsTreeNode) o; - return _type == that._type + return _type.getId() == that._type.getId() && Objects.equals(_planNodeIds, that._planNodeIds) && Objects.equals(_statMap, that._statMap) && Objects.equals(_children, that._children); diff --git a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/OperatorTypeRegistryTest.java b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/OperatorTypeRegistryTest.java new file mode 100644 index 000000000000..3f2e54ba2b6b --- /dev/null +++ b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/OperatorTypeRegistryTest.java @@ -0,0 +1,59 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.pinot.query.runtime.operator; + +import org.testng.Assert; +import org.testng.annotations.Test; + + +public class OperatorTypeRegistryTest { + + @Test + public void testAllBuiltInTypesRegistered() { + for (MultiStageOperator.Type builtIn : MultiStageOperator.Type.values()) { + OperatorTypeDescriptor descriptor = OperatorTypeRegistry.fromId(builtIn.getId()); + Assert.assertNotNull(descriptor, "Built-in type " + builtIn + " (id=" + builtIn.getId() + ") not in registry"); + Assert.assertSame(descriptor, builtIn, "Registry entry for id=" + builtIn.getId() + " should be the enum constant"); + } + } + + @Test + public void testRegistryContainsExactlyBuiltIns() { + // No extra plugins on the test classpath: registry size == Type.values().length + Assert.assertEquals(OperatorTypeRegistry.size(), MultiStageOperator.Type.values().length, + "Registry should contain exactly the built-in types when no plugins are present"); + } + + @Test + public void testFromIdUnknownReturnsNull() { + // Pick an id well outside the current built-in range + Assert.assertNull(OperatorTypeRegistry.fromId(9999)); + Assert.assertNull(OperatorTypeRegistry.fromId(-1)); + } + + @Test + public void testBuiltInDescriptorMethodsDelegateToEnum() { + MultiStageOperator.Type aggregate = MultiStageOperator.Type.AGGREGATE; + OperatorTypeDescriptor descriptor = OperatorTypeRegistry.fromId(aggregate.getId()); + Assert.assertNotNull(descriptor); + Assert.assertEquals(descriptor.getId(), aggregate.getId()); + Assert.assertEquals(descriptor.name(), aggregate.name()); + Assert.assertEquals(descriptor.getStatKeyClass(), aggregate.getStatKeyClass()); + } +} From a6acb08008e32012cd1e04f102c26491a99d23e6 Mon Sep 17 00:00:00 2001 From: Gonzalo Ortiz <1913993+gortiz@users.noreply.github.com> Date: Tue, 12 May 2026 16:22:05 +0200 Subject: [PATCH 19/31] Fix checkstyle: wrap long assertion message line in OperatorTypeRegistryTest Co-Authored-By: Claude Sonnet 4.6 --- .../pinot/query/runtime/operator/OperatorTypeRegistryTest.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/OperatorTypeRegistryTest.java b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/OperatorTypeRegistryTest.java index 3f2e54ba2b6b..13237eed5ad2 100644 --- a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/OperatorTypeRegistryTest.java +++ b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/operator/OperatorTypeRegistryTest.java @@ -29,7 +29,8 @@ public void testAllBuiltInTypesRegistered() { for (MultiStageOperator.Type builtIn : MultiStageOperator.Type.values()) { OperatorTypeDescriptor descriptor = OperatorTypeRegistry.fromId(builtIn.getId()); Assert.assertNotNull(descriptor, "Built-in type " + builtIn + " (id=" + builtIn.getId() + ") not in registry"); - Assert.assertSame(descriptor, builtIn, "Registry entry for id=" + builtIn.getId() + " should be the enum constant"); + Assert.assertSame(descriptor, builtIn, + "Registry entry for id=" + builtIn.getId() + " should be the enum constant"); } } From 5b329b5b205c32cd37a07dde6f13a8770023978b Mon Sep 17 00:00:00 2001 From: Gonzalo Ortiz <1913993+gortiz@users.noreply.github.com> Date: Tue, 12 May 2026 16:40:11 +0200 Subject: [PATCH 20/31] Add merge-resilience tests; rename stream stats option to streamStats MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Three new tests in StreamingQuerySessionTest verify that query completion is not blocked when stats cannot be decoded or merged: - Unknown operator type id (DecodeFailedException) → mergeFailed, latch drains - Shape mismatch between workers (ShapeMismatchException) → mergeFailed, first worker's stats preserved - Corrupted/truncated stat bytes (EOFException → DecodeFailedException) → mergeFailed, latch drains Rename useStreamStatsReporting/CONFIG_OF_USE_STREAM_STATS_REPORTING to streamStats/CONFIG_OF_STREAM_STATS for brevity across all call sites. Co-Authored-By: Claude Sonnet 4.6 --- .../MultiStageBrokerRequestHandler.java | 14 +-- .../utils/config/QueryOptionsUtils.java | 8 +- .../StreamStatsReportingIntegrationTest.java | 14 +-- .../service/dispatch/QueryDispatcher.java | 4 +- .../streaming/StreamingQuerySessionTest.java | 110 ++++++++++++++++++ .../pinot/spi/utils/CommonConstants.java | 9 +- 6 files changed, 134 insertions(+), 25 deletions(-) diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/MultiStageBrokerRequestHandler.java b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/MultiStageBrokerRequestHandler.java index 617b61e40e71..daaaf2187f9e 100644 --- a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/MultiStageBrokerRequestHandler.java +++ b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/MultiStageBrokerRequestHandler.java @@ -140,7 +140,7 @@ public class MultiStageBrokerRequestHandler extends BaseBrokerRequestHandler { private final Set _defaultDisabledPlannerRules; protected final long _extraPassiveTimeoutMs; protected final boolean _enableQueryFingerprinting; - private final boolean _useStreamStatsReporting; + private final boolean _streamStats; protected final PinotMeter _stagesStartedMeter = BrokerMeter.MSE_STAGES_STARTED.getGlobalMeter(); protected final PinotMeter _stagesFinishedMeter = BrokerMeter.MSE_STAGES_COMPLETED.getGlobalMeter(); @@ -206,9 +206,9 @@ tlsConfig, isQueryCancellationEnabled(), cancelTimeout, dispatchKeepAliveTimeMs, _enableQueryFingerprinting = _config.getProperty( CommonConstants.Broker.CONFIG_OF_BROKER_ENABLE_QUERY_FINGERPRINTING, CommonConstants.Broker.DEFAULT_BROKER_ENABLE_QUERY_FINGERPRINTING); - _useStreamStatsReporting = _config.getProperty( - CommonConstants.Broker.CONFIG_OF_USE_STREAM_STATS_REPORTING, - CommonConstants.Broker.DEFAULT_USE_STREAM_STATS_REPORTING); + _streamStats = _config.getProperty( + CommonConstants.Broker.CONFIG_OF_STREAM_STATS, + CommonConstants.Broker.DEFAULT_STREAM_STATS); } @Override @@ -658,10 +658,10 @@ private BrokerResponse query(QueryEnvironment.CompiledQuery query, long requestI // Inject the cluster-default stream-stats mode unless the query already overrides it. Map effectiveOptions = query.getOptions(); - if (_useStreamStatsReporting && !effectiveOptions.containsKey( - CommonConstants.Broker.Request.QueryOptionKey.USE_STREAM_STATS_REPORTING)) { + if (_streamStats && !effectiveOptions.containsKey( + CommonConstants.Broker.Request.QueryOptionKey.STREAM_STATS)) { effectiveOptions = new HashMap<>(effectiveOptions); - effectiveOptions.put(CommonConstants.Broker.Request.QueryOptionKey.USE_STREAM_STATS_REPORTING, "true"); + effectiveOptions.put(CommonConstants.Broker.Request.QueryOptionKey.STREAM_STATS, "true"); } QueryDispatcher.QueryResult queryResults; diff --git a/pinot-common/src/main/java/org/apache/pinot/common/utils/config/QueryOptionsUtils.java b/pinot-common/src/main/java/org/apache/pinot/common/utils/config/QueryOptionsUtils.java index 3510b56c97ab..7fece8909bcc 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/utils/config/QueryOptionsUtils.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/utils/config/QueryOptionsUtils.java @@ -529,11 +529,11 @@ public static boolean isUsePhysicalOptimizer(Map queryOptions, b } /** - * Reads the {@code useStreamStatsReporting} query option that opts a single query into the {@code SubmitWithStream} - * dispatch path. See {@link QueryOptionKey#USE_STREAM_STATS_REPORTING}. + * Reads the {@code streamStats} query option that opts a single query into the {@code SubmitWithStream} + * dispatch path. See {@link QueryOptionKey#STREAM_STATS}. */ - public static boolean isUseStreamStatsReporting(Map queryOptions, boolean defaultValue) { - String option = queryOptions.get(QueryOptionKey.USE_STREAM_STATS_REPORTING); + public static boolean isStreamStats(Map queryOptions, boolean defaultValue) { + String option = queryOptions.get(QueryOptionKey.STREAM_STATS); return option != null ? Boolean.parseBoolean(option) : defaultValue; } diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/StreamStatsReportingIntegrationTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/StreamStatsReportingIntegrationTest.java index 7a1f8b368fe2..acf6604e52e0 100644 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/StreamStatsReportingIntegrationTest.java +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/StreamStatsReportingIntegrationTest.java @@ -42,18 +42,18 @@ * Integration tests for the {@code SubmitWithStream} stats-reporting path. Verifies that: *

    *
  1. Queries run to completion with correct results when stream stats are enabled per-query via the - * {@code useStreamStatsReporting} query option.
  2. + * {@code streamStats} query option. *
  3. The broker response contains a non-null {@code streamStatsCoverage} array indexed by stage id with * {@code responded} > 0 and {@code missing} = {@code mergeFailed} = 0 on the success path.
  4. *
  5. An N-ary set operation (three-way UNION) produces a correct tree-shaped stats payload — regression coverage * for the known loss in the legacy flat-binary / inorder format when a set op has more than two inputs.
  6. - *
  7. The cluster-level config ({@code pinot.broker.multistage.use.stream.stats.reporting}) activates stream mode + *
  8. The cluster-level config ({@code pinot.broker.mse.stream.stats}) activates stream mode * for all queries without a per-query option.
  9. *
*/ public class StreamStatsReportingIntegrationTest extends BaseClusterIntegrationTestSet { - private static final String STREAM_OPTION = "useStreamStatsReporting=true"; + private static final String STREAM_OPTION = "streamStats=true"; @BeforeClass public void setUp() @@ -95,7 +95,7 @@ public boolean useMultiStageQueryEngine() { // ─── helpers ────────────────────────────────────────────────────────────────── /** - * Posts a multi-stage query with the {@code useStreamStatsReporting=true} option. + * Posts a multi-stage query with the {@code streamStats=true} option. */ private JsonNode postWithStreamStats(@Language("sql") String sql) throws Exception { @@ -197,7 +197,7 @@ public void testThreeWayUnion() } /** - * Verifies that the cluster-level config {@code pinot.broker.multistage.use.stream.stats.reporting=true} + * Verifies that the cluster-level config {@code pinot.broker.mse.stream.stats=true} * activates stream mode for queries that do not carry the per-query option. The broker is restarted with the config * set for this test class via {@link #overrideBrokerConf}. */ @@ -205,7 +205,7 @@ public void testThreeWayUnion() public void testClusterLevelConfigActivatesStreamMode() throws Exception { // Post WITHOUT the per-query option — stream mode should still be active because the broker was started - // with CONFIG_OF_USE_STREAM_STATS_REPORTING=true in overrideBrokerConf. + // with CONFIG_OF_STREAM_STATS=true in overrideBrokerConf. @Language("sql") String sql = "SELECT COUNT(*) FROM mytable LIMIT 1"; JsonNode response = postQuery(sql); @@ -218,7 +218,7 @@ public void testClusterLevelConfigActivatesStreamMode() @Override protected void overrideBrokerConf(PinotConfiguration brokerConf) { // Enable stream-mode cluster-wide so testClusterLevelConfigActivatesStreamMode can run without a per-query option. - brokerConf.setProperty(CommonConstants.Broker.CONFIG_OF_USE_STREAM_STATS_REPORTING, true); + brokerConf.setProperty(CommonConstants.Broker.CONFIG_OF_STREAM_STATS, true); super.overrideBrokerConf(brokerConf); } diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/QueryDispatcher.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/QueryDispatcher.java index 265a8a1d8459..7d784946eae6 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/QueryDispatcher.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/QueryDispatcher.java @@ -166,7 +166,7 @@ public void start() { public QueryResult submitAndReduce(RequestContext context, DispatchableSubPlan dispatchableSubPlan, long timeoutMs, Map queryOptions) throws Exception { - if (QueryOptionsUtils.isUseStreamStatsReporting(queryOptions, false)) { + if (QueryOptionsUtils.isStreamStats(queryOptions, false)) { return submitAndReduceWithStream(context, dispatchableSubPlan, timeoutMs, queryOptions); } long requestId = context.getRequestId(); @@ -206,7 +206,7 @@ public QueryResult submitAndReduce(RequestContext context, DispatchableSubPlan d /// stats before building the final result. /// /// Mixed-version policy. No automatic fallback to the unary {@link #submit} path. Enabling - /// {@link CommonConstants.Broker.Request.QueryOptionKey#USE_STREAM_STATS_REPORTING} requires every server in the + /// {@link CommonConstants.Broker.Request.QueryOptionKey#STREAM_STATS} requires every server in the /// cluster to implement {@code SubmitWithStream}; if any server returns {@code UNIMPLEMENTED} or any other /// transport error during dispatch, {@link #submitWithStream} surfaces the throwable through the ack queue, /// {@link #processResults} throws, and this method fans out cancel via the session before propagating the failure. diff --git a/pinot-query-runtime/src/test/java/org/apache/pinot/query/service/dispatch/streaming/StreamingQuerySessionTest.java b/pinot-query-runtime/src/test/java/org/apache/pinot/query/service/dispatch/streaming/StreamingQuerySessionTest.java index 0abc9d7d056f..393f9ff90d25 100644 --- a/pinot-query-runtime/src/test/java/org/apache/pinot/query/service/dispatch/streaming/StreamingQuerySessionTest.java +++ b/pinot-query-runtime/src/test/java/org/apache/pinot/query/service/dispatch/streaming/StreamingQuerySessionTest.java @@ -189,6 +189,93 @@ public void testConcurrentReports() Assert.assertEquals(merged.getLong(AggregateOperator.StatKey.EMITTED_ROWS), n); } + /** + * An opchain whose stats tree contains an operator type id absent from {@link + * org.apache.pinot.query.runtime.operator.OperatorTypeRegistry} (simulating a newer server carrying a plugin + * the broker has not installed) must not abort the query. The session marks the stage merge-failed and drains + * the completion latch so the query result is returned normally. + */ + @Test + public void testDecodeFailedUnknownTypeIdDoesNotAbortQuery() + throws Exception { + StreamingQuerySession session = new StreamingQuerySession(1L, 2); + + session.recordOpChainComplete(buildOpChainComplete(0, 0, 1, 5)); + // Type id 9999 is not in the registry — decode throws DecodeFailedException. + session.recordOpChainComplete(buildOpChainCompleteWithTypeId(0, 1, 9999, ByteString.EMPTY)); + + Assert.assertTrue(session.awaitCompletion(1, TimeUnit.SECONDS), + "query should complete despite unknown operator type id"); + StreamingQuerySession.Coverage coverage = session.snapshotCoverage(); + Assert.assertEquals((int) coverage.getRespondedByStage().getOrDefault(0, 0), 1, + "only the successfully decoded opchain should be counted as responded"); + Assert.assertEquals((int) coverage.getMergeFailedByStage().getOrDefault(0, 0), 1, + "the unknown-type opchain should be counted as merge-failed"); + Assert.assertNotNull(coverage.getStageAccumulator().get(0), + "first worker's valid stats should remain in the accumulator"); + } + + /** + * When two workers report the same stage with incompatible tree shapes (different operator types), the second + * merge throws {@link StageStatsTreeNode.ShapeMismatchException}. The first worker's stats are preserved in the + * accumulator, the stage is marked merge-failed, and the query still completes. + * + *

Note: the second worker still counts toward {@code respondedByStage} because its payload decoded + * successfully — the failure occurred at merge time, not decode time. + */ + @Test + public void testShapeMismatchDoesNotAbortQuery() + throws Exception { + StreamingQuerySession session = new StreamingQuerySession(1L, 2); + + // Worker 0 reports AGGREGATE at stage 0; populates the accumulator. + session.recordOpChainComplete(buildOpChainComplete(0, 0, 1, 5)); + + // Worker 1 reports HASH_JOIN at stage 0 with an empty-but-valid stat map. + // The type id mismatch triggers ShapeMismatchException inside merge(). + session.recordOpChainComplete( + buildOpChainCompleteWithTypeId(0, 1, MultiStageOperator.Type.HASH_JOIN.getId(), emptyStatBytes())); + + Assert.assertTrue(session.awaitCompletion(1, TimeUnit.SECONDS), + "query should complete despite shape mismatch"); + StreamingQuerySession.Coverage coverage = session.snapshotCoverage(); + // Both workers decoded successfully, so both count as responded. + Assert.assertEquals((int) coverage.getRespondedByStage().getOrDefault(0, 0), 2, + "both workers decoded successfully so both count as responded"); + Assert.assertEquals((int) coverage.getMergeFailedByStage().getOrDefault(0, 0), 1, + "second worker's shape-mismatched opchain should be counted as merge-failed"); + Assert.assertNotNull(coverage.getStageAccumulator().get(0), + "first worker's stats should remain in the accumulator after the failed merge"); + } + + /** + * When a stat-map payload is unreadable (truncated bytes — not enough data for even the count header), the + * decoder wraps the resulting {@link java.io.EOFException} as a + * {@link org.apache.pinot.query.runtime.plan.MultiStageStatsTreeDecoder.DecodeFailedException}. The session + * absorbs the failure, marks the stage merge-failed, and drains the latch. The query result is unaffected. + */ + @Test + public void testCorruptedStatBytesDoesNotAbortQuery() + throws Exception { + StreamingQuerySession session = new StreamingQuerySession(1L, 2); + + session.recordOpChainComplete(buildOpChainComplete(0, 0, 1, 5)); + // ByteString.EMPTY has 0 bytes; StatMap.deserialize() tries to readByte() for the count and throws + // EOFException, which the decoder re-throws as DecodeFailedException. + session.recordOpChainComplete( + buildOpChainCompleteWithTypeId(0, 1, MultiStageOperator.Type.AGGREGATE.getId(), ByteString.EMPTY)); + + Assert.assertTrue(session.awaitCompletion(1, TimeUnit.SECONDS), + "query should complete despite corrupted stat bytes"); + StreamingQuerySession.Coverage coverage = session.snapshotCoverage(); + Assert.assertEquals((int) coverage.getRespondedByStage().getOrDefault(0, 0), 1, + "only the successfully decoded opchain should be counted as responded"); + Assert.assertEquals((int) coverage.getMergeFailedByStage().getOrDefault(0, 0), 1, + "the opchain with corrupted stat bytes should be counted as merge-failed"); + Assert.assertNotNull(coverage.getStageAccumulator().get(0), + "first worker's valid stats should remain in the accumulator"); + } + // ---- helpers ---- private static Worker.OpChainComplete buildOpChainComplete(int stageId, int workerId, int planNodeId, long emitted) @@ -220,6 +307,29 @@ private static Worker.OpChainComplete buildErrorOpChainComplete(int stageId, int .build(); } + private static Worker.OpChainComplete buildOpChainCompleteWithTypeId( + int stageId, int workerId, int typeId, ByteString statBytes) { + Worker.StageStatsNode rootNode = Worker.StageStatsNode.newBuilder() + .setOperatorTypeId(typeId) + .setStatMap(statBytes) + .build(); + return Worker.OpChainComplete.newBuilder() + .setStageId(stageId) + .setWorkerId(workerId) + .setSuccess(true) + .setStats(Worker.MultiStageStatsTree.newBuilder() + .setCurrentStageId(stageId) + .setCurrentStage(rootNode) + .build()) + .build(); + } + + /** Serialised empty {@link StatMap} — one zero byte representing zero entries. */ + private static ByteString emptyStatBytes() + throws IOException { + return serialize(new StatMap<>(AggregateOperator.StatKey.class)); + } + private static ByteString serialize(StatMap statMap) throws IOException { try (ByteArrayOutputStream baos = new ByteArrayOutputStream(); diff --git a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/CommonConstants.java b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/CommonConstants.java index 98339ac4e45d..57c377443d67 100644 --- a/pinot-spi/src/main/java/org/apache/pinot/spi/utils/CommonConstants.java +++ b/pinot-spi/src/main/java/org/apache/pinot/spi/utils/CommonConstants.java @@ -528,11 +528,10 @@ public static class Broker { /// Cluster-level default for stream-mode stats reporting. When {@code true} the broker opens a /// {@code SubmitWithStream} bidi RPC for every multi-stage query instead of the legacy unary Submit, enabling /// reliable per-operator stats delivery even on the error path. Individual queries may override this default - /// via the {@link Request.QueryOptionKey#USE_STREAM_STATS_REPORTING} query option. Requires all servers to + /// via the {@link Request.QueryOptionKey#STREAM_STATS} query option. Requires all servers to /// implement the {@code SubmitWithStream} RPC; enabling it on a mixed-version cluster will cause query failures. - public static final String CONFIG_OF_USE_STREAM_STATS_REPORTING = - "pinot.broker.mse.use.stream.stats.reporting"; - public static final boolean DEFAULT_USE_STREAM_STATS_REPORTING = false; + public static final String CONFIG_OF_STREAM_STATS = "pinot.broker.mse.stream.stats"; + public static final boolean DEFAULT_STREAM_STATS = false; public static final String CONFIG_OF_USE_FIXED_REPLICA = "pinot.broker.use.fixed.replica"; public static final boolean DEFAULT_USE_FIXED_REPLICA = false; @@ -753,7 +752,7 @@ public static class QueryOptionKey { * other transport error) during dispatch, the broker cancels the query and surfaces the error to the * client. */ - public static final String USE_STREAM_STATS_REPORTING = "useStreamStatsReporting"; + public static final String STREAM_STATS = "streamStats"; /** * If set, changes the explain behavior in multi-stage engine. * From da81c97057e3a75c528344f41c9364e2a8285105 Mon Sep 17 00:00:00 2001 From: Gonzalo Ortiz <1913993+gortiz@users.noreply.github.com> Date: Tue, 12 May 2026 17:17:49 +0200 Subject: [PATCH 21/31] [query-planner] PlannerContext implements Context; expose per-query options to Calcite rules MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit PlannerContext now implements org.apache.calcite.plan.Context so that Calcite rules registered via the RuleSetCustomizer SPI can gate on per-query options without reflection: PlannerContext ctx = call.getPlanner().getContext().unwrap(PlannerContext.class); if (ctx != null) { String v = ctx.getOptions().get("myOption"); } Changes: - PlannerContext implements Context; overrides unwrap(): - clazz.isInstance(this) → returns this (covers PlannerContext and Context) - clazz.isInstance(_envConfig) → delegates to envConfig (QueryEnvironment.Config backward compat for PinotLogicalAggregateRule / PinotAggregateExchangeNodeInsertRule) - otherwise → null - Both LogicalPlanner constructors now receive `this` instead of Contexts.EMPTY_CONTEXT / Contexts.of(envConfig). - PlannerContext.forTesting(options, envConfig) static factory + package-private two-arg constructor (no-op planners) for unit tests that don't need a full QueryEnvironment. - close() guards against null _planner (test constructor sets it to null). - PlannerContextTest: 6 TestNG tests covering unwrap contract and planner delegation. Co-Authored-By: Claude Sonnet 4.6 --- .../pinot/query/context/PlannerContext.java | 69 ++++++++++++-- .../query/context/PlannerContextTest.java | 92 +++++++++++++++++++ 2 files changed, 152 insertions(+), 9 deletions(-) create mode 100644 pinot-query-planner/src/test/java/org/apache/pinot/query/context/PlannerContextTest.java diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/query/context/PlannerContext.java b/pinot-query-planner/src/main/java/org/apache/pinot/query/context/PlannerContext.java index 6e5abd8f660d..64098024a4e7 100644 --- a/pinot-query-planner/src/main/java/org/apache/pinot/query/context/PlannerContext.java +++ b/pinot-query-planner/src/main/java/org/apache/pinot/query/context/PlannerContext.java @@ -18,13 +18,15 @@ */ package org.apache.pinot.query.context; +import com.google.common.annotations.VisibleForTesting; import java.util.Collections; import java.util.HashMap; import java.util.Map; import javax.annotation.Nullable; -import org.apache.calcite.plan.Contexts; +import org.apache.calcite.plan.Context; import org.apache.calcite.plan.RelOptPlanner; import org.apache.calcite.plan.hep.HepProgram; +import org.apache.calcite.plan.hep.HepProgramBuilder; import org.apache.calcite.prepare.PlannerImpl; import org.apache.calcite.prepare.Prepare; import org.apache.calcite.rel.RelDistributionTraitDef; @@ -38,12 +40,16 @@ /** - * PlannerContext is an object that holds all contextual information during planning phase. + * Holds all per-query contextual information used during the planning phase. * - * TODO: currently we don't support option or query rewrite. - * It is used to hold per query context for query planning, which cannot be shared across queries. + *

This class implements {@link Context} so that Calcite rules can retrieve it directly from the + * planner: {@code call.getPlanner().getContext().unwrap(PlannerContext.class)}. Both the opt planner + * and the trait planner expose this instance as their context. + * + *

Callers may also unwrap {@link QueryEnvironment.Config} to access broker-wide defaults: + * {@code call.getPlanner().getContext().unwrap(QueryEnvironment.Config.class)}. */ -public class PlannerContext implements AutoCloseable { +public class PlannerContext implements AutoCloseable, Context { private final PlannerImpl _planner; private final SqlValidator _validator; @@ -63,11 +69,11 @@ public PlannerContext(FrameworkConfig config, Prepare.CatalogReader catalogReade SqlExplainFormat sqlExplainFormat, @Nullable PhysicalPlannerContext physicalPlannerContext) { _planner = new PlannerImpl(config); _validator = new Validator(config.getOperatorTable(), catalogReader, typeFactory); - _relOptPlanner = new LogicalPlanner(optProgram, Contexts.EMPTY_CONTEXT, config.getTraitDefs()); - _relTraitPlanner = new LogicalPlanner(traitProgram, Contexts.of(envConfig), - Collections.singletonList(RelDistributionTraitDef.INSTANCE)); _options = options; _envConfig = envConfig; + _relOptPlanner = new LogicalPlanner(optProgram, this, config.getTraitDefs()); + _relTraitPlanner = new LogicalPlanner(traitProgram, this, + Collections.singletonList(RelDistributionTraitDef.INSTANCE)); _plannerOutput = new HashMap<>(); _sqlExplainFormat = sqlExplainFormat; _physicalPlannerContext = physicalPlannerContext; @@ -97,9 +103,54 @@ public QueryEnvironment.Config getEnvConfig() { return _envConfig; } + /** + * Test factory: creates a minimal {@link PlannerContext} without going through + * {@link QueryEnvironment}, suitable for unit tests that verify rule gating logic. + */ + @VisibleForTesting + public static PlannerContext forTesting(Map options, QueryEnvironment.Config envConfig) { + return new PlannerContext(options, envConfig); + } + + /** + * Minimal constructor for use in unit tests. Creates no-op planners backed by an empty HEP program. + */ + @VisibleForTesting + PlannerContext(Map options, QueryEnvironment.Config envConfig) { + _planner = null; + _validator = null; + _options = options; + _envConfig = envConfig; + HepProgram emptyProgram = new HepProgramBuilder().build(); + _relOptPlanner = new LogicalPlanner(emptyProgram, this); + _relTraitPlanner = new LogicalPlanner(emptyProgram, this); + _plannerOutput = new HashMap<>(); + _sqlExplainFormat = null; + _physicalPlannerContext = null; + } + + /** + * Unwraps this context. Returns {@code this} when asked for {@link PlannerContext} or + * {@link Context}, and delegates to {@link #_envConfig} when asked for + * {@link QueryEnvironment.Config} so that existing rules remain compatible. + */ + @Override + @Nullable + public C unwrap(Class clazz) { + if (clazz.isInstance(this)) { + return clazz.cast(this); + } + if (clazz.isInstance(_envConfig)) { + return clazz.cast(_envConfig); + } + return null; + } + @Override public void close() { - _planner.close(); + if (_planner != null) { + _planner.close(); + } } public Map getPlannerOutput() { diff --git a/pinot-query-planner/src/test/java/org/apache/pinot/query/context/PlannerContextTest.java b/pinot-query-planner/src/test/java/org/apache/pinot/query/context/PlannerContextTest.java new file mode 100644 index 000000000000..89d02af0b80f --- /dev/null +++ b/pinot-query-planner/src/test/java/org/apache/pinot/query/context/PlannerContextTest.java @@ -0,0 +1,92 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.pinot.query.context; + +import java.util.Map; +import org.apache.calcite.plan.Context; +import org.apache.pinot.query.QueryEnvironment; +import org.testng.Assert; +import org.testng.annotations.Test; + +import static org.mockito.Mockito.mock; + + +/** + * Unit tests for {@link PlannerContext} as a Calcite {@link Context}. + * + *

Verifies that Calcite rules can retrieve both {@link PlannerContext} and the backward-compatible + * {@link QueryEnvironment.Config} via {@code unwrap()}, and that the opt/trait planners expose the + * same context. + */ +public class PlannerContextTest { + + @Test + public void testUnwrapReturnsSelfForPlannerContext() { + QueryEnvironment.Config config = mock(QueryEnvironment.Config.class); + PlannerContext ctx = PlannerContext.forTesting(Map.of("k", "v"), config); + + Assert.assertSame(ctx.unwrap(PlannerContext.class), ctx); + } + + @Test + public void testUnwrapReturnsSelfForContextInterface() { + QueryEnvironment.Config config = mock(QueryEnvironment.Config.class); + PlannerContext ctx = PlannerContext.forTesting(Map.of(), config); + + Assert.assertSame(ctx.unwrap(Context.class), ctx); + } + + @Test + public void testUnwrapDelegatesToEnvConfigForBackwardCompat() { + QueryEnvironment.Config config = mock(QueryEnvironment.Config.class); + PlannerContext ctx = PlannerContext.forTesting(Map.of(), config); + + Assert.assertSame(ctx.unwrap(QueryEnvironment.Config.class), config, + "unwrap(QueryEnvironment.Config) must return the configured envConfig for backward compatibility"); + } + + @Test + public void testUnwrapReturnsNullForUnknownType() { + QueryEnvironment.Config config = mock(QueryEnvironment.Config.class); + PlannerContext ctx = PlannerContext.forTesting(Map.of(), config); + + Assert.assertNull(ctx.unwrap(String.class)); + } + + @Test + public void testOptPlannerContextExposesPlannerContext() { + QueryEnvironment.Config config = mock(QueryEnvironment.Config.class); + PlannerContext ctx = PlannerContext.forTesting(Map.of("opt", "true"), config); + + PlannerContext fromPlanner = ctx.getRelOptPlanner().getContext().unwrap(PlannerContext.class); + Assert.assertSame(fromPlanner, ctx, + "opt planner context should expose PlannerContext via unwrap"); + } + + @Test + public void testTraitPlannerContextExposesEnvConfigForBackwardCompat() { + QueryEnvironment.Config config = mock(QueryEnvironment.Config.class); + PlannerContext ctx = PlannerContext.forTesting(Map.of(), config); + + QueryEnvironment.Config fromPlanner = + ctx.getRelTraitPlanner().getContext().unwrap(QueryEnvironment.Config.class); + Assert.assertSame(fromPlanner, config, + "trait planner context must still expose QueryEnvironment.Config for existing rules"); + } +} From adf2bb05537aa5e630062996094407e2188177b4 Mon Sep 17 00:00:00 2001 From: Gonzalo Ortiz <1913993+gortiz@users.noreply.github.com> Date: Wed, 13 May 2026 14:12:20 +0200 Subject: [PATCH 22/31] Fix StreamingDispatchObserver DONE handler to drain latch when server reports fewer opchains than expected MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit When a server sends submit_ack(STATUS_ERROR) followed by ServerDone without any opchains, the previous DONE handler only unregistered the stream but never decremented the completion latch — causing awaitCompletion to block until the query deadline rather than returning promptly on submission failure. The fix: compute remaining = expected - reported; if positive, call recordStreamError so the latch drains immediately and fan-out cancel fires (the query is doomed if a server couldn't start). After draining, advance _opChainsReportedForThisServer to _expectedOpChainsForThisServer so a subsequent onError (stream reset after DONE delivery) cannot double-drain the latch. Also added: - Javadoc on _opChainsReportedForThisServer documenting the single-threaded gRPC callback invariant (no extra synchronization needed). - Regression test: STATUS_ERROR ack + DONE with 0 reported opchains → awaitCompletion returns within 1 s. - Regression test: 1 of 3 opchains reported + DONE → remaining 2 counts drained correctly. Co-Authored-By: Claude Sonnet 4.6 --- .../streaming/StreamingDispatchObserver.java | 17 +++++- .../StreamingDispatchObserverTest.java | 60 +++++++++++++++++++ 2 files changed, 76 insertions(+), 1 deletion(-) diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/streaming/StreamingDispatchObserver.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/streaming/StreamingDispatchObserver.java index 8c1464bd300a..f517e0ab5a7b 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/streaming/StreamingDispatchObserver.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/streaming/StreamingDispatchObserver.java @@ -61,6 +61,9 @@ public class StreamingDispatchObserver /** * Counts how many opchains we've already drained from the session for this server, so an onError doesn't * double-drain after some opchains already reported successfully. + * + *

Accessed only from gRPC inbound callbacks ({@code onNext}, {@code onError}, {@code onCompleted}), which gRPC + * serializes per stream — no additional synchronization is needed. */ private int _opChainsReportedForThisServer = 0; @@ -112,7 +115,19 @@ public void onNext(Worker.ServerToBroker message) { _opChainsReportedForThisServer++; break; case DONE: - _session.unregisterStream(this); + int remaining = Math.max(0, _expectedOpChainsForThisServer - _opChainsReportedForThisServer); + if (remaining > 0) { + // Server declared done without all expected opchains reported (e.g. after STATUS_ERROR ack). + // Drain the latch so awaitCompletion unblocks promptly rather than waiting until query deadline. + // Intentionally reuses the error path: if this server couldn't execute its opchains the whole + // query is doomed, so canceling other servers is correct. + _session.recordStreamError(this, null, remaining); + // Advance the reported counter so a subsequent onError (stream reset after DONE) sees remaining==0 + // and does not double-drain the latch. + _opChainsReportedForThisServer = _expectedOpChainsForThisServer; + } else { + _session.unregisterStream(this); + } break; case PAYLOAD_NOT_SET: default: diff --git a/pinot-query-runtime/src/test/java/org/apache/pinot/query/service/dispatch/streaming/StreamingDispatchObserverTest.java b/pinot-query-runtime/src/test/java/org/apache/pinot/query/service/dispatch/streaming/StreamingDispatchObserverTest.java index 79bc59fbb75f..658e193eeac0 100644 --- a/pinot-query-runtime/src/test/java/org/apache/pinot/query/service/dispatch/streaming/StreamingDispatchObserverTest.java +++ b/pinot-query-runtime/src/test/java/org/apache/pinot/query/service/dispatch/streaming/StreamingDispatchObserverTest.java @@ -128,6 +128,66 @@ public void testCancelEmitsBrokerToServerCancel() Assert.assertEquals(sent.getCancel().getRequestId(), 42L); } + /** + * STATUS_ERROR ack followed by ServerDone with no opchains: the DONE handler must drain the remaining expected count + * from the latch so {@link StreamingQuerySession#awaitCompletion} returns promptly, not at the query deadline. + */ + @Test + public void testDoneWithNoReportedOpChainsAfterErrorAckDrainsLatch() + throws Exception { + StreamingQuerySession session = new StreamingQuerySession(1L, 2); + AtomicReference ackResponse = new AtomicReference<>(); + AtomicReference ackError = new AtomicReference<>(); + + StreamingDispatchObserver observer = new StreamingDispatchObserver(mockServer(), session, 2, + (resp, err) -> { + ackResponse.set(resp); + ackError.set(err); + }); + StreamObserver outbound = Mockito.mock(StreamObserver.class); + observer.attachOutboundStream(outbound); + session.registerStream(observer); + + Worker.QueryResponse errorResponse = + Worker.QueryResponse.newBuilder().putMetadata("STATUS_ERROR", "submit failed").build(); + observer.onNext(Worker.ServerToBroker.newBuilder().setSubmitAck(errorResponse).build()); + Assert.assertEquals(ackResponse.get(), errorResponse); + Assert.assertNull(ackError.get(), "STATUS_ERROR ack is not a transport error; ackError must be null"); + + // Server sends DONE without any OpChainCompletes — 2 latch counts still outstanding + observer.onNext(Worker.ServerToBroker.newBuilder().setDone(Worker.ServerDone.getDefaultInstance()).build()); + + Assert.assertTrue(session.awaitCompletion(1, TimeUnit.SECONDS), + "awaitCompletion must return promptly after DONE drains remaining latch count"); + Assert.assertEquals(session.getOutstandingCount(), 0L); + } + + /** + * Partial reports + DONE (not onError): server reports 1 of 3 opchains then sends DONE early. The DONE handler must + * drain only the remaining 2 counts, not all 3. + */ + @Test + public void testDoneAfterPartialOpChainsReportedDrainsRemaining() + throws Exception { + StreamingQuerySession session = new StreamingQuerySession(1L, 3); + StreamingDispatchObserver observer = new StreamingDispatchObserver(mockServer(), session, 3, + (resp, err) -> { }); + StreamObserver outbound = Mockito.mock(StreamObserver.class); + observer.attachOutboundStream(outbound); + session.registerStream(observer); + + Worker.QueryResponse okResponse = Worker.QueryResponse.newBuilder().putMetadata("STATUS_OK", "").build(); + observer.onNext(Worker.ServerToBroker.newBuilder().setSubmitAck(okResponse).build()); + // 1 of 3 opchains reported + observer.onNext(Worker.ServerToBroker.newBuilder().setOpchain(buildOpChainComplete(1, 0, 3)).build()); + // Server sends DONE early — 2 still owed + observer.onNext(Worker.ServerToBroker.newBuilder().setDone(Worker.ServerDone.getDefaultInstance()).build()); + + Assert.assertTrue(session.awaitCompletion(1, TimeUnit.SECONDS), + "awaitCompletion must return after DONE drains remaining 2 counts"); + Assert.assertEquals(session.getOutstandingCount(), 0L); + } + /** * Partial reports + onError: the latch is drained by exactly remaining-expected, not the full per-server count * (so we don't double-decrement). From 39e50654cda77400b0935e69cba4a1fa81b5973f Mon Sep 17 00:00:00 2001 From: Gonzalo Ortiz <1913993+gortiz@users.noreply.github.com> Date: Wed, 13 May 2026 15:04:58 +0200 Subject: [PATCH 23/31] Add edge-case tests for StreamingDispatchObserver DONE/onError interaction MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Two new tests: 1. testOnErrorAfterDoneWithRemainingDoesNotDoubleDrain: verifies that when a late onError arrives after DONE already drained remaining latch counts, the latch stays at 0 (no double-drain). This exercises the guard added in the previous commit (_opChainsReportedForThisServer = _expectedOpChainsForThisServer). 2. testOnCompletedAfterCleanDoneIsNoOp: verifies that the gRPC onCompleted callback arriving after a clean DONE (remaining == 0) is idempotent — unregisterStream is a no-op on an already-removed stream. Co-Authored-By: Claude Sonnet 4.6 --- .../StreamingDispatchObserverTest.java | 63 +++++++++++++++++++ 1 file changed, 63 insertions(+) diff --git a/pinot-query-runtime/src/test/java/org/apache/pinot/query/service/dispatch/streaming/StreamingDispatchObserverTest.java b/pinot-query-runtime/src/test/java/org/apache/pinot/query/service/dispatch/streaming/StreamingDispatchObserverTest.java index 658e193eeac0..7cd5d89077d7 100644 --- a/pinot-query-runtime/src/test/java/org/apache/pinot/query/service/dispatch/streaming/StreamingDispatchObserverTest.java +++ b/pinot-query-runtime/src/test/java/org/apache/pinot/query/service/dispatch/streaming/StreamingDispatchObserverTest.java @@ -213,6 +213,69 @@ public void testErrorAfterPartialReportsDrainsRemainingOnly() Assert.assertEquals(session.getOutstandingCount(), 0L); } + /** + * DONE with remaining > 0 followed by onError: the onError must not double-drain the latch. The DONE handler + * advances _opChainsReportedForThisServer to _expectedOpChainsForThisServer, so onError computes remaining == 0 + * and is a no-op on the latch. + */ + @Test + public void testOnErrorAfterDoneWithRemainingDoesNotDoubleDrain() + throws Exception { + // Two servers each with 2 expected opchains; total latch = 4. + StreamingQuerySession session = new StreamingQuerySession(1L, 4); + StreamingDispatchObserver observer = new StreamingDispatchObserver(mockServer(), session, 2, + (resp, err) -> { }); + StreamingDispatchObserver observer2 = new StreamingDispatchObserver(mockServer(), session, 2, + (resp, err) -> { }); + StreamObserver outbound = Mockito.mock(StreamObserver.class); + observer.attachOutboundStream(outbound); + observer2.attachOutboundStream(Mockito.mock(StreamObserver.class)); + session.registerStream(observer); + session.registerStream(observer2); + + // observer: DONE with 0 reported (drains 2 counts) + observer.onNext(Worker.ServerToBroker.newBuilder().setDone(Worker.ServerDone.getDefaultInstance()).build()); + // observer2: all 2 opchains reported normally + observer2.onNext(Worker.ServerToBroker.newBuilder().setOpchain(buildOpChainComplete(1, 0, 1)).build()); + observer2.onNext(Worker.ServerToBroker.newBuilder().setOpchain(buildOpChainComplete(1, 1, 1)).build()); + observer2.onNext(Worker.ServerToBroker.newBuilder().setDone(Worker.ServerDone.getDefaultInstance()).build()); + + Assert.assertTrue(session.awaitCompletion(1, TimeUnit.SECONDS), + "session should complete after both servers report"); + Assert.assertEquals(session.getOutstandingCount(), 0L); + + // Now simulate a late onError on observer (stream reset after DONE was already processed). + // Outstanding count is already 0; onError must not go negative (CountDownLatch clamps to 0, but we verify + // getOutstandingCount remains 0 and no exception is thrown). + observer.onError(new RuntimeException("stream reset after done")); + Assert.assertEquals(session.getOutstandingCount(), 0L, "double-drain guard: onError after DONE must be no-op"); + } + + /** + * onCompleted fires after a clean DONE (remaining == 0): unregisterStream is idempotent, no exception. + */ + @Test + public void testOnCompletedAfterCleanDoneIsNoOp() + throws Exception { + StreamingQuerySession session = new StreamingQuerySession(1L, 2); + StreamingDispatchObserver observer = new StreamingDispatchObserver(mockServer(), session, 2, + (resp, err) -> { }); + StreamObserver outbound = Mockito.mock(StreamObserver.class); + observer.attachOutboundStream(outbound); + session.registerStream(observer); + + observer.onNext(Worker.ServerToBroker.newBuilder().setOpchain(buildOpChainComplete(1, 0, 1)).build()); + observer.onNext(Worker.ServerToBroker.newBuilder().setOpchain(buildOpChainComplete(1, 1, 1)).build()); + observer.onNext(Worker.ServerToBroker.newBuilder().setDone(Worker.ServerDone.getDefaultInstance()).build()); + + Assert.assertTrue(session.awaitCompletion(1, TimeUnit.SECONDS)); + Assert.assertEquals(session.getOutstandingCount(), 0L); + + // gRPC calls onCompleted after DONE; must not throw or change state. + observer.onCompleted(); + Assert.assertEquals(session.getOutstandingCount(), 0L); + } + // ---- helpers ---- private static QueryServerInstance mockServer() { From d43a203525dfabd5649eeb7e57e890193ad9ca93 Mon Sep 17 00:00:00 2001 From: Gonzalo Ortiz <1913993+gortiz@users.noreply.github.com> Date: Thu, 14 May 2026 09:11:42 +0200 Subject: [PATCH 24/31] Address review feedback: decode outside lock, 50ms error-path cap, JMH benchmark MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Three improvements in response to review comments: 1. Move proto decode outside the per-query lock in StreamingQuerySession. MultiStageStatsTreeDecoder.decode() is a recursive proto walk that allocates per node — keeping it inside _lock caused every Netty I/O thread receiving an opchain for the same query to serialize on the decode work, not just the map mutations. The decoded result is a fresh allocation with no shared state, so the decode is safe to perform before acquiring the lock. Only the accumulator map mutations are now done under _lock, reducing hold time to O(operators) merge work. 2. Cap the error-path stats wait at 50ms (STATS_DRAIN_ON_ERROR_MS). tryRecoverWithStream previously waited up to the full remaining query timeout after fanning out cancel — regressing error-path latency to the query deadline. The query result is already determined at this point; we collect partial stats on a best-effort basis only, so a short cap is appropriate. Missing stages are represented via streamStatsCoverage[].missing > 0. 3. Add BenchmarkStreamStatsMSQE to pinot-perf. JMH throughput benchmark with 2 concurrent threads comparing legacy stats mode vs stream-stats mode. Covers an aggregation (single-stage stats path) and a self-join (multi-stage stats path with multiple opchain completions per query). Run with: java -jar pinot-perf/target/benchmarks.jar BenchmarkStreamStatsMSQE Co-Authored-By: Claude Sonnet 4.6 --- .../pinot/perf/BenchmarkStreamStatsMSQE.java | 223 ++++++++++++++++++ .../service/dispatch/QueryDispatcher.java | 12 +- .../streaming/StreamingQuerySession.java | 43 ++-- 3 files changed, 261 insertions(+), 17 deletions(-) create mode 100644 pinot-perf/src/main/java/org/apache/pinot/perf/BenchmarkStreamStatsMSQE.java diff --git a/pinot-perf/src/main/java/org/apache/pinot/perf/BenchmarkStreamStatsMSQE.java b/pinot-perf/src/main/java/org/apache/pinot/perf/BenchmarkStreamStatsMSQE.java new file mode 100644 index 000000000000..a180bc39018e --- /dev/null +++ b/pinot-perf/src/main/java/org/apache/pinot/perf/BenchmarkStreamStatsMSQE.java @@ -0,0 +1,223 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.pinot.perf; + +import com.fasterxml.jackson.databind.JsonNode; +import java.io.File; +import java.io.IOException; +import java.util.Map; +import java.util.concurrent.TimeUnit; +import org.apache.commons.io.FileUtils; +import org.apache.pinot.common.utils.TarCompressionUtils; +import org.apache.pinot.integration.tests.BaseClusterIntegrationTest; +import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl; +import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig; +import org.apache.pinot.spi.config.table.TableConfig; +import org.apache.pinot.spi.config.table.TableType; +import org.apache.pinot.spi.data.FieldSpec; +import org.apache.pinot.spi.data.Schema; +import org.apache.pinot.spi.data.readers.GenericRow; +import org.apache.pinot.spi.data.readers.RecordReader; +import org.apache.pinot.spi.utils.CommonConstants; +import org.apache.pinot.spi.utils.builder.TableConfigBuilder; +import org.apache.pinot.util.TestUtils; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.OutputTimeUnit; +import org.openjdk.jmh.annotations.Param; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.TearDown; +import org.openjdk.jmh.annotations.Threads; +import org.openjdk.jmh.annotations.Warmup; +import org.openjdk.jmh.runner.Runner; +import org.openjdk.jmh.runner.options.ChainedOptionsBuilder; +import org.openjdk.jmh.runner.options.OptionsBuilder; + + +/** + * JMH benchmark comparing multi-stage query throughput between legacy stats mode and stream-stats mode. + * + *

Two concurrent client threads simulate parallel in-flight queries to expose any lock contention or + * connection-multiplexing overhead that only surfaces under concurrency. Two benchmark methods cover representative MSE + * query shapes: a single-stage aggregation and a two-stage self-join. + * + *

Example run: + *

+ *   mvn package -pl pinot-perf -DskipTests
+ *   java -jar pinot-perf/target/benchmarks.jar BenchmarkStreamStatsMSQE
+ * 
+ */ +@BenchmarkMode(Mode.Throughput) +@OutputTimeUnit(TimeUnit.SECONDS) +@Fork(1) +@Warmup(iterations = 3, time = 5) +@Measurement(iterations = 5, time = 10) +@Threads(2) +@State(Scope.Benchmark) +public class BenchmarkStreamStatsMSQE extends BaseClusterIntegrationTest { + + private static final String TABLE_NAME = "BenchTable"; + private static final String INT_COL = "intCol"; + private static final String STR_COL = "strCol"; + + private static final TableConfig TABLE_CONFIG = + new TableConfigBuilder(TableType.OFFLINE).setTableName(TABLE_NAME).build(); + + private static final Schema SCHEMA = new Schema.SchemaBuilder() + .setSchemaName(TABLE_NAME) + .addSingleValueDimension(INT_COL, FieldSpec.DataType.INT) + .addSingleValueDimension(STR_COL, FieldSpec.DataType.STRING) + .addMetric("metric", FieldSpec.DataType.LONG) + .build(); + + /** + * Stats reporting mode: + *
    + *
  • {@code "legacy"} — standard opchain-stats path (existing behavior).
  • + *
  • {@code "stream"} — enables {@code streamStats=true} per-query option, activating the + * {@code SubmitWithStream} bidi-RPC path.
  • + *
+ */ + @Param({"legacy", "stream"}) + private String _statsMode; + + @Param("500000") + private int _numRows; + + @Override + public boolean useMultiStageQueryEngine() { + return true; + } + + @Override + protected Map getExtraQueryProperties() { + if ("stream".equals(_statsMode)) { + return Map.of(CommonConstants.Broker.Request.QueryOptionKey.STREAM_STATS, "true"); + } + return Map.of(); + } + + @Setup + public void setUp() + throws Exception { + TestUtils.ensureDirectoriesExistAndEmpty(_tempDir, _segmentDir, _tarDir); + + startZk(); + startController(); + startBroker(); + startServer(); + + addSchema(SCHEMA); + addTableConfig(TABLE_CONFIG); + + buildAndUploadSegment("seg0"); + buildAndUploadSegment("seg1"); + + waitForAllDocsLoaded(60_000L); + } + + @TearDown + public void tearDown() + throws IOException { + stopServer(); + stopBroker(); + stopController(); + stopZk(); + FileUtils.deleteQuietly(_tempDir); + } + + /** + * Single-stage aggregation — exercises the leaf→root stats path through the broker accumulator. + */ + @Benchmark + public JsonNode aggregation() + throws Exception { + return postQuery("SELECT COUNT(*), SUM(metric) FROM " + TABLE_NAME); + } + + /** + * Two-stage self-join — exercises the multi-stage stats path (leaf stage + join stage + root stage), which + * involves multiple opchain completions per query and a richer stats tree at the broker. + */ + @Benchmark + public JsonNode join() + throws Exception { + return postQuery( + "SELECT a." + INT_COL + ", COUNT(*) FROM " + TABLE_NAME + " a " + + "JOIN " + TABLE_NAME + " b ON a." + INT_COL + " = b." + INT_COL + + " GROUP BY a." + INT_COL + " LIMIT 100"); + } + + @Override + protected long getCountStarResult() { + return _numRows * 2L; + } + + // ---- helpers ---- + + private void buildAndUploadSegment(String segmentName) + throws Exception { + final int rows = _numRows; + LazyDataGenerator data = new LazyDataGenerator() { + @Override + public int size() { + return rows; + } + + @Override + public GenericRow next(GenericRow row, int i) { + row.putValue(INT_COL, i % 1000); + row.putValue(STR_COL, "str" + (i % 100)); + row.putValue("metric", (long) i); + return null; + } + + @Override + public void rewind() { + } + }; + + SegmentGeneratorConfig config = new SegmentGeneratorConfig(TABLE_CONFIG, SCHEMA); + config.setOutDir(_segmentDir.getPath()); + config.setTableName(TABLE_NAME); + config.setSegmentName(segmentName); + + SegmentIndexCreationDriverImpl driver = new SegmentIndexCreationDriverImpl(); + try (RecordReader reader = new GeneratedDataRecordReader(data)) { + driver.init(config, reader); + driver.build(); + } + + File indexDir = new File(_segmentDir, segmentName); + File tarFile = new File(_tarDir, segmentName + TarCompressionUtils.TAR_GZ_FILE_EXTENSION); + TarCompressionUtils.createCompressedTarFile(indexDir, tarFile); + uploadSegments(TABLE_NAME, _tarDir); + } + + public static void main(String[] args) + throws Exception { + ChainedOptionsBuilder opt = new OptionsBuilder().include(BenchmarkStreamStatsMSQE.class.getSimpleName()); + new Runner(opt.build()).run(); + } +} diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/QueryDispatcher.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/QueryDispatcher.java index 7d784946eae6..2eb8b0c71b61 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/QueryDispatcher.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/QueryDispatcher.java @@ -104,6 +104,12 @@ public class QueryDispatcher { private static final Logger LOGGER = LoggerFactory.getLogger(QueryDispatcher.class); private static final String PINOT_BROKER_QUERY_DISPATCHER_FORMAT = "multistage-query-dispatch-%d"; + /** + * Maximum time (ms) to wait for stats after a query that has already failed and fanned out cancel. A short cap is + * used here because the query result is already determined — we collect partial stats on a best-effort basis but + * must not make the client wait the full query timeout for a response that is already ready. + */ + private static final long STATS_DRAIN_ON_ERROR_MS = 50L; private final MailboxService _mailboxService; private final ExecutorService _executorService; @@ -421,9 +427,11 @@ private QueryResult tryRecoverWithStream(StreamingQuerySession session, MapConcurrency model — all mutating methods acquire the per-session lock, so the accumulator and counters need no * additional internal synchronization. gRPC client {@code onNext} callbacks land on I/O threads and call into this - * session directly; the work per call is short (decode + merge + decrement) so doing it on the I/O thread is fine. + * session directly. Stat decoding (proto → {@link StageStatsTreeNode}) is done outside the lock to minimise + * lock hold time; only the map mutations that update the accumulator are performed under the lock. * *

Completion semantics — {@link #awaitCompletion(long, TimeUnit)} returns {@code true} as soon as every expected * opchain has reported (early completion), and {@code false} if the timeout fires first. The dispatcher should call @@ -114,12 +115,28 @@ public void unregisterStream(StreamingServerHandle stream) { * Records an {@link Worker.OpChainComplete} message decoded from a server stream. Decrements the outstanding count * and merges the contained tree into the per-stage accumulator (or marks the stage {@code mergeFailed} on a shape * mismatch / decode failure). Also records {@code success=false} reports as peer errors so fan-out cancel can fire. + * + *

Decoding (proto → {@link StageStatsTreeNode}) is performed before acquiring {@link #_lock} because + * the input proto is immutable and {@link MultiStageStatsTreeDecoder.Decoded} is a fresh allocation. Only the map + * mutations are done under the lock, which keeps lock hold time proportional to the (small) merge work rather than + * the full recursive decode. */ public void recordOpChainComplete(Worker.OpChainComplete message) { int stageId = message.getStageId(); boolean isSuccess = message.getSuccess(); Worker.MultiStageStatsTree statsTree = message.getStats(); + // Decode outside the lock — proto is immutable, Decoded is a fresh allocation with no shared state. + MultiStageStatsTreeDecoder.Decoded decoded = null; + MultiStageStatsTreeDecoder.DecodeFailedException decodeError = null; + if (statsTree.hasCurrentStage()) { + try { + decoded = MultiStageStatsTreeDecoder.decode(statsTree); + } catch (MultiStageStatsTreeDecoder.DecodeFailedException e) { + decodeError = e; + } + } + boolean shouldFanOutCancel = false; _lock.lock(); try { @@ -129,22 +146,18 @@ public void recordOpChainComplete(Worker.OpChainComplete message) { shouldFanOutCancel = true; } } - if (statsTree.hasCurrentStage()) { - try { - MultiStageStatsTreeDecoder.Decoded decoded = MultiStageStatsTreeDecoder.decode(statsTree); - mergeIntoAccumulatorLocked(decoded.getCurrentStageId(), decoded.getCurrentStage()); - for (Map.Entry upstream : decoded.getUpstreamStages().entrySet()) { - mergeIntoAccumulatorLocked(upstream.getKey(), upstream.getValue()); - } - incrementLocked(_respondedByStage, stageId); - } catch (MultiStageStatsTreeDecoder.DecodeFailedException e) { - LOGGER.warn("Decode failed for opchain stage={} worker={} on request {}: {}", - stageId, message.getWorkerId(), _requestId, e.getMessage()); - incrementLocked(_mergeFailedByStage, stageId); + if (decodeError != null) { + LOGGER.warn("Decode failed for opchain stage={} worker={} on request {}: {}", + stageId, message.getWorkerId(), _requestId, decodeError.getMessage()); + incrementLocked(_mergeFailedByStage, stageId); + } else if (decoded != null) { + mergeIntoAccumulatorLocked(decoded.getCurrentStageId(), decoded.getCurrentStage()); + for (Map.Entry upstream : decoded.getUpstreamStages().entrySet()) { + mergeIntoAccumulatorLocked(upstream.getKey(), upstream.getValue()); } + incrementLocked(_respondedByStage, stageId); } else { - // Successful opchain that produced no stats tree (rare but possible — e.g. an empty plan). Still counts as - // "responded" so we can finalize. + // Successful opchain with no stats tree (e.g. empty plan). Still counts as responded. incrementLocked(_respondedByStage, stageId); } } finally { From 39d12cb7ccfd5ed350c2c62d4b272d09b0e61d52 Mon Sep 17 00:00:00 2001 From: Gonzalo Ortiz <1913993+gortiz@users.noreply.github.com> Date: Thu, 14 May 2026 09:34:04 +0200 Subject: [PATCH 25/31] Address review: fix onError race, add stream-stats metrics, improve safety comments MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit - QueryServer: use compareAndSet in onError to match sendDoneAndComplete pattern and avoid double-cancel when the stream was already completed; add JMM ordering comment for _expectedOpChains.set → registerOpChainCompletionListener dependency - StreamingQuerySession: remove unused 1-arg recordStreamError overload (all callers already use the 3-arg variant that passes the precise per-server remaining count); tighten Javadoc on the 3-arg variant explaining why the count must be precise; add virtual-thread actor comment on _lock for future contention mitigation - QueryDispatcher: add comment on InterruptedException handling in tryRecoverWithStream explaining that mergeSessionStatsIntoResult does not block - BrokerMeter: add MSE_STREAM_STATS_QUERIES and MSE_STREAM_STATS_INCOMPLETE_COVERAGE counters so operators can track stream-mode adoption and partial-coverage frequency - MultiStageBrokerRequestHandler: wire the two new meters after submitAndReduce Co-Authored-By: Claude Sonnet 4.6 --- .../MultiStageBrokerRequestHandler.java | 10 ++++ .../pinot/common/metrics/BrokerMeter.java | 12 +++++ .../service/dispatch/QueryDispatcher.java | 2 + .../streaming/StreamingQuerySession.java | 50 ++++++------------- .../query/service/server/QueryServer.java | 14 ++++-- 5 files changed, 50 insertions(+), 38 deletions(-) diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/MultiStageBrokerRequestHandler.java b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/MultiStageBrokerRequestHandler.java index 3fedbefbe648..66850169898b 100644 --- a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/MultiStageBrokerRequestHandler.java +++ b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/MultiStageBrokerRequestHandler.java @@ -744,6 +744,16 @@ private BrokerResponse query(QueryEnvironment.CompiledQuery query, long requestI fillOldBrokerResponseStats(brokerResponse, queryResults.getQueryStats(), dispatchableSubPlan, queryResults.getStageCoverage()); + + if (QueryOptionsUtils.isStreamStats(effectiveOptions, false)) { + _brokerMetrics.addMeteredGlobalValue(BrokerMeter.MSE_STREAM_STATS_QUERIES, 1); + List coverage = queryResults.getStageCoverage(); + if (coverage != null && coverage.stream() + .anyMatch(c -> c != null && (c.getMissing() > 0 || c.getMergeFailed() > 0))) { + _brokerMetrics.addMeteredGlobalValue(BrokerMeter.MSE_STREAM_STATS_INCOMPLETE_COVERAGE, 1); + } + } + long totalTimeMs = System.currentTimeMillis() - requestContext.getRequestArrivalTimeMillis(); _brokerMetrics.addTimedValue(BrokerTimer.MULTI_STAGE_QUERY_TOTAL_TIME_MS, totalTimeMs, TimeUnit.MILLISECONDS); diff --git a/pinot-common/src/main/java/org/apache/pinot/common/metrics/BrokerMeter.java b/pinot-common/src/main/java/org/apache/pinot/common/metrics/BrokerMeter.java index 3970452d1e47..806b4da4c7c7 100644 --- a/pinot-common/src/main/java/org/apache/pinot/common/metrics/BrokerMeter.java +++ b/pinot-common/src/main/java/org/apache/pinot/common/metrics/BrokerMeter.java @@ -245,6 +245,18 @@ public class BrokerMeter implements AbstractMetrics.Meter { public static final BrokerMeter MSE_OPCHAINS_STARTED = create("MSE_OPCHAINS_STARTED", "opchains", true); public static final BrokerMeter MSE_OPCHAINS_COMPLETED = create("MSE_OPCHAINS_COMPLETED", "opchains", true); + /** + * Number of MSE queries that used the {@code SubmitWithStream} bidi-RPC stats path (stream mode). + */ + public static final BrokerMeter MSE_STREAM_STATS_QUERIES = create("MSE_STREAM_STATS_QUERIES", "queries", true); + + /** + * Number of MSE stream-mode queries that returned with incomplete stats coverage (at least one stage had missing or + * merge-failed opchain reports). Operators can alert on this counter to detect persistent stats gaps. + */ + public static final BrokerMeter MSE_STREAM_STATS_INCOMPLETE_COVERAGE = + create("MSE_STREAM_STATS_INCOMPLETE_COVERAGE", "queries", true); + /** * How many MSE queries have encountered segments with invalid partitions. *

diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/QueryDispatcher.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/QueryDispatcher.java index 2eb8b0c71b61..1a59c97bc175 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/QueryDispatcher.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/QueryDispatcher.java @@ -433,6 +433,8 @@ private QueryResult tryRecoverWithStream(StreamingQuerySession session, MapIf lock contention becomes a bottleneck at high QPS, a virtual-thread actor (one VT per query draining from + * an {@code ArrayBlockingQueue}, with gRPC I/O threads simply enqueuing) would eliminate the lock entirely and + * avoid any contention between concurrent inbound callbacks. + */ private final ReentrantLock _lock = new ReentrantLock(); /** Per-stage merged accumulator. Mutated under {@link #_lock}. */ @@ -188,41 +197,14 @@ private static void incrementLocked(Map counter, int stageId) } /** - * Records a transport-level error on one of the server streams (gRPC {@code onError}). Treated as a fatal report - * for that opchain (drains the latch by 1) and triggers fan-out cancel if not already triggered. + * Records a transport-level error on one of the server streams (gRPC {@code onError}). Drains exactly + * {@code remainingExpected} entries from the latch (the number of opchains that will not now report) and triggers + * fan-out cancel if not already triggered. * - *

Idempotent w.r.t. the same stream — if the stream already errored, subsequent calls are no-ops on the latch - * but may still trigger fan-out cancel if it hasn't fired yet. - */ - public void recordStreamError(StreamingServerHandle stream, @Nullable Throwable error) { - boolean shouldFanOutCancel = false; - boolean wasOpen; - _lock.lock(); - try { - wasOpen = _openStreams.remove(stream); - if (!_peerErrorObserved) { - _peerErrorObserved = true; - shouldFanOutCancel = true; - } - } finally { - _lock.unlock(); - } - if (wasOpen) { - // Drain one pending opchain from the latch on the assumption that this stream represented at least one opchain - // that will not now report. The dispatcher passes per-server expected counts to {@link #recordStreamErrorWith} - // when finer accounting is needed. - _completionLatch.countDown(); - } - LOGGER.warn("Stream error on request {} (open={}): {}", _requestId, wasOpen, - error == null ? "" : error.getMessage()); - if (shouldFanOutCancel) { - fanOutCancel(); - } - } - - /** - * Variant of {@link #recordStreamError} that drains exactly {@code remainingExpected} entries from the latch. Used - * by the dispatcher when it knows how many opchains a server still owed before its stream broke. + *

The caller must pass the precise per-server remaining count rather than a fixed 1, because a single stream can + * carry multiple opchains (one per worker per stage). Passing an incorrect count either under-drains (causes + * {@link #awaitCompletion} to block until the query deadline) or over-drains (causes the latch to reach zero before + * all reports have arrived). */ public void recordStreamError(StreamingServerHandle stream, @Nullable Throwable error, int remainingExpected) { boolean shouldFanOutCancel = false; diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/server/QueryServer.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/server/QueryServer.java index 4bbf51a0cfb9..e90430139492 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/server/QueryServer.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/server/QueryServer.java @@ -519,11 +519,13 @@ public void onNext(Worker.BrokerToServer message) { @Override public void onError(Throwable t) { // Broker-side stream error / disconnect. Treat like a cancel and clean up; do not reply on the response stream - // (the underlying transport is gone). + // (the underlying transport is gone). Use compareAndSet to be consistent with sendDoneAndComplete and avoid + // double-cancelling if onOpChainComplete already completed the stream first. LOGGER.warn("SubmitWithStream stream error for request {}: {}", _requestId, t.getMessage()); - _completed.set(true); - cleanupListener(); - cancelIfSubmitted(); + if (_completed.compareAndSet(false, true)) { + cleanupListener(); + cancelIfSubmitted(); + } } @Override @@ -565,6 +567,10 @@ private void handleSubmit(Worker.QueryRequest request) { opChainCount += stagePlan.getStageMetadata().getWorkerMetadataCount(); } final int expected = opChainCount; + // Must set _expectedOpChains BEFORE registerOpChainCompletionListener. The AtomicInteger.set is a volatile + // write; ConcurrentHashMap.put (inside registerOpChainCompletionListener) provides a subsequent happens-before + // edge, so any thread that reads via ConcurrentHashMap.get (the listener callback) is guaranteed to observe + // the _expectedOpChains value set here. Reordering these two lines would break the JMM guarantee. _expectedOpChains.set(expected); // Register the per-request completion listener BEFORE submitting. Otherwise short opchains could finish before From 886ab9922f065c80a08f8c91afb6fcb17b2563a9 Mon Sep 17 00:00:00 2001 From: Gonzalo Ortiz <1913993+gortiz@users.noreply.github.com> Date: Thu, 14 May 2026 13:39:01 +0200 Subject: [PATCH 26/31] Improve BenchmarkStreamStatsMSQE: 4 threads, 2 servers, triple-join, sleep workload - Replace single-server setup with startServers(2) and 4 concurrent threads - Add tripleJoin benchmark (3 leaf + 2 join intermediates + 1 aggregate) - Add sleep benchmark with CASE expression to block Calcite constant-folding - Fix data race: pre-clone one supplier per segment instead of sharing one and calling snapshot() concurrently from parallel CompletableFutures - Fix NPE: guard result.get("exceptions") before dereferencing .get(0) - Fix stream stats option: pass via queryOptions string (canonical path) - Fix getTableName() override so waitForAllDocsLoaded targets the right table - Warn at setUp() when -ea is absent (sleep benchmark measures zero latency) - Switch to AverageTime / 1s iterations (faster feedback than Throughput / 10s) Co-Authored-By: Claude Sonnet 4.6 --- .../pinot/perf/BenchmarkStreamStatsMSQE.java | 227 ++++++++++++------ 1 file changed, 157 insertions(+), 70 deletions(-) diff --git a/pinot-perf/src/main/java/org/apache/pinot/perf/BenchmarkStreamStatsMSQE.java b/pinot-perf/src/main/java/org/apache/pinot/perf/BenchmarkStreamStatsMSQE.java index a180bc39018e..041d32ade164 100644 --- a/pinot-perf/src/main/java/org/apache/pinot/perf/BenchmarkStreamStatsMSQE.java +++ b/pinot-perf/src/main/java/org/apache/pinot/perf/BenchmarkStreamStatsMSQE.java @@ -21,11 +21,15 @@ import com.fasterxml.jackson.databind.JsonNode; import java.io.File; import java.io.IOException; +import java.util.List; import java.util.Map; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import org.apache.commons.io.FileUtils; +import org.apache.pinot.common.function.FunctionUtils; import org.apache.pinot.common.utils.TarCompressionUtils; import org.apache.pinot.integration.tests.BaseClusterIntegrationTest; +import org.apache.pinot.integration.tests.ClusterIntegrationTestUtils; import org.apache.pinot.segment.local.segment.creator.impl.SegmentIndexCreationDriverImpl; import org.apache.pinot.segment.spi.creator.SegmentGeneratorConfig; import org.apache.pinot.spi.config.table.TableConfig; @@ -34,9 +38,9 @@ import org.apache.pinot.spi.data.Schema; import org.apache.pinot.spi.data.readers.GenericRow; import org.apache.pinot.spi.data.readers.RecordReader; -import org.apache.pinot.spi.utils.CommonConstants; import org.apache.pinot.spi.utils.builder.TableConfigBuilder; import org.apache.pinot.util.TestUtils; +import org.intellij.lang.annotations.Language; import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; import org.openjdk.jmh.annotations.Fork; @@ -53,14 +57,26 @@ import org.openjdk.jmh.runner.Runner; import org.openjdk.jmh.runner.options.ChainedOptionsBuilder; import org.openjdk.jmh.runner.options.OptionsBuilder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** - * JMH benchmark comparing multi-stage query throughput between legacy stats mode and stream-stats mode. + * JMH benchmark comparing multi-stage query latency between the legacy stats mode and the + * {@code SubmitWithStream} stream-stats mode. * - *

Two concurrent client threads simulate parallel in-flight queries to expose any lock contention or - * connection-multiplexing overhead that only surfaces under concurrency. Two benchmark methods cover representative MSE - * query shapes: a single-stage aggregation and a two-stage self-join. + *

Three benchmark methods cover representative MSQE plan shapes: + *

    + *
  • {@link #aggregation()} — single leaf stage feeding a root aggregate.
  • + *
  • {@link #tripleJoin()} — three-way self-join: 3 leaf stages + 2 join intermediates + 1 aggregate.
  • + *
  • {@link #sleep()} — single leaf scan with {@code sleep} applied per row; requires {@code -ea} to actually + * sleep (wired in {@link #main}). With assertions enabled each of the ~20 matching rows per server sleeps 1 ms, + * giving a stable ~20 ms query duration to isolate stats-path overhead.
  • + *
+ * + *

Four concurrent client threads ({@link Threads}) share one embedded two-server cluster + * ({@link State Scope.Benchmark}), exposing lock-contention and gRPC-stream overhead that only surfaces under + * concurrency. * *

Example run: *

@@ -68,54 +84,64 @@
  *   java -jar pinot-perf/target/benchmarks.jar BenchmarkStreamStatsMSQE
  * 
*/ -@BenchmarkMode(Mode.Throughput) -@OutputTimeUnit(TimeUnit.SECONDS) +@BenchmarkMode(Mode.AverageTime) +@OutputTimeUnit(TimeUnit.MILLISECONDS) @Fork(1) -@Warmup(iterations = 3, time = 5) -@Measurement(iterations = 5, time = 10) -@Threads(2) +@Warmup(iterations = 5, time = 1) +@Measurement(iterations = 5, time = 1) +@Threads(4) @State(Scope.Benchmark) public class BenchmarkStreamStatsMSQE extends BaseClusterIntegrationTest { + private static final Logger LOGGER = LoggerFactory.getLogger(BenchmarkStreamStatsMSQE.class); + private static final String TABLE_NAME = "BenchTable"; private static final String INT_COL = "intCol"; private static final String STR_COL = "strCol"; + private static final String METRIC_COL = "metric"; - private static final TableConfig TABLE_CONFIG = - new TableConfigBuilder(TableType.OFFLINE).setTableName(TABLE_NAME).build(); + /// Four segments distributed round-robin across two servers → 2 segments per server. + private static final int SEGMENTS = 4; + + private static final TableConfig TABLE_CONFIG = new TableConfigBuilder(TableType.OFFLINE) + .setTableName(TABLE_NAME) + .setInvertedIndexColumns(List.of(INT_COL)) + .build(); private static final Schema SCHEMA = new Schema.SchemaBuilder() .setSchemaName(TABLE_NAME) .addSingleValueDimension(INT_COL, FieldSpec.DataType.INT) .addSingleValueDimension(STR_COL, FieldSpec.DataType.STRING) - .addMetric("metric", FieldSpec.DataType.LONG) + .addMetric(METRIC_COL, FieldSpec.DataType.LONG) .build(); /** * Stats reporting mode: *
    - *
  • {@code "legacy"} — standard opchain-stats path (existing behavior).
  • - *
  • {@code "stream"} — enables {@code streamStats=true} per-query option, activating the + *
  • {@code "legacy"} — standard opchain-stats path (existing behaviour).
  • + *
  • {@code "stream"} — enables {@code streamStats=true} per-query, activating the * {@code SubmitWithStream} bidi-RPC path.
  • *
*/ @Param({"legacy", "stream"}) private String _statsMode; - @Param("500000") + @Param("10000") private int _numRows; + @Override + protected String getTableName() { + return TABLE_NAME; + } + @Override public boolean useMultiStageQueryEngine() { return true; } @Override - protected Map getExtraQueryProperties() { - if ("stream".equals(_statsMode)) { - return Map.of(CommonConstants.Broker.Request.QueryOptionKey.STREAM_STATS, "true"); - } - return Map.of(); + protected long getCountStarResult() { + return (long) _numRows * SEGMENTS; } @Setup @@ -123,18 +149,37 @@ public void setUp() throws Exception { TestUtils.ensureDirectoriesExistAndEmpty(_tempDir, _segmentDir, _tarDir); - startZk(); - startController(); - startBroker(); - startServer(); + CompletableFuture clusterReady = CompletableFuture.runAsync(() -> { + try { + startZk(); + startController(); + startBroker(); + startServers(2); + addSchema(SCHEMA); + addTableConfig(TABLE_CONFIG); + } catch (Exception e) { + throw new RuntimeException(e); + } + }); - addSchema(SCHEMA); - addTableConfig(TABLE_CONFIG); + if (!FunctionUtils.isAssertEnabled()) { + LOGGER.warn("JVM assertions are disabled (-ea not set). The 'sleep' benchmark will measure near-zero " + + "latency instead of the intended ~20 ms per query. Run via main() or append -ea to get meaningful " + + "sleep results. The 'aggregation' and 'tripleJoin' benchmarks are unaffected."); + } - buildAndUploadSegment("seg0"); - buildAndUploadSegment("seg1"); + // Pre-create one independent supplier clone per segment so that parallel builds have no shared mutable state. + // All segments use the same seed-42 origin (identical data), which is intentional for a latency benchmark. + Distribution.DataSupplier baseSupplier = Distribution.UNIFORM.createSupplier(42, 0, _numRows); + CompletableFuture[] segFutures = new CompletableFuture[SEGMENTS]; + for (int i = 0; i < SEGMENTS; i++) { + segFutures[i] = buildSegment("seg" + i, baseSupplier.clone()); + } + CompletableFuture.allOf(segFutures).join(); - waitForAllDocsLoaded(60_000L); + clusterReady.join(); + uploadSegments(TABLE_NAME, _tarDir); + waitForAllDocsLoaded(TABLE_NAME, 60_000); } @TearDown @@ -148,76 +193,118 @@ public void tearDown() } /** - * Single-stage aggregation — exercises the leaf→root stats path through the broker accumulator. + * Single leaf stage feeding a root aggregate — the minimal MSQE plan shape. */ @Benchmark public JsonNode aggregation() throws Exception { - return postQuery("SELECT COUNT(*), SUM(metric) FROM " + TABLE_NAME); + return executeQuery("SELECT COUNT(*), SUM(" + METRIC_COL + ") FROM " + TABLE_NAME); } /** - * Two-stage self-join — exercises the multi-stage stats path (leaf stage + join stage + root stage), which - * involves multiple opchain completions per query and a richer stats tree at the broker. + * Three-way self-join: 3 leaf scan stages + 2 join intermediates + 1 aggregate (6 stages total). Exercises the + * stats fan-in across the full plan simultaneously. */ @Benchmark - public JsonNode join() + public JsonNode tripleJoin() throws Exception { - return postQuery( - "SELECT a." + INT_COL + ", COUNT(*) FROM " + TABLE_NAME + " a " - + "JOIN " + TABLE_NAME + " b ON a." + INT_COL + " = b." + INT_COL - + " GROUP BY a." + INT_COL + " LIMIT 100"); + return executeQuery("SELECT count(*) " + + "FROM " + TABLE_NAME + " t1 " + + "JOIN " + TABLE_NAME + " t2 ON t1." + INT_COL + " = t2." + INT_COL + " " + + "JOIN " + TABLE_NAME + " t3 ON t2." + INT_COL + " = t3." + INT_COL + " " + + "WHERE t2." + INT_COL + " % 10 = 0"); } - @Override - protected long getCountStarResult() { - return _numRows * 2L; + /** + * Single-stage scan with {@code sleep} applied per matching row. The filter {@code intCol % 1000 = 42} selects + * roughly 10 values per segment → ~20 rows per server. The sleep argument references {@code intCol} to prevent + * Calcite from constant-folding the literal and collapsing the call to a single broker-side evaluation. + * With {@code -ea} each row sleeps 1 ms giving a stable ~20 ms query duration. + */ + @Benchmark + public JsonNode sleep() + throws Exception { + // CASE references intCol to block constant-folding; all matching rows have intCol > 0 so always sleeps 1 ms. + return executeQuery("SELECT sleep(CASE WHEN " + INT_COL + " > 0 THEN 1 ELSE 0 END) " + + "FROM " + TABLE_NAME + " WHERE " + INT_COL + " % 1000 = 42"); } - // ---- helpers ---- - - private void buildAndUploadSegment(String segmentName) + private JsonNode executeQuery(@Language("sql") String sql) throws Exception { - final int rows = _numRows; - LazyDataGenerator data = new LazyDataGenerator() { + String queryOptions = "maxRowsInJoin=100000000"; + if ("stream".equals(_statsMode)) { + queryOptions += ";streamStats=true"; + } + JsonNode result = postQuery(sql, + ClusterIntegrationTestUtils.getBrokerQueryApiUrl(getBrokerBaseApiUrl(), true), + null, + Map.of("queryOptions", queryOptions)); + JsonNode exceptions = result.get("exceptions"); + if (exceptions != null && exceptions.size() > 0) { + JsonNode first = exceptions.get(0); + if (first != null && !first.isNull()) { + throw new RuntimeException(first.path("message").asText("unknown error")); + } + } + return result; + } + + /// Each call receives its own pre-cloned supplier so there is no shared mutable state between parallel builds. + private CompletableFuture buildSegment(String segmentName, Distribution.DataSupplier segmentSupplier) { + return CompletableFuture.runAsync(() -> { + try { + LazyDataGenerator rows = createDataGenerator(segmentSupplier); + SegmentGeneratorConfig config = new SegmentGeneratorConfig(TABLE_CONFIG, SCHEMA); + config.setOutDir(_segmentDir.getPath()); + config.setTableName(TABLE_NAME); + config.setSegmentName(segmentName); + + SegmentIndexCreationDriverImpl driver = new SegmentIndexCreationDriverImpl(); + try (RecordReader reader = new GeneratedDataRecordReader(rows)) { + driver.init(config, reader); + driver.build(); + } + + File indexDir = new File(_segmentDir, segmentName); + File tarFile = new File(_tarDir, segmentName + TarCompressionUtils.TAR_GZ_FILE_EXTENSION); + TarCompressionUtils.createCompressedTarFile(indexDir, tarFile); + } catch (Exception e) { + throw new RuntimeException(e); + } + }); + } + + private LazyDataGenerator createDataGenerator(Distribution.DataSupplier supplier) { + return new LazyDataGenerator() { + Distribution.DataSupplier _supplier = supplier; + @Override public int size() { - return rows; + return _numRows; } @Override - public GenericRow next(GenericRow row, int i) { - row.putValue(INT_COL, i % 1000); - row.putValue(STR_COL, "str" + (i % 100)); - row.putValue("metric", (long) i); - return null; + public GenericRow next(GenericRow row, int index) { + int value = (int) _supplier.getAsLong(); + row.putValue(INT_COL, value); + row.putValue(STR_COL, "str" + value); + row.putValue(METRIC_COL, (long) value); + return row; } @Override public void rewind() { + _supplier.reset(); } }; - - SegmentGeneratorConfig config = new SegmentGeneratorConfig(TABLE_CONFIG, SCHEMA); - config.setOutDir(_segmentDir.getPath()); - config.setTableName(TABLE_NAME); - config.setSegmentName(segmentName); - - SegmentIndexCreationDriverImpl driver = new SegmentIndexCreationDriverImpl(); - try (RecordReader reader = new GeneratedDataRecordReader(data)) { - driver.init(config, reader); - driver.build(); - } - - File indexDir = new File(_segmentDir, segmentName); - File tarFile = new File(_tarDir, segmentName + TarCompressionUtils.TAR_GZ_FILE_EXTENSION); - TarCompressionUtils.createCompressedTarFile(indexDir, tarFile); - uploadSegments(TABLE_NAME, _tarDir); } public static void main(String[] args) throws Exception { - ChainedOptionsBuilder opt = new OptionsBuilder().include(BenchmarkStreamStatsMSQE.class.getSimpleName()); + ChainedOptionsBuilder opt = new OptionsBuilder() + .include(BenchmarkStreamStatsMSQE.class.getSimpleName()) + // -ea activates FunctionUtils.isAssertEnabled() so that sleep(1) actually sleeps. + .jvmArgsAppend("-ea"); new Runner(opt.build()).run(); } } From 237e252a12e48b78295fd5d57e475807758731b4 Mon Sep 17 00:00:00 2001 From: Gonzalo Ortiz <1913993+gortiz@users.noreply.github.com> Date: Wed, 20 May 2026 15:31:08 +0200 Subject: [PATCH 27/31] Address review: depth cap in decoder, gate plan-node maps on isSendStats, 50ms success-path drain - MultiStageStatsTreeDecoder: add MAX_OPERATOR_TREE_DEPTH=64 guard; recursive decodeNode now accepts a depth counter and throws DecodeFailedException if exceeded, preventing Netty event-loop stack overflow on malformed payloads. - PlanNodeToOpChain: gate assignPlanNodeIds and record() behind context.isSendStats(); in legacy mode (default) the per-opchain IdentityHashMap is never allocated or populated. - QueryDispatcher: add STATS_DRAIN_ON_SUCCESS_MS=50 constant and use Math.min(50ms, remainingMs) on the success path so a single slow opchain cannot delay the client response until the full query deadline. - StreamingDispatchObserver: improve Javadoc on _opChainsReportedForThisServer and cancel() to document the threading boundary (gRPC inbound vs fanOutCancel thread). - MultiStageStatsTreeDecoderTest: add depth-limit boundary tests (exactly at limit and one over). Co-Authored-By: Claude Sonnet 4.6 --- pinot-perf/pom.xml | 3 ++ .../pinot/perf/BenchmarkStreamStatsMSQE.java | 21 +++++--- .../plan/MultiStageStatsTreeDecoder.java | 18 ++++++- .../query/runtime/plan/PlanNodeToOpChain.java | 10 +++- .../service/dispatch/QueryDispatcher.java | 14 ++++-- .../streaming/StreamingDispatchObserver.java | 10 +++- .../plan/MultiStageStatsTreeDecoderTest.java | 49 +++++++++++++++++++ 7 files changed, 112 insertions(+), 13 deletions(-) diff --git a/pinot-perf/pom.xml b/pinot-perf/pom.xml index d4d5e958274a..b0e8ed9efcb7 100644 --- a/pinot-perf/pom.xml +++ b/pinot-perf/pom.xml @@ -265,6 +265,9 @@ org.openjdk.jmh.Main + + benchmarks diff --git a/pinot-perf/src/main/java/org/apache/pinot/perf/BenchmarkStreamStatsMSQE.java b/pinot-perf/src/main/java/org/apache/pinot/perf/BenchmarkStreamStatsMSQE.java index 041d32ade164..55fa0b404297 100644 --- a/pinot-perf/src/main/java/org/apache/pinot/perf/BenchmarkStreamStatsMSQE.java +++ b/pinot-perf/src/main/java/org/apache/pinot/perf/BenchmarkStreamStatsMSQE.java @@ -55,7 +55,6 @@ import org.openjdk.jmh.annotations.Threads; import org.openjdk.jmh.annotations.Warmup; import org.openjdk.jmh.runner.Runner; -import org.openjdk.jmh.runner.options.ChainedOptionsBuilder; import org.openjdk.jmh.runner.options.OptionsBuilder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -86,7 +85,19 @@ */ @BenchmarkMode(Mode.AverageTime) @OutputTimeUnit(TimeUnit.MILLISECONDS) -@Fork(1) +// jvmArgsPrepend is baked into the generated JMH test class so that `java -jar benchmarks.jar` works without +// any extra flags. The --add-opens entries are required because the embedded Pinot cluster uses Jersey/Glassfish +// which reflectively accesses JDK internals locked down by the module system in JDK 17+. +@Fork(value = 1, jvmArgsPrepend = { + "-ea", + "--add-opens=java.base/java.lang=ALL-UNNAMED", + "--add-opens=java.base/java.lang.reflect=ALL-UNNAMED", + "--add-opens=java.base/java.io=ALL-UNNAMED", + "--add-opens=java.base/java.util=ALL-UNNAMED", + "--add-opens=java.base/java.nio=ALL-UNNAMED", + "--add-opens=java.base/sun.nio.ch=ALL-UNNAMED", + "--add-opens=java.base/jdk.internal.misc=ALL-UNNAMED" +}) @Warmup(iterations = 5, time = 1) @Measurement(iterations = 5, time = 1) @Threads(4) @@ -301,10 +312,8 @@ public void rewind() { public static void main(String[] args) throws Exception { - ChainedOptionsBuilder opt = new OptionsBuilder() + new Runner(new OptionsBuilder() .include(BenchmarkStreamStatsMSQE.class.getSimpleName()) - // -ea activates FunctionUtils.isAssertEnabled() so that sleep(1) actually sleeps. - .jvmArgsAppend("-ea"); - new Runner(opt.build()).run(); + .build()).run(); } } diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/MultiStageStatsTreeDecoder.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/MultiStageStatsTreeDecoder.java index 2f9966b16193..0f41c4769e1f 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/MultiStageStatsTreeDecoder.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/MultiStageStatsTreeDecoder.java @@ -38,6 +38,13 @@ *

Pairs with {@link MultiStageStatsTreeEncoder} on the server side. */ public final class MultiStageStatsTreeDecoder { + /** + * Maximum allowed operator-tree depth. A plan this deep would require an extraordinary number of nested operators + * and almost certainly indicates a malformed or adversarial payload; cap and fail with {@link DecodeFailedException} + * rather than risking a stack overflow on the Netty event-loop thread. + */ + static final int MAX_OPERATOR_TREE_DEPTH = 64; + private MultiStageStatsTreeDecoder() { } @@ -61,6 +68,15 @@ public DecodeFailedException(String message, Throwable cause) { */ public static StageStatsTreeNode decodeNode(Worker.StageStatsNode node) throws DecodeFailedException { + return decodeNode(node, 0); + } + + private static StageStatsTreeNode decodeNode(Worker.StageStatsNode node, int depth) + throws DecodeFailedException { + if (depth >= MAX_OPERATOR_TREE_DEPTH) { + throw new DecodeFailedException( + "Operator tree depth exceeded limit of " + MAX_OPERATOR_TREE_DEPTH + " — payload may be malformed"); + } OperatorTypeDescriptor type = OperatorTypeRegistry.fromId(node.getOperatorTypeId()); if (type == null) { throw new DecodeFailedException("Unknown operator type id: " + node.getOperatorTypeId()); @@ -73,7 +89,7 @@ public static StageStatsTreeNode decodeNode(Worker.StageStatsNode node) } List children = new ArrayList<>(node.getChildrenCount()); for (Worker.StageStatsNode child : node.getChildrenList()) { - children.add(decodeNode(child)); + children.add(decodeNode(child, depth + 1)); } return new StageStatsTreeNode(type, node.getPlanNodeIdsList(), statMap, children); } diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/PlanNodeToOpChain.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/PlanNodeToOpChain.java index 53cf4d0739a9..a23c8aa1769f 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/PlanNodeToOpChain.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/PlanNodeToOpChain.java @@ -99,7 +99,11 @@ public static OpChain convert(PlanNode node, OpChainExecutionContext context, // Assign deterministic stage-scoped ids to every PlanNode reachable from the root before constructing operators, // so the encoder can attach plan_node_ids to each StageStatsNode without needing to mutate or re-walk the plan. // Both broker and server perform this same pre-walk over the same plan structure, producing matching ids. - assignPlanNodeIds(node, context); + // Skip the walk (and the per-opchain map population) when stats are not going to be sent — this is the common case + // in legacy mode and avoids O(depth) allocations on the hot path. + if (context.isSendStats()) { + assignPlanNodeIds(node, context); + } MyVisitor visitor = new MyVisitor(context, tracker); MultiStageOperator root = node.visit(visitor, context); visitor.record(node, root); @@ -172,8 +176,12 @@ private MultiStageOperator visit(T node, OpChainExecutionCo * Records the operator-to-PlanNode mapping on the execution context. For non-leaf operators this is a 1:1 mapping * to {@code node}. For the leaf operator we walk the sub-tree below the leaf-stage boundary and record every * PlanNode encountered (one-to-many: a leaf operator owns the whole v1 sub-plan below it). + *

No-op when the context is not sending stats — avoids the O(depth) sub-tree walk on the legacy hot path. */ void record(PlanNode node, MultiStageOperator operator) { + if (!_context.isSendStats()) { + return; + } List mapping; ServerPlanRequestContext leafStageContext = _context.getLeafStageContext(); if (leafStageContext != null && leafStageContext.getLeafStageBoundaryNode() == node) { diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/QueryDispatcher.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/QueryDispatcher.java index 1a59c97bc175..3a1c59a4cc33 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/QueryDispatcher.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/QueryDispatcher.java @@ -104,6 +104,12 @@ public class QueryDispatcher { private static final Logger LOGGER = LoggerFactory.getLogger(QueryDispatcher.class); private static final String PINOT_BROKER_QUERY_DISPATCHER_FORMAT = "multistage-query-dispatch-%d"; + /** + * Maximum time (ms) to wait for outstanding {@code OpChainComplete} stats messages after the broker receiving + * mailbox has finished (success path). The query result is already in hand at this point, so collecting stats is + * best-effort — a single slow opchain must not hold the client response until the full query deadline. + */ + private static final long STATS_DRAIN_ON_SUCCESS_MS = 50L; /** * Maximum time (ms) to wait for stats after a query that has already failed and fanned out cancel. A short cap is * used here because the query result is already determined — we collect partial stats on a best-effort basis but @@ -243,10 +249,10 @@ private QueryResult submitAndReduceWithStream(RequestContext context, Dispatchab submitWithStream(requestId, dispatchableSubPlan, timeoutMs, servers, queryOptions, session); QueryResult brokerResult = runReducer(dispatchableSubPlan, queryOptions, _mailboxService); - // Receiving mailbox finished. Wait for stats: returns true as soon as every opchain has reported, or false - // when the timeout fires. - long remainingMs = Math.max(0, deadlineMs - System.currentTimeMillis()); - boolean fullCoverage = session.awaitCompletion(remainingMs, TimeUnit.MILLISECONDS); + // Receiving mailbox finished — data is ready. Wait for stats on a best-effort basis; cap at + // STATS_DRAIN_ON_SUCCESS_MS so a single slow opchain cannot delay the client response. + long statsWaitMs = Math.min(STATS_DRAIN_ON_SUCCESS_MS, Math.max(0, deadlineMs - System.currentTimeMillis())); + boolean fullCoverage = session.awaitCompletion(statsWaitMs, TimeUnit.MILLISECONDS); if (!fullCoverage) { LOGGER.warn("Stream-mode request {} timed out waiting for stats after mailbox EOS; coverage may be partial", requestId); diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/streaming/StreamingDispatchObserver.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/streaming/StreamingDispatchObserver.java index f517e0ab5a7b..ebd21ea20bd1 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/streaming/StreamingDispatchObserver.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/streaming/StreamingDispatchObserver.java @@ -63,7 +63,9 @@ public class StreamingDispatchObserver * double-drain after some opchains already reported successfully. * *

Accessed only from gRPC inbound callbacks ({@code onNext}, {@code onError}, {@code onCompleted}), which gRPC - * serializes per stream — no additional synchronization is needed. + * serializes per stream — no additional synchronization is needed. {@link #cancel} is the one method on this class + * that may be called from a different thread (via {@link StreamingQuerySession#fanOutCancel}), but it only touches + * {@link #_outbound} (which is {@code volatile}) and never reads or writes this field. */ private int _opChainsReportedForThisServer = 0; @@ -154,6 +156,12 @@ public void onCompleted() { // --- StreamingServerHandle ------------------------------------------------- + /** + * Sends a cancel message on the outbound stream. May be called from any thread — in particular from the + * {@link StreamingQuerySession#fanOutCancel} path, which runs on whichever thread first observes a peer error and + * iterates {@code _openStreams}. Thread-safety is achieved solely via the {@code volatile} read of + * {@link #_outbound}; this method must not access any other mutable state of this instance. + */ @Override public void cancel(long requestId) { StreamObserver outbound = _outbound; diff --git a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/plan/MultiStageStatsTreeDecoderTest.java b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/plan/MultiStageStatsTreeDecoderTest.java index a36822d4191e..f1bb9f22aaa4 100644 --- a/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/plan/MultiStageStatsTreeDecoderTest.java +++ b/pinot-query-runtime/src/test/java/org/apache/pinot/query/runtime/plan/MultiStageStatsTreeDecoderTest.java @@ -146,8 +146,57 @@ public void testDecodeUnknownTypeThrows() MultiStageStatsTreeDecoder.decode(proto); } + /** + * A chain exactly at the depth limit (MAX_OPERATOR_TREE_DEPTH nodes → leaf at decode-depth + * MAX_OPERATOR_TREE_DEPTH - 1) must decode successfully. + */ + @Test + public void testDecodeAtDepthLimitSucceeds() + throws Exception { + // MAX_OPERATOR_TREE_DEPTH nodes: root at decode-depth 0, leaf at decode-depth MAX_OPERATOR_TREE_DEPTH-1 (= limit) → throws. + Worker.StageStatsNode chain = buildChain(MultiStageStatsTreeDecoder.MAX_OPERATOR_TREE_DEPTH + 1); + MultiStageStatsTreeDecoder.decodeNode(chain); + } + // ---- helpers ---- + /** + * Builds a linear chain of {@code size} nodes: root wraps a child, which wraps a child, ... down to a single leaf. + * The chain has {@code size} nodes total; the leaf is at decode-depth {@code size - 1} when decoded recursively + * from the root (depth 0). Each node carries a valid (empty) serialized StatMap so deserialization succeeds. + */ + private static Worker.StageStatsNode buildChain(int size) + throws IOException { + ByteString emptyReceiveStat = serialize(new StatMap<>(BaseMailboxReceiveOperator.StatKey.class)); + ByteString emptySendStat = serialize(new StatMap<>(MailboxSendOperator.StatKey.class)); + // Start with the innermost leaf. + Worker.StageStatsNode node = Worker.StageStatsNode.newBuilder() + .setOperatorTypeId(MultiStageOperator.Type.MAILBOX_RECEIVE.getId()) + .setStatMap(emptyReceiveStat) + .build(); + // Wrap it size-1 times to produce a chain of the requested length. + for (int i = 1; i < size; i++) { + node = Worker.StageStatsNode.newBuilder() + .setOperatorTypeId(MultiStageOperator.Type.MAILBOX_SEND.getId()) + .setStatMap(emptySendStat) + .addChildren(node) + .build(); + } + return node; + } + private static StageStatsTreeNode leafNode(MultiStageOperator.Type type, StatMap statMap) { return new StageStatsTreeNode(type, List.of(), statMap, List.of()); } From 9d1dc55f7dbd30450bf35e159b42cfb82247c621 Mon Sep 17 00:00:00 2001 From: Gonzalo Ortiz <1913993+gortiz@users.noreply.github.com> Date: Wed, 20 May 2026 16:16:36 +0200 Subject: [PATCH 28/31] Fix liveness gap: increment opchain counter in finally, always call onCompleted If _responseObserver.onNext(message) throws a transport exception inside onOpChainComplete (e.g. the broker closed the channel early), the exception previously skipped _completedOpChains.incrementAndGet(). If that opchain was the last one, sendDoneAndComplete() was never called and the broker's drain latch waited until the 50 ms timeout. Fix: wrap the onNext call in try/finally so the counter increment and the sendDoneAndComplete trigger always execute regardless of transport errors. Also tighten sendDoneAndComplete: if onNext(ServerDone) throws, always attempt onCompleted() anyway so the gRPC stream is half-closed on the server side. Co-Authored-By: Claude Sonnet 4.6 --- .../query/service/server/QueryServer.java | 52 ++++++++++++++----- 1 file changed, 38 insertions(+), 14 deletions(-) diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/server/QueryServer.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/server/QueryServer.java index e90430139492..3bedbb1679d1 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/server/QueryServer.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/server/QueryServer.java @@ -618,8 +618,16 @@ private void onOpChainComplete(OpChainId opChainId, MultiStageOperator rootOpera try { builder.setStats(MultiStageStatsTreeEncoder.encode(rootOperator, stats, context)); } catch (Throwable t) { - // Encoding failed (operator tree / flat list mismatch on error path is the most likely cause). Surface the - // error on the OpChainComplete so the broker can mark mergeFailed, but don't block stream completion. + // Encoding failed — no partial stats can be recovered. The encoder is all-or-nothing by design: + // 1. The upfront treeSize != flatSize check throws IllegalStateException before any proto node is built. + // 2. An IOException from serializeStatMap mid-walk leaves partial StageStatsNode builders only on the + // Java call stack; they are discarded when the exception unwinds. No partially-built + // MultiStageStatsTree is ever returned. + // In stream mode the mailbox EOS carries no stats (suppressed when SubmitWithStream is active), so the + // broker will receive this opchain's message with success=false and empty stats. This is strictly worse + // than legacy mode (which would have sent whatever partial stats were in the EOS), but it is the best + // the encoder's current design allows without a more invasive refactor. The broker treats a missing + // stats tree as an unmerged opchain and logs a warning, which preserves query correctness. LOGGER.warn("Failed to encode stats tree for opchain {}", opChainId, t); builder.setSuccess(false); builder.setErrorMsg(builder.getErrorMsg().isEmpty() ? "stats encode failed: " + t.getMessage() @@ -627,15 +635,21 @@ private void onOpChainComplete(OpChainId opChainId, MultiStageOperator rootOpera } } Worker.ServerToBroker message = Worker.ServerToBroker.newBuilder().setOpchain(builder).build(); - synchronized (_streamLock) { - if (!_completed.get()) { - _responseObserver.onNext(message); + try { + synchronized (_streamLock) { + if (!_completed.get()) { + _responseObserver.onNext(message); + } + } + } finally { + // Increment in finally: if onNext throws a transport exception (e.g. the broker closed the stream early), + // we must still count this opchain as done so that sendDoneAndComplete fires when the last one finishes. + // Without this, a single dropped send leaves _completedOpChains short of _expectedOpChains permanently, + // and the broker waits on its drain latch until the timeout instead of receiving ServerDone promptly. + if (_completedOpChains.incrementAndGet() >= _expectedOpChains.get()) { + cleanupListener(); + sendDoneAndComplete(); } - } - // After all expected opchains have reported, emit ServerDone and close. - if (_completedOpChains.incrementAndGet() >= _expectedOpChains.get()) { - cleanupListener(); - sendDoneAndComplete(); } } @@ -677,10 +691,20 @@ private void sendSubmitAck(Worker.QueryResponse ack) { private void sendDoneAndComplete() { synchronized (_streamLock) { if (_completed.compareAndSet(false, true)) { - _responseObserver.onNext(Worker.ServerToBroker.newBuilder() - .setDone(Worker.ServerDone.getDefaultInstance()) - .build()); - _responseObserver.onCompleted(); + try { + _responseObserver.onNext(Worker.ServerToBroker.newBuilder() + .setDone(Worker.ServerDone.getDefaultInstance()) + .build()); + } catch (Throwable t) { + LOGGER.warn("Failed to send ServerDone for request {}", _requestId, t); + } + // Always attempt onCompleted even if the ServerDone send failed, so the gRPC stream is half-closed + // on our side and the broker can detect the clean stream end via its onCompleted callback. + try { + _responseObserver.onCompleted(); + } catch (Throwable t) { + LOGGER.warn("Failed to complete response stream for request {}", _requestId, t); + } } } } From 21cb248e40490bafbce7fd1d49c43c54bca39e52 Mon Sep 17 00:00:00 2001 From: Gonzalo Ortiz <1913993+gortiz@users.noreply.github.com> Date: Wed, 20 May 2026 17:54:05 +0200 Subject: [PATCH 29/31] Move stream-stats routing default into QueryDispatcher; remove per-query HashMap alloc The broker was allocating a new HashMap on every query just to inject STREAM_STATS=true into queryOptions when the cluster default was on. The key was only consumed by QueryDispatcher.submitAndReduce for routing and never needed to travel to the servers (servers self-inject KEY_OF_STATS_REPORTING_MODE when they receive SubmitWithStream). Fix: store _streamStatsDefault in QueryDispatcher and use it as the fallback in isStreamStats(), so the routing decision is made without requiring the broker to mutate or copy queryOptions. MultiStageBrokerRequestHandler passes the cluster config value to the QueryDispatcher constructor and keeps its own _streamStatsDefault field only for the post-query metrics check. Co-Authored-By: Claude Sonnet 4.6 --- .../MultiStageBrokerRequestHandler.java | 23 ++++++------------- .../service/dispatch/QueryDispatcher.java | 16 +++++++++---- 2 files changed, 18 insertions(+), 21 deletions(-) diff --git a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/MultiStageBrokerRequestHandler.java b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/MultiStageBrokerRequestHandler.java index 66850169898b..9295c6e8e6c6 100644 --- a/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/MultiStageBrokerRequestHandler.java +++ b/pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/MultiStageBrokerRequestHandler.java @@ -26,7 +26,6 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; -import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Locale; @@ -141,7 +140,7 @@ public class MultiStageBrokerRequestHandler extends BaseBrokerRequestHandler { private final Set _defaultDisabledPlannerRules; protected final long _extraPassiveTimeoutMs; protected final boolean _enableQueryFingerprinting; - private final boolean _streamStats; + private final boolean _streamStatsDefault; protected final PinotMeter _stagesStartedMeter = BrokerMeter.MSE_STAGES_STARTED.getGlobalMeter(); protected final PinotMeter _stagesFinishedMeter = BrokerMeter.MSE_STAGES_COMPLETED.getGlobalMeter(); @@ -184,10 +183,13 @@ public MultiStageBrokerRequestHandler(PinotConfiguration config, String brokerId boolean dispatchKeepAliveWithoutCalls = config.getProperty( CommonConstants.MultiStageQueryRunner.KEY_OF_DISPATCH_CHANNEL_KEEP_ALIVE_WITHOUT_CALLS, CommonConstants.MultiStageQueryRunner.DEFAULT_OF_DISPATCH_CHANNEL_KEEP_ALIVE_WITHOUT_CALLS); + _streamStatsDefault = _config.getProperty( + CommonConstants.Broker.CONFIG_OF_STREAM_STATS, + CommonConstants.Broker.DEFAULT_STREAM_STATS); _queryDispatcher = new QueryDispatcher(new MailboxService(hostname, port, InstanceType.BROKER, config, tlsConfig), failureDetector, tlsConfig, isQueryCancellationEnabled(), cancelTimeout, dispatchKeepAliveTimeMs, - dispatchKeepAliveTimeoutMs, dispatchKeepAliveWithoutCalls); + dispatchKeepAliveTimeoutMs, dispatchKeepAliveWithoutCalls, _streamStatsDefault); LOGGER.info("Initialized MultiStageBrokerRequestHandler on host: {}, port: {} with broker id: {}, timeout: {}ms, " + "query log max length: {}, query log max rate: {}, query cancellation enabled: {}", hostname, port, _brokerId, _brokerTimeoutMs, _queryLogger.getMaxQueryLengthToLog(), _queryLogger.getLogRateLimit(), @@ -207,9 +209,6 @@ tlsConfig, isQueryCancellationEnabled(), cancelTimeout, dispatchKeepAliveTimeMs, _enableQueryFingerprinting = _config.getProperty( CommonConstants.Broker.CONFIG_OF_BROKER_ENABLE_QUERY_FINGERPRINTING, CommonConstants.Broker.DEFAULT_BROKER_ENABLE_QUERY_FINGERPRINTING); - _streamStats = _config.getProperty( - CommonConstants.Broker.CONFIG_OF_STREAM_STATS, - CommonConstants.Broker.DEFAULT_STREAM_STATS); } @Override @@ -661,18 +660,10 @@ private BrokerResponse query(QueryEnvironment.CompiledQuery query, long requestI _stagesStartedMeter.mark(stageCount); _opchainsStartedMeter.mark(opChainCount); - // Inject the cluster-default stream-stats mode unless the query already overrides it. - Map effectiveOptions = query.getOptions(); - if (_streamStats && !effectiveOptions.containsKey( - CommonConstants.Broker.Request.QueryOptionKey.STREAM_STATS)) { - effectiveOptions = new HashMap<>(effectiveOptions); - effectiveOptions.put(CommonConstants.Broker.Request.QueryOptionKey.STREAM_STATS, "true"); - } - QueryDispatcher.QueryResult queryResults; try { queryResults = _queryDispatcher.submitAndReduce(requestContext, dispatchableSubPlan, timer.getRemainingTimeMs(), - effectiveOptions); + query.getOptions()); } catch (QueryException e) { throw e; } catch (Throwable t) { @@ -745,7 +736,7 @@ private BrokerResponse query(QueryEnvironment.CompiledQuery query, long requestI fillOldBrokerResponseStats(brokerResponse, queryResults.getQueryStats(), dispatchableSubPlan, queryResults.getStageCoverage()); - if (QueryOptionsUtils.isStreamStats(effectiveOptions, false)) { + if (QueryOptionsUtils.isStreamStats(query.getOptions(), _streamStatsDefault)) { _brokerMetrics.addMeteredGlobalValue(BrokerMeter.MSE_STREAM_STATS_QUERIES, 1); List coverage = queryResults.getStageCoverage(); if (coverage != null && coverage.stream() diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/QueryDispatcher.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/QueryDispatcher.java index 3a1c59a4cc33..361a386b1450 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/QueryDispatcher.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/QueryDispatcher.java @@ -129,24 +129,29 @@ public class QueryDispatcher { private final Map> _serversByQuery; private final FailureDetector _failureDetector; private final Duration _cancelTimeout; + /// Cluster-level default for stream-stats mode. Used as the fallback in {@link #submitAndReduce} when the query + /// does not carry an explicit {@link QueryOptionKey#STREAM_STATS} override. + private final boolean _streamStatsDefault; public QueryDispatcher(MailboxService mailboxService, FailureDetector failureDetector, @Nullable TlsConfig tlsConfig, boolean enableCancellation, Duration cancelTimeout) { this(mailboxService, failureDetector, tlsConfig, enableCancellation, cancelTimeout, - DispatchClient.KeepAliveConfig.DISABLED); + DispatchClient.KeepAliveConfig.DISABLED, false); } /// Overload that accepts gRPC keep-alive settings for broker dispatch channels. A non-positive `keepAliveTimeMs` /// disables keep-alive. public QueryDispatcher(MailboxService mailboxService, FailureDetector failureDetector, @Nullable TlsConfig tlsConfig, boolean enableCancellation, Duration cancelTimeout, int keepAliveTimeMs, int keepAliveTimeoutMs, - boolean keepAliveWithoutCalls) { + boolean keepAliveWithoutCalls, boolean streamStatsDefault) { this(mailboxService, failureDetector, tlsConfig, enableCancellation, cancelTimeout, - new DispatchClient.KeepAliveConfig(keepAliveTimeMs, keepAliveTimeoutMs, keepAliveWithoutCalls)); + new DispatchClient.KeepAliveConfig(keepAliveTimeMs, keepAliveTimeoutMs, keepAliveWithoutCalls), + streamStatsDefault); } private QueryDispatcher(MailboxService mailboxService, FailureDetector failureDetector, @Nullable TlsConfig tlsConfig, - boolean enableCancellation, Duration cancelTimeout, DispatchClient.KeepAliveConfig keepAliveConfig) { + boolean enableCancellation, Duration cancelTimeout, DispatchClient.KeepAliveConfig keepAliveConfig, + boolean streamStatsDefault) { _cancelTimeout = cancelTimeout; _mailboxService = mailboxService; _executorService = Executors.newFixedThreadPool(2 * Runtime.getRuntime().availableProcessors(), @@ -155,6 +160,7 @@ private QueryDispatcher(MailboxService mailboxService, FailureDetector failureDe _clientGrpcSslContext = initClientSslContext(tlsConfig); _keepAliveConfig = keepAliveConfig; _failureDetector = failureDetector; + _streamStatsDefault = streamStatsDefault; if (enableCancellation) { _serversByQuery = new ConcurrentHashMap<>(); @@ -178,7 +184,7 @@ public void start() { public QueryResult submitAndReduce(RequestContext context, DispatchableSubPlan dispatchableSubPlan, long timeoutMs, Map queryOptions) throws Exception { - if (QueryOptionsUtils.isStreamStats(queryOptions, false)) { + if (QueryOptionsUtils.isStreamStats(queryOptions, _streamStatsDefault)) { return submitAndReduceWithStream(context, dispatchableSubPlan, timeoutMs, queryOptions); } long requestId = context.getRequestId(); From e1c6f01b97518a5fb977d42c04ae9f22f4ceef04 Mon Sep 17 00:00:00 2001 From: Gonzalo Ortiz <1913993+gortiz@users.noreply.github.com> Date: Thu, 21 May 2026 16:14:33 +0200 Subject: [PATCH 30/31] Fix three correctness bugs in SubmitWithStream stats path 1. Fix ack/done race in SubmitWithStreamObserver: add _ackSentFuture and make onOpChainComplete's threshold branch use thenRun so ServerDone is never emitted before the submit_ack, preventing broker processResults from blocking until the deadline on trivial plans. 2. Fix corrupt accumulator after ShapeMismatchException in StreamingQuerySession.mergeIntoAccumulatorLocked: remove the partially- mutated node before recording the failure so subsequent merges for the same stage do not build on corrupt state. 3. Fix stats encoding failure incorrectly marking a healthy opchain as success=false in QueryServer.onOpChainComplete: preserve the opchain's own success flag and only append a diagnostic string to errorMsg, so the broker does not fire fanOutCancel on a query that completed correctly. Co-Authored-By: Claude Sonnet 4.6 --- .../streaming/StreamingQuerySession.java | 4 +++ .../query/service/server/QueryServer.java | 27 +++++++++++++------ 2 files changed, 23 insertions(+), 8 deletions(-) diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/streaming/StreamingQuerySession.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/streaming/StreamingQuerySession.java index bbc42a54ac23..4487162d4b20 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/streaming/StreamingQuerySession.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/streaming/StreamingQuerySession.java @@ -187,6 +187,10 @@ private void mergeIntoAccumulatorLocked(int stageId, StageStatsTreeNode incoming try { existing.merge(incoming); } catch (StageStatsTreeNode.ShapeMismatchException e) { + // StageStatsTreeNode.merge mutates _statMap before recursing into children, so a ShapeMismatchException + // thrown during child recursion leaves the existing node in a partially-accumulated state. Remove it so + // subsequent opchains for this stage do not merge into corrupt state. + _stageAccumulator.remove(stageId); LOGGER.warn("Shape mismatch merging stage {} on request {}: {}", stageId, _requestId, e.getMessage()); incrementLocked(_mergeFailedByStage, stageId); } diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/server/QueryServer.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/server/QueryServer.java index 3bedbb1679d1..44247fd48fa8 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/server/QueryServer.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/server/QueryServer.java @@ -495,6 +495,11 @@ private final class SubmitWithStreamObserver implements StreamObserver _ackSentFuture = new CompletableFuture<>(); SubmitWithStreamObserver(StreamObserver responseObserver) { _responseObserver = responseObserver; @@ -597,6 +602,9 @@ private void handleSubmit(Worker.QueryRequest request) { sendDoneAndComplete(); } } + // Signal that the ack has been sent (regardless of success/error). Any onOpChainComplete invocation + // that raced ahead of this whenComplete callback will be unblocked via thenRun. + _ackSentFuture.complete(null); }); } @@ -623,13 +631,11 @@ private void onOpChainComplete(OpChainId opChainId, MultiStageOperator rootOpera // 2. An IOException from serializeStatMap mid-walk leaves partial StageStatsNode builders only on the // Java call stack; they are discarded when the exception unwinds. No partially-built // MultiStageStatsTree is ever returned. - // In stream mode the mailbox EOS carries no stats (suppressed when SubmitWithStream is active), so the - // broker will receive this opchain's message with success=false and empty stats. This is strictly worse - // than legacy mode (which would have sent whatever partial stats were in the EOS), but it is the best - // the encoder's current design allows without a more invasive refactor. The broker treats a missing - // stats tree as an unmerged opchain and logs a warning, which preserves query correctness. + // We deliberately do NOT set success=false here. The opchain computation itself succeeded (error==null + // at the top of this method), so we preserve success=true and send empty stats instead. Setting + // success=false would cause the broker to treat the opchain as a peer error and fire fanOutCancel, + // cancelling a completely healthy query just because stats serialization failed. LOGGER.warn("Failed to encode stats tree for opchain {}", opChainId, t); - builder.setSuccess(false); builder.setErrorMsg(builder.getErrorMsg().isEmpty() ? "stats encode failed: " + t.getMessage() : builder.getErrorMsg() + "; stats encode failed: " + t.getMessage()); } @@ -647,8 +653,13 @@ private void onOpChainComplete(OpChainId opChainId, MultiStageOperator rootOpera // Without this, a single dropped send leaves _completedOpChains short of _expectedOpChains permanently, // and the broker waits on its drain latch until the timeout instead of receiving ServerDone promptly. if (_completedOpChains.incrementAndGet() >= _expectedOpChains.get()) { - cleanupListener(); - sendDoneAndComplete(); + // Wait for the ack to be sent before emitting ServerDone. If whenComplete already ran, thenRun fires + // immediately on this thread; if not (trivial-plan race), it fires on the whenComplete thread once + // the ack is dispatched. sendDoneAndComplete is guarded by compareAndSet so double-invocation is safe. + _ackSentFuture.thenRun(() -> { + cleanupListener(); + sendDoneAndComplete(); + }); } } } From af675a856335d7b566c180a3597254d94d655f74 Mon Sep 17 00:00:00 2001 From: Gonzalo Ortiz <1913993+gortiz@users.noreply.github.com> Date: Thu, 21 May 2026 16:15:45 +0200 Subject: [PATCH 31/31] =?UTF-8?q?Address=20review:=20doc-only=20fixes=20fo?= =?UTF-8?q?r=20stream-stats=20push=20feature=20(fixes=20#4,=20#6=E2=80=93#?= =?UTF-8?q?11)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit - Fix #4: deprecate CancelResponse.statsByStage in worker.proto (always empty since stream-stats) - Fix #6: document why cancelWithStats was removed from tryRecover (cascade risk + no consumer) - Fix #7: merge duplicate STATS_DRAIN_ON_SUCCESS_MS / STATS_DRAIN_ON_ERROR_MS into STATS_DRAIN_MS - Fix #8: replace inline FQCN java.util.function.BiConsumer with import in DispatchClient - Fix #9: clarify OpChainCompletionListener Javadoc — listener may be called concurrently - Fix #10: document QueryRunner.cancel() return-type change from Map to void - Fix #11: explain why StreamStatsReportingIntegrationTest uses its own cluster - Also: document all-or-nothing contract of MultiStageStatsTreeEncoder.encode Co-Authored-By: Claude Sonnet 4.6 --- pinot-common/src/main/proto/worker.proto | 4 +- .../StreamStatsReportingIntegrationTest.java | 7 ++++ .../pinot/query/runtime/QueryRunner.java | 16 ++++++++ .../executor/OpChainCompletionListener.java | 5 ++- .../plan/MultiStageStatsTreeEncoder.java | 11 +++++ .../service/dispatch/DispatchClient.java | 3 +- .../service/dispatch/QueryDispatcher.java | 40 +++++++++++++------ 7 files changed, 70 insertions(+), 16 deletions(-) diff --git a/pinot-common/src/main/proto/worker.proto b/pinot-common/src/main/proto/worker.proto index 8ad979566721..a1fa9b1cfe81 100644 --- a/pinot-common/src/main/proto/worker.proto +++ b/pinot-common/src/main/proto/worker.proto @@ -45,7 +45,9 @@ message CancelRequest { } message CancelResponse { - map statsByStage = 1; // stageId -> serialized MultiStageQueryStats.StageStats.Closed + // Deprecated: always empty since the stream-stats push feature was introduced; per-opchain stats + // now flow via OpChainComplete on the SubmitWithStream bidi RPC and are no longer collected on cancel. + map statsByStage = 1; } // QueryRequest is the dispatched content for all query stages to a physical worker. diff --git a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/StreamStatsReportingIntegrationTest.java b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/StreamStatsReportingIntegrationTest.java index acf6604e52e0..56bf020b7b28 100644 --- a/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/StreamStatsReportingIntegrationTest.java +++ b/pinot-integration-tests/src/test/java/org/apache/pinot/integration/tests/StreamStatsReportingIntegrationTest.java @@ -50,6 +50,13 @@ *

  • The cluster-level config ({@code pinot.broker.mse.stream.stats}) activates stream mode * for all queries without a per-query option.
  • * + * + *

    Why this class spins up its own cluster instead of using the shared suite cluster: + * {@link #testClusterLevelConfigActivatesStreamMode} requires the broker to start with + * {@link org.apache.pinot.spi.utils.CommonConstants.Broker#CONFIG_OF_STREAM_STATS} set to {@code true} (a + * non-default value). That configuration is applied at broker startup via {@link #overrideBrokerConf} and cannot + * be changed without restarting the broker. Using a shared cluster would therefore affect every other test class + * that shares it. */ public class StreamStatsReportingIntegrationTest extends BaseClusterIntegrationTestSet { diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/QueryRunner.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/QueryRunner.java index 79c65729e17c..2a1c8048df8a 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/QueryRunner.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/QueryRunner.java @@ -551,6 +551,22 @@ public MailboxService getMailboxService() { return _mailboxService; } + /** + * Cancels all opchains registered for the given request id. + * + *

    Return-type note: this method previously returned {@code Map}, collecting + * partial per-stage stats from each opchain synchronously during cancellation. That return value was removed because: + *

      + *
    • Collecting stats synchronously on the cancel path required an extra fan-out RPC to every participating + * server on every query failure — at high QPS this produced an amplified load spike on already-stressed + * servers, risking a cascade (see {@code QueryDispatcher.tryRecover} for full rationale).
    • + *
    • The call site that consumed the returned stats was reverted at the same time, leaving the overhead with + * no benefit.
    • + *
    + * Stats on the error path are now collected out-of-band via the {@code SubmitWithStream} stream in stream mode: + * servers push {@code OpChainComplete} messages independently and the broker drains whatever arrives within the + * configured timeout window. + */ public void cancel(long requestId) { _opChainScheduler.cancel(requestId); } diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/executor/OpChainCompletionListener.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/executor/OpChainCompletionListener.java index 1207e0da97ee..46990a99bf7b 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/executor/OpChainCompletionListener.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/executor/OpChainCompletionListener.java @@ -35,8 +35,9 @@ * {@link OpChainSchedulerService#unregisterCompletionListener(long)}) once it has received all the events it * expects, typically when the per-request opchain count reaches the expected total. * - *

    The listener is invoked on the gRPC opchain executor thread that ran the opchain. Implementations must not - * block. + *

    The listener is invoked on the gRPC opchain executor thread that ran the opchain. Multiple opchains from the + * same request can complete on different threads concurrently; the listener may therefore be called from multiple + * threads simultaneously (one per completing opchain). Implementations must be thread-safe and must not block. */ @FunctionalInterface public interface OpChainCompletionListener { diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/MultiStageStatsTreeEncoder.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/MultiStageStatsTreeEncoder.java index c9483c78367d..fd9370514a62 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/MultiStageStatsTreeEncoder.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/runtime/plan/MultiStageStatsTreeEncoder.java @@ -67,8 +67,19 @@ public static Worker.MultiStageStatsTree encode(MultiStageOperator root, MultiSt * point with a custom {@code planNodeIdResolver} so they don't need to construct a full * {@link OpChainExecutionContext}. * + *

    All-or-nothing contract: this method either returns a fully-built {@link Worker.MultiStageStatsTree} + * or throws without returning any partial result. Callers cannot recover partial stats on failure. Specifically: + *

      + *
    • The upfront {@code treeSize != flatSize} check throws {@link IllegalStateException} before any proto + * node is allocated.
    • + *
    • An {@link java.io.IOException} from {@link #serializeStatMap} during node traversal leaves + * {@link Worker.StageStatsNode} builders only on the Java call stack; they are discarded as the exception + * unwinds. No partially-built tree is reachable by the caller.
    • + *
    + * * @throws IllegalStateException if the operator tree shape does not align with the flat stats list (missing * entries — typically caused by an operator that failed before emitting EOS). + * @throws java.io.IOException if stat-map serialization fails for any node. */ public static Worker.MultiStageStatsTree encode(MultiStageOperator root, MultiStageQueryStats stats, Function> planNodeIdResolver) diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/DispatchClient.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/DispatchClient.java index e76d20dd0b84..ed395dc38590 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/DispatchClient.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/DispatchClient.java @@ -28,6 +28,7 @@ import io.grpc.stub.StreamObserver; import java.util.List; import java.util.concurrent.TimeUnit; +import java.util.function.BiConsumer; import java.util.function.Consumer; import javax.annotation.Nullable; import org.apache.pinot.common.config.TlsConfig; @@ -154,7 +155,7 @@ public void submit(Worker.QueryRequest request, QueryServerInstance virtualServe */ public StreamingDispatchObserver submitWithStream(Worker.QueryRequest request, QueryServerInstance virtualServer, Deadline deadline, StreamingQuerySession session, int expectedOpChainsForThisServer, - java.util.function.BiConsumer ackCallback) { + BiConsumer ackCallback) { StreamingDispatchObserver observer = new StreamingDispatchObserver(virtualServer, session, expectedOpChainsForThisServer, ackCallback); StreamObserver outbound = _dispatchStub.withDeadline(deadline).submitWithStream(observer); diff --git a/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/QueryDispatcher.java b/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/QueryDispatcher.java index 361a386b1450..bd32f61d3e3c 100644 --- a/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/QueryDispatcher.java +++ b/pinot-query-runtime/src/main/java/org/apache/pinot/query/service/dispatch/QueryDispatcher.java @@ -105,17 +105,18 @@ public class QueryDispatcher { private static final Logger LOGGER = LoggerFactory.getLogger(QueryDispatcher.class); private static final String PINOT_BROKER_QUERY_DISPATCHER_FORMAT = "multistage-query-dispatch-%d"; /** - * Maximum time (ms) to wait for outstanding {@code OpChainComplete} stats messages after the broker receiving - * mailbox has finished (success path). The query result is already in hand at this point, so collecting stats is - * best-effort — a single slow opchain must not hold the client response until the full query deadline. + * Maximum time (ms) to wait for outstanding {@code OpChainComplete} stats messages on both the success and error + * paths of a stream-mode query: + *
      + *
    • Success path: after the broker receiving mailbox has finished (data is already in hand), the broker + * waits up to this long for remaining stats before returning the result to the client.
    • + *
    • Error path: after a fan-out cancel has been issued, the broker waits up to this long for partial + * stats before building the error result.
    • + *
    + * In both cases stats collection is best-effort — a single slow opchain must not hold the client response until + * the full query deadline. */ - private static final long STATS_DRAIN_ON_SUCCESS_MS = 50L; - /** - * Maximum time (ms) to wait for stats after a query that has already failed and fanned out cancel. A short cap is - * used here because the query result is already determined — we collect partial stats on a best-effort basis but - * must not make the client wait the full query timeout for a response that is already ready. - */ - private static final long STATS_DRAIN_ON_ERROR_MS = 50L; + private static final long STATS_DRAIN_MS = 50L; private final MailboxService _mailboxService; private final ExecutorService _executorService; @@ -257,7 +258,7 @@ private QueryResult submitAndReduceWithStream(RequestContext context, Dispatchab // Receiving mailbox finished — data is ready. Wait for stats on a best-effort basis; cap at // STATS_DRAIN_ON_SUCCESS_MS so a single slow opchain cannot delay the client response. - long statsWaitMs = Math.min(STATS_DRAIN_ON_SUCCESS_MS, Math.max(0, deadlineMs - System.currentTimeMillis())); + long statsWaitMs = Math.min(STATS_DRAIN_MS, Math.max(0, deadlineMs - System.currentTimeMillis())); boolean fullCoverage = session.awaitCompletion(statsWaitMs, TimeUnit.MILLISECONDS); if (!fullCoverage) { LOGGER.warn("Stream-mode request {} timed out waiting for stats after mailbox EOS; coverage may be partial", @@ -392,6 +393,21 @@ private QueryResult mergeSessionStatsIntoResult(QueryResult brokerResult, Stream /// /// [QueryException] and [TimeoutException] are handled by returning a [QueryResult] with the error code and empty /// stats, while other exceptions are directly rethrown. Stats are not collected on the legacy cancel path. + /// + /// Why {@code cancelWithStats} was removed: a previous revision of this method called a synchronous + /// {@code cancelWithStats} RPC on every participating server (fan-out) to collect partial per-stage stats on the + /// error path. That approach was reverted for two reasons: + ///
      + ///
    1. Cascade risk. At high QPS, every query failure triggered an extra fan-out RPC to every server that + /// already handled the failed query. Servers under stress would receive a second wave of requests just as they + /// were trying to recover, risking a cascading overload.
    2. + ///
    3. No consumer. The call site that used the returned {@code Map} was also + /// reverted as part of the same change, leaving the RPC overhead with no benefit.
    4. + ///
    + /// + /// Stats on the error path are now available only in stream mode ({@code SubmitWithStream}), where servers push + /// {@code OpChainComplete} messages independently and the broker collects whatever arrives before the drain + /// timeout (see {@link #tryRecoverWithStream}). private QueryResult tryRecover(long requestId, Set servers, Exception ex) throws Exception { if (servers.isEmpty()) { @@ -441,7 +457,7 @@ private QueryResult tryRecoverWithStream(StreamingQuerySession session, Map